diff --git a/distributed/tests/test_asyncprocess.py b/distributed/tests/test_asyncprocess.py index 44ba89bcc3e..65aa0e303cc 100644 --- a/distributed/tests/test_asyncprocess.py +++ b/distributed/tests/test_asyncprocess.py @@ -16,7 +16,7 @@ from tornado.ioloop import IOLoop from tornado.locks import Event -from distributed.compatibility import WINDOWS +from distributed.compatibility import LINUX, MACOS, WINDOWS from distributed.metrics import time from distributed.process import AsyncProcess from distributed.utils import get_mp_context @@ -36,10 +36,10 @@ def exit_now(rc=0): sys.exit(rc) -def exit_with_signal(signum): +def exit_with_sigint(): signal.signal(signal.SIGINT, signal.SIG_DFL) + os.kill(os.getpid(), signal.SIGINT) while True: - os.kill(os.getpid(), signum) sleep(0.01) @@ -159,45 +159,57 @@ async def test_exitcode(): assert proc.exitcode is None q.put(5) - await proc.join(timeout=30) + await proc.join() assert not proc.is_alive() assert proc.exitcode == 5 -@pytest.mark.skipif(WINDOWS, reason="POSIX only") +def assert_exit_code(proc: AsyncProcess, expect: signal.Signals) -> None: + if WINDOWS: + # multiprocessing.Process.terminate() sets exit code -15 like in Linux, but + # os.kill(pid, signal.SIGTERM) sets exit code +15 + assert proc.exitcode in (-expect, expect) + elif MACOS: + # FIXME this happens very frequently on GitHub MacOSX CI. Reason unknown. + if expect != signal.SIGKILL and proc.exitcode == -signal.SIGKILL: + raise pytest.xfail(reason="https://github.com/dask/distributed/issues/6393") + assert proc.exitcode == -expect + else: + assert LINUX + assert proc.exitcode == -expect + + @gen_test() -async def test_signal(): - proc = AsyncProcess(target=exit_with_signal, args=(signal.SIGINT,)) - proc.daemon = True +async def test_sigint_from_same_process(): + proc = AsyncProcess(target=exit_with_sigint) assert not proc.is_alive() assert proc.exitcode is None await proc.start() - await proc.join(timeout=30) + await proc.join() assert not proc.is_alive() - # Can be 255 with forkserver, see https://bugs.python.org/issue30589 - assert proc.exitcode in (-signal.SIGINT, 255) + assert_exit_code(proc, signal.SIGINT) + +@gen_test() +async def test_sigterm_from_parent_process(): proc = AsyncProcess(target=wait) await proc.start() os.kill(proc.pid, signal.SIGTERM) - await proc.join(timeout=30) - + await proc.join() assert not proc.is_alive() - assert proc.exitcode in (-signal.SIGTERM, 255) + assert_exit_code(proc, signal.SIGTERM) @gen_test() async def test_terminate(): proc = AsyncProcess(target=wait) - proc.daemon = True await proc.start() await proc.terminate() - await proc.join() assert not proc.is_alive() - assert proc.exitcode in (-signal.SIGTERM, 255) + assert_exit_code(proc, signal.SIGTERM) @gen_test() @@ -333,7 +345,7 @@ async def test_kill(): await proc.kill() await proc.join() assert not proc.is_alive() - assert proc.exitcode in (-signal.SIGKILL, 255) + assert proc.exitcode == -signal.SIGKILL def _worker_process(worker_ready, child_pipe): diff --git a/distributed/tests/test_semaphore.py b/distributed/tests/test_semaphore.py index c922c568dd5..2c4388af628 100644 --- a/distributed/tests/test_semaphore.py +++ b/distributed/tests/test_semaphore.py @@ -3,6 +3,7 @@ import asyncio import logging import pickle +import signal from datetime import timedelta from time import sleep @@ -146,7 +147,7 @@ def f(x, sem, kill_address): if worker.address == kill_address: import os - os.kill(os.getpid(), 15) + os.kill(os.getpid(), signal.SIGTERM) return x futures = client.map(