[Core] Fix Interleaved Placement Group Creation Process due to Node Failure#52202
Merged
jjyao merged 3 commits intoray-project:masterfrom Apr 11, 2025
Merged
[Core] Fix Interleaved Placement Group Creation Process due to Node Failure#52202jjyao merged 3 commits intoray-project:masterfrom
jjyao merged 3 commits intoray-project:masterfrom
Conversation
Signed-off-by: Mengjin Yan <mengjinyan3@gmail.com>
Contributor
|
@MortalHappiness could you review it? |
MortalHappiness
approved these changes
Apr 10, 2025
Co-authored-by: Chi-Sheng Liu <chishengliu@chishengliu.com> Signed-off-by: Mengjin Yan <mengjinyan3@gmail.com>
Contributor
Author
I miss typed the sentence. I meant "this PR added corresponding tests as well". Sorry about that. In the case of |
jjyao
reviewed
Apr 10, 2025
Comment on lines
+821
to
+822
| << "Placement group that is pending rescheduling shouldn't be in PENDING " | ||
| "state. placement_group_id=" |
Contributor
There was a problem hiding this comment.
.WithField(iter->second->GetPlacementGroupID()).WithField(node_id) << "PENDING placement group should have no scheduled bundles on the dead node"
Comment on lines
+825
to
+826
| << "Placement group that is pending rescheduling shouldn't be in REMOVED " | ||
| "state. placement_group_id=" |
Contributor
There was a problem hiding this comment.
REMOVED placement group should have no scheduled bundles on the dead node
Signed-off-by: Mengjin Yan <mengjinyan3@gmail.com>
jjyao
approved these changes
Apr 11, 2025
Contributor
|
Dashboard test failure is unrelated. |
han-steve
pushed a commit
to han-steve/ray
that referenced
this pull request
Apr 11, 2025
…ailure (ray-project#52202) Signed-off-by: Mengjin Yan <mengjinyan3@gmail.com> Signed-off-by: Steve Han <stevehan2001@gmail.com>
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
Why are these changes needed?
When debugging a recent check failure, we found that when multiple nodes with the bundles from the same placement group dies at the same time, the placement group rescheduling process of the 2 nodes will interleave with each other and thus causes check failure.
To be specific, under the above scenario, the following happens:
RESCHEDULINGstate and pg is added to the placement group pending scheduling queue.SchedulePendingPlacementGroupsis called to fetch the placement groups in the pending queue and schedule themPREPARED.RESCHEDULINGstate and the pg is added to the placement group pending scheduling queueThe above issue happens mainly because 2 placement group scheduling processes of the same placement group happens in an interleaved way and we should avoid that by making the scheduling of the same placement group in a sequential manner.
The fix of the issue is updating the
OnNodeDeadlogic in the placement group management logic to only update the placement group state, add the placement group to the pending queue and trigger scheduling, when the placement group is inCREATEDstate. In this way, if the placement group state is already inRESCHEDULINGor inPREPAREDstate (meaning the scheduling process in progress), we will not update the placement group state and trigger another scheduling process.At the same time, in the current placement group scheduling process, we already have the logic to check whether all placement group bundles are placed after a placement group is created. In this sense, if the node dies during the placement group scheduling process, the rescheduling of the bundles on the dead node will happen there.
This PR also added corresponding tests as well.
Related issue number
N/A
Checks
git commit -s) in this PR.scripts/format.shto lint the changes in this PR.method in Tune, I've added it in
doc/source/tune/api/under thecorresponding
.rstfile.