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 shuts down after 20 minutes of inactivity - tasks not executed #5921

Open
lojohnson opened this issue Mar 9, 2022 · 4 comments
Open
Labels
bug Something is broken stability Issue or feature related to cluster stability (e.g. deadlock)

Comments

@lojohnson
Copy link

Reported by a Coiled user forman for his recent clusters id 121956 and 121459:

Hello! I'm currently trying to use Coiled for upscaling an ND image (= 2+D dask array) but I constantly fail.
My source is read via xr.open_zarr() from S3 and is around 2k^2 pixels. I upscale it by factor of 50 to around 100k^2 pixels. My x/y chunk size is 2048. I'm using dask.array.map_blocks(f, ...) with f being a custom function that uses numba. The output is written to S3 using xr.Dataset.to_zarr().
When I execute my job using dask local threading or even processes and the local filesystem, it runs without any problems and result is as expected and finishes within seconds.
If I run the same on Coiled using S3 (from my local JNB), the cluster shuts down after 20 minutes of inactivity - the scheduler doesn't receive a single task. I cannot tell what is happening. My local CPU is 3%, RAM 8 GB, practically no network.

User moved past this issue by setting his next cluster to use 4 workers and including scheduler_options={"idle_timeout": "2 hours"

Odd behavior was observed in worker and scheduler logs that may have lead to the bad state of the cluster. Full scheduler logs attached. For cluster 121956, worker coiled-dask-forman-121956-worker-35c5f23472, logs show "
Mar 09 13:12:09 ip-10-13-8-190 cloud-init[1860]: distributed.batched - ERROR - Error in batched write
Mar 09 13:12:09 ip-10-13-8-190 cloud-init[1860]: Traceback (most recent call last):
Mar 09 13:12:09 ip-10-13-8-190 cloud-init[1860]: BufferError: Existing exports of data: object cannot be re-sized

Scheduler for the same cluster shows that it removed this worker, and then ran into a stream of Unexpected worker completed task errors referencing this same removed worker.

Mar 09 13:12:05 ip-10-13-11-186 cloud-init[1528]: distributed.core - INFO - Event loop was unresponsive in Scheduler for 30.75s. This is often caused by long-running GIL-holding functions or moving large chunks of data. This can cause timeouts and instability.

Mar 09 13:12:10 ip-10-13-11-186 cloud-init[1528]: distributed.scheduler - INFO - Remove worker <WorkerState 'tls://10.13.8.140:44147', name: coiled-dask-forman-121956-worker-35c5f23472, status: running, memory: 952, processing: 7056>

Mar 09 13:12:12 ip-10-13-11-186 cloud-init[1528]: distributed.scheduler - INFO - Unexpected worker completed task. Expected: <WorkerState 'tls://10.13.15.152:42935', name: coiled-dask-forman-121956-worker-1f11f7b175, status: running, memory: 806, processing: 8479>, Got: <WorkerState 'tls://10.13.8.140:44147', name: coiled-dask-forman-121956-worker-35c5f23472, status: running, memory: 0, processing: 0>, Key: ('block-info-_upscale_numpy_array-2ba5457e4d03cf22addd23421859e823', 44, 25)

Possibly related to #5675

Scheduler logs:
forman-scheduler-121956-logs.zip

Task graph of stuck cluster
forman-cluster121956- graph

@lojohnson
Copy link
Author

forman-scheduler-121956-logs.log
Better scheduler logs

@gjoseph92
Copy link
Collaborator

distributed.batched - ERROR - Error in batched write

Woo, more problems with BatchedSend! Not exactly #5480, but I'm not sure what else to reference. Also xref #4239.

In the "better scheduler logs", I also see

13:13:48 distributed.scheduler - INFO - Remove client Client-fa36f346-9fa9-11ec-9020-145afc45530e
...
13:14:02 distributed.scheduler - INFO - Close client connection: Client-fa36f346-9fa9-11ec-9020-145afc45530e
...
14:14:01 distributed.scheduler - INFO - Receive client connection: Client-2b332333-9fb3-11ec-a4c0-145afc45530e

which makes me think the client disconnected and reconnected, i.e. #5667. I don't remember what the default timeouts are, but maybe the 49sec of unresponsiveness on the scheduler would be enough for the client to think the connection was broken? The disconnection would have caused all that client's tasks to be dropped, which could explain why the scheduler thought it was idle. I'm also very curious why there's a full hour delay (according to timestamps) for a client reconnecting though? The timeline doesn't add up; the user reports the cluster shutting down from 20min of inactivity, but the logs keep going for more than an hour after the client disconnects.

@lojohnson these logs don't seem to include the scheduler starting up or shutting down; I'd be curious full logs for the entire runtime if it's possible to get them.

cc @fjetter

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

I was able to get logging from the start and end of that scheduler from Cloudwatch logging (exported as .csv)
scheduler-logs-start.csv
scheduler-logs-shutdown.csv

@fjetter
Copy link
Member

fjetter commented Mar 10, 2022

don't remember what the default timeouts are, but maybe the 49sec of unresponsiveness on the scheduler would be enough for the client to think the connection was broken?

There is no such timeout. Once the connection is established we keep it open forever.

async def _handle_report(self):
"""Listen to scheduler"""
with log_errors():
try:
while True:
if self.scheduler_comm is None:
break
try:
msgs = await self.scheduler_comm.comm.read()
except CommClosedError:
if self.status == "running":
logger.info("Client report stream closed to scheduler")
logger.info("Reconnecting...")
self.status = "connecting"
await self._reconnect()
continue
else:
break

The client only disconnects if an OSError is raised. A bunch of network things can happen to tear this connection, of course.
If it closes after Xmin and this is reproducible, I would check out network/firewall settings. Some firewalls close connections after a certain amount of idleness (regardless of HTTP keep-alive). This is why we introduced our own keep-alive on the workers, see

pc = PeriodicCallback(
lambda: self.batched_stream.send({"op": "keep-alive"}), 60000
)

I'd be surprised if this was configured to 20min anywhere, though

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

No branches or pull requests

3 participants