-
Notifications
You must be signed in to change notification settings - Fork 51
[SPARK-25299] Propose a new NIO transfer API for partition writing. #535
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 1 commit
7e13ed1
81e8a86
53f6bbd
9b77268
0dd4ffa
e98661e
1faf980
d12a86c
5ae75de
ce68613
f3dac6e
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,6 +21,7 @@ | |
| import java.io.FileInputStream; | ||
| import java.io.IOException; | ||
| import java.io.OutputStream; | ||
| import java.nio.channels.Channels; | ||
| import java.nio.channels.FileChannel; | ||
| import javax.annotation.Nullable; | ||
|
|
||
|
|
@@ -92,7 +93,6 @@ final class BypassMergeSortShuffleWriter<K, V> extends ShuffleWriter<K, V> { | |
| private final int mapId; | ||
| private final Serializer serializer; | ||
| private final ShuffleWriteSupport shuffleWriteSupport; | ||
| private final IndexShuffleBlockResolver shuffleBlockResolver; | ||
|
|
||
| /** Array of file writers, one for each partition */ | ||
| private DiskBlockObjectWriter[] partitionWriters; | ||
|
|
@@ -109,7 +109,6 @@ final class BypassMergeSortShuffleWriter<K, V> extends ShuffleWriter<K, V> { | |
|
|
||
| BypassMergeSortShuffleWriter( | ||
| BlockManager blockManager, | ||
| IndexShuffleBlockResolver shuffleBlockResolver, | ||
| BypassMergeSortShuffleHandle<K, V> handle, | ||
| int mapId, | ||
| SparkConf conf, | ||
|
|
@@ -126,7 +125,6 @@ final class BypassMergeSortShuffleWriter<K, V> extends ShuffleWriter<K, V> { | |
| this.numPartitions = partitioner.numPartitions(); | ||
| this.writeMetrics = writeMetrics; | ||
| this.serializer = dep.serializer(); | ||
| this.shuffleBlockResolver = shuffleBlockResolver; | ||
| this.shuffleWriteSupport = shuffleWriteSupport; | ||
| } | ||
|
|
||
|
|
@@ -211,14 +209,19 @@ private long[] writePartitionedData(ShuffleMapOutputWriter mapOutputWriter) thro | |
| try { | ||
| for (int i = 0; i < numPartitions; i++) { | ||
| final File file = partitionWriterSegments[i].file(); | ||
| boolean copyThrewException = true; | ||
| ShufflePartitionWriter writer = mapOutputWriter.getPartitionWriter(i); | ||
| if (file.exists()) { | ||
| if (transferToEnabled && writer instanceof SupportsTransferTo) { | ||
| boolean copyThrewException = true; | ||
| if (transferToEnabled) { | ||
| FileInputStream in = new FileInputStream(file); | ||
| TransferrableWritableByteChannel outputChannel = null; | ||
| try (FileChannel inputChannel = in.getChannel()) { | ||
| outputChannel = ((SupportsTransferTo) writer).openTransferrableChannel(); | ||
| if (writer instanceof SupportsTransferTo) { | ||
| outputChannel = ((SupportsTransferTo) writer).openTransferrableChannel(); | ||
| } else { | ||
| outputChannel = new DefaultTransferrableWritableByteChannel( | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. any point in having this else branch here? shouldn't the
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is to remain consistent with |
||
| Channels.newChannel(writer.openStream())); | ||
| } | ||
| TransferrableReadableByteChannel inputTransferable = | ||
| new FileTransferrableReadableByteChannel(inputChannel, 0L); | ||
|
||
| outputChannel.transferFrom(inputTransferable, inputChannel.size()); | ||
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -15,10 +15,12 @@ | |
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.api.shuffle; | ||
| package org.apache.spark.shuffle.sort; | ||
|
|
||
| import java.io.IOException; | ||
| import java.nio.channels.WritableByteChannel; | ||
| import org.apache.spark.api.shuffle.TransferrableReadableByteChannel; | ||
| import org.apache.spark.api.shuffle.TransferrableWritableByteChannel; | ||
|
|
||
| public class DefaultTransferrableWritableByteChannel implements TransferrableWritableByteChannel { | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. move this out of the api package and into |
||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
i think you're missing the case for if
file.exists()is false, you need to still set thecopyThrewExceptionto falseThere was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The value of
copyThrewExceptionisn't used if the file does not exist.