-
Notifications
You must be signed in to change notification settings - Fork 28.9k
SPARK-24355 Spark external shuffle server improvement to better handle block fetch requests. #21402
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
…e block fetch requests.
to confirm, |
what's the strategy here? if the number of block fetch requests has reached the limitation, shall we fail following block fetch requests immediately? |
The strategy in this patch is to use a separate Processing the In this PR, the If the number of |
I think we should still respect |
Using a percentage to configure the number of threads to handle chunk fetch requests does make sense. Will update the PR for this change. |
while we are here, could we also add or at least propose some metrics around this, such as number of open block failure, or even number of block threads? we have suffer a lot from the lack of visibility |
@felixcheung |
I'm fine of adding metrics in another PR, please add a TODO in the code comment. |
ok to test |
Test build #91444 has finished for PR 21402 at commit
|
Hmm, I'm not so sure that's accurate. I think the main difference is that I don't think there is currently any code path that sends a I took a look at the code and it seems the actual change that avoids the disk thrashing is the synchronization done in the chunk fetch handler; it only allows a certain number of threads to actually do disk reads simultaneously. That's an improvement already, but a couple of questions popped in my head when I read your comment:
|
ok to test |
Test build #91593 has finished for PR 21402 at commit
|
Another question before I forget about it again: with the current code, if I think Wenchen suggested that that value should be a percentage of the number of threads for the other pool, which would avoid that problem (as long as the percentage is < 100). But a check that results in an error or at least a noisy warning log could at least help users detect a misconfiguration. |
shall we close it since #22173 is merged? |
@cloud-fan yes we can close this |
Can one of the admins verify this patch? |
@redsanket you should close it by yourself. |
…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 <schintap@yahoo-inc.com> Signed-off-by: Thomas Graves <tgraves@apache.org>
…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 <schintap@yahoo-inc.com> Signed-off-by: Thomas Graves <tgraves@apache.org>
…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 <schintap@yahoo-inc.com> Signed-off-by: Thomas Graves <tgraves@apache.org>
…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 <schintap@yahoo-inc.com> Signed-off-by: Thomas Graves <tgraves@apache.org> (cherry picked from commit ff601cf)
What changes were proposed in this pull request?
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.
How was this patch tested?
Added unit test for this patch.
In addition, we built an internal tool to stress test Spark shuffle service and have observed significant improvement after applying this patch.
Please review http://spark.apache.org/contributing.html before opening a pull request.