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 2022/11/07 17:57:02 UTC

[GitHub] [spark] pan3793 commented on a diff in pull request #38468: [SPARK-41005][CONNECT][PYTHON] Arrow-based collect

pan3793 commented on code in PR #38468:
URL: https://github.com/apache/spark/pull/38468#discussion_r1015733832


##########
connector/connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamHandler.scala:
##########
@@ -117,10 +129,91 @@ class SparkConnectStreamHandler(responseObserver: StreamObserver[Response]) exte
       responseObserver.onNext(response.build())
     }
 
-    responseObserver.onNext(sendMetricsToResponse(clientId, rows))
+    responseObserver.onNext(sendMetricsToResponse(clientId, dataframe))
     responseObserver.onCompleted()
   }
 
+  def processRowsAsArrowBatches(clientId: String, dataframe: DataFrame): Unit = {
+    val spark = dataframe.sparkSession
+    val schema = dataframe.schema
+    // TODO: control the batch size instead of max records
+    val maxRecordsPerBatch = spark.sessionState.conf.arrowMaxRecordsPerBatch
+    val timeZoneId = spark.sessionState.conf.sessionLocalTimeZone
+
+    SQLExecution.withNewExecutionId(dataframe.queryExecution, Some("collectArrow")) {
+      val pool = ThreadUtils.newDaemonSingleThreadExecutor("connect-collect-arrow")
+      val tasks = collection.mutable.ArrayBuffer.empty[Future[_]]
+      val rows = dataframe.queryExecution.executedPlan.execute()
+
+      if (rows.getNumPartitions > 0) {
+        val batches = rows.mapPartitionsInternal { iter =>
+          ArrowConverters
+            .toArrowBatchIterator(iter, schema, maxRecordsPerBatch, timeZoneId)
+        }
+
+        val processPartition = (iter: Iterator[(Array[Byte], Long, Long)]) => iter.toArray
+
+        val resultHandler = (partitionId: Int, taskResult: Array[(Array[Byte], Long, Long)]) => {
+          if (taskResult.exists(_._1.nonEmpty)) {
+            // only send non-empty partitions
+            val task = pool.submit(new Runnable {
+              override def run(): Unit = {
+                var batchId = partitionId.toLong << 33
+                taskResult.foreach { case (bytes, count, size) =>
+                  val response = proto.Response.newBuilder().setClientId(clientId)
+                  val batch = proto.Response.ArrowBatch
+                    .newBuilder()
+                    .setBatchId(batchId)
+                    .setRowCount(count)
+                    .setUncompressedBytes(size)
+                    .setCompressedBytes(bytes.length)
+                    .setData(ByteString.copyFrom(bytes))
+                    .build()
+                  response.setArrowBatch(batch)
+                  responseObserver.onNext(response.build())

Review Comment:
   If I understand correctly, the `TaskResultGetter` runs in a separate thread pool, and there is no back pressure mechanism, the current approach still may cause memory pressure on the driver if the client consumes results slowly.



-- 
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: reviews-unsubscribe@spark.apache.org

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


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