You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2019/07/01 06:11:20 UTC

[GitHub] [spark] cloud-fan commented on a change in pull request #22347: [SPARK-25353][SQL] executeTake in SparkPlan is modified to avoid unnecessary decoding.

cloud-fan commented on a change in pull request #22347: [SPARK-25353][SQL] executeTake in SparkPlan is modified to avoid unnecessary decoding.
URL: https://github.com/apache/spark/pull/22347#discussion_r298891651
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
 ##########
 @@ -348,30 +349,30 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ
         // Otherwise, interpolate the number of partitions we need to try, but overestimate
         // it by 50%. We also cap the estimation in the end.
         val limitScaleUpFactor = Math.max(sqlContext.conf.limitScaleUpFactor, 2)
-        if (buf.isEmpty) {
+        if (scannedRowCount == 0) {
           numPartsToTry = partsScanned * limitScaleUpFactor
         } else {
-          val left = n - buf.size
+          val left = n - scannedRowCount
           // As left > 0, numPartsToTry is always >= 1
-          numPartsToTry = Math.ceil(1.5 * left * partsScanned / buf.size).toInt
+          numPartsToTry = Math.ceil(1.5 * left * partsScanned / scannedRowCount).toInt
           numPartsToTry = Math.min(numPartsToTry, partsScanned * limitScaleUpFactor)
         }
       }
 
       val p = partsScanned.until(math.min(partsScanned + numPartsToTry, totalParts).toInt)
       val sc = sqlContext.sparkContext
-      val res = sc.runJob(childRDD,
-        (it: Iterator[Array[Byte]]) => if (it.hasNext) it.next() else Array.empty[Byte], p)
-
-      buf ++= res.flatMap(decodeUnsafeRows)
+      val res = sc.runJob(childRDD, (it: Iterator[(Long, Array[Byte])]) =>
+        if (it.hasNext) it.next() else (0L, Array.empty[Byte]), p)
 
+      buf ++= res.map(_._2)
+      scannedRowCount += res.map(_._1).sum
       partsScanned += p.size
     }
 
-    if (buf.size > n) {
-      buf.take(n).toArray
+    if (scannedRowCount > n) {
+      buf.iterator.flatMap(decodeUnsafeRows).take(n).toArray
 
 Review comment:
   nit: since this is perf critical code path, I think we can optimize it further, since we know the length of the result array.
   ```
   val result = new Array[InternalRow](n)
   while (result.length < n) {
     // decode
   }
   result
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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