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][pg] Fix pg bundles can't reschedule bug when the two nodes both dead #24875

Merged
merged 1 commit into from
Nov 21, 2022

Conversation

larrylian
Copy link
Contributor

@larrylian larrylian commented May 17, 2022

Why are these changes needed?

When a bundle is rescheduled due to node death, if other bundles of this pg also trigger rescheduling due to node death, there will be a bug that the bundle cannot be scheduled.

Reason:
step 1: Node A is down, and then bundle 1 of PG deployed on this node enters this GcsPlacementGroupManager::OnNodeDead process. This PG state will be RESCHEDULING and going to scheduling.

step 2: Just when this PG was being scheduled, another node B also went down. Bundle 2 of this PG also enters this GcsPlacementGroupManager::OnNodeDead process.

step 3: Because this PG state is RESCHEDULING, the bundle 2 can't be added to pending queue。 In the end, the bundle 2 cannot be rescheduled.

image

Solution:
1、After each PG is scheduled success, judge whether the PG has any unplace bundles. If there is, then schedule it.

Related issue number

Checks

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

@rkooo567
Copy link
Contributor

rkooo567 commented Jun 9, 2022

ETA for the review EoD tomorrow in Japan time zone

@fishbone
Copy link
Contributor

Could you update the description with more detail about why it's failing and how this PR fixed this?

Also could you please merge your PR with master?

