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 2019/05/21 03:42:03 UTC

[GitHub] [spark] HyukjinKwon commented on a change in pull request #24650: [SPARK-27778][PySpark] Fix toPandas conversion using arrow for DFs with no partitions

HyukjinKwon commented on a change in pull request #24650: [SPARK-27778][PySpark] Fix toPandas conversion using arrow for DFs with no partitions
URL: https://github.com/apache/spark/pull/24650#discussion_r285839287
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
 ##########
 @@ -3310,23 +3310,31 @@ class Dataset[T] private[sql](
 
           // After last batch, end the stream and write batch order indices
           if (partitionCount == numPartitions) {
-            batchWriter.end()
-            out.writeInt(batchOrder.length)
-            // Sort by (index of partition, batch index in that partition) tuple to get the
-            // overall_batch_index from 0 to N-1 batches, which can be used to put the
-            // transferred batches in the correct order
-            batchOrder.zipWithIndex.sortBy(_._1).foreach { case (_, overallBatchIndex) =>
-              out.writeInt(overallBatchIndex)
-            }
-            out.flush()
+            doAfterLastPartition()
+          }
+        }
+
+        def doAfterLastPartition(): Unit = {
+          batchWriter.end()
+          out.writeInt(batchOrder.length)
+          // Sort by (index of partition, batch index in that partition) tuple to get the
+          // overall_batch_index from 0 to N-1 batches, which can be used to put the
+          // transferred batches in the correct order
+          batchOrder.zipWithIndex.sortBy(_._1).foreach { case (_, overallBatchIndex) =>
+            out.writeInt(overallBatchIndex)
           }
+          out.flush()
         }
 
         sparkSession.sparkContext.runJob(
           arrowBatchRdd,
           (ctx: TaskContext, it: Iterator[Array[Byte]]) => it.toArray,
           0 until numPartitions,
           handlePartitionBatches)
+
+        if (numPartitions == 0) {
 
 Review comment:
   This method is well-commented. Can you add another comment that we should end stream when partitions are empty?
   
   Also, I would do:
   
   ```scala
   partitions = 0 until numPartitions
   sparkSession.sparkContext.runJob(
     arrowBatchRdd,
     (ctx: TaskContext, it: Iterator[Array[Byte]]) => it.toArray,
     partitions,
     handlePartitionBatches)
   
   if (partitions.isEmpty) {
     // Currently result handler is not called when given partitions are empty.
     // Therefore, we should end stream here.
     doAfterLastPartition()
   }
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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