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

[CoreWorker] Partially address Ray child process leaks by killing all child processes in the CoreWorker shutdown sequence. #33976

Merged
merged 18 commits into from
Apr 7, 2023

Conversation

cadedaniel
Copy link
Member

@cadedaniel cadedaniel commented Mar 31, 2023

TL;DR

We kill all child processes when a Ray worker process exits. This addresses process leaks that caused GPU OOM errors in #31451. There is some risk to this PR, particularly if Ray users rely on Ray's existing behavior of leaking processes. We don't know of any such user, but we provide a new flag RAY_kill_child_processes_on_worker_exit to provide a workaround in case someone is impacted.

Motivation

In #31451 we find that Ray workers can "leak" child processes in certain cases. This was manifest as a GPU OOM because a leaked worker child process kept a CUDA context alive, which kept the reserved GPU memory allocated. Future Ray tasks/actors could no longer use the full GPU memory until users ran ray stop. This is surprising to users because they don't expect Ray to leave their cluster in a bad state.

Thus, Ray needs some way to ensure that reasonably well-behaved code doesn't cause resource starvation due to leaked processes.

Minimal repro (from @matthewdeng #31451 (comment))

After Ctrl+C, this will leak the child process. If that child process were to hold GPU memory, future workloads on the same machine would not be able to use the memory.

import ray
import time
from multiprocessing import Process

@ray.remote
class MyActor:

    def run(self):
       p = Process(target = lambda: time.sleep(1000), daemon=True)
       p.start()
       p.join()

actor = MyActor.remote()
ray.get(actor.run.remote())

Root causes

Root cause 1

This bug is caused by a bug in the cleanup flow for worker processes, particularly actors. We set force_kill to true when killing actors that go out-of-scope. This effectuates an immediate termination of the worker process, such that atexit handlers and destructors are not guaranteed to be called. In particular, multiprocessing.Process installs an atexit handler which terminates all known daemon child processes.

Because of this bug in our cleanup logic, multiprocessing.Process is unable to clean up as it would in a vanilla Python interpreter, and we are left with a leak. Notably, this bug could strike any case where destructors/atexit handlers clean up machine state.

We should fix this bug, filed here #34124.

Root cause 2

Even if we fix the root cause of the leak, it will still be possible for buggy code to leak processes which affect the state of the cluster. For example, someone could intentionally double-fork and get a grandchild process reparented to the init process. Malicious intent is not required; bugs in user code or libraries could achieve the same thing.

This is a bad experience for Ray users expecting a native Python experience because it is relatively very difficult to clean up an entire cluster compared to cleaning up leaked state on a single machine as one would with a buggy vanilla-Python program. Thus, Ray should have some isolation guarantees so that buggy tasks or actors do not break a cluster to the point of requiring manual intervention.

Possible fix implementations

Root Cause 1

We should improve the cleanup flow for actors so that destructors and atexit handlers can run as they do in a vanilla Python program. We attempted this fix before the 2.4 release but feel that it is too risky to merge at this late stage in the release. Following up here #34124.

Root Cause 2

See this thread for discussion on potential solutions for Root Cause 2.

The cleanest solution is to configure the raylet to be a Linux child subreaper for all Ray worker processes and their descendants. The raylet would then periodically scan its immediate children and kill any it does not recognize as a worker.

A partial solution to this problem is to have the CoreWorker kill its children during the shutdown sequence. This handles the user-reported bug where the process leaks during a Ctrl+C, but does not handle leaks originating from worker crashes.

We decided to go with the partial solution, mainly because due to timing around the release and us not discovering Root Cause 1 or the PR_SET_CHILD_SUBREAPER until late in the 2.4 release process. We will follow up with a better solution to Root Cause 2, see #34125 for tracking.

Testing

I reproduce the minimal process leak case provided above and verify that all child processes (whether they're started by actors or tasks) are terminated upon sending Ctrl+C to the driver.

Closes

Closes #31451

Remaining items:

  • Automated testing
  • Clean up names / add comments

src/ray/core_worker/core_worker.cc Outdated Show resolved Hide resolved
@@ -704,6 +704,35 @@ void CoreWorker::Disconnect(
}
}

void CoreWorker::KillLeakedProcs() {
Copy link
Member Author

Choose a reason for hiding this comment

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

I call this in the CoreWorker. There are cases where we will still leak processes, like if the worker process segfaults. To handle this, we should keep track of child processes of the workers in the raylet. I am intentionally not doing that now to 1) get validation on this approach and 2) get a faster fix in because that kind of resource tracking is a larger change.

