-
-
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
Computation deadlocks due to worker rapidly running out of memory instead of spilling #6110
Comments
Has anyone tried bisecting yet?
…On Tue, Apr 12, 2022 at 10:20 AM Florian Jetter ***@***.***> wrote:
The below script is pretty reliably triggering deadlocks. I'm sure this
can be reduced further but I haven't had time to do so, yet.
import coiled.v2from distributed import Clientcluster = coiled.v2.Cluster(
n_workers=20
)client = Client(cluster)
from distributed import Clientfrom dask.datasets import timeseriesddf = timeseries(
"2020",
"2025",
partition_freq='2w',
)ddf2 = timeseries(
"2020",
"2023",
partition_freq='2w',
)def slowident(df):
import random
import time
time.sleep(random.randint(1, 5))
return df
while True:
client.restart()
demo1 = ddf.map_partitions(slowident)
(demo1.x + demo1.y).mean().compute()
demo2 = ddf.merge(ddf2)
demo2 = demo2.map_partitions(slowident)
(demo2.x + demo2.y).mean().compute()
We could confirm that version 2022.1.1 is *not* affected by this but it
appears that all follow up versions might be affected (haven't tested all
of them, can definitely confirm for 2022.4.0)
—
Reply to this email directly, view it on GitHub
<#6110>, or unsubscribe
<https://github.com/notifications/unsubscribe-auth/AACKZTEFDNKXA5XGISTMBPLVEWIDNANCNFSM5THVRWOQ>
.
You are receiving this because you are subscribed to this thread.Message
ID: ***@***.***>
|
Do you have an estimate of what fraction of runs? As @mrocklin says, this script is ideal for a bisect (behind a timeout failure). |
I'm currently trying to run this locally without Coiled. I encourage other folks to hop on as well, but I'll at least try to run with this for the rest of the day when I have free time. |
I'm also curious about how often this fails. Currently running this locally and not running into anything. I'm wondering how long I should wait before moving up to Coiled. |
I'm getting these when running locally, but I suspect that they're due to being unclean on restart TracebackFuture exception was never retrieved
future: <Future finished exception=CommClosedError("Exception while trying to call remote method 'restart' before comm was established.")>
Traceback (most recent call last):
File "/home/mrocklin/workspace/distributed/distributed/comm/tcp.py", line 226, in read
frames_nbytes = await stream.read_bytes(fmt_size)
tornado.iostream.StreamClosedError: Stream is closed
The above exception was the direct cause of the following exception:
Traceback (most recent call last):
File "/home/mrocklin/workspace/distributed/distributed/core.py", line 864, in send_recv_from_rpc
result = await send_recv(comm=comm, op=key, **kwargs)
File "/home/mrocklin/workspace/distributed/distributed/core.py", line 709, in send_recv
response = await comm.read(deserializers=deserializers)
File "/home/mrocklin/workspace/distributed/distributed/comm/tcp.py", line 242, in read
convert_stream_closed_error(self, e)
File "/home/mrocklin/workspace/distributed/distributed/comm/tcp.py", line 150, in convert_stream_closed_error
raise CommClosedError(f"in {obj}: {exc}") from exc
distributed.comm.core.CommClosedError: in <TCP (closed) rpc.restart local=tcp://127.0.0.1:51626 remote=tcp://127.0.0.1:33741>: Stream is closed
The above exception was the direct cause of the following exception:
Traceback (most recent call last):
File "/home/mrocklin/mambaforge/lib/python3.9/site-packages/tornado/gen.py", line 769, in run
yielded = self.gen.throw(*exc_info) # type: ignore
File "/home/mrocklin/workspace/distributed/distributed/utils.py", line 220, in quiet
yield task
File "/home/mrocklin/mambaforge/lib/python3.9/site-packages/tornado/gen.py", line 762, in run
value = future.result()
File "/home/mrocklin/workspace/distributed/distributed/core.py", line 867, in send_recv_from_rpc
raise type(e)(
distributed.comm.core.CommClosedError: Exception while trying to call remote method 'restart' before comm was established.
2022-04-12 10:34:22,259 - tornado.application - ERROR - Exception in callback functools.partial(<bound method IOLoop._discard_future_result of <tornado.platform.asyncio.AsyncIOLoop object at 0x7f1dc605be80>>, <Task finished name='Task-33593' coro=<Scheduler.restart() done, defined at /home/mrocklin/workspace/distributed/distributed/scheduler.py:6061> exception=CommClosedError("Exception while trying to call remote method 'restart' before comm was established.")>)
Traceback (most recent call last):
File "/home/mrocklin/workspace/distributed/distributed/comm/tcp.py", line 226, in read
frames_nbytes = await stream.read_bytes(fmt_size)
tornado.iostream.StreamClosedError: Stream is closed
The above exception was the direct cause of the following exception:
Traceback (most recent call last):
File "/home/mrocklin/workspace/distributed/distributed/core.py", line 864, in send_recv_from_rpc
result = await send_recv(comm=comm, op=key, **kwargs)
File "/home/mrocklin/workspace/distributed/distributed/core.py", line 709, in send_recv
response = await comm.read(deserializers=deserializers)
File "/home/mrocklin/workspace/distributed/distributed/comm/tcp.py", line 242, in read
convert_stream_closed_error(self, e)
File "/home/mrocklin/workspace/distributed/distributed/comm/tcp.py", line 150, in convert_stream_closed_error
raise CommClosedError(f"in {obj}: {exc}") from exc
distributed.comm.core.CommClosedError: in <TCP (closed) rpc.restart local=tcp://127.0.0.1:51462 remote=tcp://127.0.0.1:35655>: Stream is closed
The above exception was the direct cause of the following exception:
Traceback (most recent call last):
File "/home/mrocklin/mambaforge/lib/python3.9/site-packages/tornado/ioloop.py", line 741, in _run_callback
ret = callback()
File "/home/mrocklin/mambaforge/lib/python3.9/site-packages/tornado/ioloop.py", line 765, in _discard_future_result
future.result()
File "/home/mrocklin/workspace/distributed/distributed/scheduler.py", line 6116, in restart
resps = await asyncio.wait_for(resps, timeout)
File "/home/mrocklin/mambaforge/lib/python3.9/asyncio/tasks.py", line 481, in wait_for
return fut.result()
File "/home/mrocklin/workspace/distributed/distributed/utils.py", line 207, in All
result = await tasks.next()
File "/home/mrocklin/workspace/distributed/distributed/core.py", line 867, in send_recv_from_rpc
raise type(e)(
distributed.comm.core.CommClosedError: Exception while trying to call remote method 'restart' before comm was established.
Task exception was never retrieved
future: <Task finished name='Task-33599' coro=<rpc.__getattr__.<locals>.send_recv_from_rpc() done, defined at /home/mrocklin/workspace/distributed/distributed/core.py:855> exception=CommClosedError("Exception while trying to call remote method 'restart' before comm was established.")>
Traceback (most recent call last):
File "/home/mrocklin/workspace/distributed/distributed/comm/tcp.py", line 226, in read
frames_nbytes = await stream.read_bytes(fmt_size)
tornado.iostream.StreamClosedError: Stream is closed
The above exception was the direct cause of the following exception:
Traceback (most recent call last):
File "/home/mrocklin/workspace/distributed/distributed/core.py", line 864, in send_recv_from_rpc
result = await send_recv(comm=comm, op=key, **kwargs)
File "/home/mrocklin/workspace/distributed/distributed/core.py", line 709, in send_recv
response = await comm.read(deserializers=deserializers)
File "/home/mrocklin/workspace/distributed/distributed/comm/tcp.py", line 242, in read
convert_stream_closed_error(self, e)
File "/home/mrocklin/workspace/distributed/distributed/comm/tcp.py", line 150, in convert_stream_closed_error
raise CommClosedError(f"in {obj}: {exc}") from exc
distributed.comm.core.CommClosedError: in <TCP (closed) rpc.restart local=tcp://127.0.0.1:50694 remote=tcp://127.0.0.1:35275>: Stream is closed
The above exception was the direct cause of the following exception:
Traceback (most recent call last):
File "/home/mrocklin/workspace/distributed/distributed/core.py", line 867, in send_recv_from_rpc
raise type(e)(
distributed.comm.core.CommClosedError: Exception while trying to call remote method 'restart' before comm was established.
Task exception was never retrieved
future: <Task finished name='Task-33600' coro=<rpc.__getattr__.<locals>.send_recv_from_rpc() done, defined at /home/mrocklin/workspace/distributed/distributed/core.py:855> exception=CommClosedError("Exception while trying to call remote method 'restart' before comm was established.")>
Traceback (most recent call last):
File "/home/mrocklin/workspace/distributed/distributed/comm/tcp.py", line 226, in read
frames_nbytes = await stream.read_bytes(fmt_size)
tornado.iostream.StreamClosedError: Stream is closed
The above exception was the direct cause of the following exception:
Traceback (most recent call last):
File "/home/mrocklin/workspace/distributed/distributed/core.py", line 864, in send_recv_from_rpc
result = await send_recv(comm=comm, op=key, **kwargs)
File "/home/mrocklin/workspace/distributed/distributed/core.py", line 709, in send_recv
response = await comm.read(deserializers=deserializers)
File "/home/mrocklin/workspace/distributed/distributed/comm/tcp.py", line 242, in read
convert_stream_closed_error(self, e)
File "/home/mrocklin/workspace/distributed/distributed/comm/tcp.py", line 150, in convert_stream_closed_error
raise CommClosedError(f"in {obj}: {exc}") from exc
distributed.comm.core.CommClosedError: in <TCP (closed) rpc.restart local=tcp://127.0.0.1:45222 remote=tcp://127.0.0.1:37231>: Stream is closed
The above exception was the direct cause of the following exception: |
Shifting to #5951 , which seems easier to iterate on |
Shifting back. I'm now running this at scale on Coiled. I'm not yet seeing a deadlock. @fjetter can you say a bit more about how often you ran into this? |
We reverted the default environment on coiled so the above will not trigger an issue anymore. With 2022.4.1 it will deadlock within seconds / few minutes |
I've been running this on Coiled. It reliably deadlocks within 2-3 iterations. Scriptimport coiled.v2
from distributed import Client
cluster = coiled.v2.Cluster(
name="florian-deadlock",
n_workers=20,
software="florian-deadlock",
# environ=dict(
# DASK_LOGGING__DISTRIBUTED="debug",
# DASK_DISTRIBUTED__ADMIN__LOG_LENGTH="1000000"
# )
)
client = Client(cluster)
from dask.datasets import timeseries
ddf = timeseries(
"2020",
"2025",
partition_freq="2w",
)
ddf2 = timeseries(
"2020",
"2023",
partition_freq="2w",
)
def slowident(df):
import random
import time
time.sleep(random.randint(1, 5))
return df
i = 1
while True:
print(f"Iteration {i}")
client.restart()
demo1 = ddf.map_partitions(slowident)
(demo1.x + demo1.y).mean().compute()
demo2 = ddf.merge(ddf2)
demo2 = demo2.map_partitions(slowident)
(demo2.x + demo2.y).mean().compute()
i += 1 [tool.poetry]
name = "florian-deadlock"
version = "0.1.0"
description = ""
authors = ["Gabe Joseph <gjoseph92@gmail.com>"]
[tool.poetry.dependencies]
python = "=3.9.1"
dask = {extras = ["complete"], version = "^2022.4.0"}
ipython = "^8.2.0"
coiled = "^0.0.73"
scheduler-profilers = {git = "https://github.com/gjoseph92/scheduler-profilers.git", rev = "main"}
[tool.poetry.dev-dependencies]
[build-system]
requires = ["poetry-core>=1.0.0"]
build-backend = "poetry.core.masonry.api" #!/bin/bash
cat > postbuild.sh <<EOF
#!/bin/bash
pip install --no-deps "git+https://github.com/dask/distributed.git@main"
echo "export DASK_LOGGING__DISTRIBUTED=debug" >> ~/.bashrc
echo "export DASK_DISTRIBUTED__ADMIN__LOG_LENGTH=1000000" >> ~/.bashrc
EOF
poetry export --without-hashes > requirements.txt
coiled env create -n florian-deadlock --pip requirements.txt --post-build postbuild.sh
rm requirements.txt
rm postbuild.sh I've tried distributed from main to get #6112. Still deadlocks. The overall pattern I'm seeing is at least 1 worker which is effectively dead, but hasn't been removed from the scheduler yet. Other works seem to still want to fetch data from that worker. In [9]: client.run(lambda dask_worker: [ts.who_has for ts in dask_worker.data_needed], workers=['tls://10.6.0.70:34327'])
Out[9]:
{'tls://10.6.0.70:34327': [{'tls://10.6.12.222:42533'},
{'tls://10.6.12.222:42533'},
{'tls://10.6.12.222:42533'}]}
# notice the `who_has` addresses are all the same, and all reference the unresponsive red worker above Here are logs (via Coiled) from one of the unresponsive workers: 2022-04-14 18:55:24,233 - distributed.worker - ERROR - Worker stream died during communication: tls://10.6.3.191:45383
Traceback (most recent call last):
File "/opt/conda/envs/coiled/lib/python3.9/site-packages/tornado/iostream.py", line 867, in _read_to_buffer
bytes_read = self.read_from_fd(buf)
File "/opt/conda/envs/coiled/lib/python3.9/site-packages/tornado/iostream.py", line 1592, in read_from_fd
return self.socket.recv_into(buf, len(buf))
File "/opt/conda/envs/coiled/lib/python3.9/ssl.py", line 1241, in recv_into
return self.read(nbytes, buffer)
File "/opt/conda/envs/coiled/lib/python3.9/ssl.py", line 1099, in read
return self._sslobj.read(len, buffer)
ConnectionResetError: [Errno 104] Connection reset by peer
The above exception was the direct cause of the following exception:
Traceback (most recent call last):
File "/opt/conda/envs/coiled/lib/python3.9/site-packages/distributed/worker.py", line 3019, in gather_dep
response = await get_data_from_worker(
File "/opt/conda/envs/coiled/lib/python3.9/site-packages/distributed/worker.py", line 4320, in get_data_from_worker
return await retry_operation(_get_data, operation="get_data_from_worker")
File "/opt/conda/envs/coiled/lib/python3.9/site-packages/distributed/utils_comm.py", line 381, in retry_operation
return await retry(
File "/opt/conda/envs/coiled/lib/python3.9/site-packages/distributed/utils_comm.py", line 366, in retry
return await coro()
File "/opt/conda/envs/coiled/lib/python3.9/site-packages/distributed/worker.py", line 4300, in _get_data
response = await send_recv(
File "/opt/conda/envs/coiled/lib/python3.9/site-packages/distributed/core.py", line 709, in send_recv
response = await comm.read(deserializers=deserializers)
File "/opt/conda/envs/coiled/lib/python3.9/site-packages/distributed/comm/tcp.py", line 242, in read
convert_stream_closed_error(self, e)
File "/opt/conda/envs/coiled/lib/python3.9/site-packages/distributed/comm/tcp.py", line 148, in convert_stream_closed_error
raise CommClosedError(f"in {obj}: {exc.__class__.__name__}: {exc}") from exc
distributed.comm.core.CommClosedError: in <TLS (closed) Ephemeral Worker->Worker for gather local=tls://10.6.3.169:44316 remote=tls://10.6.3.191:45383>: ConnectionResetError: [Errno 104] Connection reset by peer
2022-04-14 18:56:15,295 - distributed.worker - INFO - Stopping worker at tls://10.6.3.169:43833
2022-04-14 18:56:15,318 - distributed.nanny - INFO - Worker closed
2022-04-14 18:56:16,231 - distributed.worker - INFO - Start worker at: tls://10.6.3.169:40133
2022-04-14 18:56:16,231 - distributed.worker - INFO - Listening to: tls://10.6.3.169:40133
2022-04-14 18:56:16,231 - distributed.worker - INFO - dashboard at: 10.6.3.169:39063
2022-04-14 18:56:16,231 - distributed.worker - INFO - Waiting to connect to: tls://34.221.160.71:8786
2022-04-14 18:56:16,231 - distributed.worker - INFO - -------------------------------------------------
2022-04-14 18:56:16,231 - distributed.worker - INFO - Threads: 2
2022-04-14 18:56:16,233 - distributed.worker - INFO - Memory: 3.78 GiB
2022-04-14 18:56:16,233 - distributed.worker - INFO - Local Directory: /dask-worker-space/worker-c7603wej
2022-04-14 18:56:16,234 - distributed.worker - INFO - -------------------------------------------------
2022-04-14 18:56:16,477 - distributed.worker - WARNING - Mismatched versions found
+-------------+-----------------------+-----------------------+---------------------------------------+
| Package | This Worker | scheduler | workers |
+-------------+-----------------------+-----------------------+---------------------------------------+
| blosc | MISSING | MISSING | {None, 'MISSING'} |
| distributed | 2022.4.0+33.g41ecbca2 | 2022.4.0+33.g41ecbca2 | {'2022.4.0+33.g41ecbca2', '2022.4.0'} |
| python | 3.9.12.final.0 | 3.9.12.final.0 | {'3.9.12.final.0', '3.9.1.final.0'} |
+-------------+-----------------------+-----------------------+---------------------------------------+
2022-04-14 18:56:16,478 - distributed.worker - INFO - Starting Worker plugin coiled-aws-env
2022-04-14 18:56:16,480 - distributed.worker - INFO - Registered to: tls://34.221.160.71:8786
2022-04-14 18:56:16,482 - distributed.worker - INFO - -------------------------------------------------
2022-04-14 18:56:16,483 - distributed.core - INFO - Starting established connection You'll notice:
I'm seeing the nanny-restart for all three unresponsive workers in my most recent run. Other workers don't show that pattern. I feel like the reconnection could have something to do with it. I'm also confused why the workers seem to think they've reconnected to the scheduler, yet the scheduler still sees their I have cluster dumps, but I've found them less useful so far than just poking at things manually. |
I'm playing with this. I've gotten only about as far as @gjoseph92 has. I'm seeing the same thing. I don't see any reason why the worker would have restarted, which is also strange. (I'm not seeing the same comm issue) Logs
I see a few things to inspect:
None of these will find the answer, but I think that they're all increase visibility. These are my next steps. I'm also open to other suggestions. |
Also, two quick notes, my recent PRs don't seem to help with this. If I use m5.large instances (my default type) this problem doesn't occur. My guess is that there is some disk thing happening here. We're running out of space and so failing hard when trying to touch disk. I'm going to aim towards that as a cause and just try to increase visibility along the way. |
cc @ntabris without SSDs Coiled instances have 10GB drives right now. Is that right? |
No, prod and staging are still putting 100GB boot (EBS) drives on all instances. |
Good to know. It doesn't seem like we could dump that much data this quickly with this workload. I did throw up a test to see what happens with Disk stops cooperating. We're not graceful here. #6128 I also tried to see what happens when we fail writing, and things were actually ok. |
Oh, I'm being dumb. The closed workers are due to the client.restart command 🤦 |
Sorry, should have posted that. I was also confused until I actually read the code 😁 |
So if I set |
Setting I'm going to switch away from |
Nope. Still an issue with m5's. t-type nodes and disk are both in the clear it seems. I can reliably recreate the situation way more easily now though 🙂 |
Playing with triggering worker failures. I was able to kill workers in a way such that they didn't come back somehow.
|
After merging in a bunch of things to make the other failures go away I'm back at this issue. I tried running with I tried checking to see if the event loop was stuck on something by sending in a plugin that writes what's going on in a separate thread from dask.distributed import WorkerPlugin
from dask.utils import parse_timedelta
import threading
import time
from distributed import profile
import sys
class PrintThread(WorkerPlugin):
name = "print-event-loop-thread"
def __init__(self, delay="10s"):
self.delay = parse_timedelta(delay)
async def setup(self, worker):
self.worker = worker
self.thread_id = threading.get_ident()
self.stop = threading.Event()
self._thread = threading.Thread(target=self.go, daemon=True)
self._thread.start()
def go(self):
while True:
frame = sys._current_frames()[self.thread_id]
print("-" * 40)
for line in profile.call_stack(frame):
print(line, end="")
if self.stop.wait(timeout=self.delay):
break
def teardown(self, worker):
self.stop.set()
self._thread.join()
client.register_worker_plugin(PrintThread("5s")) And while we get interesting output in logs, we stop getting that output once the worker hangs. So the problem isn't in the worker necessarily. I would expect this to run in Python pretty much no matter what. The problem is deeper I think. |
On restart the frozen workers are cleared out of the scheduler (which makes sense, it cleans metadata) but they don't come back. This means that even the Nanny is borked. Something is wrong with these instances I think. Coiled folks, I might want to engage you all on this next week. |
Example failed cluster: https://cloud.coiled.io/mrocklin/clusters/7589/details |
Yup, I just changed the worker plugin to a nanny plugin and we lose track of any ongoing output. Somehow this process has frozen in such a way that the network connection isn't severed (Dask would react well in this case) but that Python itself is also totally non-responsive. This is atypical. |
Sorry, and not just the worker process (which might have some crazy user code or something (but in this case it's just pandas)) but also the nanny process, which generally doesn't do anything at all. Something has frozen the entire box somehow. If I were a cloud provider, I might freeze VMs in this way if they were behaving badly. |
I connected to the serial console and it was logging something like this quickly and repeatedly:
After a while dask did stop with message below and the VM started the shutdown process after that:
|
yeah, so the repeated thing is the plugin. I'm asking it to report what the main event loop thread is doing every 10s. My experience with the downloaded logs is that once the worker seems to stall, those logs stop coming. Is what you're saying that it is still consistently printing every ten seconds? Or is there possibly a gap of a few minutes? If it's consistently printing every ten seconds then what is happening that they're no longer showing up in the downloaded logs? |
I don't ask because I care about the downloadable logs (although I do) I ask because something is happening at that time that is causing the whole worker to appear to freeze. |
Confirmed. I've run the reproducer on #6174 and workers aren't locking up anymore. So that's clearly the problem, but I'm not sure yet what the best solution is. Blocking the event loop like so to prevent more computations or fetches from running doesn't seem like the best long-term strategy though? Especially once disk access becomes async, this won't work. Related:
Maybe we'd want to temporarily pause the worker (once pausing includes pausing Also, though distributed can clearly be better about preventing workers from getting into this high-memory state (where the OS is probably flailing so much that nothing can run?), it's always possible that user code could do this on its own. So we should be more robust to it. That includes #6148 and #6159 (comment), but also looking into why the nanny doesn't/can't kill the worker in this situation. Perhaps Nannies should use ulimits on the child processes, instead of relying on the application-level periodic callback for termination? |
Hooray! Rejoice!
…On Thu, Apr 21, 2022, 5:58 PM Gabe Joseph ***@***.***> wrote:
Confirmed. I've run the reproducer on #6174
<#6174> and workers aren't
locking up anymore. So that's clearly the problem, but I'm not sure yet
what the best solution is.
Blocking the event loop like so to prevent more computations or fetches
from running doesn't seem like the best long-term strategy though?
Especially once disk access becomes async, this won't work. Related:
- #4424 <#4424>
- #5891 <#5891>
- #5702 <#5702>
- #5736 <#5736>
Maybe we'd want to temporarily pause the worker (once pausing includes
pausing gather_deps) until all the evictions have processed?
Also, though distributed can clearly be better about preventing workers
from getting into this high-memory state (where the OS is probably flailing
so much that nothing can run?), it's always possible that user code could
do this on its own. So we should be more robust to it. That includes #6148
<#6148> and #6159 (comment)
<#6159 (comment)>,
but also looking into why the nanny doesn't/can't kill the worker in this
situation.
Perhaps Nannies should use ulimits on the child processes, instead of
relying on the application-level periodic callback for termination?
—
Reply to this email directly, view it on GitHub
<#6110 (comment)>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/AACKZTE22ASKNMV5URTGFETVGHMPJANCNFSM5THVRWOQ>
.
You are receiving this because you were mentioned.Message ID:
***@***.***>
|
I would be curious how this would react if we weren't writing to EBS volumes, but were instead writing to SSDs. @ntabris can you provide an example coiled command to get a machine with SSDs? Then maybe @gjoseph92 could you try with those? Do you have any reason to think that they might behave differently? I'm curious why this comes up on Coiled but not as often on other systems. |
Something like |
Thanks Nat!
…On Fri, Apr 22, 2022 at 1:26 PM Nat Tabris ***@***.***> wrote:
Something like ClusterBeta(n_workers=2, worker_vm_types=["i3.large"]) or (
i3.xlarge) should work. Those each have large NVMe drive that will be
used for dask temp storage. i3.large is 2 vCPU, 15.25GiB mem; i3.xlarge
is 4 vCPU, 30.5GiB.
—
Reply to this email directly, view it on GitHub
<#6110 (comment)>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/AACKZTHDVL6FE63BMP2PUZLVGLVOPANCNFSM5THVRWOQ>
.
You are receiving this because you were mentioned.Message ID:
***@***.***>
|
Thanks! (Note that testing this will require tweaking the size of the dataframes to ensure data is still being spilled—don't assume that just because the script works on an |
@fjetter for posterity & future searches, could you rename this to "Computation deadlocks due to worker rapidly running out of memory instead of spilling" or something like that, now that we've found a root cause? Also, to summarize:
|
Done |
Update here others might find interesting: the lock-up likely is caused by swapping, but not in the way you'd think. It points to #6177 being a very worthwhile thing to do. #4345 (comment) got my curious: yeah, why isn't the kernel's OOMKiller getting involved here and just killing the Python process that's using too much memory? I spun up a Let's see what happens when we use up some memory (based on https://unix.stackexchange.com/a/254976). ubuntu@ip-10-0-9-34:~$ cat <( </dev/zero head -c 3GiB) <(sleep 10) | tail
# 3GiB goes fine, unsurprisingly.
ubuntu@ip-10-0-9-34:~$ cat <( </dev/zero head -c 4GiB) <(sleep 10) | tail
Killed
# 4GiB is more than the total physically available space. So this also goes well.
ubuntu@ip-10-0-9-34:~$ cat <( </dev/zero head -c 3200MiB) <(sleep 10) | tail
# This has now been unresponsive for 12min!
# My SSH connection is still up, but tmux panes are unresponsive.
# Clearly this is very locked up, because the 10-sec timeout isn't even working It turns out the OOMKiller is very conservative and doesn't kill things even when system performance is extremely degraded, so long as there are any pages it can reclaim. And even without a swap partition, the kernel can and will still swap pages to disk if they were on disk to begin with—such as executable code. (See https://askubuntu.com/a/432827 for an explanation.) Really, this just means the kernel is willing to remove almost everything from the disk cache, so executing new instructions in the Python interpreter or any other executables means constantly reloading pages from disk. It's all still running... just really... really... slowly. Basically, getting into the "almost all physical memory is used, but not quite all of it" terrain leads to very bad behavior. People complain/rant about this a bit: https://unix.stackexchange.com/questions/373312/oom-killer-doesnt-work-properly-leads-to-a-frozen-os, https://serverfault.com/questions/390623/how-do-i-prevent-linux-from-freezing-when-out-of-memory, etc. There doesn't seem to be a good answer for how to avoid this in linux generically. But we don't have to worry about the generic case. For dask, we can reasonably expect that the worker process tree is the only thing allocating significant amounts of memory. If we just set hard memory limits at the OS level (#6177) and don't let dask go above 95% of physically available memory (our current default), we should be able to terminate before we get into this thrashing scenario. And if we don't set hard limits at the OS level (we currently don't), we should have no reason to believe that we'd be able to successfully prevent or recover from this via our application-level memory monitor. Prevention is the only option, because once we get into this state, no userland processes can run effectively. |
Whoa, nice find. The cat-sleep example is a super-clear demonstration. I'm open to thinking about cgroups at the Dask level. If that doesn't work for some reason, we could probably also consider handling this on the Coiled side. A couple of thoughts:
|
Coiled actually used to do this (via Docker), but it was removed a while ago so that if you got heterogeneous node sizes, you'd fully use the memory on all of them (why this had to be removed is a bit of a Coiled implementation detail though). But I think Dask should do this. We already offer a memory limit parameter; this would just be an implementation fix to make it work correctly. |
I haven't tried this but it sounds like it's not recommended. In my mind, one big reason to be in the cloud is that it's really easy to scale up if you (say) need more memory. Is there a reason someone would prefer slow disk swap rather than instance with more memory? (Maybe if they only needed swap space very occasionally? Or didn't realize they'd need more memory but don't want their workers to die?) |
Great find @gjoseph92 . Just to summarize what I understand. We saw this issue appear in the 2022.4.? release but didn't in |
Maybe #5971? I'd be curious what it looks like setting the env var manually on the cluster (easy with Coiled). I'd be unsurprised to hear it's something else though. |
I'm running into a similar issue here where a dask-geopandas computation is 'deadlocking' without error messages. A worker is running out of memory and the deadlocking task is getting shipped around workers. More info here: geopandas/dask-geopandas#197 |
Ok, it seems unlikely that we're putting that much data that quickly on a
node. It does feel like the worker logic has been interrupted by something
similar though. This will be an interesting hunt. I'll try playing with a
few parameters of the experiment and see if there's a good pattern on when
it occurs. Thanks!
…On Thu, Apr 14, 2022, 6:09 PM Nat Tabris ***@***.***> wrote:
without SSDs Coiled instances have 10GB drives right now. Is that right?
No, prod and staging are still putting 100GB boot (EBS) drives on all
instances.
—
Reply to this email directly, view it on GitHub
<#6110 (comment)>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/AACKZTHXF4OG5NXJQG4UO7TVFCQTBANCNFSM5THVRWOQ>
.
You are receiving this because you were mentioned.Message ID:
***@***.***>
|
The below script is pretty reliably triggering deadlocks. I'm sure this can be reduced further but I haven't had time to do so, yet.
We could confirm that version
2022.1.1
is not affected by this but it appears that all follow up versions might be affected (haven't tested all of them, can definitely confirm for2022.4.0
)The text was updated successfully, but these errors were encountered: