From 64c3dd77971536d863109a4602c16c65941c6adf Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 09:50:17 -0700 Subject: [PATCH 01/35] save --- native/core/src/execution/operators/mod.rs | 2 ++ native/core/src/execution/planner.rs | 32 +++++++++++++++++++++- native/proto/src/proto/operator.proto | 6 ++++ 3 files changed, 39 insertions(+), 1 deletion(-) diff --git a/native/core/src/execution/operators/mod.rs b/native/core/src/execution/operators/mod.rs index b3998e2f60..b01f7857be 100644 --- a/native/core/src/execution/operators/mod.rs +++ b/native/core/src/execution/operators/mod.rs @@ -29,6 +29,8 @@ mod copy; mod expand; pub use expand::ExpandExec; mod iceberg_scan; +mod parquet_writer; +pub use parquet_writer::ParquetWriterExec; mod scan; /// Error returned during executing operators. diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 0fe04a5a41..60d3bd9b17 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -22,7 +22,7 @@ use crate::{ errors::ExpressionError, execution::{ expressions::subquery::Subquery, - operators::{ExecutionError, ExpandExec, ScanExec}, + operators::{ExecutionError, ExpandExec, ParquetWriterExec, ScanExec}, serde::to_arrow_datatype, shuffle::ShuffleWriterExec, }, @@ -1448,6 +1448,36 @@ impl PhysicalPlanner { )), )) } + OpStruct::ParquetWriter(writer) => { + assert_eq!(children.len(), 1); + let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?; + + let codec = match writer.compression.try_into() { + Ok(SparkCompressionCodec::None) => Ok(CompressionCodec::None), + Ok(SparkCompressionCodec::Snappy) => Ok(CompressionCodec::Snappy), + Ok(SparkCompressionCodec::Zstd) => Ok(CompressionCodec::Zstd(3)), + Ok(SparkCompressionCodec::Lz4) => Ok(CompressionCodec::Lz4Frame), + _ => Err(GeneralError(format!( + "Unsupported parquet compression codec: {:?}", + writer.compression + ))), + }?; + + let parquet_writer = Arc::new(ParquetWriterExec::try_new( + Arc::clone(&child.native_plan), + writer.output_path.clone(), + codec, + )?); + + Ok(( + scans, + Arc::new(SparkPlan::new( + spark_plan.plan_id, + parquet_writer, + vec![Arc::clone(&child)], + )), + )) + } OpStruct::Expand(expand) => { assert_eq!(children.len(), 1); let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?; diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 94661a20e6..44c84b7273 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -49,6 +49,7 @@ message Operator { Window window = 110; NativeScan native_scan = 111; IcebergScan iceberg_scan = 112; + ParquetWriter parquet_writer = 113; } } @@ -236,6 +237,11 @@ message ShuffleWriter { bool tracing_enabled = 7; } +message ParquetWriter { + string output_path = 1; + CompressionCodec compression = 2; +} + enum AggregateMode { Partial = 0; Final = 1; From c6fe639d6cd18bfafd99711426bbd288c9c71851 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 09:59:51 -0700 Subject: [PATCH 02/35] save --- .../src/execution/operators/parquet_writer.rs | 216 ++++++++++++++++++ .../comet/CometParquetWriterSuite.scala | 145 ++++++++++++ 2 files changed, 361 insertions(+) create mode 100644 native/core/src/execution/operators/parquet_writer.rs create mode 100644 spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala diff --git a/native/core/src/execution/operators/parquet_writer.rs b/native/core/src/execution/operators/parquet_writer.rs new file mode 100644 index 0000000000..b4084f4c9f --- /dev/null +++ b/native/core/src/execution/operators/parquet_writer.rs @@ -0,0 +1,216 @@ +// 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. + +//! Parquet writer operator for writing RecordBatches to Parquet files + +use std::{ + any::Any, + fmt, + fmt::{Debug, Formatter}, + fs::File, + path::Path, + sync::Arc, +}; + +use arrow::datatypes::SchemaRef; +use async_trait::async_trait; +use datafusion::{ + error::{DataFusionError, Result}, + execution::context::TaskContext, + physical_expr::{EquivalenceProperties, Partitioning}, + physical_plan::{ + execution_plan::{Boundedness, EmissionType}, + metrics::{ExecutionPlanMetricsSet, MetricsSet}, + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, SendableRecordBatchStream, + Statistics, + }, +}; +use futures::TryStreamExt; +use parquet::{ + arrow::ArrowWriter, + basic::{Compression, ZstdLevel}, + file::properties::WriterProperties, +}; + +use crate::execution::shuffle::CompressionCodec; + +/// Parquet writer operator that writes input batches to a Parquet file +#[derive(Debug)] +pub struct ParquetWriterExec { + /// Input execution plan + input: Arc, + /// Output file path + output_path: String, + /// Compression codec + compression: CompressionCodec, + /// Metrics + metrics: ExecutionPlanMetricsSet, + /// Cache for plan properties + cache: PlanProperties, +} + +impl ParquetWriterExec { + /// Create a new ParquetWriterExec + pub fn try_new( + input: Arc, + output_path: String, + compression: CompressionCodec, + ) -> Result { + let cache = PlanProperties::new( + EquivalenceProperties::new(Arc::clone(&input.schema())), + Partitioning::UnknownPartitioning(1), + EmissionType::Final, + Boundedness::Bounded, + ); + + Ok(ParquetWriterExec { + input, + output_path, + compression, + metrics: ExecutionPlanMetricsSet::new(), + cache, + }) + } + + fn compression_to_parquet(&self) -> Compression { + match self.compression { + CompressionCodec::None => Compression::UNCOMPRESSED, + CompressionCodec::Zstd(_) => Compression::ZSTD(ZstdLevel::default()), + CompressionCodec::Lz4Frame => Compression::LZ4, + CompressionCodec::Snappy => Compression::SNAPPY, + } + } +} + +impl DisplayAs for ParquetWriterExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!( + f, + "ParquetWriterExec: path={}, compression={:?}", + self.output_path, self.compression + ) + } + DisplayFormatType::TreeRender => unimplemented!(), + } + } +} + +#[async_trait] +impl ExecutionPlan for ParquetWriterExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "ParquetWriterExec" + } + + fn metrics(&self) -> Option { + Some(self.metrics.clone_inner()) + } + + fn statistics(&self) -> Result { + self.input.partition_statistics(None) + } + + fn properties(&self) -> &PlanProperties { + &self.cache + } + + fn schema(&self) -> SchemaRef { + self.input.schema() + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.input] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> Result> { + match children.len() { + 1 => Ok(Arc::new(ParquetWriterExec::try_new( + Arc::clone(&children[0]), + self.output_path.clone(), + self.compression.clone(), + )?)), + _ => Err(DataFusionError::Internal( + "ParquetWriterExec requires exactly one child".to_string(), + )), + } + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> Result { + let input = self.input.execute(partition, context)?; + let schema = self.schema(); + let output_path = self.output_path.clone(); + let compression = self.compression_to_parquet(); + + // Create output directory if it doesn't exist + if let Some(parent) = Path::new(&output_path).parent() { + std::fs::create_dir_all(parent).map_err(|e| { + DataFusionError::Execution(format!("Failed to create output directory: {}", e)) + })?; + } + + // Create the Parquet file + let file = File::create(&output_path).map_err(|e| { + DataFusionError::Execution(format!("Failed to create output file: {}", e)) + })?; + + // Configure writer properties + let props = WriterProperties::builder() + .set_compression(compression) + .build(); + + let mut writer = ArrowWriter::try_new(file, schema.clone(), Some(props)) + .map_err(|e| DataFusionError::Execution(format!("Failed to create writer: {}", e)))?; + + // Write batches + let write_task = async move { + let mut stream = input; + + while let Some(batch_result) = stream.try_next().await.transpose() { + let batch = batch_result?; + writer.write(&batch).map_err(|e| { + DataFusionError::Execution(format!("Failed to write batch: {}", e)) + })?; + } + + writer.close().map_err(|e| { + DataFusionError::Execution(format!("Failed to close writer: {}", e)) + })?; + + // Return empty stream to indicate completion + Ok::<_, DataFusionError>(futures::stream::empty()) + }; + + // Execute the write task and convert to a stream + use datafusion::physical_plan::stream::RecordBatchStreamAdapter; + Ok(Box::pin(RecordBatchStreamAdapter::new( + schema, + futures::stream::once(write_task).try_flatten(), + ))) + } +} diff --git a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala new file mode 100644 index 0000000000..a55646b85b --- /dev/null +++ b/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet + +import java.io.File + +import org.apache.spark.sql.{CometTestBase, Row} +import org.apache.spark.sql.comet.{CometExec, CometMetricNode} +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.types.{IntegerType, StringType} + +import org.apache.comet.serde.{OperatorOuterClass, QueryPlanSerde} +import org.apache.comet.serde.OperatorOuterClass.Operator + +class CometParquetWriterSuite extends CometTestBase { + import testImplicits._ + + test("basic parquet write with native writer") { + withTempPath { dir => + val outputPath = new File(dir, "output.parquet").getAbsolutePath + + // Create test data and write it to a temp parquet file first + withTempPath { inputDir => + val inputPath = new File(inputDir, "input.parquet").getAbsolutePath + val df = Seq((1, "a"), (2, "b"), (3, "c")).toDF("id", "value") + df.write.parquet(inputPath) + + // Now use native writer to read and write + // Collect data as ColumnarBatches (for this small test) + val inputBatches = spark.read + .parquet(inputPath) + .queryExecution + .executedPlan + .executeColumnar() + .collect() + .iterator + + // Create native plan for ParquetWriter + val scanOp = OperatorOuterClass.Scan + .newBuilder() + .setSource("test_input") + .setArrowFfiSafe(true) + .addFields(QueryPlanSerde.serializeDataType(IntegerType).get) + .addFields(QueryPlanSerde.serializeDataType(StringType).get) + .build() + + val scanOperator = Operator + .newBuilder() + .setPlanId(1) + .setScan(scanOp) + .build() + + val writerOp = OperatorOuterClass.ParquetWriter + .newBuilder() + .setOutputPath(outputPath) + .setCompression(OperatorOuterClass.CompressionCodec.Snappy) + .build() + + val writerOperator = Operator + .newBuilder() + .setPlanId(2) + .addChildren(scanOperator) + .setParquetWriter(writerOp) + .build() + + // Execute native write + val nativeMetrics = CometMetricNode(Map.empty[String, SQLMetric]) + + val cometIter = CometExec.getCometIterator( + Seq(inputBatches), + 2, // numColumns + writerOperator, + nativeMetrics, + 1, // numPartitions + 0, // partitionIndex + broadcastedHadoopConfForEncryption = None, + encryptedFilePaths = Seq.empty) + + // Consume the iterator + while (cometIter.hasNext) { + cometIter.next() + } + cometIter.close() + + // Verify the file was written + assert(new File(outputPath).exists(), s"Output file should exist at $outputPath") + assert(new File(outputPath).length() > 0, "Output file should not be empty") + + // Verify we can read the data back with Spark + val readDf = spark.read.parquet(outputPath) + val result = readDf.collect().sortBy(_.getInt(0)) + + assert(result.length == 3) + assert(result(0).getInt(0) == 1) + assert(result(0).getString(1) == "a") + assert(result(1).getInt(0) == 2) + assert(result(1).getString(1) == "b") + assert(result(2).getInt(0) == 3) + assert(result(2).getString(1) == "c") + } + } + } + + test("verify parquet writer operator serialization") { + val outputPath = "/tmp/test_output.parquet" + + val writerOp = OperatorOuterClass.ParquetWriter + .newBuilder() + .setOutputPath(outputPath) + .setCompression(OperatorOuterClass.CompressionCodec.Zstd) + .build() + + val operator = Operator + .newBuilder() + .setPlanId(1) + .setParquetWriter(writerOp) + .build() + + // Verify the operator can be serialized and deserialized + val serialized = operator.toByteArray + val deserialized = Operator.parseFrom(serialized) + + assert(deserialized.hasParquetWriter) + assert(deserialized.getParquetWriter.getOutputPath == outputPath) + assert( + deserialized.getParquetWriter.getCompression == OperatorOuterClass.CompressionCodec.Zstd) + } +} From 2a8dc525ae4af751534e63407c798cc683751e4b Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 10:14:00 -0700 Subject: [PATCH 03/35] save --- .../scala/org/apache/comet/CometConf.scala | 10 ++ .../sql/comet/CometNativeWriteExec.scala | 112 ++++++++++++++++++ .../comet/CometParquetWriterSuite.scala | 64 +++++++++- 3 files changed, 185 insertions(+), 1 deletion(-) create mode 100644 spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 0484b64f16..017b571053 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -100,6 +100,16 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(true) + val COMET_NATIVE_PARQUET_WRITE_ENABLED: ConfigEntry[Boolean] = + conf("spark.comet.parquet.write.enabled") + .category(CATEGORY_PARQUET) + .doc( + "Whether to enable native Parquet write through Comet. When enabled, " + + "Comet will intercept Parquet write operations and execute them natively " + + "for improved performance.") + .booleanConf + .createWithDefault(false) + val SCAN_NATIVE_COMET = "native_comet" val SCAN_NATIVE_DATAFUSION = "native_datafusion" val SCAN_NATIVE_ICEBERG_COMPAT = "native_iceberg_compat" diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala new file mode 100644 index 0000000000..3688d3a23e --- /dev/null +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala @@ -0,0 +1,112 @@ +/* + * 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 org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} +import org.apache.spark.sql.vectorized.ColumnarBatch + +import org.apache.comet.serde.OperatorOuterClass.Operator + +/** + * Comet physical operator for native Parquet write operations. + * + * This operator writes data to Parquet files using the native Comet engine. It wraps the child + * operator and adds a ParquetWriter operator on top. + * + * @param nativeOp + * The native operator representing the write operation + * @param child + * The child operator providing the data to write + * @param outputPath + * The path where the Parquet file will be written + */ +case class CometNativeWriteExec(nativeOp: Operator, child: SparkPlan, outputPath: String) + extends CometNativeExec + with UnaryExecNode { + + override def originalPlan: SparkPlan = child + + override def serializedPlanOpt: SerializedPlan = { + val outputStream = new java.io.ByteArrayOutputStream() + nativeOp.writeTo(outputStream) + outputStream.close() + SerializedPlan(Some(outputStream.toByteArray)) + } + + override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = + copy(child = newChild) + + override def nodeName: String = "CometNativeWrite" + + override def doExecute(): RDD[InternalRow] = { + // Execute the native write + val resultRDD = doExecuteColumnar() + // Convert to InternalRow RDD (write operations typically return empty results) + resultRDD.mapPartitions { iter => + // Consume all batches (they should be empty) + iter.foreach(_.close()) + Iterator.empty + } + } + + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + // Get the input data from the child operator + val childRDD = if (child.supportsColumnar) { + child.executeColumnar() + } else { + // If child doesn't support columnar, convert to columnar + child.execute().mapPartitionsInternal { rowIter => + // This is a simplified version - in production you'd need proper row to columnar conversion + throw new UnsupportedOperationException( + "Row-based child operators not yet supported for native write") + } + } + + // Execute native write operation + childRDD.mapPartitionsInternal { iter => + val nativeMetrics = CometMetricNode.fromCometPlan(this) + + // Serialize the native plan + val outputStream = new java.io.ByteArrayOutputStream() + nativeOp.writeTo(outputStream) + outputStream.close() + val planBytes = outputStream.toByteArray + + val cometIter = new org.apache.comet.CometExecIterator( + CometExec.newIterId, + Seq(iter), + child.output.length, + planBytes, + nativeMetrics, + childRDD.getNumPartitions, + org.apache.spark.TaskContext.getPartitionId(), + None, + Seq.empty) + + // Consume all output batches (they should be empty for write operations) + new Iterator[ColumnarBatch] { + override def hasNext: Boolean = cometIter.hasNext + override def next(): ColumnarBatch = cometIter.next() + } + } + } +} diff --git a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala index a55646b85b..73a729d2fd 100644 --- a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala @@ -22,7 +22,7 @@ package org.apache.comet import java.io.File import org.apache.spark.sql.{CometTestBase, Row} -import org.apache.spark.sql.comet.{CometExec, CometMetricNode} +import org.apache.spark.sql.comet.{CometExec, CometMetricNode, CometNativeWriteExec} import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types.{IntegerType, StringType} @@ -118,6 +118,68 @@ class CometParquetWriterSuite extends CometTestBase { } } + test("end-to-end DataFrame write with CometNativeWriteExec") { + withTempPath { dir => + val outputPath = new File(dir, "output.parquet").getAbsolutePath + + // Create test data + val df = Seq((1, "a"), (2, "b"), (3, "c")).toDF("id", "value") + + // Get the physical plan + val childPlan = df.queryExecution.executedPlan + + // Create ParquetWriter operator + val scanOp = OperatorOuterClass.Scan + .newBuilder() + .setSource("test_input") + .setArrowFfiSafe(true) + .addFields(QueryPlanSerde.serializeDataType(IntegerType).get) + .addFields(QueryPlanSerde.serializeDataType(StringType).get) + .build() + + val scanOperator = Operator + .newBuilder() + .setPlanId(1) + .setScan(scanOp) + .build() + + val writerOp = OperatorOuterClass.ParquetWriter + .newBuilder() + .setOutputPath(outputPath) + .setCompression(OperatorOuterClass.CompressionCodec.Snappy) + .build() + + val writerOperator = Operator + .newBuilder() + .setPlanId(2) + .addChildren(scanOperator) + .setParquetWriter(writerOp) + .build() + + // Create CometNativeWriteExec + val writeExec = CometNativeWriteExec(writerOperator, childPlan, outputPath) + + // Execute the write + writeExec.executeColumnar().count() + + // Verify the file was written + assert(new File(outputPath).exists(), s"Output file should exist at $outputPath") + assert(new File(outputPath).length() > 0, "Output file should not be empty") + + // Verify we can read the data back + val readDf = spark.read.parquet(outputPath) + val result = readDf.collect().sortBy(_.getInt(0)) + + assert(result.length == 3) + assert(result(0).getInt(0) == 1) + assert(result(0).getString(1) == "a") + assert(result(1).getInt(0) == 2) + assert(result(1).getString(1) == "b") + assert(result(2).getInt(0) == 3) + assert(result(2).getString(1) == "c") + } + } + test("verify parquet writer operator serialization") { val outputPath = "/tmp/test_output.parquet" From e4acf9325a8bb998885ecc4497f940f2f966f326 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 10:17:47 -0700 Subject: [PATCH 04/35] save --- .../comet/CometParquetWriterSuite.scala | 140 +++++++++--------- 1 file changed, 74 insertions(+), 66 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala index 73a729d2fd..b4c3382b75 100644 --- a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala @@ -103,17 +103,19 @@ class CometParquetWriterSuite extends CometTestBase { assert(new File(outputPath).exists(), s"Output file should exist at $outputPath") assert(new File(outputPath).length() > 0, "Output file should not be empty") - // Verify we can read the data back with Spark - val readDf = spark.read.parquet(outputPath) - val result = readDf.collect().sortBy(_.getInt(0)) - - assert(result.length == 3) - assert(result(0).getInt(0) == 1) - assert(result(0).getString(1) == "a") - assert(result(1).getInt(0) == 2) - assert(result(1).getString(1) == "b") - assert(result(2).getInt(0) == 3) - assert(result(2).getString(1) == "c") + // Verify we can read the data back with Spark (disable Comet for read) + withSQLConf("spark.comet.enabled" -> "false") { + val readDf = spark.read.parquet(outputPath) + val result = readDf.collect().sortBy(_.getInt(0)) + + assert(result.length == 3) + assert(result(0).getInt(0) == 1) + assert(result(0).getString(1) == "a") + assert(result(1).getInt(0) == 2) + assert(result(1).getString(1) == "b") + assert(result(2).getInt(0) == 3) + assert(result(2).getString(1) == "c") + } } } } @@ -122,61 +124,67 @@ class CometParquetWriterSuite extends CometTestBase { withTempPath { dir => val outputPath = new File(dir, "output.parquet").getAbsolutePath - // Create test data - val df = Seq((1, "a"), (2, "b"), (3, "c")).toDF("id", "value") - - // Get the physical plan - val childPlan = df.queryExecution.executedPlan - - // Create ParquetWriter operator - val scanOp = OperatorOuterClass.Scan - .newBuilder() - .setSource("test_input") - .setArrowFfiSafe(true) - .addFields(QueryPlanSerde.serializeDataType(IntegerType).get) - .addFields(QueryPlanSerde.serializeDataType(StringType).get) - .build() - - val scanOperator = Operator - .newBuilder() - .setPlanId(1) - .setScan(scanOp) - .build() - - val writerOp = OperatorOuterClass.ParquetWriter - .newBuilder() - .setOutputPath(outputPath) - .setCompression(OperatorOuterClass.CompressionCodec.Snappy) - .build() - - val writerOperator = Operator - .newBuilder() - .setPlanId(2) - .addChildren(scanOperator) - .setParquetWriter(writerOp) - .build() - - // Create CometNativeWriteExec - val writeExec = CometNativeWriteExec(writerOperator, childPlan, outputPath) - - // Execute the write - writeExec.executeColumnar().count() - - // Verify the file was written - assert(new File(outputPath).exists(), s"Output file should exist at $outputPath") - assert(new File(outputPath).length() > 0, "Output file should not be empty") - - // Verify we can read the data back - val readDf = spark.read.parquet(outputPath) - val result = readDf.collect().sortBy(_.getInt(0)) - - assert(result.length == 3) - assert(result(0).getInt(0) == 1) - assert(result(0).getString(1) == "a") - assert(result(1).getInt(0) == 2) - assert(result(1).getString(1) == "b") - assert(result(2).getInt(0) == 3) - assert(result(2).getString(1) == "c") + // Create test data and write it to a temp parquet file first + withTempPath { inputDir => + val inputPath = new File(inputDir, "input.parquet").getAbsolutePath + val df = Seq((1, "a"), (2, "b"), (3, "c")).toDF("id", "value") + df.write.parquet(inputPath) + + // Read from parquet to get a columnar plan + val childPlan = spark.read.parquet(inputPath).queryExecution.executedPlan + + // Create ParquetWriter operator + val scanOp = OperatorOuterClass.Scan + .newBuilder() + .setSource("test_input") + .setArrowFfiSafe(true) + .addFields(QueryPlanSerde.serializeDataType(IntegerType).get) + .addFields(QueryPlanSerde.serializeDataType(StringType).get) + .build() + + val scanOperator = Operator + .newBuilder() + .setPlanId(1) + .setScan(scanOp) + .build() + + val writerOp = OperatorOuterClass.ParquetWriter + .newBuilder() + .setOutputPath(outputPath) + .setCompression(OperatorOuterClass.CompressionCodec.Snappy) + .build() + + val writerOperator = Operator + .newBuilder() + .setPlanId(2) + .addChildren(scanOperator) + .setParquetWriter(writerOp) + .build() + + // Create CometNativeWriteExec + val writeExec = CometNativeWriteExec(writerOperator, childPlan, outputPath) + + // Execute the write + writeExec.executeColumnar().count() + + // Verify the file was written + assert(new File(outputPath).exists(), s"Output file should exist at $outputPath") + assert(new File(outputPath).length() > 0, "Output file should not be empty") + + // Verify we can read the data back (disable Comet for read) + withSQLConf("spark.comet.enabled" -> "false") { + val readDf = spark.read.parquet(outputPath) + val result = readDf.collect().sortBy(_.getInt(0)) + + assert(result.length == 3) + assert(result(0).getInt(0) == 1) + assert(result(0).getString(1) == "a") + assert(result(1).getInt(0) == 2) + assert(result(1).getString(1) == "b") + assert(result(2).getInt(0) == 3) + assert(result(2).getString(1) == "c") + } + } } } From 82a552d2dcedf3524dd5db9a90ce802db0eae9aa Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 10:49:59 -0700 Subject: [PATCH 05/35] save --- docs/source/user-guide/latest/configs.md | 1 + .../apache/comet/rules/CometExecRule.scala | 517 ++++++++++++------ .../comet/CometParquetWriterSuite.scala | 23 + 3 files changed, 383 insertions(+), 158 deletions(-) diff --git a/docs/source/user-guide/latest/configs.md b/docs/source/user-guide/latest/configs.md index 1e77032f7d..0d2489c449 100644 --- a/docs/source/user-guide/latest/configs.md +++ b/docs/source/user-guide/latest/configs.md @@ -51,6 +51,7 @@ Comet provides the following configuration settings. | `spark.comet.parquet.read.parallel.io.enabled` | Whether to enable Comet's parallel reader for Parquet files. The parallel reader reads ranges of consecutive data in a file in parallel. It is faster for large files and row groups but uses more resources. | true | | `spark.comet.parquet.read.parallel.io.thread-pool.size` | The maximum number of parallel threads the parallel reader will use in a single executor. For executors configured with a smaller number of cores, use a smaller number. | 16 | | `spark.comet.parquet.respectFilterPushdown` | Whether to respect Spark's PARQUET_FILTER_PUSHDOWN_ENABLED config. This needs to be respected when running the Spark SQL test suite but the default setting results in poor performance in Comet when using the new native scans, disabled by default | false | +| `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 for improved performance. | false | 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 4baedc9196..bac6564153 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -32,7 +32,9 @@ import org.apache.spark.sql.comet.execution.shuffle.{CometColumnarShuffle, Comet 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.ExecutedCommandExec +import org.apache.spark.sql.execution.command.{DataWritingCommandExec, ExecutedCommandExec} +import org.apache.spark.sql.execution.datasources.{FileFormat, InsertIntoHadoopFsRelationCommand, WriteFilesExec} +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat 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} @@ -49,6 +51,8 @@ import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.QueryPlanSerde.{serializeDataType, supportedDataType} import org.apache.comet.serde.operator._ +// scalastyle:off + object CometExecRule { /** @@ -110,6 +114,185 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { private def isCometNative(op: SparkPlan): Boolean = op.isInstanceOf[CometNativeExec] + /** + * Try to convert a DataWritingCommandExec to use Comet native Parquet writer. Returns + * Some(CometNativeWriteExec) if conversion is successful, None otherwise. + */ + private def tryConvertDataWritingCommand(exec: DataWritingCommandExec): Option[SparkPlan] = { + exec.cmd match { + case cmd: InsertIntoHadoopFsRelationCommand => + // Check if this is a Parquet write + cmd.fileFormat match { + case _: ParquetFileFormat => + try { + // The child should be WriteFilesExec + val childPlan = exec.child match { + case writeFiles: WriteFilesExec => writeFiles.child + case other => other + } + + // Get output path + val outputPath = cmd.outputPath.toString + + println(s"Converting DataWritingCommandExec to native Parquet write: $outputPath") + + // Create native ParquetWriter operator + val scanOp = OperatorOuterClass.Scan + .newBuilder() + .setSource("write_source") + .setArrowFfiSafe(true) + + // Add fields from the query output schema + cmd.query.output.foreach { attr => + serializeDataType(attr.dataType) match { + case Some(dataType) => scanOp.addFields(dataType) + case None => + logWarning(s"Cannot serialize data type ${attr.dataType} for native write") + return None + } + } + + val scanOperator = Operator + .newBuilder() + .setPlanId(1) + .setScan(scanOp.build()) + .build() + + val writerOp = OperatorOuterClass.ParquetWriter + .newBuilder() + .setOutputPath(outputPath) + // TODO: Get compression from options + .setCompression(OperatorOuterClass.CompressionCodec.Snappy) + .build() + + val writerOperator = Operator + .newBuilder() + .setPlanId(2) + .addChildren(scanOperator) + .setParquetWriter(writerOp) + .build() + + // Create CometNativeWriteExec with the transformed child plan + val writeExec = CometNativeWriteExec(writerOperator, childPlan, outputPath) + + println(s"Successfully converted to native Parquet write: $outputPath") + Some(writeExec) + } catch { + case e: Exception => + logWarning(s"Failed to convert DataWritingCommandExec to native execution", e) + None + } + case _ => + // Not a Parquet write, skip + None + } + case _ => + // Not a write command we handle + None + } + } + + /** + * Try to convert a WriteFilesExec to use Comet native Parquet writer. Returns + * Some(CometNativeWriteExec) if conversion is successful, None otherwise. + */ + private def tryConvertWriteFilesExec(writeOp: WriteFilesExec): Option[SparkPlan] = { + // Check if this is a Parquet write + writeOp.fileFormat match { + case _: ParquetFileFormat => + try { + // The child plan is the data source + val childPlan = writeOp.child + + // Get output path - WriteFilesExec doesn't have outputPath directly, + // we need to get it from the parent DataWritingCommandExec + // For now, we'll skip this operator and handle it in tryConvertWriteCommand + None + } catch { + case e: Exception => + logWarning(s"Failed to convert WriteFilesExec to native execution", e) + None + } + case _ => + // Not a Parquet write, skip + None + } + } + + /** + * Try to convert a write command (ExecutedCommandExec) to use Comet native Parquet writer. + * Returns Some(CometNativeWriteExec) if conversion is successful, None otherwise. + */ + private def tryConvertWriteCommand(exec: ExecutedCommandExec): Option[SparkPlan] = { + exec.cmd match { + case cmd: InsertIntoHadoopFsRelationCommand => + // Check if this is a Parquet write + cmd.fileFormat match { + case _: ParquetFileFormat => + try { + // Plan the query to get the physical plan + val queryExecution = session.sessionState.executePlan(cmd.query) + val childPlan = queryExecution.executedPlan + + // Get output path + val outputPath = cmd.outputPath.toString + + // Create native ParquetWriter operator + val scanOp = OperatorOuterClass.Scan + .newBuilder() + .setSource("write_source") + .setArrowFfiSafe(true) + + // Add fields from the query output schema + cmd.query.output.foreach { attr => + serializeDataType(attr.dataType) match { + case Some(dataType) => scanOp.addFields(dataType) + case None => + logWarning(s"Cannot serialize data type ${attr.dataType} for native write") + return None + } + } + + val scanOperator = Operator + .newBuilder() + .setPlanId(1) + .setScan(scanOp.build()) + .build() + + val writerOp = OperatorOuterClass.ParquetWriter + .newBuilder() + .setOutputPath(outputPath) + // TODO: Get compression from options + .setCompression(OperatorOuterClass.CompressionCodec.Snappy) + .build() + + val writerOperator = Operator + .newBuilder() + .setPlanId(2) + .addChildren(scanOperator) + .setParquetWriter(writerOp) + .build() + + // Create CometNativeWriteExec + val writeExec = CometNativeWriteExec(writerOperator, childPlan, outputPath) + + println(s"Converted Parquet write to native execution: $outputPath") + Some(writeExec) + } catch { + case e: Exception => + logWarning(s"Failed to convert write command to native execution", e) + None + } + case _ => + // Not a Parquet write, skip + None + } + case _ => + // Not a write command we handle + None + } + } + // spotless:off /** @@ -190,178 +373,196 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { operator2ProtoIfAllChildrenAreNative(op).map(fun).getOrElse(op) } - def convertNode(op: SparkPlan): SparkPlan = op match { - // Fully native scan for V1 - case scan: CometScanExec if scan.scanImpl == CometConf.SCAN_NATIVE_DATAFUSION => - val nativeOp = operator2Proto(scan).get - CometNativeScan.createExec(nativeOp, scan) - - // Fully native Iceberg scan for V2 (iceberg-rust path) - // Only handle scans with native metadata; SupportsComet scans fall through to isCometScan - // Config checks (COMET_ICEBERG_NATIVE_ENABLED, COMET_EXEC_ENABLED) are done in CometScanRule - case scan: CometBatchScanExec if scan.nativeIcebergScanMetadata.isDefined => - operator2Proto(scan) match { - case Some(nativeOp) => - CometIcebergNativeScan.createExec(nativeOp, scan) - case None => - // Serialization failed, fall back to CometBatchScanExec - scan - } + def convertNode(op: SparkPlan): SparkPlan = { + + println(s"convertNode: [${op.getClass}] $op") + + op match { + // Fully native scan for V1 + case scan: CometScanExec if scan.scanImpl == CometConf.SCAN_NATIVE_DATAFUSION => + val nativeOp = operator2Proto(scan).get + CometNativeScan.createExec(nativeOp, scan) + + // Fully native Iceberg scan for V2 (iceberg-rust path) + // Only handle scans with native metadata; SupportsComet scans fall through to isCometScan + // Config checks (COMET_ICEBERG_NATIVE_ENABLED, COMET_EXEC_ENABLED) are done in CometScanRule + case scan: CometBatchScanExec if scan.nativeIcebergScanMetadata.isDefined => + operator2Proto(scan) match { + case Some(nativeOp) => + CometIcebergNativeScan.createExec(nativeOp, scan) + case None => + // Serialization failed, fall back to CometBatchScanExec + scan + } - // Comet JVM + native scan for V1 and V2 - case op if isCometScan(op) => - val nativeOp = operator2Proto(op) - CometScanWrapper(nativeOp.get, op) - - case op if shouldApplySparkToColumnar(conf, op) => - val cometOp = CometSparkToColumnarExec(op) - val nativeOp = operator2Proto(cometOp) - CometScanWrapper(nativeOp.get, cometOp) - - // For AQE broadcast stage on a Comet broadcast exchange - case s @ BroadcastQueryStageExec(_, _: CometBroadcastExchangeExec, _) => - newPlanWithProto(s, CometSinkPlaceHolder(_, s, s)) - - case s @ BroadcastQueryStageExec( - _, - ReusedExchangeExec(_, _: CometBroadcastExchangeExec), - _) => - newPlanWithProto(s, CometSinkPlaceHolder(_, s, s)) - - // `CometBroadcastExchangeExec`'s broadcast output is not compatible with Spark's broadcast - // exchange. It is only used for Comet native execution. We only transform Spark broadcast - // exchange to Comet broadcast exchange if its downstream is a Comet native plan or if the - // broadcast exchange is forced to be enabled by Comet config. - case plan if plan.children.exists(_.isInstanceOf[BroadcastExchangeExec]) => - val newChildren = plan.children.map { - case b: BroadcastExchangeExec - if isCometNative(b.child) && - CometConf.COMET_EXEC_BROADCAST_EXCHANGE_ENABLED.get(conf) => - operator2Proto(b) match { - case Some(nativeOp) => - val cometOp = CometBroadcastExchangeExec(b, b.output, b.mode, b.child) - CometSinkPlaceHolder(nativeOp, b, cometOp) - case None => b + // Comet JVM + native scan for V1 and V2 + case op if isCometScan(op) => + val nativeOp = operator2Proto(op) + CometScanWrapper(nativeOp.get, op) + + case op if shouldApplySparkToColumnar(conf, op) => + val cometOp = CometSparkToColumnarExec(op) + val nativeOp = operator2Proto(cometOp) + CometScanWrapper(nativeOp.get, cometOp) + + // Intercept DataWritingCommandExec (Spark 3.5+) + case exec: DataWritingCommandExec + if CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.get(conf) => + tryConvertDataWritingCommand(exec).getOrElse(exec) + + // Intercept Parquet write commands (fallback for older Spark versions) + case exec: ExecutedCommandExec + if CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.get(conf) => + tryConvertWriteCommand(exec).getOrElse(exec) + + // For AQE broadcast stage on a Comet broadcast exchange + case s @ BroadcastQueryStageExec(_, _: CometBroadcastExchangeExec, _) => + newPlanWithProto(s, CometSinkPlaceHolder(_, s, s)) + + case s @ BroadcastQueryStageExec( + _, + ReusedExchangeExec(_, _: CometBroadcastExchangeExec), + _) => + newPlanWithProto(s, CometSinkPlaceHolder(_, s, s)) + + // `CometBroadcastExchangeExec`'s broadcast output is not compatible with Spark's broadcast + // exchange. It is only used for Comet native execution. We only transform Spark broadcast + // exchange to Comet broadcast exchange if its downstream is a Comet native plan or if the + // broadcast exchange is forced to be enabled by Comet config. + case plan if plan.children.exists(_.isInstanceOf[BroadcastExchangeExec]) => + val newChildren = plan.children.map { + case b: BroadcastExchangeExec + if isCometNative(b.child) && + CometConf.COMET_EXEC_BROADCAST_EXCHANGE_ENABLED.get(conf) => + operator2Proto(b) match { + case Some(nativeOp) => + val cometOp = CometBroadcastExchangeExec(b, b.output, b.mode, b.child) + CometSinkPlaceHolder(nativeOp, b, cometOp) + case None => b + } + case other => other + } + if (!newChildren.exists(_.isInstanceOf[BroadcastExchangeExec])) { + val newPlan = convertNode(plan.withNewChildren(newChildren)) + if (isCometNative(newPlan) || isCometBroadCastForceEnabled(conf)) { + newPlan + } else { + if (isCometNative(newPlan)) { + val reason = + getCometBroadcastNotEnabledReason(conf).getOrElse("no reason available") + withInfo(plan, s"Broadcast is not enabled: $reason") + } + plan } - case other => other - } - if (!newChildren.exists(_.isInstanceOf[BroadcastExchangeExec])) { - val newPlan = convertNode(plan.withNewChildren(newChildren)) - if (isCometNative(newPlan) || isCometBroadCastForceEnabled(conf)) { - newPlan } else { - if (isCometNative(newPlan)) { - val reason = - getCometBroadcastNotEnabledReason(conf).getOrElse("no reason available") - withInfo(plan, s"Broadcast is not enabled: $reason") - } plan } - } else { - plan - } - // For AQE shuffle stage on a Comet shuffle exchange - case s @ ShuffleQueryStageExec(_, _: CometShuffleExchangeExec, _) => - newPlanWithProto(s, CometSinkPlaceHolder(_, s, s)) - - // For AQE shuffle stage on a reused Comet shuffle exchange - // Note that we don't need to handle `ReusedExchangeExec` for non-AQE case, because - // the query plan won't be re-optimized/planned in non-AQE mode. - case s @ ShuffleQueryStageExec(_, ReusedExchangeExec(_, _: CometShuffleExchangeExec), _) => - newPlanWithProto(s, CometSinkPlaceHolder(_, s, s)) - - // Native shuffle for Comet operators - case s: ShuffleExchangeExec => - val nativeShuffle: Option[SparkPlan] = - if (nativeShuffleSupported(s)) { - val newOp = operator2ProtoIfAllChildrenAreNative(s) - newOp match { - case Some(nativeOp) => - // Switch to use Decimal128 regardless of precision, since Arrow native execution - // doesn't support Decimal32 and Decimal64 yet. - conf.setConfString(CometConf.COMET_USE_DECIMAL_128.key, "true") - val cometOp = CometShuffleExchangeExec(s, shuffleType = CometNativeShuffle) - Some(CometSinkPlaceHolder(nativeOp, s, cometOp)) - case None => - None + // For AQE shuffle stage on a Comet shuffle exchange + case s @ ShuffleQueryStageExec(_, _: CometShuffleExchangeExec, _) => + newPlanWithProto(s, CometSinkPlaceHolder(_, s, s)) + + // For AQE shuffle stage on a reused Comet shuffle exchange + // Note that we don't need to handle `ReusedExchangeExec` for non-AQE case, because + // the query plan won't be re-optimized/planned in non-AQE mode. + case s @ ShuffleQueryStageExec( + _, + ReusedExchangeExec(_, _: CometShuffleExchangeExec), + _) => + newPlanWithProto(s, CometSinkPlaceHolder(_, s, s)) + + // Native shuffle for Comet operators + case s: ShuffleExchangeExec => + val nativeShuffle: Option[SparkPlan] = + if (nativeShuffleSupported(s)) { + val newOp = operator2ProtoIfAllChildrenAreNative(s) + newOp match { + case Some(nativeOp) => + // Switch to use Decimal128 regardless of precision, since Arrow native execution + // doesn't support Decimal32 and Decimal64 yet. + conf.setConfString(CometConf.COMET_USE_DECIMAL_128.key, "true") + val cometOp = CometShuffleExchangeExec(s, shuffleType = CometNativeShuffle) + Some(CometSinkPlaceHolder(nativeOp, s, cometOp)) + case None => + None + } + } else { + None } + + val nativeOrColumnarShuffle = if (nativeShuffle.isDefined) { + nativeShuffle } else { - None + // Columnar shuffle for regular Spark operators (not Comet) and Comet operators + // (if configured). + // If the child of ShuffleExchangeExec is also a ShuffleExchangeExec, we should not + // convert it to CometColumnarShuffle, + if (columnarShuffleSupported(s)) { + val newOp = operator2Proto(s) + newOp match { + case Some(nativeOp) => + s.child match { + case n if n.isInstanceOf[CometNativeExec] || !n.supportsColumnar => + val cometOp = + CometShuffleExchangeExec(s, shuffleType = CometColumnarShuffle) + Some(CometSinkPlaceHolder(nativeOp, s, cometOp)) + case _ => + None + } + case None => + None + } + } else { + None + } } - val nativeOrColumnarShuffle = if (nativeShuffle.isDefined) { - nativeShuffle - } else { - // Columnar shuffle for regular Spark operators (not Comet) and Comet operators - // (if configured). - // If the child of ShuffleExchangeExec is also a ShuffleExchangeExec, we should not - // convert it to CometColumnarShuffle, - if (columnarShuffleSupported(s)) { - val newOp = operator2Proto(s) - newOp match { - case Some(nativeOp) => - s.child match { - case n if n.isInstanceOf[CometNativeExec] || !n.supportsColumnar => - val cometOp = - CometShuffleExchangeExec(s, shuffleType = CometColumnarShuffle) - Some(CometSinkPlaceHolder(nativeOp, s, cometOp)) - case _ => - None - } - case None => - None - } + if (nativeOrColumnarShuffle.isDefined) { + nativeOrColumnarShuffle.get } else { - None + s } - } - - if (nativeOrColumnarShuffle.isDefined) { - nativeOrColumnarShuffle.get - } else { - s - } - case op => - allExecs - .get(op.getClass) - .map(_.asInstanceOf[CometOperatorSerde[SparkPlan]]) match { - case Some(handler) => - if (op.children.forall(isCometNative)) { - if (isOperatorEnabled(handler, op)) { - val builder = OperatorOuterClass.Operator.newBuilder().setPlanId(op.id) - val childOp = op.children.map(_.asInstanceOf[CometNativeExec].nativeOp) - childOp.foreach(builder.addChildren) - return handler - .convert(op, builder, childOp: _*) - .map(handler.createExec(_, op)) - .getOrElse(op) + case op => + allExecs + .get(op.getClass) + .map(_.asInstanceOf[CometOperatorSerde[SparkPlan]]) match { + case Some(handler) => + if (op.children.forall(isCometNative)) { + if (isOperatorEnabled(handler, op)) { + val builder = OperatorOuterClass.Operator.newBuilder().setPlanId(op.id) + val childOp = op.children.map(_.asInstanceOf[CometNativeExec].nativeOp) + childOp.foreach(builder.addChildren) + return handler + .convert(op, builder, childOp: _*) + .map(handler.createExec(_, op)) + .getOrElse(op) + } + } else { + return op } - } else { - return op - } - case _ => - } + case _ => + } - op match { - case _: CometPlan | _: AQEShuffleReadExec | _: BroadcastExchangeExec | - _: BroadcastQueryStageExec | _: AdaptiveSparkPlanExec => - // Some execs should never be replaced. We include - // these cases specially here so we do not add a misleading 'info' message - op - case _: ExecutedCommandExec | _: V2CommandExec => - // Some execs that comet will not accelerate, such as command execs. - op - case _ => - if (!hasExplainInfo(op)) { - // An operator that is not supported by Comet - withInfo(op, s"${op.nodeName} is not supported") - } else { - // Already has fallback reason, do not override it + op match { + case _: CometPlan | _: AQEShuffleReadExec | _: BroadcastExchangeExec | + _: BroadcastQueryStageExec | _: AdaptiveSparkPlanExec => + // Some execs should never be replaced. We include + // these cases specially here so we do not add a misleading 'info' message op - } - } + case _: ExecutedCommandExec | _: V2CommandExec => + // Some execs that comet will not accelerate, such as command execs. + op + case _ => + if (!hasExplainInfo(op)) { + // An operator that is not supported by Comet + withInfo(op, s"${op.nodeName} is not supported") + } else { + // Already has fallback reason, do not override it + op + } + } + } } plan.transformUp { case op => @@ -424,7 +625,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = { val newPlan = _apply(plan) if (showTransformations && !newPlan.fastEquals(plan)) { - logInfo(s""" + println(s""" |=== Applying Rule $ruleName === |${sideBySide(plan.treeString, newPlan.treeString).mkString("\n")} |""".stripMargin) diff --git a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala index b4c3382b75..fffb9c8827 100644 --- a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala @@ -32,6 +32,29 @@ import org.apache.comet.serde.OperatorOuterClass.Operator class CometParquetWriterSuite extends CometTestBase { import testImplicits._ + test("basic parquet write") { + withTempPath { dir => + val outputPath = new File(dir, "output.parquet").getAbsolutePath + + // Create test data and write it to a temp parquet file first + withTempPath { inputDir => + val inputPath = new File(inputDir, "input.parquet").getAbsolutePath + val df = Seq((1, "a"), (2, "b"), (3, "c")).toDF("id", "value") + df.write.parquet(inputPath) + + withSQLConf( + CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", + CometConf.COMET_EXPLAIN_NATIVE_ENABLED.key -> "true") { + val df = spark.read.parquet(inputPath) + + df.write.parquet(outputPath) + + assert(spark.read.parquet(outputPath).count() == 3) + } + } + } + } + test("basic parquet write with native writer") { withTempPath { dir => val outputPath = new File(dir, "output.parquet").getAbsolutePath From ba93de73cf01f8e5b8d6f61e751b23b132fb5556 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 10:59:56 -0700 Subject: [PATCH 06/35] save --- .../org/apache/comet/rules/CometExecRule.scala | 13 ++++++++++--- .../spark/sql/comet/CometNativeWriteExec.scala | 8 ++++++-- 2 files changed, 16 insertions(+), 5 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 bac6564153..6313c3926f 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -125,16 +125,23 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { cmd.fileFormat match { case _: ParquetFileFormat => try { - // The child should be WriteFilesExec + // Use the already-transformed child plan from the WriteFilesExec + // The child has already been through Comet transformations val childPlan = exec.child match { - case writeFiles: WriteFilesExec => writeFiles.child - case other => other + case writeFiles: WriteFilesExec => + // The WriteFilesExec child should already be a Comet operator + writeFiles.child + case other => + // Fallback: use the child directly + other } // Get output path val outputPath = cmd.outputPath.toString println(s"Converting DataWritingCommandExec to native Parquet write: $outputPath") + println(s"Child plan class: ${childPlan.getClass.getName}") + println(s"Child plan: $childPlan") // Create native ParquetWriter operator val scanOp = OperatorOuterClass.Scan diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala index 3688d3a23e..b0096c511d 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala @@ -81,6 +81,10 @@ case class CometNativeWriteExec(nativeOp: Operator, child: SparkPlan, outputPath } } + // Capture metadata before the transformation + val numPartitions = childRDD.getNumPartitions + val numOutputCols = child.output.length + // Execute native write operation childRDD.mapPartitionsInternal { iter => val nativeMetrics = CometMetricNode.fromCometPlan(this) @@ -94,10 +98,10 @@ case class CometNativeWriteExec(nativeOp: Operator, child: SparkPlan, outputPath val cometIter = new org.apache.comet.CometExecIterator( CometExec.newIterId, Seq(iter), - child.output.length, + numOutputCols, planBytes, nativeMetrics, - childRDD.getNumPartitions, + numPartitions, org.apache.spark.TaskContext.getPartitionId(), None, Seq.empty) From 96087efdf3274b17ddb5cf03f5b2620ab53bbf12 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 11:25:37 -0700 Subject: [PATCH 07/35] save [skip ci] --- .../src/execution/operators/parquet_writer.rs | 41 +++++++++++++------ .../comet/CometParquetWriterSuite.scala | 5 +++ 2 files changed, 33 insertions(+), 13 deletions(-) diff --git a/native/core/src/execution/operators/parquet_writer.rs b/native/core/src/execution/operators/parquet_writer.rs index b4084f4c9f..4cd9f8f71c 100644 --- a/native/core/src/execution/operators/parquet_writer.rs +++ b/native/core/src/execution/operators/parquet_writer.rs @@ -22,7 +22,6 @@ use std::{ fmt, fmt::{Debug, Formatter}, fs::File, - path::Path, sync::Arc, }; @@ -31,12 +30,12 @@ use async_trait::async_trait; use datafusion::{ error::{DataFusionError, Result}, execution::context::TaskContext, - physical_expr::{EquivalenceProperties, Partitioning}, + physical_expr::EquivalenceProperties, physical_plan::{ execution_plan::{Boundedness, EmissionType}, metrics::{ExecutionPlanMetricsSet, MetricsSet}, - DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, SendableRecordBatchStream, - Statistics, + DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties, PlanProperties, + SendableRecordBatchStream, Statistics, }, }; use futures::TryStreamExt; @@ -70,9 +69,12 @@ impl ParquetWriterExec { output_path: String, compression: CompressionCodec, ) -> Result { + // Preserve the input's partitioning so each partition writes its own file + let input_partitioning = input.output_partitioning().clone(); + let cache = PlanProperties::new( EquivalenceProperties::new(Arc::clone(&input.schema())), - Partitioning::UnknownPartitioning(1), + input_partitioning, EmissionType::Final, Boundedness::Bounded, ); @@ -167,16 +169,29 @@ impl ExecutionPlan for ParquetWriterExec { let output_path = self.output_path.clone(); let compression = self.compression_to_parquet(); - // Create output directory if it doesn't exist - if let Some(parent) = Path::new(&output_path).parent() { - std::fs::create_dir_all(parent).map_err(|e| { - DataFusionError::Execution(format!("Failed to create output directory: {}", e)) - })?; - } + // Strip file:// or file: prefix if present + let local_path = output_path + .strip_prefix("file://") + .or_else(|| output_path.strip_prefix("file:")) + .unwrap_or(&output_path) + .to_string(); + + // Create output directory + std::fs::create_dir_all(&local_path).map_err(|e| { + DataFusionError::Execution(format!( + "Failed to create output directory '{}': {}", + local_path, e + )) + })?; + + // Generate part file name for this partition + let part_file = format!("{}/part-{:05}.snappy.parquet", local_path, partition); + + println!("ParquetWriter executing: partition={}, output={}", partition, part_file); // Create the Parquet file - let file = File::create(&output_path).map_err(|e| { - DataFusionError::Execution(format!("Failed to create output file: {}", e)) + let file = File::create(&part_file).map_err(|e| { + DataFusionError::Execution(format!("Failed to create output file '{}': {}", part_file, e)) })?; // Configure writer properties diff --git a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala index fffb9c8827..a7e37faea7 100644 --- a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala @@ -49,6 +49,11 @@ class CometParquetWriterSuite extends CometTestBase { df.write.parquet(outputPath) + spark.read.parquet(outputPath).show() + // scalastyle:off + println(outputPath) + + // Thread.sleep(60000) assert(spark.read.parquet(outputPath).count() == 3) } } From e6bb5cfd98b2dd2923c7820ac3cfb67650eb97d5 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 11:26:40 -0700 Subject: [PATCH 08/35] delete some tests --- .../comet/CometParquetWriterSuite.scala | 180 ------------------ 1 file changed, 180 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala index a7e37faea7..368b2e96cc 100644 --- a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala @@ -60,184 +60,4 @@ class CometParquetWriterSuite extends CometTestBase { } } - test("basic parquet write with native writer") { - withTempPath { dir => - val outputPath = new File(dir, "output.parquet").getAbsolutePath - - // Create test data and write it to a temp parquet file first - withTempPath { inputDir => - val inputPath = new File(inputDir, "input.parquet").getAbsolutePath - val df = Seq((1, "a"), (2, "b"), (3, "c")).toDF("id", "value") - df.write.parquet(inputPath) - - // Now use native writer to read and write - // Collect data as ColumnarBatches (for this small test) - val inputBatches = spark.read - .parquet(inputPath) - .queryExecution - .executedPlan - .executeColumnar() - .collect() - .iterator - - // Create native plan for ParquetWriter - val scanOp = OperatorOuterClass.Scan - .newBuilder() - .setSource("test_input") - .setArrowFfiSafe(true) - .addFields(QueryPlanSerde.serializeDataType(IntegerType).get) - .addFields(QueryPlanSerde.serializeDataType(StringType).get) - .build() - - val scanOperator = Operator - .newBuilder() - .setPlanId(1) - .setScan(scanOp) - .build() - - val writerOp = OperatorOuterClass.ParquetWriter - .newBuilder() - .setOutputPath(outputPath) - .setCompression(OperatorOuterClass.CompressionCodec.Snappy) - .build() - - val writerOperator = Operator - .newBuilder() - .setPlanId(2) - .addChildren(scanOperator) - .setParquetWriter(writerOp) - .build() - - // Execute native write - val nativeMetrics = CometMetricNode(Map.empty[String, SQLMetric]) - - val cometIter = CometExec.getCometIterator( - Seq(inputBatches), - 2, // numColumns - writerOperator, - nativeMetrics, - 1, // numPartitions - 0, // partitionIndex - broadcastedHadoopConfForEncryption = None, - encryptedFilePaths = Seq.empty) - - // Consume the iterator - while (cometIter.hasNext) { - cometIter.next() - } - cometIter.close() - - // Verify the file was written - assert(new File(outputPath).exists(), s"Output file should exist at $outputPath") - assert(new File(outputPath).length() > 0, "Output file should not be empty") - - // Verify we can read the data back with Spark (disable Comet for read) - withSQLConf("spark.comet.enabled" -> "false") { - val readDf = spark.read.parquet(outputPath) - val result = readDf.collect().sortBy(_.getInt(0)) - - assert(result.length == 3) - assert(result(0).getInt(0) == 1) - assert(result(0).getString(1) == "a") - assert(result(1).getInt(0) == 2) - assert(result(1).getString(1) == "b") - assert(result(2).getInt(0) == 3) - assert(result(2).getString(1) == "c") - } - } - } - } - - test("end-to-end DataFrame write with CometNativeWriteExec") { - withTempPath { dir => - val outputPath = new File(dir, "output.parquet").getAbsolutePath - - // Create test data and write it to a temp parquet file first - withTempPath { inputDir => - val inputPath = new File(inputDir, "input.parquet").getAbsolutePath - val df = Seq((1, "a"), (2, "b"), (3, "c")).toDF("id", "value") - df.write.parquet(inputPath) - - // Read from parquet to get a columnar plan - val childPlan = spark.read.parquet(inputPath).queryExecution.executedPlan - - // Create ParquetWriter operator - val scanOp = OperatorOuterClass.Scan - .newBuilder() - .setSource("test_input") - .setArrowFfiSafe(true) - .addFields(QueryPlanSerde.serializeDataType(IntegerType).get) - .addFields(QueryPlanSerde.serializeDataType(StringType).get) - .build() - - val scanOperator = Operator - .newBuilder() - .setPlanId(1) - .setScan(scanOp) - .build() - - val writerOp = OperatorOuterClass.ParquetWriter - .newBuilder() - .setOutputPath(outputPath) - .setCompression(OperatorOuterClass.CompressionCodec.Snappy) - .build() - - val writerOperator = Operator - .newBuilder() - .setPlanId(2) - .addChildren(scanOperator) - .setParquetWriter(writerOp) - .build() - - // Create CometNativeWriteExec - val writeExec = CometNativeWriteExec(writerOperator, childPlan, outputPath) - - // Execute the write - writeExec.executeColumnar().count() - - // Verify the file was written - assert(new File(outputPath).exists(), s"Output file should exist at $outputPath") - assert(new File(outputPath).length() > 0, "Output file should not be empty") - - // Verify we can read the data back (disable Comet for read) - withSQLConf("spark.comet.enabled" -> "false") { - val readDf = spark.read.parquet(outputPath) - val result = readDf.collect().sortBy(_.getInt(0)) - - assert(result.length == 3) - assert(result(0).getInt(0) == 1) - assert(result(0).getString(1) == "a") - assert(result(1).getInt(0) == 2) - assert(result(1).getString(1) == "b") - assert(result(2).getInt(0) == 3) - assert(result(2).getString(1) == "c") - } - } - } - } - - test("verify parquet writer operator serialization") { - val outputPath = "/tmp/test_output.parquet" - - val writerOp = OperatorOuterClass.ParquetWriter - .newBuilder() - .setOutputPath(outputPath) - .setCompression(OperatorOuterClass.CompressionCodec.Zstd) - .build() - - val operator = Operator - .newBuilder() - .setPlanId(1) - .setParquetWriter(writerOp) - .build() - - // Verify the operator can be serialized and deserialized - val serialized = operator.toByteArray - val deserialized = Operator.parseFrom(serialized) - - assert(deserialized.hasParquetWriter) - assert(deserialized.getParquetWriter.getOutputPath == outputPath) - assert( - deserialized.getParquetWriter.getCompression == OperatorOuterClass.CompressionCodec.Zstd) - } } From 8c5a6bed5878cde482478d6e00f0c9f7e7616d04 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 11:41:04 -0700 Subject: [PATCH 09/35] test passes --- .github/workflows/pr_build_linux.yml | 1 + .github/workflows/pr_build_macos.yml | 1 + .../src/execution/operators/parquet_writer.rs | 17 ++++++++++++++--- native/core/src/execution/planner.rs | 1 + native/proto/src/proto/operator.proto | 1 + .../org/apache/comet/rules/CometExecRule.scala | 8 ++++---- .../spark/sql/comet/CometNativeWriteExec.scala | 12 +++++++++--- .../apache/comet/CometParquetWriterSuite.scala | 8 +------- 8 files changed, 32 insertions(+), 17 deletions(-) diff --git a/.github/workflows/pr_build_linux.yml b/.github/workflows/pr_build_linux.yml index 8570c7e7ae..026d3ccacf 100644 --- a/.github/workflows/pr_build_linux.yml +++ b/.github/workflows/pr_build_linux.yml @@ -118,6 +118,7 @@ jobs: org.apache.comet.exec.DisableAQECometAsyncShuffleSuite - name: "parquet" value: | + org.apache.comet.CometParquetWriterSuite org.apache.comet.parquet.ParquetReadV1Suite org.apache.comet.parquet.ParquetReadV2Suite org.apache.comet.parquet.ParquetReadFromFakeHadoopFsSuite diff --git a/.github/workflows/pr_build_macos.yml b/.github/workflows/pr_build_macos.yml index 5eb6be4ee1..2c76b8fa1e 100644 --- a/.github/workflows/pr_build_macos.yml +++ b/.github/workflows/pr_build_macos.yml @@ -83,6 +83,7 @@ jobs: org.apache.comet.exec.DisableAQECometAsyncShuffleSuite - name: "parquet" value: | + org.apache.comet.CometParquetWriterSuite org.apache.comet.parquet.ParquetReadV1Suite org.apache.comet.parquet.ParquetReadV2Suite org.apache.comet.parquet.ParquetReadFromFakeHadoopFsSuite diff --git a/native/core/src/execution/operators/parquet_writer.rs b/native/core/src/execution/operators/parquet_writer.rs index 4cd9f8f71c..0bdca26dee 100644 --- a/native/core/src/execution/operators/parquet_writer.rs +++ b/native/core/src/execution/operators/parquet_writer.rs @@ -56,6 +56,8 @@ pub struct ParquetWriterExec { output_path: String, /// Compression codec compression: CompressionCodec, + /// Partition ID (from Spark TaskContext) + partition_id: i32, /// Metrics metrics: ExecutionPlanMetricsSet, /// Cache for plan properties @@ -68,6 +70,7 @@ impl ParquetWriterExec { input: Arc, output_path: String, compression: CompressionCodec, + partition_id: i32, ) -> Result { // Preserve the input's partitioning so each partition writes its own file let input_partitioning = input.output_partitioning().clone(); @@ -83,6 +86,7 @@ impl ParquetWriterExec { input, output_path, compression, + partition_id, metrics: ExecutionPlanMetricsSet::new(), cache, }) @@ -152,6 +156,7 @@ impl ExecutionPlan for ParquetWriterExec { Arc::clone(&children[0]), self.output_path.clone(), self.compression.clone(), + self.partition_id, )?)), _ => Err(DataFusionError::Internal( "ParquetWriterExec requires exactly one child".to_string(), @@ -185,13 +190,19 @@ impl ExecutionPlan for ParquetWriterExec { })?; // Generate part file name for this partition - let part_file = format!("{}/part-{:05}.snappy.parquet", local_path, partition); + let part_file = format!("{}/part-{:05}.snappy.parquet", local_path, self.partition_id); - println!("ParquetWriter executing: partition={}, output={}", partition, part_file); + println!( + "ParquetWriter executing: partition={}, output={}", + self.partition_id, part_file + ); // Create the Parquet file let file = File::create(&part_file).map_err(|e| { - DataFusionError::Execution(format!("Failed to create output file '{}': {}", part_file, e)) + DataFusionError::Execution(format!( + "Failed to create output file '{}': {}", + part_file, e + )) })?; // Configure writer properties diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 60d3bd9b17..cd8c5421dc 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1467,6 +1467,7 @@ impl PhysicalPlanner { Arc::clone(&child.native_plan), writer.output_path.clone(), codec, + writer.partition_id, )?); Ok(( diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 44c84b7273..34074135cb 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -240,6 +240,7 @@ message ShuffleWriter { message ParquetWriter { string output_path = 1; CompressionCodec compression = 2; + int32 partition_id = 3; } enum AggregateMode { 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 6313c3926f..9c85d67420 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -33,7 +33,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.{FileFormat, InsertIntoHadoopFsRelationCommand, WriteFilesExec} +import org.apache.spark.sql.execution.datasources.{InsertIntoHadoopFsRelationCommand, WriteFilesExec} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.datasources.v2.V2CommandExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} @@ -186,7 +186,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { Some(writeExec) } catch { case e: Exception => - logWarning(s"Failed to convert DataWritingCommandExec to native execution", e) + logWarning("Failed to convert DataWritingCommandExec to native execution", e) None } case _ => @@ -217,7 +217,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { None } catch { case e: Exception => - logWarning(s"Failed to convert WriteFilesExec to native execution", e) + logWarning("Failed to convert WriteFilesExec to native execution", e) None } case _ => @@ -287,7 +287,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { Some(writeExec) } catch { case e: Exception => - logWarning(s"Failed to convert write command to native execution", e) + logWarning("Failed to convert write command to native execution", e) None } case _ => diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala index b0096c511d..39083010c8 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala @@ -84,14 +84,20 @@ case class CometNativeWriteExec(nativeOp: Operator, child: SparkPlan, outputPath // Capture metadata before the transformation val numPartitions = childRDD.getNumPartitions val numOutputCols = child.output.length + val baseNativeOp = nativeOp // The base operator without partition_id set // Execute native write operation childRDD.mapPartitionsInternal { iter => val nativeMetrics = CometMetricNode.fromCometPlan(this) + val partitionId = org.apache.spark.TaskContext.getPartitionId() - // Serialize the native plan + // Create a new operator with the partition ID set for this specific partition + val writerOp = baseNativeOp.getParquetWriter.toBuilder.setPartitionId(partitionId).build() + val partitionSpecificOp = baseNativeOp.toBuilder.setParquetWriter(writerOp).build() + + // Serialize the partition-specific plan val outputStream = new java.io.ByteArrayOutputStream() - nativeOp.writeTo(outputStream) + partitionSpecificOp.writeTo(outputStream) outputStream.close() val planBytes = outputStream.toByteArray @@ -102,7 +108,7 @@ case class CometNativeWriteExec(nativeOp: Operator, child: SparkPlan, outputPath planBytes, nativeMetrics, numPartitions, - org.apache.spark.TaskContext.getPartitionId(), + partitionId, None, Seq.empty) diff --git a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala index 368b2e96cc..1d65b26a7b 100644 --- a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala @@ -21,13 +21,7 @@ package org.apache.comet import java.io.File -import org.apache.spark.sql.{CometTestBase, Row} -import org.apache.spark.sql.comet.{CometExec, CometMetricNode, CometNativeWriteExec} -import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.types.{IntegerType, StringType} - -import org.apache.comet.serde.{OperatorOuterClass, QueryPlanSerde} -import org.apache.comet.serde.OperatorOuterClass.Operator +import org.apache.spark.sql.CometTestBase class CometParquetWriterSuite extends CometTestBase { import testImplicits._ From 42e0c7953acc913ff561021bc4561ac559a5b6a1 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 11:44:02 -0700 Subject: [PATCH 10/35] prep for review --- .../src/execution/operators/parquet_writer.rs | 5 +- .../apache/comet/rules/CometExecRule.scala | 377 ++++++++---------- .../comet/CometParquetWriterSuite.scala | 6 +- 3 files changed, 171 insertions(+), 217 deletions(-) diff --git a/native/core/src/execution/operators/parquet_writer.rs b/native/core/src/execution/operators/parquet_writer.rs index 0bdca26dee..a1a2539e2d 100644 --- a/native/core/src/execution/operators/parquet_writer.rs +++ b/native/core/src/execution/operators/parquet_writer.rs @@ -190,7 +190,10 @@ impl ExecutionPlan for ParquetWriterExec { })?; // Generate part file name for this partition - let part_file = format!("{}/part-{:05}.snappy.parquet", local_path, self.partition_id); + let part_file = format!( + "{}/part-{:05}.snappy.parquet", + local_path, self.partition_id + ); println!( "ParquetWriter executing: partition={}, output={}", 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 9c85d67420..f1c3caaecb 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -51,8 +51,6 @@ import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.QueryPlanSerde.{serializeDataType, supportedDataType} import org.apache.comet.serde.operator._ -// scalastyle:off - object CometExecRule { /** @@ -139,10 +137,6 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { // Get output path val outputPath = cmd.outputPath.toString - println(s"Converting DataWritingCommandExec to native Parquet write: $outputPath") - println(s"Child plan class: ${childPlan.getClass.getName}") - println(s"Child plan: $childPlan") - // Create native ParquetWriter operator val scanOp = OperatorOuterClass.Scan .newBuilder() @@ -180,10 +174,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { .build() // Create CometNativeWriteExec with the transformed child plan - val writeExec = CometNativeWriteExec(writerOperator, childPlan, outputPath) - - println(s"Successfully converted to native Parquet write: $outputPath") - Some(writeExec) + Some(CometNativeWriteExec(writerOperator, childPlan, outputPath)) } catch { case e: Exception => logWarning("Failed to convert DataWritingCommandExec to native execution", e) @@ -199,33 +190,6 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { } } - /** - * Try to convert a WriteFilesExec to use Comet native Parquet writer. Returns - * Some(CometNativeWriteExec) if conversion is successful, None otherwise. - */ - private def tryConvertWriteFilesExec(writeOp: WriteFilesExec): Option[SparkPlan] = { - // Check if this is a Parquet write - writeOp.fileFormat match { - case _: ParquetFileFormat => - try { - // The child plan is the data source - val childPlan = writeOp.child - - // Get output path - WriteFilesExec doesn't have outputPath directly, - // we need to get it from the parent DataWritingCommandExec - // For now, we'll skip this operator and handle it in tryConvertWriteCommand - None - } catch { - case e: Exception => - logWarning("Failed to convert WriteFilesExec to native execution", e) - None - } - case _ => - // Not a Parquet write, skip - None - } - } - /** * Try to convert a write command (ExecutedCommandExec) to use Comet native Parquet writer. * Returns Some(CometNativeWriteExec) if conversion is successful, None otherwise. @@ -380,196 +344,187 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { operator2ProtoIfAllChildrenAreNative(op).map(fun).getOrElse(op) } - def convertNode(op: SparkPlan): SparkPlan = { - - println(s"convertNode: [${op.getClass}] $op") - - op match { - // Fully native scan for V1 - case scan: CometScanExec if scan.scanImpl == CometConf.SCAN_NATIVE_DATAFUSION => - val nativeOp = operator2Proto(scan).get - CometNativeScan.createExec(nativeOp, scan) - - // Fully native Iceberg scan for V2 (iceberg-rust path) - // Only handle scans with native metadata; SupportsComet scans fall through to isCometScan - // Config checks (COMET_ICEBERG_NATIVE_ENABLED, COMET_EXEC_ENABLED) are done in CometScanRule - case scan: CometBatchScanExec if scan.nativeIcebergScanMetadata.isDefined => - operator2Proto(scan) match { - case Some(nativeOp) => - CometIcebergNativeScan.createExec(nativeOp, scan) - case None => - // Serialization failed, fall back to CometBatchScanExec - scan - } + def convertNode(op: SparkPlan): SparkPlan = op match { + // Fully native scan for V1 + case scan: CometScanExec if scan.scanImpl == CometConf.SCAN_NATIVE_DATAFUSION => + val nativeOp = operator2Proto(scan).get + CometNativeScan.createExec(nativeOp, scan) - // Comet JVM + native scan for V1 and V2 - case op if isCometScan(op) => - val nativeOp = operator2Proto(op) - CometScanWrapper(nativeOp.get, op) - - case op if shouldApplySparkToColumnar(conf, op) => - val cometOp = CometSparkToColumnarExec(op) - val nativeOp = operator2Proto(cometOp) - CometScanWrapper(nativeOp.get, cometOp) - - // Intercept DataWritingCommandExec (Spark 3.5+) - case exec: DataWritingCommandExec - if CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.get(conf) => - tryConvertDataWritingCommand(exec).getOrElse(exec) - - // Intercept Parquet write commands (fallback for older Spark versions) - case exec: ExecutedCommandExec - if CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.get(conf) => - tryConvertWriteCommand(exec).getOrElse(exec) - - // For AQE broadcast stage on a Comet broadcast exchange - case s @ BroadcastQueryStageExec(_, _: CometBroadcastExchangeExec, _) => - newPlanWithProto(s, CometSinkPlaceHolder(_, s, s)) - - case s @ BroadcastQueryStageExec( - _, - ReusedExchangeExec(_, _: CometBroadcastExchangeExec), - _) => - newPlanWithProto(s, CometSinkPlaceHolder(_, s, s)) - - // `CometBroadcastExchangeExec`'s broadcast output is not compatible with Spark's broadcast - // exchange. It is only used for Comet native execution. We only transform Spark broadcast - // exchange to Comet broadcast exchange if its downstream is a Comet native plan or if the - // broadcast exchange is forced to be enabled by Comet config. - case plan if plan.children.exists(_.isInstanceOf[BroadcastExchangeExec]) => - val newChildren = plan.children.map { - case b: BroadcastExchangeExec - if isCometNative(b.child) && - CometConf.COMET_EXEC_BROADCAST_EXCHANGE_ENABLED.get(conf) => - operator2Proto(b) match { - case Some(nativeOp) => - val cometOp = CometBroadcastExchangeExec(b, b.output, b.mode, b.child) - CometSinkPlaceHolder(nativeOp, b, cometOp) - case None => b - } - case other => other - } - if (!newChildren.exists(_.isInstanceOf[BroadcastExchangeExec])) { - val newPlan = convertNode(plan.withNewChildren(newChildren)) - if (isCometNative(newPlan) || isCometBroadCastForceEnabled(conf)) { - newPlan - } else { - if (isCometNative(newPlan)) { - val reason = - getCometBroadcastNotEnabledReason(conf).getOrElse("no reason available") - withInfo(plan, s"Broadcast is not enabled: $reason") - } - plan + // Fully native Iceberg scan for V2 (iceberg-rust path) + // Only handle scans with native metadata; SupportsComet scans fall through to isCometScan + // Config checks (COMET_ICEBERG_NATIVE_ENABLED, COMET_EXEC_ENABLED) are done in CometScanRule + case scan: CometBatchScanExec if scan.nativeIcebergScanMetadata.isDefined => + operator2Proto(scan) match { + case Some(nativeOp) => + CometIcebergNativeScan.createExec(nativeOp, scan) + case None => + // Serialization failed, fall back to CometBatchScanExec + scan + } + + // Comet JVM + native scan for V1 and V2 + case op if isCometScan(op) => + val nativeOp = operator2Proto(op) + CometScanWrapper(nativeOp.get, op) + + case op if shouldApplySparkToColumnar(conf, op) => + val cometOp = CometSparkToColumnarExec(op) + val nativeOp = operator2Proto(cometOp) + CometScanWrapper(nativeOp.get, cometOp) + + // Intercept DataWritingCommandExec (Spark 3.5+) + case exec: DataWritingCommandExec + if CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.get(conf) => + tryConvertDataWritingCommand(exec).getOrElse(exec) + + // Intercept Parquet write commands (fallback for older Spark versions) + case exec: ExecutedCommandExec if CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.get(conf) => + tryConvertWriteCommand(exec).getOrElse(exec) + + // For AQE broadcast stage on a Comet broadcast exchange + case s @ BroadcastQueryStageExec(_, _: CometBroadcastExchangeExec, _) => + newPlanWithProto(s, CometSinkPlaceHolder(_, s, s)) + + case s @ BroadcastQueryStageExec( + _, + ReusedExchangeExec(_, _: CometBroadcastExchangeExec), + _) => + newPlanWithProto(s, CometSinkPlaceHolder(_, s, s)) + + // `CometBroadcastExchangeExec`'s broadcast output is not compatible with Spark's broadcast + // exchange. It is only used for Comet native execution. We only transform Spark broadcast + // exchange to Comet broadcast exchange if its downstream is a Comet native plan or if the + // broadcast exchange is forced to be enabled by Comet config. + case plan if plan.children.exists(_.isInstanceOf[BroadcastExchangeExec]) => + val newChildren = plan.children.map { + case b: BroadcastExchangeExec + if isCometNative(b.child) && + CometConf.COMET_EXEC_BROADCAST_EXCHANGE_ENABLED.get(conf) => + operator2Proto(b) match { + case Some(nativeOp) => + val cometOp = CometBroadcastExchangeExec(b, b.output, b.mode, b.child) + CometSinkPlaceHolder(nativeOp, b, cometOp) + case None => b } + case other => other + } + if (!newChildren.exists(_.isInstanceOf[BroadcastExchangeExec])) { + val newPlan = convertNode(plan.withNewChildren(newChildren)) + if (isCometNative(newPlan) || isCometBroadCastForceEnabled(conf)) { + newPlan } else { + if (isCometNative(newPlan)) { + val reason = + getCometBroadcastNotEnabledReason(conf).getOrElse("no reason available") + withInfo(plan, s"Broadcast is not enabled: $reason") + } plan } + } else { + plan + } - // For AQE shuffle stage on a Comet shuffle exchange - case s @ ShuffleQueryStageExec(_, _: CometShuffleExchangeExec, _) => - newPlanWithProto(s, CometSinkPlaceHolder(_, s, s)) - - // For AQE shuffle stage on a reused Comet shuffle exchange - // Note that we don't need to handle `ReusedExchangeExec` for non-AQE case, because - // the query plan won't be re-optimized/planned in non-AQE mode. - case s @ ShuffleQueryStageExec( - _, - ReusedExchangeExec(_, _: CometShuffleExchangeExec), - _) => - newPlanWithProto(s, CometSinkPlaceHolder(_, s, s)) - - // Native shuffle for Comet operators - case s: ShuffleExchangeExec => - val nativeShuffle: Option[SparkPlan] = - if (nativeShuffleSupported(s)) { - val newOp = operator2ProtoIfAllChildrenAreNative(s) - newOp match { - case Some(nativeOp) => - // Switch to use Decimal128 regardless of precision, since Arrow native execution - // doesn't support Decimal32 and Decimal64 yet. - conf.setConfString(CometConf.COMET_USE_DECIMAL_128.key, "true") - val cometOp = CometShuffleExchangeExec(s, shuffleType = CometNativeShuffle) - Some(CometSinkPlaceHolder(nativeOp, s, cometOp)) - case None => - None - } - } else { - None + // For AQE shuffle stage on a Comet shuffle exchange + case s @ ShuffleQueryStageExec(_, _: CometShuffleExchangeExec, _) => + newPlanWithProto(s, CometSinkPlaceHolder(_, s, s)) + + // For AQE shuffle stage on a reused Comet shuffle exchange + // Note that we don't need to handle `ReusedExchangeExec` for non-AQE case, because + // the query plan won't be re-optimized/planned in non-AQE mode. + case s @ ShuffleQueryStageExec(_, ReusedExchangeExec(_, _: CometShuffleExchangeExec), _) => + newPlanWithProto(s, CometSinkPlaceHolder(_, s, s)) + + // Native shuffle for Comet operators + case s: ShuffleExchangeExec => + val nativeShuffle: Option[SparkPlan] = + if (nativeShuffleSupported(s)) { + val newOp = operator2ProtoIfAllChildrenAreNative(s) + newOp match { + case Some(nativeOp) => + // Switch to use Decimal128 regardless of precision, since Arrow native execution + // doesn't support Decimal32 and Decimal64 yet. + conf.setConfString(CometConf.COMET_USE_DECIMAL_128.key, "true") + val cometOp = CometShuffleExchangeExec(s, shuffleType = CometNativeShuffle) + Some(CometSinkPlaceHolder(nativeOp, s, cometOp)) + case None => + None } - - val nativeOrColumnarShuffle = if (nativeShuffle.isDefined) { - nativeShuffle } else { - // Columnar shuffle for regular Spark operators (not Comet) and Comet operators - // (if configured). - // If the child of ShuffleExchangeExec is also a ShuffleExchangeExec, we should not - // convert it to CometColumnarShuffle, - if (columnarShuffleSupported(s)) { - val newOp = operator2Proto(s) - newOp match { - case Some(nativeOp) => - s.child match { - case n if n.isInstanceOf[CometNativeExec] || !n.supportsColumnar => - val cometOp = - CometShuffleExchangeExec(s, shuffleType = CometColumnarShuffle) - Some(CometSinkPlaceHolder(nativeOp, s, cometOp)) - case _ => - None - } - case None => - None - } - } else { - None - } + None } - if (nativeOrColumnarShuffle.isDefined) { - nativeOrColumnarShuffle.get + val nativeOrColumnarShuffle = if (nativeShuffle.isDefined) { + nativeShuffle + } else { + // Columnar shuffle for regular Spark operators (not Comet) and Comet operators + // (if configured). + // If the child of ShuffleExchangeExec is also a ShuffleExchangeExec, we should not + // convert it to CometColumnarShuffle, + if (columnarShuffleSupported(s)) { + val newOp = operator2Proto(s) + newOp match { + case Some(nativeOp) => + s.child match { + case n if n.isInstanceOf[CometNativeExec] || !n.supportsColumnar => + val cometOp = + CometShuffleExchangeExec(s, shuffleType = CometColumnarShuffle) + Some(CometSinkPlaceHolder(nativeOp, s, cometOp)) + case _ => + None + } + case None => + None + } } else { - s + None } + } - case op => - allExecs - .get(op.getClass) - .map(_.asInstanceOf[CometOperatorSerde[SparkPlan]]) match { - case Some(handler) => - if (op.children.forall(isCometNative)) { - if (isOperatorEnabled(handler, op)) { - val builder = OperatorOuterClass.Operator.newBuilder().setPlanId(op.id) - val childOp = op.children.map(_.asInstanceOf[CometNativeExec].nativeOp) - childOp.foreach(builder.addChildren) - return handler - .convert(op, builder, childOp: _*) - .map(handler.createExec(_, op)) - .getOrElse(op) - } - } else { - return op + if (nativeOrColumnarShuffle.isDefined) { + nativeOrColumnarShuffle.get + } else { + s + } + + case op => + allExecs + .get(op.getClass) + .map(_.asInstanceOf[CometOperatorSerde[SparkPlan]]) match { + case Some(handler) => + if (op.children.forall(isCometNative)) { + if (isOperatorEnabled(handler, op)) { + val builder = OperatorOuterClass.Operator.newBuilder().setPlanId(op.id) + val childOp = op.children.map(_.asInstanceOf[CometNativeExec].nativeOp) + childOp.foreach(builder.addChildren) + return handler + .convert(op, builder, childOp: _*) + .map(handler.createExec(_, op)) + .getOrElse(op) } - case _ => - } + } else { + return op + } + case _ => + } - op match { - case _: CometPlan | _: AQEShuffleReadExec | _: BroadcastExchangeExec | - _: BroadcastQueryStageExec | _: AdaptiveSparkPlanExec => - // Some execs should never be replaced. We include - // these cases specially here so we do not add a misleading 'info' message - op - case _: ExecutedCommandExec | _: V2CommandExec => - // Some execs that comet will not accelerate, such as command execs. + op match { + case _: CometPlan | _: AQEShuffleReadExec | _: BroadcastExchangeExec | + _: BroadcastQueryStageExec | _: AdaptiveSparkPlanExec => + // Some execs should never be replaced. We include + // these cases specially here so we do not add a misleading 'info' message + op + case _: ExecutedCommandExec | _: V2CommandExec => + // Some execs that comet will not accelerate, such as command execs. + op + case _ => + if (!hasExplainInfo(op)) { + // An operator that is not supported by Comet + withInfo(op, s"${op.nodeName} is not supported") + } else { + // Already has fallback reason, do not override it op - case _ => - if (!hasExplainInfo(op)) { - // An operator that is not supported by Comet - withInfo(op, s"${op.nodeName} is not supported") - } else { - // Already has fallback reason, do not override it - op - } - } - } + } + } } plan.transformUp { case op => diff --git a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala index 1d65b26a7b..32a6c2f2ca 100644 --- a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala @@ -41,13 +41,9 @@ class CometParquetWriterSuite extends CometTestBase { CometConf.COMET_EXPLAIN_NATIVE_ENABLED.key -> "true") { val df = spark.read.parquet(inputPath) + // perform native write df.write.parquet(outputPath) - spark.read.parquet(outputPath).show() - // scalastyle:off - println(outputPath) - - // Thread.sleep(60000) assert(spark.read.parquet(outputPath).count() == 3) } } From fac5c562919ca18b9d256b240f69a683e5eb19a3 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 11:52:58 -0700 Subject: [PATCH 11/35] improve test --- .../src/execution/operators/parquet_writer.rs | 5 ---- .../comet/CometParquetWriterSuite.scala | 23 +++++++++++++++---- 2 files changed, 19 insertions(+), 9 deletions(-) diff --git a/native/core/src/execution/operators/parquet_writer.rs b/native/core/src/execution/operators/parquet_writer.rs index a1a2539e2d..8ec12d0c1a 100644 --- a/native/core/src/execution/operators/parquet_writer.rs +++ b/native/core/src/execution/operators/parquet_writer.rs @@ -195,11 +195,6 @@ impl ExecutionPlan for ParquetWriterExec { local_path, self.partition_id ); - println!( - "ParquetWriter executing: partition={}, output={}", - self.partition_id, part_file - ); - // Create the Parquet file let file = File::create(&part_file).map_err(|e| { DataFusionError::Execution(format!( diff --git a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala index 32a6c2f2ca..1f0a63f5a2 100644 --- a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala @@ -36,15 +36,30 @@ class CometParquetWriterSuite extends CometTestBase { val df = Seq((1, "a"), (2, "b"), (3, "c")).toDF("id", "value") df.write.parquet(inputPath) - withSQLConf( - CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", - CometConf.COMET_EXPLAIN_NATIVE_ENABLED.key -> "true") { + withSQLConf(CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true") { val df = spark.read.parquet(inputPath) // perform native write df.write.parquet(outputPath) - assert(spark.read.parquet(outputPath).count() == 3) + // Verify the data was written correctly + val resultDf = spark.read.parquet(outputPath) + assert(resultDf.count() == 3, "Expected 3 rows to be written") + + // Verify correct data + // TODO native parquet writer loses column names + val rows = resultDf.orderBy("col_0").collect() + assert(rows.length == 3) + assert(rows(0).getInt(0) == 1 && rows(0).getString(1) == "a") + assert(rows(1).getInt(0) == 2 && rows(1).getString(1) == "b") + assert(rows(2).getInt(0) == 3 && rows(2).getString(1) == "c") + + // Verify multiple part files were created + val outputDir = new File(outputPath) + val partFiles = outputDir.listFiles().filter(_.getName.startsWith("part-")) + // With 3 rows and default parallelism, we should get multiple partitions + assert(partFiles.length > 0, "Expected at least one part file to be created") + println(s"Created ${partFiles.length} part file(s)") } } } From 30a503f31914edabf1b16847b21850fc4cc22b42 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 12:01:17 -0700 Subject: [PATCH 12/35] prep for review --- .../src/main/scala/org/apache/comet/CometConf.scala | 5 +++-- docs/source/user-guide/latest/configs.md | 2 +- .../scala/org/apache/comet/rules/CometExecRule.scala | 12 ++++-------- 3 files changed, 8 insertions(+), 11 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 017b571053..b34d9ccaec 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -105,8 +105,9 @@ object CometConf extends ShimCometConf { .category(CATEGORY_PARQUET) .doc( "Whether to enable native Parquet write through Comet. When enabled, " + - "Comet will intercept Parquet write operations and execute them natively " + - "for improved performance.") + "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.") .booleanConf .createWithDefault(false) diff --git a/docs/source/user-guide/latest/configs.md b/docs/source/user-guide/latest/configs.md index 0d2489c449..1fa7311bef 100644 --- a/docs/source/user-guide/latest/configs.md +++ b/docs/source/user-guide/latest/configs.md @@ -51,7 +51,7 @@ Comet provides the following configuration settings. | `spark.comet.parquet.read.parallel.io.enabled` | Whether to enable Comet's parallel reader for Parquet files. The parallel reader reads ranges of consecutive data in a file in parallel. It is faster for large files and row groups but uses more resources. | true | | `spark.comet.parquet.read.parallel.io.thread-pool.size` | The maximum number of parallel threads the parallel reader will use in a single executor. For executors configured with a smaller number of cores, use a smaller number. | 16 | | `spark.comet.parquet.respectFilterPushdown` | Whether to respect Spark's PARQUET_FILTER_PUSHDOWN_ENABLED config. This needs to be respected when running the Spark SQL test suite but the default setting results in poor performance in Comet when using the new native scans, disabled by default | false | -| `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 for improved performance. | false | +| `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 | 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 f1c3caaecb..612a8effce 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -226,7 +226,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { val scanOperator = Operator .newBuilder() - .setPlanId(1) + .setPlanId(1) // TODO use real plan id .setScan(scanOp.build()) .build() @@ -239,16 +239,12 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { val writerOperator = Operator .newBuilder() - .setPlanId(2) + .setPlanId(2) // TODO use real plan id .addChildren(scanOperator) .setParquetWriter(writerOp) .build() - // Create CometNativeWriteExec - val writeExec = CometNativeWriteExec(writerOperator, childPlan, outputPath) - - println(s"Converted Parquet write to native execution: $outputPath") - Some(writeExec) + Some(CometNativeWriteExec(writerOperator, childPlan, outputPath)) } catch { case e: Exception => logWarning("Failed to convert write command to native execution", e) @@ -587,7 +583,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = { val newPlan = _apply(plan) if (showTransformations && !newPlan.fastEquals(plan)) { - println(s""" + logInfo(s""" |=== Applying Rule $ruleName === |${sideBySide(plan.treeString, newPlan.treeString).mkString("\n")} |""".stripMargin) From 5174ba3f909df91fa6d5a4c568cb9d731d27b4ce Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 12:10:14 -0700 Subject: [PATCH 13/35] remove partition id from proto --- native/core/src/execution/planner.rs | 2 +- native/proto/src/proto/operator.proto | 1 - .../src/main/scala/org/apache/comet/rules/CometExecRule.scala | 4 ++-- .../org/apache/spark/sql/comet/CometNativeWriteExec.scala | 3 ++- .../test/scala/org/apache/comet/CometParquetWriterSuite.scala | 1 - 5 files changed, 5 insertions(+), 6 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index cd8c5421dc..314b630415 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1467,7 +1467,7 @@ impl PhysicalPlanner { Arc::clone(&child.native_plan), writer.output_path.clone(), codec, - writer.partition_id, + self.partition, )?); Ok(( diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 34074135cb..44c84b7273 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -240,7 +240,6 @@ message ShuffleWriter { message ParquetWriter { string output_path = 1; CompressionCodec compression = 2; - int32 partition_id = 3; } enum AggregateMode { 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 612a8effce..e1c21d380b 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -155,7 +155,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { val scanOperator = Operator .newBuilder() - .setPlanId(1) + .setPlanId(1) // TODO use real plan id .setScan(scanOp.build()) .build() @@ -168,7 +168,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { val writerOperator = Operator .newBuilder() - .setPlanId(2) + .setPlanId(2) // TODO use real plan id .addChildren(scanOperator) .setParquetWriter(writerOp) .build() diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala index 39083010c8..8a4e260e4c 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala @@ -92,7 +92,8 @@ case class CometNativeWriteExec(nativeOp: Operator, child: SparkPlan, outputPath val partitionId = org.apache.spark.TaskContext.getPartitionId() // Create a new operator with the partition ID set for this specific partition - val writerOp = baseNativeOp.getParquetWriter.toBuilder.setPartitionId(partitionId).build() + // TODO this code needs to be removed + val writerOp = baseNativeOp.getParquetWriter.toBuilder.build() val partitionSpecificOp = baseNativeOp.toBuilder.setParquetWriter(writerOp).build() // Serialize the partition-specific plan diff --git a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala index 1f0a63f5a2..b0988a3624 100644 --- a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala @@ -59,7 +59,6 @@ class CometParquetWriterSuite extends CometTestBase { val partFiles = outputDir.listFiles().filter(_.getName.startsWith("part-")) // With 3 rows and default parallelism, we should get multiple partitions assert(partFiles.length > 0, "Expected at least one part file to be created") - println(s"Created ${partFiles.length} part file(s)") } } } From 8748d85e6f3fb559de64b697dffc4b42847e1d55 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 12:13:38 -0700 Subject: [PATCH 14/35] prep for review --- .../scala/org/apache/comet/rules/CometExecRule.scala | 8 ++++---- .../spark/sql/comet/CometNativeWriteExec.scala | 12 ++---------- 2 files changed, 6 insertions(+), 14 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 e1c21d380b..a3affeb1a4 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -155,7 +155,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { val scanOperator = Operator .newBuilder() - .setPlanId(1) // TODO use real plan id + .setPlanId(exec.id) .setScan(scanOp.build()) .build() @@ -168,7 +168,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { val writerOperator = Operator .newBuilder() - .setPlanId(2) // TODO use real plan id + .setPlanId(exec.id) .addChildren(scanOperator) .setParquetWriter(writerOp) .build() @@ -226,7 +226,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { val scanOperator = Operator .newBuilder() - .setPlanId(1) // TODO use real plan id + .setPlanId(exec.id) .setScan(scanOp.build()) .build() @@ -239,7 +239,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { val writerOperator = Operator .newBuilder() - .setPlanId(2) // TODO use real plan id + .setPlanId(exec.id) .addChildren(scanOperator) .setParquetWriter(writerOp) .build() diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala index 8a4e260e4c..bb1a25ec28 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala @@ -84,21 +84,13 @@ case class CometNativeWriteExec(nativeOp: Operator, child: SparkPlan, outputPath // Capture metadata before the transformation val numPartitions = childRDD.getNumPartitions val numOutputCols = child.output.length - val baseNativeOp = nativeOp // The base operator without partition_id set // Execute native write operation childRDD.mapPartitionsInternal { iter => val nativeMetrics = CometMetricNode.fromCometPlan(this) - val partitionId = org.apache.spark.TaskContext.getPartitionId() - // Create a new operator with the partition ID set for this specific partition - // TODO this code needs to be removed - val writerOp = baseNativeOp.getParquetWriter.toBuilder.build() - val partitionSpecificOp = baseNativeOp.toBuilder.setParquetWriter(writerOp).build() - - // Serialize the partition-specific plan val outputStream = new java.io.ByteArrayOutputStream() - partitionSpecificOp.writeTo(outputStream) + nativeOp.writeTo(outputStream) outputStream.close() val planBytes = outputStream.toByteArray @@ -109,7 +101,7 @@ case class CometNativeWriteExec(nativeOp: Operator, child: SparkPlan, outputPath planBytes, nativeMetrics, numPartitions, - partitionId, + org.apache.spark.TaskContext.getPartitionId(), None, Seq.empty) From f1b7ba88d243b0d301f748b5cee14a1f888ea933 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 12:15:08 -0700 Subject: [PATCH 15/35] move test --- .github/workflows/pr_build_linux.yml | 2 +- .github/workflows/pr_build_macos.yml | 2 +- .../apache/comet/{ => parquet}/CometParquetWriterSuite.scala | 4 +++- 3 files changed, 5 insertions(+), 3 deletions(-) rename spark/src/test/scala/org/apache/comet/{ => parquet}/CometParquetWriterSuite.scala (97%) diff --git a/.github/workflows/pr_build_linux.yml b/.github/workflows/pr_build_linux.yml index 026d3ccacf..1d615bac9c 100644 --- a/.github/workflows/pr_build_linux.yml +++ b/.github/workflows/pr_build_linux.yml @@ -118,7 +118,7 @@ jobs: org.apache.comet.exec.DisableAQECometAsyncShuffleSuite - name: "parquet" value: | - org.apache.comet.CometParquetWriterSuite + org.apache.comet.parquet.CometParquetWriterSuite org.apache.comet.parquet.ParquetReadV1Suite org.apache.comet.parquet.ParquetReadV2Suite org.apache.comet.parquet.ParquetReadFromFakeHadoopFsSuite diff --git a/.github/workflows/pr_build_macos.yml b/.github/workflows/pr_build_macos.yml index 2c76b8fa1e..11f1f1709f 100644 --- a/.github/workflows/pr_build_macos.yml +++ b/.github/workflows/pr_build_macos.yml @@ -83,7 +83,7 @@ jobs: org.apache.comet.exec.DisableAQECometAsyncShuffleSuite - name: "parquet" value: | - org.apache.comet.CometParquetWriterSuite + org.apache.comet.parquet.CometParquetWriterSuite org.apache.comet.parquet.ParquetReadV1Suite org.apache.comet.parquet.ParquetReadV2Suite org.apache.comet.parquet.ParquetReadFromFakeHadoopFsSuite diff --git a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala similarity index 97% rename from spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala rename to spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala index b0988a3624..dd46f8bd60 100644 --- a/spark/src/test/scala/org/apache/comet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala @@ -17,12 +17,14 @@ * under the License. */ -package org.apache.comet +package org.apache.comet.parquet import java.io.File import org.apache.spark.sql.CometTestBase +import org.apache.comet.CometConf + class CometParquetWriterSuite extends CometTestBase { import testImplicits._ From 3ad4d6eaed7ffa2eef5230af364caf4a9900e2d3 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 12:25:52 -0700 Subject: [PATCH 16/35] clippy --- native/core/src/execution/operators/parquet_writer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/native/core/src/execution/operators/parquet_writer.rs b/native/core/src/execution/operators/parquet_writer.rs index 8ec12d0c1a..ee2e7472c2 100644 --- a/native/core/src/execution/operators/parquet_writer.rs +++ b/native/core/src/execution/operators/parquet_writer.rs @@ -208,7 +208,7 @@ impl ExecutionPlan for ParquetWriterExec { .set_compression(compression) .build(); - let mut writer = ArrowWriter::try_new(file, schema.clone(), Some(props)) + let mut writer = ArrowWriter::try_new(file, Arc::clone(&schema), Some(props)) .map_err(|e| DataFusionError::Execution(format!("Failed to create writer: {}", e)))?; // Write batches From f35196b3f4cc28d81688a9b0b5c5255ac4c8d348 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 13:01:18 -0700 Subject: [PATCH 17/35] code cleanup --- .../spark/sql/comet/CometNativeWriteExec.scala | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala index bb1a25ec28..2617e8c60a 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometNativeWriteExec.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.comet.CometExecIterator import org.apache.comet.serde.OperatorOuterClass.Operator /** @@ -60,7 +61,7 @@ case class CometNativeWriteExec(nativeOp: Operator, child: SparkPlan, outputPath override def doExecute(): RDD[InternalRow] = { // Execute the native write val resultRDD = doExecuteColumnar() - // Convert to InternalRow RDD (write operations typically return empty results) + // Convert to empty InternalRow RDD (write operations typically return empty results) resultRDD.mapPartitions { iter => // Consume all batches (they should be empty) iter.foreach(_.close()) @@ -74,8 +75,9 @@ case class CometNativeWriteExec(nativeOp: Operator, child: SparkPlan, outputPath child.executeColumnar() } else { // If child doesn't support columnar, convert to columnar - child.execute().mapPartitionsInternal { rowIter => - // This is a simplified version - in production you'd need proper row to columnar conversion + child.execute().mapPartitionsInternal { _ => + // TODO this could delegate to CometRowToColumnar, but maybe Comet + // does not need to support this case? throw new UnsupportedOperationException( "Row-based child operators not yet supported for native write") } @@ -94,7 +96,7 @@ case class CometNativeWriteExec(nativeOp: Operator, child: SparkPlan, outputPath outputStream.close() val planBytes = outputStream.toByteArray - val cometIter = new org.apache.comet.CometExecIterator( + new CometExecIterator( CometExec.newIterId, Seq(iter), numOutputCols, @@ -105,11 +107,6 @@ case class CometNativeWriteExec(nativeOp: Operator, child: SparkPlan, outputPath None, Seq.empty) - // Consume all output batches (they should be empty for write operations) - new Iterator[ColumnarBatch] { - override def hasNext: Boolean = cometIter.hasNext - override def next(): ColumnarBatch = cometIter.next() - } } } } From 1ea7f31ae1d86a17a3214754e843a475fc7e41a4 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 13:13:36 -0700 Subject: [PATCH 18/35] preserve column names --- .../src/execution/operators/parquet_writer.rs | 53 +++++++++++++++++-- native/core/src/execution/planner.rs | 1 + native/proto/src/proto/operator.proto | 2 + .../apache/comet/rules/CometExecRule.scala | 4 ++ .../parquet/CometParquetWriterSuite.scala | 4 +- 5 files changed, 58 insertions(+), 6 deletions(-) diff --git a/native/core/src/execution/operators/parquet_writer.rs b/native/core/src/execution/operators/parquet_writer.rs index ee2e7472c2..8f8020bbad 100644 --- a/native/core/src/execution/operators/parquet_writer.rs +++ b/native/core/src/execution/operators/parquet_writer.rs @@ -58,6 +58,8 @@ pub struct ParquetWriterExec { compression: CompressionCodec, /// Partition ID (from Spark TaskContext) partition_id: i32, + /// Column names to use in the output Parquet file + column_names: Vec, /// Metrics metrics: ExecutionPlanMetricsSet, /// Cache for plan properties @@ -71,6 +73,7 @@ impl ParquetWriterExec { output_path: String, compression: CompressionCodec, partition_id: i32, + column_names: Vec, ) -> Result { // Preserve the input's partitioning so each partition writes its own file let input_partitioning = input.output_partitioning().clone(); @@ -87,6 +90,7 @@ impl ParquetWriterExec { output_path, compression, partition_id, + column_names, metrics: ExecutionPlanMetricsSet::new(), cache, }) @@ -157,6 +161,7 @@ impl ExecutionPlan for ParquetWriterExec { self.output_path.clone(), self.compression.clone(), self.partition_id, + self.column_names.clone(), )?)), _ => Err(DataFusionError::Internal( "ParquetWriterExec requires exactly one child".to_string(), @@ -170,9 +175,31 @@ impl ExecutionPlan for ParquetWriterExec { context: Arc, ) -> Result { let input = self.input.execute(partition, context)?; - let schema = self.schema(); + let input_schema = self.schema(); let output_path = self.output_path.clone(); let compression = self.compression_to_parquet(); + let column_names = self.column_names.clone(); + + // Create output schema with correct column names + let output_schema = if !column_names.is_empty() { + // Replace the generic column names (col_0, col_1, etc.) with the actual names + let fields: Vec<_> = input_schema + .fields() + .iter() + .enumerate() + .map(|(i, field)| { + if i < column_names.len() { + Arc::new(field.as_ref().clone().with_name(&column_names[i])) + } else { + Arc::clone(field) + } + }) + .collect(); + Arc::new(arrow::datatypes::Schema::new(fields)) + } else { + // No column names provided, use input schema as-is + Arc::clone(&input_schema) + }; // Strip file:// or file: prefix if present let local_path = output_path @@ -208,16 +235,34 @@ impl ExecutionPlan for ParquetWriterExec { .set_compression(compression) .build(); - let mut writer = ArrowWriter::try_new(file, Arc::clone(&schema), Some(props)) + let mut writer = ArrowWriter::try_new(file, Arc::clone(&output_schema), Some(props)) .map_err(|e| DataFusionError::Execution(format!("Failed to create writer: {}", e)))?; + // Clone schema for use in async closure + let schema_for_write = Arc::clone(&output_schema); + // Write batches let write_task = async move { let mut stream = input; while let Some(batch_result) = stream.try_next().await.transpose() { let batch = batch_result?; - writer.write(&batch).map_err(|e| { + + // Rename columns in the batch to match output schema + let renamed_batch = if !column_names.is_empty() { + use arrow::record_batch::RecordBatch; + RecordBatch::try_new(Arc::clone(&schema_for_write), batch.columns().to_vec()) + .map_err(|e| { + DataFusionError::Execution(format!( + "Failed to rename batch columns: {}", + e + )) + })? + } else { + batch + }; + + writer.write(&renamed_batch).map_err(|e| { DataFusionError::Execution(format!("Failed to write batch: {}", e)) })?; } @@ -233,7 +278,7 @@ impl ExecutionPlan for ParquetWriterExec { // Execute the write task and convert to a stream use datafusion::physical_plan::stream::RecordBatchStreamAdapter; Ok(Box::pin(RecordBatchStreamAdapter::new( - schema, + output_schema, futures::stream::once(write_task).try_flatten(), ))) } diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 314b630415..b0746a6f84 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1468,6 +1468,7 @@ impl PhysicalPlanner { writer.output_path.clone(), codec, self.partition, + writer.column_names.clone(), )?); Ok(( diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 44c84b7273..c94f1b6190 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -240,6 +240,8 @@ message ShuffleWriter { message ParquetWriter { string output_path = 1; CompressionCodec compression = 2; + int32 partition_id = 3; + repeated string column_names = 4; } enum AggregateMode { 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 a3affeb1a4..ce56ca8e60 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -164,6 +164,8 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { .setOutputPath(outputPath) // TODO: Get compression from options .setCompression(OperatorOuterClass.CompressionCodec.Snappy) + // Add column names to preserve them across FFI boundary + .addAllColumnNames(cmd.query.output.map(_.name).asJava) .build() val writerOperator = Operator @@ -235,6 +237,8 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { .setOutputPath(outputPath) // TODO: Get compression from options .setCompression(OperatorOuterClass.CompressionCodec.Snappy) + // Add column names to preserve them across FFI boundary + .addAllColumnNames(cmd.query.output.map(_.name).asJava) .build() val writerOperator = Operator diff --git a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala index dd46f8bd60..83f91775bb 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala @@ -42,6 +42,7 @@ class CometParquetWriterSuite extends CometTestBase { val df = spark.read.parquet(inputPath) // perform native write + // TODO add assertion that this actually ran with the native writer df.write.parquet(outputPath) // Verify the data was written correctly @@ -49,8 +50,7 @@ class CometParquetWriterSuite extends CometTestBase { assert(resultDf.count() == 3, "Expected 3 rows to be written") // Verify correct data - // TODO native parquet writer loses column names - val rows = resultDf.orderBy("col_0").collect() + val rows = resultDf.orderBy("id").collect() assert(rows.length == 3) assert(rows(0).getInt(0) == 1 && rows(0).getString(1) == "a") assert(rows(1).getInt(0) == 2 && rows(1).getString(1) == "b") From 61b669060fad25daedb5d84d5541d769460b3542 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 13:14:41 -0700 Subject: [PATCH 19/35] fix assertion --- .../org/apache/comet/parquet/CometParquetWriterSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala index 83f91775bb..db6c9f68dd 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala @@ -60,7 +60,7 @@ class CometParquetWriterSuite extends CometTestBase { val outputDir = new File(outputPath) val partFiles = outputDir.listFiles().filter(_.getName.startsWith("part-")) // With 3 rows and default parallelism, we should get multiple partitions - assert(partFiles.length > 0, "Expected at least one part file to be created") + assert(partFiles.length > 1, "Expected multiple part files to be created") } } } From 1c00aae9ea7eb9c2f8463cbc26c7eb3778ebac6e Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 13:18:25 -0700 Subject: [PATCH 20/35] test --- .../comet/parquet/CometParquetWriterSuite.scala | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala index db6c9f68dd..47fb79a0a5 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala @@ -21,7 +21,7 @@ package org.apache.comet.parquet import java.io.File -import org.apache.spark.sql.CometTestBase +import org.apache.spark.sql.{CometTestBase, DataFrame} import org.apache.comet.CometConf @@ -61,6 +61,17 @@ class CometParquetWriterSuite extends CometTestBase { val partFiles = outputDir.listFiles().filter(_.getName.startsWith("part-")) // With 3 rows and default parallelism, we should get multiple partitions assert(partFiles.length > 1, "Expected multiple part files to be created") + + // read with and without Comet and compare + var sparkDf: DataFrame = null + var cometDf: DataFrame = null + withSQLConf(CometConf.COMET_NATIVE_SCAN_ENABLED.key -> "false") { + sparkDf = spark.read.parquet(outputPath) + } + withSQLConf(CometConf.COMET_NATIVE_SCAN_ENABLED.key -> "true") { + cometDf = spark.read.parquet(outputPath) + } + checkAnswer(sparkDf, cometDf) } } } From e1e357eecce1c84143b876f85613240b18d4b787 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 13:24:07 -0700 Subject: [PATCH 21/35] improve test --- .../parquet/CometParquetWriterSuite.scala | 71 +++++++++++++++++-- 1 file changed, 67 insertions(+), 4 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala index 47fb79a0a5..0df2e3ded1 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala @@ -22,6 +22,9 @@ package org.apache.comet.parquet import java.io.File import org.apache.spark.sql.{CometTestBase, DataFrame} +import org.apache.spark.sql.comet.CometNativeWriteExec +import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.execution.command.DataWritingCommandExec import org.apache.comet.CometConf @@ -38,12 +41,72 @@ class CometParquetWriterSuite extends CometTestBase { val df = Seq((1, "a"), (2, "b"), (3, "c")).toDF("id", "value") df.write.parquet(inputPath) - withSQLConf(CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true") { + withSQLConf( + CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true") { val df = spark.read.parquet(inputPath) - // perform native write - // TODO add assertion that this actually ran with the native writer - df.write.parquet(outputPath) + // Use a listener to capture the execution plan during write + var capturedPlan: Option[QueryExecution] = None + + val listener = new org.apache.spark.sql.util.QueryExecutionListener { + override def onSuccess( + funcName: String, + qe: QueryExecution, + durationNs: Long): Unit = { + // Capture plans from write operations + if (funcName == "save" || funcName.contains("command")) { + capturedPlan = Some(qe) + } + } + + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = {} + } + + spark.listenerManager.register(listener) + + try { + // Perform native write + df.write.parquet(outputPath) + + // Wait for listener to be called with timeout + val maxWaitTimeMs = 15000 + val checkIntervalMs = 100 + val maxIterations = maxWaitTimeMs / checkIntervalMs + var iterations = 0 + + while (capturedPlan.isEmpty && iterations < maxIterations) { + Thread.sleep(checkIntervalMs) + iterations += 1 + } + + // Verify that CometNativeWriteExec was used + assert( + capturedPlan.isDefined, + s"Listener was not called within ${maxWaitTimeMs}ms - no execution plan captured") + + capturedPlan.foreach { qe => + val executedPlan = qe.executedPlan + val hasNativeWrite = executedPlan.exists { + case _: CometNativeWriteExec => true + case d: DataWritingCommandExec => + d.child.exists { + case _: CometNativeWriteExec => true + case _ => false + } + case _ => false + } + + assert( + hasNativeWrite, + s"Expected CometNativeWriteExec in the plan, but got:\n${executedPlan.treeString}") + } + } finally { + spark.listenerManager.unregister(listener) + } // Verify the data was written correctly val resultDf = spark.read.parquet(outputPath) From 9fb2b532902b9f623293629426852f80fbb272fb Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 14:14:01 -0700 Subject: [PATCH 22/35] refactor to use operator serde framework --- .../apache/comet/rules/CometExecRule.scala | 186 +++--------------- .../operator/CometDataWritingCommand.scala | 133 +++++++++++++ 2 files changed, 157 insertions(+), 162 deletions(-) create mode 100644 spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala 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 ce56ca8e60..2560ada2d6 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -33,8 +33,6 @@ 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.{InsertIntoHadoopFsRelationCommand, WriteFilesExec} -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat 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} @@ -50,6 +48,7 @@ import org.apache.comet.serde.{CometOperatorSerde, Compatible, Incompatible, Ope import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.QueryPlanSerde.{serializeDataType, supportedDataType} import org.apache.comet.serde.operator._ +import org.apache.comet.serde.operator.CometDataWritingCommandExec object CometExecRule { @@ -72,6 +71,13 @@ object CometExecRule { classOf[LocalTableScanExec] -> CometLocalTableScanExec, classOf[WindowExec] -> CometWindowExec) + /** + * DataWritingCommandExec is handled separately in convertNode since it doesn't follow the + * standard pattern of having CometNativeExec children. + */ + val writeExecs: Map[Class[_ <: SparkPlan], CometOperatorSerde[_]] = + Map(classOf[DataWritingCommandExec] -> CometDataWritingCommandExec) + /** * Sinks that have a native plan of ScanExec. */ @@ -112,158 +118,6 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { private def isCometNative(op: SparkPlan): Boolean = op.isInstanceOf[CometNativeExec] - /** - * Try to convert a DataWritingCommandExec to use Comet native Parquet writer. Returns - * Some(CometNativeWriteExec) if conversion is successful, None otherwise. - */ - private def tryConvertDataWritingCommand(exec: DataWritingCommandExec): Option[SparkPlan] = { - exec.cmd match { - case cmd: InsertIntoHadoopFsRelationCommand => - // Check if this is a Parquet write - cmd.fileFormat match { - case _: ParquetFileFormat => - try { - // Use the already-transformed child plan from the WriteFilesExec - // The child has already been through Comet transformations - val childPlan = exec.child match { - case writeFiles: WriteFilesExec => - // The WriteFilesExec child should already be a Comet operator - writeFiles.child - case other => - // Fallback: use the child directly - other - } - - // Get output path - val outputPath = cmd.outputPath.toString - - // Create native ParquetWriter operator - val scanOp = OperatorOuterClass.Scan - .newBuilder() - .setSource("write_source") - .setArrowFfiSafe(true) - - // Add fields from the query output schema - cmd.query.output.foreach { attr => - serializeDataType(attr.dataType) match { - case Some(dataType) => scanOp.addFields(dataType) - case None => - logWarning(s"Cannot serialize data type ${attr.dataType} for native write") - return None - } - } - - val scanOperator = Operator - .newBuilder() - .setPlanId(exec.id) - .setScan(scanOp.build()) - .build() - - val writerOp = OperatorOuterClass.ParquetWriter - .newBuilder() - .setOutputPath(outputPath) - // TODO: Get compression from options - .setCompression(OperatorOuterClass.CompressionCodec.Snappy) - // Add column names to preserve them across FFI boundary - .addAllColumnNames(cmd.query.output.map(_.name).asJava) - .build() - - val writerOperator = Operator - .newBuilder() - .setPlanId(exec.id) - .addChildren(scanOperator) - .setParquetWriter(writerOp) - .build() - - // Create CometNativeWriteExec with the transformed child plan - Some(CometNativeWriteExec(writerOperator, childPlan, outputPath)) - } catch { - case e: Exception => - logWarning("Failed to convert DataWritingCommandExec to native execution", e) - None - } - case _ => - // Not a Parquet write, skip - None - } - case _ => - // Not a write command we handle - None - } - } - - /** - * Try to convert a write command (ExecutedCommandExec) to use Comet native Parquet writer. - * Returns Some(CometNativeWriteExec) if conversion is successful, None otherwise. - */ - private def tryConvertWriteCommand(exec: ExecutedCommandExec): Option[SparkPlan] = { - exec.cmd match { - case cmd: InsertIntoHadoopFsRelationCommand => - // Check if this is a Parquet write - cmd.fileFormat match { - case _: ParquetFileFormat => - try { - // Plan the query to get the physical plan - val queryExecution = session.sessionState.executePlan(cmd.query) - val childPlan = queryExecution.executedPlan - - // Get output path - val outputPath = cmd.outputPath.toString - - // Create native ParquetWriter operator - val scanOp = OperatorOuterClass.Scan - .newBuilder() - .setSource("write_source") - .setArrowFfiSafe(true) - - // Add fields from the query output schema - cmd.query.output.foreach { attr => - serializeDataType(attr.dataType) match { - case Some(dataType) => scanOp.addFields(dataType) - case None => - logWarning(s"Cannot serialize data type ${attr.dataType} for native write") - return None - } - } - - val scanOperator = Operator - .newBuilder() - .setPlanId(exec.id) - .setScan(scanOp.build()) - .build() - - val writerOp = OperatorOuterClass.ParquetWriter - .newBuilder() - .setOutputPath(outputPath) - // TODO: Get compression from options - .setCompression(OperatorOuterClass.CompressionCodec.Snappy) - // Add column names to preserve them across FFI boundary - .addAllColumnNames(cmd.query.output.map(_.name).asJava) - .build() - - val writerOperator = Operator - .newBuilder() - .setPlanId(exec.id) - .addChildren(scanOperator) - .setParquetWriter(writerOp) - .build() - - Some(CometNativeWriteExec(writerOperator, childPlan, outputPath)) - } catch { - case e: Exception => - logWarning("Failed to convert write command to native execution", e) - None - } - case _ => - // Not a Parquet write, skip - None - } - case _ => - // Not a write command we handle - None - } - } - // spotless:off /** @@ -372,14 +226,22 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { val nativeOp = operator2Proto(cometOp) CometScanWrapper(nativeOp.get, cometOp) - // Intercept DataWritingCommandExec (Spark 3.5+) - case exec: DataWritingCommandExec - if CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.get(conf) => - tryConvertDataWritingCommand(exec).getOrElse(exec) - - // Intercept Parquet write commands (fallback for older Spark versions) - case exec: ExecutedCommandExec if CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.get(conf) => - tryConvertWriteCommand(exec).getOrElse(exec) + // Handle DataWritingCommandExec specially since it doesn't follow the standard pattern + case exec: DataWritingCommandExec => + CometExecRule.writeExecs.get(classOf[DataWritingCommandExec]) match { + case Some(handler) if isOperatorEnabled(handler, exec) => + val builder = OperatorOuterClass.Operator.newBuilder().setPlanId(exec.id) + handler + .asInstanceOf[CometOperatorSerde[DataWritingCommandExec]] + .convert(exec, builder) + .map(nativeOp => + handler + .asInstanceOf[CometOperatorSerde[DataWritingCommandExec]] + .createExec(nativeOp, exec)) + .getOrElse(exec) + case _ => + exec + } // For AQE broadcast stage on a Comet broadcast exchange case s @ BroadcastQueryStageExec(_, _: CometBroadcastExchangeExec, _) => diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala new file mode 100644 index 0000000000..a219462574 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala @@ -0,0 +1,133 @@ +/* + * 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.operator + +import scala.jdk.CollectionConverters._ + +import org.apache.spark.sql.comet.{CometNativeExec, CometNativeWriteExec} +import org.apache.spark.sql.execution.command.DataWritingCommandExec +import org.apache.spark.sql.execution.datasources.{InsertIntoHadoopFsRelationCommand, WriteFilesExec} +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat + +import org.apache.comet.{CometConf, ConfigEntry} +import org.apache.comet.CometSparkSessionExtensions.withInfo +import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} +import org.apache.comet.serde.OperatorOuterClass.Operator +import org.apache.comet.serde.QueryPlanSerde.serializeDataType + +/** + * CometOperatorSerde implementation for DataWritingCommandExec that converts Parquet write + * operations to use Comet's native Parquet writer. + */ +object CometDataWritingCommandExec extends CometOperatorSerde[DataWritingCommandExec] { + + override def enabledConfig: Option[ConfigEntry[Boolean]] = + Some(CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED) + + override def convert( + op: DataWritingCommandExec, + builder: Operator.Builder, + childOp: Operator*): Option[OperatorOuterClass.Operator] = { + op.cmd match { + case cmd: InsertIntoHadoopFsRelationCommand => + // Check if this is a Parquet write + cmd.fileFormat match { + case _: ParquetFileFormat => + try { + // Create native ParquetWriter operator + val scanOp = OperatorOuterClass.Scan + .newBuilder() + .setSource("write_source") + .setArrowFfiSafe(true) + + // Add fields from the query output schema + val scanTypes = cmd.query.output.flatMap { attr => + serializeDataType(attr.dataType) + } + + if (scanTypes.length != cmd.query.output.length) { + withInfo(op, "Cannot serialize data types for native write") + return None + } + + scanTypes.foreach(scanOp.addFields) + + val scanOperator = Operator + .newBuilder() + .setPlanId(op.id) + .setScan(scanOp.build()) + .build() + + // Get output path + val outputPath = cmd.outputPath.toString + + val writerOp = OperatorOuterClass.ParquetWriter + .newBuilder() + .setOutputPath(outputPath) + // TODO: Get compression from options + .setCompression(OperatorOuterClass.CompressionCodec.Snappy) + // Add column names to preserve them across FFI boundary + .addAllColumnNames(cmd.query.output.map(_.name).asJava) + .build() + + // Build the ParquetWriter operator + val writerOperator = Operator + .newBuilder() + .setPlanId(op.id) + .addChildren(scanOperator) + .setParquetWriter(writerOp) + .build() + + Some(writerOperator) + } catch { + case e: Exception => + withInfo( + op, + "Failed to convert DataWritingCommandExec to native execution: " + + s"${e.getMessage}") + None + } + case _ => + // Not a Parquet write, skip + None + } + case _ => + // Not a write command we handle + None + } + } + + override def createExec(nativeOp: Operator, op: DataWritingCommandExec): CometNativeExec = { + val cmd = op.cmd.asInstanceOf[InsertIntoHadoopFsRelationCommand] + val outputPath = cmd.outputPath.toString + + // Get the child plan from the WriteFilesExec or use the child directly + val childPlan = op.child match { + case writeFiles: WriteFilesExec => + // The WriteFilesExec child should already be a Comet operator + writeFiles.child + case other => + // Fallback: use the child directly + other + } + + CometNativeWriteExec(nativeOp, childPlan, outputPath) + } +} From 457a9b9ef61e0bc93d1d40e5785e01aff9c8e6c2 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 14:15:13 -0700 Subject: [PATCH 23/35] skip testing for native_datafusion for now --- .../org/apache/comet/parquet/CometParquetWriterSuite.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala index 0df2e3ded1..bcbc4ade63 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala @@ -32,6 +32,9 @@ class CometParquetWriterSuite extends CometTestBase { import testImplicits._ test("basic parquet write") { + // no support for fully native scan as input yet + assume(CometConf.COMET_NATIVE_SCAN_IMPL.get() != CometConf.SCAN_NATIVE_DATAFUSION) + withTempPath { dir => val outputPath = new File(dir, "output.parquet").getAbsolutePath From 757ba8101983ecee429d76a205d778af30857253 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 15:30:34 -0700 Subject: [PATCH 24/35] remove hard-coded compression codec --- .../operator/CometDataWritingCommand.scala | 27 ++++++++++++++----- 1 file changed, 21 insertions(+), 6 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala index a219462574..abb45c58a2 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.comet.{CometNativeExec, CometNativeWriteExec} import org.apache.spark.sql.execution.command.DataWritingCommandExec import org.apache.spark.sql.execution.datasources.{InsertIntoHadoopFsRelationCommand, WriteFilesExec} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.internal.SQLConf import org.apache.comet.{CometConf, ConfigEntry} import org.apache.comet.CometSparkSessionExtensions.withInfo @@ -78,12 +79,26 @@ object CometDataWritingCommandExec extends CometOperatorSerde[DataWritingCommand // Get output path val outputPath = cmd.outputPath.toString + val codecName = cmd.options.getOrElse( + "compression", + SQLConf.get.getConfString( + SQLConf.PARQUET_COMPRESSION.key, + SQLConf.PARQUET_COMPRESSION.defaultValueString)) + + val codec = codecName match { + case "snappy" => OperatorOuterClass.CompressionCodec.Snappy + case "lz4" => OperatorOuterClass.CompressionCodec.Lz4 + case "zstd" => OperatorOuterClass.CompressionCodec.Zstd + case "none" => OperatorOuterClass.CompressionCodec.None + case other => + withInfo(op, s"Unsupported compression codec: $other") + return None + } + val writerOp = OperatorOuterClass.ParquetWriter .newBuilder() .setOutputPath(outputPath) - // TODO: Get compression from options - .setCompression(OperatorOuterClass.CompressionCodec.Snappy) - // Add column names to preserve them across FFI boundary + .setCompression(codec) .addAllColumnNames(cmd.query.output.map(_.name).asJava) .build() @@ -105,11 +120,11 @@ object CometDataWritingCommandExec extends CometOperatorSerde[DataWritingCommand None } case _ => - // Not a Parquet write, skip + withInfo(op, "Only Parquet writes are supported") None } - case _ => - // Not a write command we handle + case other => + withInfo(op, s"Unsupported write command: ${other.getClass}") None } } From 7b9e925df332f40ebc6d7e4ac52f6167ca133cac Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 15:35:20 -0700 Subject: [PATCH 25/35] lz4 level --- .../core/src/execution/operators/parquet_writer.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/native/core/src/execution/operators/parquet_writer.rs b/native/core/src/execution/operators/parquet_writer.rs index 8f8020bbad..9b6b7f007b 100644 --- a/native/core/src/execution/operators/parquet_writer.rs +++ b/native/core/src/execution/operators/parquet_writer.rs @@ -96,12 +96,12 @@ impl ParquetWriterExec { }) } - fn compression_to_parquet(&self) -> Compression { + fn compression_to_parquet(&self) -> Result { match self.compression { - CompressionCodec::None => Compression::UNCOMPRESSED, - CompressionCodec::Zstd(_) => Compression::ZSTD(ZstdLevel::default()), - CompressionCodec::Lz4Frame => Compression::LZ4, - CompressionCodec::Snappy => Compression::SNAPPY, + CompressionCodec::None => Ok(Compression::UNCOMPRESSED), + CompressionCodec::Zstd(level) => Ok(Compression::ZSTD(ZstdLevel::try_new(level)?)), + CompressionCodec::Lz4Frame => Ok(Compression::LZ4), + CompressionCodec::Snappy => Ok(Compression::SNAPPY), } } } @@ -177,7 +177,7 @@ impl ExecutionPlan for ParquetWriterExec { let input = self.input.execute(partition, context)?; let input_schema = self.schema(); let output_path = self.output_path.clone(); - let compression = self.compression_to_parquet(); + let compression = self.compression_to_parquet()?; let column_names = self.column_names.clone(); // Create output schema with correct column names From 913f2baac861b795b2401968bd08113bbf4cd7f8 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 21 Nov 2025 15:47:45 -0700 Subject: [PATCH 26/35] remove partition id from proto --- native/proto/src/proto/operator.proto | 1 - 1 file changed, 1 deletion(-) diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index c94f1b6190..a958327099 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -240,7 +240,6 @@ message ShuffleWriter { message ParquetWriter { string output_path = 1; CompressionCodec compression = 2; - int32 partition_id = 3; repeated string column_names = 4; } From de07d45cf9464568e9ee05072bf650a1e5c99774 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 22 Nov 2025 09:24:48 -0700 Subject: [PATCH 27/35] implement getSupportLevel --- .../operator/CometDataWritingCommand.scala | 179 ++++++++++-------- .../parquet/CometParquetWriterSuite.scala | 1 + 2 files changed, 103 insertions(+), 77 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala index abb45c58a2..7afdcea6b2 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala @@ -27,9 +27,9 @@ import org.apache.spark.sql.execution.datasources.{InsertIntoHadoopFsRelationCom import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.internal.SQLConf -import org.apache.comet.{CometConf, ConfigEntry} +import org.apache.comet.{CometConf, ConfigEntry, DataTypeSupport} import org.apache.comet.CometSparkSessionExtensions.withInfo -import org.apache.comet.serde.{CometOperatorSerde, OperatorOuterClass} +import org.apache.comet.serde.{CometOperatorSerde, Incompatible, OperatorOuterClass, SupportLevel, Unsupported} import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.QueryPlanSerde.serializeDataType @@ -39,92 +39,108 @@ import org.apache.comet.serde.QueryPlanSerde.serializeDataType */ object CometDataWritingCommandExec extends CometOperatorSerde[DataWritingCommandExec] { + private val supportedCompressionCodes = Set("none", "snappy", "lz4", "zstd") + override def enabledConfig: Option[ConfigEntry[Boolean]] = Some(CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED) - override def convert( - op: DataWritingCommandExec, - builder: Operator.Builder, - childOp: Operator*): Option[OperatorOuterClass.Operator] = { + override def getSupportLevel(op: DataWritingCommandExec): SupportLevel = { op.cmd match { case cmd: InsertIntoHadoopFsRelationCommand => - // Check if this is a Parquet write cmd.fileFormat match { case _: ParquetFileFormat => - try { - // Create native ParquetWriter operator - val scanOp = OperatorOuterClass.Scan - .newBuilder() - .setSource("write_source") - .setArrowFfiSafe(true) - - // Add fields from the query output schema - val scanTypes = cmd.query.output.flatMap { attr => - serializeDataType(attr.dataType) - } - - if (scanTypes.length != cmd.query.output.length) { - withInfo(op, "Cannot serialize data types for native write") - return None - } - - scanTypes.foreach(scanOp.addFields) - - val scanOperator = Operator - .newBuilder() - .setPlanId(op.id) - .setScan(scanOp.build()) - .build() - - // Get output path - val outputPath = cmd.outputPath.toString - - val codecName = cmd.options.getOrElse( - "compression", - SQLConf.get.getConfString( - SQLConf.PARQUET_COMPRESSION.key, - SQLConf.PARQUET_COMPRESSION.defaultValueString)) - - val codec = codecName match { - case "snappy" => OperatorOuterClass.CompressionCodec.Snappy - case "lz4" => OperatorOuterClass.CompressionCodec.Lz4 - case "zstd" => OperatorOuterClass.CompressionCodec.Zstd - case "none" => OperatorOuterClass.CompressionCodec.None - case other => - withInfo(op, s"Unsupported compression codec: $other") - return None - } - - val writerOp = OperatorOuterClass.ParquetWriter - .newBuilder() - .setOutputPath(outputPath) - .setCompression(codec) - .addAllColumnNames(cmd.query.output.map(_.name).asJava) - .build() - - // Build the ParquetWriter operator - val writerOperator = Operator - .newBuilder() - .setPlanId(op.id) - .addChildren(scanOperator) - .setParquetWriter(writerOp) - .build() - - Some(writerOperator) - } catch { - case e: Exception => - withInfo( - op, - "Failed to convert DataWritingCommandExec to native execution: " + - s"${e.getMessage}") - None + if (cmd.bucketSpec.isDefined) { + return Unsupported(Some("Bucketed writes are not supported")) + } + + if (cmd.partitionColumns.nonEmpty) { + return Unsupported(Some("Partitioned writes are not supported")) + } + + if (cmd.query.output.exists(attr => DataTypeSupport.isComplexType(attr.dataType))) { + return Unsupported(Some("Complex types are not supported")) } + + val codec = parseCompressionCodec(cmd) + if (!supportedCompressionCodes.contains(codec)) { + return Unsupported(Some(s"Unsupported compression codec: $codec")) + } + + Incompatible(Some("Parquet write support is highly experimental")) case _ => - withInfo(op, "Only Parquet writes are supported") - None + Unsupported(Some("Only Parquet writes are supported")) } case other => - withInfo(op, s"Unsupported write command: ${other.getClass}") + Unsupported(Some(s"Unsupported write command: ${other.getClass}")) + } + } + + override def convert( + op: DataWritingCommandExec, + builder: Operator.Builder, + childOp: Operator*): Option[OperatorOuterClass.Operator] = { + + val cmd = op.cmd.asInstanceOf[InsertIntoHadoopFsRelationCommand] + try { + // Create native ParquetWriter operator + val scanOp = OperatorOuterClass.Scan + .newBuilder() + .setSource("write_source") + .setArrowFfiSafe(true) + + // Add fields from the query output schema + val scanTypes = cmd.query.output.flatMap { attr => + serializeDataType(attr.dataType) + } + + if (scanTypes.length != cmd.query.output.length) { + withInfo(op, "Cannot serialize data types for native write") + return None + } + + scanTypes.foreach(scanOp.addFields) + + val scanOperator = Operator + .newBuilder() + .setPlanId(op.id) + .setScan(scanOp.build()) + .build() + + // Get output path + val outputPath = cmd.outputPath.toString + + val codec = parseCompressionCodec(cmd) match { + case "snappy" => OperatorOuterClass.CompressionCodec.Snappy + case "lz4" => OperatorOuterClass.CompressionCodec.Lz4 + case "zstd" => OperatorOuterClass.CompressionCodec.Zstd + case "none" => OperatorOuterClass.CompressionCodec.None + case other => + withInfo(op, s"Unsupported compression codec: $other") + return None + } + + val writerOp = OperatorOuterClass.ParquetWriter + .newBuilder() + .setOutputPath(outputPath) + .setCompression(codec) + .addAllColumnNames(cmd.query.output.map(_.name).asJava) + .build() + + // Build the ParquetWriter operator + val writerOperator = Operator + .newBuilder() + .setPlanId(op.id) + .addChildren(scanOperator) + .setParquetWriter(writerOp) + .build() + + Some(writerOperator) + } catch { + case e: Exception => + withInfo( + op, + "Failed to convert DataWritingCommandExec to native execution: " + + s"${e.getMessage}") None } } @@ -145,4 +161,13 @@ object CometDataWritingCommandExec extends CometOperatorSerde[DataWritingCommand CometNativeWriteExec(nativeOp, childPlan, outputPath) } + + private def parseCompressionCodec(cmd: InsertIntoHadoopFsRelationCommand) = { + cmd.options.getOrElse( + "compression", + SQLConf.get.getConfString( + SQLConf.PARQUET_COMPRESSION.key, + SQLConf.PARQUET_COMPRESSION.defaultValueString)) + } + } diff --git a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala index bcbc4ade63..7163dd02f7 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala @@ -46,6 +46,7 @@ class CometParquetWriterSuite extends CometTestBase { withSQLConf( CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", + CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true", CometConf.COMET_EXEC_ENABLED.key -> "true") { val df = spark.read.parquet(inputPath) From eb4fee4a10c587b8251998fa1bb093917f1a5ae9 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 22 Nov 2025 09:30:48 -0700 Subject: [PATCH 28/35] move config to testing category --- common/src/main/scala/org/apache/comet/CometConf.scala | 2 +- docs/source/user-guide/latest/configs.md | 2 +- .../comet/serde/operator/CometDataWritingCommand.scala | 10 ++++------ 3 files changed, 6 insertions(+), 8 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index b34d9ccaec..1e5d19ee23 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -102,7 +102,7 @@ object CometConf extends ShimCometConf { val COMET_NATIVE_PARQUET_WRITE_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.parquet.write.enabled") - .category(CATEGORY_PARQUET) + .category(CATEGORY_TESTING) .doc( "Whether to enable native Parquet write through Comet. When enabled, " + "Comet will intercept Parquet write operations and execute them natively. This " + diff --git a/docs/source/user-guide/latest/configs.md b/docs/source/user-guide/latest/configs.md index 1fa7311bef..a1c3212c20 100644 --- a/docs/source/user-guide/latest/configs.md +++ b/docs/source/user-guide/latest/configs.md @@ -51,7 +51,6 @@ Comet provides the following configuration settings. | `spark.comet.parquet.read.parallel.io.enabled` | Whether to enable Comet's parallel reader for Parquet files. The parallel reader reads ranges of consecutive data in a file in parallel. It is faster for large files and row groups but uses more resources. | true | | `spark.comet.parquet.read.parallel.io.thread-pool.size` | The maximum number of parallel threads the parallel reader will use in a single executor. For executors configured with a smaller number of cores, use a smaller number. | 16 | | `spark.comet.parquet.respectFilterPushdown` | Whether to respect Spark's PARQUET_FILTER_PUSHDOWN_ENABLED config. This needs to be respected when running the Spark SQL test suite but the default setting results in poor performance in Comet when using the new native scans, disabled by default | false | -| `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 | @@ -143,6 +142,7 @@ These settings can be used to determine which parts of the plan are accelerated | `spark.comet.exec.onHeap.enabled` | Whether to allow Comet to run in on-heap mode. Required for running Spark SQL tests. It can be overridden by the environment variable `ENABLE_COMET_ONHEAP`. | false | | `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.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/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala index 7afdcea6b2..6580064cd5 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala @@ -80,13 +80,13 @@ object CometDataWritingCommandExec extends CometOperatorSerde[DataWritingCommand builder: Operator.Builder, childOp: Operator*): Option[OperatorOuterClass.Operator] = { - val cmd = op.cmd.asInstanceOf[InsertIntoHadoopFsRelationCommand] try { - // Create native ParquetWriter operator + val cmd = op.cmd.asInstanceOf[InsertIntoHadoopFsRelationCommand] + val scanOp = OperatorOuterClass.Scan .newBuilder() - .setSource("write_source") - .setArrowFfiSafe(true) + .setSource(cmd.query.nodeName) + .setArrowFfiSafe(false) // Add fields from the query output schema val scanTypes = cmd.query.output.flatMap { attr => @@ -106,7 +106,6 @@ object CometDataWritingCommandExec extends CometOperatorSerde[DataWritingCommand .setScan(scanOp.build()) .build() - // Get output path val outputPath = cmd.outputPath.toString val codec = parseCompressionCodec(cmd) match { @@ -126,7 +125,6 @@ object CometDataWritingCommandExec extends CometOperatorSerde[DataWritingCommand .addAllColumnNames(cmd.query.output.map(_.name).asJava) .build() - // Build the ParquetWriter operator val writerOperator = Operator .newBuilder() .setPlanId(op.id) From 9a6e48ca768f734db3deeaaaa0e2f0fe0a6f2e12 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 22 Nov 2025 10:22:15 -0700 Subject: [PATCH 29/35] fuzz test --- .../parquet/CometParquetWriterSuite.scala | 32 ++++++++++++------- 1 file changed, 20 insertions(+), 12 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala index 7163dd02f7..e4b8b53856 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/CometParquetWriterSuite.scala @@ -21,15 +21,18 @@ package org.apache.comet.parquet import java.io.File +import scala.util.Random + import org.apache.spark.sql.{CometTestBase, DataFrame} import org.apache.spark.sql.comet.CometNativeWriteExec import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.command.DataWritingCommandExec +import org.apache.spark.sql.internal.SQLConf import org.apache.comet.CometConf +import org.apache.comet.testing.{DataGenOptions, FuzzDataGenerator, SchemaGenOptions} class CometParquetWriterSuite extends CometTestBase { - import testImplicits._ test("basic parquet write") { // no support for fully native scan as input yet @@ -41,11 +44,23 @@ class CometParquetWriterSuite extends CometTestBase { // Create test data and write it to a temp parquet file first withTempPath { inputDir => val inputPath = new File(inputDir, "input.parquet").getAbsolutePath - val df = Seq((1, "a"), (2, "b"), (3, "c")).toDF("id", "value") - df.write.parquet(inputPath) + val schema = FuzzDataGenerator.generateSchema( + SchemaGenOptions(generateArray = false, generateStruct = false, generateMap = false)) + val df = FuzzDataGenerator.generateDataFrame( + new Random(42), + spark, + schema, + 1000, + DataGenOptions(generateNegativeZero = false)) + withSQLConf( + CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "false", + SQLConf.SESSION_LOCAL_TIMEZONE.key -> "America/Denver") { + df.write.parquet(inputPath) + } withSQLConf( CometConf.COMET_NATIVE_PARQUET_WRITE_ENABLED.key -> "true", + SQLConf.SESSION_LOCAL_TIMEZONE.key -> "America/Halifax", CometConf.getOperatorAllowIncompatConfigKey(classOf[DataWritingCommandExec]) -> "true", CometConf.COMET_EXEC_ENABLED.key -> "true") { val df = spark.read.parquet(inputPath) @@ -114,19 +129,12 @@ class CometParquetWriterSuite extends CometTestBase { // Verify the data was written correctly val resultDf = spark.read.parquet(outputPath) - assert(resultDf.count() == 3, "Expected 3 rows to be written") - - // Verify correct data - val rows = resultDf.orderBy("id").collect() - assert(rows.length == 3) - assert(rows(0).getInt(0) == 1 && rows(0).getString(1) == "a") - assert(rows(1).getInt(0) == 2 && rows(1).getString(1) == "b") - assert(rows(2).getInt(0) == 3 && rows(2).getString(1) == "c") + assert(resultDf.count() == 1000, "Expected 1000 rows to be written") // Verify multiple part files were created val outputDir = new File(outputPath) val partFiles = outputDir.listFiles().filter(_.getName.startsWith("part-")) - // With 3 rows and default parallelism, we should get multiple partitions + // With 1000 rows and default parallelism, we should get multiple partitions assert(partFiles.length > 1, "Expected multiple part files to be created") // read with and without Comet and compare From 8b1cb0b14a6d841afeb6ae444527d029f12ad25a Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 22 Nov 2025 10:27:58 -0700 Subject: [PATCH 30/35] remove snappy from filename --- native/core/src/execution/operators/parquet_writer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/native/core/src/execution/operators/parquet_writer.rs b/native/core/src/execution/operators/parquet_writer.rs index 9b6b7f007b..08dcadf686 100644 --- a/native/core/src/execution/operators/parquet_writer.rs +++ b/native/core/src/execution/operators/parquet_writer.rs @@ -218,7 +218,7 @@ impl ExecutionPlan for ParquetWriterExec { // Generate part file name for this partition let part_file = format!( - "{}/part-{:05}.snappy.parquet", + "{}/part-{:05}.parquet", local_path, self.partition_id ); From 58c056d639c1cb493d8edb21730b521ad9fcc31f Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 22 Nov 2025 10:38:18 -0700 Subject: [PATCH 31/35] remove snappy from filename --- native/core/src/execution/operators/parquet_writer.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/native/core/src/execution/operators/parquet_writer.rs b/native/core/src/execution/operators/parquet_writer.rs index 08dcadf686..12d633b22a 100644 --- a/native/core/src/execution/operators/parquet_writer.rs +++ b/native/core/src/execution/operators/parquet_writer.rs @@ -217,10 +217,7 @@ impl ExecutionPlan for ParquetWriterExec { })?; // Generate part file name for this partition - let part_file = format!( - "{}/part-{:05}.parquet", - local_path, self.partition_id - ); + let part_file = format!("{}/part-{:05}.parquet", local_path, self.partition_id); // Create the Parquet file let file = File::create(&part_file).map_err(|e| { From bd042747897d880b2d75236c64b4cedbcc230c90 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 22 Nov 2025 10:54:58 -0700 Subject: [PATCH 32/35] docs --- docs/source/user-guide/latest/operators.md | 1 + .../apache/comet/serde/operator/CometDataWritingCommand.scala | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/source/user-guide/latest/operators.md b/docs/source/user-guide/latest/operators.md index fdfbcef687..09934c6e81 100644 --- a/docs/source/user-guide/latest/operators.md +++ b/docs/source/user-guide/latest/operators.md @@ -27,6 +27,7 @@ not supported by Comet will fall back to regular Spark execution. | BatchScanExec | Yes | Supports Parquet files and Apache Iceberg Parquet scans. See the [Comet Compatibility Guide] for more information. | | BroadcastExchangeExec | Yes | | | BroadcastHashJoinExec | Yes | | +| DataWritingCommandExec | No | Experimental support for native Parquet writes. Disabled by default. | | ExpandExec | Yes | | | FileSourceScanExec | Yes | Supports Parquet files. See the [Comet Compatibility Guide] for more information. | | FilterExec | Yes | | diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala index 6580064cd5..ce524e8a41 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala @@ -53,7 +53,7 @@ object CometDataWritingCommandExec extends CometOperatorSerde[DataWritingCommand return Unsupported(Some("Bucketed writes are not supported")) } - if (cmd.partitionColumns.nonEmpty) { + if (cmd.partitionColumns.nonEmpty || cmd.staticPartitions.nonEmpty) { return Unsupported(Some("Partitioned writes are not supported")) } From 55d5d48c6c74b4ea49e498df388178e6d8994184 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 22 Nov 2025 12:43:45 -0700 Subject: [PATCH 33/35] fix ci --- .github/actions/java-test/action.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/actions/java-test/action.yaml b/.github/actions/java-test/action.yaml index ab2bcd74fe..d42053494c 100644 --- a/.github/actions/java-test/action.yaml +++ b/.github/actions/java-test/action.yaml @@ -55,7 +55,7 @@ runs: path: | ~/.m2/repository /root/.m2/repository - key: ${{ runner.os }}-java-maven-${{ hashFiles('**/pom.xml') }} + key: ${{ runner.os }}-java-maven-${{ hashFiles(format('{0}/**/pom.xml', github.workspace)) }} restore-keys: | ${{ runner.os }}-java-maven- From cda832231803be342331e5d95cabca417411b8ca Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 22 Nov 2025 13:17:06 -0700 Subject: [PATCH 34/35] Revert "fix ci" This reverts commit 55d5d48c6c74b4ea49e498df388178e6d8994184. --- .github/actions/java-test/action.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/actions/java-test/action.yaml b/.github/actions/java-test/action.yaml index d42053494c..ab2bcd74fe 100644 --- a/.github/actions/java-test/action.yaml +++ b/.github/actions/java-test/action.yaml @@ -55,7 +55,7 @@ runs: path: | ~/.m2/repository /root/.m2/repository - key: ${{ runner.os }}-java-maven-${{ hashFiles(format('{0}/**/pom.xml', github.workspace)) }} + key: ${{ runner.os }}-java-maven-${{ hashFiles('**/pom.xml') }} restore-keys: | ${{ runner.os }}-java-maven- From 1d1d430fbc3c06384f1d7cd851da522a76756e7f Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 24 Nov 2025 13:29:32 -0700 Subject: [PATCH 35/35] partially address feedback --- .../main/scala/org/apache/comet/rules/CometExecRule.scala | 4 ++-- .../comet/serde/operator/CometDataWritingCommand.scala | 6 +++++- 2 files changed, 7 insertions(+), 3 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 2560ada2d6..124188b64d 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -48,7 +48,7 @@ import org.apache.comet.serde.{CometOperatorSerde, Compatible, Incompatible, Ope import org.apache.comet.serde.OperatorOuterClass.Operator import org.apache.comet.serde.QueryPlanSerde.{serializeDataType, supportedDataType} import org.apache.comet.serde.operator._ -import org.apache.comet.serde.operator.CometDataWritingCommandExec +import org.apache.comet.serde.operator.CometDataWritingCommand object CometExecRule { @@ -76,7 +76,7 @@ object CometExecRule { * standard pattern of having CometNativeExec children. */ val writeExecs: Map[Class[_ <: SparkPlan], CometOperatorSerde[_]] = - Map(classOf[DataWritingCommandExec] -> CometDataWritingCommandExec) + Map(classOf[DataWritingCommandExec] -> CometDataWritingCommand) /** * Sinks that have a native plan of ScanExec. diff --git a/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala b/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala index ce524e8a41..af91d59680 100644 --- a/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala +++ b/spark/src/main/scala/org/apache/comet/serde/operator/CometDataWritingCommand.scala @@ -37,7 +37,7 @@ import org.apache.comet.serde.QueryPlanSerde.serializeDataType * CometOperatorSerde implementation for DataWritingCommandExec that converts Parquet write * operations to use Comet's native Parquet writer. */ -object CometDataWritingCommandExec extends CometOperatorSerde[DataWritingCommandExec] { +object CometDataWritingCommand extends CometOperatorSerde[DataWritingCommandExec] { private val supportedCompressionCodes = Set("none", "snappy", "lz4", "zstd") @@ -49,6 +49,10 @@ object CometDataWritingCommandExec extends CometOperatorSerde[DataWritingCommand case cmd: InsertIntoHadoopFsRelationCommand => cmd.fileFormat match { case _: ParquetFileFormat => + if (!cmd.outputPath.toString.startsWith("file:")) { + return Unsupported(Some("Only local filesystem output paths are supported")) + } + if (cmd.bucketSpec.isDefined) { return Unsupported(Some("Bucketed writes are not supported")) }