Note that even if we track child processes in the raylet, we'll still need code here in case the raylet dies.

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 is one of those cases where you do want an e2e correct solution on the first try. I don't think it makes sense to invest effort on a solution that isn't fully correct to this extent..

Copy link
Contributor

Choose a reason for hiding this comment

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

What was the result of the investigation around the raylet killing the reparented child processes as the process group leader?

Copy link
Member Author

Choose a reason for hiding this comment

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

What was the result of the investigation around the raylet killing the reparented child processes as the process group leader?

Killing the reparented child processes by process group was not part of the investigation because they are in the raylet's process group (so all workers and any child processes would also get killed).

As for killing the reparented child processes, I assumed that they would be re-parented to the raylet. This was incorrect; I tried it and they are reparented to init (pid 1).

I think this is one of those cases where you do want an e2e correct solution on the first try. I don't think it makes sense to invest effort on a solution that isn't fully correct to this extent..

The spectrum of solutions I see are as follows:

  1. Kill processes by cgroup (best userspace solution).
  • Sureproof cleaning up of accidentally-leaked processes.
  • Significant re-architecture of how Ray creates/manages worker processes.
  • Requires handling Raylet crashes.
  1. Kill processes by process-group.
  • Sureproof cleaning up of accidentally-leaked processes.
  • Nontrivial changes in Ray given that Ray currently places all workers in the same process group and uses that for cleanup.
  • Requires handling Raylet crashes.
  1. Track worker child processes in the raylet; kill child processes once worker dies.
  • Not sureproof clean up -- will leak processes in certain race conditions, and can kill arbitrary new processes in other race conditions.
  • Straightforward but nontrivial addition to raylet.
  • Requires handling Raylet crashes.
  1. Kill worker child process when a worker exits, either from Cancellation (ctrl+C) or on normal teardown.
  • Sureproof clean up of accidentally-leaked processes except in worker segfault.
  • Straightforward and small addition to core worker.
  • The raylet can crash and workers still clean up correctly.

Horable mention: Raylet kills processes that are in the Raylet process group but have ppid 1. It is not guaranteed that ppid==1 for orphaned processes.

I see (2) as the best medium-term solution. I think (4) is the better short-term solution between (3, 4) because it is sureproof under user cancellation or normal teardown, and easy to implement.

Copy link
Contributor

Choose a reason for hiding this comment

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

Horable mention: Raylet kills processes that are in the Raylet process group but have ppid 1. It is not guaranteed that ppid==1 for orphaned processes.

Why isn't this a candidate solution? It seems to (1) be correct e2e, and (2) pretty straightforward to implement via polling, for example. Even if ppid != 1, we could do it for any unknown ppid.

Copy link
Contributor

@ericl ericl Mar 31, 2023

Choose a reason for hiding this comment

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

Btw, https://unix.stackexchange.com/questions/149319/new-parent-process-when-the-parent-process-dies

suggests that you can use a prctl call to request orphaned children to be reparented to yourself.

From the man page:

