Skip to content

Commit

Permalink
Clean up cluster process reaping (#6840)
Browse files Browse the repository at this point in the history
  • Loading branch information
gjoseph92 committed Aug 18, 2022
1 parent 7768f6c commit 61fca1c
Showing 1 changed file with 12 additions and 14 deletions.
26 changes: 12 additions & 14 deletions distributed/utils_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -568,14 +568,14 @@ def security():
return tls_only_security()


def _kill_join(proc, timeout):
proc.kill()
proc.join(timeout)
if proc.is_alive():
raise multiprocessing.TimeoutError(
f"Process {proc} did not shut down within {timeout}s"
)
proc.close()
def _kill_join_processes(processes):
# Join may hang or cause issues, so make sure all are killed first.
# Note that we don't use a timeout, but rely on the overall pytest timeout.
for proc in processes:
proc.kill()
for proc in processes:
proc.join()
proc.close()


def _close_queue(q):
Expand All @@ -590,15 +590,13 @@ def cluster(
nanny=False,
worker_kwargs=None,
active_rpc_timeout=10,
shutdown_timeout=20,
scheduler_kwargs=None,
config=None,
):
worker_kwargs = worker_kwargs or {}
scheduler_kwargs = scheduler_kwargs or {}
config = config or {}

ws = weakref.WeakSet()
enable_proctitle_on_children()

with check_process_leak(check=True), check_instances(), config_for_cluster_tests():
Expand All @@ -608,6 +606,8 @@ def cluster(
_run_worker = run_worker

with contextlib.ExitStack() as stack:
processes = []
stack.callback(_kill_join_processes, processes)
# The scheduler queue will receive the scheduler's address
scheduler_q = get_mp_context().Queue()
stack.callback(_close_queue, scheduler_q)
Expand All @@ -620,9 +620,8 @@ def cluster(
kwargs=scheduler_kwargs,
daemon=True,
)
ws.add(scheduler)
scheduler.start()
stack.callback(_kill_join, scheduler, shutdown_timeout)
processes.append(scheduler)

# Launch workers
workers_by_pid = {}
Expand All @@ -642,9 +641,8 @@ def cluster(
args=(q, scheduler_q, config),
kwargs=kwargs,
)
ws.add(proc)
proc.start()
stack.callback(_kill_join, proc, shutdown_timeout)
processes.append(proc)
workers_by_pid[proc.pid] = {"proc": proc}

saddr_or_exception = scheduler_q.get()
Expand Down

0 comments on commit 61fca1c

Please sign in to comment.