-
-
Notifications
You must be signed in to change notification settings - Fork 748
Replace loop.call_later and loop.add_callback with background tasks added to Server.
#6478
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
Changes from all commits
33ae2ef
e680be9
0af0741
d9ce34f
89f70db
76edf23
e415d97
3d688ad
d91a4cf
8312041
488b111
9155235
62c9383
96c6346
0248587
3a5695a
aaa3cd4
b271783
5459964
773188a
734e893
bc45f99
a932b79
bb9d71b
eda6b73
79970d7
bdb1b70
d3f04a5
0532a21
ac53b94
2f7d90c
4f739b2
f6348ee
b66fae6
b064ba1
52061d9
1e41e20
688aaa1
f3d9fa9
ad70db2
1b28c3b
683d837
350c2b4
43c7b3c
fddd34c
936710a
f46bc59
06749bb
9ca5bea
daa845a
be622fd
075a539
8df26a5
a77c61e
50f5851
22d0952
d5b99ef
cd04ab2
2daae46
55bcdd3
6c4536c
53643cd
2494e40
2751c52
93ff2ba
e76489c
09a3b2c
a7df081
8873ccb
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||
|---|---|---|---|---|
|
|
@@ -15,11 +15,10 @@ | |||
| from contextlib import suppress | ||||
| from enum import Enum | ||||
| from functools import partial | ||||
| from typing import Callable, ClassVar, TypedDict, TypeVar | ||||
| from typing import Callable, ClassVar, Coroutine, TypedDict, TypeVar | ||||
|
|
||||
| import tblib | ||||
| from tlz import merge | ||||
| from tornado import gen | ||||
| from tornado.ioloop import IOLoop, PeriodicCallback | ||||
|
|
||||
| import dask | ||||
|
|
@@ -38,9 +37,11 @@ | |||
| from distributed.metrics import time | ||||
| from distributed.system_monitor import SystemMonitor | ||||
| from distributed.utils import ( | ||||
| NoOpAwaitable, | ||||
| delayed, | ||||
| get_traceback, | ||||
| has_keyword, | ||||
| is_coroutine_function, | ||||
| iscoroutinefunction, | ||||
| recursive_to_dict, | ||||
| truncate_exception, | ||||
| ) | ||||
|
|
@@ -109,6 +110,146 @@ def _expects_comm(func: Callable) -> bool: | |||
| return False | ||||
|
|
||||
|
|
||||
| class _LoopBoundMixin: | ||||
| """Backport of the private asyncio.mixins._LoopBoundMixin from 3.11""" | ||||
|
|
||||
| _global_lock = threading.Lock() | ||||
|
|
||||
| _loop = None | ||||
|
|
||||
| def _get_loop(self): | ||||
| loop = asyncio.get_running_loop() | ||||
|
|
||||
| if self._loop is None: | ||||
| with self._global_lock: | ||||
| if self._loop is None: | ||||
| self._loop = loop | ||||
| if loop is not self._loop: | ||||
| raise RuntimeError(f"{self!r} is bound to a different event loop") | ||||
| return loop | ||||
|
|
||||
|
|
||||
| class AsyncTaskGroupClosedError(RuntimeError): | ||||
| pass | ||||
|
|
||||
|
|
||||
| class AsyncTaskGroup(_LoopBoundMixin): | ||||
| """Collection tracking all currently running asynchronous tasks within a group""" | ||||
|
|
||||
| #: If True, the group is closed and does not allow adding new tasks. | ||||
| closed: bool | ||||
|
|
||||
| def __init__(self) -> None: | ||||
| self.closed = False | ||||
| self._ongoing_tasks: set[asyncio.Task] = set() | ||||
|
|
||||
| def call_soon( | ||||
| self, afunc: Callable[..., Coroutine], *args, **kwargs | ||||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. does ParamSpec work on these?
|
||||
| ) -> asyncio.Task: | ||||
| """Schedule a coroutine function to be executed as an `asyncio.Task`. | ||||
|
|
||||
| The coroutine function `afunc` is scheduled with `args` arguments and `kwargs` keyword arguments | ||||
| as an `asyncio.Task`. | ||||
|
|
||||
| Parameters | ||||
| ---------- | ||||
| afunc | ||||
| Coroutine function to schedule. | ||||
| *args | ||||
| Arguments to be passed to `afunc`. | ||||
| **kwargs | ||||
| Keyword arguments to be passed to `afunc` | ||||
|
|
||||
| Returns | ||||
| ------- | ||||
| The scheduled Task object. | ||||
|
|
||||
| Raises | ||||
| ------ | ||||
| AsyncTaskGroupClosedError | ||||
| If the task group is closed. | ||||
| """ | ||||
| if self.closed: # Avoid creating a coroutine | ||||
| raise AsyncTaskGroupClosedError( | ||||
| "Cannot schedule a new coroutine function as the group is already closed." | ||||
| ) | ||||
| task = self._get_loop().create_task(afunc(*args, **kwargs)) | ||||
| task.add_done_callback(self._ongoing_tasks.remove) | ||||
| self._ongoing_tasks.add(task) | ||||
| return task | ||||
|
|
||||
| def call_later( | ||||
| self, delay: float, afunc: Callable[..., Coroutine], *args, **kwargs | ||||
| ) -> asyncio.Task: | ||||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. do we use the return value of this task anywhere? if we don't return it we can avoid cases where people call |
||||
| """Schedule a coroutine function to be executed after `delay` seconds as an `asyncio.Task`. | ||||
|
|
||||
| The coroutine function `afunc` is scheduled with `args` arguments and `kwargs` keyword arguments | ||||
| as an `asyncio.Task` that is executed after `delay` seconds. | ||||
|
|
||||
| Parameters | ||||
| ---------- | ||||
| delay | ||||
| Delay in seconds. | ||||
| afunc | ||||
| Coroutine function to schedule. | ||||
| *args | ||||
| Arguments to be passed to `afunc`. | ||||
| **kwargs | ||||
| Keyword arguments to be passed to `afunc` | ||||
|
|
||||
| Returns | ||||
| ------- | ||||
| The scheduled Task object. | ||||
|
|
||||
| Raises | ||||
| ------ | ||||
| AsyncTaskGroupClosedError | ||||
| If the task group is closed. | ||||
| """ | ||||
| return self.call_soon(delayed(afunc, delay), *args, **kwargs) | ||||
|
|
||||
| def close(self) -> None: | ||||
| """Closes the task group so that no new tasks can be scheduled. | ||||
|
|
||||
| Existing tasks continue to run. | ||||
| """ | ||||
| self.closed = True | ||||
|
|
||||
| async def stop(self, timeout=1) -> None: | ||||
| """Close the group and stop all currently running tasks. | ||||
|
|
||||
| Closes the task group and waits `timeout` seconds for all tasks to gracefully finish. | ||||
| After the timeout, all remaining tasks are cancelled. | ||||
| """ | ||||
| self.close() | ||||
|
|
||||
| current_task = asyncio.current_task(self._get_loop()) | ||||
| tasks_to_stop = [t for t in self._ongoing_tasks if t is not current_task] | ||||
|
|
||||
| if tasks_to_stop: | ||||
| # Wrap gather in task to avoid Python3.8 issue, | ||||
| # see https://github.com/dask/distributed/pull/6478#discussion_r885696827 | ||||
| async def gather(): | ||||
| return await asyncio.gather(*tasks_to_stop, return_exceptions=True) | ||||
|
|
||||
| try: | ||||
| await asyncio.wait_for( | ||||
| gather(), | ||||
| timeout, | ||||
| ) | ||||
| except asyncio.TimeoutError: | ||||
| # The timeout on gather has cancelled the tasks, so this will not hang indefinitely | ||||
| await asyncio.gather(*tasks_to_stop, return_exceptions=True) | ||||
|
Collaborator
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Might be good to comment that the timeout on |
||||
|
|
||||
| if [t for t in self._ongoing_tasks if t is not current_task]: | ||||
| raise RuntimeError( | ||||
| f"Expected all ongoing tasks to be cancelled and removed, found {self._ongoing_tasks}." | ||||
| ) | ||||
|
|
||||
| def __len__(self): | ||||
| return len(self._ongoing_tasks) | ||||
|
|
||||
|
|
||||
| class Server: | ||||
| """Dask Distributed Server | ||||
|
|
||||
|
|
@@ -194,7 +335,8 @@ def __init__( | |||
| self.monitor = SystemMonitor() | ||||
| self.counters = None | ||||
| self.digests = None | ||||
| self._ongoing_coroutines = set() | ||||
| self._ongoing_background_tasks = AsyncTaskGroup() | ||||
| self._ongoing_comm_handlers = AsyncTaskGroup() | ||||
| self._event_finished = asyncio.Event() | ||||
|
|
||||
| self.listeners = [] | ||||
|
|
@@ -227,7 +369,7 @@ def stop() -> bool: | |||
|
|
||||
| self.counters = defaultdict(partial(Counter, loop=self.io_loop)) | ||||
|
|
||||
| self.periodic_callbacks = dict() | ||||
| self.periodic_callbacks = {} | ||||
|
|
||||
| pc = PeriodicCallback( | ||||
| self.monitor.update, | ||||
|
|
@@ -348,29 +490,30 @@ def start_periodic_callbacks(self): | |||
| """Start Periodic Callbacks consistently | ||||
|
|
||||
| This starts all PeriodicCallbacks stored in self.periodic_callbacks if | ||||
| they are not yet running. It does this safely on the IOLoop. | ||||
| they are not yet running. It does this safely by checking that it is using the | ||||
| correct event loop. | ||||
| """ | ||||
| self._last_tick = time() | ||||
|
|
||||
| def start_pcs(): | ||||
| for pc in self.periodic_callbacks.values(): | ||||
| if not pc.is_running(): | ||||
| pc.start() | ||||
| if self.io_loop.asyncio_loop is not asyncio.get_running_loop(): | ||||
| raise RuntimeError(f"{self!r} is bound to a different event loop") | ||||
|
|
||||
| self.io_loop.add_callback(start_pcs) | ||||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this was originally written so that
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. if we're making it so start_periodic_callbacks can only be called from the event loop thread we can simplify this to be: if self.io_loop.asyncio_loop is not asyncio.get_running_loop():
raise RuntimeError(f"{self!r} is bound to a different event loop")
self._last_tick = time()
for pc in self.periodic_callbacks.values():
if not pc.is_running(): # we probably want to issue a warning here - and also check `pc.io_loop is self.io_loop`
pc.start()
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I wasn't aware of that. Is there a need to keep this as-is or can we safely simplify this? cc @fjetter?
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is always called as part of
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes that's correct, currently all our calls of start_periodic_callbacks are called from the running loop That's why I think it's better to use an assertion that we're on the correct loop than support running from any thread
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Alright, I'll adjust according to the outline above then.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let's address checking |
||||
| self._last_tick = time() | ||||
| for pc in self.periodic_callbacks.values(): | ||||
| if not pc.is_running(): | ||||
| logger.info(f"Starting periodic callback {pc!r}") | ||||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think this might be causing the failures in test_worker_cli_nprocs_renamed_to_nworkers see also https://github.com/dask/distributed/pulls/
Suggested change
|
||||
| pc.start() | ||||
|
|
||||
| def stop(self): | ||||
| if not self.__stopped: | ||||
| self.__stopped = True | ||||
|
|
||||
| for listener in self.listeners: | ||||
| # Delay closing the server socket until the next IO loop tick. | ||||
| # Otherwise race conditions can appear if an event handler | ||||
| # for an accept() call is already scheduled by the IO loop, | ||||
| # raising EBADF. | ||||
| # The demonstrator for this is Worker.terminate(), which | ||||
| # closes the server socket in response to an incoming message. | ||||
| # See https://github.com/tornadoweb/tornado/issues/2069 | ||||
| self.io_loop.add_callback(listener.stop) | ||||
|
|
||||
| async def stop_listener(listener): | ||||
| v = listener.stop() | ||||
| if inspect.isawaitable(v): | ||||
| await v | ||||
|
|
||||
| self._ongoing_background_tasks.call_soon(stop_listener, listener) | ||||
|
|
||||
| @property | ||||
| def listener(self): | ||||
|
|
@@ -495,7 +638,15 @@ async def listen(self, port_or_addr=None, allow_offload=True, **kwargs): | |||
| ) | ||||
| self.listeners.append(listener) | ||||
|
|
||||
| async def handle_comm(self, comm): | ||||
| def handle_comm(self, comm): | ||||
| """Start a background task that dispatches new communications to coroutine-handlers""" | ||||
| try: | ||||
| self._ongoing_background_tasks.call_soon(self._handle_comm, comm) | ||||
| except AsyncTaskGroupClosedError: | ||||
| comm.abort() | ||||
| return NoOpAwaitable() | ||||
|
|
||||
| async def _handle_comm(self, comm): | ||||
| """Dispatch new communications to coroutine-handlers | ||||
|
|
||||
| Handlers is a dictionary mapping operation names to functions or | ||||
|
|
@@ -589,11 +740,6 @@ async def handle_comm(self, comm): | |||
| else: | ||||
| result = handler(**msg) | ||||
| if inspect.iscoroutine(result): | ||||
| result = asyncio.create_task( | ||||
| result, name=f"handle-comm-{address}-{op}" | ||||
| ) | ||||
| self._ongoing_coroutines.add(result) | ||||
| result.add_done_callback(self._ongoing_coroutines.remove) | ||||
| result = await result | ||||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this can result in a
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Dealt with as specified above. |
||||
| elif inspect.isawaitable(result): | ||||
| raise RuntimeError( | ||||
|
|
@@ -660,9 +806,11 @@ async def handle_stream(self, comm, extra=None): | |||
| closed = True | ||||
| break | ||||
| handler = self.stream_handlers[op] | ||||
| if is_coroutine_function(handler): | ||||
| self.loop.add_callback(handler, **merge(extra, msg)) | ||||
| await gen.sleep(0) | ||||
| if iscoroutinefunction(handler): | ||||
| self._ongoing_background_tasks.call_soon( | ||||
| handler, **merge(extra, msg) | ||||
| ) | ||||
| await asyncio.sleep(0) | ||||
| else: | ||||
| handler(**merge(extra, msg)) | ||||
| else: | ||||
|
|
@@ -695,20 +843,11 @@ async def close(self, timeout=None): | |||
| _stops.add(future) | ||||
| await asyncio.gather(*_stops) | ||||
|
|
||||
| def _ongoing_tasks(): | ||||
| return ( | ||||
| t for t in self._ongoing_coroutines if t is not asyncio.current_task() | ||||
| ) | ||||
| # TODO: Deal with exceptions | ||||
| await self._ongoing_background_tasks.stop(timeout=1) | ||||
|
|
||||
| # TODO: Deal with exceptions | ||||
| try: | ||||
| # Give the handlers a bit of time to finish gracefully | ||||
| await asyncio.wait_for( | ||||
| asyncio.gather(*_ongoing_tasks(), return_exceptions=True), 1 | ||||
| ) | ||||
| except asyncio.TimeoutError: | ||||
| # the timeout on gather should've cancelled all the tasks | ||||
| await asyncio.gather(*_ongoing_tasks(), return_exceptions=True) | ||||
| await self._ongoing_comm_handlers.stop(timeout=1) | ||||
|
|
||||
| await self.rpc.close() | ||||
| await asyncio.gather(*[comm.close() for comm in list(self._comms)]) | ||||
|
|
@@ -878,12 +1017,10 @@ async def _close_comm(comm): | |||
| tasks = [] | ||||
| for comm in list(self.comms): | ||||
| if comm and not comm.closed(): | ||||
| # IOLoop.current().add_callback(_close_comm, comm) | ||||
| task = asyncio.ensure_future(_close_comm(comm)) | ||||
| tasks.append(task) | ||||
| for comm in list(self._created): | ||||
| if comm and not comm.closed(): | ||||
| # IOLoop.current().add_callback(_close_comm, comm) | ||||
| task = asyncio.ensure_future(_close_comm(comm)) | ||||
| tasks.append(task) | ||||
|
|
||||
|
|
||||
| Original file line number | Diff line number | Diff line change | ||||||||||||||||||||||||||||||||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -498,7 +498,7 @@ def run(self, comm, *args, **kwargs): | |||||||||||||||||||||||||||||||||||||||||||||
| return run(self, comm, *args, **kwargs) | ||||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||||
| def _on_exit_sync(self, exitcode): | ||||||||||||||||||||||||||||||||||||||||||||||
| self.loop.add_callback(self._on_exit, exitcode) | ||||||||||||||||||||||||||||||||||||||||||||||
| self._ongoing_background_tasks.call_soon(self._on_exit, exitcode) | ||||||||||||||||||||||||||||||||||||||||||||||
|
Collaborator
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Was this method relying on the fact that
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. no distributed/distributed/nanny.py Line 408 in 8873ccb
distributed/distributed/nanny.py Line 660 in 8873ccb
distributed/distributed/process.py Lines 314 to 326 in 8873ccb
distributed/distributed/process.py Line 240 in 8873ccb
distributed/distributed/process.py Lines 129 to 134 in 8873ccb
|
||||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||||
| @log_errors | ||||||||||||||||||||||||||||||||||||||||||||||
| async def _on_exit(self, exitcode): | ||||||||||||||||||||||||||||||||||||||||||||||
|
|
@@ -595,7 +595,7 @@ async def _log_event(self, topic, msg): | |||||||||||||||||||||||||||||||||||||||||||||
| ) | ||||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||||
| def log_event(self, topic, msg): | ||||||||||||||||||||||||||||||||||||||||||||||
| self.loop.add_callback(self._log_event, topic, msg) | ||||||||||||||||||||||||||||||||||||||||||||||
| self._ongoing_background_tasks.call_soon(self._log_event, topic, msg) | ||||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||||
| class WorkerProcess: | ||||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't intend to delay merging this PR with this at all. But I'm a little surprised to see something so similar to, but not quite the same as, the asyncio TaskGroup added in 3.11. Maybe once this gets in, we'd want to refactor to something consistent with the new asyncio API?
I also feel like
AsyncTaskGroupshould maybe be its own file?Also, this looks interesting: https://github.com/Tinche/quattro
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
the task group of 3.11 cannot be easily copy-pasted, it relies on some 3.11 features, e.g.
Task.uncancelI'm hoping to see some backports about the 3.11 async features, though.
Either way, I'm very open to refactor this and align with stdlib. I'm also fine with moving this it's own file