-
Notifications
You must be signed in to change notification settings - Fork 6.2k
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 Core] After the ray job is finished, it will stably trigger resource leakage #49999
Comments
I'm able to repro. Thanks for reporting. |
@jjyao Thanks for your quick confirmation and looking forward to having this fixed! |
Hi @jjyao , it seems this problem is quite serious, is there any progress or update? |
@Moonquakes we figured out the root cause and will fix it soon. |
Hi @jjyao, I have also done some research on this issue, but have not made much progress. I am curious about the root cause of these phenomena. If it is convenient, can you explain it in more detail? Thank you! BTW, I noticed this PR (#50280), but there is no relevant description. Is it related to this issue? |
Hi @jjyao , sorry to bother you, but this problem really bothers our daily use. I would be very grateful if I could understand its root cause or get a quick fix. |
Hi @Moonquakes sorry for the late reply. The issue happens because raylet fails to detect the death of the driver immediately (it will eventually detect after hours) and that's because each of the task has unique resource (memory) requirement which makes raylet very slow during scheduling. In your real case, do you really need unique resource requirement for each task? |
@jjyao Thank you for your reply! Yes, in our scenario, different tasks need to specify different memory to prevent triggering OOM because the file sizes they process are very different. At the same time, we hope to use full resources as much as possible, so we will specify more fine-grained memory requirements. Are the above three problems all caused by this reason? Because I tested that it seems to also be related to ray.wait(num_returns=1). In addition, is there any quick fix for the situation you mentioned (For example, patch the judgment logic of raylet to detect whether the driver has ended first, but I am not sure where the code logic of this block is)? We can try it quickly after modification. |
Are you able to call ray.get() less frequently: only when all objects are ready? |
@jjyao I can try to modify the logic of this part, but in the actual code, there is downstream logic behind this part, so if it is changed to a one-time ray.get, it will have a certain impact on the overall time of the job, so I prefer to make a patch for this to ensure that the existing logic will not change too much. |
Yea that's just a mitigation. I'll get the actual fix out asap. |
@jjyao Thank you very much for your work! It would be a great help to us if it can be fixed! |
@Moonquakes thanks for the info, let me test that and see the behavior difference. Btw, one suggestion to mitigate the issue and also to improve the performance of this generally would be to use "buckets" for the memory requirements. Instead of having a unique requirement for every file, have a predefined set (1MiB, 5MiB, 10MiB, etc.) and then round up for each task. |
@edoakes Thanks for your feedback! At the same time, I tried your PR, and it seems that the resources can be recycled correctly now, but I don't understand why changing to calling |
@Moonquakes I will update the PR description to describe the underlying issue. Hoping to get this merged in the next couple of days :) |
@edoakes Thanks a lot! And It would be great to know the exact reason why the bug arose before this PR (e.g. where the logic slowed down to cause the worker recovery to be blocked, as it seems to be recovering normally after a while), and why it could be released immediately within ten seconds after the PR was added, was it taking a "fast track" to some process that wasn't being blocked? |
Basically the issue is there were many pending messages in the unix domain socket that the worker uses to communicate with the raylet. We don't get an error from the socket until the messages are drained. Typically these are drained fast but here because you're using so many unique scheduling classes, processing each message takes a long time and chews through CPU. The minor code change I made explicitly disconnects the client in the raylet and disregards future messages. This is the path we should've been going through the whole time. We then stop processing the expensive messages immediately when terminating the worker. |
Hi @edoakes , When executing the above script (about three minutes), this alarm message will be encountered. This seems to cause the stop job to fail. Is it related to the failure to receive socket information? |
Was this running on my PR branch or on master? I think it's likely because the raylet is so overloaded. Have you implemented my suggestion of bucketing the memory resource requirements? |
…51033) Currently, the worker fires-and-forgets a disconnect message to the raylet, then goes through its shutdown procedure and closes the socket. However, in #50812 we want to add detect for unexpected disconnects on the socket. With the current procedure the worker may send the disconnect and close its socket immediately, causing the raylet to detect the unexpected disconnect prior to processing the graceful disconnect message. This PR adds a `DisconnectClientReply` that the raylet will send once it has processed the disconnect message. The worker blocks until it receives this reply, then proceeds with the disconnect procedure and closes its socket. ## Related issue number Towards #49999 --------- Signed-off-by: Edward Oakes <ed.nmi.oakes@gmail.com>
Hi @edoakes, there seem to be a lot of scattered PRs related to this issue. Can you provide which PRs need to be applied to fix this issue? Also, can you introduce what logic has been updated to fix it? Thank you! |
@Moonquakes this PR is the one that fixes the issue: #50812 It depends on these:
You can test out the fix using the nightly wheels |
…ay-project#50812) Adds logic to periodically check for unexpected disconnects from worker processes and proactively disconnect them when it happens. Currently, we only mark a worker as disconnected once we've processed all messages from the socket and get the EOF. In some cases, we may process these messages very slowly or never (see linked issue). The disconnect detection is implemented by periodically `poll`ing for a `SIGHUP` on the file descriptor of all register workers' sockets every 1s (configurable). This is left unimplemented on Windows. ## Related issue number Closes ray-project#49999 --------- Signed-off-by: Edward Oakes <ed.nmi.oakes@gmail.com>
…ay-project#51033) Currently, the worker fires-and-forgets a disconnect message to the raylet, then goes through its shutdown procedure and closes the socket. However, in ray-project#50812 we want to add detect for unexpected disconnects on the socket. With the current procedure the worker may send the disconnect and close its socket immediately, causing the raylet to detect the unexpected disconnect prior to processing the graceful disconnect message. This PR adds a `DisconnectClientReply` that the raylet will send once it has processed the disconnect message. The worker blocks until it receives this reply, then proceeds with the disconnect procedure and closes its socket. ## Related issue number Towards ray-project#49999 --------- Signed-off-by: Edward Oakes <ed.nmi.oakes@gmail.com>
…ay-project#50812) Adds logic to periodically check for unexpected disconnects from worker processes and proactively disconnect them when it happens. Currently, we only mark a worker as disconnected once we've processed all messages from the socket and get the EOF. In some cases, we may process these messages very slowly or never (see linked issue). The disconnect detection is implemented by periodically `poll`ing for a `SIGHUP` on the file descriptor of all register workers' sockets every 1s (configurable). This is left unimplemented on Windows. ## Related issue number Closes ray-project#49999 --------- Signed-off-by: Edward Oakes <ed.nmi.oakes@gmail.com>
…ay-project#51033) Currently, the worker fires-and-forgets a disconnect message to the raylet, then goes through its shutdown procedure and closes the socket. However, in ray-project#50812 we want to add detect for unexpected disconnects on the socket. With the current procedure the worker may send the disconnect and close its socket immediately, causing the raylet to detect the unexpected disconnect prior to processing the graceful disconnect message. This PR adds a `DisconnectClientReply` that the raylet will send once it has processed the disconnect message. The worker blocks until it receives this reply, then proceeds with the disconnect procedure and closes its socket. ## Related issue number Towards ray-project#49999 --------- Signed-off-by: Edward Oakes <ed.nmi.oakes@gmail.com> Signed-off-by: Jay Chia <17691182+jaychia@users.noreply.github.com>
…ay-project#50812) Adds logic to periodically check for unexpected disconnects from worker processes and proactively disconnect them when it happens. Currently, we only mark a worker as disconnected once we've processed all messages from the socket and get the EOF. In some cases, we may process these messages very slowly or never (see linked issue). The disconnect detection is implemented by periodically `poll`ing for a `SIGHUP` on the file descriptor of all register workers' sockets every 1s (configurable). This is left unimplemented on Windows. ## Related issue number Closes ray-project#49999 --------- Signed-off-by: Edward Oakes <ed.nmi.oakes@gmail.com> Signed-off-by: Jay Chia <17691182+jaychia@users.noreply.github.com>
…ay-project#51033) Currently, the worker fires-and-forgets a disconnect message to the raylet, then goes through its shutdown procedure and closes the socket. However, in ray-project#50812 we want to add detect for unexpected disconnects on the socket. With the current procedure the worker may send the disconnect and close its socket immediately, causing the raylet to detect the unexpected disconnect prior to processing the graceful disconnect message. This PR adds a `DisconnectClientReply` that the raylet will send once it has processed the disconnect message. The worker blocks until it receives this reply, then proceeds with the disconnect procedure and closes its socket. ## Related issue number Towards ray-project#49999 --------- Signed-off-by: Edward Oakes <ed.nmi.oakes@gmail.com> Signed-off-by: Jay Chia <17691182+jaychia@users.noreply.github.com>
…ay-project#50812) Adds logic to periodically check for unexpected disconnects from worker processes and proactively disconnect them when it happens. Currently, we only mark a worker as disconnected once we've processed all messages from the socket and get the EOF. In some cases, we may process these messages very slowly or never (see linked issue). The disconnect detection is implemented by periodically `poll`ing for a `SIGHUP` on the file descriptor of all register workers' sockets every 1s (configurable). This is left unimplemented on Windows. ## Related issue number Closes ray-project#49999 --------- Signed-off-by: Edward Oakes <ed.nmi.oakes@gmail.com> Signed-off-by: Jay Chia <17691182+jaychia@users.noreply.github.com>
…ay-project#51033) Currently, the worker fires-and-forgets a disconnect message to the raylet, then goes through its shutdown procedure and closes the socket. However, in ray-project#50812 we want to add detect for unexpected disconnects on the socket. With the current procedure the worker may send the disconnect and close its socket immediately, causing the raylet to detect the unexpected disconnect prior to processing the graceful disconnect message. This PR adds a `DisconnectClientReply` that the raylet will send once it has processed the disconnect message. The worker blocks until it receives this reply, then proceeds with the disconnect procedure and closes its socket. ## Related issue number Towards ray-project#49999 --------- Signed-off-by: Edward Oakes <ed.nmi.oakes@gmail.com>
…ay-project#50812) Adds logic to periodically check for unexpected disconnects from worker processes and proactively disconnect them when it happens. Currently, we only mark a worker as disconnected once we've processed all messages from the socket and get the EOF. In some cases, we may process these messages very slowly or never (see linked issue). The disconnect detection is implemented by periodically `poll`ing for a `SIGHUP` on the file descriptor of all register workers' sockets every 1s (configurable). This is left unimplemented on Windows. ## Related issue number Closes ray-project#49999 --------- Signed-off-by: Edward Oakes <ed.nmi.oakes@gmail.com>
What happened + What you expected to happen
When I was running a task, I found that a type of job can stably trigger the resource leakage problem after the ray job ends. I tested the code part to the most simplified version and provided it below. The Ray Cluster used in the test is configured as a 128c896g worker node, minWorkerNum is 0, maxWorkerNum is 1, and no environment variables are configured. After the job runs for one minute, you need to manually trigger the
ray job stop
to trigger the exception.This task will have the following abnormal phenomena, I think it is worth your in-depth investigation to see what bug is triggered.
These three problems will occasionally be triggered separately in some other jobs, but in this given code, they can be almost stably triggered at the same time. Please take a look at where the problem occurs. Thank you!
Versions / Dependencies
Ray v2.40.0
Kuberay v1.2.2
Reproduction script
Submit job:
ray job submit --address=http://localhost:8265 --working-dir=. -- python3 test_resource_leak.py
And execute
ray job stop 02000000 --address=http://localhost:8265
after the ray job runs for one minute.Issue Severity
High: It blocks me from completing my task.
The text was updated successfully, but these errors were encountered: