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

    https://github.com/apache/spark/pull/11359#discussion_r54369578
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/Sort.scala 
---
    @@ -93,4 +97,75 @@ case class Sort(
           sortedIterator
         }
       }
    +
    +  override def upstreams(): Seq[RDD[InternalRow]] = {
    +    child.asInstanceOf[CodegenSupport].upstreams()
    +  }
    +
    +  // 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 metrics = ctx.freshName("metrics")
    +    ctx.addMutableState(classOf[TaskMetrics].getName, metrics,
    +      s"$metrics = org.apache.spark.TaskContext.get().taskMetrics();")
    +    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")
    +    val dataSize = metricTerm(ctx, "dataSize")
    +    val spillSize = metricTerm(ctx, "spillSize")
    +    val spillSizeBefore = ctx.freshName("spillSizeBefore")
    +    ctx.addMutableState(classOf[Long].getName, spillSizeBefore, "")
    +    s"""
    +       | if ($needToSort) {
    +       |   $addToSorter();
    +       |   $spillSizeBefore = $metrics.memoryBytesSpilled();
    +       |   $sortedIterator = $sorterVariable.sort();
    +       |   $dataSize.add($sorterVariable.getPeakMemoryUsage());
    +       |   $spillSize.add($metrics.memoryBytesSpilled() - 
$spillSizeBefore);
    +       |   
$metrics.incPeakExecutionMemory($sorterVariable.getPeakMemoryUsage());
    +       |   $needToSort = false;
    +       | }
    +       |
    +       | while ($sortedIterator.hasNext()) {
    +       |   UnsafeRow $outputRow = (UnsafeRow)$sortedIterator.next();
    +       |   ${consume(ctx, null, outputRow)}
    +       |   if (shouldStop()) return;
    +       | }
    +     """.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, 
useSubexprElimination = false)
    --- End diff --
    
    Any reason to explicilty set subexpr to false?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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

Reply via email to