Skip to content

[Core] Put pg state to kv store when pg rescheduling#34467

Merged
rkooo567 merged 1 commit intoray-project:masterfrom
larrylian:pg_fo_when_gcs_fo
Apr 27, 2023
Merged

[Core] Put pg state to kv store when pg rescheduling#34467
rkooo567 merged 1 commit intoray-project:masterfrom
larrylian:pg_fo_when_gcs_fo

Conversation

@larrylian
Copy link
Contributor

@larrylian larrylian commented Apr 17, 2023

Why are these changes needed?

When a PG fails over but has not been scheduled successfully, the restart of gcs will cause the PG to no longer be rescheduled.

  1. A node is down, triggering the rescheduling of the PG bundle on this node
  2. However, due to insufficient resources, this PG bunlde cannot be scheduled successfully
  3. The gcs server sent FO
  4. In the end, even if the resources are sufficient, the PG bundle is still not rescheduled.

Reproduce command:

pytest -sv python/ray/tests/test_placement_group_failover.py::test_gcs_restart_when_placement_group_failover

Because the rescheduling state of PG is lost when gcs restarts.
solution:
It is necessary to save the PG to kvstore when the PG is changed to the rescheduling state.

image

Related issue number

When a PG fails over but has not been scheduled successfully, the restart of gcs will cause the PG to no longer be rescheduled. #34468

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 :(

@larrylian larrylian requested a review from a team as a code owner April 17, 2023 12:20
@larrylian larrylian self-assigned this Apr 17, 2023
@rkooo567 rkooo567 self-assigned this Apr 17, 2023
@larrylian larrylian requested a review from jjyao April 18, 2023 06:06
Signed-off-by: LarryLian <554538252@qq.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.

This makes sense! Btw can you also add a cpp test to simulate both cases?

  1. rescheduling -> node dead -> GCS restarts -> rescheduling succeeds
  2. rescheduling started -> GCS restarts -> rescheduling succeeds

@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 Apr 25, 2023
@larrylian
Copy link
Contributor Author

Btw can you also add a cpp test to simulate both cases?

@rkooo567
In fact, my personal opinion is not to add this use case to gcs_placement_group_manager_test.

  1. My current python "test_gcs_restart_when_placement_group_failover" use case has been able to fully cover this scenario, and it is also a really effective use case.
  2. Many operations in the current gcs_placement_group_manager_test use case are mocked and very complicated. After I add a use case with "gcs restart" logic to it, the later maintenance cost will be even greater.
  3. Now many operations in the gcs_placement_group_manager_test use case are mocked. Even if I add the use case you mentioned, its protection effect is not as good as the python use case.

@rkooo567
Copy link
Contributor

I don't think it is complicated to test it? It seems like we already have the way to test GCS restart

TEST_F(GcsPlacementGroupManagerTest, TestSchedulerReinitializeAfterGcsRestart) {
  // Create a placement group and make sure it has been created successfully.
  auto request = Mocker::GenCreatePlacementGroupRequest();
  std::atomic<int> registered_placement_group_count(0);
  RegisterPlacementGroup(request, [&registered_placement_group_count](Status status) {
    ++registered_placement_group_count;
  });
  ASSERT_EQ(registered_placement_group_count, 1);
  WaitForExpectedPgCount(1);

  auto placement_group = mock_placement_group_scheduler_->placement_groups_.back();
  placement_group->GetMutableBundle(0)->set_node_id(NodeID::FromRandom().Binary());
  placement_group->GetMutableBundle(1)->set_node_id(NodeID::FromRandom().Binary());
  mock_placement_group_scheduler_->placement_groups_.pop_back();
  OnPlacementGroupCreationSuccess(placement_group);
  ASSERT_EQ(placement_group->GetState(), rpc::PlacementGroupTableData::CREATED);
  // Reinitialize the placement group manager and test the node dead case.
  auto gcs_init_data = LoadDataFromDataStorage();
  ASSERT_EQ(1, gcs_init_data->PlacementGroups().size());
  EXPECT_TRUE(
      gcs_init_data->PlacementGroups().find(placement_group->GetPlacementGroupID()) !=
      gcs_init_data->PlacementGroups().end());
  EXPECT_CALL(*mock_placement_group_scheduler_, ReleaseUnusedBundles(_)).Times(1);
  EXPECT_CALL(
      *mock_placement_group_scheduler_,
      Initialize(testing::Contains(testing::Key(placement_group->GetPlacementGroupID()))))
      .Times(1);
  gcs_placement_group_manager_->Initialize(*gcs_init_data);
}

The pro of testing it in cpp is we can test the exact edge cases. The python level test depends on the timing, and it is easy to miss edge cases.

@larrylian
Copy link
Contributor Author

The pro of testing it in cpp is we can test the exact edge cases. The python level test depends on the timing, and it is easy to miss edge cases.

@rkooo567
I agree with this point of view.
Thank you for the example. I'll resubmit a new PR to complement this C++ test case.
This PR has now been run through CI, I think it will be more efficient to merge it first.

@rkooo567
Copy link
Contributor

We will merge this PR first and @larrylian will submit a follow up PR for unit test!

@rkooo567
Copy link
Contributor

Failures are unlikely related

@rkooo567 rkooo567 merged commit af018f6 into ray-project:master Apr 27, 2023
rkooo567 added a commit to rkooo567/ray that referenced this pull request May 1, 2023
rkooo567 added a commit that referenced this pull request May 2, 2023
ProjectsByJackHe pushed a commit to ProjectsByJackHe/ray that referenced this pull request May 4, 2023
When a PG fails over but has not been scheduled successfully, the restart of gcs will cause the PG to no longer be rescheduled.

A node is down, triggering the rescheduling of the PG bundle on this node
However, due to insufficient resources, this PG bunlde cannot be scheduled successfully
The gcs server sent FO
In the end, even if the resources are sufficient, the PG bundle is still not rescheduled.
Reproduce command:

pytest -sv python/ray/tests/test_placement_group_failover.py::test_gcs_restart_when_placement_group_failover
Because the rescheduling state of PG is lost when gcs restarts.
solution:
It is necessary to save the PG to kvstore when the PG is changed to the rescheduling state.

Signed-off-by: Jack He <jackhe2345@gmail.com>
architkulkarni pushed a commit to architkulkarni/ray that referenced this pull request May 16, 2023
When a PG fails over but has not been scheduled successfully, the restart of gcs will cause the PG to no longer be rescheduled.

A node is down, triggering the rescheduling of the PG bundle on this node
However, due to insufficient resources, this PG bunlde cannot be scheduled successfully
The gcs server sent FO
In the end, even if the resources are sufficient, the PG bundle is still not rescheduled.
Reproduce command:

pytest -sv python/ray/tests/test_placement_group_failover.py::test_gcs_restart_when_placement_group_failover
Because the rescheduling state of PG is lost when gcs restarts.
solution:
It is necessary to save the PG to kvstore when the PG is changed to the rescheduling state.
architkulkarni pushed a commit to architkulkarni/ray that referenced this pull request May 16, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

@author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer.

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants