yma11 commented on code in PR #5462:
URL: https://github.com/apache/incubator-gluten/pull/5462#discussion_r1575695397


##########
backends-velox/src/main/scala/org/apache/gluten/execution/python/ColumnarArrowEvalPythonExec.scala:
##########
@@ -0,0 +1,365 @@
+/*
+ * 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.api.python
+
+import org.apache.gluten.columnarbatch.ColumnarBatches
+import org.apache.gluten.extension.GlutenPlan
+import org.apache.gluten.memory.arrowalloc.ArrowBufferAllocators
+import org.apache.gluten.utils.Iterators
+import org.apache.gluten.vectorized.ArrowWritableColumnVector
+
+import org.apache.spark.{ContextAwareIterator, SparkEnv, TaskContext}
+import org.apache.spark.internal.Logging
+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.execution.SparkPlan
+import org.apache.spark.sql.execution.python.{BasePythonRunnerShim, 
EvalPythonExec, PythonUDFRunner}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import org.apache.spark.sql.utils.{SparkArrowUtil, SparkSchemaUtil, 
SparkVectorUtil}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
+import org.apache.spark.util.Utils
+
+import org.apache.arrow.vector.{VectorLoader, VectorSchemaRoot}
+import org.apache.arrow.vector.ipc.{ArrowStreamReader, ArrowStreamWriter}
+
+import java.io.{DataInputStream, DataOutputStream}
+import java.net.Socket
+import java.util.concurrent.atomic.AtomicBoolean
+
+import scala.collection.mutable.ArrayBuffer
+
+class ColumnarArrowPythonRunner(
+    funcs: Seq[ChainedPythonFunctions],
+    evalType: Int,
+    argOffsets: Array[Array[Int]],
+    schema: StructType,
+    timeZoneId: String,
+    conf: Map[String, String])
+  extends BasePythonRunnerShim(funcs, evalType, argOffsets) {
+
+  override val simplifiedTraceback: Boolean = 
SQLConf.get.pysparkSimplifiedTraceback
+
+  override val bufferSize: Int = SQLConf.get.pandasUDFBufferSize
+  require(
+    bufferSize >= 4,
+    "Pandas execution requires more than 4 bytes. Please set higher buffer. " +
+      s"Please change '${SQLConf.PANDAS_UDF_BUFFER_SIZE.key}'.")
+
+  protected def newReaderIterator(
+      stream: DataInputStream,
+      writerThread: WriterThread,
+      startTime: Long,
+      env: SparkEnv,
+      worker: Socket,
+      pid: scala.Option[scala.Int],
+      releasedOrClosed: AtomicBoolean,
+      context: TaskContext): Iterator[ColumnarBatch] = {
+
+    new ReaderIterator(
+      stream,
+      writerThread,
+      startTime,
+      env,
+      worker,
+      None,
+      releasedOrClosed,
+      context) {
+      private val allocator = ArrowBufferAllocators.contextInstance()
+
+      private var reader: ArrowStreamReader = _
+      private var root: VectorSchemaRoot = _
+      private var schema: StructType = _
+      private var vectors: Array[ColumnVector] = _
+
+      context.addTaskCompletionListener[Unit] {
+        _ =>
+          if (reader != null) {
+            reader.close(false)
+          }
+          if (root != null) {
+            root.close()
+          }
+      }
+
+      private var batchLoaded = true
+
+      override protected def read(): ColumnarBatch = {
+        if (writerThread.exception.isDefined) {
+          throw writerThread.exception.get
+        }
+        try {
+          if (reader != null && batchLoaded) {
+            batchLoaded = reader.loadNextBatch()
+            if (batchLoaded) {
+              val batch = new ColumnarBatch(vectors)
+              batch.setNumRows(root.getRowCount)
+              batch
+            } else {
+              reader.close(false)
+              // Reach end of stream. Call `read()` again to read control data.
+              read()
+            }
+          } else {
+            stream.readInt() match {
+              case SpecialLengths.START_ARROW_STREAM =>
+                reader = new ArrowStreamReader(stream, allocator)
+                root = reader.getVectorSchemaRoot()
+                schema = SparkArrowUtil.fromArrowSchema(root.getSchema())
+                vectors = ArrowWritableColumnVector
+                  .loadColumns(root.getRowCount, root.getFieldVectors)
+                  .toArray[ColumnVector]
+                read()
+              case SpecialLengths.TIMING_DATA =>
+                handleTimingData()
+                read()
+              case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
+                throw handlePythonException()
+              case SpecialLengths.END_OF_DATA_SECTION =>
+                handleEndOfDataSection()
+                null
+            }
+          }
+        } catch handleException

Review Comment:
   `handleException` is implemented in base class for exception processing, 
refer 
[code](https://github.com/apache/spark/blob/branch-3.3/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala#L581-L601).



-- 
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