diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index 62ed8922db..792aa1a82c 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -677,7 +677,7 @@ index 7af826583bd..3c3def1eb67 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 44c8cb92fc3..e29cb93ecda 100644 +index 44c8cb92fc3..f098beeca26 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 @@ -31,7 +31,8 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation @@ -825,19 +825,7 @@ index 44c8cb92fc3..e29cb93ecda 100644 checkAnswer(shjNonCodegenDF, Seq.empty) } } -@@ -1470,7 +1491,10 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan - "/*+ BROADCAST(t2) */ t1.k as k" - } - val plan = sql(getAggQuery(selectExpr, joinType)).queryExecution.executedPlan -- assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) -+ assert(collect(plan) { -+ case _: BroadcastNestedLoopJoinExec | _: CometBroadcastNestedLoopJoinExec => -+ true -+ }.size === 1) - // No extra shuffle before aggregation - assert(collect(plan) { case _: ShuffleExchangeExec => true }.size === 0) - } -@@ -1486,7 +1510,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1486,7 +1507,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 @@ -847,17 +835,12 @@ index 44c8cb92fc3..e29cb93ecda 100644 } def getJoinQuery(selectExpr: String, joinType: String): String = { -@@ -1514,10 +1539,16 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan - "/*+ BROADCAST(right_t) */ k1 as k0" +@@ -1515,9 +1537,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) + assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) - assert(collect(plan) { case _: SortMergeJoinExec => true }.size === 3) + assert(collect(plan) { -+ case _: BroadcastNestedLoopJoinExec | _: CometBroadcastNestedLoopJoinExec => -+ true -+ }.size === 1) -+ assert(collect(plan) { + case _: SortMergeJoinExec => true + case _: CometSortMergeJoinExec => true + }.size === 3) @@ -867,7 +850,7 @@ index 44c8cb92fc3..e29cb93ecda 100644 } // Test output ordering is not preserved -@@ -1526,9 +1557,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1526,9 +1551,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) @@ -882,7 +865,7 @@ index 44c8cb92fc3..e29cb93ecda 100644 } // Test singe partition -@@ -1538,7 +1572,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1538,7 +1566,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 @@ -892,7 +875,7 @@ index 44c8cb92fc3..e29cb93ecda 100644 checkAnswer(fullJoinDF, Row(100)) } } -@@ -1611,6 +1646,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1611,6 +1640,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan Seq(semiJoinDF, antiJoinDF).foreach { df => assert(collect(df.queryExecution.executedPlan) { case j: ShuffledHashJoinExec if j.ignoreDuplicatedKey == ignoreDuplicatedKey => true @@ -902,7 +885,7 @@ index 44c8cb92fc3..e29cb93ecda 100644 }.size == 1) } } -@@ -1655,14 +1693,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1655,14 +1687,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan test("SPARK-43113: Full outer join with duplicate stream-side references in condition (SMJ)") { def check(plan: SparkPlan): Unit = { @@ -925,7 +908,7 @@ index 44c8cb92fc3..e29cb93ecda 100644 } dupStreamSideColTest("SHUFFLE_HASH", check) } -@@ -1798,7 +1842,8 @@ class ThreadLeakInSortMergeJoinSuite +@@ -1798,7 +1836,8 @@ class ThreadLeakInSortMergeJoinSuite sparkConf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, 20)) } diff --git a/dev/diffs/4.0.2.diff b/dev/diffs/4.0.2.diff index 037f15398f..d074583cf4 100644 --- a/dev/diffs/4.0.2.diff +++ b/dev/diffs/4.0.2.diff @@ -824,7 +824,7 @@ index 53e47f428c3..a55d8f0c161 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 aaac0ebc9aa..276c592ec88 100644 +index aaac0ebc9aa..fbef0774d46 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,7 +29,8 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation @@ -972,19 +972,7 @@ index aaac0ebc9aa..276c592ec88 100644 checkAnswer(shjNonCodegenDF, Seq.empty) } } -@@ -1473,7 +1494,10 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan - "/*+ BROADCAST(t2) */ t1.k as k" - } - val plan = sql(getAggQuery(selectExpr, joinType)).queryExecution.executedPlan -- assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) -+ assert(collect(plan) { -+ case _: BroadcastNestedLoopJoinExec | _: CometBroadcastNestedLoopJoinExec => -+ true -+ }.size === 1) - // No extra shuffle before aggregation - assert(collect(plan) { case _: ShuffleExchangeExec => true }.size === 0) - } -@@ -1489,7 +1513,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1489,7 +1510,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 @@ -994,17 +982,12 @@ index aaac0ebc9aa..276c592ec88 100644 } def getJoinQuery(selectExpr: String, joinType: String): String = { -@@ -1517,10 +1542,16 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan - "/*+ BROADCAST(right_t) */ k1 as k0" +@@ -1518,9 +1540,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) + assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) - assert(collect(plan) { case _: SortMergeJoinExec => true }.size === 3) + assert(collect(plan) { -+ case _: BroadcastNestedLoopJoinExec | _: CometBroadcastNestedLoopJoinExec => -+ true -+ }.size === 1) -+ assert(collect(plan) { + case _: SortMergeJoinExec => true + case _: CometSortMergeJoinExec => true + }.size === 3) @@ -1014,7 +997,7 @@ index aaac0ebc9aa..276c592ec88 100644 } // Test output ordering is not preserved -@@ -1529,9 +1560,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1529,9 +1554,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) @@ -1029,7 +1012,7 @@ index aaac0ebc9aa..276c592ec88 100644 } // Test singe partition -@@ -1541,7 +1575,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1541,7 +1569,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 @@ -1039,7 +1022,7 @@ index aaac0ebc9aa..276c592ec88 100644 checkAnswer(fullJoinDF, Row(100)) } } -@@ -1614,6 +1649,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1614,6 +1643,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan Seq(semiJoinDF, antiJoinDF).foreach { df => assert(collect(df.queryExecution.executedPlan) { case j: ShuffledHashJoinExec if j.ignoreDuplicatedKey == ignoreDuplicatedKey => true @@ -1049,7 +1032,7 @@ index aaac0ebc9aa..276c592ec88 100644 }.size == 1) } } -@@ -1658,14 +1696,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1658,14 +1690,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan test("SPARK-43113: Full outer join with duplicate stream-side references in condition (SMJ)") { def check(plan: SparkPlan): Unit = { @@ -1072,7 +1055,7 @@ index aaac0ebc9aa..276c592ec88 100644 } dupStreamSideColTest("SHUFFLE_HASH", check) } -@@ -1801,7 +1845,8 @@ class ThreadLeakInSortMergeJoinSuite +@@ -1801,7 +1839,8 @@ class ThreadLeakInSortMergeJoinSuite sparkConf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, 20)) } diff --git a/dev/diffs/4.1.2.diff b/dev/diffs/4.1.2.diff index 8b5fd9ba24..a53f14843c 100644 --- a/dev/diffs/4.1.2.diff +++ b/dev/diffs/4.1.2.diff @@ -1043,18 +1043,6 @@ index 885512d4d19..113ae17ad9f 100644 checkAnswer(shjNonCodegenDF, Seq.empty) } } -@@ -1485,7 +1507,10 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan - "/*+ BROADCAST(t2) */ t1.k as k" - } - val plan = sql(getAggQuery(selectExpr, joinType)).queryExecution.executedPlan -- assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) -+ assert(collect(plan) { -+ case _: BroadcastNestedLoopJoinExec | _: CometBroadcastNestedLoopJoinExec => -+ true -+ }.size === 1) - // No extra shuffle before aggregation - assert(collect(plan) { case _: ShuffleExchangeExec => true }.size === 0) - } @@ -1501,7 +1523,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) @@ -1065,16 +1053,12 @@ index 885512d4d19..113ae17ad9f 100644 } def getJoinQuery(selectExpr: String, joinType: String): String = { -@@ -1530,10 +1556,16 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1530,9 +1553,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) + assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) - assert(collect(plan) { case _: SortMergeJoinExec => true }.size === 3) + assert(collect(plan) { -+ case _: BroadcastNestedLoopJoinExec | _: CometBroadcastNestedLoopJoinExec => -+ true -+ }.size === 1) -+ assert(collect(plan) { + case _: SortMergeJoinExec => true + case _: CometSortMergeJoinExec => true + }.size === 3) diff --git a/docs/source/user-guide/latest/operators.md b/docs/source/user-guide/latest/operators.md index d4f016f1ae..495141f9bb 100644 --- a/docs/source/user-guide/latest/operators.md +++ b/docs/source/user-guide/latest/operators.md @@ -27,7 +27,6 @@ not supported by Comet will fall back to regular Spark execution. | BatchScanExec | Yes | Supports Parquet files and Apache Iceberg Parquet scans. See the [Comet Compatibility Guide] for more information. | | BroadcastExchangeExec | Yes | | | BroadcastHashJoinExec | Yes | | -| BroadcastNestedLoopJoinExec | Yes | Falls back to Spark when the preserved side is broadcast (e.g. LEFT OUTER with BROADCAST on the left). | | ExpandExec | Yes | | | FileSourceScanExec | Yes | Supports Parquet files. See the [Comet Compatibility Guide] for more information. | | FilterExec | Yes | | diff --git a/docs/source/user-guide/latest/understanding-comet-plans.md b/docs/source/user-guide/latest/understanding-comet-plans.md index 8adccc0929..a4d14a1228 100644 --- a/docs/source/user-guide/latest/understanding-comet-plans.md +++ b/docs/source/user-guide/latest/understanding-comet-plans.md @@ -202,22 +202,21 @@ by role. Names match what is shown in the plan output. These run natively in DataFusion. When several appear consecutively in a plan, they execute as a single fused native block. -| Node | Spark equivalent | -| ------------------------------ | ----------------------------------------------- | -| `CometProject` | `ProjectExec` | -| `CometFilter` | `FilterExec` | -| `CometSort` | `SortExec` | -| `CometLocalLimit` | `LocalLimitExec` | -| `CometGlobalLimit` | `GlobalLimitExec` | -| `CometExpand` | `ExpandExec` | -| `CometExplode` | `GenerateExec` (for `explode` and `posexplode`) | -| `CometHashAggregate` | `HashAggregateExec`, `ObjectHashAggregateExec` | -| `CometHashJoin` | `ShuffledHashJoinExec` | -| `CometBroadcastHashJoin` | `BroadcastHashJoinExec` | -| `CometBroadcastNestedLoopJoin` | `BroadcastNestedLoopJoinExec` | -| `CometSortMergeJoin` | `SortMergeJoinExec` | -| `CometWindow` | `WindowExec` | -| `CometTakeOrderedAndProject` | `TakeOrderedAndProjectExec` | +| Node | Spark equivalent | +| ---------------------------- | ----------------------------------------------- | +| `CometProject` | `ProjectExec` | +| `CometFilter` | `FilterExec` | +| `CometSort` | `SortExec` | +| `CometLocalLimit` | `LocalLimitExec` | +| `CometGlobalLimit` | `GlobalLimitExec` | +| `CometExpand` | `ExpandExec` | +| `CometExplode` | `GenerateExec` (for `explode` and `posexplode`) | +| `CometHashAggregate` | `HashAggregateExec`, `ObjectHashAggregateExec` | +| `CometHashJoin` | `ShuffledHashJoinExec` | +| `CometBroadcastHashJoin` | `BroadcastHashJoinExec` | +| `CometSortMergeJoin` | `SortMergeJoinExec` | +| `CometWindow` | `WindowExec` | +| `CometTakeOrderedAndProject` | `TakeOrderedAndProjectExec` | ### JVM-Side Operators diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 4a7a21006d..c0c6319d7a 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -239,7 +239,6 @@ fn op_name(op: &OpStruct) -> &'static str { OpStruct::Explode(_) => "Explode", OpStruct::CsvScan(_) => "CsvScan", OpStruct::ShuffleScan(_) => "ShuffleScan", - OpStruct::BroadcastNestedLoopJoin(_) => "BroadcastNestedLoopJoin", } } diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index c07a92d700..6ae17120d1 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -109,7 +109,6 @@ use arrow::row::{OwnedRow, RowConverter, SortField}; use datafusion::common::utils::SingleRowListArrayBuilder; use datafusion::common::UnnestOptions; use datafusion::physical_plan::filter::FilterExec; -use datafusion::physical_plan::joins::NestedLoopJoinExec; use datafusion::physical_plan::limit::GlobalLimitExec; use datafusion::physical_plan::unnest::{ListUnnest, UnnestExec}; use datafusion_comet_proto::spark_expression::ListLiteral; @@ -1229,57 +1228,6 @@ impl PhysicalPlanner { Arc::new(SparkPlan::new(spark_plan.plan_id, aggregate, vec![child])), )) } - - OpStruct::BroadcastNestedLoopJoin(bnlj) => { - let (join_params, scans, shuffle_scans) = self.parse_join_parameters( - inputs, - children, - &[], - &[], - bnlj.join_type, - &bnlj.condition, - partition_count, - )?; - - let left = Arc::clone(&join_params.left.native_plan); - let right = Arc::clone(&join_params.right.native_plan); - - let nested_loop_join = Arc::new(NestedLoopJoinExec::try_new( - left, - right, - join_params.join_filter, - &join_params.join_type, - None, - )?); - - if bnlj.build_side == BuildSide::BuildRight as i32 { - let swapped_join = nested_loop_join.as_ref().swap_inputs()?; - let mut additional_native_plans = vec![]; - if swapped_join.as_any().is::() { - additional_native_plans.push(Arc::clone(swapped_join.children()[0])); - } - Ok(( - scans, - shuffle_scans, - Arc::new(SparkPlan::new_with_additional( - spark_plan.plan_id, - swapped_join, - vec![join_params.left, join_params.right], - additional_native_plans, - )), - )) - } else { - Ok(( - scans, - shuffle_scans, - Arc::new(SparkPlan::new( - spark_plan.plan_id, - nested_loop_join, - vec![join_params.left, join_params.right], - )), - )) - } - } OpStruct::Limit(limit) => { assert_eq!(children.len(), 1); let num = limit.limit; diff --git a/native/core/src/execution/planner/operator_registry.rs b/native/core/src/execution/planner/operator_registry.rs index dc98a32fb0..eb31184461 100644 --- a/native/core/src/execution/planner/operator_registry.rs +++ b/native/core/src/execution/planner/operator_registry.rs @@ -151,6 +151,5 @@ fn get_operator_type(spark_operator: &Operator) -> Option { OpStruct::Explode(_) => None, // Not yet in OperatorType enum OpStruct::CsvScan(_) => Some(OperatorType::CsvScan), OpStruct::ShuffleScan(_) => None, // Not yet in OperatorType enum - OpStruct::BroadcastNestedLoopJoin(_) => None, } } diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 15265f1d86..a180d71309 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -53,7 +53,6 @@ message Operator { Explode explode = 114; CsvScan csv_scan = 115; ShuffleScan shuffle_scan = 116; - BroadcastNestedLoopJoin broadcast_nested_loop_join = 117; } } @@ -398,12 +397,6 @@ message SortMergeJoin { optional spark.spark_expression.Expr condition = 5; } -message BroadcastNestedLoopJoin { - JoinType join_type = 1; - BuildSide build_side = 2; - optional spark.spark_expression.Expr condition = 3; -} - enum JoinType { Inner = 0; LeftOuter = 1; diff --git a/spark/src/main/scala/org/apache/comet/CometConf.scala b/spark/src/main/scala/org/apache/comet/CometConf.scala index aabd64b9b3..af8fbb5815 100644 --- a/spark/src/main/scala/org/apache/comet/CometConf.scala +++ b/spark/src/main/scala/org/apache/comet/CometConf.scala @@ -254,8 +254,6 @@ object CometConf extends ShimCometConf { createExecEnabledConfig("broadcastExchange", defaultValue = true) val COMET_EXEC_HASH_JOIN_ENABLED: ConfigEntry[Boolean] = createExecEnabledConfig("hashJoin", defaultValue = true) - val COMET_EXEC_BROADCAST_NESTED_LOOP_JOIN_ENABLED: ConfigEntry[Boolean] = - createExecEnabledConfig("broadcastNestedLoopJoin", defaultValue = true) val COMET_EXEC_SORT_MERGE_JOIN_ENABLED: ConfigEntry[Boolean] = createExecEnabledConfig("sortMergeJoin", defaultValue = true) val COMET_EXEC_AGGREGATE_ENABLED: ConfigEntry[Boolean] = diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index e261ac45d1..d116d2f407 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -44,7 +44,7 @@ import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan import org.apache.spark.sql.execution.datasources.v2.json.JsonScan import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -81,7 +81,6 @@ object CometExecRule { classOf[HashAggregateExec] -> CometHashAggregateExec, classOf[ObjectHashAggregateExec] -> CometObjectHashAggregateExec, classOf[BroadcastHashJoinExec] -> CometBroadcastHashJoinExec, - classOf[BroadcastNestedLoopJoinExec] -> CometBroadcastNestedLoopJoinExec, classOf[ShuffledHashJoinExec] -> CometHashJoinExec, classOf[SortMergeJoinExec] -> CometSortMergeJoinExec, classOf[SortExec] -> CometSortExec, diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala index 0352e3d83c..b3b89d043a 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometMetricNode.scala @@ -268,10 +268,9 @@ object CometMetricNode { } /** - * SQL Metrics for DataFusion joins that use the BuildProbeJoinMetrics shape (HashJoinExec and - * NestedLoopJoinExec). + * SQL Metrics for DataFusion HashJoin */ - def joinMetrics(sc: SparkContext): Map[String, SQLMetric] = { + def hashJoinMetrics(sc: SparkContext): Map[String, SQLMetric] = { Map( "build_time" -> SQLMetrics.createNanoTimingMetric(sc, "Total time for collecting build-side of join"), diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index 53b09e92b3..8ace88eebb 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -41,7 +41,7 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, BroadcastQueryStageExec, ShuffleQueryStageExec} import org.apache.spark.sql.execution.aggregate.{BaseAggregateExec, HashAggregateExec, ObjectHashAggregateExec} import org.apache.spark.sql.execution.exchange.ReusedExchangeExec -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, HashJoin, ShuffledHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, HashJoin, ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.types.{ArrayType, BooleanType, ByteType, DataType, DateType, DecimalType, DoubleType, FloatType, IntegerType, LongType, MapType, ShortType, StringType, TimestampNTZType, TimestampType} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -2054,161 +2054,6 @@ trait CometHashJoin { } } -case class CometBroadcastNestedLoopJoinExec( - override val nativeOp: Operator, - override val originalPlan: SparkPlan, - override val output: Seq[Attribute], - override val outputOrdering: Seq[SortOrder], - joinType: JoinType, - condition: Option[Expression], - buildSide: BuildSide, - override val left: SparkPlan, - override val right: SparkPlan, - override val serializedPlanOpt: SerializedPlan) - extends CometBinaryExec { - - // Mirror Spark's BroadcastNestedLoopJoinExec: output partitioning derives from the streamed - // (non-broadcast) side. Reading from live left/right rather than originalPlan keeps this - // correct after an AQE child swap. - private def streamedPlan: SparkPlan = buildSide match { - case BuildLeft => right - case BuildRight => left - } - - override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning - - override def withNewChildrenInternal(newLeft: SparkPlan, newRight: SparkPlan): SparkPlan = - this.copy(left = newLeft, right = newRight) - - override def stringArgs: Iterator[Any] = - Iterator(joinType, buildSide, condition, left, right) - - override def equals(obj: Any): Boolean = { - obj match { - case other: CometBroadcastNestedLoopJoinExec => - this.output == other.output && - this.joinType == other.joinType && - this.condition == other.condition && - this.buildSide == other.buildSide && - this.left == other.left && - this.right == other.right && - this.serializedPlanOpt == other.serializedPlanOpt - case _ => - false - } - } - - override def hashCode(): Int = - Objects.hashCode(output, joinType, condition, buildSide, left, right) - - override lazy val metrics: Map[String, SQLMetric] = - CometMetricNode.joinMetrics(sparkContext) -} - -object CometBroadcastNestedLoopJoinExec extends CometOperatorSerde[BroadcastNestedLoopJoinExec] { - - /** - * Get the optional Comet configuration entry that is used to enable or disable native support - * for this operator. - */ - override def enabledConfig: Option[ConfigEntry[Boolean]] = { - Some(CometConf.COMET_EXEC_BROADCAST_NESTED_LOOP_JOIN_ENABLED) - } - - private val broadcastBuildReplicationReason = - "BNLJ combinations that emit per-build-row results need a cross-partition merge that" + - " DataFusion's NestedLoopJoinExec does not provide. Affects: LeftOuter+BuildLeft," + - " RightOuter+BuildRight, FullOuter, LeftSemi+BuildLeft, LeftAnti+BuildLeft." - - override def getSupportLevel(op: BroadcastNestedLoopJoinExec): SupportLevel = - (op.joinType, op.buildSide) match { - case (Inner, _) | (LeftOuter, BuildRight) | (RightOuter, BuildLeft) | - (LeftSemi, BuildRight) | (LeftAnti, BuildRight) => - Compatible(None) - case (LeftOuter | RightOuter | FullOuter | LeftSemi | LeftAnti, _) => - Unsupported(Some(broadcastBuildReplicationReason)) - case _ => Unsupported(Some(s"Join type ${op.joinType} is not supported")) - } - - /** - * Convert a Spark operator into a protocol buffer representation that can be passed into native - * code. - * - * @param op - * The Spark operator. - * @param builder - * The protobuf builder for the operator. - * @param childOp - * Child operators that have already been converted to Comet. - * @return - * Protocol buffer representation, or None if the operator could not be converted. In this - * case it is expected that the input operator will have been tagged with reasons why it could - * not be converted. - */ - override def convert( - op: BroadcastNestedLoopJoinExec, - builder: Operator.Builder, - childOp: Operator*): Option[Operator] = { - - val buildSide = op.buildSide match { - case BuildLeft => OperatorOuterClass.BuildSide.BuildLeft - case BuildRight => OperatorOuterClass.BuildSide.BuildRight - } - - val join = op.joinType - val joinType = { - import OperatorOuterClass.JoinType - join match { - case Inner => JoinType.Inner - case LeftOuter => JoinType.LeftOuter - case RightOuter => JoinType.RightOuter - case FullOuter => JoinType.FullOuter - case LeftSemi => JoinType.LeftSemi - case LeftAnti => JoinType.LeftAnti - case _ => - // Spark doesn't support other join types - withFallbackReason(op, s"Unsupported join type $join") - return None - } - } - - val joinCondition = op.condition.map({ cond => - val condProto = exprToProto(cond, op.left.output ++ op.right.output) - if (condProto.isEmpty) { - withFallbackReason(op, cond) - return None - } - condProto.get - }) - - val joinBuilder = OperatorOuterClass.BroadcastNestedLoopJoin - .newBuilder() - .setJoinType(joinType) - .setBuildSide(buildSide) - - joinCondition.foreach(joinBuilder.setCondition) - - Some(builder.setBroadcastNestedLoopJoin(joinBuilder).build()) - } - - override def createExec( - nativeOp: Operator, - op: BroadcastNestedLoopJoinExec): CometNativeExec = { - - CometBroadcastNestedLoopJoinExec( - nativeOp, - op, - op.output, - op.outputOrdering, - op.joinType, - op.condition, - op.buildSide, - op.left, - op.right, - SerializedPlan(None)) - } -} - object CometBroadcastHashJoinExec extends CometOperatorSerde[HashJoin] with CometHashJoin { override def enabledConfig: Option[ConfigEntry[Boolean]] = @@ -2317,7 +2162,7 @@ case class CometHashJoinExec( Objects.hashCode(output, leftKeys, rightKeys, condition, buildSide, left, right) override lazy val metrics: Map[String, SQLMetric] = - CometMetricNode.joinMetrics(sparkContext) + CometMetricNode.hashJoinMetrics(sparkContext) } case class CometBroadcastHashJoinExec( @@ -2458,7 +2303,7 @@ case class CometBroadcastHashJoinExec( Objects.hashCode(output, leftKeys, rightKeys, condition, buildSide, left, right) override lazy val metrics: Map[String, SQLMetric] = - CometMetricNode.joinMetrics(sparkContext) + CometMetricNode.hashJoinMetrics(sparkContext) } object CometSortMergeJoinExec extends CometOperatorSerde[SortMergeJoinExec] { diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/extended.txt deleted file mode 100644 index 6bc95e1a33..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/extended.txt +++ /dev/null @@ -1,67 +0,0 @@ -CometNativeColumnarToRow -+- CometBroadcastNestedLoopJoin - :- CometBroadcastNestedLoopJoin - : :- CometBroadcastNestedLoopJoin - : : :- CometBroadcastNestedLoopJoin - : : : :- CometBroadcastNestedLoopJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometBroadcastExchange - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store_sales - -Comet accelerated 64 out of 64 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/extended.txt deleted file mode 100644 index 1fd67d1103..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/extended.txt +++ /dev/null @@ -1,86 +0,0 @@ -CometNativeColumnarToRow -+- CometProject - +- CometBroadcastNestedLoopJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.promotion - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 81 out of 83 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/extended.txt deleted file mode 100644 index 67c31a6ac1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/extended.txt +++ /dev/null @@ -1,112 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- CometProject - : +- CometBroadcastNestedLoopJoin - : :- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page - -Comet accelerated 103 out of 109 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/extended.txt deleted file mode 100644 index 603d8e0b52..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/extended.txt +++ /dev/null @@ -1,209 +0,0 @@ -CometNativeColumnarToRow -+- CometBroadcastNestedLoopJoin - :- CometBroadcastNestedLoopJoin - : :- CometBroadcastNestedLoopJoin - : : :- CometBroadcastNestedLoopJoin - : : : :- CometBroadcastNestedLoopJoin - : : : : :- CometBroadcastNestedLoopJoin - : : : : : :- CometBroadcastNestedLoopJoin - : : : : : : :- CometHashAggregate - : : : : : : : +- CometExchange - : : : : : : : +- CometHashAggregate - : : : : : : : +- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometHashAggregate - : : : : : : +- CometExchange - : : : : : : +- CometHashAggregate - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : : +- CometBroadcastExchange - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : +- CometBroadcastExchange - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.time_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.time_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store - -Comet accelerated 206 out of 206 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/extended.txt deleted file mode 100644 index ebe68e5447..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/extended.txt +++ /dev/null @@ -1,54 +0,0 @@ -CometNativeColumnarToRow -+- CometProject - +- CometBroadcastNestedLoopJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page - -Comet accelerated 51 out of 51 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/extended.txt deleted file mode 100644 index 6bc95e1a33..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/extended.txt +++ /dev/null @@ -1,67 +0,0 @@ -CometNativeColumnarToRow -+- CometBroadcastNestedLoopJoin - :- CometBroadcastNestedLoopJoin - : :- CometBroadcastNestedLoopJoin - : : :- CometBroadcastNestedLoopJoin - : : : :- CometBroadcastNestedLoopJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometBroadcastExchange - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store_sales - -Comet accelerated 64 out of 64 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/extended.txt deleted file mode 100644 index 1fd67d1103..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/extended.txt +++ /dev/null @@ -1,86 +0,0 @@ -CometNativeColumnarToRow -+- CometProject - +- CometBroadcastNestedLoopJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.promotion - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.item - -Comet accelerated 81 out of 83 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/extended.txt deleted file mode 100644 index 67c31a6ac1..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/extended.txt +++ /dev/null @@ -1,112 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- CometProject - : +- CometBroadcastNestedLoopJoin - : :- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page - -Comet accelerated 103 out of 109 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/extended.txt deleted file mode 100644 index 603d8e0b52..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/extended.txt +++ /dev/null @@ -1,209 +0,0 @@ -CometNativeColumnarToRow -+- CometBroadcastNestedLoopJoin - :- CometBroadcastNestedLoopJoin - : :- CometBroadcastNestedLoopJoin - : : :- CometBroadcastNestedLoopJoin - : : : :- CometBroadcastNestedLoopJoin - : : : : :- CometBroadcastNestedLoopJoin - : : : : : :- CometBroadcastNestedLoopJoin - : : : : : : :- CometHashAggregate - : : : : : : : +- CometExchange - : : : : : : : +- CometHashAggregate - : : : : : : : +- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometHashAggregate - : : : : : : +- CometExchange - : : : : : : +- CometHashAggregate - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : : +- CometBroadcastExchange - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : +- CometBroadcastExchange - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.time_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.time_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.store - -Comet accelerated 206 out of 206 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/extended.txt deleted file mode 100644 index ebe68e5447..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/extended.txt +++ /dev/null @@ -1,54 +0,0 @@ -CometNativeColumnarToRow -+- CometProject - +- CometBroadcastNestedLoopJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.time_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page - -Comet accelerated 51 out of 51 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/extended.txt index 6bc95e1a33..6dbeba7e9a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q28/extended.txt @@ -1,59 +1,64 @@ -CometNativeColumnarToRow -+- CometBroadcastNestedLoopJoin - :- CometBroadcastNestedLoopJoin - : :- CometBroadcastNestedLoopJoin - : : :- CometBroadcastNestedLoopJoin - : : : :- CometBroadcastNestedLoopJoin - : : : : :- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometBroadcastExchange - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store_sales - +- CometBroadcastExchange +BroadcastNestedLoopJoin +:- BroadcastNestedLoopJoin +: :- BroadcastNestedLoopJoin +: : :- BroadcastNestedLoopJoin +: : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] +: : : : :- CometNativeColumnarToRow +: : : : : +- CometHashAggregate +: : : : : +- CometExchange +: : : : : +- CometHashAggregate +: : : : : +- CometHashAggregate +: : : : : +- CometExchange +: : : : : +- CometHashAggregate +: : : : : +- CometProject +: : : : : +- CometFilter +: : : : : +- CometNativeScan parquet spark_catalog.default.store_sales +: : : : +- BroadcastExchange +: : : : +- CometNativeColumnarToRow +: : : : +- CometHashAggregate +: : : : +- CometExchange +: : : : +- CometHashAggregate +: : : : +- CometHashAggregate +: : : : +- CometExchange +: : : : +- CometHashAggregate +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometNativeScan parquet spark_catalog.default.store_sales +: : : +- BroadcastExchange +: : : +- CometNativeColumnarToRow +: : : +- CometHashAggregate +: : : +- CometExchange +: : : +- CometHashAggregate +: : : +- CometHashAggregate +: : : +- CometExchange +: : : +- CometHashAggregate +: : : +- CometProject +: : : +- CometFilter +: : : +- CometNativeScan parquet spark_catalog.default.store_sales +: : +- BroadcastExchange +: : +- CometNativeColumnarToRow +: : +- CometHashAggregate +: : +- CometExchange +: : +- CometHashAggregate +: : +- CometHashAggregate +: : +- CometExchange +: : +- CometHashAggregate +: : +- CometProject +: : +- CometFilter +: : +- CometNativeScan parquet spark_catalog.default.store_sales +: +- BroadcastExchange +: +- CometNativeColumnarToRow +: +- CometHashAggregate +: +- CometExchange +: +- CometHashAggregate +: +- CometHashAggregate +: +- CometExchange +: +- CometHashAggregate +: +- CometProject +: +- CometFilter +: +- CometNativeScan parquet spark_catalog.default.store_sales ++- BroadcastExchange + +- CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -64,4 +69,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store_sales -Comet accelerated 64 out of 64 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 54 out of 64 eligible operators (84%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/extended.txt index 1fd67d1103..39f985df28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/extended.txt @@ -1,52 +1,53 @@ -CometNativeColumnarToRow -+- CometProject - +- CometBroadcastNestedLoopJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometSubqueryBroadcast - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.promotion - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange +Project ++- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + :- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : : : +- CometSubqueryBroadcast + : : : : : : : +- CometBroadcastExchange + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometNativeScan parquet spark_catalog.default.store + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.promotion + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.customer + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- BroadcastExchange + +- CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -83,4 +84,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 81 out of 83 eligible operators (97%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 78 out of 83 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/extended.txt index 67c31a6ac1..5bffdf3688 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/extended.txt @@ -1,76 +1,79 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometUnion - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- CometProject - : +- CometBroadcastNestedLoopJoin - : :- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim +TakeOrderedAndProject ++- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] + +- Exchange + +- HashAggregate + +- Expand + +- Union + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- Project + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + : :- BroadcastExchange + : : +- CometNativeColumnarToRow + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometNativeColumnarToRow +- CometProject +- CometBroadcastHashJoin :- CometHashAggregate @@ -109,4 +112,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 103 out of 109 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 94 out of 109 eligible operators (86%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/extended.txt index 603d8e0b52..87f19deeb8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/extended.txt @@ -1,186 +1,193 @@ -CometNativeColumnarToRow -+- CometBroadcastNestedLoopJoin - :- CometBroadcastNestedLoopJoin - : :- CometBroadcastNestedLoopJoin - : : :- CometBroadcastNestedLoopJoin - : : : :- CometBroadcastNestedLoopJoin - : : : : :- CometBroadcastNestedLoopJoin - : : : : : :- CometBroadcastNestedLoopJoin - : : : : : : :- CometHashAggregate - : : : : : : : +- CometExchange - : : : : : : : +- CometHashAggregate - : : : : : : : +- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : : :- CometProject - : : : : : : : : : : +- CometFilter - : : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : : +- CometBroadcastExchange - : : : : : : : : : +- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometHashAggregate - : : : : : : +- CometExchange - : : : : : : +- CometHashAggregate - : : : : : : +- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometBroadcastHashJoin - : : : : : : : : :- CometProject - : : : : : : : : : +- CometFilter - : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : : +- CometBroadcastExchange - : : : : : : : : +- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : : +- CometBroadcastExchange - : : : : : +- CometHashAggregate - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometBroadcastHashJoin - : : : : : : : :- CometProject - : : : : : : : : +- CometFilter - : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : : +- CometBroadcastExchange - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : : +- CometBroadcastExchange - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.time_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.time_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - +- CometBroadcastExchange +BroadcastNestedLoopJoin +:- BroadcastNestedLoopJoin +: :- BroadcastNestedLoopJoin +: : :- BroadcastNestedLoopJoin +: : : :- BroadcastNestedLoopJoin +: : : : :- BroadcastNestedLoopJoin +: : : : : :- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] +: : : : : : :- CometNativeColumnarToRow +: : : : : : : +- CometHashAggregate +: : : : : : : +- CometExchange +: : : : : : : +- CometHashAggregate +: : : : : : : +- CometProject +: : : : : : : +- CometBroadcastHashJoin +: : : : : : : :- CometProject +: : : : : : : : +- CometBroadcastHashJoin +: : : : : : : : :- CometProject +: : : : : : : : : +- CometBroadcastHashJoin +: : : : : : : : : :- CometProject +: : : : : : : : : : +- CometFilter +: : : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales +: : : : : : : : : +- CometBroadcastExchange +: : : : : : : : : +- CometProject +: : : : : : : : : +- CometFilter +: : : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics +: : : : : : : : +- CometBroadcastExchange +: : : : : : : : +- CometProject +: : : : : : : : +- CometFilter +: : : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim +: : : : : : : +- CometBroadcastExchange +: : : : : : : +- CometProject +: : : : : : : +- CometFilter +: : : : : : : +- CometNativeScan parquet spark_catalog.default.store +: : : : : : +- BroadcastExchange +: : : : : : +- CometNativeColumnarToRow +: : : : : : +- CometHashAggregate +: : : : : : +- CometExchange +: : : : : : +- CometHashAggregate +: : : : : : +- CometProject +: : : : : : +- CometBroadcastHashJoin +: : : : : : :- CometProject +: : : : : : : +- CometBroadcastHashJoin +: : : : : : : :- CometProject +: : : : : : : : +- CometBroadcastHashJoin +: : : : : : : : :- CometProject +: : : : : : : : : +- CometFilter +: : : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales +: : : : : : : : +- CometBroadcastExchange +: : : : : : : : +- CometProject +: : : : : : : : +- CometFilter +: : : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics +: : : : : : : +- CometBroadcastExchange +: : : : : : : +- CometProject +: : : : : : : +- CometFilter +: : : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim +: : : : : : +- CometBroadcastExchange +: : : : : : +- CometProject +: : : : : : +- CometFilter +: : : : : : +- CometNativeScan parquet spark_catalog.default.store +: : : : : +- BroadcastExchange +: : : : : +- CometNativeColumnarToRow +: : : : : +- CometHashAggregate +: : : : : +- CometExchange +: : : : : +- CometHashAggregate +: : : : : +- CometProject +: : : : : +- CometBroadcastHashJoin +: : : : : :- CometProject +: : : : : : +- CometBroadcastHashJoin +: : : : : : :- CometProject +: : : : : : : +- CometBroadcastHashJoin +: : : : : : : :- CometProject +: : : : : : : : +- CometFilter +: : : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales +: : : : : : : +- CometBroadcastExchange +: : : : : : : +- CometProject +: : : : : : : +- CometFilter +: : : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics +: : : : : : +- CometBroadcastExchange +: : : : : : +- CometProject +: : : : : : +- CometFilter +: : : : : : +- CometNativeScan parquet spark_catalog.default.time_dim +: : : : : +- CometBroadcastExchange +: : : : : +- CometProject +: : : : : +- CometFilter +: : : : : +- CometNativeScan parquet spark_catalog.default.store +: : : : +- BroadcastExchange +: : : : +- CometNativeColumnarToRow +: : : : +- CometHashAggregate +: : : : +- CometExchange +: : : : +- CometHashAggregate +: : : : +- CometProject +: : : : +- CometBroadcastHashJoin +: : : : :- CometProject +: : : : : +- CometBroadcastHashJoin +: : : : : :- CometProject +: : : : : : +- CometBroadcastHashJoin +: : : : : : :- CometProject +: : : : : : : +- CometFilter +: : : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales +: : : : : : +- CometBroadcastExchange +: : : : : : +- CometProject +: : : : : : +- CometFilter +: : : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics +: : : : : +- CometBroadcastExchange +: : : : : +- CometProject +: : : : : +- CometFilter +: : : : : +- CometNativeScan parquet spark_catalog.default.time_dim +: : : : +- CometBroadcastExchange +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometNativeScan parquet spark_catalog.default.store +: : : +- BroadcastExchange +: : : +- CometNativeColumnarToRow +: : : +- CometHashAggregate +: : : +- CometExchange +: : : +- CometHashAggregate +: : : +- CometProject +: : : +- CometBroadcastHashJoin +: : : :- CometProject +: : : : +- CometBroadcastHashJoin +: : : : :- CometProject +: : : : : +- CometBroadcastHashJoin +: : : : : :- CometProject +: : : : : : +- CometFilter +: : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales +: : : : : +- CometBroadcastExchange +: : : : : +- CometProject +: : : : : +- CometFilter +: : : : : +- CometNativeScan parquet spark_catalog.default.household_demographics +: : : : +- CometBroadcastExchange +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometNativeScan parquet spark_catalog.default.time_dim +: : : +- CometBroadcastExchange +: : : +- CometProject +: : : +- CometFilter +: : : +- CometNativeScan parquet spark_catalog.default.store +: : +- BroadcastExchange +: : +- CometNativeColumnarToRow +: : +- CometHashAggregate +: : +- CometExchange +: : +- CometHashAggregate +: : +- CometProject +: : +- CometBroadcastHashJoin +: : :- CometProject +: : : +- CometBroadcastHashJoin +: : : :- CometProject +: : : : +- CometBroadcastHashJoin +: : : : :- CometProject +: : : : : +- CometFilter +: : : : : +- CometNativeScan parquet spark_catalog.default.store_sales +: : : : +- CometBroadcastExchange +: : : : +- CometProject +: : : : +- CometFilter +: : : : +- CometNativeScan parquet spark_catalog.default.household_demographics +: : : +- CometBroadcastExchange +: : : +- CometProject +: : : +- CometFilter +: : : +- CometNativeScan parquet spark_catalog.default.time_dim +: : +- CometBroadcastExchange +: : +- CometProject +: : +- CometFilter +: : +- CometNativeScan parquet spark_catalog.default.store +: +- BroadcastExchange +: +- CometNativeColumnarToRow +: +- CometHashAggregate +: +- CometExchange +: +- CometHashAggregate +: +- CometProject +: +- CometBroadcastHashJoin +: :- CometProject +: : +- CometBroadcastHashJoin +: : :- CometProject +: : : +- CometBroadcastHashJoin +: : : :- CometProject +: : : : +- CometFilter +: : : : +- CometNativeScan parquet spark_catalog.default.store_sales +: : : +- CometBroadcastExchange +: : : +- CometProject +: : : +- CometFilter +: : : +- CometNativeScan parquet spark_catalog.default.household_demographics +: : +- CometBroadcastExchange +: : +- CometProject +: : +- CometFilter +: : +- CometNativeScan parquet spark_catalog.default.time_dim +: +- CometBroadcastExchange +: +- CometProject +: +- CometFilter +: +- CometNativeScan parquet spark_catalog.default.store ++- BroadcastExchange + +- CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -206,4 +213,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 206 out of 206 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 192 out of 206 eligible operators (93%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/extended.txt index ebe68e5447..b1125e15e4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/extended.txt @@ -1,31 +1,32 @@ -CometNativeColumnarToRow -+- CometProject - +- CometBroadcastNestedLoopJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.time_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- CometBroadcastExchange +Project ++- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + :- CometNativeColumnarToRow + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.household_demographics + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.time_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_page + +- BroadcastExchange + +- CometNativeColumnarToRow +- CometHashAggregate +- CometExchange +- CometHashAggregate @@ -51,4 +52,4 @@ CometNativeColumnarToRow +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 51 out of 51 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 48 out of 51 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt deleted file mode 100644 index 43cba378a4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastNestedLoopJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometNativeScan parquet spark_catalog.default.warehouse - -Comet accelerated 27 out of 28 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/extended.txt deleted file mode 100644 index f7614895a6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/extended.txt +++ /dev/null @@ -1,335 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- CometProject - : : +- CometBroadcastNestedLoopJoin - : : :- CometBroadcastExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- CometProject - : : +- CometBroadcastNestedLoopJoin - : : :- CometBroadcastExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- CometProject - : +- CometBroadcastNestedLoopJoin - : :- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page - -Comet accelerated 314 out of 332 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/extended.txt deleted file mode 100644 index 43cba378a4..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/extended.txt +++ /dev/null @@ -1,31 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastNestedLoopJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometNativeScan parquet spark_catalog.default.warehouse - -Comet accelerated 27 out of 28 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/extended.txt deleted file mode 100644 index f7614895a6..0000000000 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/extended.txt +++ /dev/null @@ -1,335 +0,0 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- CometProject - : : +- CometBroadcastNestedLoopJoin - : : :- CometBroadcastExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- CometProject - : : +- CometBroadcastNestedLoopJoin - : : :- CometBroadcastExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- CometProject - : +- CometBroadcastNestedLoopJoin - : :- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page - -Comet accelerated 314 out of 332 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/extended.txt index 43cba378a4..7d564dd7a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22/extended.txt @@ -1,31 +1,32 @@ -CometNativeColumnarToRow -+- CometTakeOrderedAndProject - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastNestedLoopJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.inventory - : : : +- CometSubqueryBroadcast - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.item - +- CometBroadcastExchange +TakeOrderedAndProject ++- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] + +- Exchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.inventory + : : : +- CometSubqueryBroadcast + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.item + +- BroadcastExchange + +- CometNativeColumnarToRow +- CometNativeScan parquet spark_catalog.default.warehouse -Comet accelerated 27 out of 28 eligible operators (96%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 19 out of 28 eligible operators (67%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/extended.txt index f7614895a6..7831e37634 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/extended.txt @@ -1,335 +1,347 @@ CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] + : +- Exchange + : +- HashAggregate + : +- Union + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : :- Project + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + : : :- BroadcastExchange + : : : +- CometNativeColumnarToRow : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate : : +- CometProject : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- ReusedSubquery : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- CometProject - : : +- CometBroadcastNestedLoopJoin - : : :- CometBroadcastExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometHashAggregate - : : : +- CometExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_page + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_page + :- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] + : +- Exchange + : +- HashAggregate + : +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] + : +- Exchange + : +- HashAggregate + : +- Union + : :- CometNativeColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : : +- CometSubqueryBroadcast + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : :- Project + : : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + : : :- BroadcastExchange + : : : +- CometNativeColumnarToRow : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : : +- CometSubqueryBroadcast - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store - : : +- CometBroadcastExchange + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometNativeColumnarToRow : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate : : +- CometProject : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : : +- ReusedSubquery : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : :- CometProject - : : +- CometBroadcastNestedLoopJoin - : : :- CometBroadcastExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_page - : +- CometBroadcastExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometBroadcastHashJoin - : :- CometHashAggregate - : : +- CometExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_page + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_page + +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] + +- Exchange + +- HashAggregate + +- HashAggregate [COMET: Spark Final aggregate without Comet Partial requires compatible intermediate buffer formats] + +- Exchange + +- HashAggregate + +- Union + :- CometNativeColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_sales + : : : : +- CometSubqueryBroadcast + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.store + :- Project + : +- BroadcastNestedLoopJoin [COMET: BroadcastNestedLoopJoin is not supported] + : :- BroadcastExchange + : : +- CometNativeColumnarToRow : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.store_sales - : : : : +- CometSubqueryBroadcast - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.store - : +- CometBroadcastExchange + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometNativeScan parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometNativeColumnarToRow : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate : +- CometProject : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim + : :- CometNativeScan parquet spark_catalog.default.catalog_returns + : : +- ReusedSubquery : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.store - :- CometProject - : +- CometBroadcastNestedLoopJoin - : :- CometBroadcastExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometProject - +- CometBroadcastHashJoin - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.web_page - +- CometBroadcastExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.web_returns - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometNativeScan parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometNativeScan parquet spark_catalog.default.web_page + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometNativeColumnarToRow + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.web_page + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.web_returns + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometNativeScan parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 314 out of 332 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 285 out of 332 eligible operators (85%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala index 96deae66e0..29c9a7833f 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala @@ -25,7 +25,7 @@ import org.scalatest.Tag import org.apache.spark.sql.CometTestBase import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation -import org.apache.spark.sql.comet.{CometBroadcastExchangeExec, CometBroadcastHashJoinExec, CometBroadcastNestedLoopJoinExec, CometSortMergeJoinExec} +import org.apache.spark.sql.comet.{CometBroadcastExchangeExec, CometBroadcastHashJoinExec, CometSortMergeJoinExec} import org.apache.spark.sql.execution.adaptive.AQEShuffleReadExec import org.apache.spark.sql.internal.SQLConf @@ -702,211 +702,4 @@ class CometJoinSuite extends CometTestBase { } } } - - test("BroadcastNestedLoopJoin with unequal filter") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - // Include NULL keys: predicate `_1 > _1` returns NULL with a NULL operand, so - // those rows must not contribute to the join output. - val left: Seq[(Integer, Int)] = - (0 until 100).map(i => ((i: Integer), i % 5)) ++ Seq[(Integer, Int)]((null, 7), (50, -1)) - val right: Seq[(Integer, Int)] = - (0 until 10).map(i => ((i: Integer), i + 5)) ++ Seq[(Integer, Int)]((null, 1)) - withParquetTable(left, "tbl_a") { - withParquetTable(right, "tbl_b") { - val df = - sql("SELECT /*+ BROADCAST(tbl_b) */ * FROM tbl_a JOIN tbl_b ON tbl_a._1 > tbl_b._1") - checkSparkAnswerAndOperator( - df, - Seq(classOf[CometBroadcastExchangeExec], classOf[CometBroadcastNestedLoopJoinExec])) - } - } - } - } - - test("BroadcastNestedLoopJoin cross join with count-only output") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - val left: Seq[(Integer, Int)] = - (0 until 100).map(i => ((i: Integer), i % 5)) ++ Seq[(Integer, Int)]((null, 9)) - val right: Seq[(Integer, String)] = - (0 until 5).map(i => ((i: Integer), s"w_$i")) ++ Seq[(Integer, String)]((null, "w_null")) - withParquetTable(left, "tbl_a") { - withParquetTable(right, "tbl_b") { - val df = sql("SELECT /*+ BROADCAST(tbl_b) */ count(*) FROM tbl_a, tbl_b") - checkSparkAnswerAndOperator( - df, - Seq(classOf[CometBroadcastExchangeExec], classOf[CometBroadcastNestedLoopJoinExec])) - } - } - } - } - - test("BroadcastNestedLoopJoin LEFT OUTER with inequality") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - // NULL left keys must still appear in the output (LEFT OUTER preserves them) - val left: Seq[(Integer, Int)] = - (0 until 100).map(i => ((i: Integer), i % 5)) ++ Seq[(Integer, Int)]((null, 7), (50, -1)) - val right: Seq[(Integer, Int)] = - (0 until 10).map(i => ((i: Integer), i + 5)) ++ Seq[(Integer, Int)]((null, 1)) - withParquetTable(left, "tbl_a") { - withParquetTable(right, "tbl_b") { - val df = - sql( - "SELECT /*+ BROADCAST(tbl_b) */ * FROM tbl_a LEFT JOIN tbl_b ON tbl_a._1 > tbl_b._1") - checkSparkAnswerAndOperator( - df, - Seq(classOf[CometBroadcastExchangeExec], classOf[CometBroadcastNestedLoopJoinExec])) - } - } - } - } - - test("BroadcastNestedLoopJoin LEFT SEMI with inequality") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - // NULL keys never match (predicate evaluates to NULL - val left: Seq[(Integer, Int)] = - (0 until 100).map(i => ((i: Integer), i % 5)) ++ Seq[(Integer, Int)]((null, 7), (50, -1)) - val right: Seq[(Integer, Int)] = - (0 until 10).map(i => ((i: Integer), i + 5)) ++ Seq[(Integer, Int)]((null, 1)) - withParquetTable(left, "tbl_a") { - withParquetTable(right, "tbl_b") { - val df = - sql("SELECT /*+ BROADCAST(tbl_b) */ * FROM tbl_a LEFT SEMI JOIN tbl_b ON tbl_a._1 > tbl_b._1") - checkSparkAnswerAndOperator( - df, - Seq(classOf[CometBroadcastExchangeExec], classOf[CometBroadcastNestedLoopJoinExec])) - } - } - } - } - - test("BroadcastNestedLoopJoin LEFT ANTI with inequality") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - // LEFT ANTI keeps left rows that have NO match (left rows with NULL keys must appear in the output) - val left: Seq[(Integer, Int)] = - (0 until 100).map(i => ((i: Integer), i % 5)) ++ Seq[(Integer, Int)]((null, 7), (50, -1)) - val right: Seq[(Integer, Int)] = - (0 until 10).map(i => ((i: Integer), i + 5)) ++ Seq[(Integer, Int)]((null, 1)) - withParquetTable(left, "tbl_a") { - withParquetTable(right, "tbl_b") { - val df = - sql("SELECT /*+ BROADCAST(tbl_b) */ * FROM tbl_a LEFT ANTI JOIN tbl_b ON tbl_a._1 > tbl_b._1") - checkSparkAnswerAndOperator( - df, - Seq(classOf[CometBroadcastExchangeExec], classOf[CometBroadcastNestedLoopJoinExec])) - } - } - } - } - - test("BroadcastNestedLoopJoin RIGHT OUTER with inequality (BuildLeft, swap path)") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - // RIGHT OUTER preserves right rows. - val left: Seq[(Integer, Int)] = - (0 until 10).map(i => ((i: Integer), i + 5)) ++ Seq[(Integer, Int)]((null, 9)) - val right: Seq[(Integer, Int)] = - (0 until 100).map(i => ((i: Integer), i % 5)) ++ Seq[(Integer, Int)]((null, 7), (50, -1)) - withParquetTable(left, "tbl_a") { - withParquetTable(right, "tbl_b") { - val df = - sql("SELECT /*+ BROADCAST(tbl_a) */ * FROM tbl_a RIGHT JOIN tbl_b ON tbl_a._1 < tbl_b._1") - checkSparkAnswerAndOperator( - df, - Seq(classOf[CometBroadcastExchangeExec], classOf[CometBroadcastNestedLoopJoinExec])) - } - } - } - } - - test("BroadcastNestedLoopJoin cross join without condition (materialized rows)") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - val left: Seq[(Integer, Int)] = - (0 until 5).map(i => ((i: Integer), i * 10)) ++ Seq[(Integer, Int)]((null, 99)) - val right: Seq[(Integer, String)] = - (0 until 4).map(i => ((i: Integer), s"v_$i")) ++ Seq[(Integer, String)]((null, "v_null")) - withParquetTable(left, "tbl_a") { - withParquetTable(right, "tbl_b") { - val df = - sql("SELECT /*+ BROADCAST(tbl_b) */ tbl_a._1, tbl_b._2 FROM tbl_a, tbl_b") - checkSparkAnswerAndOperator( - df, - Seq(classOf[CometBroadcastExchangeExec], classOf[CometBroadcastNestedLoopJoinExec])) - } - } - } - } - - test("BroadcastNestedLoopJoin LEFT OUTER without condition") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - val left: Seq[(Integer, Int)] = - (0 until 5).map(i => ((i: Integer), i * 10)) ++ Seq[(Integer, Int)]((null, 99)) - val right: Seq[(Integer, String)] = - (0 until 4).map(i => ((i: Integer), s"v_$i")) ++ Seq[(Integer, String)]((null, "v_null")) - withParquetTable(left, "tbl_a") { - withParquetTable(right, "tbl_b") { - val df = - sql( - "SELECT /*+ BROADCAST(tbl_b) */ tbl_a._1, tbl_b._2" + - " FROM tbl_a LEFT JOIN tbl_b ON true") - checkSparkAnswerAndOperator( - df, - Seq(classOf[CometBroadcastExchangeExec], classOf[CometBroadcastNestedLoopJoinExec])) - } - } - } - } - - test("BroadcastNestedLoopJoin broadcast reuse across two joins") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - // Same broadcast relation (tbl_b) feeds two separate BNLJs. Spark/AQE handles - // broadcast-exchange reuse generically rather than inside BNLJ, so this verifies - // we still produce correct results when reuse fires across CometBNLJ consumers. - withParquetTable((0 until 50).map(i => (i, i % 10)), "tbl_a") { - withParquetTable((0 until 50).map(i => (i, i + 1)), "tbl_c") { - withParquetTable((0 until 5).map(i => (i, i * 10)), "tbl_b") { - val df = sql( - "SELECT count(*) FROM" + - " (SELECT /*+ BROADCAST(tbl_b) */ tbl_a._1 AS k FROM tbl_a JOIN tbl_b" + - " ON tbl_a._1 > tbl_b._1) a" + - " JOIN" + - " (SELECT /*+ BROADCAST(tbl_b) */ tbl_c._1 AS k FROM tbl_c JOIN tbl_b" + - " ON tbl_c._1 > tbl_b._1) c" + - " ON a.k = c.k") - checkSparkAnswerAndOperator( - df, - Seq(classOf[CometBroadcastExchangeExec], classOf[CometBroadcastNestedLoopJoinExec])) - } - } - } - } - } - - test("BroadcastNestedLoopJoin FULL OUTER falls back to Spark") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - withParquetTable((0 until 50).map(i => (i, i % 5)), "tbl_a") { - withParquetTable((0 until 10).map(i => (i, i + 100)), "tbl_b") { - val df = - sql( - "SELECT /*+ BROADCAST(tbl_b) */ * FROM tbl_a FULL OUTER JOIN tbl_b" + - " ON tbl_a._1 > tbl_b._1") - checkSparkAnswer(df) - } - } - } - } - - test("BroadcastNestedLoopJoin LEFT OUTER with BuildLeft falls back to Spark") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { - withParquetTable((0 until 10).map(i => (i, i + 100)), "tbl_a") { - withParquetTable((0 until 50).map(i => (i, i % 5)), "tbl_b") { - // Broadcasting the preserved (left) side forces BuildLeft + LeftOuter, an - // unsupported combo. Comet should fall back to Spark and still match. - val df = - sql( - "SELECT /*+ BROADCAST(tbl_a) */ * FROM tbl_a LEFT OUTER JOIN tbl_b" + - " ON tbl_a._1 > tbl_b._1") - checkSparkAnswer(df) - } - } - } - } } diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBroadcastNestedLoopJoinBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBroadcastNestedLoopJoinBenchmark.scala deleted file mode 100644 index e86903f6ea..0000000000 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBroadcastNestedLoopJoinBenchmark.scala +++ /dev/null @@ -1,123 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.spark.sql.benchmark - -import org.apache.spark.SparkConf -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.internal.SQLConf - -import org.apache.comet.{CometConf, CometSparkSessionExtensions} - -/** - * Benchmark to measure performance of Comet BroadcastNestedLoopJoin. To run this benchmark: - * {{{ - * SPARK_GENERATE_BENCHMARK_FILES=1 make benchmark-org.apache.spark.sql.benchmark.CometBroadcastNestedLoopJoinBenchmark - * }}} - * Results will be written to - * "spark/benchmarks/CometBroadcastNestedLoopJoinBenchmark-**results.txt". - */ -object CometBroadcastNestedLoopJoinBenchmark extends CometBenchmarkBase { - - override def getSparkSession: SparkSession = { - val conf = new SparkConf() - .setAppName("CometBroadcastNestedLoopJoinBenchmark") - .set("spark.master", "local[5]") - .setIfMissing("spark.driver.memory", "3g") - .setIfMissing("spark.executor.memory", "3g") - .set( - "spark.shuffle.manager", - "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") - - val sparkSession = SparkSession.builder - .config(conf) - .withExtensions(new CometSparkSessionExtensions) - .getOrCreate() - - sparkSession.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true") - sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") - sparkSession.conf.set(CometConf.COMET_ENABLED.key, "false") - sparkSession.conf.set(CometConf.COMET_EXEC_ENABLED.key, "false") - sparkSession.conf.set(SQLConf.ANSI_ENABLED.key, "false") - sparkSession.conf.set("spark.sql.shuffle.partitions", "2") - - sparkSession - } - - private val cometConfigs: Map[String, String] = Map.empty[String, String] - - override def runCometBenchmark(mainArgs: Array[String]): Unit = { - val probeRows = 1024 * 1024 - val buildRows = 100 - - withTempPath { dir => - withTempTable("probe", "build") { - spark - .range(probeRows) - .selectExpr("id AS k", "id % 100 AS v") - .write - .parquet(s"${dir.getAbsolutePath}/probe") - spark - .range(buildRows) - .selectExpr("id * 1000 AS lo", "id * 1000 + 500 AS hi") - .write - .parquet(s"${dir.getAbsolutePath}/build") - - spark.read.parquet(s"${dir.getAbsolutePath}/probe").createOrReplaceTempView("probe") - spark.read.parquet(s"${dir.getAbsolutePath}/build").createOrReplaceTempView("build") - - runBenchmark("BroadcastNestedLoopJoin - range") { - runExpressionBenchmark( - "range join (BETWEEN)", - probeRows, - "SELECT /*+ BROADCAST(b) */ count(*) FROM probe p " + - "JOIN build b ON p.k BETWEEN b.lo AND b.hi", - cometConfigs) - } - - runBenchmark("BroadcastNestedLoopJoin - inequality") { - runExpressionBenchmark( - "inequality join (>)", - probeRows, - "SELECT /*+ BROADCAST(b) */ count(*) FROM probe p " + - "JOIN build b ON p.k > b.lo", - cometConfigs) - } - - runBenchmark("BroadcastNestedLoopJoin - left outer with non-equi") { - runExpressionBenchmark( - "left outer non-equi", - probeRows, - "SELECT /*+ BROADCAST(b) */ count(*) FROM probe p " + - "LEFT OUTER JOIN build b ON p.k BETWEEN b.lo AND b.hi", - cometConfigs) - } - - runBenchmark("BroadcastNestedLoopJoin - range, materialized rows") { - runExpressionBenchmark( - "range join (BETWEEN, projected)", - probeRows, - "SELECT /*+ BROADCAST(b) */ p.k, p.v, b.lo, b.hi FROM probe p " + - "JOIN build b ON p.k BETWEEN b.lo AND b.hi", - cometConfigs) - } - } - } - } -}