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

Use subreaper to kill unowned subprocesses in raylet. #42992

Merged
merged 47 commits into from Mar 9, 2024

Conversation

rynewang
Copy link
Contributor

@rynewang rynewang commented Feb 5, 2024

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 direct child procs on core worker exit (#33976), but if a worker crashed (e.g. sigkill'd), or there are grandchild processes those processes leak. They may still hold valuable resources e.g. GPU memory.

This patches adds a feature to kill any recursive children from a core worker on its death. It's gated by a flag RAY_kill_child_processes_on_worker_exit_with_raylet_subreaper default disabled.

Once enabled:

  • raylet as Linux subreaper.
  • raylet tracks "known subprocesses" it spawned.
  • raylet auto-kills unknown children every 10s.
  • core worker as Linux subreaper, no auto-kills.

so that

  • if a core worker is running, a (grand)child dies -> all other (grand)children keeps running
  • if a core worker dies -> all (grand)children killed by raylet.

To avoid zombies, sets core worker to auto reap zombies by ignoring SIGCHLD. The raylet already reaps zombies, but now it also removes the dead children in the "known subprocesses".

Added a linux only unit test, and a doc page.

Fixes #42861, #26118

Fixes ray-project#42861

Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
@rynewang
Copy link
Contributor Author

rynewang commented Feb 5, 2024

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)
  }
}

Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
// Register a signal handler for the given signal.
// The handler will be called with the signal_set and the error code.
// After the handler is called, the signal will be re-registered.
void RegisterSignalHandlerLoop(boost::asio::signal_set &signals,
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is it possible for the reference to be invalid during shutdown

}

