-
Notifications
You must be signed in to change notification settings - Fork 5.7k
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] Autoscaler kills working nodes unexpectedly #46492
Comments
I have tried autoscaler v2 and the bug did go away, but it introduced other new problems, based on the fact that development of autoscaler v2 is currently on hold, so I'm wondering if there's a better solution? |
Thanks for reproing on ASv2 ... we'll take a look at this @yx367563 |
@anyscalesam Or I would also be happy to fix this issue in autoscaler v1. Where can I get the documentation for autoscaler v1? I feel it is difficult to find the problem just by reading the source code. Or you can share with me the possible reasons you can think of. |
@yx367563 can you provide a more detailed repro setup script including the scaling down part? We plan to repro it and try to fix. Expected behavior: when a task is assigned to a node, in state "waiting for resources", and the node dead. The task should be reassigned to other nodes, and here it looks like they are marked dead. |
@kevin85421 can you try to repro this with @yx367563 ? |
Of course, I set up a single worker with minWorkerNum of 2, maxWorkerNum of 1000, each worker requires 8 cpu, and idleTimeoutSeconds of 60. The cluster can only provide 24 workers at most, so if the above code is executed, at most 24 tasks can be executed at the same time, and the rest of the tasks will be in pending state. This bug is triggered almost every time the last tasks are scheduled and scale-down occurs. cc @rynewang @kevin85421 |
@rynewang is P1 right for this; (OSS AS killing nodes unexpectedly feels like an important issue). |
Additional discovery: When using |
What other issues come up when ASv2 is turne don @yx367563 ? |
@anyscalesam You can refer to ray-project/kuberay#2223. |
Oh! I seem to have found the root cause! In the code snippet above, the outside_ray_task takes up 1 cpu, the inside_ray_task takes up 8 cpus, and there are only 8 cpus on a single worker node, but in reality, the outside_ray_task and a certain inside_ray_task are assigned to a same worker node, resulting in this node being marked as idle after the inside_ray_task on this node completes. I've found that in the case of nested, no matter how much cpu the outer task declares it needs, it will run on the same node as a certain inside task, which should not be expected. cc @anyscalesam @rynewang @kevin85421 |
If I set the memory requirement of the outer task to 1000 it runs fine, there should be a problem with the cpu setting of the outer task |
It seems that when ray.get is called, the CPU resources requested on the current node will be temporarily released, so that other tasks can be scheduled. This design is reasonable, but it will cause the autoscaler to kill such nodes by mistake. Is there any solution? |
Nice going finding the root cause! This is one of Ray's classic sharp edges -- logical CPU resources of the parent task are released while the child is running, but other resources are not released. It sort of makes sense, in that physical CPU is not consumed while the parent is waiting, whereas some other resources (like memory) are still blocked. But then you get some funny edge cases, like this one. Looking forward to the fix! |
We have recently identified the root cause of the issue and developed a fully reproducible script. We would like to contribute to this issue. However, there may be multiple potential solutions, and it is a high impact issue, so I would like to know if you can provide any guidance. The root cause is related to a nested setting: when a child task, located on the same node as its parent task, completes, and no new tasks are scheduled on that node, the node is removed after the idle timeout. As a result, unfinished child tasks fail. |
Way to go to go deep here @mimiliaogo > so so your expected that the outside tasks persists until all of it's inked inside tasks (on other worker nodes) complete? can you link the repro script on this issue here? |
This is the repro script: https://pastecode.io/s/wozizu6u It's reasonable that the parent task shouldn't occupy CPU resources, but its state (memory,... ) should be preserved to other active nodes before removing that node during the downscaling. |
I will schedule a meeting with @mimiliaogo to discuss this issue. |
What happened + What you expected to happen
I have found that Ray autoscaler sometimes mistakenly kills some nodes that are working. My scenario is that 400 Ray Tasks are submitted at the same time, but not all of them can be allocated resources at the beginning, and I find that almost every time I start scaler down at the final stage, I get an error report ray.exceptions.NodeDiedError: Task failed due to the node dying.
The reason I have troubleshooted so far is that the kill of a working node causes an exception, and then the whole job fails. I am currently using Ray 2.23.0, Kuberay 1.0.0, without autoscaler v2 enabled.
Since the development of autoscaler v2 has been temporarily suspended, I'd like to ask if there is any solution, or if I can find out what is causing this and what the code logic is?
Versions / Dependencies
Ray 2.23.0
Kuberay 1.0.0
Reproduction script
You can reproduce my problem with the following code, be careful not to make all tasks executable at the same time, there needs to be a task in pending for the bug to be triggered. looking forward to your replies!
Issue Severity
High: It blocks me from completing my task.
The text was updated successfully, but these errors were encountered: