用户生成进程的生命周期#

当你从 Ray 工作进程中生成子进程时,你需要负责管理子进程的生命周期。然而,这并不总是可能的,尤其是在工作进程崩溃且子进程由库(如 torch dataloader)生成时。

为了避免用户启动的进程泄露,Ray 提供了在启动这些进程的工作者退出时杀死所有用户启动进程的机制。此功能防止了子进程(例如,torch)的 GPU 内存泄露。

我们有两个环境变量来处理工作进程退出时的子进程终止:

  • RAY_kill_child_processes_on_worker_exit (默认 true): 仅在Linux上有效。如果为真,工作进程在退出时会杀死所有 直接 子进程。如果工作进程崩溃,这将不起作用。这不是递归的,因此孙进程不会被此机制杀死。

  • RAY_kill_child_processes_on_worker_exit_with_raylet_subreaper (默认 false): 仅在 Linux 3.4 及以上版本有效。如果为 true,Raylet 会在 worker 退出后 递归地 杀死由该 worker 启动的所有子进程和孙进程。即使 worker 崩溃,此功能也有效。在 worker 死亡后的 10 秒内进行杀进程操作。

在非Linux平台上,用户启动的进程不由Ray控制。用户负责管理子进程的生命周期。如果父Ray工作进程死亡,子进程将继续运行。

注意:该功能旨在作为杀死孤立进程的最后手段。它不是适当进程管理的替代品。用户仍应管理其进程的生命周期并正确清理。

用户启动的进程在工作进程退出时被终止#

在下面的示例中,我们使用 Ray Actor 来生成一个用户进程。用户进程是一个长时间运行的进程,每秒打印一次“Hello, world!”。当 Actor 被终止时,用户进程也会被终止。

import ray
import psutil
import subprocess
import time
import os

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

@ray.remote
class MyActor:
  def __init__(self):
    pass

  def start(self):
    # Start a user process
    process = subprocess.Popen(["/bin/bash", "-c", "sleep 10000"])
    return process.pid

  def signal_my_pid(self):
    import signal
    os.kill(os.getpid(), signal.SIGKILL)


actor = MyActor.remote()

pid = ray.get(actor.start.remote())
assert psutil.pid_exists(pid)  # the subprocess running

actor.signal_my_pid.remote()  # sigkill'ed, the worker's subprocess killing no longer works
time.sleep(11)  # raylet kills orphans every 10s
assert not psutil.pid_exists(pid)

启用该功能#

要启用子收割者功能,请在**启动 Ray 集群时**将环境变量 RAY_kill_child_processes_on_worker_exit_with_raylet_subreaper 设置为 true。如果 Ray 集群已经在运行,您需要重启 Ray 集群以应用更改。在运行时环境中设置 env_var 将无效。

RAY_kill_child_processes_on_worker_exit_with_raylet_subreaper=true ray start --head

另一种方法是,在 ray.init() 时通过添加 _system_config 来启用它,如下所示:

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

⚠️ 注意:核心工作进程现在会收割僵尸进程,如果你需要 waitpid,请切换回来。#

当该功能启用时,工作进程成为子收割者(参见下一节),这意味着可能会有一些孙进程被重新父进程到工作进程。为了收割这些进程,工作进程将 SIGCHLD 信号设置为 SIG_IGN。这使得工作进程在其子进程退出时不接收 SIGCHLD 信号。如果你需要等待子进程退出,你需要将 SIGCHLD 信号重置为 SIG_DFL

import signal
signal.signal(signal.SIGCHLD, signal.SIG_DFL)

在底层#

此功能通过在生成所有 Ray 工作者的 Raylet 进程上设置 prctl(PR_SET_CHILD_SUBREAPER, 1) 标志来实现。参见 prctl(2)。此标志使 Raylet 进程成为“子收割者”,这意味着如果一个后代子进程死亡,该死亡子进程的子进程将重新父化为 Raylet 进程。

Raylet 维护一个它生成的“已知”直接子进程 pid 列表,当 Raylet 进程接收到 SIGCHLD 信号时,它知道它的一个子进程(例如工作者)已经死亡,并且可能存在被重新父进程的孤儿进程。Raylet 列出所有子进程 pid(ppid = raylet pid),如果一个子进程 pid 是“未知”的(即不在直接子进程 pid 列表中),Raylet 认为它是一个孤儿进程并通过 SIGKILL 杀死它。

对于一个深度创建进程的链,Raylet 会一步一步地进行终止。例如,在一个这样的链中:

raylet -> the worker -> user process A -> user process B -> user process C

the worker 死亡时,Raylet 会杀死 user process A,因为它不在“已知”的子进程列表中。当 user process A 死亡时,Raylet 会杀死 user process B,依此类推。

边缘情况是,如果 the worker 仍然存活但 user process A 已经死亡,那么 user process B 会被重新父进程化并面临被杀的风险。为了缓解这种情况,Ray 也将 the worker 设置为子收割者,因此它可以收养重新父进程化的进程。Core worker 不会杀死未知的孩子进程,因此用户“守护进程”进程例如 user process Buser process A 存活更久的情况下可以继续存活。然而,如果 the worker 死亡,用户守护进程会被重新父进程化为 raylet 并被杀死。

相关 PR: 在 raylet 中使用 subreaper 杀死未拥有的子进程。(#42992)