Skip to content

[Autoscaler][V2] Fix autoscaler terminating more nodes than exist of a type #52760

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

Merged
merged 12 commits into from
May 15, 2025

Conversation

ryanaoleary
Copy link
Contributor

@ryanaoleary ryanaoleary commented May 2, 2025

Why are these changes needed?

We're seeing an issue where the value of worker_to_delete_set is incorrect, stopping the autoscaler from reconciling correctly due to the following assertion failing:

assert num_workers_dict[to_delete_instance.node_type] >= 0

Currently in _get_workers_delete_info, if there are multiple nodes in the same worker group with a pending deletion, only the first one is added to worker_to_delete_set. This set is then used in _initialize_scale_request here:

if to_delete_instance.cloud_instance_id in worker_to_delete_set:
  # If the instance is already in the workersToDelete field of
  # any worker group, skip it.
  continue

but since all workers aren't being added to the set, the above assertion fails and future calls to intialize_scale_request are unsuccessful.

Related issue number

#52264

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

@ryanaoleary ryanaoleary requested a review from a team as a code owner May 2, 2025 23:40
@ryanaoleary
Copy link
Contributor Author

ryanaoleary commented May 2, 2025

cc: @kevin85421 I tested this with the V2 autoscaler unit tests, but haven't been able to manually replicate the issue to where I could test the fix with my dev image. I added a guard and logging in 503eeea, since it seems like bad behavior for the autoscaler to block all future scale up and terminate calls (since they both rely on initialize_scale_request) if the workers_to_delete set is out of sync.

Copy link
Member

@kevin85421 kevin85421 left a comment

Choose a reason for hiding this comment

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

@ryanaoleary, I recall you mentioned that a user can consistently reproduce the issue. Could you build an image for them to verify it? Additionally, if possible, please do your best to reproduce the issue so we can add tests to prevent it from happening again.

@ryanaoleary ryanaoleary requested a review from kevin85421 May 5, 2025 22:30
Copy link
Member

@kevin85421 kevin85421 left a comment

Choose a reason for hiding this comment

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

Chat with @ryanaoleary offline. The customer can't use the dev image. Let's merge this PR and ask the users to verify whether the issue is resolved or not.

@kevin85421 kevin85421 added the go add ONLY when ready to merge, run all tests label May 5, 2025
@kevin85421
Copy link
Member

cc @jjyao @edoakes for merging this PR. Thanks!

@hainesmichaelc hainesmichaelc added the community-contribution Contributed by the community label May 7, 2025
@ryanaoleary
Copy link
Contributor Author

I wrote a unit test that sets replicas and minReplicas to 0 and then attempts to terminate a node on the same reconciliation (I haven't pushed it yet because it's failing), this results in the error occurring even with the fix:

            num_workers_dict[to_delete_instance.node_type] -= 1
>           assert num_workers_dict[to_delete_instance.node_type] >= 0
E           AssertionError

../instance_manager/cloud_providers/kuberay/cloud_provider.py:253: AssertionError

 python/ray/autoscaler/v2/tests/test_node_provider.py::KubeRayProviderIntegrationTest.test_scale_down_pods_with_replica_limit ⨯93% █████████▍2025-05-08 18:36:39,480  INFO cloud_provider.py:121 -- Terminating worker pods: ['raycluster-autoscaler-worker-small-group-dkz2r']
2025-05-08 18:36:39,480 INFO cloud_provider.py:480 -- Fetched pod data at resource version .
2025-05-08 18:36:39,481 INFO cloud_provider.py:331 -- Submitting a scale request: KubeRayProvider.ScaleRequest(desired_num_workers=defaultdict(<class 'int'>, {'small-group': 0, 'gpu-group': 1, 'tpu-group': 1}), workers_to_delete=defaultdict(<class 'list'>, {'small-group': [CloudInstance(cloud_instance_id='raycluster-autoscaler-worker-small-group-dkz2r', node_type='small-group', node_kind=2, is_running=False, request_id=None)]}), worker_groups_without_pending_deletes=set(), worker_groups_with_pending_deletes=set())

since cur_instances still contains the instance to be deleted, but the value of replicas has been updated to a lower value. This is why I think we need the change from b4f32fa that adds a guard and a warning (rather than an assertion) when a value of num_workers_dict goes below 0, since I think the autoscaler shouldn't continuously error on each subsequent iteration that calls _initialize_scale_request if a user updates replicas/minReplicas manually. We can instead log a warning and allow it to reconcile on the next iteration - since the Pod will terminate after the scale request gets submitted, workers_to_delete will clear, and the value of replicas/minReplicas will become accurate. What do you think @kevin85421 @jjyao?

