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/03 17:03:03 UTC

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

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


##########
connector/connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamHandler.scala:
##########
@@ -117,7 +131,36 @@ 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 schema = dataframe.schema
+    val maxRecordsPerBatch = dataframe.sparkSession.sessionState.conf.arrowMaxRecordsPerBatch
+    val timeZoneId = dataframe.sparkSession.sessionState.conf.sessionLocalTimeZone
+
+    val batches = dataframe.queryExecution.executedPlan
+      .execute()
+      .mapPartitionsInternal { iter =>
+        ArrowConverters
+          .toArrowBatchIterator(iter, schema, maxRecordsPerBatch, timeZoneId, TaskContext.get)
+      }
+
+    batches.toLocalIterator.foreach { case (bytes, count, size) =>

Review Comment:
   Let's also avoid using local iterator. That will make things unnecessarily slow because it limits parallelism. I was thinking doing the following:
   ```scala
   def collectArrow(callback: Array[Array[Byte]] => ()): Unit = {
       withAction("collectArrow", queryExecution) { plan =>
         val rdd = toArrowBatchRdd(plan)
         val signal = new Object
         val numPartitions = rdd.getNumPartitions
         val availablePartitions = mutable.Map.empty[Int, Array[Array[Byte]]]
         def onNewPartition(partitionId: Int, batches: Array[Array[Byte]]): Unit = {
           signal.synchronized {
             availablePartitions(partitionId) = batches
             signal.notify()
           }
         }
         sparkSession.sparkContext.submitJob(
           rdd = rdd,
           processPartition = (i: Iterator[Array[Byte]]) => i.toArray,
           partitions = 0 until numPartitions,
           resultHandler = onNewPartition,
           resultFunc = () => ())
   
         var currentPartitionId = 0
         while (currentPartitionId < numPartitions) {
           val batches = signal.synchronized {
             var result = availablePartitions.remove(currentPartitionId)
             while (result.isEmpty) {
               signal.wait()
               result = availablePartitions.remove(currentPartitionId)
             }
             result.get
           }
           callback(batches)
           currentPartitionId += 1
         }
       }
     }
   ```
   This was hacked together in Dataset, but IMO it is better to do this somewhere in here. 



-- 
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