Skip to content

Commit

Permalink
Add distributed Dask Event that mimics threading.Event (#3821)
Browse files Browse the repository at this point in the history
  • Loading branch information
nils-braun authored May 22, 2020
1 parent b377f6a commit b9e4953
Show file tree
Hide file tree
Showing 5 changed files with 472 additions and 0 deletions.
1 change: 1 addition & 0 deletions distributed/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
from .queues import Queue
from .security import Security
from .semaphore import Semaphore
from .event import Event
from .scheduler import Scheduler
from .threadpoolexecutor import rejoin
from .utils import sync, TimeoutError, CancelledError
Expand Down
246 changes: 246 additions & 0 deletions distributed/event.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,246 @@
import asyncio
from collections import defaultdict
from contextlib import suppress
import logging
import uuid

from .client import Client
from .utils import log_errors, TimeoutError
from .worker import get_worker

logger = logging.getLogger(__name__)


class EventExtension:
""" An extension for the scheduler to manage Events
This adds the following routes to the scheduler
* event_wait
* event_set
* event_clear
* event_is_set
In principle, the implementation logic is quite simple
as we can reuse the asyncio.Event as much as possible:
we keep a mapping from name to an asyncio.Event and call
every function (wait, set, clear, is_set) directly on these
events.
However, this would cause a memory leak: created events in the
dictionary are never removed.
For this, we also keep a counter for the number of waiters on
a specific event.
If an event is set, we need to keep track of this state so
we can not remove it (the default flag is false).
If it is unset but there are waiters, we can also not remove
it, as those waiters would then have dangling futures.
Therefore the only time we can remove the event from our dict
is when the number of waiters is 0 and the event flag is cleared.
"""

def __init__(self, scheduler):
self.scheduler = scheduler
# Keep track of all current events, identified by their name
self._events = defaultdict(asyncio.Event)
# Keep track on how many waiters are present, so we know when
# we can remove the event
self._waiter_count = defaultdict(int)

self.scheduler.handlers.update(
{
"event_wait": self.event_wait,
"event_set": self.event_set,
"event_clear": self.event_clear,
"event_is_set": self.event_is_set,
}
)

self.scheduler.extensions["events"] = self

async def event_wait(self, comm=None, name=None, timeout=None):
""" Wait until the event is set to true.
Returns false, when this did not happen in the given time
and true otherwise.
"""
with log_errors():
name = self._normalize_name(name)

event = self._events[name]
future = event.wait()
if timeout is not None:
future = asyncio.wait_for(future, timeout)

self._waiter_count[name] += 1
try:
await future
except TimeoutError:
return False
finally:
self._waiter_count[name] -= 1

if not self._waiter_count[name] and not event.is_set():
# No one is waiting for this
# and as the default flag for an event is false
# we can safely remove it
self._delete_event(name)

return True

def event_set(self, comm=None, name=None):
""" Set the event with the given name to true.
All waiters on this event will be notified.
"""
with log_errors():
name = self._normalize_name(name)
# No matter if someone is listening or not,
# we set the event to true
self._events[name].set()

def event_clear(self, comm=None, name=None):
"""Set the event with the given name to false."""
with log_errors():
name = self._normalize_name(name)
if not self._waiter_count[name]:
# No one is waiting for this
# and as the default flag for an event is false
# we can safely remove it
self._delete_event(name)

else:
# There are waiters
# This can happen if an event is "double-cleared"
# In principle, the event should be unset at this point
# (because if it is set, all waiters should have been
# notified). But to prevent race conditions
# due to unlucky timing, we clear anyways
assert name in self._events
event = self._events[name]
event.clear()

def event_is_set(self, comm=None, name=None):
with log_errors():
name = self._normalize_name(name)
# the default flag value is false
# we could also let the defaultdict
# create a new event for us, but that
# could produce many unused events
if name not in self._events:
return False

return self._events[name].is_set()

def _normalize_name(self, name):
""" Helper function to normalize an event name """
if isinstance(name, list):
name = tuple(name)

return name

def _delete_event(self, name):
""" Helper function to delete an event """
# suppress key errors to make calling this method
# also possible if we do not even have such an event
with suppress(KeyError):
del self._waiter_count[name]
with suppress(KeyError):
del self._events[name]


class Event:
""" Distributed Centralized Event equivalent to asyncio.Event
An event stores a single flag, which is set to false on start.
The flag can be set to true (using the set() call) or back to false
(with the clear() call).
Every call to wait() blocks until the event flag is set to true.
Parameters
----------
name: string (optional)
Name of the event. Choosing the same name allows two
disconnected processes to coordinate an event.
If not given, a random name will be generated.
client: Client (optional)
Client to use for communication with the scheduler.
If not given, the default global client will be used.
Examples
--------
>>> event_1 = Event('a') # doctest: +SKIP
>>> event_1.wait(timeout=1) # doctest: +SKIP
>>> # in another process
>>> event_2 = Event('a') # doctest: +SKIP
>>> event_2.set() # doctest: +SKIP
>>> # now event_1 will stop waiting
"""

def __init__(self, name=None, client=None):
try:
self.client = client or Client.current()
except ValueError:
# Initialise new client
self.client = get_worker().client
self.name = name or "event-" + uuid.uuid4().hex

def __await__(self):
""" async constructor
Make it possible to write
>>> event = await Event("x") # doctest: +SKIP
even though no waiting is implied
"""

async def _():
return self

return _().__await__()

def wait(self, timeout=None):
""" Wait until the event is set.
Parameters
----------
timeout : number, optional
Seconds to wait on the event in the scheduler. This does not
include local coroutine time, network transfer time, etc..
Examples
--------
>>> event = Event('a') # doctest: +SKIP
>>> event.wait(timeout=1) # doctest: +SKIP
Returns
-------
True if the event was set of false, if a timeout happend
"""
result = self.client.sync(
self.client.scheduler.event_wait, name=self.name, timeout=timeout,
)
return result

def clear(self):
""" Clear the event (set its flag to false).
All waiters will now block.
"""
return self.client.sync(self.client.scheduler.event_clear, name=self.name)

def set(self):
""" Set the event (set its flag to false).
All waiters will now be released.
"""
result = self.client.sync(self.client.scheduler.event_set, name=self.name,)
return result

def is_set(self):
""" Check if the event is set """
result = self.client.sync(self.client.scheduler.event_is_set, name=self.name,)
return result

def __reduce__(self):
return (Event, (self.name,))
2 changes: 2 additions & 0 deletions distributed/scheduler.py
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,7 @@
from .semaphore import SemaphoreExtension
from .recreate_exceptions import ReplayExceptionScheduler
from .lock import LockExtension
from .event import EventExtension
from .pubsub import PubSubSchedulerExtension
from .stealing import WorkStealing
from .variable import VariableExtension
Expand All @@ -102,6 +103,7 @@
VariableExtension,
PubSubSchedulerExtension,
SemaphoreExtension,
EventExtension,
]

ALL_TASK_STATES = {"released", "waiting", "no-worker", "processing", "erred", "memory"}
Expand Down
Loading

0 comments on commit b9e4953

Please sign in to comment.