zhengruifeng commented on code in PR #41867:
URL: https://github.com/apache/spark/pull/41867#discussion_r1260613692


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonUDTFExec.scala:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.execution.python
+
+import java.io.File
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.{ContextAwareIterator, SparkEnv, 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.execution.UnaryExecNode
+import org.apache.spark.sql.types.{DataType, StructField, StructType}
+import org.apache.spark.util.Utils
+
+/**
+ * A physical plan that evaluates a [[PythonUDTF]], one partition of tuples at 
a time.
+ * This is similar to [[EvalPythonExec]].
+ */
+trait EvalPythonUDTFExec extends UnaryExecNode {
+  def udtf: PythonUDTF
+
+  def requiredChildOutput: Seq[Attribute]
+
+  def resultAttrs: Seq[Attribute]
+
+  override def output: Seq[Attribute] = requiredChildOutput ++ resultAttrs
+
+  override def producedAttributes: AttributeSet = AttributeSet(resultAttrs)
+
+  protected def evaluate(
+      argOffsets: Array[Int],
+      iter: Iterator[InternalRow],
+      schema: StructType,
+      context: TaskContext): Iterator[Iterator[InternalRow]]
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    val inputRDD = child.execute().map(_.copy())
+
+    inputRDD.mapPartitions { iter =>
+      val context = TaskContext.get()
+      val contextAwareIterator = new ContextAwareIterator(context, iter)
+
+      // The queue used to buffer input rows so we can drain it to
+      // combine input with output from Python.
+      val queue = HybridRowQueue(context.taskMemoryManager(),
+        new File(Utils.getLocalDir(SparkEnv.get.conf)), child.output.length)
+      context.addTaskCompletionListener[Unit] { ctx =>
+        queue.close()
+      }
+
+      // flatten all the arguments
+      val allInputs = new ArrayBuffer[Expression]
+      val dataTypes = new ArrayBuffer[DataType]
+      val argOffsets = udtf.children.map { e =>
+        if (allInputs.exists(_.semanticEquals(e))) {
+          allInputs.indexWhere(_.semanticEquals(e))
+        } else {
+          allInputs += e
+          dataTypes += e.dataType
+          allInputs.length - 1
+        }
+      }.toArray
+      val projection = MutableProjection.create(allInputs.toSeq, child.output)
+      projection.initialize(context.partitionId())
+      val schema = StructType(dataTypes.zipWithIndex.map { case (dt, i) =>
+        StructField(s"_$i", dt)
+      }.toArray)
+
+      // Add rows to the queue to join later with the result.
+      // Also keep track of the number rows added to the queue.
+      // This is needed to process extra output rows from the `terminate()` 
call of the UDTF.
+      var count = 0L
+      val projectedRowIter = contextAwareIterator.map { inputRow =>
+        queue.add(inputRow.asInstanceOf[UnsafeRow])
+        count += 1
+        projection(inputRow)
+      }
+
+      val outputRowIterator = evaluate(argOffsets, projectedRowIter, schema, 
context)
+
+      val pruneChildForResult: InternalRow => InternalRow =
+        if (child.outputSet == AttributeSet(requiredChildOutput)) {
+          identity
+        } else {
+          UnsafeProjection.create(requiredChildOutput, child.output)

Review Comment:
   not related to this PR itself:
   why not introducing a `Project` after `child`, when `child.output` doesn't 
match `requiredChildOutput`?
   



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