用户创建进程的生命周期#
当您从 Ray worker 进程创建子进程时,您需要负责管理子进程的生命周期。然而,这并非总是可行的,特别是当 worker 崩溃且子进程是从库(如 torch dataloader)创建的时。
为了避免用户创建的进程泄露,Ray 提供了机制,在启动它们的 worker 退出时终止所有用户创建的进程。此功能可防止子进程(例如 torch)导致 GPU 内存泄露。
Ray 提供了以下机制来处理 worker 退出时的子进程终止:
RAY_kill_child_processes_on_worker_exit(默认true): 仅在 Linux 上有效。如果为 true,worker 在退出时会终止所有直接子进程。如果 worker 崩溃,此设置将不起作用。此设置不是递归的,即孙子进程不会被此机制终止。RAY_kill_child_processes_on_worker_exit_with_raylet_subreaper(默认false): 仅在 Linux 3.4 及以上版本上有效。如果为 true,Raylet 会在 worker 退出后递归终止由该 worker 启动的所有子进程和孙子进程。即使 worker 崩溃,此设置也有效。终止操作在 worker 死亡后的 10 秒内完成。RAY_process_group_cleanup_enabled(默认false): 如果为 true (POSIX),Ray 会在创建时将每个 worker 隔离到其自己的进程组,并在 worker 退出时通过killpg清理 worker 的进程组。有意调用setsid()的进程将脱离,不会被此清理操作终止。
在非 Linux 平台上,不可用子进程收割器。每个 worker 的进程组在 POSIX 平台上受支持;在 Windows 上,子进程收割器和进程组均不适用。用户应在不支持的平台上显式管理子进程。
注意:此功能旨在作为终止孤儿进程的最后手段。它不能替代正确的进程管理。用户仍应管理其进程的生命周期并妥善清理。
Worker 退出时用户创建的进程被终止#
以下示例使用 Ray Actor 来创建一个用户进程。该用户进程是一个 sleep 进程。
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)
启用该功能#
要启用子进程收割器功能(已弃用),请在启动时通过 _system_config 或等效的集群配置进行设置。您必须重启集群才能应用更改。建议优先启用 process_group_cleanup_enabled。
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})
⚠️ 注意:核心 worker 现在会收割僵尸进程,如果您需要 waitpid,请将其切换回#
当启用子进程收割器时,worker 进程也会成为一个子进程收割器(Linux),这意味着一些孙子进程可能会被重新挂载到 worker 进程。worker 将 SIGCHLD 设置为 SIG_IGN。如果您需要等待子进程退出,请先将 SIGCHLD 重置为 SIG_DFL。
import signal
signal.signal(signal.SIGCHLD, signal.SIG_DFL)
内部机制#
此功能通过在创建所有 Ray worker 的 Raylet 进程上设置 prctl(PR_SET_CHILD_SUBREAPER, 1) 标志来实现。请参阅 prctl(2)。此标志使 Raylet 进程成为“子进程收割器”,这意味着如果某个后代子进程死亡,死去的子进程的子进程将被重新挂载到 Raylet 进程。子进程收割器已弃用,建议使用每个 worker 的进程组。
Raylet 维护一个它创建的“已知”直接子进程 PID 列表,当 Raylet 进程收到 SIGCHLD 信号时,它就知道其子进程之一(例如 worker)已死亡,并且可能存在被重新挂载的孤儿进程。Raylet 列出所有子进程 PID(ppid = raylet pid),如果某个子进程 PID 不在“已知”列表中(即不在直接子进程 PID 列表中),Raylet 则认为它是孤儿进程并将其通过 SIGKILL 终止。
对于深度进程创建链,Raylet 将分步进行终止。例如,在以下链中:
raylet -> the worker -> user process A -> user process B -> user process C
当 worker 进程死亡时,Raylet 会终止 用户进程 A,因为它不在“已知”子进程列表中。当 用户进程 A 死亡时,Raylet 会终止 用户进程 B,依此类推。
一种特殊情况是,如果 worker 进程仍然存活但 用户进程 A 已死亡,那么 用户进程 B 将被重新挂载并有被终止的风险。为了缓解这种情况,Ray 还将 worker 设置为子进程收割器,以便它可以接收被重新挂载的进程。核心 worker 不会终止未知的子进程,因此一个用户“守护进程”(例如 用户进程 B)如果比 用户进程 A 存活时间长,就可以继续运行。但是,如果 worker 进程死亡,用户守护进程将被重新挂载到 raylet 并被终止。