PR_SET_CHILD_SUBREAPER (since Linux 3.4)
              If arg2 is nonzero, set the "child subreaper" attribute of
              the calling process; if arg2 is zero, unset the attribute.

              A subreaper fulfills the role of [init(1)](https://man7.org/linux/man-pages/man1/init.1.html) for its
              descendant processes.  When a process becomes orphaned
              (i.e., its immediate parent terminates), then that process
              will be reparented to the nearest still living ancestor
              subreaper.  Subsequently, calls to [getppid(2)](https://man7.org/linux/man-pages/man2/getppid.2.html) in the
              orphaned process will now return the PID of the subreaper
              process, and when the orphan terminates, it is the
              subreaper process that will receive a SIGCHLD signal and
              will be able to [wait(2)](https://man7.org/linux/man-pages/man2/wait.2.html) on the process to discover its
              termination status.

              The setting of the "child subreaper" attribute is not
              inherited by children created by [fork(2)](https://man7.org/linux/man-pages/man2/fork.2.html) and [clone(2)](https://man7.org/linux/man-pages/man2/clone.2.html).
              The setting is preserved across [execve(2)](https://man7.org/linux/man-pages/man2/execve.2.html).

              Establishing a subreaper process is useful in session
              management frameworks where a hierarchical group of
              processes is managed by a subreaper process that needs to
              be informed when one of the processes—for example, a
              double-forked daemon—terminates (perhaps so that it can
              restart that process).  Some [init(1)](https://man7.org/linux/man-pages/man1/init.1.html) frameworks (e.g.,
              [systemd(1)](https://man7.org/linux/man-pages/man1/systemd.1.html)) employ a subreaper process for similar
              reasons.

Copy link
Collaborator

Choose a reason for hiding this comment

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

Oh, didn't know that we have this. Seems that's what we want.

Copy link
Member Author

Choose a reason for hiding this comment

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

Horable mention: Raylet kills processes that are in the Raylet process group but have ppid 1. It is not guaranteed that ppid==1 for orphaned processes.

Why isn't this a candidate solution? It seems to (1) be correct e2e, and (2) pretty straightforward to implement via polling, for example. Even if ppid != 1, we could do it for any unknown ppid.

I avoided that solution because you have to distinguish between leaked child processes and non-leaked child processes (since at depth>1 child processes will have unknown ppids). And climbing the ppid tree has race conditions which are especially risky if there are many child processes being created.

That said, we could do something like a mark-and-sweep approach to drastically reduce the chance of these race conditions; on the first poll, mark candidate processes for termination, on the second poll confirm the tree structure is the same and then kill the child process.

This still fails when the raylet crashes, since the entire old raylet process group will not be known.

I think this is potentially something we do before (2) above. But even this case will require the core-worker to kill child processes on exit to handle raylet crashes (which is the code in this PR).

Copy link
Member Author

Choose a reason for hiding this comment

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

Btw, https://unix.stackexchange.com/questions/149319/new-parent-process-when-the-parent-process-dies

suggests that you can use a prctl call to request orphaned children to be reparented to yourself.

Oh, nice find. Seems that we can use this so that all leaked processes get reparented to the raylet. The missing piece then would be to periodically poll children of the raylet and if a process is not recognized (i.e. isn't a known worker), then we assume it is a leaked child process and we kill it.

Can we still merge this coreworker change to handle the raylet crash case, and since we believe it addresses the high-frequency torch dataloader process leak?

Copy link
Collaborator

Choose a reason for hiding this comment

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

I think if raylet and worker crashes the same time, we can still have leaks.

This is my current understanding (might be wrong): with the current linux mechanisms, unless the child process is cooperative (e.g. using PR_SET_PDEATHSIG, listening to pipe), there is always corner case where the child process is leaked.

src/ray/core_worker/core_worker.cc Show resolved Hide resolved
@rkooo567 rkooo567 self-assigned this Apr 4, 2023
@rkooo567
Copy link
Contributor

rkooo567 commented Apr 4, 2023

Hey guys. I'd like to make a counter proposal to fix this issue here. I'd like to hear your opinions.

I made a PR (#34045) that fixes the multi-processing leak issue that @matthewdeng created. It is not ready yet (need more testing to merge), but I verified it fixes the multi-processing leak issue.

This fix assumes one thing; (potentially controversial); When it comes to child procs cleanup, we don't need to support more than what Python provides. For example, daemon processes are not cleaned up when the Python script is killed by SIGKILL, and that's the expected behavior. IMO, we don't need to provide a stronger cleanup guarantee here (since users are aware of the limitation, and most of the libraries should be implemented with this in mind). But this can be discussed.

Problems

For both Pytorch dataloader and MultiProcessing daemon processes, they are cleaned up from the destructor, __del__ . __del__ is called under the proper process termination process. However, it is not called when SIGKILL is received or the process is terminated via _Exit(), which is the implementation detail of QuickExit in Ray (which is supposed to be used under "emergency scenario").

The root cause of the issue is that we don't properly exit the process when actors are terminated. We bypass the proper termination path "even under common scenarios". Here's more detail about how actors exit in Ray.

  1. When the actor handle goes out of scope, GCS sends an RPC to the core worker which calls QuickExit -> However, it bypasses the destructor (thus we can have leaks like this).
  2. When the job terminates, it sends SIGTERM -> SIGKILL to all owned workers. However, we have bugs in handling SIGTERM for actors (that's something I fixed in my PR). Simply put, if we send a SIGTERM to an actor, it triggers the check failure. Also SIGTERM -> SIGKILL interval is really short (100ms).
  3. When ray.kill is called, GCS sends an RPC to the core worker to call QuickExit, which has the same problem as (1).

Although QuickExit comes in handy sometimes, for these 3 scenarios, users would expect the process to exit properly, calling all of their __del__ methods.

Also note that since most of the cleanup logic is implemented in the destructor, it means we always have a risk of random "resource leaks" from any Python code although we fix the process leak issues by the proposal in this PR.

Proposal

My counter-proposal is to make sure we exit actor processes properly. And then we can probably work on child processes cleaning things if necessary (I think this will solve 99% of user issues). I believe it is already the case for tasks (I verified). It is a bug that actor processes don't properly exit.

There are 3 work items;

  • We kill actors in the same way in every scenario. Raylet sends SIGTERM. It waits for the grace period (I am thinking 5~10 seconds). And then send a SIGKILL
  • We always make Raylet kill the actors. When GCS wants to kill actors, send an RPC to the corresponding raylet.
  • We fix the issue the actor processes cannot handle SIGTERM properly. I verified when I just send a SIGTERM to actors, it has a really bad check failure.

The PR I have fixed the 3rd issue. Though I need more testing to make sure this works well.

@jjyao
Copy link
Collaborator

jjyao commented Apr 4, 2023

I think there are several decisions we need to make:

  1. When a worker process normally exits, is it ok to have child process leak
  2. When a worker process crashes, is it ok to have child process leak
  3. When raylet crashes, is it ok to have child process leak

@rkooo567 I think your PR tries to fix 1 and we are trying to fix 2 & 3.

1 is noncontroversial so it's really about 2 & 3.

@cadedaniel
Copy link
Member Author

Yeah, @rkooo567 your fix should also go in (nice find) but doesn't address case where child process misbehaves.

But your approach highlights something interesting.. really, we should kill child processes once the task or actor finishes, not when the worker process exits. Otherwise unkilled child processes from a dead task will continue to consume system resources (e.g. GPU memory) since the worker process is alive.

I think fixing this iteratively is a good approach.. not sure if this is an issue people see.

@ericl
Copy link
Contributor

ericl commented Apr 4, 2023 via email

@cadedaniel
Copy link
Member Author

After more investigation, I found that Sang's proposal requires a bugfix in actor cleanup behavior so that actor processes are properly cleaned up. The fix is not huge but there is risk if we want to get it in by 2.4.

This PR addresses the process leak reported by users, where processes leak after pressing Ctrl+C. It does not address leaks that come from crashes in worker processes. We'll need to improve upon this in future releases to harden Ray against worker child process leaks.

@cadedaniel cadedaniel added core Issues that should be addressed in Ray Core v2.4.0-pick Ray 2.4 labels Apr 6, 2023
@cadedaniel cadedaniel changed the title [WIP] [CoreWorker] Kill child processes of worker process on exit [CoreWorker] Partially address Ray child process leaks by killing all child processes in the CoreWorker shutdown sequence. Apr 6, 2023
@cadedaniel cadedaniel marked this pull request as ready for review April 6, 2023 03:58
@cadedaniel cadedaniel requested a review from a team April 6, 2023 04:06
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.

nits only

@@ -347,6 +352,103 @@ def pid(self):
ray.get(t)


def test_no_worker_child_process_leaks(ray_start_cluster, tmp_path):
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 to add a test with torch data loader?

Copy link
Member Author

Choose a reason for hiding this comment

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

Sure, let me try today

Copy link
Member Author

Choose a reason for hiding this comment

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

Still can't reproduce (torch shuts down successfully whenever I leak the processes). #31451 (comment)

Already spent a lot of time on this a few weeks ago, let's see if this fixes the issue in the user environment.

src/ray/core_worker/core_worker.cc Outdated Show resolved Hide resolved
src/ray/core_worker/core_worker.cc Outdated Show resolved Hide resolved
// returned. Ideally, we use a library for this, but at the time of writing one is not
// available in Ray C++.

std::filesystem::directory_iterator dir(kProcDirectory);
Copy link
Contributor

Choose a reason for hiding this comment

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

does this work in docker container too (and is it configurable?)

Copy link
Member Author

Choose a reason for hiding this comment

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

Yes, works in docker container. The man pages suggest that there is a way you can change it, but I think it's really rare.

The proc filesystem is a pseudo-filesystem which provides an interface to kernel data
structures. It is commonly mounted at /proc. Most of it is read-only, but some files
allow kernel variables to be changed.


for (const auto &child_pid : child_procs) {
auto maybe_error_code = KillProc(child_pid);
RAY_CHECK(maybe_error_code &&
Copy link
Collaborator

Choose a reason for hiding this comment

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

RAY_CHECK(maybe_error_code) << "Expected ...."

}

#if defined(__linux__)
static inline std::vector<pid_t> GetAllProcsWithPpidLinux(pid_t parent_pid) {
Copy link
Collaborator

Choose a reason for hiding this comment

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

oh, it's a bummer that linux doesn't provide an API to return all child processes and I think this code is needed later on when we implement the raylet subreaper.

Actually would just Process.Exec("kill $(ps -o pid= --ppid XXX)") just work

Copy link
Member Author

Choose a reason for hiding this comment

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

oh, it's a bummer that linux doesn't provide an API to return all child processes and I think this code is needed later on when we implement the raylet subreaper.

Linux does, the API is this file :) ps actually uses this file under the covers. Long-term, if we need more interaction with /proc in the Ray backend, we should build and link against libproc-dev.

Actually would just Process.Exec("kill $(ps -o pid= --ppid XXX)") just work

Yep -- this is the same as that. Using Process.Exec is a bit worse from a quality perspective as it assumes these binaries are on the PATH; in some environments like minimal container images this is not true.

continue;
}

// Determine if the directory name consists of only digits (means it's a PID).
Copy link
Collaborator

Choose a reason for hiding this comment

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

Is it always true that a digit only file name is the pid? Would reading /proc/<pid>/task/<tid>/children be safer?

Copy link
Member Author

Choose a reason for hiding this comment

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

Is it always true that a digit only file name is the pid?

Are you asking if a pid can be non-digit? No, it is always an integer value.

Would reading /proc//task//children be safer?

What is the concern? This has additional constraints; if there are multiple threads in the worker process, then we need to check each one.

Copy link
Collaborator

@jjyao jjyao left a comment

Choose a reason for hiding this comment

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

Approve since we think this is the lowest rick change for 2.4. In 2.5 we will fix the normal shutdown bug and raylet subreaper.

src/ray/core_worker/core_worker.cc Outdated Show resolved Hide resolved
@cadedaniel
Copy link
Member Author

Tests look good, the failures are unrelated. There are a few MacOS tests that still need to run but the C++ ones have already passed, which are the important ones.

Can we merge? cc @jjyao @scv119

@rkooo567
Copy link
Contributor

rkooo567 commented Apr 7, 2023

@cadedaniel can you run release tests before merging this?

@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 Apr 7, 2023
@cadedaniel
Copy link
Member Author

cadedaniel commented Apr 7, 2023

@cadedaniel can you run release tests before merging this?

kicked off release tests here https://buildkite.com/ray-project/release-tests-pr/builds/34108

@cadedaniel
Copy link
Member Author

cadedaniel commented Apr 7, 2023

Looking through release test failures (will update this as I go through them all):

@jjyao
Copy link
Collaborator

jjyao commented Apr 7, 2023

Failed release tests look unrelated.

@jjyao jjyao merged commit 0ae95fb into ray-project:master Apr 7, 2023
cadedaniel added a commit to cadedaniel/ray that referenced this pull request Apr 7, 2023
… child processes in the CoreWorker shutdown sequence. (ray-project#33976)

We kill all child processes when a Ray worker process exits. This addresses process leaks that caused GPU OOM errors in ray-project#31451. There is some risk to this PR, particularly if Ray users rely on Ray's existing behavior of leaking processes. We don't know of any such user, but we provide a new flag RAY_kill_child_processes_on_worker_exit to provide a workaround in case someone is impacted.
scv119 pushed a commit that referenced this pull request Apr 9, 2023
… child processes in the CoreWorker shutdown sequence. (#33976) (#34181)

We kill all child processes when a Ray worker process exits. This addresses process leaks that caused GPU OOM errors in #31451. There is some risk to this PR, particularly if Ray users rely on Ray's existing behavior of leaking processes. We don't know of any such user, but we provide a new flag RAY_kill_child_processes_on_worker_exit to provide a workaround in case someone is impacted.
elliottower pushed a commit to elliottower/ray that referenced this pull request Apr 22, 2023
… child processes in the CoreWorker shutdown sequence. (ray-project#33976)

We kill all child processes when a Ray worker process exits. This addresses process leaks that caused GPU OOM errors in ray-project#31451. There is some risk to this PR, particularly if Ray users rely on Ray's existing behavior of leaking processes. We don't know of any such user, but we provide a new flag RAY_kill_child_processes_on_worker_exit to provide a workaround in case someone is impacted.

Signed-off-by: elliottower <[email protected]>
@cadedaniel cadedaniel deleted the daemon-subprocess-leak branch May 1, 2023 22:56
ProjectsByJackHe pushed a commit to ProjectsByJackHe/ray that referenced this pull request May 4, 2023
… child processes in the CoreWorker shutdown sequence. (ray-project#33976)

We kill all child processes when a Ray worker process exits. This addresses process leaks that caused GPU OOM errors in ray-project#31451. There is some risk to this PR, particularly if Ray users rely on Ray's existing behavior of leaking processes. We don't know of any such user, but we provide a new flag RAY_kill_child_processes_on_worker_exit to provide a workaround in case someone is impacted.

Signed-off-by: Jack He <[email protected]>
pcmoritz added a commit that referenced this pull request May 27, 2023
Thanks @iycheng for debugging this!

In #33976 we don't always guarantee that children of worker processes are killed -- namely if the worker dies in an unexpected way, they are not (see #26118).

This unfortunately happens in the `test_no_worker_child_process_leaks` test case since the Ray cluster is torn down when the driver exits, which can lead to worker processes being forcefully terminated [the solution in this PR is to not initialize the Ray cluster in the driver script but start a separate head node with `.add_node`].

As a result of this forceful killing, the test becomes flaky (it passes if the workers are killed through the normal termination, and it doesn't pass if the workers are killed otherwise). The flakiness is not super big right now but becomes big with some other changes.

This should eventually be fixed with #26118.
scv119 pushed a commit to scv119/ray that referenced this pull request Jun 16, 2023
Thanks @iycheng for debugging this!

In ray-project#33976 we don't always guarantee that children of worker processes are killed -- namely if the worker dies in an unexpected way, they are not (see ray-project#26118).

This unfortunately happens in the `test_no_worker_child_process_leaks` test case since the Ray cluster is torn down when the driver exits, which can lead to worker processes being forcefully terminated [the solution in this PR is to not initialize the Ray cluster in the driver script but start a separate head node with `.add_node`].

As a result of this forceful killing, the test becomes flaky (it passes if the workers are killed through the normal termination, and it doesn't pass if the workers are killed otherwise). The flakiness is not super big right now but becomes big with some other changes.

This should eventually be fixed with ray-project#26118.
rkooo567 pushed a commit that referenced this pull request Aug 17, 2023
Summary
This PR has CoreWorkers kill their child processes when they notice the raylet has died. This fixes an edge case missing from #33976.

Details
The CoreWorker processes have a periodic check that validates the raylet process is alive. If the raylet is no longer alive, then the CoreWorker processes invoke QuickExit. This PR modifies this behavior so that before calling QuickExit, the CoreWorker processes kill all child processes.

Note that this does not address the case where CoreWorker processes crash or are killed with SIGKILL. That condition requires the subreaper work proposed in #34125 / #26118.

The ray stop --force case also requires the subreaper work. This is because --force kills CoreWorker processes with SIGKILL, leaving them no opportunity to clean up child processes. We can add the logic which catches leaked child processes, but the best design for this is the subreaper design.
arvind-chandra pushed a commit to lmco/ray that referenced this pull request Aug 31, 2023
Thanks @iycheng for debugging this!

In ray-project#33976 we don't always guarantee that children of worker processes are killed -- namely if the worker dies in an unexpected way, they are not (see ray-project#26118).

This unfortunately happens in the `test_no_worker_child_process_leaks` test case since the Ray cluster is torn down when the driver exits, which can lead to worker processes being forcefully terminated [the solution in this PR is to not initialize the Ray cluster in the driver script but start a separate head node with `.add_node`].

As a result of this forceful killing, the test becomes flaky (it passes if the workers are killed through the normal termination, and it doesn't pass if the workers are killed otherwise). The flakiness is not super big right now but becomes big with some other changes.

This should eventually be fixed with ray-project#26118.

Signed-off-by: e428265 <[email protected]>
arvind-chandra pushed a commit to lmco/ray that referenced this pull request Aug 31, 2023
…ect#38439)

Summary
This PR has CoreWorkers kill their child processes when they notice the raylet has died. This fixes an edge case missing from ray-project#33976.

Details
The CoreWorker processes have a periodic check that validates the raylet process is alive. If the raylet is no longer alive, then the CoreWorker processes invoke QuickExit. This PR modifies this behavior so that before calling QuickExit, the CoreWorker processes kill all child processes.

Note that this does not address the case where CoreWorker processes crash or are killed with SIGKILL. That condition requires the subreaper work proposed in ray-project#34125 / ray-project#26118.

The ray stop --force case also requires the subreaper work. This is because --force kills CoreWorker processes with SIGKILL, leaving them no opportunity to clean up child processes. We can add the logic which catches leaked child processes, but the best design for this is the subreaper design.

Signed-off-by: e428265 <[email protected]>
vymao pushed a commit to vymao/ray that referenced this pull request Oct 11, 2023
…ect#38439)

Summary
This PR has CoreWorkers kill their child processes when they notice the raylet has died. This fixes an edge case missing from ray-project#33976.

Details
The CoreWorker processes have a periodic check that validates the raylet process is alive. If the raylet is no longer alive, then the CoreWorker processes invoke QuickExit. This PR modifies this behavior so that before calling QuickExit, the CoreWorker processes kill all child processes.

Note that this does not address the case where CoreWorker processes crash or are killed with SIGKILL. That condition requires the subreaper work proposed in ray-project#34125 / ray-project#26118.

The ray stop --force case also requires the subreaper work. This is because --force kills CoreWorker processes with SIGKILL, leaving them no opportunity to clean up child processes. We can add the logic which catches leaked child processes, but the best design for this is the subreaper design.

Signed-off-by: Victor <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
@author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. core Issues that should be addressed in Ray Core Ray 2.4
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[core] Multi-process(?) / GPU processes do not seem to be freed after ctrl+c on cluster
5 participants