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

@gen_cluster has become 1s slower; CI takes 50% longer #6632

Closed
crusaderky opened this issue Jun 26, 2022 · 4 comments · Fixed by #6633
Closed

@gen_cluster has become 1s slower; CI takes 50% longer #6632

crusaderky opened this issue Jun 26, 2022 · 4 comments · Fixed by #6633
Assignees

Comments

@crusaderky
Copy link
Collaborator

Since #6603, a test decorated with @gen_cluster:

@gen_cluster()
async def test1(s, a, b):
    pass

has increased in runtime on my host from 220ms to 1220ms.

With the same PR, the CI test suites (ci1) have increased:
Ubuntu: 19m23s -> 34m18s
Windows: 23m36s -> 48m55s
MacOSX: 32m15s -> 46m39s

CC @graingert

@graingert
Copy link
Member

graingert commented Jun 27, 2022

the issue is the scheduler.close() call is now always taking at least 1s to finish:

await asyncio.gather(*(end_worker(w) for w in workers))
await s.close() # wait until scheduler stops completely
s.stop()

the workers are already closed so there should be no remaining comm_handlers left:
# TODO: Deal with exceptions
await self._ongoing_background_tasks.stop(timeout=1)
# TODO: Deal with exceptions
await self._ongoing_comm_handlers.stop(timeout=1)

however looking at the tasks in self._ongoing_background_tasks._ongoing_tasks:
async def remove_worker_from_events():
# If the worker isn't registered anymore after the delay, remove from events
if address not in self.workers and address in self.events:
del self.events[address]
cleanup_delay = parse_timedelta(
dask.config.get("distributed.scheduler.events-cleanup-delay")
)
self._ongoing_background_tasks.call_later(
cleanup_delay, remove_worker_from_events
)
logger.debug("Removed worker %s", ws)

there's a pair of tasks waiting to call remove_worker_from_events, these get a 1 second grace period to stop and are then cancelled
async def remove_worker_from_events():
# If the worker isn't registered anymore after the delay, remove from events
if address not in self.workers and address in self.events:
del self.events[address]
cleanup_delay = parse_timedelta(
dask.config.get("distributed.scheduler.events-cleanup-delay")
)
self._ongoing_background_tasks.call_later(
cleanup_delay, remove_worker_from_events
)
logger.debug("Removed worker %s", ws)
return "OK"

I think there should not be a grace period of 1s for background tasks in Server.close and I think remove_worker_from_events should be called immediately when a worker reports that it is closing.

@hendrikmakait
Copy link
Member

I'd be open to dropping the grace period and directly cancelling tasks. Are there any tasks for which we'd actually prefer graceful shutdown? In that case we could adjust the signature of call_soon to include a graceful=False keyword that would get stored together with the task itself and we could wait only for graceful shutdown of those tasks (likely with a shorter grace period).

@fjetter
Copy link
Member

fjetter commented Jun 27, 2022

remove_worker_from_events should be called immediately when a worker reports that it is closing.

  1. We need to remove workers from events, eventually. Depending on the deployment, this can cause a memory leak otherwise. I introduced this ~2 years ago because we were using clusters that were always-on but we scaled them up and down based on demand, i.e. over time this events dict became huge. Eventually this dict should be cleaned up. The solution back then was to introduce a timeout
  2. It should be delayed such that the data is available after a while for debugging. The hour was very arbitrary.

If this causes big problems we can drop it. Some information about entering/leaving workers is also stored in the all topic but the worker specific topic is more granular.

Is there a problem with just canceling this task?

@fjetter
Copy link
Member

fjetter commented Jun 27, 2022

I agree that we don't necessarily want a grace period for when we're closing. I'm fine with cancelling stuff right away

@fjetter fjetter self-assigned this Jun 30, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging a pull request may close this issue.

4 participants