From d9ab5cfd191213c0ad11c8ba95a2bafb582afa1b Mon Sep 17 00:00:00 2001 From: Florian Jetter Date: Thu, 23 Jun 2022 15:18:38 +0200 Subject: [PATCH] Restore log message about received signals in CLI (#6618) --- distributed/_signals.py | 7 +++++++ distributed/cli/tests/test_dask_scheduler.py | 1 + distributed/cli/tests/test_dask_worker.py | 1 + 3 files changed, 9 insertions(+) diff --git a/distributed/_signals.py b/distributed/_signals.py index c80215d7617..ee70d10f08a 100644 --- a/distributed/_signals.py +++ b/distributed/_signals.py @@ -14,10 +14,13 @@ async def wait_for_signals(signals: list[signal.Signals]) -> None: event = asyncio.Event() old_handlers: dict[int, Any] = {} + caught_signal: int | None = None def handle_signal(signum, frame): # *** Do not log or print anything in here # https://stackoverflow.com/questions/45680378/how-to-explain-the-reentrant-runtimeerror-caused-by-printing-in-signal-handlers + nonlocal caught_signal + caught_signal = signum # Restore old signal handler to allow for quicker exit # if the user sends the signal again. signal.signal(signum, old_handlers[signum]) @@ -28,6 +31,10 @@ def handle_signal(signum, frame): try: await event.wait() + assert caught_signal + logger.info( + "Received signal %s (%d)", signal.Signals(caught_signal).name, caught_signal + ) finally: for sig in signals: signal.signal(sig, old_handlers[sig]) diff --git a/distributed/cli/tests/test_dask_scheduler.py b/distributed/cli/tests/test_dask_scheduler.py index cbd00c41427..75667551ccc 100644 --- a/distributed/cli/tests/test_dask_scheduler.py +++ b/distributed/cli/tests/test_dask_scheduler.py @@ -549,6 +549,7 @@ def test_signal_handling(loop, sig): stdout, stderr = scheduler.communicate() logs = stdout.decode().lower() assert stderr is None + assert sig.name.lower() in logs assert scheduler.returncode == 0 assert "scheduler closing" in logs assert "end scheduler" in logs diff --git a/distributed/cli/tests/test_dask_worker.py b/distributed/cli/tests/test_dask_worker.py index d59a7ae25fb..c6605660110 100644 --- a/distributed/cli/tests/test_dask_worker.py +++ b/distributed/cli/tests/test_dask_worker.py @@ -689,6 +689,7 @@ async def test_signal_handling(c, s, nanny, sig): stdout, stderr = worker.communicate() logs = stdout.decode().lower() assert stderr is None + assert sig.name.lower() in logs assert worker.returncode == 0 if nanny == "--nanny": assert "closing nanny" in logs