Skip to content

[Core] Fix Interleaved Placement Group Creation Process due to Node Failure#52202

Merged
jjyao merged 3 commits intoray-project:masterfrom
MengjinYan:core-1068
Apr 11, 2025
Merged

[Core] Fix Interleaved Placement Group Creation Process due to Node Failure#52202
jjyao merged 3 commits intoray-project:masterfrom
MengjinYan:core-1068

Conversation

@MengjinYan
Copy link
Copy Markdown
Contributor

@MengjinYan MengjinYan commented Apr 10, 2025

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:

  1. Node 1 with bundle 1 of placement group pg dies
  2. pg is set to RESCHEDULING state and pg is added to the placement group pending scheduling queue.
  3. SchedulePendingPlacementGroups is called to fetch the placement groups in the pending queue and schedule them
  4. pg is fetched from the queue. Prepare resources requests for bundle 1 received with success state. pg's state is updated to PREPARED.
  5. Node 2 with bundle 2 of pg dies
  6. pg is then set to RESCHEDULING state and the pg is added to the placement group pending scheduling queue
  7. Resource commit responses for bundle 1 is received. GCS tries to update pg's state to CREATED.
  8. However, when the state update, GCS checks whether the current pg state is PREPARED. Since the pg state is updated to RESCHEDULING in 6, the check fails and thus the job fails.

The 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 OnNodeDead logic 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 in CREATED state. In this way, if the placement group state is already in RESCHEDULING or in PREPARED state (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

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
    • I've added any new APIs to the API Reference. For example, if I added a
      method in Tune, I've added it in doc/source/tune/api/ under the
      corresponding .rst file.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

Signed-off-by: Mengjin Yan <mengjinyan3@gmail.com>
@MengjinYan MengjinYan added the go add ONLY when ready to merge, run all tests label Apr 10, 2025
@MengjinYan MengjinYan marked this pull request as ready for review April 10, 2025 05:51
@MengjinYan MengjinYan requested a review from a team as a code owner April 10, 2025 05:51
@jjyao
Copy link
Copy Markdown
Contributor

jjyao commented Apr 10, 2025

@MortalHappiness could you review it?

Copy link
Copy Markdown
Member

@MortalHappiness MortalHappiness left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM. But what is "This PR also added corresponding PRs as well" in the PR description? Besides, what are the cases for PENDING and REMOVED?

Co-authored-by: Chi-Sheng Liu <chishengliu@chishengliu.com>
Signed-off-by: Mengjin Yan <mengjinyan3@gmail.com>
@MengjinYan
Copy link
Copy Markdown
Contributor Author

LGTM. But what is "This PR also added corresponding PRs as well" in the PR description? Besides, what are the cases for PENDING and REMOVED?

I miss typed the sentence. I meant "this PR added corresponding tests as well". Sorry about that.

In the case of PENDING and REMOVED, the placement group doesn't exist. So we don't need to consider them in the rescheduling logic.

Copy link
Copy Markdown
Contributor

@jjyao jjyao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lg

Comment on lines +821 to +822
<< "Placement group that is pending rescheduling shouldn't be in PENDING "
"state. placement_group_id="
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

.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="
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

REMOVED placement group should have no scheduled bundles on the dead node

Signed-off-by: Mengjin Yan <mengjinyan3@gmail.com>
@jjyao
Copy link
Copy Markdown
Contributor

jjyao commented Apr 11, 2025

Dashboard test failure is unrelated.

@jjyao jjyao merged commit 028b9f5 into ray-project:master Apr 11, 2025
4 of 5 checks passed
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>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

community-backlog go add ONLY when ready to merge, run all tests

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants