wForget commented on code in PR #4234:
URL: https://github.com/apache/datafusion-comet/pull/4234#discussion_r3212277309


##########
spark/src/main/scala/org/apache/spark/sql/comet/CometMapInBatchExec.scala:
##########
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.comet
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.{ContextAwareIterator, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.PythonUDF
+import org.apache.spark.sql.catalyst.plans.physical.Partitioning
+import org.apache.spark.sql.comet.shims.ShimCometMapInBatch
+import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, 
UnaryExecNode}
+import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
+import org.apache.spark.sql.execution.python.{BatchIterator, PythonSQLMetrics}
+import org.apache.spark.sql.types.{StructField, StructType}
+import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch}
+
+/**
+ * Comet replacement for Spark's `MapInBatchExec` family 
(`PythonMapInArrowExec` /
+ * `MapInArrowExec` in 4.1+ / `MapInPandasExec`). Accepts columnar input 
directly from a Comet
+ * child instead of going through the per-row `UnsafeProjection` that 
`ColumnarToRowExec` applies,
+ * and keeps the Python runner output as `ColumnarBatch` so downstream Comet 
operators consume it
+ * natively.
+ *
+ * What this eliminates: two `UnsafeProjection` copies (input and output) and 
the row transition
+ * between Comet and the Python operator. The internal row-to-Arrow IPC 
re-encoding inside
+ * `ArrowPythonRunner` is unchanged; full round-trip elimination is tracked in 
#4240.
+ */
+case class CometMapInBatchExec(
+    func: Expression,
+    output: Seq[Attribute],
+    child: SparkPlan,
+    isBarrier: Boolean,
+    pythonEvalType: Int)
+    extends UnaryExecNode
+    with CometPlan
+    with PythonSQLMetrics
+    with ShimCometMapInBatch {
+
+  override def supportsColumnar: Boolean = true
+
+  override def producedAttributes: AttributeSet = AttributeSet(output)
+
+  override def outputPartitioning: Partitioning = child.outputPartitioning
+
+  override lazy val metrics: Map[String, SQLMetric] = Map(
+    "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output 
rows"),
+    "numOutputBatches" -> SQLMetrics.createMetric(sparkContext, "number of 
output batches"),
+    "numInputRows" -> SQLMetrics.createMetric(sparkContext, "number of input 
rows")) ++
+    pythonMetrics
+
+  // Fallback for row-consuming parents (e.g. a top-level `collect()` that 
produces rows).
+  // Wraps this columnar exec in `ColumnarToRowExec`, reintroducing exactly 
the row transition
+  // this operator otherwise eliminates. Only fires when nothing downstream 
consumes columnar.
+  override def doExecute(): RDD[InternalRow] = {
+    ColumnarToRowExec(this).doExecute()
+  }
+
+  override def doExecuteColumnar(): RDD[ColumnarBatch] = {
+    val numOutputRows = longMetric("numOutputRows")
+    val numOutputBatches = longMetric("numOutputBatches")
+    val numInputRows = longMetric("numInputRows")
+
+    val pythonUDF = func.asInstanceOf[PythonUDF]
+    val outputAttrs = output
+    val childSchema = child.schema
+    val batchSize = conf.arrowMaxRecordsPerBatch
+    val evalType = pythonEvalType
+    val sqlConf = conf
+    val metricsCopy = pythonMetrics
+
+    val inputRDD = child.executeColumnar()
+
+    def processPartition(batches: Iterator[ColumnarBatch]): 
Iterator[ColumnarBatch] = {
+      val context = TaskContext.get()
+      val argOffsets = Array(Array(0))
+
+      val rowIter = batches.flatMap { batch =>
+        numInputRows += batch.numRows()
+        batch.rowIterator().asScala
+      }
+
+      val contextAwareIterator = new ContextAwareIterator(context, rowIter)
+
+      // Wrap rows as a struct, matching MapInBatchEvaluatorFactory behavior
+      val wrappedIter = contextAwareIterator.map(InternalRow(_))
+
+      val batchIter =
+        if (batchSize > 0) new BatchIterator(wrappedIter, batchSize) else 
Iterator(wrappedIter)
+
+      val columnarBatchIter = computeArrowPython(
+        pythonUDF,
+        evalType,
+        argOffsets,
+        StructType(Array(StructField("struct", childSchema))),
+        sqlConf,

Review Comment:
   `sqlConf` object should not be passed in the task calculation logic.
   
   ```
   E                   : org.apache.spark.SparkException: Job aborted due to 
stage failure: Task 0 in stage 2.0 failed 1 times, most recent failure: Lost 
task 0.0 in stage 2.0 (TID 3) (836a973dfdc3 executor driver): 
java.lang.NullPointerException: Cannot invoke 
"org.apache.spark.internal.config.ConfigReader.get(String)" because "reader" is 
null
   E                    at 
org.apache.spark.internal.config.ConfigEntry.readString(ConfigEntry.scala:96)
   E                    at 
org.apache.spark.internal.config.ConfigEntryWithDefaultFunction.readFrom(ConfigEntry.scala:173)
   E                    at 
org.apache.spark.sql.internal.SQLConf.getConf(SQLConf.scala:6661)
   E                    at 
org.apache.spark.sql.internal.SQLConf.sessionLocalTimeZone(SQLConf.scala:6231)
   E                    at 
org.apache.spark.sql.comet.shims.Spark4xMapInBatchSupport.runnerInputs(Spark4xMapInBatchSupport.scala:77)
   E                    at 
org.apache.spark.sql.comet.shims.Spark4xMapInBatchSupport.runnerInputs$(Spark4xMapInBatchSupport.scala:74)
   E                    at 
org.apache.spark.sql.comet.CometMapInBatchExec.runnerInputs(CometMapInBatchExec.scala:48)
   E                    at 
org.apache.spark.sql.comet.shims.ShimCometMapInBatch.computeArrowPython(ShimCometMapInBatch.scala:43)
   E                    at 
org.apache.spark.sql.comet.shims.ShimCometMapInBatch.computeArrowPython$(ShimCometMapInBatch.scala:33)
   E                    at 
org.apache.spark.sql.comet.CometMapInBatchExec.computeArrowPython(CometMapInBatchExec.scala:48)
   E                    at 
org.apache.spark.sql.comet.CometMapInBatchExec.processPartition$1(CometMapInBatchExec.scala:119)
   E                    at 
org.apache.spark.sql.comet.CometMapInBatchExec.$anonfun$doExecuteColumnar$6(CometMapInBatchExec.scala:137)
   E                    at 
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:901)
   E                    at 
org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:901)
   E                    at 
org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to