ray
ray copied to clipboard
Use subreaper to kill unowned subprocesses in raylet.
Currently when user code spawns subprocess (from core worker), we don't have a good way to track them. We do best effort to kill any child procs on core worker exit, but if a worker crashed (e.g. sigkill'd) those processes leak. They may still hold valuable resources e.g. GPU memory.
This patches adds raylet as a Linux subreaper, so any worker crash makes subparents reparented to raylet. Then in raylet sigchld we kill them. We use a mutex protected set of pids to bookkeep what pids we recognize from Process ctor and Process::spawn, and in sigchld we compare all pids with ppid=this process and kill the extra ones.
To make this work, the double fork stuff is removed. They were used to save the hussle of writing a sigchld handler but here we have them.
Added a linux only unit test.
DO NOT SUBMIT: think if this is robust, e.g. will we accidentally kill creating subprocesses not yet bookkeeped? pid wrap around? shall we use the existing flag kill_child_processes_on_worker_exit or to create a new one?
Fixes #42861
removeOwnedChild needs a redo. Now we call it in ProcessFD dtor. However there are times we deallocate ProcessFD and does not track the process anymore, yet we don't want to kill it immediately either, e.g. when you spawn a one time util script. So we really need to track exit of those processes, i.e. in sigchld handler.
while (waitpid(&pid)) {
if pid in children {
children.remove(pid)
}
}
I will review this by today!
Updated, now the subreaper test works. i.e. if the worker is dead, its subprocesses are sigkilled.
Note:
- we may not want a hard sigkill at first. We can do something like "sigterm, after 5s then sigkill".
- I am wondering how waterproof is this approach. If we spawn a one time command (
bash start_something.sh
) which spawns a tool, will it be killed? Maybe we can bring back the "decouple" arg and only care about the "coupled" processes, i.e. workers.
btw to be more clear about feedback (not 100% sure if it is possible);
- Keep everything (decouple or how we track child process' health) as it is.
- Ignore sigchld from owned children
- Only handle sigchld from unonwned children and sigkill.
- We can remove owned children pid when a worker is killed (there must be some sort of hook here)
- Make sure things are logged properly with pid with INFO.
- The behavior should be clearly specified from core doc.
- Probably we can remove parent core worker killing child workers and convert it to this mechanism?
Some other comments;
- Do you think we need a way to exclude subprocesses from being killed? E.g., if an actor start a new job (.sh file) and exits, is there a way to not kill it? My guess is it is probably not a requirement given we already kill child procs and no one complained (meaning no regression)
- I wonder if we want to do this for subprocesses started from agent.py. Maybe it is okay (because agent.py fate share with raylet).
Btw, can we accelerate the progress of this PR as the branch cut is coming? We'd like to merge this by this week to meet 2/29
Q: all comments ^ are addressed? Also, there seems to be some premerge failures
addressed
decouple
Intention of the decouple was to make the spawned process be reparented to 1 (init), however if we have subreaper the double forked subprocess will be reparented to raylet so it won't work.
how we track child process' health
Yes I'm keeping it
Only handle sigchld for unowned children
sigchld actually is triggered from the direct children death and we need to take care of all children deaths, which we do via waitpid
. For the killing, yes we kill onowned children only.
We can remove owned children pid when a worker is killed (there must be some sort of hook here)
Right, we remove owned children in waitpid
where Linux tells us the child is gone.
Make sure things are logged properly with pid with INFO.
Yes we have the logs on child exited and child killed.
Probably we can remove parent core worker killing child workers and convert it to this mechanism?
Maybe. but let's at least keep it in this PR.
Not addressed
The behavior should be clearly specified from core doc.
ok, will write doc in this PR
premerge
It's strange, can't repro on my linux devbox. will fix
Lmk when it is ready again. Per private discussion
- we will make both raylet and worker subreaper
- this means granchildren can be zomebie in some circumstances. we should document this well
- off by default
- timer based approach
In spirit of not breaking anything, set the flag to default=false and added back decouple
.
please remove the label when it is ready to review again! I will take a look asap
Only failure is a linkcheck which is in rllib, not relevant. Should be ok now.
@fishbone we will turn it on by default. It is off because it is merged in the last minute and a bit of breaking change.
also before merge, let's make sure to run mac test/window test/release test
I'll merge this for now. @angelinalg could you still review the doc related changes and we will address comments in the follow up PR.