You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@kyuubi.apache.org by "cfmcgrady (via GitHub)" <gi...@apache.org> on 2023/04/04 07:34:49 UTC

[GitHub] [kyuubi] cfmcgrady commented on a diff in pull request #4662: [ARROW] Arrow serialization should not introduce extra shuffle for outermost limit

cfmcgrady commented on code in PR #4662:
URL: https://github.com/apache/kyuubi/pull/4662#discussion_r1156847207


##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/spark/sql/kyuubi/SparkDatasetHelper.scala:
##########
@@ -17,18 +17,95 @@
 
 package org.apache.spark.sql.kyuubi
 
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.TaskContext
+import org.apache.spark.network.util.{ByteUnit, JavaUtils}
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.{DataFrame, Dataset, Row}
+import org.apache.spark.sql.execution.{CollectLimitExec, SparkPlan, SQLExecution}
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec
+import org.apache.spark.sql.execution.arrow.{ArrowCollectUtils, ArrowConverters, KyuubiArrowUtils}
 import org.apache.spark.sql.functions._
 import org.apache.spark.sql.types._
 
+import org.apache.kyuubi.engine.spark.KyuubiSparkUtil
 import org.apache.kyuubi.engine.spark.schema.RowSet
 
 object SparkDatasetHelper {
+
   def toArrowBatchRdd[T](ds: Dataset[T]): RDD[Array[Byte]] = {
     ds.toArrowBatchRdd
   }
 
+  /**
+   * Forked from [[Dataset.toArrowBatchRdd(plan: SparkPlan)]].
+   * Convert to an RDD of serialized ArrowRecordBatches.
+   */
+  def toArrowBatchRdd(plan: SparkPlan): RDD[Array[Byte]] = {
+    val schemaCaptured = plan.schema
+    val maxRecordsPerBatch = plan.session.sessionState.conf.arrowMaxRecordsPerBatch
+    val timeZoneId = plan.session.sessionState.conf.sessionLocalTimeZone
+    plan.execute().mapPartitionsInternal { iter =>
+      val context = TaskContext.get()
+      ArrowConverters.toBatchIterator(
+        iter,
+        schemaCaptured,
+        maxRecordsPerBatch,
+        timeZoneId,
+        context)
+    }
+  }
+
+  def doCollectLimit(collectLimit: CollectLimitExec): Array[Array[Byte]] = {
+    val timeZoneId = collectLimit.session.sessionState.conf.sessionLocalTimeZone
+    val maxRecordsPerBatch = collectLimit.session.sessionState.conf.arrowMaxRecordsPerBatch
+
+    val batches = ArrowCollectUtils.takeAsArrowBatches(
+      collectLimit,
+      maxRecordsPerBatch,
+      maxBatchSize,
+      timeZoneId)
+
+    // note that the number of rows in the returned arrow batches may be >= `limit`, preform
+    // the slicing operation of result
+    val result = ArrayBuffer[Array[Byte]]()
+    var i = 0
+    var rest = collectLimit.limit
+    while (i < batches.length && rest > 0) {
+      val (batch, size) = batches(i)
+      if (size <= rest) {
+        result += batch
+        // returned ArrowRecordBatch has less than `limit` row count, safety to do conversion
+        rest -= size.toInt
+      } else { // size > rest
+        result += KyuubiArrowUtils.slice(collectLimit.schema, timeZoneId, batch, 0, rest)
+        rest = 0
+      }
+      i += 1
+    }
+    result.toArray
+  }
+
+  def executeCollect(df: DataFrame): Array[Array[Byte]] = withNewExecutionId(df) {
+    executeArrowBatchCollect(df.queryExecution.executedPlan)
+  }
+
+  def toArrowBatchLocalIterator(df: DataFrame): Iterator[Array[Byte]] = {
+    withNewExecutionId(df) {
+      toArrowBatchRdd(df).toLocalIterator
+    }
+  }
+
+  def executeArrowBatchCollect: SparkPlan => Array[Array[Byte]] = {
+    case adaptiveSparkPlan: AdaptiveSparkPlanExec =>
+      executeArrowBatchCollect(adaptiveSparkPlan.finalPhysicalPlan)

Review Comment:
   the `AdaptiveSparkPlanExec.finalPhysicalPlan` function was introduced in [SPARK-41914](https://github.com/apache/spark/pull/39431), and it may present compatibility issues if the underlying Spark runtime lacks the corresponding patch.



-- 
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: notifications-unsubscribe@kyuubi.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org