@fishbone fishbone changed the title [core/pg] Fix pg bundles can't reschedule bug when the two nodes both dead [core][pg] Fix pg bundles can't reschedule bug when the two nodes both dead Jul 6, 2022
@fishbone fishbone self-assigned this Jul 6, 2022
@@ -354,6 +360,12 @@ class GcsPlacementGroupManager : public rpc::PlacementGroupInfoHandler {
// Update placement group load information so that the autoscaler can use it.
void UpdatePlacementGroupLoad();

/// Check if this placement group is waiting for scheduling.
bool IsPlacmentGroupIDInPendingQueue(const PlacementGroupID &placement_group_id) const;
Copy link
Contributor

Choose a reason for hiding this comment

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

no strong opinion, but feel the name is too long.

IsInPendingQueue(...)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

@fishbone
Copy link
Contributor

fishbone commented Jul 6, 2022

I'm not sure, but shouldn't we just allow it to be pushed to pending queue?

@rkooo567 could you take a look?

@larrylian
Copy link
Contributor Author

I'm not sure, but shouldn't we just allow it to be pushed to pending queue?
Pushing to the pending queue is fine.

@fishbone
Copy link
Contributor

I'm trying to understand the PR and I tried this on my dev and it always pass (on master):

https://gist.github.com/6edaf987a00e185f50bfaf1f7168d758

I'm not sure what's the issue here. It'll be good if you can provide a simple script which can reproduce the issue.

@larrylian
Copy link
Contributor Author

I'm trying to understand the PR and I tried this on my dev and it always pass (on master):

https://gist.github.com/6edaf987a00e185f50bfaf1f7168d758

I'm not sure what's the issue here. It'll be good if you can provide a simple script which can reproduce the issue.

This scene is not easy to reproduce. This can only modify the code of void NodeManager::HandlePrepareBundleResources and let HandlePrepareBundleResources wait for a while. In this way, during the scheduling process of bundle 1, bundle 2 triggers the OnNodeDead process.
image

a simple script which can reproduce the issue:

import pytest
import sys
import time
import ray
import ray.cluster_utils
from ray._private.test_utils import (
    get_other_nodes,
)

MB = 1024 * 1024


@pytest.mark.parametrize("repeat", list(range(3)))
def test_placement_group_failover(ray_start_cluster, repeat):
    @ray.remote(num_cpus=1)
    class Actor(object):
        def __init__(self):
            self.n = 0

        def value(self):
            return self.n

    cluster = ray_start_cluster
    num_nodes = 6
    nodes = []
    for _ in range(num_nodes):
        nodes.append(cluster.add_node(num_cpus=1))
    ray.init(address=cluster.address)
    bundles = [{"CPU": 1, "memory": 100 * MB} for _ in range(num_nodes)]
    placement_group = ray.util.placement_group(
        name="name", strategy="STRICT_SPREAD", bundles=bundles
    )
    assert placement_group.wait(10000)
    other_nodes = get_other_nodes(cluster, exclude_head=True)
    other_nodes_num = len(other_nodes)
    for i in range(other_nodes_num):
        cluster.add_node(num_cpus=1)
    cluster.wait_for_nodes()
    for node in other_nodes:
        cluster.remove_node(node)
    time.sleep(1)
    for i in range(num_nodes):
        actor = Actor.options(
            placement_group=placement_group, placement_group_bundle_index=i
        ).remote()
        object_ref = actor.value.remote()
        ray.get(object_ref, timeout=5)


if __name__ == "__main__":
    sys.exit(pytest.main(["-v", __file__]))

@fishbone
Copy link
Contributor

fishbone commented Jul 20, 2022

@larrylian we have latency injection here https://docs.ray.io/en/latest/ray-contribute/debugging.html#callback-latency-injection

Maybe you can give that a try.

Btw, the script you give is what I tried locally. I build your branch and run the test script in this branch.
What I mean is that it always pass. Maybe because of the setup of the machine (I saw you have repeat 3 in the decorator, I guess it's not always reproducing?)

@larrylian
Copy link
Contributor Author

larrylian commented Jul 20, 2022

@iycheng
It is difficult to reproduce without injecting the delay. The injection delay method you mentioned, I will understand it first.

@fishbone
Copy link
Contributor

@iycheng It is difficult to reproduce without injecting the delay. The injection delay method you mentioned, I will understand it first.

Here are some examples which you can refer to https://sourcegraph.com/search?patternType=regexp&case=yes&q=context:%40iycheng/ray+RAY_testing_asio_delay_us

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.

Can you make SchedulePendingPlacementGroups no-op if all bundles are already placed? I think we need this as a safety check

python/ray/tests/test_placement_group_fo.py 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
python/ray/tests/test_placement_group_fo.py Outdated Show resolved Hide resolved
python/ray/tests/test_placement_group_fo.py Outdated Show resolved Hide resolved
python/ray/tests/test_placement_group_fo.py Outdated Show resolved Hide resolved
python/ray/tests/test_placement_group_fo.py Outdated Show resolved Hide resolved
python/ray/tests/test_placement_group_fo.py Outdated Show resolved Hide resolved
python/ray/tests/test_placement_group_fo.py Outdated Show resolved Hide resolved
@rkooo567
Copy link
Contributor

I think we can easily reproduce it if we add some delay on either commit or prepare rpcs

@stale
Copy link

stale bot commented Sep 8, 2022

This pull request has been automatically marked as stale because it has not had recent activity. It will be closed in 14 days if no further activity occurs. Thank you for your contributions.

  • If you'd like to keep this open, just leave any comment, and the stale label will be removed.

@stale stale bot added the stale The issue is stale. It will be closed within 7 days unless there are further conversation label Sep 8, 2022
@stale
Copy link

stale bot commented Sep 24, 2022

Hi again! The issue will be closed because there has been no more activity in the 14 days since the last message.

Please feel free to reopen or open a new issue if you'd still like it to be addressed.

Again, you can always ask for help on our discussion forum or Ray's public slack channel.

Thanks again for opening the issue!

@stale stale bot closed this Sep 24, 2022
@larrylian larrylian reopened this Oct 11, 2022
@larrylian larrylian force-pushed the fix_pg_bundle_both_fo_bug branch 2 times, most recently from 85583d7 to 44d2a2f Compare October 11, 2022 08:19
@stale stale bot removed the stale The issue is stale. It will be closed within 7 days unless there are further conversation label Oct 12, 2022
@larrylian larrylian force-pushed the fix_pg_bundle_both_fo_bug branch 3 times, most recently from 8d81072 to 5fce60b Compare November 3, 2022 03:27
@@ -728,6 +737,9 @@ void GcsPlacementGroupManager::OnNodeDead(const NodeID &node_id) {
if (iter != registered_placement_groups_.end()) {
for (const auto &bundle_index : bundle.second) {
iter->second->GetMutableBundle(bundle_index)->clear_node_id();
RAY_LOG(INFO) << "Rescheduling a bundle when a node dies, placement group id:"
Copy link
Contributor

Choose a reason for hiding this comment

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

remove or just change to debug log

Copy link
Contributor Author

Choose a reason for hiding this comment

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

  1. This function OnNodeDead will only come in when the node is abnormally down. Therefore, this log is usually not printed and will not be printed frequently. It will only be printed when the node is abnormal.
  2. The production environment uses INFO level logs, and DEBUG logs cannot be used. Then, if a node fails in the production environment, you need to check this log to determine which bundles have FO.

So this log I think to keep and use INFO level.

Copy link
Contributor

Choose a reason for hiding this comment

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

+1. Can you address this?

@fishbone fishbone added release-blocker P0 Issue that blocks the release Ray 2.2 labels Nov 17, 2022
@scv119 scv119 added the P0 Issues that should be fixed in short order label Nov 17, 2022
@xwjiang2010
Copy link
Contributor

Reminder, the 2.2 release is scheduled for branch cut next Monday. Do folks feel that we are pretty close on this PR? Only asking as I notice that this PR has been out for a while.

@rkooo567
Copy link
Contributor

Let me run the nightly test before merging.

@rkooo567
Copy link
Contributor

rkooo567 commented Nov 18, 2022

[x] placement_group_stress_test
[x] pg_long_running_performance_test
[x] placement_group_performance_test
[ ] many_pgs (Running)

@larrylian
Copy link
Contributor Author

[x] placement_group_stress_test [ ] pg_long_running_performance_test (Running) [ ] placement_group_performance_test [ ] many_pgs

Thanks, please contact me if you have any problem.

@larrylian
Copy link
Contributor Author

[x] placement_group_stress_test [x] pg_long_running_performance_test [ ] placement_group_performance_test (Running) [ ] many_pgs

I see that most CIs have passed. Is there any problem with your night test?

@rkooo567
Copy link
Contributor

Running another 2 now. It seems promising so far

@rkooo567
Copy link
Contributor

rkooo567 commented Nov 21, 2022

@larrylian I will merge this by today (branch cut is tmrw). We should be able to verify all tests are passing by today

@larrylian
Copy link
Contributor Author

@larrylian I will merge this by today (branch cut is tmrw). We should be able to verify all tests are passing by today

OK, Thanks.

@rkooo567 rkooo567 merged commit ddfeae3 into ray-project:master Nov 21, 2022
@fishbone
Copy link
Contributor

Thanks @rkooo567 for covering this. My bad, somehow I forgot it :(

WeichenXu123 pushed a commit to WeichenXu123/ray that referenced this pull request Dec 19, 2022
…h dead (ray-project#24875)

When a bundle is rescheduled due to node death, if other bundles of this pg also trigger rescheduling due to node death, there will be a bug that the bundle cannot be scheduled.

Reason:
step 1: Node A is down, and then bundle 1 of PG deployed on this node enters this GcsPlacementGroupManager::OnNodeDead process. This PG state will be RESCHEDULING and going to scheduling.

step 2: Just when this PG was being scheduled, another node B also went down. Bundle 2 of this PG also enters this GcsPlacementGroupManager::OnNodeDead process.

step 3: Because this PG state is RESCHEDULING, the bundle 2 can't be added to pending queue。 In the end, the bundle 2 cannot be rescheduled.

Signed-off-by: Weichen Xu <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
P0 Issues that should be fixed in short order release-blocker P0 Issue that blocks the release
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants