Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11008#discussion_r53856345
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/Sort.scala 
---
    @@ -93,4 +98,63 @@ case class Sort(
           sortedIterator
         }
       }
    +
    +  override def upstream(): RDD[InternalRow] = {
    +    child.asInstanceOf[CodegenSupport].upstream()
    +  }
    +
    +  // Name of sorter variable used in codegen.
    +  private var sorterVariable: String = _
    +
    +  override protected def doProduce(ctx: CodegenContext): String = {
    +    val needToSort = ctx.freshName("needToSort")
    +    ctx.addMutableState("boolean", needToSort, s"$needToSort = true;")
    +
    +
    +    // Initialize the class member variables. This includes the instance 
of the Sorter and
    +    // the iterator to return sorted rows.
    +    val thisPlan = ctx.addReferenceObj("plan", this)
    +    sorterVariable = ctx.freshName("sorter")
    +    ctx.addMutableState(classOf[UnsafeExternalRowSorter].getName, 
sorterVariable,
    +      s"$sorterVariable = $thisPlan.createSorter();")
    +    val sortedIterator = ctx.freshName("sortedIter")
    +    ctx.addMutableState("scala.collection.Iterator<UnsafeRow>", 
sortedIterator, "")
    +
    +    val addToSorter = ctx.freshName("addToSorter")
    +    ctx.addNewFunction(addToSorter,
    +      s"""
    +        | private void $addToSorter() throws java.io.IOException {
    +        |   ${child.asInstanceOf[CodegenSupport].produce(ctx, this)}
    +        | }
    +      """.stripMargin.trim)
    +
    +    val outputRow = ctx.freshName("outputRow")
    +    s"""
    +       | if ($needToSort) {
    +       |   $addToSorter();
    +       |   $sortedIterator = $sorterVariable.sort();
    +       |   $needToSort = false;
    +       | }
    +       |
    +       | while ($sortedIterator.hasNext()) {
    +       |   UnsafeRow $outputRow = (UnsafeRow)$sortedIterator.next();
    +       |   ${consume(ctx, null, outputRow)}
    +       | }
    +     """.stripMargin.trim
    +  }
    +
    +  override def doConsume(ctx: CodegenContext, input: Seq[ExprCode]): 
String = {
    +    val colExprs = child.output.zipWithIndex.map { case (attr, i) =>
    +      BoundReference(i, attr.dataType, attr.nullable)
    +    }
    +
    +    ctx.currentVars = input
    +    val code = GenerateUnsafeProjection.createCode(ctx, colExprs, false)
    --- End diff --
    
    If the child can produce UnsafeRow (for example, Exchange), we should have 
a way to avoid this unpack and pack again, or we will see regression (generated 
version slower than non-generated).
    
    I think we can pass the variable for input row into `doCosume`, could be 
null. It's better to do this after #11274 , then we don't need to worry about 
whether should we create variables for input or not.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

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

Reply via email to