-
Notifications
You must be signed in to change notification settings - Fork 28.1k
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-37675][CORE][SHUFFLE] Return PushMergedRemoteMetaFailedFetchResult if no available push-merged block #34934
Conversation
Can one of the admins verify this patch? |
Found another issue with this change, investigating |
This reverts commit d1e23d9.
+CC @otterc |
Will take a look on Monday. It's unusual that there isn't even a single chunk because that means that no blocks were merged. If there aren't any blocks that were merged then there shouldn't be a push-merged block for the iterator to fetch at all. In this case there isn't any need to fetch the meta. Seems like that there could be a bug somewhere else where empty mergeStatus for a partition is getting created. |
After the change, my 1T TPCDS passed one round, but failed in the second round, I'm not sure if it's related or just another issue. I'm a newbie in this area, I'll appreciate it if you can give me some advices.
|
+CC @Ngone51 Looks like there is an additional interaction between shuffle corruption diagnosis and push based shuffle. Failure to fetch push based shuffle chunks should have resulted in fallback to fetching the original shuffle blocks - needs to be investigated why it dropped down to diagnosis directly (CC @otterc ) |
Yeah, we should skip diagnosis for push-based shuffle, created a ticket for it: SPARK-37695 |
It looks like the fetch to push-based chunks doesn't fail. However, it fails when consuming the fetched chunk data. In this case, we can only throw exception since the downstream RDDs could have already consumed partial data. |
iterator.addToResultsQueue(PushMergedRemoteMetaFetchResult(shuffleId, shuffleMergeId, | ||
reduceId, sizeMap((shuffleId, reduceId)), bitmaps, address)) | ||
} else { | ||
logInfo(s"No available push-merged block for ($shuffleId, $shuffleMergeId," + |
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 thought we only raise the merged status request when there are merged blocks. Because we ensured this when fetching merge status:
val remainingMapStatuses = if (mergeStatus != null && mergeStatus.totalSize > 0) { |
Sounds like there is something wrong earlier than I dealt with, I will investigate in this direction, and use this change as a workaround until the issue has been addressed to unblock my push-based shuffle feature test. |
#32287 (comment) |
@otterc I reproduced this issue today and sent a email to you with logs and spark confs you requested here, highly suspect that SPARK-37675 and SPARK-37793 share the same root cause. Please let me know if any other things I can do. |
To clarify, these logs are with a version of spark/shuffle service without modifications ? |
Oops, I forgot to link the code pan3793@c38459d |
@pan3793 Would you be able to add these changes and rerun this test?
Please let me know if you can rerun with these changes and share the logs with me. |
Thanks @otterc, changed code as you suggested, pan3793@1c14eb9, and have sent logs to you via email, please take a look again. |
Thanks @pan3793. The new logs, however, don't have the statement on the shuffle server which logs the partition that is finalized. I see that you removed this:
I think that the partitions that the reducer is trying to read are not finalized by the shuffle server. This log statement would have shown that. I still don't know how the reducer is getting these merged block but at least we can rule out the shuffle service. |
Is there possible that shuffle service overwites the finalized merged blocks?
I think BTW, I will lose the access of this test cluster after 1.31, hopes we can find the root cause before then. spark/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala Lines 486 to 536 in ef00554
|
@pan3793, @mridulm identified the issue on the server side. You are right that the service was overwriting the finalized merged blocks. The bug was introduced with SPARK-32923. Will update later about the bug. |
We're closing this PR because it hasn't been updated in a while. This isn't a judgement on the merit of the PR in any way. It's just a way of keeping the PR queue manageable. |
What changes were proposed in this pull request?
When push-based shuffle enabled, reduce task will ask ESS for MergedMetas, then
PushMergedRemoteMetaFailedFetchResult
should be returned instead ofPushMergedRemoteMetaFetchResult
if there is no available push-merged block on ESS.Why are the changes needed?
Because that push-based shuffle works as best-effort, there are opportunities that no chunks of the block are available on ESS, in current implementation, it will cause reduce task failed with
ArithmeticException: / by zero
.After the change, a
PushMergedRemoteMetaFetchResult
will let reduce task fall back to fetching the original blocks.Does this PR introduce any user-facing change?
Yes, it'a bug fix. The change makes push-based shuffle stable. Before this change, my 1T TPCDS test failed several times w/
ArithmeticException: / by zero
, after the change, passed w/o any exception.How was this patch tested?
Existing tests, and run 1T TPCDS manually.