Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
33 commits
Select commit Hold shift + click to select a range
1957e82
[SPARK-25299] Introduce the new shuffle writer API (#5) (#520)
mccheah Mar 20, 2019
857552a
[SPARK-25299] Local shuffle implementation of the shuffle writer API …
mccheah Apr 3, 2019
d13037f
[SPARK-25299] Make UnsafeShuffleWriter use the new API (#536)
mccheah Apr 17, 2019
8f5fb60
[SPARK-25299] Use the shuffle writer plugin for the SortShuffleWriter…
mccheah Apr 15, 2019
e17c7ea
[SPARK-25299] Shuffle locations api (#517)
mccheah Apr 19, 2019
3f0c131
[SPARK-25299] Move shuffle writers back to being given specific parti…
mccheah Apr 19, 2019
f982df7
[SPARK-25299] Don't set map status twice in bypass merge sort shuffle…
mccheah Apr 19, 2019
6891197
[SPARK-25299] Propose a new NIO transfer API for partition writing. (…
mccheah May 24, 2019
7b44ed2
Remove shuffle location support.
mccheah Jun 27, 2019
df75f1f
Remove changes to UnsafeShuffleWriter
mccheah Jun 27, 2019
a8558af
Revert changes for SortShuffleWriter
mccheah Jun 27, 2019
806d7bb
Revert a bunch of other stuff
mccheah Jun 27, 2019
3167030
More reverts
mccheah Jun 27, 2019
70f59db
Set task contexts in failing test
mccheah Jun 28, 2019
3083d86
Fix style
mccheah Jun 28, 2019
4c3d692
Check for null on the block manager as well.
mccheah Jun 28, 2019
2421c92
Add task attempt id in the APIs
mccheah Jul 1, 2019
982f207
Address comments
mccheah Jul 8, 2019
594d1e2
Fix style
mccheah Jul 8, 2019
66aae91
Address comments.
mccheah Jul 12, 2019
8b432f9
Merge remote-tracking branch 'origin/master' into spark-shuffle-write…
mccheah Jul 17, 2019
9f597dd
Address comments.
mccheah Jul 18, 2019
86c1829
Restructure test
mccheah Jul 18, 2019
a7885ae
Add ShuffleWriteMetricsReporter to the createMapOutputWriter API.
mccheah Jul 19, 2019
9893c6c
Add more documentation
mccheah Jul 19, 2019
cd897e7
REfactor reading records from file in test
mccheah Jul 19, 2019
9f17b9b
Address comments
mccheah Jul 24, 2019
e53a001
Code tags
mccheah Jul 24, 2019
56fa450
Add some docs
mccheah Jul 24, 2019
b8b7b8d
Change mockito format in BypassMergeSortShuffleWriterSuite
mccheah Jul 25, 2019
2d29404
Remove metrics from the API.
mccheah Jul 29, 2019
06ea01a
Address more comments.
mccheah Jul 29, 2019
7dceec9
Args per line
mccheah Jul 30, 2019
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
Address more comments.
  • Loading branch information
mccheah committed Jul 29, 2019
commit 06ea01afc0660025564acff8f1d1e695a72a35cc
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.spark.shuffle.api;

import java.io.IOException;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: white space between different import groups.


import org.apache.spark.annotation.Private;

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,10 @@
package org.apache.spark.shuffle.api;

import java.io.IOException;
import java.util.Optional;
import java.io.OutputStream;
import java.nio.channels.Channels;

import org.apache.spark.annotation.Private;
import org.apache.spark.shuffle.sort.io.DefaultWritableByteChannelWrapper;

/**
* :: Private ::
Expand Down Expand Up @@ -67,25 +66,24 @@ public interface ShufflePartitionWriter {
* Implementations that intend on combining the bytes for all the partitions written by this
* map task should reuse the same channel instance across all the partition writers provided
* by the parent {@link ShuffleMapOutputWriter}. If one does so, ensure that
* {@link WritableByteChannelWrapper#close()} does not close the resource, since it
* {@link WritableByteChannelWrapper#close()} does not close the resource, since the channel
* will be reused across partition writes. The underlying resources should be cleaned up in
* {@link ShuffleMapOutputWriter#commitAllPartitions()} and
* {@link ShuffleMapOutputWriter#abort(Throwable)}.
* <p>
* This method is primarily for advanced optimizations where bytes can be copied from the input
* spill files to the output channel without copying data into memory.
* <p>
* The default implementation should be sufficient for most situations. Only override this
* method if there is a very specific optimization that needs to be built.
* spill files to the output channel without copying data into memory. If such optimizations are
* not supported, the implementation should return {@link Optional#empty()}. By default, the
* implementation returns {@link Optional#empty()}.
* <p>
* Note that the returned {@link WritableByteChannelWrapper} itself is closed, but not the
* underlying channel that is returned by {@link WritableByteChannelWrapper#channel()}. Ensure
* that the underlying channel is cleaned up in {@link WritableByteChannelWrapper#close()},
* {@link ShuffleMapOutputWriter#commitAllPartitions()}, or
* {@link ShuffleMapOutputWriter#abort(Throwable)}.
*/
default WritableByteChannelWrapper openChannelWrapper() throws IOException {
return new DefaultWritableByteChannelWrapper(Channels.newChannel(openStream()));
default Optional<WritableByteChannelWrapper> openChannelWrapper() throws IOException {
return Optional.empty();
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import java.io.IOException;
import java.io.OutputStream;
import java.nio.channels.FileChannel;
import java.util.Optional;
import javax.annotation.Nullable;

import scala.None$;
Expand Down Expand Up @@ -205,45 +206,23 @@ private long[] writePartitionedData(ShuffleMapOutputWriter mapOutputWriter) thro
final File file = partitionWriterSegments[i].file();
ShufflePartitionWriter writer = mapOutputWriter.getPartitionWriter(i);
if (file.exists()) {
boolean copyThrewException = true;
if (transferToEnabled) {
FileInputStream in = new FileInputStream(file);
// Using WritableByteChannelWrapper to make resource closing consistent between
// this implementation and UnsafeShuffleWriter.
try {
WritableByteChannelWrapper outputChannel = writer.openChannelWrapper();
try (FileChannel inputChannel = in.getChannel()) {
Utils.copyFileStreamNIO(
inputChannel, outputChannel.channel(), 0L, inputChannel.size());
copyThrewException = false;
} finally {
Closeables.close(outputChannel, copyThrewException);
}
} finally {
Closeables.close(in, copyThrewException);
Optional<WritableByteChannelWrapper> maybeOutputChannel = writer.openChannelWrapper();
if (maybeOutputChannel.isPresent()) {
writePartitionedDataWithChannel(file, maybeOutputChannel.get());
} else {
writePartitionedDataWithStream(file, writer);
}
} else {
FileInputStream in = new FileInputStream(file);
OutputStream outputStream;
try {
outputStream = writer.openStream();
try {
Utils.copyStream(in, outputStream, false, false);
copyThrewException = false;
} finally {
Closeables.close(outputStream, copyThrewException);
}
} finally {
Closeables.close(in, copyThrewException);
}
writePartitionedDataWithStream(file, writer);
}
if (!file.delete()) {
logger.error("Unable to delete file for partition {}", i);
}
}
long numBytesWritten = writer.getNumBytesWritten();
lengths[i] = numBytesWritten;
writeMetrics.incBytesWritten(numBytesWritten);
lengths[i] = writer.getNumBytesWritten();
}
} finally {
writeMetrics.incWriteTime(System.nanoTime() - writeStartTime);
Expand All @@ -252,6 +231,41 @@ private long[] writePartitionedData(ShuffleMapOutputWriter mapOutputWriter) thro
return lengths;
}

private void writePartitionedDataWithChannel(
File file, WritableByteChannelWrapper outputChannel) throws IOException {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: one arg per line

boolean copyThrewException = true;
try {
FileInputStream in = new FileInputStream(file);
try (FileChannel inputChannel = in.getChannel()) {
Utils.copyFileStreamNIO(
inputChannel, outputChannel.channel(), 0L, inputChannel.size());
copyThrewException = false;
} finally {
Closeables.close(in, copyThrewException);
}
} finally {
Closeables.close(outputChannel, copyThrewException);
}
}

private void writePartitionedDataWithStream(File file, ShufflePartitionWriter writer)
throws IOException {
boolean copyThrewException = true;
FileInputStream in = new FileInputStream(file);
OutputStream outputStream;
try {
outputStream = writer.openStream();
try {
Utils.copyStream(in, outputStream, false, false);
copyThrewException = false;
} finally {
Closeables.close(outputStream, copyThrewException);
}
} finally {
Closeables.close(in, copyThrewException);
}
}

@Override
public Option<MapStatus> stop(boolean success) {
if (stopping) {
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import java.nio.channels.FileChannel;
import java.nio.channels.WritableByteChannel;

import java.util.Optional;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -164,7 +165,7 @@ public OutputStream openStream() throws IOException {
}

@Override
public WritableByteChannelWrapper openChannelWrapper() throws IOException {
public Optional<WritableByteChannelWrapper> openChannelWrapper() throws IOException {
if (partChannel == null) {
if (partStream != null) {
throw new IllegalStateException("Requested an output stream for a previous write but" +
Expand All @@ -174,7 +175,7 @@ public WritableByteChannelWrapper openChannelWrapper() throws IOException {
initChannel();
partChannel = new PartitionWriterChannel(partitionId);
}
return partChannel;
return Optional.of(partChannel);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.shuffle.sort.io

import java.io.{File, FileInputStream, FileOutputStream}
import java.io.{File, FileInputStream}
import java.nio.channels.FileChannel
import java.nio.file.Files
import java.util.Arrays
Expand All @@ -30,7 +30,6 @@ import org.mockito.MockitoAnnotations
import org.scalatest.BeforeAndAfterEach

import org.apache.spark.{SparkConf, SparkFunSuite}
import org.apache.spark.executor.ShuffleWriteMetrics
import org.apache.spark.shuffle.IndexShuffleBlockResolver
import org.apache.spark.util.Utils

Expand All @@ -39,9 +38,6 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA
@Mock(answer = RETURNS_SMART_NULLS)
private var blockResolver: IndexShuffleBlockResolver = _

@Mock(answer = RETURNS_SMART_NULLS)
private var shuffleWriteMetrics: ShuffleWriteMetrics = _

private val NUM_PARTITIONS = 4
private val data: Array[Array[Byte]] = (0 until NUM_PARTITIONS).map { p =>
if (p == 3) {
Expand Down Expand Up @@ -93,7 +89,6 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA
0,
0,
NUM_PARTITIONS,
shuffleWriteMetrics,
blockResolver,
conf)
}
Expand All @@ -116,13 +111,11 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA
(0 until NUM_PARTITIONS).foreach { p =>
val writer = mapOutputWriter.getPartitionWriter(p)
val outputTempFile = File.createTempFile("channelTemp", "", tempDir)
val outputTempFileStream = new FileOutputStream(outputTempFile)
outputTempFileStream.write(data(p))
outputTempFileStream.close()
Files.write(outputTempFile.toPath, data(p))
val tempFileInput = new FileInputStream(outputTempFile)
val channel = writer.openChannelWrapper()
Utils.tryWithResource(new FileInputStream(outputTempFile)) { tempFileInput =>
Utils.tryWithResource(writer.openChannelWrapper()) { channelWrapper =>
Utils.tryWithResource(writer.openChannelWrapper().get) { channelWrapper =>
assert(channelWrapper.channel().isInstanceOf[FileChannel],
"Underlying channel should be a file channel")
Utils.copyFileStreamNIO(
Expand Down