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/07/31 02:10:03 UTC

[GitHub] [spark] squito commented on a change in pull request #25304: [SPARK-28570][CORE][SHUFFLE] Make UnsafeShuffleWriter use the new API.

squito commented on a change in pull request #25304: [SPARK-28570][CORE][SHUFFLE] Make UnsafeShuffleWriter use the new API.
URL: https://github.com/apache/spark/pull/25304#discussion_r309011967
 
 

 ##########
 File path: core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java
 ##########
 @@ -316,102 +304,100 @@ void forceSorterToSpill() throws IOException {
           // that doesn't need to interpret the spilled bytes.
           if (transferToEnabled && !encryptionEnabled) {
             logger.debug("Using transferTo-based fast merge");
-            partitionLengths = mergeSpillsWithTransferTo(spills, outputFile);
+            partitionLengths = mergeSpillsWithTransferTo(spills, mapWriter);
           } else {
             logger.debug("Using fileStream-based fast merge");
-            partitionLengths = mergeSpillsWithFileStream(spills, outputFile, null);
+            partitionLengths = mergeSpillsWithFileStream(spills, mapWriter, null);
           }
         } else {
           logger.debug("Using slow merge");
-          partitionLengths = mergeSpillsWithFileStream(spills, outputFile, compressionCodec);
+          partitionLengths = mergeSpillsWithFileStream(spills, mapWriter, compressionCodec);
         }
         // When closing an UnsafeShuffleExternalSorter that has already spilled once but also has
         // in-memory records, we write out the in-memory records to a file but do not count that
         // final write as bytes spilled (instead, it's accounted as shuffle write). The merge needs
         // to be counted as shuffle write, but this will lead to double-counting of the final
         // SpillInfo's bytes.
         writeMetrics.decBytesWritten(spills[spills.length - 1].file.length());
-        writeMetrics.incBytesWritten(outputFile.length());
         return partitionLengths;
       }
     } catch (IOException e) {
-      if (outputFile.exists() && !outputFile.delete()) {
-        logger.error("Unable to delete output file {}", outputFile.getPath());
-      }
       throw e;
     }
   }
 
   /**
    * Merges spill files using Java FileStreams. This code path is typically slower than
    * the NIO-based merge, {@link UnsafeShuffleWriter#mergeSpillsWithTransferTo(SpillInfo[],
-   * File)}, and it's mostly used in cases where the IO compression codec does not support
-   * concatenation of compressed data, when encryption is enabled, or when users have
-   * explicitly disabled use of {@code transferTo} in order to work around kernel bugs.
+   * ShuffleMapOutputWriter)}, and it's mostly used in cases where the IO compression codec
+   * does not support concatenation of compressed data, when encryption is enabled, or when
+   * users have explicitly disabled use of {@code transferTo} in order to work around kernel bugs.
    * This code path might also be faster in cases where individual partition size in a spill
    * is small and UnsafeShuffleWriter#mergeSpillsWithTransferTo method performs many small
    * disk ios which is inefficient. In those case, Using large buffers for input and output
    * files helps reducing the number of disk ios, making the file merging faster.
    *
    * @param spills the spills to merge.
-   * @param outputFile the file to write the merged data to.
+   * @param mapWriter the map output writer to use for output.
    * @param compressionCodec the IO compression codec, or null if shuffle compression is disabled.
    * @return the partition lengths in the merged file.
    */
   private long[] mergeSpillsWithFileStream(
       SpillInfo[] spills,
-      File outputFile,
+      ShuffleMapOutputWriter mapWriter,
       @Nullable CompressionCodec compressionCodec) throws IOException {
-    assert (spills.length >= 2);
     final int numPartitions = partitioner.numPartitions();
     final long[] partitionLengths = new long[numPartitions];
     final InputStream[] spillInputStreams = new InputStream[spills.length];
 
-    final OutputStream bos = new BufferedOutputStream(
-            new FileOutputStream(outputFile),
-            outputBufferSizeInBytes);
-    // Use a counting output stream to avoid having to close the underlying file and ask
-    // the file system for its size after each partition is written.
-    final CountingOutputStream mergedFileOutputStream = new CountingOutputStream(bos);
-
     boolean threwException = true;
     try {
       for (int i = 0; i < spills.length; i++) {
         spillInputStreams[i] = new NioBufferedFileInputStream(
-            spills[i].file,
-            inputBufferSizeInBytes);
+          spills[i].file,
+          inputBufferSizeInBytes);
       }
       for (int partition = 0; partition < numPartitions; partition++) {
-        final long initialFileLength = mergedFileOutputStream.getByteCount();
-        // Shield the underlying output stream from close() and flush() calls, so that we can close
-        // the higher level streams to make sure all data is really flushed and internal state is
-        // cleaned.
-        OutputStream partitionOutput = new CloseAndFlushShieldOutputStream(
-          new TimeTrackingOutputStream(writeMetrics, mergedFileOutputStream));
-        partitionOutput = blockManager.serializerManager().wrapForEncryption(partitionOutput);
-        if (compressionCodec != null) {
-          partitionOutput = compressionCodec.compressedOutputStream(partitionOutput);
-        }
-        for (int i = 0; i < spills.length; i++) {
-          final long partitionLengthInSpill = spills[i].partitionLengths[partition];
-          if (partitionLengthInSpill > 0) {
-            InputStream partitionInputStream = new LimitedInputStream(spillInputStreams[i],
-              partitionLengthInSpill, false);
-            try {
-              partitionInputStream = blockManager.serializerManager().wrapForEncryption(
-                partitionInputStream);
-              if (compressionCodec != null) {
-                partitionInputStream = compressionCodec.compressedInputStream(partitionInputStream);
+        boolean copyThrewExecption = true;
+        ShufflePartitionWriter writer = mapWriter.getPartitionWriter(partition);
+        OutputStream partitionOutput = writer.openStream();
+        try {
+          // Shield the underlying output stream from close() calls, so that we can close the
+          // higher level streams to make sure all data is really flushed and internal state
+          // is cleaned
 
 Review comment:
   comment doesn't belong here anymore -- you're not doing any shielding here, its left to individual implementations to to do (as you do in LocalDiskShuffleMapOutputWriter)

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