-
-
Notifications
You must be signed in to change notification settings - Fork 719
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
Worker can memory overflow by fetching too much data at once #6208
Comments
I think it's also likely this wouldn't always kill the worker, but freeze it like in #6110. If the 25GiB come somewhat staggered in those 500MB increments, physical memory usage will creep up very close to the 10GB max, without actually requesting more than it. Once it gets close enough, the system will thrash to the point that everything stops, so more memory isn't allocated (which would put it over the limit and kill it). The smaller the key size and larger the key count, this more likely this is to happen (probably why a shuffle was a good reproducer). Since we know the (approximate) size of all the keys we're fetching, this seems like something we can avoid doing in the first place (don't fetch more than we have memory capacity for, as a start). Maybe also a use-case for receiving data directly to disk #5996? |
pseudocode: if current managed_in_memory + self.comm_nbytes > threshold% * max_memory:
stop scheduling further gather_dep's or with better encapsulation: if self.data.would_exceed_threshold(self.comm_nbytes):
stop scheduling further gather_dep's There are two ways to resume gathering:
Note that the above mitigates the issue, but doesn't actually prevent the use case that @fjetter showed above in case of a task that individually requires more memory than what's available: CHUNK_SIZE = 500 // 8 * 2**20 # 500 MiB
a = da.random.random(CHUNK_SIZE * 20, CHUNK_SIZE) # 10 GiB total
a = a.persist()
f = client.submit(lambda _: None, futures_of(a)) What will happen is that whatever worker runs f will start accumulating all chunks of a locally.
And this is not considering unrelated unmanaged memory - e.g. on a multithreaded worker, there may be other tasks running. |
Two thoughts:
If planning for worst case would cripple things (You must have 25GB of memory free before you do any work!) then I would not want to plan for worst case. I think that we could reduce this burden in a few ways:
Just some thoughts to reduce the burden here. Please don't let these distract too much from the main conversation. |
The receiving worker already knows in advance how large the data will be before it asks for it. That's returned by
I would not want to count workers towards the maximum if they are going to send me only a few kilobytes . |
Ah, great then. |
In coiled/benchmarks#229, we have found a reliable reproducer for a worker getting EDIT: Apart from 100 MiB being the default chosen by |
One quick yet imperfect fix might be to introduce a parameter that limits the amount of memory we want to dedicate to communications, effectively adjusting the above if-clause to if (
(
len(self.in_flight_workers) >= self.total_out_connections
and self.comm_nbytes >= self.comm_threshold_bytes
) or self.comm_nbytes >= self.comm_memory_limit
):
return {}, [] This value could be set as a percentage of To better understand the burden comms put onto our workers, we may also want to add metrics about the number of open connections and the size of In addition, we may want to drop cc @fjetter |
It's also worth noting that the same issue exists for workers sending data. There's a limit to the number of incoming connections a worker can have at once (default 10), but not the number of bytes it can be sending at once. If a worker is paused, it abruptly limits itself to 1 connection, but that message could still be large. Serialization may not be zero-copy. If compression is used, it's never zero-copy. TLS comms will also require a copy in the SSL buffer. So even if the sent data is already in memory, we could potentially be doubling (or more) memory usage of that data until the other worker confirms it received the message. The worst case is when the data being sent is currently spilled to disk. Ideally, a worker shouldn't be willing to send more than |
I wrote this a big ago, but I'd like to point to #6212 as a more holistic approach to this problem. The overall idea is that:
It's common to use a synchronization primitive like a semaphore or |
Not quite correct. If unspilling a key causes the reported managed memory to get above The mechanism you mention happens specifically and only when managed memory is below 60% and process memory is above |
We do not know how much data sending/receiving actually requires, nor do we know how much a user task requires. Without reliable measurements I would prefer sticking to a pragmatic, low complexity approach for now. I do believe that simply introducing a threshold as proposed by @hendrikmakait in #6208 (comment) will improve things significantly already. I'm not sure if an absolute threshold, a relative (to memory_limit) threshold or a dynamic threshold (percentage of free/available memory) fits best. My gut feeling tells me to start with a relative threshold (based on memory_limit). Together with the added instrumentation, we can iterate on this choice incrementally, if necessary. If there are still significant shortcomings afterwards, we can think about a more complex solution. |
This is in, by now |
There is a possibility for a worker to suicide by fetching too much data at once
Old version:
distributed/distributed/worker.py
Lines 2754 to 2757 in 84cbb09
New version:
distributed/distributed/worker_state_machine.py
Lines 1425 to 1429 in 3647cfe
Simple example
Let's assume that keys have on average ~500MB
Local memory available is ~10GB
total_out_connections: 50 (default)
That would cause us to fetch up to 25GB at once which would kill the worker immediately.
This problem is exacerbated if the keys data sizes are underestimated (e.g. bias/bug in dask.sizeof)
cc @crusaderky
The text was updated successfully, but these errors were encountered: