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: Ray cluster nodes underutilization during autoscaling #47355

Open
aljeshishe opened this issue Aug 27, 2024 · 12 comments
Open

Core: Ray cluster nodes underutilization during autoscaling #47355

aljeshishe opened this issue Aug 27, 2024 · 12 comments
Assignees
Labels
bug Something that is supposed to be working; but isn't core Issues that should be addressed in Ray Core P0 Issues that should be fixed in short order

Comments

@aljeshishe
Copy link

aljeshishe commented Aug 27, 2024

What happened + What you expected to happen

We have ray cluster with ec2 cluster with autoscaling (min:2 max: 10 nodes).
Our load is ~3000 tasks.
After we start our load we expect cluster utilization to rise until all tasks are processed.
But that is not the case.
Running tasks:
Screenshot 2024-08-26 at 11 50 48
Nodes:
Screenshot 2024-08-26 at 11 50 53

It looks like there is a scheduling slowdown during autoscaling even on already running nodes.
Our observations:

  1. There is no issue with fresh cluster.
    Here is how utilization looks on fresh cluster:
Screenshot 2024-08-23 at 19 09 01

Slowdown develops smoothly, we are trying to understand the exact reason, but all we can say it begins to manifest itself after ~1 day or ~500 jobs on cluster.

  1. It is probably related to runtime environment, as if I don't pass runtime_env = dict(py_modules=[pandas]) to ray.init, there is no issue.

  2. Nothing wrong in logs except lots of messages during autoscaling

Node 8927887737694498143 is not available for request {CPU: 10000} with critical resource utilization 1 based on local view {"total":{CPU: 300000, object_store_memory: 198586257400000, node:10.17.7.106: 10000, mem│
ory: 481036337150000}}, "available": {node:10.17.7.106: 10000, memory: 481036337150000, object_store_memory: 198586257400000}}, "labels":{"ray.io/node_id":"9b08defb38e6058b1b0830777ff3b32bbb13a96e2097f530ae6482d3",}, "is_draining": 0, "draining_deadline_timestamp_ms": -1}

We would really appreciate if anyone could provide any suggestions. I’m glad to provide any further information if needed.

Versions / Dependencies

I use ray==2.9.2. It's less pronounced on ray==2.34 but issue still exist.

Reproduction script

import ray
from tqdm import tqdm
import pandas
runtime_env = dict(py_modules=[pandas])
ray.init("ray:/...", runtime_env=runtime_env)


@ray.remote
def func():
    import time
    print("started")
    time.sleep(20)
    return 42

COUNT = 1000
tasks = [func.remote() for i in range(COUNT)]
with tqdm(total=COUNT) as pbar:
    while tasks:
        done, tasks = ray.wait(tasks, num_returns=len(tasks), timeout=1)
        pbar.update(len(done))

Issue Severity

High: It blocks me from completing my task, because after several days nodes become dramatically underutilized: only 10-50 CPUs from ~700 CPUs cluster are used.

@aljeshishe aljeshishe added bug Something that is supposed to be working; but isn't triage Needs triage (eg: priority, bug/not-bug, and owning component) labels Aug 27, 2024
@anyscalesam anyscalesam added the core Issues that should be addressed in Ray Core label Sep 3, 2024
@anyscalesam
Copy link
Contributor

Thanks - this is a really good bug report @aljeshishe ... out of curiosity are you able to workaround by spinning up a fresh Ray Cluster (or, put in another way, any reason not to)?

@jjyao on the attempt to repro...

@jjyao
Copy link
Collaborator

jjyao commented Sep 9, 2024

@aljeshishe it's probably best to set up a call to do a live debug when the cluster is in this bad state.

@anyscalesam
Copy link
Contributor

I can help coordinate some time here > @aljeshishe can you Slack me and Jiajun on Ray Slack and we can go from there?

@anyscalesam anyscalesam added the @external-author-action-required Alternate tag for PRs where the author doesn't have labeling permission. label Sep 9, 2024
@aljeshishe
Copy link
Author

Thanks - this is a really good bug report @aljeshishe ... out of curiosity are you able to workaround by spinning up a fresh Ray Cluster (or, put in another way, any reason not to)?

@jjyao on the attempt to repro...

Usually, I kill the docker container with head and then I do ray up(to avoid head instance recreation).
Now I have to do it almost every day.
Ok I'll contact you in slack later.

@rkooo567
Copy link
Contributor

