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: [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]

Reply via email to