-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-24355] Spark external shuffle server improvement to better handle block fetch requests. #22173
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
Conversation
|
@tgravescs @vanzin @Victsm please review thanks |
|
ok to test |
|
|
Test build #95042 has finished for PR 22173 at commit
|
| .addLast("handler", channelHandler); | ||
| .addLast("handler", channelHandler) | ||
| // Use a separate EventLoopGroup to handle ChunkFetchRequest messages. | ||
| .addLast(chunkFetchWorkers, "chunkFetchHandler", chunkFetchHandler); |
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.
Hmm... I think there is some waste here. Not all channels actually need the chunk fetch handler. Basically only the shuffle server (external or not) does. So for all other cases - RpcEnv server and clients, shuffle clients - you'd have this new thread pool just sitting there.
It would be good to avoid that.
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.
yes i did notice that... makes sense
|
Test build #95289 has finished for PR 22173 at commit
|
|
Test build #95293 has finished for PR 22173 at commit
|
complete expanion of imports rearrange imports
|
Test build #95299 has finished for PR 22173 at commit
|
|
Test build #95300 has finished for PR 22173 at commit
|
|
Test build #95352 has finished for PR 22173 at commit
|
|
Test build #95423 has finished for PR 22173 at commit
|
|
Test build #95434 has finished for PR 22173 at commit
|
|
retest this please |
|
(I haven't forgotten about this, just haven't had the time to look at it.) |
|
Test build #95499 has finished for PR 22173 at commit
|
|
thanks @vanzin, also @tgravescs gentle ping... |
tgravescs
left a comment
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 assume we have the same issue here with processStreamRequest since its going to hold a thread. Same goes I think for the uploadStream request. We could move that to be under the same threadpool as chunkedfetch request. Or we could do those as separate jira as well since the majority of requests will be the chunkedfetch requests.
| } | ||
| int chunkFetchHandlerThreadsPercent = | ||
| conf.getInt("spark.shuffle.server.chunkFetchHandlerThreadsPercent", 0); | ||
| return this.serverThreads() > 0? (this.serverThreads() * chunkFetchHandlerThreadsPercent)/100: |
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.
space between 0 and ?, and space between 100 and :
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 assume we aren't documenting chunkFetchHandlerThreadsPercent since the serverthreads config isn't documented
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 it is a good idea to document both as this is an important config. Let me know your thoughts
| * which equals 0.1 * 2*#cores or 0.1 * io.serverThreads. | ||
| */ | ||
| public int chunkFetchHandlerThreads() { | ||
| if(!this.getModuleName().equalsIgnoreCase("shuffle")) { |
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.
space after if before (
| * higher number of shuffler server threads, we are able to reserve some threads for | ||
| * handling other RPC messages, thus making the Client less likely to experience timeout | ||
| * when sending RPC messages to the shuffle server. Default to 0, which is 2*#cores | ||
| * or io.serverThreads. 10 would mean 10% of 2*#cores or 10% of io.serverThreads |
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 realize these are just examples but normally I would expect a user to have the threads processing chunk request much more then those doing other things so perhaps the example should be like 90%.
Were any tests done to see how many threads were needed for the others, I would expect very few?
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.
No based on how many threads required for other rpc calls, i have not tested them, but the whole point would be to reduce the dependency how much time the chunkFetchedRequests will be spending doing disk I/O
| int chunkFetchHandlerThreadsPercent = | ||
| conf.getInt("spark.shuffle.server.chunkFetchHandlerThreadsPercent", 0); | ||
| return this.serverThreads() > 0? (this.serverThreads() * chunkFetchHandlerThreadsPercent)/100: | ||
| (2* NettyRuntime.availableProcessors() * chunkFetchHandlerThreadsPercent)/100; |
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.
space between 2 and *
| import io.netty.util.concurrent.Future; | ||
| import io.netty.util.concurrent.GenericFutureListener; | ||
|
|
||
|
|
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.
remove extra line.
| private ChannelFuture respond(final Channel channel, | ||
| final Encodable result) throws InterruptedException { | ||
| final SocketAddress remoteAddress = channel.remoteAddress(); | ||
| return channel.writeAndFlush(result).sync().addListener((ChannelFutureListener) future -> { |
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.
do we actually want to use await() here instead of sync(). Sync says "Waits for this future until it is done, and rethrows the cause of the failure if this future failed."
I'm not sure we want a rethrow here since we have the addListener to handle the future failure.
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.
Yes await can be used as well... i will test it out and let you know its ramifications if any, thanks
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.
ok i figured the following... i think it better to rethrow a.k.a. use sync() instead of quitely logging the exception via await(). The reason being as follows....
The respond call made here https://github.com/apache/spark/pull/22173/files#diff-a37b07d454be4d6cd26edb294661d4e3R106 waits for sending the failed or success response back. Hence, it is a blocking call. If an exception is thrown using await() we log quitely and do no rethrow, the underlying rpc handler will not know about the request status here https://github.com/apache/spark/blob/master/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java#L183, rethrowing the exception allows it to handle and throw an rpc failure here https://github.com/apache/spark/blob/master/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java#L194 else we have to wait for a timeout exception instead of an Interrupt exception. Either cases this seems fine but using sync() we will fail fast...
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.
so it doesn't go through https://github.com/apache/spark/blob/master/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java#L183 since that is hte rpchandler . The TransportChannelHandler won't handle it at all, it will go to the pipeline which calls the ChunkFetchRequestHandler. So whatever is calling that is what would get the exception propogated. The reason I said this is because previously when this was handled in TransportChannelHandler, it wasn't throwing the exception up. The TrasnportRequestHandler.respond is calling hte writeAndFlush async and then just adding a listener which would simply log an error if the future wasn't successful.
I want to make sure if an exception is thrown here it doesn't kill the entire external shuffle service for instance.
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.
ok i had chunkFetchHandler as an instance of rpcHandler in my mind...
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.
It justs logs the exception but the behaviour is same… as in the first case an additional warning from netty side is passed onto the listener and the channel is closed… in the later we quitely log error and the channel is closed… the executors deal with the closed channel and retry or schedule the task on different executor… and eventually the job seems to succeed… in the example i ran… i expect the retry to go through in such scenarios… i will just use await() as the ERROR is logged and we dont have to be more verbose with the warning from the netty side...
|
Test build #95876 has finished for PR 22173 at commit
|
|
test this please |
|
Test build #96140 has finished for PR 22173 at commit
|
| this.rpcHandler = rpcHandler; | ||
| this.closeIdleConnections = closeIdleConnections; | ||
|
|
||
| synchronized(this.getClass()) { |
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 synchronized(this.getClass()) is not recommended due to it not handling inheritance and such. Use synchronized(TransportContext.class)
|
|
||
| synchronized(this.getClass()) { | ||
| if (chunkFetchWorkers == null && conf.getModuleName() != null && | ||
| conf.getModuleName().equalsIgnoreCase("shuffle")) { |
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.
fix spacing here, line up conf.getModuleName with the chunkFetchWorkers
| chunkFetchWorkers = NettyUtils.createEventLoop( | ||
| IOMode.valueOf(conf.ioMode()), | ||
| conf.chunkFetchHandlerThreads(), | ||
| "chunk-fetch-handler"); |
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.
for consistency perhaps name thread shuffle-chunk-fetch-handler
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.
like you mention if we can not create the event loop when on the client side that would be best
| TransportChannelHandler channelHandler = createChannelHandler(channel, channelRpcHandler); | ||
| channel.pipeline() | ||
| ChunkFetchRequestHandler chunkFetchHandler = | ||
| createChunkFetchHandler(channelHandler, channelRpcHandler); |
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.
fix spacing, only indented 2 spaces
| @Override | ||
| public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { | ||
| logger.warn("Exception in connection from " + getRemoteAddress(ctx.channel()), | ||
| 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.
spacing 2, fix throughout the file
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.
this can actually be unwrapped here
| logger.error(String.format("Error opening block %s for request from %s", | ||
| msg.streamChunkId, getRemoteAddress(channel)), e); | ||
| respond(channel, | ||
| new ChunkFetchFailure(msg.streamChunkId, |
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.
fix wrapping and sapcing
| .addLast("decoder", DECODER) | ||
| .addLast("idleStateHandler", new IdleStateHandler(0, 0, conf.connectionTimeoutMs() / 1000)) | ||
| .addLast("idleStateHandler", | ||
| new IdleStateHandler(0, 0, conf.connectionTimeoutMs() / 1000)) |
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.
fix indentation
| import org.apache.spark.network.buffer.NioManagedBuffer; | ||
| import org.apache.spark.network.client.*; | ||
| import org.apache.spark.network.protocol.*; | ||
| import org.apache.spark.network.client.RpcResponseCallback; |
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.
revert the imports to be .*
| private ChannelFuture respond(final Channel channel, | ||
| final Encodable result) throws InterruptedException { | ||
| final SocketAddress remoteAddress = channel.remoteAddress(); | ||
| return channel.writeAndFlush(result).sync().addListener((ChannelFutureListener) future -> { |
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.
so it doesn't go through https://github.com/apache/spark/blob/master/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java#L183 since that is hte rpchandler . The TransportChannelHandler won't handle it at all, it will go to the pipeline which calls the ChunkFetchRequestHandler. So whatever is calling that is what would get the exception propogated. The reason I said this is because previously when this was handled in TransportChannelHandler, it wasn't throwing the exception up. The TrasnportRequestHandler.respond is calling hte writeAndFlush async and then just adding a listener which would simply log an error if the future wasn't successful.
I want to make sure if an exception is thrown here it doesn't kill the entire external shuffle service for instance.
| return 0; | ||
| } | ||
| int chunkFetchHandlerThreadsPercent = | ||
| conf.getInt("spark.shuffle.server.chunkFetchHandlerThreadsPercent", 0); |
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.
fix spacing
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.
Yes it is documented above... if it is 0 or 100 it is 2*#cores or io.serverThreads
|
Test build #96307 has finished for PR 22173 at commit
|
tgravescs
left a comment
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.
+1. looks good.
I'm not super fond of the isClientOnly parameter to TransportContext (which I recommended instead of other ways) but I can't think of with a more elegant solution at this point.
|
merged into master (2.5.0) |
|
closes #21402 |
| // Separate thread pool for handling ChunkFetchRequest. This helps to enable throttling | ||
| // max number of TransportServer worker threads that are blocked on writing response | ||
| // of ChunkFetchRequest message back to the client via the underlying channel. | ||
| private static EventLoopGroup chunkFetchWorkers; |
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.
Is there any special reason that this must be a global one? I have not yet looked the details. But looks like this may cause ChunkFetchIntegrationSuite flaky as there is no isolation between tests.
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 haven't been able to reproduce this but the number of threads used for this tests are 2* number of cores or spark.shuffle.io.serverThreads.
…dle block fetch requests. ## What changes were proposed in this pull request? Description: Right now, the default server side netty handler threads is 2 * # cores, and can be further configured with parameter spark.shuffle.io.serverThreads. In order to process a client request, it would require one available server netty handler thread. However, when the server netty handler threads start to process ChunkFetchRequests, they will be blocked on disk I/O, mostly due to disk contentions from the random read operations initiated by all the ChunkFetchRequests received from clients. As a result, when the shuffle server is serving many concurrent ChunkFetchRequests, the server side netty handler threads could all be blocked on reading shuffle files, thus leaving no handler thread available to process other types of requests which should all be very quick to process. This issue could potentially be fixed by limiting the number of netty handler threads that could get blocked when processing ChunkFetchRequest. We have a patch to do this by using a separate EventLoopGroup with a dedicated ChannelHandler to process ChunkFetchRequest. This enables shuffle server to reserve netty handler threads for non-ChunkFetchRequest, thus enabling consistent processing time for these requests which are fast to process. After deploying the patch in our infrastructure, we no longer see timeout issues with either executor registration with local shuffle server or shuffle client establishing connection with remote shuffle server. (Please fill in changes proposed in this fix) For Original PR please refer here apache#21402 ## How was this patch tested? Unit tests and stress testing. (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review http://spark.apache.org/contributing.html before opening a pull request. Closes apache#22173 from redsanket/SPARK-24335. Authored-by: Sanket Chintapalli <[email protected]> Signed-off-by: Thomas Graves <[email protected]>
…dle block fetch requests. (#89) ## What changes were proposed in this pull request? Description: Right now, the default server side netty handler threads is 2 * # cores, and can be further configured with parameter spark.shuffle.io.serverThreads. In order to process a client request, it would require one available server netty handler thread. However, when the server netty handler threads start to process ChunkFetchRequests, they will be blocked on disk I/O, mostly due to disk contentions from the random read operations initiated by all the ChunkFetchRequests received from clients. As a result, when the shuffle server is serving many concurrent ChunkFetchRequests, the server side netty handler threads could all be blocked on reading shuffle files, thus leaving no handler thread available to process other types of requests which should all be very quick to process. This issue could potentially be fixed by limiting the number of netty handler threads that could get blocked when processing ChunkFetchRequest. We have a patch to do this by using a separate EventLoopGroup with a dedicated ChannelHandler to process ChunkFetchRequest. This enables shuffle server to reserve netty handler threads for non-ChunkFetchRequest, thus enabling consistent processing time for these requests which are fast to process. After deploying the patch in our infrastructure, we no longer see timeout issues with either executor registration with local shuffle server or shuffle client establishing connection with remote shuffle server. (Please fill in changes proposed in this fix) For Original PR please refer here apache#21402 ## How was this patch tested? Unit tests and stress testing. (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review http://spark.apache.org/contributing.html before opening a pull request. Closes apache#22173 from redsanket/SPARK-24335. Authored-by: Sanket Chintapalli <[email protected]> Signed-off-by: Thomas Graves <[email protected]>
…dle block fetch requests. Description: Right now, the default server side netty handler threads is 2 * # cores, and can be further configured with parameter spark.shuffle.io.serverThreads. In order to process a client request, it would require one available server netty handler thread. However, when the server netty handler threads start to process ChunkFetchRequests, they will be blocked on disk I/O, mostly due to disk contentions from the random read operations initiated by all the ChunkFetchRequests received from clients. As a result, when the shuffle server is serving many concurrent ChunkFetchRequests, the server side netty handler threads could all be blocked on reading shuffle files, thus leaving no handler thread available to process other types of requests which should all be very quick to process. This issue could potentially be fixed by limiting the number of netty handler threads that could get blocked when processing ChunkFetchRequest. We have a patch to do this by using a separate EventLoopGroup with a dedicated ChannelHandler to process ChunkFetchRequest. This enables shuffle server to reserve netty handler threads for non-ChunkFetchRequest, thus enabling consistent processing time for these requests which are fast to process. After deploying the patch in our infrastructure, we no longer see timeout issues with either executor registration with local shuffle server or shuffle client establishing connection with remote shuffle server. (Please fill in changes proposed in this fix) For Original PR please refer here apache#21402 Unit tests and stress testing. (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review http://spark.apache.org/contributing.html before opening a pull request. Closes apache#22173 from redsanket/SPARK-24335. Authored-by: Sanket Chintapalli <[email protected]> Signed-off-by: Thomas Graves <[email protected]>
…dle block fetch requests. Description: Right now, the default server side netty handler threads is 2 * # cores, and can be further configured with parameter spark.shuffle.io.serverThreads. In order to process a client request, it would require one available server netty handler thread. However, when the server netty handler threads start to process ChunkFetchRequests, they will be blocked on disk I/O, mostly due to disk contentions from the random read operations initiated by all the ChunkFetchRequests received from clients. As a result, when the shuffle server is serving many concurrent ChunkFetchRequests, the server side netty handler threads could all be blocked on reading shuffle files, thus leaving no handler thread available to process other types of requests which should all be very quick to process. This issue could potentially be fixed by limiting the number of netty handler threads that could get blocked when processing ChunkFetchRequest. We have a patch to do this by using a separate EventLoopGroup with a dedicated ChannelHandler to process ChunkFetchRequest. This enables shuffle server to reserve netty handler threads for non-ChunkFetchRequest, thus enabling consistent processing time for these requests which are fast to process. After deploying the patch in our infrastructure, we no longer see timeout issues with either executor registration with local shuffle server or shuffle client establishing connection with remote shuffle server. (Please fill in changes proposed in this fix) For Original PR please refer here apache#21402 Unit tests and stress testing. (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review http://spark.apache.org/contributing.html before opening a pull request. Closes apache#22173 from redsanket/SPARK-24335. Authored-by: Sanket Chintapalli <[email protected]> Signed-off-by: Thomas Graves <[email protected]> (cherry picked from commit ff601cf)
|
We hit significant performance regression in our internal workload caused by this commit. After this commit, the executor can handle at most N chunk fetch requests at the same time, where N is the value of This commit does fix a nasty problem, and I'm fine with it even if it may introduce perf regression, but there should be a way to turn it off. Unfortunately, we can't turn off this feature. We can set Looking back at this problem, I think we can either create a dedicated channel for non chunk fetch request, or ask netty to handle channel write of non chunk fetch request first. Both seem hard to implement. Shall we revert it first, and think of a good fix later? |
|
Can you clarify? The default for spark.shuffle.server.chunkFetchHandlerThreadsPercent is 0 which should be the same number of chunk fetcher threads as previously. It wasn't previously unlimited as Netty would limit to 2*number of cores by default. (https://github.com/netty/netty/blob/9621a5b98120f9596b5d2a337330339dda199bde/transport/src/main/java/io/netty/channel/MultithreadEventLoopGroup.java#L40) Were you configuring Netty or something to make it unlimited? Or perhaps the default for Netty changed or we missed something? The intention was no perf regression and same default as previous behavior. Note there was a follow on to this pr to fix the default calculation properly. Please explain more what you are seeing and what settings you are using. |
|
It's good to know that the underlying channel write thread pool has the same concurrency with the request handling thread pool. So there are 2 thread pools: one to handle the requests, one to write data to the channel. Previously, fetch requests were handled by the request handling thread pool, and return immediately after reading shuffle blocks. Now, fetch requests are handled by a new fetch request thread pool, and return until channel write is completed. It's kind of handle fetch requests in sync mode, while previously it's more likely to keep both thread pools busy with reading shuffle blocks/writing data to channel. Unfortunately I can't share our internal workload (we don't have special settings), I'll try to write a microbenchmark. |
|
sorry I'm not understanding quite what you are saying and I think perhaps you mean fetch requests in "async" mode? |
|
When I say sync mode, I mean a thread that handles fetch request has to finish reading the shuffle blocks and writing the response to channel, before handling the next requests, although the channel writing is done by another thread pool. Previously it's fully async: the thread can handle the next request once it finishes reading the shuffle blocks of the current request. That said, the throughput of handling fetch requests is reduced now. Anyway let me come up with a microbenchmark first. |
|
the jobs we ran didn't see an overall performance impact and it greatly helped when you ran on busy multi-tenant cluster. How about if the value of spark.shuffle.server.chunkFetchHandlerThreadsPercent isn't explicitly set by the user we just don't add the extra event loop or thread pool, see TransportContext - just like it does it its not shuffle, that should be a fairly easy change. |
|
Sounds good to me. We can explore different solutions after 3.0. |
|
I think This adds a The code in Here it records how much time it took to perform the IO operations, that is, execute
This ensures that the default server-IO thread always gets time to process the ready channels. Its not always busy processing |
|
When we worked on the original fix in #21402, we were holding a wrong view of how Netty handles both event loop groups. We have an internal stress testing framework that can help to validate if the original timeout issue still occurs after removing |
|
@Victsm @tgravescs Since there is a delay in reading the channel, I suspect this is because the hardcoding in netty code I can test this theory by lowering this number in a fork of netty and building spark against it. However, for now we can't remove Note: This test was with a dedicated boss event loop group which is why we don't see any delay in channel registration. |
|
@cloud-fan |
|
SPARK-30602 looks too big for this particular issue. What I am looking for is to disable this feature completely by default (no await). We can further improve it that brings no perf regression, or replace it by SPARK-30602. @xuanyuanking can you help to do it? |
|
Sure, will give a follow up for this. |
…event loop group ### What changes were proposed in this pull request? Fix the regression caused by #22173. The original PR changes the logic of handling `ChunkFetchReqeust` from async to sync, that's causes the shuffle benchmark regression. This PR fixes the regression back to the async mode by reusing the config `spark.shuffle.server.chunkFetchHandlerThreadsPercent`. When the user sets the config, ChunkFetchReqeust will be processed in a separate event loop group, otherwise, the code path is exactly the same as before. ### Why are the changes needed? Fix the shuffle performance regression described in #22173 (comment) ### Does this PR introduce any user-facing change? Yes, this PR disable the separate event loop for FetchRequest by default. ### How was this patch tested? Existing UT. Closes #27665 from xuanyuanking/SPARK-24355-follow. Authored-by: Yuanjian Li <[email protected]> Signed-off-by: Wenchen Fan <[email protected]>
…event loop group ### What changes were proposed in this pull request? Fix the regression caused by #22173. The original PR changes the logic of handling `ChunkFetchReqeust` from async to sync, that's causes the shuffle benchmark regression. This PR fixes the regression back to the async mode by reusing the config `spark.shuffle.server.chunkFetchHandlerThreadsPercent`. When the user sets the config, ChunkFetchReqeust will be processed in a separate event loop group, otherwise, the code path is exactly the same as before. ### Why are the changes needed? Fix the shuffle performance regression described in #22173 (comment) ### Does this PR introduce any user-facing change? Yes, this PR disable the separate event loop for FetchRequest by default. ### How was this patch tested? Existing UT. Closes #27665 from xuanyuanking/SPARK-24355-follow. Authored-by: Yuanjian Li <[email protected]> Signed-off-by: Wenchen Fan <[email protected]> (cherry picked from commit 0fe203e) Signed-off-by: Wenchen Fan <[email protected]>
…dle block fetch requests. Description: Right now, the default server side netty handler threads is 2 * # cores, and can be further configured with parameter spark.shuffle.io.serverThreads. In order to process a client request, it would require one available server netty handler thread. However, when the server netty handler threads start to process ChunkFetchRequests, they will be blocked on disk I/O, mostly due to disk contentions from the random read operations initiated by all the ChunkFetchRequests received from clients. As a result, when the shuffle server is serving many concurrent ChunkFetchRequests, the server side netty handler threads could all be blocked on reading shuffle files, thus leaving no handler thread available to process other types of requests which should all be very quick to process. This issue could potentially be fixed by limiting the number of netty handler threads that could get blocked when processing ChunkFetchRequest. We have a patch to do this by using a separate EventLoopGroup with a dedicated ChannelHandler to process ChunkFetchRequest. This enables shuffle server to reserve netty handler threads for non-ChunkFetchRequest, thus enabling consistent processing time for these requests which are fast to process. After deploying the patch in our infrastructure, we no longer see timeout issues with either executor registration with local shuffle server or shuffle client establishing connection with remote shuffle server. (Please fill in changes proposed in this fix) For Original PR please refer here apache#21402 Unit tests and stress testing. (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review http://spark.apache.org/contributing.html before opening a pull request. Closes apache#22173 from redsanket/SPARK-24335. Authored-by: Sanket Chintapalli <[email protected]> Signed-off-by: Thomas Graves <[email protected]> (cherry picked from commit ff601cf) (cherry picked from commit 8094f59a8128166dd283d97c49322ec09fa2c93c) Change-Id: Id91ae9d20ffe9aae09585b197c6a15f4e8044895 (cherry picked from commit 5261726732ef5a9cf21e02a25bba0d14a808c274) Change-Id: Ic666004b8b3d17038d8f42f4154058c0d00e9a37
…event loop group ### What changes were proposed in this pull request? Fix the regression caused by apache#22173. The original PR changes the logic of handling `ChunkFetchReqeust` from async to sync, that's causes the shuffle benchmark regression. This PR fixes the regression back to the async mode by reusing the config `spark.shuffle.server.chunkFetchHandlerThreadsPercent`. When the user sets the config, ChunkFetchReqeust will be processed in a separate event loop group, otherwise, the code path is exactly the same as before. ### Why are the changes needed? Fix the shuffle performance regression described in apache#22173 (comment) ### Does this PR introduce any user-facing change? Yes, this PR disable the separate event loop for FetchRequest by default. ### How was this patch tested? Existing UT. Closes apache#27665 from xuanyuanking/SPARK-24355-follow. Authored-by: Yuanjian Li <[email protected]> Signed-off-by: Wenchen Fan <[email protected]>




What changes were proposed in this pull request?
Description:
Right now, the default server side netty handler threads is 2 * # cores, and can be further configured with parameter spark.shuffle.io.serverThreads.
In order to process a client request, it would require one available server netty handler thread.
However, when the server netty handler threads start to process ChunkFetchRequests, they will be blocked on disk I/O, mostly due to disk contentions from the random read operations initiated by all the ChunkFetchRequests received from clients.
As a result, when the shuffle server is serving many concurrent ChunkFetchRequests, the server side netty handler threads could all be blocked on reading shuffle files, thus leaving no handler thread available to process other types of requests which should all be very quick to process.
This issue could potentially be fixed by limiting the number of netty handler threads that could get blocked when processing ChunkFetchRequest. We have a patch to do this by using a separate EventLoopGroup with a dedicated ChannelHandler to process ChunkFetchRequest. This enables shuffle server to reserve netty handler threads for non-ChunkFetchRequest, thus enabling consistent processing time for these requests which are fast to process. After deploying the patch in our infrastructure, we no longer see timeout issues with either executor registration with local shuffle server or shuffle client establishing connection with remote shuffle server.
(Please fill in changes proposed in this fix)
For Original PR please refer here
#21402
How was this patch tested?
Unit tests and stress testing.
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
Please review http://spark.apache.org/contributing.html before opening a pull request.