It is probably related to runtime environment, as if I don't pass runtime_env = dict(py_modules=[pandas]) to ray.init, there is no issue.

This is super interesting. afaik, we should only download this once and reuse, but based on what you are saying, it seems like this cause issues

I am curious if you use "pip" instead of pymodules (runtime_env={"pip": ["pandas"]}), you have the same issue ? Or as a workaround, you can probably just add pandas to your docker images.

@kevin85421 kevin85421 self-assigned this Sep 24, 2024
@aljeshishe
Copy link
Author

@rkooo567
I've tried pip/working_dir/py_modules, same issue.

@jjyao finally found out how to reproduce the issue

To degrade cluster run:

import random
from pathlib import Path
import ray
import logging

logger = logging.getLogger(__name__)

def main():
    logging.basicConfig(level=logging.INFO, format="%(asctime)s %(levelname)s %(message)s")
    submodule_path = Path(__file__).parent / "submodule"
    submodule_path.mkdir(exist_ok=True)
    (submodule_path / "__init__.py").write_text(f"a = {random.random()}")
    
    ray.init("ray://...:10001", 
             runtime_env=dict(py_modules=[submodule_path])
             )
    @ray.remote(scheduling_strategy='SPREAD')
    def task():
        pass

    print(ray.get(task.remote()))

        
if __name__ == "__main__":
    main()

run this script 200 times to create 200 envs:

for i in {0..200}; do python run.py; done;

Now lets check degraded state:

from ray.util.queue import Queue
import time
import ray
import pandas
import logging


logger = logging.getLogger(__name__)
NUM_TASKS = 3000
SLEEP_TIME = 1

def main():
    logging.basicConfig(level=logging.INFO, format="%(asctime)s %(levelname)s %(message)s")

    ray.init("ray://...:10001", 
             runtime_env=dict(py_modules=[pandas])
             )
    q = Queue()
    @ray.remote(scheduling_strategy='SPREAD')
    def task():
        q.put(1)
        
        ts = time.time()
        while time.time() - ts < 1: pass
        
        return 42


    
    logger.info("scheduling tasks")
    refs = [task.remote() for _ in range(NUM_TASKS)]
    logger.info("Done")

    ts = time.time()

    while True:
        time.sleep(SLEEP_TIME)
        items = len(q.get_nowait_batch(q.size()))
        new_ts = time.time()
        refs += [task.remote() for _ in range(items)]
        
        elapsed = new_ts - ts
        scheduling_speed = items / elapsed
        availible_cpus = ray.available_resources().get("CPU", 0)
        cluster_cpus = ray.cluster_resources().get("CPU", 0)
        logger.info(f"CPUs: {availible_cpus=:.2f} {cluster_cpus=:.2f} {scheduling_speed=:.2f}")
        
        ts = new_ts

        
if __name__ == "__main__":
    main()

As you may see in script output, lots of environments degrade cluster, so scheduling almost stops for some time
Screenshot 2024-10-02 at 13 14 36

@aljeshishe
Copy link
Author

aljeshishe commented Oct 2, 2024

For comparison you may checkout scheduling on fresh cluster
Screenshot 2024-10-02 at 13 31 00
There is a slight 2-second pause.

In an ideal world, I expect no scheduling degradation at all.

@aljeshishe
Copy link
Author

@jjyao have you managed to reproduce ?

@anyscalesam
Copy link
Contributor

@kevin85421 did you get a chance to look into this deeper?

@DmitriGekhtman
Copy link
Contributor

Looks vaguely similar to #45373
I wonder if it's connected.

@jjyao jjyao added P0 Issues that should be fixed in short order and removed @external-author-action-required Alternate tag for PRs where the author doesn't have labeling permission. triage Needs triage (eg: priority, bug/not-bug, and owning component) labels Oct 28, 2024
@jjyao jjyao assigned rynewang and unassigned kevin85421 Nov 4, 2024
@aljeshishe
Copy link
Author

Hello, guys, is it going to be fixed?

@DmitriGekhtman
Copy link
Contributor

I noticed in the repro script that the job is submitted from outside the cluster using Ray Client (the service running at port 10001 on the head node). I've heard that this submission method is not well-supported (though perhaps this has changed, I'm not sure)

I'm curious -- Does the issue persist if you run the job directly on the head node, or use Ray Job Submission (the other way of submitting jobs, using the dashboard port 8265 by default.)

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 P0 Issues that should be fixed in short order
Projects
None yet
Development

No branches or pull requests

7 participants