Skip to content

Send SIGKILL after SIGTERM when passing 95% memory#6419

Merged
crusaderky merged 2 commits intodask:mainfrom
crusaderky:slow_terminate
May 25, 2022
Merged

Send SIGKILL after SIGTERM when passing 95% memory#6419
crusaderky merged 2 commits intodask:mainfrom
crusaderky:slow_terminate

Conversation

@crusaderky
Copy link
Collaborator

@crusaderky crusaderky commented May 23, 2022

Closes #6373

CC @hendrikmakait

@crusaderky crusaderky self-assigned this May 23, 2022
@crusaderky crusaderky changed the title Send SIGKILL if the worker ignores SIGTERM Send SIGKILL after SIGTERM when passing 95% memory May 23, 2022
self._last_terminated_pid = -1

if memory / self.memory_limit > self.memory_terminate_fraction:
if self._last_terminated_pid != process.pid:
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The default interval is 100ms. Isn't this a bit short for us to escalate to kill?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not really - that's 100ms worth of extra leaking from any task that is currently running.

Note that, to the best of my understanding, no cleanup code whatsoever runs on SIGTERM by default. So it should be immediate, unless the user tampered with the signal handlers - which they could legitimately do for their own cleanup code. A realistic use case is a database library that installs a SIGTERM handler to cleanly shut down its sockets.

In this case, I argue that prompt termination for a process that is close to go beyond 100% is more important the clean teardown.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We do install our own signal handlers and close workers/nannies, e.g.

async def wait_for_signals(signals: list[signal.Signals]) -> None:

async def wait_for_signals_and_close():
"""Wait for SIGINT or SIGTERM and close all nannies upon receiving one of those signals"""
nonlocal signal_fired
await wait_for_signals([signal.SIGINT, signal.SIGTERM])
signal_fired = True
if nanny:
# Unregister all workers from scheduler
await asyncio.gather(*(n.close(timeout=10) for n in nannies))

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I know. This doesn't exclude third party handlers.
Typical pattern:

def my_handler(signo, frame):
    # TODO do your thing
    prev(signo, frame)

prev = signal.signal(signal.SIGTERM, my_handler)

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

From what I understand, this should not be a problem since we install the handler on the parent process (i.e., the nanny itself), so it shouldn't be triggered by a SIGTERM to the worker child process.

@crusaderky crusaderky marked this pull request as draft May 23, 2022 15:03
@github-actions
Copy link
Contributor

github-actions bot commented May 23, 2022

Unit Test Results

       15 files  ±  0         15 suites  ±0   6h 58m 30s ⏱️ + 14m 13s
  2 808 tests +  2    2 728 ✔️ +  3    79 💤  - 1  1 ±0 
20 818 runs  +15  19 884 ✔️ +10  933 💤 +5  1 ±0 

For more details on these failures, see this check.

Results for commit 1042953. ± Comparison against base commit 97a7eb6.

♻️ This comment has been updated with latest results.

@crusaderky crusaderky marked this pull request as ready for review May 23, 2022 18:46
@crusaderky
Copy link
Collaborator Author

Ready for review and merge

Copy link
Member

@hendrikmakait hendrikmakait left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM!

self._last_terminated_pid = -1

if memory / self.memory_limit > self.memory_terminate_fraction:
if self._last_terminated_pid != process.pid:
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

From what I understand, this should not be a problem since we install the handler on the parent process (i.e., the nanny itself), so it shouldn't be triggered by a SIGTERM to the worker child process.

@crusaderky crusaderky merged commit ba39915 into dask:main May 25, 2022
@crusaderky crusaderky deleted the slow_terminate branch May 25, 2022 09:27
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Flaky test_slow_terminate

3 participants