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 2020/09/28 16:20:38 UTC

[GitHub] [spark] attilapiros commented on a change in pull request #28618: [SPARK-31801][API][SHUFFLE] Register map output metadata

attilapiros commented on a change in pull request #28618:
URL: https://github.com/apache/spark/pull/28618#discussion_r495953526



##########
File path: core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java
##########
@@ -262,33 +268,35 @@ void forceSorterToSpill() throws IOException {
    *
    * @return the partition lengths in the merged file.
    */
-  private long[] mergeSpills(SpillInfo[] spills) throws IOException {
-    long[] partitionLengths;
+  private MapOutputCommitMessage mergeSpills(SpillInfo[] spills) throws IOException {
+    MapOutputCommitMessage mapOutputCommitMessage;
     if (spills.length == 0) {
       final ShuffleMapOutputWriter mapWriter = shuffleExecutorComponents
           .createMapOutputWriter(shuffleId, mapId, partitioner.numPartitions());
-      return mapWriter.commitAllPartitions().getPartitionLengths();
+      return mapWriter.commitAllPartitions();
     } else if (spills.length == 1) {
       Optional<SingleSpillShuffleMapOutputWriter> maybeSingleFileWriter =
           shuffleExecutorComponents.createSingleFileMapOutputWriter(shuffleId, mapId);
       if (maybeSingleFileWriter.isPresent()) {
         // Here, we don't need to perform any metrics updates because the bytes written to this
         // output file would have already been counted as shuffle bytes written.
-        partitionLengths = spills[0].partitionLengths;
+        long[] partitionLengths = spills[0].partitionLengths;
         logger.debug("Merge shuffle spills for mapId {} with length {}", mapId,

Review comment:
       Is not this log a bit misleading? I mean here is no merge done as there is only one spill.
   
   Even `transferMapSpillFile ` says:
   > The map spill file already has the proper format, and it contains all of the partition data.
   > So just transfer it directly to the destination without any merging.
   

##########
File path: core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java
##########
@@ -262,33 +268,35 @@ void forceSorterToSpill() throws IOException {
    *
    * @return the partition lengths in the merged file.
    */
-  private long[] mergeSpills(SpillInfo[] spills) throws IOException {
-    long[] partitionLengths;
+  private MapOutputCommitMessage mergeSpills(SpillInfo[] spills) throws IOException {
+    MapOutputCommitMessage mapOutputCommitMessage;
     if (spills.length == 0) {
       final ShuffleMapOutputWriter mapWriter = shuffleExecutorComponents
           .createMapOutputWriter(shuffleId, mapId, partitioner.numPartitions());
-      return mapWriter.commitAllPartitions().getPartitionLengths();
+      return mapWriter.commitAllPartitions();

Review comment:
       Nit:
   ```suggestion
         mapOutputCommitMessage = mapWriter.commitAllPartitions();
   ```




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



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