-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-21175] Reject OpenBlocks when memory shortage on shuffle service. #18388
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 4 commits
3cc29a7
70aaf9a
7dd2cec
ef89321
98123ee
4de417f
3a018b1
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 |
|---|---|---|
|
|
@@ -25,6 +25,8 @@ | |
|
|
||
| import com.google.common.base.Preconditions; | ||
| import io.netty.channel.Channel; | ||
| import org.apache.commons.lang3.tuple.ImmutablePair; | ||
| import org.apache.commons.lang3.tuple.Pair; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
|
|
@@ -53,9 +55,13 @@ private static class StreamState { | |
| // that the caller only requests each chunk one at a time, in order. | ||
| int curChunk = 0; | ||
|
|
||
| // Used to keep track of the number of chunks being transferred and not finished yet. | ||
| AtomicLong chunksBeingTransferred; | ||
|
|
||
| StreamState(String appId, Iterator<ManagedBuffer> buffers) { | ||
| this.appId = appId; | ||
| this.buffers = Preconditions.checkNotNull(buffers); | ||
| this.chunksBeingTransferred = new AtomicLong(0L); | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -84,6 +90,7 @@ public ManagedBuffer getChunk(long streamId, int chunkIndex) { | |
| "Requested chunk index beyond end %s", chunkIndex)); | ||
| } | ||
| state.curChunk += 1; | ||
| state.chunksBeingTransferred.incrementAndGet(); | ||
| ManagedBuffer nextChunk = state.buffers.next(); | ||
|
|
||
| if (!state.buffers.hasNext()) { | ||
|
|
@@ -96,18 +103,23 @@ public ManagedBuffer getChunk(long streamId, int chunkIndex) { | |
|
|
||
| @Override | ||
| public ManagedBuffer openStream(String streamChunkId) { | ||
| 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 getChunk(streamId, chunkIndex); | ||
| Pair<Long, Integer> streamChunkIdPair = parseStreamChunkId(streamChunkId); | ||
| return getChunk(streamChunkIdPair.getLeft(), streamChunkIdPair.getRight()); | ||
| } | ||
|
|
||
| public static String genStreamChunkId(long streamId, int chunkId) { | ||
| return String.format("%d_%d", streamId, chunkId); | ||
| } | ||
|
|
||
| public static Pair<Long, Integer> parseStreamChunkId(String streamChunkId) { | ||
|
Member
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. nit: please document the meaning of the return value for this public method. |
||
| String[] array = streamChunkId.split("_"); | ||
| assert array.length == 2: | ||
| "Stream id and chunk index should be specified."; | ||
| long streamId = Long.valueOf(array[0]); | ||
| int chunkIndex = Integer.valueOf(array[1]); | ||
| return ImmutablePair.of(streamId, chunkIndex); | ||
| } | ||
|
|
||
| @Override | ||
| public void connectionTerminated(Channel channel) { | ||
| // Close all streams which have been associated with the channel. | ||
|
|
@@ -122,6 +134,7 @@ public void connectionTerminated(Channel channel) { | |
| } | ||
| } | ||
| } | ||
|
|
||
|
||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -139,6 +152,23 @@ public void checkAuthorization(TransportClient client, long streamId) { | |
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void chunkSent(long streamId) { | ||
| StreamState streamState = streams.get(streamId); | ||
| if (streamState != null) { | ||
| streamState.chunksBeingTransferred.decrementAndGet(); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public long chunksBeingTransferred() { | ||
| long sum = 0L; | ||
| for (StreamState streamState: streams.values()) { | ||
| sum += streamState.chunksBeingTransferred.get(); | ||
| } | ||
| return sum; | ||
| } | ||
|
|
||
| /** | ||
| * Registers a stream of ManagedBuffers which are served as individual chunks one at a time to | ||
| * callers. Each ManagedBuffer will be release()'d after it is transferred on the wire. If a | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -83,4 +83,16 @@ public void connectionTerminated(Channel channel) { } | |
| */ | ||
| public void checkAuthorization(TransportClient client, long streamId) { } | ||
|
|
||
| /** | ||
| * Return the number of chunks being transferred and not finished yet in this StreamManager. | ||
| */ | ||
| public long chunksBeingTransferred() { | ||
| return 0; | ||
| } | ||
|
|
||
| /** | ||
| * Called when a chunk is successfully sent. | ||
| */ | ||
| public void chunkSent(long streamId) { } | ||
|
Contributor
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 think it makes more sense to have 2 methods:
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. Yes, it would be much better |
||
|
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,6 +22,7 @@ | |
|
|
||
| import com.google.common.base.Throwables; | ||
| import io.netty.channel.Channel; | ||
| import io.netty.channel.ChannelFuture; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
|
|
@@ -65,14 +66,19 @@ public class TransportRequestHandler extends MessageHandler<RequestMessage> { | |
| /** Returns each chunk part of a stream. */ | ||
| private final StreamManager streamManager; | ||
|
|
||
| /** The max number of chunks being transferred and not finished yet. */ | ||
| private final long maxChunksBeingTransferred; | ||
|
|
||
| public TransportRequestHandler( | ||
| Channel channel, | ||
| TransportClient reverseClient, | ||
| RpcHandler rpcHandler) { | ||
| RpcHandler rpcHandler, | ||
| Long maxChunksBeingTransferred) { | ||
| this.channel = channel; | ||
| this.reverseClient = reverseClient; | ||
| this.rpcHandler = rpcHandler; | ||
| this.streamManager = rpcHandler.getStreamManager(); | ||
| this.maxChunksBeingTransferred = maxChunksBeingTransferred; | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -118,6 +124,13 @@ private void processFetchRequest(final ChunkFetchRequest req) { | |
| req.streamChunkId); | ||
| } | ||
|
|
||
| long chunksBeingTransferred = streamManager.chunksBeingTransferred(); | ||
| if (chunksBeingTransferred > maxChunksBeingTransferred) { | ||
| logger.warn("The number of chunks being transferred {} is above {}, close the connection.", | ||
| chunksBeingTransferred, maxChunksBeingTransferred); | ||
| channel.close(); | ||
| } | ||
|
Member
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. missing |
||
|
|
||
| ManagedBuffer buf; | ||
| try { | ||
| streamManager.checkAuthorization(reverseClient, req.streamChunkId.streamId); | ||
|
|
@@ -130,11 +143,25 @@ private void processFetchRequest(final ChunkFetchRequest req) { | |
| return; | ||
| } | ||
|
|
||
| respond(new ChunkFetchSuccess(req.streamChunkId, buf)); | ||
| respond(new ChunkFetchSuccess(req.streamChunkId, buf)).addListener(future -> { | ||
| streamManager.chunkSent(req.streamChunkId.streamId); | ||
| }); | ||
| } | ||
|
|
||
| private void processStreamRequest(final StreamRequest req) { | ||
| if (logger.isTraceEnabled()) { | ||
| logger.trace("Received req from {} to fetch stream {}", getRemoteAddress(channel), | ||
| req.streamId); | ||
| } | ||
|
|
||
| long chunksBeingTransferred = streamManager.chunksBeingTransferred(); | ||
| if (chunksBeingTransferred > maxChunksBeingTransferred) { | ||
| logger.warn("The number of chunks being transferred {} is above {}, close the connection.", | ||
| chunksBeingTransferred, maxChunksBeingTransferred); | ||
| channel.close(); | ||
| } | ||
|
Member
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. missing
Member
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. Also please decrease
Member
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. To make the error handling simple, you can increase chunksBeingTransferred just before writing the chunk to the channel, and decrease it in the future returned by write. |
||
| ManagedBuffer buf; | ||
|
|
||
|
||
| try { | ||
| buf = streamManager.openStream(req.streamId); | ||
| } catch (Exception e) { | ||
|
|
@@ -145,7 +172,12 @@ private void processStreamRequest(final StreamRequest req) { | |
| } | ||
|
|
||
| if (buf != null) { | ||
| respond(new StreamResponse(req.streamId, buf.size(), buf)); | ||
| respond(new StreamResponse(req.streamId, buf.size(), buf)).addListener(future -> { | ||
| if (streamManager instanceof OneForOneStreamManager) { | ||
|
||
| streamManager.chunkSent(OneForOneStreamManager.parseStreamChunkId(req.streamId) | ||
| .getLeft()); | ||
| } | ||
| }); | ||
| } else { | ||
| respond(new StreamFailure(req.streamId, String.format( | ||
| "Stream '%s' was not found.", req.streamId))); | ||
|
|
@@ -187,9 +219,9 @@ private void processOneWayMessage(OneWayMessage req) { | |
| * Responds to a single message with some Encodable object. If a failure occurs while sending, | ||
| * it will be logged and the channel closed. | ||
| */ | ||
| private void respond(Encodable result) { | ||
| private ChannelFuture respond(Encodable result) { | ||
| SocketAddress remoteAddress = channel.remoteAddress(); | ||
| channel.writeAndFlush(result).addListener(future -> { | ||
| return channel.writeAndFlush(result).addListener(future -> { | ||
| if (future.isSuccess()) { | ||
| logger.trace("Sent result {} to client {}", result, remoteAddress); | ||
| } else { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -257,4 +257,11 @@ public Properties cryptoConf() { | |
| return CryptoUtils.toCryptoConf("spark.network.crypto.config.", conf.getAll()); | ||
| } | ||
|
|
||
| /** | ||
| * The max number of chunks being transferred at the same time. This config helps avoid OOM on | ||
| * shuffle server. | ||
| */ | ||
| public long maxChunksBeingTransferred() { | ||
| return conf.getLong("spark.network.shuffle.maxChunksBeingTransferred", Long.MAX_VALUE); | ||
|
||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -1808,6 +1808,14 @@ Apart from these, the following properties are also available, and may be useful | |
| Disable unencrypted connections for services that support SASL authentication. | ||
| </td> | ||
| </tr> | ||
| <tr> | ||
| <td><code>spark.network.shuffle.maxChunksBeingTransferred</code></td> | ||
| <td>Long.MAX_VALUE</td> | ||
| <td> | ||
| The max number of chunks being transferred at the same time. This config helps avoid OOM on | ||
|
||
| shuffle server. | ||
| </td> | ||
| </tr> | ||
| <tr> | ||
| <td><code>spark.core.connection.ack.wait.timeout</code></td> | ||
| <td><code>spark.network.timeout</code></td> | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.
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.
according to the usage, seems
volatile long chunksBeingTransferredis enough?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.
Really? we do
chunksBeingTransferred++andchunksBeingTransferred--? I think it's not safe.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.
you are right
Uh oh!
There was an error while loading. Please reload this page.
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
chunksBeingTransferredis modified in the same thread. Not a big deal though.