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

[Ray autoscaler v2] Can't scaler up when using autoscaler v2 #46473

Closed
yx367563 opened this issue Jul 8, 2024 · 0 comments · Fixed by #48909
Closed

[Ray autoscaler v2] Can't scaler up when using autoscaler v2 #46473

yx367563 opened this issue Jul 8, 2024 · 0 comments · Fixed by #48909
Assignees
Labels
bug Something that is supposed to be working; but isn't core Issues that should be addressed in Ray Core core-autoscaler autoscaler related issues P2 Important issue, but not time-critical

Comments

@yx367563
Copy link

yx367563 commented Jul 8, 2024

What happened + What you expected to happen

For the same environment, only changing the use of autoscaler v1 or v2, for a one-time submission of 8000 tasks, v1 can work normally, but v2 will always be stuck, can not be scaled up.
Also, I want to know what has made recent progress in AutoScaler V2? It seems that it has not been updated for a long time.

Versions / Dependencies

Ray 2.23.0
Kuberay 1.1.1

Reproduction script

import ray
import time
import os
import random

@ray.remote(max_retries=5, num_cpus=8)
def inside_ray_task():
    sleep_time = random.randint(120, 600)

    start_time = time.perf_counter()
    while True:
        if(time.perf_counter() - start_time < sleep_time):
            time.sleep(0.001)
        else:
            break
  
@ray.remote(max_retries=0)
def outside_ray_task():
    future_list = []
    for i in range(8000):
        future_list.append(inside_ray_task.remote())
    ray.get(future_list)

if __name__ == '__main__':
    ray.init("ray://localhost:10001")
    ray.get(outside_ray_task.remote())

3adc4197-8928-4f55-9bce-a332d21b3b07

Issue Severity

High: It blocks me from completing my task.

@yx367563 yx367563 added bug Something that is supposed to be working; but isn't triage Needs triage (eg: priority, bug/not-bug, and owning component) labels Jul 8, 2024
@anyscalesam anyscalesam added the core Issues that should be addressed in Ray Core label Jul 8, 2024
@jjyao jjyao added core-autoscaler autoscaler related issues P1.5 Issues that will be fixed in a couple releases. It will be bumped once all P1s are cleared and removed triage Needs triage (eg: priority, bug/not-bug, and owning component) labels Jul 8, 2024
@jjyao jjyao added P2 Important issue, but not time-critical and removed P1.5 Issues that will be fixed in a couple releases. It will be bumped once all P1s are cleared labels Nov 12, 2024
@kevin85421 kevin85421 self-assigned this Nov 19, 2024
jecsand838 pushed a commit to jecsand838/ray that referenced this issue Dec 4, 2024
…eRay RayCluster is not in the goal state (ray-project#48909)

<!-- Thank you for your contribution! Please review
https://github.com/ray-project/ray/blob/master/CONTRIBUTING.rst before
opening a pull request. -->

<!-- Please add a reviewer to the assignee section when you create a PR.
If you don't have the access to it, we will shortly find a reviewer and
assign them to your PR. -->

## Why are these changes needed?

### Issue
  * Create a Autoscaler V2 RayCluster CR.
    * head Pod: `num-cpus: 0`
* worker Pod: Each worker Pod has 1 CPU, and the `maxReplicas` of the
worker group is 10.
* Run the following script in the head Pod:
https://gist.github.com/kevin85421/6f09368ba48572e28f53654dca854b57
* There are 10 scale requests to add a new node. However, only some of
them will be created (e.g., 5).

### Reason

In the reproduction script above, the `cloud_instance_updater` will send
a request to scale up one worker Pod 10 times because the `maxReplicas`
of the worker group is set to 10.

However, the construction of the scale_request depends on the Pods in
the Kubernetes cluster. For example,

* cluster state: RayCluster Replicas: 2, Ray Pods: 1
* 1st scale request: launch 1 node --> goal state: RayCluster Replicas:
2 (Ray Pods + 1)
* 2nd scale request: launch 1 node --> goal state: RayCluster Replicas:
2 (Ray Pods + 1) --> **this should be 3!**

The above example is expected to create 3 Pods. However, it will
ultimately create only 2 Pods.

### Solution

Use RayCluster CR instead of Ray Pods to build scale requests.

## Related issue number

Closes ray-project#46473

## Checks

10 worker Pods are created successfully.

<img width="1373" alt="Screenshot 2024-11-24 at 2 11 39 AM"
src="https://github.com/user-attachments/assets/c42c6cdd-3bf0-4aa9-a928-630c12ff5569">

- [ ] 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: kaihsun <kaihsun@anyscale.com>
Signed-off-by: Connor Sanders <connor@elastiflow.com>
dentiny pushed a commit to dentiny/ray that referenced this issue Dec 7, 2024
…eRay RayCluster is not in the goal state (ray-project#48909)

<!-- Thank you for your contribution! Please review
https://github.com/ray-project/ray/blob/master/CONTRIBUTING.rst before
opening a pull request. -->

<!-- Please add a reviewer to the assignee section when you create a PR.
If you don't have the access to it, we will shortly find a reviewer and
assign them to your PR. -->

## Why are these changes needed?

### Issue
  * Create a Autoscaler V2 RayCluster CR.
    * head Pod: `num-cpus: 0`
* worker Pod: Each worker Pod has 1 CPU, and the `maxReplicas` of the
worker group is 10.
* Run the following script in the head Pod:
https://gist.github.com/kevin85421/6f09368ba48572e28f53654dca854b57
* There are 10 scale requests to add a new node. However, only some of
them will be created (e.g., 5).

### Reason

In the reproduction script above, the `cloud_instance_updater` will send
a request to scale up one worker Pod 10 times because the `maxReplicas`
of the worker group is set to 10.

However, the construction of the scale_request depends on the Pods in
the Kubernetes cluster. For example,

* cluster state: RayCluster Replicas: 2, Ray Pods: 1
* 1st scale request: launch 1 node --> goal state: RayCluster Replicas:
2 (Ray Pods + 1)
* 2nd scale request: launch 1 node --> goal state: RayCluster Replicas:
2 (Ray Pods + 1) --> **this should be 3!**

The above example is expected to create 3 Pods. However, it will
ultimately create only 2 Pods.

### Solution

Use RayCluster CR instead of Ray Pods to build scale requests.


## Related issue number

Closes ray-project#46473

## Checks

10 worker Pods are created successfully.

<img width="1373" alt="Screenshot 2024-11-24 at 2 11 39 AM"
src="https://github.com/user-attachments/assets/c42c6cdd-3bf0-4aa9-a928-630c12ff5569">


- [ ] 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: kaihsun <kaihsun@anyscale.com>
Signed-off-by: hjiang <dentinyhao@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something that is supposed to be working; but isn't core Issues that should be addressed in Ray Core core-autoscaler autoscaler related issues P2 Important issue, but not time-critical
Projects
None yet
4 participants