From 692cc374eddf05001e02ef564938d3af15f040be Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Sun, 24 May 2026 23:42:41 -0700 Subject: [PATCH 01/24] native_support_nested_loop_join --- native/core/src/execution/jni_api.rs | 1 + native/core/src/execution/planner.rs | 35 ++++++ .../execution/planner/operator_registry.rs | 1 + native/proto/src/proto/operator.proto | 7 ++ .../apache/comet/rules/CometExecRule.scala | 4 +- .../apache/spark/sql/comet/operators.scala | 108 +++++++++++++++++- 6 files changed, 154 insertions(+), 2 deletions(-) diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 97e3f851c5..395cb2be08 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -240,6 +240,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 89d118b059..c271b71f55 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -134,6 +134,7 @@ use num::{BigInt, ToPrimitive}; use object_store::path::Path; use std::cmp::max; use std::{collections::HashMap, sync::Arc}; +use datafusion::physical_plan::joins::NestedLoopJoinExec; use url::Url; // For clippy error on type_complexity. @@ -1197,6 +1198,40 @@ impl PhysicalPlanner { )) } } + + OpStruct::BroadcastNestedLoopJoin(bnlj) => { + let (join_params, scans, shuffle_scans) = self.parse_join_parameters( + inputs, + &[], + &[], + &[], + 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 + )?); + + 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..3b96652c90 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(_) => todo!(), } } diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index ed1684b240..b8e6b13499 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; } } @@ -384,6 +385,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/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index aeb7db40ad..b94b2c7f93 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._ @@ -54,6 +54,7 @@ import org.apache.comet.CometConf.{COMET_SPARK_TO_ARROW_ENABLED, COMET_SPARK_TO_ import org.apache.comet.CometSparkSessionExtensions._ import org.apache.comet.rules.CometExecRule.allExecs import org.apache.comet.serde._ +import org.apache.comet.serde.OperatorOuterClass.BroadcastNestedLoopJoin import org.apache.comet.serde.operator._ import org.apache.comet.shims.{ShimCometStreaming, ShimSubqueryBroadcast} @@ -81,6 +82,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 7d5398ae62..ee13f9bf84 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 @@ -1918,6 +1918,112 @@ 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 { + + override def withNewChildrenInternal(newLeft: SparkPlan, newRight: SparkPlan): SparkPlan = + this.copy(left = newLeft, right = newRight) +} + +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_HASH_JOIN_ENABLED) + } + + /** + * 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 + withInfo(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) { + withInfo(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]] = From 27e7bf06c90fad888b6136def9ac9b6b106f3ca9 Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Mon, 25 May 2026 10:21:33 -0700 Subject: [PATCH 02/24] native_support_nested_loop_join --- native/core/src/execution/planner.rs | 8 ++++---- native/core/src/execution/planner/operator_registry.rs | 2 +- spark/src/main/scala/org/apache/comet/CometConf.scala | 2 ++ .../main/scala/org/apache/spark/sql/comet/operators.scala | 5 ++++- 4 files changed, 11 insertions(+), 6 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index c271b71f55..a9f802e6f4 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; @@ -134,7 +135,6 @@ use num::{BigInt, ToPrimitive}; use object_store::path::Path; use std::cmp::max; use std::{collections::HashMap, sync::Arc}; -use datafusion::physical_plan::joins::NestedLoopJoinExec; use url::Url; // For clippy error on type_complexity. @@ -1202,12 +1202,12 @@ impl PhysicalPlanner { OpStruct::BroadcastNestedLoopJoin(bnlj) => { let (join_params, scans, shuffle_scans) = self.parse_join_parameters( inputs, - &[], + children, &[], &[], bnlj.join_type, &bnlj.condition, - partition_count + partition_count, )?; let left = Arc::clone(&join_params.left.native_plan); @@ -1218,7 +1218,7 @@ impl PhysicalPlanner { right, join_params.join_filter, &join_params.join_type, - None + None, )?); Ok(( diff --git a/native/core/src/execution/planner/operator_registry.rs b/native/core/src/execution/planner/operator_registry.rs index 3b96652c90..dc98a32fb0 100644 --- a/native/core/src/execution/planner/operator_registry.rs +++ b/native/core/src/execution/planner/operator_registry.rs @@ -151,6 +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(_) => todo!(), + OpStruct::BroadcastNestedLoopJoin(_) => None, } } diff --git a/spark/src/main/scala/org/apache/comet/CometConf.scala b/spark/src/main/scala/org/apache/comet/CometConf.scala index 82700a939e..43b8a5cf3f 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 = false) 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/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index ee13f9bf84..053f9a9f2d 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 @@ -2026,8 +2026,11 @@ object CometBroadcastNestedLoopJoinExec extends CometOperatorSerde[BroadcastNest object CometBroadcastHashJoinExec extends CometOperatorSerde[HashJoin] with CometHashJoin { + override def getSupportLevel(operator: HashJoin): SupportLevel = { + Incompatible(Some("Broadcast nested loop join is experimental")) + } override def enabledConfig: Option[ConfigEntry[Boolean]] = - Some(CometConf.COMET_EXEC_HASH_JOIN_ENABLED) + Some(CometConf.COMET_EXEC_BROADCAST_NESTED_LOOP_JOIN_ENABLED) override def convert( join: HashJoin, From 57c26b8181db9e8e6edd14eafc5a97dbeb8a1693 Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Mon, 25 May 2026 10:40:22 -0700 Subject: [PATCH 03/24] native_support_nested_loop_join --- spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala | 1 - 1 file changed, 1 deletion(-) 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 b94b2c7f93..95fdaae67e 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -54,7 +54,6 @@ import org.apache.comet.CometConf.{COMET_SPARK_TO_ARROW_ENABLED, COMET_SPARK_TO_ import org.apache.comet.CometSparkSessionExtensions._ import org.apache.comet.rules.CometExecRule.allExecs import org.apache.comet.serde._ -import org.apache.comet.serde.OperatorOuterClass.BroadcastNestedLoopJoin import org.apache.comet.serde.operator._ import org.apache.comet.shims.{ShimCometStreaming, ShimSubqueryBroadcast} From 8c4e6498484e81a8d7387f6d03b72329f75c72d9 Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Mon, 25 May 2026 11:27:18 -0700 Subject: [PATCH 04/24] native_support_nested_loop_join --- .../apache/comet/exec/CometJoinSuite.scala | 20 ++++++++++++++++--- 1 file changed, 17 insertions(+), 3 deletions(-) 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..d3d495c87b 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala @@ -21,14 +21,12 @@ package org.apache.comet.exec import org.scalactic.source.Position 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 - import org.apache.comet.CometConf class CometJoinSuite extends CometTestBase { @@ -702,4 +700,20 @@ class CometJoinSuite extends CometTestBase { } } } + + test("BroadcastNestedLoopJoin with unequal filter") { + withSQLConf( + CometConf.COMET_EXEC_BROADCAST_NESTED_LOOP_JOIN_ENABLED.key -> "true", + CometConf.getOperatorAllowIncompatConfigKey("BroadcastNestedLoopJoinExec") -> "true", + 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])) + } + } + } + } } From 7811135682106ec11dc1326b5078c0b3b5dc2d95 Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Mon, 25 May 2026 11:38:09 -0700 Subject: [PATCH 05/24] native_support_nested_loop_join --- .../test/scala/org/apache/comet/exec/CometJoinSuite.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) 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 d3d495c87b..d72cf9b777 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala @@ -21,12 +21,14 @@ package org.apache.comet.exec import org.scalactic.source.Position import org.scalatest.Tag + import org.apache.spark.sql.CometTestBase import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.comet.{CometBroadcastExchangeExec, CometBroadcastHashJoinExec, CometBroadcastNestedLoopJoinExec, CometSortMergeJoinExec} import org.apache.spark.sql.execution.adaptive.AQEShuffleReadExec import org.apache.spark.sql.internal.SQLConf + import org.apache.comet.CometConf class CometJoinSuite extends CometTestBase { @@ -708,7 +710,8 @@ class CometJoinSuite extends CometTestBase { 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") + 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])) From c4596fffd6c7c4ed2016be6822236795ef26145e Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Mon, 25 May 2026 11:57:50 -0700 Subject: [PATCH 06/24] native_support_nested_loop_join --- spark/src/main/scala/org/apache/comet/CometConf.scala | 2 +- .../main/scala/org/apache/spark/sql/comet/operators.scala | 5 ++++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/CometConf.scala b/spark/src/main/scala/org/apache/comet/CometConf.scala index 43b8a5cf3f..c851ef4b62 100644 --- a/spark/src/main/scala/org/apache/comet/CometConf.scala +++ b/spark/src/main/scala/org/apache/comet/CometConf.scala @@ -255,7 +255,7 @@ object CometConf extends ShimCometConf { 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 = false) + 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/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index 053f9a9f2d..6e07a9de43 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 @@ -1942,9 +1942,12 @@ object CometBroadcastNestedLoopJoinExec extends CometOperatorSerde[BroadcastNest * for this operator. */ override def enabledConfig: Option[ConfigEntry[Boolean]] = { - Some(CometConf.COMET_EXEC_HASH_JOIN_ENABLED) + Some(CometConf.COMET_EXEC_BROADCAST_NESTED_LOOP_JOIN_ENABLED) } + override def getSupportLevel(op: BroadcastNestedLoopJoinExec): SupportLevel = + Compatible(None) + /** * Convert a Spark operator into a protocol buffer representation that can be passed into native * code. From 381ad1c6f1be96892a7f862cac1bcf71ad27d649 Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Mon, 25 May 2026 18:41:19 -0700 Subject: [PATCH 07/24] native_support_nested_loop_join_regen_golden_files --- .../apache/spark/sql/comet/operators.scala | 3 - .../q28/extended.txt | 67 ++++ .../q61/extended.txt | 86 +++++ .../q77/extended.txt | 112 ++++++ .../q88/extended.txt | 209 +++++++++++ .../q90/extended.txt | 54 +++ .../q28/extended.txt | 67 ++++ .../q61/extended.txt | 86 +++++ .../q77/extended.txt | 112 ++++++ .../q88/extended.txt | 209 +++++++++++ .../q90/extended.txt | 54 +++ .../q22/extended.txt | 31 ++ .../q77a/extended.txt | 335 ++++++++++++++++++ .../q22/extended.txt | 31 ++ .../q77a/extended.txt | 335 ++++++++++++++++++ 15 files changed, 1788 insertions(+), 3 deletions(-) create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q28/extended.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q61/extended.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q77/extended.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q88/extended.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q90/extended.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/extended.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/extended.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/extended.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/extended.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/extended.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q22/extended.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q77a/extended.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/extended.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/extended.txt 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 6e07a9de43..987dda7bad 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 @@ -2029,9 +2029,6 @@ object CometBroadcastNestedLoopJoinExec extends CometOperatorSerde[BroadcastNest object CometBroadcastHashJoinExec extends CometOperatorSerde[HashJoin] with CometHashJoin { - override def getSupportLevel(operator: HashJoin): SupportLevel = { - Incompatible(Some("Broadcast nested loop join is experimental")) - } override def enabledConfig: Option[ConfigEntry[Boolean]] = Some(CometConf.COMET_EXEC_BROADCAST_NESTED_LOOP_JOIN_ENABLED) 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-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 From be5e50fd4101edf53e6eb071958015bd0cac714e Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Tue, 26 May 2026 11:13:30 -0700 Subject: [PATCH 08/24] bnlj_test_fix --- spark/src/main/scala/org/apache/spark/sql/comet/operators.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 987dda7bad..e99b2c7083 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 @@ -2030,7 +2030,7 @@ object CometBroadcastNestedLoopJoinExec extends CometOperatorSerde[BroadcastNest object CometBroadcastHashJoinExec extends CometOperatorSerde[HashJoin] with CometHashJoin { override def enabledConfig: Option[ConfigEntry[Boolean]] = - Some(CometConf.COMET_EXEC_BROADCAST_NESTED_LOOP_JOIN_ENABLED) + Some(CometConf.COMET_EXEC_HASH_JOIN_ENABLED) override def convert( join: HashJoin, From 69fd2bb835f28ea9537456cecfef060b693c0ba8 Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Thu, 28 May 2026 16:26:08 -0700 Subject: [PATCH 09/24] add_benches_rebase_main --- .../apache/comet/exec/CometJoinSuite.scala | 80 +++++++++++- ...ometBroadcastNestedLoopJoinBenchmark.scala | 114 ++++++++++++++++++ 2 files changed, 189 insertions(+), 5 deletions(-) create mode 100644 spark/src/test/scala/org/apache/spark/sql/benchmark/CometBroadcastNestedLoopJoinBenchmark.scala 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 d72cf9b777..91a346fdfd 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala @@ -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 } } @@ -704,10 +707,7 @@ class CometJoinSuite extends CometTestBase { } test("BroadcastNestedLoopJoin with unequal filter") { - withSQLConf( - CometConf.COMET_EXEC_BROADCAST_NESTED_LOOP_JOIN_ENABLED.key -> "true", - CometConf.getOperatorAllowIncompatConfigKey("BroadcastNestedLoopJoinExec") -> "true", - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + 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 = @@ -719,4 +719,74 @@ class CometJoinSuite extends CometTestBase { } } } + + 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 RIGHT 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 RIGHT 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])) + } + } + } + } } 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..836382f2c5 --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBroadcastNestedLoopJoinBenchmark.scala @@ -0,0 +1,114 @@ +/* + * 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 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 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 count(*) FROM probe p " + + "LEFT OUTER JOIN build b ON p.k BETWEEN b.lo AND b.hi", + cometConfigs) + } + } + } + } +} From 18497888a3091db6f9d6e8849698244ed927532d Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Thu, 28 May 2026 22:36:34 -0700 Subject: [PATCH 10/24] swap_build_probe --- native/core/src/execution/planner.rs | 37 ++++++++++++++++++++-------- 1 file changed, 27 insertions(+), 10 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index c3f9d815f0..2bebca33b6 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1225,17 +1225,34 @@ impl PhysicalPlanner { None, )?); - Ok(( - scans, - shuffle_scans, - Arc::new(SparkPlan::new( - spark_plan.plan_id, - nested_loop_join, - vec![join_params.left, join_params.right], - )), - )) + 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; From ab4d039c6825bd72f0632691560ae8925a30717f Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Fri, 29 May 2026 11:02:34 -0700 Subject: [PATCH 11/24] mark_joins_needing_dedupe_incompatible --- .../org/apache/spark/sql/comet/operators.scala | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) 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 e99b2c7083..53aee16438 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 @@ -1945,8 +1945,19 @@ object CometBroadcastNestedLoopJoinExec extends CometOperatorSerde[BroadcastNest Some(CometConf.COMET_EXEC_BROADCAST_NESTED_LOOP_JOIN_ENABLED) } + private val unmatchedDuplicationReason = + "BNLJ with preserved-build side (LeftOuter+BuildLeft, RightOuter+BuildRight, FullOuter)" + + " duplicates unmatched rows across partitions because the broadcast side is replicated" + override def getSupportLevel(op: BroadcastNestedLoopJoinExec): SupportLevel = - Compatible(None) + (op.joinType, op.buildSide) match { + case (LeftOuter, BuildLeft) => Unsupported(Some(unmatchedDuplicationReason)) + case (RightOuter, BuildRight) => Unsupported(Some(unmatchedDuplicationReason)) + case (FullOuter, _) => Unsupported(Some(unmatchedDuplicationReason)) + case _ => Compatible(None) + } + + override def getUnsupportedReasons(): Seq[String] = Seq(unmatchedDuplicationReason) /** * Convert a Spark operator into a protocol buffer representation that can be passed into native From 242c8b845127f089bedf93389f3fb1a0f46968ca Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Fri, 29 May 2026 11:37:30 -0700 Subject: [PATCH 12/24] mark_joins_needing_dedupe_incompatible --- spark/src/main/scala/org/apache/spark/sql/comet/operators.scala | 2 -- 1 file changed, 2 deletions(-) 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 53aee16438..a565f46c6a 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 @@ -1957,8 +1957,6 @@ object CometBroadcastNestedLoopJoinExec extends CometOperatorSerde[BroadcastNest case _ => Compatible(None) } - override def getUnsupportedReasons(): Seq[String] = Seq(unmatchedDuplicationReason) - /** * Convert a Spark operator into a protocol buffer representation that can be passed into native * code. From eb3fe00052904c19165f110252b5e7053500ea72 Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Fri, 29 May 2026 15:28:25 -0700 Subject: [PATCH 13/24] mark_joins_needing_dedupe_incompatible --- .../main/scala/org/apache/spark/sql/comet/operators.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 a565f46c6a..1b566f0292 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 @@ -1951,10 +1951,10 @@ object CometBroadcastNestedLoopJoinExec extends CometOperatorSerde[BroadcastNest override def getSupportLevel(op: BroadcastNestedLoopJoinExec): SupportLevel = (op.joinType, op.buildSide) match { - case (LeftOuter, BuildLeft) => Unsupported(Some(unmatchedDuplicationReason)) + case (LeftOuter, BuildLeft) => Unsupported(Some(unmatchedDuplicationReason)) case (RightOuter, BuildRight) => Unsupported(Some(unmatchedDuplicationReason)) - case (FullOuter, _) => Unsupported(Some(unmatchedDuplicationReason)) - case _ => Compatible(None) + case (FullOuter, _) => Unsupported(Some(unmatchedDuplicationReason)) + case _ => Compatible(None) } /** From d5a4bf5961a450ccc291ea350afff99ae7c147fa Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Fri, 29 May 2026 16:35:03 -0700 Subject: [PATCH 14/24] mark_joins_needing_dedupe_incompatible --- .../org/apache/comet/exec/CometJoinSuite.scala | 14 -------------- .../CometBroadcastNestedLoopJoinBenchmark.scala | 6 +++--- 2 files changed, 3 insertions(+), 17 deletions(-) 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 91a346fdfd..1da17e290b 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala @@ -748,20 +748,6 @@ class CometJoinSuite extends CometTestBase { } } - test("BroadcastNestedLoopJoin RIGHT 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 RIGHT 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") { 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 index 836382f2c5..3b242c3ada 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBroadcastNestedLoopJoinBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBroadcastNestedLoopJoinBenchmark.scala @@ -86,7 +86,7 @@ object CometBroadcastNestedLoopJoinBenchmark extends CometBenchmarkBase { runExpressionBenchmark( "range join (BETWEEN)", probeRows, - "SELECT count(*) FROM probe p " + + "SELECT /*+ BROADCAST(b) */ count(*) FROM probe p " + "JOIN build b ON p.k BETWEEN b.lo AND b.hi", cometConfigs) } @@ -95,7 +95,7 @@ object CometBroadcastNestedLoopJoinBenchmark extends CometBenchmarkBase { runExpressionBenchmark( "inequality join (>)", probeRows, - "SELECT count(*) FROM probe p " + + "SELECT /*+ BROADCAST(b) */ count(*) FROM probe p " + "JOIN build b ON p.k > b.lo", cometConfigs) } @@ -104,7 +104,7 @@ object CometBroadcastNestedLoopJoinBenchmark extends CometBenchmarkBase { runExpressionBenchmark( "left outer non-equi", probeRows, - "SELECT count(*) FROM probe p " + + "SELECT /*+ BROADCAST(b) */ count(*) FROM probe p " + "LEFT OUTER JOIN build b ON p.k BETWEEN b.lo AND b.hi", cometConfigs) } From a64845cd5ee4adaa417d9df266f25cb5f143a2f2 Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Sat, 30 May 2026 09:51:25 -0700 Subject: [PATCH 15/24] mark_joins_needing_dedupe_incompatible_regen_tpcds --- .../approved-plans-v1_4/q28/extended.txt | 119 ++-- .../approved-plans-v1_4/q61/extended.txt | 101 ++- .../approved-plans-v1_4/q77/extended.txt | 151 ++--- .../approved-plans-v1_4/q88/extended.txt | 375 +++++------ .../approved-plans-v1_4/q90/extended.txt | 59 +- .../approved-plans-v2_7/q22/extended.txt | 59 +- .../approved-plans-v2_7/q77a/extended.txt | 632 +++++++++--------- 7 files changed, 733 insertions(+), 763 deletions(-) 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/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 From 6d0148edbdb3bf5b1a84f7bc7be90973047fe2ef Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Sat, 30 May 2026 10:19:29 -0700 Subject: [PATCH 16/24] mark_joins_needing_dedupe_incompatible_regen_tpcds --- .../org/apache/spark/sql/comet/operators.scala | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) 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 1b566f0292..f6869b9252 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 @@ -1945,15 +1945,18 @@ object CometBroadcastNestedLoopJoinExec extends CometOperatorSerde[BroadcastNest Some(CometConf.COMET_EXEC_BROADCAST_NESTED_LOOP_JOIN_ENABLED) } - private val unmatchedDuplicationReason = - "BNLJ with preserved-build side (LeftOuter+BuildLeft, RightOuter+BuildRight, FullOuter)" + - " duplicates unmatched rows across partitions because the broadcast side is replicated" + 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 (LeftOuter, BuildLeft) => Unsupported(Some(unmatchedDuplicationReason)) - case (RightOuter, BuildRight) => Unsupported(Some(unmatchedDuplicationReason)) - case (FullOuter, _) => Unsupported(Some(unmatchedDuplicationReason)) + case (LeftOuter, BuildLeft) => Unsupported(Some(broadcastBuildReplicationReason)) + case (RightOuter, BuildRight) => Unsupported(Some(broadcastBuildReplicationReason)) + case (FullOuter, _) => Unsupported(Some(broadcastBuildReplicationReason)) + case (LeftSemi, BuildLeft) => Unsupported(Some(broadcastBuildReplicationReason)) + case (LeftAnti, BuildLeft) => Unsupported(Some(broadcastBuildReplicationReason)) case _ => Compatible(None) } From 14cd3fee37727150de06920b9033c7ea44c75cea Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Sat, 30 May 2026 19:35:46 -0700 Subject: [PATCH 17/24] mark_joins_needing_dedupe_incompatible_regen_tpcds --- .../src/main/scala/org/apache/spark/sql/comet/operators.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 2b18979064..85354a43d8 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 @@ -2003,7 +2003,7 @@ object CometBroadcastNestedLoopJoinExec extends CometOperatorSerde[BroadcastNest case LeftAnti => JoinType.LeftAnti case _ => // Spark doesn't support other join types - withInfo(op, s"Unsupported join type $join") + withFallbackReason(op, s"Unsupported join type $join") return None } } @@ -2011,7 +2011,7 @@ object CometBroadcastNestedLoopJoinExec extends CometOperatorSerde[BroadcastNest val joinCondition = op.condition.map({ cond => val condProto = exprToProto(cond, op.left.output ++ op.right.output) if (condProto.isEmpty) { - withInfo(op, cond) + withFallbackReason(op, cond) return None } condProto.get From f51b5a7713cc998bf30771a9e0d3b07d13799732 Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Tue, 2 Jun 2026 11:21:01 -0700 Subject: [PATCH 18/24] update_diffs --- dev/diffs/3.5.8.diff | 20 ++++++++++++++++++-- dev/diffs/4.0.2.diff | 35 ++++++++++++++++++++++++++--------- dev/diffs/4.1.1.diff | 20 ++++++++++++++++++-- 3 files changed, 62 insertions(+), 13 deletions(-) diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index 792aa1a82c..f6ceefa335 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -825,6 +825,18 @@ index 44c8cb92fc3..f098beeca26 100644 checkAnswer(shjNonCodegenDF, Seq.empty) } } +@@ -1470,7 +1491,10 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + "/*+ BROADCAST(t2) */ t1.k as k" + } + val plan = sql(getAggQuery(selectExpr, joinType)).queryExecution.executedPlan +- assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) ++ assert(collect(plan) { ++ case _: BroadcastNestedLoopJoinExec | _: CometBroadcastNestedLoopJoinExec => ++ true ++ }.size === 1) + // No extra shuffle before aggregation + assert(collect(plan) { case _: ShuffleExchangeExec => true }.size === 0) + } @@ -1486,7 +1507,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val plan = sql(getAggQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) @@ -835,12 +847,16 @@ index 44c8cb92fc3..f098beeca26 100644 } def getJoinQuery(selectExpr: String, joinType: String): String = { -@@ -1515,9 +1537,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1515,10 +1540,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/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.1.diff b/dev/diffs/4.1.1.diff index 6721ca5b0f..4f6d4ee60a 100644 --- a/dev/diffs/4.1.1.diff +++ b/dev/diffs/4.1.1.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) From 544148466978643db5e167ee4f6e5b2fe2acdc10 Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Tue, 2 Jun 2026 13:21:14 -0700 Subject: [PATCH 19/24] update_diffs --- dev/diffs/4.1.1.diff | 68 +++----------------------------------------- 1 file changed, 4 insertions(+), 64 deletions(-) diff --git a/dev/diffs/4.1.1.diff b/dev/diffs/4.1.1.diff index 4f6d4ee60a..5cf6326dbf 100644 --- a/dev/diffs/4.1.1.diff +++ b/dev/diffs/4.1.1.diff @@ -1043,18 +1043,6 @@ index 885512d4d19..113ae17ad9f 100644 checkAnswer(shjNonCodegenDF, Seq.empty) } } -@@ -1485,7 +1507,10 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan - "/*+ BROADCAST(t2) */ t1.k as k" - } - val plan = sql(getAggQuery(selectExpr, joinType)).queryExecution.executedPlan -- assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) -+ assert(collect(plan) { -+ case _: BroadcastNestedLoopJoinExec | _: CometBroadcastNestedLoopJoinExec => -+ true -+ }.size === 1) - // No extra shuffle before aggregation - assert(collect(plan) { case _: ShuffleExchangeExec => true }.size === 0) - } @@ -1501,7 +1523,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val plan = sql(getAggQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) @@ -1065,16 +1053,12 @@ index 885512d4d19..113ae17ad9f 100644 } def getJoinQuery(selectExpr: String, joinType: String): String = { -@@ -1530,10 +1556,16 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1530,9 +1553,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan } val plan = sql(getJoinQuery(selectExpr, joinType)).queryExecution.executedPlan -- assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) + assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) - assert(collect(plan) { case _: SortMergeJoinExec => true }.size === 3) + assert(collect(plan) { -+ case _: BroadcastNestedLoopJoinExec | _: CometBroadcastNestedLoopJoinExec => -+ true -+ }.size === 1) -+ assert(collect(plan) { + case _: SortMergeJoinExec => true + case _: CometSortMergeJoinExec => true + }.size === 3) @@ -1166,7 +1150,7 @@ index e4b5e10f7c3..c6efde09c8a 100644 protected val baseResourcePath = { // use the same way as `SQLQueryTestSuite` to get the resource path diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala -index 74cdee49e55..f7452c9abb7 100644 +index 74cdee49e55..3decf393ed0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1521,7 +1521,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark @@ -1179,25 +1163,8 @@ index 74cdee49e55..f7452c9abb7 100644 AccumulatorSuite.verifyPeakExecutionMemorySet(sparkContext, "external sort") { sql("SELECT * FROM testData2 ORDER BY a ASC, b ASC").collect() } -@@ -1982,8 +1983,15 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark - countAcc.add(1) - x - }) -+ // Comet's `CometProject` and `CometHashAggregate` do not implement Spark's cross-sibling -+ // subexpression elimination over `ScalaUDF`, so each reference invokes the UDF body -+ // separately. The other call sites in this test pass against Comet because the source -+ // (`testData2`, a `LocalRelation`) is not Comet-scannable and the project runs on Spark's -+ // path; the `agg` case routes through `CometHashAggregate` once an Exchange enters the plan. -+ // Tracking issue: https://github.com/apache/datafusion-comet/issues/4516 - verifyCallCount( -- df.agg(sum(testUdf($"b") + testUdf($"b") + testUdf($"b"))), Row(3.0), 1) -+ df.agg(sum(testUdf($"b") + testUdf($"b") + testUdf($"b"))), Row(3.0), -+ if (isCometEnabled) 3 else 1) - - verifyCallCount( - df.selectExpr("testUdf(a + 1) + testUdf(1 + a)", "testUdf(a + 1)"), Row(4, 2), 1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala -index 23f0144dcec..2586d93d630 100644 +index 23f0144dcec..df845f7295a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -166,7 +166,16 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper @@ -1218,33 +1185,6 @@ index 23f0144dcec..2586d93d630 100644 ) ++ otherIgnoreList /** List of test cases that require TPCDS table schemas to be loaded. */ private def requireTPCDSCases: Seq[String] = Seq("pipe-operators.sql") -@@ -682,9 +691,24 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper - s"Schema did not match for query #$i\n${expected.sql}: $output") { - output.schema - } -- assertResult(expected.output, s"Result did not match" + -+ // Comet may surface errors as `CometNativeException` instead of the matching Spark -+ // exception class when a `ScalaUDF` dispatched into the native plan evaluates a -+ // divide-by-zero (DataFusion wraps the typed error so the JNI bridge cannot downcast it). -+ // Same category, different surface. Collapse both sides to a placeholder when this happens -+ // so the literal compare passes. -+ // Tracking issue: https://github.com/apache/datafusion-comet/issues/4517 -+ val (expectedOut, actualOut) = if (isCometEnabled && -+ expected.output.startsWith("org.apache.spark.SparkArithmeticException") && -+ expected.output.contains("\"DIVIDE_BY_ZERO\"") && -+ output.output.startsWith("org.apache.comet.CometNativeException") && -+ output.output.contains("DivideByZero")) { -+ ("[DIVIDE_BY_ZERO]", "[DIVIDE_BY_ZERO]") -+ } else { -+ (expected.output, output.output) -+ } -+ assertResult(expectedOut, s"Result did not match" + - s" for query #$i\n${expected.sql}") { -- output.output -+ actualOut - } - } - } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala index 66826a9ca76..ab4265a5fb9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala From bfe9254b39be82cc00853ebca64387c6b5f623f3 Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Tue, 2 Jun 2026 13:47:16 -0700 Subject: [PATCH 20/24] update_diffs --- dev/diffs/{4.1.1.diff => 4.1.2.diff} | 110 +++++++++++++++++++++------ 1 file changed, 85 insertions(+), 25 deletions(-) rename dev/diffs/{4.1.1.diff => 4.1.2.diff} (97%) diff --git a/dev/diffs/4.1.1.diff b/dev/diffs/4.1.2.diff similarity index 97% rename from dev/diffs/4.1.1.diff rename to dev/diffs/4.1.2.diff index 5cf6326dbf..0fec94e6d5 100644 --- a/dev/diffs/4.1.1.diff +++ b/dev/diffs/4.1.2.diff @@ -39,7 +39,7 @@ index 6df8bc85b51..dabb75e2b75 100644 withSpark(sc) { sc => TestUtils.waitUntilExecutorsUp(sc, 2, 60000) diff --git a/pom.xml b/pom.xml -index dc757d78812..10f7b202e71 100644 +index dc201151999..3e278cfb34c 100644 --- a/pom.xml +++ b/pom.xml @@ -152,6 +152,8 @@ @@ -78,7 +78,7 @@ index dc757d78812..10f7b202e71 100644 org.apache.datasketches diff --git a/sql/core/pom.xml b/sql/core/pom.xml -index d2d07a08aa9..d89f80e5b68 100644 +index c25b83c355b..5e23b863dcf 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -97,6 +97,10 @@ @@ -392,7 +392,7 @@ index 0d807aeae4d..6d7744e771b 100644 withTempView("t0", "t1", "t2") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala -index 0dfd37ebeae..66340218c7c 100644 +index bfe15b33768..55c23a38ccc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.errors.DataTypeErrors.toSQLId @@ -695,10 +695,10 @@ index e1a2fd33c7c..632f4b695df 100644 } assert(scanOption.isDefined) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -index b27122a8de2..3c690dbe788 100644 +index 4c62c47971a..ecc7ed21546 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -@@ -267,7 +267,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -268,7 +268,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } @@ -708,7 +708,7 @@ index b27122a8de2..3c690dbe788 100644 withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true") { withTempView("df") { val df1 = spark.range(1, 100) -@@ -470,7 +471,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -471,7 +472,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } @@ -718,7 +718,7 @@ index b27122a8de2..3c690dbe788 100644 withTempDir { dir => Seq("parquet", "orc", "csv", "json").foreach { fmt => val basePath = dir.getCanonicalPath + "/" + fmt -@@ -548,7 +550,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -549,7 +551,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } @@ -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) @@ -1150,7 +1166,7 @@ index e4b5e10f7c3..c6efde09c8a 100644 protected val baseResourcePath = { // use the same way as `SQLQueryTestSuite` to get the resource path diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala -index 74cdee49e55..3decf393ed0 100644 +index 74cdee49e55..f7452c9abb7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1521,7 +1521,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark @@ -1163,8 +1179,25 @@ index 74cdee49e55..3decf393ed0 100644 AccumulatorSuite.verifyPeakExecutionMemorySet(sparkContext, "external sort") { sql("SELECT * FROM testData2 ORDER BY a ASC, b ASC").collect() } +@@ -1982,8 +1983,15 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark + countAcc.add(1) + x + }) ++ // Comet's `CometProject` and `CometHashAggregate` do not implement Spark's cross-sibling ++ // subexpression elimination over `ScalaUDF`, so each reference invokes the UDF body ++ // separately. The other call sites in this test pass against Comet because the source ++ // (`testData2`, a `LocalRelation`) is not Comet-scannable and the project runs on Spark's ++ // path; the `agg` case routes through `CometHashAggregate` once an Exchange enters the plan. ++ // Tracking issue: https://github.com/apache/datafusion-comet/issues/4516 + verifyCallCount( +- df.agg(sum(testUdf($"b") + testUdf($"b") + testUdf($"b"))), Row(3.0), 1) ++ df.agg(sum(testUdf($"b") + testUdf($"b") + testUdf($"b"))), Row(3.0), ++ if (isCometEnabled) 3 else 1) + + verifyCallCount( + df.selectExpr("testUdf(a + 1) + testUdf(1 + a)", "testUdf(a + 1)"), Row(4, 2), 1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala -index 23f0144dcec..df845f7295a 100644 +index 23f0144dcec..2586d93d630 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -166,7 +166,16 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper @@ -1185,6 +1218,33 @@ index 23f0144dcec..df845f7295a 100644 ) ++ otherIgnoreList /** List of test cases that require TPCDS table schemas to be loaded. */ private def requireTPCDSCases: Seq[String] = Seq("pipe-operators.sql") +@@ -682,9 +691,24 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper + s"Schema did not match for query #$i\n${expected.sql}: $output") { + output.schema + } +- assertResult(expected.output, s"Result did not match" + ++ // Comet may surface errors as `CometNativeException` instead of the matching Spark ++ // exception class when a `ScalaUDF` dispatched into the native plan evaluates a ++ // divide-by-zero (DataFusion wraps the typed error so the JNI bridge cannot downcast it). ++ // Same category, different surface. Collapse both sides to a placeholder when this happens ++ // so the literal compare passes. ++ // Tracking issue: https://github.com/apache/datafusion-comet/issues/4517 ++ val (expectedOut, actualOut) = if (isCometEnabled && ++ expected.output.startsWith("org.apache.spark.SparkArithmeticException") && ++ expected.output.contains("\"DIVIDE_BY_ZERO\"") && ++ output.output.startsWith("org.apache.comet.CometNativeException") && ++ output.output.contains("DivideByZero")) { ++ ("[DIVIDE_BY_ZERO]", "[DIVIDE_BY_ZERO]") ++ } else { ++ (expected.output, output.output) ++ } ++ assertResult(expectedOut, s"Result did not match" + + s" for query #$i\n${expected.sql}") { +- output.output ++ actualOut + } + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala index 66826a9ca76..ab4265a5fb9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala @@ -1232,7 +1292,7 @@ index d7b2511eac2..d5f5b940b94 100644 val session = classic.SparkSession.builder().sparkContext(sc).getOrCreate() import session.implicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala -index ff0ee19ae97..01958e0c45b 100644 +index 7bfc8cf4fa6..7a425b74184 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -17,6 +17,8 @@ @@ -1393,7 +1453,7 @@ index fee375db10a..8c2c24e2c5f 100644 val v = VariantBuilder.parseJson(s, false) new VariantVal(v.getValue, v.getMetadata) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSuite.scala -index 6cdf681d65c..34a0e3714bd 100644 +index 8f7a68bcbe6..88dbe1793c9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/CollationSuite.scala @@ -26,6 +26,8 @@ import org.apache.spark.sql.{AnalysisException, Row} @@ -1549,7 +1609,7 @@ index 2a0ab21ddb0..6030e7c2b9b 100644 } finally { spark.listenerManager.unregister(listener) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala -index 7c830bf6c6e..6d9c643d83e 100644 +index 122c511bf83..9bea26c5225 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala @@ -24,6 +24,8 @@ import org.apache.spark.sql.{DataFrame, Row} @@ -1562,7 +1622,7 @@ index 7c830bf6c6e..6d9c643d83e 100644 import org.apache.spark.sql.connector.catalog.functions._ import org.apache.spark.sql.connector.distributions.Distributions @@ -32,7 +34,7 @@ import org.apache.spark.sql.connector.expressions.Expressions._ - import org.apache.spark.sql.execution.SparkPlan + import org.apache.spark.sql.execution.{RDDScanExec, SparkPlan} import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation -import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec @@ -1596,7 +1656,7 @@ index 7c830bf6c6e..6d9c643d83e 100644 } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala -index 7c4852c5e22..d1a34456bdc 100644 +index ede5d285932..c9a8abb5a94 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/WriteDistributionAndOrderingSuite.scala @@ -21,7 +21,7 @@ package org.apache.spark.sql.connector @@ -1605,10 +1665,10 @@ index 7c4852c5e22..d1a34456bdc 100644 -import org.apache.spark.sql.{catalyst, AnalysisException, DataFrame, Row} +import org.apache.spark.sql.{catalyst, AnalysisException, DataFrame, IgnoreCometSuite, Row} - import org.apache.spark.sql.catalyst.expressions.{ApplyFunctionExpression, Cast, Literal} + import org.apache.spark.sql.catalyst.expressions.{ApplyFunctionExpression, Cast, Literal, TransformExpression} import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.catalyst.plans.physical -@@ -45,7 +45,8 @@ import org.apache.spark.sql.util.QueryExecutionListener +@@ -46,7 +46,8 @@ import org.apache.spark.sql.util.QueryExecutionListener import org.apache.spark.tags.SlowSQLTest @SlowSQLTest @@ -2845,7 +2905,7 @@ index 6b73cc8618d..e67aaeff9df 100644 case _ => assert(false, "Can not match ParquetTable in the query.") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -index 3072657a095..599d169cf8a 100644 +index 6ba790deddf..34b2f424c8f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -40,6 +40,7 @@ import org.apache.parquet.schema.{MessageType, MessageTypeParser} @@ -2856,7 +2916,7 @@ index 3072657a095..599d169cf8a 100644 import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericInternalRow, UnsafeRow} import org.apache.spark.sql.catalyst.util.{DateTimeConstants, DateTimeUtils} -@@ -953,7 +954,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession +@@ -971,7 +972,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession } } @@ -2866,7 +2926,7 @@ index 3072657a095..599d169cf8a 100644 val data = Seq( Tuple1((null, null)), Tuple1((null, null)), -@@ -1567,7 +1569,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession +@@ -1585,7 +1587,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession } } @@ -3230,7 +3290,7 @@ index 38e5b15465b..ca3e8fef27a 100644 testWithColumnFamilies("RocksDBStateStore", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala -index e839ccd35ec..d182aa07b44 100644 +index 232332a6575..324afe9ebb7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -32,7 +32,8 @@ import org.apache.hadoop.conf.Configuration @@ -3595,7 +3655,7 @@ index 465da3cd469..92ac998929d 100644 val aggregateExecsWithoutPartialAgg = allAggregateExecs.filter { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala -index 22028a585e2..20c6b7c796a 100644 +index 6cdca9fb530..6542bc8dced 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.{DataFrame, Row, SparkSession} @@ -3775,10 +3835,10 @@ index f0f3f94b811..b7d18771314 100644 spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -index 245219c1756..b566f970ccd 100644 +index 720b13b812e..71b20c79a12 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -@@ -75,6 +75,21 @@ trait SharedSparkSessionBase +@@ -98,6 +98,21 @@ trait SharedSparkSessionBase // this rule may potentially block testing of other optimization rules such as // ConstantPropagation etc. .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) From c372da8d059dc92084949f52a2daf25ae35ac653 Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Tue, 2 Jun 2026 14:05:06 -0700 Subject: [PATCH 21/24] update_diffs --- dev/diffs/4.1.2.diff | 20 ++++++++++++++++++-- 1 file changed, 18 insertions(+), 2 deletions(-) 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) From 79cd4241a25051f4ca2a4c54cda8c4bda528709a Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Tue, 2 Jun 2026 16:32:58 -0700 Subject: [PATCH 22/24] update_diffs --- dev/diffs/3.5.8.diff | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index f6ceefa335..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 @@ -837,7 +837,7 @@ index 44c8cb92fc3..f098beeca26 100644 // No extra shuffle before aggregation assert(collect(plan) { case _: ShuffleExchangeExec => true }.size === 0) } -@@ -1486,7 +1507,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -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 @@ -847,7 +847,8 @@ index 44c8cb92fc3..f098beeca26 100644 } def getJoinQuery(selectExpr: String, joinType: String): String = { -@@ -1515,10 +1540,16 @@ 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) @@ -866,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) @@ -881,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 @@ -891,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 @@ -901,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 = { @@ -924,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)) } From 8a3fbbb021f43e1ef374f1c987cac52d403dcbc6 Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Wed, 3 Jun 2026 07:47:54 -0700 Subject: [PATCH 23/24] additional_tests_implement_hashcode --- docs/source/user-guide/latest/operators.md | 1 + .../latest/understanding-comet-plans.md | 1 + .../apache/spark/sql/comet/operators.scala | 43 ++++++++++++++++--- .../apache/comet/exec/CometJoinSuite.scala | 28 ++++++++++++ ...ometBroadcastNestedLoopJoinBenchmark.scala | 9 ++++ 5 files changed, 76 insertions(+), 6 deletions(-) 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 7fb93c8f53..f71fd592fc 100644 --- a/docs/source/user-guide/latest/understanding-comet-plans.md +++ b/docs/source/user-guide/latest/understanding-comet-plans.md @@ -169,6 +169,7 @@ they execute as a single fused native block. | `CometHashAggregate` | `HashAggregateExec`, `ObjectHashAggregateExec` | | `CometHashJoin` | `ShuffledHashJoinExec` | | `CometBroadcastHashJoin` | `BroadcastHashJoinExec` | +| `CometBroadcastNestedLoopJoin` | `BroadcastNestedLoopJoinExec` | | `CometSortMergeJoin` | `SortMergeJoinExec` | | `CometWindow` | `WindowExec` | | `CometTakeOrderedAndProject` | `TakeOrderedAndProjectExec` | 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 85354a43d8..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 @@ -1937,8 +1937,39 @@ case class CometBroadcastNestedLoopJoinExec( 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] { @@ -1958,12 +1989,12 @@ object CometBroadcastNestedLoopJoinExec extends CometOperatorSerde[BroadcastNest override def getSupportLevel(op: BroadcastNestedLoopJoinExec): SupportLevel = (op.joinType, op.buildSide) match { - case (LeftOuter, BuildLeft) => Unsupported(Some(broadcastBuildReplicationReason)) - case (RightOuter, BuildRight) => Unsupported(Some(broadcastBuildReplicationReason)) - case (FullOuter, _) => Unsupported(Some(broadcastBuildReplicationReason)) - case (LeftSemi, BuildLeft) => Unsupported(Some(broadcastBuildReplicationReason)) - case (LeftAnti, BuildLeft) => Unsupported(Some(broadcastBuildReplicationReason)) - case _ => Compatible(None) + 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")) } /** 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 1da17e290b..3db36b782e 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometJoinSuite.scala @@ -775,4 +775,32 @@ class CometJoinSuite extends CometTestBase { } } } + + 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 index 3b242c3ada..e86903f6ea 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBroadcastNestedLoopJoinBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometBroadcastNestedLoopJoinBenchmark.scala @@ -108,6 +108,15 @@ object CometBroadcastNestedLoopJoinBenchmark extends CometBenchmarkBase { "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) + } } } } From 4c32ce5f32dc55d7e10f43d8dd46c59e9ec90f40 Mon Sep 17 00:00:00 2001 From: Bhargava Vadlamani Date: Wed, 3 Jun 2026 07:52:37 -0700 Subject: [PATCH 24/24] additional_tests_implement_hashcode --- .../latest/understanding-comet-plans.md | 32 +++++++++---------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/docs/source/user-guide/latest/understanding-comet-plans.md b/docs/source/user-guide/latest/understanding-comet-plans.md index f71fd592fc..8a61989150 100644 --- a/docs/source/user-guide/latest/understanding-comet-plans.md +++ b/docs/source/user-guide/latest/understanding-comet-plans.md @@ -157,22 +157,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` | -| `CometBroadcastNestedLoopJoin` | `BroadcastNestedLoopJoinExec` | -| `CometSortMergeJoin` | `SortMergeJoinExec` | -| `CometWindow` | `WindowExec` | -| `CometTakeOrderedAndProject` | `TakeOrderedAndProjectExec` | +| Node | Spark equivalent | +| ------------------------------ | ----------------------------------------------- | +| `CometProject` | `ProjectExec` | +| `CometFilter` | `FilterExec` | +| `CometSort` | `SortExec` | +| `CometLocalLimit` | `LocalLimitExec` | +| `CometGlobalLimit` | `GlobalLimitExec` | +| `CometExpand` | `ExpandExec` | +| `CometExplode` | `GenerateExec` (for `explode` and `posexplode`) | +| `CometHashAggregate` | `HashAggregateExec`, `ObjectHashAggregateExec` | +| `CometHashJoin` | `ShuffledHashJoinExec` | +| `CometBroadcastHashJoin` | `BroadcastHashJoinExec` | +| `CometBroadcastNestedLoopJoin` | `BroadcastNestedLoopJoinExec` | +| `CometSortMergeJoin` | `SortMergeJoinExec` | +| `CometWindow` | `WindowExec` | +| `CometTakeOrderedAndProject` | `TakeOrderedAndProjectExec` | ### JVM-Side Operators