Skip to content
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

[core] Add PREPARED state for placement groups. #46858

Merged
merged 11 commits into from
Aug 23, 2024

Conversation

rynewang
Copy link
Contributor

Ray Placement Groups use 2pc protocol to reserve resources. However we don't track PREPARED state; we only had PENDING (before any prepare) and CREATED (after all committed). This means when the GCS restarts, it does not know any Commits ever happened. Indeed, it instructs all raylets to return all bundles prepared or committed.

If the GCS restart happens when a PG has finished all Prepare and is doing Commit, the already-Committed bundles may have scheduled actors/tasks, and those actors/tasks may be killed by the GCS restart, making the GCS restart user-visible.

This PR introduces a PREPARED state between PENDING and CREATED. Rules:

  • (New!) On a PG received all Prepare replies successfully: state change PENDING -> PREPARED, and persist to Redis. This also persists all bundle locations (node IDs).
  • After the persist: send all Commit calls.
  • On a PG received all Commit replies successfully: state change PREPARED -> CREATED, and persist to Redis.
  • On GCS Restart, if a PG is PENDING: return any bundles, reschedule from zero.
  • (New!) On GCS Restart, if a PG is PREPARED, continue to send all Commit calls.

In order to do this cleanly, did a bunch of refactoring. The idea is to keep all interface changes to the GcsPlacementGroupScheduler but not to GcsPlacementGroupManager. Specifically:

  • GcsPlacementGroupScheduler::ScheduleUnplacedBundles now accepts an aggregated arg struct SchedulePgRequest.
  • GcsPlacementGroupScheduler::Initialize accepts one more vector of arg struct SchedulePreparedPgRequest, which is the regular SchedulePgRequest plus the prepared bundle information.
  • GcsPlacementGroupManager::Initialize now distinguishes PREPARED PGs by creating requests and send them to the Scheduler. They are also kept in used_bundles to restrain from being removed (<- actual fix).
  • Unit test for the original issue. Failing on master and passing on this PR.
  • Quality of life changes: RAY_LOG().WithField(PlacementGroupID).

rynewang added 2 commits July 29, 2024 16:34
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
@rynewang rynewang added the go add ONLY when ready to merge, run all tests label Jul 30, 2024
@jjyao
Copy link
Collaborator

jjyao commented Jul 30, 2024

Do we have tests that during recommitting the prepared bundles, some recommits failed and the PG goes to RESCHEDULING state?

@rkooo567 rkooo567 self-assigned this Jul 30, 2024
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
@rynewang
Copy link
Contributor Author

Do we have tests that during recommitting the prepared bundles, some recommits failed and the PG goes to RESCHEDULING state?

updated python test.

Copy link
Collaborator

@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.

lgtm.

In the future, we can refactor the code to be more reconciliation style.

Please wait for @rkooo567's approval before merging.

# 5.2. Kill the slow committing node.
cluster.remove_node(bad_node)

time.sleep(1)
Copy link
Collaborator

Choose a reason for hiding this comment

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

can we get rid of time.sleep using wait_for_condition etc?

Copy link
Contributor

Choose a reason for hiding this comment

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

+1

@rynewang
Copy link
Contributor Author

@rkooo567 can you take a look on this one? if you are ok with it pls merge

@rkooo567
Copy link
Contributor

rkooo567 commented Aug 20, 2024

