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