You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "Hisoka-X (via GitHub)" <gi...@apache.org> on 2023/07/15 04:13:19 UTC

[GitHub] [spark] Hisoka-X commented on a diff in pull request #42014: [SPARK-44412][SQL] Use PartitionEvaluator API in ArrowEvalPythonUDTFExec & BatchEvalPythonUDTFExec

Hisoka-X commented on code in PR #42014:
URL: https://github.com/apache/spark/pull/42014#discussion_r1264316395


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonUDTFExec.scala:
##########
@@ -40,85 +33,19 @@ trait EvalPythonUDTFExec extends UnaryExecNode {
 
   def resultAttrs: Seq[Attribute]
 
+  protected def evaluatorFactory: EvalPythonUDTFEvaluatorFactory
+
   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)
-        }
-
-      val joined = new JoinedRow
-      val resultProj = UnsafeProjection.create(output, output)
-
-      outputRowIterator.flatMap { outputRows =>
-        // If `count` is greater than zero, it means there are remaining input rows in the queue.
-        // In this case, the output rows of the UDTF are joined with the corresponding input row
-        // in the queue.
-        if (count > 0) {
-          val left = queue.remove()
-          count -= 1
-          joined.withLeft(pruneChildForResult(left))
-        }
-        // If `count` is zero, it means all input rows have been consumed. Any additional rows
-        // from the UDTF are from the `terminate()` call. We leave the left side as the last
-        // element of its child output to keep it consistent with the Generate implementation
-        // and Hive UDTFs.
-        outputRows.map(r => resultProj(joined.withRight(r)))
+    if (conf.usePartitionEvaluator) {
+      inputRDD.mapPartitionsWithEvaluator(evaluatorFactory)
+    } else {
+      inputRDD.mapPartitions { iter =>
+        evaluatorFactory.createEvaluator().eval(0, iter)

Review Comment:
   Hardcoding `partIndex` may lead to unpredictable situations in maintaining `EvalPythonUDTFEvaluatorFactory`, if subsequent `EvalPythonUDTFEvaluatorFactory` needs to use `partIndex`.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org