From 6c01551e4bfc2d59d34de39d395150d93cdd0949 Mon Sep 17 00:00:00 2001 From: Adam Binford Date: Fri, 2 Jan 2026 22:05:22 +0000 Subject: [PATCH 1/9] work on getting delta scans working --- native/core/Cargo.toml | 2 +- spark/pom.xml | 17 + .../apache/comet/delta/DeltaReflection.scala | 150 ++ .../apache/comet/rules/CometScanRule.scala | 4 + .../spark/sql/comet/CometScanExec.scala | 36 +- .../apache/comet/CometDeltaNativeSuite.scala | 2298 +++++++++++++++++ 6 files changed, 2505 insertions(+), 2 deletions(-) create mode 100644 spark/src/main/scala/org/apache/comet/delta/DeltaReflection.scala create mode 100644 spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 7b32be36a2..b181dcf971 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -76,7 +76,7 @@ parking_lot = "0.12.5" datafusion-comet-objectstore-hdfs = { path = "../hdfs", optional = true, default-features = false, features = ["hdfs"] } reqwest = { version = "0.12", default-features = false, features = ["rustls-tls-native-roots", "http2"] } object_store_opendal = {version = "0.55.0", optional = true} -hdfs-sys = {version = "0.3", optional = true, features = ["hdfs_3_3"]} +hdfs-sys = {version = "0.3", optional = true, features = ["hdfs_3_3", "vendored"]} opendal = { version ="0.55.0", optional = true, features = ["services-hdfs"] } iceberg = { workspace = true } serde_json = "1.0" diff --git a/spark/pom.xml b/spark/pom.xml index 3b832e37a2..23a5675423 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -112,6 +112,11 @@ under the License. + + com.google.guava + failureaccess + 1.0.3 + org.codehaus.jackson jackson-mapper-asl @@ -216,6 +221,12 @@ under the License. 1.8.1 test + + io.delta + delta-spark_${scala.binary.version} + 3.3.2 + test + org.eclipse.jetty @@ -241,6 +252,12 @@ under the License. 1.10.0 test + + io.delta + delta-spark_${scala.binary.version} + 4.0.0 + test + org.eclipse.jetty diff --git a/spark/src/main/scala/org/apache/comet/delta/DeltaReflection.scala b/spark/src/main/scala/org/apache/comet/delta/DeltaReflection.scala new file mode 100644 index 0000000000..be792c618b --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/delta/DeltaReflection.scala @@ -0,0 +1,150 @@ +/* + * 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.comet.delta + +import org.apache.spark.internal.Logging + +/** + * Shared reflection utilities for Delta operations. + * + * This object provides common reflection methods used across Comet for interacting with Delta + * classes without requiring a runtime dependency on Delta + */ +object DeltaReflection extends Logging { + + /** + * Iceberg class names used throughout Comet. + */ + object ClassNames { + } + + /** + * Iceberg content types. + */ + object ContentTypes { + val POSITION_DELETES = "POSITION_DELETES" + val EQUALITY_DELETES = "EQUALITY_DELETES" + } + + /** + * Iceberg file formats. + */ + object FileFormats { + val PARQUET = "PARQUET" + } + + /** + * Gets the Delta Protocol from a Spark FileFormat + */ + def getProtocol(fileFormat: Any): Option[Any] = { + try { + val field = fileFormat.getClass.getDeclaredField("protocol") + field.setAccessible(true) + Some(field.get(fileFormat)) + } catch { + case e: Exception => + logError( + s"Delta reflection failure: Failed to get protocol from FileFormat: ${e.getMessage}") + None + } + } + + /** + * Gets the tasks from a SparkScan. + * + * The tasks() method is protected in SparkScan, requiring reflection to access. + */ + def getMinReaderVersion(protocol: Any): Option[Int] = { + try { + val method = protocol.getClass.getDeclaredMethod("getMinReaderVersion") + Some(method.invoke(protocol).asInstanceOf[Int]) + } catch { + case e: Exception => + logError( + s"Delta reflection failure: Failed to get minReaderVersion from protocol: ${e.getMessage}") + None + } + } + + def getReaderFeatures(protocol: Any): Option[java.util.Set[String]] = { + try { + val method = protocol.getClass.getDeclaredMethod("getReaderFeatures") + Some(method.invoke(protocol).asInstanceOf[java.util.Set[String]]) + } catch { + case e: Exception => + logError( + s"Delta reflection failure: Failed to get minReaderVersion from protocol: ${e.getMessage}") + None + } + } +} + +/** + * Pre-extracted Iceberg metadata for native scan execution. + * + * This class holds all metadata extracted from Iceberg during the planning/validation phase in + * CometScanRule. By extracting all metadata once during validation (where reflection failures + * trigger fallback to Spark), we avoid redundant reflection during serialization (where failures + * would be fatal runtime errors). + * + * @param minReaderVersion + * The minimum reader version of the table + * @param minWriterVersion + * The minimum writer version of the table + * @param readerFeatures + * A list of enabled reader features on the table + * @param writerFeatures + * A list of enabled writer features on the table + */ +case class CometDeltaNativeScanMetadata( + minReaderVersion: Int, + minWriterVersion: Int, + readerFeatures: java.util.Set[String], + writerFeatures: java.util.Set[String]) + +object CometDeltaNativeScanMetadata extends Logging { + import DeltaReflection._ + + /** + * Extracts all Iceberg metadata needed for native scan execution. + * + * This method performs all reflection operations once during planning/validation. If any + * reflection operation fails, returns None to trigger fallback to Spark. + * + * @param scan + * The Spark BatchScanExec.scan (SparkBatchQueryScan) + * @param metadataLocation + * Path to the table metadata file (already extracted) + * @param catalogProperties + * Catalog properties for FileIO (already extracted) + * @return + * Some(metadata) if all reflection succeeds, None to trigger fallback + */ + def extract(fileFormat: Any): Option[CometDeltaNativeScanMetadata] = { + getProtocol(fileFormat).flatMap { protocol => + for { + minReaderVersion <- getMinReaderVersion(protocol) + readerFeatures <- getReaderFeatures(protocol) + } yield { + CometDeltaNativeScanMetadata(minReaderVersion, 0, readerFeatures, java.util.Set.of()) + } + } + } +} diff --git a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala index 01e385b0ae..23e9600137 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -142,6 +142,7 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com scanExec.relation match { case r: HadoopFsRelation => + println(s"Found HadoopFsRelation: $r") if (!CometScanExec.isFileFormatSupported(r.fileFormat)) { return withInfo(scanExec, s"Unsupported file format ${r.fileFormat}") } @@ -156,6 +157,8 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com scanImpl = selectScan(scanExec, r.partitionSchema, hadoopConf) } + println(s"Using scan impl: $scanImpl") + if (scanImpl == SCAN_NATIVE_DATAFUSION && !CometNativeScan.isSupported(scanExec)) { return scanExec } @@ -188,6 +191,7 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com } else { // this is confusing, but we always insert a CometScanExec here, which may replaced // with a CometNativeExec when CometExecRule runs, depending on the scanImpl value. + println("Creatin scan") CometScanExec(scanExec, session, scanImpl) } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala index e283f6b2cf..1f951465d6 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala @@ -21,6 +21,7 @@ package org.apache.spark.sql.comet import scala.collection.mutable.HashMap import scala.concurrent.duration.NANOSECONDS +import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag import org.apache.hadoop.fs.Path @@ -45,6 +46,7 @@ import org.apache.spark.util.SerializableConfiguration import org.apache.spark.util.collection._ import org.apache.comet.{CometConf, MetricsSupport} +import org.apache.comet.delta.CometDeltaNativeScanMetadata import org.apache.comet.parquet.{CometParquetFileFormat, CometParquetPartitionReaderFactory} /** @@ -583,7 +585,39 @@ object CometScanExec { def isFileFormatSupported(fileFormat: FileFormat): Boolean = { // Only support Spark's built-in Parquet scans, not others such as Delta which use a subclass // of ParquetFileFormat. - fileFormat.getClass().equals(classOf[ParquetFileFormat]) + fileFormat.getClass().equals(classOf[ParquetFileFormat]) || isSupportedDeltaScan(fileFormat) + } + + val unsupportedDeltaReaderFeatures: Set[String] = Set( + "columnMapping", + "deletionVectors" + ) + + def isSupportedDeltaScan(fileFormat: FileFormat): Boolean = { + if (fileFormat.getClass().getName() != "org.apache.spark.sql.delta.DeltaParquetFileFormat") { + return false + } + + println("Getting metadata from delta") + + // Delta scans without certain features enabled are simply normal Parquet scans that can + // take advantage of the native scan, so check to see if it is compatible + val deltaMetadata = CometDeltaNativeScanMetadata.extract(fileFormat) match { + case Some(m) => m + case None => return false + } + + println(s"Checking metadata from $deltaMetadata") + + // Version 1 has no special features + // Version 2 introduced column mapping, which is not supported + // Version 3 changes to use the readerFeatures list instead, so we check for incompatible + // features + deltaMetadata.minReaderVersion match { + case 1 => true + case 2 => false + case 3 => deltaMetadata.readerFeatures.asScala.intersect(unsupportedDeltaReaderFeatures).isEmpty + } } } diff --git a/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala new file mode 100644 index 0000000000..33d23d0590 --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala @@ -0,0 +1,2298 @@ +/* + * 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.comet + +import java.io.File +import java.nio.file.Files + +import org.apache.spark.sql.CometTestBase +import org.apache.spark.sql.comet.CometNativeScanExec +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.SparkConf + +/** + * Test suite for native Iceberg scan using FileScanTasks and iceberg-rust. + * + * Note: Requires Iceberg dependencies to be added to pom.xml + */ +class CometDeltaNativeSuite extends CometTestBase { + + override protected def sparkConf: SparkConf = { + super.sparkConf + .set("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") + .set("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog") + .set(CometConf.COMET_NATIVE_SCAN_IMPL.key, "native_datafusion") + } + + // Skip these tests if Iceberg is not available in classpath + private def deltaAvailable: Boolean = { + try { + Class.forName("org.apache.spark.sql.delta.DeltaLog") + true + } catch { + case _: ClassNotFoundException => false + } + } + + /** Collects all ComeNativeScanExec nodes from a plan */ + private def collectNativeScans(plan: SparkPlan): Seq[CometNativeScanExec] = { + collect(plan) { case scan: CometNativeScanExec => + scan + } + } + + /** + * Helper to verify query correctness and that exactly one CometIcebergNativeScanExec is used. + * This ensures both correct results and that the native Iceberg scan operator is being used. + */ + private def checkNativeScan(query: String): Unit = { + val (_, cometPlan) = checkSparkAnswer(query) + val nativeScans = collectNativeScans(cometPlan) + assert( + nativeScans.length == 1, + s"Expected exactly 1 CometNativeScanExec but found ${nativeScans.length}. Plan:\n$cometPlan") + } + + test("create and query simple Delta table") { + assume(deltaAvailable, "Delta not available in classpath") + + withTempDeltaDir { warehouseDir => + spark.sql(s""" + CREATE TABLE test_table ( + id INT, + name STRING, + value DOUBLE + ) USING delta LOCATION '${warehouseDir.toString()}' + """) + + spark.sql(""" + INSERT INTO test_table + VALUES (1, 'Alice', 10.5), (2, 'Bob', 20.3), (3, 'Charlie', 30.7) + """) + + checkNativeScan("SELECT * FROM test_table ORDER BY id") + + spark.sql("DROP TABLE test_table") + } + } + + // test("filter pushdown - equality predicates") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.filter_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.filter_cat.type" -> "hadoop", + // "spark.sql.catalog.filter_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE filter_cat.db.filter_test ( + // id INT, + // name STRING, + // value DOUBLE, + // active BOOLEAN + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO filter_cat.db.filter_test VALUES + // (1, 'Alice', 10.5, true), + // (2, 'Bob', 20.3, false), + // (3, 'Charlie', 30.7, true), + // (4, 'Diana', 15.2, false), + // (5, 'Eve', 25.8, true) + // """) + + // checkIcebergNativeScan("SELECT * FROM filter_cat.db.filter_test WHERE id = 3") + + // checkIcebergNativeScan("SELECT * FROM filter_cat.db.filter_test WHERE name = 'Bob'") + + // checkIcebergNativeScan("SELECT * FROM filter_cat.db.filter_test WHERE active = true") + + // spark.sql("DROP TABLE filter_cat.db.filter_test") + // } + // } + // } + + // test("filter pushdown - comparison operators") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.filter_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.filter_cat.type" -> "hadoop", + // "spark.sql.catalog.filter_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE filter_cat.db.comparison_test ( + // id INT, + // value DOUBLE + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO filter_cat.db.comparison_test VALUES + // (1, 10.5), (2, 20.3), (3, 30.7), (4, 15.2), (5, 25.8) + // """) + + // checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE value > 20.0") + + // checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE value >= 20.3") + + // checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE value < 20.0") + + // checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE value <= 20.3") + + // checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE id != 3") + + // spark.sql("DROP TABLE filter_cat.db.comparison_test") + // } + // } + // } + + // test("filter pushdown - AND/OR combinations") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.filter_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.filter_cat.type" -> "hadoop", + // "spark.sql.catalog.filter_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE filter_cat.db.logical_test ( + // id INT, + // category STRING, + // value DOUBLE + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO filter_cat.db.logical_test VALUES + // (1, 'A', 10.5), (2, 'B', 20.3), (3, 'A', 30.7), + // (4, 'B', 15.2), (5, 'A', 25.8), (6, 'C', 35.0) + // """) + + // checkIcebergNativeScan( + // "SELECT * FROM filter_cat.db.logical_test WHERE category = 'A' AND value > 20.0") + + // checkIcebergNativeScan( + // "SELECT * FROM filter_cat.db.logical_test WHERE category = 'B' OR value > 30.0") + + // checkIcebergNativeScan("""SELECT * FROM filter_cat.db.logical_test + // WHERE (category = 'A' AND value > 20.0) OR category = 'C'""") + + // spark.sql("DROP TABLE filter_cat.db.logical_test") + // } + // } + // } + + // test("filter pushdown - NULL checks") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.filter_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.filter_cat.type" -> "hadoop", + // "spark.sql.catalog.filter_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE filter_cat.db.null_test ( + // id INT, + // optional_value DOUBLE + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO filter_cat.db.null_test VALUES + // (1, 10.5), (2, NULL), (3, 30.7), (4, NULL), (5, 25.8) + // """) + + // checkIcebergNativeScan( + // "SELECT * FROM filter_cat.db.null_test WHERE optional_value IS NULL") + + // checkIcebergNativeScan( + // "SELECT * FROM filter_cat.db.null_test WHERE optional_value IS NOT NULL") + + // spark.sql("DROP TABLE filter_cat.db.null_test") + // } + // } + // } + + // test("filter pushdown - IN list") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.filter_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.filter_cat.type" -> "hadoop", + // "spark.sql.catalog.filter_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE filter_cat.db.in_test ( + // id INT, + // name STRING + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO filter_cat.db.in_test VALUES + // (1, 'Alice'), (2, 'Bob'), (3, 'Charlie'), + // (4, 'Diana'), (5, 'Eve'), (6, 'Frank') + // """) + + // checkIcebergNativeScan("SELECT * FROM filter_cat.db.in_test WHERE id IN (2, 4, 6)") + + // checkIcebergNativeScan( + // "SELECT * FROM filter_cat.db.in_test WHERE name IN ('Alice', 'Charlie', 'Eve')") + + // checkIcebergNativeScan("SELECT * FROM filter_cat.db.in_test WHERE id IS NOT NULL") + + // checkIcebergNativeScan("SELECT * FROM filter_cat.db.in_test WHERE id NOT IN (1, 3, 5)") + + // spark.sql("DROP TABLE filter_cat.db.in_test") + // } + // } + // } + + // test("verify filters are pushed to native scan") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.filter_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.filter_cat.type" -> "hadoop", + // "spark.sql.catalog.filter_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE filter_cat.db.filter_debug ( + // id INT, + // value DOUBLE + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO filter_cat.db.filter_debug VALUES + // (1, 10.5), (2, 20.3), (3, 30.7), (4, 15.2), (5, 25.8) + // """) + + // checkIcebergNativeScan("SELECT * FROM filter_cat.db.filter_debug WHERE id > 2") + + // spark.sql("DROP TABLE filter_cat.db.filter_debug") + // } + // } + // } + + // test("small table - verify no duplicate rows (1 file)") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.small_table ( + // id INT, + // name STRING + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.small_table + // VALUES (1, 'Alice'), (2, 'Bob'), (3, 'Charlie') + // """) + + // checkIcebergNativeScan("SELECT * FROM test_cat.db.small_table ORDER BY id") + // checkIcebergNativeScan("SELECT COUNT(DISTINCT id) FROM test_cat.db.small_table") + + // spark.sql("DROP TABLE test_cat.db.small_table") + // } + // } + // } + + // test("medium table - verify correct partition count (multiple files)") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", + // "spark.sql.files.maxRecordsPerFile" -> "10") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.medium_table ( + // id INT, + // value DOUBLE + // ) USING iceberg + // """) + + // // Insert 100 rows - should create multiple files with maxRecordsPerFile=10 + // spark.sql(""" + // INSERT INTO test_cat.db.medium_table + // SELECT id, CAST(id * 1.5 AS DOUBLE) as value + // FROM range(100) + // """) + + // // Verify results match Spark native (catches duplicates across partitions) + // checkIcebergNativeScan("SELECT * FROM test_cat.db.medium_table ORDER BY id") + // checkIcebergNativeScan("SELECT COUNT(DISTINCT id) FROM test_cat.db.medium_table") + // checkIcebergNativeScan("SELECT SUM(value) FROM test_cat.db.medium_table") + + // spark.sql("DROP TABLE test_cat.db.medium_table") + // } + // } + // } + + // test("large table - verify no duplicates with many files") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", + // "spark.sql.files.maxRecordsPerFile" -> "100") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.large_table ( + // id BIGINT, + // category STRING, + // value DOUBLE + // ) USING iceberg + // """) + + // // Insert 10,000 rows - with maxRecordsPerFile=100, creates ~100 files + // spark.sql(""" + // INSERT INTO test_cat.db.large_table + // SELECT + // id, + // CASE WHEN id % 3 = 0 THEN 'A' WHEN id % 3 = 1 THEN 'B' ELSE 'C' END as category, + // CAST(id * 2.5 AS DOUBLE) as value + // FROM range(10000) + // """) + + // checkIcebergNativeScan("SELECT COUNT(DISTINCT id) FROM test_cat.db.large_table") + // checkIcebergNativeScan("SELECT SUM(value) FROM test_cat.db.large_table") + // checkIcebergNativeScan( + // "SELECT category, COUNT(*) FROM test_cat.db.large_table GROUP BY category ORDER BY category") + + // spark.sql("DROP TABLE test_cat.db.large_table") + // } + // } + // } + + // test("partitioned table - verify key-grouped partitioning") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.partitioned_table ( + // id INT, + // category STRING, + // value DOUBLE + // ) USING iceberg + // PARTITIONED BY (category) + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.partitioned_table VALUES + // (1, 'A', 10.5), (2, 'B', 20.3), (3, 'C', 30.7), + // (4, 'A', 15.2), (5, 'B', 25.8), (6, 'C', 35.0), + // (7, 'A', 12.1), (8, 'B', 22.5), (9, 'C', 32.9) + // """) + + // checkIcebergNativeScan("SELECT * FROM test_cat.db.partitioned_table ORDER BY id") + // checkIcebergNativeScan( + // "SELECT * FROM test_cat.db.partitioned_table WHERE category = 'A' ORDER BY id") + // checkIcebergNativeScan( + // "SELECT * FROM test_cat.db.partitioned_table WHERE category = 'B' ORDER BY id") + // checkIcebergNativeScan( + // "SELECT category, COUNT(*) FROM test_cat.db.partitioned_table GROUP BY category ORDER BY category") + + // spark.sql("DROP TABLE test_cat.db.partitioned_table") + // } + // } + // } + + // test("empty table - verify graceful handling") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.empty_table ( + // id INT, + // name STRING + // ) USING iceberg + // """) + + // checkIcebergNativeScan("SELECT * FROM test_cat.db.empty_table") + // checkIcebergNativeScan("SELECT * FROM test_cat.db.empty_table WHERE id > 0") + + // spark.sql("DROP TABLE test_cat.db.empty_table") + // } + // } + // } + + // // MOR (Merge-On-Read) delete file tests. + // // Delete files are extracted from FileScanTasks and handled by iceberg-rust's ArrowReader, + // // which automatically applies both positional and equality deletes during scan execution. + // test("MOR table with POSITIONAL deletes - verify deletes are applied") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.positional_delete_test ( + // id INT, + // name STRING, + // value DOUBLE + // ) USING iceberg + // TBLPROPERTIES ( + // 'write.delete.mode' = 'merge-on-read', + // 'write.merge.mode' = 'merge-on-read' + // ) + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.positional_delete_test + // VALUES + // (1, 'Alice', 10.5), (2, 'Bob', 20.3), (3, 'Charlie', 30.7), + // (4, 'Diana', 15.2), (5, 'Eve', 25.8), (6, 'Frank', 35.0), + // (7, 'Grace', 12.1), (8, 'Hank', 22.5) + // """) + + // spark.sql("DELETE FROM test_cat.db.positional_delete_test WHERE id IN (2, 4, 6)") + + // checkIcebergNativeScan("SELECT * FROM test_cat.db.positional_delete_test ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.positional_delete_test") + // } + // } + // } + + // test("MOR table with EQUALITY deletes - verify deletes are applied") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // // Create table with equality delete columns specified + // // This forces Spark to use equality deletes instead of positional deletes + // spark.sql(""" + // CREATE TABLE test_cat.db.equality_delete_test ( + // id INT, + // category STRING, + // value DOUBLE + // ) USING iceberg + // TBLPROPERTIES ( + // 'write.delete.mode' = 'merge-on-read', + // 'write.merge.mode' = 'merge-on-read', + // 'write.delete.equality-delete-columns' = 'id' + // ) + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.equality_delete_test + // VALUES + // (1, 'A', 10.5), (2, 'B', 20.3), (3, 'A', 30.7), + // (4, 'B', 15.2), (5, 'A', 25.8), (6, 'C', 35.0) + // """) + + // spark.sql("DELETE FROM test_cat.db.equality_delete_test WHERE id IN (2, 4)") + + // checkIcebergNativeScan("SELECT * FROM test_cat.db.equality_delete_test ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.equality_delete_test") + // } + // } + // } + + // test("MOR table with multiple delete operations - mixed delete types") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.multi_delete_test ( + // id INT, + // data STRING + // ) USING iceberg + // TBLPROPERTIES ( + // 'write.delete.mode' = 'merge-on-read', + // 'write.merge.mode' = 'merge-on-read' + // ) + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.multi_delete_test + // SELECT id, CONCAT('data_', CAST(id AS STRING)) as data + // FROM range(100) + // """) + + // spark.sql("DELETE FROM test_cat.db.multi_delete_test WHERE id < 10") + // spark.sql("DELETE FROM test_cat.db.multi_delete_test WHERE id > 90") + // spark.sql("DELETE FROM test_cat.db.multi_delete_test WHERE id % 10 = 5") + + // checkIcebergNativeScan("SELECT * FROM test_cat.db.multi_delete_test ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.multi_delete_test") + // } + // } + // } + + // test("verify no duplicate rows across multiple partitions") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", + // // Create multiple files to ensure multiple partitions + // "spark.sql.files.maxRecordsPerFile" -> "50") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.multipart_test ( + // id INT, + // data STRING + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.multipart_test + // SELECT id, CONCAT('data_', CAST(id AS STRING)) as data + // FROM range(500) + // """) + + // // Critical: COUNT(*) vs COUNT(DISTINCT id) catches duplicates across partitions + // checkIcebergNativeScan("SELECT COUNT(DISTINCT id) FROM test_cat.db.multipart_test") + // checkIcebergNativeScan( + // "SELECT * FROM test_cat.db.multipart_test WHERE id < 10 ORDER BY id") + // checkIcebergNativeScan( + // "SELECT * FROM test_cat.db.multipart_test WHERE id >= 490 ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.multipart_test") + // } + // } + // } + + // test("filter pushdown with multi-partition table") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", + // "spark.sql.files.maxRecordsPerFile" -> "20") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.filter_multipart ( + // id INT, + // category STRING, + // value DOUBLE + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.filter_multipart + // SELECT + // id, + // CASE WHEN id % 2 = 0 THEN 'even' ELSE 'odd' END as category, + // CAST(id * 1.5 AS DOUBLE) as value + // FROM range(200) + // """) + + // checkIcebergNativeScan( + // "SELECT * FROM test_cat.db.filter_multipart WHERE id > 150 ORDER BY id") + // checkIcebergNativeScan( + // "SELECT * FROM test_cat.db.filter_multipart WHERE category = 'even' AND id < 50 ORDER BY id") + // checkIcebergNativeScan( + // "SELECT COUNT(DISTINCT id) FROM test_cat.db.filter_multipart WHERE id BETWEEN 50 AND 100") + // checkIcebergNativeScan( + // "SELECT SUM(value) FROM test_cat.db.filter_multipart WHERE category = 'odd'") + + // spark.sql("DROP TABLE test_cat.db.filter_multipart") + // } + // } + // } + + // test("date partitioned table with date range queries") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.date_partitioned ( + // id INT, + // event_date DATE, + // value STRING + // ) USING iceberg + // PARTITIONED BY (days(event_date)) + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.date_partitioned VALUES + // (1, DATE '2024-01-01', 'a'), (2, DATE '2024-01-02', 'b'), + // (3, DATE '2024-01-03', 'c'), (4, DATE '2024-01-15', 'd'), + // (5, DATE '2024-01-16', 'e'), (6, DATE '2024-02-01', 'f') + // """) + + // checkIcebergNativeScan("SELECT * FROM test_cat.db.date_partitioned ORDER BY id") + // checkIcebergNativeScan( + // "SELECT * FROM test_cat.db.date_partitioned WHERE event_date = DATE '2024-01-01'") + // checkIcebergNativeScan( + // "SELECT * FROM test_cat.db.date_partitioned WHERE event_date BETWEEN DATE '2024-01-01' AND DATE '2024-01-03' ORDER BY id") + // checkIcebergNativeScan( + // "SELECT event_date, COUNT(*) FROM test_cat.db.date_partitioned GROUP BY event_date ORDER BY event_date") + + // spark.sql("DROP TABLE test_cat.db.date_partitioned") + // } + // } + // } + + // test("bucket partitioned table") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.bucket_partitioned ( + // id INT, + // value DOUBLE + // ) USING iceberg + // PARTITIONED BY (bucket(4, id)) + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.bucket_partitioned + // SELECT id, CAST(id * 1.5 AS DOUBLE) as value + // FROM range(100) + // """) + + // checkIcebergNativeScan("SELECT * FROM test_cat.db.bucket_partitioned ORDER BY id") + // checkIcebergNativeScan("SELECT COUNT(DISTINCT id) FROM test_cat.db.bucket_partitioned") + // checkIcebergNativeScan("SELECT SUM(value) FROM test_cat.db.bucket_partitioned") + // checkIcebergNativeScan( + // "SELECT * FROM test_cat.db.bucket_partitioned WHERE id < 20 ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.bucket_partitioned") + // } + // } + // } + + // test("partition pruning - bucket transform verifies files are skipped") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.bucket_pruning ( + // id INT, + // data STRING + // ) USING iceberg + // PARTITIONED BY (bucket(8, id)) + // """) + + // (0 until 8).foreach { bucket => + // spark.sql(s""" + // INSERT INTO test_cat.db.bucket_pruning + // SELECT id, CONCAT('data_', CAST(id AS STRING)) as data + // FROM range(${bucket * 100}, ${(bucket + 1) * 100}) + // """) + // } + + // val specificIds = Seq(5, 15, 25) + // val df = spark.sql(s""" + // SELECT * FROM test_cat.db.bucket_pruning + // WHERE id IN (${specificIds.mkString(",")}) + // """) + + // val scanNodes = df.queryExecution.executedPlan + // .collectLeaves() + // .collect { case s: CometIcebergNativeScanExec => s } + + // assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") + + // val metrics = scanNodes.head.metrics + + // val result = df.collect() + // assert(result.length == specificIds.length) + + // // With bucket partitioning, pruning occurs at the file level, not manifest level + // // Bucket transforms use hash-based bucketing, so manifests may contain files from + // // multiple buckets. Iceberg can skip individual files based on bucket metadata, + // // but cannot skip entire manifests. + // assert( + // metrics("resultDataFiles").value < 8, + // "Bucket pruning should skip some files, but read " + + // s"${metrics("resultDataFiles").value} out of 8") + // assert( + // metrics("skippedDataFiles").value > 0, + // "Expected skipped data files due to bucket pruning, got" + + // s"${metrics("skippedDataFiles").value}") + + // spark.sql("DROP TABLE test_cat.db.bucket_pruning") + // } + // } + // } + + // test("partition pruning - truncate transform verifies files are skipped") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.truncate_pruning ( + // id INT, + // message STRING + // ) USING iceberg + // PARTITIONED BY (truncate(5, message)) + // """) + + // val prefixes = Seq("alpha", "bravo", "charlie", "delta", "echo") + // prefixes.zipWithIndex.foreach { case (prefix, idx) => + // spark.sql(s""" + // INSERT INTO test_cat.db.truncate_pruning + // SELECT + // id, + // CONCAT('$prefix', '_suffix_', CAST(id AS STRING)) as message + // FROM range(${idx * 10}, ${(idx + 1) * 10}) + // """) + // } + + // val df = spark.sql(""" + // SELECT * FROM test_cat.db.truncate_pruning + // WHERE message LIKE 'alpha%' + // """) + + // val scanNodes = df.queryExecution.executedPlan + // .collectLeaves() + // .collect { case s: CometIcebergNativeScanExec => s } + + // assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") + + // val metrics = scanNodes.head.metrics + + // val result = df.collect() + // assert(result.length == 10) + // assert(result.forall(_.getString(1).startsWith("alpha"))) + + // // Partition pruning occurs at the manifest level, not file level + // // Each INSERT creates one manifest, so we verify skippedDataManifests + // assert( + // metrics("resultDataFiles").value == 1, + // s"Truncate pruning should only read 1 file, read ${metrics("resultDataFiles").value}") + // assert( + // metrics("skippedDataManifests").value == 4, + // s"Expected 4 skipped manifests, got ${metrics("skippedDataManifests").value}") + + // spark.sql("DROP TABLE test_cat.db.truncate_pruning") + // } + // } + // } + + // test("partition pruning - hour transform verifies files are skipped") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.hour_pruning ( + // id INT, + // event_time TIMESTAMP, + // data STRING + // ) USING iceberg + // PARTITIONED BY (hour(event_time)) + // """) + + // (0 until 6).foreach { hour => + // spark.sql(s""" + // INSERT INTO test_cat.db.hour_pruning + // SELECT + // id, + // CAST('2024-01-01 $hour:00:00' AS TIMESTAMP) as event_time, + // CONCAT('event_', CAST(id AS STRING)) as data + // FROM range(${hour * 10}, ${(hour + 1) * 10}) + // """) + // } + + // val df = spark.sql(""" + // SELECT * FROM test_cat.db.hour_pruning + // WHERE event_time >= CAST('2024-01-01 04:00:00' AS TIMESTAMP) + // """) + + // val scanNodes = df.queryExecution.executedPlan + // .collectLeaves() + // .collect { case s: CometIcebergNativeScanExec => s } + + // assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") + + // val metrics = scanNodes.head.metrics + + // val result = df.collect() + // assert(result.length == 20) + + // // Partition pruning occurs at the manifest level, not file level + // // Each INSERT creates one manifest, so we verify skippedDataManifests + // assert( + // metrics("resultDataFiles").value == 2, + // s"Hour pruning should read 2 files (hours 4-5), read ${metrics("resultDataFiles").value}") + // assert( + // metrics("skippedDataManifests").value == 4, + // s"Expected 4 skipped manifests (hours 0-3), got ${metrics("skippedDataManifests").value}") + + // spark.sql("DROP TABLE test_cat.db.hour_pruning") + // } + // } + // } + + // test("schema evolution - add column") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.schema_evolution ( + // id INT, + // name STRING + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.schema_evolution VALUES (1, 'Alice'), (2, 'Bob') + // """) + + // spark.sql("ALTER TABLE test_cat.db.schema_evolution ADD COLUMN age INT") + + // spark.sql(""" + // INSERT INTO test_cat.db.schema_evolution VALUES (3, 'Charlie', 30), (4, 'Diana', 25) + // """) + + // checkIcebergNativeScan("SELECT * FROM test_cat.db.schema_evolution ORDER BY id") + // checkIcebergNativeScan("SELECT id, name FROM test_cat.db.schema_evolution ORDER BY id") + // checkIcebergNativeScan( + // "SELECT id, age FROM test_cat.db.schema_evolution WHERE age IS NOT NULL ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.schema_evolution") + // } + // } + // } + + // test("schema evolution - drop column") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.drop_column_test ( + // id INT, + // name STRING, + // age INT + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.drop_column_test VALUES (1, 'Alice', 30), (2, 'Bob', 25) + // """) + + // // Drop the age column + // spark.sql("ALTER TABLE test_cat.db.drop_column_test DROP COLUMN age") + + // // Insert new data without the age column + // spark.sql(""" + // INSERT INTO test_cat.db.drop_column_test VALUES (3, 'Charlie'), (4, 'Diana') + // """) + + // // Read all data - must handle old files (with age) and new files (without age) + // checkIcebergNativeScan("SELECT * FROM test_cat.db.drop_column_test ORDER BY id") + // checkIcebergNativeScan("SELECT id, name FROM test_cat.db.drop_column_test ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.drop_column_test") + // } + // } + // } + + // test("migration - basic read after migration (fallback for no field ID)") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // val sourceName = "parquet_source" + // val destName = "test_cat.db.iceberg_dest" + // val dataPath = s"${warehouseDir.getAbsolutePath}/source_data" + + // // Step 1: Create regular Parquet table (without field IDs) + // spark + // .range(10) + // .selectExpr( + // "CAST(id AS INT) as id", + // "CONCAT('name_', CAST(id AS STRING)) as name", + // "CAST(id * 2 AS DOUBLE) as value") + // .write + // .mode("overwrite") + // .option("path", dataPath) + // .saveAsTable(sourceName) + + // // Step 2: Snapshot the Parquet table into Iceberg using SparkActions API + // try { + // val actionsClass = Class.forName("org.apache.iceberg.spark.actions.SparkActions") + // val getMethod = actionsClass.getMethod("get") + // val actions = getMethod.invoke(null) + // val snapshotMethod = actions.getClass.getMethod("snapshotTable", classOf[String]) + // val snapshotAction = snapshotMethod.invoke(actions, sourceName) + // val asMethod = snapshotAction.getClass.getMethod("as", classOf[String]) + // val snapshotWithDest = asMethod.invoke(snapshotAction, destName) + // val executeMethod = snapshotWithDest.getClass.getMethod("execute") + // executeMethod.invoke(snapshotWithDest) + + // // Step 3: Read the Iceberg table - Parquet files have no field IDs, so position-based mapping is used + // checkIcebergNativeScan(s"SELECT * FROM $destName ORDER BY id") + // checkIcebergNativeScan(s"SELECT id, name FROM $destName ORDER BY id") + // checkIcebergNativeScan(s"SELECT value FROM $destName WHERE id < 5 ORDER BY id") + + // spark.sql(s"DROP TABLE $destName") + // spark.sql(s"DROP TABLE $sourceName") + // } catch { + // case _: ClassNotFoundException => + // cancel("Iceberg Actions API not available - requires iceberg-spark-runtime") + // } + // } + // } + // } + + // test("migration - hive-style partitioned table has partition values") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // val sourceName = "parquet_partitioned_source" + // val destName = "test_cat.db.iceberg_partitioned" + // val dataPath = s"${warehouseDir.getAbsolutePath}/partitioned_data" + + // // Hive-style partitioning stores partition values in directory paths, not in data files + // spark + // .range(10) + // .selectExpr( + // "CAST(id AS INT) as partition_col", + // "CONCAT('data_', CAST(id AS STRING)) as data") + // .write + // .mode("overwrite") + // .partitionBy("partition_col") + // .option("path", dataPath) + // .saveAsTable(sourceName) + + // try { + // val actionsClass = Class.forName("org.apache.iceberg.spark.actions.SparkActions") + // val getMethod = actionsClass.getMethod("get") + // val actions = getMethod.invoke(null) + // val snapshotMethod = actions.getClass.getMethod("snapshotTable", classOf[String]) + // val snapshotAction = snapshotMethod.invoke(actions, sourceName) + // val asMethod = snapshotAction.getClass.getMethod("as", classOf[String]) + // val snapshotWithDest = asMethod.invoke(snapshotAction, destName) + // val executeMethod = snapshotWithDest.getClass.getMethod("execute") + // executeMethod.invoke(snapshotWithDest) + + // // Partition columns must have actual values from manifests, not NULL + // checkIcebergNativeScan(s"SELECT * FROM $destName ORDER BY partition_col") + // checkIcebergNativeScan( + // s"SELECT partition_col, data FROM $destName WHERE partition_col < 5 ORDER BY partition_col") + + // spark.sql(s"DROP TABLE $destName") + // spark.sql(s"DROP TABLE $sourceName") + // } catch { + // case _: ClassNotFoundException => + // cancel("Iceberg Actions API not available - requires iceberg-spark-runtime") + // } + // } + // } + // } + + // test("projection - column subset, reordering, and duplication") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // // Create table with multiple columns + // spark.sql(""" + // CREATE TABLE test_cat.db.proj_test ( + // id INT, + // name STRING, + // value DOUBLE, + // flag BOOLEAN + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.proj_test + // VALUES (1, 'Alice', 10.5, true), + // (2, 'Bob', 20.3, false), + // (3, 'Charlie', 30.7, true) + // """) + + // // Test 1: Column subset (only 2 of 4 columns) + // checkIcebergNativeScan("SELECT name, value FROM test_cat.db.proj_test ORDER BY id") + + // // Test 2: Reordered columns (reverse order) + // checkIcebergNativeScan("SELECT value, name, id FROM test_cat.db.proj_test ORDER BY id") + + // // Test 3: Duplicate columns + // checkIcebergNativeScan( + // "SELECT id, name, id AS id2 FROM test_cat.db.proj_test ORDER BY id") + + // // Test 4: Single column + // checkIcebergNativeScan("SELECT name FROM test_cat.db.proj_test ORDER BY name") + + // // Test 5: Different ordering with subset + // checkIcebergNativeScan("SELECT flag, id FROM test_cat.db.proj_test ORDER BY id") + + // // Test 6: Multiple duplicates + // checkIcebergNativeScan( + // "SELECT name, value, name AS name2, value AS value2 FROM test_cat.db.proj_test ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.proj_test") + // } + // } + // } + + // test("complex type - array") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.array_test ( + // id INT, + // name STRING, + // values ARRAY + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.array_test + // VALUES (1, 'Alice', array(1, 2, 3)), (2, 'Bob', array(4, 5, 6)) + // """) + + // checkIcebergNativeScan("SELECT * FROM test_cat.db.array_test ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.array_test") + // } + // } + // } + + // test("complex type - map") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.map_test ( + // id INT, + // name STRING, + // properties MAP + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.map_test + // VALUES (1, 'Alice', map('age', 30, 'score', 95)), (2, 'Bob', map('age', 25, 'score', 87)) + // """) + + // checkIcebergNativeScan("SELECT * FROM test_cat.db.map_test ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.map_test") + // } + // } + // } + + // test("complex type - struct") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.struct_test ( + // id INT, + // name STRING, + // address STRUCT + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.struct_test + // VALUES (1, 'Alice', struct('NYC', 10001)), (2, 'Bob', struct('LA', 90001)) + // """) + + // checkIcebergNativeScan("SELECT * FROM test_cat.db.struct_test ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.struct_test") + // } + // } + // } + + // test("UUID type - native Iceberg UUID column (reproduces type mismatch)") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // import org.apache.iceberg.catalog.TableIdentifier + // import org.apache.iceberg.spark.SparkCatalog + // import org.apache.iceberg.types.Types + // import org.apache.iceberg.{PartitionSpec, Schema} + + // // Use Iceberg API to create table with native UUID type + // // (not possible via Spark SQL CREATE TABLE) + // // Get Spark's catalog instance to ensure the table is visible to Spark + // val sparkCatalog = spark.sessionState.catalogManager + // .catalog("test_cat") + // .asInstanceOf[SparkCatalog] + + // spark.sql("CREATE NAMESPACE IF NOT EXISTS test_cat.db") + + // // UUID is stored as FixedSizeBinary(16) but must be presented as Utf8 to Spark + // val schema = new Schema( + // Types.NestedField.required(1, "id", Types.IntegerType.get()), + // Types.NestedField.optional(2, "uuid", Types.UUIDType.get())) + // val tableIdent = TableIdentifier.of("db", "uuid_test") + // sparkCatalog.icebergCatalog.createTable(tableIdent, schema, PartitionSpec.unpartitioned()) + + // spark.sql(""" + // INSERT INTO test_cat.db.uuid_test VALUES + // (1, 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11'), + // (2, 'b1ffcd88-8d1a-3de7-aa5c-5aa8ac269a00'), + // (3, 'c2aade77-7e0b-2cf6-99e4-4998bc158b22') + // """) + + // checkIcebergNativeScan("SELECT * FROM test_cat.db.uuid_test ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.uuid_test") + // } + // } + // } + + // test("verify all Iceberg planning metrics are populated") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // val icebergPlanningMetricNames = Seq( + // "totalPlanningDuration", + // "totalDataManifest", + // "scannedDataManifests", + // "skippedDataManifests", + // "resultDataFiles", + // "skippedDataFiles", + // "totalDataFileSize", + // "totalDeleteManifests", + // "scannedDeleteManifests", + // "skippedDeleteManifests", + // "totalDeleteFileSize", + // "resultDeleteFiles", + // "equalityDeleteFiles", + // "indexedDeleteFiles", + // "positionalDeleteFiles", + // "skippedDeleteFiles") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.metrics_test ( + // id INT, + // value DOUBLE + // ) USING iceberg + // """) + + // // Create multiple files to ensure non-zero manifest/file counts + // spark + // .range(10000) + // .selectExpr("CAST(id AS INT)", "CAST(id * 1.5 AS DOUBLE) as value") + // .coalesce(1) + // .write + // .format("iceberg") + // .mode("append") + // .saveAsTable("test_cat.db.metrics_test") + + // spark + // .range(10001, 20000) + // .selectExpr("CAST(id AS INT)", "CAST(id * 1.5 AS DOUBLE) as value") + // .coalesce(1) + // .write + // .format("iceberg") + // .mode("append") + // .saveAsTable("test_cat.db.metrics_test") + + // val df = spark.sql("SELECT * FROM test_cat.db.metrics_test WHERE id < 10000") + + // // Must extract metrics before collect() because planning happens at plan creation + // val scanNodes = df.queryExecution.executedPlan + // .collectLeaves() + // .collect { case s: CometIcebergNativeScanExec => s } + + // assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") + + // val metrics = scanNodes.head.metrics + + // icebergPlanningMetricNames.foreach { metricName => + // assert(metrics.contains(metricName), s"metric $metricName was not found") + // } + + // // Planning metrics are populated during plan creation, so they're already available + // assert(metrics("totalDataManifest").value > 0, "totalDataManifest should be > 0") + // assert(metrics("resultDataFiles").value > 0, "resultDataFiles should be > 0") + // assert(metrics("totalDataFileSize").value > 0, "totalDataFileSize should be > 0") + + // df.collect() + + // assert(metrics("output_rows").value == 10000) + // assert(metrics("num_splits").value > 0) + // assert(metrics("time_elapsed_opening").value > 0) + // assert(metrics("time_elapsed_scanning_until_data").value > 0) + // assert(metrics("time_elapsed_scanning_total").value > 0) + // assert(metrics("time_elapsed_processing").value > 0) + // // ImmutableSQLMetric prevents these from being reset to 0 after execution + // assert( + // metrics("totalDataManifest").value > 0, + // "totalDataManifest should still be > 0 after execution") + // assert( + // metrics("resultDataFiles").value > 0, + // "resultDataFiles should still be > 0 after execution") + + // spark.sql("DROP TABLE test_cat.db.metrics_test") + // } + // } + // } + + // test("verify manifest pruning metrics") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // // Partition by category to enable manifest-level pruning + // spark.sql(""" + // CREATE TABLE test_cat.db.pruning_test ( + // id INT, + // category STRING, + // value DOUBLE + // ) USING iceberg + // PARTITIONED BY (category) + // """) + + // // Each category gets its own manifest entry + // spark.sql(""" + // INSERT INTO test_cat.db.pruning_test + // SELECT id, 'A' as category, CAST(id * 1.5 AS DOUBLE) as value + // FROM range(1000) + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.pruning_test + // SELECT id, 'B' as category, CAST(id * 2.0 AS DOUBLE) as value + // FROM range(1000, 2000) + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.pruning_test + // SELECT id, 'C' as category, CAST(id * 2.5 AS DOUBLE) as value + // FROM range(2000, 3000) + // """) + + // // Filter should prune B and C partitions at manifest level + // val df = spark.sql("SELECT * FROM test_cat.db.pruning_test WHERE category = 'A'") + + // val scanNodes = df.queryExecution.executedPlan + // .collectLeaves() + // .collect { case s: CometIcebergNativeScanExec => s } + + // assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") + + // val metrics = scanNodes.head.metrics + + // // Iceberg prunes entire manifests when all files in a manifest don't match the filter + // assert( + // metrics("resultDataFiles").value == 1, + // s"Expected 1 result data file, got ${metrics("resultDataFiles").value}") + // assert( + // metrics("scannedDataManifests").value == 1, + // s"Expected 1 scanned manifest, got ${metrics("scannedDataManifests").value}") + // assert( + // metrics("skippedDataManifests").value == 2, + // s"Expected 2 skipped manifests, got ${metrics("skippedDataManifests").value}") + + // // Verify the query actually returns correct results + // val result = df.collect() + // assert(metrics("output_rows").value == 1000) + // assert(result.length == 1000, s"Expected 1000 rows, got ${result.length}") + + // spark.sql("DROP TABLE test_cat.db.pruning_test") + // } + // } + // } + + // test("verify delete file metrics - MOR table") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // // Equality delete columns force MOR behavior instead of COW + // spark.sql(""" + // CREATE TABLE test_cat.db.delete_metrics ( + // id INT, + // category STRING, + // value DOUBLE + // ) USING iceberg + // TBLPROPERTIES ( + // 'write.delete.mode' = 'merge-on-read', + // 'write.merge.mode' = 'merge-on-read', + // 'write.delete.equality-delete-columns' = 'id' + // ) + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.delete_metrics + // VALUES + // (1, 'A', 10.5), (2, 'B', 20.3), (3, 'A', 30.7), + // (4, 'B', 15.2), (5, 'A', 25.8), (6, 'C', 35.0) + // """) + + // spark.sql("DELETE FROM test_cat.db.delete_metrics WHERE id IN (2, 4, 6)") + + // val df = spark.sql("SELECT * FROM test_cat.db.delete_metrics") + + // val scanNodes = df.queryExecution.executedPlan + // .collectLeaves() + // .collect { case s: CometIcebergNativeScanExec => s } + + // assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") + + // val metrics = scanNodes.head.metrics + + // // Iceberg may convert equality deletes to positional deletes internally + // assert( + // metrics("resultDeleteFiles").value > 0, + // s"Expected result delete files > 0, got ${metrics("resultDeleteFiles").value}") + // assert( + // metrics("totalDeleteFileSize").value > 0, + // s"Expected total delete file size > 0, got ${metrics("totalDeleteFileSize").value}") + + // val hasDeletes = metrics("positionalDeleteFiles").value > 0 || + // metrics("equalityDeleteFiles").value > 0 + // assert(hasDeletes, "Expected either positional or equality delete files > 0") + + // val result = df.collect() + // assert(metrics("output_rows").value == 3) + // assert(result.length == 3, s"Expected 3 rows after deletes, got ${result.length}") + + // spark.sql("DROP TABLE test_cat.db.delete_metrics") + // } + // } + // } + + // test("verify output_rows metric reflects row-level filtering in scan") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", + // // Create relatively small files to get multiple row groups per file + // "spark.sql.files.maxRecordsPerFile" -> "1000") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.filter_metric_test ( + // id INT, + // category STRING, + // value DOUBLE + // ) USING iceberg + // """) + + // // Insert 10,000 rows with mixed category values + // // This ensures row groups will have mixed data that can't be completely eliminated + // spark.sql(""" + // INSERT INTO test_cat.db.filter_metric_test + // SELECT + // id, + // CASE WHEN id % 2 = 0 THEN 'even' ELSE 'odd' END as category, + // CAST(id * 1.5 AS DOUBLE) as value + // FROM range(10000) + // """) + + // // Apply a highly selective filter on id that will filter ~99% of rows + // // This filter requires row-level evaluation because: + // // - Row groups contain ranges of IDs (0-999, 1000-1999, etc.) + // // - The first row group (0-999) cannot be fully eliminated by stats alone + // // - Row-level filtering must apply "id < 100" to filter out rows 100-999 + // val df = spark.sql(""" + // SELECT * FROM test_cat.db.filter_metric_test + // WHERE id < 100 + // """) + + // val scanNodes = df.queryExecution.executedPlan + // .collectLeaves() + // .collect { case s: CometIcebergNativeScanExec => s } + + // assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") + + // val metrics = scanNodes.head.metrics + + // // Execute the query to populate metrics + // val result = df.collect() + + // // The filter "id < 100" should match exactly 100 rows (0-99) + // assert(result.length == 100, s"Expected 100 rows after filter, got ${result.length}") + + // // CRITICAL: Verify output_rows metric matches the filtered count + // // If row-level filtering is working, this should be 100 + // // If only row group filtering is working, this would be ~1000 (entire first row group) + // assert( + // metrics("output_rows").value == 100, + // s"Expected output_rows=100 (filtered count), got ${metrics("output_rows").value}. " + + // "This indicates row-level filtering may not be working correctly.") + + // // Verify the filter actually selected the right rows + // val ids = result.map(_.getInt(0)).sorted + // assert(ids.head == 0, s"Expected first id=0, got ${ids.head}") + // assert(ids.last == 99, s"Expected last id=99, got ${ids.last}") + // assert(ids.forall(_ < 100), "All IDs should be < 100") + + // spark.sql("DROP TABLE test_cat.db.filter_metric_test") + // } + // } + // } + + // test("schema evolution - read old snapshot after column drop (VERSION AS OF)") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", + // // Force LOCAL mode to use iceberg-rust + // "spark.sql.iceberg.read.data-planning-mode" -> "local") { + + // // This test verifies that Comet correctly handles reading old snapshots after schema changes, + // // which is a form of backward schema evolution. This corresponds to these Iceberg Java tests: + // // - TestIcebergSourceHadoopTables::testSnapshotReadAfterDropColumn + // // - TestIcebergSourceHadoopTables::testSnapshotReadAfterAddAndDropColumn + // // - TestIcebergSourceHiveTables::testSnapshotReadAfterDropColumn + // // - TestIcebergSourceHiveTables::testSnapshotReadAfterAddAndDropColumn + // // - TestSnapshotSelection::testSnapshotSelectionByTagWithSchemaChange + + // // Step 1: Create table with columns (id, data, category) + // spark.sql(""" + // CREATE TABLE test_cat.db.schema_evolution_test ( + // id INT, + // data STRING, + // category STRING + // ) USING iceberg + // """) + + // // Step 2: Write data with all three columns + // spark.sql(""" + // INSERT INTO test_cat.db.schema_evolution_test + // VALUES (1, 'x', 'A'), (2, 'y', 'A'), (3, 'z', 'B') + // """) + + // // Get snapshot ID before schema change + // val snapshotIdBefore = spark + // .sql("SELECT snapshot_id FROM test_cat.db.schema_evolution_test.snapshots ORDER BY committed_at DESC LIMIT 1") + // .collect()(0) + // .getLong(0) + + // // Verify data is correct before schema change + // checkIcebergNativeScan("SELECT * FROM test_cat.db.schema_evolution_test ORDER BY id") + + // // Step 3: Drop the "data" column + // spark.sql("ALTER TABLE test_cat.db.schema_evolution_test DROP COLUMN data") + + // // Step 4: Read the old snapshot (before column was dropped) using VERSION AS OF + // // This requires using the snapshot's schema, not the current table schema + // checkIcebergNativeScan( + // s"SELECT * FROM test_cat.db.schema_evolution_test VERSION AS OF $snapshotIdBefore ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.schema_evolution_test") + // } + // } + // } + + // test("schema evolution - branch read after adding DATE column") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", + // "spark.sql.iceberg.read.data-planning-mode" -> "local") { + + // // Reproduces: TestSelect::readAndWriteWithBranchAfterSchemaChange + // // Error: "Iceberg scan error: Unexpected => unexpected target column type Date32" + // // + // // Issue: When reading old data from a branch after the table schema evolved to add + // // a DATE column, the schema adapter fails to handle Date32 type conversion. + + // // Step 1: Create table with (id, data, float_col) + // spark.sql(""" + // CREATE TABLE test_cat.db.date_branch_test ( + // id BIGINT, + // data STRING, + // float_col FLOAT + // ) USING iceberg + // """) + + // // Step 2: Insert data + // spark.sql(""" + // INSERT INTO test_cat.db.date_branch_test + // VALUES (1, 'a', 1.0), (2, 'b', 2.0), (3, 'c', CAST('NaN' AS FLOAT)) + // """) + + // // Step 3: Create a branch at this point using Iceberg API + // val catalog = spark.sessionState.catalogManager.catalog("test_cat") + // val ident = + // org.apache.spark.sql.connector.catalog.Identifier.of(Array("db"), "date_branch_test") + // val sparkTable = catalog + // .asInstanceOf[org.apache.iceberg.spark.SparkCatalog] + // .loadTable(ident) + // .asInstanceOf[org.apache.iceberg.spark.source.SparkTable] + // val table = sparkTable.table() + // val snapshotId = table.currentSnapshot().snapshotId() + // table.manageSnapshots().createBranch("test_branch", snapshotId).commit() + + // // Step 4: Evolve schema - drop float_col, add date_col + // spark.sql("ALTER TABLE test_cat.db.date_branch_test DROP COLUMN float_col") + // spark.sql("ALTER TABLE test_cat.db.date_branch_test ADD COLUMN date_col DATE") + + // // Step 5: Insert more data with the new schema + // spark.sql(""" + // INSERT INTO test_cat.db.date_branch_test + // VALUES (4, 'd', DATE '2024-04-04'), (5, 'e', DATE '2024-05-05') + // """) + + // // Step 6: Read from the branch using VERSION AS OF + // // This reads old data (id, data, float_col) but applies the current schema (id, data, date_col) + // // The old data files don't have date_col, so it should be NULL + // checkIcebergNativeScan( + // "SELECT * FROM test_cat.db.date_branch_test VERSION AS OF 'test_branch' ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.date_branch_test") + // } + // } + // } + + // // Complex type filter tests + // test("complex type filter - struct column IS NULL") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.struct_filter_test ( + // id INT, + // name STRING, + // address STRUCT + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.struct_filter_test + // VALUES + // (1, 'Alice', struct('NYC', 10001)), + // (2, 'Bob', struct('LA', 90001)), + // (3, 'Charlie', NULL) + // """) + + // // Test filtering on struct IS NULL - this should fall back to Spark + // // (iceberg-rust doesn't support IS NULL on complex type columns yet) + // checkSparkAnswer( + // "SELECT * FROM test_cat.db.struct_filter_test WHERE address IS NULL ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.struct_filter_test") + // } + // } + // } + + // test("complex type filter - struct field filter") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.struct_field_filter_test ( + // id INT, + // name STRING, + // address STRUCT + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.struct_field_filter_test + // VALUES + // (1, 'Alice', struct('NYC', 10001)), + // (2, 'Bob', struct('LA', 90001)), + // (3, 'Charlie', struct('NYC', 10002)) + // """) + + // // Test filtering on struct field - this should use native scan now! + // // iceberg-rust supports nested field filters like address.city = 'NYC' + // checkIcebergNativeScan( + // "SELECT * FROM test_cat.db.struct_field_filter_test WHERE address.city = 'NYC' ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.struct_field_filter_test") + // } + // } + // } + + // test("complex type filter - entire struct value") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.struct_value_filter_test ( + // id INT, + // name STRING, + // address STRUCT + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.struct_value_filter_test + // VALUES + // (1, 'Alice', named_struct('city', 'NYC', 'zip', 10001)), + // (2, 'Bob', named_struct('city', 'LA', 'zip', 90001)), + // (3, 'Charlie', named_struct('city', 'NYC', 'zip', 10001)) + // """) + + // // Test filtering on entire struct value - this falls back to Spark + // // (Iceberg Java doesn't push down this type of filter) + // checkSparkAnswer( + // "SELECT * FROM test_cat.db.struct_value_filter_test WHERE address = named_struct('city', 'NYC', 'zip', 10001) ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.struct_value_filter_test") + // } + // } + // } + + // test("complex type filter - array column IS NULL") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.array_filter_test ( + // id INT, + // name STRING, + // values ARRAY + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.array_filter_test + // VALUES + // (1, 'Alice', array(1, 2, 3)), + // (2, 'Bob', array(4, 5, 6)), + // (3, 'Charlie', NULL) + // """) + + // // Test filtering on array IS NULL - this should fall back to Spark + // // (iceberg-rust doesn't support IS NULL on complex type columns yet) + // checkSparkAnswer( + // "SELECT * FROM test_cat.db.array_filter_test WHERE values IS NULL ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.array_filter_test") + // } + // } + // } + + // test("complex type filter - array element filter") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.array_element_filter_test ( + // id INT, + // name STRING, + // values ARRAY + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.array_element_filter_test + // VALUES + // (1, 'Alice', array(1, 2, 3)), + // (2, 'Bob', array(4, 5, 6)), + // (3, 'Charlie', array(1, 7, 8)) + // """) + + // // Test filtering with array_contains - this should fall back to Spark + // // (Iceberg Java only pushes down NOT NULL, which fails in iceberg-rust) + // checkSparkAnswer( + // "SELECT * FROM test_cat.db.array_element_filter_test WHERE array_contains(values, 1) ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.array_element_filter_test") + // } + // } + // } + + // test("complex type filter - entire array value") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.array_value_filter_test ( + // id INT, + // name STRING, + // values ARRAY + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.array_value_filter_test + // VALUES + // (1, 'Alice', array(1, 2, 3)), + // (2, 'Bob', array(4, 5, 6)), + // (3, 'Charlie', array(1, 2, 3)) + // """) + + // // Test filtering on entire array value - this should fall back to Spark + // // (Iceberg Java only pushes down NOT NULL, which fails in iceberg-rust) + // checkSparkAnswer( + // "SELECT * FROM test_cat.db.array_value_filter_test WHERE values = array(1, 2, 3) ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.array_value_filter_test") + // } + // } + // } + + // test("complex type filter - map column IS NULL") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.map_filter_test ( + // id INT, + // name STRING, + // properties MAP + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.map_filter_test + // VALUES + // (1, 'Alice', map('age', 30, 'score', 95)), + // (2, 'Bob', map('age', 25, 'score', 87)), + // (3, 'Charlie', NULL) + // """) + + // // Test filtering on map IS NULL - this should fall back to Spark + // // (iceberg-rust doesn't support IS NULL on complex type columns yet) + // checkSparkAnswer( + // "SELECT * FROM test_cat.db.map_filter_test WHERE properties IS NULL ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.map_filter_test") + // } + // } + // } + + // test("complex type filter - map key access filter") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // spark.sql(""" + // CREATE TABLE test_cat.db.map_key_filter_test ( + // id INT, + // name STRING, + // properties MAP + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.map_key_filter_test + // VALUES + // (1, 'Alice', map('age', 30, 'score', 95)), + // (2, 'Bob', map('age', 25, 'score', 87)), + // (3, 'Charlie', map('age', 30, 'score', 80)) + // """) + + // // Test filtering with map key access - this should fall back to Spark + // // (Iceberg Java only pushes down NOT NULL, which fails in iceberg-rust) + // checkSparkAnswer( + // "SELECT * FROM test_cat.db.map_key_filter_test WHERE properties['age'] = 30 ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.map_key_filter_test") + // } + // } + // } + + // // Test to reproduce "Field X not found in schema" errors + // // Mimics TestAggregatePushDown.testNaN() where aggregate output schema differs from table schema + // test("partitioned table with aggregates - reproduces Field not found error") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // // Create table partitioned by id, like TestAggregatePushDown.testNaN + // spark.sql(""" + // CREATE TABLE test_cat.db.agg_test ( + // id INT, + // data FLOAT + // ) USING iceberg + // PARTITIONED BY (id) + // """) + + // spark.sql(""" + // INSERT INTO test_cat.db.agg_test VALUES + // (1, CAST('NaN' AS FLOAT)), + // (1, CAST('NaN' AS FLOAT)), + // (2, 2.0), + // (2, CAST('NaN' AS FLOAT)), + // (3, CAST('NaN' AS FLOAT)), + // (3, 1.0) + // """) + + // // This aggregate query's output schema is completely different from table schema + // // When iceberg-rust tries to look up partition field 'id' (field 1 in table schema), + // // it needs to find it in the full table schema, not the aggregate output schema + // checkIcebergNativeScan( + // "SELECT count(*), max(data), min(data), count(data) FROM test_cat.db.agg_test") + + // spark.sql("DROP TABLE test_cat.db.agg_test") + // } + // } + // } + + // test("MOR partitioned table with timestamp_ntz - reproduces NULL partition issue") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // // Create partitioned table like TestRewritePositionDeleteFiles.testTimestampNtz + // spark.sql(""" + // CREATE TABLE test_cat.db.timestamp_ntz_partition_test ( + // id LONG, + // ts TIMESTAMP_NTZ, + // c1 STRING, + // c2 STRING + // ) USING iceberg + // PARTITIONED BY (ts) + // TBLPROPERTIES ( + // 'format-version' = '2', + // 'write.delete.mode' = 'merge-on-read', + // 'write.merge.mode' = 'merge-on-read' + // ) + // """) + + // // Insert data into multiple partitions + // spark.sql(""" + // INSERT INTO test_cat.db.timestamp_ntz_partition_test + // VALUES + // (1, TIMESTAMP_NTZ '2023-01-01 15:30:00', 'a', 'b'), + // (2, TIMESTAMP_NTZ '2023-01-02 15:30:00', 'c', 'd'), + // (3, TIMESTAMP_NTZ '2023-01-03 15:30:00', 'e', 'f') + // """) + + // // Delete some rows to create position delete files + // spark.sql("DELETE FROM test_cat.db.timestamp_ntz_partition_test WHERE id = 2") + + // // Query should work with NULL partition handling + // checkIcebergNativeScan( + // "SELECT * FROM test_cat.db.timestamp_ntz_partition_test ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.timestamp_ntz_partition_test") + // } + // } + // } + + // test("MOR partitioned table with decimal - reproduces NULL partition issue") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withTempIcebergDir { warehouseDir => + // withSQLConf( + // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.test_cat.type" -> "hadoop", + // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + + // // Create partitioned table like TestRewritePositionDeleteFiles.testDecimalPartition + // spark.sql(""" + // CREATE TABLE test_cat.db.decimal_partition_test ( + // id LONG, + // dec DECIMAL(18, 10), + // c1 STRING, + // c2 STRING + // ) USING iceberg + // PARTITIONED BY (dec) + // TBLPROPERTIES ( + // 'format-version' = '2', + // 'write.delete.mode' = 'merge-on-read', + // 'write.merge.mode' = 'merge-on-read' + // ) + // """) + + // // Insert data into multiple partitions + // spark.sql(""" + // INSERT INTO test_cat.db.decimal_partition_test + // VALUES + // (1, 1.0, 'a', 'b'), + // (2, 2.0, 'c', 'd'), + // (3, 3.0, 'e', 'f') + // """) + + // // Delete some rows to create position delete files + // spark.sql("DELETE FROM test_cat.db.decimal_partition_test WHERE id = 2") + + // // Query should work with NULL partition handling + // checkIcebergNativeScan("SELECT * FROM test_cat.db.decimal_partition_test ORDER BY id") + + // spark.sql("DROP TABLE test_cat.db.decimal_partition_test") + // } + // } + // } + + // test("REST catalog with native Iceberg scan") { + // assume(icebergAvailable, "Iceberg not available in classpath") + + // withRESTCatalog { (restUri, _, warehouseDir) => + // withSQLConf( + // "spark.sql.catalog.rest_cat" -> "org.apache.iceberg.spark.SparkCatalog", + // "spark.sql.catalog.rest_cat.catalog-impl" -> "org.apache.iceberg.rest.RESTCatalog", + // "spark.sql.catalog.rest_cat.uri" -> restUri, + // "spark.sql.catalog.rest_cat.warehouse" -> warehouseDir.getAbsolutePath, + // CometConf.COMET_ENABLED.key -> "true", + // CometConf.COMET_EXEC_ENABLED.key -> "true", + // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", + // CometConf.COMET_EXPLAIN_FALLBACK_ENABLED.key -> "true") { + + // // Create namespace first (REST catalog requires explicit namespace creation) + // spark.sql("CREATE NAMESPACE rest_cat.db") + + // // Create a table via REST catalog + // spark.sql(""" + // CREATE TABLE rest_cat.db.test_table ( + // id INT, + // name STRING, + // value DOUBLE + // ) USING iceberg + // """) + + // spark.sql(""" + // INSERT INTO rest_cat.db.test_table + // VALUES (1, 'Alice', 10.5), (2, 'Bob', 20.3), (3, 'Charlie', 30.7) + // """) + + // checkIcebergNativeScan("SELECT * FROM rest_cat.db.test_table ORDER BY id") + + // spark.sql("DROP TABLE rest_cat.db.test_table") + // spark.sql("DROP NAMESPACE rest_cat.db") + // } + // } + // } + + // Helper to create temp directory + def withTempDeltaDir(f: File => Unit): Unit = { + val dir = Files.createTempDirectory("comet-delta-test").toFile + try { + f(dir) + } finally { + def deleteRecursively(file: File): Unit = { + if (file.isDirectory) { + file.listFiles().foreach(deleteRecursively) + } + file.delete() + } + deleteRecursively(dir) + } + } +} From 82dada0cd32623ca12967d563ab2fb4da4e2a1bb Mon Sep 17 00:00:00 2001 From: Adam Binford Date: Sat, 3 Jan 2026 15:16:40 -0500 Subject: [PATCH 2/9] More tests --- .../apache/comet/delta/DeltaReflection.scala | 9 +- .../apache/comet/rules/CometScanRule.scala | 4 - .../spark/sql/comet/CometScanExec.scala | 12 +- .../apache/comet/CometDeltaNativeSuite.scala | 2370 +---------------- 4 files changed, 153 insertions(+), 2242 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/delta/DeltaReflection.scala b/spark/src/main/scala/org/apache/comet/delta/DeltaReflection.scala index be792c618b..fed617b1dd 100644 --- a/spark/src/main/scala/org/apache/comet/delta/DeltaReflection.scala +++ b/spark/src/main/scala/org/apache/comet/delta/DeltaReflection.scala @@ -32,8 +32,7 @@ object DeltaReflection extends Logging { /** * Iceberg class names used throughout Comet. */ - object ClassNames { - } + object ClassNames {} /** * Iceberg content types. @@ -78,7 +77,8 @@ object DeltaReflection extends Logging { } catch { case e: Exception => logError( - s"Delta reflection failure: Failed to get minReaderVersion from protocol: ${e.getMessage}") + "Delta reflection failure: Failed to get minReaderVersion from protocol: " + + s"${e.getMessage}") None } } @@ -90,7 +90,8 @@ object DeltaReflection extends Logging { } catch { case e: Exception => logError( - s"Delta reflection failure: Failed to get minReaderVersion from protocol: ${e.getMessage}") + "Delta reflection failure: Failed to get minReaderVersion from protocol: " + + s"${e.getMessage}") None } } diff --git a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala index 23e9600137..01e385b0ae 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -142,7 +142,6 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com scanExec.relation match { case r: HadoopFsRelation => - println(s"Found HadoopFsRelation: $r") if (!CometScanExec.isFileFormatSupported(r.fileFormat)) { return withInfo(scanExec, s"Unsupported file format ${r.fileFormat}") } @@ -157,8 +156,6 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com scanImpl = selectScan(scanExec, r.partitionSchema, hadoopConf) } - println(s"Using scan impl: $scanImpl") - if (scanImpl == SCAN_NATIVE_DATAFUSION && !CometNativeScan.isSupported(scanExec)) { return scanExec } @@ -191,7 +188,6 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com } else { // this is confusing, but we always insert a CometScanExec here, which may replaced // with a CometNativeExec when CometExecRule runs, depending on the scanImpl value. - println("Creatin scan") CometScanExec(scanExec, session, scanImpl) } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala index 1f951465d6..52ce7da65e 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala @@ -588,18 +588,13 @@ object CometScanExec { fileFormat.getClass().equals(classOf[ParquetFileFormat]) || isSupportedDeltaScan(fileFormat) } - val unsupportedDeltaReaderFeatures: Set[String] = Set( - "columnMapping", - "deletionVectors" - ) + val unsupportedDeltaReaderFeatures: Set[String] = Set("columnMapping", "deletionVectors") def isSupportedDeltaScan(fileFormat: FileFormat): Boolean = { if (fileFormat.getClass().getName() != "org.apache.spark.sql.delta.DeltaParquetFileFormat") { return false } - println("Getting metadata from delta") - // Delta scans without certain features enabled are simply normal Parquet scans that can // take advantage of the native scan, so check to see if it is compatible val deltaMetadata = CometDeltaNativeScanMetadata.extract(fileFormat) match { @@ -607,8 +602,6 @@ object CometScanExec { case None => return false } - println(s"Checking metadata from $deltaMetadata") - // Version 1 has no special features // Version 2 introduced column mapping, which is not supported // Version 3 changes to use the readerFeatures list instead, so we check for incompatible @@ -616,7 +609,8 @@ object CometScanExec { deltaMetadata.minReaderVersion match { case 1 => true case 2 => false - case 3 => deltaMetadata.readerFeatures.asScala.intersect(unsupportedDeltaReaderFeatures).isEmpty + case 3 => + deltaMetadata.readerFeatures.asScala.intersect(unsupportedDeltaReaderFeatures).isEmpty } } diff --git a/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala index 33d23d0590..4475896854 100644 --- a/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala @@ -22,15 +22,18 @@ package org.apache.comet import java.io.File import java.nio.file.Files +import scala.jdk.CollectionConverters._ + +import org.apache.spark.SparkConf import org.apache.spark.sql.CometTestBase +import org.apache.spark.sql.Row import org.apache.spark.sql.comet.CometNativeScanExec import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.SparkConf + +import io.delta.tables.DeltaTable /** - * Test suite for native Iceberg scan using FileScanTasks and iceberg-rust. - * - * Note: Requires Iceberg dependencies to be added to pom.xml + * Test suite for native Delta scan using built-in native Parquet reader for simple reads */ class CometDeltaNativeSuite extends CometTestBase { @@ -41,16 +44,6 @@ class CometDeltaNativeSuite extends CometTestBase { .set(CometConf.COMET_NATIVE_SCAN_IMPL.key, "native_datafusion") } - // Skip these tests if Iceberg is not available in classpath - private def deltaAvailable: Boolean = { - try { - Class.forName("org.apache.spark.sql.delta.DeltaLog") - true - } catch { - case _: ClassNotFoundException => false - } - } - /** Collects all ComeNativeScanExec nodes from a plan */ private def collectNativeScans(plan: SparkPlan): Seq[CometNativeScanExec] = { collect(plan) { case scan: CometNativeScanExec => @@ -59,2240 +52,167 @@ class CometDeltaNativeSuite extends CometTestBase { } /** - * Helper to verify query correctness and that exactly one CometIcebergNativeScanExec is used. - * This ensures both correct results and that the native Iceberg scan operator is being used. + * Helper to verify query correctness and that exactly one CometNativeScanExec is used. This + * ensures both correct results and that the native scan operator is being used. */ - private def checkNativeScan(query: String): Unit = { - val (_, cometPlan) = checkSparkAnswer(query) - val nativeScans = collectNativeScans(cometPlan) - assert( - nativeScans.length == 1, - s"Expected exactly 1 CometNativeScanExec but found ${nativeScans.length}. Plan:\n$cometPlan") + private def checkNativeScan(query: String, native: Boolean): Unit = { + val (_, plan) = checkSparkAnswer(query) + val nativeScans = collectNativeScans(plan) + + if (native) { + assert( + nativeScans.length == 1, + s"Expected exactly 1 CometNativeScanExec but found ${nativeScans.length}. Plan:\n$plan") + } else { + assert( + nativeScans.length == 0, + s"Expected no CometNativeScanExec but found ${nativeScans.length}. Plan:\n$plan") + } } test("create and query simple Delta table") { - assume(deltaAvailable, "Delta not available in classpath") - - withTempDeltaDir { warehouseDir => - spark.sql(s""" - CREATE TABLE test_table ( - id INT, - name STRING, - value DOUBLE - ) USING delta LOCATION '${warehouseDir.toString()}' - """) - - spark.sql(""" - INSERT INTO test_table - VALUES (1, 'Alice', 10.5), (2, 'Bob', 20.3), (3, 'Charlie', 30.7) - """) - - checkNativeScan("SELECT * FROM test_table ORDER BY id") + import testImplicits._ + + withTempDir { dir => + DeltaTable + .create(spark) + .tableName("test_table") + .addColumn("id", "INT") + .addColumn("name", "STRING") + .addColumn("value", "DOUBLE") + .location(dir.getAbsolutePath) + .execute() + + Seq((1, "Alice", 10.5), (2, "Bob", 20.3), (3, "Charlie", 30.7)) + .toDF("id", "name", "value") + .write + .format("delta") + .mode("append") + .save(dir.getAbsolutePath) + + checkNativeScan("SELECT * FROM test_table ORDER BY id", true) spark.sql("DROP TABLE test_table") } } - // test("filter pushdown - equality predicates") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.filter_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.filter_cat.type" -> "hadoop", - // "spark.sql.catalog.filter_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE filter_cat.db.filter_test ( - // id INT, - // name STRING, - // value DOUBLE, - // active BOOLEAN - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO filter_cat.db.filter_test VALUES - // (1, 'Alice', 10.5, true), - // (2, 'Bob', 20.3, false), - // (3, 'Charlie', 30.7, true), - // (4, 'Diana', 15.2, false), - // (5, 'Eve', 25.8, true) - // """) - - // checkIcebergNativeScan("SELECT * FROM filter_cat.db.filter_test WHERE id = 3") - - // checkIcebergNativeScan("SELECT * FROM filter_cat.db.filter_test WHERE name = 'Bob'") - - // checkIcebergNativeScan("SELECT * FROM filter_cat.db.filter_test WHERE active = true") - - // spark.sql("DROP TABLE filter_cat.db.filter_test") - // } - // } - // } - - // test("filter pushdown - comparison operators") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.filter_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.filter_cat.type" -> "hadoop", - // "spark.sql.catalog.filter_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE filter_cat.db.comparison_test ( - // id INT, - // value DOUBLE - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO filter_cat.db.comparison_test VALUES - // (1, 10.5), (2, 20.3), (3, 30.7), (4, 15.2), (5, 25.8) - // """) - - // checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE value > 20.0") - - // checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE value >= 20.3") - - // checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE value < 20.0") - - // checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE value <= 20.3") - - // checkIcebergNativeScan("SELECT * FROM filter_cat.db.comparison_test WHERE id != 3") - - // spark.sql("DROP TABLE filter_cat.db.comparison_test") - // } - // } - // } - - // test("filter pushdown - AND/OR combinations") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.filter_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.filter_cat.type" -> "hadoop", - // "spark.sql.catalog.filter_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE filter_cat.db.logical_test ( - // id INT, - // category STRING, - // value DOUBLE - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO filter_cat.db.logical_test VALUES - // (1, 'A', 10.5), (2, 'B', 20.3), (3, 'A', 30.7), - // (4, 'B', 15.2), (5, 'A', 25.8), (6, 'C', 35.0) - // """) - - // checkIcebergNativeScan( - // "SELECT * FROM filter_cat.db.logical_test WHERE category = 'A' AND value > 20.0") - - // checkIcebergNativeScan( - // "SELECT * FROM filter_cat.db.logical_test WHERE category = 'B' OR value > 30.0") - - // checkIcebergNativeScan("""SELECT * FROM filter_cat.db.logical_test - // WHERE (category = 'A' AND value > 20.0) OR category = 'C'""") - - // spark.sql("DROP TABLE filter_cat.db.logical_test") - // } - // } - // } - - // test("filter pushdown - NULL checks") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.filter_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.filter_cat.type" -> "hadoop", - // "spark.sql.catalog.filter_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE filter_cat.db.null_test ( - // id INT, - // optional_value DOUBLE - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO filter_cat.db.null_test VALUES - // (1, 10.5), (2, NULL), (3, 30.7), (4, NULL), (5, 25.8) - // """) - - // checkIcebergNativeScan( - // "SELECT * FROM filter_cat.db.null_test WHERE optional_value IS NULL") - - // checkIcebergNativeScan( - // "SELECT * FROM filter_cat.db.null_test WHERE optional_value IS NOT NULL") - - // spark.sql("DROP TABLE filter_cat.db.null_test") - // } - // } - // } - - // test("filter pushdown - IN list") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.filter_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.filter_cat.type" -> "hadoop", - // "spark.sql.catalog.filter_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE filter_cat.db.in_test ( - // id INT, - // name STRING - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO filter_cat.db.in_test VALUES - // (1, 'Alice'), (2, 'Bob'), (3, 'Charlie'), - // (4, 'Diana'), (5, 'Eve'), (6, 'Frank') - // """) - - // checkIcebergNativeScan("SELECT * FROM filter_cat.db.in_test WHERE id IN (2, 4, 6)") - - // checkIcebergNativeScan( - // "SELECT * FROM filter_cat.db.in_test WHERE name IN ('Alice', 'Charlie', 'Eve')") - - // checkIcebergNativeScan("SELECT * FROM filter_cat.db.in_test WHERE id IS NOT NULL") - - // checkIcebergNativeScan("SELECT * FROM filter_cat.db.in_test WHERE id NOT IN (1, 3, 5)") - - // spark.sql("DROP TABLE filter_cat.db.in_test") - // } - // } - // } - - // test("verify filters are pushed to native scan") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.filter_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.filter_cat.type" -> "hadoop", - // "spark.sql.catalog.filter_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE filter_cat.db.filter_debug ( - // id INT, - // value DOUBLE - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO filter_cat.db.filter_debug VALUES - // (1, 10.5), (2, 20.3), (3, 30.7), (4, 15.2), (5, 25.8) - // """) - - // checkIcebergNativeScan("SELECT * FROM filter_cat.db.filter_debug WHERE id > 2") - - // spark.sql("DROP TABLE filter_cat.db.filter_debug") - // } - // } - // } - - // test("small table - verify no duplicate rows (1 file)") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.small_table ( - // id INT, - // name STRING - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.small_table - // VALUES (1, 'Alice'), (2, 'Bob'), (3, 'Charlie') - // """) - - // checkIcebergNativeScan("SELECT * FROM test_cat.db.small_table ORDER BY id") - // checkIcebergNativeScan("SELECT COUNT(DISTINCT id) FROM test_cat.db.small_table") - - // spark.sql("DROP TABLE test_cat.db.small_table") - // } - // } - // } - - // test("medium table - verify correct partition count (multiple files)") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", - // "spark.sql.files.maxRecordsPerFile" -> "10") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.medium_table ( - // id INT, - // value DOUBLE - // ) USING iceberg - // """) - - // // Insert 100 rows - should create multiple files with maxRecordsPerFile=10 - // spark.sql(""" - // INSERT INTO test_cat.db.medium_table - // SELECT id, CAST(id * 1.5 AS DOUBLE) as value - // FROM range(100) - // """) - - // // Verify results match Spark native (catches duplicates across partitions) - // checkIcebergNativeScan("SELECT * FROM test_cat.db.medium_table ORDER BY id") - // checkIcebergNativeScan("SELECT COUNT(DISTINCT id) FROM test_cat.db.medium_table") - // checkIcebergNativeScan("SELECT SUM(value) FROM test_cat.db.medium_table") - - // spark.sql("DROP TABLE test_cat.db.medium_table") - // } - // } - // } - - // test("large table - verify no duplicates with many files") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", - // "spark.sql.files.maxRecordsPerFile" -> "100") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.large_table ( - // id BIGINT, - // category STRING, - // value DOUBLE - // ) USING iceberg - // """) - - // // Insert 10,000 rows - with maxRecordsPerFile=100, creates ~100 files - // spark.sql(""" - // INSERT INTO test_cat.db.large_table - // SELECT - // id, - // CASE WHEN id % 3 = 0 THEN 'A' WHEN id % 3 = 1 THEN 'B' ELSE 'C' END as category, - // CAST(id * 2.5 AS DOUBLE) as value - // FROM range(10000) - // """) - - // checkIcebergNativeScan("SELECT COUNT(DISTINCT id) FROM test_cat.db.large_table") - // checkIcebergNativeScan("SELECT SUM(value) FROM test_cat.db.large_table") - // checkIcebergNativeScan( - // "SELECT category, COUNT(*) FROM test_cat.db.large_table GROUP BY category ORDER BY category") - - // spark.sql("DROP TABLE test_cat.db.large_table") - // } - // } - // } - - // test("partitioned table - verify key-grouped partitioning") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.partitioned_table ( - // id INT, - // category STRING, - // value DOUBLE - // ) USING iceberg - // PARTITIONED BY (category) - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.partitioned_table VALUES - // (1, 'A', 10.5), (2, 'B', 20.3), (3, 'C', 30.7), - // (4, 'A', 15.2), (5, 'B', 25.8), (6, 'C', 35.0), - // (7, 'A', 12.1), (8, 'B', 22.5), (9, 'C', 32.9) - // """) - - // checkIcebergNativeScan("SELECT * FROM test_cat.db.partitioned_table ORDER BY id") - // checkIcebergNativeScan( - // "SELECT * FROM test_cat.db.partitioned_table WHERE category = 'A' ORDER BY id") - // checkIcebergNativeScan( - // "SELECT * FROM test_cat.db.partitioned_table WHERE category = 'B' ORDER BY id") - // checkIcebergNativeScan( - // "SELECT category, COUNT(*) FROM test_cat.db.partitioned_table GROUP BY category ORDER BY category") - - // spark.sql("DROP TABLE test_cat.db.partitioned_table") - // } - // } - // } - - // test("empty table - verify graceful handling") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.empty_table ( - // id INT, - // name STRING - // ) USING iceberg - // """) - - // checkIcebergNativeScan("SELECT * FROM test_cat.db.empty_table") - // checkIcebergNativeScan("SELECT * FROM test_cat.db.empty_table WHERE id > 0") - - // spark.sql("DROP TABLE test_cat.db.empty_table") - // } - // } - // } - - // // MOR (Merge-On-Read) delete file tests. - // // Delete files are extracted from FileScanTasks and handled by iceberg-rust's ArrowReader, - // // which automatically applies both positional and equality deletes during scan execution. - // test("MOR table with POSITIONAL deletes - verify deletes are applied") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.positional_delete_test ( - // id INT, - // name STRING, - // value DOUBLE - // ) USING iceberg - // TBLPROPERTIES ( - // 'write.delete.mode' = 'merge-on-read', - // 'write.merge.mode' = 'merge-on-read' - // ) - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.positional_delete_test - // VALUES - // (1, 'Alice', 10.5), (2, 'Bob', 20.3), (3, 'Charlie', 30.7), - // (4, 'Diana', 15.2), (5, 'Eve', 25.8), (6, 'Frank', 35.0), - // (7, 'Grace', 12.1), (8, 'Hank', 22.5) - // """) - - // spark.sql("DELETE FROM test_cat.db.positional_delete_test WHERE id IN (2, 4, 6)") - - // checkIcebergNativeScan("SELECT * FROM test_cat.db.positional_delete_test ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.positional_delete_test") - // } - // } - // } - - // test("MOR table with EQUALITY deletes - verify deletes are applied") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // // Create table with equality delete columns specified - // // This forces Spark to use equality deletes instead of positional deletes - // spark.sql(""" - // CREATE TABLE test_cat.db.equality_delete_test ( - // id INT, - // category STRING, - // value DOUBLE - // ) USING iceberg - // TBLPROPERTIES ( - // 'write.delete.mode' = 'merge-on-read', - // 'write.merge.mode' = 'merge-on-read', - // 'write.delete.equality-delete-columns' = 'id' - // ) - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.equality_delete_test - // VALUES - // (1, 'A', 10.5), (2, 'B', 20.3), (3, 'A', 30.7), - // (4, 'B', 15.2), (5, 'A', 25.8), (6, 'C', 35.0) - // """) - - // spark.sql("DELETE FROM test_cat.db.equality_delete_test WHERE id IN (2, 4)") - - // checkIcebergNativeScan("SELECT * FROM test_cat.db.equality_delete_test ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.equality_delete_test") - // } - // } - // } - - // test("MOR table with multiple delete operations - mixed delete types") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.multi_delete_test ( - // id INT, - // data STRING - // ) USING iceberg - // TBLPROPERTIES ( - // 'write.delete.mode' = 'merge-on-read', - // 'write.merge.mode' = 'merge-on-read' - // ) - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.multi_delete_test - // SELECT id, CONCAT('data_', CAST(id AS STRING)) as data - // FROM range(100) - // """) - - // spark.sql("DELETE FROM test_cat.db.multi_delete_test WHERE id < 10") - // spark.sql("DELETE FROM test_cat.db.multi_delete_test WHERE id > 90") - // spark.sql("DELETE FROM test_cat.db.multi_delete_test WHERE id % 10 = 5") - - // checkIcebergNativeScan("SELECT * FROM test_cat.db.multi_delete_test ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.multi_delete_test") - // } - // } - // } - - // test("verify no duplicate rows across multiple partitions") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", - // // Create multiple files to ensure multiple partitions - // "spark.sql.files.maxRecordsPerFile" -> "50") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.multipart_test ( - // id INT, - // data STRING - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.multipart_test - // SELECT id, CONCAT('data_', CAST(id AS STRING)) as data - // FROM range(500) - // """) - - // // Critical: COUNT(*) vs COUNT(DISTINCT id) catches duplicates across partitions - // checkIcebergNativeScan("SELECT COUNT(DISTINCT id) FROM test_cat.db.multipart_test") - // checkIcebergNativeScan( - // "SELECT * FROM test_cat.db.multipart_test WHERE id < 10 ORDER BY id") - // checkIcebergNativeScan( - // "SELECT * FROM test_cat.db.multipart_test WHERE id >= 490 ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.multipart_test") - // } - // } - // } - - // test("filter pushdown with multi-partition table") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", - // "spark.sql.files.maxRecordsPerFile" -> "20") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.filter_multipart ( - // id INT, - // category STRING, - // value DOUBLE - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.filter_multipart - // SELECT - // id, - // CASE WHEN id % 2 = 0 THEN 'even' ELSE 'odd' END as category, - // CAST(id * 1.5 AS DOUBLE) as value - // FROM range(200) - // """) - - // checkIcebergNativeScan( - // "SELECT * FROM test_cat.db.filter_multipart WHERE id > 150 ORDER BY id") - // checkIcebergNativeScan( - // "SELECT * FROM test_cat.db.filter_multipart WHERE category = 'even' AND id < 50 ORDER BY id") - // checkIcebergNativeScan( - // "SELECT COUNT(DISTINCT id) FROM test_cat.db.filter_multipart WHERE id BETWEEN 50 AND 100") - // checkIcebergNativeScan( - // "SELECT SUM(value) FROM test_cat.db.filter_multipart WHERE category = 'odd'") - - // spark.sql("DROP TABLE test_cat.db.filter_multipart") - // } - // } - // } - - // test("date partitioned table with date range queries") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.date_partitioned ( - // id INT, - // event_date DATE, - // value STRING - // ) USING iceberg - // PARTITIONED BY (days(event_date)) - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.date_partitioned VALUES - // (1, DATE '2024-01-01', 'a'), (2, DATE '2024-01-02', 'b'), - // (3, DATE '2024-01-03', 'c'), (4, DATE '2024-01-15', 'd'), - // (5, DATE '2024-01-16', 'e'), (6, DATE '2024-02-01', 'f') - // """) - - // checkIcebergNativeScan("SELECT * FROM test_cat.db.date_partitioned ORDER BY id") - // checkIcebergNativeScan( - // "SELECT * FROM test_cat.db.date_partitioned WHERE event_date = DATE '2024-01-01'") - // checkIcebergNativeScan( - // "SELECT * FROM test_cat.db.date_partitioned WHERE event_date BETWEEN DATE '2024-01-01' AND DATE '2024-01-03' ORDER BY id") - // checkIcebergNativeScan( - // "SELECT event_date, COUNT(*) FROM test_cat.db.date_partitioned GROUP BY event_date ORDER BY event_date") - - // spark.sql("DROP TABLE test_cat.db.date_partitioned") - // } - // } - // } - - // test("bucket partitioned table") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.bucket_partitioned ( - // id INT, - // value DOUBLE - // ) USING iceberg - // PARTITIONED BY (bucket(4, id)) - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.bucket_partitioned - // SELECT id, CAST(id * 1.5 AS DOUBLE) as value - // FROM range(100) - // """) - - // checkIcebergNativeScan("SELECT * FROM test_cat.db.bucket_partitioned ORDER BY id") - // checkIcebergNativeScan("SELECT COUNT(DISTINCT id) FROM test_cat.db.bucket_partitioned") - // checkIcebergNativeScan("SELECT SUM(value) FROM test_cat.db.bucket_partitioned") - // checkIcebergNativeScan( - // "SELECT * FROM test_cat.db.bucket_partitioned WHERE id < 20 ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.bucket_partitioned") - // } - // } - // } - - // test("partition pruning - bucket transform verifies files are skipped") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.bucket_pruning ( - // id INT, - // data STRING - // ) USING iceberg - // PARTITIONED BY (bucket(8, id)) - // """) - - // (0 until 8).foreach { bucket => - // spark.sql(s""" - // INSERT INTO test_cat.db.bucket_pruning - // SELECT id, CONCAT('data_', CAST(id AS STRING)) as data - // FROM range(${bucket * 100}, ${(bucket + 1) * 100}) - // """) - // } - - // val specificIds = Seq(5, 15, 25) - // val df = spark.sql(s""" - // SELECT * FROM test_cat.db.bucket_pruning - // WHERE id IN (${specificIds.mkString(",")}) - // """) - - // val scanNodes = df.queryExecution.executedPlan - // .collectLeaves() - // .collect { case s: CometIcebergNativeScanExec => s } - - // assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") - - // val metrics = scanNodes.head.metrics - - // val result = df.collect() - // assert(result.length == specificIds.length) - - // // With bucket partitioning, pruning occurs at the file level, not manifest level - // // Bucket transforms use hash-based bucketing, so manifests may contain files from - // // multiple buckets. Iceberg can skip individual files based on bucket metadata, - // // but cannot skip entire manifests. - // assert( - // metrics("resultDataFiles").value < 8, - // "Bucket pruning should skip some files, but read " + - // s"${metrics("resultDataFiles").value} out of 8") - // assert( - // metrics("skippedDataFiles").value > 0, - // "Expected skipped data files due to bucket pruning, got" + - // s"${metrics("skippedDataFiles").value}") - - // spark.sql("DROP TABLE test_cat.db.bucket_pruning") - // } - // } - // } - - // test("partition pruning - truncate transform verifies files are skipped") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.truncate_pruning ( - // id INT, - // message STRING - // ) USING iceberg - // PARTITIONED BY (truncate(5, message)) - // """) - - // val prefixes = Seq("alpha", "bravo", "charlie", "delta", "echo") - // prefixes.zipWithIndex.foreach { case (prefix, idx) => - // spark.sql(s""" - // INSERT INTO test_cat.db.truncate_pruning - // SELECT - // id, - // CONCAT('$prefix', '_suffix_', CAST(id AS STRING)) as message - // FROM range(${idx * 10}, ${(idx + 1) * 10}) - // """) - // } - - // val df = spark.sql(""" - // SELECT * FROM test_cat.db.truncate_pruning - // WHERE message LIKE 'alpha%' - // """) - - // val scanNodes = df.queryExecution.executedPlan - // .collectLeaves() - // .collect { case s: CometIcebergNativeScanExec => s } - - // assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") - - // val metrics = scanNodes.head.metrics - - // val result = df.collect() - // assert(result.length == 10) - // assert(result.forall(_.getString(1).startsWith("alpha"))) - - // // Partition pruning occurs at the manifest level, not file level - // // Each INSERT creates one manifest, so we verify skippedDataManifests - // assert( - // metrics("resultDataFiles").value == 1, - // s"Truncate pruning should only read 1 file, read ${metrics("resultDataFiles").value}") - // assert( - // metrics("skippedDataManifests").value == 4, - // s"Expected 4 skipped manifests, got ${metrics("skippedDataManifests").value}") - - // spark.sql("DROP TABLE test_cat.db.truncate_pruning") - // } - // } - // } - - // test("partition pruning - hour transform verifies files are skipped") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.hour_pruning ( - // id INT, - // event_time TIMESTAMP, - // data STRING - // ) USING iceberg - // PARTITIONED BY (hour(event_time)) - // """) - - // (0 until 6).foreach { hour => - // spark.sql(s""" - // INSERT INTO test_cat.db.hour_pruning - // SELECT - // id, - // CAST('2024-01-01 $hour:00:00' AS TIMESTAMP) as event_time, - // CONCAT('event_', CAST(id AS STRING)) as data - // FROM range(${hour * 10}, ${(hour + 1) * 10}) - // """) - // } - - // val df = spark.sql(""" - // SELECT * FROM test_cat.db.hour_pruning - // WHERE event_time >= CAST('2024-01-01 04:00:00' AS TIMESTAMP) - // """) - - // val scanNodes = df.queryExecution.executedPlan - // .collectLeaves() - // .collect { case s: CometIcebergNativeScanExec => s } - - // assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") - - // val metrics = scanNodes.head.metrics - - // val result = df.collect() - // assert(result.length == 20) - - // // Partition pruning occurs at the manifest level, not file level - // // Each INSERT creates one manifest, so we verify skippedDataManifests - // assert( - // metrics("resultDataFiles").value == 2, - // s"Hour pruning should read 2 files (hours 4-5), read ${metrics("resultDataFiles").value}") - // assert( - // metrics("skippedDataManifests").value == 4, - // s"Expected 4 skipped manifests (hours 0-3), got ${metrics("skippedDataManifests").value}") - - // spark.sql("DROP TABLE test_cat.db.hour_pruning") - // } - // } - // } - - // test("schema evolution - add column") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.schema_evolution ( - // id INT, - // name STRING - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.schema_evolution VALUES (1, 'Alice'), (2, 'Bob') - // """) - - // spark.sql("ALTER TABLE test_cat.db.schema_evolution ADD COLUMN age INT") - - // spark.sql(""" - // INSERT INTO test_cat.db.schema_evolution VALUES (3, 'Charlie', 30), (4, 'Diana', 25) - // """) - - // checkIcebergNativeScan("SELECT * FROM test_cat.db.schema_evolution ORDER BY id") - // checkIcebergNativeScan("SELECT id, name FROM test_cat.db.schema_evolution ORDER BY id") - // checkIcebergNativeScan( - // "SELECT id, age FROM test_cat.db.schema_evolution WHERE age IS NOT NULL ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.schema_evolution") - // } - // } - // } - - // test("schema evolution - drop column") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.drop_column_test ( - // id INT, - // name STRING, - // age INT - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.drop_column_test VALUES (1, 'Alice', 30), (2, 'Bob', 25) - // """) - - // // Drop the age column - // spark.sql("ALTER TABLE test_cat.db.drop_column_test DROP COLUMN age") - - // // Insert new data without the age column - // spark.sql(""" - // INSERT INTO test_cat.db.drop_column_test VALUES (3, 'Charlie'), (4, 'Diana') - // """) - - // // Read all data - must handle old files (with age) and new files (without age) - // checkIcebergNativeScan("SELECT * FROM test_cat.db.drop_column_test ORDER BY id") - // checkIcebergNativeScan("SELECT id, name FROM test_cat.db.drop_column_test ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.drop_column_test") - // } - // } - // } - - // test("migration - basic read after migration (fallback for no field ID)") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // val sourceName = "parquet_source" - // val destName = "test_cat.db.iceberg_dest" - // val dataPath = s"${warehouseDir.getAbsolutePath}/source_data" - - // // Step 1: Create regular Parquet table (without field IDs) - // spark - // .range(10) - // .selectExpr( - // "CAST(id AS INT) as id", - // "CONCAT('name_', CAST(id AS STRING)) as name", - // "CAST(id * 2 AS DOUBLE) as value") - // .write - // .mode("overwrite") - // .option("path", dataPath) - // .saveAsTable(sourceName) - - // // Step 2: Snapshot the Parquet table into Iceberg using SparkActions API - // try { - // val actionsClass = Class.forName("org.apache.iceberg.spark.actions.SparkActions") - // val getMethod = actionsClass.getMethod("get") - // val actions = getMethod.invoke(null) - // val snapshotMethod = actions.getClass.getMethod("snapshotTable", classOf[String]) - // val snapshotAction = snapshotMethod.invoke(actions, sourceName) - // val asMethod = snapshotAction.getClass.getMethod("as", classOf[String]) - // val snapshotWithDest = asMethod.invoke(snapshotAction, destName) - // val executeMethod = snapshotWithDest.getClass.getMethod("execute") - // executeMethod.invoke(snapshotWithDest) - - // // Step 3: Read the Iceberg table - Parquet files have no field IDs, so position-based mapping is used - // checkIcebergNativeScan(s"SELECT * FROM $destName ORDER BY id") - // checkIcebergNativeScan(s"SELECT id, name FROM $destName ORDER BY id") - // checkIcebergNativeScan(s"SELECT value FROM $destName WHERE id < 5 ORDER BY id") - - // spark.sql(s"DROP TABLE $destName") - // spark.sql(s"DROP TABLE $sourceName") - // } catch { - // case _: ClassNotFoundException => - // cancel("Iceberg Actions API not available - requires iceberg-spark-runtime") - // } - // } - // } - // } - - // test("migration - hive-style partitioned table has partition values") { - // assume(icebergAvailable, "Iceberg not available in classpath") + test("supported reader features") { + import testImplicits._ + + withTempDir { dir => + DeltaTable + .create(spark) + .tableName("test_table") + .addColumn("id", "INT") + .addColumn("name", "STRING") + .addColumn("value", "DOUBLE") + .property("delta.checkpointPolicy", "v2") + .location(dir.getAbsolutePath) + .execute() + + Seq((1, "Alice", 10.5), (2, "Bob", 20.3), (3, "Charlie", 30.7)) + .toDF("id", "name", "value") + .write + .format("delta") + .mode("append") + .save(dir.getAbsolutePath) + + checkNativeScan("SELECT * FROM test_table ORDER BY id", true) - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // val sourceName = "parquet_partitioned_source" - // val destName = "test_cat.db.iceberg_partitioned" - // val dataPath = s"${warehouseDir.getAbsolutePath}/partitioned_data" - - // // Hive-style partitioning stores partition values in directory paths, not in data files - // spark - // .range(10) - // .selectExpr( - // "CAST(id AS INT) as partition_col", - // "CONCAT('data_', CAST(id AS STRING)) as data") - // .write - // .mode("overwrite") - // .partitionBy("partition_col") - // .option("path", dataPath) - // .saveAsTable(sourceName) - - // try { - // val actionsClass = Class.forName("org.apache.iceberg.spark.actions.SparkActions") - // val getMethod = actionsClass.getMethod("get") - // val actions = getMethod.invoke(null) - // val snapshotMethod = actions.getClass.getMethod("snapshotTable", classOf[String]) - // val snapshotAction = snapshotMethod.invoke(actions, sourceName) - // val asMethod = snapshotAction.getClass.getMethod("as", classOf[String]) - // val snapshotWithDest = asMethod.invoke(snapshotAction, destName) - // val executeMethod = snapshotWithDest.getClass.getMethod("execute") - // executeMethod.invoke(snapshotWithDest) - - // // Partition columns must have actual values from manifests, not NULL - // checkIcebergNativeScan(s"SELECT * FROM $destName ORDER BY partition_col") - // checkIcebergNativeScan( - // s"SELECT partition_col, data FROM $destName WHERE partition_col < 5 ORDER BY partition_col") - - // spark.sql(s"DROP TABLE $destName") - // spark.sql(s"DROP TABLE $sourceName") - // } catch { - // case _: ClassNotFoundException => - // cancel("Iceberg Actions API not available - requires iceberg-spark-runtime") - // } - // } - // } - // } - - // test("projection - column subset, reordering, and duplication") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // // Create table with multiple columns - // spark.sql(""" - // CREATE TABLE test_cat.db.proj_test ( - // id INT, - // name STRING, - // value DOUBLE, - // flag BOOLEAN - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.proj_test - // VALUES (1, 'Alice', 10.5, true), - // (2, 'Bob', 20.3, false), - // (3, 'Charlie', 30.7, true) - // """) - - // // Test 1: Column subset (only 2 of 4 columns) - // checkIcebergNativeScan("SELECT name, value FROM test_cat.db.proj_test ORDER BY id") - - // // Test 2: Reordered columns (reverse order) - // checkIcebergNativeScan("SELECT value, name, id FROM test_cat.db.proj_test ORDER BY id") - - // // Test 3: Duplicate columns - // checkIcebergNativeScan( - // "SELECT id, name, id AS id2 FROM test_cat.db.proj_test ORDER BY id") - - // // Test 4: Single column - // checkIcebergNativeScan("SELECT name FROM test_cat.db.proj_test ORDER BY name") - - // // Test 5: Different ordering with subset - // checkIcebergNativeScan("SELECT flag, id FROM test_cat.db.proj_test ORDER BY id") - - // // Test 6: Multiple duplicates - // checkIcebergNativeScan( - // "SELECT name, value, name AS name2, value AS value2 FROM test_cat.db.proj_test ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.proj_test") - // } - // } - // } - - // test("complex type - array") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.array_test ( - // id INT, - // name STRING, - // values ARRAY - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.array_test - // VALUES (1, 'Alice', array(1, 2, 3)), (2, 'Bob', array(4, 5, 6)) - // """) - - // checkIcebergNativeScan("SELECT * FROM test_cat.db.array_test ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.array_test") - // } - // } - // } - - // test("complex type - map") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.map_test ( - // id INT, - // name STRING, - // properties MAP - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.map_test - // VALUES (1, 'Alice', map('age', 30, 'score', 95)), (2, 'Bob', map('age', 25, 'score', 87)) - // """) - - // checkIcebergNativeScan("SELECT * FROM test_cat.db.map_test ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.map_test") - // } - // } - // } - - // test("complex type - struct") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.struct_test ( - // id INT, - // name STRING, - // address STRUCT - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.struct_test - // VALUES (1, 'Alice', struct('NYC', 10001)), (2, 'Bob', struct('LA', 90001)) - // """) - - // checkIcebergNativeScan("SELECT * FROM test_cat.db.struct_test ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.struct_test") - // } - // } - // } - - // test("UUID type - native Iceberg UUID column (reproduces type mismatch)") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // import org.apache.iceberg.catalog.TableIdentifier - // import org.apache.iceberg.spark.SparkCatalog - // import org.apache.iceberg.types.Types - // import org.apache.iceberg.{PartitionSpec, Schema} - - // // Use Iceberg API to create table with native UUID type - // // (not possible via Spark SQL CREATE TABLE) - // // Get Spark's catalog instance to ensure the table is visible to Spark - // val sparkCatalog = spark.sessionState.catalogManager - // .catalog("test_cat") - // .asInstanceOf[SparkCatalog] - - // spark.sql("CREATE NAMESPACE IF NOT EXISTS test_cat.db") - - // // UUID is stored as FixedSizeBinary(16) but must be presented as Utf8 to Spark - // val schema = new Schema( - // Types.NestedField.required(1, "id", Types.IntegerType.get()), - // Types.NestedField.optional(2, "uuid", Types.UUIDType.get())) - // val tableIdent = TableIdentifier.of("db", "uuid_test") - // sparkCatalog.icebergCatalog.createTable(tableIdent, schema, PartitionSpec.unpartitioned()) - - // spark.sql(""" - // INSERT INTO test_cat.db.uuid_test VALUES - // (1, 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11'), - // (2, 'b1ffcd88-8d1a-3de7-aa5c-5aa8ac269a00'), - // (3, 'c2aade77-7e0b-2cf6-99e4-4998bc158b22') - // """) - - // checkIcebergNativeScan("SELECT * FROM test_cat.db.uuid_test ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.uuid_test") - // } - // } - // } - - // test("verify all Iceberg planning metrics are populated") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // val icebergPlanningMetricNames = Seq( - // "totalPlanningDuration", - // "totalDataManifest", - // "scannedDataManifests", - // "skippedDataManifests", - // "resultDataFiles", - // "skippedDataFiles", - // "totalDataFileSize", - // "totalDeleteManifests", - // "scannedDeleteManifests", - // "skippedDeleteManifests", - // "totalDeleteFileSize", - // "resultDeleteFiles", - // "equalityDeleteFiles", - // "indexedDeleteFiles", - // "positionalDeleteFiles", - // "skippedDeleteFiles") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.metrics_test ( - // id INT, - // value DOUBLE - // ) USING iceberg - // """) - - // // Create multiple files to ensure non-zero manifest/file counts - // spark - // .range(10000) - // .selectExpr("CAST(id AS INT)", "CAST(id * 1.5 AS DOUBLE) as value") - // .coalesce(1) - // .write - // .format("iceberg") - // .mode("append") - // .saveAsTable("test_cat.db.metrics_test") - - // spark - // .range(10001, 20000) - // .selectExpr("CAST(id AS INT)", "CAST(id * 1.5 AS DOUBLE) as value") - // .coalesce(1) - // .write - // .format("iceberg") - // .mode("append") - // .saveAsTable("test_cat.db.metrics_test") - - // val df = spark.sql("SELECT * FROM test_cat.db.metrics_test WHERE id < 10000") - - // // Must extract metrics before collect() because planning happens at plan creation - // val scanNodes = df.queryExecution.executedPlan - // .collectLeaves() - // .collect { case s: CometIcebergNativeScanExec => s } - - // assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") - - // val metrics = scanNodes.head.metrics - - // icebergPlanningMetricNames.foreach { metricName => - // assert(metrics.contains(metricName), s"metric $metricName was not found") - // } - - // // Planning metrics are populated during plan creation, so they're already available - // assert(metrics("totalDataManifest").value > 0, "totalDataManifest should be > 0") - // assert(metrics("resultDataFiles").value > 0, "resultDataFiles should be > 0") - // assert(metrics("totalDataFileSize").value > 0, "totalDataFileSize should be > 0") - - // df.collect() - - // assert(metrics("output_rows").value == 10000) - // assert(metrics("num_splits").value > 0) - // assert(metrics("time_elapsed_opening").value > 0) - // assert(metrics("time_elapsed_scanning_until_data").value > 0) - // assert(metrics("time_elapsed_scanning_total").value > 0) - // assert(metrics("time_elapsed_processing").value > 0) - // // ImmutableSQLMetric prevents these from being reset to 0 after execution - // assert( - // metrics("totalDataManifest").value > 0, - // "totalDataManifest should still be > 0 after execution") - // assert( - // metrics("resultDataFiles").value > 0, - // "resultDataFiles should still be > 0 after execution") - - // spark.sql("DROP TABLE test_cat.db.metrics_test") - // } - // } - // } - - // test("verify manifest pruning metrics") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // // Partition by category to enable manifest-level pruning - // spark.sql(""" - // CREATE TABLE test_cat.db.pruning_test ( - // id INT, - // category STRING, - // value DOUBLE - // ) USING iceberg - // PARTITIONED BY (category) - // """) - - // // Each category gets its own manifest entry - // spark.sql(""" - // INSERT INTO test_cat.db.pruning_test - // SELECT id, 'A' as category, CAST(id * 1.5 AS DOUBLE) as value - // FROM range(1000) - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.pruning_test - // SELECT id, 'B' as category, CAST(id * 2.0 AS DOUBLE) as value - // FROM range(1000, 2000) - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.pruning_test - // SELECT id, 'C' as category, CAST(id * 2.5 AS DOUBLE) as value - // FROM range(2000, 3000) - // """) - - // // Filter should prune B and C partitions at manifest level - // val df = spark.sql("SELECT * FROM test_cat.db.pruning_test WHERE category = 'A'") - - // val scanNodes = df.queryExecution.executedPlan - // .collectLeaves() - // .collect { case s: CometIcebergNativeScanExec => s } - - // assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") - - // val metrics = scanNodes.head.metrics - - // // Iceberg prunes entire manifests when all files in a manifest don't match the filter - // assert( - // metrics("resultDataFiles").value == 1, - // s"Expected 1 result data file, got ${metrics("resultDataFiles").value}") - // assert( - // metrics("scannedDataManifests").value == 1, - // s"Expected 1 scanned manifest, got ${metrics("scannedDataManifests").value}") - // assert( - // metrics("skippedDataManifests").value == 2, - // s"Expected 2 skipped manifests, got ${metrics("skippedDataManifests").value}") - - // // Verify the query actually returns correct results - // val result = df.collect() - // assert(metrics("output_rows").value == 1000) - // assert(result.length == 1000, s"Expected 1000 rows, got ${result.length}") - - // spark.sql("DROP TABLE test_cat.db.pruning_test") - // } - // } - // } - - // test("verify delete file metrics - MOR table") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // // Equality delete columns force MOR behavior instead of COW - // spark.sql(""" - // CREATE TABLE test_cat.db.delete_metrics ( - // id INT, - // category STRING, - // value DOUBLE - // ) USING iceberg - // TBLPROPERTIES ( - // 'write.delete.mode' = 'merge-on-read', - // 'write.merge.mode' = 'merge-on-read', - // 'write.delete.equality-delete-columns' = 'id' - // ) - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.delete_metrics - // VALUES - // (1, 'A', 10.5), (2, 'B', 20.3), (3, 'A', 30.7), - // (4, 'B', 15.2), (5, 'A', 25.8), (6, 'C', 35.0) - // """) - - // spark.sql("DELETE FROM test_cat.db.delete_metrics WHERE id IN (2, 4, 6)") - - // val df = spark.sql("SELECT * FROM test_cat.db.delete_metrics") - - // val scanNodes = df.queryExecution.executedPlan - // .collectLeaves() - // .collect { case s: CometIcebergNativeScanExec => s } - - // assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") - - // val metrics = scanNodes.head.metrics - - // // Iceberg may convert equality deletes to positional deletes internally - // assert( - // metrics("resultDeleteFiles").value > 0, - // s"Expected result delete files > 0, got ${metrics("resultDeleteFiles").value}") - // assert( - // metrics("totalDeleteFileSize").value > 0, - // s"Expected total delete file size > 0, got ${metrics("totalDeleteFileSize").value}") - - // val hasDeletes = metrics("positionalDeleteFiles").value > 0 || - // metrics("equalityDeleteFiles").value > 0 - // assert(hasDeletes, "Expected either positional or equality delete files > 0") - - // val result = df.collect() - // assert(metrics("output_rows").value == 3) - // assert(result.length == 3, s"Expected 3 rows after deletes, got ${result.length}") - - // spark.sql("DROP TABLE test_cat.db.delete_metrics") - // } - // } - // } - - // test("verify output_rows metric reflects row-level filtering in scan") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", - // // Create relatively small files to get multiple row groups per file - // "spark.sql.files.maxRecordsPerFile" -> "1000") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.filter_metric_test ( - // id INT, - // category STRING, - // value DOUBLE - // ) USING iceberg - // """) - - // // Insert 10,000 rows with mixed category values - // // This ensures row groups will have mixed data that can't be completely eliminated - // spark.sql(""" - // INSERT INTO test_cat.db.filter_metric_test - // SELECT - // id, - // CASE WHEN id % 2 = 0 THEN 'even' ELSE 'odd' END as category, - // CAST(id * 1.5 AS DOUBLE) as value - // FROM range(10000) - // """) - - // // Apply a highly selective filter on id that will filter ~99% of rows - // // This filter requires row-level evaluation because: - // // - Row groups contain ranges of IDs (0-999, 1000-1999, etc.) - // // - The first row group (0-999) cannot be fully eliminated by stats alone - // // - Row-level filtering must apply "id < 100" to filter out rows 100-999 - // val df = spark.sql(""" - // SELECT * FROM test_cat.db.filter_metric_test - // WHERE id < 100 - // """) - - // val scanNodes = df.queryExecution.executedPlan - // .collectLeaves() - // .collect { case s: CometIcebergNativeScanExec => s } - - // assert(scanNodes.nonEmpty, "Expected at least one CometIcebergNativeScanExec node") - - // val metrics = scanNodes.head.metrics - - // // Execute the query to populate metrics - // val result = df.collect() - - // // The filter "id < 100" should match exactly 100 rows (0-99) - // assert(result.length == 100, s"Expected 100 rows after filter, got ${result.length}") - - // // CRITICAL: Verify output_rows metric matches the filtered count - // // If row-level filtering is working, this should be 100 - // // If only row group filtering is working, this would be ~1000 (entire first row group) - // assert( - // metrics("output_rows").value == 100, - // s"Expected output_rows=100 (filtered count), got ${metrics("output_rows").value}. " + - // "This indicates row-level filtering may not be working correctly.") - - // // Verify the filter actually selected the right rows - // val ids = result.map(_.getInt(0)).sorted - // assert(ids.head == 0, s"Expected first id=0, got ${ids.head}") - // assert(ids.last == 99, s"Expected last id=99, got ${ids.last}") - // assert(ids.forall(_ < 100), "All IDs should be < 100") - - // spark.sql("DROP TABLE test_cat.db.filter_metric_test") - // } - // } - // } - - // test("schema evolution - read old snapshot after column drop (VERSION AS OF)") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", - // // Force LOCAL mode to use iceberg-rust - // "spark.sql.iceberg.read.data-planning-mode" -> "local") { - - // // This test verifies that Comet correctly handles reading old snapshots after schema changes, - // // which is a form of backward schema evolution. This corresponds to these Iceberg Java tests: - // // - TestIcebergSourceHadoopTables::testSnapshotReadAfterDropColumn - // // - TestIcebergSourceHadoopTables::testSnapshotReadAfterAddAndDropColumn - // // - TestIcebergSourceHiveTables::testSnapshotReadAfterDropColumn - // // - TestIcebergSourceHiveTables::testSnapshotReadAfterAddAndDropColumn - // // - TestSnapshotSelection::testSnapshotSelectionByTagWithSchemaChange - - // // Step 1: Create table with columns (id, data, category) - // spark.sql(""" - // CREATE TABLE test_cat.db.schema_evolution_test ( - // id INT, - // data STRING, - // category STRING - // ) USING iceberg - // """) - - // // Step 2: Write data with all three columns - // spark.sql(""" - // INSERT INTO test_cat.db.schema_evolution_test - // VALUES (1, 'x', 'A'), (2, 'y', 'A'), (3, 'z', 'B') - // """) - - // // Get snapshot ID before schema change - // val snapshotIdBefore = spark - // .sql("SELECT snapshot_id FROM test_cat.db.schema_evolution_test.snapshots ORDER BY committed_at DESC LIMIT 1") - // .collect()(0) - // .getLong(0) - - // // Verify data is correct before schema change - // checkIcebergNativeScan("SELECT * FROM test_cat.db.schema_evolution_test ORDER BY id") - - // // Step 3: Drop the "data" column - // spark.sql("ALTER TABLE test_cat.db.schema_evolution_test DROP COLUMN data") - - // // Step 4: Read the old snapshot (before column was dropped) using VERSION AS OF - // // This requires using the snapshot's schema, not the current table schema - // checkIcebergNativeScan( - // s"SELECT * FROM test_cat.db.schema_evolution_test VERSION AS OF $snapshotIdBefore ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.schema_evolution_test") - // } - // } - // } - - // test("schema evolution - branch read after adding DATE column") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", - // "spark.sql.iceberg.read.data-planning-mode" -> "local") { - - // // Reproduces: TestSelect::readAndWriteWithBranchAfterSchemaChange - // // Error: "Iceberg scan error: Unexpected => unexpected target column type Date32" - // // - // // Issue: When reading old data from a branch after the table schema evolved to add - // // a DATE column, the schema adapter fails to handle Date32 type conversion. - - // // Step 1: Create table with (id, data, float_col) - // spark.sql(""" - // CREATE TABLE test_cat.db.date_branch_test ( - // id BIGINT, - // data STRING, - // float_col FLOAT - // ) USING iceberg - // """) - - // // Step 2: Insert data - // spark.sql(""" - // INSERT INTO test_cat.db.date_branch_test - // VALUES (1, 'a', 1.0), (2, 'b', 2.0), (3, 'c', CAST('NaN' AS FLOAT)) - // """) - - // // Step 3: Create a branch at this point using Iceberg API - // val catalog = spark.sessionState.catalogManager.catalog("test_cat") - // val ident = - // org.apache.spark.sql.connector.catalog.Identifier.of(Array("db"), "date_branch_test") - // val sparkTable = catalog - // .asInstanceOf[org.apache.iceberg.spark.SparkCatalog] - // .loadTable(ident) - // .asInstanceOf[org.apache.iceberg.spark.source.SparkTable] - // val table = sparkTable.table() - // val snapshotId = table.currentSnapshot().snapshotId() - // table.manageSnapshots().createBranch("test_branch", snapshotId).commit() - - // // Step 4: Evolve schema - drop float_col, add date_col - // spark.sql("ALTER TABLE test_cat.db.date_branch_test DROP COLUMN float_col") - // spark.sql("ALTER TABLE test_cat.db.date_branch_test ADD COLUMN date_col DATE") - - // // Step 5: Insert more data with the new schema - // spark.sql(""" - // INSERT INTO test_cat.db.date_branch_test - // VALUES (4, 'd', DATE '2024-04-04'), (5, 'e', DATE '2024-05-05') - // """) - - // // Step 6: Read from the branch using VERSION AS OF - // // This reads old data (id, data, float_col) but applies the current schema (id, data, date_col) - // // The old data files don't have date_col, so it should be NULL - // checkIcebergNativeScan( - // "SELECT * FROM test_cat.db.date_branch_test VERSION AS OF 'test_branch' ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.date_branch_test") - // } - // } - // } - - // // Complex type filter tests - // test("complex type filter - struct column IS NULL") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.struct_filter_test ( - // id INT, - // name STRING, - // address STRUCT - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.struct_filter_test - // VALUES - // (1, 'Alice', struct('NYC', 10001)), - // (2, 'Bob', struct('LA', 90001)), - // (3, 'Charlie', NULL) - // """) - - // // Test filtering on struct IS NULL - this should fall back to Spark - // // (iceberg-rust doesn't support IS NULL on complex type columns yet) - // checkSparkAnswer( - // "SELECT * FROM test_cat.db.struct_filter_test WHERE address IS NULL ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.struct_filter_test") - // } - // } - // } - - // test("complex type filter - struct field filter") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.struct_field_filter_test ( - // id INT, - // name STRING, - // address STRUCT - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.struct_field_filter_test - // VALUES - // (1, 'Alice', struct('NYC', 10001)), - // (2, 'Bob', struct('LA', 90001)), - // (3, 'Charlie', struct('NYC', 10002)) - // """) - - // // Test filtering on struct field - this should use native scan now! - // // iceberg-rust supports nested field filters like address.city = 'NYC' - // checkIcebergNativeScan( - // "SELECT * FROM test_cat.db.struct_field_filter_test WHERE address.city = 'NYC' ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.struct_field_filter_test") - // } - // } - // } - - // test("complex type filter - entire struct value") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.struct_value_filter_test ( - // id INT, - // name STRING, - // address STRUCT - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.struct_value_filter_test - // VALUES - // (1, 'Alice', named_struct('city', 'NYC', 'zip', 10001)), - // (2, 'Bob', named_struct('city', 'LA', 'zip', 90001)), - // (3, 'Charlie', named_struct('city', 'NYC', 'zip', 10001)) - // """) - - // // Test filtering on entire struct value - this falls back to Spark - // // (Iceberg Java doesn't push down this type of filter) - // checkSparkAnswer( - // "SELECT * FROM test_cat.db.struct_value_filter_test WHERE address = named_struct('city', 'NYC', 'zip', 10001) ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.struct_value_filter_test") - // } - // } - // } - - // test("complex type filter - array column IS NULL") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.array_filter_test ( - // id INT, - // name STRING, - // values ARRAY - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.array_filter_test - // VALUES - // (1, 'Alice', array(1, 2, 3)), - // (2, 'Bob', array(4, 5, 6)), - // (3, 'Charlie', NULL) - // """) - - // // Test filtering on array IS NULL - this should fall back to Spark - // // (iceberg-rust doesn't support IS NULL on complex type columns yet) - // checkSparkAnswer( - // "SELECT * FROM test_cat.db.array_filter_test WHERE values IS NULL ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.array_filter_test") - // } - // } - // } - - // test("complex type filter - array element filter") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.array_element_filter_test ( - // id INT, - // name STRING, - // values ARRAY - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.array_element_filter_test - // VALUES - // (1, 'Alice', array(1, 2, 3)), - // (2, 'Bob', array(4, 5, 6)), - // (3, 'Charlie', array(1, 7, 8)) - // """) - - // // Test filtering with array_contains - this should fall back to Spark - // // (Iceberg Java only pushes down NOT NULL, which fails in iceberg-rust) - // checkSparkAnswer( - // "SELECT * FROM test_cat.db.array_element_filter_test WHERE array_contains(values, 1) ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.array_element_filter_test") - // } - // } - // } - - // test("complex type filter - entire array value") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.array_value_filter_test ( - // id INT, - // name STRING, - // values ARRAY - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.array_value_filter_test - // VALUES - // (1, 'Alice', array(1, 2, 3)), - // (2, 'Bob', array(4, 5, 6)), - // (3, 'Charlie', array(1, 2, 3)) - // """) - - // // Test filtering on entire array value - this should fall back to Spark - // // (Iceberg Java only pushes down NOT NULL, which fails in iceberg-rust) - // checkSparkAnswer( - // "SELECT * FROM test_cat.db.array_value_filter_test WHERE values = array(1, 2, 3) ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.array_value_filter_test") - // } - // } - // } - - // test("complex type filter - map column IS NULL") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.map_filter_test ( - // id INT, - // name STRING, - // properties MAP - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.map_filter_test - // VALUES - // (1, 'Alice', map('age', 30, 'score', 95)), - // (2, 'Bob', map('age', 25, 'score', 87)), - // (3, 'Charlie', NULL) - // """) - - // // Test filtering on map IS NULL - this should fall back to Spark - // // (iceberg-rust doesn't support IS NULL on complex type columns yet) - // checkSparkAnswer( - // "SELECT * FROM test_cat.db.map_filter_test WHERE properties IS NULL ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.map_filter_test") - // } - // } - // } - - // test("complex type filter - map key access filter") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // spark.sql(""" - // CREATE TABLE test_cat.db.map_key_filter_test ( - // id INT, - // name STRING, - // properties MAP - // ) USING iceberg - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.map_key_filter_test - // VALUES - // (1, 'Alice', map('age', 30, 'score', 95)), - // (2, 'Bob', map('age', 25, 'score', 87)), - // (3, 'Charlie', map('age', 30, 'score', 80)) - // """) - - // // Test filtering with map key access - this should fall back to Spark - // // (Iceberg Java only pushes down NOT NULL, which fails in iceberg-rust) - // checkSparkAnswer( - // "SELECT * FROM test_cat.db.map_key_filter_test WHERE properties['age'] = 30 ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.map_key_filter_test") - // } - // } - // } - - // // Test to reproduce "Field X not found in schema" errors - // // Mimics TestAggregatePushDown.testNaN() where aggregate output schema differs from table schema - // test("partitioned table with aggregates - reproduces Field not found error") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // // Create table partitioned by id, like TestAggregatePushDown.testNaN - // spark.sql(""" - // CREATE TABLE test_cat.db.agg_test ( - // id INT, - // data FLOAT - // ) USING iceberg - // PARTITIONED BY (id) - // """) - - // spark.sql(""" - // INSERT INTO test_cat.db.agg_test VALUES - // (1, CAST('NaN' AS FLOAT)), - // (1, CAST('NaN' AS FLOAT)), - // (2, 2.0), - // (2, CAST('NaN' AS FLOAT)), - // (3, CAST('NaN' AS FLOAT)), - // (3, 1.0) - // """) - - // // This aggregate query's output schema is completely different from table schema - // // When iceberg-rust tries to look up partition field 'id' (field 1 in table schema), - // // it needs to find it in the full table schema, not the aggregate output schema - // checkIcebergNativeScan( - // "SELECT count(*), max(data), min(data), count(data) FROM test_cat.db.agg_test") - - // spark.sql("DROP TABLE test_cat.db.agg_test") - // } - // } - // } - - // test("MOR partitioned table with timestamp_ntz - reproduces NULL partition issue") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { - - // // Create partitioned table like TestRewritePositionDeleteFiles.testTimestampNtz - // spark.sql(""" - // CREATE TABLE test_cat.db.timestamp_ntz_partition_test ( - // id LONG, - // ts TIMESTAMP_NTZ, - // c1 STRING, - // c2 STRING - // ) USING iceberg - // PARTITIONED BY (ts) - // TBLPROPERTIES ( - // 'format-version' = '2', - // 'write.delete.mode' = 'merge-on-read', - // 'write.merge.mode' = 'merge-on-read' - // ) - // """) - - // // Insert data into multiple partitions - // spark.sql(""" - // INSERT INTO test_cat.db.timestamp_ntz_partition_test - // VALUES - // (1, TIMESTAMP_NTZ '2023-01-01 15:30:00', 'a', 'b'), - // (2, TIMESTAMP_NTZ '2023-01-02 15:30:00', 'c', 'd'), - // (3, TIMESTAMP_NTZ '2023-01-03 15:30:00', 'e', 'f') - // """) - - // // Delete some rows to create position delete files - // spark.sql("DELETE FROM test_cat.db.timestamp_ntz_partition_test WHERE id = 2") - - // // Query should work with NULL partition handling - // checkIcebergNativeScan( - // "SELECT * FROM test_cat.db.timestamp_ntz_partition_test ORDER BY id") - - // spark.sql("DROP TABLE test_cat.db.timestamp_ntz_partition_test") - // } - // } - // } - - // test("MOR partitioned table with decimal - reproduces NULL partition issue") { - // assume(icebergAvailable, "Iceberg not available in classpath") - - // withTempIcebergDir { warehouseDir => - // withSQLConf( - // "spark.sql.catalog.test_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.test_cat.type" -> "hadoop", - // "spark.sql.catalog.test_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true") { + spark.sql("DROP TABLE test_table") + } + } - // // Create partitioned table like TestRewritePositionDeleteFiles.testDecimalPartition - // spark.sql(""" - // CREATE TABLE test_cat.db.decimal_partition_test ( - // id LONG, - // dec DECIMAL(18, 10), - // c1 STRING, - // c2 STRING - // ) USING iceberg - // PARTITIONED BY (dec) - // TBLPROPERTIES ( - // 'format-version' = '2', - // 'write.delete.mode' = 'merge-on-read', - // 'write.merge.mode' = 'merge-on-read' - // ) - // """) + test("deletion vectors not supported") { + import testImplicits._ + + withTempDir { dir => + DeltaTable + .create(spark) + .tableName("test_table") + .addColumn("id", "INT") + .addColumn("name", "STRING") + .addColumn("value", "DOUBLE") + .property("delta.enableDeletionVectors", "true") + .location(dir.getAbsolutePath) + .execute() + + Seq((1, "Alice", 10.5), (2, "Bob", 20.3), (3, "Charlie", 30.7)) + .toDF("id", "name", "value") + .write + .format("delta") + .mode("append") + .save(dir.getAbsolutePath) + + checkNativeScan("SELECT * FROM test_table ORDER BY id", false) - // // Insert data into multiple partitions - // spark.sql(""" - // INSERT INTO test_cat.db.decimal_partition_test - // VALUES - // (1, 1.0, 'a', 'b'), - // (2, 2.0, 'c', 'd'), - // (3, 3.0, 'e', 'f') - // """) + spark.sql("DROP TABLE test_table") + } + } - // // Delete some rows to create position delete files - // spark.sql("DELETE FROM test_cat.db.decimal_partition_test WHERE id = 2") + test("column mapping not supported") { + import testImplicits._ - // // Query should work with NULL partition handling - // checkIcebergNativeScan("SELECT * FROM test_cat.db.decimal_partition_test ORDER BY id") + withTempDir { dir => + // Creating a table just with column mapping results in reader version 2 + val table = DeltaTable + .create(spark) + .tableName("test_table") + .addColumn("id", "INT") + .addColumn("name", "STRING") + .addColumn("value", "DOUBLE") + .property("delta.columnMapping.mode", "name") + .location(dir.getAbsolutePath) + .execute() - // spark.sql("DROP TABLE test_cat.db.decimal_partition_test") - // } - // } - // } + assert(table.detail().select("minReaderVersion").first().getInt(0) == 2) - // test("REST catalog with native Iceberg scan") { - // assume(icebergAvailable, "Iceberg not available in classpath") + Seq((1, "Alice", 10.5), (2, "Bob", 20.3), (3, "Charlie", 30.7)) + .toDF("id", "name", "value") + .write + .format("delta") + .mode("append") + .save(dir.getAbsolutePath) - // withRESTCatalog { (restUri, _, warehouseDir) => - // withSQLConf( - // "spark.sql.catalog.rest_cat" -> "org.apache.iceberg.spark.SparkCatalog", - // "spark.sql.catalog.rest_cat.catalog-impl" -> "org.apache.iceberg.rest.RESTCatalog", - // "spark.sql.catalog.rest_cat.uri" -> restUri, - // "spark.sql.catalog.rest_cat.warehouse" -> warehouseDir.getAbsolutePath, - // CometConf.COMET_ENABLED.key -> "true", - // CometConf.COMET_EXEC_ENABLED.key -> "true", - // CometConf.COMET_ICEBERG_NATIVE_ENABLED.key -> "true", - // CometConf.COMET_EXPLAIN_FALLBACK_ENABLED.key -> "true") { + checkNativeScan("SELECT * FROM test_table ORDER BY id", false) - // // Create namespace first (REST catalog requires explicit namespace creation) - // spark.sql("CREATE NAMESPACE rest_cat.db") + // Now add a new feature that requires enable reader features + table.addFeatureSupport("v2Checkpoint") - // // Create a table via REST catalog - // spark.sql(""" - // CREATE TABLE rest_cat.db.test_table ( - // id INT, - // name STRING, - // value DOUBLE - // ) USING iceberg - // """) + assert(table.detail().select("minReaderVersion").first().getInt(0) == 3) - // spark.sql(""" - // INSERT INTO rest_cat.db.test_table - // VALUES (1, 'Alice', 10.5), (2, 'Bob', 20.3), (3, 'Charlie', 30.7) - // """) + checkNativeScan("SELECT * FROM test_table ORDER BY id", false) - // checkIcebergNativeScan("SELECT * FROM rest_cat.db.test_table ORDER BY id") + spark.sql("DROP TABLE test_table") + } + } - // spark.sql("DROP TABLE rest_cat.db.test_table") - // spark.sql("DROP NAMESPACE rest_cat.db") - // } - // } - // } + test("complex Delta table") { + import testImplicits._ + + withTempDir { dir => + val table = DeltaTable + .create(spark) + .tableName("test_table") + .addColumn("nested", "struct") + .addColumn("arr", "array>") + .location(dir.getAbsolutePath) + .execute() + + spark + .createDataFrame( + List( + Row((1, 10.5), Seq((1, 10.5), (2, 15.0))), + Row((2, 20.3), Seq((2, 20.3), (3, 25.5)))).asJava, + table.toDF.schema) + .write + .format("delta") + .mode("append") + .save(dir.getAbsolutePath) + + checkNativeScan("SELECT * FROM test_table ORDER BY nested.id", true) - // Helper to create temp directory - def withTempDeltaDir(f: File => Unit): Unit = { - val dir = Files.createTempDirectory("comet-delta-test").toFile - try { - f(dir) - } finally { - def deleteRecursively(file: File): Unit = { - if (file.isDirectory) { - file.listFiles().foreach(deleteRecursively) - } - file.delete() - } - deleteRecursively(dir) + spark.sql("DROP TABLE test_table") } } } From 0a995ef2edb40ff6820723f70e8e46b85eb4e198 Mon Sep 17 00:00:00 2001 From: Adam Binford Date: Sat, 3 Jan 2026 20:31:14 -0500 Subject: [PATCH 3/9] Scalafix --- .../test/scala/org/apache/comet/CometDeltaNativeSuite.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala index 4475896854..a67aa8c7ad 100644 --- a/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala @@ -19,9 +19,6 @@ package org.apache.comet -import java.io.File -import java.nio.file.Files - import scala.jdk.CollectionConverters._ import org.apache.spark.SparkConf @@ -188,7 +185,6 @@ class CometDeltaNativeSuite extends CometTestBase { } test("complex Delta table") { - import testImplicits._ withTempDir { dir => val table = DeltaTable From 86ff296d9d6c59083f0f92b4d0ac43ce999b8b3d Mon Sep 17 00:00:00 2001 From: Adam Binford Date: Sat, 3 Jan 2026 21:59:51 -0500 Subject: [PATCH 4/9] Support Spark 3.4 --- spark/pom.xml | 7 + .../apache/comet/delta/DeltaReflection.scala | 58 ++--- .../spark/sql/comet/CometScanExec.scala | 3 +- .../apache/comet/CometDeltaNativeSuite.scala | 223 +++++++++--------- 4 files changed, 134 insertions(+), 157 deletions(-) diff --git a/spark/pom.xml b/spark/pom.xml index 23a5675423..793b37920a 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -116,6 +116,7 @@ under the License. com.google.guava failureaccess 1.0.3 + test org.codehaus.jackson @@ -193,6 +194,12 @@ under the License. 1.5.2 test + + io.delta + delta-core_${scala.binary.version} + 2.4.0 + test + org.eclipse.jetty diff --git a/spark/src/main/scala/org/apache/comet/delta/DeltaReflection.scala b/spark/src/main/scala/org/apache/comet/delta/DeltaReflection.scala index fed617b1dd..ee1eeca4ea 100644 --- a/spark/src/main/scala/org/apache/comet/delta/DeltaReflection.scala +++ b/spark/src/main/scala/org/apache/comet/delta/DeltaReflection.scala @@ -29,26 +29,6 @@ import org.apache.spark.internal.Logging */ object DeltaReflection extends Logging { - /** - * Iceberg class names used throughout Comet. - */ - object ClassNames {} - - /** - * Iceberg content types. - */ - object ContentTypes { - val POSITION_DELETES = "POSITION_DELETES" - val EQUALITY_DELETES = "EQUALITY_DELETES" - } - - /** - * Iceberg file formats. - */ - object FileFormats { - val PARQUET = "PARQUET" - } - /** * Gets the Delta Protocol from a Spark FileFormat */ @@ -72,8 +52,9 @@ object DeltaReflection extends Logging { */ def getMinReaderVersion(protocol: Any): Option[Int] = { try { - val method = protocol.getClass.getDeclaredMethod("getMinReaderVersion") - Some(method.invoke(protocol).asInstanceOf[Int]) + val field = protocol.getClass.getDeclaredField("minReaderVersion") + field.setAccessible(true) + Some(field.get(protocol).asInstanceOf[Int]) } catch { case e: Exception => logError( @@ -83,10 +64,11 @@ object DeltaReflection extends Logging { } } - def getReaderFeatures(protocol: Any): Option[java.util.Set[String]] = { + def getReaderFeatures(protocol: Any): Option[Set[String]] = { try { - val method = protocol.getClass.getDeclaredMethod("getReaderFeatures") - Some(method.invoke(protocol).asInstanceOf[java.util.Set[String]]) + val field = protocol.getClass.getDeclaredField("readerFeatures") + field.setAccessible(true) + Some(field.get(protocol).asInstanceOf[Option[Set[String]]].getOrElse(Set.empty)) } catch { case e: Exception => logError( @@ -98,43 +80,31 @@ object DeltaReflection extends Logging { } /** - * Pre-extracted Iceberg metadata for native scan execution. + * Pre-extracted Delta metadata for native scan execution. * - * This class holds all metadata extracted from Iceberg during the planning/validation phase in + * This class holds all metadata extracted from Delta during the planning/validation phase in * CometScanRule. By extracting all metadata once during validation (where reflection failures * trigger fallback to Spark), we avoid redundant reflection during serialization (where failures * would be fatal runtime errors). * * @param minReaderVersion * The minimum reader version of the table - * @param minWriterVersion - * The minimum writer version of the table * @param readerFeatures * A list of enabled reader features on the table - * @param writerFeatures - * A list of enabled writer features on the table */ -case class CometDeltaNativeScanMetadata( - minReaderVersion: Int, - minWriterVersion: Int, - readerFeatures: java.util.Set[String], - writerFeatures: java.util.Set[String]) +case class CometDeltaNativeScanMetadata(minReaderVersion: Int, readerFeatures: Set[String]) object CometDeltaNativeScanMetadata extends Logging { import DeltaReflection._ /** - * Extracts all Iceberg metadata needed for native scan execution. + * Extracts all Delta metadata needed for native scan execution. * * This method performs all reflection operations once during planning/validation. If any * reflection operation fails, returns None to trigger fallback to Spark. * - * @param scan - * The Spark BatchScanExec.scan (SparkBatchQueryScan) - * @param metadataLocation - * Path to the table metadata file (already extracted) - * @param catalogProperties - * Catalog properties for FileIO (already extracted) + * @param fileFormat + * The FileFormat instance from the HadoopFsRelation * @return * Some(metadata) if all reflection succeeds, None to trigger fallback */ @@ -144,7 +114,7 @@ object CometDeltaNativeScanMetadata extends Logging { minReaderVersion <- getMinReaderVersion(protocol) readerFeatures <- getReaderFeatures(protocol) } yield { - CometDeltaNativeScanMetadata(minReaderVersion, 0, readerFeatures, java.util.Set.of()) + CometDeltaNativeScanMetadata(minReaderVersion, readerFeatures) } } } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala index 52ce7da65e..df30fd6874 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala @@ -21,7 +21,6 @@ package org.apache.spark.sql.comet import scala.collection.mutable.HashMap import scala.concurrent.duration.NANOSECONDS -import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag import org.apache.hadoop.fs.Path @@ -610,7 +609,7 @@ object CometScanExec { case 1 => true case 2 => false case 3 => - deltaMetadata.readerFeatures.asScala.intersect(unsupportedDeltaReaderFeatures).isEmpty + deltaMetadata.readerFeatures.intersect(unsupportedDeltaReaderFeatures).isEmpty } } diff --git a/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala index a67aa8c7ad..2a377a23bf 100644 --- a/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala @@ -71,25 +71,25 @@ class CometDeltaNativeSuite extends CometTestBase { import testImplicits._ withTempDir { dir => - DeltaTable - .create(spark) - .tableName("test_table") - .addColumn("id", "INT") - .addColumn("name", "STRING") - .addColumn("value", "DOUBLE") - .location(dir.getAbsolutePath) - .execute() - - Seq((1, "Alice", 10.5), (2, "Bob", 20.3), (3, "Charlie", 30.7)) - .toDF("id", "name", "value") - .write - .format("delta") - .mode("append") - .save(dir.getAbsolutePath) - - checkNativeScan("SELECT * FROM test_table ORDER BY id", true) - - spark.sql("DROP TABLE test_table") + withTable("test_table") { + DeltaTable + .create(spark) + .tableName("test_table") + .addColumn("id", "INT") + .addColumn("name", "STRING") + .addColumn("value", "DOUBLE") + .location(dir.getAbsolutePath) + .execute() + + Seq((1, "Alice", 10.5), (2, "Bob", 20.3), (3, "Charlie", 30.7)) + .toDF("id", "name", "value") + .write + .format("delta") + .mode("append") + .save(dir.getAbsolutePath) + + checkNativeScan("SELECT * FROM test_table ORDER BY id", true) + } } } @@ -97,26 +97,27 @@ class CometDeltaNativeSuite extends CometTestBase { import testImplicits._ withTempDir { dir => - DeltaTable - .create(spark) - .tableName("test_table") - .addColumn("id", "INT") - .addColumn("name", "STRING") - .addColumn("value", "DOUBLE") - .property("delta.checkpointPolicy", "v2") - .location(dir.getAbsolutePath) - .execute() - - Seq((1, "Alice", 10.5), (2, "Bob", 20.3), (3, "Charlie", 30.7)) - .toDF("id", "name", "value") - .write - .format("delta") - .mode("append") - .save(dir.getAbsolutePath) - - checkNativeScan("SELECT * FROM test_table ORDER BY id", true) - - spark.sql("DROP TABLE test_table") + withTable("test_table") { + val table = DeltaTable + .create(spark) + .tableName("test_table") + .addColumn("id", "INT") + .addColumn("name", "STRING") + .addColumn("value", "DOUBLE") + .location(dir.getAbsolutePath) + .execute() + + table.addFeatureSupport("timestampNtz") + + Seq((1, "Alice", 10.5), (2, "Bob", 20.3), (3, "Charlie", 30.7)) + .toDF("id", "name", "value") + .write + .format("delta") + .mode("append") + .save(dir.getAbsolutePath) + + checkNativeScan("SELECT * FROM test_table ORDER BY id", true) + } } } @@ -124,26 +125,26 @@ class CometDeltaNativeSuite extends CometTestBase { import testImplicits._ withTempDir { dir => - DeltaTable - .create(spark) - .tableName("test_table") - .addColumn("id", "INT") - .addColumn("name", "STRING") - .addColumn("value", "DOUBLE") - .property("delta.enableDeletionVectors", "true") - .location(dir.getAbsolutePath) - .execute() - - Seq((1, "Alice", 10.5), (2, "Bob", 20.3), (3, "Charlie", 30.7)) - .toDF("id", "name", "value") - .write - .format("delta") - .mode("append") - .save(dir.getAbsolutePath) - - checkNativeScan("SELECT * FROM test_table ORDER BY id", false) - - spark.sql("DROP TABLE test_table") + withTable("test_table") { + DeltaTable + .create(spark) + .tableName("test_table") + .addColumn("id", "INT") + .addColumn("name", "STRING") + .addColumn("value", "DOUBLE") + .property("delta.enableDeletionVectors", "true") + .location(dir.getAbsolutePath) + .execute() + + Seq((1, "Alice", 10.5), (2, "Bob", 20.3), (3, "Charlie", 30.7)) + .toDF("id", "name", "value") + .write + .format("delta") + .mode("append") + .save(dir.getAbsolutePath) + + checkNativeScan("SELECT * FROM test_table ORDER BY id", false) + } } } @@ -151,64 +152,64 @@ class CometDeltaNativeSuite extends CometTestBase { import testImplicits._ withTempDir { dir => - // Creating a table just with column mapping results in reader version 2 - val table = DeltaTable - .create(spark) - .tableName("test_table") - .addColumn("id", "INT") - .addColumn("name", "STRING") - .addColumn("value", "DOUBLE") - .property("delta.columnMapping.mode", "name") - .location(dir.getAbsolutePath) - .execute() - - assert(table.detail().select("minReaderVersion").first().getInt(0) == 2) - - Seq((1, "Alice", 10.5), (2, "Bob", 20.3), (3, "Charlie", 30.7)) - .toDF("id", "name", "value") - .write - .format("delta") - .mode("append") - .save(dir.getAbsolutePath) - - checkNativeScan("SELECT * FROM test_table ORDER BY id", false) - - // Now add a new feature that requires enable reader features - table.addFeatureSupport("v2Checkpoint") - - assert(table.detail().select("minReaderVersion").first().getInt(0) == 3) - - checkNativeScan("SELECT * FROM test_table ORDER BY id", false) - - spark.sql("DROP TABLE test_table") + withTable("test_table") { + // Creating a table just with column mapping results in reader version 2 + val table = DeltaTable + .create(spark) + .tableName("test_table") + .addColumn("id", "INT") + .addColumn("name", "STRING") + .addColumn("value", "DOUBLE") + .property("delta.columnMapping.mode", "name") + .location(dir.getAbsolutePath) + .execute() + + assert(table.detail().select("minReaderVersion").first().getInt(0) == 2) + + Seq((1, "Alice", 10.5), (2, "Bob", 20.3), (3, "Charlie", 30.7)) + .toDF("id", "name", "value") + .write + .format("delta") + .mode("append") + .save(dir.getAbsolutePath) + + checkNativeScan("SELECT * FROM test_table ORDER BY id", false) + + // Now upgrade to reader version 3 which uses the reader features + table.upgradeTableProtocol(3, 7) + + assert(table.detail().select("minReaderVersion").first().getInt(0) == 3) + + checkNativeScan("SELECT * FROM test_table ORDER BY id", false) + } } } test("complex Delta table") { withTempDir { dir => - val table = DeltaTable - .create(spark) - .tableName("test_table") - .addColumn("nested", "struct") - .addColumn("arr", "array>") - .location(dir.getAbsolutePath) - .execute() - - spark - .createDataFrame( - List( - Row((1, 10.5), Seq((1, 10.5), (2, 15.0))), - Row((2, 20.3), Seq((2, 20.3), (3, 25.5)))).asJava, - table.toDF.schema) - .write - .format("delta") - .mode("append") - .save(dir.getAbsolutePath) - - checkNativeScan("SELECT * FROM test_table ORDER BY nested.id", true) - - spark.sql("DROP TABLE test_table") + withTable("test_table") { + val table = DeltaTable + .create(spark) + .tableName("test_table") + .addColumn("nested", "struct") + .addColumn("arr", "array>") + .location(dir.getAbsolutePath) + .execute() + + spark + .createDataFrame( + List( + Row((1, 10.5), Seq((1, 10.5), (2, 15.0))), + Row((2, 20.3), Seq((2, 20.3), (3, 25.5)))).asJava, + table.toDF.schema) + .write + .format("delta") + .mode("append") + .save(dir.getAbsolutePath) + + checkNativeScan("SELECT * FROM test_table ORDER BY nested.id", true) + } } } } From a31f3a0d6e76932a0bef9b963c58a38bc0376fb6 Mon Sep 17 00:00:00 2001 From: Adam Binford Date: Sat, 3 Jan 2026 22:47:22 -0500 Subject: [PATCH 5/9] Remove unnecessary dependency and fix tests for Spark 4 --- native/Cargo.lock | 1 - native/core/Cargo.toml | 3 --- .../org/apache/comet/CometDeltaNativeSuite.scala | 13 ++----------- 3 files changed, 2 insertions(+), 15 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index dd908ce467..6eacfe258d 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -1793,7 +1793,6 @@ dependencies = [ "datafusion-spark", "futures", "hdfs-sys", - "hdrs", "hex", "iceberg", "itertools 0.14.0", diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index b181dcf971..9fa5dde1d9 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -85,9 +85,6 @@ uuid = "1.18.1" [target.'cfg(target_os = "linux")'.dependencies] procfs = "0.18.0" -[target.'cfg(target_os = "macos")'.dependencies] -hdrs = { version = "0.3.2", features = ["vendored"] } - [dev-dependencies] pprof = { version = "0.15", features = ["flamegraph"] } criterion = { version = "0.7", features = ["async", "async_tokio", "async_std"] } diff --git a/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala index 2a377a23bf..8e80e013cf 100644 --- a/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala @@ -38,6 +38,7 @@ class CometDeltaNativeSuite extends CometTestBase { super.sparkConf .set("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") .set("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog") + .set("spark.databricks.delta.snapshotPartitions", "2") .set(CometConf.COMET_NATIVE_SCAN_IMPL.key, "native_datafusion") } @@ -153,8 +154,7 @@ class CometDeltaNativeSuite extends CometTestBase { withTempDir { dir => withTable("test_table") { - // Creating a table just with column mapping results in reader version 2 - val table = DeltaTable + DeltaTable .create(spark) .tableName("test_table") .addColumn("id", "INT") @@ -164,8 +164,6 @@ class CometDeltaNativeSuite extends CometTestBase { .location(dir.getAbsolutePath) .execute() - assert(table.detail().select("minReaderVersion").first().getInt(0) == 2) - Seq((1, "Alice", 10.5), (2, "Bob", 20.3), (3, "Charlie", 30.7)) .toDF("id", "name", "value") .write @@ -174,13 +172,6 @@ class CometDeltaNativeSuite extends CometTestBase { .save(dir.getAbsolutePath) checkNativeScan("SELECT * FROM test_table ORDER BY id", false) - - // Now upgrade to reader version 3 which uses the reader features - table.upgradeTableProtocol(3, 7) - - assert(table.detail().select("minReaderVersion").first().getInt(0) == 3) - - checkNativeScan("SELECT * FROM test_table ORDER BY id", false) } } } From b0bbba824c476b6f514c9ddc9bff86dd51a4ee65 Mon Sep 17 00:00:00 2001 From: Adam Binford Date: Sun, 4 Jan 2026 08:46:40 -0500 Subject: [PATCH 6/9] Add test for CDF --- .../apache/comet/CometDeltaNativeSuite.scala | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala b/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala index 8e80e013cf..b1923b1bf1 100644 --- a/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometDeltaNativeSuite.scala @@ -176,6 +176,38 @@ class CometDeltaNativeSuite extends CometTestBase { } } + test("change data feed enabled") { + import testImplicits._ + + withTempDir { dir => + withTable("test_table") { + DeltaTable + .create(spark) + .tableName("test_table") + .addColumn("id", "INT") + .addColumn("name", "STRING") + .addColumn("value", "DOUBLE") + .property("delta.enableChangeDataFeed", "true") + .location(dir.getAbsolutePath) + .execute() + + Seq((1, "Alice", 10.5), (2, "Bob", 20.3), (3, "Charlie", 30.7)) + .toDF("id", "name", "value") + .write + .format("delta") + .mode("append") + .save(dir.getAbsolutePath) + + // Reading from a table with change data feed enabled is supported if not reading change + // feed + checkNativeScan("SELECT * FROM test_table ORDER BY id", true) + + // Read a change feed uses a different relation that is not supported + checkNativeScan("SELECT * FROM table_changes('test_table', 0)", false) + } + } + } + test("complex Delta table") { withTempDir { dir => From 2076f1e69fd0e237fa0b02e2f8c04bc8512742df Mon Sep 17 00:00:00 2001 From: Adam Binford Date: Sun, 4 Jan 2026 09:05:19 -0500 Subject: [PATCH 7/9] Add new suite to CI --- .github/workflows/pr_build_linux.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/pr_build_linux.yml b/.github/workflows/pr_build_linux.yml index e3b0e40566..712f6900c0 100644 --- a/.github/workflows/pr_build_linux.yml +++ b/.github/workflows/pr_build_linux.yml @@ -127,6 +127,7 @@ jobs: org.apache.spark.sql.comet.ParquetEncryptionITCase org.apache.comet.exec.CometNativeReaderSuite org.apache.comet.CometIcebergNativeSuite + org.apache.comet.CometDeltaNativeSuite - name: "exec" value: | org.apache.comet.exec.CometAggregateSuite From ef51e5e72ccc5ad6b86bfac12836e07bf75d1b28 Mon Sep 17 00:00:00 2001 From: Adam Binford Date: Wed, 7 Jan 2026 16:27:57 +0000 Subject: [PATCH 8/9] Add a snippet to supported sources --- docs/source/user-guide/latest/datasources.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/source/user-guide/latest/datasources.md b/docs/source/user-guide/latest/datasources.md index cfce4d13d4..4cb7d171a9 100644 --- a/docs/source/user-guide/latest/datasources.md +++ b/docs/source/user-guide/latest/datasources.md @@ -34,6 +34,9 @@ Comet accelerates Iceberg scans of Parquet files. See the [Iceberg Guide] for mo [Iceberg Guide]: iceberg.md +### Delta Lake +Comet can perform native scans of Delta tables when they do not have column mapping or deletion vectors enabled. Scans of Delta tables without these features are simply regular Spark parquet scans, so they follow the same behavior as the `parquet` format behavior described above. Scans of tables using these features may be supported in the future. + ### CSV Comet does not provide native CSV scan, but when `spark.comet.convert.csv.enabled` is enabled, data is immediately From aa51b22867c02e47d46ea48bf5637cba65068dfd Mon Sep 17 00:00:00 2001 From: Adam Binford Date: Wed, 7 Jan 2026 16:28:43 +0000 Subject: [PATCH 9/9] Fix md --- docs/source/user-guide/latest/datasources.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/source/user-guide/latest/datasources.md b/docs/source/user-guide/latest/datasources.md index 4cb7d171a9..a29aac8a9b 100644 --- a/docs/source/user-guide/latest/datasources.md +++ b/docs/source/user-guide/latest/datasources.md @@ -35,6 +35,7 @@ Comet accelerates Iceberg scans of Parquet files. See the [Iceberg Guide] for mo [Iceberg Guide]: iceberg.md ### Delta Lake + Comet can perform native scans of Delta tables when they do not have column mapping or deletion vectors enabled. Scans of Delta tables without these features are simply regular Spark parquet scans, so they follow the same behavior as the `parquet` format behavior described above. Scans of tables using these features may be supported in the future. ### CSV