From b3cf832bb84f1e1223d9b91b69392b13cd378ed5 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 13 May 2025 11:37:05 -0600 Subject: [PATCH 01/42] Remove COMET_SHUFFLE_FALLBACK_TO_COLUMNAR config --- .../src/main/scala/org/apache/comet/CometConf.scala | 7 ------- .../org/apache/comet/rules/CometExecRule.scala | 13 ++----------- .../scala/org/apache/spark/sql/CometTestBase.scala | 1 - .../spark/sql/comet/CometPlanStabilitySuite.scala | 1 - 4 files changed, 2 insertions(+), 20 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 3f35133a58..d06311b68e 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -294,13 +294,6 @@ object CometConf extends ShimCometConf { .checkValues(Set("native", "jvm", "auto")) .createWithDefault("auto") - val COMET_SHUFFLE_FALLBACK_TO_COLUMNAR: ConfigEntry[Boolean] = - conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.fallbackToColumnar") - .doc("Whether to try falling back to columnar shuffle when native shuffle is not supported") - .internal() - .booleanConf - .createWithDefault(false) - val COMET_EXEC_BROADCAST_FORCE_ENABLED: ConfigEntry[Boolean] = conf(s"$COMET_EXEC_CONFIG_PREFIX.broadcast.enabled") .doc( 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 dd1b0d516d..bf399f9e91 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.types.{DoubleType, FloatType} import org.apache.comet.{CometConf, ExtendedExplainInfo} -import org.apache.comet.CometConf.{COMET_ANSI_MODE_ENABLED, COMET_NATIVE_SCAN_IMPL, COMET_SHUFFLE_FALLBACK_TO_COLUMNAR} +import org.apache.comet.CometConf.{COMET_ANSI_MODE_ENABLED, COMET_NATIVE_SCAN_IMPL} import org.apache.comet.CometSparkSessionExtensions.{createMessage, getCometBroadcastNotEnabledReason, getCometShuffleNotEnabledReason, isANSIEnabled, isCometBroadCastForceEnabled, isCometExecEnabled, isCometJVMShuffleMode, isCometLoaded, isCometNativeShuffleMode, isCometScan, isCometShuffleEnabled, isSpark40Plus, shouldApplySparkToColumnar, withInfo} import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.QueryPlanSerde @@ -486,16 +486,9 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { None } - // this is a temporary workaround because some Spark SQL tests fail - // when we enable COMET_SHUFFLE_FALLBACK_TO_COLUMNAR due to valid bugs - // that we had not previously seen - val tryColumnarNext = - !nativePrecondition || (nativePrecondition && nativeShuffle.isEmpty && - COMET_SHUFFLE_FALLBACK_TO_COLUMNAR.get(conf)) - val nativeOrColumnarShuffle = if (nativeShuffle.isDefined) { nativeShuffle - } else if (tryColumnarNext) { + } else { // Columnar shuffle for regular Spark operators (not Comet) and Comet operators // (if configured). // If the child of ShuffleExchangeExec is also a ShuffleExchangeExec, we should not @@ -521,8 +514,6 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { } else { None } - } else { - None } if (nativeOrColumnarShuffle.isDefined) { diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala index 9257dc3d7b..ae972bef6a 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala @@ -78,7 +78,6 @@ abstract class CometTestBase conf.set(CometConf.COMET_ENABLED.key, "true") conf.set(CometConf.COMET_EXEC_ENABLED.key, "true") conf.set(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key, "true") - conf.set(CometConf.COMET_SHUFFLE_FALLBACK_TO_COLUMNAR.key, "true") conf.set(CometConf.COMET_SPARK_TO_ARROW_ENABLED.key, "true") conf.set(CometConf.COMET_NATIVE_SCAN_ENABLED.key, "true") conf.set(CometConf.COMET_SCAN_ALLOW_INCOMPATIBLE.key, "true") diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala index 9f611612f6..cf887a1013 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala @@ -272,7 +272,6 @@ trait CometPlanStabilitySuite extends DisableAdaptiveExecutionSuite with TPCDSBa CometConf.COMET_DPP_FALLBACK_ENABLED.key -> "false", SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> dppEnabled.toString, CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_SHUFFLE_FALLBACK_TO_COLUMNAR.key -> "true", CometConf.COMET_EXEC_SORT_MERGE_JOIN_WITH_JOIN_FILTER_ENABLED.key -> "true", CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true", // needed for v1.4/q9, v1.4/q44, v2.7.0/q6, v2.7.0/q64 SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10MB") { From 81b338497159555090fed4ad5fd7de9b5a502b73 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 30 May 2025 11:32:26 -0600 Subject: [PATCH 02/42] format --- spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala | 2 +- 1 file changed, 1 insertion(+), 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 5e9845f14d..54e6e63649 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.types.{DoubleType, FloatType} import org.apache.comet.{CometConf, ExtendedExplainInfo} -import org.apache.comet.CometConf.{COMET_ANSI_MODE_ENABLED, COMET_NATIVE_SCAN_IMPL} +import org.apache.comet.CometConf.COMET_ANSI_MODE_ENABLED import org.apache.comet.CometSparkSessionExtensions.{createMessage, getCometBroadcastNotEnabledReason, getCometShuffleNotEnabledReason, isANSIEnabled, isCometBroadCastForceEnabled, isCometExecEnabled, isCometJVMShuffleMode, isCometLoaded, isCometNativeShuffleMode, isCometScan, isCometShuffleEnabled, isSpark40Plus, shouldApplySparkToColumnar, withInfo} import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.QueryPlanSerde From 8626f51ab52c0a1f235cbafe47ae6dc5f80aac1b Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 30 May 2025 13:55:53 -0600 Subject: [PATCH 03/42] Specify -Dsbt.log.noformat=true in sbt CI runs --- .github/workflows/spark_sql_test.yml | 2 +- .github/workflows/spark_sql_test_ansi.yml | 2 +- .github/workflows/spark_sql_test_native_datafusion.yml | 2 +- .github/workflows/spark_sql_test_native_iceberg_compat.yml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/spark_sql_test.yml b/.github/workflows/spark_sql_test.yml index 58defe6608..bb704c20ec 100644 --- a/.github/workflows/spark_sql_test.yml +++ b/.github/workflows/spark_sql_test.yml @@ -114,6 +114,6 @@ jobs: run: | cd apache-spark rm -rf /root/.m2/repository/org/apache/parquet # somehow parquet cache requires cleanups - ENABLE_COMET=true ENABLE_COMET_SHUFFLE=true COMET_SPARK_PARQUET_IMPL=native_iceberg_compat build/sbt ${{ matrix.module.args1 }} "${{ matrix.module.args2 }}" + ENABLE_COMET=true ENABLE_COMET_SHUFFLE=true COMET_SPARK_PARQUET_IMPL=native_iceberg_compat build/sbt -Dsbt.log.noformat=true ${{ matrix.module.args1 }} "${{ matrix.module.args2 }}" env: LC_ALL: "C.UTF-8" diff --git a/.github/workflows/spark_sql_test_ansi.yml b/.github/workflows/spark_sql_test_ansi.yml index 170c10644b..a3e85307d9 100644 --- a/.github/workflows/spark_sql_test_ansi.yml +++ b/.github/workflows/spark_sql_test_ansi.yml @@ -73,7 +73,7 @@ jobs: run: | cd apache-spark rm -rf /root/.m2/repository/org/apache/parquet # somehow parquet cache requires cleanups - RUST_BACKTRACE=1 ENABLE_COMET=true ENABLE_COMET_ANSI_MODE=true ENABLE_COMET_SHUFFLE=true build/sbt ${{ matrix.module.args1 }} "${{ matrix.module.args2 }}" + RUST_BACKTRACE=1 ENABLE_COMET=true ENABLE_COMET_ANSI_MODE=true ENABLE_COMET_SHUFFLE=true build/sbt -Dsbt.log.noformat=true ${{ matrix.module.args1 }} "${{ matrix.module.args2 }}" env: LC_ALL: "C.UTF-8" diff --git a/.github/workflows/spark_sql_test_native_datafusion.yml b/.github/workflows/spark_sql_test_native_datafusion.yml index 39164abb9f..3086b5f11c 100644 --- a/.github/workflows/spark_sql_test_native_datafusion.yml +++ b/.github/workflows/spark_sql_test_native_datafusion.yml @@ -65,7 +65,7 @@ jobs: run: | cd apache-spark rm -rf /root/.m2/repository/org/apache/parquet # somehow parquet cache requires cleanups - ENABLE_COMET=true ENABLE_COMET_SHUFFLE=true COMET_PARQUET_SCAN_IMPL=native_datafusion build/sbt ${{ matrix.module.args1 }} "${{ matrix.module.args2 }}" + ENABLE_COMET=true ENABLE_COMET_SHUFFLE=true COMET_PARQUET_SCAN_IMPL=native_datafusion build/sbt -Dsbt.log.noformat=true ${{ matrix.module.args1 }} "${{ matrix.module.args2 }}" env: LC_ALL: "C.UTF-8" diff --git a/.github/workflows/spark_sql_test_native_iceberg_compat.yml b/.github/workflows/spark_sql_test_native_iceberg_compat.yml index ed3c94e9be..b9aaf3f0f0 100644 --- a/.github/workflows/spark_sql_test_native_iceberg_compat.yml +++ b/.github/workflows/spark_sql_test_native_iceberg_compat.yml @@ -65,7 +65,7 @@ jobs: run: | cd apache-spark rm -rf /root/.m2/repository/org/apache/parquet # somehow parquet cache requires cleanups - ENABLE_COMET=true ENABLE_COMET_SHUFFLE=true COMET_PARQUET_SCAN_IMPL=native_iceberg_compat build/sbt ${{ matrix.module.args1 }} "${{ matrix.module.args2 }}" + ENABLE_COMET=true ENABLE_COMET_SHUFFLE=true COMET_PARQUET_SCAN_IMPL=native_iceberg_compat build/sbt -Dsbt.log.noformat=true ${{ matrix.module.args1 }} "${{ matrix.module.args2 }}" env: LC_ALL: "C.UTF-8" From dab5fbf7b774d8eb1356ba786d19861d6dbf43e9 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 3 Jun 2025 16:42:56 -0600 Subject: [PATCH 04/42] update test --- dev/diffs/3.5.5.diff | 38 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/dev/diffs/3.5.5.diff b/dev/diffs/3.5.5.diff index 2c7fe57f9b..991db5540d 100644 --- a/dev/diffs/3.5.5.diff +++ b/dev/diffs/3.5.5.diff @@ -332,6 +332,44 @@ index 7ee18df3756..64f01a68048 100644 withTable("tbl") { sql( """ +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +index 47a311c71d5..342e71cfdd4 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +@@ -24,8 +24,9 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression + import org.apache.spark.sql.catalyst.optimizer.TransposeWindow + import org.apache.spark.sql.catalyst.plans.logical.{Window => LogicalWindow} + import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning ++import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +-import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, Exchange, ShuffleExchangeExec} ++import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, Exchange, ShuffleExchangeExec, ShuffleExchangeLike} + import org.apache.spark.sql.execution.window.WindowExec + import org.apache.spark.sql.expressions.{Aggregator, MutableAggregationBuffer, UserDefinedAggregateFunction, Window} + import org.apache.spark.sql.functions._ +@@ -1187,10 +1188,12 @@ class DataFrameWindowFunctionsSuite extends QueryTest + } + + def isShuffleExecByRequirement( +- plan: ShuffleExchangeExec, ++ plan: ShuffleExchangeLike, + desiredClusterColumns: Seq[String]): Boolean = plan match { + case ShuffleExchangeExec(op: HashPartitioning, _, ENSURE_REQUIREMENTS, _) => + partitionExpressionsColumns(op.expressions) === desiredClusterColumns ++ case CometShuffleExchangeExec(op: HashPartitioning, _, _, ENSURE_REQUIREMENTS, _, _) => ++ partitionExpressionsColumns(op.expressions) === desiredClusterColumns + case _ => false + } + +@@ -1213,7 +1216,7 @@ class DataFrameWindowFunctionsSuite extends QueryTest + val shuffleByRequirement = windowed.queryExecution.executedPlan.exists { + case w: WindowExec => + w.child.exists { +- case s: ShuffleExchangeExec => isShuffleExecByRequirement(s, Seq("key1", "key2")) ++ case s: ShuffleExchangeLike => isShuffleExecByRequirement(s, Seq("key1", "key2")) + case _ => false + } + case _ => false diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index f32b32ffc5a..447d7c6416e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala From 8dc4e9b595d4c6b77b1cf6125629e6e557a0e2bc Mon Sep 17 00:00:00 2001 From: Kristin Cowalcijk Date: Wed, 4 Jun 2025 23:42:20 +0800 Subject: [PATCH 05/42] Fix shuffle writing rows containing null struct fields --- native/core/src/execution/shuffle/row.rs | 66 ++++++++++++++----- .../exec/CometColumnarShuffleSuite.scala | 30 +++++++++ 2 files changed, 80 insertions(+), 16 deletions(-) diff --git a/native/core/src/execution/shuffle/row.rs b/native/core/src/execution/shuffle/row.rs index bb1401e263..e19136b37e 100644 --- a/native/core/src/execution/shuffle/row.rs +++ b/native/core/src/execution/shuffle/row.rs @@ -444,25 +444,18 @@ pub(crate) fn append_field( // Appending value into struct field builder of Arrow struct builder. let field_builder = struct_builder.field_builder::(idx).unwrap(); - if row.is_null_row() { - // The row is null. + let nested_row = if row.is_null_row() || row.is_null_at(idx) { + // The row is null, or the field in the row is null, i.e., a null nested row. + // Append a null value to the row builder. field_builder.append_null(); + SparkUnsafeRow::default() } else { - let is_null = row.is_null_at(idx); + field_builder.append(true); + row.get_struct(idx, fields.len()) + }; - let nested_row = if is_null { - // The field in the row is null, i.e., a null nested row. - // Append a null value to the row builder. - field_builder.append_null(); - SparkUnsafeRow::default() - } else { - field_builder.append(true); - row.get_struct(idx, fields.len()) - }; - - for (field_idx, field) in fields.into_iter().enumerate() { - append_field(field.data_type(), field_builder, &nested_row, field_idx)?; - } + for (field_idx, field) in fields.into_iter().enumerate() { + append_field(field.data_type(), field_builder, &nested_row, field_idx)?; } } DataType::Map(field, _) => { @@ -3302,3 +3295,44 @@ fn make_batch(arrays: Vec, row_count: usize) -> Result + val testData = "{}\n" + val path = Paths.get(dir.toString, "test.json") + Files.write(path, testData.getBytes) + + // Define the nested struct schema + val readSchema = StructType( + Array( + StructField( + "metaData", + StructType( + Array(StructField( + "format", + StructType(Array(StructField("provider", StringType, nullable = true))), + nullable = true))), + nullable = true))) + + // Read JSON with custom schema and repartition, this will repartition rows that contain + // null struct fields. + val df = spark.read.format("json").schema(readSchema).load(path.toString).repartition(2) + assert(df.count() == 1) + val row = df.collect()(0) + assert(row.getAs[org.apache.spark.sql.Row]("metaData") == null) + } + } + /** * Checks that `df` produces the same answer as Spark does, and has the `expectedNum` Comet * exchange operators. From 0fcd7ebcccbd7c2cd8b4a8d98ac21a8e8914deef Mon Sep 17 00:00:00 2001 From: Kristin Cowalcijk Date: Thu, 5 Jun 2025 00:15:25 +0800 Subject: [PATCH 06/42] Ignore miri for test_append_null_struct_field_to_struct_builder --- native/core/src/execution/shuffle/row.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/native/core/src/execution/shuffle/row.rs b/native/core/src/execution/shuffle/row.rs index e19136b37e..c98cc54387 100644 --- a/native/core/src/execution/shuffle/row.rs +++ b/native/core/src/execution/shuffle/row.rs @@ -3319,6 +3319,7 @@ mod test { } #[test] + #[cfg_attr(miri, ignore)] // Unaligned memory access in SparkUnsafeRow fn test_append_null_struct_field_to_struct_builder() { let data_type = DataType::Struct(Fields::from(vec![ Field::new("a", DataType::Boolean, true), From 2cc00c2b159b63ad6d0ced779d5425b737643b38 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 4 Jun 2025 16:24:00 -0600 Subject: [PATCH 07/42] ignore test for 3.5.5 --- dev/diffs/3.5.5.diff | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/dev/diffs/3.5.5.diff b/dev/diffs/3.5.5.diff index 310aa68815..26a51f6c43 100644 --- a/dev/diffs/3.5.5.diff +++ b/dev/diffs/3.5.5.diff @@ -226,7 +226,7 @@ index 9815cb816c9..95b5f9992b0 100644 test("A cached table preserves the partitioning and ordering of its cached SparkPlan") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala -index 5a8681aed97..da9d25e2eb4 100644 +index 5a8681aed97..db69fde723a 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 @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical.Expand @@ -247,6 +247,16 @@ index 5a8681aed97..da9d25e2eb4 100644 } assert(exchangePlans.length == 1) } +@@ -1255,7 +1255,8 @@ class DataFrameAggregateSuite extends QueryTest + } + } + +- test("SPARK-32038: NormalizeFloatingNumbers should work on distinct aggregate") { ++ test("SPARK-32038: NormalizeFloatingNumbers should work on distinct aggregate", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1824")) { + withTempView("view") { + val nan1 = java.lang.Float.intBitsToFloat(0x7f800001) + val nan2 = java.lang.Float.intBitsToFloat(0x7fffffff) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index 56e9520fdab..917932336df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala From 0fda09b3c6a34ba2ab85b2c93cfe67376991a40d Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 4 Jun 2025 16:28:59 -0600 Subject: [PATCH 08/42] update docs --- docs/source/user-guide/compatibility.md | 9 +++------ docs/templates/compatibility-template.md | 9 +++------ 2 files changed, 6 insertions(+), 12 deletions(-) diff --git a/docs/source/user-guide/compatibility.md b/docs/source/user-guide/compatibility.md index 212dc0dc31..677c291b9a 100644 --- a/docs/source/user-guide/compatibility.md +++ b/docs/source/user-guide/compatibility.md @@ -29,12 +29,6 @@ Comet aims to provide consistent results with the version of Apache Spark that i This guide offers information about areas of functionality where there are known differences. -# Compatibility Guide - -Comet aims to provide consistent results with the version of Apache Spark that is being used. - -This guide offers information about areas of functionality where there are known differences. - ## Parquet Scans Comet currently has three distinct implementations of the Parquet scan operator. The configuration property @@ -89,6 +83,9 @@ because they are handled well in Spark (e.g., `SQLOrderingUtil.compareFloats`). functions of arrow-rs used by DataFusion do not normalize NaN and zero (e.g., [arrow::compute::kernels::cmp::eq](https://docs.rs/arrow/latest/arrow/compute/kernels/cmp/fn.eq.html#)). So Comet will add additional normalization expression of NaN and zero for comparison. +There is a known bug with using count(distinct) within aggregate queries, where each NaN value will be counted +separately (#1824)[https://github.com/apache/datafusion-comet/issues/1824]. + ## Incompatible Expressions Some Comet native expressions are not 100% compatible with Spark and are disabled by default. These expressions diff --git a/docs/templates/compatibility-template.md b/docs/templates/compatibility-template.md index 9f9f9911b1..27be0eca9a 100644 --- a/docs/templates/compatibility-template.md +++ b/docs/templates/compatibility-template.md @@ -29,12 +29,6 @@ Comet aims to provide consistent results with the version of Apache Spark that i This guide offers information about areas of functionality where there are known differences. -# Compatibility Guide - -Comet aims to provide consistent results with the version of Apache Spark that is being used. - -This guide offers information about areas of functionality where there are known differences. - ## Parquet Scans Comet currently has three distinct implementations of the Parquet scan operator. The configuration property @@ -89,6 +83,9 @@ because they are handled well in Spark (e.g., `SQLOrderingUtil.compareFloats`). functions of arrow-rs used by DataFusion do not normalize NaN and zero (e.g., [arrow::compute::kernels::cmp::eq](https://docs.rs/arrow/latest/arrow/compute/kernels/cmp/fn.eq.html#)). So Comet will add additional normalization expression of NaN and zero for comparison. +There is a known bug with using count(distinct) within aggregate queries, where each NaN value will be counted +separately (#1824)[https://github.com/apache/datafusion-comet/issues/1824]. + ## Incompatible Expressions Some Comet native expressions are not 100% compatible with Spark and are disabled by default. These expressions From c4219cdbf8b15d79da09438ee249839b5968f895 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 4 Jun 2025 16:37:54 -0600 Subject: [PATCH 09/42] 3.4.3 --- dev/diffs/3.4.3.diff | 12 +++++++++++- docs/source/user-guide/compatibility.md | 2 +- docs/templates/compatibility-template.md | 2 +- 3 files changed, 13 insertions(+), 3 deletions(-) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 049671c2ed..bf44e3d7ab 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -247,7 +247,7 @@ index cf40e944c09..bdd5be4f462 100644 test("A cached table preserves the partitioning and ordering of its cached SparkPlan") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala -index 1cc09c3d7fc..f031fa45c33 100644 +index 1cc09c3d7fc..b85b53a9688 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 @@ -27,7 +27,7 @@ import org.apache.spark.SparkException @@ -268,6 +268,16 @@ index 1cc09c3d7fc..f031fa45c33 100644 } assert(exchangePlans.length == 1) } +@@ -1100,7 +1100,8 @@ class DataFrameAggregateSuite extends QueryTest + } + } + +- test("SPARK-32038: NormalizeFloatingNumbers should work on distinct aggregate") { ++ test("SPARK-32038: NormalizeFloatingNumbers should work on distinct aggregate", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1824")) { + withTempView("view") { + val nan1 = java.lang.Float.intBitsToFloat(0x7f800001) + val nan2 = java.lang.Float.intBitsToFloat(0x7fffffff) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index 56e9520fdab..917932336df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala diff --git a/docs/source/user-guide/compatibility.md b/docs/source/user-guide/compatibility.md index 677c291b9a..39cd3a0582 100644 --- a/docs/source/user-guide/compatibility.md +++ b/docs/source/user-guide/compatibility.md @@ -84,7 +84,7 @@ functions of arrow-rs used by DataFusion do not normalize NaN and zero (e.g., [a So Comet will add additional normalization expression of NaN and zero for comparison. There is a known bug with using count(distinct) within aggregate queries, where each NaN value will be counted -separately (#1824)[https://github.com/apache/datafusion-comet/issues/1824]. +separately [#1824](https://github.com/apache/datafusion-comet/issues/1824). ## Incompatible Expressions diff --git a/docs/templates/compatibility-template.md b/docs/templates/compatibility-template.md index 27be0eca9a..e304d933fa 100644 --- a/docs/templates/compatibility-template.md +++ b/docs/templates/compatibility-template.md @@ -84,7 +84,7 @@ functions of arrow-rs used by DataFusion do not normalize NaN and zero (e.g., [a So Comet will add additional normalization expression of NaN and zero for comparison. There is a known bug with using count(distinct) within aggregate queries, where each NaN value will be counted -separately (#1824)[https://github.com/apache/datafusion-comet/issues/1824]. +separately [#1824](https://github.com/apache/datafusion-comet/issues/1824). ## Incompatible Expressions From eb69d509ffe093c01c65d5cd6e5c07feca5ad06b Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 4 Jun 2025 16:43:02 -0600 Subject: [PATCH 10/42] 3.5.4 --- dev/diffs/3.5.4.diff | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/dev/diffs/3.5.4.diff b/dev/diffs/3.5.4.diff index c594fbbe42..8b7f052802 100644 --- a/dev/diffs/3.5.4.diff +++ b/dev/diffs/3.5.4.diff @@ -226,7 +226,7 @@ index 9815cb816c9..95b5f9992b0 100644 test("A cached table preserves the partitioning and ordering of its cached SparkPlan") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala -index 5a8681aed97..da9d25e2eb4 100644 +index 5a8681aed97..db69fde723a 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 @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical.Expand @@ -247,6 +247,16 @@ index 5a8681aed97..da9d25e2eb4 100644 } assert(exchangePlans.length == 1) } +@@ -1255,7 +1255,8 @@ class DataFrameAggregateSuite extends QueryTest + } + } + +- test("SPARK-32038: NormalizeFloatingNumbers should work on distinct aggregate") { ++ test("SPARK-32038: NormalizeFloatingNumbers should work on distinct aggregate", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1824")) { + withTempView("view") { + val nan1 = java.lang.Float.intBitsToFloat(0x7f800001) + val nan2 = java.lang.Float.intBitsToFloat(0x7fffffff) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index 56e9520fdab..917932336df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala From 6f90a2b3e19e35391952344fb0acf5188c6e3d9c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 4 Jun 2025 16:45:42 -0600 Subject: [PATCH 11/42] 4.0.0-preview1 --- dev/diffs/4.0.0-preview1.diff | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/dev/diffs/4.0.0-preview1.diff b/dev/diffs/4.0.0-preview1.diff index c014660cf5..99cbd0325d 100644 --- a/dev/diffs/4.0.0-preview1.diff +++ b/dev/diffs/4.0.0-preview1.diff @@ -268,7 +268,7 @@ index d023fb82185..0f4f03bda6c 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 620ee430cab..9d383a4bff9 100644 +index 620ee430cab..f5df9218fc1 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 @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.util.AUTO_GENERATED_ALIAS @@ -289,6 +289,16 @@ index 620ee430cab..9d383a4bff9 100644 } assert(exchangePlans.length == 1) } +@@ -1275,7 +1275,8 @@ class DataFrameAggregateSuite extends QueryTest + } + } + +- test("SPARK-32038: NormalizeFloatingNumbers should work on distinct aggregate") { ++ test("SPARK-32038: NormalizeFloatingNumbers should work on distinct aggregate", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1824")) { + withTempView("view") { + val nan1 = java.lang.Float.intBitsToFloat(0x7f800001) + val nan2 = java.lang.Float.intBitsToFloat(0x7fffffff) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index f6fd6b501d7..11870c85d82 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala From 25fcccf3109cdf69f0ac9ac69cb91089b73511d5 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 4 Jun 2025 18:55:54 -0600 Subject: [PATCH 12/42] fall back to Spark for window ranges --- .../apache/comet/serde/QueryPlanSerde.scala | 18 ++++-------------- .../apache/comet/CometExpressionSuite.scala | 17 +++++++++++++++++ 2 files changed, 21 insertions(+), 14 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index 3fedaa7e36..f5dbe391b0 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -288,21 +288,11 @@ object QueryPlanSerde extends Logging with CometExprShim { .newBuilder() .setCurrentRow(OperatorOuterClass.CurrentRow.newBuilder().build()) .build() - case e => - val offset = e.eval() match { - case i: Integer => i.toLong - case l: Long => l - case _ => return None - } - OperatorOuterClass.UpperWindowFrameBound - .newBuilder() - .setFollowing( - OperatorOuterClass.Following - .newBuilder() - .setOffset(offset) - .build()) - .build() + case _ => + // TODO add support for numeric and temporal offsets + // see https://github.com/apache/datafusion-comet/issues/1246 + return None } (frameProto, lBoundProto, uBoundProto) diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 6273ab9b0d..7aa2a68576 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -21,6 +21,7 @@ package org.apache.comet import java.time.{Duration, Period} +import scala.collection.immutable.Seq import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag import scala.util.Random @@ -31,6 +32,7 @@ import org.apache.spark.sql.catalyst.optimizer.SimplifyExtractValueOps import org.apache.spark.sql.comet.{CometColumnarToRowExec, CometProjectExec} import org.apache.spark.sql.execution.{InputAdapter, ProjectExec, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.SESSION_LOCAL_TIMEZONE @@ -2705,4 +2707,19 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } + test("type coercion with window range") { + + // values are int + val df = Seq(1, 2, 4, 3, 2, 1).toDF("value") + val window = Window.orderBy($"value".desc) + + // ranges are long + val df2 = df.select( + $"value", + sum($"value").over(window.rangeBetween(Window.unboundedPreceding, 1L)), + sum($"value").over(window.rangeBetween(1L, Window.unboundedFollowing))) + + checkSparkAnswer(df2) + } + } From cd03a1cce3f75cc48d36615cce52e5e509525d96 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 4 Jun 2025 19:17:51 -0600 Subject: [PATCH 13/42] ignore DPP test with 3.5.5 --- dev/diffs/3.5.5.diff | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/dev/diffs/3.5.5.diff b/dev/diffs/3.5.5.diff index 028530978b..8f73864301 100644 --- a/dev/diffs/3.5.5.diff +++ b/dev/diffs/3.5.5.diff @@ -403,7 +403,7 @@ index f32b32ffc5a..447d7c6416e 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index f33432ddb6f..fe9f74ff8f1 100644 +index f33432ddb6f..e0bc502eda0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen @@ -424,7 +424,17 @@ index f33432ddb6f..fe9f74ff8f1 100644 case _ => Nil } } -@@ -1729,6 +1733,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1423,7 +1427,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("SPARK-34637: DPP side broadcast query stage is created firstly") { ++ test("SPARK-34637: DPP side broadcast query stage is created firstly", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1839")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + val df = sql( + """ WITH v as ( +@@ -1729,6 +1734,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat case s: BatchScanExec => // we use f1 col for v2 tables due to schema pruning s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) From a79b445faaf257566238f5e4c5c2a8c39c35244f Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 4 Jun 2025 20:23:17 -0600 Subject: [PATCH 14/42] fix --- .../test/scala/org/apache/comet/exec/CometExecSuite.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index 26cae2f8e9..d3d02b267f 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -1979,7 +1979,12 @@ class CometExecSuite extends CometTestBase { s"SELECT $function OVER(order by _2 rows between current row and 1 following) FROM t1") queries.foreach { query => - checkSparkAnswerAndOperator(query) + if (query.contains("rows between 1 preceding and 1 following")) { + // https://github.com/apache/datafusion-comet/issues/1246 + checkSparkAnswerAndOperator(query) + } else { + checkSparkAnswer(query) + } } } } From 3bf0e72d4a3e4aaeaccaf1d61f5a036aab7e1871 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 4 Jun 2025 22:11:22 -0600 Subject: [PATCH 15/42] fix --- .../apache/comet/serde/QueryPlanSerde.scala | 23 ++++++++++++++++--- .../apache/comet/exec/CometExecSuite.scala | 7 +----- 2 files changed, 21 insertions(+), 9 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index f5dbe391b0..bb278e4231 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -261,7 +261,7 @@ object QueryPlanSerde extends Logging with CometExprShim { .newBuilder() .setCurrentRow(OperatorOuterClass.CurrentRow.newBuilder().build()) .build() - case e => + case e if frameType == RowFrame => val offset = e.eval() match { case i: Integer => i.toLong case l: Long => l @@ -275,6 +275,10 @@ object QueryPlanSerde extends Logging with CometExprShim { .setOffset(offset) .build()) .build() + case _ => + // TODO add support for numeric and temporal range offsets + // see https://github.com/apache/datafusion-comet/issues/1246 + return None } val uBoundProto = uBound match { @@ -288,9 +292,22 @@ object QueryPlanSerde extends Logging with CometExprShim { .newBuilder() .setCurrentRow(OperatorOuterClass.CurrentRow.newBuilder().build()) .build() - + case e if frameType == RowFrame => + val offset = e.eval() match { + case i: Integer => i.toLong + case l: Long => l + case _ => return None + } + OperatorOuterClass.UpperWindowFrameBound + .newBuilder() + .setFollowing( + OperatorOuterClass.Following + .newBuilder() + .setOffset(offset) + .build()) + .build() case _ => - // TODO add support for numeric and temporal offsets + // TODO add support for numeric and temporal range offsets // see https://github.com/apache/datafusion-comet/issues/1246 return None } diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index d3d02b267f..26cae2f8e9 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -1979,12 +1979,7 @@ class CometExecSuite extends CometTestBase { s"SELECT $function OVER(order by _2 rows between current row and 1 following) FROM t1") queries.foreach { query => - if (query.contains("rows between 1 preceding and 1 following")) { - // https://github.com/apache/datafusion-comet/issues/1246 - checkSparkAnswerAndOperator(query) - } else { - checkSparkAnswer(query) - } + checkSparkAnswerAndOperator(query) } } } From 4d7919cc45769d8ab88ddbdd577ca4f387d53861 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 4 Jun 2025 22:15:18 -0600 Subject: [PATCH 16/42] fix --- .../main/scala/org/apache/comet/serde/QueryPlanSerde.scala | 4 ++-- .../test/scala/org/apache/comet/CometExpressionSuite.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index bb278e4231..13bea457d3 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -276,7 +276,7 @@ object QueryPlanSerde extends Logging with CometExprShim { .build()) .build() case _ => - // TODO add support for numeric and temporal range offsets + // TODO add support for numeric and temporal RANGE BETWEEN expressions // see https://github.com/apache/datafusion-comet/issues/1246 return None } @@ -307,7 +307,7 @@ object QueryPlanSerde extends Logging with CometExprShim { .build()) .build() case _ => - // TODO add support for numeric and temporal range offsets + // TODO add support for numeric and temporal RANGE BETWEEN expressions // see https://github.com/apache/datafusion-comet/issues/1246 return None } diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 7aa2a68576..dc9aa2866d 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -2707,7 +2707,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } - test("type coercion with window range") { + test("window query with rangeBetween") { // values are int val df = Seq(1, 2, 4, 3, 2, 1).toDF("value") From 5aca868fc0833bc536e0b5581a2f3940685232fc Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 5 Jun 2025 09:57:44 -0600 Subject: [PATCH 17/42] disable some tests temporarily --- .github/workflows/spark_sql_test.yml | 3 +- .github/workflows/spark_sql_test_ansi.yml | 154 +++++++++++----------- 2 files changed, 79 insertions(+), 78 deletions(-) diff --git a/.github/workflows/spark_sql_test.yml b/.github/workflows/spark_sql_test.yml index bb704c20ec..9598a5accc 100644 --- a/.github/workflows/spark_sql_test.yml +++ b/.github/workflows/spark_sql_test.yml @@ -45,7 +45,8 @@ jobs: matrix: os: [ubuntu-24.04] java-version: [11] - spark-version: [{short: '3.4', full: '3.4.3'}, {short: '3.5', full: '3.5.4'}, {short: '3.5', full: '3.5.5'}] + # TODO enbable other versions once tests are passing with 3.5.5 + spark-version: [{short: '3.5', full: '3.5.5'}] module: - {name: "catalyst", args1: "catalyst/test", args2: ""} - {name: "sql/core-1", args1: "", args2: sql/testOnly * -- -l org.apache.spark.tags.ExtendedSQLTest -l org.apache.spark.tags.SlowSQLTest} diff --git a/.github/workflows/spark_sql_test_ansi.yml b/.github/workflows/spark_sql_test_ansi.yml index a3e85307d9..878e9a803c 100644 --- a/.github/workflows/spark_sql_test_ansi.yml +++ b/.github/workflows/spark_sql_test_ansi.yml @@ -1,79 +1,79 @@ -# 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 +## 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. # -# http://www.apache.org/licenses/LICENSE-2.0 +#name: Spark SQL Tests (ANSI mode) +# +#concurrency: +# group: ${{ github.repository }}-${{ github.head_ref || github.sha }}-${{ github.workflow }} +# cancel-in-progress: true +# +#on: +# push: +# paths-ignore: +# - "docs/**" +# - "**.md" +# pull_request: +# paths-ignore: +# - "docs/**" +# - "**.md" +# # manual trigger +# # https://docs.github.com/en/actions/managing-workflow-runs/manually-running-a-workflow +# workflow_dispatch: +# +#env: +# RUST_VERSION: stable +# +#jobs: +# spark-sql-catalyst: +# strategy: +# matrix: +# os: [ubuntu-24.04] +# java-version: [17] +# spark-version: [{short: '4.0', full: '4.0.0-preview1'}] +# module: +# - {name: "catalyst", args1: "catalyst/test", args2: ""} +# - {name: "sql/core-1", args1: "", args2: sql/testOnly * -- -l org.apache.spark.tags.ExtendedSQLTest -l org.apache.spark.tags.SlowSQLTest} +# - {name: "sql/core-2", args1: "", args2: "sql/testOnly * -- -n org.apache.spark.tags.ExtendedSQLTest"} +# - {name: "sql/core-3", args1: "", args2: "sql/testOnly * -- -n org.apache.spark.tags.SlowSQLTest"} +# - {name: "sql/hive-1", args1: "", args2: "hive/testOnly * -- -l org.apache.spark.tags.ExtendedHiveTest -l org.apache.spark.tags.SlowHiveTest"} +# - {name: "sql/hive-2", args1: "", args2: "hive/testOnly * -- -n org.apache.spark.tags.ExtendedHiveTest"} +# - {name: "sql/hive-3", args1: "", args2: "hive/testOnly * -- -n org.apache.spark.tags.SlowHiveTest"} +# fail-fast: false +# name: spark-sql-${{ matrix.module.name }}/${{ matrix.os }}/spark-${{ matrix.spark-version.full }}/java-${{ matrix.java-version }} +# runs-on: ${{ matrix.os }} +# container: +# image: amd64/rust +# steps: +# - uses: actions/checkout@v4 +# - name: Setup Rust & Java toolchain +# uses: ./.github/actions/setup-builder +# with: +# rust-version: ${{env.RUST_VERSION}} +# jdk-version: ${{ matrix.java-version }} +# - name: Setup Spark +# uses: ./.github/actions/setup-spark-builder +# with: +# spark-version: ${{ matrix.spark-version.full }} +# spark-short-version: ${{ matrix.spark-version.short }} +# - name: Run Spark tests +# run: | +# cd apache-spark +# rm -rf /root/.m2/repository/org/apache/parquet # somehow parquet cache requires cleanups +# RUST_BACKTRACE=1 ENABLE_COMET=true ENABLE_COMET_ANSI_MODE=true ENABLE_COMET_SHUFFLE=true build/sbt -Dsbt.log.noformat=true ${{ matrix.module.args1 }} "${{ matrix.module.args2 }}" +# env: +# LC_ALL: "C.UTF-8" # -# 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. - -name: Spark SQL Tests (ANSI mode) - -concurrency: - group: ${{ github.repository }}-${{ github.head_ref || github.sha }}-${{ github.workflow }} - cancel-in-progress: true - -on: - push: - paths-ignore: - - "docs/**" - - "**.md" - pull_request: - paths-ignore: - - "docs/**" - - "**.md" - # manual trigger - # https://docs.github.com/en/actions/managing-workflow-runs/manually-running-a-workflow - workflow_dispatch: - -env: - RUST_VERSION: stable - -jobs: - spark-sql-catalyst: - strategy: - matrix: - os: [ubuntu-24.04] - java-version: [17] - spark-version: [{short: '4.0', full: '4.0.0-preview1'}] - module: - - {name: "catalyst", args1: "catalyst/test", args2: ""} - - {name: "sql/core-1", args1: "", args2: sql/testOnly * -- -l org.apache.spark.tags.ExtendedSQLTest -l org.apache.spark.tags.SlowSQLTest} - - {name: "sql/core-2", args1: "", args2: "sql/testOnly * -- -n org.apache.spark.tags.ExtendedSQLTest"} - - {name: "sql/core-3", args1: "", args2: "sql/testOnly * -- -n org.apache.spark.tags.SlowSQLTest"} - - {name: "sql/hive-1", args1: "", args2: "hive/testOnly * -- -l org.apache.spark.tags.ExtendedHiveTest -l org.apache.spark.tags.SlowHiveTest"} - - {name: "sql/hive-2", args1: "", args2: "hive/testOnly * -- -n org.apache.spark.tags.ExtendedHiveTest"} - - {name: "sql/hive-3", args1: "", args2: "hive/testOnly * -- -n org.apache.spark.tags.SlowHiveTest"} - fail-fast: false - name: spark-sql-${{ matrix.module.name }}/${{ matrix.os }}/spark-${{ matrix.spark-version.full }}/java-${{ matrix.java-version }} - runs-on: ${{ matrix.os }} - container: - image: amd64/rust - steps: - - uses: actions/checkout@v4 - - name: Setup Rust & Java toolchain - uses: ./.github/actions/setup-builder - with: - rust-version: ${{env.RUST_VERSION}} - jdk-version: ${{ matrix.java-version }} - - name: Setup Spark - uses: ./.github/actions/setup-spark-builder - with: - spark-version: ${{ matrix.spark-version.full }} - spark-short-version: ${{ matrix.spark-version.short }} - - name: Run Spark tests - run: | - cd apache-spark - rm -rf /root/.m2/repository/org/apache/parquet # somehow parquet cache requires cleanups - RUST_BACKTRACE=1 ENABLE_COMET=true ENABLE_COMET_ANSI_MODE=true ENABLE_COMET_SHUFFLE=true build/sbt -Dsbt.log.noformat=true ${{ matrix.module.args1 }} "${{ matrix.module.args2 }}" - env: - LC_ALL: "C.UTF-8" - From a0f70f15a5cf9721eef21d410b26c76dddd72ff5 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 5 Jun 2025 10:09:21 -0600 Subject: [PATCH 18/42] ignore some tests --- dev/diffs/3.5.5.diff | 95 +++++++++++++++++++++++++++++++++----------- 1 file changed, 72 insertions(+), 23 deletions(-) diff --git a/dev/diffs/3.5.5.diff b/dev/diffs/3.5.5.diff index 8f73864301..3abd040972 100644 --- a/dev/diffs/3.5.5.diff +++ b/dev/diffs/3.5.5.diff @@ -1239,15 +1239,13 @@ index de24b8c82b0..1f835481290 100644 setupTestData() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala -index 9e9d717db3b..c1a7caf56e0 100644 +index 9e9d717db3b..91a4f9a38d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala -@@ -17,7 +17,8 @@ - +@@ -18,6 +18,7 @@ package org.apache.spark.sql.execution --import org.apache.spark.sql.{DataFrame, QueryTest, Row} -+import org.apache.spark.sql.{DataFrame, QueryTest, Row} + import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.spark.sql.comet.CometProjectExec import org.apache.spark.sql.connector.SimpleWritableDataSource import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} @@ -1264,15 +1262,6 @@ index 9e9d717db3b..c1a7caf56e0 100644 assert(actual == expected) } } -@@ -112,7 +116,7 @@ abstract class RemoveRedundantProjectsSuiteBase - assertProjectExec(query, 1, 3) - } - -- test("join with ordering requirement") { -+ test("join with ordering requirement") { - val query = "select * from (select key, a, c, b from testView) as t1 join " + - "(select key, a, b, c from testView) as t2 on t1.key = t2.key where t2.a > 50" - assertProjectExec(query, 2, 2) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantSortsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantSortsSuite.scala index 005e764cc30..92ec088efab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantSortsSuite.scala @@ -1340,18 +1329,42 @@ index b14f4a405f6..ab7baf434a5 100644 spark.range(1).foreach { _ => columnarToRowExec.canonicalized diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala -index 5a413c77754..a6f97dccb67 100644 +index 5a413c77754..be3a38e6c4a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala -@@ -20,6 +20,7 @@ package org.apache.spark.sql.execution - import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} +@@ -17,9 +17,10 @@ + + package org.apache.spark.sql.execution + +-import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} ++import org.apache.spark.sql.{Dataset, IgnoreComet, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeAndComment, CodeGenerator} +import org.apache.spark.sql.comet.{CometSortExec, CometSortMergeJoinExec} import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec -@@ -235,6 +236,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -161,7 +162,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + assert(df.collect() === Array(Row(1, 1, "1"), Row(1, 1, "1"), Row(2, 2, "2"))) + } + +- test("Inner ShuffledHashJoin should be included in WholeStageCodegen") { ++ test("Inner ShuffledHashJoin should be included in WholeStageCodegen", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1852")) { + val df1 = spark.range(5).select($"id".as("k1")) + val df2 = spark.range(15).select($"id".as("k2")) + val df3 = spark.range(6).select($"id".as("k3")) +@@ -193,7 +195,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + } + } + +- test("Full Outer ShuffledHashJoin and SortMergeJoin should be included in WholeStageCodegen") { ++ test("Full Outer ShuffledHashJoin and SortMergeJoin should be included in WholeStageCodegen", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1852")) { + val df1 = spark.range(5).select($"id".as("k1")) + val df2 = spark.range(10).select($"id".as("k2")) + val df3 = spark.range(3).select($"id".as("k3")) +@@ -235,6 +238,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession assert(twoJoinsDF.queryExecution.executedPlan.collect { case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true @@ -1359,7 +1372,27 @@ index 5a413c77754..a6f97dccb67 100644 }.size === 2) checkAnswer(twoJoinsDF, Seq(Row(0, 0, 0), Row(1, 1, null), Row(2, 2, 2), Row(3, 3, null), Row(4, 4, null), -@@ -358,6 +360,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -244,7 +248,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + } + + +- test("SPARK-44060 Code-gen for build side outer shuffled hash join") { ++ test("SPARK-44060 Code-gen for build side outer shuffled hash join", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1852")) { + val df1 = spark.range(0, 5).select($"id".as("k1")) + val df2 = spark.range(1, 11).select($"id".as("k2")) + val df3 = spark.range(2, 5).select($"id".as("k3")) +@@ -332,7 +337,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + } + } + +- test("Left/Right Outer SortMergeJoin should be included in WholeStageCodegen") { ++ test("Left/Right Outer SortMergeJoin should be included in WholeStageCodegen", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1852")) { + val df1 = spark.range(10).select($"id".as("k1")) + val df2 = spark.range(4).select($"id".as("k2")) + val df3 = spark.range(6).select($"id".as("k3")) +@@ -358,13 +364,15 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "right_outer") assert(twoJoinsDF.queryExecution.executedPlan.collect { case WholeStageCodegenExec(_ : SortMergeJoinExec) => true @@ -1367,7 +1400,16 @@ index 5a413c77754..a6f97dccb67 100644 }.size === 2) checkAnswer(twoJoinsDF, Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, null, 4), Row(5, null, 5), -@@ -380,8 +383,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + Row(null, null, 6), Row(null, null, 7), Row(null, null, 8), Row(null, null, 9))) + } + +- test("Left Semi SortMergeJoin should be included in WholeStageCodegen") { ++ test("Left Semi SortMergeJoin should be included in WholeStageCodegen", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1852")) { + val df1 = spark.range(10).select($"id".as("k1")) + val df2 = spark.range(4).select($"id".as("k2")) + val df3 = spark.range(6).select($"id".as("k3")) +@@ -380,13 +388,13 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession val twoJoinsDF = df3.join(df2.hint("SHUFFLE_MERGE"), $"k3" === $"k2", "left_semi") .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "left_semi") assert(twoJoinsDF.queryExecution.executedPlan.collect { @@ -1377,7 +1419,14 @@ index 5a413c77754..a6f97dccb67 100644 }.size === 2) checkAnswer(twoJoinsDF, Seq(Row(0), Row(1), Row(2), Row(3))) } -@@ -402,8 +404,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + +- test("Left Anti SortMergeJoin should be included in WholeStageCodegen") { ++ test("Left Anti SortMergeJoin should be included in WholeStageCodegen", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1852")) { + val df1 = spark.range(10).select($"id".as("k1")) + val df2 = spark.range(4).select($"id".as("k2")) + val df3 = spark.range(6).select($"id".as("k3")) +@@ -402,8 +410,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession val twoJoinsDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_anti") .join(df3.hint("SHUFFLE_MERGE"), $"k1" === $"k3", "left_anti") assert(twoJoinsDF.queryExecution.executedPlan.collect { @@ -1387,7 +1436,7 @@ index 5a413c77754..a6f97dccb67 100644 }.size === 2) checkAnswer(twoJoinsDF, Seq(Row(6), Row(7), Row(8), Row(9))) } -@@ -536,7 +537,10 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -536,7 +543,10 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession val plan = df.queryExecution.executedPlan assert(plan.exists(p => p.isInstanceOf[WholeStageCodegenExec] && @@ -1399,7 +1448,7 @@ index 5a413c77754..a6f97dccb67 100644 assert(df.collect() === Array(Row(1), Row(2), Row(3))) } -@@ -716,7 +720,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession +@@ -716,7 +726,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession .write.mode(SaveMode.Overwrite).parquet(path) withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "255", From c5a115315f75077275036704e9b95f0d812495f6 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 5 Jun 2025 11:54:56 -0600 Subject: [PATCH 19/42] update tests --- dev/diffs/3.5.5.diff | 126 ++++++++++++++++++++++--------------------- 1 file changed, 64 insertions(+), 62 deletions(-) diff --git a/dev/diffs/3.5.5.diff b/dev/diffs/3.5.5.diff index 3abd040972..8650133738 100644 --- a/dev/diffs/3.5.5.diff +++ b/dev/diffs/3.5.5.diff @@ -403,7 +403,7 @@ index f32b32ffc5a..447d7c6416e 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index f33432ddb6f..e0bc502eda0 100644 +index f33432ddb6f..0e1499a24ca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen @@ -424,7 +424,27 @@ index f33432ddb6f..e0bc502eda0 100644 case _ => Nil } } -@@ -1423,7 +1427,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1027,7 +1031,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("avoid reordering broadcast join keys to match input hash partitioning") { ++ test("avoid reordering broadcast join keys to match input hash partitioning", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1839")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + withTable("large", "dimTwo", "dimThree") { +@@ -1215,7 +1220,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + + test("SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + +- "canonicalization and exchange reuse") { ++ "canonicalization and exchange reuse", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1839")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df = sql( +@@ -1423,7 +1429,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -434,7 +454,7 @@ index f33432ddb6f..e0bc502eda0 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ WITH v as ( -@@ -1729,6 +1734,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1729,6 +1736,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat case s: BatchScanExec => // we use f1 col for v2 tables due to schema pruning s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) @@ -531,57 +551,6 @@ index 93275487f29..01e5c601763 100644 }.flatten assert(filters.contains(GreaterThan(scan.logicalPlan.output.head, Literal(5L)))) } -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala -new file mode 100644 -index 00000000000..5691536c114 ---- /dev/null -+++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala -@@ -0,0 +1,45 @@ -+/* -+ * 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 -+ -+import org.scalactic.source.Position -+import org.scalatest.Tag -+ -+import org.apache.spark.sql.test.SQLTestUtils -+ -+/** -+ * Tests with this tag will be ignored when Comet is enabled (e.g., via `ENABLE_COMET`). -+ */ -+case class IgnoreComet(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeIcebergCompat(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeDataFusion(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeScan(reason: String) extends Tag("DisableComet") -+ -+/** -+ * Helper trait that disables Comet for all tests regardless of default config values. -+ */ -+trait IgnoreCometSuite extends SQLTestUtils { -+ override protected def test(testName: String, testTags: Tag*)(testFun: => Any) -+ (implicit pos: Position): Unit = { -+ if (isCometEnabled) { -+ ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) -+ } else { -+ super.test(testName, testTags: _*)(testFun) -+ } -+ } -+} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala index 7af826583bd..3c3def1eb67 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala @@ -611,7 +580,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 4d256154c85..43f0bebb00c 100644 +index 4d256154c85..66a5473852d 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 @@ -737,7 +706,7 @@ index 4d256154c85..43f0bebb00c 100644 }.size === 1) // Same result between shuffled hash join and sort merge join checkAnswer(shjDF, smjResult) -@@ -1432,13 +1446,19 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1432,13 +1446,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan assert(shjCodegenDF.queryExecution.executedPlan.collect { case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true case WholeStageCodegenExec(ProjectExec(_, _ : ShuffledHashJoinExec)) => true @@ -745,6 +714,7 @@ index 4d256154c85..43f0bebb00c 100644 + true + case WholeStageCodegenExec(ColumnarToRowExec( + InputAdapter(CometProjectExec(_, _, _, _, _: CometHashJoinExec, _)))) => true ++ case _: CometHashJoinExec => true }.size === 1) checkAnswer(shjCodegenDF, Seq.empty) @@ -758,7 +728,7 @@ index 4d256154c85..43f0bebb00c 100644 checkAnswer(shjNonCodegenDF, Seq.empty) } } -@@ -1486,7 +1506,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1486,7 +1507,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val plan = sql(getAggQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) // Have shuffle before aggregation @@ -768,7 +738,7 @@ index 4d256154c85..43f0bebb00c 100644 } def getJoinQuery(selectExpr: String, joinType: String): String = { -@@ -1515,9 +1536,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1515,9 +1537,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan } val plan = sql(getJoinQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) @@ -783,7 +753,7 @@ index 4d256154c85..43f0bebb00c 100644 } // Test output ordering is not preserved -@@ -1526,9 +1550,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1526,9 +1551,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val selectExpr = "/*+ BROADCAST(left_t) */ k1 as k0" val plan = sql(getJoinQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) @@ -798,7 +768,7 @@ index 4d256154c85..43f0bebb00c 100644 } // Test singe partition -@@ -1538,7 +1565,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1538,7 +1566,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan |FROM range(0, 10, 1, 1) t1 FULL OUTER JOIN range(0, 10, 1, 1) t2 |""".stripMargin) val plan = fullJoinDF.queryExecution.executedPlan @@ -808,7 +778,7 @@ index 4d256154c85..43f0bebb00c 100644 checkAnswer(fullJoinDF, Row(100)) } } -@@ -1583,6 +1611,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1583,6 +1612,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 @@ -818,7 +788,7 @@ index 4d256154c85..43f0bebb00c 100644 }.size == 1) } } -@@ -1627,14 +1658,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1627,14 +1659,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 = { @@ -841,6 +811,16 @@ index 4d256154c85..43f0bebb00c 100644 } dupStreamSideColTest("SHUFFLE_HASH", check) } +@@ -1770,7 +1808,8 @@ class ThreadLeakInSortMergeJoinSuite + sparkConf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, 20)) + } + +- test("SPARK-47146: thread leak when doing SortMergeJoin (with spill)") { ++ test("SPARK-47146: thread leak when doing SortMergeJoin (with spill)", ++ IgnoreComet("Comet does not support spilling")) { + + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala index c26757c9cff..d55775f09d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala @@ -1304,6 +1284,28 @@ index 47679ed7865..9ffbaecb98e 100644 }.length == hashAggCount) assert(collectWithSubqueries(plan) { case s: SortAggregateExec => s }.length == sortAggCount) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala +index eec396b2e39..bf3f1c769d6 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala +@@ -18,7 +18,7 @@ + package org.apache.spark.sql.execution + + import org.apache.spark.TestUtils.assertSpilled +-import org.apache.spark.sql.{AnalysisException, QueryTest, Row} ++import org.apache.spark.sql.{AnalysisException, IgnoreComet, QueryTest, Row} + import org.apache.spark.sql.internal.SQLConf.{WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD, WINDOW_EXEC_BUFFER_SPILL_THRESHOLD} + import org.apache.spark.sql.test.SharedSparkSession + +@@ -470,7 +470,7 @@ class SQLWindowFunctionSuite extends QueryTest with SharedSparkSession { + Row(1, 3, null) :: Row(2, null, 4) :: Nil) + } + +- test("test with low buffer spill threshold") { ++ test("test with low buffer spill threshold", IgnoreComet("Comet does not support spilling")) { + val nums = sparkContext.parallelize(1 to 10).map(x => (x, x % 2)).toDF("x", "y") + nums.createOrReplaceTempView("nums") + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala index b14f4a405f6..ab7baf434a5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala From f17601c1b89a4fe068220fb6cc11ecb5090a0c9c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 5 Jun 2025 11:55:07 -0600 Subject: [PATCH 20/42] update tests --- dev/diffs/3.5.5.diff | 51 ++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 51 insertions(+) diff --git a/dev/diffs/3.5.5.diff b/dev/diffs/3.5.5.diff index 8650133738..5090ec90c6 100644 --- a/dev/diffs/3.5.5.diff +++ b/dev/diffs/3.5.5.diff @@ -551,6 +551,57 @@ index 93275487f29..01e5c601763 100644 }.flatten assert(filters.contains(GreaterThan(scan.logicalPlan.output.head, Literal(5L)))) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala +new file mode 100644 +index 00000000000..5691536c114 +--- /dev/null ++++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala +@@ -0,0 +1,45 @@ ++/* ++ * 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 ++ ++import org.scalactic.source.Position ++import org.scalatest.Tag ++ ++import org.apache.spark.sql.test.SQLTestUtils ++ ++/** ++ * Tests with this tag will be ignored when Comet is enabled (e.g., via `ENABLE_COMET`). ++ */ ++case class IgnoreComet(reason: String) extends Tag("DisableComet") ++case class IgnoreCometNativeIcebergCompat(reason: String) extends Tag("DisableComet") ++case class IgnoreCometNativeDataFusion(reason: String) extends Tag("DisableComet") ++case class IgnoreCometNativeScan(reason: String) extends Tag("DisableComet") ++ ++/** ++ * Helper trait that disables Comet for all tests regardless of default config values. ++ */ ++trait IgnoreCometSuite extends SQLTestUtils { ++ override protected def test(testName: String, testTags: Tag*)(testFun: => Any) ++ (implicit pos: Position): Unit = { ++ if (isCometEnabled) { ++ ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) ++ } else { ++ super.test(testName, testTags: _*)(testFun) ++ } ++ } ++} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala index 7af826583bd..3c3def1eb67 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala From 18a993eff7470a45e17f1030ff857900ba9c52fd Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 5 Jun 2025 12:19:47 -0600 Subject: [PATCH 21/42] trigger ci From 57b5ffbb70a15e87214d9b0729e77dd4eb63403b Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 5 Jun 2025 15:47:22 -0600 Subject: [PATCH 22/42] debug --- .../test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala index 8d084fd75d..0e582ddaba 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala @@ -188,6 +188,7 @@ class CometTPCDSQuerySuite conf.set(CometConf.COMET_NATIVE_SCAN_ENABLED.key, "true") conf.set(CometConf.COMET_EXEC_SHUFFLE_ENABLED.key, "true") conf.set(CometConf.COMET_MEMORY_OVERHEAD.key, "15g") + conf.set(CometConf.COMET_EXPLAIN_TRANSFORMATIONS.key, "true") conf.set("spark.sql.adaptive.coalescePartitions.enabled", "true") conf.set(MEMORY_OFFHEAP_ENABLED.key, "true") conf.set(MEMORY_OFFHEAP_SIZE.key, "15g") From 09519e7ef8f31fe27b22af3fd174dfc7830749ac Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 5 Jun 2025 16:51:07 -0600 Subject: [PATCH 23/42] debug --- .../comet/rules/EliminateRedundantTransitions.scala | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala b/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala index ecc0823d60..8209fb391d 100644 --- a/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala +++ b/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala @@ -21,11 +21,10 @@ package org.apache.comet.rules import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.comet.{CometCollectLimitExec, CometColumnarToRowExec, CometPlan, CometSparkToColumnarExec} +import org.apache.spark.sql.comet.{CometBroadcastExchangeExec, CometCollectLimitExec, CometColumnarToRowExec, CometPlan, CometSparkToColumnarExec} import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometShuffleExchangeExec} import org.apache.spark.sql.execution.{ColumnarToRowExec, RowToColumnarExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.QueryStageExec - import org.apache.comet.CometConf // This rule is responsible for eliminating redundant transitions between row-based and @@ -56,7 +55,8 @@ case class EliminateRedundantTransitions(session: SparkSession) extends Rule[Spa override def apply(plan: SparkPlan): SparkPlan = { val newPlan = _apply(plan) if (showTransformations) { - logInfo(s"\nINPUT: $plan\nOUTPUT: $newPlan") + // scalastyle:off println + System.err.println(s"EliminateRedundantTransitions:\nINPUT: $plan\nOUTPUT: $newPlan") } newPlan } @@ -64,8 +64,11 @@ case class EliminateRedundantTransitions(session: SparkSession) extends Rule[Spa private def _apply(plan: SparkPlan): SparkPlan = { val eliminatedPlan = plan transformUp { case ColumnarToRowExec(shuffleExchangeExec: CometShuffleExchangeExec) - if (plan.conf.adaptiveExecutionEnabled) => + if plan.conf.adaptiveExecutionEnabled => shuffleExchangeExec + case ColumnarToRowExec(broadcastExchangeExec: CometBroadcastExchangeExec) + if plan.conf.adaptiveExecutionEnabled => + broadcastExchangeExec case ColumnarToRowExec(sparkToColumnar: CometSparkToColumnarExec) => if (sparkToColumnar.child.supportsColumnar) { // For Spark Columnar to Comet Columnar, we should keep the ColumnarToRowExec From 019bcd44bcc8f3c327cf579bef0c30a1d86b4edd Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 5 Jun 2025 18:23:23 -0600 Subject: [PATCH 24/42] fmt --- .../org/apache/comet/rules/EliminateRedundantTransitions.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala b/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala index 8209fb391d..d7fd210a35 100644 --- a/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala +++ b/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.comet.{CometBroadcastExchangeExec, CometCollectLimit import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometShuffleExchangeExec} import org.apache.spark.sql.execution.{ColumnarToRowExec, RowToColumnarExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.QueryStageExec + import org.apache.comet.CometConf // This rule is responsible for eliminating redundant transitions between row-based and @@ -67,7 +68,7 @@ case class EliminateRedundantTransitions(session: SparkSession) extends Rule[Spa if plan.conf.adaptiveExecutionEnabled => shuffleExchangeExec case ColumnarToRowExec(broadcastExchangeExec: CometBroadcastExchangeExec) - if plan.conf.adaptiveExecutionEnabled => + if plan.conf.adaptiveExecutionEnabled => broadcastExchangeExec case ColumnarToRowExec(sparkToColumnar: CometSparkToColumnarExec) => if (sparkToColumnar.child.supportsColumnar) { From d054f6bc905fd19eb372d1638da74c1c2bf741e7 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 6 Jun 2025 11:09:44 -0600 Subject: [PATCH 25/42] fix? --- .../org/apache/comet/rules/EliminateRedundantTransitions.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala b/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala index d7fd210a35..96b816698f 100644 --- a/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala +++ b/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala @@ -25,8 +25,8 @@ import org.apache.spark.sql.comet.{CometBroadcastExchangeExec, CometCollectLimit import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometShuffleExchangeExec} import org.apache.spark.sql.execution.{ColumnarToRowExec, RowToColumnarExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.QueryStageExec - import org.apache.comet.CometConf +import org.apache.spark.sql.execution.exchange.ReusedExchangeExec // This rule is responsible for eliminating redundant transitions between row-based and // columnar-based operators for Comet. Currently, three potential redundant transitions are: @@ -116,6 +116,7 @@ case class EliminateRedundantTransitions(session: SparkSession) extends Rule[Spa private def hasCometNativeChild(op: SparkPlan): Boolean = { op match { case c: QueryStageExec => hasCometNativeChild(c.plan) + case c: ReusedExchangeExec => hasCometNativeChild(c.child) case _ => op.exists(_.isInstanceOf[CometPlan]) } } From ccd710cb7a50c5cb4a6f4a19b70fdc23c2f89bf5 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 6 Jun 2025 11:18:43 -0600 Subject: [PATCH 26/42] format --- .../org/apache/comet/rules/EliminateRedundantTransitions.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala b/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala index 96b816698f..e15a2fb677 100644 --- a/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala +++ b/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala @@ -25,9 +25,10 @@ import org.apache.spark.sql.comet.{CometBroadcastExchangeExec, CometCollectLimit import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometShuffleExchangeExec} import org.apache.spark.sql.execution.{ColumnarToRowExec, RowToColumnarExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.QueryStageExec -import org.apache.comet.CometConf import org.apache.spark.sql.execution.exchange.ReusedExchangeExec +import org.apache.comet.CometConf + // This rule is responsible for eliminating redundant transitions between row-based and // columnar-based operators for Comet. Currently, three potential redundant transitions are: // 1. `ColumnarToRowExec` on top of an ending `CometCollectLimitExec` operator, which is From 01c43aa35f99a7ae874110da78ebbbbb1a493fda Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 6 Jun 2025 11:56:11 -0600 Subject: [PATCH 27/42] fix? --- .../org/apache/spark/sql/comet/CometColumnarToRowExec.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometColumnarToRowExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometColumnarToRowExec.scala index 0391a1c3b3..95e03ca69c 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometColumnarToRowExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometColumnarToRowExec.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.comet.util.{Utils => CometUtils} import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.{CodegenSupport, ColumnarToRowTransition, SparkPlan, SQLExecution} import org.apache.spark.sql.execution.adaptive.BroadcastQueryStageExec +import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.execution.vectorized.{ConstantColumnVector, WritableColumnVector} import org.apache.spark.sql.types._ @@ -172,6 +173,7 @@ case class CometColumnarToRowExec(child: SparkPlan) op match { case b: CometBroadcastExchangeExec => Some(b) case b: BroadcastQueryStageExec => findCometBroadcastExchange(b.plan) + case b: ReusedExchangeExec => findCometBroadcastExchange(b.child) case _ => op.children.collectFirst(Function.unlift(findCometBroadcastExchange)) } } From 28787ed6a99f975ac5fdde736de58cf5317a565c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 6 Jun 2025 13:17:22 -0600 Subject: [PATCH 28/42] upmerge and remove one change --- .../org/apache/comet/rules/EliminateRedundantTransitions.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala b/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala index e15a2fb677..277bdeae63 100644 --- a/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala +++ b/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala @@ -68,9 +68,6 @@ case class EliminateRedundantTransitions(session: SparkSession) extends Rule[Spa case ColumnarToRowExec(shuffleExchangeExec: CometShuffleExchangeExec) if plan.conf.adaptiveExecutionEnabled => shuffleExchangeExec - case ColumnarToRowExec(broadcastExchangeExec: CometBroadcastExchangeExec) - if plan.conf.adaptiveExecutionEnabled => - broadcastExchangeExec case ColumnarToRowExec(sparkToColumnar: CometSparkToColumnarExec) => if (sparkToColumnar.child.supportsColumnar) { // For Spark Columnar to Comet Columnar, we should keep the ColumnarToRowExec From ab127faeb4789edc2b4841b8f6f0f1977d541cef Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 6 Jun 2025 13:25:56 -0600 Subject: [PATCH 29/42] format --- .../org/apache/comet/rules/EliminateRedundantTransitions.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala b/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala index 277bdeae63..a1a96d321b 100644 --- a/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala +++ b/spark/src/main/scala/org/apache/comet/rules/EliminateRedundantTransitions.scala @@ -21,7 +21,7 @@ package org.apache.comet.rules import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.comet.{CometBroadcastExchangeExec, CometCollectLimitExec, CometColumnarToRowExec, CometPlan, CometSparkToColumnarExec} +import org.apache.spark.sql.comet.{CometCollectLimitExec, CometColumnarToRowExec, CometPlan, CometSparkToColumnarExec} import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, CometShuffleExchangeExec} import org.apache.spark.sql.execution.{ColumnarToRowExec, RowToColumnarExec, SparkPlan} import org.apache.spark.sql.execution.adaptive.QueryStageExec From 940a046a07ca2a8a96c2658c34e67857b9c59237 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 6 Jun 2025 13:36:39 -0600 Subject: [PATCH 30/42] Update 3.5.5 diff --- dev/diffs/3.5.5.diff | 96 +++++++++----------------------------------- 1 file changed, 18 insertions(+), 78 deletions(-) diff --git a/dev/diffs/3.5.5.diff b/dev/diffs/3.5.5.diff index 310aa68815..763f1ac34d 100644 --- a/dev/diffs/3.5.5.diff +++ b/dev/diffs/3.5.5.diff @@ -1181,15 +1181,13 @@ index de24b8c82b0..1f835481290 100644 setupTestData() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala -index 9e9d717db3b..c1a7caf56e0 100644 +index 9e9d717db3b..91a4f9a38d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala -@@ -17,7 +17,8 @@ - +@@ -18,6 +18,7 @@ package org.apache.spark.sql.execution --import org.apache.spark.sql.{DataFrame, QueryTest, Row} -+import org.apache.spark.sql.{DataFrame, QueryTest, Row} + import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.spark.sql.comet.CometProjectExec import org.apache.spark.sql.connector.SimpleWritableDataSource import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} @@ -1206,15 +1204,6 @@ index 9e9d717db3b..c1a7caf56e0 100644 assert(actual == expected) } } -@@ -112,7 +116,7 @@ abstract class RemoveRedundantProjectsSuiteBase - assertProjectExec(query, 1, 3) - } - -- test("join with ordering requirement") { -+ test("join with ordering requirement") { - val query = "select * from (select key, a, c, b from testView) as t1 join " + - "(select key, a, b, c from testView) as t2 on t1.key = t2.key where t2.a > 50" - assertProjectExec(query, 2, 2) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantSortsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantSortsSuite.scala index 005e764cc30..92ec088efab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantSortsSuite.scala @@ -1282,76 +1271,27 @@ index b14f4a405f6..ab7baf434a5 100644 spark.range(1).foreach { _ => columnarToRowExec.canonicalized diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala -index 5a413c77754..a6f97dccb67 100644 +index 5a413c77754..207b66e1d7b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala -@@ -20,6 +20,7 @@ package org.apache.spark.sql.execution - import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} +@@ -17,7 +17,7 @@ + + package org.apache.spark.sql.execution + +-import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} ++import org.apache.spark.sql.{Dataset, IgnoreCometSuite, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeAndComment, CodeGenerator} -+import org.apache.spark.sql.comet.{CometSortExec, CometSortMergeJoinExec} import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite - import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec} - import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec -@@ -235,6 +236,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - assert(twoJoinsDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true - case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true - }.size === 2) - checkAnswer(twoJoinsDF, - Seq(Row(0, 0, 0), Row(1, 1, null), Row(2, 2, 2), Row(3, 3, null), Row(4, 4, null), -@@ -358,6 +360,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "right_outer") - assert(twoJoinsDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: CometSortMergeJoinExec => true - }.size === 2) - checkAnswer(twoJoinsDF, - Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, null, 4), Row(5, null, 5), -@@ -380,8 +383,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val twoJoinsDF = df3.join(df2.hint("SHUFFLE_MERGE"), $"k3" === $"k2", "left_semi") - .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "left_semi") - assert(twoJoinsDF.queryExecution.executedPlan.collect { -- case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | -- WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: SortMergeJoinExec => true - }.size === 2) - checkAnswer(twoJoinsDF, Seq(Row(0), Row(1), Row(2), Row(3))) - } -@@ -402,8 +404,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val twoJoinsDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_anti") - .join(df3.hint("SHUFFLE_MERGE"), $"k1" === $"k3", "left_anti") - assert(twoJoinsDF.queryExecution.executedPlan.collect { -- case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | -- WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: SortMergeJoinExec => true - }.size === 2) - checkAnswer(twoJoinsDF, Seq(Row(6), Row(7), Row(8), Row(9))) - } -@@ -536,7 +537,10 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val plan = df.queryExecution.executedPlan - assert(plan.exists(p => - p.isInstanceOf[WholeStageCodegenExec] && -- p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[SortExec])) -+ p.asInstanceOf[WholeStageCodegenExec].collect { -+ case _: SortExec => true -+ case _: CometSortExec => true -+ }.nonEmpty)) - assert(df.collect() === Array(Row(1), Row(2), Row(3))) - } - -@@ -716,7 +720,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - .write.mode(SaveMode.Overwrite).parquet(path) - - withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "255", -- SQLConf.WHOLESTAGE_SPLIT_CONSUME_FUNC_BY_OPERATOR.key -> "true") { -+ SQLConf.WHOLESTAGE_SPLIT_CONSUME_FUNC_BY_OPERATOR.key -> "true", -+ // Disable Comet native execution because this checks wholestage codegen. -+ "spark.comet.exec.enabled" -> "false") { - val projection = Seq.tabulate(columnNum)(i => s"c$i + c$i as newC$i") - val df = spark.read.parquet(path).selectExpr(projection: _*) +@@ -30,7 +30,7 @@ import org.apache.spark.sql.test.SharedSparkSession + import org.apache.spark.sql.types.{IntegerType, StringType, StructType} + + // Disable AQE because the WholeStageCodegenExec is added when running QueryStageExec +-class WholeStageCodegenSuite extends QueryTest with SharedSparkSession ++class WholeStageCodegenSuite extends QueryTest with SharedSparkSession with IgnoreCometSuite + with DisableAdaptiveExecutionSuite { + import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 2f8e401e743..a4f94417dcc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala From 89ce244a1ecaded21fde753054201b30d8901cbd Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 6 Jun 2025 13:44:56 -0600 Subject: [PATCH 31/42] Update 3.4.3 diff --- dev/diffs/3.4.3.diff | 205 +++---------------------------------------- 1 file changed, 14 insertions(+), 191 deletions(-) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 049671c2ed..50f9da84d1 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -497,57 +497,6 @@ index 2796b1cf154..4816349d690 100644 }.flatten assert(filters.contains(GreaterThan(scan.logicalPlan.output.head, Literal(5L)))) } -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala -new file mode 100644 -index 00000000000..5691536c114 ---- /dev/null -+++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala -@@ -0,0 +1,45 @@ -+/* -+ * 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 -+ -+import org.scalactic.source.Position -+import org.scalatest.Tag -+ -+import org.apache.spark.sql.test.SQLTestUtils -+ -+/** -+ * Tests with this tag will be ignored when Comet is enabled (e.g., via `ENABLE_COMET`). -+ */ -+case class IgnoreComet(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeIcebergCompat(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeDataFusion(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeScan(reason: String) extends Tag("DisableComet") -+ -+/** -+ * Helper trait that disables Comet for all tests regardless of default config values. -+ */ -+trait IgnoreCometSuite extends SQLTestUtils { -+ override protected def test(testName: String, testTags: Tag*)(testFun: => Any) -+ (implicit pos: Position): Unit = { -+ if (isCometEnabled) { -+ ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) -+ } else { -+ super.test(testName, testTags: _*)(testFun) -+ } -+ } -+} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala index fda442eeef0..1b69e4f280e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala @@ -1295,153 +1244,27 @@ index b14f4a405f6..ab7baf434a5 100644 spark.range(1).foreach { _ => columnarToRowExec.canonicalized diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala -index ac710c32296..baae214c6ee 100644 +index ac710c32296..2854b433dd3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala -@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution +@@ -17,7 +17,7 @@ + + package org.apache.spark.sql.execution - import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} +-import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} ++import org.apache.spark.sql.{Dataset, IgnoreCometSuite, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeAndComment, CodeGenerator} -+import org.apache.spark.sql.comet.{CometHashJoinExec, CometSortMergeJoinExec} import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec} - import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec -@@ -169,6 +170,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val oneJoinDF = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2") - assert(oneJoinDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true -+ case _: CometHashJoinExec => true - }.size === 1) - checkAnswer(oneJoinDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, 4))) - -@@ -177,6 +179,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - .join(df3.hint("SHUFFLE_HASH"), $"k1" === $"k3") - assert(twoJoinsDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true -+ case _: CometHashJoinExec => true - }.size === 2) - checkAnswer(twoJoinsDF, - Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, 4, 4))) -@@ -193,6 +196,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - assert(joinUniqueDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true - case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true -+ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true - }.size === 1) - checkAnswer(joinUniqueDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, 4), - Row(null, 5), Row(null, 6), Row(null, 7), Row(null, 8), Row(null, 9))) -@@ -203,6 +208,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - assert(joinNonUniqueDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true - case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true -+ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true - }.size === 1) - checkAnswer(joinNonUniqueDF, Seq(Row(0, 0), Row(0, 3), Row(0, 6), Row(0, 9), Row(1, 1), - Row(1, 4), Row(1, 7), Row(2, 2), Row(2, 5), Row(2, 8), Row(3, null), Row(4, null))) -@@ -213,6 +220,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - assert(joinWithNonEquiDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true - case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true -+ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true - }.size === 1) - checkAnswer(joinWithNonEquiDF, Seq(Row(0, 0), Row(0, 6), Row(0, 9), Row(1, 1), - Row(1, 7), Row(2, 2), Row(2, 8), Row(3, null), Row(4, null), Row(null, 3), Row(null, 4), -@@ -224,6 +233,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - assert(twoJoinsDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true - case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true -+ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true - }.size === 2) - checkAnswer(twoJoinsDF, - Seq(Row(0, 0, 0), Row(1, 1, null), Row(2, 2, 2), Row(3, 3, null), Row(4, 4, null), -@@ -241,6 +252,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val oneLeftOuterJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_outer") - assert(oneLeftOuterJoinDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: CometSortMergeJoinExec => true - }.size === 1) - checkAnswer(oneLeftOuterJoinDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, null), - Row(5, null), Row(6, null), Row(7, null), Row(8, null), Row(9, null))) -@@ -249,6 +261,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val oneRightOuterJoinDF = df2.join(df3.hint("SHUFFLE_MERGE"), $"k2" === $"k3", "right_outer") - assert(oneRightOuterJoinDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: CometSortMergeJoinExec => true - }.size === 1) - checkAnswer(oneRightOuterJoinDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(null, 4), - Row(null, 5))) -@@ -258,6 +271,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "right_outer") - assert(twoJoinsDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: CometSortMergeJoinExec => true - }.size === 2) - checkAnswer(twoJoinsDF, - Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, null, 4), Row(5, null, 5), -@@ -273,6 +287,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val oneJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_semi") - assert(oneJoinDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) => true -+ case _: CometSortMergeJoinExec => true - }.size === 1) - checkAnswer(oneJoinDF, Seq(Row(0), Row(1), Row(2), Row(3))) - -@@ -280,8 +295,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val twoJoinsDF = df3.join(df2.hint("SHUFFLE_MERGE"), $"k3" === $"k2", "left_semi") - .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "left_semi") - assert(twoJoinsDF.queryExecution.executedPlan.collect { -- case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | -- WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: SortMergeJoinExec => true -+ case _: CometSortMergeJoinExec => true - }.size === 2) - checkAnswer(twoJoinsDF, Seq(Row(0), Row(1), Row(2), Row(3))) - } -@@ -295,6 +310,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val oneJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_anti") - assert(oneJoinDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) => true -+ case _: CometSortMergeJoinExec => true - }.size === 1) - checkAnswer(oneJoinDF, Seq(Row(4), Row(5), Row(6), Row(7), Row(8), Row(9))) - -@@ -302,8 +318,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val twoJoinsDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_anti") - .join(df3.hint("SHUFFLE_MERGE"), $"k1" === $"k3", "left_anti") - assert(twoJoinsDF.queryExecution.executedPlan.collect { -- case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | -- WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: SortMergeJoinExec => true -+ case _: CometSortMergeJoinExec => true - }.size === 2) - checkAnswer(twoJoinsDF, Seq(Row(6), Row(7), Row(8), Row(9))) - } -@@ -433,10 +449,6 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - - test("Sort should be included in WholeStageCodegen") { - val df = spark.range(3, 0, -1).toDF().sort(col("id")) -- val plan = df.queryExecution.executedPlan -- assert(plan.exists(p => -- p.isInstanceOf[WholeStageCodegenExec] && -- p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[SortExec])) - assert(df.collect() === Array(Row(1), Row(2), Row(3))) - } - -@@ -616,7 +628,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - .write.mode(SaveMode.Overwrite).parquet(path) - - withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "255", -- SQLConf.WHOLESTAGE_SPLIT_CONSUME_FUNC_BY_OPERATOR.key -> "true") { -+ SQLConf.WHOLESTAGE_SPLIT_CONSUME_FUNC_BY_OPERATOR.key -> "true", -+ // Disable Comet native execution because this checks wholestage codegen. -+ "spark.comet.exec.enabled" -> "false") { - val projection = Seq.tabulate(columnNum)(i => s"c$i + c$i as newC$i") - val df = spark.read.parquet(path).selectExpr(projection: _*) +@@ -29,7 +29,7 @@ import org.apache.spark.sql.test.SharedSparkSession + import org.apache.spark.sql.types.{IntegerType, StringType, StructType} + // Disable AQE because the WholeStageCodegenExec is added when running QueryStageExec +-class WholeStageCodegenSuite extends QueryTest with SharedSparkSession ++class WholeStageCodegenSuite extends QueryTest with SharedSparkSession with IgnoreCometSuite + with DisableAdaptiveExecutionSuite { + + import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 593bd7bb4ba..32af28b0238 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala From e4d57cea56bcba69a7f4680f65c8d15b6200cfac Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 6 Jun 2025 13:47:37 -0600 Subject: [PATCH 32/42] Update 3.5.4 diff --- dev/diffs/3.5.4.diff | 79 +++++++++----------------------------------- 1 file changed, 15 insertions(+), 64 deletions(-) diff --git a/dev/diffs/3.5.4.diff b/dev/diffs/3.5.4.diff index c594fbbe42..33e3d4b057 100644 --- a/dev/diffs/3.5.4.diff +++ b/dev/diffs/3.5.4.diff @@ -1466,76 +1466,27 @@ index b14f4a405f6..ab7baf434a5 100644 spark.range(1).foreach { _ => columnarToRowExec.canonicalized diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala -index 5a413c77754..a6f97dccb67 100644 +index 5a413c77754..207b66e1d7b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala -@@ -20,6 +20,7 @@ package org.apache.spark.sql.execution - import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} +@@ -17,7 +17,7 @@ + + package org.apache.spark.sql.execution + +-import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} ++import org.apache.spark.sql.{Dataset, IgnoreCometSuite, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeAndComment, CodeGenerator} -+import org.apache.spark.sql.comet.{CometSortExec, CometSortMergeJoinExec} import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite - import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec} - import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec -@@ -235,6 +236,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - assert(twoJoinsDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true - case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true - }.size === 2) - checkAnswer(twoJoinsDF, - Seq(Row(0, 0, 0), Row(1, 1, null), Row(2, 2, 2), Row(3, 3, null), Row(4, 4, null), -@@ -358,6 +360,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "right_outer") - assert(twoJoinsDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: CometSortMergeJoinExec => true - }.size === 2) - checkAnswer(twoJoinsDF, - Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, null, 4), Row(5, null, 5), -@@ -380,8 +383,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val twoJoinsDF = df3.join(df2.hint("SHUFFLE_MERGE"), $"k3" === $"k2", "left_semi") - .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "left_semi") - assert(twoJoinsDF.queryExecution.executedPlan.collect { -- case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | -- WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: SortMergeJoinExec => true - }.size === 2) - checkAnswer(twoJoinsDF, Seq(Row(0), Row(1), Row(2), Row(3))) - } -@@ -402,8 +404,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val twoJoinsDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_anti") - .join(df3.hint("SHUFFLE_MERGE"), $"k1" === $"k3", "left_anti") - assert(twoJoinsDF.queryExecution.executedPlan.collect { -- case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | -- WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: SortMergeJoinExec => true - }.size === 2) - checkAnswer(twoJoinsDF, Seq(Row(6), Row(7), Row(8), Row(9))) - } -@@ -536,7 +537,10 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val plan = df.queryExecution.executedPlan - assert(plan.exists(p => - p.isInstanceOf[WholeStageCodegenExec] && -- p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[SortExec])) -+ p.asInstanceOf[WholeStageCodegenExec].collect { -+ case _: SortExec => true -+ case _: CometSortExec => true -+ }.nonEmpty)) - assert(df.collect() === Array(Row(1), Row(2), Row(3))) - } - -@@ -716,7 +720,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - .write.mode(SaveMode.Overwrite).parquet(path) - - withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "255", -- SQLConf.WHOLESTAGE_SPLIT_CONSUME_FUNC_BY_OPERATOR.key -> "true") { -+ SQLConf.WHOLESTAGE_SPLIT_CONSUME_FUNC_BY_OPERATOR.key -> "true", -+ // Disable Comet native execution because this checks wholestage codegen. -+ "spark.comet.exec.enabled" -> "false") { - val projection = Seq.tabulate(columnNum)(i => s"c$i + c$i as newC$i") - val df = spark.read.parquet(path).selectExpr(projection: _*) +@@ -30,7 +30,7 @@ import org.apache.spark.sql.test.SharedSparkSession + import org.apache.spark.sql.types.{IntegerType, StringType, StructType} + + // Disable AQE because the WholeStageCodegenExec is added when running QueryStageExec +-class WholeStageCodegenSuite extends QueryTest with SharedSparkSession ++class WholeStageCodegenSuite extends QueryTest with SharedSparkSession with IgnoreCometSuite + with DisableAdaptiveExecutionSuite { + import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 2f8e401e743..a4f94417dcc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala From 980d17554040030dfd9e65d971d1fc5a1397fb7c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 6 Jun 2025 13:47:59 -0600 Subject: [PATCH 33/42] Update 3.4.3 diff --- dev/diffs/3.4.3.diff | 51 ++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 51 insertions(+) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 50f9da84d1..2087c40e9a 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -497,6 +497,57 @@ index 2796b1cf154..4816349d690 100644 }.flatten assert(filters.contains(GreaterThan(scan.logicalPlan.output.head, Literal(5L)))) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala +new file mode 100644 +index 00000000000..5691536c114 +--- /dev/null ++++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala +@@ -0,0 +1,45 @@ ++/* ++ * 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 ++ ++import org.scalactic.source.Position ++import org.scalatest.Tag ++ ++import org.apache.spark.sql.test.SQLTestUtils ++ ++/** ++ * Tests with this tag will be ignored when Comet is enabled (e.g., via `ENABLE_COMET`). ++ */ ++case class IgnoreComet(reason: String) extends Tag("DisableComet") ++case class IgnoreCometNativeIcebergCompat(reason: String) extends Tag("DisableComet") ++case class IgnoreCometNativeDataFusion(reason: String) extends Tag("DisableComet") ++case class IgnoreCometNativeScan(reason: String) extends Tag("DisableComet") ++ ++/** ++ * Helper trait that disables Comet for all tests regardless of default config values. ++ */ ++trait IgnoreCometSuite extends SQLTestUtils { ++ override protected def test(testName: String, testTags: Tag*)(testFun: => Any) ++ (implicit pos: Position): Unit = { ++ if (isCometEnabled) { ++ ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) ++ } else { ++ super.test(testName, testTags: _*)(testFun) ++ } ++ } ++} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala index fda442eeef0..1b69e4f280e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala From 97a19b928cc57b52c15d9f5987f39fe49c2e9747 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 6 Jun 2025 13:50:57 -0600 Subject: [PATCH 34/42] Update 4.0.0-preview1 diff --- dev/diffs/4.0.0-preview1.diff | 229 +++------------------------------- 1 file changed, 15 insertions(+), 214 deletions(-) diff --git a/dev/diffs/4.0.0-preview1.diff b/dev/diffs/4.0.0-preview1.diff index c014660cf5..716ae50520 100644 --- a/dev/diffs/4.0.0-preview1.diff +++ b/dev/diffs/4.0.0-preview1.diff @@ -1466,226 +1466,27 @@ index 966f4e74712..8017e22d7f8 100644 spark.range(1).foreach { _ => columnarToRowExec.canonicalized diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala -index 3aaf61ffba4..4130ece2283 100644 +index 3aaf61ffba4..b9dd7090799 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala -@@ -22,6 +22,7 @@ import org.apache.spark.rdd.MapPartitionsWithEvaluatorRDD - import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} +@@ -19,7 +19,7 @@ package org.apache.spark.sql.execution + + import org.apache.spark.SparkException + import org.apache.spark.rdd.MapPartitionsWithEvaluatorRDD +-import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} ++import org.apache.spark.sql.{Dataset, IgnoreCometSuite, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeAndComment, CodeGenerator} -+import org.apache.spark.sql.comet.{CometHashJoinExec, CometSortExec, CometSortMergeJoinExec} import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite - import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec} - import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec -@@ -172,6 +173,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val oneJoinDF = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2") - assert(oneJoinDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true -+ case _: CometHashJoinExec => true - }.size === 1) - checkAnswer(oneJoinDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, 4))) - -@@ -180,6 +182,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - .join(df3.hint("SHUFFLE_HASH"), $"k1" === $"k3") - assert(twoJoinsDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true -+ case _: CometHashJoinExec => true - }.size === 2) - checkAnswer(twoJoinsDF, - Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, 4, 4))) -@@ -206,6 +209,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - assert(joinUniqueDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true - case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true -+ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true - }.size === 1) - checkAnswer(joinUniqueDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, 4), - Row(null, 5), Row(null, 6), Row(null, 7), Row(null, 8), Row(null, 9))) -@@ -216,6 +221,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - assert(joinNonUniqueDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true - case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true -+ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true - }.size === 1) - checkAnswer(joinNonUniqueDF, Seq(Row(0, 0), Row(0, 3), Row(0, 6), Row(0, 9), Row(1, 1), - Row(1, 4), Row(1, 7), Row(2, 2), Row(2, 5), Row(2, 8), Row(3, null), Row(4, null))) -@@ -226,6 +233,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - assert(joinWithNonEquiDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true - case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true -+ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true - }.size === 1) - checkAnswer(joinWithNonEquiDF, Seq(Row(0, 0), Row(0, 6), Row(0, 9), Row(1, 1), - Row(1, 7), Row(2, 2), Row(2, 8), Row(3, null), Row(4, null), Row(null, 3), Row(null, 4), -@@ -237,6 +246,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - assert(twoJoinsDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true - case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true -+ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true - }.size === 2) - checkAnswer(twoJoinsDF, - Seq(Row(0, 0, 0), Row(1, 1, null), Row(2, 2, 2), Row(3, 3, null), Row(4, 4, null), -@@ -258,6 +269,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - assert(rightJoinUniqueDf.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_: ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true - case WholeStageCodegenExec(_: SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true -+ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true - }.size === 1) - checkAnswer(rightJoinUniqueDf, Seq(Row(1, 1), Row(2, 2), Row(3, 3), Row(4, 4), - Row(null, 5), Row(null, 6), Row(null, 7), Row(null, 8), Row(null, 9), -@@ -269,6 +282,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - assert(leftJoinUniqueDf.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_: ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true - case WholeStageCodegenExec(_: SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true -+ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true - }.size === 1) - checkAnswer(leftJoinUniqueDf, Seq(Row(0, null), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, 4))) - assert(leftJoinUniqueDf.count() === 5) -@@ -278,6 +293,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - assert(rightJoinNonUniqueDf.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_: ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true - case WholeStageCodegenExec(_: SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true -+ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true - }.size === 1) - checkAnswer(rightJoinNonUniqueDf, Seq(Row(0, 3), Row(0, 6), Row(0, 9), Row(1, 1), - Row(1, 4), Row(1, 7), Row(1, 10), Row(2, 2), Row(2, 5), Row(2, 8))) -@@ -287,6 +304,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - assert(leftJoinNonUniqueDf.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_: ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true - case WholeStageCodegenExec(_: SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true -+ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true - }.size === 1) - checkAnswer(leftJoinNonUniqueDf, Seq(Row(0, 3), Row(0, 6), Row(0, 9), Row(1, 1), - Row(1, 4), Row(1, 7), Row(1, 10), Row(2, 2), Row(2, 5), Row(2, 8), Row(3, null), -@@ -298,6 +317,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - assert(rightJoinWithNonEquiDf.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_: ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true - case WholeStageCodegenExec(_: SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true -+ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true - }.size === 1) - checkAnswer(rightJoinWithNonEquiDf, Seq(Row(0, 6), Row(0, 9), Row(1, 1), Row(1, 7), - Row(1, 10), Row(2, 2), Row(2, 8), Row(null, 3), Row(null, 4), Row(null, 5))) -@@ -308,6 +329,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - assert(leftJoinWithNonEquiDf.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_: ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true - case WholeStageCodegenExec(_: SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true -+ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true - }.size === 1) - checkAnswer(leftJoinWithNonEquiDf, Seq(Row(0, 6), Row(0, 9), Row(1, 1), Row(1, 7), - Row(1, 10), Row(2, 2), Row(2, 8), Row(3, null), Row(4, null))) -@@ -318,6 +341,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - assert(twoRightJoinsDf.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_: ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true - case WholeStageCodegenExec(_: SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true -+ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true - }.size === 2) - checkAnswer(twoRightJoinsDf, Seq(Row(2, 2, 2), Row(3, 3, 3), Row(4, 4, 4))) - -@@ -327,6 +352,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - assert(twoLeftJoinsDf.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_: ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true - case WholeStageCodegenExec(_: SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case _: CometHashJoinExec if hint == "SHUFFLE_HASH" => true -+ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true - }.size === 2) - checkAnswer(twoLeftJoinsDf, - Seq(Row(0, null, null), Row(1, 1, null), Row(2, 2, 2), Row(3, 3, 3), Row(4, 4, 4))) -@@ -343,6 +370,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val oneLeftOuterJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_outer") - assert(oneLeftOuterJoinDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: CometSortMergeJoinExec => true - }.size === 1) - checkAnswer(oneLeftOuterJoinDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, null), - Row(5, null), Row(6, null), Row(7, null), Row(8, null), Row(9, null))) -@@ -351,6 +379,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val oneRightOuterJoinDF = df2.join(df3.hint("SHUFFLE_MERGE"), $"k2" === $"k3", "right_outer") - assert(oneRightOuterJoinDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: CometSortMergeJoinExec => true - }.size === 1) - checkAnswer(oneRightOuterJoinDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(null, 4), - Row(null, 5))) -@@ -360,6 +389,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "right_outer") - assert(twoJoinsDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: CometSortMergeJoinExec => true - }.size === 2) - checkAnswer(twoJoinsDF, - Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, null, 4), Row(5, null, 5), -@@ -375,6 +405,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val oneJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_semi") - assert(oneJoinDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) => true -+ case _: CometSortMergeJoinExec => true - }.size === 1) - checkAnswer(oneJoinDF, Seq(Row(0), Row(1), Row(2), Row(3))) - -@@ -382,8 +413,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val twoJoinsDF = df3.join(df2.hint("SHUFFLE_MERGE"), $"k3" === $"k2", "left_semi") - .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "left_semi") - assert(twoJoinsDF.queryExecution.executedPlan.collect { -- case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | -- WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: SortMergeJoinExec => true -+ case _: CometSortMergeJoinExec => true - }.size === 2) - checkAnswer(twoJoinsDF, Seq(Row(0), Row(1), Row(2), Row(3))) - } -@@ -397,6 +428,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val oneJoinDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_anti") - assert(oneJoinDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) => true -+ case _: CometSortMergeJoinExec => true - }.size === 1) - checkAnswer(oneJoinDF, Seq(Row(4), Row(5), Row(6), Row(7), Row(8), Row(9))) - -@@ -404,8 +436,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val twoJoinsDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_anti") - .join(df3.hint("SHUFFLE_MERGE"), $"k1" === $"k3", "left_anti") - assert(twoJoinsDF.queryExecution.executedPlan.collect { -- case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | -- WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: SortMergeJoinExec => true -+ case _: CometSortMergeJoinExec => true - }.size === 2) - checkAnswer(twoJoinsDF, Seq(Row(6), Row(7), Row(8), Row(9))) - } -@@ -538,7 +570,10 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val plan = df.queryExecution.executedPlan - assert(plan.exists(p => - p.isInstanceOf[WholeStageCodegenExec] && -- p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[SortExec])) -+ p.asInstanceOf[WholeStageCodegenExec].collect { -+ case _: SortExec => true -+ case _: CometSortExec => true -+ }.nonEmpty)) - assert(df.collect() === Array(Row(1), Row(2), Row(3))) - } - -@@ -718,7 +753,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - .write.mode(SaveMode.Overwrite).parquet(path) - - withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "255", -- SQLConf.WHOLESTAGE_SPLIT_CONSUME_FUNC_BY_OPERATOR.key -> "true") { -+ SQLConf.WHOLESTAGE_SPLIT_CONSUME_FUNC_BY_OPERATOR.key -> "true", -+ // Disable Comet native execution because this checks wholestage codegen. -+ "spark.comet.exec.enabled" -> "false") { - val projection = Seq.tabulate(columnNum)(i => s"c$i + c$i as newC$i") - val df = spark.read.parquet(path).selectExpr(projection: _*) +@@ -32,7 +32,7 @@ import org.apache.spark.sql.test.SharedSparkSession + import org.apache.spark.sql.types.{IntegerType, StringType, StructType} + + // Disable AQE because the WholeStageCodegenExec is added when running QueryStageExec +-class WholeStageCodegenSuite extends QueryTest with SharedSparkSession ++class WholeStageCodegenSuite extends QueryTest with SharedSparkSession with IgnoreCometSuite + with DisableAdaptiveExecutionSuite { + import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index a7efd0aa75e..baae0967a2a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala From 99f6cb9ec0ac63089da8a920e56fb2a14610c0cd Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 6 Jun 2025 16:24:40 -0600 Subject: [PATCH 35/42] ignore wholestagecodegen tests --- dev/diffs/3.5.5.diff | 131 ++++--------------------------------------- 1 file changed, 11 insertions(+), 120 deletions(-) diff --git a/dev/diffs/3.5.5.diff b/dev/diffs/3.5.5.diff index 5090ec90c6..1fa55686d9 100644 --- a/dev/diffs/3.5.5.diff +++ b/dev/diffs/3.5.5.diff @@ -1382,136 +1382,27 @@ index b14f4a405f6..ab7baf434a5 100644 spark.range(1).foreach { _ => columnarToRowExec.canonicalized diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala -index 5a413c77754..be3a38e6c4a 100644 +index 5a413c77754..207b66e1d7b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala -@@ -17,9 +17,10 @@ +@@ -17,7 +17,7 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} -+import org.apache.spark.sql.{Dataset, IgnoreComet, QueryTest, Row, SaveMode} ++import org.apache.spark.sql.{Dataset, IgnoreCometSuite, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeAndComment, CodeGenerator} -+import org.apache.spark.sql.comet.{CometSortExec, CometSortMergeJoinExec} import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite - import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec} - import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec -@@ -161,7 +162,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - assert(df.collect() === Array(Row(1, 1, "1"), Row(1, 1, "1"), Row(2, 2, "2"))) - } - -- test("Inner ShuffledHashJoin should be included in WholeStageCodegen") { -+ test("Inner ShuffledHashJoin should be included in WholeStageCodegen", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1852")) { - val df1 = spark.range(5).select($"id".as("k1")) - val df2 = spark.range(15).select($"id".as("k2")) - val df3 = spark.range(6).select($"id".as("k3")) -@@ -193,7 +195,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - } - } - -- test("Full Outer ShuffledHashJoin and SortMergeJoin should be included in WholeStageCodegen") { -+ test("Full Outer ShuffledHashJoin and SortMergeJoin should be included in WholeStageCodegen", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1852")) { - val df1 = spark.range(5).select($"id".as("k1")) - val df2 = spark.range(10).select($"id".as("k2")) - val df3 = spark.range(3).select($"id".as("k3")) -@@ -235,6 +238,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - assert(twoJoinsDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true - case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true -+ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true - }.size === 2) - checkAnswer(twoJoinsDF, - Seq(Row(0, 0, 0), Row(1, 1, null), Row(2, 2, 2), Row(3, 3, null), Row(4, 4, null), -@@ -244,7 +248,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - } - - -- test("SPARK-44060 Code-gen for build side outer shuffled hash join") { -+ test("SPARK-44060 Code-gen for build side outer shuffled hash join", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1852")) { - val df1 = spark.range(0, 5).select($"id".as("k1")) - val df2 = spark.range(1, 11).select($"id".as("k2")) - val df3 = spark.range(2, 5).select($"id".as("k3")) -@@ -332,7 +337,8 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - } - } - -- test("Left/Right Outer SortMergeJoin should be included in WholeStageCodegen") { -+ test("Left/Right Outer SortMergeJoin should be included in WholeStageCodegen", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1852")) { - val df1 = spark.range(10).select($"id".as("k1")) - val df2 = spark.range(4).select($"id".as("k2")) - val df3 = spark.range(6).select($"id".as("k3")) -@@ -358,13 +364,15 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "right_outer") - assert(twoJoinsDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: CometSortMergeJoinExec => true - }.size === 2) - checkAnswer(twoJoinsDF, - Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, null, 4), Row(5, null, 5), - Row(null, null, 6), Row(null, null, 7), Row(null, null, 8), Row(null, null, 9))) - } - -- test("Left Semi SortMergeJoin should be included in WholeStageCodegen") { -+ test("Left Semi SortMergeJoin should be included in WholeStageCodegen", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1852")) { - val df1 = spark.range(10).select($"id".as("k1")) - val df2 = spark.range(4).select($"id".as("k2")) - val df3 = spark.range(6).select($"id".as("k3")) -@@ -380,13 +388,13 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val twoJoinsDF = df3.join(df2.hint("SHUFFLE_MERGE"), $"k3" === $"k2", "left_semi") - .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "left_semi") - assert(twoJoinsDF.queryExecution.executedPlan.collect { -- case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | -- WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: SortMergeJoinExec => true - }.size === 2) - checkAnswer(twoJoinsDF, Seq(Row(0), Row(1), Row(2), Row(3))) - } - -- test("Left Anti SortMergeJoin should be included in WholeStageCodegen") { -+ test("Left Anti SortMergeJoin should be included in WholeStageCodegen", -+ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1852")) { - val df1 = spark.range(10).select($"id".as("k1")) - val df2 = spark.range(4).select($"id".as("k2")) - val df3 = spark.range(6).select($"id".as("k3")) -@@ -402,8 +410,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val twoJoinsDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_anti") - .join(df3.hint("SHUFFLE_MERGE"), $"k1" === $"k3", "left_anti") - assert(twoJoinsDF.queryExecution.executedPlan.collect { -- case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | -- WholeStageCodegenExec(_ : SortMergeJoinExec) => true -+ case _: SortMergeJoinExec => true - }.size === 2) - checkAnswer(twoJoinsDF, Seq(Row(6), Row(7), Row(8), Row(9))) - } -@@ -536,7 +543,10 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - val plan = df.queryExecution.executedPlan - assert(plan.exists(p => - p.isInstanceOf[WholeStageCodegenExec] && -- p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[SortExec])) -+ p.asInstanceOf[WholeStageCodegenExec].collect { -+ case _: SortExec => true -+ case _: CometSortExec => true -+ }.nonEmpty)) - assert(df.collect() === Array(Row(1), Row(2), Row(3))) - } - -@@ -716,7 +726,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession - .write.mode(SaveMode.Overwrite).parquet(path) - - withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "255", -- SQLConf.WHOLESTAGE_SPLIT_CONSUME_FUNC_BY_OPERATOR.key -> "true") { -+ SQLConf.WHOLESTAGE_SPLIT_CONSUME_FUNC_BY_OPERATOR.key -> "true", -+ // Disable Comet native execution because this checks wholestage codegen. -+ "spark.comet.exec.enabled" -> "false") { - val projection = Seq.tabulate(columnNum)(i => s"c$i + c$i as newC$i") - val df = spark.read.parquet(path).selectExpr(projection: _*) +@@ -30,7 +30,7 @@ import org.apache.spark.sql.test.SharedSparkSession + import org.apache.spark.sql.types.{IntegerType, StringType, StructType} + // Disable AQE because the WholeStageCodegenExec is added when running QueryStageExec +-class WholeStageCodegenSuite extends QueryTest with SharedSparkSession ++class WholeStageCodegenSuite extends QueryTest with SharedSparkSession with IgnoreCometSuite + with DisableAdaptiveExecutionSuite { + + import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 2f8e401e743..a4f94417dcc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala From fe8b887d2dd977e79122135a9c81f8d23fb21198 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 6 Jun 2025 16:30:52 -0600 Subject: [PATCH 36/42] enable tests for 3.4.3 --- .github/workflows/spark_sql_test.yml | 2 +- dev/diffs/3.4.3.diff | 101 ++++++++++++--------------- 2 files changed, 47 insertions(+), 56 deletions(-) diff --git a/.github/workflows/spark_sql_test.yml b/.github/workflows/spark_sql_test.yml index 9598a5accc..1fc8c8c606 100644 --- a/.github/workflows/spark_sql_test.yml +++ b/.github/workflows/spark_sql_test.yml @@ -46,7 +46,7 @@ jobs: os: [ubuntu-24.04] java-version: [11] # TODO enbable other versions once tests are passing with 3.5.5 - spark-version: [{short: '3.5', full: '3.5.5'}] + spark-version: [{short: '3.4', full: '3.4.3'}, {short: '3.5', full: '3.5.5'}] module: - {name: "catalyst", args1: "catalyst/test", args2: ""} - {name: "sql/core-1", args1: "", args2: sql/testOnly * -- -l org.apache.spark.tags.ExtendedSQLTest -l org.apache.spark.tags.SlowSQLTest} diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 18e91d9da6..51401a739a 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -386,7 +386,7 @@ index daef11ae4d6..9f3cc9181f2 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index f33432ddb6f..cc5224af735 100644 +index f33432ddb6f..1925aac8d97 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen @@ -417,17 +417,37 @@ index f33432ddb6f..cc5224af735 100644 Given("disable broadcast pruning and disable subquery duplication") withSQLConf( SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", -@@ -1215,7 +1220,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1027,7 +1032,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("avoid reordering broadcast join keys to match input hash partitioning") { ++ test("avoid reordering broadcast join keys to match input hash partitioning", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1839")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + withTable("large", "dimTwo", "dimThree") { +@@ -1215,7 +1221,8 @@ abstract class DynamicPartitionPruningSuiteBase } test("SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + - "canonicalization and exchange reuse") { + "canonicalization and exchange reuse", -+ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #1737")) { ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1839")) { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val df = sql( -@@ -1729,6 +1735,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1423,7 +1430,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("SPARK-34637: DPP side broadcast query stage is created firstly") { ++ test("SPARK-34637: DPP side broadcast query stage is created firstly", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1839")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + val df = sql( + """ WITH v as ( +@@ -1729,6 +1737,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat case s: BatchScanExec => // we use f1 col for v2 tables due to schema pruning s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) @@ -507,57 +527,6 @@ index 2796b1cf154..4816349d690 100644 }.flatten assert(filters.contains(GreaterThan(scan.logicalPlan.output.head, Literal(5L)))) } -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala -new file mode 100644 -index 00000000000..5691536c114 ---- /dev/null -+++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala -@@ -0,0 +1,45 @@ -+/* -+ * 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 -+ -+import org.scalactic.source.Position -+import org.scalatest.Tag -+ -+import org.apache.spark.sql.test.SQLTestUtils -+ -+/** -+ * Tests with this tag will be ignored when Comet is enabled (e.g., via `ENABLE_COMET`). -+ */ -+case class IgnoreComet(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeIcebergCompat(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeDataFusion(reason: String) extends Tag("DisableComet") -+case class IgnoreCometNativeScan(reason: String) extends Tag("DisableComet") -+ -+/** -+ * Helper trait that disables Comet for all tests regardless of default config values. -+ */ -+trait IgnoreCometSuite extends SQLTestUtils { -+ override protected def test(testName: String, testTags: Tag*)(testFun: => Any) -+ (implicit pos: Position): Unit = { -+ if (isCometEnabled) { -+ ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) -+ } else { -+ super.test(testName, testTags: _*)(testFun) -+ } -+ } -+} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala index fda442eeef0..1b69e4f280e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala @@ -1280,6 +1249,28 @@ index 47679ed7865..9ffbaecb98e 100644 }.length == hashAggCount) assert(collectWithSubqueries(plan) { case s: SortAggregateExec => s }.length == sortAggCount) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala +index eec396b2e39..bf3f1c769d6 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala +@@ -18,7 +18,7 @@ + package org.apache.spark.sql.execution + + import org.apache.spark.TestUtils.assertSpilled +-import org.apache.spark.sql.{AnalysisException, QueryTest, Row} ++import org.apache.spark.sql.{AnalysisException, IgnoreComet, QueryTest, Row} + import org.apache.spark.sql.internal.SQLConf.{WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD, WINDOW_EXEC_BUFFER_SPILL_THRESHOLD} + import org.apache.spark.sql.test.SharedSparkSession + +@@ -470,7 +470,7 @@ class SQLWindowFunctionSuite extends QueryTest with SharedSparkSession { + Row(1, 3, null) :: Row(2, null, 4) :: Nil) + } + +- test("test with low buffer spill threshold") { ++ test("test with low buffer spill threshold", IgnoreComet("Comet does not support spilling")) { + val nums = sparkContext.parallelize(1 to 10).map(x => (x, x % 2)).toDF("x", "y") + nums.createOrReplaceTempView("nums") + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala index b14f4a405f6..ab7baf434a5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala From eaf276879c96d1eb4f278d00136f28b22c489ebb Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 6 Jun 2025 16:33:53 -0600 Subject: [PATCH 37/42] enable tests for 3.4.3 --- dev/diffs/3.4.3.diff | 57 +++++++++++++++++++++++++++++++++++++------- 1 file changed, 48 insertions(+), 9 deletions(-) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 51401a739a..35e4732d9f 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -363,6 +363,44 @@ index a9f69ab28a1..5d9d4f2cb83 100644 withTable("tbl") { sql( """ +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +index 433b4741979..07148eee480 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +@@ -23,8 +23,9 @@ import org.apache.spark.TestUtils.{assertNotSpilled, assertSpilled} + import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, Lag, Literal, NonFoldableLiteral} + import org.apache.spark.sql.catalyst.optimizer.TransposeWindow + import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning ++import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +-import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, Exchange, ShuffleExchangeExec} ++import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, Exchange, ShuffleExchangeExec, ShuffleExchangeLike} + import org.apache.spark.sql.execution.window.WindowExec + import org.apache.spark.sql.expressions.{Aggregator, MutableAggregationBuffer, UserDefinedAggregateFunction, Window} + import org.apache.spark.sql.functions._ +@@ -1186,10 +1187,12 @@ class DataFrameWindowFunctionsSuite extends QueryTest + } + + def isShuffleExecByRequirement( +- plan: ShuffleExchangeExec, ++ plan: ShuffleExchangeLike, + desiredClusterColumns: Seq[String]): Boolean = plan match { + case ShuffleExchangeExec(op: HashPartitioning, _, ENSURE_REQUIREMENTS) => + partitionExpressionsColumns(op.expressions) === desiredClusterColumns ++ case CometShuffleExchangeExec(op: HashPartitioning, _, _, ENSURE_REQUIREMENTS, _, _) => ++ partitionExpressionsColumns(op.expressions) === desiredClusterColumns + case _ => false + } + +@@ -1212,7 +1215,7 @@ class DataFrameWindowFunctionsSuite extends QueryTest + val shuffleByRequirement = windowed.queryExecution.executedPlan.exists { + case w: WindowExec => + w.child.exists { +- case s: ShuffleExchangeExec => isShuffleExecByRequirement(s, Seq("key1", "key2")) ++ case s: ShuffleExchangeLike => isShuffleExecByRequirement(s, Seq("key1", "key2")) + case _ => false + } + case _ => false diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index daef11ae4d6..9f3cc9181f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -580,7 +618,7 @@ index 1792b4c32eb..1616e6f39bd 100644 assert(shuffleMergeJoins.size == 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala -index 7f062bfb899..b347ef905d2 100644 +index 7f062bfb899..0ed85486e80 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 @@ -30,7 +30,8 @@ import org.apache.spark.sql.catalyst.TableIdentifier @@ -676,7 +714,7 @@ index 7f062bfb899..b347ef905d2 100644 // Same result between shuffled hash join and sort merge join checkAnswer(shjDF, smjResult) } -@@ -1282,18 +1292,25 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1282,18 +1292,26 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan } // Test shuffled hash join @@ -691,6 +729,7 @@ index 7f062bfb899..b347ef905d2 100644 + true + case WholeStageCodegenExec(ColumnarToRowExec( + InputAdapter(CometProjectExec(_, _, _, _, _: CometHashJoinExec, _)))) => true ++ case _: CometHashJoinExec => true }.size === 1) checkAnswer(shjCodegenDF, Seq.empty) @@ -704,7 +743,7 @@ index 7f062bfb899..b347ef905d2 100644 checkAnswer(shjNonCodegenDF, Seq.empty) } } -@@ -1341,7 +1358,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1341,7 +1359,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 @@ -714,7 +753,7 @@ index 7f062bfb899..b347ef905d2 100644 } def getJoinQuery(selectExpr: String, joinType: String): String = { -@@ -1370,9 +1388,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1370,9 +1389,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) @@ -729,7 +768,7 @@ index 7f062bfb899..b347ef905d2 100644 } // Test output ordering is not preserved -@@ -1381,9 +1402,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1381,9 +1403,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) @@ -744,7 +783,7 @@ index 7f062bfb899..b347ef905d2 100644 } // Test singe partition -@@ -1393,7 +1417,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1393,7 +1418,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 @@ -754,7 +793,7 @@ index 7f062bfb899..b347ef905d2 100644 checkAnswer(fullJoinDF, Row(100)) } } -@@ -1438,6 +1463,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1438,6 +1464,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 @@ -764,7 +803,7 @@ index 7f062bfb899..b347ef905d2 100644 }.size == 1) } } -@@ -1482,14 +1510,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1482,14 +1511,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 = { @@ -787,7 +826,7 @@ index 7f062bfb899..b347ef905d2 100644 } dupStreamSideColTest("SHUFFLE_HASH", check) } -@@ -1605,7 +1639,8 @@ class ThreadLeakInSortMergeJoinSuite +@@ -1605,7 +1640,8 @@ class ThreadLeakInSortMergeJoinSuite sparkConf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, 20)) } From 3c2d06bf45a5fdbd7415b06ed49ad42055fa1f3c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 6 Jun 2025 16:50:05 -0600 Subject: [PATCH 38/42] 3.5.4 --- .github/workflows/spark_sql_test.yml | 2 +- dev/diffs/3.5.4.diff | 101 +++++++++++++++++++++++---- 2 files changed, 90 insertions(+), 13 deletions(-) diff --git a/.github/workflows/spark_sql_test.yml b/.github/workflows/spark_sql_test.yml index 1fc8c8c606..4b762476bf 100644 --- a/.github/workflows/spark_sql_test.yml +++ b/.github/workflows/spark_sql_test.yml @@ -46,7 +46,7 @@ jobs: os: [ubuntu-24.04] java-version: [11] # TODO enbable other versions once tests are passing with 3.5.5 - spark-version: [{short: '3.4', full: '3.4.3'}, {short: '3.5', full: '3.5.5'}] + spark-version: [{short: '3.4', full: '3.4.3'}, {short: '3.5', full: '3.5.4'}, {short: '3.5', full: '3.5.5'}] module: - {name: "catalyst", args1: "catalyst/test", args2: ""} - {name: "sql/core-1", args1: "", args2: sql/testOnly * -- -l org.apache.spark.tags.ExtendedSQLTest -l org.apache.spark.tags.SlowSQLTest} diff --git a/dev/diffs/3.5.4.diff b/dev/diffs/3.5.4.diff index 32d4d617e8..4b8d5c2d0a 100644 --- a/dev/diffs/3.5.4.diff +++ b/dev/diffs/3.5.4.diff @@ -342,6 +342,50 @@ index 7ee18df3756..64f01a68048 100644 withTable("tbl") { sql( """ +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +index 47a311c71d5..909a12bd830 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +@@ -18,14 +18,14 @@ + package org.apache.spark.sql + + import org.scalatest.matchers.must.Matchers.the +- + import org.apache.spark.TestUtils.{assertNotSpilled, assertSpilled} + import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, Lag, Literal, NonFoldableLiteral} + import org.apache.spark.sql.catalyst.optimizer.TransposeWindow + import org.apache.spark.sql.catalyst.plans.logical.{Window => LogicalWindow} + import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning ++import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +-import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, Exchange, ShuffleExchangeExec} ++import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, Exchange, ShuffleExchangeExec, ShuffleExchangeLike} + import org.apache.spark.sql.execution.window.WindowExec + import org.apache.spark.sql.expressions.{Aggregator, MutableAggregationBuffer, UserDefinedAggregateFunction, Window} + import org.apache.spark.sql.functions._ +@@ -1187,10 +1187,12 @@ class DataFrameWindowFunctionsSuite extends QueryTest + } + + def isShuffleExecByRequirement( +- plan: ShuffleExchangeExec, ++ plan: ShuffleExchangeLike, + desiredClusterColumns: Seq[String]): Boolean = plan match { + case ShuffleExchangeExec(op: HashPartitioning, _, ENSURE_REQUIREMENTS, _) => + partitionExpressionsColumns(op.expressions) === desiredClusterColumns ++ case CometShuffleExchangeExec(op: HashPartitioning, _, _, ENSURE_REQUIREMENTS, _, _) => ++ partitionExpressionsColumns(op.expressions) === desiredClusterColumns + case _ => false + } + +@@ -1213,7 +1215,7 @@ class DataFrameWindowFunctionsSuite extends QueryTest + val shuffleByRequirement = windowed.queryExecution.executedPlan.exists { + case w: WindowExec => + w.child.exists { +- case s: ShuffleExchangeExec => isShuffleExecByRequirement(s, Seq("key1", "key2")) ++ case s: ShuffleExchangeLike => isShuffleExecByRequirement(s, Seq("key1", "key2")) + case _ => false + } + case _ => false diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index f32b32ffc5a..447d7c6416e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -365,7 +409,7 @@ index f32b32ffc5a..447d7c6416e 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index f33432ddb6f..19ce507e82b 100644 +index f33432ddb6f..0fa49fb3f0b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen @@ -422,7 +466,7 @@ index f33432ddb6f..19ce507e82b 100644 - test("avoid reordering broadcast join keys to match input hash partitioning") { + test("avoid reordering broadcast join keys to match input hash partitioning", -+ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1839")) { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { withTable("large", "dimTwo", "dimThree") { @@ -442,7 +486,7 @@ index f33432ddb6f..19ce507e82b 100644 test("SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + - "canonicalization and exchange reuse") { + "canonicalization and exchange reuse", -+ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1839")) { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val df = sql( @@ -482,7 +526,7 @@ index f33432ddb6f..19ce507e82b 100644 - test("SPARK-34637: DPP side broadcast query stage is created firstly") { + test("SPARK-34637: DPP side broadcast query stage is created firstly", -+ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1839")) { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ WITH v as ( @@ -746,7 +790,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 4d256154c85..43f0bebb00c 100644 +index 4d256154c85..66a5473852d 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 @@ -872,7 +916,7 @@ index 4d256154c85..43f0bebb00c 100644 }.size === 1) // Same result between shuffled hash join and sort merge join checkAnswer(shjDF, smjResult) -@@ -1432,13 +1446,19 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1432,13 +1446,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan assert(shjCodegenDF.queryExecution.executedPlan.collect { case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true case WholeStageCodegenExec(ProjectExec(_, _ : ShuffledHashJoinExec)) => true @@ -880,6 +924,7 @@ index 4d256154c85..43f0bebb00c 100644 + true + case WholeStageCodegenExec(ColumnarToRowExec( + InputAdapter(CometProjectExec(_, _, _, _, _: CometHashJoinExec, _)))) => true ++ case _: CometHashJoinExec => true }.size === 1) checkAnswer(shjCodegenDF, Seq.empty) @@ -893,7 +938,7 @@ index 4d256154c85..43f0bebb00c 100644 checkAnswer(shjNonCodegenDF, Seq.empty) } } -@@ -1486,7 +1506,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1486,7 +1507,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val plan = sql(getAggQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) // Have shuffle before aggregation @@ -903,7 +948,7 @@ index 4d256154c85..43f0bebb00c 100644 } def getJoinQuery(selectExpr: String, joinType: String): String = { -@@ -1515,9 +1536,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1515,9 +1537,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan } val plan = sql(getJoinQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) @@ -918,7 +963,7 @@ index 4d256154c85..43f0bebb00c 100644 } // Test output ordering is not preserved -@@ -1526,9 +1550,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1526,9 +1551,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val selectExpr = "/*+ BROADCAST(left_t) */ k1 as k0" val plan = sql(getJoinQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) @@ -933,7 +978,7 @@ index 4d256154c85..43f0bebb00c 100644 } // Test singe partition -@@ -1538,7 +1565,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1538,7 +1566,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan |FROM range(0, 10, 1, 1) t1 FULL OUTER JOIN range(0, 10, 1, 1) t2 |""".stripMargin) val plan = fullJoinDF.queryExecution.executedPlan @@ -943,7 +988,7 @@ index 4d256154c85..43f0bebb00c 100644 checkAnswer(fullJoinDF, Row(100)) } } -@@ -1583,6 +1611,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1583,6 +1612,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 @@ -953,7 +998,7 @@ index 4d256154c85..43f0bebb00c 100644 }.size == 1) } } -@@ -1627,14 +1658,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1627,14 +1659,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 = { @@ -976,6 +1021,16 @@ index 4d256154c85..43f0bebb00c 100644 } dupStreamSideColTest("SHUFFLE_HASH", check) } +@@ -1770,7 +1808,8 @@ class ThreadLeakInSortMergeJoinSuite + sparkConf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, 20)) + } + +- test("SPARK-47146: thread leak when doing SortMergeJoin (with spill)") { ++ test("SPARK-47146: thread leak when doing SortMergeJoin (with spill)", ++ IgnoreComet("Comet does not support spilling")) { + + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala index c26757c9cff..d55775f09d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala @@ -1451,6 +1506,28 @@ index 47679ed7865..9ffbaecb98e 100644 }.length == hashAggCount) assert(collectWithSubqueries(plan) { case s: SortAggregateExec => s }.length == sortAggCount) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala +index eec396b2e39..bf3f1c769d6 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala +@@ -18,7 +18,7 @@ + package org.apache.spark.sql.execution + + import org.apache.spark.TestUtils.assertSpilled +-import org.apache.spark.sql.{AnalysisException, QueryTest, Row} ++import org.apache.spark.sql.{AnalysisException, IgnoreComet, QueryTest, Row} + import org.apache.spark.sql.internal.SQLConf.{WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD, WINDOW_EXEC_BUFFER_SPILL_THRESHOLD} + import org.apache.spark.sql.test.SharedSparkSession + +@@ -470,7 +470,7 @@ class SQLWindowFunctionSuite extends QueryTest with SharedSparkSession { + Row(1, 3, null) :: Row(2, null, 4) :: Nil) + } + +- test("test with low buffer spill threshold") { ++ test("test with low buffer spill threshold", IgnoreComet("Comet does not support spilling")) { + val nums = sparkContext.parallelize(1 to 10).map(x => (x, x % 2)).toDF("x", "y") + nums.createOrReplaceTempView("nums") + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala index b14f4a405f6..ab7baf434a5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala From 30c962c3e13c8c78d385dc137fe28b8a51d3b4d2 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 6 Jun 2025 16:55:21 -0600 Subject: [PATCH 39/42] 4.0.0 --- dev/diffs/4.0.0-preview1.diff | 109 +++++++++++++++++++++++++++++----- 1 file changed, 95 insertions(+), 14 deletions(-) diff --git a/dev/diffs/4.0.0-preview1.diff b/dev/diffs/4.0.0-preview1.diff index 2fec4297d8..3fb9b2c025 100644 --- a/dev/diffs/4.0.0-preview1.diff +++ b/dev/diffs/4.0.0-preview1.diff @@ -374,6 +374,44 @@ index 760ee802608..b77133ffd37 100644 } assert(exchanges.size == 2) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +index e3aff9b36ae..06196517935 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +@@ -24,8 +24,9 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression + import org.apache.spark.sql.catalyst.optimizer.TransposeWindow + import org.apache.spark.sql.catalyst.plans.logical.{Window => LogicalWindow} + import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning ++import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +-import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, Exchange, ShuffleExchangeExec} ++import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, Exchange, ShuffleExchangeExec, ShuffleExchangeLike} + import org.apache.spark.sql.execution.window.WindowExec + import org.apache.spark.sql.expressions.{Aggregator, MutableAggregationBuffer, UserDefinedAggregateFunction, Window} + import org.apache.spark.sql.functions._ +@@ -1142,10 +1143,12 @@ class DataFrameWindowFunctionsSuite extends QueryTest + } + + def isShuffleExecByRequirement( +- plan: ShuffleExchangeExec, ++ plan: ShuffleExchangeLike, + desiredClusterColumns: Seq[String]): Boolean = plan match { + case ShuffleExchangeExec(op: HashPartitioning, _, ENSURE_REQUIREMENTS, _) => + partitionExpressionsColumns(op.expressions) === desiredClusterColumns ++ case CometShuffleExchangeExec(op: HashPartitioning, _, _, ENSURE_REQUIREMENTS, _, _) => ++ partitionExpressionsColumns(op.expressions) === desiredClusterColumns + case _ => false + } + +@@ -1168,7 +1171,7 @@ class DataFrameWindowFunctionsSuite extends QueryTest + val shuffleByRequirement = windowed.queryExecution.executedPlan.exists { + case w: WindowExec => + w.child.exists { +- case s: ShuffleExchangeExec => isShuffleExecByRequirement(s, Seq("key1", "key2")) ++ case s: ShuffleExchangeLike => isShuffleExecByRequirement(s, Seq("key1", "key2")) + case _ => false + } + case _ => false diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 16a493b5290..3f0b70e2d59 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -397,7 +435,7 @@ index 16a493b5290..3f0b70e2d59 100644 assert(exchanges.size == 2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala -index 2c24cc7d570..3e6a8632fa6 100644 +index 2c24cc7d570..21d36ebc6f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen @@ -428,17 +466,37 @@ index 2c24cc7d570..3e6a8632fa6 100644 Given("disable broadcast pruning and disable subquery duplication") withSQLConf( SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", -@@ -1215,7 +1220,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1027,7 +1032,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("avoid reordering broadcast join keys to match input hash partitioning") { ++ test("avoid reordering broadcast join keys to match input hash partitioning", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1839")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + withTable("large", "dimTwo", "dimThree") { +@@ -1215,7 +1221,8 @@ abstract class DynamicPartitionPruningSuiteBase } test("SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + - "canonicalization and exchange reuse") { + "canonicalization and exchange reuse", -+ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #1737")) { ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1839")) { withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { val df = sql( -@@ -1455,7 +1461,8 @@ abstract class DynamicPartitionPruningSuiteBase +@@ -1424,7 +1431,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("SPARK-34637: DPP side broadcast query stage is created firstly") { ++ test("SPARK-34637: DPP side broadcast query stage is created firstly", ++ IgnoreComet("TODO: https://github.com/apache/datafusion-comet/issues/1839")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + val df = sql( + """ WITH v as ( +@@ -1455,7 +1463,8 @@ abstract class DynamicPartitionPruningSuiteBase } } @@ -448,7 +506,7 @@ index 2c24cc7d570..3e6a8632fa6 100644 val df = sql( """ |SELECT s.store_id, f.product_id -@@ -1730,6 +1737,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1730,6 +1739,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat case s: BatchScanExec => // we use f1 col for v2 tables due to schema pruning s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) @@ -649,7 +707,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 fcb937d82ba..fafe8e8d08b 100644 +index fcb937d82ba..fc208087a69 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 @@ -775,7 +833,7 @@ index fcb937d82ba..fafe8e8d08b 100644 }.size === 1) // Same result between shuffled hash join and sort merge join checkAnswer(shjDF, smjResult) -@@ -1435,13 +1449,19 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1435,13 +1449,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan assert(shjCodegenDF.queryExecution.executedPlan.collect { case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true case WholeStageCodegenExec(ProjectExec(_, _ : ShuffledHashJoinExec)) => true @@ -783,6 +841,7 @@ index fcb937d82ba..fafe8e8d08b 100644 + true + case WholeStageCodegenExec(ColumnarToRowExec( + InputAdapter(CometProjectExec(_, _, _, _, _: CometHashJoinExec, _)))) => true ++ case _: CometHashJoinExec => true }.size === 1) checkAnswer(shjCodegenDF, Seq.empty) @@ -796,7 +855,7 @@ index fcb937d82ba..fafe8e8d08b 100644 checkAnswer(shjNonCodegenDF, Seq.empty) } } -@@ -1489,7 +1509,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1489,7 +1510,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val plan = sql(getAggQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) // Have shuffle before aggregation @@ -806,7 +865,7 @@ index fcb937d82ba..fafe8e8d08b 100644 } def getJoinQuery(selectExpr: String, joinType: String): String = { -@@ -1518,9 +1539,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1518,9 +1540,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan } val plan = sql(getJoinQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) @@ -821,7 +880,7 @@ index fcb937d82ba..fafe8e8d08b 100644 } // Test output ordering is not preserved -@@ -1529,9 +1553,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1529,9 +1554,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val selectExpr = "/*+ BROADCAST(left_t) */ k1 as k0" val plan = sql(getJoinQuery(selectExpr, joinType)).queryExecution.executedPlan assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) @@ -836,7 +895,7 @@ index fcb937d82ba..fafe8e8d08b 100644 } // Test singe partition -@@ -1541,7 +1568,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1541,7 +1569,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan |FROM range(0, 10, 1, 1) t1 FULL OUTER JOIN range(0, 10, 1, 1) t2 |""".stripMargin) val plan = fullJoinDF.queryExecution.executedPlan @@ -846,7 +905,7 @@ index fcb937d82ba..fafe8e8d08b 100644 checkAnswer(fullJoinDF, Row(100)) } } -@@ -1586,6 +1614,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1586,6 +1615,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 @@ -856,7 +915,7 @@ index fcb937d82ba..fafe8e8d08b 100644 }.size == 1) } } -@@ -1630,14 +1661,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1630,14 +1662,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 = { @@ -879,7 +938,7 @@ index fcb937d82ba..fafe8e8d08b 100644 } dupStreamSideColTest("SHUFFLE_HASH", check) } -@@ -1773,7 +1810,8 @@ class ThreadLeakInSortMergeJoinSuite +@@ -1773,7 +1811,8 @@ class ThreadLeakInSortMergeJoinSuite sparkConf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, 20)) } @@ -1451,6 +1510,28 @@ index 47679ed7865..9ffbaecb98e 100644 }.length == hashAggCount) assert(collectWithSubqueries(plan) { case s: SortAggregateExec => s }.length == sortAggCount) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala +index eec396b2e39..bf3f1c769d6 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala +@@ -18,7 +18,7 @@ + package org.apache.spark.sql.execution + + import org.apache.spark.TestUtils.assertSpilled +-import org.apache.spark.sql.{AnalysisException, QueryTest, Row} ++import org.apache.spark.sql.{AnalysisException, IgnoreComet, QueryTest, Row} + import org.apache.spark.sql.internal.SQLConf.{WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD, WINDOW_EXEC_BUFFER_SPILL_THRESHOLD} + import org.apache.spark.sql.test.SharedSparkSession + +@@ -470,7 +470,7 @@ class SQLWindowFunctionSuite extends QueryTest with SharedSparkSession { + Row(1, 3, null) :: Row(2, null, 4) :: Nil) + } + +- test("test with low buffer spill threshold") { ++ test("test with low buffer spill threshold", IgnoreComet("Comet does not support spilling")) { + val nums = sparkContext.parallelize(1 to 10).map(x => (x, x % 2)).toDF("x", "y") + nums.createOrReplaceTempView("nums") + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala index 966f4e74712..8017e22d7f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala From 2b416d39c2a201304ad14197f5ded6e7bcb2104d Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 6 Jun 2025 16:56:03 -0600 Subject: [PATCH 40/42] prep for review --- .github/workflows/spark_sql_test.yml | 1 - .github/workflows/spark_sql_test_ansi.yml | 154 +++++++++++----------- 2 files changed, 77 insertions(+), 78 deletions(-) diff --git a/.github/workflows/spark_sql_test.yml b/.github/workflows/spark_sql_test.yml index 4b762476bf..bb704c20ec 100644 --- a/.github/workflows/spark_sql_test.yml +++ b/.github/workflows/spark_sql_test.yml @@ -45,7 +45,6 @@ jobs: matrix: os: [ubuntu-24.04] java-version: [11] - # TODO enbable other versions once tests are passing with 3.5.5 spark-version: [{short: '3.4', full: '3.4.3'}, {short: '3.5', full: '3.5.4'}, {short: '3.5', full: '3.5.5'}] module: - {name: "catalyst", args1: "catalyst/test", args2: ""} diff --git a/.github/workflows/spark_sql_test_ansi.yml b/.github/workflows/spark_sql_test_ansi.yml index 878e9a803c..a3e85307d9 100644 --- a/.github/workflows/spark_sql_test_ansi.yml +++ b/.github/workflows/spark_sql_test_ansi.yml @@ -1,79 +1,79 @@ -## 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. +# 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 # -#name: Spark SQL Tests (ANSI mode) -# -#concurrency: -# group: ${{ github.repository }}-${{ github.head_ref || github.sha }}-${{ github.workflow }} -# cancel-in-progress: true -# -#on: -# push: -# paths-ignore: -# - "docs/**" -# - "**.md" -# pull_request: -# paths-ignore: -# - "docs/**" -# - "**.md" -# # manual trigger -# # https://docs.github.com/en/actions/managing-workflow-runs/manually-running-a-workflow -# workflow_dispatch: -# -#env: -# RUST_VERSION: stable -# -#jobs: -# spark-sql-catalyst: -# strategy: -# matrix: -# os: [ubuntu-24.04] -# java-version: [17] -# spark-version: [{short: '4.0', full: '4.0.0-preview1'}] -# module: -# - {name: "catalyst", args1: "catalyst/test", args2: ""} -# - {name: "sql/core-1", args1: "", args2: sql/testOnly * -- -l org.apache.spark.tags.ExtendedSQLTest -l org.apache.spark.tags.SlowSQLTest} -# - {name: "sql/core-2", args1: "", args2: "sql/testOnly * -- -n org.apache.spark.tags.ExtendedSQLTest"} -# - {name: "sql/core-3", args1: "", args2: "sql/testOnly * -- -n org.apache.spark.tags.SlowSQLTest"} -# - {name: "sql/hive-1", args1: "", args2: "hive/testOnly * -- -l org.apache.spark.tags.ExtendedHiveTest -l org.apache.spark.tags.SlowHiveTest"} -# - {name: "sql/hive-2", args1: "", args2: "hive/testOnly * -- -n org.apache.spark.tags.ExtendedHiveTest"} -# - {name: "sql/hive-3", args1: "", args2: "hive/testOnly * -- -n org.apache.spark.tags.SlowHiveTest"} -# fail-fast: false -# name: spark-sql-${{ matrix.module.name }}/${{ matrix.os }}/spark-${{ matrix.spark-version.full }}/java-${{ matrix.java-version }} -# runs-on: ${{ matrix.os }} -# container: -# image: amd64/rust -# steps: -# - uses: actions/checkout@v4 -# - name: Setup Rust & Java toolchain -# uses: ./.github/actions/setup-builder -# with: -# rust-version: ${{env.RUST_VERSION}} -# jdk-version: ${{ matrix.java-version }} -# - name: Setup Spark -# uses: ./.github/actions/setup-spark-builder -# with: -# spark-version: ${{ matrix.spark-version.full }} -# spark-short-version: ${{ matrix.spark-version.short }} -# - name: Run Spark tests -# run: | -# cd apache-spark -# rm -rf /root/.m2/repository/org/apache/parquet # somehow parquet cache requires cleanups -# RUST_BACKTRACE=1 ENABLE_COMET=true ENABLE_COMET_ANSI_MODE=true ENABLE_COMET_SHUFFLE=true build/sbt -Dsbt.log.noformat=true ${{ matrix.module.args1 }} "${{ matrix.module.args2 }}" -# env: -# LC_ALL: "C.UTF-8" +# 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. + +name: Spark SQL Tests (ANSI mode) + +concurrency: + group: ${{ github.repository }}-${{ github.head_ref || github.sha }}-${{ github.workflow }} + cancel-in-progress: true + +on: + push: + paths-ignore: + - "docs/**" + - "**.md" + pull_request: + paths-ignore: + - "docs/**" + - "**.md" + # manual trigger + # https://docs.github.com/en/actions/managing-workflow-runs/manually-running-a-workflow + workflow_dispatch: + +env: + RUST_VERSION: stable + +jobs: + spark-sql-catalyst: + strategy: + matrix: + os: [ubuntu-24.04] + java-version: [17] + spark-version: [{short: '4.0', full: '4.0.0-preview1'}] + module: + - {name: "catalyst", args1: "catalyst/test", args2: ""} + - {name: "sql/core-1", args1: "", args2: sql/testOnly * -- -l org.apache.spark.tags.ExtendedSQLTest -l org.apache.spark.tags.SlowSQLTest} + - {name: "sql/core-2", args1: "", args2: "sql/testOnly * -- -n org.apache.spark.tags.ExtendedSQLTest"} + - {name: "sql/core-3", args1: "", args2: "sql/testOnly * -- -n org.apache.spark.tags.SlowSQLTest"} + - {name: "sql/hive-1", args1: "", args2: "hive/testOnly * -- -l org.apache.spark.tags.ExtendedHiveTest -l org.apache.spark.tags.SlowHiveTest"} + - {name: "sql/hive-2", args1: "", args2: "hive/testOnly * -- -n org.apache.spark.tags.ExtendedHiveTest"} + - {name: "sql/hive-3", args1: "", args2: "hive/testOnly * -- -n org.apache.spark.tags.SlowHiveTest"} + fail-fast: false + name: spark-sql-${{ matrix.module.name }}/${{ matrix.os }}/spark-${{ matrix.spark-version.full }}/java-${{ matrix.java-version }} + runs-on: ${{ matrix.os }} + container: + image: amd64/rust + steps: + - uses: actions/checkout@v4 + - name: Setup Rust & Java toolchain + uses: ./.github/actions/setup-builder + with: + rust-version: ${{env.RUST_VERSION}} + jdk-version: ${{ matrix.java-version }} + - name: Setup Spark + uses: ./.github/actions/setup-spark-builder + with: + spark-version: ${{ matrix.spark-version.full }} + spark-short-version: ${{ matrix.spark-version.short }} + - name: Run Spark tests + run: | + cd apache-spark + rm -rf /root/.m2/repository/org/apache/parquet # somehow parquet cache requires cleanups + RUST_BACKTRACE=1 ENABLE_COMET=true ENABLE_COMET_ANSI_MODE=true ENABLE_COMET_SHUFFLE=true build/sbt -Dsbt.log.noformat=true ${{ matrix.module.args1 }} "${{ matrix.module.args2 }}" + env: + LC_ALL: "C.UTF-8" + From 0968c29538925b58e920074730367e8fe5255b55 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 6 Jun 2025 17:40:23 -0600 Subject: [PATCH 41/42] fix 3.4.3 diff --- dev/diffs/3.4.3.diff | 51 ++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 51 insertions(+) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 35e4732d9f..aeb1472c67 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -565,6 +565,57 @@ index 2796b1cf154..4816349d690 100644 }.flatten assert(filters.contains(GreaterThan(scan.logicalPlan.output.head, Literal(5L)))) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala +new file mode 100644 +index 00000000000..5691536c114 +--- /dev/null ++++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala +@@ -0,0 +1,45 @@ ++/* ++ * 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 ++ ++import org.scalactic.source.Position ++import org.scalatest.Tag ++ ++import org.apache.spark.sql.test.SQLTestUtils ++ ++/** ++ * Tests with this tag will be ignored when Comet is enabled (e.g., via `ENABLE_COMET`). ++ */ ++case class IgnoreComet(reason: String) extends Tag("DisableComet") ++case class IgnoreCometNativeIcebergCompat(reason: String) extends Tag("DisableComet") ++case class IgnoreCometNativeDataFusion(reason: String) extends Tag("DisableComet") ++case class IgnoreCometNativeScan(reason: String) extends Tag("DisableComet") ++ ++/** ++ * Helper trait that disables Comet for all tests regardless of default config values. ++ */ ++trait IgnoreCometSuite extends SQLTestUtils { ++ override protected def test(testName: String, testTags: Tag*)(testFun: => Any) ++ (implicit pos: Position): Unit = { ++ if (isCometEnabled) { ++ ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) ++ } else { ++ super.test(testName, testTags: _*)(testFun) ++ } ++ } ++} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala index fda442eeef0..1b69e4f280e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala From 3f6d03d71fd7490f4eb45251d9710993bb4f86cf Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 7 Jun 2025 09:46:55 -0600 Subject: [PATCH 42/42] fix 3.5.4 diff --- dev/diffs/3.5.4.diff | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/dev/diffs/3.5.4.diff b/dev/diffs/3.5.4.diff index 4b8d5c2d0a..68e0b8e362 100644 --- a/dev/diffs/3.5.4.diff +++ b/dev/diffs/3.5.4.diff @@ -343,16 +343,10 @@ index 7ee18df3756..64f01a68048 100644 sql( """ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala -index 47a311c71d5..909a12bd830 100644 +index 47a311c71d5..342e71cfdd4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala -@@ -18,14 +18,14 @@ - package org.apache.spark.sql - - import org.scalatest.matchers.must.Matchers.the -- - import org.apache.spark.TestUtils.{assertNotSpilled, assertSpilled} - import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, Lag, Literal, NonFoldableLiteral} +@@ -24,8 +24,9 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression import org.apache.spark.sql.catalyst.optimizer.TransposeWindow import org.apache.spark.sql.catalyst.plans.logical.{Window => LogicalWindow} import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning @@ -363,7 +357,7 @@ index 47a311c71d5..909a12bd830 100644 import org.apache.spark.sql.execution.window.WindowExec import org.apache.spark.sql.expressions.{Aggregator, MutableAggregationBuffer, UserDefinedAggregateFunction, Window} import org.apache.spark.sql.functions._ -@@ -1187,10 +1187,12 @@ class DataFrameWindowFunctionsSuite extends QueryTest +@@ -1187,10 +1188,12 @@ class DataFrameWindowFunctionsSuite extends QueryTest } def isShuffleExecByRequirement( @@ -377,7 +371,7 @@ index 47a311c71d5..909a12bd830 100644 case _ => false } -@@ -1213,7 +1215,7 @@ class DataFrameWindowFunctionsSuite extends QueryTest +@@ -1213,7 +1216,7 @@ class DataFrameWindowFunctionsSuite extends QueryTest val shuffleByRequirement = windowed.queryExecution.executedPlan.exists { case w: WindowExec => w.child.exists {