void SigchldHandlerPlain(const boost::system::error_code &error, int signal_number) {
if (!error) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what does this case mean?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1. Can you comment here? Also it is no-op if it is an error. Is this the legit behavior?

if (!error) {
int status;
pid_t pid;
while ((pid = waitpid(-1, &status, WNOHANG)) > 0) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

seems we should check for EINVAL to fail loudly.

Comment on lines 111 to 118
if (WIFEXITED(status)) {
RAY_LOG(INFO) << "Child process " << pid << " exited with status "
<< WEXITSTATUS(status);
} else if (WIFSIGNALED(status)) {
RAY_LOG(INFO) << "Child process " << pid << " exited from signal "
<< WTERMSIG(status);
}
removeOwnedChild(pid);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does this replace existing raylet-side process cleanup in MacOS / when the new code path is disabled on linux? I am not familiar with the double forking stuff in Ray but if it's meant to replace it then will need to think a bit to make sure edge cases are handled

// Use pipe to track process lifetime. (The pipe closes when process terminates.)
fd = pipefds[0];
if (pid == -1) {
ec = std::error_code(errno, std::system_category());
}
#endif
addOwnedChild(pid);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You need a signal block between process creation and adding it to the owned set. Otherwise there is a race where the process dies before it is added to the owned set. I'll leave to you whether this is something that should be fixed now or not.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

(maybe boost magic solves this for you)

// Use pipe to track process lifetime. (The pipe closes when process terminates.)
fd = pipefds[0];
if (pid == -1) {
ec = std::error_code(errno, std::system_category());
}
#endif
addOwnedChild(pid);
return ProcessFD(pid, fd);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

seems the constructor is public, what happens if someone creates a process using another mechanism? tricky

Comment on lines 111 to 117
if (WIFEXITED(status)) {
RAY_LOG(INFO) << "Child process " << pid << " exited with status "
<< WEXITSTATUS(status);
} else if (WIFSIGNALED(status)) {
RAY_LOG(INFO) << "Child process " << pid << " exited from signal "
<< WTERMSIG(status);
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why do we need the notion of ownership at all? seems this will only run for dead child procs which need to be reaped anyways

@rkooo567
Copy link
Contributor

rkooo567 commented Feb 6, 2024

I will review this by today!

Copy link
Contributor

@rkooo567 rkooo567 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Current status: it is not working yet. Waiting for it to work

@rkooo567 rkooo567 added the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Feb 8, 2024
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
@rynewang
Copy link
Contributor Author

Updated, now the subreaper test works. i.e. if the worker is dead, its subprocesses are sigkilled.

@rynewang rynewang removed the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Feb 12, 2024
@rynewang
Copy link
Contributor Author

Note:

  1. we may not want a hard sigkill at first. We can do something like "sigterm, after 5s then sigkill".
  2. 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.

Copy link
Contributor

@rkooo567 rkooo567 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In general, it lgtm. But I feel like we don't need SigchldHandlerPlain for this PR? I think it is the best we only handle killing unowned children in this PR, and replace stdin read -> wait on child procs in another PR. wait on child procs will also improve observabiilty.

we may not want a hard sigkill at first. We can do something like "sigterm, after 5s then sigkill".

SGTM

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.

Do we ever do this from raylet though? Besides, agreed to keep decouple here.

@@ -71,6 +72,14 @@ DEFINE_string(session_dir, "", "The path of this ray session directory.");
DEFINE_string(log_dir, "", "The path of the dir where log files are created.");
DEFINE_string(resource_dir, "", "The path of this ray resource directory.");
DEFINE_int32(ray_debugger_external, 0, "Make Ray debugger externally accessible.");
// TODO(ryw): maybe instead of a new flag, we use kill_child_processes_on_worker_exit ?
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this makes sense

// become zombies instead of dying gracefully.
signal(SIGCHLD, SIG_IGN);
#endif
// No need to treat SIGCHLD as it's handled in raylet main.cc.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What's going to happen to this behavior?>

  // Ignore SIGCHLD signals. If we don't do this, then worker processes will
  // become zombies instead of dying gracefully.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

also do you have any guess when this happens? This seems like an old behavior (when workers don't fate share with raylet)?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@@ -123,7 +244,6 @@ class ProcessFD {
}
#ifdef _WIN32

(void)decouple; // Windows doesn't require anything particular for decoupling.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can you explain me what this argument was for?


void addOwnedChild(pid_t pid) {
#ifdef __linux__
std::lock_guard<std::mutex> guard(m);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit, but just use absl::MutexLock lock(&mutex_);? (we use this for locks)

// Set this process as a subreaper.
void SetThisProcessAsSubreaper() {
if (prctl(PR_SET_CHILD_SUBREAPER, 1) == -1) {
perror("prctl");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Consider RAY_LOG(FATAL) here?

});
}

void SigchldHandlerPlain(const boost::system::error_code &error, int signal_number) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it possible for owned children (the workers), we just handle it in the same way as before? I think it is probably unnecessary change for this PR?

#ifdef __linux__

void KillUnownedChildren() {
auto maybe_child_procs = GetAllProcsWithPpid(GetPID());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how slow is this?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

1 file read per process in the current linux. so not very fast. Fortunately we are not blocking anything here.


// Enumerating child procs is not supported on this platform.
if (!maybe_child_procs) {
RAY_LOG(WARNING)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

maybe this should just exit here and we shouldn't even start this function if it is not linux

std::vector<pid_t> to_kill;
to_kill.reserve(maybe_child_procs->size());
{
std::lock_guard<std::mutex> guard(m);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why do we need a lock btw? is it happening in a different thread?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if 1 thread is spawning a child process, while another thread is gathering this owned children list.

RAY_LOG(INFO) << "Killing leaked child process " << pid;
auto error = KillProc(pid);
if (error) {
RAY_LOG(ERROR) << "Failed to kill leaked child process " << pid << " with error "
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Use WARNING here. ERROR will be printed to the user driver.

@rkooo567
Copy link
Contributor

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).

Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
@rkooo567
Copy link
Contributor

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

Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
@rkooo567
Copy link
Contributor

rkooo567 commented Mar 1, 2024

please remove the label when it is ready to review again! I will take a look asap

rynewang and others added 5 commits March 1, 2024 16:55
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
@rynewang
Copy link
Contributor Author

rynewang commented Mar 4, 2024

Only failure is a linkcheck which is in rllib, not relevant. Should be ok now.

@rynewang rynewang removed the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Mar 4, 2024
@fishbone fishbone self-assigned this Mar 8, 2024

On non-Linux platforms, user-spawned process is not controlled by Ray. The user is responsible for managing the lifetime of the child processes. If the parent Ray worker process dies, the child processes will continue to run.


Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What we provide is more like a last resort. We should mention that ideally child processes should catch the death of the parent and self-exit so this feature doesn't need to be turned on.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since I reversed the default sigchld handler in core worker to reap-all-dead, users need to toggle it back to really do the signal handling. See the new ⚠️ Caution section.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh, I mean not the signal handle on the parent side but on the child side: https://stackoverflow.com/questions/284325/how-to-make-child-process-die-after-parent-exits.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Understood. Added Note in the last section in the intro.


- ``RAY_kill_child_processes_on_worker_exit`` (default ``true``): Only works on Linux. If true, core worker kills all subprocesses on exit. This won't work if the core worker crashed or was killed by a signal. If a process is created as a daemon, e.g. double forked, it will not be killed by this mechanism.

- ``RAY_kill_child_processes_on_worker_exit_with_raylet_subreaper`` (default ``false``): Only works on Linux greater than or equal to 3.4. If true, Raylet kills any subprocesses that were spawned by the core worker after the core worker exits. This works even if the core worker crashed or was killed by a signal. Even if a process is created as a daemon, e.g. double forked, it will still be killed by this mechanism. The killing happens within 10 seconds after the core worker death.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What's the plan to turn this on by default and remove the other one?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we should never remove the flag as the behavior is kind of inconsistent with pure python

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

maybe better way is to allow this per task/actor eventually (if there's a request)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I mean removing the other RAY_kill_child_processes_on_worker_exit flag and the related code since we don't need two cleanup mechanisms.

ray.init(_system_config={"kill_child_processes_on_worker_exit_with_raylet_subreaper":True})


⚠️ Caution: Core worker needs to reap zombies
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it more of an issue if the core work is long live since otherwise they will be killed by raylet.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, long running driver for example

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I can reverse the default on core worker: it defaults to reap all zombies, and if users want to call waitpid they can set the signal handler to default (no waiting).

Copy link
Contributor

@rkooo567 rkooo567 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Generally LGTM!

PR description still relevant? can you update it?


We have 2 environment variables to handle subprocess killing on core worker exit:

- ``RAY_kill_child_processes_on_worker_exit`` (default ``true``): Only works on Linux. If true, core worker kills all subprocesses on exit. This won't work if the core worker crashed or was killed by a signal. If a process is created as a daemon, e.g. double forked, it will not be killed by this mechanism.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

e.g. double forked, it will not be killed by this mechanism.

I don't think people can understand this. Either remove it or add more details?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

changed to "grandchild processes" which should be more understandable.

Lifetimes of a User-Spawn Process
=================================

To avoid leaking user-spawned processes, Ray provides mechanisms to kill all user-spawned processes when a worker that starts it exits. This feature prevents GPU memory leaks from child processes (e.g., torch).
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's improve this a bit.

When you spawns child processes from Ray workers, you are responsible for managing the lifetime of child processes. However, it is not always possible, especially when worker crashes and child processes are spawned from libraries (torch dataloader).

To avoid leak...

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done


When the feature is enabled, the core worker process becomes a subreaper (see the next section), meaning there can be some grandchildren processes that are reparented to the core worker process. If these processes exit, core worker needs to reap them to avoid zombies, even though they are not spawn by core worker. If core worker does not reap them, the zombies will accumulate and eventually cause the system to run out of resources like memory.

You can add this code to the Ray Actors or Tasks to reap zombies, if you choose to enable the feature:
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is this tested? can you add a test?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

so it is the default behavior now right? Can you add info here?

⚠️ Caution: Core worker needs to reap zombies
----------------------------------------------

When the feature is enabled, the core worker process becomes a subreaper (see the next section), meaning there can be some grandchildren processes that are reparented to the core worker process. If these processes exit, core worker needs to reap them to avoid zombies, even though they are not spawn by core worker. If core worker does not reap them, the zombies will accumulate and eventually cause the system to run out of resources like memory.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should not mention "core worker". It should be just "worker"

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

renamed all core worker to the worker

sys.platform != "linux",
reason="Orphan process killing only works on Linux.",
)
def test_daemon_processes_not_killed_until_actor_dead(enable_subreaper, shutdown_only):
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you add a test where adding signal can reap grandchildren?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

no need, reversed to core worker to default-reap all children.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

got it! please make sure you added a test case for reap. (and document how to diable automatic reap)

// The handler will be called with the signal_set and the error code.
// After the handler is called, the signal will be re-registered.
// The callback keeps a reference of the shared ptr to make sure it's not destroyed.
void RegisterSignalHandlerLoop(std::shared_ptr<boost::asio::signal_set> signals,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is this necessary? no better API from asio that does this automatically?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not any I know of. I did a quick search in github and did not find a good one

pid_t pid;
// Reaps any children that have exited. WNOHANG makes waitpid non-blocking and returns
// 0 if there's no zombie children.
while ((pid = waitpid(-1, &status, WNOHANG)) > 0) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this should be non-blocking right? (since proc should have already dead)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Non-blocking, yes. It's controlled by the WNOHANG. If there's no child dead already, it returns -1

void SigchldHandlerReapZombieAndRemoveKnownChildren(
const boost::system::error_code &error, int signal_number) {
if (error) {
RAY_LOG(WARNING) << "Error in SIGCHLD handler: " << error.message();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should we just return here? what kind of error can happen?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The only error is "signal set cancelled" as boost::asio::error::operation_aborted but we won't ever cancel it.

https://www.boost.org/doc/libs/1_84_0/doc/html/boost_asio/reference/basic_signal_set/async_wait.html

It does not hurt to waitpid anyway.

Copy link
Contributor

@rkooo567 rkooo567 Mar 8, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can you comment in code? Also comment in the waitpid that it is non blocking (so reader can know although it is due to abort, it is okay)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

added comments for the error and WNOHANG.

// TODO: Checking PIDs is not 100% reliable because of PID recycling. If we find issues
// later due to this, we can use pidfd.
void KillUnknownChildren() {
auto to_kill =
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Q: Is this automatically optimized by compiler?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it can do guaranteed copy elision but not 100% sure

create_child_fn();
return;
}
absl::MutexLock lock(&m_);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we don't need a lock when we create a child proc?

pid_t pid = create_child_fn();
absl::MutexLock lock(&m_);
children_.insert(pid);

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We don't want the creation of a pid (in procfs) happen when we are reading the procfs for pids, I assume

Copy link
Contributor Author

@rynewang rynewang Mar 8, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

e.g. if 2 racing threads:

  1. create process
    a. then add pid to the known list
  2. kill unknown children

If the order is 1 -> 2 -> a, the new process is killed which is not what we want. We have to make 1 and a be atomic on the list.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why call the create child fn inside the AddKnownChild function? It should be cleaner if the creation and adding is decoupled. Like, in the caller:

child = create()
tracker.addknownchild(child)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

see the race condition I gave: if a killing happens between create() and addknownchild it may kill the newborn child process.

Co-authored-by: SangBin Cho <rkooo567@gmail.com>
Signed-off-by: Ruiyang Wang <56065503+rynewang@users.noreply.github.com>
// until this worker exits and raylet reaps it.
if (SetThisProcessAsSubreaper()) {
RAY_LOG(INFO) << "Set this core_worker process as subreaper: " << getpid();
} else {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When kill_child_processes_on_worker_exit_with_raylet_subreaper is set, why do we still need to set the core worker ? The raylet should just work as the reaper I think?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

See https://github.com/ray-project/ray/pull/42992/files#diff-f6c49babcc6278c29b15f311f28011254c7e7d20a0ff06b03a1ac17a6fe352bbR103

tldr: if a child of core worker spawns a grandchild and dies, the grandchild is reparented to the nearest subreaper. But we don't want it to be killed (since the core worker is still alive), so we set the core worker as a subreaper as well to avoid it be noticed and killed by raylet.

Copy link
Contributor

@fishbone fishbone Mar 8, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, let me know if I understand it wrong:

  • if the core worker is dead => the grand child reparented to the raylet
  • if the raylet is dead => the core worker is still alive => later it'll kill itself => before it exits, it'll kill all the children?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it handles this case:

raylet -> core worker -> A -> B

then

A exited, we don't want to kill B.

Now, B is reparented to core worker who does NOT kill B, so it's good. If we don't do that, B'd be reparented to raylet and be killed, which is what we don't want

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks!

Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
// later due to this, we can use pidfd.
void KillUnknownChildren() {
auto to_kill =
KnownChildrenTracker::instance().ListUnknownChildren([]() -> std::vector<pid_t> {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we pass the lambda to the function instead of just move the lambda to the function body?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is to ensure the KnownChildrenTracker only cares about the lock on the hash set, and it separates from the children finding code.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, I doubt the separation making things simpler, but I'm fine with it.

// not allow a macro invocation (#ifdef) in another macro invocation (RAY_CHECK_OK),
// so we have to put it here.
auto enable_subreaper = [&]() {
#ifdef __linux__
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This probably is not an issue. Maybe we can put the linux into the SetThisProcessAsSubreaper.

For the platform specific code, the lower level we put it, the better.

Copy link
Contributor

@fishbone fishbone left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Overall it looks good.

Question is that, should we just turn it on by default? At least let raylet be the reaper?

The lifecycle of subprocess now becomes complicated, in different cases, the behavior is different.

IMO, if we just turned it on by default for most cases, and let the user know this might be better. Some special flag in remote options to turn it off for corner cases.

@rkooo567
Copy link
Contributor

rkooo567 commented Mar 8, 2024

@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.

Copy link
Contributor

@rkooo567 rkooo567 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM if tests pass. Let's make sure the core worker reaping grand children is the default behavior

@rkooo567
Copy link
Contributor

rkooo567 commented Mar 8, 2024

also before merge, let's make sure to run mac test/window test/release test

Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
@jjyao
Copy link
Contributor

jjyao commented Mar 9, 2024

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.

@jjyao jjyao merged commit 944bfe7 into ray-project:master Mar 9, 2024
9 checks passed
@rynewang rynewang deleted the raylet-subreaper branch March 9, 2024 01:55
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[core] subprocess leaks if Ray worker crashes
6 participants