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

Scheduler stops itself due to idle timeout, even though workers should still be working #5675

Open
gjoseph92 opened this issue Jan 20, 2022 · 7 comments
Labels
bug Something is broken needs info Needs further information from the user stability Issue or feature related to cluster stability (e.g. deadlock)

Comments

@gjoseph92
Copy link
Collaborator

gjoseph92 commented Jan 20, 2022

A user has reported that a long-running scheduler (up for ~18h) appears to have shut itself down because it thought it was idle. However, the scheduler had plenty of work left to do.

The scheduler logs looked like:

distributed.core - INFO - Event loop was unresponsive in Scheduler for 4.35s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.
distributed.core - INFO - Event loop was unresponsive in Scheduler for 4.60s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.
distributed.core - INFO - Event loop was unresponsive in Scheduler for 3.08s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.
distributed.core - INFO - Event loop was unresponsive in Scheduler for 3.24s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.
distributed.core - INFO - Event loop was unresponsive in Scheduler for 3.48s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.
distributed.core - INFO - Event loop was unresponsive in Scheduler for 3.76s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.
distributed.core - INFO - Event loop was unresponsive in Scheduler for 3.74s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.
distributed.core - INFO - Event loop was unresponsive in Scheduler for 4.22s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.
distributed.core - INFO - Event loop was unresponsive in Scheduler for 4.61s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.
distributed.core - INFO - Event loop was unresponsive in Scheduler for 4.93s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.
distributed.core - INFO - Event loop was unresponsive in Scheduler for 5.44s.  This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.
distributed.scheduler - INFO - Scheduler closing after being idle for 300.00 s
distributed.scheduler - INFO - Scheduler closing...
distributed.scheduler - INFO - Scheduler closing all comms
distributed.scheduler - INFO - Remove worker <WorkerState 'tcp://172.31.10.95:34519', name: tcp://172.31.10.95:34519, status: running, memory: 139350, processing: 0>
distributed.core - INFO - Removing comms to tcp://172.31.10.95:34519
distributed.scheduler - INFO - Remove worker <WorkerState 'tcp://172.31.31.188:40365', name: tcp://172.31.31.188:40365, status: running, memory: 131425, processing: 1>
distributed.core - INFO - Removing comms to tcp://172.31.31.188:40365
distributed.scheduler - INFO - Remove worker <WorkerState 'tcp://172.31.7.23:33299', name: tcp://172.31.7.23:33299, status: running, memory: 133166, processing: 2>
distributed.core - INFO - Removing comms to tcp://172.31.7.23:33299
distributed.scheduler - INFO - Remove worker <WorkerState 'tcp://172.31.11.172:34905', name: tcp://172.31.11.172:34905, status: running, memory: 141044, processing: 3>
distributed.core - INFO - Removing comms to tcp://172.31.11.172:34905
distributed.scheduler - INFO - Remove worker <WorkerState 'tcp://172.31.32.43:40345', name: tcp://172.31.32.43:40345, status: running, memory: 128135, processing: 4>
distributed.core - INFO - Removing comms to tcp://172.31.32.43:40345
distributed.scheduler - INFO - Remove worker <WorkerState 'tcp://172.31.2.153:40059', name: tcp://172.31.2.153:40059, status: running, memory: 135440, processing: 5>
distributed.core - INFO - Removing comms to tcp://172.31.2.153:40059
distributed.scheduler - INFO - Remove worker <WorkerState 'tcp://172.31.33.86:43825', name: tcp://172.31.33.86:43825, status: running, memory: 133801, processing: 7>
distributed.core - INFO - Removing comms to tcp://172.31.33.86:43825
distributed.scheduler - INFO - Remove worker <WorkerState 'tcp://172.31.30.46:40461', name: tcp://172.31.30.46:40461, status: running, memory: 131706, processing: 3>
distributed.core - INFO - Removing comms to tcp://172.31.30.46:40461
distributed.scheduler - INFO - Remove worker <WorkerState 'tcp://172.31.43.190:45543', name: tcp://172.31.43.190:45543, status: running, memory: 135540, processing: 2>
distributed.core - INFO - Removing comms to tcp://172.31.43.190:45543
distributed.scheduler - INFO - Remove worker <WorkerState 'tcp://172.31.0.126:44369', name: tcp://172.31.0.126:44369, status: running, memory: 134745, processing: 12>
distributed.core - INFO - Removing comms to tcp://172.31.0.126:44369
distributed.scheduler - INFO - Remove worker <WorkerState 'tcp://172.31.31.190:44953', name: tcp://172.31.31.190:44953, status: running, memory: 134171, processing: 12>
distributed.core - INFO - Removing comms to tcp://172.31.31.190:44953
distributed.scheduler - INFO - Remove worker <WorkerState 'tcp://172.31.26.162:45605', name: tcp://172.31.26.162:45605, status: running, memory: 126044, processing: 14>
distributed.core - INFO - Removing comms to tcp://172.31.26.162:45605

