vinodkc commented on code in PR #42014:
URL: https://github.com/apache/spark/pull/42014#discussion_r1264555511
##########
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:
Done. Used mapPartitionsWithIndexInternal
--
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]