-
-
Notifications
You must be signed in to change notification settings - Fork 710
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
WARNING - Memory use is high but worker has no data to store to disk #4193
Comments
A related issue I hit when residing workers a couple of times: At some point, these high-memory workers are just stuck and do not do anything anymore. |
I don't think there is a method to auto-restart. You can manually call |
Thanks for your answer @quasiben. |
Hmm, could there in fact be a leak ? Can you run with more memory ?Can you describe the kind of operation(s) the worker is doing during this time ? |
My function is fetching data from a SQL server and reads some data from TileDB. Afterwards it uses Pandas to do some aggregations and return a numpy array. I made sure that all connections are one-shot and therefore get removed after loading the data. More memory helps indeed, but only to keep the workers alive longer. |
I like this idea. In practice Dask is unable to control memory leaks. These often come from other libraries that weren't designed to run in a multi-threaded context (indeed, glibc malloc itself has some issues here). I think that smoothly retiring workers in this situation would be a good way for us to add some robustness to this situation. One challenge to this is what to do with the memory that is currently stored in the worker. It would be nice to transfer it to other workers before restarting (this will happen naturally as part of the worker retire process), but it might be hard to bring it back from disk, given the memory leak that has occurred. As a first step, we could call diff --git a/distributed/worker.py b/distributed/worker.py
index 00d69329..06013718 100644
--- a/distributed/worker.py
+++ b/distributed/worker.py
@@ -1178,12 +1178,15 @@ class Worker(ServerNode):
setproctitle("dask-worker [closed]")
return "OK"
- async def close_gracefully(self):
+ async def close_gracefully(self, nanny=None):
""" Gracefully shut down a worker
This first informs the scheduler that we're shutting down, and asks it
to move our data elsewhere. Afterwards, we close as normal
"""
+ if nanny is None:
+ nanny = not self.lifetime_restart
+
if self.status in (Status.closing, Status.closing_gracefully):
await self.finished()
@@ -1193,7 +1196,7 @@ class Worker(ServerNode):
logger.info("Closing worker gracefully: %s", self.address)
self.status = Status.closing_gracefully
await self.scheduler.retire_workers(workers=[self.address], remove=False)
- await self.close(safe=True, nanny=not self.lifetime_restart)
+ await self.close(safe=True, nanny=nanny)
async def terminate(self, comm=None, report=True, **kwargs):
await self.close(report=report, **kwargs)
@@ -2683,6 +2686,7 @@ class Worker(ServerNode):
if self.memory_limit is not None
else "None",
)
+ await self.close_gracefully(nanny=False)
break
k, v, weight = self.data.fast.evict()
del k, v |
Thanks a lot @mrocklin, I will test your patch in the next week. |
Thanks! To be clear *I *haven't tested that, or even verified that it's
valid Python. Some work on your end may still be necessary.
…On Sat, Oct 31, 2020 at 10:29 AM Florian R. Hölzlwimmer < ***@***.***> wrote:
Thanks a lot @mrocklin <https://github.com/mrocklin>, I will test your
patch in the next week.
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
<#4193 (comment)>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/AACKZTD4QRTZYZRF6B5P54DSNRCQVANCNFSM4TBKFTIQ>
.
|
OK sure. I'm happy to debug :) |
The testing infrastructure is pretty good for this, especially if you can
write async code:
https://distributed.dask.org/en/latest/develop.html#writing-tests
Dask can run itself entirely in one thread, which really helps with
debugging.
…On Sat, Oct 31, 2020 at 11:59 AM Florian R. Hölzlwimmer < ***@***.***> wrote:
OK sure. I'm happy to debug :)
Nonetheless if you have tips how to get dask workers into debug mode (with
pycharm) I'd be even more happy :D
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
<#4193 (comment)>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/AACKZTHXTTLIXQJJJP63MGLSNRM7XANCNFSM4TBKFTIQ>
.
|
Hi @mrocklin, I implemented your solution in #4221. However, there are still workers that end up freezing. import dask
import dask.distributed
import numpy as np
import time
cluster = dask.distributed.LocalCluster(n_workers=2, threads_per_worker=1, memory_limit="512M")
client = dask.distributed.Client(cluster)
x = {}
def memory_leaking_fn(data, sleep=0):
x[data] = np.random.randint(100, size=12 * 1024**2 // 8)
time.sleep(sleep)
return data
futures = client.map(memory_leaking_fn, range(1000))
futures = client.map(memory_leaking_fn, range(1000), np.repeat(0.1, 1000))
for f in futures:
print(f.result()) |
I'm curious, what happens? Are there any logs or signs about what might be wrong? |
@mrocklin it's the last remaining worker that never gets restarted. Basically, I can reproduce two scenarios:
This happens a bit randomly, depending on the time delay between both workers restart. (I edited the test script in my last post slightly: change the sleep time + have only two workers) |
Hey @Hoeze, the errors in your traceback look a lot like the kinds of problems that |
@KrishanBhasin I am currently running with a fork of d7f532c (see #4221). The workers still keep freezing and I cannot get my calculations to finish. |
Hi all, are there some news on this topic?
I imagine all those instabilities could be fixed by in-place worker restarts + having the nanny hold all results, as this would make them failure-proof to worker failures. Is there a way to push progress on this issue or get professional support on it? |
@mrocklin @Hoeze @quasiben @KrishanBhasin @jedbrown |
I end up getting a lot of error messages like this:
I'm relatively sure that this warning is actually true.
Also, the workers hitting this warning end up in idling all the time.
Is there a way to automatically retire + restart the workers that hit this error message?
Anything else we need to know?:
I'm using Dask v2.30
Environment:
The text was updated successfully, but these errors were encountered: