Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Automatically restart memory-leaking workers when they reach a critical limit #4221

Open
wants to merge 7 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
26 changes: 26 additions & 0 deletions distributed/tests/test_worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -427,6 +427,32 @@ async def test_io_loop(cleanup):
assert w.io_loop is s.loop


@pytest.mark.slow
@gen_cluster(
client=True,
Worker=Nanny,
nthreads=[("127.0.0.1", 1), ("127.0.0.1", 1)],
worker_kwargs={"memory_limit": "512MB"},
)
async def test_memory_leak(c, s, a, b):
import numpy as np
import time

x = {}

def memory_leaking_fn(data):
# leak 12MB blocks
x[data] = np.random.randint(100, size=12 * 1024 ** 2 // 8)

time.sleep(0.1)
return data

futures = c.map(memory_leaking_fn, range(1000))

for f in futures:
f.result()


@gen_cluster(client=True, nthreads=[])
async def test_spill_to_disk(c, s):
np = pytest.importorskip("numpy")
Expand Down
1 change: 1 addition & 0 deletions distributed/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -2709,6 +2709,7 @@ async def check_pause(memory):
if self.memory_limit is not None
else "None",
)
await self.close_gracefully(restart=True)
Copy link
Member

Choose a reason for hiding this comment

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

I'm trying to anticipate cases where restarting might not be welcome. I can think of a couple:

  1. There is some slow cleanup process that needs a couple of seconds to clear memory out (I haven't seen this personally). Maybe it makes sense to wait a few times before restarting? This might not be worth the complexity though
  2. The worker is currently working on something, and we should give it a moment to try to finish it. This could be determined by checking the self.executing set of currently executing tasks. If this is empty then sure, restarting seems ok. If not, then we might want to wait for a bit, but not forever.

break
k, v, weight = self.data.fast.evict()
del k, v
Expand Down