@@ -26,16 +26,19 @@ import scala.jdk.CollectionConverters._
2626import org .apache .hadoop .fs .Path
2727import org .apache .hadoop .mapreduce .{Job , TaskAttemptContext , TaskAttemptID , TaskID , TaskType }
2828import org .apache .hadoop .mapreduce .task .TaskAttemptContextImpl
29+ import org .apache .spark .TaskContext
2930import org .apache .spark .internal .io .FileCommitProtocol
3031import org .apache .spark .rdd .RDD
3132import org .apache .spark .sql .catalyst .InternalRow
33+ import org .apache .spark .sql .comet .execution .arrow .CometArrowConverters
3234import org .apache .spark .sql .execution .{SparkPlan , UnaryExecNode }
3335import org .apache .spark .sql .execution .metric .{SQLMetric , SQLMetrics }
3436import org .apache .spark .sql .vectorized .ColumnarBatch
3537import org .apache .spark .util .Utils
3638
37- import org .apache .comet .CometExecIterator
39+ import org .apache .comet .{ CometConf , CometExecIterator }
3840import org .apache .comet .serde .OperatorOuterClass .Operator
41+ import org .apache .comet .vector .CometVector
3942
4043/**
4144 * Comet physical operator for native Parquet write operations with FileCommitProtocol support.
@@ -138,16 +141,21 @@ case class CometNativeWriteExec(
138141 }
139142
140143 override def doExecuteColumnar (): RDD [ColumnarBatch ] = {
144+ // Check if the child produces Arrow/Comet batches or Spark batches
145+ val childIsComet = child.isInstanceOf [CometPlan ]
146+
141147 // Get the input data from the child operator
142148 val childRDD = if (child.supportsColumnar) {
143149 child.executeColumnar()
144150 } else {
145- // If child doesn't support columnar, convert to columnar
146- child.execute().mapPartitionsInternal { _ =>
147- // TODO this could delegate to CometRowToColumnar, but maybe Comet
148- // does not need to support this case?
149- throw new UnsupportedOperationException (
150- " Row-based child operators not yet supported for native write" )
151+ // If child doesn't support columnar, convert rows to Arrow columnar batches
152+ val maxRecordsPerBatch = CometConf .COMET_BATCH_SIZE .get(conf)
153+ val timeZoneId = conf.sessionLocalTimeZone
154+ val schema = child.schema
155+ child.execute().mapPartitionsInternal { rowIter =>
156+ val context = TaskContext .get()
157+ CometArrowConverters
158+ .rowToArrowBatchIter(rowIter, schema, maxRecordsPerBatch, timeZoneId, context)
151159 }
152160 }
153161
@@ -158,6 +166,10 @@ case class CometNativeWriteExec(
158166 val capturedJobTrackerID = jobTrackerID
159167 val capturedNativeOp = nativeOp
160168 val capturedAccumulator = taskCommitMessagesAccum // Capture accumulator for use in tasks
169+ val capturedChildIsComet = childIsComet
170+ val capturedSchema = child.schema
171+ val capturedMaxRecordsPerBatch = CometConf .COMET_BATCH_SIZE .get(conf)
172+ val capturedTimeZoneId = conf.sessionLocalTimeZone
161173
162174 // Execute native write operation with task-level commit protocol
163175 childRDD.mapPartitionsInternal { iter =>
@@ -201,9 +213,28 @@ case class CometNativeWriteExec(
201213 outputStream.close()
202214 val planBytes = outputStream.toByteArray
203215
216+ // Convert Spark columnar batches to Arrow format if child is not a Comet operator.
217+ // Comet native execution expects Arrow arrays, but Spark operators like RangeExec
218+ // produce OnHeapColumnVector which must be converted.
219+ val arrowIter = if (capturedChildIsComet) {
220+ // Child is already producing Arrow/Comet batches
221+ iter
222+ } else {
223+ // Convert Spark columnar batches to Arrow format
224+ val context = TaskContext .get()
225+ iter.flatMap { sparkBatch =>
226+ CometArrowConverters .columnarBatchToArrowBatchIter(
227+ sparkBatch,
228+ capturedSchema,
229+ capturedMaxRecordsPerBatch,
230+ capturedTimeZoneId,
231+ context)
232+ }
233+ }
234+
204235 val execIterator = new CometExecIterator (
205236 CometExec .newIterId,
206- Seq (iter ),
237+ Seq (arrowIter ),
207238 numOutputCols,
208239 planBytes,
209240 nativeMetrics,
0 commit comments