@kevin85421
Copy link
Member

Move our slack messages to here:

In my opinion, we shouldn’t allow users to manually update replicas; they should only be able to adjust the minimum and maximum replica counts.

I think we shouldn't have the autoscaler completely crash if a user updates replicas/minReplicas manually

This makes sense, but I’m worried it could bury other issues even deeper. Could you ask the users if they’re trying to manually lower both minReplicas and replicas?

If we can confirm that the issue is caused by either _get_workers_delete_info or manually lowering both minReplicas and replicas, I’m OK with accepting b4f32fa.

@ryanaoleary ryanaoleary requested a review from jjyao May 12, 2025 16:56
@ryanaoleary
Copy link
Contributor Author

Move our slack messages to here:

In my opinion, we shouldn’t allow users to manually update replicas; they should only be able to adjust the minimum and maximum replica counts.

I think we shouldn't have the autoscaler completely crash if a user updates replicas/minReplicas manually

This makes sense, but I’m worried it could bury other issues even deeper. Could you ask the users if they’re trying to manually lower both minReplicas and replicas?

If we can confirm that the issue is caused by either _get_workers_delete_info or manually lowering both minReplicas and replicas, I’m OK with accepting b4f32fa.

I added a unit test in 9a6b143 that verifies that the provider can handle multiple node deletions of the same type in one autoscaler iteration. This should cover the bug with workers_to_delete not being correctly counted that is fixed by this PR. I don't think they're manually updating replicas or minReplicas, can we merge this PR as-is and then revisit the guard/logging change in b4f32fa if it's still required based on their response?

@kevin85421
Copy link
Member

can we merge this PR as-is and then revisit the guard/logging change in b4f32fa if it's still required based on their response?

SG. I will review the new changes.

Copy link
Member

@kevin85421 kevin85421 left a comment

Choose a reason for hiding this comment

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

@ryanaoleary ryanaoleary requested a review from kevin85421 May 13, 2025 00:21
Copy link
Member

@kevin85421 kevin85421 left a comment

Choose a reason for hiding this comment

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

Approve to unblock the user. I will open a follow up PR to do some improvements.

@kevin85421
Copy link
Member

cc @jjyao would you mind merging this PR?

pending_deletes,
finished_deletes,
workers_to_delete,
) = self.provider._get_workers_delete_info(
Copy link
Collaborator

Choose a reason for hiding this comment

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

We just had a conversation about not testing private method (i.e. implementation detail): #52812 (comment). Can we test the public interface instead?

Copy link
Member

@kevin85421 kevin85421 May 13, 2025

Choose a reason for hiding this comment

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

I don’t agree with this philosophy in this case. Unit tests are much more appropriate here. We should

(1) Add more real e2e tests for KubeRay.
(2) Add more unit tests for well-defined functions.

We should avoid tests between these two which we need to mock the behaviors, and it's pretty common that the mock behavior is wrong.

Copy link
Collaborator

Choose a reason for hiding this comment

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

Yea, is there a way to test it through the public APIs?

Copy link
Member

Choose a reason for hiding this comment

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

Let's chat in-person.

pending_deletes,
finished_deletes,
workers_to_delete,
) = self.provider._get_workers_delete_info(
Copy link
Collaborator

Choose a reason for hiding this comment

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

_get_workers_delete_info is a static method so we we don't need to create a KubeRayProvider instance at all.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Oh oops yeah, I was using the provider to get the list of cloud instances, but since it's KubeRay we can just use the Pod names. This should be fixed in 2b10b1a.

@ryanaoleary ryanaoleary requested a review from jjyao May 15, 2025 00:00
@jjyao jjyao merged commit aa4deb3 into ray-project:master May 15, 2025
5 checks passed
@jjyao
Copy link
Collaborator

jjyao commented May 15, 2025

Merging to unblock the fix. @kevin85421 will follow-up on some of the tests cleanup.

iamjustinhsu pushed a commit to iamjustinhsu/ray that referenced this pull request May 15, 2025
lk-chen pushed a commit to lk-chen/ray that referenced this pull request May 17, 2025
@JKBGIT1
Copy link

JKBGIT1 commented May 26, 2025

Hello guys, what's the ETA for release, please?

vickytsang pushed a commit to ROCm/ray that referenced this pull request Jun 3, 2025
rebel-scottlee pushed a commit to rebellions-sw/ray that referenced this pull request Jun 21, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
clusters community-backlog community-contribution Contributed by the community go add ONLY when ready to merge, run all tests stability
Projects
None yet
Development

Successfully merging this pull request may close these issues.

7 participants