let me try this by today. If it is urgent, please merge it without my approval (if I don't review tonight)

Copy link
Contributor

@rkooo567 rkooo567 left a comment

Choose a reason for hiding this comment

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

WIP review. I think the motivation makes a lot of sense. One thing I'd like to be careful is to handle edge cases in removing pg (I will need time to review this part of code one more time..). In the meantime, can you run a release test?

  • stress_test_placement_group
  • pg_autoscaling_regression_test
  • placement_group_performance_test

I think if stress_test_placement_group passes, I don't worry much about handling removal of pg after introducing a new state

src/ray/gcs/gcs_server/gcs_placement_group_manager.cc Outdated Show resolved Hide resolved
Copy link
Contributor

@rkooo567 rkooo567 left a comment

Choose a reason for hiding this comment

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

Looks pretty good! Just a couple comments on unit tests; AFter that, let's merge

  • Can you make sure we handle removing placement group is handled properly (add unit tests) if it is removed after redis.Put is called (and before it is returned)? I barely remembered we don't remove pg that's in progress to simplify the protocol, but I am not 100% sure if it is still the case.
  • CreatePrepared
  • cpp tests in gcs placement group manager (if exists)?

# 5.2. Kill the slow committing node.
cluster.remove_node(bad_node)

time.sleep(1)
Copy link
Contributor

Choose a reason for hiding this comment

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

+1

src/ray/gcs/gcs_server/gcs_placement_group_manager.cc Outdated Show resolved Hide resolved
src/ray/gcs/gcs_server/gcs_placement_group_manager.cc Outdated Show resolved Hide resolved
src/ray/gcs/gcs_server/gcs_placement_group_manager.cc Outdated Show resolved Hide resolved
placement_group->GetPlacementGroupTableData(),
[this, lease_status_tracker, schedule_failure_handler, schedule_success_handler](
Status status) {
RAY_CHECK_OK(status);
Copy link
Contributor

Choose a reason for hiding this comment

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

I think it may be possible the pg is removed before this Put is replied (since it is async). Can you add an unit test here? I.e., Update redis -> remove pg -> make sure we don't send commit bundle requests

src/ray/gcs/gcs_server/gcs_placement_group_scheduler.h Outdated Show resolved Hide resolved
src/ray/gcs/gcs_server/gcs_placement_group_scheduler.h Outdated Show resolved Hide resolved
@@ -346,10 +354,12 @@ class GcsPlacementGroupScheduler : public GcsPlacementGroupSchedulerInterface {
/// This should be called when GCS server restarts after a failure.
///
/// \param node_to_bundles Bundles used by each node.
Copy link
Contributor

Choose a reason for hiding this comment

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

can you update this docstring? seems to be incorrect

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is still correct

@rkooo567 rkooo567 added the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Aug 22, 2024
@rynewang
Copy link
Contributor Author

So the 3 release tests passed: https://buildkite.com/ray-project/release/builds/20250

rynewang and others added 4 commits August 22, 2024 11:45
Co-authored-by: SangBin Cho <rkooo567@gmail.com>
Signed-off-by: Ruiyang Wang <56065503+rynewang@users.noreply.github.com>
Co-authored-by: SangBin Cho <rkooo567@gmail.com>
Signed-off-by: Ruiyang Wang <56065503+rynewang@users.noreply.github.com>
Co-authored-by: SangBin Cho <rkooo567@gmail.com>
Signed-off-by: Ruiyang Wang <56065503+rynewang@users.noreply.github.com>
Co-authored-by: SangBin Cho <rkooo567@gmail.com>
Signed-off-by: Ruiyang Wang <56065503+rynewang@users.noreply.github.com>
Copy link
Contributor

@rkooo567 rkooo567 left a comment

Choose a reason for hiding this comment

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

let's merge it after adding remove unit tests to cpp (remove happens after Put(PREPARED) is sent and before it is returned). I think there's no race condition now given stress tests pass but just in case

@rynewang
Copy link
Contributor Author

Just to be safe I plan to add a "is cancelled?" check after the put (in the put callback).

@rkooo567
Copy link
Contributor

that sounds good. or some kind of assert that it is not removed if we don't remove pg while it is scheduled (I think this was the case last time)

@rynewang
Copy link
Contributor Author

For the Put-then-cancel case it should already work: the pg is committed, and after all-committed the scheduler finds it marked cancelled and cancel it. But sure I added a TOCTOU check before doing any committing, and destroy it. this should prevent any tasks/actors from starting.

@rynewang
Copy link
Contributor Author

wydm "we don't remove pg while it is scheduled"? I think user can always cancel it at any time?

Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
@rynewang
Copy link
Contributor Author

@rkooo567 cpp test added, "is it cancelled?" check before starting any commit added

@rynewang rynewang removed the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Aug 22, 2024
@rkooo567 rkooo567 enabled auto-merge (squash) August 22, 2024 23:37
@rkooo567
Copy link
Contributor

let's merge!

Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
@github-actions github-actions bot disabled auto-merge August 23, 2024 17:34
@rynewang rynewang enabled auto-merge (squash) August 23, 2024 17:34
@rynewang rynewang merged commit 2897f46 into ray-project:master Aug 23, 2024
6 checks passed
@rynewang rynewang deleted the prepared-pg branch August 23, 2024 20:12
@can-anyscale
Copy link
Collaborator

this is broken window builds https://buildkite.com/ray-project/postmerge/builds/5995#_, i'm reverting to unblock @rynewang

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
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