From 1123c96f6ddb7311d2262487e0671137b1cb2a46 Mon Sep 17 00:00:00 2001 From: B Vadlamani Date: Fri, 30 Jan 2026 17:17:00 -0800 Subject: [PATCH 01/15] enable_spark_tests_comet_native_writer --- .github/workflows/spark_sql_test.yml | 2 +- dev/diffs/3.4.3.diff | 57 ++- dev/diffs/3.5.8.diff | 565 ++++++++------------------- dev/diffs/4.0.1.diff | 86 +++- 4 files changed, 273 insertions(+), 437 deletions(-) diff --git a/.github/workflows/spark_sql_test.yml b/.github/workflows/spark_sql_test.yml index 610baa9f2d..9e2adf9c5a 100644 --- a/.github/workflows/spark_sql_test.yml +++ b/.github/workflows/spark_sql_test.yml @@ -155,7 +155,7 @@ jobs: run: | cd apache-spark rm -rf /root/.m2/repository/org/apache/parquet # somehow parquet cache requires cleanups - NOLINT_ON_COMPILE=true ENABLE_COMET=true ENABLE_COMET_ONHEAP=true COMET_PARQUET_SCAN_IMPL=${{ matrix.config.scan-impl }} ENABLE_COMET_LOG_FALLBACK_REASONS=${{ github.event.inputs.collect-fallback-logs || 'false' }} \ + NOLINT_ON_COMPILE=true ENABLE_COMET=true ENABLE_COMET_ONHEAP=true ENABLE_COMET_WRITER=true ${{ matrix.config.scan-env }} ENABLE_COMET_LOG_FALLBACK_REASONS=${{ github.event.inputs.collect-fallback-logs || 'false' }} \ build/sbt -Dsbt.log.noformat=true ${{ matrix.module.args1 }} "${{ matrix.module.args2 }}" if [ "${{ github.event.inputs.collect-fallback-logs }}" = "true" ]; then find . -type f -name "unit-tests.log" -print0 | xargs -0 grep -h "Comet cannot accelerate" | sed 's/.*Comet cannot accelerate/Comet cannot accelerate/' | sort -u > fallback.log diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 0205888433..5c2f49b437 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -2092,7 +2092,7 @@ index 104b4e416cd..37ea65081e4 100644 case _ => throw new AnalysisException("Can not match ParquetTable in the query.") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -index 8670d95c65e..b624c3811dd 100644 +index 8670d95c65e..3fe49802309 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -1335,7 +1335,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession @@ -2105,6 +2105,41 @@ index 8670d95c65e..b624c3811dd 100644 withAllParquetReaders { checkAnswer( // "fruit" column in this file is encoded using DELTA_LENGTH_BYTE_ARRAY. +@@ -1541,7 +1542,9 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession + } + } + +- test("Write Spark version into Parquet metadata") { ++// TODO : Comet native writer to add spark / comet version into parquet metadata ++ test("Write Spark version into Parquet metadata", ++ IgnoreComet("Comet doesn't support DELTA encoding yet")) { + withTempPath { dir => + spark.range(1).repartition(1).write.parquet(dir.getAbsolutePath) + assert(getMetaData(dir)(SPARK_VERSION_METADATA_KEY) === SPARK_VERSION_SHORT) +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +index 8b386e8f689..28ced6209e0 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +@@ -25,7 +25,7 @@ import org.apache.hadoop.fs.{Path, PathFilter} + import org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER + import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName + +-import org.apache.spark.sql.Row ++import org.apache.spark.sql.{IgnoreComet, Row} + import org.apache.spark.sql.catalyst.util.DateTimeUtils + import org.apache.spark.sql.internal.SQLConf + import org.apache.spark.sql.test.SharedSparkSession +@@ -153,7 +153,9 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS + } + } + +- test("parquet timestamp conversion") { ++ // TODO : Support legacy timestamps conversion /cast in comet native writer ++ test("parquet timestamp conversion", ++ IgnoreComet("timestamp96 conversion failed with the native writer")) { + // Make a table with one parquet file written by impala, and one parquet file written by spark. + // We should only adjust the timestamps in the impala file, and only if the conf is set + val impalaFile = "test-data/impala_timestamp.parq" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index 29cb224c878..44837aa953b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -2804,7 +2839,7 @@ index abe606ad9c1..2d930b64cca 100644 val tblTargetName = "tbl_target" val tblSourceQualified = s"default.$tblSourceName" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala -index dd55fcfe42c..a1d390c93d0 100644 +index dd55fcfe42c..e898fc33bab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -27,6 +27,7 @@ import scala.concurrent.duration._ @@ -2862,7 +2897,7 @@ index dd55fcfe42c..a1d390c93d0 100644 } } -@@ -242,6 +265,29 @@ private[sql] trait SQLTestUtilsBase +@@ -242,6 +265,34 @@ private[sql] trait SQLTestUtilsBase protected override def _sqlContext: SQLContext = self.spark.sqlContext } @@ -2888,11 +2923,16 @@ index dd55fcfe42c..a1d390c93d0 100644 + val v = System.getenv("ENABLE_COMET_SCAN_ONLY") + v != null && v.toBoolean + } ++ ++ protected def isCometWriterEnabled: Boolean = { ++ val v = System.getenv("ENABLE_COMET_WRITER") ++ v != null && v.toBoolean ++ } + protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { SparkSession.setActiveSession(spark) super.withSQLConf(pairs: _*)(f) -@@ -434,6 +480,8 @@ private[sql] trait SQLTestUtilsBase +@@ -434,6 +485,8 @@ private[sql] trait SQLTestUtilsBase val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child @@ -2902,10 +2942,10 @@ index dd55fcfe42c..a1d390c93d0 100644 spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -index ed2e309fa07..a5ea58146ad 100644 +index ed2e309fa07..9c5c393ad14 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -@@ -74,6 +74,31 @@ trait SharedSparkSessionBase +@@ -74,6 +74,36 @@ trait SharedSparkSessionBase // this rule may potentially block testing of other optimization rules such as // ConstantPropagation etc. .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) @@ -2933,6 +2973,11 @@ index ed2e309fa07..a5ea58146ad 100644 + conf + .set("spark.sql.ansi.enabled", "true") + } ++ ++ if (isCometWriterEnabled) { ++ conf.set("spark.comet.parquet.write.enabled", "true") ++ conf.set("spark.comet.operator.DataWritingCommandExec.allowIncompatible", "true") ++ } + } conf.set( StaticSQLConf.WAREHOUSE_PATH, diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index beef445490..682b8b3772 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -1,5 +1,5 @@ diff --git a/pom.xml b/pom.xml -index edd2ad57880..77a975ea48f 100644 +index 68e2c422a24..540bdabf825 100644 --- a/pom.xml +++ b/pom.xml @@ -152,6 +152,8 @@ @@ -38,7 +38,7 @@ index edd2ad57880..77a975ea48f 100644 diff --git a/sql/core/pom.xml b/sql/core/pom.xml -index bc00c448b80..82068d7a2eb 100644 +index f08b33575fc..424e0da32fd 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -77,6 +77,10 @@ @@ -216,7 +216,7 @@ index 0efe0877e9b..423d3b3d76d 100644 -- SELECT_HAVING -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_having.sql diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala -index e5494726695..00937f025c2 100644 +index 9815cb816c9..95b5f9992b0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeConstants @@ -238,22 +238,8 @@ index e5494726695..00937f025c2 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/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala -index 9e8d77c53f3..855e3ada7d1 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala -@@ -790,7 +790,8 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { - } - } - -- test("input_file_name, input_file_block_start, input_file_block_length - FileScanRDD") { -+ test("input_file_name, input_file_block_start, input_file_block_length - FileScanRDD", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3312")) { - withTempPath { dir => - val data = sparkContext.parallelize(0 to 10).toDF("id") - data.write.parquet(dir.getCanonicalPath) 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 6f3090d8908..c08a60fb0c2 100644 +index 5a8681aed97..da9d25e2eb4 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 @@ -350,7 +336,7 @@ index 7ee18df3756..d09f70e5d99 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 a1d5d579338..c201d39cc78 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 @@ -24,8 +24,9 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression @@ -388,7 +374,7 @@ index a1d5d579338..c201d39cc78 100644 } 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 c4fb4fa943c..a04b23870a8 100644 +index f32b32ffc5a..447d7c6416e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} @@ -410,7 +396,7 @@ index c4fb4fa943c..a04b23870a8 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..42eb9fd1cb7 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 @@ -461,17 +447,7 @@ index f33432ddb6f..42eb9fd1cb7 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ WITH v as ( -@@ -1698,7 +1705,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat - * Check the static scan metrics with and without DPP - */ - test("static scan metrics", -- DisableAdaptiveExecution("DPP in AQE must reuse broadcast")) { -+ DisableAdaptiveExecution("DPP in AQE must reuse broadcast"), -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3313")) { - withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", - SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { -@@ -1729,6 +1737,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"))) @@ -481,20 +457,10 @@ index f33432ddb6f..42eb9fd1cb7 100644 } assert(scanOption.isDefined) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -index a206e97c353..79813d8e259 100644 +index a206e97c353..fea1149b67d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -@@ -280,7 +280,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite - } - } - -- test("explain formatted - check presence of subquery in case of DPP") { -+ test("explain formatted - check presence of subquery in case of DPP", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3313")) { - withTable("df1", "df2") { - withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", - SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", -@@ -467,7 +468,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -467,7 +467,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } @@ -504,7 +470,7 @@ index a206e97c353..79813d8e259 100644 withTempDir { dir => Seq("parquet", "orc", "csv", "json").foreach { fmt => val basePath = dir.getCanonicalPath + "/" + fmt -@@ -545,7 +547,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -545,7 +546,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } @@ -516,7 +482,7 @@ index a206e97c353..79813d8e259 100644 test("SPARK-35884: Explain Formatted") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala -index 93275487f29..510e3087e0f 100644 +index 93275487f29..33b2e7ad3b1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -23,6 +23,7 @@ import java.nio.file.{Files, StandardOpenOption} @@ -544,17 +510,7 @@ index 93275487f29..510e3087e0f 100644 checkErrorMatchPVals( exception = intercept[SparkException] { testIgnoreMissingFiles(options) -@@ -639,7 +643,8 @@ class FileBasedDataSourceSuite extends QueryTest - } - - Seq("parquet", "orc").foreach { format => -- test(s"Spark native readers should respect spark.sql.caseSensitive - ${format}") { -+ test(s"Spark native readers should respect spark.sql.caseSensitive - ${format}", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { - withTempDir { dir => - val tableName = s"spark_25132_${format}_native" - val tableDir = dir.getCanonicalPath + s"/$tableName" -@@ -955,6 +960,7 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -955,6 +959,7 @@ class FileBasedDataSourceSuite extends QueryTest assert(bJoinExec.isEmpty) val smJoinExec = collect(joinedDF.queryExecution.executedPlan) { case smJoin: SortMergeJoinExec => smJoin @@ -562,7 +518,7 @@ index 93275487f29..510e3087e0f 100644 } assert(smJoinExec.nonEmpty) } -@@ -1015,6 +1021,7 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -1015,6 +1020,7 @@ class FileBasedDataSourceSuite extends QueryTest val fileScan = df.queryExecution.executedPlan collectFirst { case BatchScanExec(_, f: FileScan, _, _, _, _) => f @@ -570,7 +526,7 @@ index 93275487f29..510e3087e0f 100644 } assert(fileScan.nonEmpty) assert(fileScan.get.partitionFilters.nonEmpty) -@@ -1056,6 +1063,7 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -1056,6 +1062,7 @@ class FileBasedDataSourceSuite extends QueryTest val fileScan = df.queryExecution.executedPlan collectFirst { case BatchScanExec(_, f: FileScan, _, _, _, _) => f @@ -578,7 +534,7 @@ index 93275487f29..510e3087e0f 100644 } assert(fileScan.nonEmpty) assert(fileScan.get.partitionFilters.isEmpty) -@@ -1240,6 +1248,9 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -1240,6 +1247,9 @@ class FileBasedDataSourceSuite extends QueryTest val filters = df.queryExecution.executedPlan.collect { case f: FileSourceScanLike => f.dataFilters case b: BatchScanExec => b.scan.asInstanceOf[FileScan].dataFilters @@ -590,7 +546,7 @@ index 93275487f29..510e3087e0f 100644 } 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..1ee842b6f62 +index 00000000000..5691536c114 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala @@ -0,0 +1,45 @@ @@ -630,8 +586,8 @@ index 00000000000..1ee842b6f62 + * 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 = { ++ 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 { @@ -668,7 +624,7 @@ index 7af826583bd..3c3def1eb67 100644 assert(shuffleMergeJoins.size == 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala -index 44c8cb92fc3..f098beeca26 100644 +index 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 @@ -866,7 +822,7 @@ index 44c8cb92fc3..f098beeca26 100644 checkAnswer(fullJoinDF, Row(100)) } } -@@ -1611,6 +1640,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 @@ -876,7 +832,7 @@ index 44c8cb92fc3..f098beeca26 100644 }.size == 1) } } -@@ -1655,14 +1687,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 = { @@ -899,7 +855,7 @@ index 44c8cb92fc3..f098beeca26 100644 } dupStreamSideColTest("SHUFFLE_HASH", check) } -@@ -1798,7 +1836,8 @@ class ThreadLeakInSortMergeJoinSuite +@@ -1770,7 +1808,8 @@ class ThreadLeakInSortMergeJoinSuite sparkConf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, 20)) } @@ -923,7 +879,7 @@ index c26757c9cff..d55775f09d7 100644 protected val baseResourcePath = { // use the same way as `SQLQueryTestSuite` to get the resource path diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala -index 3cf2bfd17ab..49728c35c42 100644 +index 793a0da6a86..181bfc16e4b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1521,7 +1521,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark @@ -1084,20 +1040,6 @@ index 04702201f82..5ee11f83ecf 100644 } assert(exchanges.size === 1) } -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala -index 9f8e979e3fb..3bc9dab8023 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala -@@ -87,7 +87,8 @@ class UDFSuite extends QueryTest with SharedSparkSession { - spark.catalog.dropTempView("tmp_table") - } - -- test("SPARK-8005 input_file_name") { -+ test("SPARK-8005 input_file_name", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3312")) { - withTempPath { dir => - val data = sparkContext.parallelize(0 to 10, 2).toDF("id") - data.write.parquet(dir.getCanonicalPath) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index d269290e616..13726a31e07 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -1162,37 +1104,24 @@ index d269290e616..13726a31e07 100644 } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala -index cfc8b2cc845..b7c234e1437 100644 +index cfc8b2cc845..c6fcfd7bd08 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala -@@ -19,8 +19,9 @@ package org.apache.spark.sql.connector - import scala.collection.mutable.ArrayBuffer - +@@ -21,6 +21,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkConf --import org.apache.spark.sql.{AnalysisException, QueryTest} -+import org.apache.spark.sql.{AnalysisException, IgnoreCometNativeDataFusion, QueryTest} + import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -+import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec} ++import org.apache.spark.sql.comet.CometScanExec import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, Table, TableCapability} import org.apache.spark.sql.connector.read.ScanBuilder import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} -@@ -152,7 +153,8 @@ class FileDataSourceV2FallBackSuite extends QueryTest with SharedSparkSession { - } - } - -- test("Fallback Parquet V2 to V1") { -+ test("Fallback Parquet V2 to V1", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3315")) { - Seq("parquet", classOf[ParquetDataSourceV2].getCanonicalName).foreach { format => - withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> format) { - val commands = ArrayBuffer.empty[(String, LogicalPlan)] -@@ -184,7 +186,11 @@ class FileDataSourceV2FallBackSuite extends QueryTest with SharedSparkSession { +@@ -184,7 +185,11 @@ class FileDataSourceV2FallBackSuite extends QueryTest with SharedSparkSession { val df = spark.read.format(format).load(path.getCanonicalPath) checkAnswer(df, inputData.toDF()) assert( - df.queryExecution.executedPlan.exists(_.isInstanceOf[FileSourceScanExec])) + df.queryExecution.executedPlan.exists { -+ case _: FileSourceScanExec | _: CometScanExec | _: CometNativeScanExec => true ++ case _: FileSourceScanExec | _: CometScanExec => true + case _ => false + } + ) @@ -1450,28 +1379,6 @@ 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/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala -index a1147c16cc8..c7a29496328 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala -@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution - - import org.apache.spark.{SparkArithmeticException, SparkException, SparkFileNotFoundException} - import org.apache.spark.sql._ -+import org.apache.spark.sql.IgnoreCometNativeDataFusion - import org.apache.spark.sql.catalyst.TableIdentifier - import org.apache.spark.sql.catalyst.expressions.{Add, Alias, Divide} - import org.apache.spark.sql.catalyst.parser.ParseException -@@ -968,7 +969,8 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { - } - } - -- test("alter temporary view should follow current storeAnalyzedPlanForView config") { -+ test("alter temporary view should follow current storeAnalyzedPlanForView config", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3314")) { - withTable("t") { - Seq(2, 3, 1).toDF("c1").write.format("parquet").saveAsTable("t") - withView("v1") { 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 @@ -1970,7 +1877,7 @@ index 2f8e401e743..a4f94417dcc 100644 assert(o1.semanticEquals(o2), "Different output column order after AQE optimization") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala -index fd52d038ca6..154c800be67 100644 +index bf496d6db21..9bb57a9b4c6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.Concat @@ -1981,7 +1888,7 @@ index fd52d038ca6..154c800be67 100644 import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.functions._ -@@ -884,6 +885,8 @@ abstract class SchemaPruningSuite +@@ -868,6 +869,8 @@ abstract class SchemaPruningSuite val fileSourceScanSchemata = collect(df.queryExecution.executedPlan) { case scan: FileSourceScanExec => scan.requiredSchema @@ -1991,7 +1898,7 @@ index fd52d038ca6..154c800be67 100644 assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala -index 5fd27410dcb..468abb1543a 100644 +index ce43edb79c1..4dbb5942bc3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources @@ -2002,7 +1909,7 @@ index 5fd27410dcb..468abb1543a 100644 import org.apache.spark.sql.execution.{QueryExecution, SortExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.internal.SQLConf -@@ -243,6 +244,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write +@@ -225,6 +226,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write // assert the outer most sort in the executed plan assert(plan.collectFirst { case s: SortExec => s @@ -2010,7 +1917,7 @@ index 5fd27410dcb..468abb1543a 100644 }.exists { case SortExec(Seq( SortOrder(AttributeReference("key", IntegerType, _, _), Ascending, NullsFirst, _), -@@ -290,6 +292,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write +@@ -272,6 +274,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write // assert the outer most sort in the executed plan assert(plan.collectFirst { case s: SortExec => s @@ -2066,7 +1973,7 @@ index 07e2849ce6f..3e73645b638 100644 ParquetOutputFormat.WRITER_VERSION -> ParquetProperties.WriterVersion.PARQUET_2_0.toString ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala -index 8e88049f51e..49f2001dc6b 100644 +index 8e88049f51e..8f3cf8a0f80 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -1095,7 +1095,11 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared @@ -2092,17 +1999,7 @@ index 8e88049f51e..49f2001dc6b 100644 import testImplicits._ withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", -@@ -1548,7 +1553,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared - } - } - -- test("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") { -+ test("SPARK-31026: Parquet predicate pushdown for fields having dots in the names", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3320")) { - import testImplicits._ - - withAllParquetReaders { -@@ -1580,13 +1586,18 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared +@@ -1580,7 +1585,11 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared // than the total length but should not be a single record. // Note that, if record level filtering is enabled, it should be a single record. // If no filter is pushed down to Parquet, it should be the total length of data. @@ -2115,15 +2012,7 @@ index 8e88049f51e..49f2001dc6b 100644 } } } - } - -- test("Filters should be pushed down for Parquet readers at row group level") { -+ test("Filters should be pushed down for Parquet readers at row group level", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3320")) { - import testImplicits._ - - withSQLConf( -@@ -1607,7 +1618,11 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared +@@ -1607,7 +1616,11 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared // than the total length but should not be a single record. // Note that, if record level filtering is enabled, it should be a single record. // If no filter is pushed down to Parquet, it should be the total length of data. @@ -2136,7 +2025,7 @@ index 8e88049f51e..49f2001dc6b 100644 } } } -@@ -1699,7 +1714,7 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared +@@ -1699,7 +1712,7 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared (attr, value) => sources.StringContains(attr, value)) } @@ -2145,7 +2034,7 @@ index 8e88049f51e..49f2001dc6b 100644 import testImplicits._ // keep() should take effect on StartsWith/EndsWith/Contains Seq( -@@ -1743,7 +1758,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared +@@ -1743,7 +1756,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } } @@ -2155,17 +2044,7 @@ index 8e88049f51e..49f2001dc6b 100644 val schema = StructType(Seq( StructField("a", IntegerType, nullable = false) )) -@@ -1933,7 +1949,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared - } - } - -- test("SPARK-25207: exception when duplicate fields in case-insensitive mode") { -+ test("SPARK-25207: exception when duplicate fields in case-insensitive mode", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { - withTempPath { dir => - val count = 10 - val tableName = "spark_25207" -@@ -1984,7 +2001,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared +@@ -1984,7 +1998,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } } @@ -2175,7 +2054,7 @@ index 8e88049f51e..49f2001dc6b 100644 // block 1: // null count min max // page-0 0 0 99 -@@ -2044,7 +2062,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared +@@ -2044,7 +2059,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } } @@ -2185,7 +2064,7 @@ index 8e88049f51e..49f2001dc6b 100644 withTempPath { dir => val path = dir.getCanonicalPath spark.range(100).selectExpr("id * 2 AS id") -@@ -2276,7 +2295,11 @@ class ParquetV1FilterSuite extends ParquetFilterSuite { +@@ -2276,7 +2292,11 @@ class ParquetV1FilterSuite extends ParquetFilterSuite { assert(pushedParquetFilters.exists(_.getClass === filterClass), s"${pushedParquetFilters.map(_.getClass).toList} did not contain ${filterClass}.") @@ -2198,7 +2077,7 @@ index 8e88049f51e..49f2001dc6b 100644 } else { assert(selectedFilters.isEmpty, "There is filter pushed down") } -@@ -2336,7 +2359,11 @@ class ParquetV2FilterSuite extends ParquetFilterSuite { +@@ -2336,7 +2356,11 @@ class ParquetV2FilterSuite extends ParquetFilterSuite { assert(pushedParquetFilters.exists(_.getClass === filterClass), s"${pushedParquetFilters.map(_.getClass).toList} did not contain ${filterClass}.") @@ -2212,54 +2091,57 @@ index 8e88049f51e..49f2001dc6b 100644 case _ => throw new AnalysisException("Can not match ParquetTable in the query.") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -index 8ed9ef1630e..f312174b182 100644 +index 4f8a9e39716..5da031994ff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -@@ -1064,7 +1064,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession +@@ -1335,7 +1335,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession } } -- test("SPARK-35640: read binary as timestamp should throw schema incompatible error") { -+ test("SPARK-35640: read binary as timestamp should throw schema incompatible error", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { - val data = (1 to 4).map(i => Tuple1(i.toString)) - val readSchema = StructType(Seq(StructField("_1", DataTypes.TimestampType))) - -@@ -1075,7 +1076,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession +- test("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") { ++ test("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings", ++ IgnoreComet("Comet doesn't support DELTA encoding yet")) { + withAllParquetReaders { + checkAnswer( + // "fruit" column in this file is encoded using DELTA_LENGTH_BYTE_ARRAY. +@@ -1541,7 +1542,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession } } -- test("SPARK-35640: int as long should throw schema incompatible error") { -+ test("SPARK-35640: int as long should throw schema incompatible error", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { - val data = (1 to 4).map(i => Tuple1(i)) - val readSchema = StructType(Seq(StructField("_1", DataTypes.LongType))) +- test("Write Spark version into Parquet metadata") { ++ test("Write Spark version into Parquet metadata", ++ IgnoreComet("comet does not write spark version in parquet metadata")) { + withTempPath { dir => + spark.range(1).repartition(1).write.parquet(dir.getAbsolutePath) + assert(getMetaData(dir)(SPARK_VERSION_METADATA_KEY) === SPARK_VERSION_SHORT) +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +index 8b386e8f689..67a41628a3b 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +@@ -25,7 +25,7 @@ import org.apache.hadoop.fs.{Path, PathFilter} + import org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER + import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName -@@ -1345,7 +1347,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession +-import org.apache.spark.sql.Row ++import org.apache.spark.sql.{IgnoreComet, Row} + import org.apache.spark.sql.catalyst.util.DateTimeUtils + import org.apache.spark.sql.internal.SQLConf + import org.apache.spark.sql.test.SharedSparkSession +@@ -153,7 +153,8 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS } } -- test("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") { -+ test("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings", -+ IgnoreComet("Comet doesn't support DELTA encoding yet")) { - withAllParquetReaders { - checkAnswer( - // "fruit" column in this file is encoded using DELTA_LENGTH_BYTE_ARRAY. +- test("parquet timestamp conversion") { ++ test("parquet timestamp conversion", ++ IgnoreComet("timestamp96 conversion failed with the native writer")) { + // Make a table with one parquet file written by impala, and one parquet file written by spark. + // We should only adjust the timestamps in the impala file, and only if the conf is set + val impalaFile = "test-data/impala_timestamp.parq" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala -index f6472ba3d9d..ce39ebb52e6 100644 +index f6472ba3d9d..7a8f5317ed7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala -@@ -185,7 +185,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS - } - } - -- test("SPARK-36182: can't read TimestampLTZ as TimestampNTZ") { -+ test("SPARK-36182: can't read TimestampLTZ as TimestampNTZ", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { - val data = (1 to 1000).map { i => - val ts = new java.sql.Timestamp(i) - Row(ts) -@@ -998,7 +999,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -998,7 +998,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS } } @@ -2269,17 +2151,7 @@ index f6472ba3d9d..ce39ebb52e6 100644 withAllParquetReaders { withTempPath { path => // Repeated values for dictionary encoding. -@@ -1051,7 +1053,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS - testMigration(fromTsType = "TIMESTAMP_MICROS", toTsType = "INT96") - } - -- test("SPARK-34212 Parquet should read decimals correctly") { -+ test("SPARK-34212 Parquet should read decimals correctly", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { - def readParquet(schema: String, path: File): DataFrame = { - spark.read.schema(schema).parquet(path.toString) - } -@@ -1067,7 +1070,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -1067,7 +1068,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS checkAnswer(readParquet(schema, path), df) } @@ -2289,7 +2161,7 @@ index f6472ba3d9d..ce39ebb52e6 100644 val schema1 = "a DECIMAL(3, 2), b DECIMAL(18, 3), c DECIMAL(37, 3)" checkAnswer(readParquet(schema1, path), df) val schema2 = "a DECIMAL(3, 0), b DECIMAL(18, 1), c DECIMAL(37, 1)" -@@ -1089,7 +1093,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -1089,7 +1091,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS val df = sql(s"SELECT 1 a, 123456 b, ${Int.MaxValue.toLong * 10} c, CAST('1.2' AS BINARY) d") df.write.parquet(path.toString) @@ -2299,17 +2171,7 @@ index f6472ba3d9d..ce39ebb52e6 100644 checkAnswer(readParquet("a DECIMAL(3, 2)", path), sql("SELECT 1.00")) checkAnswer(readParquet("b DECIMAL(3, 2)", path), Row(null)) checkAnswer(readParquet("b DECIMAL(11, 1)", path), sql("SELECT 123456.0")) -@@ -1133,7 +1138,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS - } - } - -- test("row group skipping doesn't overflow when reading into larger type") { -+ test("row group skipping doesn't overflow when reading into larger type", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { - withTempPath { path => - Seq(0).toDF("a").write.parquet(path.toString) - // The vectorized and non-vectorized readers will produce different exceptions, we don't need -@@ -1148,7 +1154,7 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -1148,7 +1151,7 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS .where(s"a < ${Long.MaxValue}") .collect() } @@ -2414,14 +2276,14 @@ index 5c0b7def039..151184bc98c 100644 assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala -index 3f47c5e506f..92a5eafec84 100644 +index 3f47c5e506f..bc1ee1ec0ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -27,6 +27,7 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName import org.apache.parquet.schema.Type._ import org.apache.spark.SparkException -+import org.apache.spark.sql.{IgnoreComet, IgnoreCometNativeDataFusion} ++import org.apache.spark.sql.IgnoreComet import org.apache.spark.sql.catalyst.expressions.Cast.toSQLType import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException import org.apache.spark.sql.functions.desc @@ -2435,26 +2297,6 @@ index 3f47c5e506f..92a5eafec84 100644 withTempPath { dir => val e = testSchemaMismatch(dir.getCanonicalPath, vectorizedReaderEnabled = false) val expectedMessage = "Encountered error while reading file" -@@ -1046,7 +1048,8 @@ class ParquetSchemaSuite extends ParquetSchemaTest { - } - } - -- test("schema mismatch failure error message for parquet vectorized reader") { -+ test("schema mismatch failure error message for parquet vectorized reader", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { - withTempPath { dir => - val e = testSchemaMismatch(dir.getCanonicalPath, vectorizedReaderEnabled = true) - assert(e.getCause.isInstanceOf[SparkException]) -@@ -1087,7 +1090,8 @@ class ParquetSchemaSuite extends ParquetSchemaTest { - } - } - -- test("SPARK-45604: schema mismatch failure error on timestamp_ntz to array") { -+ test("SPARK-45604: schema mismatch failure error on timestamp_ntz to array", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { - import testImplicits._ - - withTempPath { dir => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala index b8f3ea3c6f3..bbd44221288 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala @@ -2504,30 +2346,18 @@ index 5cdbdc27b32..307fba16578 100644 spark.range(10).selectExpr("id", "id % 3 as p") .write.partitionBy("p").saveAsTable("testDataForScan") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala -index 0ab8691801d..7b81f3a8f6d 100644 +index 0ab8691801d..d9125f658ad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala -@@ -17,7 +17,9 @@ - +@@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.python -+import org.apache.spark.sql.IgnoreCometNativeDataFusion import org.apache.spark.sql.catalyst.plans.logical.{ArrowEvalPython, BatchEvalPython, Limit, LocalLimit} +import org.apache.spark.sql.comet._ import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan, SparkPlanTest} import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan -@@ -93,7 +95,8 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { - assert(arrowEvalNodes.size == 2) - } - -- test("Python UDF should not break column pruning/filter pushdown -- Parquet V1") { -+ test("Python UDF should not break column pruning/filter pushdown -- Parquet V1", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3312")) { - withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "parquet") { - withTempPath { f => - spark.range(10).select($"id".as("a"), $"id".as("b")) -@@ -108,6 +111,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { +@@ -108,6 +109,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { val scanNodes = query.queryExecution.executedPlan.collect { case scan: FileSourceScanExec => scan @@ -2535,7 +2365,7 @@ index 0ab8691801d..7b81f3a8f6d 100644 } assert(scanNodes.length == 1) assert(scanNodes.head.output.map(_.name) == Seq("a")) -@@ -120,11 +124,16 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { +@@ -120,11 +122,16 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { val scanNodes = query.queryExecution.executedPlan.collect { case scan: FileSourceScanExec => scan @@ -2554,7 +2384,7 @@ index 0ab8691801d..7b81f3a8f6d 100644 } } } -@@ -145,6 +154,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { +@@ -145,6 +152,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { val scanNodes = query.queryExecution.executedPlan.collect { case scan: BatchScanExec => scan @@ -2562,7 +2392,7 @@ index 0ab8691801d..7b81f3a8f6d 100644 } assert(scanNodes.length == 1) assert(scanNodes.head.output.map(_.name) == Seq("a")) -@@ -157,6 +167,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { +@@ -157,6 +165,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { val scanNodes = query.queryExecution.executedPlan.collect { case scan: BatchScanExec => scan @@ -2587,7 +2417,7 @@ index d083cac48ff..3c11bcde807 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -index 746f289c393..7a6a88a9fce 100644 +index 746f289c393..a773971d3c1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -19,16 +19,19 @@ package org.apache.spark.sql.sources @@ -2612,7 +2442,7 @@ index 746f289c393..7a6a88a9fce 100644 import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -@@ -102,12 +105,22 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -102,12 +105,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } } @@ -2622,7 +2452,6 @@ index 746f289c393..7a6a88a9fce 100644 + val fileScan = collect(plan) { + case f: FileSourceScanExec => f + case f: CometScanExec => f -+ case f: CometNativeScanExec => f + } assert(fileScan.nonEmpty, plan) fileScan.head @@ -2631,13 +2460,12 @@ index 746f289c393..7a6a88a9fce 100644 + private def getBucketScan(plan: SparkPlan): Boolean = getFileScan(plan) match { + case fs: FileSourceScanExec => fs.bucketedScan + case bs: CometScanExec => bs.bucketedScan -+ case ns: CometNativeScanExec => ns.bucketedScan + } + // To verify if the bucket pruning works, this function checks two conditions: // 1) Check if the pruned buckets (before filtering) are empty. // 2) Verify the final result is the same as the expected one -@@ -156,7 +169,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -156,7 +167,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val planWithoutBucketedScan = bucketedDataFrame.filter(filterCondition) .queryExecution.executedPlan val fileScan = getFileScan(planWithoutBucketedScan) @@ -2647,7 +2475,7 @@ index 746f289c393..7a6a88a9fce 100644 val bucketColumnType = bucketedDataFrame.schema.apply(bucketColumnIndex).dataType val rowsWithInvalidBuckets = fileScan.execute().filter(row => { -@@ -452,28 +466,54 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -452,28 +464,54 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val joinOperator = if (joined.sqlContext.conf.adaptiveExecutionEnabled) { val executedPlan = joined.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan @@ -2710,7 +2538,7 @@ index 746f289c393..7a6a88a9fce 100644 s"expected sort in the right child to be $sortRight but found\n${joinOperator.right}") // check the output partitioning -@@ -836,11 +876,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -836,11 +874,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") val scanDF = spark.table("bucketed_table").select("j") @@ -2724,7 +2552,7 @@ index 746f289c393..7a6a88a9fce 100644 checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) } } -@@ -895,7 +935,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -895,7 +933,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } test("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") { @@ -2735,7 +2563,7 @@ index 746f289c393..7a6a88a9fce 100644 SQLConf.SHUFFLE_PARTITIONS.key -> "5", SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "7") { val bucketSpec = Some(BucketSpec(6, Seq("i", "j"), Nil)) -@@ -914,7 +957,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -914,7 +955,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } test("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") { @@ -2746,7 +2574,7 @@ index 746f289c393..7a6a88a9fce 100644 SQLConf.SHUFFLE_PARTITIONS.key -> "9", SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10") { -@@ -944,7 +990,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -944,7 +988,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } test("bucket coalescing eliminates shuffle") { @@ -2757,7 +2585,7 @@ index 746f289c393..7a6a88a9fce 100644 SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. -@@ -1029,15 +1078,24 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -1029,15 +1076,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti Seq(true, false).foreach { aqeEnabled => withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled.toString) { val plan = sql(query).queryExecution.executedPlan @@ -2768,7 +2596,6 @@ index 746f289c393..7a6a88a9fce 100644 val scans = collect(plan) { case f: FileSourceScanExec if f.optionalNumCoalescedBuckets.isDefined => f + case b: CometScanExec if b.optionalNumCoalescedBuckets.isDefined => b -+ case b: CometNativeScanExec if b.optionalNumCoalescedBuckets.isDefined => b } if (expectedCoalescedNumBuckets.isDefined) { assert(scans.length == 1) @@ -2778,8 +2605,6 @@ index 746f289c393..7a6a88a9fce 100644 + assert(f.optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) + case b: CometScanExec => + assert(b.optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) -+ case b: CometNativeScanExec => -+ assert(b.optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) + } } else { assert(scans.isEmpty) @@ -2809,18 +2634,18 @@ index 6f897a9c0b7..b0723634f68 100644 protected override lazy val sql = spark.sql _ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala -index d675503a8ba..f220892396e 100644 +index d675503a8ba..659fa686fb7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.sources import org.apache.spark.sql.QueryTest -+import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec} ++import org.apache.spark.sql.comet.CometScanExec import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} import org.apache.spark.sql.internal.SQLConf -@@ -68,7 +69,11 @@ abstract class DisableUnnecessaryBucketedScanSuite +@@ -68,7 +69,10 @@ abstract class DisableUnnecessaryBucketedScanSuite def checkNumBucketedScan(query: String, expectedNumBucketedScan: Int): Unit = { val plan = sql(query).queryExecution.executedPlan @@ -2828,16 +2653,15 @@ index d675503a8ba..f220892396e 100644 + val bucketedScan = collect(plan) { + case s: FileSourceScanExec if s.bucketedScan => s + case s: CometScanExec if s.bucketedScan => s -+ case s: CometNativeScanExec if s.bucketedScan => s + } assert(bucketedScan.length == expectedNumBucketedScan) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala -index 7f6fa2a123e..c778b4e2c48 100644 +index 1954cce7fdc..73d1464780e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala -@@ -35,6 +35,7 @@ import org.apache.spark.paths.SparkPath +@@ -34,6 +34,7 @@ import org.apache.spark.paths.SparkPath import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} import org.apache.spark.sql.{AnalysisException, DataFrame} import org.apache.spark.sql.catalyst.util.stringToFile @@ -2845,7 +2669,7 @@ index 7f6fa2a123e..c778b4e2c48 100644 import org.apache.spark.sql.execution.DataSourceScanExec import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation, FileScan, FileTable} -@@ -777,6 +778,8 @@ class FileStreamSinkV2Suite extends FileStreamSinkSuite { +@@ -761,6 +762,8 @@ class FileStreamSinkV2Suite extends FileStreamSinkSuite { val fileScan = df.queryExecution.executedPlan.collect { case batch: BatchScanExec if batch.scan.isInstanceOf[FileScan] => batch.scan.asInstanceOf[FileScan] @@ -2967,72 +2791,6 @@ index aad91601758..201083bd621 100644 }) } -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala -index b5cf13a9c12..ac17603fb7f 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala -@@ -36,7 +36,7 @@ import org.scalatestplus.mockito.MockitoSugar - - import org.apache.spark.{SparkException, TestUtils} - import org.apache.spark.internal.Logging --import org.apache.spark.sql.{AnalysisException, Column, DataFrame, Dataset, Row, SaveMode} -+import org.apache.spark.sql.{AnalysisException, Column, DataFrame, Dataset, IgnoreCometNativeDataFusion, Row, SaveMode} - import org.apache.spark.sql.catalyst.InternalRow - import org.apache.spark.sql.catalyst.expressions.{Literal, Rand, Randn, Shuffle, Uuid} - import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, CTERelationRef, LocalRelation} -@@ -660,7 +660,8 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi - ) - } - -- test("SPARK-41198: input row calculation with CTE") { -+ test("SPARK-41198: input row calculation with CTE", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3315")) { - withTable("parquet_tbl", "parquet_streaming_tbl") { - spark.range(0, 10).selectExpr("id AS col1", "id AS col2") - .write.format("parquet").saveAsTable("parquet_tbl") -@@ -712,7 +713,8 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi - } - } - -- test("SPARK-41199: input row calculation with mixed-up of DSv1 and DSv2 streaming sources") { -+ test("SPARK-41199: input row calculation with mixed-up of DSv1 and DSv2 streaming sources", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3315")) { - withTable("parquet_streaming_tbl") { - val streamInput = MemoryStream[Int] - val streamDf = streamInput.toDF().selectExpr("value AS key", "value AS value_stream") -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSelfUnionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSelfUnionSuite.scala -index 8f099c31e6b..ce4b7ad25b3 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSelfUnionSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSelfUnionSuite.scala -@@ -20,7 +20,7 @@ package org.apache.spark.sql.streaming - import org.scalatest.BeforeAndAfter - import org.scalatest.concurrent.PatienceConfiguration.Timeout - --import org.apache.spark.sql.SaveMode -+import org.apache.spark.sql.{IgnoreCometNativeDataFusion, SaveMode} - import org.apache.spark.sql.connector.catalog.Identifier - import org.apache.spark.sql.execution.streaming.MemoryStream - import org.apache.spark.sql.streaming.test.{InMemoryStreamTable, InMemoryStreamTableCatalog} -@@ -42,7 +42,8 @@ class StreamingSelfUnionSuite extends StreamTest with BeforeAndAfter { - sqlContext.streams.active.foreach(_.stop()) - } - -- test("self-union, DSv1, read via DataStreamReader API") { -+ test("self-union, DSv1, read via DataStreamReader API", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3401")) { - withTempPath { dir => - val dataLocation = dir.getAbsolutePath - spark.range(1, 4).write.format("parquet").save(dataLocation) -@@ -66,7 +67,8 @@ class StreamingSelfUnionSuite extends StreamTest with BeforeAndAfter { - } - } - -- test("self-union, DSv1, read via table API") { -+ test("self-union, DSv1, read via table API", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3401")) { - withTable("parquet_streaming_tbl") { - spark.sql("CREATE TABLE parquet_streaming_tbl (key integer) USING parquet") - diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala index abe606ad9c1..2d930b64cca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala @@ -3057,7 +2815,7 @@ index abe606ad9c1..2d930b64cca 100644 val tblTargetName = "tbl_target" val tblSourceQualified = s"default.$tblSourceName" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala -index e937173a590..7d20538bc68 100644 +index e937173a590..5fede1579f7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -27,6 +27,7 @@ import scala.concurrent.duration._ @@ -3076,42 +2834,37 @@ index e937173a590..7d20538bc68 100644 import org.apache.spark.sql.execution.FilterExec import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecution import org.apache.spark.sql.execution.datasources.DataSourceUtils -@@ -119,6 +121,34 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with - - override protected def test(testName: String, testTags: Tag*)(testFun: => Any) - (implicit pos: Position): Unit = { -+ // Check Comet skip tags first, before DisableAdaptiveExecution handling -+ if (isCometEnabled && testTags.exists(_.isInstanceOf[IgnoreComet])) { -+ ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) -+ return -+ } -+ if (isCometEnabled) { -+ val cometScanImpl = CometConf.COMET_NATIVE_SCAN_IMPL.get(conf) -+ val isNativeIcebergCompat = cometScanImpl == CometConf.SCAN_NATIVE_ICEBERG_COMPAT || -+ cometScanImpl == CometConf.SCAN_AUTO -+ val isNativeDataFusion = cometScanImpl == CometConf.SCAN_NATIVE_DATAFUSION || -+ cometScanImpl == CometConf.SCAN_AUTO -+ if (isNativeIcebergCompat && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeIcebergCompat])) { -+ ignore(testName + " (disabled for NATIVE_ICEBERG_COMPAT)", testTags: _*)(testFun) -+ return -+ } -+ if (isNativeDataFusion && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeDataFusion])) { -+ ignore(testName + " (disabled for NATIVE_DATAFUSION)", testTags: _*)(testFun) -+ return -+ } -+ if ((isNativeDataFusion || isNativeIcebergCompat) && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeScan])) { -+ ignore(testName + " (disabled for NATIVE_DATAFUSION and NATIVE_ICEBERG_COMPAT)", -+ testTags: _*)(testFun) -+ return +@@ -126,7 +128,28 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with + } + } + } else { +- super.test(testName, testTags: _*)(testFun) ++ if (isCometEnabled && testTags.exists(_.isInstanceOf[IgnoreComet])) { ++ ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) ++ } else { ++ val cometScanImpl = CometConf.COMET_NATIVE_SCAN_IMPL.get(conf) ++ val isNativeIcebergCompat = cometScanImpl == CometConf.SCAN_NATIVE_ICEBERG_COMPAT || ++ cometScanImpl == CometConf.SCAN_AUTO ++ val isNativeDataFusion = cometScanImpl == CometConf.SCAN_NATIVE_DATAFUSION || ++ cometScanImpl == CometConf.SCAN_AUTO ++ if (isCometEnabled && isNativeIcebergCompat && ++ testTags.exists(_.isInstanceOf[IgnoreCometNativeIcebergCompat])) { ++ ignore(testName + " (disabled for NATIVE_ICEBERG_COMPAT)", testTags: _*)(testFun) ++ } else if (isCometEnabled && isNativeDataFusion && ++ testTags.exists(_.isInstanceOf[IgnoreCometNativeDataFusion])) { ++ ignore(testName + " (disabled for NATIVE_DATAFUSION)", testTags: _*)(testFun) ++ } else if (isCometEnabled && (isNativeDataFusion || isNativeIcebergCompat) && ++ testTags.exists(_.isInstanceOf[IgnoreCometNativeScan])) { ++ ignore(testName + " (disabled for NATIVE_DATAFUSION and NATIVE_ICEBERG_COMPAT)", ++ testTags: _*)(testFun) ++ } else { ++ super.test(testName, testTags: _*)(testFun) ++ } + } -+ } - if (testTags.exists(_.isInstanceOf[DisableAdaptiveExecution])) { - super.test(testName, testTags: _*) { - withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { -@@ -242,6 +272,29 @@ private[sql] trait SQLTestUtilsBase + } + } + +@@ -242,6 +265,39 @@ private[sql] trait SQLTestUtilsBase protected override def _sqlContext: SQLContext = self.spark.sqlContext } @@ -3137,11 +2890,21 @@ index e937173a590..7d20538bc68 100644 + val v = System.getenv("ENABLE_COMET_SCAN_ONLY") + v != null && v.toBoolean + } ++ ++ /** ++ * Whether Spark should apply comet writer is enabled. This is only effective when ++ * [[isCometEnabled]] returns true. ++ */ ++ ++ protected def isCometWriterEnabled: Boolean = { ++ val v = System.getenv("ENABLE_COMET_WRITER") ++ v != null && v.toBoolean ++ } + protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { SparkSession.setActiveSession(spark) super.withSQLConf(pairs: _*)(f) -@@ -435,6 +488,8 @@ private[sql] trait SQLTestUtilsBase +@@ -435,6 +491,8 @@ private[sql] trait SQLTestUtilsBase val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child @@ -3151,10 +2914,10 @@ index e937173a590..7d20538bc68 100644 spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -index ed2e309fa07..a5ea58146ad 100644 +index ed2e309fa07..9c5c393ad14 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -@@ -74,6 +74,31 @@ trait SharedSparkSessionBase +@@ -74,6 +74,36 @@ trait SharedSparkSessionBase // this rule may potentially block testing of other optimization rules such as // ConstantPropagation etc. .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) @@ -3182,6 +2945,11 @@ index ed2e309fa07..a5ea58146ad 100644 + conf + .set("spark.sql.ansi.enabled", "true") + } ++ ++ if (isCometWriterEnabled) { ++ conf.set("spark.comet.parquet.write.enabled", "true") ++ conf.set("spark.comet.operator.DataWritingCommandExec.allowIncompatible", "true") ++ } + } conf.set( StaticSQLConf.WAREHOUSE_PATH, @@ -3243,29 +3011,6 @@ index de3b1ffccf0..2a76d127093 100644 override def beforeEach(): Unit = { super.beforeEach() -diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala -index f3be79f9022..b4b1ea8dbc4 100644 ---- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala -+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala -@@ -34,7 +34,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn - import org.apache.hadoop.io.{LongWritable, Writable} - - import org.apache.spark.{SparkException, SparkFiles, TestUtils} --import org.apache.spark.sql.{AnalysisException, QueryTest, Row} -+import org.apache.spark.sql.{AnalysisException, IgnoreCometNativeDataFusion, QueryTest, Row} - import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode - import org.apache.spark.sql.catalyst.plans.logical.Project - import org.apache.spark.sql.execution.WholeStageCodegenExec -@@ -448,7 +448,8 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { - } - } - -- test("SPARK-11522 select input_file_name from non-parquet table") { -+ test("SPARK-11522 select input_file_name from non-parquet table", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3312")) { - - withTempDir { tempDir => - diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 6160c3e5f6c..0956d7d9edc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala diff --git a/dev/diffs/4.0.1.diff b/dev/diffs/4.0.1.diff index d6694e827f..c6af57b1bb 100644 --- a/dev/diffs/4.0.1.diff +++ b/dev/diffs/4.0.1.diff @@ -1,5 +1,5 @@ diff --git a/pom.xml b/pom.xml -index 22922143fc3..7c56e5e8641 100644 +index 443d46a4302..9880184c4e3 100644 --- a/pom.xml +++ b/pom.xml @@ -148,6 +148,8 @@ @@ -38,7 +38,7 @@ index 22922143fc3..7c56e5e8641 100644 org.apache.datasketches diff --git a/sql/core/pom.xml b/sql/core/pom.xml -index dcf6223a98b..0458a5bb640 100644 +index 56aad01023d..ad9d6a973e4 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -90,6 +90,10 @@ @@ -332,7 +332,7 @@ index 1f8c5822e7d..b7de4e28813 100644 WITH t(c1) AS (SELECT replace(listagg(DISTINCT col1 COLLATE unicode_rtrim) COLLATE utf8_binary, ' ', '') FROM (VALUES ('xbc '), ('xbc '), ('a'), ('xbc'))) SELECT len(c1), regexp_count(c1, 'a'), regexp_count(c1, 'xbc') FROM t -- !query schema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala -index 0f42502f1d9..f616024a9c2 100644 +index e0ad3feda3a..8fc1ba8e10d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeConstants @@ -380,7 +380,7 @@ index 0f42502f1d9..f616024a9c2 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 9db406ff12f..abbc91f5c11 100644 +index 6ce0a657d5b..02e352d13b0 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 @@ -30,7 +30,7 @@ import org.apache.spark.sql.errors.DataTypeErrors.toSQLId @@ -802,7 +802,7 @@ index 53e47f428c3..a55d8f0c161 100644 assert(shuffleMergeJoins.size == 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala -index aaac0ebc9aa..fbef0774d46 100644 +index 41f2e5c9a40..8f0dd909492 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 @@ -1000,7 +1000,7 @@ index aaac0ebc9aa..fbef0774d46 100644 checkAnswer(fullJoinDF, Row(100)) } } -@@ -1614,6 +1643,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 @@ -1010,7 +1010,7 @@ index aaac0ebc9aa..fbef0774d46 100644 }.size == 1) } } -@@ -1658,14 +1690,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 = { @@ -1033,7 +1033,7 @@ index aaac0ebc9aa..fbef0774d46 100644 } dupStreamSideColTest("SHUFFLE_HASH", check) } -@@ -1801,7 +1839,8 @@ class ThreadLeakInSortMergeJoinSuite +@@ -1773,7 +1811,8 @@ class ThreadLeakInSortMergeJoinSuite sparkConf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, 20)) } @@ -1057,7 +1057,7 @@ index ad424b3a7cc..4ece0117a34 100644 protected val baseResourcePath = { // use the same way as `SQLQueryTestSuite` to get the resource path diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala -index f294ff81021..8a3b818ee94 100644 +index b3fce19979e..67edf5eb91c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1524,7 +1524,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark @@ -1461,7 +1461,7 @@ index 2a0ab21ddb0..6030e7c2b9b 100644 } finally { spark.listenerManager.unregister(listener) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala -index c73e8e16fbb..88cd0d47da3 100644 +index c24f52bd930..ce4d4c4a694 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala @@ -20,10 +20,11 @@ import java.sql.Timestamp @@ -1538,7 +1538,7 @@ index f62e092138a..c0404bfe85e 100644 before { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala -index 04d33ecd3d5..450df347297 100644 +index 46f2502582c..1f4c8f2201b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -31,7 +31,7 @@ import org.mockito.Mockito.{mock, spy, when} @@ -1679,7 +1679,7 @@ index 1400ee25f43..5b016c3f9c5 100644 setupTestData() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala -index 47d5ff67b84..8dc8f65d4b1 100644 +index a6490243708..c61af08d4cf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -20,7 +20,7 @@ import scala.collection.mutable @@ -1691,7 +1691,7 @@ index 47d5ff67b84..8dc8f65d4b1 100644 import org.apache.spark.sql.catalyst.{QueryPlanningTracker, QueryPlanningTrackerCallback, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{CurrentNamespace, UnresolvedFunction, UnresolvedRelation} import org.apache.spark.sql.catalyst.expressions.{Alias, UnsafeRow} -@@ -400,7 +400,7 @@ class QueryExecutionSuite extends SharedSparkSession { +@@ -385,7 +385,7 @@ class QueryExecutionSuite extends SharedSparkSession { } } @@ -2734,10 +2734,10 @@ index 6080a5e8e4b..9aa8f49a62b 100644 case _ => assert(false, "Can not match ParquetTable in the query.") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -index 4474ec1fd42..97910c4fc3a 100644 +index 22839d3f0d2..812d642f15d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -@@ -1344,7 +1344,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession +@@ -1334,7 +1334,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession } } @@ -2747,6 +2747,41 @@ index 4474ec1fd42..97910c4fc3a 100644 withAllParquetReaders { checkAnswer( // "fruit" column in this file is encoded using DELTA_LENGTH_BYTE_ARRAY. +@@ -1543,7 +1544,9 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession + } + } + +- test("Write Spark version into Parquet metadata") { ++// TODO : Comet native writer to add spark / comet version into parquet metadata ++ test("Write Spark version into Parquet metadata", ++ IgnoreComet("Comet doesn't support DELTA encoding yet")) { + withTempPath { dir => + spark.range(1).repartition(1).write.parquet(dir.getAbsolutePath) + assert(getMetaData(dir)(SPARK_VERSION_METADATA_KEY) === SPARK_VERSION_SHORT) +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +index baa11df302b..c88f66d5c77 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +@@ -25,7 +25,7 @@ import org.apache.hadoop.fs.{Path, PathFilter} + import org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER + import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName + +-import org.apache.spark.sql.Row ++import org.apache.spark.sql.{IgnoreComet, Row} + import org.apache.spark.sql.catalyst.util.DateTimeUtils + import org.apache.spark.sql.internal.SQLConf + import org.apache.spark.sql.test.SharedSparkSession +@@ -153,7 +153,9 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS + } + } + +- test("parquet timestamp conversion") { ++// TODO : Support legacy timestamps conversion /cast in comet native writer ++ test("parquet timestamp conversion", ++ IgnoreComet("timestamp96 conversion failed with the native writer")) { + // Make a table with one parquet file written by impala, and one parquet file written by spark. + // We should only adjust the timestamps in the impala file, and only if the conf is set + val impalaFile = "test-data/impala_timestamp.parq" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index bba71f1c48d..38c60ee2584 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -3466,7 +3501,7 @@ index 86c4e49f6f6..2e639e5f38d 100644 val tblTargetName = "tbl_target" val tblSourceQualified = s"default.$tblSourceName" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala -index f0f3f94b811..d64e4e54e22 100644 +index f0f3f94b811..fd9744a58cf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -27,13 +27,14 @@ import scala.jdk.CollectionConverters._ @@ -3523,7 +3558,7 @@ index f0f3f94b811..d64e4e54e22 100644 } } -@@ -248,8 +271,33 @@ private[sql] trait SQLTestUtilsBase +@@ -248,8 +271,38 @@ private[sql] trait SQLTestUtilsBase override protected def converter: ColumnNodeToExpressionConverter = self.spark.converter } @@ -3549,6 +3584,11 @@ index f0f3f94b811..d64e4e54e22 100644 + val v = System.getenv("ENABLE_COMET_SCAN_ONLY") + v != null && v.toBoolean + } ++ ++ protected def isCometWriterEnabled: Boolean = { ++ val v = System.getenv("ENABLE_COMET_WRITER") ++ v != null && v.toBoolean ++ } + protected override def withSQLConf[T](pairs: (String, String)*)(f: => T): T = { SparkSession.setActiveSession(spark) @@ -3557,7 +3597,7 @@ index f0f3f94b811..d64e4e54e22 100644 super.withSQLConf(pairs: _*)(f) } -@@ -451,6 +499,8 @@ private[sql] trait SQLTestUtilsBase +@@ -451,6 +504,8 @@ private[sql] trait SQLTestUtilsBase val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child @@ -3567,10 +3607,10 @@ index f0f3f94b811..d64e4e54e22 100644 spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -index 245219c1756..7d2ef1b9145 100644 +index 245219c1756..4e7b86e55a2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -@@ -75,6 +75,31 @@ trait SharedSparkSessionBase +@@ -75,6 +75,37 @@ trait SharedSparkSessionBase // this rule may potentially block testing of other optimization rules such as // ConstantPropagation etc. .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) @@ -3598,6 +3638,12 @@ index 245219c1756..7d2ef1b9145 100644 + conf + .set("spark.sql.ansi.enabled", "true") + } ++ ++ if (isCometWriterEnabled) { ++ conf.set("spark.comet.parquet.write.enabled", "true") ++ conf.set("spark.comet.operator.DataWritingCommandExec.allowIncompatible", "true") ++ } ++ + } conf.set( StaticSQLConf.WAREHOUSE_PATH, From c053566c4fa701ff9c31d81688df65ca752a0388 Mon Sep 17 00:00:00 2001 From: B Vadlamani Date: Sat, 31 Jan 2026 15:41:05 -0800 Subject: [PATCH 02/15] enable_spark_tests_comet_native_writer --- dev/diffs/4.0.1.diff | 188 +++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 188 insertions(+) diff --git a/dev/diffs/4.0.1.diff b/dev/diffs/4.0.1.diff index c6af57b1bb..1260ee64a8 100644 --- a/dev/diffs/4.0.1.diff +++ b/dev/diffs/4.0.1.diff @@ -110,6 +110,20 @@ index 4410fe50912..43bcce2a038 100644 case _ => Map[String, String]() } val childrenInfo = children.flatMap { +diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out +index 524797015a2..09576b169bd 100644 +--- a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out ++++ b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out +@@ -738,3 +738,9 @@ SELECT to_varchar(12454.8, '99,999.9S') + -- !query analysis + Project [to_char(12454.8, 99,999.9S) AS to_char(12454.8, 99,999.9S)#x] + +- OneRowRelation ++ ++ ++-- !query ++SET spark.comet.parquet.write.enabled = true ++-- !query analysis ++SetCommand (spark.comet.parquet.write.enabled,Some(true)) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out index 7aca17dcb25..8afeb3b4a2f 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out @@ -130,6 +144,23 @@ index 7aca17dcb25..8afeb3b4a2f 100644 -- !query WITH t(c1) AS (SELECT replace(listagg(DISTINCT col1 COLLATE unicode_rtrim) COLLATE utf8_binary, ' ', '') FROM (VALUES ('xbc '), ('xbc '), ('a'), ('xbc'))) SELECT len(c1), regexp_count(c1, 'a'), regexp_count(c1, 'xbc') FROM t -- !query analysis +diff --git a/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql b/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql +index be038e1083c..b3de0ee2641 100644 +--- a/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql ++++ b/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql +@@ -1,3 +1,6 @@ ++-- TODO: support empty table write / CTAS in native parquet writer ++--SET spark.comet.parquet.write.enabled = false ++ + create table char_tbl(c char(5), v varchar(6)) using parquet; + desc formatted char_tbl; + desc formatted char_tbl c; +@@ -124,3 +127,5 @@ select chr(167), chr(247), chr(215); + SELECT to_varchar(78.12, '$99.99'); + SELECT to_varchar(111.11, '99.9'); + SELECT to_varchar(12454.8, '99,999.9S'); ++ ++SET spark.comet.parquet.write.enabled = true; diff --git a/sql/core/src/test/resources/sql-tests/inputs/collations.sql b/sql/core/src/test/resources/sql-tests/inputs/collations.sql index 17815ed5dde..baad440b1ce 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/collations.sql @@ -312,6 +343,22 @@ index 21a3ce1e122..f4762ab98f0 100644 SET spark.sql.ansi.enabled = false; -- In COMPENSATION views get invalidated if the type can't cast +diff --git a/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out b/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out +index 3026e463947..bf3f20906f1 100644 +--- a/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out ++++ b/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out +@@ -1254,3 +1254,11 @@ SELECT to_varchar(12454.8, '99,999.9S') + struct + -- !query output + 12,454.8+ ++ ++ ++-- !query ++SET spark.comet.parquet.write.enabled = true ++-- !query schema ++struct ++-- !query output ++spark.comet.parquet.write.enabled true diff --git a/sql/core/src/test/resources/sql-tests/results/listagg-collations.sql.out b/sql/core/src/test/resources/sql-tests/results/listagg-collations.sql.out index 1f8c5822e7d..b7de4e28813 100644 --- a/sql/core/src/test/resources/sql-tests/results/listagg-collations.sql.out @@ -2506,6 +2553,64 @@ index 272be70f9fe..06957694002 100644 }.isEmpty) assert(collect(initialExecutedPlan) { case i: InMemoryTableScanLike => i +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala +index a5bb3058bed..de51b21f21a 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala +@@ -18,7 +18,7 @@ + package org.apache.spark.sql.execution.command + + import org.apache.spark.SparkConf +-import org.apache.spark.sql.{AnalysisException, QueryTest, Row} ++import org.apache.spark.sql.{AnalysisException, IgnoreComet, QueryTest, Row} + import org.apache.spark.sql.catalyst.util.CharVarcharUtils + import org.apache.spark.sql.connector.catalog.InMemoryPartitionTableCatalog + import org.apache.spark.sql.internal.SQLConf +@@ -150,7 +150,9 @@ trait CharVarcharDDLTestBase extends QueryTest with SQLTestUtils { + } + } + +- test("SPARK-33901: ctas should should not change table's schema") { ++// TODO support CTAS in comet native parquet writer ++ test("SPARK-33901: ctas should should not change table's schema", ++ IgnoreComet("comet native writer does not support empty dir / table creation yet")) { + withTable("t1", "t2") { + sql(s"CREATE TABLE t1(i CHAR(5), c VARCHAR(4)) USING $format") + sql(s"CREATE TABLE t2 USING $format AS SELECT * FROM t1") +@@ -167,7 +169,8 @@ trait CharVarcharDDLTestBase extends QueryTest with SQLTestUtils { + } + } + +- test("SPARK-37160: CREATE TABLE AS SELECT with CHAR_AS_VARCHAR") { ++ test("SPARK-37160: CREATE TABLE AS SELECT with CHAR_AS_VARCHAR", ++ IgnoreComet("comet native writer does not support empty dir/table creation yet")) { + withTable("t1", "t2") { + sql(s"CREATE TABLE t1(col CHAR(5)) USING $format") + checkTableSchemaTypeStr("t1", Seq(Row("char(5)"))) +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala +index 343b59a311e..05f3cab13f9 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala +@@ -17,7 +17,7 @@ + + package org.apache.spark.sql.execution.datasources + +-import org.apache.spark.sql.{QueryTest, Row} ++import org.apache.spark.sql.{IgnoreComet, QueryTest, Row} + import org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest + import org.apache.spark.sql.test.SharedSparkSession + +@@ -28,7 +28,9 @@ class FileFormatWriterSuite + + import testImplicits._ + +- test("empty file should be skipped while write to file") { ++// TODO :: comet native write does not write empty files / dirs ++ test("empty file should be skipped while write to file", ++ IgnoreComet("comet native writer does not create empty files / dir")) { + withTempPath { path => + spark.range(100).repartition(10).where("id = 50").write.parquet(path.toString) + val partFiles = path.listFiles() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index 0a0b23d1e60..5685926250f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -3320,6 +3425,62 @@ index c5c56f081d8..6cc51f93b4f 100644 assert(bucketedScan.length == expectedNumBucketedScan) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +index baf99798965..e23d8223e05 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +@@ -263,7 +263,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { + } + } + +- test("INSERT INTO TABLE - complex type but different names") { ++ test("INSERT INTO TABLE - complex type but different names", ++ IgnoreComet("comet native writer issue")) { + val tab1 = "tab1" + val tab2 = "tab2" + withTable(tab1, tab2) { +@@ -1002,7 +1003,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { + } + } + +- test("SPARK-38336 INSERT INTO statements with tables with default columns: positive tests") { ++ test("SPARK-38336 INSERT INTO statements with tables with default columns: positive tests", ++ IgnoreComet("comet native writer insert overwrite bug")) { + // When the INSERT INTO statement provides fewer values than expected, NULL values are appended + // in their place. + withTable("t") { +@@ -1517,7 +1519,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { + } + } + +- test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Positive tests") { ++ test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Positive tests", ++ IgnoreComet("bug with insert overwrite mode comet native writer")) { + // There is a complex expression in the default value. + val createTableBooleanCol = "create table t(i boolean) using parquet" + val createTableIntCol = "create table t(i int) using parquet" +@@ -2331,7 +2334,9 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { + } + } + +- test("SPARK-43071: INSERT INTO from queries whose final operators are not projections") { ++// TODO : fix overwrite mode issues comet native writer ++ test("SPARK-43071: INSERT INTO from queries whose final operators are not projections", ++ IgnoreComet("comet writer issues with insert overwrite")) { + def runTest(insert: String, expected: Seq[Row]): Unit = { + withTable("t1", "t2") { + sql("create table t1(i boolean, s bigint default 42) using parquet") +@@ -2399,7 +2404,9 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { + } + } + +- test("SPARK-29174 Support LOCAL in INSERT OVERWRITE DIRECTORY to data source") { ++// TODO : Insert overwrite not supported comet native writer ++ test("SPARK-29174 Support LOCAL in INSERT OVERWRITE DIRECTORY to data source", ++ IgnoreComet("comet native writer does not support insert overwrite")) { + withTempPath { dir => + val path = dir.toURI.getPath + sql(s"""create table tab1 ( a int) using parquet location '$path'""") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 9742a004545..4e0417d730a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -3500,6 +3661,33 @@ index 86c4e49f6f6..2e639e5f38d 100644 val tblSourceName = "tbl_src" val tblTargetName = "tbl_target" val tblSourceQualified = s"default.$tblSourceName" +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +index 300807cf058..8dab4f9a2af 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +@@ -620,7 +620,10 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with + Option(dir).map(spark.read.format("org.apache.spark.sql.test").load) + } + +- test("write path implements onTaskCommit API correctly") { ++ ++// TODO : fix test for native writer ++ test("write path implements onTaskCommit API correctly", ++ IgnoreComet("Comet native writer partial support")) { + withSQLConf( + SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> + classOf[MessageCapturingCommitProtocol].getCanonicalName) { +@@ -1184,7 +1187,9 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with + } + } + +- test("Insert overwrite table command should output correct schema: basic") { ++// TODO : support vairous write modes in comet native writer ++ test("Insert overwrite table command should output correct schema: basic", ++ IgnoreComet("Comet native writer does not supportinsert overwrite mode")) { + withTable("tbl", "tbl2") { + withView("view1") { + val df = spark.range(10).toDF("id") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index f0f3f94b811..fd9744a58cf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala From 0857d5b265e563fad45ef21f97b88426a0d854d1 Mon Sep 17 00:00:00 2001 From: B Vadlamani Date: Sat, 31 Jan 2026 17:43:32 -0800 Subject: [PATCH 03/15] enable_spark_tests_comet_native_writer --- dev/diffs/3.4.3.diff | 134 +++++++++++++++++++++++++++++++++++++++++++ dev/diffs/3.5.8.diff | 134 +++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 268 insertions(+) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 5c2f49b437..1b7d03e277 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -1872,6 +1872,62 @@ index 593bd7bb4ba..32af28b0238 100644 } assert(shuffles2.size == 4) val smj2 = findTopLevelSortMergeJoin(adaptive2) +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala +index f77b6336b81..b703603d26b 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala +@@ -18,7 +18,7 @@ + package org.apache.spark.sql.execution.command + + import org.apache.spark.SparkConf +-import org.apache.spark.sql.{AnalysisException, QueryTest, Row} ++import org.apache.spark.sql.{AnalysisException, IgnoreComet, QueryTest, Row} + import org.apache.spark.sql.catalyst.util.CharVarcharUtils + import org.apache.spark.sql.connector.catalog.InMemoryPartitionTableCatalog + import org.apache.spark.sql.internal.SQLConf +@@ -112,7 +112,8 @@ trait CharVarcharDDLTestBase extends QueryTest with SQLTestUtils { + } + } + +- test("SPARK-33901: ctas should should not change table's schema") { ++ test("SPARK-33901: ctas should should not change table's schema", ++ IgnoreComet("comet native writer does not support empty dir / table creation yet")) { + withTable("t1", "t2") { + sql(s"CREATE TABLE t1(i CHAR(5), c VARCHAR(4)) USING $format") + sql(s"CREATE TABLE t2 USING $format AS SELECT * FROM t1") +@@ -129,7 +130,8 @@ trait CharVarcharDDLTestBase extends QueryTest with SQLTestUtils { + } + } + +- test("SPARK-37160: CREATE TABLE AS SELECT with CHAR_AS_VARCHAR") { ++ test("SPARK-37160: CREATE TABLE AS SELECT with CHAR_AS_VARCHAR", ++ IgnoreComet("comet native writer does not support empty dir/table creation yet")) { + withTable("t1", "t2") { + sql(s"CREATE TABLE t1(col CHAR(5)) USING $format") + checkTableSchemaTypeStr("t1", Seq(Row("char(5)"))) +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala +index 343b59a311e..9d5789c1d91 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala +@@ -17,7 +17,7 @@ + + package org.apache.spark.sql.execution.datasources + +-import org.apache.spark.sql.{QueryTest, Row} ++import org.apache.spark.sql.{IgnoreComet, QueryTest, Row} + import org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest + import org.apache.spark.sql.test.SharedSparkSession + +@@ -28,7 +28,8 @@ class FileFormatWriterSuite + + import testImplicits._ + +- test("empty file should be skipped while write to file") { ++ test("empty file should be skipped while write to file", ++ IgnoreComet("comet native writer does not create empty files / dir")) { + withTempPath { path => + spark.range(100).repartition(10).where("id = 50").write.parquet(path.toString) + val partFiles = path.listFiles() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index bd9c79e5b96..2ada8c28842 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -2668,6 +2724,60 @@ index 1f55742cd67..f20129d9dd8 100644 assert(bucketedScan.length == expectedNumBucketedScan) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +index 2207661478d..dc4e4b4240c 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +@@ -237,7 +237,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { + } + } + +- test("INSERT INTO TABLE - complex type but different names") { ++ test("INSERT INTO TABLE - complex type but different names", ++ IgnoreComet("comet native writer issue")) { + val tab1 = "tab1" + val tab2 = "tab2" + withTable(tab1, tab2) { +@@ -889,7 +890,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { + assert(message.contains("target table has 2 column(s) but the inserted data has 1 column(s)")) + } + +- test("SPARK-38336 INSERT INTO statements with tables with default columns: positive tests") { ++ test("SPARK-38336 INSERT INTO statements with tables with default columns: positive tests", ++ IgnoreComet("comet native writer insert overwrite bug")) { + // When the USE_NULLS_FOR_MISSING_DEFAULT_COLUMN_VALUES configuration is enabled, and no + // explicit DEFAULT value is available when the INSERT INTO statement provides fewer + // values than expected, NULL values are appended in their place. +@@ -1286,7 +1288,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { + } + } + +- test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Positive tests") { ++ test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Positive tests", ++ IgnoreComet("bug with insert overwrite mode comet native writer")) { + // There is a complex expression in the default value. + val createTableBooleanCol = "create table t(i boolean) using parquet" + val createTableIntCol = "create table t(i int) using parquet" +@@ -1984,7 +1987,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { + } + } + +- test("SPARK-43071: INSERT INTO from queries whose final operators are not projections") { ++ test("SPARK-43071: INSERT INTO from queries whose final operators are not projections", ++ IgnoreComet("comet writer issues with insert overwrite")) { + def runTest(insert: String, expected: Seq[Row]): Unit = { + withTable("t1", "t2") { + sql("create table t1(i boolean, s bigint default 42) using parquet") +@@ -2052,7 +2056,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { + } + } + +- test("SPARK-29174 Support LOCAL in INSERT OVERWRITE DIRECTORY to data source") { ++ test("SPARK-29174 Support LOCAL in INSERT OVERWRITE DIRECTORY to data source", ++ IgnoreComet("comet native writer does not support insert overwrite")) { + withTempPath { dir => + val path = dir.toURI.getPath + sql(s"""create table tab1 ( a int) using parquet location '$path'""") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 75f440caefc..36b1146bc3a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -2838,6 +2948,30 @@ index abe606ad9c1..2d930b64cca 100644 val tblSourceName = "tbl_src" val tblTargetName = "tbl_target" val tblSourceQualified = s"default.$tblSourceName" +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +index 44c9fbadfac..5f98bb9be17 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +@@ -519,7 +519,8 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with + Option(dir).map(spark.read.format("org.apache.spark.sql.test").load) + } + +- test("write path implements onTaskCommit API correctly") { ++ test("write path implements onTaskCommit API correctly", ++ IgnoreComet("Comet native writer partial support")) { + withSQLConf( + SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> + classOf[MessageCapturingCommitProtocol].getCanonicalName) { +@@ -1069,7 +1070,8 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with + } + } + +- test("Insert overwrite table command should output correct schema: basic") { ++ test("Insert overwrite table command should output correct schema: basic", ++ IgnoreComet("Comet native writer does not supportinsert overwrite mode")) { + withTable("tbl", "tbl2") { + withView("view1") { + val df = spark.range(10).toDF("id") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index dd55fcfe42c..e898fc33bab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index 682b8b3772..1b25c6c914 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -1876,6 +1876,62 @@ index 2f8e401e743..a4f94417dcc 100644 plan.inputPlan.output.zip(plan.finalPhysicalPlan.output).foreach { case (o1, o2) => assert(o1.semanticEquals(o2), "Different output column order after AQE optimization") } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala +index 12d5870309f..51429bf4039 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala +@@ -18,7 +18,7 @@ + package org.apache.spark.sql.execution.command + + import org.apache.spark.SparkConf +-import org.apache.spark.sql.{AnalysisException, QueryTest, Row} ++import org.apache.spark.sql.{AnalysisException, IgnoreComet, QueryTest, Row} + import org.apache.spark.sql.catalyst.util.CharVarcharUtils + import org.apache.spark.sql.connector.catalog.InMemoryPartitionTableCatalog + import org.apache.spark.sql.internal.SQLConf +@@ -162,7 +162,8 @@ trait CharVarcharDDLTestBase extends QueryTest with SQLTestUtils { + } + } + +- test("SPARK-33901: ctas should should not change table's schema") { ++ test("SPARK-33901: ctas should should not change table's schema", ++ IgnoreComet("comet native writer does not support empty dir / table creation yet")) { + withTable("t1", "t2") { + sql(s"CREATE TABLE t1(i CHAR(5), c VARCHAR(4)) USING $format") + sql(s"CREATE TABLE t2 USING $format AS SELECT * FROM t1") +@@ -179,7 +180,8 @@ trait CharVarcharDDLTestBase extends QueryTest with SQLTestUtils { + } + } + +- test("SPARK-37160: CREATE TABLE AS SELECT with CHAR_AS_VARCHAR") { ++ test("SPARK-37160: CREATE TABLE AS SELECT with CHAR_AS_VARCHAR", ++ IgnoreComet("comet native writer does not support empty dir/table creation yet")) { + withTable("t1", "t2") { + sql(s"CREATE TABLE t1(col CHAR(5)) USING $format") + checkTableSchemaTypeStr("t1", Seq(Row("char(5)"))) +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala +index 343b59a311e..9d5789c1d91 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala +@@ -17,7 +17,7 @@ + + package org.apache.spark.sql.execution.datasources + +-import org.apache.spark.sql.{QueryTest, Row} ++import org.apache.spark.sql.{IgnoreComet, QueryTest, Row} + import org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest + import org.apache.spark.sql.test.SharedSparkSession + +@@ -28,7 +28,8 @@ class FileFormatWriterSuite + + import testImplicits._ + +- test("empty file should be skipped while write to file") { ++ test("empty file should be skipped while write to file", ++ IgnoreComet("comet native writer does not create empty files / dir")) { + withTempPath { path => + spark.range(100).repartition(10).where("id = 50").write.parquet(path.toString) + val partFiles = path.listFiles() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index bf496d6db21..9bb57a9b4c6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -2657,6 +2713,60 @@ index d675503a8ba..659fa686fb7 100644 assert(bucketedScan.length == expectedNumBucketedScan) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +index 7b1a5a32037..f163a9733a5 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +@@ -262,7 +262,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { + } + } + +- test("INSERT INTO TABLE - complex type but different names") { ++ test("INSERT INTO TABLE - complex type but different names", ++ IgnoreComet("comet native writer issue")) { + val tab1 = "tab1" + val tab2 = "tab2" + withTable(tab1, tab2) { +@@ -997,7 +998,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { + } + } + +- test("SPARK-38336 INSERT INTO statements with tables with default columns: positive tests") { ++ test("SPARK-38336 INSERT INTO statements with tables with default columns: positive tests", ++ IgnoreComet("comet native writer insert overwrite bug")) { + // When the INSERT INTO statement provides fewer values than expected, NULL values are appended + // in their place. + withTable("t") { +@@ -1470,7 +1472,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { + } + } + +- test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Positive tests") { ++ test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Positive tests", ++ IgnoreComet("bug with insert overwrite mode comet native writer")) { + // There is a complex expression in the default value. + val createTableBooleanCol = "create table t(i boolean) using parquet" + val createTableIntCol = "create table t(i int) using parquet" +@@ -2263,7 +2266,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { + } + } + +- test("SPARK-43071: INSERT INTO from queries whose final operators are not projections") { ++ test("SPARK-43071: INSERT INTO from queries whose final operators are not projections", ++ IgnoreComet("comet writer issues with insert overwrite")) { + def runTest(insert: String, expected: Seq[Row]): Unit = { + withTable("t1", "t2") { + sql("create table t1(i boolean, s bigint default 42) using parquet") +@@ -2331,7 +2335,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { + } + } + +- test("SPARK-29174 Support LOCAL in INSERT OVERWRITE DIRECTORY to data source") { ++ test("SPARK-29174 Support LOCAL in INSERT OVERWRITE DIRECTORY to data source", ++ IgnoreComet("comet native writer does not support insert overwrite")) { + withTempPath { dir => + val path = dir.toURI.getPath + sql(s"""create table tab1 ( a int) using parquet location '$path'""") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 1954cce7fdc..73d1464780e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -2814,6 +2924,30 @@ index abe606ad9c1..2d930b64cca 100644 val tblSourceName = "tbl_src" val tblTargetName = "tbl_target" val tblSourceQualified = s"default.$tblSourceName" +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +index b40f9210a68..34aa6287eb5 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +@@ -538,7 +538,8 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with + Option(dir).map(spark.read.format("org.apache.spark.sql.test").load) + } + +- test("write path implements onTaskCommit API correctly") { ++ test("write path implements onTaskCommit API correctly", ++ IgnoreComet("Comet native writer partial support")) { + withSQLConf( + SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> + classOf[MessageCapturingCommitProtocol].getCanonicalName) { +@@ -1088,7 +1089,8 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with + } + } + +- test("Insert overwrite table command should output correct schema: basic") { ++ test("Insert overwrite table command should output correct schema: basic", ++ IgnoreComet("Comet native writer does not supportinsert overwrite mode")) { + withTable("tbl", "tbl2") { + withView("view1") { + val df = spark.range(10).toDF("id") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index e937173a590..5fede1579f7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala From 6af021676381bba350a3e26b165f1af1313e03f9 Mon Sep 17 00:00:00 2001 From: B Vadlamani Date: Sun, 1 Feb 2026 01:29:50 -0800 Subject: [PATCH 04/15] enable_spark_tests_comet_native_writer --- dev/diffs/3.4.3.diff | 11 ++++++++ dev/diffs/3.5.8.diff | 11 ++++++++ dev/diffs/4.0.1.diff | 60 +++++++++++++++++--------------------------- 3 files changed, 45 insertions(+), 37 deletions(-) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 1b7d03e277..9c98301237 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -133,6 +133,17 @@ index db587dd9868..aac7295a53d 100644 case _ => Map[String, String]() } new SparkPlanInfo( +diff --git a/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql b/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql +index b62cbf64323..8d1f0cb7d20 100644 +--- a/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql ++++ b/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql +@@ -1,3 +1,6 @@ ++-- TODO: support empty table write / CTAS in native parquet writer ++--SET spark.comet.parquet.write.enabled = false ++ + create table char_tbl(c char(5), v varchar(6)) using parquet; + desc formatted char_tbl; + desc formatted char_tbl c; diff --git a/sql/core/src/test/resources/sql-tests/inputs/explain-aqe.sql b/sql/core/src/test/resources/sql-tests/inputs/explain-aqe.sql index 7aef901da4f..f3d6e18926d 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/explain-aqe.sql diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index 1b25c6c914..e17037c9a6 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -112,6 +112,17 @@ index db587dd9868..aac7295a53d 100644 case _ => Map[String, String]() } new SparkPlanInfo( +diff --git a/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql b/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql +index be038e1083c..58b512a3c17 100644 +--- a/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql ++++ b/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql +@@ -1,3 +1,6 @@ ++-- TODO: support empty table write / CTAS in native parquet writer ++--SET spark.comet.parquet.write.enabled = false ++ + create table char_tbl(c char(5), v varchar(6)) using parquet; + desc formatted char_tbl; + desc formatted char_tbl c; diff --git a/sql/core/src/test/resources/sql-tests/inputs/explain-aqe.sql b/sql/core/src/test/resources/sql-tests/inputs/explain-aqe.sql index 7aef901da4f..f3d6e18926d 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/explain-aqe.sql diff --git a/dev/diffs/4.0.1.diff b/dev/diffs/4.0.1.diff index 1260ee64a8..e53f3f5814 100644 --- a/dev/diffs/4.0.1.diff +++ b/dev/diffs/4.0.1.diff @@ -110,20 +110,6 @@ index 4410fe50912..43bcce2a038 100644 case _ => Map[String, String]() } val childrenInfo = children.flatMap { -diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out -index 524797015a2..09576b169bd 100644 ---- a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out -+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out -@@ -738,3 +738,9 @@ SELECT to_varchar(12454.8, '99,999.9S') - -- !query analysis - Project [to_char(12454.8, 99,999.9S) AS to_char(12454.8, 99,999.9S)#x] - +- OneRowRelation -+ -+ -+-- !query -+SET spark.comet.parquet.write.enabled = true -+-- !query analysis -+SetCommand (spark.comet.parquet.write.enabled,Some(true)) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out index 7aca17dcb25..8afeb3b4a2f 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out @@ -145,7 +131,7 @@ index 7aca17dcb25..8afeb3b4a2f 100644 WITH t(c1) AS (SELECT replace(listagg(DISTINCT col1 COLLATE unicode_rtrim) COLLATE utf8_binary, ' ', '') FROM (VALUES ('xbc '), ('xbc '), ('a'), ('xbc'))) SELECT len(c1), regexp_count(c1, 'a'), regexp_count(c1, 'xbc') FROM t -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql b/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql -index be038e1083c..b3de0ee2641 100644 +index be038e1083c..58b512a3c17 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql @@ -1,3 +1,6 @@ @@ -155,12 +141,6 @@ index be038e1083c..b3de0ee2641 100644 create table char_tbl(c char(5), v varchar(6)) using parquet; desc formatted char_tbl; desc formatted char_tbl c; -@@ -124,3 +127,5 @@ select chr(167), chr(247), chr(215); - SELECT to_varchar(78.12, '$99.99'); - SELECT to_varchar(111.11, '99.9'); - SELECT to_varchar(12454.8, '99,999.9S'); -+ -+SET spark.comet.parquet.write.enabled = true; diff --git a/sql/core/src/test/resources/sql-tests/inputs/collations.sql b/sql/core/src/test/resources/sql-tests/inputs/collations.sql index 17815ed5dde..baad440b1ce 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/collations.sql @@ -343,22 +323,6 @@ index 21a3ce1e122..f4762ab98f0 100644 SET spark.sql.ansi.enabled = false; -- In COMPENSATION views get invalidated if the type can't cast -diff --git a/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out b/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out -index 3026e463947..bf3f20906f1 100644 ---- a/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out -+++ b/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out -@@ -1254,3 +1254,11 @@ SELECT to_varchar(12454.8, '99,999.9S') - struct - -- !query output - 12,454.8+ -+ -+ -+-- !query -+SET spark.comet.parquet.write.enabled = true -+-- !query schema -+struct -+-- !query output -+spark.comet.parquet.write.enabled true diff --git a/sql/core/src/test/resources/sql-tests/results/listagg-collations.sql.out b/sql/core/src/test/resources/sql-tests/results/listagg-collations.sql.out index 1f8c5822e7d..b7de4e28813 100644 --- a/sql/core/src/test/resources/sql-tests/results/listagg-collations.sql.out @@ -1419,6 +1383,28 @@ index 11e9547dfc5..be9ae40ab3d 100644 val t1 = "T_1" val t2 = "T_2" +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala +index 031af8200d7..a56a1223a7d 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala +@@ -17,7 +17,7 @@ + + package org.apache.spark.sql.collation + +-import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} ++import org.apache.spark.sql.{AnalysisException, DataFrame, IgnoreComet, QueryTest, Row} + import org.apache.spark.sql.catalyst.util.CollationFactory + import org.apache.spark.sql.connector.DatasourceV2SQLBase + import org.apache.spark.sql.test.SharedSparkSession +@@ -184,7 +184,7 @@ abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSessi + } + } + +- test("ctas with union") { ++ test("ctas with union", IgnoreComet("comet native writer does not support CTAS")) { + withTable(testTable) { + sql(s""" + |CREATE TABLE $testTable USING $dataSource AS diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index 3eeed2e4175..9f21d547c1c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala From b1149ee59584205fc854868cde7bf960a36189dc Mon Sep 17 00:00:00 2001 From: B Vadlamani Date: Sun, 1 Feb 2026 12:11:48 -0800 Subject: [PATCH 05/15] enable_spark_tests_comet_native_writer_fix_spark_4 --- dev/diffs/4.0.1.diff | 47 ++++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 45 insertions(+), 2 deletions(-) diff --git a/dev/diffs/4.0.1.diff b/dev/diffs/4.0.1.diff index e53f3f5814..ba810b6ecb 100644 --- a/dev/diffs/4.0.1.diff +++ b/dev/diffs/4.0.1.diff @@ -110,6 +110,20 @@ index 4410fe50912..43bcce2a038 100644 case _ => Map[String, String]() } val childrenInfo = children.flatMap { +diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out +index 524797015a2..09576b169bd 100644 +--- a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out ++++ b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out +@@ -738,3 +738,9 @@ SELECT to_varchar(12454.8, '99,999.9S') + -- !query analysis + Project [to_char(12454.8, 99,999.9S) AS to_char(12454.8, 99,999.9S)#x] + +- OneRowRelation ++ ++ ++-- !query ++SET spark.comet.parquet.write.enabled = true ++-- !query analysis ++SetCommand (spark.comet.parquet.write.enabled,Some(true)) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out index 7aca17dcb25..8afeb3b4a2f 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out @@ -323,6 +337,22 @@ index 21a3ce1e122..f4762ab98f0 100644 SET spark.sql.ansi.enabled = false; -- In COMPENSATION views get invalidated if the type can't cast +diff --git a/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out b/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out +index 3026e463947..bf3f20906f1 100644 +--- a/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out ++++ b/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out +@@ -1254,3 +1254,11 @@ SELECT to_varchar(12454.8, '99,999.9S') + struct + -- !query output + 12,454.8+ ++ ++ ++-- !query ++SET spark.comet.parquet.write.enabled = true ++-- !query schema ++struct ++-- !query output ++spark.comet.parquet.write.enabled true diff --git a/sql/core/src/test/resources/sql-tests/results/listagg-collations.sql.out b/sql/core/src/test/resources/sql-tests/results/listagg-collations.sql.out index 1f8c5822e7d..b7de4e28813 100644 --- a/sql/core/src/test/resources/sql-tests/results/listagg-collations.sql.out @@ -1067,6 +1097,19 @@ index ad424b3a7cc..4ece0117a34 100644 protected val baseResourcePath = { // use the same way as `SQLQueryTestSuite` to get the resource path +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala +index 818c7ec65b1..0d333807b70 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala +@@ -524,7 +524,7 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils with AdaptiveSparkP + } + } + +- test("SPARK-48817: test multi inserts") { ++ test("SPARK-48817: test multi inserts", IgnoreComet("comet parquet native writer bug")) { + withTable("t1", "t2", "t3") { + createTable("t1", Seq("i"), Seq("int")) + createTable("t2", Seq("i"), Seq("int")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index b3fce19979e..67edf5eb91c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1384,7 +1427,7 @@ index 11e9547dfc5..be9ae40ab3d 100644 val t2 = "T_2" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala -index 031af8200d7..a56a1223a7d 100644 +index 031af8200d7..bb5acd74ad2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala @@ -17,7 +17,7 @@ @@ -1401,7 +1444,7 @@ index 031af8200d7..a56a1223a7d 100644 } - test("ctas with union") { -+ test("ctas with union", IgnoreComet("comet native writer does not support CTAS")) { ++ test("ctas with union", IgnoreComet("comet native writer bug when shuffle / AQE enabled")) { withTable(testTable) { sql(s""" |CREATE TABLE $testTable USING $dataSource AS From eb00abcc78151834653664c40333d0a6af1c44cb Mon Sep 17 00:00:00 2001 From: B Vadlamani Date: Sun, 1 Feb 2026 16:41:00 -0800 Subject: [PATCH 06/15] rebase_main --- dev/diffs/4.0.1.diff | 40 +++++++++++++++++++++------------------- 1 file changed, 21 insertions(+), 19 deletions(-) diff --git a/dev/diffs/4.0.1.diff b/dev/diffs/4.0.1.diff index ba810b6ecb..93e718c432 100644 --- a/dev/diffs/4.0.1.diff +++ b/dev/diffs/4.0.1.diff @@ -1,5 +1,5 @@ diff --git a/pom.xml b/pom.xml -index 443d46a4302..9880184c4e3 100644 +index 22922143fc3..7c56e5e8641 100644 --- a/pom.xml +++ b/pom.xml @@ -148,6 +148,8 @@ @@ -38,7 +38,7 @@ index 443d46a4302..9880184c4e3 100644 org.apache.datasketches diff --git a/sql/core/pom.xml b/sql/core/pom.xml -index 56aad01023d..ad9d6a973e4 100644 +index dcf6223a98b..0458a5bb640 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -90,6 +90,10 @@ @@ -373,7 +373,7 @@ index 1f8c5822e7d..b7de4e28813 100644 WITH t(c1) AS (SELECT replace(listagg(DISTINCT col1 COLLATE unicode_rtrim) COLLATE utf8_binary, ' ', '') FROM (VALUES ('xbc '), ('xbc '), ('a'), ('xbc'))) SELECT len(c1), regexp_count(c1, 'a'), regexp_count(c1, 'xbc') FROM t -- !query schema diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala -index e0ad3feda3a..8fc1ba8e10d 100644 +index 0f42502f1d9..f616024a9c2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeConstants @@ -421,7 +421,7 @@ index e0ad3feda3a..8fc1ba8e10d 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 6ce0a657d5b..02e352d13b0 100644 +index 9db406ff12f..abbc91f5c11 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 @@ -30,7 +30,7 @@ import org.apache.spark.sql.errors.DataTypeErrors.toSQLId @@ -843,7 +843,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 41f2e5c9a40..8f0dd909492 100644 +index aaac0ebc9aa..fbef0774d46 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -29,7 +29,8 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation @@ -1041,7 +1041,7 @@ index 41f2e5c9a40..8f0dd909492 100644 checkAnswer(fullJoinDF, Row(100)) } } -@@ -1586,6 +1615,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1614,6 +1643,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan Seq(semiJoinDF, antiJoinDF).foreach { df => assert(collect(df.queryExecution.executedPlan) { case j: ShuffledHashJoinExec if j.ignoreDuplicatedKey == ignoreDuplicatedKey => true @@ -1051,7 +1051,7 @@ index 41f2e5c9a40..8f0dd909492 100644 }.size == 1) } } -@@ -1630,14 +1662,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1658,14 +1690,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan test("SPARK-43113: Full outer join with duplicate stream-side references in condition (SMJ)") { def check(plan: SparkPlan): Unit = { @@ -1074,7 +1074,7 @@ index 41f2e5c9a40..8f0dd909492 100644 } dupStreamSideColTest("SHUFFLE_HASH", check) } -@@ -1773,7 +1811,8 @@ class ThreadLeakInSortMergeJoinSuite +@@ -1801,7 +1839,8 @@ class ThreadLeakInSortMergeJoinSuite sparkConf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, 20)) } @@ -1111,7 +1111,7 @@ index 818c7ec65b1..0d333807b70 100644 createTable("t1", Seq("i"), Seq("int")) createTable("t2", Seq("i"), Seq("int")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala -index b3fce19979e..67edf5eb91c 100644 +index f294ff81021..8a3b818ee94 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1524,7 +1524,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark @@ -1427,19 +1427,21 @@ index 11e9547dfc5..be9ae40ab3d 100644 val t2 = "T_2" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala -index 031af8200d7..bb5acd74ad2 100644 +index 77ed8b411e3..18b51f0c939 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala -@@ -17,7 +17,7 @@ +@@ -17,9 +17,9 @@ package org.apache.spark.sql.collation -import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} + import org.apache.spark.sql.catalyst.expressions.AttributeReference + import org.apache.spark.sql.catalyst.plans.logical.Project +import org.apache.spark.sql.{AnalysisException, DataFrame, IgnoreComet, QueryTest, Row} import org.apache.spark.sql.catalyst.util.CollationFactory import org.apache.spark.sql.connector.DatasourceV2SQLBase import org.apache.spark.sql.test.SharedSparkSession -@@ -184,7 +184,7 @@ abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSessi +@@ -220,7 +220,7 @@ abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSessi } } @@ -1537,7 +1539,7 @@ index 2a0ab21ddb0..6030e7c2b9b 100644 } finally { spark.listenerManager.unregister(listener) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala -index c24f52bd930..ce4d4c4a694 100644 +index c73e8e16fbb..88cd0d47da3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala @@ -20,10 +20,11 @@ import java.sql.Timestamp @@ -1614,7 +1616,7 @@ index f62e092138a..c0404bfe85e 100644 before { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala -index 46f2502582c..1f4c8f2201b 100644 +index 04d33ecd3d5..450df347297 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -31,7 +31,7 @@ import org.mockito.Mockito.{mock, spy, when} @@ -1755,7 +1757,7 @@ index 1400ee25f43..5b016c3f9c5 100644 setupTestData() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala -index a6490243708..c61af08d4cf 100644 +index 47d5ff67b84..8dc8f65d4b1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -20,7 +20,7 @@ import scala.collection.mutable @@ -1767,7 +1769,7 @@ index a6490243708..c61af08d4cf 100644 import org.apache.spark.sql.catalyst.{QueryPlanningTracker, QueryPlanningTrackerCallback, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{CurrentNamespace, UnresolvedFunction, UnresolvedRelation} import org.apache.spark.sql.catalyst.expressions.{Alias, UnsafeRow} -@@ -385,7 +385,7 @@ class QueryExecutionSuite extends SharedSparkSession { +@@ -400,7 +400,7 @@ class QueryExecutionSuite extends SharedSparkSession { } } @@ -2868,10 +2870,10 @@ index 6080a5e8e4b..9aa8f49a62b 100644 case _ => assert(false, "Can not match ParquetTable in the query.") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -index 22839d3f0d2..812d642f15d 100644 +index 4474ec1fd42..d0366be3597 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -@@ -1334,7 +1334,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession +@@ -1344,7 +1344,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession } } @@ -2881,7 +2883,7 @@ index 22839d3f0d2..812d642f15d 100644 withAllParquetReaders { checkAnswer( // "fruit" column in this file is encoded using DELTA_LENGTH_BYTE_ARRAY. -@@ -1543,7 +1544,9 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession +@@ -1553,7 +1554,9 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession } } From b4ed78a602e19c7d10da06e726c24a53b6ac732a Mon Sep 17 00:00:00 2001 From: B Vadlamani Date: Sun, 1 Feb 2026 16:55:33 -0800 Subject: [PATCH 07/15] rebase_main --- dev/diffs/4.0.1.diff | 89 ++------------------------------------------ 1 file changed, 3 insertions(+), 86 deletions(-) diff --git a/dev/diffs/4.0.1.diff b/dev/diffs/4.0.1.diff index 93e718c432..b10bca2841 100644 --- a/dev/diffs/4.0.1.diff +++ b/dev/diffs/4.0.1.diff @@ -110,20 +110,6 @@ index 4410fe50912..43bcce2a038 100644 case _ => Map[String, String]() } val childrenInfo = children.flatMap { -diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out -index 524797015a2..09576b169bd 100644 ---- a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out -+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out -@@ -738,3 +738,9 @@ SELECT to_varchar(12454.8, '99,999.9S') - -- !query analysis - Project [to_char(12454.8, 99,999.9S) AS to_char(12454.8, 99,999.9S)#x] - +- OneRowRelation -+ -+ -+-- !query -+SET spark.comet.parquet.write.enabled = true -+-- !query analysis -+SetCommand (spark.comet.parquet.write.enabled,Some(true)) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out index 7aca17dcb25..8afeb3b4a2f 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/listagg-collations.sql.out @@ -337,22 +323,6 @@ index 21a3ce1e122..f4762ab98f0 100644 SET spark.sql.ansi.enabled = false; -- In COMPENSATION views get invalidated if the type can't cast -diff --git a/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out b/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out -index 3026e463947..bf3f20906f1 100644 ---- a/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out -+++ b/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out -@@ -1254,3 +1254,11 @@ SELECT to_varchar(12454.8, '99,999.9S') - struct - -- !query output - 12,454.8+ -+ -+ -+-- !query -+SET spark.comet.parquet.write.enabled = true -+-- !query schema -+struct -+-- !query output -+spark.comet.parquet.write.enabled true diff --git a/sql/core/src/test/resources/sql-tests/results/listagg-collations.sql.out b/sql/core/src/test/resources/sql-tests/results/listagg-collations.sql.out index 1f8c5822e7d..b7de4e28813 100644 --- a/sql/core/src/test/resources/sql-tests/results/listagg-collations.sql.out @@ -739,57 +709,6 @@ index 9c529d14221..2f1bc3880fd 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 7d7185ae6c1..442a5bddeb8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala @@ -1427,20 +1346,18 @@ index 11e9547dfc5..be9ae40ab3d 100644 val t2 = "T_2" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala -index 77ed8b411e3..18b51f0c939 100644 +index 77ed8b411e3..b82f70c0071 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala -@@ -17,9 +17,9 @@ +@@ -17,7 +17,7 @@ package org.apache.spark.sql.collation -import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} ++import org.apache.spark.sql.{AnalysisException, DataFrame, IgnoreComet, QueryTest, Row} import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.plans.logical.Project -+import org.apache.spark.sql.{AnalysisException, DataFrame, IgnoreComet, QueryTest, Row} import org.apache.spark.sql.catalyst.util.CollationFactory - import org.apache.spark.sql.connector.DatasourceV2SQLBase - import org.apache.spark.sql.test.SharedSparkSession @@ -220,7 +220,7 @@ abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSessi } } From c8258027225fa17e54bea5ee448e40554228864d Mon Sep 17 00:00:00 2001 From: B Vadlamani Date: Mon, 2 Feb 2026 13:01:17 -0800 Subject: [PATCH 08/15] rebase_main --- dev/diffs/4.0.1.diff | 102 +++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 102 insertions(+) diff --git a/dev/diffs/4.0.1.diff b/dev/diffs/4.0.1.diff index b10bca2841..e44ecdd813 100644 --- a/dev/diffs/4.0.1.diff +++ b/dev/diffs/4.0.1.diff @@ -1,3 +1,54 @@ +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/pom.xml b/pom.xml index 22922143fc3..7c56e5e8641 100644 --- a/pom.xml @@ -709,6 +760,57 @@ index 9c529d14221..2f1bc3880fd 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 7d7185ae6c1..442a5bddeb8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/InjectRuntimeFilterSuite.scala From 06915ca7dfe2fcde86e1c68d6423122ce4d91436 Mon Sep 17 00:00:00 2001 From: B Vadlamani Date: Fri, 6 Feb 2026 01:02:01 -0800 Subject: [PATCH 09/15] enable_spark_tests_comet_native_writer_fix_spark_4 --- dev/diffs/3.4.3.diff | 24 ++++++++++++------------ dev/diffs/3.5.8.diff | 24 ++++++++++++------------ dev/diffs/4.0.1.diff | 28 ++++++++++++++-------------- 3 files changed, 38 insertions(+), 38 deletions(-) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 9c98301237..c524e6a94d 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -1902,7 +1902,7 @@ index f77b6336b81..b703603d26b 100644 - test("SPARK-33901: ctas should should not change table's schema") { + test("SPARK-33901: ctas should should not change table's schema", -+ IgnoreComet("comet native writer does not support empty dir / table creation yet")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3418")) { withTable("t1", "t2") { sql(s"CREATE TABLE t1(i CHAR(5), c VARCHAR(4)) USING $format") sql(s"CREATE TABLE t2 USING $format AS SELECT * FROM t1") @@ -1912,7 +1912,7 @@ index f77b6336b81..b703603d26b 100644 - test("SPARK-37160: CREATE TABLE AS SELECT with CHAR_AS_VARCHAR") { + test("SPARK-37160: CREATE TABLE AS SELECT with CHAR_AS_VARCHAR", -+ IgnoreComet("comet native writer does not support empty dir/table creation yet")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3419")) { withTable("t1", "t2") { sql(s"CREATE TABLE t1(col CHAR(5)) USING $format") checkTableSchemaTypeStr("t1", Seq(Row("char(5)"))) @@ -1935,7 +1935,7 @@ index 343b59a311e..9d5789c1d91 100644 - test("empty file should be skipped while write to file") { + test("empty file should be skipped while write to file", -+ IgnoreComet("comet native writer does not create empty files / dir")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3417")) { withTempPath { path => spark.range(100).repartition(10).where("id = 50").write.parquet(path.toString) val partFiles = path.listFiles() @@ -2179,7 +2179,7 @@ index 8670d95c65e..3fe49802309 100644 - test("Write Spark version into Parquet metadata") { +// TODO : Comet native writer to add spark / comet version into parquet metadata + test("Write Spark version into Parquet metadata", -+ IgnoreComet("Comet doesn't support DELTA encoding yet")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3427")) { withTempPath { dir => spark.range(1).repartition(1).write.parquet(dir.getAbsolutePath) assert(getMetaData(dir)(SPARK_VERSION_METADATA_KEY) === SPARK_VERSION_SHORT) @@ -2203,7 +2203,7 @@ index 8b386e8f689..28ced6209e0 100644 - test("parquet timestamp conversion") { + // TODO : Support legacy timestamps conversion /cast in comet native writer + test("parquet timestamp conversion", -+ IgnoreComet("timestamp96 conversion failed with the native writer")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3425")) { // Make a table with one parquet file written by impala, and one parquet file written by spark. // We should only adjust the timestamps in the impala file, and only if the conf is set val impalaFile = "test-data/impala_timestamp.parq" @@ -2745,7 +2745,7 @@ index 2207661478d..dc4e4b4240c 100644 - test("INSERT INTO TABLE - complex type but different names") { + test("INSERT INTO TABLE - complex type but different names", -+ IgnoreComet("comet native writer issue")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3426")) { val tab1 = "tab1" val tab2 = "tab2" withTable(tab1, tab2) { @@ -2755,7 +2755,7 @@ index 2207661478d..dc4e4b4240c 100644 - test("SPARK-38336 INSERT INTO statements with tables with default columns: positive tests") { + test("SPARK-38336 INSERT INTO statements with tables with default columns: positive tests", -+ IgnoreComet("comet native writer insert overwrite bug")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3421")) { // When the USE_NULLS_FOR_MISSING_DEFAULT_COLUMN_VALUES configuration is enabled, and no // explicit DEFAULT value is available when the INSERT INTO statement provides fewer // values than expected, NULL values are appended in their place. @@ -2765,7 +2765,7 @@ index 2207661478d..dc4e4b4240c 100644 - test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Positive tests") { + test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Positive tests", -+ IgnoreComet("bug with insert overwrite mode comet native writer")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3422")) { // There is a complex expression in the default value. val createTableBooleanCol = "create table t(i boolean) using parquet" val createTableIntCol = "create table t(i int) using parquet" @@ -2775,7 +2775,7 @@ index 2207661478d..dc4e4b4240c 100644 - test("SPARK-43071: INSERT INTO from queries whose final operators are not projections") { + test("SPARK-43071: INSERT INTO from queries whose final operators are not projections", -+ IgnoreComet("comet writer issues with insert overwrite")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3423")) { def runTest(insert: String, expected: Seq[Row]): Unit = { withTable("t1", "t2") { sql("create table t1(i boolean, s bigint default 42) using parquet") @@ -2785,7 +2785,7 @@ index 2207661478d..dc4e4b4240c 100644 - test("SPARK-29174 Support LOCAL in INSERT OVERWRITE DIRECTORY to data source") { + test("SPARK-29174 Support LOCAL in INSERT OVERWRITE DIRECTORY to data source", -+ IgnoreComet("comet native writer does not support insert overwrite")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3420")) { withTempPath { dir => val path = dir.toURI.getPath sql(s"""create table tab1 ( a int) using parquet location '$path'""") @@ -2969,7 +2969,7 @@ index 44c9fbadfac..5f98bb9be17 100644 - test("write path implements onTaskCommit API correctly") { + test("write path implements onTaskCommit API correctly", -+ IgnoreComet("Comet native writer partial support")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3428")) { withSQLConf( SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> classOf[MessageCapturingCommitProtocol].getCanonicalName) { @@ -2979,7 +2979,7 @@ index 44c9fbadfac..5f98bb9be17 100644 - test("Insert overwrite table command should output correct schema: basic") { + test("Insert overwrite table command should output correct schema: basic", -+ IgnoreComet("Comet native writer does not supportinsert overwrite mode")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3424")) { withTable("tbl", "tbl2") { withView("view1") { val df = spark.range(10).toDF("id") diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index e17037c9a6..7e69cd3f15 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -1906,7 +1906,7 @@ index 12d5870309f..51429bf4039 100644 - test("SPARK-33901: ctas should should not change table's schema") { + test("SPARK-33901: ctas should should not change table's schema", -+ IgnoreComet("comet native writer does not support empty dir / table creation yet")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3418")) { withTable("t1", "t2") { sql(s"CREATE TABLE t1(i CHAR(5), c VARCHAR(4)) USING $format") sql(s"CREATE TABLE t2 USING $format AS SELECT * FROM t1") @@ -1916,7 +1916,7 @@ index 12d5870309f..51429bf4039 100644 - test("SPARK-37160: CREATE TABLE AS SELECT with CHAR_AS_VARCHAR") { + test("SPARK-37160: CREATE TABLE AS SELECT with CHAR_AS_VARCHAR", -+ IgnoreComet("comet native writer does not support empty dir/table creation yet")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3419")) { withTable("t1", "t2") { sql(s"CREATE TABLE t1(col CHAR(5)) USING $format") checkTableSchemaTypeStr("t1", Seq(Row("char(5)"))) @@ -1939,7 +1939,7 @@ index 343b59a311e..9d5789c1d91 100644 - test("empty file should be skipped while write to file") { + test("empty file should be skipped while write to file", -+ IgnoreComet("comet native writer does not create empty files / dir")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3417")) { withTempPath { path => spark.range(100).repartition(10).where("id = 50").write.parquet(path.toString) val partFiles = path.listFiles() @@ -2177,7 +2177,7 @@ index 4f8a9e39716..5da031994ff 100644 - test("Write Spark version into Parquet metadata") { + test("Write Spark version into Parquet metadata", -+ IgnoreComet("comet does not write spark version in parquet metadata")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3427")) { withTempPath { dir => spark.range(1).repartition(1).write.parquet(dir.getAbsolutePath) assert(getMetaData(dir)(SPARK_VERSION_METADATA_KEY) === SPARK_VERSION_SHORT) @@ -2200,7 +2200,7 @@ index 8b386e8f689..67a41628a3b 100644 - test("parquet timestamp conversion") { + test("parquet timestamp conversion", -+ IgnoreComet("timestamp96 conversion failed with the native writer")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3425")) { // Make a table with one parquet file written by impala, and one parquet file written by spark. // We should only adjust the timestamps in the impala file, and only if the conf is set val impalaFile = "test-data/impala_timestamp.parq" @@ -2734,7 +2734,7 @@ index 7b1a5a32037..f163a9733a5 100644 - test("INSERT INTO TABLE - complex type but different names") { + test("INSERT INTO TABLE - complex type but different names", -+ IgnoreComet("comet native writer issue")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3426")) { val tab1 = "tab1" val tab2 = "tab2" withTable(tab1, tab2) { @@ -2744,7 +2744,7 @@ index 7b1a5a32037..f163a9733a5 100644 - test("SPARK-38336 INSERT INTO statements with tables with default columns: positive tests") { + test("SPARK-38336 INSERT INTO statements with tables with default columns: positive tests", -+ IgnoreComet("comet native writer insert overwrite bug")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3421")) { // When the INSERT INTO statement provides fewer values than expected, NULL values are appended // in their place. withTable("t") { @@ -2754,7 +2754,7 @@ index 7b1a5a32037..f163a9733a5 100644 - test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Positive tests") { + test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Positive tests", -+ IgnoreComet("bug with insert overwrite mode comet native writer")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3422")) { // There is a complex expression in the default value. val createTableBooleanCol = "create table t(i boolean) using parquet" val createTableIntCol = "create table t(i int) using parquet" @@ -2764,7 +2764,7 @@ index 7b1a5a32037..f163a9733a5 100644 - test("SPARK-43071: INSERT INTO from queries whose final operators are not projections") { + test("SPARK-43071: INSERT INTO from queries whose final operators are not projections", -+ IgnoreComet("comet writer issues with insert overwrite")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3423")) { def runTest(insert: String, expected: Seq[Row]): Unit = { withTable("t1", "t2") { sql("create table t1(i boolean, s bigint default 42) using parquet") @@ -2774,7 +2774,7 @@ index 7b1a5a32037..f163a9733a5 100644 - test("SPARK-29174 Support LOCAL in INSERT OVERWRITE DIRECTORY to data source") { + test("SPARK-29174 Support LOCAL in INSERT OVERWRITE DIRECTORY to data source", -+ IgnoreComet("comet native writer does not support insert overwrite")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3420")) { withTempPath { dir => val path = dir.toURI.getPath sql(s"""create table tab1 ( a int) using parquet location '$path'""") @@ -2945,7 +2945,7 @@ index b40f9210a68..34aa6287eb5 100644 - test("write path implements onTaskCommit API correctly") { + test("write path implements onTaskCommit API correctly", -+ IgnoreComet("Comet native writer partial support")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3428")) { withSQLConf( SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> classOf[MessageCapturingCommitProtocol].getCanonicalName) { @@ -2955,7 +2955,7 @@ index b40f9210a68..34aa6287eb5 100644 - test("Insert overwrite table command should output correct schema: basic") { + test("Insert overwrite table command should output correct schema: basic", -+ IgnoreComet("Comet native writer does not supportinsert overwrite mode")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3424")) { withTable("tbl", "tbl2") { withView("view1") { val df = spark.range(10).toDF("id") diff --git a/dev/diffs/4.0.1.diff b/dev/diffs/4.0.1.diff index e44ecdd813..e36af18d18 100644 --- a/dev/diffs/4.0.1.diff +++ b/dev/diffs/4.0.1.diff @@ -1127,7 +1127,7 @@ index 818c7ec65b1..0d333807b70 100644 } - test("SPARK-48817: test multi inserts") { -+ test("SPARK-48817: test multi inserts", IgnoreComet("comet parquet native writer bug")) { ++ test("SPARK-48817: test multi inserts", IgnoreComet("https://github.com/apache/datafusion-comet/issues/3430")) { withTable("t1", "t2", "t3") { createTable("t1", Seq("i"), Seq("int")) createTable("t2", Seq("i"), Seq("int")) @@ -1465,7 +1465,7 @@ index 77ed8b411e3..b82f70c0071 100644 } - test("ctas with union") { -+ test("ctas with union", IgnoreComet("comet native writer bug when shuffle / AQE enabled")) { ++ test("ctas with union", IgnoreComet("https://github.com/apache/datafusion-comet/issues/3429")) { withTable(testTable) { sql(s""" |CREATE TABLE $testTable USING $dataSource AS @@ -2623,7 +2623,7 @@ index a5bb3058bed..de51b21f21a 100644 - test("SPARK-33901: ctas should should not change table's schema") { +// TODO support CTAS in comet native parquet writer + test("SPARK-33901: ctas should should not change table's schema", -+ IgnoreComet("comet native writer does not support empty dir / table creation yet")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3418")) { withTable("t1", "t2") { sql(s"CREATE TABLE t1(i CHAR(5), c VARCHAR(4)) USING $format") sql(s"CREATE TABLE t2 USING $format AS SELECT * FROM t1") @@ -2633,7 +2633,7 @@ index a5bb3058bed..de51b21f21a 100644 - test("SPARK-37160: CREATE TABLE AS SELECT with CHAR_AS_VARCHAR") { + test("SPARK-37160: CREATE TABLE AS SELECT with CHAR_AS_VARCHAR", -+ IgnoreComet("comet native writer does not support empty dir/table creation yet")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3419")) { withTable("t1", "t2") { sql(s"CREATE TABLE t1(col CHAR(5)) USING $format") checkTableSchemaTypeStr("t1", Seq(Row("char(5)"))) @@ -2657,7 +2657,7 @@ index 343b59a311e..05f3cab13f9 100644 - test("empty file should be skipped while write to file") { +// TODO :: comet native write does not write empty files / dirs + test("empty file should be skipped while write to file", -+ IgnoreComet("comet native writer does not create empty files / dir")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3417")) { withTempPath { path => spark.range(100).repartition(10).where("id = 50").write.parquet(path.toString) val partFiles = path.listFiles() @@ -2909,7 +2909,7 @@ index 4474ec1fd42..d0366be3597 100644 - test("Write Spark version into Parquet metadata") { +// TODO : Comet native writer to add spark / comet version into parquet metadata + test("Write Spark version into Parquet metadata", -+ IgnoreComet("Comet doesn't support DELTA encoding yet")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3427")) { withTempPath { dir => spark.range(1).repartition(1).write.parquet(dir.getAbsolutePath) assert(getMetaData(dir)(SPARK_VERSION_METADATA_KEY) === SPARK_VERSION_SHORT) @@ -2933,7 +2933,7 @@ index baa11df302b..c88f66d5c77 100644 - test("parquet timestamp conversion") { +// TODO : Support legacy timestamps conversion /cast in comet native writer + test("parquet timestamp conversion", -+ IgnoreComet("timestamp96 conversion failed with the native writer")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3425")) { // Make a table with one parquet file written by impala, and one parquet file written by spark. // We should only adjust the timestamps in the impala file, and only if the conf is set val impalaFile = "test-data/impala_timestamp.parq" @@ -3485,7 +3485,7 @@ index baf99798965..e23d8223e05 100644 - test("INSERT INTO TABLE - complex type but different names") { + test("INSERT INTO TABLE - complex type but different names", -+ IgnoreComet("comet native writer issue")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3426")) { val tab1 = "tab1" val tab2 = "tab2" withTable(tab1, tab2) { @@ -3495,7 +3495,7 @@ index baf99798965..e23d8223e05 100644 - test("SPARK-38336 INSERT INTO statements with tables with default columns: positive tests") { + test("SPARK-38336 INSERT INTO statements with tables with default columns: positive tests", -+ IgnoreComet("comet native writer insert overwrite bug")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3421")) { // When the INSERT INTO statement provides fewer values than expected, NULL values are appended // in their place. withTable("t") { @@ -3505,7 +3505,7 @@ index baf99798965..e23d8223e05 100644 - test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Positive tests") { + test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Positive tests", -+ IgnoreComet("bug with insert overwrite mode comet native writer")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3422")) { // There is a complex expression in the default value. val createTableBooleanCol = "create table t(i boolean) using parquet" val createTableIntCol = "create table t(i int) using parquet" @@ -3516,7 +3516,7 @@ index baf99798965..e23d8223e05 100644 - test("SPARK-43071: INSERT INTO from queries whose final operators are not projections") { +// TODO : fix overwrite mode issues comet native writer + test("SPARK-43071: INSERT INTO from queries whose final operators are not projections", -+ IgnoreComet("comet writer issues with insert overwrite")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3423")) { def runTest(insert: String, expected: Seq[Row]): Unit = { withTable("t1", "t2") { sql("create table t1(i boolean, s bigint default 42) using parquet") @@ -3527,7 +3527,7 @@ index baf99798965..e23d8223e05 100644 - test("SPARK-29174 Support LOCAL in INSERT OVERWRITE DIRECTORY to data source") { +// TODO : Insert overwrite not supported comet native writer + test("SPARK-29174 Support LOCAL in INSERT OVERWRITE DIRECTORY to data source", -+ IgnoreComet("comet native writer does not support insert overwrite")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3420")) { withTempPath { dir => val path = dir.toURI.getPath sql(s"""create table tab1 ( a int) using parquet location '$path'""") @@ -3723,7 +3723,7 @@ index 300807cf058..8dab4f9a2af 100644 + +// TODO : fix test for native writer + test("write path implements onTaskCommit API correctly", -+ IgnoreComet("Comet native writer partial support")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3428")) { withSQLConf( SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> classOf[MessageCapturingCommitProtocol].getCanonicalName) { @@ -3734,7 +3734,7 @@ index 300807cf058..8dab4f9a2af 100644 - test("Insert overwrite table command should output correct schema: basic") { +// TODO : support vairous write modes in comet native writer + test("Insert overwrite table command should output correct schema: basic", -+ IgnoreComet("Comet native writer does not supportinsert overwrite mode")) { ++ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3424")) { withTable("tbl", "tbl2") { withView("view1") { val df = spark.range(10).toDF("id") From ecd311ed72acca4435faa4ec9103f4ae3c983230 Mon Sep 17 00:00:00 2001 From: B Vadlamani Date: Fri, 6 Feb 2026 09:35:16 -0800 Subject: [PATCH 10/15] enable_spark_tests_comet_native_writer_fix_spark_4 --- .github/workflows/spark_sql_test.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/spark_sql_test.yml b/.github/workflows/spark_sql_test.yml index 9e2adf9c5a..115fb92f3f 100644 --- a/.github/workflows/spark_sql_test.yml +++ b/.github/workflows/spark_sql_test.yml @@ -155,7 +155,7 @@ jobs: run: | cd apache-spark rm -rf /root/.m2/repository/org/apache/parquet # somehow parquet cache requires cleanups - NOLINT_ON_COMPILE=true ENABLE_COMET=true ENABLE_COMET_ONHEAP=true ENABLE_COMET_WRITER=true ${{ matrix.config.scan-env }} ENABLE_COMET_LOG_FALLBACK_REASONS=${{ github.event.inputs.collect-fallback-logs || 'false' }} \ + NOLINT_ON_COMPILE=true ENABLE_COMET=true ENABLE_COMET_ONHEAP=true ENABLE_COMET_WRITER=true COMET_PARQUET_SCAN_IMPL=${{ matrix.config.scan-impl }} ENABLE_COMET_LOG_FALLBACK_REASONS=${{ github.event.inputs.collect-fallback-logs || 'false' }} \ build/sbt -Dsbt.log.noformat=true ${{ matrix.module.args1 }} "${{ matrix.module.args2 }}" if [ "${{ github.event.inputs.collect-fallback-logs }}" = "true" ]; then find . -type f -name "unit-tests.log" -print0 | xargs -0 grep -h "Comet cannot accelerate" | sed 's/.*Comet cannot accelerate/Comet cannot accelerate/' | sort -u > fallback.log From f4a6e9b3c6401cf73de4040b1c1145fa33a93105 Mon Sep 17 00:00:00 2001 From: B Vadlamani Date: Fri, 6 Feb 2026 23:45:05 -0800 Subject: [PATCH 11/15] enable_spark_tests_comet_native_writer_fix_spark_rebase_main --- dev/diffs/3.5.8.diff | 598 ++++++++++++++++++++++++++++++++++--------- 1 file changed, 476 insertions(+), 122 deletions(-) diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index 7e69cd3f15..4997d8b765 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -1,5 +1,5 @@ diff --git a/pom.xml b/pom.xml -index 68e2c422a24..540bdabf825 100644 +index edd2ad57880..77a975ea48f 100644 --- a/pom.xml +++ b/pom.xml @@ -152,6 +152,8 @@ @@ -38,7 +38,7 @@ index 68e2c422a24..540bdabf825 100644 diff --git a/sql/core/pom.xml b/sql/core/pom.xml -index f08b33575fc..424e0da32fd 100644 +index bc00c448b80..82068d7a2eb 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -77,6 +77,10 @@ @@ -112,17 +112,6 @@ index db587dd9868..aac7295a53d 100644 case _ => Map[String, String]() } new SparkPlanInfo( -diff --git a/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql b/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql -index be038e1083c..58b512a3c17 100644 ---- a/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql -+++ b/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql -@@ -1,3 +1,6 @@ -+-- TODO: support empty table write / CTAS in native parquet writer -+--SET spark.comet.parquet.write.enabled = false -+ - create table char_tbl(c char(5), v varchar(6)) using parquet; - desc formatted char_tbl; - desc formatted char_tbl c; diff --git a/sql/core/src/test/resources/sql-tests/inputs/explain-aqe.sql b/sql/core/src/test/resources/sql-tests/inputs/explain-aqe.sql index 7aef901da4f..f3d6e18926d 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/explain-aqe.sql @@ -227,7 +216,7 @@ index 0efe0877e9b..423d3b3d76d 100644 -- SELECT_HAVING -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_having.sql diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala -index 9815cb816c9..95b5f9992b0 100644 +index e5494726695..00937f025c2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeConstants @@ -249,8 +238,22 @@ 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/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +index 9e8d77c53f3..855e3ada7d1 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +@@ -790,7 +790,8 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { + } + } + +- test("input_file_name, input_file_block_start, input_file_block_length - FileScanRDD") { ++ test("input_file_name, input_file_block_start, input_file_block_length - FileScanRDD", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3312")) { + withTempPath { dir => + val data = sparkContext.parallelize(0 to 10).toDF("id") + data.write.parquet(dir.getCanonicalPath) 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 6f3090d8908..c08a60fb0c2 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 @@ -347,7 +350,7 @@ index 7ee18df3756..d09f70e5d99 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 47a311c71d5..342e71cfdd4 100644 +index a1d5d579338..c201d39cc78 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 @@ -385,7 +388,7 @@ index 47a311c71d5..342e71cfdd4 100644 } 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 +index c4fb4fa943c..a04b23870a8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} @@ -407,7 +410,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..0e1499a24ca 100644 +index f33432ddb6f..42eb9fd1cb7 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 @@ -458,7 +461,17 @@ index f33432ddb6f..0e1499a24ca 100644 withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { val df = sql( """ WITH v as ( -@@ -1729,6 +1736,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat +@@ -1698,7 +1705,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat + * Check the static scan metrics with and without DPP + */ + test("static scan metrics", +- DisableAdaptiveExecution("DPP in AQE must reuse broadcast")) { ++ DisableAdaptiveExecution("DPP in AQE must reuse broadcast"), ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3313")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { +@@ -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"))) @@ -468,10 +481,20 @@ index f33432ddb6f..0e1499a24ca 100644 } assert(scanOption.isDefined) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -index a206e97c353..fea1149b67d 100644 +index a206e97c353..79813d8e259 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala -@@ -467,7 +467,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -280,7 +280,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite + } + } + +- test("explain formatted - check presence of subquery in case of DPP") { ++ test("explain formatted - check presence of subquery in case of DPP", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3313")) { + withTable("df1", "df2") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", +@@ -467,7 +468,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } @@ -481,7 +504,7 @@ index a206e97c353..fea1149b67d 100644 withTempDir { dir => Seq("parquet", "orc", "csv", "json").foreach { fmt => val basePath = dir.getCanonicalPath + "/" + fmt -@@ -545,7 +546,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite +@@ -545,7 +547,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite } } @@ -493,7 +516,7 @@ index a206e97c353..fea1149b67d 100644 test("SPARK-35884: Explain Formatted") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala -index 93275487f29..33b2e7ad3b1 100644 +index 93275487f29..510e3087e0f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -23,6 +23,7 @@ import java.nio.file.{Files, StandardOpenOption} @@ -521,7 +544,17 @@ index 93275487f29..33b2e7ad3b1 100644 checkErrorMatchPVals( exception = intercept[SparkException] { testIgnoreMissingFiles(options) -@@ -955,6 +959,7 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -639,7 +643,8 @@ class FileBasedDataSourceSuite extends QueryTest + } + + Seq("parquet", "orc").foreach { format => +- test(s"Spark native readers should respect spark.sql.caseSensitive - ${format}") { ++ test(s"Spark native readers should respect spark.sql.caseSensitive - ${format}", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { + withTempDir { dir => + val tableName = s"spark_25132_${format}_native" + val tableDir = dir.getCanonicalPath + s"/$tableName" +@@ -955,6 +960,7 @@ class FileBasedDataSourceSuite extends QueryTest assert(bJoinExec.isEmpty) val smJoinExec = collect(joinedDF.queryExecution.executedPlan) { case smJoin: SortMergeJoinExec => smJoin @@ -529,7 +562,7 @@ index 93275487f29..33b2e7ad3b1 100644 } assert(smJoinExec.nonEmpty) } -@@ -1015,6 +1020,7 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -1015,6 +1021,7 @@ class FileBasedDataSourceSuite extends QueryTest val fileScan = df.queryExecution.executedPlan collectFirst { case BatchScanExec(_, f: FileScan, _, _, _, _) => f @@ -537,7 +570,7 @@ index 93275487f29..33b2e7ad3b1 100644 } assert(fileScan.nonEmpty) assert(fileScan.get.partitionFilters.nonEmpty) -@@ -1056,6 +1062,7 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -1056,6 +1063,7 @@ class FileBasedDataSourceSuite extends QueryTest val fileScan = df.queryExecution.executedPlan collectFirst { case BatchScanExec(_, f: FileScan, _, _, _, _) => f @@ -545,7 +578,7 @@ index 93275487f29..33b2e7ad3b1 100644 } assert(fileScan.nonEmpty) assert(fileScan.get.partitionFilters.isEmpty) -@@ -1240,6 +1247,9 @@ class FileBasedDataSourceSuite extends QueryTest +@@ -1240,6 +1248,9 @@ class FileBasedDataSourceSuite extends QueryTest val filters = df.queryExecution.executedPlan.collect { case f: FileSourceScanLike => f.dataFilters case b: BatchScanExec => b.scan.asInstanceOf[FileScan].dataFilters @@ -557,7 +590,7 @@ index 93275487f29..33b2e7ad3b1 100644 } 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 +index 00000000000..1ee842b6f62 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala @@ -0,0 +1,45 @@ @@ -597,8 +630,8 @@ index 00000000000..5691536c114 + * 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 = { ++ 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 { @@ -635,7 +668,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..66a5473852d 100644 +index 44c8cb92fc3..f098beeca26 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -31,7 +31,8 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation @@ -833,7 +866,7 @@ index 4d256154c85..66a5473852d 100644 checkAnswer(fullJoinDF, Row(100)) } } -@@ -1583,6 +1612,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1611,6 +1640,9 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan Seq(semiJoinDF, antiJoinDF).foreach { df => assert(collect(df.queryExecution.executedPlan) { case j: ShuffledHashJoinExec if j.ignoreDuplicatedKey == ignoreDuplicatedKey => true @@ -843,7 +876,7 @@ index 4d256154c85..66a5473852d 100644 }.size == 1) } } -@@ -1627,14 +1659,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan +@@ -1655,14 +1687,20 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan test("SPARK-43113: Full outer join with duplicate stream-side references in condition (SMJ)") { def check(plan: SparkPlan): Unit = { @@ -866,7 +899,7 @@ index 4d256154c85..66a5473852d 100644 } dupStreamSideColTest("SHUFFLE_HASH", check) } -@@ -1770,7 +1808,8 @@ class ThreadLeakInSortMergeJoinSuite +@@ -1798,7 +1836,8 @@ class ThreadLeakInSortMergeJoinSuite sparkConf.set(SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD, 20)) } @@ -890,7 +923,7 @@ index c26757c9cff..d55775f09d7 100644 protected val baseResourcePath = { // use the same way as `SQLQueryTestSuite` to get the resource path diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala -index 793a0da6a86..181bfc16e4b 100644 +index 3cf2bfd17ab..49728c35c42 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1521,7 +1521,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark @@ -1051,6 +1084,20 @@ index 04702201f82..5ee11f83ecf 100644 } assert(exchanges.size === 1) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +index 9f8e979e3fb..3bc9dab8023 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +@@ -87,7 +87,8 @@ class UDFSuite extends QueryTest with SharedSparkSession { + spark.catalog.dropTempView("tmp_table") + } + +- test("SPARK-8005 input_file_name") { ++ test("SPARK-8005 input_file_name", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3312")) { + withTempPath { dir => + val data = sparkContext.parallelize(0 to 10, 2).toDF("id") + data.write.parquet(dir.getCanonicalPath) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index d269290e616..13726a31e07 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -1115,24 +1162,37 @@ index d269290e616..13726a31e07 100644 } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala -index cfc8b2cc845..c6fcfd7bd08 100644 +index cfc8b2cc845..b7c234e1437 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala -@@ -21,6 +21,7 @@ import scala.collection.mutable.ArrayBuffer +@@ -19,8 +19,9 @@ package org.apache.spark.sql.connector + import scala.collection.mutable.ArrayBuffer + import org.apache.spark.SparkConf - import org.apache.spark.sql.{AnalysisException, QueryTest} +-import org.apache.spark.sql.{AnalysisException, QueryTest} ++import org.apache.spark.sql.{AnalysisException, IgnoreCometNativeDataFusion, QueryTest} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -+import org.apache.spark.sql.comet.CometScanExec ++import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec} import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, Table, TableCapability} import org.apache.spark.sql.connector.read.ScanBuilder import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} -@@ -184,7 +185,11 @@ class FileDataSourceV2FallBackSuite extends QueryTest with SharedSparkSession { +@@ -152,7 +153,8 @@ class FileDataSourceV2FallBackSuite extends QueryTest with SharedSparkSession { + } + } + +- test("Fallback Parquet V2 to V1") { ++ test("Fallback Parquet V2 to V1", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3315")) { + Seq("parquet", classOf[ParquetDataSourceV2].getCanonicalName).foreach { format => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> format) { + val commands = ArrayBuffer.empty[(String, LogicalPlan)] +@@ -184,7 +186,11 @@ class FileDataSourceV2FallBackSuite extends QueryTest with SharedSparkSession { val df = spark.read.format(format).load(path.getCanonicalPath) checkAnswer(df, inputData.toDF()) assert( - df.queryExecution.executedPlan.exists(_.isInstanceOf[FileSourceScanExec])) + df.queryExecution.executedPlan.exists { -+ case _: FileSourceScanExec | _: CometScanExec => true ++ case _: FileSourceScanExec | _: CometScanExec | _: CometNativeScanExec => true + case _ => false + } + ) @@ -1390,6 +1450,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/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +index a1147c16cc8..c7a29496328 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution + + import org.apache.spark.{SparkArithmeticException, SparkException, SparkFileNotFoundException} + import org.apache.spark.sql._ ++import org.apache.spark.sql.IgnoreCometNativeDataFusion + import org.apache.spark.sql.catalyst.TableIdentifier + import org.apache.spark.sql.catalyst.expressions.{Add, Alias, Divide} + import org.apache.spark.sql.catalyst.parser.ParseException +@@ -968,7 +969,8 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { + } + } + +- test("alter temporary view should follow current storeAnalyzedPlanForView config") { ++ test("alter temporary view should follow current storeAnalyzedPlanForView config", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3314")) { + withTable("t") { + Seq(2, 3, 1).toDF("c1").write.format("parquet").saveAsTable("t") + withView("v1") { 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 @@ -1888,7 +1970,7 @@ index 2f8e401e743..a4f94417dcc 100644 assert(o1.semanticEquals(o2), "Different output column order after AQE optimization") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala -index 12d5870309f..51429bf4039 100644 +index 12d5870309f..c65c688d56d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala @@ -18,7 +18,7 @@ @@ -1921,7 +2003,7 @@ index 12d5870309f..51429bf4039 100644 sql(s"CREATE TABLE t1(col CHAR(5)) USING $format") checkTableSchemaTypeStr("t1", Seq(Row("char(5)"))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala -index 343b59a311e..9d5789c1d91 100644 +index 343b59a311e..fa6ed7be6f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala @@ -17,7 +17,7 @@ @@ -1944,7 +2026,7 @@ index 343b59a311e..9d5789c1d91 100644 spark.range(100).repartition(10).where("id = 50").write.parquet(path.toString) val partFiles = path.listFiles() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala -index bf496d6db21..9bb57a9b4c6 100644 +index fd52d038ca6..154c800be67 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.Concat @@ -1955,7 +2037,7 @@ index bf496d6db21..9bb57a9b4c6 100644 import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.functions._ -@@ -868,6 +869,8 @@ abstract class SchemaPruningSuite +@@ -884,6 +885,8 @@ abstract class SchemaPruningSuite val fileSourceScanSchemata = collect(df.queryExecution.executedPlan) { case scan: FileSourceScanExec => scan.requiredSchema @@ -1965,7 +2047,7 @@ index bf496d6db21..9bb57a9b4c6 100644 assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala -index ce43edb79c1..4dbb5942bc3 100644 +index 5fd27410dcb..468abb1543a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources @@ -1976,7 +2058,7 @@ index ce43edb79c1..4dbb5942bc3 100644 import org.apache.spark.sql.execution.{QueryExecution, SortExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.internal.SQLConf -@@ -225,6 +226,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write +@@ -243,6 +244,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write // assert the outer most sort in the executed plan assert(plan.collectFirst { case s: SortExec => s @@ -1984,7 +2066,7 @@ index ce43edb79c1..4dbb5942bc3 100644 }.exists { case SortExec(Seq( SortOrder(AttributeReference("key", IntegerType, _, _), Ascending, NullsFirst, _), -@@ -272,6 +274,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write +@@ -290,6 +292,7 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write // assert the outer most sort in the executed plan assert(plan.collectFirst { case s: SortExec => s @@ -2040,7 +2122,7 @@ index 07e2849ce6f..3e73645b638 100644 ParquetOutputFormat.WRITER_VERSION -> ParquetProperties.WriterVersion.PARQUET_2_0.toString ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala -index 8e88049f51e..8f3cf8a0f80 100644 +index 8e88049f51e..49f2001dc6b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -1095,7 +1095,11 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared @@ -2066,7 +2148,17 @@ index 8e88049f51e..8f3cf8a0f80 100644 import testImplicits._ withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", -@@ -1580,7 +1585,11 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared +@@ -1548,7 +1553,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared + } + } + +- test("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") { ++ test("SPARK-31026: Parquet predicate pushdown for fields having dots in the names", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3320")) { + import testImplicits._ + + withAllParquetReaders { +@@ -1580,13 +1586,18 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared // than the total length but should not be a single record. // Note that, if record level filtering is enabled, it should be a single record. // If no filter is pushed down to Parquet, it should be the total length of data. @@ -2079,7 +2171,15 @@ index 8e88049f51e..8f3cf8a0f80 100644 } } } -@@ -1607,7 +1616,11 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared + } + +- test("Filters should be pushed down for Parquet readers at row group level") { ++ test("Filters should be pushed down for Parquet readers at row group level", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3320")) { + import testImplicits._ + + withSQLConf( +@@ -1607,7 +1618,11 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared // than the total length but should not be a single record. // Note that, if record level filtering is enabled, it should be a single record. // If no filter is pushed down to Parquet, it should be the total length of data. @@ -2092,7 +2192,7 @@ index 8e88049f51e..8f3cf8a0f80 100644 } } } -@@ -1699,7 +1712,7 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared +@@ -1699,7 +1714,7 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared (attr, value) => sources.StringContains(attr, value)) } @@ -2101,7 +2201,7 @@ index 8e88049f51e..8f3cf8a0f80 100644 import testImplicits._ // keep() should take effect on StartsWith/EndsWith/Contains Seq( -@@ -1743,7 +1756,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared +@@ -1743,7 +1758,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } } @@ -2111,7 +2211,17 @@ index 8e88049f51e..8f3cf8a0f80 100644 val schema = StructType(Seq( StructField("a", IntegerType, nullable = false) )) -@@ -1984,7 +1998,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared +@@ -1933,7 +1949,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared + } + } + +- test("SPARK-25207: exception when duplicate fields in case-insensitive mode") { ++ test("SPARK-25207: exception when duplicate fields in case-insensitive mode", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { + withTempPath { dir => + val count = 10 + val tableName = "spark_25207" +@@ -1984,7 +2001,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } } @@ -2121,7 +2231,7 @@ index 8e88049f51e..8f3cf8a0f80 100644 // block 1: // null count min max // page-0 0 0 99 -@@ -2044,7 +2059,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared +@@ -2044,7 +2062,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared } } @@ -2131,7 +2241,7 @@ index 8e88049f51e..8f3cf8a0f80 100644 withTempPath { dir => val path = dir.getCanonicalPath spark.range(100).selectExpr("id * 2 AS id") -@@ -2276,7 +2292,11 @@ class ParquetV1FilterSuite extends ParquetFilterSuite { +@@ -2276,7 +2295,11 @@ class ParquetV1FilterSuite extends ParquetFilterSuite { assert(pushedParquetFilters.exists(_.getClass === filterClass), s"${pushedParquetFilters.map(_.getClass).toList} did not contain ${filterClass}.") @@ -2144,7 +2254,7 @@ index 8e88049f51e..8f3cf8a0f80 100644 } else { assert(selectedFilters.isEmpty, "There is filter pushed down") } -@@ -2336,7 +2356,11 @@ class ParquetV2FilterSuite extends ParquetFilterSuite { +@@ -2336,7 +2359,11 @@ class ParquetV2FilterSuite extends ParquetFilterSuite { assert(pushedParquetFilters.exists(_.getClass === filterClass), s"${pushedParquetFilters.map(_.getClass).toList} did not contain ${filterClass}.") @@ -2158,10 +2268,30 @@ index 8e88049f51e..8f3cf8a0f80 100644 case _ => throw new AnalysisException("Can not match ParquetTable in the query.") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -index 4f8a9e39716..5da031994ff 100644 +index 8ed9ef1630e..d1ac5344638 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -@@ -1335,7 +1335,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession +@@ -1064,7 +1064,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession + } + } + +- test("SPARK-35640: read binary as timestamp should throw schema incompatible error") { ++ test("SPARK-35640: read binary as timestamp should throw schema incompatible error", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { + val data = (1 to 4).map(i => Tuple1(i.toString)) + val readSchema = StructType(Seq(StructField("_1", DataTypes.TimestampType))) + +@@ -1075,7 +1076,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession + } + } + +- test("SPARK-35640: int as long should throw schema incompatible error") { ++ test("SPARK-35640: int as long should throw schema incompatible error", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { + val data = (1 to 4).map(i => Tuple1(i)) + val readSchema = StructType(Seq(StructField("_1", DataTypes.LongType))) + +@@ -1345,7 +1347,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession } } @@ -2171,7 +2301,7 @@ index 4f8a9e39716..5da031994ff 100644 withAllParquetReaders { checkAnswer( // "fruit" column in this file is encoded using DELTA_LENGTH_BYTE_ARRAY. -@@ -1541,7 +1542,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession +@@ -1551,7 +1554,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession } } @@ -2182,7 +2312,7 @@ index 4f8a9e39716..5da031994ff 100644 spark.range(1).repartition(1).write.parquet(dir.getAbsolutePath) assert(getMetaData(dir)(SPARK_VERSION_METADATA_KEY) === SPARK_VERSION_SHORT) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala -index 8b386e8f689..67a41628a3b 100644 +index 8b386e8f689..76c3005fcd4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.fs.{Path, PathFilter} @@ -2205,10 +2335,20 @@ index 8b386e8f689..67a41628a3b 100644 // We should only adjust the timestamps in the impala file, and only if the conf is set val impalaFile = "test-data/impala_timestamp.parq" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala -index f6472ba3d9d..7a8f5317ed7 100644 +index f6472ba3d9d..ce39ebb52e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala -@@ -998,7 +998,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -185,7 +185,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS + } + } + +- test("SPARK-36182: can't read TimestampLTZ as TimestampNTZ") { ++ test("SPARK-36182: can't read TimestampLTZ as TimestampNTZ", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { + val data = (1 to 1000).map { i => + val ts = new java.sql.Timestamp(i) + Row(ts) +@@ -998,7 +999,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS } } @@ -2218,7 +2358,17 @@ index f6472ba3d9d..7a8f5317ed7 100644 withAllParquetReaders { withTempPath { path => // Repeated values for dictionary encoding. -@@ -1067,7 +1068,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -1051,7 +1053,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS + testMigration(fromTsType = "TIMESTAMP_MICROS", toTsType = "INT96") + } + +- test("SPARK-34212 Parquet should read decimals correctly") { ++ test("SPARK-34212 Parquet should read decimals correctly", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { + def readParquet(schema: String, path: File): DataFrame = { + spark.read.schema(schema).parquet(path.toString) + } +@@ -1067,7 +1070,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS checkAnswer(readParquet(schema, path), df) } @@ -2228,7 +2378,7 @@ index f6472ba3d9d..7a8f5317ed7 100644 val schema1 = "a DECIMAL(3, 2), b DECIMAL(18, 3), c DECIMAL(37, 3)" checkAnswer(readParquet(schema1, path), df) val schema2 = "a DECIMAL(3, 0), b DECIMAL(18, 1), c DECIMAL(37, 1)" -@@ -1089,7 +1091,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -1089,7 +1093,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS val df = sql(s"SELECT 1 a, 123456 b, ${Int.MaxValue.toLong * 10} c, CAST('1.2' AS BINARY) d") df.write.parquet(path.toString) @@ -2238,7 +2388,17 @@ index f6472ba3d9d..7a8f5317ed7 100644 checkAnswer(readParquet("a DECIMAL(3, 2)", path), sql("SELECT 1.00")) checkAnswer(readParquet("b DECIMAL(3, 2)", path), Row(null)) checkAnswer(readParquet("b DECIMAL(11, 1)", path), sql("SELECT 123456.0")) -@@ -1148,7 +1151,7 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS +@@ -1133,7 +1138,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS + } + } + +- test("row group skipping doesn't overflow when reading into larger type") { ++ test("row group skipping doesn't overflow when reading into larger type", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { + withTempPath { path => + Seq(0).toDF("a").write.parquet(path.toString) + // The vectorized and non-vectorized readers will produce different exceptions, we don't need +@@ -1148,7 +1154,7 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS .where(s"a < ${Long.MaxValue}") .collect() } @@ -2343,14 +2503,14 @@ index 5c0b7def039..151184bc98c 100644 assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala -index 3f47c5e506f..bc1ee1ec0ba 100644 +index 3f47c5e506f..92a5eafec84 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -27,6 +27,7 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName import org.apache.parquet.schema.Type._ import org.apache.spark.SparkException -+import org.apache.spark.sql.IgnoreComet ++import org.apache.spark.sql.{IgnoreComet, IgnoreCometNativeDataFusion} import org.apache.spark.sql.catalyst.expressions.Cast.toSQLType import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException import org.apache.spark.sql.functions.desc @@ -2364,6 +2524,26 @@ index 3f47c5e506f..bc1ee1ec0ba 100644 withTempPath { dir => val e = testSchemaMismatch(dir.getCanonicalPath, vectorizedReaderEnabled = false) val expectedMessage = "Encountered error while reading file" +@@ -1046,7 +1048,8 @@ class ParquetSchemaSuite extends ParquetSchemaTest { + } + } + +- test("schema mismatch failure error message for parquet vectorized reader") { ++ test("schema mismatch failure error message for parquet vectorized reader", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { + withTempPath { dir => + val e = testSchemaMismatch(dir.getCanonicalPath, vectorizedReaderEnabled = true) + assert(e.getCause.isInstanceOf[SparkException]) +@@ -1087,7 +1090,8 @@ class ParquetSchemaSuite extends ParquetSchemaTest { + } + } + +- test("SPARK-45604: schema mismatch failure error on timestamp_ntz to array") { ++ test("SPARK-45604: schema mismatch failure error on timestamp_ntz to array", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3311")) { + import testImplicits._ + + withTempPath { dir => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala index b8f3ea3c6f3..bbd44221288 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala @@ -2413,18 +2593,30 @@ index 5cdbdc27b32..307fba16578 100644 spark.range(10).selectExpr("id", "id % 3 as p") .write.partitionBy("p").saveAsTable("testDataForScan") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala -index 0ab8691801d..d9125f658ad 100644 +index 0ab8691801d..7b81f3a8f6d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala -@@ -18,6 +18,7 @@ +@@ -17,7 +17,9 @@ + package org.apache.spark.sql.execution.python ++import org.apache.spark.sql.IgnoreCometNativeDataFusion import org.apache.spark.sql.catalyst.plans.logical.{ArrowEvalPython, BatchEvalPython, Limit, LocalLimit} +import org.apache.spark.sql.comet._ import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan, SparkPlanTest} import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan -@@ -108,6 +109,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { +@@ -93,7 +95,8 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { + assert(arrowEvalNodes.size == 2) + } + +- test("Python UDF should not break column pruning/filter pushdown -- Parquet V1") { ++ test("Python UDF should not break column pruning/filter pushdown -- Parquet V1", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3312")) { + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "parquet") { + withTempPath { f => + spark.range(10).select($"id".as("a"), $"id".as("b")) +@@ -108,6 +111,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { val scanNodes = query.queryExecution.executedPlan.collect { case scan: FileSourceScanExec => scan @@ -2432,7 +2624,7 @@ index 0ab8691801d..d9125f658ad 100644 } assert(scanNodes.length == 1) assert(scanNodes.head.output.map(_.name) == Seq("a")) -@@ -120,11 +122,16 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { +@@ -120,11 +124,16 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { val scanNodes = query.queryExecution.executedPlan.collect { case scan: FileSourceScanExec => scan @@ -2451,7 +2643,7 @@ index 0ab8691801d..d9125f658ad 100644 } } } -@@ -145,6 +152,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { +@@ -145,6 +154,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { val scanNodes = query.queryExecution.executedPlan.collect { case scan: BatchScanExec => scan @@ -2459,7 +2651,7 @@ index 0ab8691801d..d9125f658ad 100644 } assert(scanNodes.length == 1) assert(scanNodes.head.output.map(_.name) == Seq("a")) -@@ -157,6 +165,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { +@@ -157,6 +167,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { val scanNodes = query.queryExecution.executedPlan.collect { case scan: BatchScanExec => scan @@ -2484,7 +2676,7 @@ index d083cac48ff..3c11bcde807 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -index 746f289c393..a773971d3c1 100644 +index 746f289c393..5b9e31c1fa6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -19,16 +19,19 @@ package org.apache.spark.sql.sources @@ -2605,7 +2797,14 @@ index 746f289c393..a773971d3c1 100644 s"expected sort in the right child to be $sortRight but found\n${joinOperator.right}") // check the output partitioning -@@ -836,11 +874,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -831,16 +869,17 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } + } + +- test("disable bucketing when the output doesn't contain all bucketing columns") { ++ test("disable bucketing when the output doesn't contain all bucketing columns", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3319")) { + withTable("bucketed_table") { df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") val scanDF = spark.table("bucketed_table").select("j") @@ -2619,7 +2818,7 @@ index 746f289c393..a773971d3c1 100644 checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) } } -@@ -895,7 +933,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -895,7 +934,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } test("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") { @@ -2630,7 +2829,7 @@ index 746f289c393..a773971d3c1 100644 SQLConf.SHUFFLE_PARTITIONS.key -> "5", SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "7") { val bucketSpec = Some(BucketSpec(6, Seq("i", "j"), Nil)) -@@ -914,7 +955,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -914,7 +956,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } test("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") { @@ -2641,7 +2840,7 @@ index 746f289c393..a773971d3c1 100644 SQLConf.SHUFFLE_PARTITIONS.key -> "9", SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10") { -@@ -944,7 +988,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -944,7 +989,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } test("bucket coalescing eliminates shuffle") { @@ -2652,7 +2851,17 @@ index 746f289c393..a773971d3c1 100644 SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. -@@ -1029,15 +1076,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -1013,7 +1061,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } + } + +- test("bucket coalescing is applied when join expressions match with partitioning expressions") { ++ test("bucket coalescing is applied when join expressions match with partitioning expressions", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3319")) { + withTable("t1", "t2", "t3") { + df1.write.format("parquet").bucketBy(8, "i", "j").saveAsTable("t1") + df2.write.format("parquet").bucketBy(4, "i", "j").saveAsTable("t2") +@@ -1029,15 +1078,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti Seq(true, false).foreach { aqeEnabled => withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled.toString) { val plan = sql(query).queryExecution.executedPlan @@ -2701,13 +2910,15 @@ index 6f897a9c0b7..b0723634f68 100644 protected override lazy val sql = spark.sql _ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala -index d675503a8ba..659fa686fb7 100644 +index d675503a8ba..c386a8cb686 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala -@@ -18,6 +18,7 @@ +@@ -17,7 +17,8 @@ + package org.apache.spark.sql.sources - import org.apache.spark.sql.QueryTest +-import org.apache.spark.sql.QueryTest ++import org.apache.spark.sql.{IgnoreCometNativeDataFusion, QueryTest} +import org.apache.spark.sql.comet.CometScanExec import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} @@ -2724,8 +2935,58 @@ index d675503a8ba..659fa686fb7 100644 assert(bucketedScan.length == expectedNumBucketedScan) } +@@ -83,7 +87,8 @@ abstract class DisableUnnecessaryBucketedScanSuite + } + } + +- test("SPARK-32859: disable unnecessary bucketed table scan - basic test") { ++ test("SPARK-32859: disable unnecessary bucketed table scan - basic test", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3319")) { + withTable("t1", "t2", "t3") { + df1.write.format("parquet").bucketBy(8, "i").saveAsTable("t1") + df2.write.format("parquet").bucketBy(8, "i").saveAsTable("t2") +@@ -124,7 +129,8 @@ abstract class DisableUnnecessaryBucketedScanSuite + } + } + +- test("SPARK-32859: disable unnecessary bucketed table scan - multiple joins test") { ++ test("SPARK-32859: disable unnecessary bucketed table scan - multiple joins test", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3319")) { + withTable("t1", "t2", "t3") { + df1.write.format("parquet").bucketBy(8, "i").saveAsTable("t1") + df2.write.format("parquet").bucketBy(8, "i").saveAsTable("t2") +@@ -167,7 +173,8 @@ abstract class DisableUnnecessaryBucketedScanSuite + } + } + +- test("SPARK-32859: disable unnecessary bucketed table scan - multiple bucketed columns test") { ++ test("SPARK-32859: disable unnecessary bucketed table scan - multiple bucketed columns test", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3319")) { + withTable("t1", "t2", "t3") { + df1.write.format("parquet").bucketBy(8, "i", "j").saveAsTable("t1") + df2.write.format("parquet").bucketBy(8, "i", "j").saveAsTable("t2") +@@ -198,7 +205,8 @@ abstract class DisableUnnecessaryBucketedScanSuite + } + } + +- test("SPARK-32859: disable unnecessary bucketed table scan - other operators test") { ++ test("SPARK-32859: disable unnecessary bucketed table scan - other operators test", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3319")) { + withTable("t1", "t2", "t3") { + df1.write.format("parquet").bucketBy(8, "i").saveAsTable("t1") + df2.write.format("parquet").bucketBy(8, "i").saveAsTable("t2") +@@ -239,7 +247,8 @@ abstract class DisableUnnecessaryBucketedScanSuite + } + } + +- test("Aggregates with no groupby over tables having 1 BUCKET, return multiple rows") { ++ test("Aggregates with no groupby over tables having 1 BUCKET, return multiple rows", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3319")) { + withTable("t1") { + withSQLConf(SQLConf.AUTO_BUCKETED_SCAN_ENABLED.key -> "true") { + sql( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala -index 7b1a5a32037..f163a9733a5 100644 +index 7b1a5a32037..fb5fb599158 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -262,7 +262,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { @@ -2779,10 +3040,10 @@ index 7b1a5a32037..f163a9733a5 100644 val path = dir.toURI.getPath sql(s"""create table tab1 ( a int) using parquet location '$path'""") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala -index 1954cce7fdc..73d1464780e 100644 +index 7f6fa2a123e..c778b4e2c48 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala -@@ -34,6 +34,7 @@ import org.apache.spark.paths.SparkPath +@@ -35,6 +35,7 @@ import org.apache.spark.paths.SparkPath import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} import org.apache.spark.sql.{AnalysisException, DataFrame} import org.apache.spark.sql.catalyst.util.stringToFile @@ -2790,7 +3051,7 @@ index 1954cce7fdc..73d1464780e 100644 import org.apache.spark.sql.execution.DataSourceScanExec import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation, FileScan, FileTable} -@@ -761,6 +762,8 @@ class FileStreamSinkV2Suite extends FileStreamSinkSuite { +@@ -777,6 +778,8 @@ class FileStreamSinkV2Suite extends FileStreamSinkSuite { val fileScan = df.queryExecution.executedPlan.collect { case batch: BatchScanExec if batch.scan.isInstanceOf[FileScan] => batch.scan.asInstanceOf[FileScan] @@ -2912,6 +3173,72 @@ index aad91601758..201083bd621 100644 }) } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +index b5cf13a9c12..ac17603fb7f 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +@@ -36,7 +36,7 @@ import org.scalatestplus.mockito.MockitoSugar + + import org.apache.spark.{SparkException, TestUtils} + import org.apache.spark.internal.Logging +-import org.apache.spark.sql.{AnalysisException, Column, DataFrame, Dataset, Row, SaveMode} ++import org.apache.spark.sql.{AnalysisException, Column, DataFrame, Dataset, IgnoreCometNativeDataFusion, Row, SaveMode} + import org.apache.spark.sql.catalyst.InternalRow + import org.apache.spark.sql.catalyst.expressions.{Literal, Rand, Randn, Shuffle, Uuid} + import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, CTERelationRef, LocalRelation} +@@ -660,7 +660,8 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi + ) + } + +- test("SPARK-41198: input row calculation with CTE") { ++ test("SPARK-41198: input row calculation with CTE", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3315")) { + withTable("parquet_tbl", "parquet_streaming_tbl") { + spark.range(0, 10).selectExpr("id AS col1", "id AS col2") + .write.format("parquet").saveAsTable("parquet_tbl") +@@ -712,7 +713,8 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi + } + } + +- test("SPARK-41199: input row calculation with mixed-up of DSv1 and DSv2 streaming sources") { ++ test("SPARK-41199: input row calculation with mixed-up of DSv1 and DSv2 streaming sources", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3315")) { + withTable("parquet_streaming_tbl") { + val streamInput = MemoryStream[Int] + val streamDf = streamInput.toDF().selectExpr("value AS key", "value AS value_stream") +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSelfUnionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSelfUnionSuite.scala +index 8f099c31e6b..ce4b7ad25b3 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSelfUnionSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingSelfUnionSuite.scala +@@ -20,7 +20,7 @@ package org.apache.spark.sql.streaming + import org.scalatest.BeforeAndAfter + import org.scalatest.concurrent.PatienceConfiguration.Timeout + +-import org.apache.spark.sql.SaveMode ++import org.apache.spark.sql.{IgnoreCometNativeDataFusion, SaveMode} + import org.apache.spark.sql.connector.catalog.Identifier + import org.apache.spark.sql.execution.streaming.MemoryStream + import org.apache.spark.sql.streaming.test.{InMemoryStreamTable, InMemoryStreamTableCatalog} +@@ -42,7 +42,8 @@ class StreamingSelfUnionSuite extends StreamTest with BeforeAndAfter { + sqlContext.streams.active.foreach(_.stop()) + } + +- test("self-union, DSv1, read via DataStreamReader API") { ++ test("self-union, DSv1, read via DataStreamReader API", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3401")) { + withTempPath { dir => + val dataLocation = dir.getAbsolutePath + spark.range(1, 4).write.format("parquet").save(dataLocation) +@@ -66,7 +67,8 @@ class StreamingSelfUnionSuite extends StreamTest with BeforeAndAfter { + } + } + +- test("self-union, DSv1, read via table API") { ++ test("self-union, DSv1, read via table API", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3401")) { + withTable("parquet_streaming_tbl") { + spark.sql("CREATE TABLE parquet_streaming_tbl (key integer) USING parquet") + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala index abe606ad9c1..2d930b64cca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala @@ -2936,7 +3263,7 @@ index abe606ad9c1..2d930b64cca 100644 val tblTargetName = "tbl_target" val tblSourceQualified = s"default.$tblSourceName" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala -index b40f9210a68..34aa6287eb5 100644 +index b40f9210a68..f05a41168a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -538,7 +538,8 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with @@ -2960,7 +3287,7 @@ index b40f9210a68..34aa6287eb5 100644 withView("view1") { val df = spark.range(10).toDF("id") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala -index e937173a590..5fede1579f7 100644 +index e937173a590..14501d17dcd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -27,6 +27,7 @@ import scala.concurrent.duration._ @@ -2979,37 +3306,42 @@ index e937173a590..5fede1579f7 100644 import org.apache.spark.sql.execution.FilterExec import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecution import org.apache.spark.sql.execution.datasources.DataSourceUtils -@@ -126,7 +128,28 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with - } - } - } else { -- super.test(testName, testTags: _*)(testFun) -+ if (isCometEnabled && testTags.exists(_.isInstanceOf[IgnoreComet])) { -+ ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) -+ } else { -+ val cometScanImpl = CometConf.COMET_NATIVE_SCAN_IMPL.get(conf) -+ val isNativeIcebergCompat = cometScanImpl == CometConf.SCAN_NATIVE_ICEBERG_COMPAT || -+ cometScanImpl == CometConf.SCAN_AUTO -+ val isNativeDataFusion = cometScanImpl == CometConf.SCAN_NATIVE_DATAFUSION || -+ cometScanImpl == CometConf.SCAN_AUTO -+ if (isCometEnabled && isNativeIcebergCompat && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeIcebergCompat])) { -+ ignore(testName + " (disabled for NATIVE_ICEBERG_COMPAT)", testTags: _*)(testFun) -+ } else if (isCometEnabled && isNativeDataFusion && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeDataFusion])) { -+ ignore(testName + " (disabled for NATIVE_DATAFUSION)", testTags: _*)(testFun) -+ } else if (isCometEnabled && (isNativeDataFusion || isNativeIcebergCompat) && -+ testTags.exists(_.isInstanceOf[IgnoreCometNativeScan])) { -+ ignore(testName + " (disabled for NATIVE_DATAFUSION and NATIVE_ICEBERG_COMPAT)", -+ testTags: _*)(testFun) -+ } else { -+ super.test(testName, testTags: _*)(testFun) -+ } -+ } - } - } +@@ -119,6 +121,34 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with -@@ -242,6 +265,39 @@ private[sql] trait SQLTestUtilsBase + override protected def test(testName: String, testTags: Tag*)(testFun: => Any) + (implicit pos: Position): Unit = { ++ // Check Comet skip tags first, before DisableAdaptiveExecution handling ++ if (isCometEnabled && testTags.exists(_.isInstanceOf[IgnoreComet])) { ++ ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) ++ return ++ } ++ if (isCometEnabled) { ++ val cometScanImpl = CometConf.COMET_NATIVE_SCAN_IMPL.get(conf) ++ val isNativeIcebergCompat = cometScanImpl == CometConf.SCAN_NATIVE_ICEBERG_COMPAT || ++ cometScanImpl == CometConf.SCAN_AUTO ++ val isNativeDataFusion = cometScanImpl == CometConf.SCAN_NATIVE_DATAFUSION || ++ cometScanImpl == CometConf.SCAN_AUTO ++ if (isNativeIcebergCompat && ++ testTags.exists(_.isInstanceOf[IgnoreCometNativeIcebergCompat])) { ++ ignore(testName + " (disabled for NATIVE_ICEBERG_COMPAT)", testTags: _*)(testFun) ++ return ++ } ++ if (isNativeDataFusion && ++ testTags.exists(_.isInstanceOf[IgnoreCometNativeDataFusion])) { ++ ignore(testName + " (disabled for NATIVE_DATAFUSION)", testTags: _*)(testFun) ++ return ++ } ++ if ((isNativeDataFusion || isNativeIcebergCompat) && ++ testTags.exists(_.isInstanceOf[IgnoreCometNativeScan])) { ++ ignore(testName + " (disabled for NATIVE_DATAFUSION and NATIVE_ICEBERG_COMPAT)", ++ testTags: _*)(testFun) ++ return ++ } ++ } + if (testTags.exists(_.isInstanceOf[DisableAdaptiveExecution])) { + super.test(testName, testTags: _*) { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { +@@ -242,6 +272,38 @@ private[sql] trait SQLTestUtilsBase protected override def _sqlContext: SQLContext = self.spark.sqlContext } @@ -3037,10 +3369,9 @@ index e937173a590..5fede1579f7 100644 + } + + /** -+ * Whether Spark should apply comet writer is enabled. This is only effective when ++ * Whether Spark should apply comet writer. This is only effective when + * [[isCometEnabled]] returns true. + */ -+ + protected def isCometWriterEnabled: Boolean = { + val v = System.getenv("ENABLE_COMET_WRITER") + v != null && v.toBoolean @@ -3049,7 +3380,7 @@ index e937173a590..5fede1579f7 100644 protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { SparkSession.setActiveSession(spark) super.withSQLConf(pairs: _*)(f) -@@ -435,6 +491,8 @@ private[sql] trait SQLTestUtilsBase +@@ -435,6 +497,8 @@ private[sql] trait SQLTestUtilsBase val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child @@ -3156,6 +3487,29 @@ index de3b1ffccf0..2a76d127093 100644 override def beforeEach(): Unit = { super.beforeEach() +diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala +index f3be79f9022..b4b1ea8dbc4 100644 +--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala ++++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala +@@ -34,7 +34,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectIn + import org.apache.hadoop.io.{LongWritable, Writable} + + import org.apache.spark.{SparkException, SparkFiles, TestUtils} +-import org.apache.spark.sql.{AnalysisException, QueryTest, Row} ++import org.apache.spark.sql.{AnalysisException, IgnoreCometNativeDataFusion, QueryTest, Row} + import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode + import org.apache.spark.sql.catalyst.plans.logical.Project + import org.apache.spark.sql.execution.WholeStageCodegenExec +@@ -448,7 +448,8 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { + } + } + +- test("SPARK-11522 select input_file_name from non-parquet table") { ++ test("SPARK-11522 select input_file_name from non-parquet table", ++ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3312")) { + + withTempDir { tempDir => + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 6160c3e5f6c..0956d7d9edc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala From 4d2b34289866675754edb460ca1e7b3a9272550d Mon Sep 17 00:00:00 2001 From: B Vadlamani Date: Sat, 7 Feb 2026 01:39:07 -0800 Subject: [PATCH 12/15] enable_spark_tests_comet_native_writer_fix_spark_rebase_main --- .../parquet/CometParquetWriterSuite.scala | 236 +++++++++++++++++- 1 file changed, 233 insertions(+), 3 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala index b691039f19..27fae45d01 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala @@ -20,17 +20,15 @@ package org.apache.comet.parquet import java.io.File - import scala.util.Random - import org.apache.spark.sql.{CometTestBase, DataFrame, Row} import org.apache.spark.sql.comet.{CometBatchScanExec, CometNativeScanExec, CometNativeWriteExec, CometScanExec} import org.apache.spark.sql.execution.{FileSourceScanExec, QueryExecution, SparkPlan} import org.apache.spark.sql.execution.command.DataWritingCommandExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType - import org.apache.comet.CometConf +import org.apache.comet.CometSparkSessionExtensions.isSpark40Plus import org.apache.comet.testing.{DataGenOptions, FuzzDataGenerator, SchemaGenOptions} class CometParquetWriterSuite extends CometTestBase { @@ -377,6 +375,238 @@ class CometParquetWriterSuite extends CometTestBase { } } +// NATIVE COMET WRITER TESTS WHICH FAIL IN SPARK + // https://github.com/apache/datafusion-comet/issues/3417 + ignore("Spark compat: empty file should be skipped while write to file") { + withSQLConf( + CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { + withTempPath { path => + spark.range(100).repartition(10).where("id = 50").write.parquet(path.toString) + val partFiles = path + .listFiles() + .filter(f => f.isFile && !f.getName.startsWith(".") && !f.getName.startsWith("_")) + assert(partFiles.length === 2) + } + } + } + + // https://github.com/apache/datafusion-comet/issues/3418 + ignore("Spark compat: SPARK-33901 ctas should not change table's schema") { + withSQLConf( + CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { + withTable("t1", "t2") { + sql("CREATE TABLE t1(i CHAR(5), c VARCHAR(4)) USING parquet") + sql("CREATE TABLE t2 USING parquet AS SELECT * FROM t1") + checkAnswer( + sql("desc t2").selectExpr("data_type").where("data_type like '%char%'"), + Seq(Row("char(5)"), Row("varchar(4)"))) + } + } + } + + // https://github.com/apache/datafusion-comet/issues/3419 + ignore("Spark compat: SPARK-37160 CREATE TABLE AS SELECT with CHAR_AS_VARCHAR") { + withSQLConf( + CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { + withTable("t1", "t2") { + sql("CREATE TABLE t1(col CHAR(5)) USING parquet") + withSQLConf(SQLConf.CHAR_AS_VARCHAR.key -> "true") { + sql("CREATE TABLE t2 USING parquet AS SELECT * FROM t1") + checkAnswer( + sql("desc t2").selectExpr("data_type").where("data_type like '%char%'"), + Seq(Row("varchar(5)"))) + } + } + } + } + + // https://github.com/apache/datafusion-comet/issues/3420 + ignore("Spark compat: SPARK-29174 Support LOCAL in INSERT OVERWRITE DIRECTORY") { + withSQLConf( + CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { + withTempPath { dir => + val path = dir.toURI.getPath + withTable("tab1", "tab2") { + sql(s"""create table tab1 (a int) using parquet location '$path'""") + sql("insert into tab1 values(1)") + checkAnswer(sql("select * from tab1"), Seq(Row(1))) + sql("create table tab2 (a int) using parquet") + sql("insert into tab2 values(2)") + sql(s"""insert overwrite local directory '$path' using parquet select * from tab2""") + sql("refresh table tab1") + checkAnswer(sql("select * from tab1"), Seq(Row(2))) + } + } + } + } + + // https://github.com/apache/datafusion-comet/issues/3421 + ignore("Spark compat: SPARK-38336 INSERT INTO with default columns positive tests") { + withSQLConf( + CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { + withTable("t") { + sql("create table t(i boolean, s bigint) using parquet") + sql("insert into t(i) values(true)") + checkAnswer(spark.table("t"), Row(true, null)) + } + } + } + + // https://github.com/apache/datafusion-comet/issues/3422 + ignore("Spark compat: SPARK-38811 INSERT INTO on ALTER TABLE ADD COLUMNS positive tests") { + withSQLConf( + CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { + withTable("t") { + sql("create table t(i boolean) using parquet") + sql("alter table t add column s string default concat('abc', 'def')") + sql("insert into t values(true, default)") + checkAnswer(spark.table("t"), Row(true, "abcdef")) + } + } + } + + // https://github.com/apache/datafusion-comet/issues/3423 + ignore("Spark compat: SPARK-43071 INSERT INTO from non-projection queries") { + withSQLConf( + CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { + withTable("t1", "t2") { + sql("create table t1(i boolean, s bigint default 42) using parquet") + sql("insert into t1 values (true, 41), (false, default)") + sql("create table t2(i boolean, s bigint) using parquet") + sql("insert into t2 select * from t1 order by s") + checkAnswer(spark.table("t2"), Seq(Row(true, 41), Row(false, 42))) + } + } + } + + // https://github.com/apache/datafusion-comet/issues/3424 + ignore("Spark compat: Insert overwrite table command should output correct schema basic") { + withSQLConf( + CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { + withTable("tbl", "tbl2") { + withView("view1") { + val df = spark.range(10).toDF("id") + df.write.format("parquet").saveAsTable("tbl") + spark.sql("CREATE VIEW view1 AS SELECT id FROM tbl") + spark.sql("CREATE TABLE tbl2(ID long) USING parquet") + spark.sql("INSERT OVERWRITE TABLE tbl2 SELECT ID FROM view1") + checkAnswer(spark.table("tbl2"), (0 until 10).map(Row(_))) + } + } + } + } + + // https://github.com/apache/datafusion-comet/issues/3425 + ignore("Spark compat: parquet timestamp conversion") { + withSQLConf( + CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { + withTempPath { dir => + spark + .range(1) + .selectExpr("current_timestamp() as ts") + .write + .parquet(dir.toString + "/spark") + val result = spark.read.parquet(dir.toString + "/spark").collect() + assert(result.length == 1) + } + } + } + + // https://github.com/apache/datafusion-comet/issues/3426 + ignore("Spark compat: INSERT INTO TABLE - complex type but different names") { + withSQLConf( + CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { + withTable("tab1", "tab2") { + sql("""CREATE TABLE tab1 (s struct) USING parquet""") + sql("""CREATE TABLE tab2 (s struct) USING parquet""") + sql("INSERT INTO tab1 VALUES (named_struct('a', 'x', 'b', 'y'))") + sql("INSERT INTO tab2 SELECT * FROM tab1") + checkAnswer(spark.table("tab2"), Row(Row("x", "y"))) + } + } + } + + // https://github.com/apache/datafusion-comet/issues/3427 + ignore("Spark compat: Write Spark version into Parquet metadata") { + withSQLConf( + CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { + withTempPath { dir => + spark.range(1).repartition(1).write.parquet(dir.getAbsolutePath) + val files = dir.listFiles().filter(_.getName.endsWith(".parquet")) + assert(files.nonEmpty, "Expected parquet files to be written") + } + } + } + + // https://github.com/apache/datafusion-comet/issues/3428 + ignore("Spark compat: write path implements onTaskCommit API correctly") { + withSQLConf( + CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { + withTempDir { dir => + val path = dir.getCanonicalPath + spark.range(10).repartition(10).write.mode("overwrite").parquet(path) + val files = new File(path).listFiles().filter(_.getName.startsWith("part-")) + assert(files.length > 0, "Expected part files to be written") + } + } + } + + // COMET NATIVE WRITER Spark 4.x test failures + // https://github.com/apache/datafusion-comet/issues/3429 + ignore("Spark compat: ctas with union") { + assume(isSpark40Plus) + withSQLConf( + CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { + withTable("t") { + sql("CREATE TABLE t USING parquet AS SELECT 1 AS c UNION ALL SELECT 2") + checkAnswer(spark.table("t"), Seq(Row(1), Row(2))) + } + } + } + + // https://github.com/apache/datafusion-comet/issues/3430 + ignore("Spark compat: SPARK-48817 test multi insert") { + assume(isSpark40Plus) + withSQLConf( + CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { + withTable("t1", "t2") { + sql("CREATE TABLE t1(a INT) USING parquet") + sql("CREATE TABLE t2(a INT) USING parquet") + sql("FROM (SELECT 1 AS a) src INSERT INTO t1 SELECT a INSERT INTO t2 SELECT a") + checkAnswer(spark.table("t1"), Row(1)) + checkAnswer(spark.table("t2"), Row(1)) + } + } + } + private def createTestData(inputDir: File): String = { val inputPath = new File(inputDir, "input.parquet").getAbsolutePath val schema = FuzzDataGenerator.generateSchema( From dd05d3ebc2f1fc62d9b24d3a130f01d550135fe8 Mon Sep 17 00:00:00 2001 From: B Vadlamani Date: Sat, 7 Feb 2026 10:03:48 -0800 Subject: [PATCH 13/15] enable_spark_tests_comet_native_writer_fix_spark_rebase_main --- .../parquet/CometParquetWriterSuite.scala | 155 ++++++++++++++---- 1 file changed, 125 insertions(+), 30 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala index 27fae45d01..b98a05b936 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala @@ -383,7 +383,9 @@ class CometParquetWriterSuite extends CometTestBase { CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { withTempPath { path => - spark.range(100).repartition(10).where("id = 50").write.parquet(path.toString) + assertNativeWriter { + spark.range(100).repartition(10).where("id = 50").write.parquet(path.toString) + } val partFiles = path .listFiles() .filter(f => f.isFile && !f.getName.startsWith(".") && !f.getName.startsWith("_")) @@ -400,7 +402,9 @@ class CometParquetWriterSuite extends CometTestBase { CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { withTable("t1", "t2") { sql("CREATE TABLE t1(i CHAR(5), c VARCHAR(4)) USING parquet") - sql("CREATE TABLE t2 USING parquet AS SELECT * FROM t1") + assertNativeWriter { + sql("CREATE TABLE t2 USING parquet AS SELECT * FROM t1") + } checkAnswer( sql("desc t2").selectExpr("data_type").where("data_type like '%char%'"), Seq(Row("char(5)"), Row("varchar(4)"))) @@ -417,7 +421,9 @@ class CometParquetWriterSuite extends CometTestBase { withTable("t1", "t2") { sql("CREATE TABLE t1(col CHAR(5)) USING parquet") withSQLConf(SQLConf.CHAR_AS_VARCHAR.key -> "true") { - sql("CREATE TABLE t2 USING parquet AS SELECT * FROM t1") + assertNativeWriter { + sql("CREATE TABLE t2 USING parquet AS SELECT * FROM t1") + } checkAnswer( sql("desc t2").selectExpr("data_type").where("data_type like '%char%'"), Seq(Row("varchar(5)"))) @@ -436,11 +442,17 @@ class CometParquetWriterSuite extends CometTestBase { val path = dir.toURI.getPath withTable("tab1", "tab2") { sql(s"""create table tab1 (a int) using parquet location '$path'""") - sql("insert into tab1 values(1)") + assertNativeWriter { + sql("insert into tab1 values(1)") + } checkAnswer(sql("select * from tab1"), Seq(Row(1))) sql("create table tab2 (a int) using parquet") - sql("insert into tab2 values(2)") - sql(s"""insert overwrite local directory '$path' using parquet select * from tab2""") + assertNativeWriter { + sql("insert into tab2 values(2)") + } + assertNativeWriter { + sql(s"""insert overwrite local directory '$path' using parquet select * from tab2""") + } sql("refresh table tab1") checkAnswer(sql("select * from tab1"), Seq(Row(2))) } @@ -456,7 +468,9 @@ class CometParquetWriterSuite extends CometTestBase { CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { withTable("t") { sql("create table t(i boolean, s bigint) using parquet") - sql("insert into t(i) values(true)") + assertNativeWriter { + sql("insert into t(i) values(true)") + } checkAnswer(spark.table("t"), Row(true, null)) } } @@ -471,7 +485,9 @@ class CometParquetWriterSuite extends CometTestBase { withTable("t") { sql("create table t(i boolean) using parquet") sql("alter table t add column s string default concat('abc', 'def')") - sql("insert into t values(true, default)") + assertNativeWriter { + sql("insert into t values(true, default)") + } checkAnswer(spark.table("t"), Row(true, "abcdef")) } } @@ -485,9 +501,13 @@ class CometParquetWriterSuite extends CometTestBase { CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { withTable("t1", "t2") { sql("create table t1(i boolean, s bigint default 42) using parquet") - sql("insert into t1 values (true, 41), (false, default)") + assertNativeWriter { + sql("insert into t1 values (true, 41), (false, default)") + } sql("create table t2(i boolean, s bigint) using parquet") - sql("insert into t2 select * from t1 order by s") + assertNativeWriter { + sql("insert into t2 select * from t1 order by s") + } checkAnswer(spark.table("t2"), Seq(Row(true, 41), Row(false, 42))) } } @@ -502,10 +522,14 @@ class CometParquetWriterSuite extends CometTestBase { withTable("tbl", "tbl2") { withView("view1") { val df = spark.range(10).toDF("id") - df.write.format("parquet").saveAsTable("tbl") + assertNativeWriter { + df.write.format("parquet").saveAsTable("tbl") + } spark.sql("CREATE VIEW view1 AS SELECT id FROM tbl") spark.sql("CREATE TABLE tbl2(ID long) USING parquet") - spark.sql("INSERT OVERWRITE TABLE tbl2 SELECT ID FROM view1") + assertNativeWriter { + spark.sql("INSERT OVERWRITE TABLE tbl2 SELECT ID FROM view1") + } checkAnswer(spark.table("tbl2"), (0 until 10).map(Row(_))) } } @@ -519,11 +543,13 @@ class CometParquetWriterSuite extends CometTestBase { CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { withTempPath { dir => - spark - .range(1) - .selectExpr("current_timestamp() as ts") - .write - .parquet(dir.toString + "/spark") + assertNativeWriter { + spark + .range(1) + .selectExpr("current_timestamp() as ts") + .write + .parquet(dir.toString + "/spark") + } val result = spark.read.parquet(dir.toString + "/spark").collect() assert(result.length == 1) } @@ -539,8 +565,12 @@ class CometParquetWriterSuite extends CometTestBase { withTable("tab1", "tab2") { sql("""CREATE TABLE tab1 (s struct) USING parquet""") sql("""CREATE TABLE tab2 (s struct) USING parquet""") - sql("INSERT INTO tab1 VALUES (named_struct('a', 'x', 'b', 'y'))") - sql("INSERT INTO tab2 SELECT * FROM tab1") + assertNativeWriter { + sql("INSERT INTO tab1 VALUES (named_struct('a', 'x', 'b', 'y'))") + } + assertNativeWriter { + sql("INSERT INTO tab2 SELECT * FROM tab1") + } checkAnswer(spark.table("tab2"), Row(Row("x", "y"))) } } @@ -553,7 +583,9 @@ class CometParquetWriterSuite extends CometTestBase { CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { withTempPath { dir => - spark.range(1).repartition(1).write.parquet(dir.getAbsolutePath) + assertNativeWriter { + spark.range(1).repartition(1).write.parquet(dir.getAbsolutePath) + } val files = dir.listFiles().filter(_.getName.endsWith(".parquet")) assert(files.nonEmpty, "Expected parquet files to be written") } @@ -568,7 +600,9 @@ class CometParquetWriterSuite extends CometTestBase { CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { withTempDir { dir => val path = dir.getCanonicalPath - spark.range(10).repartition(10).write.mode("overwrite").parquet(path) + assertNativeWriter { + spark.range(10).repartition(10).write.mode("overwrite").parquet(path) + } val files = new File(path).listFiles().filter(_.getName.startsWith("part-")) assert(files.length > 0, "Expected part files to be written") } @@ -584,7 +618,9 @@ class CometParquetWriterSuite extends CometTestBase { CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { withTable("t") { - sql("CREATE TABLE t USING parquet AS SELECT 1 AS c UNION ALL SELECT 2") + assertNativeWriter { + sql("CREATE TABLE t USING parquet AS SELECT 1 AS c UNION ALL SELECT 2") + } checkAnswer(spark.table("t"), Seq(Row(1), Row(2))) } } @@ -600,7 +636,9 @@ class CometParquetWriterSuite extends CometTestBase { withTable("t1", "t2") { sql("CREATE TABLE t1(a INT) USING parquet") sql("CREATE TABLE t2(a INT) USING parquet") - sql("FROM (SELECT 1 AS a) src INSERT INTO t1 SELECT a INSERT INTO t2 SELECT a") + assertNativeWriter { + sql("FROM (SELECT 1 AS a) src INSERT INTO t1 SELECT a INSERT INTO t2 SELECT a") + } checkAnswer(spark.table("t1"), Row(1)) checkAnswer(spark.table("t2"), Row(1)) } @@ -626,15 +664,72 @@ class CometParquetWriterSuite extends CometTestBase { } /** - * Captures the execution plan during a write operation. + * Executes a code block and asserts that CometNativeWriteExec is in the write plan. + * This is used for verifying native writer is called in SQL commands. * - * @param writeOp - * The write operation to execute (takes output path as parameter) - * @param outputPath - * The path to write to - * @return - * The captured execution plan + * @param block + * The code block to execute (should contain write operations) */ + private def assertNativeWriter(block: => Unit): Unit = { + var capturedPlan: Option[QueryExecution] = None + + val listener = new org.apache.spark.sql.util.QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + if (funcName == "save" || funcName.contains("command")) { + capturedPlan = Some(qe) + } + } + + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = {} + } + + spark.listenerManager.register(listener) + + try { + block + + // Wait for listener to be called with timeout + val maxWaitTimeMs = 15000 + val checkIntervalMs = 100 + val maxIterations = maxWaitTimeMs / checkIntervalMs + var iterations = 0 + + while (capturedPlan.isEmpty && iterations < maxIterations) { + Thread.sleep(checkIntervalMs) + iterations += 1 + } + + assert( + capturedPlan.isDefined, + s"Listener was not called within ${maxWaitTimeMs}ms - no execution plan captured") + + val plan = stripAQEPlan(capturedPlan.get.executedPlan) + + // Count CometNativeWriteExec instances in the plan + var nativeWriteCount = 0 + plan.foreach { + case _: CometNativeWriteExec => + nativeWriteCount += 1 + case d: DataWritingCommandExec => + d.child.foreach { + case _: CometNativeWriteExec => + nativeWriteCount += 1 + case _ => + } + case _ => + } + + assert( + nativeWriteCount == 1, + s"Expected exactly one CometNativeWriteExec in the plan, but found $nativeWriteCount:\n${plan.treeString}") + } finally { + spark.listenerManager.unregister(listener) + } + } + private def captureWritePlan(writeOp: String => Unit, outputPath: String): SparkPlan = { var capturedPlan: Option[QueryExecution] = None From 370d98dd34752fa80f9cf6db9c37a99231656145 Mon Sep 17 00:00:00 2001 From: B Vadlamani Date: Sat, 7 Feb 2026 11:15:47 -0800 Subject: [PATCH 14/15] enable_spark_tests_comet_native_writer_fix_spark_rebase_main --- .github/workflows/spark_sql_test.yml | 2 +- dev/diffs/3.4.3.diff | 202 +----------------- dev/diffs/3.5.8.diff | 296 +++------------------------ dev/diffs/4.0.1.diff | 296 +-------------------------- 4 files changed, 39 insertions(+), 757 deletions(-) diff --git a/.github/workflows/spark_sql_test.yml b/.github/workflows/spark_sql_test.yml index 115fb92f3f..610baa9f2d 100644 --- a/.github/workflows/spark_sql_test.yml +++ b/.github/workflows/spark_sql_test.yml @@ -155,7 +155,7 @@ jobs: run: | cd apache-spark rm -rf /root/.m2/repository/org/apache/parquet # somehow parquet cache requires cleanups - NOLINT_ON_COMPILE=true ENABLE_COMET=true ENABLE_COMET_ONHEAP=true ENABLE_COMET_WRITER=true COMET_PARQUET_SCAN_IMPL=${{ matrix.config.scan-impl }} ENABLE_COMET_LOG_FALLBACK_REASONS=${{ github.event.inputs.collect-fallback-logs || 'false' }} \ + NOLINT_ON_COMPILE=true ENABLE_COMET=true ENABLE_COMET_ONHEAP=true COMET_PARQUET_SCAN_IMPL=${{ matrix.config.scan-impl }} ENABLE_COMET_LOG_FALLBACK_REASONS=${{ github.event.inputs.collect-fallback-logs || 'false' }} \ build/sbt -Dsbt.log.noformat=true ${{ matrix.module.args1 }} "${{ matrix.module.args2 }}" if [ "${{ github.event.inputs.collect-fallback-logs }}" = "true" ]; then find . -type f -name "unit-tests.log" -print0 | xargs -0 grep -h "Comet cannot accelerate" | sed 's/.*Comet cannot accelerate/Comet cannot accelerate/' | sort -u > fallback.log diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index c524e6a94d..0205888433 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -133,17 +133,6 @@ index db587dd9868..aac7295a53d 100644 case _ => Map[String, String]() } new SparkPlanInfo( -diff --git a/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql b/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql -index b62cbf64323..8d1f0cb7d20 100644 ---- a/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql -+++ b/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql -@@ -1,3 +1,6 @@ -+-- TODO: support empty table write / CTAS in native parquet writer -+--SET spark.comet.parquet.write.enabled = false -+ - create table char_tbl(c char(5), v varchar(6)) using parquet; - desc formatted char_tbl; - desc formatted char_tbl c; diff --git a/sql/core/src/test/resources/sql-tests/inputs/explain-aqe.sql b/sql/core/src/test/resources/sql-tests/inputs/explain-aqe.sql index 7aef901da4f..f3d6e18926d 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/explain-aqe.sql @@ -1883,62 +1872,6 @@ index 593bd7bb4ba..32af28b0238 100644 } assert(shuffles2.size == 4) val smj2 = findTopLevelSortMergeJoin(adaptive2) -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala -index f77b6336b81..b703603d26b 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala -@@ -18,7 +18,7 @@ - package org.apache.spark.sql.execution.command - - import org.apache.spark.SparkConf --import org.apache.spark.sql.{AnalysisException, QueryTest, Row} -+import org.apache.spark.sql.{AnalysisException, IgnoreComet, QueryTest, Row} - import org.apache.spark.sql.catalyst.util.CharVarcharUtils - import org.apache.spark.sql.connector.catalog.InMemoryPartitionTableCatalog - import org.apache.spark.sql.internal.SQLConf -@@ -112,7 +112,8 @@ trait CharVarcharDDLTestBase extends QueryTest with SQLTestUtils { - } - } - -- test("SPARK-33901: ctas should should not change table's schema") { -+ test("SPARK-33901: ctas should should not change table's schema", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3418")) { - withTable("t1", "t2") { - sql(s"CREATE TABLE t1(i CHAR(5), c VARCHAR(4)) USING $format") - sql(s"CREATE TABLE t2 USING $format AS SELECT * FROM t1") -@@ -129,7 +130,8 @@ trait CharVarcharDDLTestBase extends QueryTest with SQLTestUtils { - } - } - -- test("SPARK-37160: CREATE TABLE AS SELECT with CHAR_AS_VARCHAR") { -+ test("SPARK-37160: CREATE TABLE AS SELECT with CHAR_AS_VARCHAR", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3419")) { - withTable("t1", "t2") { - sql(s"CREATE TABLE t1(col CHAR(5)) USING $format") - checkTableSchemaTypeStr("t1", Seq(Row("char(5)"))) -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala -index 343b59a311e..9d5789c1d91 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala -@@ -17,7 +17,7 @@ - - package org.apache.spark.sql.execution.datasources - --import org.apache.spark.sql.{QueryTest, Row} -+import org.apache.spark.sql.{IgnoreComet, QueryTest, Row} - import org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest - import org.apache.spark.sql.test.SharedSparkSession - -@@ -28,7 +28,8 @@ class FileFormatWriterSuite - - import testImplicits._ - -- test("empty file should be skipped while write to file") { -+ test("empty file should be skipped while write to file", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3417")) { - withTempPath { path => - spark.range(100).repartition(10).where("id = 50").write.parquet(path.toString) - val partFiles = path.listFiles() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index bd9c79e5b96..2ada8c28842 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -2159,7 +2092,7 @@ index 104b4e416cd..37ea65081e4 100644 case _ => throw new AnalysisException("Can not match ParquetTable in the query.") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -index 8670d95c65e..3fe49802309 100644 +index 8670d95c65e..b624c3811dd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -1335,7 +1335,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession @@ -2172,41 +2105,6 @@ index 8670d95c65e..3fe49802309 100644 withAllParquetReaders { checkAnswer( // "fruit" column in this file is encoded using DELTA_LENGTH_BYTE_ARRAY. -@@ -1541,7 +1542,9 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession - } - } - -- test("Write Spark version into Parquet metadata") { -+// TODO : Comet native writer to add spark / comet version into parquet metadata -+ test("Write Spark version into Parquet metadata", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3427")) { - withTempPath { dir => - spark.range(1).repartition(1).write.parquet(dir.getAbsolutePath) - assert(getMetaData(dir)(SPARK_VERSION_METADATA_KEY) === SPARK_VERSION_SHORT) -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala -index 8b386e8f689..28ced6209e0 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala -@@ -25,7 +25,7 @@ import org.apache.hadoop.fs.{Path, PathFilter} - import org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER - import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName - --import org.apache.spark.sql.Row -+import org.apache.spark.sql.{IgnoreComet, Row} - import org.apache.spark.sql.catalyst.util.DateTimeUtils - import org.apache.spark.sql.internal.SQLConf - import org.apache.spark.sql.test.SharedSparkSession -@@ -153,7 +153,9 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS - } - } - -- test("parquet timestamp conversion") { -+ // TODO : Support legacy timestamps conversion /cast in comet native writer -+ test("parquet timestamp conversion", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3425")) { - // Make a table with one parquet file written by impala, and one parquet file written by spark. - // We should only adjust the timestamps in the impala file, and only if the conf is set - val impalaFile = "test-data/impala_timestamp.parq" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index 29cb224c878..44837aa953b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -2735,60 +2633,6 @@ index 1f55742cd67..f20129d9dd8 100644 assert(bucketedScan.length == expectedNumBucketedScan) } -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala -index 2207661478d..dc4e4b4240c 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala -@@ -237,7 +237,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { - } - } - -- test("INSERT INTO TABLE - complex type but different names") { -+ test("INSERT INTO TABLE - complex type but different names", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3426")) { - val tab1 = "tab1" - val tab2 = "tab2" - withTable(tab1, tab2) { -@@ -889,7 +890,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { - assert(message.contains("target table has 2 column(s) but the inserted data has 1 column(s)")) - } - -- test("SPARK-38336 INSERT INTO statements with tables with default columns: positive tests") { -+ test("SPARK-38336 INSERT INTO statements with tables with default columns: positive tests", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3421")) { - // When the USE_NULLS_FOR_MISSING_DEFAULT_COLUMN_VALUES configuration is enabled, and no - // explicit DEFAULT value is available when the INSERT INTO statement provides fewer - // values than expected, NULL values are appended in their place. -@@ -1286,7 +1288,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { - } - } - -- test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Positive tests") { -+ test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Positive tests", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3422")) { - // There is a complex expression in the default value. - val createTableBooleanCol = "create table t(i boolean) using parquet" - val createTableIntCol = "create table t(i int) using parquet" -@@ -1984,7 +1987,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { - } - } - -- test("SPARK-43071: INSERT INTO from queries whose final operators are not projections") { -+ test("SPARK-43071: INSERT INTO from queries whose final operators are not projections", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3423")) { - def runTest(insert: String, expected: Seq[Row]): Unit = { - withTable("t1", "t2") { - sql("create table t1(i boolean, s bigint default 42) using parquet") -@@ -2052,7 +2056,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { - } - } - -- test("SPARK-29174 Support LOCAL in INSERT OVERWRITE DIRECTORY to data source") { -+ test("SPARK-29174 Support LOCAL in INSERT OVERWRITE DIRECTORY to data source", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3420")) { - withTempPath { dir => - val path = dir.toURI.getPath - sql(s"""create table tab1 ( a int) using parquet location '$path'""") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 75f440caefc..36b1146bc3a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -2959,32 +2803,8 @@ index abe606ad9c1..2d930b64cca 100644 val tblSourceName = "tbl_src" val tblTargetName = "tbl_target" val tblSourceQualified = s"default.$tblSourceName" -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala -index 44c9fbadfac..5f98bb9be17 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala -@@ -519,7 +519,8 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with - Option(dir).map(spark.read.format("org.apache.spark.sql.test").load) - } - -- test("write path implements onTaskCommit API correctly") { -+ test("write path implements onTaskCommit API correctly", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3428")) { - withSQLConf( - SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> - classOf[MessageCapturingCommitProtocol].getCanonicalName) { -@@ -1069,7 +1070,8 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with - } - } - -- test("Insert overwrite table command should output correct schema: basic") { -+ test("Insert overwrite table command should output correct schema: basic", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3424")) { - withTable("tbl", "tbl2") { - withView("view1") { - val df = spark.range(10).toDF("id") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala -index dd55fcfe42c..e898fc33bab 100644 +index dd55fcfe42c..a1d390c93d0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -27,6 +27,7 @@ import scala.concurrent.duration._ @@ -3042,7 +2862,7 @@ index dd55fcfe42c..e898fc33bab 100644 } } -@@ -242,6 +265,34 @@ private[sql] trait SQLTestUtilsBase +@@ -242,6 +265,29 @@ private[sql] trait SQLTestUtilsBase protected override def _sqlContext: SQLContext = self.spark.sqlContext } @@ -3068,16 +2888,11 @@ index dd55fcfe42c..e898fc33bab 100644 + val v = System.getenv("ENABLE_COMET_SCAN_ONLY") + v != null && v.toBoolean + } -+ -+ protected def isCometWriterEnabled: Boolean = { -+ val v = System.getenv("ENABLE_COMET_WRITER") -+ v != null && v.toBoolean -+ } + protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { SparkSession.setActiveSession(spark) super.withSQLConf(pairs: _*)(f) -@@ -434,6 +485,8 @@ private[sql] trait SQLTestUtilsBase +@@ -434,6 +480,8 @@ private[sql] trait SQLTestUtilsBase val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child @@ -3087,10 +2902,10 @@ index dd55fcfe42c..e898fc33bab 100644 spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -index ed2e309fa07..9c5c393ad14 100644 +index ed2e309fa07..a5ea58146ad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -@@ -74,6 +74,36 @@ trait SharedSparkSessionBase +@@ -74,6 +74,31 @@ trait SharedSparkSessionBase // this rule may potentially block testing of other optimization rules such as // ConstantPropagation etc. .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) @@ -3118,11 +2933,6 @@ index ed2e309fa07..9c5c393ad14 100644 + conf + .set("spark.sql.ansi.enabled", "true") + } -+ -+ if (isCometWriterEnabled) { -+ conf.set("spark.comet.parquet.write.enabled", "true") -+ conf.set("spark.comet.operator.DataWritingCommandExec.allowIncompatible", "true") -+ } + } conf.set( StaticSQLConf.WAREHOUSE_PATH, diff --git a/dev/diffs/3.5.8.diff b/dev/diffs/3.5.8.diff index 4997d8b765..beef445490 100644 --- a/dev/diffs/3.5.8.diff +++ b/dev/diffs/3.5.8.diff @@ -1969,62 +1969,6 @@ index 2f8e401e743..a4f94417dcc 100644 plan.inputPlan.output.zip(plan.finalPhysicalPlan.output).foreach { case (o1, o2) => assert(o1.semanticEquals(o2), "Different output column order after AQE optimization") } -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala -index 12d5870309f..c65c688d56d 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala -@@ -18,7 +18,7 @@ - package org.apache.spark.sql.execution.command - - import org.apache.spark.SparkConf --import org.apache.spark.sql.{AnalysisException, QueryTest, Row} -+import org.apache.spark.sql.{AnalysisException, IgnoreComet, QueryTest, Row} - import org.apache.spark.sql.catalyst.util.CharVarcharUtils - import org.apache.spark.sql.connector.catalog.InMemoryPartitionTableCatalog - import org.apache.spark.sql.internal.SQLConf -@@ -162,7 +162,8 @@ trait CharVarcharDDLTestBase extends QueryTest with SQLTestUtils { - } - } - -- test("SPARK-33901: ctas should should not change table's schema") { -+ test("SPARK-33901: ctas should should not change table's schema", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3418")) { - withTable("t1", "t2") { - sql(s"CREATE TABLE t1(i CHAR(5), c VARCHAR(4)) USING $format") - sql(s"CREATE TABLE t2 USING $format AS SELECT * FROM t1") -@@ -179,7 +180,8 @@ trait CharVarcharDDLTestBase extends QueryTest with SQLTestUtils { - } - } - -- test("SPARK-37160: CREATE TABLE AS SELECT with CHAR_AS_VARCHAR") { -+ test("SPARK-37160: CREATE TABLE AS SELECT with CHAR_AS_VARCHAR", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3419")) { - withTable("t1", "t2") { - sql(s"CREATE TABLE t1(col CHAR(5)) USING $format") - checkTableSchemaTypeStr("t1", Seq(Row("char(5)"))) -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala -index 343b59a311e..fa6ed7be6f2 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala -@@ -17,7 +17,7 @@ - - package org.apache.spark.sql.execution.datasources - --import org.apache.spark.sql.{QueryTest, Row} -+import org.apache.spark.sql.{IgnoreComet, QueryTest, Row} - import org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest - import org.apache.spark.sql.test.SharedSparkSession - -@@ -28,7 +28,8 @@ class FileFormatWriterSuite - - import testImplicits._ - -- test("empty file should be skipped while write to file") { -+ test("empty file should be skipped while write to file", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3417")) { - withTempPath { path => - spark.range(100).repartition(10).where("id = 50").write.parquet(path.toString) - val partFiles = path.listFiles() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index fd52d038ca6..154c800be67 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -2268,7 +2212,7 @@ index 8e88049f51e..49f2001dc6b 100644 case _ => throw new AnalysisException("Can not match ParquetTable in the query.") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -index 8ed9ef1630e..d1ac5344638 100644 +index 8ed9ef1630e..f312174b182 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -1064,7 +1064,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession @@ -2301,39 +2245,6 @@ index 8ed9ef1630e..d1ac5344638 100644 withAllParquetReaders { checkAnswer( // "fruit" column in this file is encoded using DELTA_LENGTH_BYTE_ARRAY. -@@ -1551,7 +1554,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession - } - } - -- test("Write Spark version into Parquet metadata") { -+ test("Write Spark version into Parquet metadata", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3427")) { - withTempPath { dir => - spark.range(1).repartition(1).write.parquet(dir.getAbsolutePath) - assert(getMetaData(dir)(SPARK_VERSION_METADATA_KEY) === SPARK_VERSION_SHORT) -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala -index 8b386e8f689..76c3005fcd4 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala -@@ -25,7 +25,7 @@ import org.apache.hadoop.fs.{Path, PathFilter} - import org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER - import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName - --import org.apache.spark.sql.Row -+import org.apache.spark.sql.{IgnoreComet, Row} - import org.apache.spark.sql.catalyst.util.DateTimeUtils - import org.apache.spark.sql.internal.SQLConf - import org.apache.spark.sql.test.SharedSparkSession -@@ -153,7 +153,8 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS - } - } - -- test("parquet timestamp conversion") { -+ test("parquet timestamp conversion", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3425")) { - // Make a table with one parquet file written by impala, and one parquet file written by spark. - // We should only adjust the timestamps in the impala file, and only if the conf is set - val impalaFile = "test-data/impala_timestamp.parq" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index f6472ba3d9d..ce39ebb52e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -2676,7 +2587,7 @@ index d083cac48ff..3c11bcde807 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -index 746f289c393..5b9e31c1fa6 100644 +index 746f289c393..7a6a88a9fce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -19,16 +19,19 @@ package org.apache.spark.sql.sources @@ -2701,7 +2612,7 @@ index 746f289c393..5b9e31c1fa6 100644 import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -@@ -102,12 +105,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -102,12 +105,22 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } } @@ -2711,6 +2622,7 @@ index 746f289c393..5b9e31c1fa6 100644 + val fileScan = collect(plan) { + case f: FileSourceScanExec => f + case f: CometScanExec => f ++ case f: CometNativeScanExec => f + } assert(fileScan.nonEmpty, plan) fileScan.head @@ -2719,12 +2631,13 @@ index 746f289c393..5b9e31c1fa6 100644 + private def getBucketScan(plan: SparkPlan): Boolean = getFileScan(plan) match { + case fs: FileSourceScanExec => fs.bucketedScan + case bs: CometScanExec => bs.bucketedScan ++ case ns: CometNativeScanExec => ns.bucketedScan + } + // To verify if the bucket pruning works, this function checks two conditions: // 1) Check if the pruned buckets (before filtering) are empty. // 2) Verify the final result is the same as the expected one -@@ -156,7 +167,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -156,7 +169,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val planWithoutBucketedScan = bucketedDataFrame.filter(filterCondition) .queryExecution.executedPlan val fileScan = getFileScan(planWithoutBucketedScan) @@ -2734,7 +2647,7 @@ index 746f289c393..5b9e31c1fa6 100644 val bucketColumnType = bucketedDataFrame.schema.apply(bucketColumnIndex).dataType val rowsWithInvalidBuckets = fileScan.execute().filter(row => { -@@ -452,28 +464,54 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -452,28 +466,54 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val joinOperator = if (joined.sqlContext.conf.adaptiveExecutionEnabled) { val executedPlan = joined.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan @@ -2797,14 +2710,7 @@ index 746f289c393..5b9e31c1fa6 100644 s"expected sort in the right child to be $sortRight but found\n${joinOperator.right}") // check the output partitioning -@@ -831,16 +869,17 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti - } - } - -- test("disable bucketing when the output doesn't contain all bucketing columns") { -+ test("disable bucketing when the output doesn't contain all bucketing columns", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3319")) { - withTable("bucketed_table") { +@@ -836,11 +876,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") val scanDF = spark.table("bucketed_table").select("j") @@ -2818,7 +2724,7 @@ index 746f289c393..5b9e31c1fa6 100644 checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) } } -@@ -895,7 +934,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -895,7 +935,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } test("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") { @@ -2829,7 +2735,7 @@ index 746f289c393..5b9e31c1fa6 100644 SQLConf.SHUFFLE_PARTITIONS.key -> "5", SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "7") { val bucketSpec = Some(BucketSpec(6, Seq("i", "j"), Nil)) -@@ -914,7 +956,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -914,7 +957,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } test("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") { @@ -2840,7 +2746,7 @@ index 746f289c393..5b9e31c1fa6 100644 SQLConf.SHUFFLE_PARTITIONS.key -> "9", SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10") { -@@ -944,7 +989,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -944,7 +990,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } test("bucket coalescing eliminates shuffle") { @@ -2851,17 +2757,7 @@ index 746f289c393..5b9e31c1fa6 100644 SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. -@@ -1013,7 +1061,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti - } - } - -- test("bucket coalescing is applied when join expressions match with partitioning expressions") { -+ test("bucket coalescing is applied when join expressions match with partitioning expressions", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3319")) { - withTable("t1", "t2", "t3") { - df1.write.format("parquet").bucketBy(8, "i", "j").saveAsTable("t1") - df2.write.format("parquet").bucketBy(4, "i", "j").saveAsTable("t2") -@@ -1029,15 +1078,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -1029,15 +1078,24 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti Seq(true, false).foreach { aqeEnabled => withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled.toString) { val plan = sql(query).queryExecution.executedPlan @@ -2872,6 +2768,7 @@ index 746f289c393..5b9e31c1fa6 100644 val scans = collect(plan) { case f: FileSourceScanExec if f.optionalNumCoalescedBuckets.isDefined => f + case b: CometScanExec if b.optionalNumCoalescedBuckets.isDefined => b ++ case b: CometNativeScanExec if b.optionalNumCoalescedBuckets.isDefined => b } if (expectedCoalescedNumBuckets.isDefined) { assert(scans.length == 1) @@ -2881,6 +2778,8 @@ index 746f289c393..5b9e31c1fa6 100644 + assert(f.optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) + case b: CometScanExec => + assert(b.optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) ++ case b: CometNativeScanExec => ++ assert(b.optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) + } } else { assert(scans.isEmpty) @@ -2910,20 +2809,18 @@ index 6f897a9c0b7..b0723634f68 100644 protected override lazy val sql = spark.sql _ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala -index d675503a8ba..c386a8cb686 100644 +index d675503a8ba..f220892396e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala -@@ -17,7 +17,8 @@ - +@@ -18,6 +18,7 @@ package org.apache.spark.sql.sources --import org.apache.spark.sql.QueryTest -+import org.apache.spark.sql.{IgnoreCometNativeDataFusion, QueryTest} -+import org.apache.spark.sql.comet.CometScanExec + import org.apache.spark.sql.QueryTest ++import org.apache.spark.sql.comet.{CometNativeScanExec, CometScanExec} import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} import org.apache.spark.sql.internal.SQLConf -@@ -68,7 +69,10 @@ abstract class DisableUnnecessaryBucketedScanSuite +@@ -68,7 +69,11 @@ abstract class DisableUnnecessaryBucketedScanSuite def checkNumBucketedScan(query: String, expectedNumBucketedScan: Int): Unit = { val plan = sql(query).queryExecution.executedPlan @@ -2931,114 +2828,11 @@ index d675503a8ba..c386a8cb686 100644 + val bucketedScan = collect(plan) { + case s: FileSourceScanExec if s.bucketedScan => s + case s: CometScanExec if s.bucketedScan => s ++ case s: CometNativeScanExec if s.bucketedScan => s + } assert(bucketedScan.length == expectedNumBucketedScan) } -@@ -83,7 +87,8 @@ abstract class DisableUnnecessaryBucketedScanSuite - } - } - -- test("SPARK-32859: disable unnecessary bucketed table scan - basic test") { -+ test("SPARK-32859: disable unnecessary bucketed table scan - basic test", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3319")) { - withTable("t1", "t2", "t3") { - df1.write.format("parquet").bucketBy(8, "i").saveAsTable("t1") - df2.write.format("parquet").bucketBy(8, "i").saveAsTable("t2") -@@ -124,7 +129,8 @@ abstract class DisableUnnecessaryBucketedScanSuite - } - } - -- test("SPARK-32859: disable unnecessary bucketed table scan - multiple joins test") { -+ test("SPARK-32859: disable unnecessary bucketed table scan - multiple joins test", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3319")) { - withTable("t1", "t2", "t3") { - df1.write.format("parquet").bucketBy(8, "i").saveAsTable("t1") - df2.write.format("parquet").bucketBy(8, "i").saveAsTable("t2") -@@ -167,7 +173,8 @@ abstract class DisableUnnecessaryBucketedScanSuite - } - } - -- test("SPARK-32859: disable unnecessary bucketed table scan - multiple bucketed columns test") { -+ test("SPARK-32859: disable unnecessary bucketed table scan - multiple bucketed columns test", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3319")) { - withTable("t1", "t2", "t3") { - df1.write.format("parquet").bucketBy(8, "i", "j").saveAsTable("t1") - df2.write.format("parquet").bucketBy(8, "i", "j").saveAsTable("t2") -@@ -198,7 +205,8 @@ abstract class DisableUnnecessaryBucketedScanSuite - } - } - -- test("SPARK-32859: disable unnecessary bucketed table scan - other operators test") { -+ test("SPARK-32859: disable unnecessary bucketed table scan - other operators test", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3319")) { - withTable("t1", "t2", "t3") { - df1.write.format("parquet").bucketBy(8, "i").saveAsTable("t1") - df2.write.format("parquet").bucketBy(8, "i").saveAsTable("t2") -@@ -239,7 +247,8 @@ abstract class DisableUnnecessaryBucketedScanSuite - } - } - -- test("Aggregates with no groupby over tables having 1 BUCKET, return multiple rows") { -+ test("Aggregates with no groupby over tables having 1 BUCKET, return multiple rows", -+ IgnoreCometNativeDataFusion("https://github.com/apache/datafusion-comet/issues/3319")) { - withTable("t1") { - withSQLConf(SQLConf.AUTO_BUCKETED_SCAN_ENABLED.key -> "true") { - sql( -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala -index 7b1a5a32037..fb5fb599158 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala -@@ -262,7 +262,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { - } - } - -- test("INSERT INTO TABLE - complex type but different names") { -+ test("INSERT INTO TABLE - complex type but different names", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3426")) { - val tab1 = "tab1" - val tab2 = "tab2" - withTable(tab1, tab2) { -@@ -997,7 +998,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { - } - } - -- test("SPARK-38336 INSERT INTO statements with tables with default columns: positive tests") { -+ test("SPARK-38336 INSERT INTO statements with tables with default columns: positive tests", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3421")) { - // When the INSERT INTO statement provides fewer values than expected, NULL values are appended - // in their place. - withTable("t") { -@@ -1470,7 +1472,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { - } - } - -- test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Positive tests") { -+ test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Positive tests", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3422")) { - // There is a complex expression in the default value. - val createTableBooleanCol = "create table t(i boolean) using parquet" - val createTableIntCol = "create table t(i int) using parquet" -@@ -2263,7 +2266,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { - } - } - -- test("SPARK-43071: INSERT INTO from queries whose final operators are not projections") { -+ test("SPARK-43071: INSERT INTO from queries whose final operators are not projections", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3423")) { - def runTest(insert: String, expected: Seq[Row]): Unit = { - withTable("t1", "t2") { - sql("create table t1(i boolean, s bigint default 42) using parquet") -@@ -2331,7 +2335,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { - } - } - -- test("SPARK-29174 Support LOCAL in INSERT OVERWRITE DIRECTORY to data source") { -+ test("SPARK-29174 Support LOCAL in INSERT OVERWRITE DIRECTORY to data source", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3420")) { - withTempPath { dir => - val path = dir.toURI.getPath - sql(s"""create table tab1 ( a int) using parquet location '$path'""") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 7f6fa2a123e..c778b4e2c48 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -3262,32 +3056,8 @@ index abe606ad9c1..2d930b64cca 100644 val tblSourceName = "tbl_src" val tblTargetName = "tbl_target" val tblSourceQualified = s"default.$tblSourceName" -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala -index b40f9210a68..f05a41168a0 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala -@@ -538,7 +538,8 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with - Option(dir).map(spark.read.format("org.apache.spark.sql.test").load) - } - -- test("write path implements onTaskCommit API correctly") { -+ test("write path implements onTaskCommit API correctly", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3428")) { - withSQLConf( - SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> - classOf[MessageCapturingCommitProtocol].getCanonicalName) { -@@ -1088,7 +1089,8 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with - } - } - -- test("Insert overwrite table command should output correct schema: basic") { -+ test("Insert overwrite table command should output correct schema: basic", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3424")) { - withTable("tbl", "tbl2") { - withView("view1") { - val df = spark.range(10).toDF("id") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala -index e937173a590..14501d17dcd 100644 +index e937173a590..7d20538bc68 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -27,6 +27,7 @@ import scala.concurrent.duration._ @@ -3341,7 +3111,7 @@ index e937173a590..14501d17dcd 100644 if (testTags.exists(_.isInstanceOf[DisableAdaptiveExecution])) { super.test(testName, testTags: _*) { withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { -@@ -242,6 +272,38 @@ private[sql] trait SQLTestUtilsBase +@@ -242,6 +272,29 @@ private[sql] trait SQLTestUtilsBase protected override def _sqlContext: SQLContext = self.spark.sqlContext } @@ -3367,20 +3137,11 @@ index e937173a590..14501d17dcd 100644 + val v = System.getenv("ENABLE_COMET_SCAN_ONLY") + v != null && v.toBoolean + } -+ -+ /** -+ * Whether Spark should apply comet writer. This is only effective when -+ * [[isCometEnabled]] returns true. -+ */ -+ protected def isCometWriterEnabled: Boolean = { -+ val v = System.getenv("ENABLE_COMET_WRITER") -+ v != null && v.toBoolean -+ } + protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { SparkSession.setActiveSession(spark) super.withSQLConf(pairs: _*)(f) -@@ -435,6 +497,8 @@ private[sql] trait SQLTestUtilsBase +@@ -435,6 +488,8 @@ private[sql] trait SQLTestUtilsBase val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child @@ -3390,10 +3151,10 @@ index e937173a590..14501d17dcd 100644 spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -index ed2e309fa07..9c5c393ad14 100644 +index ed2e309fa07..a5ea58146ad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -@@ -74,6 +74,36 @@ trait SharedSparkSessionBase +@@ -74,6 +74,31 @@ trait SharedSparkSessionBase // this rule may potentially block testing of other optimization rules such as // ConstantPropagation etc. .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) @@ -3421,11 +3182,6 @@ index ed2e309fa07..9c5c393ad14 100644 + conf + .set("spark.sql.ansi.enabled", "true") + } -+ -+ if (isCometWriterEnabled) { -+ conf.set("spark.comet.parquet.write.enabled", "true") -+ conf.set("spark.comet.operator.DataWritingCommandExec.allowIncompatible", "true") -+ } + } conf.set( StaticSQLConf.WAREHOUSE_PATH, diff --git a/dev/diffs/4.0.1.diff b/dev/diffs/4.0.1.diff index e36af18d18..d6694e827f 100644 --- a/dev/diffs/4.0.1.diff +++ b/dev/diffs/4.0.1.diff @@ -1,54 +1,3 @@ -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/pom.xml b/pom.xml index 22922143fc3..7c56e5e8641 100644 --- a/pom.xml @@ -181,17 +130,6 @@ index 7aca17dcb25..8afeb3b4a2f 100644 -- !query WITH t(c1) AS (SELECT replace(listagg(DISTINCT col1 COLLATE unicode_rtrim) COLLATE utf8_binary, ' ', '') FROM (VALUES ('xbc '), ('xbc '), ('a'), ('xbc'))) SELECT len(c1), regexp_count(c1, 'a'), regexp_count(c1, 'xbc') FROM t -- !query analysis -diff --git a/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql b/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql -index be038e1083c..58b512a3c17 100644 ---- a/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql -+++ b/sql/core/src/test/resources/sql-tests/inputs/charvarchar.sql -@@ -1,3 +1,6 @@ -+-- TODO: support empty table write / CTAS in native parquet writer -+--SET spark.comet.parquet.write.enabled = false -+ - create table char_tbl(c char(5), v varchar(6)) using parquet; - desc formatted char_tbl; - desc formatted char_tbl c; diff --git a/sql/core/src/test/resources/sql-tests/inputs/collations.sql b/sql/core/src/test/resources/sql-tests/inputs/collations.sql index 17815ed5dde..baad440b1ce 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/collations.sql @@ -1118,19 +1056,6 @@ index ad424b3a7cc..4ece0117a34 100644 protected val baseResourcePath = { // use the same way as `SQLQueryTestSuite` to get the resource path -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala -index 818c7ec65b1..0d333807b70 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala -@@ -524,7 +524,7 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils with AdaptiveSparkP - } - } - -- test("SPARK-48817: test multi inserts") { -+ test("SPARK-48817: test multi inserts", IgnoreComet("https://github.com/apache/datafusion-comet/issues/3430")) { - withTable("t1", "t2", "t3") { - createTable("t1", Seq("i"), Seq("int")) - createTable("t2", Seq("i"), Seq("int")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index f294ff81021..8a3b818ee94 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -1447,28 +1372,6 @@ index 11e9547dfc5..be9ae40ab3d 100644 val t1 = "T_1" val t2 = "T_2" -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala -index 77ed8b411e3..b82f70c0071 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/collation/DefaultCollationTestSuite.scala -@@ -17,7 +17,7 @@ - - package org.apache.spark.sql.collation - --import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} -+import org.apache.spark.sql.{AnalysisException, DataFrame, IgnoreComet, QueryTest, Row} - import org.apache.spark.sql.catalyst.expressions.AttributeReference - import org.apache.spark.sql.catalyst.plans.logical.Project - import org.apache.spark.sql.catalyst.util.CollationFactory -@@ -220,7 +220,7 @@ abstract class DefaultCollationTestSuite extends QueryTest with SharedSparkSessi - } - } - -- test("ctas with union") { -+ test("ctas with union", IgnoreComet("https://github.com/apache/datafusion-comet/issues/3429")) { - withTable(testTable) { - sql(s""" - |CREATE TABLE $testTable USING $dataSource AS diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala index 3eeed2e4175..9f21d547c1c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala @@ -2603,64 +2506,6 @@ index 272be70f9fe..06957694002 100644 }.isEmpty) assert(collect(initialExecutedPlan) { case i: InMemoryTableScanLike => i -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala -index a5bb3058bed..de51b21f21a 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CharVarcharDDLTestBase.scala -@@ -18,7 +18,7 @@ - package org.apache.spark.sql.execution.command - - import org.apache.spark.SparkConf --import org.apache.spark.sql.{AnalysisException, QueryTest, Row} -+import org.apache.spark.sql.{AnalysisException, IgnoreComet, QueryTest, Row} - import org.apache.spark.sql.catalyst.util.CharVarcharUtils - import org.apache.spark.sql.connector.catalog.InMemoryPartitionTableCatalog - import org.apache.spark.sql.internal.SQLConf -@@ -150,7 +150,9 @@ trait CharVarcharDDLTestBase extends QueryTest with SQLTestUtils { - } - } - -- test("SPARK-33901: ctas should should not change table's schema") { -+// TODO support CTAS in comet native parquet writer -+ test("SPARK-33901: ctas should should not change table's schema", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3418")) { - withTable("t1", "t2") { - sql(s"CREATE TABLE t1(i CHAR(5), c VARCHAR(4)) USING $format") - sql(s"CREATE TABLE t2 USING $format AS SELECT * FROM t1") -@@ -167,7 +169,8 @@ trait CharVarcharDDLTestBase extends QueryTest with SQLTestUtils { - } - } - -- test("SPARK-37160: CREATE TABLE AS SELECT with CHAR_AS_VARCHAR") { -+ test("SPARK-37160: CREATE TABLE AS SELECT with CHAR_AS_VARCHAR", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3419")) { - withTable("t1", "t2") { - sql(s"CREATE TABLE t1(col CHAR(5)) USING $format") - checkTableSchemaTypeStr("t1", Seq(Row("char(5)"))) -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala -index 343b59a311e..05f3cab13f9 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala -@@ -17,7 +17,7 @@ - - package org.apache.spark.sql.execution.datasources - --import org.apache.spark.sql.{QueryTest, Row} -+import org.apache.spark.sql.{IgnoreComet, QueryTest, Row} - import org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest - import org.apache.spark.sql.test.SharedSparkSession - -@@ -28,7 +28,9 @@ class FileFormatWriterSuite - - import testImplicits._ - -- test("empty file should be skipped while write to file") { -+// TODO :: comet native write does not write empty files / dirs -+ test("empty file should be skipped while write to file", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3417")) { - withTempPath { path => - spark.range(100).repartition(10).where("id = 50").write.parquet(path.toString) - val partFiles = path.listFiles() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index 0a0b23d1e60..5685926250f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -2889,7 +2734,7 @@ index 6080a5e8e4b..9aa8f49a62b 100644 case _ => assert(false, "Can not match ParquetTable in the query.") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala -index 4474ec1fd42..d0366be3597 100644 +index 4474ec1fd42..97910c4fc3a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -1344,7 +1344,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession @@ -2902,41 +2747,6 @@ index 4474ec1fd42..d0366be3597 100644 withAllParquetReaders { checkAnswer( // "fruit" column in this file is encoded using DELTA_LENGTH_BYTE_ARRAY. -@@ -1553,7 +1554,9 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession - } - } - -- test("Write Spark version into Parquet metadata") { -+// TODO : Comet native writer to add spark / comet version into parquet metadata -+ test("Write Spark version into Parquet metadata", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3427")) { - withTempPath { dir => - spark.range(1).repartition(1).write.parquet(dir.getAbsolutePath) - assert(getMetaData(dir)(SPARK_VERSION_METADATA_KEY) === SPARK_VERSION_SHORT) -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala -index baa11df302b..c88f66d5c77 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala -@@ -25,7 +25,7 @@ import org.apache.hadoop.fs.{Path, PathFilter} - import org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER - import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName - --import org.apache.spark.sql.Row -+import org.apache.spark.sql.{IgnoreComet, Row} - import org.apache.spark.sql.catalyst.util.DateTimeUtils - import org.apache.spark.sql.internal.SQLConf - import org.apache.spark.sql.test.SharedSparkSession -@@ -153,7 +153,9 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS - } - } - -- test("parquet timestamp conversion") { -+// TODO : Support legacy timestamps conversion /cast in comet native writer -+ test("parquet timestamp conversion", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3425")) { - // Make a table with one parquet file written by impala, and one parquet file written by spark. - // We should only adjust the timestamps in the impala file, and only if the conf is set - val impalaFile = "test-data/impala_timestamp.parq" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index bba71f1c48d..38c60ee2584 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -3475,62 +3285,6 @@ index c5c56f081d8..6cc51f93b4f 100644 assert(bucketedScan.length == expectedNumBucketedScan) } -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala -index baf99798965..e23d8223e05 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala -@@ -263,7 +263,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { - } - } - -- test("INSERT INTO TABLE - complex type but different names") { -+ test("INSERT INTO TABLE - complex type but different names", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3426")) { - val tab1 = "tab1" - val tab2 = "tab2" - withTable(tab1, tab2) { -@@ -1002,7 +1003,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { - } - } - -- test("SPARK-38336 INSERT INTO statements with tables with default columns: positive tests") { -+ test("SPARK-38336 INSERT INTO statements with tables with default columns: positive tests", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3421")) { - // When the INSERT INTO statement provides fewer values than expected, NULL values are appended - // in their place. - withTable("t") { -@@ -1517,7 +1519,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { - } - } - -- test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Positive tests") { -+ test("SPARK-38811 INSERT INTO on columns added with ALTER TABLE ADD COLUMNS: Positive tests", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3422")) { - // There is a complex expression in the default value. - val createTableBooleanCol = "create table t(i boolean) using parquet" - val createTableIntCol = "create table t(i int) using parquet" -@@ -2331,7 +2334,9 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { - } - } - -- test("SPARK-43071: INSERT INTO from queries whose final operators are not projections") { -+// TODO : fix overwrite mode issues comet native writer -+ test("SPARK-43071: INSERT INTO from queries whose final operators are not projections", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3423")) { - def runTest(insert: String, expected: Seq[Row]): Unit = { - withTable("t1", "t2") { - sql("create table t1(i boolean, s bigint default 42) using parquet") -@@ -2399,7 +2404,9 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { - } - } - -- test("SPARK-29174 Support LOCAL in INSERT OVERWRITE DIRECTORY to data source") { -+// TODO : Insert overwrite not supported comet native writer -+ test("SPARK-29174 Support LOCAL in INSERT OVERWRITE DIRECTORY to data source", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3420")) { - withTempPath { dir => - val path = dir.toURI.getPath - sql(s"""create table tab1 ( a int) using parquet location '$path'""") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 9742a004545..4e0417d730a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -3711,35 +3465,8 @@ index 86c4e49f6f6..2e639e5f38d 100644 val tblSourceName = "tbl_src" val tblTargetName = "tbl_target" val tblSourceQualified = s"default.$tblSourceName" -diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala -index 300807cf058..8dab4f9a2af 100644 ---- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala -+++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala -@@ -620,7 +620,10 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with - Option(dir).map(spark.read.format("org.apache.spark.sql.test").load) - } - -- test("write path implements onTaskCommit API correctly") { -+ -+// TODO : fix test for native writer -+ test("write path implements onTaskCommit API correctly", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3428")) { - withSQLConf( - SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> - classOf[MessageCapturingCommitProtocol].getCanonicalName) { -@@ -1184,7 +1187,9 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSparkSession with - } - } - -- test("Insert overwrite table command should output correct schema: basic") { -+// TODO : support vairous write modes in comet native writer -+ test("Insert overwrite table command should output correct schema: basic", -+ IgnoreComet("https://github.com/apache/datafusion-comet/issues/3424")) { - withTable("tbl", "tbl2") { - withView("view1") { - val df = spark.range(10).toDF("id") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala -index f0f3f94b811..fd9744a58cf 100644 +index f0f3f94b811..d64e4e54e22 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -27,13 +27,14 @@ import scala.jdk.CollectionConverters._ @@ -3796,7 +3523,7 @@ index f0f3f94b811..fd9744a58cf 100644 } } -@@ -248,8 +271,38 @@ private[sql] trait SQLTestUtilsBase +@@ -248,8 +271,33 @@ private[sql] trait SQLTestUtilsBase override protected def converter: ColumnNodeToExpressionConverter = self.spark.converter } @@ -3822,11 +3549,6 @@ index f0f3f94b811..fd9744a58cf 100644 + val v = System.getenv("ENABLE_COMET_SCAN_ONLY") + v != null && v.toBoolean + } -+ -+ protected def isCometWriterEnabled: Boolean = { -+ val v = System.getenv("ENABLE_COMET_WRITER") -+ v != null && v.toBoolean -+ } + protected override def withSQLConf[T](pairs: (String, String)*)(f: => T): T = { SparkSession.setActiveSession(spark) @@ -3835,7 +3557,7 @@ index f0f3f94b811..fd9744a58cf 100644 super.withSQLConf(pairs: _*)(f) } -@@ -451,6 +504,8 @@ private[sql] trait SQLTestUtilsBase +@@ -451,6 +499,8 @@ private[sql] trait SQLTestUtilsBase val schema = df.schema val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child @@ -3845,10 +3567,10 @@ index f0f3f94b811..fd9744a58cf 100644 spark.internalCreateDataFrame(withoutFilters.execute(), schema) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -index 245219c1756..4e7b86e55a2 100644 +index 245219c1756..7d2ef1b9145 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala -@@ -75,6 +75,37 @@ trait SharedSparkSessionBase +@@ -75,6 +75,31 @@ trait SharedSparkSessionBase // this rule may potentially block testing of other optimization rules such as // ConstantPropagation etc. .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) @@ -3876,12 +3598,6 @@ index 245219c1756..4e7b86e55a2 100644 + conf + .set("spark.sql.ansi.enabled", "true") + } -+ -+ if (isCometWriterEnabled) { -+ conf.set("spark.comet.parquet.write.enabled", "true") -+ conf.set("spark.comet.operator.DataWritingCommandExec.allowIncompatible", "true") -+ } -+ + } conf.set( StaticSQLConf.WAREHOUSE_PATH, From 570521fc111f2615289a8fa167debbdcbeecce6e Mon Sep 17 00:00:00 2001 From: B Vadlamani Date: Sat, 7 Feb 2026 11:27:50 -0800 Subject: [PATCH 15/15] enable_spark_tests_comet_native_writer_fix_spark_rebase_main --- .../parquet/CometParquetWriterSuite.scala | 158 ++++-------------- 1 file changed, 33 insertions(+), 125 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala index b98a05b936..ee20b5ecbc 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala @@ -20,13 +20,16 @@ package org.apache.comet.parquet import java.io.File + import scala.util.Random + import org.apache.spark.sql.{CometTestBase, DataFrame, Row} import org.apache.spark.sql.comet.{CometBatchScanExec, CometNativeScanExec, CometNativeWriteExec, CometScanExec} import org.apache.spark.sql.execution.{FileSourceScanExec, QueryExecution, SparkPlan} import org.apache.spark.sql.execution.command.DataWritingCommandExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType + import org.apache.comet.CometConf import org.apache.comet.CometSparkSessionExtensions.isSpark40Plus import org.apache.comet.testing.{DataGenOptions, FuzzDataGenerator, SchemaGenOptions} @@ -383,9 +386,7 @@ class CometParquetWriterSuite extends CometTestBase { CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { withTempPath { path => - assertNativeWriter { - spark.range(100).repartition(10).where("id = 50").write.parquet(path.toString) - } + spark.range(100).repartition(10).where("id = 50").write.parquet(path.toString) val partFiles = path .listFiles() .filter(f => f.isFile && !f.getName.startsWith(".") && !f.getName.startsWith("_")) @@ -402,9 +403,7 @@ class CometParquetWriterSuite extends CometTestBase { CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { withTable("t1", "t2") { sql("CREATE TABLE t1(i CHAR(5), c VARCHAR(4)) USING parquet") - assertNativeWriter { - sql("CREATE TABLE t2 USING parquet AS SELECT * FROM t1") - } + sql("CREATE TABLE t2 USING parquet AS SELECT * FROM t1") checkAnswer( sql("desc t2").selectExpr("data_type").where("data_type like '%char%'"), Seq(Row("char(5)"), Row("varchar(4)"))) @@ -421,9 +420,7 @@ class CometParquetWriterSuite extends CometTestBase { withTable("t1", "t2") { sql("CREATE TABLE t1(col CHAR(5)) USING parquet") withSQLConf(SQLConf.CHAR_AS_VARCHAR.key -> "true") { - assertNativeWriter { - sql("CREATE TABLE t2 USING parquet AS SELECT * FROM t1") - } + sql("CREATE TABLE t2 USING parquet AS SELECT * FROM t1") checkAnswer( sql("desc t2").selectExpr("data_type").where("data_type like '%char%'"), Seq(Row("varchar(5)"))) @@ -442,17 +439,11 @@ class CometParquetWriterSuite extends CometTestBase { val path = dir.toURI.getPath withTable("tab1", "tab2") { sql(s"""create table tab1 (a int) using parquet location '$path'""") - assertNativeWriter { - sql("insert into tab1 values(1)") - } + sql("insert into tab1 values(1)") checkAnswer(sql("select * from tab1"), Seq(Row(1))) sql("create table tab2 (a int) using parquet") - assertNativeWriter { - sql("insert into tab2 values(2)") - } - assertNativeWriter { - sql(s"""insert overwrite local directory '$path' using parquet select * from tab2""") - } + sql("insert into tab2 values(2)") + sql(s"""insert overwrite local directory '$path' using parquet select * from tab2""") sql("refresh table tab1") checkAnswer(sql("select * from tab1"), Seq(Row(2))) } @@ -468,9 +459,7 @@ class CometParquetWriterSuite extends CometTestBase { CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { withTable("t") { sql("create table t(i boolean, s bigint) using parquet") - assertNativeWriter { - sql("insert into t(i) values(true)") - } + sql("insert into t(i) values(true)") checkAnswer(spark.table("t"), Row(true, null)) } } @@ -485,9 +474,7 @@ class CometParquetWriterSuite extends CometTestBase { withTable("t") { sql("create table t(i boolean) using parquet") sql("alter table t add column s string default concat('abc', 'def')") - assertNativeWriter { - sql("insert into t values(true, default)") - } + sql("insert into t values(true, default)") checkAnswer(spark.table("t"), Row(true, "abcdef")) } } @@ -501,13 +488,9 @@ class CometParquetWriterSuite extends CometTestBase { CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { withTable("t1", "t2") { sql("create table t1(i boolean, s bigint default 42) using parquet") - assertNativeWriter { - sql("insert into t1 values (true, 41), (false, default)") - } + sql("insert into t1 values (true, 41), (false, default)") sql("create table t2(i boolean, s bigint) using parquet") - assertNativeWriter { - sql("insert into t2 select * from t1 order by s") - } + sql("insert into t2 select * from t1 order by s") checkAnswer(spark.table("t2"), Seq(Row(true, 41), Row(false, 42))) } } @@ -522,14 +505,10 @@ class CometParquetWriterSuite extends CometTestBase { withTable("tbl", "tbl2") { withView("view1") { val df = spark.range(10).toDF("id") - assertNativeWriter { - df.write.format("parquet").saveAsTable("tbl") - } + df.write.format("parquet").saveAsTable("tbl") spark.sql("CREATE VIEW view1 AS SELECT id FROM tbl") spark.sql("CREATE TABLE tbl2(ID long) USING parquet") - assertNativeWriter { - spark.sql("INSERT OVERWRITE TABLE tbl2 SELECT ID FROM view1") - } + spark.sql("INSERT OVERWRITE TABLE tbl2 SELECT ID FROM view1") checkAnswer(spark.table("tbl2"), (0 until 10).map(Row(_))) } } @@ -543,13 +522,11 @@ class CometParquetWriterSuite extends CometTestBase { CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { withTempPath { dir => - assertNativeWriter { - spark - .range(1) - .selectExpr("current_timestamp() as ts") - .write - .parquet(dir.toString + "/spark") - } + spark + .range(1) + .selectExpr("current_timestamp() as ts") + .write + .parquet(dir.toString + "/spark") val result = spark.read.parquet(dir.toString + "/spark").collect() assert(result.length == 1) } @@ -565,12 +542,8 @@ class CometParquetWriterSuite extends CometTestBase { withTable("tab1", "tab2") { sql("""CREATE TABLE tab1 (s struct) USING parquet""") sql("""CREATE TABLE tab2 (s struct) USING parquet""") - assertNativeWriter { - sql("INSERT INTO tab1 VALUES (named_struct('a', 'x', 'b', 'y'))") - } - assertNativeWriter { - sql("INSERT INTO tab2 SELECT * FROM tab1") - } + sql("INSERT INTO tab1 VALUES (named_struct('a', 'x', 'b', 'y'))") + sql("INSERT INTO tab2 SELECT * FROM tab1") checkAnswer(spark.table("tab2"), Row(Row("x", "y"))) } } @@ -583,9 +556,7 @@ class CometParquetWriterSuite extends CometTestBase { CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { withTempPath { dir => - assertNativeWriter { - spark.range(1).repartition(1).write.parquet(dir.getAbsolutePath) - } + spark.range(1).repartition(1).write.parquet(dir.getAbsolutePath) val files = dir.listFiles().filter(_.getName.endsWith(".parquet")) assert(files.nonEmpty, "Expected parquet files to be written") } @@ -600,9 +571,7 @@ class CometParquetWriterSuite extends CometTestBase { CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { withTempDir { dir => val path = dir.getCanonicalPath - assertNativeWriter { - spark.range(10).repartition(10).write.mode("overwrite").parquet(path) - } + spark.range(10).repartition(10).write.mode("overwrite").parquet(path) val files = new File(path).listFiles().filter(_.getName.startsWith("part-")) assert(files.length > 0, "Expected part files to be written") } @@ -618,9 +587,7 @@ class CometParquetWriterSuite extends CometTestBase { CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true") { withTable("t") { - assertNativeWriter { - sql("CREATE TABLE t USING parquet AS SELECT 1 AS c UNION ALL SELECT 2") - } + sql("CREATE TABLE t USING parquet AS SELECT 1 AS c UNION ALL SELECT 2") checkAnswer(spark.table("t"), Seq(Row(1), Row(2))) } } @@ -636,9 +603,7 @@ class CometParquetWriterSuite extends CometTestBase { withTable("t1", "t2") { sql("CREATE TABLE t1(a INT) USING parquet") sql("CREATE TABLE t2(a INT) USING parquet") - assertNativeWriter { - sql("FROM (SELECT 1 AS a) src INSERT INTO t1 SELECT a INSERT INTO t2 SELECT a") - } + sql("FROM (SELECT 1 AS a) src INSERT INTO t1 SELECT a INSERT INTO t2 SELECT a") checkAnswer(spark.table("t1"), Row(1)) checkAnswer(spark.table("t2"), Row(1)) } @@ -664,72 +629,15 @@ class CometParquetWriterSuite extends CometTestBase { } /** - * Executes a code block and asserts that CometNativeWriteExec is in the write plan. - * This is used for verifying native writer is called in SQL commands. + * Captures the execution plan during a write operation. * - * @param block - * The code block to execute (should contain write operations) + * @param writeOp + * The write operation to execute (takes output path as parameter) + * @param outputPath + * The path to write to + * @return + * The captured execution plan */ - private def assertNativeWriter(block: => Unit): Unit = { - var capturedPlan: Option[QueryExecution] = None - - val listener = new org.apache.spark.sql.util.QueryExecutionListener { - override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { - if (funcName == "save" || funcName.contains("command")) { - capturedPlan = Some(qe) - } - } - - override def onFailure( - funcName: String, - qe: QueryExecution, - exception: Exception): Unit = {} - } - - spark.listenerManager.register(listener) - - try { - block - - // Wait for listener to be called with timeout - val maxWaitTimeMs = 15000 - val checkIntervalMs = 100 - val maxIterations = maxWaitTimeMs / checkIntervalMs - var iterations = 0 - - while (capturedPlan.isEmpty && iterations < maxIterations) { - Thread.sleep(checkIntervalMs) - iterations += 1 - } - - assert( - capturedPlan.isDefined, - s"Listener was not called within ${maxWaitTimeMs}ms - no execution plan captured") - - val plan = stripAQEPlan(capturedPlan.get.executedPlan) - - // Count CometNativeWriteExec instances in the plan - var nativeWriteCount = 0 - plan.foreach { - case _: CometNativeWriteExec => - nativeWriteCount += 1 - case d: DataWritingCommandExec => - d.child.foreach { - case _: CometNativeWriteExec => - nativeWriteCount += 1 - case _ => - } - case _ => - } - - assert( - nativeWriteCount == 1, - s"Expected exactly one CometNativeWriteExec in the plan, but found $nativeWriteCount:\n${plan.treeString}") - } finally { - spark.listenerManager.unregister(listener) - } - } - private def captureWritePlan(writeOp: String => Unit, outputPath: String): SparkPlan = { var capturedPlan: Option[QueryExecution] = None