Without timestamps (#4762), we don't know if those "event loop unresponsive" pauses were related and happened immediately prior to the idle shutdown, or if they were long before.

The confusing thing is that the idle shutdown only happens if no workers are processing any tasks (and there are no unrunnable tasks). Meaning that, during every check for 300s, either:

  1. The scheduler thought no workers had processing tasks
  2. No workers were connected

check_idle code, for reference:

def check_idle(self):
parent: SchedulerState = cast(SchedulerState, self)
ws: WorkerState
if (
any([ws._processing for ws in parent._workers_dv.values()])
or parent._unrunnable
):
self.idle_since = None
return
elif not self.idle_since:
self.idle_since = time()
if time() > self.idle_since + self.idle_timeout:
logger.info(
"Scheduler closing after being idle for %s",
format_time(self.idle_timeout),
)
self.loop.add_callback(self.close)

Yet in the logs immediately following from close, we can see:

  1. Almost all workers do have some tasks processing (processing: 2, processing: 12, etc.)
  2. There are clearly workers connected

One thing I do notice is that time() is not monotonic (we should use time.monotonic() instead, xref #4528). So in theory, if the system clock changed (possible on that long-running of a scheduler?), we might not be waiting the full 300s. That still doesn't explain how we got in a situation where the scheduler thought there were no workers processing, though—but if that situation happened to overlap with a system clock forward-jump, the shutdown could get triggered immediately instead of actually requiring 300s to pass.

One other thing to note is that (according to logs), over the lifetime of the cluster, workers connected 114 times, and disconnected 79 times. So there was a lot of worker churn. When a worker disconnects, you'd see logs like:

distributed.scheduler - INFO - Remove worker <WorkerState 'tcp://172.31.36.136:32833', name: tcp://172.31.36.136:32833, status: closing, memory: 15652, processing: 2214>
distributed.core - INFO - Removing comms to tcp://172.31.36.136:32833
distributed.batched - INFO - Batched Comm Closed <TCP (closed) Scheduler connection to worker local=tcp://172.31.35.227:8786 remote=tcp://172.31.39.2:46620>
Traceback (most recent call last):
  File "/usr/local/lib/python3.8/site-packages/distributed/batched.py", line 93, in _background_send
    nbytes = yield self.comm.write(
  File "/usr/local/lib/python3.8/site-packages/tornado/gen.py", line 762, in run
    value = future.result()
  File "/usr/local/lib/python3.8/site-packages/distributed/comm/tcp.py", line 248, in write
    raise CommClosedError()
distributed.comm.core.CommClosedError
distributed.batched - INFO - Batched Comm Closed <TCP (closed) Scheduler connection to worker local=tcp://172.31.35.227:8786 remote=tcp://172.31.45.150:36594>

I mention this also because there are known issues around BatchedSend reconnecting (#5481, review of #5457 for more issues). I'm not sure whether it matters in this particular case though.

Note that these numbers leave 35 workers unaccounted for. That is, after the Scheduler closing after being idle for 300.00 s message, we see 12 Remove worker messages. However, by my count of how many times the scheduler registered a worker, we should have seen 47 Remove worker messages at the end. (When the cluster first started, 47 workers connected, so this number lines up.)

cc @sevberg

@gjoseph92 gjoseph92 added bug Something is broken stability Issue or feature related to cluster stability (e.g. deadlock) labels Jan 20, 2022
@gjoseph92
Copy link
Collaborator Author

Just want to note that with such a long-running scheduler, #5667 could be an issue too. Didn't see anything in the logs related to client disconnect, though.

@fjetter
Copy link
Member

fjetter commented Jun 14, 2022

Sorry for the late reply. What version was this running on?
Are you still experiencing issues on more recent versions?

@fjetter fjetter added the needs info Needs further information from the user label Jun 14, 2022
@gjoseph92
Copy link
Collaborator Author

@sevberg I don't think we were ever able to reproduce it, right? Have you still been having problems with it?

@dthompson-maystreet
Copy link

dthompson-maystreet commented Oct 17, 2022

Sorry to add a "me too" but we're seeing the same thing; the scheduler is shutting down even if there's work pending...

distributed.worker - INFO -         Registered to:       tcp://127.0.0.1:8786
distributed.worker - INFO - -------------------------------------------------
distributed.core - INFO - Starting established connection
distributed.scheduler - INFO - Remove client Client-0c79b08b-4e35-11ed-8042-ce99a9cff7ff
distributed.scheduler - INFO - Remove client Client-0c79b08b-4e35-11ed-8042-ce99a9cff7ff
distributed.scheduler - INFO - Close client connection: Client-0c79b08b-4e35-11ed-8042-ce99a9cff7ff
distributed.scheduler - INFO - Receive client connection: Client-5b3a832b-4e35-11ed-861a-0a58a9feac02
distributed.core - INFO - Starting established connection
distributed.scheduler - INFO - Receive client connection: Client-746f0016-4e35-11ed-861a-0a58a9feac02
distributed.core - INFO - Starting established connection
distributed.scheduler - INFO - Receive client connection: Client-b8936600-4e35-11ed-861a-0a58a9feac02
distributed.core - INFO - Starting established connection
distributed.scheduler - INFO - Receive client connection: Client-465fa488-4e36-11ed-861a-0a58a9feac02
distributed.core - INFO - Starting established connection
distributed.scheduler - INFO - Scheduler closing after being idle for 600.00 s
distributed.scheduler - INFO - Scheduler closing...
distributed.scheduler - INFO - Scheduler closing all comms
distributed.worker - INFO - Stopping worker at tcp://127.0.0.1:37945
distributed.scheduler - INFO - Remove worker <WorkerState 'tcp://127.0.0.1:37945', name: wb-dask-984562ae-8f35-41f2-a40e-7d3b1ad80618-worker-31bc82cb, status: running, memory: 1, processing: 0>

It looks like an issue with running on EC2 instances - I'll try and get some more logging information from CloudWatch and at least put together a sample which allows us to recreate the issue.

@fjetter
Copy link
Member

fjetter commented Oct 18, 2022

@dthompson-maystreet please also provide the version you are using. Since your logs do not contain any timestamps but use otherwise default formatting, I'm wondering if you are using a relatively old version (we added timestamps to the default log format in march / #5897; 2022.03.0)

If this issue still persists on newer versions, it would be helpful if you could provide more context about the multiple clients, the computations you submitted, etc.


For everyone that might be affected by this, please confirm that you are running on a reasonably new version. There was a fix about idle detection in June (#6563) which was released in 2022.6.1.

@dthompson-maystreet
Copy link

Thank you very much for replying; we did some investigations and here's what we found.

What we've found out is we think we're using Dask in a way it wasn't designed to do.

As part of a test we were firing single short-running jobs at a Dask cluster on a timer; these would run so quickly that when the check_idle code periodically ran nothing was active, so eventually the scheduler timed out even though work was run, say, 1 minute before the timeout occurred.

If we enqueue work on a backlog, or we're constantly firing work at the scheduler, then everything works fine.

I think there could be an argument that the check_idle function should be run when a job is enqueued or picked up to ensure the self.idle_since variable is set to None, therefore stopping the scheduler from timing out if a job has recently been started, but in the meantime we'll tweak our documentation so our users know what's going on.

Would be interested in your thoughts here too. Thank you for your time!

@fjetter
Copy link
Member

fjetter commented Oct 18, 2022

As part of a test we were firing single short-running jobs at a Dask cluster on a timer; these would run so quickly that when the check_idle code periodically ran nothing was active, so eventually the scheduler timed out even though work was run, say, 1 minute before the timeout occurred.

The patch in #6563 / 2022.6.1 addresses this problem (It's covered by the transition counter if you look at the implementation)

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something is broken needs info Needs further information from the user stability Issue or feature related to cluster stability (e.g. deadlock)
Projects
None yet
Development

No branches or pull requests

3 participants