From 1c19b516c8bbde21013d29e1cdc6dbaa15fc44ea Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Wed, 26 Nov 2025 20:14:57 +0400 Subject: [PATCH 01/23] Work --- .../csv/CometCsvPartitionReaderFactory.scala | 5 +++++ .../org/apache/comet/csv/CometCsvScan.scala | 5 +++++ .../apache/comet/rules/CometScanRule.scala | 9 ++++---- .../serde/operator/CometCsvNativeScan.scala | 21 +++++++++++++++++++ .../sql/comet/CometCsvNativeScanExec.scala | 3 +++ 5 files changed, 39 insertions(+), 4 deletions(-) create mode 100644 spark/src/main/scala/org/apache/comet/csv/CometCsvPartitionReaderFactory.scala create mode 100644 spark/src/main/scala/org/apache/comet/csv/CometCsvScan.scala create mode 100644 spark/src/main/scala/org/apache/comet/serde/operator/CometCsvNativeScan.scala create mode 100644 spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala diff --git a/spark/src/main/scala/org/apache/comet/csv/CometCsvPartitionReaderFactory.scala b/spark/src/main/scala/org/apache/comet/csv/CometCsvPartitionReaderFactory.scala new file mode 100644 index 0000000000..d908925a28 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/csv/CometCsvPartitionReaderFactory.scala @@ -0,0 +1,5 @@ +package org.apache.comet.csv + +class CometCsvPartitionReaderFactory { + +} diff --git a/spark/src/main/scala/org/apache/comet/csv/CometCsvScan.scala b/spark/src/main/scala/org/apache/comet/csv/CometCsvScan.scala new file mode 100644 index 0000000000..3d7f97a0ef --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/csv/CometCsvScan.scala @@ -0,0 +1,5 @@ +package org.apache.comet.csv + +class CometCsvScan { + +} 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 ad86471766..1f8d3b0fa9 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -20,17 +20,15 @@ package org.apache.comet.rules import java.net.URI - import scala.collection.mutable import scala.collection.mutable.ListBuffer import scala.jdk.CollectionConverters._ - import org.apache.hadoop.conf.Configuration import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, GenericInternalRow, PlanExpression} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.util.{sideBySide, ArrayBasedMapData, GenericArrayData, MetadataColumnHelper} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MetadataColumnHelper, sideBySide} import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.getExistenceDefaultValues import org.apache.spark.sql.comet.{CometBatchScanExec, CometScanExec} import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} @@ -39,7 +37,6 @@ import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ - import org.apache.comet.{CometConf, CometNativeException, DataTypeSupport} import org.apache.comet.CometConf._ import org.apache.comet.CometSparkSessionExtensions.{isCometLoaded, isCometScanEnabled, withInfo, withInfos} @@ -49,6 +46,7 @@ import org.apache.comet.objectstore.NativeConfig import org.apache.comet.parquet.{CometParquetScan, Native, SupportsComet} import org.apache.comet.parquet.CometParquetUtils.{encryptionEnabled, isEncryptionConfigSupported} import org.apache.comet.shims.CometTypeShim +import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan /** * Spark physical optimizer rule for replacing Spark scans with Comet scans. @@ -266,6 +264,9 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com withInfos(scanExec, fallbackReasons.toSet) } + case scan: CSVScan => + CometBatchScanExec(scanExec, Seq.empty) + // Iceberg scan - patched version implementing SupportsComet interface case s: SupportsComet if !COMET_ICEBERG_NATIVE_ENABLED.get() => val fallbackReasons = new ListBuffer[String]() diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometCsvNativeScan.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometCsvNativeScan.scala new file mode 100644 index 0000000000..9028390502 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometCsvNativeScan.scala @@ -0,0 +1,21 @@ +package org.apache.comet.serde.operator + +import org.apache.spark.sql.comet.{CometBatchScanExec, CometNativeExec} + +import org.apache.comet.ConfigEntry +import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} +import org.apache.comet.serde.OperatorOuterClass.Operator + +object CometCsvNativeScan extends CometOperatorSerde[CometBatchScanExec] { + + override def enabledConfig: Option[ConfigEntry[Boolean]] = ??? + + override def convert( + op: CometBatchScanExec, + builder: Operator.Builder, + childOp: OperatorOuterClass.Operator*): Option[OperatorOuterClass.Operator] = ??? + + override def createExec( + nativeOp: OperatorOuterClass.Operator, + op: CometBatchScanExec): CometNativeExec = ??? +} diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala new file mode 100644 index 0000000000..47014c8edf --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala @@ -0,0 +1,3 @@ +package org.apache.spark.sql.comet + +case class CometCsvNativeScanExec() From 0d9355ffa530162543db2749c156861839b95346 Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Fri, 5 Dec 2025 20:08:33 +0400 Subject: [PATCH 02/23] Work --- .../csv/CometCsvPartitionReaderFactory.scala | 64 ++++++++++++++++++- .../org/apache/comet/csv/CometCsvScan.scala | 36 ++++++++++- .../apache/comet/rules/CometScanRule.scala | 1 + .../sql/comet/CometCsvNativeScanExec.scala | 3 - 4 files changed, 99 insertions(+), 5 deletions(-) delete mode 100644 spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala diff --git a/spark/src/main/scala/org/apache/comet/csv/CometCsvPartitionReaderFactory.scala b/spark/src/main/scala/org/apache/comet/csv/CometCsvPartitionReaderFactory.scala index d908925a28..dfeb38413b 100644 --- a/spark/src/main/scala/org/apache/comet/csv/CometCsvPartitionReaderFactory.scala +++ b/spark/src/main/scala/org/apache/comet/csv/CometCsvPartitionReaderFactory.scala @@ -1,5 +1,67 @@ package org.apache.comet.csv -class CometCsvPartitionReaderFactory { +import org.apache.spark.TaskContext +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.csv.CSVOptions +import org.apache.spark.sql.comet.execution.arrow.CometArrowConverters +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader} +import org.apache.spark.sql.execution.datasources.{FilePartition, HadoopFileLinesReader, PartitionedFile} +import org.apache.spark.sql.execution.datasources.v2.FilePartitionReaderFactory +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{ArrayType, BinaryType, ByteType, StructType} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.SerializableConfiguration +import org.apache.comet.CometConf + +case class CometCsvPartitionReaderFactory( + sqlConf: SQLConf, + options: CSVOptions, + broadcastedConf: Broadcast[SerializableConfiguration]) + extends FilePartitionReaderFactory { + + private val schema = + new StructType().add("value", ArrayType(elementType = ByteType), nullable = false) + + override def createColumnarReader(partition: InputPartition): PartitionReader[ColumnarBatch] = { + val filePartition = partition.asInstanceOf[FilePartition] + filePartition.files.map { file => + file.filePath.toPath + } + null + } + + override def supportColumnarReads(partition: InputPartition): Boolean = true + + override def buildReader(partitionedFile: PartitionedFile): PartitionReader[InternalRow] = + throw new UnsupportedOperationException("Comet doesn't support 'buildReader'") + + private def buildCometCsvPartitionReader(file: PartitionedFile): CometCsvPartitionReader = { + val conf = broadcastedConf.value.value + val lines = { + val linesReader = new HadoopFileLinesReader(file, options.lineSeparatorInRead, conf) + Option(TaskContext.get()) + .foreach(_.addTaskCompletionListener[Unit](_ => linesReader.close())) + linesReader.map(line => InternalRow.apply(line.getBytes)) + } + val maxRecordsPerBatch = CometConf.COMET_BATCH_SIZE.get(sqlConf) + val timeZoneId = sqlConf.sessionLocalTimeZone + val batches = CometArrowConverters.rowToArrowBatchIter( + lines, + schema, + maxRecordsPerBatch, + timeZoneId, + TaskContext.get()) + CometCsvPartitionReader(batches) + } +} + +private case class CometCsvPartitionReader(it: Iterator[ColumnarBatch]) + extends PartitionReader[ColumnarBatch] { + override def next(): Boolean = it.hasNext + + override def get(): ColumnarBatch = it.next() + + override def close(): Unit = () } diff --git a/spark/src/main/scala/org/apache/comet/csv/CometCsvScan.scala b/spark/src/main/scala/org/apache/comet/csv/CometCsvScan.scala index 3d7f97a0ef..9275539df9 100644 --- a/spark/src/main/scala/org/apache/comet/csv/CometCsvScan.scala +++ b/spark/src/main/scala/org/apache/comet/csv/CometCsvScan.scala @@ -1,5 +1,39 @@ package org.apache.comet.csv -class CometCsvScan { +import scala.jdk.CollectionConverters.mapAsScalaMapConverter +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.csv.CSVOptions +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.connector.read.PartitionReaderFactory +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.v2.TextBasedFileScan +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.SerializableConfiguration + +case class CometCsvScan( + sparkSession: SparkSession, + options: CaseInsensitiveStringMap, + fileIndex: PartitioningAwareFileIndex, + dataSchema: StructType, + readDataSchema: StructType, + readPartitionSchema: StructType, + partitionFilters: Seq[Expression], + dataFilters: Seq[Expression]) + extends TextBasedFileScan(sparkSession, options) { + + private lazy val parsedOptions: CSVOptions = new CSVOptions( + options.asScala.toMap, + columnPruning = false, + sparkSession.sessionState.conf.sessionLocalTimeZone, + sparkSession.sessionState.conf.columnNameOfCorruptRecord) + + override def createReaderFactory(): PartitionReaderFactory = { + val hadoopConf = + sparkSession.sessionState.newHadoopConfWithOptions(options.asCaseSensitiveMap.asScala.toMap) + val broadcastedConf = + sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) + CometCsvPartitionReaderFactory(parsedOptions, broadcastedConf) + } } 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 1f8d3b0fa9..e70c26e10e 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -265,6 +265,7 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com } case scan: CSVScan => + CometBatchScanExec(scanExec, Seq.empty) // Iceberg scan - patched version implementing SupportsComet interface diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala deleted file mode 100644 index 47014c8edf..0000000000 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala +++ /dev/null @@ -1,3 +0,0 @@ -package org.apache.spark.sql.comet - -case class CometCsvNativeScanExec() From 6c12812be5098438265322da5b52226da9dbae96 Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Sat, 6 Dec 2025 20:21:30 +0400 Subject: [PATCH 03/23] Work --- .../scala/org/apache/comet/CometConf.scala | 10 ++++ .../core/src/execution/operators/csv_scan.rs | 46 +++++++++++++++++++ native/core/src/execution/operators/mod.rs | 1 + .../csv/CometCsvPartitionReaderFactory.scala | 44 +++++++++++++++--- .../org/apache/comet/csv/CometCsvScan.scala | 36 ++++++++++++++- .../apache/comet/rules/CometScanRule.scala | 31 +++++++++++-- .../serde/operator/CometCsvNativeScan.scala | 21 ++++----- .../sql/comet/CometCsvNativeScanExec.scala | 10 ++++ 8 files changed, 176 insertions(+), 23 deletions(-) create mode 100644 native/core/src/execution/operators/csv_scan.rs create mode 100644 spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 1e5d19ee23..6077eee131 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -143,6 +143,16 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(false) + val COMET_CSV_V2_NATIVE_ENABLED: ConfigEntry[Boolean] = + conf("spark.comet.scan.csv.v2.enabled") + .category(CATEGORY_SCAN) + .doc( + "Whether to use the native Comet V2 CSV reader for improved performance. " + + "Default: false (uses standard Spark CSV reader) " + + "Experimental: Performance benefits are workload-dependent.") + .booleanConf + .createWithDefault(false) + val COMET_RESPECT_PARQUET_FILTER_PUSHDOWN: ConfigEntry[Boolean] = conf("spark.comet.parquet.respectFilterPushdown") .category(CATEGORY_PARQUET) diff --git a/native/core/src/execution/operators/csv_scan.rs b/native/core/src/execution/operators/csv_scan.rs new file mode 100644 index 0000000000..3bf467933f --- /dev/null +++ b/native/core/src/execution/operators/csv_scan.rs @@ -0,0 +1,46 @@ +// 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. + +use crate::execution::operators::ExecutionError; +use arrow::datatypes::SchemaRef; +use datafusion::datasource::object_store::ObjectStoreUrl; +use datafusion::datasource::physical_plan::CsvSource; +use datafusion_datasource::file_groups::FileGroup; +use datafusion_datasource::file_scan_config::FileScanConfigBuilder; +use datafusion_datasource::source::DataSourceExec; +use datafusion_datasource::PartitionedFile; +use std::sync::Arc; + +pub(crate) fn init_csv_datasource_exec( + object_store_url: ObjectStoreUrl, + file_groups: Vec>, + data_schema: SchemaRef, +) -> Result, ExecutionError> { + let csv_source = CsvSource::new(false, 0, 1); + + let file_groups = file_groups + .iter() + .map(|files| FileGroup::new(files.clone())) + .collect(); + + let file_scan_config = + FileScanConfigBuilder::new(object_store_url, data_schema, Arc::new(csv_source)) + .with_file_groups(file_groups) + .build(); + + Ok(Arc::new(DataSourceExec::new(Arc::new(file_scan_config)))) +} diff --git a/native/core/src/execution/operators/mod.rs b/native/core/src/execution/operators/mod.rs index b01f7857be..958f1aac4a 100644 --- a/native/core/src/execution/operators/mod.rs +++ b/native/core/src/execution/operators/mod.rs @@ -32,6 +32,7 @@ mod iceberg_scan; mod parquet_writer; pub use parquet_writer::ParquetWriterExec; mod scan; +mod csv_scan; /// Error returned during executing operators. #[derive(thiserror::Error, Debug)] diff --git a/spark/src/main/scala/org/apache/comet/csv/CometCsvPartitionReaderFactory.scala b/spark/src/main/scala/org/apache/comet/csv/CometCsvPartitionReaderFactory.scala index dfeb38413b..f7a33335b8 100644 --- a/spark/src/main/scala/org/apache/comet/csv/CometCsvPartitionReaderFactory.scala +++ b/spark/src/main/scala/org/apache/comet/csv/CometCsvPartitionReaderFactory.scala @@ -1,3 +1,22 @@ +/* + * 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.csv import org.apache.spark.TaskContext @@ -21,15 +40,23 @@ case class CometCsvPartitionReaderFactory( broadcastedConf: Broadcast[SerializableConfiguration]) extends FilePartitionReaderFactory { - private val schema = - new StructType().add("value", ArrayType(elementType = ByteType), nullable = false) + private var partitionReaders: Iterator[CometCsvPartitionReader] = _ override def createColumnarReader(partition: InputPartition): PartitionReader[ColumnarBatch] = { val filePartition = partition.asInstanceOf[FilePartition] - filePartition.files.map { file => - file.filePath.toPath + this.partitionReaders = filePartition.files + .map(file => buildCometCsvPartitionReader(file)) + .toSeq + .toIterator + super.createColumnarReader(partition) + } + + override def buildColumnarReader( + partitionedFile: PartitionedFile): PartitionReader[ColumnarBatch] = { + if (partitionReaders.hasNext) { + return partitionReaders.next() } - null + buildCometCsvPartitionReader(partitionedFile) } override def supportColumnarReads(partition: InputPartition): Boolean = true @@ -49,7 +76,7 @@ case class CometCsvPartitionReaderFactory( val timeZoneId = sqlConf.sessionLocalTimeZone val batches = CometArrowConverters.rowToArrowBatchIter( lines, - schema, + CometCsvPartitionReaderFactory.SCHEMA, maxRecordsPerBatch, timeZoneId, TaskContext.get()) @@ -57,6 +84,11 @@ case class CometCsvPartitionReaderFactory( } } +object CometCsvPartitionReaderFactory { + private val SCHEMA = + new StructType().add("value", ArrayType(elementType = ByteType), nullable = false) +} + private case class CometCsvPartitionReader(it: Iterator[ColumnarBatch]) extends PartitionReader[ColumnarBatch] { override def next(): Boolean = it.hasNext diff --git a/spark/src/main/scala/org/apache/comet/csv/CometCsvScan.scala b/spark/src/main/scala/org/apache/comet/csv/CometCsvScan.scala index 9275539df9..d1716aef2f 100644 --- a/spark/src/main/scala/org/apache/comet/csv/CometCsvScan.scala +++ b/spark/src/main/scala/org/apache/comet/csv/CometCsvScan.scala @@ -1,3 +1,22 @@ +/* + * 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.csv import scala.jdk.CollectionConverters.mapAsScalaMapConverter @@ -8,6 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.TextBasedFileScan +import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration @@ -34,6 +54,20 @@ case class CometCsvScan( sparkSession.sessionState.newHadoopConfWithOptions(options.asCaseSensitiveMap.asScala.toMap) val broadcastedConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) - CometCsvPartitionReaderFactory(parsedOptions, broadcastedConf) + CometCsvPartitionReaderFactory(sparkSession.sessionState.conf, parsedOptions, broadcastedConf) + } +} + +object CometCsvScan { + def apply(session: SparkSession, scan: CSVScan): CometCsvScan = { + CometCsvScan( + session, + scan.options, + scan.fileIndex, + scan.dataSchema, + scan.readDataSchema, + scan.readPartitionSchema, + scan.partitionFilters, + scan.dataFilters) } } 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 e70c26e10e..b7c8f7d03b 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -20,33 +20,37 @@ package org.apache.comet.rules import java.net.URI + import scala.collection.mutable import scala.collection.mutable.ListBuffer import scala.jdk.CollectionConverters._ + import org.apache.hadoop.conf.Configuration import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, GenericInternalRow, PlanExpression} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MetadataColumnHelper, sideBySide} +import org.apache.spark.sql.catalyst.util.{sideBySide, ArrayBasedMapData, GenericArrayData, MetadataColumnHelper} import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.getExistenceDefaultValues import org.apache.spark.sql.comet.{CometBatchScanExec, CometScanExec} import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.datasources.HadoopFsRelation import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ + import org.apache.comet.{CometConf, CometNativeException, DataTypeSupport} import org.apache.comet.CometConf._ import org.apache.comet.CometSparkSessionExtensions.{isCometLoaded, isCometScanEnabled, withInfo, withInfos} import org.apache.comet.DataTypeSupport.isComplexType +import org.apache.comet.csv.CometCsvScan import org.apache.comet.iceberg.{CometIcebergNativeScanMetadata, IcebergReflection} import org.apache.comet.objectstore.NativeConfig import org.apache.comet.parquet.{CometParquetScan, Native, SupportsComet} import org.apache.comet.parquet.CometParquetUtils.{encryptionEnabled, isEncryptionConfigSupported} import org.apache.comet.shims.CometTypeShim -import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan /** * Spark physical optimizer rule for replacing Spark scans with Comet scans. @@ -264,9 +268,26 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com withInfos(scanExec, fallbackReasons.toSet) } - case scan: CSVScan => - - CometBatchScanExec(scanExec, Seq.empty) + case scan: CSVScan if COMET_CSV_V2_NATIVE_ENABLED.get() => + val fallbackReasons = new ListBuffer[String]() + val schemaSupported = + CometBatchScanExec.isSchemaSupported(scan.readDataSchema, fallbackReasons) + if (!schemaSupported) { + fallbackReasons += s"Schema ${scan.readDataSchema} is not supported" + } + val partitionSchemaSupported = + CometBatchScanExec.isSchemaSupported(scan.readPartitionSchema, fallbackReasons) + if (!partitionSchemaSupported) { + fallbackReasons += s"Partition schema ${scan.readPartitionSchema} is not supported" + } + if (schemaSupported && partitionSchemaSupported) { + val cometCsvScan = CometCsvScan(session, scan) + CometBatchScanExec( + scanExec.copy(scan = cometCsvScan), + runtimeFilters = scanExec.runtimeFilters) + } else { + withInfos(scanExec, fallbackReasons.toSet) + } // Iceberg scan - patched version implementing SupportsComet interface case s: SupportsComet if !COMET_ICEBERG_NATIVE_ENABLED.get() => diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometCsvNativeScan.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometCsvNativeScan.scala index 9028390502..d771fb3113 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometCsvNativeScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometCsvNativeScan.scala @@ -2,20 +2,19 @@ package org.apache.comet.serde.operator import org.apache.spark.sql.comet.{CometBatchScanExec, CometNativeExec} -import org.apache.comet.ConfigEntry -import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} -import org.apache.comet.serde.OperatorOuterClass.Operator +import org.apache.comet.{CometConf, ConfigEntry} +import org.apache.comet.serde.{CometOperatorSerde, Compatible, SupportLevel} object CometCsvNativeScan extends CometOperatorSerde[CometBatchScanExec] { - override def enabledConfig: Option[ConfigEntry[Boolean]] = ??? + override def enabledConfig: Option[ConfigEntry[Boolean]] = Some( + CometConf.COMET_CSV_V2_NATIVE_ENABLED) - override def convert( - op: CometBatchScanExec, - builder: Operator.Builder, - childOp: OperatorOuterClass.Operator*): Option[OperatorOuterClass.Operator] = ??? + override def getSupportLevel(operator: CometBatchScanExec): SupportLevel = { + Compatible(None) + } - override def createExec( - nativeOp: OperatorOuterClass.Operator, - op: CometBatchScanExec): CometNativeExec = ??? + override def convert(op: CometBatchScanExec, builder: Any, childOp: Any*): Option[Any] = ??? + + override def createExec(nativeOp: Any, op: CometBatchScanExec): CometNativeExec = ??? } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala new file mode 100644 index 0000000000..dbced992f6 --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala @@ -0,0 +1,10 @@ +package org.apache.spark.sql.comet +import org.apache.spark.sql.execution.SparkPlan + +case class CometCsvNativeScanExec() extends CometLeafExec { + override def serializedPlanOpt: SerializedPlan = ??? + + override def nativeOp = ??? + + override def originalPlan: SparkPlan = ??? +} From b601956b572c1372049c4681d3b8345b2dac994a Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Fri, 12 Dec 2025 19:56:28 +0400 Subject: [PATCH 04/23] work --- .../scala/org/apache/comet/CometConf.scala | 2 +- docs/source/user-guide/latest/configs.md | 1 + .../core/src/execution/operators/csv_scan.rs | 30 ++++- native/core/src/execution/operators/mod.rs | 1 + native/core/src/execution/planner.rs | 35 +++++- native/proto/src/proto/operator.proto | 8 ++ .../apache/comet/rules/CometExecRule.scala | 78 ++++++------ .../apache/comet/rules/CometScanRule.scala | 7 +- .../serde/operator/CometCsvNativeScan.scala | 20 ---- .../spark/sql/comet/CometBatchScanExec.scala | 5 +- .../sql/comet/CometCsvNativeScanExec.scala | 112 +++++++++++++++++- .../apache/spark/sql/comet/operators.scala | 9 +- .../comet/csv/CometNativeCsvScanSuite.scala | 47 ++++++++ .../org/apache/spark/sql/CometTestBase.scala | 5 +- 14 files changed, 275 insertions(+), 85 deletions(-) delete mode 100644 spark/src/main/scala/org/apache/comet/serde/operator/CometCsvNativeScan.scala create mode 100644 spark/src/test/scala/org/apache/comet/csv/CometNativeCsvScanSuite.scala diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 6077eee131..0cd803ee7b 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -145,7 +145,7 @@ object CometConf extends ShimCometConf { val COMET_CSV_V2_NATIVE_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.scan.csv.v2.enabled") - .category(CATEGORY_SCAN) + .category(CATEGORY_TESTING) .doc( "Whether to use the native Comet V2 CSV reader for improved performance. " + "Default: false (uses standard Spark CSV reader) " + diff --git a/docs/source/user-guide/latest/configs.md b/docs/source/user-guide/latest/configs.md index a1c3212c20..47fc295b06 100644 --- a/docs/source/user-guide/latest/configs.md +++ b/docs/source/user-guide/latest/configs.md @@ -143,6 +143,7 @@ These settings can be used to determine which parts of the plan are accelerated | `spark.comet.exec.onHeap.memoryPool` | The type of memory pool to be used for Comet native execution when running Spark in on-heap mode. Available pool types are `greedy`, `fair_spill`, `greedy_task_shared`, `fair_spill_task_shared`, `greedy_global`, `fair_spill_global`, and `unbounded`. | greedy_task_shared | | `spark.comet.memoryOverhead` | The amount of additional memory to be allocated per executor process for Comet, in MiB, when running Spark in on-heap mode. | 1024 MiB | | `spark.comet.parquet.write.enabled` | Whether to enable native Parquet write through Comet. When enabled, Comet will intercept Parquet write operations and execute them natively. This feature is highly experimental and only partially implemented. It should not be used in production. | false | +| `spark.comet.scan.csv.v2.enabled` | Whether to use the native Comet V2 CSV reader for improved performance. Default: false (uses standard Spark CSV reader) Experimental: Performance benefits are workload-dependent. | false | | `spark.comet.sparkToColumnar.enabled` | Whether to enable Spark to Arrow columnar conversion. When this is turned on, Comet will convert operators in `spark.comet.sparkToColumnar.supportedOperatorList` into Arrow columnar format before processing. This is an experimental feature and has known issues with non-UTC timezones. | false | | `spark.comet.sparkToColumnar.supportedOperatorList` | A comma-separated list of operators that will be converted to Arrow columnar format when `spark.comet.sparkToColumnar.enabled` is true. | Range,InMemoryTableScan,RDDScan | | `spark.comet.testing.strict` | Experimental option to enable strict testing, which will fail tests that could be more comprehensive, such as checking for a specific fallback reason. It can be overridden by the environment variable `ENABLE_COMET_STRICT_TESTING`. | false | diff --git a/native/core/src/execution/operators/csv_scan.rs b/native/core/src/execution/operators/csv_scan.rs index 3bf467933f..9db667ab7b 100644 --- a/native/core/src/execution/operators/csv_scan.rs +++ b/native/core/src/execution/operators/csv_scan.rs @@ -15,32 +15,52 @@ // specific language governing permissions and limitations // under the License. +use std::collections::HashMap; use crate::execution::operators::ExecutionError; use arrow::datatypes::SchemaRef; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::CsvSource; use datafusion_datasource::file_groups::FileGroup; -use datafusion_datasource::file_scan_config::FileScanConfigBuilder; +use datafusion_datasource::file_scan_config::{FileScanConfig, FileScanConfigBuilder}; use datafusion_datasource::source::DataSourceExec; use datafusion_datasource::PartitionedFile; use std::sync::Arc; +use crate::execution::spark_config::SparkConfig; -pub(crate) fn init_csv_datasource_exec( +pub fn init_csv_datasource_exec( object_store_url: ObjectStoreUrl, file_groups: Vec>, data_schema: SchemaRef, + options: &HashMap ) -> Result, ExecutionError> { - let csv_source = CsvSource::new(false, 0, 1); + let csv_source = build_csv_source(options); let file_groups = file_groups .iter() .map(|files| FileGroup::new(files.clone())) .collect(); - let file_scan_config = - FileScanConfigBuilder::new(object_store_url, data_schema, Arc::new(csv_source)) + let file_scan_config: FileScanConfig = + FileScanConfigBuilder::new(object_store_url, data_schema, csv_source) .with_file_groups(file_groups) .build(); Ok(Arc::new(DataSourceExec::new(Arc::new(file_scan_config)))) } + +fn build_csv_source(options: &HashMap) -> Arc { + let has_header = options.get_bool("header"); + let delimiter = options + .get("delimiter") + .and_then(|s| s.chars().next()) + .map(|c| c as u8) + .unwrap_or(b','); + let quote = options + .get("quote") + .and_then(|s| s.chars().next()) + .map(|c| c as u8) + .unwrap_or(b'"'); + let csv_source = CsvSource::new(has_header, delimiter, quote); + + Arc::new(csv_source) +} diff --git a/native/core/src/execution/operators/mod.rs b/native/core/src/execution/operators/mod.rs index 958f1aac4a..0eee9b2005 100644 --- a/native/core/src/execution/operators/mod.rs +++ b/native/core/src/execution/operators/mod.rs @@ -33,6 +33,7 @@ mod parquet_writer; pub use parquet_writer::ParquetWriterExec; mod scan; mod csv_scan; +pub use csv_scan::init_csv_datasource_exec; /// Error returned during executing operators. #[derive(thiserror::Error, Debug)] diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index ccbd0b2508..418e85d8ef 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -17,7 +17,7 @@ //! Converts Spark physical plan to DataFusion physical plan -use crate::execution::operators::IcebergScanExec; +use crate::execution::operators::{init_csv_datasource_exec, IcebergScanExec}; use crate::{ errors::ExpressionError, execution::{ @@ -88,6 +88,7 @@ use datafusion::physical_expr::window::WindowExpr; use datafusion::physical_expr::LexOrdering; use crate::parquet::parquet_exec::init_datasource_exec; + use arrow::array::{ new_empty_array, Array, ArrayRef, BinaryBuilder, BooleanArray, Date32Array, Decimal128Array, Float32Array, Float64Array, Int16Array, Int32Array, Int64Array, Int8Array, ListArray, @@ -1338,6 +1339,38 @@ impl PhysicalPlanner { Arc::new(SparkPlan::new(spark_plan.plan_id, scan, vec![])), )) } + OpStruct::CsvScan(scan) => { + let data_schema = convert_spark_types_to_arrow_schema(scan.schema.as_slice()); + let object_store_options: HashMap = scan + .object_store_options + .iter() + .map(|(k, v)| (k.clone(), v.clone())) + .collect(); + let one_file = scan + .file_partitions + .first() + .and_then(|f| f.partitioned_file.first()) + .map(|f| f.file_path.clone()) + .ok_or(GeneralError("Failed to locate file".to_string()))?; + let (object_store_url, _) = prepare_object_store_with_configs( + self.session_ctx.runtime_env(), + one_file, + &object_store_options, + )?; + let files = + self.get_partitioned_files(&scan.file_partitions[self.partition as usize])?; + let file_groups: Vec> = vec![files]; + let scan = init_csv_datasource_exec( + object_store_url, + file_groups, + data_schema, + &scan.options + )?; + Ok(( + vec![], + Arc::new(SparkPlan::new(spark_plan.plan_id, scan, vec![])), + )) + } OpStruct::Scan(scan) => { let data_types = scan.fields.iter().map(to_arrow_datatype).collect_vec(); diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index f09695b7c9..fe5a494ec9 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -50,6 +50,7 @@ message Operator { NativeScan native_scan = 111; IcebergScan iceberg_scan = 112; ParquetWriter parquet_writer = 113; + CsvScan csv_scan = 114; } } @@ -109,6 +110,13 @@ message NativeScan { bool encryption_enabled = 14; } +message CsvScan { + repeated SparkStructField schema = 1; + repeated SparkFilePartition file_partitions = 2; + map object_store_options = 3; + map options = 4; +} + message IcebergScan { // Schema to read repeated SparkStructField required_schema = 1; diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index 8e8098fd03..b44bbf1885 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AQEShuffleReadExec, BroadcastQueryStageExec, ShuffleQueryStageExec} import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec} import org.apache.spark.sql.execution.command.{DataWritingCommandExec, ExecutedCommandExec} +import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.v2.V2CommandExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} @@ -113,63 +114,49 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { /** * Tries to transform a Spark physical plan into a Comet plan. * - * This rule traverses bottom-up from the original Spark plan and for each plan node, there - * are a few cases to consider: + * This rule traverses bottom-up from the original Spark plan and for each plan node, there are + * a few cases to consider: * - * 1. The child(ren) of the current node `p` cannot be converted to native - * In this case, we'll simply return the original Spark plan, since Comet native - * execution cannot start from an arbitrary Spark operator (unless it is special node - * such as scan or sink such as shuffle exchange, union etc., which are wrapped by - * `CometScanWrapper` and `CometSinkPlaceHolder` respectively). + * 1. The child(ren) of the current node `p` cannot be converted to native In this case, we'll + * simply return the original Spark plan, since Comet native execution cannot start from an + * arbitrary Spark operator (unless it is special node such as scan or sink such as shuffle + * exchange, union etc., which are wrapped by `CometScanWrapper` and `CometSinkPlaceHolder` + * respectively). * - * 2. The child(ren) of the current node `p` can be converted to native - * There are two sub-cases for this scenario: 1) This node `p` can also be converted to - * native. In this case, we'll create a new native Comet operator for `p` and connect it with - * its previously converted child(ren); 2) This node `p` cannot be converted to native. In - * this case, similar to 1) above, we simply return `p` as it is. Its child(ren) would still - * be native Comet operators. + * 2. The child(ren) of the current node `p` can be converted to native There are two sub-cases + * for this scenario: 1) This node `p` can also be converted to native. In this case, we'll + * create a new native Comet operator for `p` and connect it with its previously converted + * child(ren); 2) This node `p` cannot be converted to native. In this case, similar to 1) + * above, we simply return `p` as it is. Its child(ren) would still be native Comet operators. * * After this rule finishes, we'll do another pass on the final plan to convert all adjacent - * Comet native operators into a single native execution block. Please see where - * `convertBlock` is called below. + * Comet native operators into a single native execution block. Please see where `convertBlock` + * is called below. * * Here are a few examples: * - * Scan ======> CometScan - * | | - * Filter CometFilter - * | | - * HashAggregate CometHashAggregate - * | | - * Exchange CometExchange - * | | - * HashAggregate CometHashAggregate - * | | - * UnsupportedOperator UnsupportedOperator + * Scan ======> CometScan + * \| | Filter CometFilter + * \| | HashAggregate CometHashAggregate + * \| | Exchange CometExchange + * \| | HashAggregate CometHashAggregate + * \| | UnsupportedOperator UnsupportedOperator * * Native execution doesn't necessarily have to start from `CometScan`: * - * Scan =======> CometScan - * | | - * UnsupportedOperator UnsupportedOperator - * | | - * HashAggregate HashAggregate - * | | - * Exchange CometExchange - * | | - * HashAggregate CometHashAggregate - * | | - * UnsupportedOperator UnsupportedOperator + * Scan =======> CometScan + * \| | UnsupportedOperator UnsupportedOperator + * \| | HashAggregate HashAggregate + * \| | Exchange CometExchange + * \| | HashAggregate CometHashAggregate + * \| | UnsupportedOperator UnsupportedOperator * * A sink can also be Comet operators other than `CometExchange`, for instance `CometUnion`: * - * Scan Scan =======> CometScan CometScan - * | | | | - * Filter Filter CometFilter CometFilter - * | | | | - * Union CometUnion - * | | - * Project CometProject + * Scan Scan =======> CometScan CometScan + * \| | | | Filter Filter CometFilter CometFilter + * \| | | | Union CometUnion + * \| | Project CometProject */ // spotless:on private def transform(plan: SparkPlan): SparkPlan = { @@ -184,6 +171,9 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { case scan: CometBatchScanExec if scan.nativeIcebergScanMetadata.isDefined => convertToComet(scan, CometIcebergNativeScan).getOrElse(scan) + case scan: CometBatchScanExec if scan.fileFormat.isInstanceOf[CSVFileFormat] => + convertToComet(scan, CometCsvNativeScanExec).getOrElse(scan) + // Comet JVM + native scan for V1 and V2 case op if isCometScan(op) => convertToComet(op, CometScanWrapper).getOrElse(op) 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 b7c8f7d03b..c2727463d7 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.getExistenceDefa import org.apache.spark.sql.comet.{CometBatchScanExec, CometScanExec} import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.datasources.HadoopFsRelation +import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan @@ -281,10 +282,10 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com fallbackReasons += s"Partition schema ${scan.readPartitionSchema} is not supported" } if (schemaSupported && partitionSchemaSupported) { - val cometCsvScan = CometCsvScan(session, scan) CometBatchScanExec( - scanExec.copy(scan = cometCsvScan), - runtimeFilters = scanExec.runtimeFilters) + scanExec.clone().asInstanceOf[BatchScanExec], + runtimeFilters = scanExec.runtimeFilters, + fileFormat = new CSVFileFormat) } else { withInfos(scanExec, fallbackReasons.toSet) } diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometCsvNativeScan.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometCsvNativeScan.scala deleted file mode 100644 index d771fb3113..0000000000 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometCsvNativeScan.scala +++ /dev/null @@ -1,20 +0,0 @@ -package org.apache.comet.serde.operator - -import org.apache.spark.sql.comet.{CometBatchScanExec, CometNativeExec} - -import org.apache.comet.{CometConf, ConfigEntry} -import org.apache.comet.serde.{CometOperatorSerde, Compatible, SupportLevel} - -object CometCsvNativeScan extends CometOperatorSerde[CometBatchScanExec] { - - override def enabledConfig: Option[ConfigEntry[Boolean]] = Some( - CometConf.COMET_CSV_V2_NATIVE_ENABLED) - - override def getSupportLevel(operator: CometBatchScanExec): SupportLevel = { - Compatible(None) - } - - override def convert(op: CometBatchScanExec, builder: Any, childOp: Any*): Option[Any] = ??? - - override def createExec(nativeOp: Any, op: CometBatchScanExec): CometNativeExec = ??? -} diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala index e4ccb4c63e..4115ded70d 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala @@ -28,6 +28,8 @@ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.connector.read._ import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.execution.metric._ import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType} @@ -41,7 +43,8 @@ import org.apache.comet.iceberg.CometIcebergNativeScanMetadata case class CometBatchScanExec( wrapped: BatchScanExec, runtimeFilters: Seq[Expression], - nativeIcebergScanMetadata: Option[CometIcebergNativeScanMetadata] = None) + nativeIcebergScanMetadata: Option[CometIcebergNativeScanMetadata] = None, + fileFormat: FileFormat = ParquetFileFormat.asInstanceOf[FileFormat]) extends DataSourceV2ScanExecBase with CometPlan { def ordering: Option[Seq[SortOrder]] = wrapped.ordering diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala index dbced992f6..be47c8b7b6 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala @@ -1,10 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.spark.sql.comet + +import scala.jdk.CollectionConverters._ + +import org.apache.spark.sql.catalyst.csv.CSVOptions +import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} +import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.FilePartition +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.types.{StructField, StructType} + +import org.apache.comet.{CometConf, ConfigEntry} +import org.apache.comet.objectstore.NativeConfig +import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} +import org.apache.comet.serde.OperatorOuterClass.Operator +import org.apache.comet.serde.QueryPlanSerde.serializeDataType + +case class CometCsvNativeScanExec( + override val nativeOp: Operator, + override val output: Seq[Attribute], + @transient override val originalPlan: BatchScanExec, + override val serializedPlanOpt: SerializedPlan) + extends CometLeafExec { + override val supportsColumnar: Boolean = true + + override val nodeName: String = "CometCsvNativeScan" + + override def outputPartitioning: Partitioning = UnknownPartitioning( + originalPlan.inputPartitions.length) + + override def outputOrdering: Seq[SortOrder] = Nil + + override protected def doCanonicalize(): SparkPlan = { + CometCsvNativeScanExec(nativeOp, output, originalPlan, serializedPlanOpt) + } +} + +object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { + + override def enabledConfig: Option[ConfigEntry[Boolean]] = Some( + CometConf.COMET_CSV_V2_NATIVE_ENABLED) + + override def convert( + op: CometBatchScanExec, + builder: Operator.Builder, + childOp: Operator*): Option[Operator] = { + val csvScanBuilder = OperatorOuterClass.CsvScan.newBuilder() + val schemaProto = schema2Proto(op.schema) + val partitionsProto = + op.inputPartitions.map(partition => partition2Proto(partition.asInstanceOf[FilePartition])) + csvScanBuilder.addAllFilePartitions(partitionsProto.asJava) + val hadoopConf = op.session.sessionState + .newHadoopConfWithOptions(op.session.sparkContext.conf.getAll.toMap) + op.inputPartitions.headOption.foreach { partitionFile => + val objectStoreOptions = + NativeConfig.extractObjectStoreOptions( + hadoopConf, + partitionFile.asInstanceOf[FilePartition].files.head.pathUri) + objectStoreOptions.foreach { case (key, value) => + csvScanBuilder.putObjectStoreOptions(key, value) + } + } + csvScanBuilder.addAllSchema(schemaProto.asJava) + Some(builder.setCsvScan(csvScanBuilder).build()) + } -case class CometCsvNativeScanExec() extends CometLeafExec { - override def serializedPlanOpt: SerializedPlan = ??? + override def createExec(nativeOp: Operator, op: CometBatchScanExec): CometNativeExec = { + CometCsvNativeScanExec(nativeOp, op.output, op.wrapped, SerializedPlan(None)) + } - override def nativeOp = ??? + private def schema2Proto(schema: StructType): Seq[OperatorOuterClass.SparkStructField] = { + val fieldBuilder = OperatorOuterClass.SparkStructField.newBuilder() + schema.fields.map { sf => + fieldBuilder.setDataType(serializeDataType(sf.dataType).get) + fieldBuilder.setName(sf.name) + fieldBuilder.setNullable(sf.nullable) + fieldBuilder.build() + }.toSeq + } - override def originalPlan: SparkPlan = ??? + private def partition2Proto(partition: FilePartition): OperatorOuterClass.SparkFilePartition = { + val partitionBuilder = OperatorOuterClass.SparkFilePartition.newBuilder() + partition.files.foreach { file => + val filePartitionBuilder = OperatorOuterClass.SparkPartitionedFile.newBuilder() + filePartitionBuilder + .setLength(file.length) + .setFilePath(file.filePath.toString) + .setStart(file.start) + .setFileSize(file.fileSize) + partitionBuilder.addPartitionedFile(filePartitionBuilder.build()) + } + partitionBuilder.build() + } } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala index 28688e9045..6ca9e530da 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/operators.scala @@ -393,10 +393,11 @@ abstract class CometNativeExec extends CometExec { def foreachUntilCometInput(plan: SparkPlan)(func: SparkPlan => Unit): Unit = { plan match { case _: CometNativeScanExec | _: CometScanExec | _: CometBatchScanExec | - _: CometIcebergNativeScanExec | _: ShuffleQueryStageExec | _: AQEShuffleReadExec | - _: CometShuffleExchangeExec | _: CometUnionExec | _: CometTakeOrderedAndProjectExec | - _: CometCoalesceExec | _: ReusedExchangeExec | _: CometBroadcastExchangeExec | - _: BroadcastQueryStageExec | _: CometSparkToColumnarExec | _: CometLocalTableScanExec => + _: CometIcebergNativeScanExec | _: CometCsvNativeScanExec | _: ShuffleQueryStageExec | + _: AQEShuffleReadExec | _: CometShuffleExchangeExec | _: CometUnionExec | + _: CometTakeOrderedAndProjectExec | _: CometCoalesceExec | _: ReusedExchangeExec | + _: CometBroadcastExchangeExec | _: BroadcastQueryStageExec | + _: CometSparkToColumnarExec | _: CometLocalTableScanExec => func(plan) case _: CometPlan => // Other Comet operators, continue to traverse the tree. diff --git a/spark/src/test/scala/org/apache/comet/csv/CometNativeCsvScanSuite.scala b/spark/src/test/scala/org/apache/comet/csv/CometNativeCsvScanSuite.scala new file mode 100644 index 0000000000..9230153c96 --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/csv/CometNativeCsvScanSuite.scala @@ -0,0 +1,47 @@ +/* + * 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.csv + +import org.apache.spark.sql.{CometTestBase, SaveMode} + +import org.apache.comet.CometConf + +class CometNativeCsvScanSuite extends CometTestBase { + + test("Native csv scan") { + withSQLConf( + CometConf.COMET_ENABLED.key -> "false", + CometConf.COMET_CSV_V2_NATIVE_ENABLED.key -> "false", + CometConf.COMET_EXPLAIN_FALLBACK_ENABLED.key -> "true", + CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "false", + "spark.sql.sources.useV1SourceList" -> "") { + spark.time { + val df = spark.read + .schema(Schemas.schema) + .options(Map("header" -> "false", "delimiter" -> ",")) + .csv("src/test/resources/test-data/songs/") + df.cache() + df.count() + df.explain(true) + } + } + Thread.sleep(10000000) + } +} diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala index 54854cb831..a82c94717c 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala @@ -67,12 +67,13 @@ abstract class CometTestBase protected def sparkConf: SparkConf = { val conf = new SparkConf() conf.set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName) - conf.set("spark.ui.enabled", "false") + conf.set("spark.ui.enabled", "true") conf.set(SQLConf.SHUFFLE_PARTITIONS, 10) // reduce parallelism in tests conf.set(SQLConf.ANSI_ENABLED.key, "false") conf.set(SHUFFLE_MANAGER, shuffleManager) conf.set(MEMORY_OFFHEAP_ENABLED.key, "true") - conf.set(MEMORY_OFFHEAP_SIZE.key, "2g") + conf.set(MEMORY_OFFHEAP_SIZE.key, "1200m") + conf.set("spark.driver.memory", "2g") conf.set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "1g") conf.set(SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key, "1g") conf.set(CometConf.COMET_ENABLED.key, "true") From c685235170c155f378408dd13712d8bb0e53e972 Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Sat, 13 Dec 2025 19:30:37 +0400 Subject: [PATCH 05/23] work --- .../core/src/execution/operators/csv_scan.rs | 21 ++-- native/core/src/execution/planner.rs | 2 +- .../execution/planner/operator_registry.rs | 2 + native/proto/src/proto/operator.proto | 33 ++++++- .../csv/CometCsvPartitionReaderFactory.scala | 99 ------------------- .../org/apache/comet/csv/CometCsvScan.scala | 73 -------------- .../apache/comet/rules/CometExecRule.scala | 3 +- .../apache/comet/rules/CometScanRule.scala | 1 - .../serde/operator/CometNativeScan.scala | 11 --- .../apache/comet/serde/operator/package.scala | 37 +++++++ .../sql/comet/CometCsvNativeScanExec.scala | 30 +++--- .../comet/csv/CometNativeCsvScanSuite.scala | 15 ++- 12 files changed, 102 insertions(+), 225 deletions(-) delete mode 100644 spark/src/main/scala/org/apache/comet/csv/CometCsvPartitionReaderFactory.scala delete mode 100644 spark/src/main/scala/org/apache/comet/csv/CometCsvScan.scala create mode 100644 spark/src/main/scala/org/apache/comet/serde/operator/package.scala diff --git a/native/core/src/execution/operators/csv_scan.rs b/native/core/src/execution/operators/csv_scan.rs index 9db667ab7b..d6afcbed85 100644 --- a/native/core/src/execution/operators/csv_scan.rs +++ b/native/core/src/execution/operators/csv_scan.rs @@ -25,15 +25,16 @@ use datafusion_datasource::file_scan_config::{FileScanConfig, FileScanConfigBuil use datafusion_datasource::source::DataSourceExec; use datafusion_datasource::PartitionedFile; use std::sync::Arc; +use datafusion_comet_proto::spark_operator::CsvOptions; use crate::execution::spark_config::SparkConfig; pub fn init_csv_datasource_exec( object_store_url: ObjectStoreUrl, file_groups: Vec>, data_schema: SchemaRef, - options: &HashMap + csv_options: &CsvOptions ) -> Result, ExecutionError> { - let csv_source = build_csv_source(options); + let csv_source = build_csv_source(csv_options); let file_groups = file_groups .iter() @@ -48,19 +49,9 @@ pub fn init_csv_datasource_exec( Ok(Arc::new(DataSourceExec::new(Arc::new(file_scan_config)))) } -fn build_csv_source(options: &HashMap) -> Arc { - let has_header = options.get_bool("header"); - let delimiter = options - .get("delimiter") - .and_then(|s| s.chars().next()) - .map(|c| c as u8) - .unwrap_or(b','); - let quote = options - .get("quote") - .and_then(|s| s.chars().next()) - .map(|c| c as u8) - .unwrap_or(b'"'); - let csv_source = CsvSource::new(has_header, delimiter, quote); +fn build_csv_source(options: &CsvOptions) -> Arc { + let csv_source = CsvSource::new(options.has_header, options.delimiter as u8, options.quote as u8) + .with_terminator(None); Arc::new(csv_source) } diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index e08d8e72b5..ab248ceaab 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1179,7 +1179,7 @@ impl PhysicalPlanner { object_store_url, file_groups, data_schema, - &scan.options + &scan.csv_options )?; Ok(( vec![], diff --git a/native/core/src/execution/planner/operator_registry.rs b/native/core/src/execution/planner/operator_registry.rs index e4899280b7..fe32e4a5cf 100644 --- a/native/core/src/execution/planner/operator_registry.rs +++ b/native/core/src/execution/planner/operator_registry.rs @@ -60,6 +60,7 @@ pub enum OperatorType { SortMergeJoin, HashJoin, Window, + CsvScan, } /// Global registry of operator builders @@ -151,5 +152,6 @@ fn get_operator_type(spark_operator: &Operator) -> Option { OpStruct::HashJoin(_) => Some(OperatorType::HashJoin), OpStruct::Window(_) => Some(OperatorType::Window), OpStruct::Explode(_) => None, // Not yet in OperatorType enum + OpStruct::CsvScan(_) => Some(OperatorType::CsvScan) } } diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index a77184df93..77aef1d812 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -112,10 +112,39 @@ message NativeScan { } message CsvScan { - repeated SparkStructField schema = 1; + repeated SparkStructField required_schema = 1; repeated SparkFilePartition file_partitions = 2; map object_store_options = 3; - map options = 4; + CsvOptions csv_options = 4; +} + +message CsvOptions { + bool has_header = 1; + uint32 delimiter = 2; + uint32 quote = 3; + uint32 escape = 4; + CompressionTypeVariant compression = 5; + optional uint64 schema_infer_max_rec = 6; + string date_format = 7; + string datetime_format = 8; + string timestamp_format = 9; + string timestamp_tz_format = 10; + string time_format = 11; + string null_value = 12; + string null_regex = 13; + bytes comment = 14; + bytes double_quote = 15; + bool newlines_in_values = 16; + bytes terminator = 17; + bytes truncated_rows = 18; +} + +enum CompressionTypeVariant { + GZIP = 0; + BZIP2 = 1; + XZ = 2; + ZSTD = 3; + UNCOMPRESSED = 4; } message IcebergScan { diff --git a/spark/src/main/scala/org/apache/comet/csv/CometCsvPartitionReaderFactory.scala b/spark/src/main/scala/org/apache/comet/csv/CometCsvPartitionReaderFactory.scala deleted file mode 100644 index f7a33335b8..0000000000 --- a/spark/src/main/scala/org/apache/comet/csv/CometCsvPartitionReaderFactory.scala +++ /dev/null @@ -1,99 +0,0 @@ -/* - * 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.csv - -import org.apache.spark.TaskContext -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.csv.CSVOptions -import org.apache.spark.sql.comet.execution.arrow.CometArrowConverters -import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader} -import org.apache.spark.sql.execution.datasources.{FilePartition, HadoopFileLinesReader, PartitionedFile} -import org.apache.spark.sql.execution.datasources.v2.FilePartitionReaderFactory -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{ArrayType, BinaryType, ByteType, StructType} -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.SerializableConfiguration - -import org.apache.comet.CometConf - -case class CometCsvPartitionReaderFactory( - sqlConf: SQLConf, - options: CSVOptions, - broadcastedConf: Broadcast[SerializableConfiguration]) - extends FilePartitionReaderFactory { - - private var partitionReaders: Iterator[CometCsvPartitionReader] = _ - - override def createColumnarReader(partition: InputPartition): PartitionReader[ColumnarBatch] = { - val filePartition = partition.asInstanceOf[FilePartition] - this.partitionReaders = filePartition.files - .map(file => buildCometCsvPartitionReader(file)) - .toSeq - .toIterator - super.createColumnarReader(partition) - } - - override def buildColumnarReader( - partitionedFile: PartitionedFile): PartitionReader[ColumnarBatch] = { - if (partitionReaders.hasNext) { - return partitionReaders.next() - } - buildCometCsvPartitionReader(partitionedFile) - } - - override def supportColumnarReads(partition: InputPartition): Boolean = true - - override def buildReader(partitionedFile: PartitionedFile): PartitionReader[InternalRow] = - throw new UnsupportedOperationException("Comet doesn't support 'buildReader'") - - private def buildCometCsvPartitionReader(file: PartitionedFile): CometCsvPartitionReader = { - val conf = broadcastedConf.value.value - val lines = { - val linesReader = new HadoopFileLinesReader(file, options.lineSeparatorInRead, conf) - Option(TaskContext.get()) - .foreach(_.addTaskCompletionListener[Unit](_ => linesReader.close())) - linesReader.map(line => InternalRow.apply(line.getBytes)) - } - val maxRecordsPerBatch = CometConf.COMET_BATCH_SIZE.get(sqlConf) - val timeZoneId = sqlConf.sessionLocalTimeZone - val batches = CometArrowConverters.rowToArrowBatchIter( - lines, - CometCsvPartitionReaderFactory.SCHEMA, - maxRecordsPerBatch, - timeZoneId, - TaskContext.get()) - CometCsvPartitionReader(batches) - } -} - -object CometCsvPartitionReaderFactory { - private val SCHEMA = - new StructType().add("value", ArrayType(elementType = ByteType), nullable = false) -} - -private case class CometCsvPartitionReader(it: Iterator[ColumnarBatch]) - extends PartitionReader[ColumnarBatch] { - override def next(): Boolean = it.hasNext - - override def get(): ColumnarBatch = it.next() - - override def close(): Unit = () -} diff --git a/spark/src/main/scala/org/apache/comet/csv/CometCsvScan.scala b/spark/src/main/scala/org/apache/comet/csv/CometCsvScan.scala deleted file mode 100644 index d1716aef2f..0000000000 --- a/spark/src/main/scala/org/apache/comet/csv/CometCsvScan.scala +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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.csv - -import scala.jdk.CollectionConverters.mapAsScalaMapConverter - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.csv.CSVOptions -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.connector.read.PartitionReaderFactory -import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex -import org.apache.spark.sql.execution.datasources.v2.TextBasedFileScan -import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.CaseInsensitiveStringMap -import org.apache.spark.util.SerializableConfiguration - -case class CometCsvScan( - sparkSession: SparkSession, - options: CaseInsensitiveStringMap, - fileIndex: PartitioningAwareFileIndex, - dataSchema: StructType, - readDataSchema: StructType, - readPartitionSchema: StructType, - partitionFilters: Seq[Expression], - dataFilters: Seq[Expression]) - extends TextBasedFileScan(sparkSession, options) { - - private lazy val parsedOptions: CSVOptions = new CSVOptions( - options.asScala.toMap, - columnPruning = false, - sparkSession.sessionState.conf.sessionLocalTimeZone, - sparkSession.sessionState.conf.columnNameOfCorruptRecord) - - override def createReaderFactory(): PartitionReaderFactory = { - val hadoopConf = - sparkSession.sessionState.newHadoopConfWithOptions(options.asCaseSensitiveMap.asScala.toMap) - val broadcastedConf = - sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) - CometCsvPartitionReaderFactory(sparkSession.sessionState.conf, parsedOptions, broadcastedConf) - } -} - -object CometCsvScan { - def apply(session: SparkSession, scan: CSVScan): CometCsvScan = { - CometCsvScan( - session, - scan.options, - scan.fileIndex, - scan.dataSchema, - scan.readDataSchema, - scan.readPartitionSchema, - scan.partitionFilters, - scan.dataFilters) - } -} diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index 67b973f385..8ac6899901 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -34,11 +34,10 @@ import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AQEShuffl import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec} import org.apache.spark.sql.execution.command.{DataWritingCommandExec, ExecutedCommandExec} import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat -import org.apache.spark.sql.execution.datasources.v2.V2CommandExec -import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.json.JsonFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, V2CommandExec} +import org.apache.spark.sql.execution.datasources.v2.V2CommandExec import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan import org.apache.spark.sql.execution.datasources.v2.json.JsonScan import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan 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 4e41951571..4953d30df5 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -46,7 +46,6 @@ import org.apache.comet.{CometConf, CometNativeException, DataTypeSupport} import org.apache.comet.CometConf._ import org.apache.comet.CometSparkSessionExtensions.{hasExplainInfo, isCometLoaded, withInfo, withInfos} import org.apache.comet.DataTypeSupport.isComplexType -import org.apache.comet.csv.CometCsvScan import org.apache.comet.iceberg.{CometIcebergNativeScanMetadata, IcebergReflection} import org.apache.comet.objectstore.NativeConfig import org.apache.comet.parquet.{CometParquetScan, Native, SupportsComet} diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala index 6381d1333f..75b4266edb 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala @@ -222,17 +222,6 @@ object CometNativeScan extends CometOperatorSerde[CometScanExec] with Logging { } - private def schema2Proto( - fields: Array[StructField]): Array[OperatorOuterClass.SparkStructField] = { - val fieldBuilder = OperatorOuterClass.SparkStructField.newBuilder() - fields.map(field => { - fieldBuilder.setName(field.name) - fieldBuilder.setDataType(serializeDataType(field.dataType).get) - fieldBuilder.setNullable(field.nullable) - fieldBuilder.build() - }) - } - private def partition2Proto( partition: FilePartition, nativeScanBuilder: OperatorOuterClass.NativeScan.Builder, diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/package.scala b/spark/src/main/scala/org/apache/comet/serde/operator/package.scala new file mode 100644 index 0000000000..f407f52bfa --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/serde/operator/package.scala @@ -0,0 +1,37 @@ +/* + * 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.serde + +import org.apache.spark.sql.types.StructField + +import org.apache.comet.serde.QueryPlanSerde.serializeDataType + +package object operator { + + def schema2Proto(fields: Array[StructField]): Array[OperatorOuterClass.SparkStructField] = { + val fieldBuilder = OperatorOuterClass.SparkStructField.newBuilder() + fields.map { field => + fieldBuilder.setName(field.name) + fieldBuilder.setDataType(serializeDataType(field.dataType).get) + fieldBuilder.setNullable(field.nullable) + fieldBuilder.build() + } + } +} diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala index be47c8b7b6..d67a52c41a 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala @@ -27,13 +27,16 @@ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartit import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.datasources.FilePartition import org.apache.spark.sql.execution.datasources.v2.BatchScanExec -import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +import com.google.protobuf.ByteString import org.apache.comet.{CometConf, ConfigEntry} import org.apache.comet.objectstore.NativeConfig import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} import org.apache.comet.serde.OperatorOuterClass.Operator -import org.apache.comet.serde.QueryPlanSerde.serializeDataType +import org.apache.comet.serde.operator.schema2Proto case class CometCsvNativeScanExec( override val nativeOp: Operator, @@ -65,7 +68,9 @@ object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { builder: Operator.Builder, childOp: Operator*): Option[Operator] = { val csvScanBuilder = OperatorOuterClass.CsvScan.newBuilder() - val schemaProto = schema2Proto(op.schema) + val csvScan = op.wrapped.scan.asInstanceOf[CSVScan] + csvOptions2Proto(csvScan.options) + val schemaProto = schema2Proto(op.schema.fields) val partitionsProto = op.inputPartitions.map(partition => partition2Proto(partition.asInstanceOf[FilePartition])) csvScanBuilder.addAllFilePartitions(partitionsProto.asJava) @@ -80,7 +85,7 @@ object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { csvScanBuilder.putObjectStoreOptions(key, value) } } - csvScanBuilder.addAllSchema(schemaProto.asJava) + csvScanBuilder.addAllRequiredSchema(schemaProto.toIterable.asJava) Some(builder.setCsvScan(csvScanBuilder).build()) } @@ -88,14 +93,15 @@ object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { CometCsvNativeScanExec(nativeOp, op.output, op.wrapped, SerializedPlan(None)) } - private def schema2Proto(schema: StructType): Seq[OperatorOuterClass.SparkStructField] = { - val fieldBuilder = OperatorOuterClass.SparkStructField.newBuilder() - schema.fields.map { sf => - fieldBuilder.setDataType(serializeDataType(sf.dataType).get) - fieldBuilder.setName(sf.name) - fieldBuilder.setNullable(sf.nullable) - fieldBuilder.build() - }.toSeq + private def csvOptions2Proto( + parameters: CaseInsensitiveStringMap, + columnPruning: Boolean, + timeZone: String): OperatorOuterClass.CsvOptions = { + val csvOptionsBuilder = OperatorOuterClass.CsvOptions.newBuilder() + val options = new CSVOptions(parameters.asScala.toMap, columnPruning, timeZone) + csvOptionsBuilder.setDelimiter(ByteString.copyFromUtf8(options.delimiter)) + csvOptionsBuilder.setHasHeader() + csvOptionsBuilder.build() } private def partition2Proto(partition: FilePartition): OperatorOuterClass.SparkFilePartition = { diff --git a/spark/src/test/scala/org/apache/comet/csv/CometNativeCsvScanSuite.scala b/spark/src/test/scala/org/apache/comet/csv/CometNativeCsvScanSuite.scala index 9230153c96..820003bc51 100644 --- a/spark/src/test/scala/org/apache/comet/csv/CometNativeCsvScanSuite.scala +++ b/spark/src/test/scala/org/apache/comet/csv/CometNativeCsvScanSuite.scala @@ -19,7 +19,7 @@ package org.apache.comet.csv -import org.apache.spark.sql.{CometTestBase, SaveMode} +import org.apache.spark.sql.CometTestBase import org.apache.comet.CometConf @@ -27,19 +27,16 @@ class CometNativeCsvScanSuite extends CometTestBase { test("Native csv scan") { withSQLConf( - CometConf.COMET_ENABLED.key -> "false", - CometConf.COMET_CSV_V2_NATIVE_ENABLED.key -> "false", + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_CSV_V2_NATIVE_ENABLED.key -> "true", CometConf.COMET_EXPLAIN_FALLBACK_ENABLED.key -> "true", - CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "false", + CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", "spark.sql.sources.useV1SourceList" -> "") { spark.time { val df = spark.read - .schema(Schemas.schema) .options(Map("header" -> "false", "delimiter" -> ",")) - .csv("src/test/resources/test-data/songs/") - df.cache() - df.count() - df.explain(true) + .csv("/Users/tendoo/Desktop/datafusion-comet/spark/src/test/resources/test-data/csv-test-1.csv") + df.show(false) } } Thread.sleep(10000000) From 768b3e90f261c7aea58bdb98dc698b90deeeae34 Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Sun, 14 Dec 2025 16:24:01 +0400 Subject: [PATCH 06/23] impl map_from_entries --- native/core/src/execution/jni_api.rs | 2 + .../apache/comet/serde/QueryPlanSerde.scala | 3 +- .../scala/org/apache/comet/serde/maps.scala | 29 +++++++++++- .../comet/CometMapExpressionSuite.scala | 45 +++++++++++++++++++ 4 files changed, 77 insertions(+), 2 deletions(-) diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index a24d993059..4f53cea3e6 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -46,6 +46,7 @@ use datafusion_spark::function::datetime::date_add::SparkDateAdd; use datafusion_spark::function::datetime::date_sub::SparkDateSub; use datafusion_spark::function::hash::sha1::SparkSha1; use datafusion_spark::function::hash::sha2::SparkSha2; +use datafusion_spark::function::map::map_from_entries::MapFromEntries; use datafusion_spark::function::math::expm1::SparkExpm1; use datafusion_spark::function::string::char::CharFunc; use datafusion_spark::function::string::concat::SparkConcat; @@ -337,6 +338,7 @@ fn register_datafusion_spark_function(session_ctx: &SessionContext) { session_ctx.register_udf(ScalarUDF::new_from_impl(SparkSha1::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkConcat::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkBitwiseNot::default())); + session_ctx.register_udf(ScalarUDF::new_from_impl(MapFromEntries::default())); } /// Prepares arrow arrays for output. diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index 54df2f1688..a99cf3824b 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -125,7 +125,8 @@ object QueryPlanSerde extends Logging with CometExprShim { classOf[MapKeys] -> CometMapKeys, classOf[MapEntries] -> CometMapEntries, classOf[MapValues] -> CometMapValues, - classOf[MapFromArrays] -> CometMapFromArrays) + classOf[MapFromArrays] -> CometMapFromArrays, + classOf[MapFromEntries] -> CometMapFromEntries) private val structExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] = Map( classOf[CreateNamedStruct] -> CometCreateNamedStruct, diff --git a/spark/src/main/scala/org/apache/comet/serde/maps.scala b/spark/src/main/scala/org/apache/comet/serde/maps.scala index 2e217f6af0..498aa3594c 100644 --- a/spark/src/main/scala/org/apache/comet/serde/maps.scala +++ b/spark/src/main/scala/org/apache/comet/serde/maps.scala @@ -19,9 +19,12 @@ package org.apache.comet.serde +import scala.annotation.tailrec + import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types.{ArrayType, MapType} +import org.apache.spark.sql.types.{ArrayType, BinaryType, DataType, MapType, StructType} +import org.apache.comet.serde.CometArrayReverse.containsBinary import org.apache.comet.serde.QueryPlanSerde.{exprToProtoInternal, optExprWithInfo, scalarFunctionExprToProto, scalarFunctionExprToProtoWithReturnType} object CometMapKeys extends CometExpressionSerde[MapKeys] { @@ -89,3 +92,27 @@ object CometMapFromArrays extends CometExpressionSerde[MapFromArrays] { optExprWithInfo(mapFromArraysExpr, expr, expr.children: _*) } } + +object CometMapFromEntries extends CometScalarFunction[MapFromEntries]("map_from_entries") { + val keyUnsupportedReason = "Using BinaryType as Map keys is not allowed in map_from_entries" + val valueUnsupportedReason = "Using BinaryType as Map values is not allowed in map_from_entries" + + private def containsBinary(dataType: DataType): Boolean = { + dataType match { + case BinaryType => true + case StructType(fields) => fields.exists(field => containsBinary(field.dataType)) + case ArrayType(elementType, _) => containsBinary(elementType) + case _ => false + } + } + + override def getSupportLevel(expr: MapFromEntries): SupportLevel = { + if (containsBinary(expr.dataType.keyType)) { + return Incompatible(Some(keyUnsupportedReason)) + } + if (containsBinary(expr.dataType.valueType)) { + return Incompatible(Some(valueUnsupportedReason)) + } + Compatible(None) + } +} diff --git a/spark/src/test/scala/org/apache/comet/CometMapExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometMapExpressionSuite.scala index 88c13391a6..01b9744ed6 100644 --- a/spark/src/test/scala/org/apache/comet/CometMapExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometMapExpressionSuite.scala @@ -25,7 +25,9 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.CometTestBase import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.BinaryType +import org.apache.comet.serde.CometMapFromEntries import org.apache.comet.testing.{DataGenOptions, ParquetGenerator, SchemaGenOptions} class CometMapExpressionSuite extends CometTestBase { @@ -125,4 +127,47 @@ class CometMapExpressionSuite extends CometTestBase { } } + test("map_from_entries") { + withTempDir { dir => + val path = new Path(dir.toURI.toString, "test.parquet") + val filename = path.toString + val random = new Random(42) + withSQLConf(CometConf.COMET_ENABLED.key -> "false") { + val schemaGenOptions = + SchemaGenOptions( + generateArray = true, + generateStruct = true, + primitiveTypes = SchemaGenOptions.defaultPrimitiveTypes.filterNot(_ == BinaryType)) + val dataGenOptions = DataGenOptions(allowNull = false, generateNegativeZero = false) + ParquetGenerator.makeParquetFile( + random, + spark, + filename, + 100, + schemaGenOptions, + dataGenOptions) + } + val df = spark.read.parquet(filename) + df.createOrReplaceTempView("t1") + for (field <- df.schema.fieldNames) { + checkSparkAnswerAndOperator( + spark.sql(s"SELECT map_from_entries(array(struct($field as a, $field as b))) FROM t1")) + } + } + } + + test("map_from_entries - fallback for binary type") { + val table = "t2" + withTable(table) { + sql( + s"create table $table using parquet as select cast(array() as array) as c1 from range(10)") + checkSparkAnswerAndFallbackReason( + sql(s"select map_from_entries(array(struct(c1, 0))) from $table"), + CometMapFromEntries.keyUnsupportedReason) + checkSparkAnswerAndFallbackReason( + sql(s"select map_from_entries(array(struct(0, c1))) from $table"), + CometMapFromEntries.valueUnsupportedReason) + } + } + } From c68c3428676b5d991e7ba9e13464bf2ce1ec84e8 Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Tue, 16 Dec 2025 16:10:43 +0400 Subject: [PATCH 07/23] Revert "impl map_from_entries" This reverts commit 768b3e90f261c7aea58bdb98dc698b90deeeae34. --- native/core/src/execution/jni_api.rs | 2 - .../apache/comet/serde/QueryPlanSerde.scala | 3 +- .../scala/org/apache/comet/serde/maps.scala | 29 +----------- .../comet/CometMapExpressionSuite.scala | 45 ------------------- 4 files changed, 2 insertions(+), 77 deletions(-) diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 4f53cea3e6..a24d993059 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -46,7 +46,6 @@ use datafusion_spark::function::datetime::date_add::SparkDateAdd; use datafusion_spark::function::datetime::date_sub::SparkDateSub; use datafusion_spark::function::hash::sha1::SparkSha1; use datafusion_spark::function::hash::sha2::SparkSha2; -use datafusion_spark::function::map::map_from_entries::MapFromEntries; use datafusion_spark::function::math::expm1::SparkExpm1; use datafusion_spark::function::string::char::CharFunc; use datafusion_spark::function::string::concat::SparkConcat; @@ -338,7 +337,6 @@ fn register_datafusion_spark_function(session_ctx: &SessionContext) { session_ctx.register_udf(ScalarUDF::new_from_impl(SparkSha1::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkConcat::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkBitwiseNot::default())); - session_ctx.register_udf(ScalarUDF::new_from_impl(MapFromEntries::default())); } /// Prepares arrow arrays for output. diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index a99cf3824b..54df2f1688 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -125,8 +125,7 @@ object QueryPlanSerde extends Logging with CometExprShim { classOf[MapKeys] -> CometMapKeys, classOf[MapEntries] -> CometMapEntries, classOf[MapValues] -> CometMapValues, - classOf[MapFromArrays] -> CometMapFromArrays, - classOf[MapFromEntries] -> CometMapFromEntries) + classOf[MapFromArrays] -> CometMapFromArrays) private val structExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] = Map( classOf[CreateNamedStruct] -> CometCreateNamedStruct, diff --git a/spark/src/main/scala/org/apache/comet/serde/maps.scala b/spark/src/main/scala/org/apache/comet/serde/maps.scala index 498aa3594c..2e217f6af0 100644 --- a/spark/src/main/scala/org/apache/comet/serde/maps.scala +++ b/spark/src/main/scala/org/apache/comet/serde/maps.scala @@ -19,12 +19,9 @@ package org.apache.comet.serde -import scala.annotation.tailrec - import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types.{ArrayType, BinaryType, DataType, MapType, StructType} +import org.apache.spark.sql.types.{ArrayType, MapType} -import org.apache.comet.serde.CometArrayReverse.containsBinary import org.apache.comet.serde.QueryPlanSerde.{exprToProtoInternal, optExprWithInfo, scalarFunctionExprToProto, scalarFunctionExprToProtoWithReturnType} object CometMapKeys extends CometExpressionSerde[MapKeys] { @@ -92,27 +89,3 @@ object CometMapFromArrays extends CometExpressionSerde[MapFromArrays] { optExprWithInfo(mapFromArraysExpr, expr, expr.children: _*) } } - -object CometMapFromEntries extends CometScalarFunction[MapFromEntries]("map_from_entries") { - val keyUnsupportedReason = "Using BinaryType as Map keys is not allowed in map_from_entries" - val valueUnsupportedReason = "Using BinaryType as Map values is not allowed in map_from_entries" - - private def containsBinary(dataType: DataType): Boolean = { - dataType match { - case BinaryType => true - case StructType(fields) => fields.exists(field => containsBinary(field.dataType)) - case ArrayType(elementType, _) => containsBinary(elementType) - case _ => false - } - } - - override def getSupportLevel(expr: MapFromEntries): SupportLevel = { - if (containsBinary(expr.dataType.keyType)) { - return Incompatible(Some(keyUnsupportedReason)) - } - if (containsBinary(expr.dataType.valueType)) { - return Incompatible(Some(valueUnsupportedReason)) - } - Compatible(None) - } -} diff --git a/spark/src/test/scala/org/apache/comet/CometMapExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometMapExpressionSuite.scala index 01b9744ed6..88c13391a6 100644 --- a/spark/src/test/scala/org/apache/comet/CometMapExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometMapExpressionSuite.scala @@ -25,9 +25,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.CometTestBase import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.BinaryType -import org.apache.comet.serde.CometMapFromEntries import org.apache.comet.testing.{DataGenOptions, ParquetGenerator, SchemaGenOptions} class CometMapExpressionSuite extends CometTestBase { @@ -127,47 +125,4 @@ class CometMapExpressionSuite extends CometTestBase { } } - test("map_from_entries") { - withTempDir { dir => - val path = new Path(dir.toURI.toString, "test.parquet") - val filename = path.toString - val random = new Random(42) - withSQLConf(CometConf.COMET_ENABLED.key -> "false") { - val schemaGenOptions = - SchemaGenOptions( - generateArray = true, - generateStruct = true, - primitiveTypes = SchemaGenOptions.defaultPrimitiveTypes.filterNot(_ == BinaryType)) - val dataGenOptions = DataGenOptions(allowNull = false, generateNegativeZero = false) - ParquetGenerator.makeParquetFile( - random, - spark, - filename, - 100, - schemaGenOptions, - dataGenOptions) - } - val df = spark.read.parquet(filename) - df.createOrReplaceTempView("t1") - for (field <- df.schema.fieldNames) { - checkSparkAnswerAndOperator( - spark.sql(s"SELECT map_from_entries(array(struct($field as a, $field as b))) FROM t1")) - } - } - } - - test("map_from_entries - fallback for binary type") { - val table = "t2" - withTable(table) { - sql( - s"create table $table using parquet as select cast(array() as array) as c1 from range(10)") - checkSparkAnswerAndFallbackReason( - sql(s"select map_from_entries(array(struct(c1, 0))) from $table"), - CometMapFromEntries.keyUnsupportedReason) - checkSparkAnswerAndFallbackReason( - sql(s"select map_from_entries(array(struct(0, c1))) from $table"), - CometMapFromEntries.valueUnsupportedReason) - } - } - } From 7ea16ee38eec8d39717d4c8f801902931c6ebc9f Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Wed, 17 Dec 2025 20:48:14 +0400 Subject: [PATCH 08/23] work --- native/proto/src/proto/operator.proto | 14 +++++++------- .../org/apache/comet/rules/CometExecRule.scala | 1 - .../spark/sql/comet/CometCsvNativeScanExec.scala | 15 ++++++++++++--- 3 files changed, 19 insertions(+), 11 deletions(-) diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 0a468392f3..208d9d6b7b 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -120,9 +120,9 @@ message CsvScan { message CsvOptions { bool has_header = 1; - uint32 delimiter = 2; - uint32 quote = 3; - uint32 escape = 4; + string delimiter = 2; + string quote = 3; + string escape = 4; CompressionTypeVariant compression = 5; optional uint64 schema_infer_max_rec = 6; string date_format = 7; @@ -132,11 +132,11 @@ message CsvOptions { string time_format = 11; string null_value = 12; string null_regex = 13; - bytes comment = 14; - bytes double_quote = 15; + string comment = 14; + string double_quote = 15; bool newlines_in_values = 16; - bytes terminator = 17; - bytes truncated_rows = 18; + string terminator = 17; + string truncated_rows = 18; } enum CompressionTypeVariant { diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index 8ac6899901..a463414090 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -37,7 +37,6 @@ import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.json.JsonFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, V2CommandExec} -import org.apache.spark.sql.execution.datasources.v2.V2CommandExec import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan import org.apache.spark.sql.execution.datasources.v2.json.JsonScan import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala index d67a52c41a..b5a8de657c 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala @@ -60,6 +60,8 @@ case class CometCsvNativeScanExec( object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { + private val DEFAULT_DATE_FORMAT = "yyyy-MM-dd" + override def enabledConfig: Option[ConfigEntry[Boolean]] = Some( CometConf.COMET_CSV_V2_NATIVE_ENABLED) @@ -69,7 +71,7 @@ object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { childOp: Operator*): Option[Operator] = { val csvScanBuilder = OperatorOuterClass.CsvScan.newBuilder() val csvScan = op.wrapped.scan.asInstanceOf[CSVScan] - csvOptions2Proto(csvScan.options) + csvOptions2Proto(csvScan.options, true, "") val schemaProto = schema2Proto(op.schema.fields) val partitionsProto = op.inputPartitions.map(partition => partition2Proto(partition.asInstanceOf[FilePartition])) @@ -99,8 +101,15 @@ object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { timeZone: String): OperatorOuterClass.CsvOptions = { val csvOptionsBuilder = OperatorOuterClass.CsvOptions.newBuilder() val options = new CSVOptions(parameters.asScala.toMap, columnPruning, timeZone) - csvOptionsBuilder.setDelimiter(ByteString.copyFromUtf8(options.delimiter)) - csvOptionsBuilder.setHasHeader() + csvOptionsBuilder.setDelimiter(options.delimiter) + csvOptionsBuilder.setHasHeader(options.headerFlag) + csvOptionsBuilder.setQuote(options.quote.toString) + csvOptionsBuilder.setEscape(options.escape.toString) + csvOptionsBuilder.setNullValue(options.nullValue) + if (options.isCommentSet) { + csvOptionsBuilder.setComment(options.comment.toString) + } + csvOptionsBuilder.setDateFormat(options.dateFormatInRead.getOrElse(DEFAULT_DATE_FORMAT)) csvOptionsBuilder.build() } From c521006784c1d6e94205b62363d7cb970fd02018 Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Fri, 19 Dec 2025 22:40:38 +0400 Subject: [PATCH 09/23] work --- .../core/src/execution/operators/csv_scan.rs | 21 ++++-- native/core/src/execution/planner.rs | 6 +- .../apache/comet/rules/CometExecRule.scala | 2 +- .../apache/comet/rules/CometScanRule.scala | 2 +- .../apache/comet/testing/CsvGenerator.scala} | 34 ++++----- .../spark/sql/comet/CometBatchScanExec.scala | 2 +- .../sql/comet/CometCsvNativeScanExec.scala | 7 +- .../CometNativeCsvReadBenchmark.scala | 73 +++++++++++++++++++ 8 files changed, 115 insertions(+), 32 deletions(-) rename spark/src/{test/scala/org/apache/comet/csv/CometNativeCsvScanSuite.scala => main/scala/org/apache/comet/testing/CsvGenerator.scala} (51%) create mode 100644 spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala diff --git a/native/core/src/execution/operators/csv_scan.rs b/native/core/src/execution/operators/csv_scan.rs index d6afcbed85..bbd500b0a9 100644 --- a/native/core/src/execution/operators/csv_scan.rs +++ b/native/core/src/execution/operators/csv_scan.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -use std::collections::HashMap; use crate::execution::operators::ExecutionError; use arrow::datatypes::SchemaRef; +use datafusion::common::Result; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::CsvSource; use datafusion_datasource::file_groups::FileGroup; @@ -25,8 +25,8 @@ use datafusion_datasource::file_scan_config::{FileScanConfig, FileScanConfigBuil use datafusion_datasource::source::DataSourceExec; use datafusion_datasource::PartitionedFile; use std::sync::Arc; +use datafusion_common::DataFusionError; use datafusion_comet_proto::spark_operator::CsvOptions; -use crate::execution::spark_config::SparkConfig; pub fn init_csv_datasource_exec( object_store_url: ObjectStoreUrl, @@ -34,7 +34,7 @@ pub fn init_csv_datasource_exec( data_schema: SchemaRef, csv_options: &CsvOptions ) -> Result, ExecutionError> { - let csv_source = build_csv_source(csv_options); + let csv_source = build_csv_source(csv_options.clone()); let file_groups = file_groups .iter() @@ -49,9 +49,18 @@ pub fn init_csv_datasource_exec( Ok(Arc::new(DataSourceExec::new(Arc::new(file_scan_config)))) } -fn build_csv_source(options: &CsvOptions) -> Arc { - let csv_source = CsvSource::new(options.has_header, options.delimiter as u8, options.quote as u8) - .with_terminator(None); +fn build_csv_source(options: CsvOptions) -> Arc { + let delimiter = char_to_u8(options.delimiter.chars().next().unwrap(), "delimiter").unwrap(); + let quote = char_to_u8(options.quote.chars().next().unwrap(), "quote").unwrap(); + let csv_source = CsvSource::new(options.has_header, delimiter, quote); Arc::new(csv_source) } + +fn char_to_u8(c: char, option: &str) -> Result { + if c.is_ascii() { + Ok(c as u8) + } else { + Err(DataFusionError::Configuration(format!("invalid {option} character '{c}': must be an ASCII character"))) + } +} diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index d7068bcffb..b366f5074b 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -22,7 +22,7 @@ pub mod macros; pub mod operator_registry; use crate::execution::operators::IcebergScanExec; -use crate::execution::operators::{init_csv_datasource_exec, IcebergScanExec}; +use crate::execution::operators::init_csv_datasource_exec; use crate::{ errors::ExpressionError, execution::{ @@ -1155,7 +1155,7 @@ impl PhysicalPlanner { )) } OpStruct::CsvScan(scan) => { - let data_schema = convert_spark_types_to_arrow_schema(scan.schema.as_slice()); + let data_schema = convert_spark_types_to_arrow_schema(scan.required_schema.as_slice()); let object_store_options: HashMap = scan .object_store_options .iter() @@ -1179,7 +1179,7 @@ impl PhysicalPlanner { object_store_url, file_groups, data_schema, - &scan.csv_options + &scan.csv_options.clone().unwrap() )?; Ok(( vec![], diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index a463414090..587423b03b 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -190,7 +190,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { case scan: CometBatchScanExec if scan.nativeIcebergScanMetadata.isDefined => convertToComet(scan, CometIcebergNativeScan).getOrElse(scan) - case scan: CometBatchScanExec if scan.fileFormat.isInstanceOf[CSVFileFormat] => + case scan: CometBatchScanExec if scan.fileFormat == "csv" => convertToComet(scan, CometCsvNativeScanExec).getOrElse(scan) // Comet JVM + native scan for V1 and V2 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 a9f71144f2..7254d5f2aa 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -244,7 +244,7 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com CometBatchScanExec( scanExec.clone().asInstanceOf[BatchScanExec], runtimeFilters = scanExec.runtimeFilters, - fileFormat = new CSVFileFormat) + fileFormat = "csv") } else { withInfos(scanExec, fallbackReasons.toSet) } diff --git a/spark/src/test/scala/org/apache/comet/csv/CometNativeCsvScanSuite.scala b/spark/src/main/scala/org/apache/comet/testing/CsvGenerator.scala similarity index 51% rename from spark/src/test/scala/org/apache/comet/csv/CometNativeCsvScanSuite.scala rename to spark/src/main/scala/org/apache/comet/testing/CsvGenerator.scala index 820003bc51..8825aeec8e 100644 --- a/spark/src/test/scala/org/apache/comet/csv/CometNativeCsvScanSuite.scala +++ b/spark/src/main/scala/org/apache/comet/testing/CsvGenerator.scala @@ -17,28 +17,24 @@ * under the License. */ -package org.apache.comet.csv +package org.apache.comet.testing -import org.apache.spark.sql.CometTestBase +import scala.util.Random -import org.apache.comet.CometConf +import org.apache.spark.sql.{SaveMode, SparkSession} +import org.apache.spark.sql.types.StructType -class CometNativeCsvScanSuite extends CometTestBase { +object CsvGenerator { - test("Native csv scan") { - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_CSV_V2_NATIVE_ENABLED.key -> "true", - CometConf.COMET_EXPLAIN_FALLBACK_ENABLED.key -> "true", - CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", - "spark.sql.sources.useV1SourceList" -> "") { - spark.time { - val df = spark.read - .options(Map("header" -> "false", "delimiter" -> ",")) - .csv("/Users/tendoo/Desktop/datafusion-comet/spark/src/test/resources/test-data/csv-test-1.csv") - df.show(false) - } - } - Thread.sleep(10000000) + def makeCsvFile( + r: Random, + spark: SparkSession, + schema: StructType, + filename: String, + numRows: Int): Unit = { + val options = DataGenOptions(allowNull = false, generateNaN = false) + val df = FuzzDataGenerator.generateDataFrame(r, spark, schema, numRows, options) + df.write.mode(SaveMode.Overwrite).csv(filename) } + } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala index 4115ded70d..5c0e4af51b 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala @@ -44,7 +44,7 @@ case class CometBatchScanExec( wrapped: BatchScanExec, runtimeFilters: Seq[Expression], nativeIcebergScanMetadata: Option[CometIcebergNativeScanMetadata] = None, - fileFormat: FileFormat = ParquetFileFormat.asInstanceOf[FileFormat]) + fileFormat: String = "parquet") extends DataSourceV2ScanExecBase with CometPlan { def ordering: Option[Seq[SortOrder]] = wrapped.ordering diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala index b5a8de657c..8ea853aa8a 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala @@ -71,7 +71,12 @@ object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { childOp: Operator*): Option[Operator] = { val csvScanBuilder = OperatorOuterClass.CsvScan.newBuilder() val csvScan = op.wrapped.scan.asInstanceOf[CSVScan] - csvOptions2Proto(csvScan.options, true, "") + val columnPruning = op.session.sessionState.conf.csvColumnPruning + val timeZone = op.session.sessionState.conf.sessionLocalTimeZone + + val csvOptionsProto = csvOptions2Proto(csvScan.options, columnPruning, timeZone) + csvScanBuilder.setCsvOptions(csvOptionsProto) + val schemaProto = schema2Proto(op.schema.fields) val partitionsProto = op.inputPartitions.map(partition => partition2Proto(partition.asInstanceOf[FilePartition])) diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala new file mode 100644 index 0000000000..1bd7d72d4d --- /dev/null +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.benchmark + +import java.io.File + +import scala.util.Random + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.benchmark.CometExecBenchmark.withSQLConf +import org.apache.spark.sql.types.{DataTypes, StructType} + +import org.apache.comet.CometConf +import org.apache.comet.testing.{CsvGenerator, FuzzDataGenerator, SchemaGenOptions} + +object CometNativeCsvReadBenchmark extends CometBenchmarkBase { + + private def prepareCsvTable(dir: File, schema: StructType, numRows: Int): Unit = { + val random = new Random(42) + CsvGenerator.makeCsvFile(random, spark, schema, dir.getCanonicalPath, numRows) + } + + override def runCometBenchmark(args: Array[String]): Unit = { + val numRows = 2000000 + val benchmark = new Benchmark(s"Native csv read - $numRows rows", numRows, output = output) + withTempPath { dir => + val schema = FuzzDataGenerator.generateSchema( + SchemaGenOptions(primitiveTypes = Seq( + DataTypes.BooleanType, + DataTypes.ByteType, + DataTypes.ShortType, + DataTypes.IntegerType, + DataTypes.LongType, + DataTypes.FloatType, + DataTypes.DoubleType, + DataTypes.createDecimalType(10, 2), + DataTypes.createDecimalType(36, 18), + DataTypes.DateType, + DataTypes.StringType))) + prepareCsvTable(dir, schema, numRows) + benchmark.addCase("Simple read") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_CSV_V2_NATIVE_ENABLED.key -> "true", + CometConf.COMET_EXPLAIN_FALLBACK_ENABLED.key -> "true", + "spark.sql.sources.useV1SourceList" -> "") { + spark.read + .schema(schema) + .csv(dir.getCanonicalPath) + .foreach(_ => ()) + } + } + benchmark.run() + } + } +} From 0f069369b9ed723d2e127dec87741b6a154789ec Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Tue, 30 Dec 2025 18:47:34 +0400 Subject: [PATCH 10/23] WIP --- .../core/src/execution/operators/csv_scan.rs | 50 ++++++++++++++----- native/core/src/execution/planner.rs | 2 + native/proto/src/proto/operator.proto | 32 +++--------- .../sql/comet/CometCsvNativeScanExec.scala | 24 +++++---- .../apache/comet/csv/CometCsvReadSuite.scala | 47 +++++++++++++++++ 5 files changed, 107 insertions(+), 48 deletions(-) create mode 100644 spark/src/test/scala/org/apache/comet/csv/CometCsvReadSuite.scala diff --git a/native/core/src/execution/operators/csv_scan.rs b/native/core/src/execution/operators/csv_scan.rs index bbd500b0a9..7cc6d41bc2 100644 --- a/native/core/src/execution/operators/csv_scan.rs +++ b/native/core/src/execution/operators/csv_scan.rs @@ -16,23 +16,25 @@ // under the License. use crate::execution::operators::ExecutionError; -use arrow::datatypes::SchemaRef; +use arrow::datatypes::{Field, SchemaRef}; +use datafusion::common::DataFusionError; use datafusion::common::Result; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::CsvSource; +use datafusion_comet_proto::spark_operator::CsvOptions; use datafusion_datasource::file_groups::FileGroup; use datafusion_datasource::file_scan_config::{FileScanConfig, FileScanConfigBuilder}; use datafusion_datasource::source::DataSourceExec; use datafusion_datasource::PartitionedFile; +use itertools::Itertools; use std::sync::Arc; -use datafusion_common::DataFusionError; -use datafusion_comet_proto::spark_operator::CsvOptions; pub fn init_csv_datasource_exec( object_store_url: ObjectStoreUrl, file_groups: Vec>, data_schema: SchemaRef, - csv_options: &CsvOptions + partition_schema: Option, + csv_options: &CsvOptions, ) -> Result, ExecutionError> { let csv_source = build_csv_source(csv_options.clone()); @@ -41,26 +43,48 @@ pub fn init_csv_datasource_exec( .map(|files| FileGroup::new(files.clone())) .collect(); + let partition_fields = partition_schema + .map(|schema| { + schema + .fields() + .iter() + .map(|field| { + Field::new(field.name(), field.data_type().clone(), field.is_nullable()) + }) + .collect_vec() + }) + .unwrap_or(vec![]); + let file_scan_config: FileScanConfig = FileScanConfigBuilder::new(object_store_url, data_schema, csv_source) .with_file_groups(file_groups) + .with_table_partition_cols(partition_fields) .build(); Ok(Arc::new(DataSourceExec::new(Arc::new(file_scan_config)))) } fn build_csv_source(options: CsvOptions) -> Arc { - let delimiter = char_to_u8(options.delimiter.chars().next().unwrap(), "delimiter").unwrap(); - let quote = char_to_u8(options.quote.chars().next().unwrap(), "quote").unwrap(); - let csv_source = CsvSource::new(options.has_header, delimiter, quote); - + let delimiter = string_to_u8(&options.delimiter, "delimiter").unwrap(); + let quote = string_to_u8(&options.quote, "quote").unwrap(); + let escape = string_to_u8(&options.escape, "escape").unwrap(); + let terminator = string_to_u8(&options.terminator, "terminator").unwrap(); + let comment = options + .comment + .map(|c| string_to_u8(&c, "comment").unwrap()); + let csv_source = CsvSource::new(options.has_header, delimiter, quote) + .with_escape(Some(escape)) + .with_comment(comment) + .with_terminator(Some(terminator)) + .with_truncate_rows(options.truncated_rows); Arc::new(csv_source) } -fn char_to_u8(c: char, option: &str) -> Result { - if c.is_ascii() { - Ok(c as u8) - } else { - Err(DataFusionError::Configuration(format!("invalid {option} character '{c}': must be an ASCII character"))) +fn string_to_u8(option: &str, option_name: &str) -> Result { + match option.as_bytes().first() { + Some(&ch) if ch.is_ascii() => Ok(ch), + _ => Err(DataFusionError::Configuration(format!( + "invalid {option_name} character '{option}': must be an ASCII character" + ))), } } diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 211eff30fb..3d72306118 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1124,6 +1124,7 @@ impl PhysicalPlanner { } OpStruct::CsvScan(scan) => { let data_schema = convert_spark_types_to_arrow_schema(scan.required_schema.as_slice()); + let partition_schema = convert_spark_types_to_arrow_schema(scan.partition_schema.as_slice()); let object_store_options: HashMap = scan .object_store_options .iter() @@ -1147,6 +1148,7 @@ impl PhysicalPlanner { object_store_url, file_groups, data_schema, + Some(partition_schema), &scan.csv_options.clone().unwrap() )?; Ok(( diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 06bd2b5abd..3372f7a4d3 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -113,9 +113,10 @@ message NativeScan { message CsvScan { repeated SparkStructField required_schema = 1; - repeated SparkFilePartition file_partitions = 2; - map object_store_options = 3; - CsvOptions csv_options = 4; + repeated SparkStructField partition_schema = 2; + repeated SparkFilePartition file_partitions = 3; + map object_store_options = 4; + CsvOptions csv_options = 5; } message CsvOptions { @@ -123,28 +124,9 @@ message CsvOptions { string delimiter = 2; string quote = 3; string escape = 4; - CompressionTypeVariant compression = 5; - optional uint64 schema_infer_max_rec = 6; - string date_format = 7; - string datetime_format = 8; - string timestamp_format = 9; - string timestamp_tz_format = 10; - string time_format = 11; - string null_value = 12; - string null_regex = 13; - string comment = 14; - string double_quote = 15; - bool newlines_in_values = 16; - string terminator = 17; - string truncated_rows = 18; -} - -enum CompressionTypeVariant { - GZIP = 0; - BZIP2 = 1; - XZ = 2; - ZSTD = 3; - UNCOMPRESSED = 4; + optional string comment = 5; + string terminator = 7; + bool truncated_rows = 8; } message IcebergScan { diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala index 8ea853aa8a..54845df87b 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala @@ -34,7 +34,7 @@ import com.google.protobuf.ByteString import org.apache.comet.{CometConf, ConfigEntry} import org.apache.comet.objectstore.NativeConfig -import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} +import org.apache.comet.serde.{CometOperatorSerde, Incompatible, OperatorOuterClass, SupportLevel} import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.operator.schema2Proto @@ -60,11 +60,13 @@ case class CometCsvNativeScanExec( object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { - private val DEFAULT_DATE_FORMAT = "yyyy-MM-dd" - override def enabledConfig: Option[ConfigEntry[Boolean]] = Some( CometConf.COMET_CSV_V2_NATIVE_ENABLED) + override def getSupportLevel(operator: CometBatchScanExec): SupportLevel = { + Incompatible() + } + override def convert( op: CometBatchScanExec, builder: Operator.Builder, @@ -74,13 +76,12 @@ object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { val columnPruning = op.session.sessionState.conf.csvColumnPruning val timeZone = op.session.sessionState.conf.sessionLocalTimeZone + val filePartitions = op.inputPartitions.map(_.asInstanceOf[FilePartition]) val csvOptionsProto = csvOptions2Proto(csvScan.options, columnPruning, timeZone) - csvScanBuilder.setCsvOptions(csvOptionsProto) - val schemaProto = schema2Proto(op.schema.fields) - val partitionsProto = - op.inputPartitions.map(partition => partition2Proto(partition.asInstanceOf[FilePartition])) - csvScanBuilder.addAllFilePartitions(partitionsProto.asJava) + val partitionSchemaProto = schema2Proto(csvScan.readPartitionSchema.fields) + val partitionsProto = filePartitions.map(partition2Proto) + val hadoopConf = op.session.sessionState .newHadoopConfWithOptions(op.session.sparkContext.conf.getAll.toMap) op.inputPartitions.headOption.foreach { partitionFile => @@ -92,7 +93,10 @@ object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { csvScanBuilder.putObjectStoreOptions(key, value) } } + csvScanBuilder.setCsvOptions(csvOptionsProto) + csvScanBuilder.addAllFilePartitions(partitionsProto.asJava) csvScanBuilder.addAllRequiredSchema(schemaProto.toIterable.asJava) + csvScanBuilder.addAllPartitionSchema(partitionSchemaProto.toIterable.asJava) Some(builder.setCsvScan(csvScanBuilder).build()) } @@ -110,11 +114,11 @@ object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { csvOptionsBuilder.setHasHeader(options.headerFlag) csvOptionsBuilder.setQuote(options.quote.toString) csvOptionsBuilder.setEscape(options.escape.toString) - csvOptionsBuilder.setNullValue(options.nullValue) + csvOptionsBuilder.setTerminator(options.lineSeparator.getOrElse("\n")) + csvOptionsBuilder.setTruncatedRows(options.multiLine) if (options.isCommentSet) { csvOptionsBuilder.setComment(options.comment.toString) } - csvOptionsBuilder.setDateFormat(options.dateFormatInRead.getOrElse(DEFAULT_DATE_FORMAT)) csvOptionsBuilder.build() } diff --git a/spark/src/test/scala/org/apache/comet/csv/CometCsvReadSuite.scala b/spark/src/test/scala/org/apache/comet/csv/CometCsvReadSuite.scala new file mode 100644 index 0000000000..817adfdcd1 --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/csv/CometCsvReadSuite.scala @@ -0,0 +1,47 @@ +/* + * 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.csv + +import org.apache.spark.sql.CometTestBase +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{IntegerType, StructType} + +import org.apache.comet.CometConf + +class CometCsvReadSuite extends CometTestBase { + + test("native csv read") { + withSQLConf( + CometConf.COMET_CSV_V2_NATIVE_ENABLED.key -> "true", + SQLConf.USE_V1_SOURCE_LIST.key -> "") { + val schema = new StructType() + .add("a", IntegerType) + .add("b", IntegerType) + .add("c", IntegerType) + + val df = spark.read + .options(Map("header" -> "false", "delimiter" -> ",")) + .schema(schema) + .csv("/Users/tendoo/Desktop/datafusion-comet/spark/src/test/resources/test-data/csv-test-1.csv") + df.explain(true) + df.show(false) + } + } +} From 033ba8b4dde4245e66a54f99305622f7046c3522 Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Tue, 30 Dec 2025 20:18:16 +0400 Subject: [PATCH 11/23] WIP --- .../apache/comet/rules/CometExecRule.scala | 68 +++++++------------ .../apache/comet/rules/CometScanRule.scala | 2 +- .../serde/operator/CometNativeScan.scala | 38 +---------- .../apache/comet/serde/operator/package.scala | 37 +++++++++- .../spark/sql/comet/CometBatchScanExec.scala | 2 +- .../sql/comet/CometCsvNativeScanExec.scala | 52 +++++--------- 6 files changed, 81 insertions(+), 118 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index 5b100c69af..d965de374e 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -120,63 +120,41 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { /** * Tries to transform a Spark physical plan into a Comet plan. * - * This rule traverses bottom-up from the original Spark plan and for each plan node, there - * are a few cases to consider: + * This rule traverses bottom-up from the original Spark plan and for each plan node, there are + * a few cases to consider: * - * 1. The child(ren) of the current node `p` cannot be converted to native - * In this case, we'll simply return the original Spark plan, since Comet native - * execution cannot start from an arbitrary Spark operator (unless it is special node - * such as scan or sink such as shuffle exchange, union etc., which are wrapped by - * `CometScanWrapper` and `CometSinkPlaceHolder` respectively). + * 1. The child(ren) of the current node `p` cannot be converted to native In this case, we'll + * simply return the original Spark plan, since Comet native execution cannot start from an + * arbitrary Spark operator (unless it is special node such as scan or sink such as shuffle + * exchange, union etc., which are wrapped by `CometScanWrapper` and `CometSinkPlaceHolder` + * respectively). * - * 2. The child(ren) of the current node `p` can be converted to native - * There are two sub-cases for this scenario: 1) This node `p` can also be converted to - * native. In this case, we'll create a new native Comet operator for `p` and connect it with - * its previously converted child(ren); 2) This node `p` cannot be converted to native. In - * this case, similar to 1) above, we simply return `p` as it is. Its child(ren) would still - * be native Comet operators. + * 2. The child(ren) of the current node `p` can be converted to native There are two sub-cases + * for this scenario: 1) This node `p` can also be converted to native. In this case, we'll + * create a new native Comet operator for `p` and connect it with its previously converted + * child(ren); 2) This node `p` cannot be converted to native. In this case, similar to 1) + * above, we simply return `p` as it is. Its child(ren) would still be native Comet operators. * * After this rule finishes, we'll do another pass on the final plan to convert all adjacent - * Comet native operators into a single native execution block. Please see where - * `convertBlock` is called below. + * Comet native operators into a single native execution block. Please see where `convertBlock` + * is called below. * * Here are a few examples: * - * Scan ======> CometScan - * | | - * Filter CometFilter - * | | - * HashAggregate CometHashAggregate - * | | - * Exchange CometExchange - * | | - * HashAggregate CometHashAggregate - * | | - * UnsupportedOperator UnsupportedOperator + * Scan ======> CometScan \| | Filter CometFilter \| | HashAggregate CometHashAggregate \| | + * Exchange CometExchange \| | HashAggregate CometHashAggregate \| | UnsupportedOperator + * UnsupportedOperator * * Native execution doesn't necessarily have to start from `CometScan`: * - * Scan =======> CometScan - * | | - * UnsupportedOperator UnsupportedOperator - * | | - * HashAggregate HashAggregate - * | | - * Exchange CometExchange - * | | - * HashAggregate CometHashAggregate - * | | - * UnsupportedOperator UnsupportedOperator + * Scan =======> CometScan \| | UnsupportedOperator UnsupportedOperator \| | HashAggregate + * HashAggregate \| | Exchange CometExchange \| | HashAggregate CometHashAggregate \| | + * UnsupportedOperator UnsupportedOperator * * A sink can also be Comet operators other than `CometExchange`, for instance `CometUnion`: * - * Scan Scan =======> CometScan CometScan - * | | | | - * Filter Filter CometFilter CometFilter - * | | | | - * Union CometUnion - * | | - * Project CometProject + * Scan Scan =======> CometScan CometScan \| | | | Filter Filter CometFilter CometFilter \| | | + * \| Union CometUnion \| | Project CometProject */ // spotless:on private def transform(plan: SparkPlan): SparkPlan = { @@ -191,7 +169,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { case scan: CometBatchScanExec if scan.nativeIcebergScanMetadata.isDefined => convertToComet(scan, CometIcebergNativeScan).getOrElse(scan) - case scan: CometBatchScanExec if scan.fileFormat == "csv" => + case scan: CometBatchScanExec if scan.fileFormat.isInstanceOf[CSVFileFormat] => convertToComet(scan, CometCsvNativeScanExec).getOrElse(scan) // Comet JVM + native scan for V1 and V2 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 e970c1b163..2e8bdcfa89 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -244,7 +244,7 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com CometBatchScanExec( scanExec.clone().asInstanceOf[BatchScanExec], runtimeFilters = scanExec.runtimeFilters, - fileFormat = "csv") + fileFormat = new CSVFileFormat) } else { withInfos(scanExec, fallbackReasons.toSet) } diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala index 5f814e1cc0..c83a89b626 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala @@ -144,12 +144,13 @@ object CometNativeScan extends CometOperatorSerde[CometScanExec] with Logging { var firstPartition: Option[PartitionedFile] = None val filePartitions = scan.getFilePartitions() - filePartitions.foreach { partition => + val filePartitionsProto = filePartitions.map { partition => if (firstPartition.isEmpty) { firstPartition = partition.files.headOption } - partition2Proto(partition, nativeScanBuilder, scan.relation.partitionSchema) + partition2Proto(partition, scan.relation.partitionSchema) } + nativeScanBuilder.addAllFilePartitions(filePartitionsProto.asJava) val partitionSchema = schema2Proto(scan.relation.partitionSchema.fields) val requiredSchema = schema2Proto(scan.requiredSchema.fields) @@ -203,39 +204,6 @@ object CometNativeScan extends CometOperatorSerde[CometScanExec] with Logging { } - private def partition2Proto( - partition: FilePartition, - nativeScanBuilder: OperatorOuterClass.NativeScan.Builder, - partitionSchema: StructType): Unit = { - val partitionBuilder = OperatorOuterClass.SparkFilePartition.newBuilder() - partition.files.foreach(file => { - // Process the partition values - val partitionValues = file.partitionValues - assert(partitionValues.numFields == partitionSchema.length) - val partitionVals = - partitionValues.toSeq(partitionSchema).zipWithIndex.map { case (value, i) => - val attr = partitionSchema(i) - val valueProto = exprToProto(Literal(value, attr.dataType), Seq.empty) - // In `CometScanRule`, we have already checked that all partition values are - // supported. So, we can safely use `get` here. - assert( - valueProto.isDefined, - s"Unsupported partition value: $value, type: ${attr.dataType}") - valueProto.get - } - - val fileBuilder = OperatorOuterClass.SparkPartitionedFile.newBuilder() - partitionVals.foreach(fileBuilder.addPartitionValues) - fileBuilder - .setFilePath(file.filePath.toString) - .setStart(file.start) - .setLength(file.length) - .setFileSize(file.fileSize) - partitionBuilder.addPartitionedFile(fileBuilder.build()) - }) - nativeScanBuilder.addFilePartitions(partitionBuilder.build()) - } - override def createExec(nativeOp: Operator, op: CometScanExec): CometNativeExec = { CometNativeScanExec(nativeOp, op.wrapped, op.session) } diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/package.scala b/spark/src/main/scala/org/apache/comet/serde/operator/package.scala index f407f52bfa..7b811d09e7 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/package.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/package.scala @@ -19,9 +19,11 @@ package org.apache.comet.serde -import org.apache.spark.sql.types.StructField +import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.execution.datasources.FilePartition +import org.apache.spark.sql.types.{StructField, StructType} -import org.apache.comet.serde.QueryPlanSerde.serializeDataType +import org.apache.comet.serde.QueryPlanSerde.{exprToProto, serializeDataType} package object operator { @@ -34,4 +36,35 @@ package object operator { fieldBuilder.build() } } + + def partition2Proto( + partition: FilePartition, + partitionSchema: StructType): OperatorOuterClass.SparkFilePartition = { + val partitionBuilder = OperatorOuterClass.SparkFilePartition.newBuilder() + partition.files.foreach(file => { + // Process the partition values + val partitionValues = file.partitionValues + assert(partitionValues.numFields == partitionSchema.length) + val partitionVals = + partitionValues.toSeq(partitionSchema).zipWithIndex.map { case (value, i) => + val attr = partitionSchema(i) + val valueProto = exprToProto(Literal(value, attr.dataType), Seq.empty) + // In `CometScanRule`, we have already checked that all partition values are + // supported. So, we can safely use `get` here. + assert( + valueProto.isDefined, + s"Unsupported partition value: $value, type: ${attr.dataType}") + valueProto.get + } + val fileBuilder = OperatorOuterClass.SparkPartitionedFile.newBuilder() + partitionVals.foreach(fileBuilder.addPartitionValues) + fileBuilder + .setFilePath(file.filePath.toString) + .setStart(file.start) + .setLength(file.length) + .setFileSize(file.fileSize) + partitionBuilder.addPartitionedFile(fileBuilder.build()) + }) + partitionBuilder.build() + } } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala index 5c0e4af51b..4115ded70d 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala @@ -44,7 +44,7 @@ case class CometBatchScanExec( wrapped: BatchScanExec, runtimeFilters: Seq[Expression], nativeIcebergScanMetadata: Option[CometIcebergNativeScanMetadata] = None, - fileFormat: String = "parquet") + fileFormat: FileFormat = ParquetFileFormat.asInstanceOf[FileFormat]) extends DataSourceV2ScanExecBase with CometPlan { def ordering: Option[Seq[SortOrder]] = wrapped.ordering diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala index 54845df87b..5e42153cb2 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala @@ -30,13 +30,11 @@ import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan import org.apache.spark.sql.util.CaseInsensitiveStringMap -import com.google.protobuf.ByteString - import org.apache.comet.{CometConf, ConfigEntry} import org.apache.comet.objectstore.NativeConfig -import org.apache.comet.serde.{CometOperatorSerde, Incompatible, OperatorOuterClass, SupportLevel} +import org.apache.comet.serde.{CometOperatorSerde, Compatible, OperatorOuterClass, SupportLevel} import org.apache.comet.serde.OperatorOuterClass.Operator -import org.apache.comet.serde.operator.schema2Proto +import org.apache.comet.serde.operator.{partition2Proto, schema2Proto} case class CometCsvNativeScanExec( override val nativeOp: Operator, @@ -64,7 +62,7 @@ object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { CometConf.COMET_CSV_V2_NATIVE_ENABLED) override def getSupportLevel(operator: CometBatchScanExec): SupportLevel = { - Incompatible() + Compatible() } override def convert( @@ -73,26 +71,26 @@ object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { childOp: Operator*): Option[Operator] = { val csvScanBuilder = OperatorOuterClass.CsvScan.newBuilder() val csvScan = op.wrapped.scan.asInstanceOf[CSVScan] - val columnPruning = op.session.sessionState.conf.csvColumnPruning - val timeZone = op.session.sessionState.conf.sessionLocalTimeZone + val sessionState = op.session.sessionState + val columnPruning = sessionState.conf.csvColumnPruning + val timeZone = sessionState.conf.sessionLocalTimeZone val filePartitions = op.inputPartitions.map(_.asInstanceOf[FilePartition]) val csvOptionsProto = csvOptions2Proto(csvScan.options, columnPruning, timeZone) - val schemaProto = schema2Proto(op.schema.fields) + val schemaProto = schema2Proto(csvScan.readDataSchema.fields) val partitionSchemaProto = schema2Proto(csvScan.readPartitionSchema.fields) - val partitionsProto = filePartitions.map(partition2Proto) - - val hadoopConf = op.session.sessionState - .newHadoopConfWithOptions(op.session.sparkContext.conf.getAll.toMap) - op.inputPartitions.headOption.foreach { partitionFile => - val objectStoreOptions = - NativeConfig.extractObjectStoreOptions( - hadoopConf, - partitionFile.asInstanceOf[FilePartition].files.head.pathUri) - objectStoreOptions.foreach { case (key, value) => - csvScanBuilder.putObjectStoreOptions(key, value) + val partitionsProto = filePartitions.map(partition2Proto(_, csvScan.readPartitionSchema)) + + val objectStoreOptions = filePartitions.headOption + .flatMap { partitionFile => + val hadoopConf = sessionState + .newHadoopConfWithOptions(op.session.sparkContext.conf.getAll.toMap) + partitionFile.files.headOption + .map(file => NativeConfig.extractObjectStoreOptions(hadoopConf, file.pathUri)) } - } + .getOrElse(Map.empty) + + csvScanBuilder.putAllObjectStoreOptions(objectStoreOptions.asJava) csvScanBuilder.setCsvOptions(csvOptionsProto) csvScanBuilder.addAllFilePartitions(partitionsProto.asJava) csvScanBuilder.addAllRequiredSchema(schemaProto.toIterable.asJava) @@ -121,18 +119,4 @@ object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { } csvOptionsBuilder.build() } - - private def partition2Proto(partition: FilePartition): OperatorOuterClass.SparkFilePartition = { - val partitionBuilder = OperatorOuterClass.SparkFilePartition.newBuilder() - partition.files.foreach { file => - val filePartitionBuilder = OperatorOuterClass.SparkPartitionedFile.newBuilder() - filePartitionBuilder - .setLength(file.length) - .setFilePath(file.filePath.toString) - .setStart(file.start) - .setFileSize(file.fileSize) - partitionBuilder.addPartitionedFile(filePartitionBuilder.build()) - } - partitionBuilder.build() - } } From dafa0deb6665bb7fedf52f71e409a8f3d120cf7f Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Tue, 30 Dec 2025 20:39:58 +0400 Subject: [PATCH 12/23] WIP --- .../sql/benchmark/CometNativeCsvReadBenchmark.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala index 1bd7d72d4d..ead0b8fd89 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala @@ -25,11 +25,18 @@ import scala.util.Random import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.benchmark.CometExecBenchmark.withSQLConf +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataTypes, StructType} import org.apache.comet.CometConf import org.apache.comet.testing.{CsvGenerator, FuzzDataGenerator, SchemaGenOptions} +/** + * Benchmark to measure Comet read performance. To run this benchmark: + * `SPARK_GENERATE_BENCHMARK_FILES=1 make + * benchmark-org.apache.spark.sql.benchmark.CometNativeCsvReadBenchmark` Results will be written + * to "spark/benchmarks/CometNativeCsvReadBenchmark-**results.txt". + */ object CometNativeCsvReadBenchmark extends CometBenchmarkBase { private def prepareCsvTable(dir: File, schema: StructType, numRows: Int): Unit = { @@ -58,13 +65,14 @@ object CometNativeCsvReadBenchmark extends CometBenchmarkBase { benchmark.addCase("Simple read") { _ => withSQLConf( CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", CometConf.COMET_CSV_V2_NATIVE_ENABLED.key -> "true", CometConf.COMET_EXPLAIN_FALLBACK_ENABLED.key -> "true", - "spark.sql.sources.useV1SourceList" -> "") { + SQLConf.USE_V1_SOURCE_LIST.key -> "") { spark.read .schema(schema) .csv(dir.getCanonicalPath) - .foreach(_ => ()) + .noop() } } benchmark.run() From 1809df8a274a0a360cd80272574f6ddfdb17f6b9 Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Thu, 1 Jan 2026 20:52:53 +0400 Subject: [PATCH 13/23] Work --- .../apache/comet/rules/CometExecRule.scala | 5 +- .../apache/comet/rules/CometScanRule.scala | 24 +++++++-- .../spark/sql/comet/CometBatchScanExec.scala | 5 +- .../sql/comet/CometCsvNativeScanExec.scala | 17 +++--- .../test/resources/test-data/csv-test-2.csv | 4 ++ .../apache/comet/csv/CometCsvReadSuite.scala | 52 +++++++++++++++++-- .../CometNativeCsvReadBenchmark.scala | 10 +++- 7 files changed, 91 insertions(+), 26 deletions(-) create mode 100644 spark/src/test/resources/test-data/csv-test-2.csv diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index d965de374e..e11cee0d48 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -51,9 +51,8 @@ import org.apache.comet.{CometConf, CometExplainInfo, ExtendedExplainInfo} import org.apache.comet.CometConf.{COMET_SPARK_TO_ARROW_ENABLED, COMET_SPARK_TO_ARROW_SUPPORTED_OPERATOR_LIST} import org.apache.comet.CometSparkSessionExtensions._ import org.apache.comet.rules.CometExecRule.allExecs -import org.apache.comet.serde.{CometOperatorSerde, Compatible, Incompatible, OperatorOuterClass, Unsupported} +import org.apache.comet.serde._ import org.apache.comet.serde.operator._ -import org.apache.comet.serde.operator.CometDataWritingCommand object CometExecRule { @@ -169,7 +168,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { case scan: CometBatchScanExec if scan.nativeIcebergScanMetadata.isDefined => convertToComet(scan, CometIcebergNativeScan).getOrElse(scan) - case scan: CometBatchScanExec if scan.fileFormat.isInstanceOf[CSVFileFormat] => + case scan: CometBatchScanExec if scan.wrapped.scan.isInstanceOf[CSVScan] => convertToComet(scan, CometCsvNativeScanExec).getOrElse(scan) // Comet JVM + native scan for V1 and V2 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 2e8bdcfa89..da4dc165d7 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -41,6 +41,7 @@ import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.comet.{CometConf, CometNativeException, DataTypeSupport} import org.apache.comet.CometConf._ @@ -240,11 +241,28 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com if (!partitionSchemaSupported) { fallbackReasons += s"Partition schema ${scan.readPartitionSchema} is not supported" } - if (schemaSupported && partitionSchemaSupported) { + val columnNameOfCorruptedRecords = + SQLConf.get.getConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD) + val hasNoCorruptedColumn = + !scan.readDataSchema.fieldNames.contains(columnNameOfCorruptedRecords) + if (!hasNoCorruptedColumn) { + fallbackReasons += "Comet doesn't support the processing of corrupted records in Spark" + } + val inferSchemaEnabled = scan.options.getBoolean("inferSchema", false) + if (inferSchemaEnabled) { + fallbackReasons += "Comet doesn't support inferSchema=true option" + } + val delimiter = scan.options.get("delimiter") + val isSingleCharDelimiter = delimiter.length == 1 + if (!isSingleCharDelimiter) { + fallbackReasons += s"Comet doesn't support delimiter: '$delimiter' " + + s"with more then one character" + } + if (schemaSupported && partitionSchemaSupported && hasNoCorruptedColumn + && !inferSchemaEnabled && isSingleCharDelimiter) { CometBatchScanExec( scanExec.clone().asInstanceOf[BatchScanExec], - runtimeFilters = scanExec.runtimeFilters, - fileFormat = new CSVFileFormat) + runtimeFilters = scanExec.runtimeFilters) } else { withInfos(scanExec, fallbackReasons.toSet) } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala index 4115ded70d..e4ccb4c63e 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometBatchScanExec.scala @@ -28,8 +28,6 @@ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.connector.read._ import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.datasources.FileFormat -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.execution.metric._ import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType} @@ -43,8 +41,7 @@ import org.apache.comet.iceberg.CometIcebergNativeScanMetadata case class CometBatchScanExec( wrapped: BatchScanExec, runtimeFilters: Seq[Expression], - nativeIcebergScanMetadata: Option[CometIcebergNativeScanMetadata] = None, - fileFormat: FileFormat = ParquetFileFormat.asInstanceOf[FileFormat]) + nativeIcebergScanMetadata: Option[CometIcebergNativeScanMetadata] = None) extends DataSourceV2ScanExecBase with CometPlan { def ordering: Option[Seq[SortOrder]] = wrapped.ordering diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala index 5e42153cb2..ba9a6bda31 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala @@ -28,7 +28,6 @@ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.datasources.FilePartition import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan -import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.comet.{CometConf, ConfigEntry} import org.apache.comet.objectstore.NativeConfig @@ -72,11 +71,13 @@ object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { val csvScanBuilder = OperatorOuterClass.CsvScan.newBuilder() val csvScan = op.wrapped.scan.asInstanceOf[CSVScan] val sessionState = op.session.sessionState - val columnPruning = sessionState.conf.csvColumnPruning - val timeZone = sessionState.conf.sessionLocalTimeZone - + val options = { + val columnPruning = sessionState.conf.csvColumnPruning + val timeZone = sessionState.conf.sessionLocalTimeZone + new CSVOptions(csvScan.options.asScala.toMap, columnPruning, timeZone) + } val filePartitions = op.inputPartitions.map(_.asInstanceOf[FilePartition]) - val csvOptionsProto = csvOptions2Proto(csvScan.options, columnPruning, timeZone) + val csvOptionsProto = csvOptions2Proto(options) val schemaProto = schema2Proto(csvScan.readDataSchema.fields) val partitionSchemaProto = schema2Proto(csvScan.readPartitionSchema.fields) val partitionsProto = filePartitions.map(partition2Proto(_, csvScan.readPartitionSchema)) @@ -102,12 +103,8 @@ object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { CometCsvNativeScanExec(nativeOp, op.output, op.wrapped, SerializedPlan(None)) } - private def csvOptions2Proto( - parameters: CaseInsensitiveStringMap, - columnPruning: Boolean, - timeZone: String): OperatorOuterClass.CsvOptions = { + private def csvOptions2Proto(options: CSVOptions): OperatorOuterClass.CsvOptions = { val csvOptionsBuilder = OperatorOuterClass.CsvOptions.newBuilder() - val options = new CSVOptions(parameters.asScala.toMap, columnPruning, timeZone) csvOptionsBuilder.setDelimiter(options.delimiter) csvOptionsBuilder.setHasHeader(options.headerFlag) csvOptionsBuilder.setQuote(options.quote.toString) diff --git a/spark/src/test/resources/test-data/csv-test-2.csv b/spark/src/test/resources/test-data/csv-test-2.csv new file mode 100644 index 0000000000..1c7c834f14 --- /dev/null +++ b/spark/src/test/resources/test-data/csv-test-2.csv @@ -0,0 +1,4 @@ +a,b,c +1,2,3 +4,5,6 +7,0,8 \ No newline at end of file diff --git a/spark/src/test/scala/org/apache/comet/csv/CometCsvReadSuite.scala b/spark/src/test/scala/org/apache/comet/csv/CometCsvReadSuite.scala index 817adfdcd1..39d3189a74 100644 --- a/spark/src/test/scala/org/apache/comet/csv/CometCsvReadSuite.scala +++ b/spark/src/test/scala/org/apache/comet/csv/CometCsvReadSuite.scala @@ -21,13 +21,15 @@ package org.apache.comet.csv import org.apache.spark.sql.CometTestBase import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.types.{IntegerType, StringType, StructType} import org.apache.comet.CometConf class CometCsvReadSuite extends CometTestBase { + private val TEST_CSV_PATH_NO_HEADER = "src/test/resources/test-data/csv-test-1.csv" + private val TEST_CSV_PATH_HAS_HEADER = "src/test/resources/test-data/csv-test-2.csv" - test("native csv read") { + test("Native csv read - with schema") { withSQLConf( CometConf.COMET_CSV_V2_NATIVE_ENABLED.key -> "true", SQLConf.USE_V1_SOURCE_LIST.key -> "") { @@ -35,13 +37,53 @@ class CometCsvReadSuite extends CometTestBase { .add("a", IntegerType) .add("b", IntegerType) .add("c", IntegerType) + val df = spark.read + .options(Map("header" -> "false", "delimiter" -> ",")) + .schema(schema) + .csv(TEST_CSV_PATH_NO_HEADER) + checkSparkAnswerAndOperator(df) + } + } + test("Native csv read - without schema") { + withSQLConf( + CometConf.COMET_CSV_V2_NATIVE_ENABLED.key -> "true", + SQLConf.USE_V1_SOURCE_LIST.key -> "") { val df = spark.read + .options(Map("header" -> "true", "delimiter" -> ",")) + .csv(TEST_CSV_PATH_HAS_HEADER) + checkSparkAnswerAndOperator(df) + } + } + + test("Native csv read - test fallback reasons") { + withSQLConf( + CometConf.COMET_CSV_V2_NATIVE_ENABLED.key -> "true", + SQLConf.USE_V1_SOURCE_LIST.key -> "") { + val columnNameOfCorruptedRecords = + SQLConf.get.getConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD) + val schema = new StructType() + .add("a", IntegerType) + .add("b", IntegerType) + .add("c", IntegerType) + .add(columnNameOfCorruptedRecords, StringType) + var df = spark.read .options(Map("header" -> "false", "delimiter" -> ",")) .schema(schema) - .csv("/Users/tendoo/Desktop/datafusion-comet/spark/src/test/resources/test-data/csv-test-1.csv") - df.explain(true) - df.show(false) + .csv(TEST_CSV_PATH_NO_HEADER) + checkSparkAnswerAndFallbackReason( + df, + "Comet doesn't support the processing of corrupted records in Spark") + df = spark.read + .options(Map("header" -> "false", "delimiter" -> ",", "inferSchema" -> "true")) + .csv(TEST_CSV_PATH_NO_HEADER) + checkSparkAnswerAndFallbackReason(df, "Comet doesn't support inferSchema=true option") + df = spark.read + .options(Map("header" -> "false", "delimiter" -> ",,")) + .csv(TEST_CSV_PATH_NO_HEADER) + checkSparkAnswerAndFallbackReason( + df, + "Comet doesn't support delimiter: ',,' with more then one character") } } } diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala index ead0b8fd89..f4d4ea7209 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala @@ -62,7 +62,15 @@ object CometNativeCsvReadBenchmark extends CometBenchmarkBase { DataTypes.DateType, DataTypes.StringType))) prepareCsvTable(dir, schema, numRows) - benchmark.addCase("Simple read") { _ => + benchmark.addCase("Simple csv v2 read - spark") { _ => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") { + spark.read + .schema(schema) + .csv(dir.getCanonicalPath) + .noop() + } + } + benchmark.addCase("Simple csv v2 read - comet native") { _ => withSQLConf( CometConf.COMET_ENABLED.key -> "true", CometConf.COMET_EXEC_ENABLED.key -> "true", From d8c7760f43a78db532ca372ee1d5d17dddf069a3 Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Tue, 6 Jan 2026 19:15:50 +0400 Subject: [PATCH 14/23] Final approach --- .../apache/comet/rules/CometScanRule.scala | 26 +-- .../apache/comet/testing/CsvGenerator.scala | 40 ---- .../sql/comet/CometCsvNativeScanExec.scala | 26 ++- ...te.scala => CometCsvNativeReadSuite.scala} | 6 +- .../CometNativeCsvReadBenchmark.scala | 201 +++++++++++++----- 5 files changed, 186 insertions(+), 113 deletions(-) delete mode 100644 spark/src/main/scala/org/apache/comet/testing/CsvGenerator.scala rename spark/src/test/scala/org/apache/comet/csv/{CometCsvReadSuite.scala => CometCsvNativeReadSuite.scala} (95%) 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 da4dc165d7..83d425bf59 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -241,25 +241,25 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com if (!partitionSchemaSupported) { fallbackReasons += s"Partition schema ${scan.readPartitionSchema} is not supported" } - val columnNameOfCorruptedRecords = + val corruptedRecordsColumnName = SQLConf.get.getConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD) - val hasNoCorruptedColumn = - !scan.readDataSchema.fieldNames.contains(columnNameOfCorruptedRecords) - if (!hasNoCorruptedColumn) { - fallbackReasons += "Comet doesn't support the processing of corrupted records in Spark" + val containsCorruptedRecordsColumn = + !scan.readDataSchema.fieldNames.contains(corruptedRecordsColumnName) + if (!containsCorruptedRecordsColumn) { + fallbackReasons += "Comet doesn't support the processing of corrupted records" } - val inferSchemaEnabled = scan.options.getBoolean("inferSchema", false) - if (inferSchemaEnabled) { + val isInferSchemaEnabled = scan.options.getBoolean("inferSchema", false) + if (isInferSchemaEnabled) { fallbackReasons += "Comet doesn't support inferSchema=true option" } val delimiter = scan.options.get("delimiter") - val isSingleCharDelimiter = delimiter.length == 1 - if (!isSingleCharDelimiter) { - fallbackReasons += s"Comet doesn't support delimiter: '$delimiter' " + - s"with more then one character" + val isSingleCharacterDelimiter = delimiter.length == 1 + if (!isSingleCharacterDelimiter) { + fallbackReasons += + s"Comet supports only single-character delimiters, but got: '$delimiter'" } - if (schemaSupported && partitionSchemaSupported && hasNoCorruptedColumn - && !inferSchemaEnabled && isSingleCharDelimiter) { + if (schemaSupported && partitionSchemaSupported && containsCorruptedRecordsColumn + && !isInferSchemaEnabled && isSingleCharacterDelimiter) { CometBatchScanExec( scanExec.clone().asInstanceOf[BatchScanExec], runtimeFilters = scanExec.runtimeFilters) diff --git a/spark/src/main/scala/org/apache/comet/testing/CsvGenerator.scala b/spark/src/main/scala/org/apache/comet/testing/CsvGenerator.scala deleted file mode 100644 index 8825aeec8e..0000000000 --- a/spark/src/main/scala/org/apache/comet/testing/CsvGenerator.scala +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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.testing - -import scala.util.Random - -import org.apache.spark.sql.{SaveMode, SparkSession} -import org.apache.spark.sql.types.StructType - -object CsvGenerator { - - def makeCsvFile( - r: Random, - spark: SparkSession, - schema: StructType, - filename: String, - numRows: Int): Unit = { - val options = DataGenOptions(allowNull = false, generateNaN = false) - val df = FuzzDataGenerator.generateDataFrame(r, spark, schema, numRows, options) - df.write.mode(SaveMode.Overwrite).csv(filename) - } - -} diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala index ba9a6bda31..1477401306 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala @@ -29,12 +29,17 @@ import org.apache.spark.sql.execution.datasources.FilePartition import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan +import com.google.common.base.Objects + import org.apache.comet.{CometConf, ConfigEntry} import org.apache.comet.objectstore.NativeConfig -import org.apache.comet.serde.{CometOperatorSerde, Compatible, OperatorOuterClass, SupportLevel} +import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.operator.{partition2Proto, schema2Proto} +/* + * Native CSV scan operator that delegates file reading to datafusion. + */ case class CometCsvNativeScanExec( override val nativeOp: Operator, override val output: Seq[Attribute], @@ -53,6 +58,21 @@ case class CometCsvNativeScanExec( override protected def doCanonicalize(): SparkPlan = { CometCsvNativeScanExec(nativeOp, output, originalPlan, serializedPlanOpt) } + + override def equals(obj: Any): Boolean = { + obj match { + case other: CometCsvNativeScanExec => + this.output == other.output && + this.serializedPlanOpt == other.serializedPlanOpt && + this.originalPlan == other.originalPlan + case _ => + false + } + } + + override def hashCode(): Int = { + Objects.hashCode(output, serializedPlanOpt, originalPlan) + } } object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { @@ -60,10 +80,6 @@ object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { override def enabledConfig: Option[ConfigEntry[Boolean]] = Some( CometConf.COMET_CSV_V2_NATIVE_ENABLED) - override def getSupportLevel(operator: CometBatchScanExec): SupportLevel = { - Compatible() - } - override def convert( op: CometBatchScanExec, builder: Operator.Builder, diff --git a/spark/src/test/scala/org/apache/comet/csv/CometCsvReadSuite.scala b/spark/src/test/scala/org/apache/comet/csv/CometCsvNativeReadSuite.scala similarity index 95% rename from spark/src/test/scala/org/apache/comet/csv/CometCsvReadSuite.scala rename to spark/src/test/scala/org/apache/comet/csv/CometCsvNativeReadSuite.scala index 39d3189a74..e9a18a18dc 100644 --- a/spark/src/test/scala/org/apache/comet/csv/CometCsvReadSuite.scala +++ b/spark/src/test/scala/org/apache/comet/csv/CometCsvNativeReadSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.types.{IntegerType, StringType, StructType} import org.apache.comet.CometConf -class CometCsvReadSuite extends CometTestBase { +class CometCsvNativeReadSuite extends CometTestBase { private val TEST_CSV_PATH_NO_HEADER = "src/test/resources/test-data/csv-test-1.csv" private val TEST_CSV_PATH_HAS_HEADER = "src/test/resources/test-data/csv-test-2.csv" @@ -73,7 +73,7 @@ class CometCsvReadSuite extends CometTestBase { .csv(TEST_CSV_PATH_NO_HEADER) checkSparkAnswerAndFallbackReason( df, - "Comet doesn't support the processing of corrupted records in Spark") + "Comet doesn't support the processing of corrupted records") df = spark.read .options(Map("header" -> "false", "delimiter" -> ",", "inferSchema" -> "true")) .csv(TEST_CSV_PATH_NO_HEADER) @@ -83,7 +83,7 @@ class CometCsvReadSuite extends CometTestBase { .csv(TEST_CSV_PATH_NO_HEADER) checkSparkAnswerAndFallbackReason( df, - "Comet doesn't support delimiter: ',,' with more then one character") + "Comet supports only single-character delimiters, but got: ',,'") } } } diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala index f4d4ea7209..4abe98f0c2 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala @@ -19,71 +19,168 @@ package org.apache.spark.sql.benchmark -import java.io.File - -import scala.util.Random - import org.apache.spark.benchmark.Benchmark -import org.apache.spark.sql.benchmark.CometExecBenchmark.withSQLConf +import org.apache.spark.sql.benchmark.CometNativeCsvReadBenchmark.TPCHSchemas._ +import org.apache.spark.sql.execution.benchmark.TPCDSQueryBenchmarkArguments import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{DataTypes, StructType} +import org.apache.spark.sql.types._ import org.apache.comet.CometConf -import org.apache.comet.testing.{CsvGenerator, FuzzDataGenerator, SchemaGenOptions} /** - * Benchmark to measure Comet read performance. To run this benchmark: + * @param tableName + * Name of the TPC-H table. Must match one of the standard table names: region, nation, part, + * supplier, partsupp, customer, orders, lineitem. + * + * @param schema + * Table data structure in Spark StructType format. + */ +case class NativeCsvReadConfig(tableName: String, schema: StructType) + +/** + * Benchmark to measure Comet csv read performance. To run this benchmark: * `SPARK_GENERATE_BENCHMARK_FILES=1 make - * benchmark-org.apache.spark.sql.benchmark.CometNativeCsvReadBenchmark` Results will be written - * to "spark/benchmarks/CometNativeCsvReadBenchmark-**results.txt". + * benchmark-org.apache.spark.sql.benchmark.CometNativeCsvReadBenchmark -- --data-location + * /tmp/tpcds` Results will be written to + * "spark/benchmarks/CometNativeCsvReadBenchmark-**results.txt". */ object CometNativeCsvReadBenchmark extends CometBenchmarkBase { - private def prepareCsvTable(dir: File, schema: StructType, numRows: Int): Unit = { - val random = new Random(42) - CsvGenerator.makeCsvFile(random, spark, schema, dir.getCanonicalPath, numRows) + private def runNativeCsvBenchmark( + dataLocation: String, + tableName: String, + schema: StructType, + valuesPerPartition: Int, + numIters: Int): Unit = { + val benchmark = + new Benchmark(s"Native csv read - `$tableName` table", valuesPerPartition, output = output) + val filePath = s"$dataLocation/$tableName.csv" + benchmark.addCase("Spark", numIters) { _ => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") { + spark.read + .schema(schema) + .options(Map("header" -> "true", "delimiter" -> ",")) + .csv(filePath) + .noop() + } + } + benchmark.addCase("Native", numIters) { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_CSV_V2_NATIVE_ENABLED.key -> "true", + SQLConf.USE_V1_SOURCE_LIST.key -> "") { + spark.read + .schema(schema) + .options(Map("header" -> "true", "delimiter" -> ",")) + .csv(filePath) + .noop() + } + } + benchmark.run() } + private val testCases = Seq( + /* NativeCsvReadConfig("orders", ordersSchema), + NativeCsvReadConfig("region", regionSchema),*/ + NativeCsvReadConfig("nation", nationSchema) + /*NativeCsvReadConfig("part", partSchema), + NativeCsvReadConfig("supplier", supplierSchema), + NativeCsvReadConfig("partsupp", partsuppSchema), + NativeCsvReadConfig("customer", customerSchema), + NativeCsvReadConfig("lineitem", lineitemSchema)*/ ) + override def runCometBenchmark(args: Array[String]): Unit = { - val numRows = 2000000 - val benchmark = new Benchmark(s"Native csv read - $numRows rows", numRows, output = output) - withTempPath { dir => - val schema = FuzzDataGenerator.generateSchema( - SchemaGenOptions(primitiveTypes = Seq( - DataTypes.BooleanType, - DataTypes.ByteType, - DataTypes.ShortType, - DataTypes.IntegerType, - DataTypes.LongType, - DataTypes.FloatType, - DataTypes.DoubleType, - DataTypes.createDecimalType(10, 2), - DataTypes.createDecimalType(36, 18), - DataTypes.DateType, - DataTypes.StringType))) - prepareCsvTable(dir, schema, numRows) - benchmark.addCase("Simple csv v2 read - spark") { _ => - withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") { - spark.read - .schema(schema) - .csv(dir.getCanonicalPath) - .noop() - } - } - benchmark.addCase("Simple csv v2 read - comet native") { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_CSV_V2_NATIVE_ENABLED.key -> "true", - CometConf.COMET_EXPLAIN_FALLBACK_ENABLED.key -> "true", - SQLConf.USE_V1_SOURCE_LIST.key -> "") { - spark.read - .schema(schema) - .csv(dir.getCanonicalPath) - .noop() - } - } - benchmark.run() + val benchmarkArgs = new TPCDSQueryBenchmarkArguments(args) + val valuesPerPartition = 1024 * 1024 * 2 + val numIters = 1 + testCases.foreach { config => + runNativeCsvBenchmark( + benchmarkArgs.dataLocation, + config.tableName, + config.schema, + valuesPerPartition, + numIters) } } + + object TPCHSchemas { + + val regionSchema: StructType = new StructType() + .add("r_regionkey", IntegerType, nullable = true) + .add("r_name", StringType, nullable = true) + .add("r_comment", StringType, nullable = true) + + val nationSchema: StructType = new StructType() + .add("n_nationkey", IntegerType, nullable = true) + .add("n_name", StringType, nullable = true) + .add("n_regionkey", IntegerType, nullable = true) + .add("n_comment", StringType, nullable = true) + + val partSchema: StructType = new StructType() + .add("p_partkey", IntegerType, nullable = true) + .add("p_name", StringType, nullable = true) + .add("p_mfgr", StringType, nullable = true) + .add("p_brand", StringType, nullable = true) + .add("p_type", StringType, nullable = true) + .add("p_size", IntegerType, nullable = true) + .add("p_container", StringType, nullable = true) + .add("p_retailprice", DoubleType, nullable = true) + .add("p_comment", StringType, nullable = true) + + val supplierSchema: StructType = new StructType() + .add("s_suppkey", IntegerType, nullable = true) + .add("s_name", StringType, nullable = true) + .add("s_address", StringType, nullable = true) + .add("s_nationkey", IntegerType, nullable = true) + .add("s_phone", StringType, nullable = true) + .add("s_acctbal", DoubleType, nullable = true) + .add("s_comment", StringType, nullable = true) + + val partsuppSchema: StructType = new StructType() + .add("ps_partkey", IntegerType, nullable = true) + .add("ps_suppkey", IntegerType, nullable = true) + .add("ps_availqty", IntegerType, nullable = true) + .add("ps_supplycost", DoubleType, nullable = true) + .add("ps_comment", StringType, nullable = true) + + val customerSchema: StructType = new StructType() + .add("c_custkey", IntegerType, nullable = true) + .add("c_name", StringType, nullable = true) + .add("c_address", StringType, nullable = true) + .add("c_nationkey", IntegerType, nullable = true) + .add("c_phone", StringType, nullable = true) + .add("c_acctbal", DoubleType, nullable = true) + .add("c_mktsegment", StringType, nullable = true) + .add("c_comment", StringType, nullable = true) + + val ordersSchema: StructType = new StructType() + .add("o_orderkey", IntegerType, nullable = true) + .add("o_custkey", IntegerType, nullable = true) + .add("o_orderstatus", StringType, nullable = true) + .add("o_totalprice", DoubleType, nullable = true) + .add("o_orderdate", DateType, nullable = true) + .add("o_orderpriority", StringType, nullable = true) + .add("o_clerk", StringType, nullable = true) + .add("o_shippriority", IntegerType, nullable = true) + .add("o_comment", StringType, nullable = true) + + val lineitemSchema: StructType = new StructType() + .add("l_orderkey", IntegerType, nullable = true) + .add("l_partkey", IntegerType, nullable = true) + .add("l_suppkey", IntegerType, nullable = true) + .add("l_linenumber", IntegerType, nullable = true) + .add("l_quantity", IntegerType, nullable = true) + .add("l_extendedprice", DoubleType, nullable = true) + .add("l_discount", DoubleType, nullable = true) + .add("l_tax", DoubleType, nullable = true) + .add("l_returnflag", StringType, nullable = true) + .add("l_linestatus", StringType, nullable = true) + .add("l_shipdate", DateType, nullable = true) + .add("l_commitdate", DateType, nullable = true) + .add("l_receiptdate", DateType, nullable = true) + .add("l_shipinstruct", StringType, nullable = true) + .add("l_shipmode", StringType, nullable = true) + .add("l_comment", StringType, nullable = true) + } } From 88aeb33709ee51773c13739338bc627eb1cfa9cd Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Tue, 6 Jan 2026 19:18:11 +0400 Subject: [PATCH 15/23] Fix workflows --- .github/workflows/pr_build_linux.yml | 3 +++ .github/workflows/pr_build_macos.yml | 3 +++ 2 files changed, 6 insertions(+) diff --git a/.github/workflows/pr_build_linux.yml b/.github/workflows/pr_build_linux.yml index e3b0e40566..aa6f955ab3 100644 --- a/.github/workflows/pr_build_linux.yml +++ b/.github/workflows/pr_build_linux.yml @@ -127,6 +127,9 @@ jobs: org.apache.spark.sql.comet.ParquetEncryptionITCase org.apache.comet.exec.CometNativeReaderSuite org.apache.comet.CometIcebergNativeSuite + - name: "csv" + value: | + org.apache.comet.csv.CometCsvNativeReadSuite - name: "exec" value: | org.apache.comet.exec.CometAggregateSuite diff --git a/.github/workflows/pr_build_macos.yml b/.github/workflows/pr_build_macos.yml index 0ad40c1932..996831c7a2 100644 --- a/.github/workflows/pr_build_macos.yml +++ b/.github/workflows/pr_build_macos.yml @@ -90,6 +90,9 @@ jobs: org.apache.spark.sql.comet.ParquetEncryptionITCase org.apache.comet.exec.CometNativeReaderSuite org.apache.comet.CometIcebergNativeSuite + - name: "csv" + value: | + org.apache.comet.csv.CometCsvNativeReadSuite - name: "exec" value: | org.apache.comet.exec.CometAggregateSuite From b2a0c28de44c43e93491223428dbccbb52e8d761 Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Tue, 6 Jan 2026 19:46:11 +0400 Subject: [PATCH 16/23] Fix fmt --- .../apache/comet/rules/CometExecRule.scala | 66 ++++++++++++------- 1 file changed, 44 insertions(+), 22 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index e11cee0d48..9c23b1be68 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -119,41 +119,63 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { /** * Tries to transform a Spark physical plan into a Comet plan. * - * This rule traverses bottom-up from the original Spark plan and for each plan node, there are - * a few cases to consider: + * This rule traverses bottom-up from the original Spark plan and for each plan node, there + * are a few cases to consider: * - * 1. The child(ren) of the current node `p` cannot be converted to native In this case, we'll - * simply return the original Spark plan, since Comet native execution cannot start from an - * arbitrary Spark operator (unless it is special node such as scan or sink such as shuffle - * exchange, union etc., which are wrapped by `CometScanWrapper` and `CometSinkPlaceHolder` - * respectively). + * 1. The child(ren) of the current node `p` cannot be converted to native + * In this case, we'll simply return the original Spark plan, since Comet native + * execution cannot start from an arbitrary Spark operator (unless it is special node + * such as scan or sink such as shuffle exchange, union etc., which are wrapped by + * `CometScanWrapper` and `CometSinkPlaceHolder` respectively). * - * 2. The child(ren) of the current node `p` can be converted to native There are two sub-cases - * for this scenario: 1) This node `p` can also be converted to native. In this case, we'll - * create a new native Comet operator for `p` and connect it with its previously converted - * child(ren); 2) This node `p` cannot be converted to native. In this case, similar to 1) - * above, we simply return `p` as it is. Its child(ren) would still be native Comet operators. + * 2. The child(ren) of the current node `p` can be converted to native + * There are two sub-cases for this scenario: 1) This node `p` can also be converted to + * native. In this case, we'll create a new native Comet operator for `p` and connect it with + * its previously converted child(ren); 2) This node `p` cannot be converted to native. In + * this case, similar to 1) above, we simply return `p` as it is. Its child(ren) would still + * be native Comet operators. * * After this rule finishes, we'll do another pass on the final plan to convert all adjacent - * Comet native operators into a single native execution block. Please see where `convertBlock` - * is called below. + * Comet native operators into a single native execution block. Please see where + * `convertBlock` is called below. * * Here are a few examples: * - * Scan ======> CometScan \| | Filter CometFilter \| | HashAggregate CometHashAggregate \| | - * Exchange CometExchange \| | HashAggregate CometHashAggregate \| | UnsupportedOperator - * UnsupportedOperator + * Scan ======> CometScan + * | | + * Filter CometFilter + * | | + * HashAggregate CometHashAggregate + * | | + * Exchange CometExchange + * | | + * HashAggregate CometHashAggregate + * | | + * UnsupportedOperator UnsupportedOperator * * Native execution doesn't necessarily have to start from `CometScan`: * - * Scan =======> CometScan \| | UnsupportedOperator UnsupportedOperator \| | HashAggregate - * HashAggregate \| | Exchange CometExchange \| | HashAggregate CometHashAggregate \| | - * UnsupportedOperator UnsupportedOperator + * Scan =======> CometScan + * | | + * UnsupportedOperator UnsupportedOperator + * | | + * HashAggregate HashAggregate + * | | + * Exchange CometExchange + * | | + * HashAggregate CometHashAggregate + * | | + * UnsupportedOperator UnsupportedOperator * * A sink can also be Comet operators other than `CometExchange`, for instance `CometUnion`: * - * Scan Scan =======> CometScan CometScan \| | | | Filter Filter CometFilter CometFilter \| | | - * \| Union CometUnion \| | Project CometProject + * Scan Scan =======> CometScan CometScan + * | | | | + * Filter Filter CometFilter CometFilter + * | | | | + * Union CometUnion + * | | + * Project CometProject */ // spotless:on private def transform(plan: SparkPlan): SparkPlan = { From ba98e37d39234602fe5337db59475d4902a8724c Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Tue, 6 Jan 2026 20:02:04 +0400 Subject: [PATCH 17/23] Fix params --- .../src/test/scala/org/apache/spark/sql/CometTestBase.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala index 0093e62a87..7dba24bff7 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala @@ -67,13 +67,12 @@ abstract class CometTestBase protected def sparkConf: SparkConf = { val conf = new SparkConf() conf.set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName) - conf.set("spark.ui.enabled", "true") + conf.set("spark.ui.enabled", "false") conf.set(SQLConf.SHUFFLE_PARTITIONS, 10) // reduce parallelism in tests conf.set(SQLConf.ANSI_ENABLED.key, "false") conf.set(SHUFFLE_MANAGER, shuffleManager) conf.set(MEMORY_OFFHEAP_ENABLED.key, "true") - conf.set(MEMORY_OFFHEAP_SIZE.key, "1200m") - conf.set("spark.driver.memory", "2g") + conf.set(MEMORY_OFFHEAP_SIZE.key, "2g") conf.set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "1g") conf.set(SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key, "1g") conf.set(CometConf.COMET_ENABLED.key, "true") From cd449c5a432df140a0e66ce688ac2dee0722c2be Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Tue, 6 Jan 2026 20:02:51 +0400 Subject: [PATCH 18/23] Fix tests --- .../sql/benchmark/CometNativeCsvReadBenchmark.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala index 4abe98f0c2..d3a0a4f34a 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala @@ -81,14 +81,14 @@ object CometNativeCsvReadBenchmark extends CometBenchmarkBase { } private val testCases = Seq( - /* NativeCsvReadConfig("orders", ordersSchema), - NativeCsvReadConfig("region", regionSchema),*/ - NativeCsvReadConfig("nation", nationSchema) - /*NativeCsvReadConfig("part", partSchema), + NativeCsvReadConfig("orders", ordersSchema), + NativeCsvReadConfig("region", regionSchema), + NativeCsvReadConfig("nation", nationSchema), + NativeCsvReadConfig("part", partSchema), NativeCsvReadConfig("supplier", supplierSchema), NativeCsvReadConfig("partsupp", partsuppSchema), NativeCsvReadConfig("customer", customerSchema), - NativeCsvReadConfig("lineitem", lineitemSchema)*/ ) + NativeCsvReadConfig("lineitem", lineitemSchema)) override def runCometBenchmark(args: Array[String]): Unit = { val benchmarkArgs = new TPCDSQueryBenchmarkArguments(args) From a1801c1cdc565a81c07971e3d594a61b32f5b66d Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Tue, 6 Jan 2026 20:07:07 +0400 Subject: [PATCH 19/23] Fix rust fmt --- native/core/src/execution/operators/mod.rs | 2 +- native/core/src/execution/planner.rs | 10 ++++++---- native/core/src/execution/planner/operator_registry.rs | 2 +- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/native/core/src/execution/operators/mod.rs b/native/core/src/execution/operators/mod.rs index caefaeaffb..07ee995367 100644 --- a/native/core/src/execution/operators/mod.rs +++ b/native/core/src/execution/operators/mod.rs @@ -31,9 +31,9 @@ pub use expand::ExpandExec; mod iceberg_scan; mod parquet_writer; pub use parquet_writer::ParquetWriterExec; +mod csv_scan; pub mod projection; mod scan; -mod csv_scan; pub use csv_scan::init_csv_datasource_exec; /// Error returned during executing operators. diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 3d72306118..06af03cbe0 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -21,8 +21,8 @@ pub mod expression_registry; pub mod macros; pub mod operator_registry; -use crate::execution::operators::IcebergScanExec; use crate::execution::operators::init_csv_datasource_exec; +use crate::execution::operators::IcebergScanExec; use crate::{ errors::ExpressionError, execution::{ @@ -1123,8 +1123,10 @@ impl PhysicalPlanner { )) } OpStruct::CsvScan(scan) => { - let data_schema = convert_spark_types_to_arrow_schema(scan.required_schema.as_slice()); - let partition_schema = convert_spark_types_to_arrow_schema(scan.partition_schema.as_slice()); + let data_schema = + convert_spark_types_to_arrow_schema(scan.required_schema.as_slice()); + let partition_schema = + convert_spark_types_to_arrow_schema(scan.partition_schema.as_slice()); let object_store_options: HashMap = scan .object_store_options .iter() @@ -1149,7 +1151,7 @@ impl PhysicalPlanner { file_groups, data_schema, Some(partition_schema), - &scan.csv_options.clone().unwrap() + &scan.csv_options.clone().unwrap(), )?; Ok(( vec![], diff --git a/native/core/src/execution/planner/operator_registry.rs b/native/core/src/execution/planner/operator_registry.rs index fe32e4a5cf..b34a80df95 100644 --- a/native/core/src/execution/planner/operator_registry.rs +++ b/native/core/src/execution/planner/operator_registry.rs @@ -152,6 +152,6 @@ fn get_operator_type(spark_operator: &Operator) -> Option { OpStruct::HashJoin(_) => Some(OperatorType::HashJoin), OpStruct::Window(_) => Some(OperatorType::Window), OpStruct::Explode(_) => None, // Not yet in OperatorType enum - OpStruct::CsvScan(_) => Some(OperatorType::CsvScan) + OpStruct::CsvScan(_) => Some(OperatorType::CsvScan), } } From 65251eb40b003101d2307cb1ef094e2df4adae34 Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Tue, 6 Jan 2026 21:57:05 +0400 Subject: [PATCH 20/23] Fix fmt --- .../src/main/scala/org/apache/comet/rules/CometScanRule.scala | 2 -- .../org/apache/comet/serde/operator/CometNativeScan.scala | 3 +-- 2 files changed, 1 insertion(+), 4 deletions(-) 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 83d425bf59..3d095f9409 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -35,13 +35,11 @@ import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.getExistenceDefa import org.apache.spark.sql.comet.{CometBatchScanExec, CometScanExec} import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.datasources.HadoopFsRelation -import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.comet.{CometConf, CometNativeException, DataTypeSupport} import org.apache.comet.CometConf._ diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala index c83a89b626..b7909b67cb 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometNativeScan.scala @@ -27,9 +27,8 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, PlanExpre import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns.getExistenceDefaultValues import org.apache.spark.sql.comet.{CometNativeExec, CometNativeScanExec, CometScanExec} import org.apache.spark.sql.execution.FileSourceScanExec -import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile} +import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{StructField, StructType} import org.apache.comet.{CometConf, ConfigEntry} import org.apache.comet.CometConf.COMET_EXEC_ENABLED From a21ae077f94e22f08a39ed29df5c59e4631523c8 Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Sun, 11 Jan 2026 20:06:38 +0400 Subject: [PATCH 21/23] Fix tests --- .../CometNativeCsvReadBenchmark.scala | 186 ------------------ 1 file changed, 186 deletions(-) delete mode 100644 spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala deleted file mode 100644 index d3a0a4f34a..0000000000 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometNativeCsvReadBenchmark.scala +++ /dev/null @@ -1,186 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.spark.sql.benchmark - -import org.apache.spark.benchmark.Benchmark -import org.apache.spark.sql.benchmark.CometNativeCsvReadBenchmark.TPCHSchemas._ -import org.apache.spark.sql.execution.benchmark.TPCDSQueryBenchmarkArguments -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types._ - -import org.apache.comet.CometConf - -/** - * @param tableName - * Name of the TPC-H table. Must match one of the standard table names: region, nation, part, - * supplier, partsupp, customer, orders, lineitem. - * - * @param schema - * Table data structure in Spark StructType format. - */ -case class NativeCsvReadConfig(tableName: String, schema: StructType) - -/** - * Benchmark to measure Comet csv read performance. To run this benchmark: - * `SPARK_GENERATE_BENCHMARK_FILES=1 make - * benchmark-org.apache.spark.sql.benchmark.CometNativeCsvReadBenchmark -- --data-location - * /tmp/tpcds` Results will be written to - * "spark/benchmarks/CometNativeCsvReadBenchmark-**results.txt". - */ -object CometNativeCsvReadBenchmark extends CometBenchmarkBase { - - private def runNativeCsvBenchmark( - dataLocation: String, - tableName: String, - schema: StructType, - valuesPerPartition: Int, - numIters: Int): Unit = { - val benchmark = - new Benchmark(s"Native csv read - `$tableName` table", valuesPerPartition, output = output) - val filePath = s"$dataLocation/$tableName.csv" - benchmark.addCase("Spark", numIters) { _ => - withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") { - spark.read - .schema(schema) - .options(Map("header" -> "true", "delimiter" -> ",")) - .csv(filePath) - .noop() - } - } - benchmark.addCase("Native", numIters) { _ => - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_ENABLED.key -> "true", - CometConf.COMET_CSV_V2_NATIVE_ENABLED.key -> "true", - SQLConf.USE_V1_SOURCE_LIST.key -> "") { - spark.read - .schema(schema) - .options(Map("header" -> "true", "delimiter" -> ",")) - .csv(filePath) - .noop() - } - } - benchmark.run() - } - - private val testCases = Seq( - NativeCsvReadConfig("orders", ordersSchema), - NativeCsvReadConfig("region", regionSchema), - NativeCsvReadConfig("nation", nationSchema), - NativeCsvReadConfig("part", partSchema), - NativeCsvReadConfig("supplier", supplierSchema), - NativeCsvReadConfig("partsupp", partsuppSchema), - NativeCsvReadConfig("customer", customerSchema), - NativeCsvReadConfig("lineitem", lineitemSchema)) - - override def runCometBenchmark(args: Array[String]): Unit = { - val benchmarkArgs = new TPCDSQueryBenchmarkArguments(args) - val valuesPerPartition = 1024 * 1024 * 2 - val numIters = 1 - testCases.foreach { config => - runNativeCsvBenchmark( - benchmarkArgs.dataLocation, - config.tableName, - config.schema, - valuesPerPartition, - numIters) - } - } - - object TPCHSchemas { - - val regionSchema: StructType = new StructType() - .add("r_regionkey", IntegerType, nullable = true) - .add("r_name", StringType, nullable = true) - .add("r_comment", StringType, nullable = true) - - val nationSchema: StructType = new StructType() - .add("n_nationkey", IntegerType, nullable = true) - .add("n_name", StringType, nullable = true) - .add("n_regionkey", IntegerType, nullable = true) - .add("n_comment", StringType, nullable = true) - - val partSchema: StructType = new StructType() - .add("p_partkey", IntegerType, nullable = true) - .add("p_name", StringType, nullable = true) - .add("p_mfgr", StringType, nullable = true) - .add("p_brand", StringType, nullable = true) - .add("p_type", StringType, nullable = true) - .add("p_size", IntegerType, nullable = true) - .add("p_container", StringType, nullable = true) - .add("p_retailprice", DoubleType, nullable = true) - .add("p_comment", StringType, nullable = true) - - val supplierSchema: StructType = new StructType() - .add("s_suppkey", IntegerType, nullable = true) - .add("s_name", StringType, nullable = true) - .add("s_address", StringType, nullable = true) - .add("s_nationkey", IntegerType, nullable = true) - .add("s_phone", StringType, nullable = true) - .add("s_acctbal", DoubleType, nullable = true) - .add("s_comment", StringType, nullable = true) - - val partsuppSchema: StructType = new StructType() - .add("ps_partkey", IntegerType, nullable = true) - .add("ps_suppkey", IntegerType, nullable = true) - .add("ps_availqty", IntegerType, nullable = true) - .add("ps_supplycost", DoubleType, nullable = true) - .add("ps_comment", StringType, nullable = true) - - val customerSchema: StructType = new StructType() - .add("c_custkey", IntegerType, nullable = true) - .add("c_name", StringType, nullable = true) - .add("c_address", StringType, nullable = true) - .add("c_nationkey", IntegerType, nullable = true) - .add("c_phone", StringType, nullable = true) - .add("c_acctbal", DoubleType, nullable = true) - .add("c_mktsegment", StringType, nullable = true) - .add("c_comment", StringType, nullable = true) - - val ordersSchema: StructType = new StructType() - .add("o_orderkey", IntegerType, nullable = true) - .add("o_custkey", IntegerType, nullable = true) - .add("o_orderstatus", StringType, nullable = true) - .add("o_totalprice", DoubleType, nullable = true) - .add("o_orderdate", DateType, nullable = true) - .add("o_orderpriority", StringType, nullable = true) - .add("o_clerk", StringType, nullable = true) - .add("o_shippriority", IntegerType, nullable = true) - .add("o_comment", StringType, nullable = true) - - val lineitemSchema: StructType = new StructType() - .add("l_orderkey", IntegerType, nullable = true) - .add("l_partkey", IntegerType, nullable = true) - .add("l_suppkey", IntegerType, nullable = true) - .add("l_linenumber", IntegerType, nullable = true) - .add("l_quantity", IntegerType, nullable = true) - .add("l_extendedprice", DoubleType, nullable = true) - .add("l_discount", DoubleType, nullable = true) - .add("l_tax", DoubleType, nullable = true) - .add("l_returnflag", StringType, nullable = true) - .add("l_linestatus", StringType, nullable = true) - .add("l_shipdate", DateType, nullable = true) - .add("l_commitdate", DateType, nullable = true) - .add("l_receiptdate", DateType, nullable = true) - .add("l_shipinstruct", StringType, nullable = true) - .add("l_shipmode", StringType, nullable = true) - .add("l_comment", StringType, nullable = true) - } -} From bb5debe9d5c795a6caef4ce674d04b0f870c5062 Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Wed, 14 Jan 2026 21:34:04 +0400 Subject: [PATCH 22/23] Run tpch --- dev/benchmarks/comet-tpch.sh | 3 ++- dev/benchmarks/spark-tpch.sh | 3 ++- dev/benchmarks/tpcbench.py | 11 ++++++----- .../core/src/execution/operators/csv_scan.rs | 18 +++++++----------- native/core/src/execution/planner.rs | 8 +++++--- native/proto/src/proto/operator.proto | 9 +++++---- .../org/apache/comet/rules/CometScanRule.scala | 5 ++++- .../sql/comet/CometCsvNativeScanExec.scala | 11 +++++++++-- 8 files changed, 40 insertions(+), 28 deletions(-) diff --git a/dev/benchmarks/comet-tpch.sh b/dev/benchmarks/comet-tpch.sh index 8b63f87ed3..f0709b7ef0 100755 --- a/dev/benchmarks/comet-tpch.sh +++ b/dev/benchmarks/comet-tpch.sh @@ -50,4 +50,5 @@ $SPARK_HOME/bin/spark-submit \ --data $TPCH_DATA \ --queries $TPCH_QUERIES \ --output . \ - --iterations 1 + --iterations 1 \ + --format parquet diff --git a/dev/benchmarks/spark-tpch.sh b/dev/benchmarks/spark-tpch.sh index beebbd60fd..ae359f049f 100755 --- a/dev/benchmarks/spark-tpch.sh +++ b/dev/benchmarks/spark-tpch.sh @@ -42,4 +42,5 @@ $SPARK_HOME/bin/spark-submit \ --data $TPCH_DATA \ --queries $TPCH_QUERIES \ --output . \ - --iterations 1 + --iterations 1 \ + --format parquet diff --git a/dev/benchmarks/tpcbench.py b/dev/benchmarks/tpcbench.py index 75944883df..25a8fdec8f 100644 --- a/dev/benchmarks/tpcbench.py +++ b/dev/benchmarks/tpcbench.py @@ -37,7 +37,7 @@ def dedup_columns(df): new_cols.append(f"{c}_{counts[c]}") return df.toDF(*new_cols) -def main(benchmark: str, data_path: str, query_path: str, iterations: int, output: str, name: str, query_num: int = None, write_path: str = None): +def main(benchmark: str, data_path: str, query_path: str, iterations: int, output: str, name: str, format: str, query_num: int = None, write_path: str = None): # Initialize a SparkSession spark = SparkSession.builder \ @@ -58,9 +58,9 @@ def main(benchmark: str, data_path: str, query_path: str, iterations: int, outpu raise "invalid benchmark" for table in table_names: - path = f"{data_path}/{table}.parquet" + path = f"{data_path}/{table}.{format}" print(f"Registering table {table} using path {path}") - df = spark.read.parquet(path) + df = spark.read.format(format).load(path) df.createOrReplaceTempView(table) conf_dict = {k: v for k, v in spark.sparkContext.getConf().getAll()} @@ -146,7 +146,7 @@ def main(benchmark: str, data_path: str, query_path: str, iterations: int, outpu if __name__ == "__main__": parser = argparse.ArgumentParser(description="DataFusion benchmark derived from TPC-H / TPC-DS") - parser.add_argument("--benchmark", required=True, help="Benchmark to run (tpch or tpcds)") + parser.add_argument("--benchmark", required=True, default="tpch", help="Benchmark to run (tpch or tpcds)") parser.add_argument("--data", required=True, help="Path to data files") parser.add_argument("--queries", required=True, help="Path to query files") parser.add_argument("--iterations", required=False, default="1", help="How many iterations to run") @@ -154,7 +154,8 @@ def main(benchmark: str, data_path: str, query_path: str, iterations: int, outpu parser.add_argument("--name", required=True, help="Prefix for result file e.g. spark/comet/gluten") parser.add_argument("--query", required=False, type=int, help="Specific query number to run (1-based). If not specified, all queries will be run.") parser.add_argument("--write", required=False, help="Path to save query results to, in Parquet format.") + parser.add_argument("--format", required=True, default="parquet", help="Input file format (parquet, csv, json)") args = parser.parse_args() - main(args.benchmark, args.data, args.queries, int(args.iterations), args.output, args.name, args.query, args.write) + main(args.benchmark, args.data, args.queries, int(args.iterations), args.output, args.name, args.format, args.query, args.write) diff --git a/native/core/src/execution/operators/csv_scan.rs b/native/core/src/execution/operators/csv_scan.rs index 7cc6d41bc2..622386f0b5 100644 --- a/native/core/src/execution/operators/csv_scan.rs +++ b/native/core/src/execution/operators/csv_scan.rs @@ -33,7 +33,8 @@ pub fn init_csv_datasource_exec( object_store_url: ObjectStoreUrl, file_groups: Vec>, data_schema: SchemaRef, - partition_schema: Option, + partition_schema: SchemaRef, + projection_vector: Vec, csv_options: &CsvOptions, ) -> Result, ExecutionError> { let csv_source = build_csv_source(csv_options.clone()); @@ -44,21 +45,16 @@ pub fn init_csv_datasource_exec( .collect(); let partition_fields = partition_schema - .map(|schema| { - schema - .fields() - .iter() - .map(|field| { - Field::new(field.name(), field.data_type().clone(), field.is_nullable()) - }) - .collect_vec() - }) - .unwrap_or(vec![]); + .fields() + .iter() + .map(|field| Field::new(field.name(), field.data_type().clone(), field.is_nullable())) + .collect_vec(); let file_scan_config: FileScanConfig = FileScanConfigBuilder::new(object_store_url, data_schema, csv_source) .with_file_groups(file_groups) .with_table_partition_cols(partition_fields) + .with_projection_indices(Some(projection_vector)) .build(); Ok(Arc::new(DataSourceExec::new(Arc::new(file_scan_config)))) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index b0f0c8dfcf..036166e524 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1123,10 +1123,11 @@ impl PhysicalPlanner { )) } OpStruct::CsvScan(scan) => { - let data_schema = - convert_spark_types_to_arrow_schema(scan.required_schema.as_slice()); + let data_schema = convert_spark_types_to_arrow_schema(scan.data_schema.as_slice()); let partition_schema = convert_spark_types_to_arrow_schema(scan.partition_schema.as_slice()); + let projection_vector: Vec = + scan.projection_vector.iter().map(|i| *i as usize).collect(); let object_store_options: HashMap = scan .object_store_options .iter() @@ -1150,7 +1151,8 @@ impl PhysicalPlanner { object_store_url, file_groups, data_schema, - Some(partition_schema), + partition_schema, + projection_vector, &scan.csv_options.clone().unwrap(), )?; Ok(( diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 3372f7a4d3..77b18bc27b 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -112,11 +112,12 @@ message NativeScan { } message CsvScan { - repeated SparkStructField required_schema = 1; + repeated SparkStructField data_schema = 1; repeated SparkStructField partition_schema = 2; - repeated SparkFilePartition file_partitions = 3; - map object_store_options = 4; - CsvOptions csv_options = 5; + repeated int32 projection_vector = 3; + repeated SparkFilePartition file_partitions = 4; + map object_store_options = 5; + CsvOptions csv_options = 6; } message CsvOptions { 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 53ef20c4ae..4310605f22 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometScanRule.scala @@ -279,7 +279,10 @@ case class CometScanRule(session: SparkSession) extends Rule[SparkPlan] with Com if (isInferSchemaEnabled) { fallbackReasons += "Comet doesn't support inferSchema=true option" } - val delimiter = scan.options.get("delimiter") + val delimiter = + Option(scan.options.get("delimiter")) + .orElse(Option(scan.options.get("sep"))) + .getOrElse(",") val isSingleCharacterDelimiter = delimiter.length == 1 if (!isSingleCharacterDelimiter) { fallbackReasons += diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala index 1477401306..39ebee49ab 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometCsvNativeScanExec.scala @@ -94,7 +94,13 @@ object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { } val filePartitions = op.inputPartitions.map(_.asInstanceOf[FilePartition]) val csvOptionsProto = csvOptions2Proto(options) - val schemaProto = schema2Proto(csvScan.readDataSchema.fields) + val dataSchemaProto = schema2Proto(csvScan.dataSchema.fields) + val readSchemaFieldNames = csvScan.readDataSchema.fieldNames + val projectionVector = csvScan.dataSchema.fields.zipWithIndex + .filter { case (field, _) => + readSchemaFieldNames.contains(field.name) + } + .map(_._2.asInstanceOf[Integer]) val partitionSchemaProto = schema2Proto(csvScan.readPartitionSchema.fields) val partitionsProto = filePartitions.map(partition2Proto(_, csvScan.readPartitionSchema)) @@ -110,7 +116,8 @@ object CometCsvNativeScanExec extends CometOperatorSerde[CometBatchScanExec] { csvScanBuilder.putAllObjectStoreOptions(objectStoreOptions.asJava) csvScanBuilder.setCsvOptions(csvOptionsProto) csvScanBuilder.addAllFilePartitions(partitionsProto.asJava) - csvScanBuilder.addAllRequiredSchema(schemaProto.toIterable.asJava) + csvScanBuilder.addAllDataSchema(dataSchemaProto.toIterable.asJava) + csvScanBuilder.addAllProjectionVector(projectionVector.toIterable.asJava) csvScanBuilder.addAllPartitionSchema(partitionSchemaProto.toIterable.asJava) Some(builder.setCsvScan(csvScanBuilder).build()) } From 139ecffa663c86e8a830390b688af0709be8c513 Mon Sep 17 00:00:00 2001 From: Kazantsev Maksim Date: Thu, 15 Jan 2026 19:30:25 +0400 Subject: [PATCH 23/23] Add spark options to tpcbench.py --- dev/benchmarks/tpcbench.py | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/dev/benchmarks/tpcbench.py b/dev/benchmarks/tpcbench.py index 25a8fdec8f..130db7a628 100644 --- a/dev/benchmarks/tpcbench.py +++ b/dev/benchmarks/tpcbench.py @@ -20,6 +20,7 @@ import json from pyspark.sql import SparkSession import time +from typing import Dict # rename same columns aliases # a, a, b, b -> a, a_1, b, b_1 @@ -37,7 +38,7 @@ def dedup_columns(df): new_cols.append(f"{c}_{counts[c]}") return df.toDF(*new_cols) -def main(benchmark: str, data_path: str, query_path: str, iterations: int, output: str, name: str, format: str, query_num: int = None, write_path: str = None): +def main(benchmark: str, data_path: str, query_path: str, iterations: int, output: str, name: str, format: str, query_num: int = None, write_path: str = None, options: Dict[str, str] = None): # Initialize a SparkSession spark = SparkSession.builder \ @@ -60,7 +61,7 @@ def main(benchmark: str, data_path: str, query_path: str, iterations: int, outpu for table in table_names: path = f"{data_path}/{table}.{format}" print(f"Registering table {table} using path {path}") - df = spark.read.format(format).load(path) + df = spark.read.format(format).options(**options).load(path) df.createOrReplaceTempView(table) conf_dict = {k: v for k, v in spark.sparkContext.getConf().getAll()} @@ -155,7 +156,8 @@ def main(benchmark: str, data_path: str, query_path: str, iterations: int, outpu parser.add_argument("--query", required=False, type=int, help="Specific query number to run (1-based). If not specified, all queries will be run.") parser.add_argument("--write", required=False, help="Path to save query results to, in Parquet format.") parser.add_argument("--format", required=True, default="parquet", help="Input file format (parquet, csv, json)") + parser.add_argument("--options", type=json.loads, required=False, default={}, help='Spark options as JSON string, e.g., \'{"header": "true", "delimiter": ","}\'') args = parser.parse_args() - main(args.benchmark, args.data, args.queries, int(args.iterations), args.output, args.name, args.format, args.query, args.write) + main(args.benchmark, args.data, args.queries, int(args.iterations), args.output, args.name, args.format, args.query, args.write, args.options)