diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index 792aa1a82c..62ed8922db 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..f098beeca26 100644 +index 44c8cb92fc3..e29cb93ecda 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,7 +825,19 @@ index 44c8cb92fc3..f098beeca26 100644 checkAnswer(shjNonCodegenDF, Seq.empty) } } -@@ -1486,7 +1507,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -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 val plan = sql(getAggQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) // Have shuffle before aggregation @@ -835,12 +847,17 @@ index 44c8cb92fc3..f098beeca26 100644 } def getJoinQuery(selectExpr: String, joinType: String): String = { -@@ -1515,9 +1537,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1514,10 +1539,16 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + "/*+ BROADCAST(right_t) */ k1 as k0" } 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) @@ -850,7 +867,7 @@ index 44c8cb92fc3..f098beeca26 100644 } // Test output ordering is not preserved -@@ -1526,9 +1551,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1526,9 +1557,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) @@ -865,7 +882,7 @@ index 44c8cb92fc3..f098beeca26 100644 } // Test singe partition -@@ -1538,7 +1566,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1538,7 +1572,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 @@ -875,7 +892,7 @@ index 44c8cb92fc3..f098beeca26 100644 checkAnswer(fullJoinDF, Row(100)) } } -@@ -1611,6 +1640,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1611,6 +1646,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 @@ -885,7 +902,7 @@ index 44c8cb92fc3..f098beeca26 100644 }.size == 1) } } -@@ -1655,14 +1687,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1655,14 +1693,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 = { @@ -908,7 +925,7 @@ index 44c8cb92fc3..f098beeca26 100644 } dupStreamSideColTest("SHUFFLE_HASH", check) } -@@ -1798,7 +1836,8 @@ class ThreadLeakInSortMergeJoinSuite +@@ -1798,7 +1842,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 b7dcc042c2..6b78928624 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..fbef0774d46 100644 +index aaac0ebc9aa..276c592ec88 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,7 +972,19 @@ index aaac0ebc9aa..fbef0774d46 100644 checkAnswer(shjNonCodegenDF, Seq.empty) } } -@@ -1489,7 +1510,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -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 val plan = sql(getAggQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) // Have shuffle before aggregation @@ -982,12 +994,17 @@ index aaac0ebc9aa..fbef0774d46 100644 } def getJoinQuery(selectExpr: String, joinType: String): String = { -@@ -1518,9 +1540,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1517,10 +1542,16 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + "/*+ BROADCAST(right_t) */ k1 as k0" } 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) @@ -997,7 +1014,7 @@ index aaac0ebc9aa..fbef0774d46 100644 } // Test output ordering is not preserved -@@ -1529,9 +1554,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1529,9 +1560,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) @@ -1012,7 +1029,7 @@ index aaac0ebc9aa..fbef0774d46 100644 } // Test singe partition -@@ -1541,7 +1569,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1541,7 +1575,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 @@ -1022,7 +1039,7 @@ index aaac0ebc9aa..fbef0774d46 100644 checkAnswer(fullJoinDF, Row(100)) } } -@@ -1614,6 +1643,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1614,6 +1649,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 @@ -1032,7 +1049,7 @@ index aaac0ebc9aa..fbef0774d46 100644 }.size == 1) } } -@@ -1658,14 +1690,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1658,14 +1696,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 = { @@ -1055,7 +1072,7 @@ index aaac0ebc9aa..fbef0774d46 100644 } dupStreamSideColTest("SHUFFLE_HASH", check) } -@@ -1801,7 +1839,8 @@ class ThreadLeakInSortMergeJoinSuite +@@ -1801,7 +1845,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 f9911ce56d..0fec94e6d5 100644 --- a/dev/diffs/4.1.2.diff +++ b/dev/diffs/4.1.2.diff @@ -1043,6 +1043,18 @@ 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) @@ -1053,12 +1065,16 @@ index 885512d4d19..113ae17ad9f 100644 } def getJoinQuery(selectExpr: String, joinType: String): String = { -@@ -1530,9 +1553,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1530,10 +1556,16 @@ 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 495141f9bb..d4f016f1ae 100644 --- a/docs/source/user-guide/latest/operators.md +++ b/docs/source/user-guide/latest/operators.md @@ -27,6 +27,7 @@ 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 a4d14a1228..8adccc0929 100644 --- a/docs/source/user-guide/latest/understanding-comet-plans.md +++ b/docs/source/user-guide/latest/understanding-comet-plans.md @@ -202,21 +202,22 @@ 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` | -| `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` | +| `CometBroadcastNestedLoopJoin` | `BroadcastNestedLoopJoinExec` | +| `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 c0c6319d7a..4a7a21006d 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -239,6 +239,7 @@ 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 c6160bddd4..2bebca33b6 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -106,6 +106,7 @@ 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; @@ -1201,6 +1202,57 @@ impl PhysicalPlanner { )) } } + + 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 eb31184461..dc98a32fb0 100644 --- a/native/core/src/execution/planner/operator_registry.rs +++ b/native/core/src/execution/planner/operator_registry.rs @@ -151,5 +151,6 @@ 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 7f50aa928c..612cb20f8e 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -53,6 +53,7 @@ message Operator { Explode explode = 114; CsvScan csv_scan = 115; ShuffleScan shuffle_scan = 116; + BroadcastNestedLoopJoin broadcast_nested_loop_join = 117; } } @@ -390,6 +391,12 @@ 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 78ea0f0168..e1b93604fd 100644 --- a/spark/src/main/scala/org/apache/comet/CometConf.scala +++ b/spark/src/main/scala/org/apache/comet/CometConf.scala @@ -254,6 +254,8 @@ 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 d116d2f407..e261ac45d1 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, ShuffledHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -81,6 +81,7 @@ 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/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index 8cbf7c9189..a96ec3d6b1 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 @@ -40,7 +40,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, HashJoin, ShuffledHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, 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 @@ -1924,6 +1924,158 @@ 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) +} + +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]] = 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 new file mode 100644 index 0000000000..6bc95e1a33 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/extended.txt @@ -0,0 +1,67 @@ +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 new file mode 100644 index 0000000000..1fd67d1103 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/extended.txt @@ -0,0 +1,86 @@ +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 new file mode 100644 index 0000000000..67c31a6ac1 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/extended.txt @@ -0,0 +1,112 @@ +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 new file mode 100644 index 0000000000..603d8e0b52 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/extended.txt @@ -0,0 +1,209 @@ +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 new file mode 100644 index 0000000000..ebe68e5447 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/extended.txt @@ -0,0 +1,54 @@ +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 new file mode 100644 index 0000000000..6bc95e1a33 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/extended.txt @@ -0,0 +1,67 @@ +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 new file mode 100644 index 0000000000..1fd67d1103 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/extended.txt @@ -0,0 +1,86 @@ +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 new file mode 100644 index 0000000000..67c31a6ac1 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/extended.txt @@ -0,0 +1,112 @@ +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 new file mode 100644 index 0000000000..603d8e0b52 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/extended.txt @@ -0,0 +1,209 @@ +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 new file mode 100644 index 0000000000..ebe68e5447 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/extended.txt @@ -0,0 +1,54 @@ +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 6dbeba7e9a..6bc95e1a33 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,64 +1,59 @@ -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 +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 @@ -69,4 +64,4 @@ BroadcastNestedLoopJoin +- CometFilter +- CometNativeScan parquet spark_catalog.default.store_sales -Comet accelerated 54 out of 64 eligible operators (84%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +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/q61/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/extended.txt index 39f985df28..1fd67d1103 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,53 +1,52 @@ -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 +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 @@ -84,4 +83,4 @@ Project +- CometFilter +- CometNativeScan parquet spark_catalog.default.item -Comet accelerated 78 out of 83 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +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/q77/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/extended.txt index 5bffdf3688..67c31a6ac1 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,79 +1,76 @@ -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 +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 @@ -112,4 +109,4 @@ TakeOrderedAndProject +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 94 out of 109 eligible operators (86%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +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/q88/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q88/extended.txt index 87f19deeb8..603d8e0b52 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,193 +1,186 @@ -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 +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 @@ -213,4 +206,4 @@ BroadcastNestedLoopJoin +- CometFilter +- CometNativeScan parquet spark_catalog.default.store -Comet accelerated 192 out of 206 eligible operators (93%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +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/q90/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/extended.txt index b1125e15e4..ebe68e5447 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,32 +1,31 @@ -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 +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 @@ -52,4 +51,4 @@ Project +- CometFilter +- CometNativeScan parquet spark_catalog.default.web_page -Comet accelerated 48 out of 51 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +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-v2_7-spark3_5/q22/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt new file mode 100644 index 0000000000..43cba378a4 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt @@ -0,0 +1,31 @@ +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 new file mode 100644 index 0000000000..f7614895a6 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/extended.txt @@ -0,0 +1,335 @@ +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 new file mode 100644 index 0000000000..43cba378a4 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/extended.txt @@ -0,0 +1,31 @@ +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 new file mode 100644 index 0000000000..f7614895a6 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/extended.txt @@ -0,0 +1,335 @@ +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 7d564dd7a2..43cba378a4 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,32 +1,31 @@ -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 +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 19 out of 28 eligible operators (67%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +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/q77a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q77a/extended.txt index 7831e37634..f7614895a6 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,347 +1,335 @@ CometNativeColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- 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 + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeColumnarToRow + : : : +- 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 - : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange - : : +- 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 + : : +- 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 - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometNativeScan parquet spark_catalog.default.date_dim - : : +- CometNativeColumnarToRow + : : : +- 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 - : : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : : +- ReusedSubquery + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.date_dim : : +- CometBroadcastExchange - : : +- 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 + : : +- 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 - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometNativeScan parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometNativeScan parquet spark_catalog.default.date_dim - : +- CometNativeColumnarToRow + : : +- 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 - : :- CometNativeScan parquet spark_catalog.default.catalog_returns - : : +- ReusedSubquery + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometNativeScan parquet spark_catalog.default.store_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometNativeScan parquet spark_catalog.default.date_dim : +- CometBroadcastExchange - : +- 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 + : +- 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 285 out of 332 eligible operators (85%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +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/scala/org/apache/comet/exec/CometJoinSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala index 29c9a7833f..3db36b782e 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, CometSortMergeJoinExec} +import org.apache.spark.sql.comet.{CometBroadcastExchangeExec, CometBroadcastHashJoinExec, CometBroadcastNestedLoopJoinExec, CometSortMergeJoinExec} import org.apache.spark.sql.execution.adaptive.AQEShuffleReadExec import org.apache.spark.sql.internal.SQLConf @@ -38,7 +38,10 @@ class CometJoinSuite extends CometTestBase { override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit pos: Position): Unit = { super.test(testName, testTags: _*) { - withSQLConf(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true") { + withSQLConf( + CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_EXEC_BROADCAST_NESTED_LOOP_JOIN_ENABLED.key -> "true", + CometConf.getOperatorAllowIncompatConfigKey("BroadcastNestedLoopJoinExec") -> "true") { testFun } } @@ -702,4 +705,102 @@ class CometJoinSuite extends CometTestBase { } } } + + test("BroadcastNestedLoopJoin with unequal filter") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + withParquetTable((0 until 100).map(i => (i, i % 5)), "tbl_a") { + withParquetTable((0 until 10).map(i => (i, i + 5)), "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") { + withParquetTable((0 until 100).map(i => (i, i % 5)), "tbl_a") { + withParquetTable((0 until 5).map(i => (i, s"w_$i")), "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") { + withParquetTable((0 until 100).map(i => (i, i % 5)), "tbl_a") { + withParquetTable((0 until 10).map(i => (i, i + 5)), "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") { + withParquetTable((0 until 100).map(i => (i, i % 5)), "tbl_a") { + withParquetTable((0 until 10).map(i => (i, i + 5)), "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") { + withParquetTable((0 until 100).map(i => (i, i % 5)), "tbl_a") { + withParquetTable((0 until 10).map(i => (i, i + 5)), "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") { + withParquetTable((0 until 10).map(i => (i, i + 5)), "tbl_a") { + withParquetTable((0 until 100).map(i => (i, i % 5)), "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") { + withParquetTable((0 until 5).map(i => (i, i * 10)), "tbl_a") { + withParquetTable((0 until 4).map(i => (i, s"v_$i")), "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])) + } + } + } + } } 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 new file mode 100644 index 0000000000..e86903f6ea --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBroadcastNestedLoopJoinBenchmark.scala @@ -0,0 +1,123 @@ +/* + * 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) + } + } + } + } +}