-
Notifications
You must be signed in to change notification settings - Fork 28.2k
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
[SPARK-19659] Fetch big blocks to disk when shuffle-read. #16989
Changes from all commits
4095cd2
63b3292
ac030fa
222680c
2ce2699
b07a3b6
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -23,6 +23,8 @@ | |
import java.util.concurrent.ConcurrentHashMap; | ||
import java.util.concurrent.atomic.AtomicLong; | ||
|
||
import scala.Tuple2; | ||
|
||
import com.google.common.base.Preconditions; | ||
import io.netty.channel.Channel; | ||
import org.slf4j.Logger; | ||
|
@@ -94,6 +96,25 @@ public ManagedBuffer getChunk(long streamId, int chunkIndex) { | |
return nextChunk; | ||
} | ||
|
||
@Override | ||
public ManagedBuffer openStream(String streamChunkId) { | ||
Tuple2<Long, Integer> streamIdAndChunkId = parseStreamChunkId(streamChunkId); | ||
return getChunk(streamIdAndChunkId._1, streamIdAndChunkId._2); | ||
} | ||
|
||
public static String genStreamChunkId(long streamId, int chunkId) { | ||
return String.format("%d_%d", streamId, chunkId); | ||
} | ||
|
||
public static Tuple2<Long, Integer> parseStreamChunkId(String streamChunkId) { | ||
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 method is called in a single place. Because it needs to return two values I see that you had to import the Scala tuple class in this Java code. You could avoid the need to do this by simply inlining this at its sole call site, which I think would make the code simpler to read. 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. Idea was to ensure that string parsing and unparsing it localized to a single place (genStreamChunkId and parseStreamChunkId) - so that we can modify it in future if required without needing to look for where all we did a split by "_" 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. we can inline it if it becomes a bottleneck in the future. |
||
String[] array = streamChunkId.split("_"); | ||
assert array.length == 2: | ||
"Stream id and chunk index should be specified when open stream for fetching block."; | ||
long streamId = Long.valueOf(array[0]); | ||
int chunkIndex = Integer.valueOf(array[1]); | ||
return new Tuple2<>(streamId, chunkIndex); | ||
} | ||
|
||
@Override | ||
public void connectionTerminated(Channel channel) { | ||
// Close all streams which have been associated with the channel. | ||
|
Original file line number | Diff line number | Diff line change | ||
---|---|---|---|---|
|
@@ -17,19 +17,28 @@ | |||
|
||||
package org.apache.spark.network.shuffle; | ||||
|
||||
import java.io.File; | ||||
import java.io.FileOutputStream; | ||||
import java.io.IOException; | ||||
import java.nio.ByteBuffer; | ||||
import java.nio.channels.Channels; | ||||
import java.nio.channels.WritableByteChannel; | ||||
import java.util.Arrays; | ||||
|
||||
import org.slf4j.Logger; | ||||
import org.slf4j.LoggerFactory; | ||||
|
||||
import org.apache.spark.network.buffer.FileSegmentManagedBuffer; | ||||
import org.apache.spark.network.buffer.ManagedBuffer; | ||||
import org.apache.spark.network.client.ChunkReceivedCallback; | ||||
import org.apache.spark.network.client.RpcResponseCallback; | ||||
import org.apache.spark.network.client.StreamCallback; | ||||
import org.apache.spark.network.client.TransportClient; | ||||
import org.apache.spark.network.server.OneForOneStreamManager; | ||||
import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; | ||||
import org.apache.spark.network.shuffle.protocol.OpenBlocks; | ||||
import org.apache.spark.network.shuffle.protocol.StreamHandle; | ||||
import org.apache.spark.network.util.TransportConf; | ||||
|
||||
/** | ||||
* Simple wrapper on top of a TransportClient which interprets each chunk as a whole block, and | ||||
|
@@ -48,6 +57,8 @@ public class OneForOneBlockFetcher { | |||
private final String[] blockIds; | ||||
private final BlockFetchingListener listener; | ||||
private final ChunkReceivedCallback chunkCallback; | ||||
private TransportConf transportConf = null; | ||||
private File[] shuffleFiles = null; | ||||
|
||||
private StreamHandle streamHandle = null; | ||||
|
||||
|
@@ -56,12 +67,20 @@ public OneForOneBlockFetcher( | |||
String appId, | ||||
String execId, | ||||
String[] blockIds, | ||||
BlockFetchingListener listener) { | ||||
BlockFetchingListener listener, | ||||
TransportConf transportConf, | ||||
File[] shuffleFiles) { | ||||
this.client = client; | ||||
this.openMessage = new OpenBlocks(appId, execId, blockIds); | ||||
this.blockIds = blockIds; | ||||
this.listener = listener; | ||||
this.chunkCallback = new ChunkCallback(); | ||||
this.transportConf = transportConf; | ||||
if (shuffleFiles != null) { | ||||
this.shuffleFiles = shuffleFiles; | ||||
assert this.shuffleFiles.length == blockIds.length: | ||||
"Number of shuffle files should equal to blocks"; | ||||
} | ||||
} | ||||
|
||||
/** Callback invoked on receipt of each chunk. We equate a single chunk to a single block. */ | ||||
|
@@ -100,7 +119,12 @@ public void onSuccess(ByteBuffer response) { | |||
// Immediately request all chunks -- we expect that the total size of the request is | ||||
// reasonable due to higher level chunking in [[ShuffleBlockFetcherIterator]]. | ||||
for (int i = 0; i < streamHandle.numChunks; i++) { | ||||
client.fetchChunk(streamHandle.streamId, i, chunkCallback); | ||||
if (shuffleFiles != null) { | ||||
client.stream(OneForOneStreamManager.genStreamChunkId(streamHandle.streamId, i), | ||||
new DownloadCallback(shuffleFiles[i], i)); | ||||
} else { | ||||
client.fetchChunk(streamHandle.streamId, i, chunkCallback); | ||||
} | ||||
} | ||||
} catch (Exception e) { | ||||
logger.error("Failed while starting block fetches after success", e); | ||||
|
@@ -126,4 +150,38 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) { | |||
} | ||||
} | ||||
} | ||||
|
||||
private class DownloadCallback implements StreamCallback { | ||||
|
||||
private WritableByteChannel channel = null; | ||||
private File targetFile = null; | ||||
private int chunkIndex; | ||||
|
||||
public DownloadCallback(File targetFile, int chunkIndex) throws IOException { | ||||
this.targetFile = targetFile; | ||||
this.channel = Channels.newChannel(new FileOutputStream(targetFile)); | ||||
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. Does this work with 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. One possible fix is writing to a temp file and renaming it to the target file. 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. 👍 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. @zsxwing @cloud-fan 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. @jinxing64 The retry logic is here: Line 215 in 88a536b
The issue is there will be two 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. Pardon, I could hardly believe there are two
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. @zsxwing |
||||
this.chunkIndex = chunkIndex; | ||||
} | ||||
|
||||
@Override | ||||
public void onData(String streamId, ByteBuffer buf) throws IOException { | ||||
channel.write(buf); | ||||
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. As an impl detail (since channel is a FOS), this will work - but in general, channel.write() need not write buf.remain(); which actually breaks spark code iirc - since it expects odData to completely consume the data. 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. Sorry, @mridulm , I'm still not quite sure how to refine this. Could you please give more details? 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. I was just noting it for review :-) in case someone else goes through the same concern. 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. I am super-late on reviewing this, apologies, just asking questions for my own understanding, and to consider possible future improvements -- this won't do a zero-copy transfer, will it? That ByteBuffer is still in user space? From my understanding, we'd need to do special handling to use netty's but I'm still working on putting all the pieces together here and admittedly this is out of my area of expertise 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. @squito This is a Java Channel. Not sure how to call By the way, I think this is a zero-copy transfer since the underlying buffer is an off heap buffer. Anyway, I found a bug here... 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. right, I realize there isn't a simple one-line change here to switch to using spliceTo, I was wondering what the behavior is. I actually thought zero-copy and offheap were orthogonal -- anytime netty gives you direct access to bytes, it has to be copied to user space, right? 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. @squito You are right. It needs a copy between user space and kernel space. |
||||
} | ||||
|
||||
@Override | ||||
public void onComplete(String streamId) throws IOException { | ||||
channel.close(); | ||||
ManagedBuffer buffer = new FileSegmentManagedBuffer(transportConf, targetFile, 0, | ||||
targetFile.length()); | ||||
listener.onBlockFetchSuccess(blockIds[chunkIndex], buffer); | ||||
} | ||||
|
||||
@Override | ||||
public void onFailure(String streamId, Throwable cause) throws IOException { | ||||
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. shall we remove the partial written file when failing? 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. Yes, that will be good ! |
||||
channel.close(); | ||||
// On receipt of a failure, fail every block from chunkIndex onwards. | ||||
String[] remainingBlockIds = Arrays.copyOfRange(blockIds, chunkIndex, blockIds.length); | ||||
failRemainingBlocks(remainingBlockIds, cause); | ||||
} | ||||
} | ||||
} |
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.
@jinxing64 this breaks old shuffle service. We should avoid change server side codes. Right now I just disabled this feature in #18467
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.
Thanks, I will try make a pr as soon as possible.