diff --git a/distributed/__init__.py b/distributed/__init__.py index b8cf71ef24..ec22fc6afd 100644 --- a/distributed/__init__.py +++ b/distributed/__init__.py @@ -20,18 +20,18 @@ from dask.config import config # type: ignore from distributed._version import get_versions -from distributed.actor import Actor, ActorFuture, BaseActorFuture +from distributed.actor import Actor, ActorTask, BaseActorTask from distributed.client import ( Client, CompatibleExecutor, - Future, + Task, as_completed, default_client, fire_and_forget, - futures_of, get_task_metadata, get_task_stream, performance_report, + tasks_of, wait, ) from distributed.core import Status, connect, rpc @@ -120,16 +120,16 @@ def _(): __all__ = [ "Actor", - "ActorFuture", + "ActorTask", "Adaptive", - "BaseActorFuture", + "BaseActorTask", "CancelledError", "Client", "CompatibleExecutor", "CondaInstall", "Environ", "Event", - "Future", + "Task", "KilledWorker", "LocalCluster", "Lock", @@ -162,7 +162,7 @@ def _(): "dask", "default_client", "fire_and_forget", - "futures_of", + "tasks_of", "get_client", "get_task_metadata", "get_task_stream", diff --git a/distributed/_concurrent_futures_thread.py b/distributed/_concurrent_futures_thread.py index 86de8a2a42..b960bcfb28 100644 --- a/distributed/_concurrent_futures_thread.py +++ b/distributed/_concurrent_futures_thread.py @@ -51,22 +51,22 @@ def _python_exit(): class _WorkItem: - def __init__(self, future, fn, args, kwargs): - self.future = future + def __init__(self, task, fn, args, kwargs): + self.task = task self.fn = fn self.args = args self.kwargs = kwargs def run(self): - if not self.future.set_running_or_notify_cancel(): # pragma: no cover + if not self.task.set_running_or_notify_cancel(): # pragma: no cover return try: result = self.fn(*self.args, **self.kwargs) except BaseException as e: - self.future.set_exception(e) + self.task.set_exception(e) else: - self.future.set_result(result) + self.task.set_result(result) def _worker(executor_reference, work_queue): @@ -123,7 +123,7 @@ def __init__(self, max_workers=None, thread_name_prefix=""): def submit(self, fn, *args, **kwargs): with self._shutdown_lock: if self._shutdown: # pragma: no cover - raise RuntimeError("cannot schedule new futures after shutdown") + raise RuntimeError("cannot schedule new tasks after shutdown") f = _base.Future() w = _WorkItem(f, fn, args, kwargs) diff --git a/distributed/actor.py b/distributed/actor.py index 1fdbf5dae4..c22ec3f3bc 100644 --- a/distributed/actor.py +++ b/distributed/actor.py @@ -10,7 +10,7 @@ from tornado.ioloop import IOLoop -from distributed.client import Future +from distributed.client import Task from distributed.protocol import to_serialize from distributed.utils import LateLoopEvent, iscoroutinefunction, sync, thread_state from distributed.utils_comm import WrappedKey @@ -41,20 +41,20 @@ class Actor(WrappedKey): You can create an actor by submitting a class with the keyword ``actor=True``. - >>> future = client.submit(Counter, actor=True) - >>> counter = future.result() + >>> task = client.submit(Counter, actor=True) + >>> counter = task.result() >>> counter - Calling methods on this object immediately returns deferred ``BaseActorFuture`` + Calling methods on this object immediately returns deferred ``BaseActorTask`` objects. You can call ``.result()`` on these objects to block and get the result of the function call. - >>> future = counter.increment() - >>> future.result() + >>> task = counter.increment() + >>> task.result() 1 - >>> future = counter.increment() - >>> future.result() + >>> task = counter.increment() + >>> task.result() 2 """ @@ -63,7 +63,7 @@ def __init__(self, cls, address, key, worker=None): self._cls = cls self._address = address self._key = key - self._future = None + self._task = None self._worker = worker self._client = None self._try_bind_worker_client() @@ -77,7 +77,7 @@ def _try_bind_worker_client(self): if not self._client: try: self._client = get_client() - self._future = Future(self._key, inform=False) + self._task = Task(self._key, inform=False) # ^ When running on a worker, only hold a weak reference to the key, otherwise the key could become unreleasable. except ValueError: self._client = None @@ -139,9 +139,9 @@ def __dir__(self): return sorted(o) def __getattr__(self, key): - if self._future and self._future.status not in ("finished", "pending"): + if self._task and self._task.status not in ("finished", "pending"): raise ValueError( - "Worker holding Actor was lost. Status: " + self._future.status + "Worker holding Actor was lost. Status: " + self._task.status ) self._try_bind_worker_client() if ( @@ -157,7 +157,7 @@ def __getattr__(self, key): return attr elif callable(attr): - return lambda *args, **kwargs: EagerActorFuture(attr(*args, **kwargs)) + return lambda *args, **kwargs: EagerActorTask(attr(*args, **kwargs)) else: return attr @@ -176,8 +176,8 @@ async def run_actor_function_on_worker(): kwargs={k: to_serialize(v) for k, v in kwargs.items()}, ) except OSError: - if self._future and not self._future.done(): - await self._future + if self._task and not self._task.done(): + await self._task return await run_actor_function_on_worker() else: exc = OSError("Unable to contact Actor's worker") @@ -186,13 +186,13 @@ async def run_actor_function_on_worker(): return _OK(result["result"]) return _Error(result["exception"]) - actor_future = ActorFuture(io_loop=self._io_loop) + actor_task = ActorTask(io_loop=self._io_loop) async def wait_then_set_result(): - actor_future._set_result(await run_actor_function_on_worker()) + actor_task._set_result(await run_actor_function_on_worker()) self._io_loop.add_callback(wait_then_set_result) - return actor_future + return actor_task return func @@ -211,7 +211,7 @@ async def get_actor_attribute_from_worker(): @property def client(self): - return self._future.client + return self._task.client class ProxyRPC: @@ -232,10 +232,10 @@ async def func(**msg): return func -class BaseActorFuture(abc.ABC, Awaitable[_T]): - """Future to an actor's method call +class BaseActorTask(abc.ABC, Awaitable[_T]): + """Task to an actor's method call - Whenever you call a method on an Actor you get a BaseActorFuture immediately + Whenever you call a method on an Actor you get a BaseActorTask immediately while the computation happens in the background. You can call ``.result`` to block and collect the full result @@ -252,13 +252,13 @@ def result(self, timeout: str | timedelta | float | None = None) -> _T: def done(self) -> bool: ... - def __repr__(self) -> Literal[""]: - return "" + def __repr__(self) -> Literal[""]: + return "" @dataclass(frozen=True, eq=False) -class EagerActorFuture(BaseActorFuture[_T]): - """Future to an actor's method call when an actor calls another actor on the same worker""" +class EagerActorTask(BaseActorTask[_T]): + """Task to an actor's method call when an actor calls another actor on the same worker""" _result: _T @@ -289,7 +289,7 @@ def unwrap(self) -> NoReturn: raise self._e -class ActorFuture(BaseActorFuture[_T]): +class ActorTask(BaseActorTask[_T]): def __init__(self, io_loop: IOLoop): self._io_loop = io_loop self._event = LateLoopEvent() diff --git a/distributed/cfexecutor.py b/distributed/cfexecutor.py index 1eadfcac19..7c90a3c3ab 100644 --- a/distributed/cfexecutor.py +++ b/distributed/cfexecutor.py @@ -13,13 +13,13 @@ @gen.coroutine -def _cascade_future(future, cf_future): +def _cascade_task(task, cf_future): """ - Coroutine that waits on Dask future, then transmits its outcome to + Coroutine that waits on Dask task, then transmits its outcome to cf_future. """ - result = yield future._result(raiseit=False) - status = future.status + result = yield task._result(raiseit=False) + status = task.status if status == "finished": cf_future.set_result(result) elif status == "cancelled": @@ -35,8 +35,8 @@ def _cascade_future(future, cf_future): @gen.coroutine -def _wait_on_futures(futures): - for fut in futures: +def _wait_on_tasks(tasks): + for fut in tasks: try: yield fut except Exception: @@ -60,41 +60,41 @@ def __init__(self, client, **kwargs): % sorted(sk - self._allowed_kwargs) ) self._client = client - self._futures = weakref.WeakSet() + self._tasks = weakref.WeakSet() self._shutdown = False self._kwargs = kwargs - def _wrap_future(self, future): + def _wrap_task(self, task): """ - Wrap a distributed Future in a concurrent.futures Future. + Wrap a distributed Task in a concurrent.futures Task. """ cf_future = cf.Future() - # Support cancelling task through .cancel() on c.f.Future + # Support cancelling task through .cancel() on c.f.Task def cf_callback(cf_future): - if cf_future.cancelled() and future.status != "cancelled": - future.cancel() + if cf_future.cancelled() and task.status != "cancelled": + task.cancel() cf_future.add_done_callback(cf_callback) - self._client.loop.add_callback(_cascade_future, future, cf_future) + self._client.loop.add_callback(_cascade_task, task, cf_future) return cf_future def submit(self, fn, *args, **kwargs): """Submits a callable to be executed with the given arguments. Schedules the callable to be executed as ``fn(*args, **kwargs)`` - and returns a Future instance representing the execution of the callable. + and returns a Task instance representing the execution of the callable. Returns ------- - A Future representing the given call. + A Task representing the given call. """ if self._shutdown: - raise RuntimeError("cannot schedule new futures after shutdown") - future = self._client.submit(fn, *args, **merge(self._kwargs, kwargs)) - self._futures.add(future) - return self._wrap_future(future) + raise RuntimeError("cannot schedule new tasks after shutdown") + task = self._client.submit(fn, *args, **merge(self._kwargs, kwargs)) + self._tasks.add(task) + return self._wrap_task(task) def map(self, fn, *iterables, **kwargs): """Returns an iterator equivalent to ``map(fn, *iterables)``. @@ -133,25 +133,25 @@ def map(self, fn, *iterables, **kwargs): fs = self._client.map(fn, *iterables, **self._kwargs) # Below iterator relies on fs being an iterator itself, and not just an iterable - # (such as a list), in order to cancel remaining futures + # (such as a list), in order to cancel remaining tasks fs = iter(fs) # Yield must be hidden in closure so that the tasks are submitted # before the first iterator value is required. def result_iterator(): try: - for future in fs: - self._futures.add(future) + for task in fs: + self._tasks.add(task) if timeout is not None: try: - yield future.result(end_time - time()) + yield task.result(end_time - time()) except TimeoutError: raise cf.TimeoutError else: - yield future.result() + yield task.result() finally: remaining = list(fs) - self._futures.update(remaining) + self._tasks.update(remaining) self._client.cancel(remaining) return result_iterator() @@ -165,13 +165,13 @@ def shutdown(self, wait=True): Parameters ---------- wait : If True then shutdown will not return until all running - futures have finished executing. If False then all running - futures are cancelled immediately. + tasks have finished executing. If False then all running + tasks are cancelled immediately. """ if not self._shutdown: self._shutdown = True - fs = list(self._futures) + fs = list(self._tasks) if wait: - sync(self._client.loop, _wait_on_futures, fs) + sync(self._client.loop, _wait_on_tasks, fs) else: self._client.cancel(fs) diff --git a/distributed/client.py b/distributed/client.py index 5bea5b6de7..b653fc836c 100644 --- a/distributed/client.py +++ b/distributed/client.py @@ -169,42 +169,42 @@ def _del_global_client(c: Client) -> None: pass -class Future(WrappedKey): +class Task(WrappedKey): """A remotely running computation - A Future is a local proxy to a result running on a remote worker. A user - manages future objects in the local Python process to determine what + A Task is a local proxy to a result running on a remote worker. A user + manages task objects in the local Python process to determine what happens in the larger cluster. Parameters ---------- key: str, or tuple - Key of remote data to which this future refers + Key of remote data to which this task refers client: Client - Client that should own this future. Defaults to _get_global_client() + Client that should own this task. Defaults to _get_global_client() inform: bool - Do we inform the scheduler that we need an update on this future - state: FutureState - The state of the future + Do we inform the scheduler that we need an update on this task + state: TaskState + The state of the task Examples -------- Futures typically emerge from Client computations - >>> my_future = client.submit(add, 1, 2) # doctest: +SKIP + >>> my_task = client.submit(add, 1, 2) # doctest: +SKIP - We can track the progress and results of a future + We can track the progress and results of a task - >>> my_future # doctest: +SKIP - + >>> my_task # doctest: +SKIP + - We can get the result or the exception and traceback from the future + We can get the result or the exception and traceback from the task - >>> my_future.result() # doctest: +SKIP + >>> my_task.result() # doctest: +SKIP See Also -------- - Client: Creates futures + Client: Creates tasks """ _cb_executor = None @@ -235,10 +235,10 @@ def _bind_late(self): self._client._inc_ref(self.key) self._generation = self._client.generation - if self.key in self._client.futures: - self._state = self._client.futures[self.key] + if self.key in self._client.tasks: + self._state = self._client.tasks[self.key] else: - self._state = self._client.futures[self.key] = FutureState() + self._state = self._client.tasks[self.key] = TaskState() if self._inform: self._client._send_to_scheduler( @@ -367,16 +367,16 @@ def exception(self, timeout=None, **kwargs): See Also -------- - Future.traceback + Task.traceback """ self._verify_initialized() return self.client.sync(self._exception, callback_timeout=timeout, **kwargs) def add_done_callback(self, fn): - """Call callback on future when future has finished + """Call callback on task when task has finished - The callback ``fn`` should take the future as its only argument. This - will be called regardless of if the future completes successfully, + The callback ``fn`` should take the task as its only argument. This + will be called regardless of if the task completes successfully, errs, or is cancelled The callback is executed in a separate thread. @@ -387,7 +387,7 @@ def add_done_callback(self, fn): The method or function to be called """ self._verify_initialized() - cls = Future + cls = Task if cls._cb_executor is None or cls._cb_executor_pid != os.getpid(): try: cls._cb_executor = ThreadPoolExecutor( @@ -408,7 +408,7 @@ def execute_callback(fut): ) def cancel(self, **kwargs): - """Cancel the request to run this future + """Cancel the request to run this task See Also -------- @@ -418,7 +418,7 @@ def cancel(self, **kwargs): return self.client.cancel([self], **kwargs) def retry(self, **kwargs): - """Retry this future if it has failed + """Retry this task if it has failed See Also -------- @@ -428,12 +428,12 @@ def retry(self, **kwargs): return self.client.retry([self], **kwargs) def cancelled(self): - """Returns True if the future has been cancelled + """Returns True if the task has been cancelled Returns ------- bool - True if the future was 'cancelled', otherwise False + True if the task was 'cancelled', otherwise False """ return self._state.status == "cancelled" @@ -448,7 +448,7 @@ def traceback(self, timeout=None, **kwargs): """Return the traceback of a failed task This returns a traceback object. You can inspect this object using the - ``traceback`` module. Alternatively if you call ``future.result()`` + ``traceback`` module. Alternatively if you call ``task.result()`` this traceback will accompany the raised exception. Parameters @@ -462,7 +462,7 @@ def traceback(self, timeout=None, **kwargs): Examples -------- >>> import traceback # doctest: +SKIP - >>> tb = future.traceback() # doctest: +SKIP + >>> tb = task.traceback() # doctest: +SKIP >>> traceback.format_tb(tb) # doctest: +SKIP [...] @@ -473,7 +473,7 @@ def traceback(self, timeout=None, **kwargs): See Also -------- - Future.exception + Task.exception """ self._verify_initialized() return self.client.sync(self._traceback, callback_timeout=timeout, **kwargs) @@ -488,7 +488,7 @@ def release(self): Notes ----- This method can be called from different threads - (see e.g. Client.get() or Future.__del__()) + (see e.g. Client.get() or Task.__del__()) """ self._verify_initialized() if not self._cleared and self.client.generation == self._generation: @@ -499,7 +499,7 @@ def release(self): pass # Shutting down, add_callback may be None def __reduce__(self) -> str | tuple[Any, ...]: - return Future, (self.key,) + return Task, (self.key,) def __del__(self): try: @@ -515,10 +515,10 @@ def __del__(self): def __repr__(self): if self.type: return ( - f"" + f"" ) else: - return f"" + return f"" def _repr_html_(self): return get_template("future.html.j2").render( @@ -531,8 +531,8 @@ def __await__(self): return self.result().__await__() -class FutureState: - """A Future's internal state. +class TaskState: + """A Task's internal state. This is shared between all Futures with the same key and client. """ @@ -627,23 +627,23 @@ def __repr__(self): return f"<{self.__class__.__name__}: {self.status}>" -async def done_callback(future, callback): - """Coroutine that waits on the future, then calls the callback +async def done_callback(task, callback): + """Coroutine that waits on the task, then calls the callback Parameters ---------- - future : asyncio.Future - The future + task : asyncio.Future + The task callback : callable The callback """ - while future.status == "pending": - await future._state.wait() - callback(future) + while task.status == "pending": + await task._state.wait() + callback(task) -@partial(normalize_token.register, Future) -def normalize_future(f): +@partial(normalize_token.register, Task) +def normalize_task(f): """Returns the key and the type as a list Parameters @@ -740,8 +740,8 @@ class Client(SyncMethodMixin): """Connect to and submit computation to a Dask cluster The Client connects users to a Dask cluster. It provides an asynchronous - user interface around functions and futures. This class resembles - executors in ``concurrent.futures`` but also allows ``Future`` objects + user interface around functions and tasks. This class resembles + executors in ``concurrent.futures`` but also allows ``Task`` objects within ``submit/map`` calls. When a Client is instantiated it takes over all ``dask.compute`` and ``dask.persist`` calls by default. @@ -861,7 +861,7 @@ def __init__( timeout = parse_timedelta(timeout, "s") self._timeout = timeout - self.futures = dict() + self.tasks = dict() self.refcount = defaultdict(int) self._handle_report_task = None if name is None: @@ -880,9 +880,9 @@ def __init__( self.cluster = None self.scheduler = None self._scheduler_identity = {} - # A reentrant-lock on the refcounts for futures associated with this + # A reentrant-lock on the refcounts for tasks associated with this # client. Should be held by individual operations modifying refcounts, - # or any bulk operation that needs to ensure the set of futures doesn't + # or any bulk operation that needs to ensure the set of tasks doesn't # change during operation. self._refcount_lock = threading.RLock() self.datasets = Datasets(self) @@ -952,7 +952,7 @@ def __init__( self._connecting_to_scheduler = False self._gather_keys = None - self._gather_future = None + self._gather_task = None if heartbeat_interval is None: heartbeat_interval = dask.config.get("distributed.client.heartbeat") @@ -1053,7 +1053,7 @@ def loop(self, value: IOLoop) -> None: @contextmanager def as_current(self): """Thread-local, Task-local context manager that causes the Client.current - class method to return self. Any Future objects deserialized inside this + class method to return self. Any Task objects deserialized inside this context manager will be automatically attached to this Client. """ tok = _current_client.set(self) @@ -1317,9 +1317,9 @@ async def _reconnect(self): self.status = "connecting" self.scheduler_comm = None - for st in self.futures.values(): + for st in self.tasks.values(): st.cancel() - self.futures.clear() + self.tasks.clear() timeout = self._timeout deadline = time() + timeout @@ -1535,7 +1535,7 @@ def _dec_ref(self, key): def _release_key(self, key): """Release key from distributed memory""" logger.debug("Release key %s", key) - st = self.futures.pop(key, None) + st = self.tasks.pop(key, None) if st is not None: st.cancel() if self.status != "closed": @@ -1601,7 +1601,7 @@ async def _handle_report(self): pass def _handle_key_in_memory(self, key=None, type=None, workers=None): - state = self.futures.get(key) + state = self.tasks.get(key) if state is not None: if type and not state.type: # Type exists and not yet set try: @@ -1615,31 +1615,31 @@ def _handle_key_in_memory(self, key=None, type=None, workers=None): state.finish(type) def _handle_lost_data(self, key=None): - state = self.futures.get(key) + state = self.tasks.get(key) if state is not None: state.lose() def _handle_cancelled_keys(self, keys): for key in keys: - state = self.futures.get(key) + state = self.tasks.get(key) if state is not None: state.cancel() def _handle_retried_key(self, key=None): - state = self.futures.get(key) + state = self.tasks.get(key) if state is not None: state.retry() def _handle_task_erred(self, key=None, exception=None, traceback=None): - state = self.futures.get(key) + state = self.tasks.get(key) if state is not None: state.set_error(exception, traceback) def _handle_restart(self): logger.info("Receive restart signal from scheduler") - for state in self.futures.values(): + for state in self.tasks.values(): state.cancel() - self.futures.clear() + self.tasks.clear() self.generation += 1 with self._refcount_lock: self.refcount.clear() @@ -1714,7 +1714,7 @@ async def _close(self, fast=False): ): await self.scheduler_comm.close() - for key in list(self.futures): + for key in list(self.tasks): self._release_key(key=key) if self._start_arg is None: @@ -1903,7 +1903,7 @@ def submit( Notes ----- The current implementation of a task graph resolution searches for occurrences of ``key`` - and replaces it with a corresponding ``Future`` result. That can lead to unwanted + and replaces it with a corresponding ``Task`` result. That can lead to unwanted substitution of strings passed as arguments to a task if these strings match some ``key`` that already exists on a cluster. To avoid these situations it is required to use unique values if a ``key`` is set manually. @@ -1911,8 +1911,8 @@ def submit( Returns ------- - Future - If running in asynchronous mode, returns the future. Otherwise + Task + If running in asynchronous mode, returns the task. Otherwise returns the concrete value Raises @@ -1944,8 +1944,8 @@ def submit( key = funcname(func) + "-" + str(uuid.uuid4()) with self._refcount_lock: - if key in self.futures: - return Future(key, self, inform=False) + if key in self.tasks: + return Task(key, self, inform=False) if allow_other_workers and workers is None: raise ValueError("Only use allow_other_workers= if using workers=") @@ -1958,7 +1958,7 @@ def submit( else: dsk = {key: (func,) + tuple(args)} - futures = self._graph_to_futures( + tasks = self._graph_to_tasks( dsk, [key], workers=workers, @@ -1973,7 +1973,7 @@ def submit( logger.debug("Submit %s(...), %s", funcname(func), key) - return futures[key] + return tasks[key] def map( self, @@ -2053,7 +2053,7 @@ def map( Notes ----- The current implementation of a task graph resolution searches for occurrences of ``key`` - and replaces it with a corresponding ``Future`` result. That can lead to unwanted + and replaces it with a corresponding ``Task`` result. That can lead to unwanted substitution of strings passed as arguments to a task if these strings match some ``key`` that already exists on a cluster. To avoid these situations it is required to use unique values if a ``key`` is set manually. @@ -2061,7 +2061,7 @@ def map( Returns ------- - List, iterator, or Queue of futures, depending on the type of the + List, iterator, or Queue of tasks, depending on the type of the inputs. See Also @@ -2164,7 +2164,7 @@ def map( internal_priority = dict(zip(keys, range(len(keys)))) - futures = self._graph_to_futures( + tasks = self._graph_to_tasks( dsk, keys, workers=workers, @@ -2178,19 +2178,19 @@ def map( ) logger.debug("map(%s, ...)", funcname(func)) - return [futures[k] for k in keys] + return [tasks[k] for k in keys] - async def _gather(self, futures, errors="raise", direct=None, local_worker=None): - unpacked, future_set = unpack_remotedata(futures, byte_keys=True) - mismatched_futures = [f for f in future_set if f.client is not self] - if mismatched_futures: + async def _gather(self, tasks, errors="raise", direct=None, local_worker=None): + unpacked, task_set = unpack_remotedata(tasks, byte_keys=True) + mismatched_tasks = [f for f in task_set if f.client is not self] + if mismatched_tasks: raise ValueError( "Cannot gather Futures created by another client. " - f"These are the {len(mismatched_futures)} (out of {len(futures)}) " + f"These are the {len(mismatched_tasks)} (out of {len(tasks)}) " f"mismatched Futures and their client IDs (this client is {self.id}): " - f"{ {f: f.client.id for f in mismatched_futures} }" # noqa: E201, E202 + f"{ {f: f.client.id for f in mismatched_tasks} }" # noqa: E201, E202 ) - keys = [future.key for future in future_set] + keys = [task.key for task in task_set] bad_data = dict() data = {} @@ -2208,7 +2208,7 @@ async def _gather(self, futures, errors="raise", direct=None, local_worker=None) async def wait(k): """Want to stop the All(...) early if we find an error""" try: - st = self.futures[k] + st = self.tasks[k] except KeyError: raise AllExit() else: @@ -2217,11 +2217,11 @@ async def wait(k): raise AllExit() while True: - logger.debug("Waiting on futures to clear before gather") + logger.debug("Waiting on tasks to clear before gather") with suppress(AllExit): await distributed.utils.All( - [wait(key) for key in keys if key in self.futures], + [wait(key) for key in keys if key in self.tasks], quiet_exceptions=AllExit, ) @@ -2230,11 +2230,11 @@ async def wait(k): exceptions = set() bad_keys = set() for key in keys: - if key not in self.futures or self.futures[key].status in failed: + if key not in self.tasks or self.tasks[key].status in failed: exceptions.add(key) if errors == "raise": try: - st = self.futures[key] + st = self.tasks[key] exception = st.exception traceback = st.traceback except (KeyError, AttributeError): @@ -2257,19 +2257,17 @@ async def wait(k): keys = [k for k in keys if k not in data] # We now do an actual remote communication with workers or scheduler - if self._gather_future: # attach onto another pending gather request + if self._gather_task: # attach onto another pending gather request self._gather_keys |= set(keys) - response = await self._gather_future + response = await self._gather_task else: # no one waiting, go ahead self._gather_keys = set(keys) - future = asyncio.ensure_future( - self._gather_remote(direct, local_worker) - ) + task = asyncio.ensure_future(self._gather_remote(direct, local_worker)) if self._gather_keys is None: - self._gather_future = None + self._gather_task = None else: - self._gather_future = future - response = await future + self._gather_task = task + response = await task if response["status"] == "error": log = logger.warning if errors == "raise" else logger.debug @@ -2282,7 +2280,7 @@ async def wait(k): self._send_to_scheduler({"op": "report-key", "key": key}) for key in response["keys"]: try: - self.futures[key].reset() + self.tasks[key].reset() except KeyError: # TODO: verify that this is safe pass else: # pragma: no cover @@ -2305,7 +2303,7 @@ async def _gather_remote(self, direct: bool, local_worker: bool) -> dict[str, An async with self._gather_semaphore: keys = list(self._gather_keys) self._gather_keys = None # clear state, these keys are being sent off - self._gather_future = None + self._gather_task = None if direct or local_worker: # gather directly from workers who_has = await retry_operation(self.scheduler.who_has, keys=keys) @@ -2325,19 +2323,19 @@ async def _gather_remote(self, direct: bool, local_worker: bool) -> dict[str, An return response - def gather(self, futures, errors="raise", direct=None, asynchronous=None): - """Gather futures from distributed memory + def gather(self, tasks, errors="raise", direct=None, asynchronous=None): + """Gather tasks from distributed memory - Accepts a future, nested container of futures, iterator, or queue. + Accepts a task, nested container of tasks, iterator, or queue. The return type will match the input type. Parameters ---------- - futures : Collection of futures - This can be a possibly nested collection of Future objects. + tasks : Collection of tasks + This can be a possibly nested collection of Task objects. Collections can be lists, sets, or dictionaries errors : string - Either 'raise' or 'skip' if we should raise if a future has erred + Either 'raise' or 'skip' if we should raise if a task has erred or skip its inclusion in the output collection direct : boolean Whether or not to connect directly to the workers, or to ask @@ -2349,7 +2347,7 @@ def gather(self, futures, errors="raise", direct=None, asynchronous=None): Returns ------- results: a collection of the same type as the input, but now with - gathered results rather than futures + gathered results rather than tasks Examples -------- @@ -2365,14 +2363,14 @@ def gather(self, futures, errors="raise", direct=None, asynchronous=None): -------- Client.scatter : Send data out to cluster """ - if isinstance(futures, pyQueue): + if isinstance(tasks, pyQueue): raise TypeError( "Dask no longer supports gathering over Iterators and Queues. " "Consider using a normal for loop and Client.submit/gather" ) - if isinstance(futures, Iterator): - return (self.gather(f, errors=errors, direct=direct) for f in futures) + if isinstance(tasks, Iterator): + return (self.gather(f, errors=errors, direct=direct) for f in tasks) try: local_worker = get_worker() @@ -2382,7 +2380,7 @@ def gather(self, futures, errors="raise", direct=None, asynchronous=None): with shorten_traceback(): return self.sync( self._gather, - futures, + tasks, errors=errors, direct=direct, local_worker=local_worker, @@ -2478,9 +2476,9 @@ async def _scatter( timeout=timeout, ) - out = {k: Future(k, self, inform=False) for k in data} + out = {k: Task(k, self, inform=False) for k in data} for key, typ in types.items(): - self.futures[key].finish(type=typ) + self.tasks[key].finish(type=typ) if direct and broadcast: n = None if broadcast is True else broadcast @@ -2542,23 +2540,23 @@ def scatter( Returns ------- - List, dict, iterator, or queue of futures matching the type of input. + List, dict, iterator, or queue of tasks matching the type of input. Examples -------- >>> c = Client('127.0.0.1:8787') # doctest: +SKIP >>> c.scatter(1) # doctest: +SKIP - + >>> c.scatter([1, 2, 3]) # doctest: +SKIP - [, - , - ] + [, + , + ] >>> c.scatter({'x': 1, 'y': 2, 'z': 3}) # doctest: +SKIP - {'x': , - 'y': , - 'z': } + {'x': , + 'y': , + 'z': } Constrain location of data to subset of workers @@ -2566,9 +2564,9 @@ def scatter( Broadcast data to all workers - >>> [future] = c.scatter([element], broadcast=True) # doctest: +SKIP + >>> [task] = c.scatter([element], broadcast=True) # doctest: +SKIP - Send scattered data to parallelized function using client futures + Send scattered data to parallelized function using client tasks interface >>> data = c.scatter(data, broadcast=True) # doctest: +SKIP @@ -2576,9 +2574,9 @@ def scatter( Notes ----- - Scattering a dictionary uses ``dict`` keys to create ``Future`` keys. + Scattering a dictionary uses ``dict`` keys to create ``Task`` keys. The current implementation of a task graph resolution searches for occurrences of ``key`` - and replaces it with a corresponding ``Future`` result. That can lead to unwanted + and replaces it with a corresponding ``Task`` result. That can lead to unwanted substitution of strings passed as arguments to a task if these strings match some ``key`` that already exists on a cluster. To avoid these situations it is required to use unique values if a ``key`` is set manually. @@ -2612,59 +2610,59 @@ def scatter( hash=hash, ) - async def _cancel(self, futures, force=False): - # FIXME: This method is asynchronous since interacting with the FutureState below requires an event loop. - keys = list({f.key for f in futures_of(futures)}) + async def _cancel(self, tasks, force=False): + # FIXME: This method is asynchronous since interacting with the TaskState below requires an event loop. + keys = list({f.key for f in tasks_of(tasks)}) self._send_to_scheduler({"op": "cancel-keys", "keys": keys, "force": force}) for k in keys: - st = self.futures.pop(k, None) + st = self.tasks.pop(k, None) if st is not None: st.cancel() - def cancel(self, futures, asynchronous=None, force=False): + def cancel(self, tasks, asynchronous=None, force=False): """ - Cancel running futures - This stops future tasks from being scheduled if they have not yet run + Cancel running tasks + This stops task tasks from being scheduled if they have not yet run and deletes them if they have already run. After calling, this result and all dependent results will no longer be accessible Parameters ---------- - futures : List[Future] + tasks : List[Task] The list of Futures asynchronous: bool If True the client is in asynchronous mode force : boolean (False) - Cancel this future even if other clients desire it + Cancel this task even if other clients desire it """ - return self.sync(self._cancel, futures, asynchronous=asynchronous, force=force) + return self.sync(self._cancel, tasks, asynchronous=asynchronous, force=force) - async def _retry(self, futures): - keys = list({f.key for f in futures_of(futures)}) + async def _retry(self, tasks): + keys = list({f.key for f in tasks_of(tasks)}) response = await self.scheduler.retry(keys=keys, client=self.id) for key in response: - st = self.futures[key] + st = self.tasks[key] st.retry() - def retry(self, futures, asynchronous=None): + def retry(self, tasks, asynchronous=None): """ - Retry failed futures + Retry failed tasks Parameters ---------- - futures : list of Futures + tasks : list of Futures The list of Futures asynchronous: bool If True the client is in asynchronous mode """ - return self.sync(self._retry, futures, asynchronous=asynchronous) + return self.sync(self._retry, tasks, asynchronous=asynchronous) @log_errors async def _publish_dataset(self, *args, name=None, override=False, **kwargs): coroutines = [] def add_coro(name, data): - keys = [f.key for f in futures_of(data)] + keys = [f.key for f in tasks_of(data)] coroutines.append( self.scheduler.publish_put( keys=keys, @@ -2695,9 +2693,9 @@ def publish_dataset(self, *args, **kwargs): """ Publish named datasets to scheduler - This stores a named reference to a dask collection or list of futures + This stores a named reference to a dask collection or list of tasks on the scheduler. These references are available to other Clients - which can download the collection or futures with ``get_dataset``. + which can download the collection or tasks with ``get_dataset``. Datasets are not immediately computed. You may wish to call ``Client.persist`` prior to publishing a dataset. @@ -3102,7 +3100,7 @@ def _get_computation_code( return tuple(reversed(code)) - def _graph_to_futures( + def _graph_to_tasks( self, dsk, keys, @@ -3149,8 +3147,8 @@ def _graph_to_futures( for key in keyset: validate_key(key) - # Create futures before sending graph (helps avoid contention) - futures = {key: Future(key, self, inform=False) for key in keyset} + # Create tasks before sending graph (helps avoid contention) + tasks = {key: Task(key, self, inform=False) for key in keyset} # Circular import from distributed.protocol import serialize from distributed.protocol.serialize import ToPickle @@ -3162,7 +3160,7 @@ def _graph_to_futures( warnings.warn( f"Sending large graph of size {format_bytes(pickled_size)}.\n" "This may cause some slowdown.\n" - "Consider scattering data ahead of time and using futures." + "Consider scattering data ahead of time and using tasks." ) computations = self._get_computation_code( @@ -3182,7 +3180,7 @@ def _graph_to_futures( "annotations": ToPickle(annotations), } ) - return futures + return tasks def get( self, @@ -3257,7 +3255,7 @@ def get( -------- Client.compute : Compute asynchronous collections """ - futures = self._graph_to_futures( + tasks = self._graph_to_tasks( dsk, keys=set(flatten([keys])), workers=workers, @@ -3268,7 +3266,7 @@ def get( user_priority=priority, actors=actors, ) - packed = pack_data(keys, futures) + packed = pack_data(keys, tasks) if sync: if getattr(thread_state, "key", False): try: @@ -3279,18 +3277,18 @@ def get( try: results = self.gather(packed, asynchronous=asynchronous, direct=direct) finally: - for f in futures.values(): + for f in tasks.values(): f.release() if getattr(thread_state, "key", False) and should_rejoin: rejoin() return results return packed - def _optimize_insert_futures(self, dsk, keys): + def _optimize_insert_tasks(self, dsk, keys): """Replace known keys in dask graph with Futures When given a Dask graph that might have overlapping keys with our known - results we replace the values of that graph with futures. This can be + results we replace the values of that graph with tasks. This can be used as an optimization to avoid recomputation. This returns the same graph if unchanged but a new graph if any changes @@ -3299,11 +3297,11 @@ def _optimize_insert_futures(self, dsk, keys): with self._refcount_lock: changed = False for key in list(dsk): - if key in self.futures: + if key in self.tasks: if not changed: changed = True dsk = ensure_dict(dsk) - dsk[key] = Future(key, self, inform=False) + dsk[key] = Task(key, self, inform=False) if changed: dsk, _ = dask.optimization.cull(dsk, keys) @@ -3312,11 +3310,11 @@ def _optimize_insert_futures(self, dsk, keys): def normalize_collection(self, collection): """ - Replace collection's tasks by already existing futures if they exist + Replace collection's tasks by already existing tasks if they exist This normalizes the tasks within a collections task graph against the - known futures within the scheduler. It returns a copy of the - collection with a task graph that includes the overlapping futures. + known tasks within the scheduler. It returns a copy of the + collection with a task graph that includes the overlapping tasks. Parameters ---------- @@ -3326,13 +3324,13 @@ def normalize_collection(self, collection): Returns ------- collection : dask object - Collection with its tasks replaced with any existing futures. + Collection with its tasks replaced with any existing tasks. Examples -------- >>> len(x.__dask_graph__()) # x is a dask collection with 100 tasks # doctest: +SKIP 100 - >>> set(client.futures).intersection(x.__dask_graph__()) # some overlap exists # doctest: +SKIP + >>> set(client.tasks).intersection(x.__dask_graph__()) # some overlap exists # doctest: +SKIP 10 >>> x = client.normalize_collection(x) # doctest: +SKIP @@ -3344,7 +3342,7 @@ def normalize_collection(self, collection): Client.persist : trigger computation of collection's tasks """ dsk_orig = collection.__dask_graph__() - dsk = self._optimize_insert_futures(dsk_orig, collection.__dask_keys__()) + dsk = self._optimize_insert_tasks(dsk_orig, collection.__dask_keys__()) if dsk is dsk_orig: return collection @@ -3408,7 +3406,7 @@ def compute( Returns ------- - List of Futures if input is a sequence, or a single future otherwise + List of Futures if input is a sequence, or a single task otherwise Examples -------- @@ -3418,7 +3416,7 @@ def compute( >>> y = delayed(add)(x, x) >>> xx, yy = client.compute([x, y]) # doctest: +SKIP >>> xx # doctest: +SKIP - + >>> xx.result() # doctest: +SKIP 3 >>> yy.result() # doctest: +SKIP @@ -3470,7 +3468,7 @@ def compute( dependencies.update(dsk.dependencies) dsk = HighLevelGraph(layers, dependencies) - futures_dict = self._graph_to_futures( + tasks_dict = self._graph_to_tasks( dsk, names, workers=workers, @@ -3483,18 +3481,18 @@ def compute( ) i = 0 - futures = [] + tasks = [] for arg in collections: if dask.is_dask_collection(arg): - futures.append(futures_dict[names[i]]) + tasks.append(tasks_dict[names[i]]) i += 1 else: - futures.append(arg) + tasks.append(arg) if sync: - result = self.gather(futures) + result = self.gather(tasks) else: - result = futures + result = tasks if singleton: return first(result) @@ -3518,7 +3516,7 @@ def persist( Starts computation of the collection on the cluster in the background. Provides a new dask collection that is semantically identical to the - previous one, but now based off of futures currently in execution. + previous one, but now based off of tasks currently in execution. Parameters ---------- @@ -3576,7 +3574,7 @@ def persist( names = {k for c in collections for k in flatten(c.__dask_keys__())} - futures = self._graph_to_futures( + tasks = self._graph_to_tasks( dsk, names, workers=workers, @@ -3590,7 +3588,7 @@ def persist( postpersists = [c.__dask_postpersist__() for c in collections] result = [ - func({k: futures[k] for k in flatten(c.__dask_keys__())}, *args) + func({k: tasks[k] for k in flatten(c.__dask_keys__())}, *args) for (func, args), c in zip(postpersists, collections) ] @@ -3749,9 +3747,9 @@ async def _upload_large_file(self, local_filename, remote_filename=None): with open(local_filename, "rb") as f: data = f.read() - [future] = await self._scatter([data]) - key = future.key - await self._replicate(future) + [task] = await self._scatter([data]) + key = task.key + await self._replicate(task) def dump_to_file(dask_worker=None): if not os.path.isabs(remote_filename): @@ -3813,10 +3811,10 @@ async def _(): return self.sync(_) - async def _rebalance(self, futures=None, workers=None): - if futures is not None: - await _wait(futures) - keys = list({f.key for f in self.futures_of(futures)}) + async def _rebalance(self, tasks=None, workers=None): + if tasks is not None: + await _wait(tasks) + keys = list({f.key for f in self.tasks_of(tasks)}) else: keys = None result = await self.scheduler.rebalance(keys=keys, workers=workers) @@ -3824,7 +3822,7 @@ async def _rebalance(self, futures=None, workers=None): raise KeyError(f"Could not rebalance keys: {result['keys']}") assert result["status"] == "OK", result - def rebalance(self, futures=None, workers=None, **kwargs): + def rebalance(self, tasks=None, workers=None, **kwargs): """Rebalance data within network Move data between workers to roughly balance memory burden. This @@ -3840,32 +3838,32 @@ def rebalance(self, futures=None, workers=None, **kwargs): Parameters ---------- - futures : list, optional - A list of futures to balance, defaults all data + tasks : list, optional + A list of tasks to balance, defaults all data workers : list, optional A list of workers on which to balance, defaults to all workers **kwargs : dict Optional keyword arguments for the function """ - return self.sync(self._rebalance, futures, workers, **kwargs) + return self.sync(self._rebalance, tasks, workers, **kwargs) - async def _replicate(self, futures, n=None, workers=None, branching_factor=2): - futures = self.futures_of(futures) - await _wait(futures) - keys = {f.key for f in futures} + async def _replicate(self, tasks, n=None, workers=None, branching_factor=2): + tasks = self.tasks_of(tasks) + await _wait(tasks) + keys = {f.key for f in tasks} await self.scheduler.replicate( keys=list(keys), n=n, workers=workers, branching_factor=branching_factor ) - def replicate(self, futures, n=None, workers=None, branching_factor=2, **kwargs): - """Set replication of futures within network + def replicate(self, tasks, n=None, workers=None, branching_factor=2, **kwargs): + """Set replication of tasks within network Copy data onto many workers. This helps to broadcast frequently accessed data and can improve resilience. This performs a tree copy of the data throughout the network individually on each piece of data. This operation blocks until - complete. It does not guarantee replication of data to future workers. + complete. It does not guarantee replication of data to task workers. .. note:: This method is incompatible with the Active Memory Manager's @@ -3874,7 +3872,7 @@ def replicate(self, futures, n=None, workers=None, branching_factor=2, **kwargs) Parameters ---------- - futures : list of futures + tasks : list of tasks Futures we wish to replicate n : int, optional Number of processes on the cluster on which to replicate the data. @@ -3902,7 +3900,7 @@ def replicate(self, futures, n=None, workers=None, branching_factor=2, **kwargs) """ return self.sync( self._replicate, - futures, + tasks, n=n, workers=workers, branching_factor=branching_factor, @@ -3943,13 +3941,13 @@ def nthreads(self, workers=None, **kwargs): ncores = nthreads - def who_has(self, futures=None, **kwargs): - """The workers storing each future's data + def who_has(self, tasks=None, **kwargs): + """The workers storing each task's data Parameters ---------- - futures : list (optional) - A list of futures, defaults to all data + tasks : list (optional) + A list of tasks, defaults to all data **kwargs : dict Optional keyword arguments for the remote function @@ -3971,9 +3969,9 @@ def who_has(self, futures=None, **kwargs): Client.has_what Client.nthreads """ - if futures is not None: - futures = self.futures_of(futures) - keys = list({f.key for f in futures}) + if tasks is not None: + tasks = self.tasks_of(tasks) + keys = list({f.key for f in tasks}) else: keys = None @@ -4084,18 +4082,18 @@ def nbytes(self, keys=None, summary=True, **kwargs): """ return self.sync(self.scheduler.nbytes, keys=keys, summary=summary, **kwargs) - def call_stack(self, futures=None, keys=None): + def call_stack(self, tasks=None, keys=None): """The actively running call stack of all relevant keys - You can specify data of interest either by providing futures or - collections in the ``futures=`` keyword or a list of explicit keys in + You can specify data of interest either by providing tasks or + collections in the ``tasks=`` keyword or a list of explicit keys in the ``keys=`` keyword. If neither are provided then all call stacks will be returned. Parameters ---------- - futures : list (optional) - List of futures, defaults to all data + tasks : list (optional) + List of tasks, defaults to all data keys : list (optional) List of key names, defaults to all data @@ -4107,9 +4105,9 @@ def call_stack(self, futures=None, keys=None): >>> client.call_stack() # Or call with no arguments for all activity # doctest: +SKIP """ keys = keys or [] - if futures is not None: - futures = self.futures_of(futures) - keys += list({f.key for f in futures}) + if tasks is not None: + tasks = self.tasks_of(tasks) + keys += list({f.key for f in tasks}) return self.sync(self.scheduler.call_stack, keys=keys or None) def profile( @@ -4673,15 +4671,15 @@ async def _get_versions( return result - def futures_of(self, futures): - """Wrapper method of futures_of + def tasks_of(self, tasks): + """Wrapper method of tasks_of Parameters ---------- - futures : tuple - The futures + tasks : tuple + The tasks """ - return futures_of(futures, client=self) + return tasks_of(tasks, client=self) @classmethod def _expand_key(cls, k): @@ -4995,12 +4993,12 @@ def unregister_scheduler_plugin(self, name): def register_worker_callbacks(self, setup=None): """ - Registers a setup callback function for all current and future workers. + Registers a setup callback function for all current and task workers. This registers a new setup function for workers in this cluster. The function will run immediately on all currently connected workers. It will also be run upon connection by any workers that are added in the - future. Multiple setup functions can be registered - these will be + task. Multiple setup functions can be registered - these will be called in the order they were added. If the function takes an input argument named ``dask_worker`` then @@ -5050,7 +5048,7 @@ def register_worker_plugin( nanny: bool | None = None, ): """ - Registers a lifecycle worker plugin for all current and future workers. + Registers a lifecycle worker plugin for all current and task workers. .. deprecated:: 2023.9.2 Use :meth:`Client.register_plugin` instead. @@ -5110,7 +5108,7 @@ def register_worker_plugin( ... plugin = worker.plugins['my-plugin'] ... return plugin.my_state - >>> future = client.run(f) + >>> task = client.run(f) See Also -------- @@ -5408,19 +5406,19 @@ async def _wait(fs, timeout=None, return_when=ALL_COMPLETED): " Bad: wait(x, y, z)\n" " Good: wait([x, y, z])" ) - fs = futures_of(fs) + fs = tasks_of(fs) if return_when == ALL_COMPLETED: - future = distributed.utils.All({f._state.wait() for f in fs}) + task = distributed.utils.All({f._state.wait() for f in fs}) elif return_when == FIRST_COMPLETED: - future = distributed.utils.Any({f._state.wait() for f in fs}) + task = distributed.utils.Any({f._state.wait() for f in fs}) else: raise NotImplementedError( "Only return_when='ALL_COMPLETED' and 'FIRST_COMPLETED' are supported" ) if timeout is not None: - future = wait_for(future, timeout) - await future + task = wait_for(task, timeout) + await task done, not_done = ( {fu for fu in fs if fu.status != "pending"}, @@ -5434,11 +5432,11 @@ async def _wait(fs, timeout=None, return_when=ALL_COMPLETED): def wait(fs, timeout=None, return_when=ALL_COMPLETED): - """Wait until all/any futures are finished + """Wait until all/any tasks are finished Parameters ---------- - fs : List[Future] + fs : List[Task] timeout : number, string, optional Time after which to raise a ``dask.distributed.TimeoutError``. Can be a string like ``"10 minutes"`` or a number of seconds to wait. @@ -5457,7 +5455,7 @@ def wait(fs, timeout=None, return_when=ALL_COMPLETED): async def _as_completed(fs, queue): - fs = futures_of(fs) + fs = tasks_of(fs) groups = groupby(lambda f: f.key, fs) firsts = [v[0] for v in groups.values()] wait_iterator = gen.WaitIterator( @@ -5466,45 +5464,45 @@ async def _as_completed(fs, queue): while not wait_iterator.done(): await wait_iterator.next() - # TODO: handle case of restarted futures - future = firsts[wait_iterator.current_index] - for f in groups[future.key]: + # TODO: handle case of restarted tasks + task = firsts[wait_iterator.current_index] + for f in groups[task.key]: queue.put_nowait(f) -async def _first_completed(futures): - """Return a single completed future +async def _first_completed(tasks): + """Return a single completed task See Also: _as_completed """ q = asyncio.Queue() - await _as_completed(futures, q) + await _as_completed(tasks, q) result = await q.get() return result class as_completed: """ - Return futures in the order in which they complete + Return tasks in the order in which they complete - This returns an iterator that yields the input future objects in the order + This returns an iterator that yields the input task objects in the order in which they complete. Calling ``next`` on the iterator will block until - the next future completes, irrespective of order. + the next task completes, irrespective of order. - Additionally, you can also add more futures to this object during + Additionally, you can also add more tasks to this object during computation with the ``.add`` method Parameters ---------- - futures: Collection of futures - A list of Future objects to be iterated over in the order in which they + tasks: Collection of tasks + A list of Task objects to be iterated over in the order in which they complete with_results: bool (False) - Whether to wait and include results of futures as well; - in this case ``as_completed`` yields a tuple of (future, result) + Whether to wait and include results of tasks as well; + in this case ``as_completed`` yields a tuple of (task, result) raise_errors: bool (True) - Whether we should raise when the result of a future raises an + Whether we should raise when the result of a task raises an exception; only affects behavior when ``with_results=True``. timeout: int (optional) The returned iterator raises a ``dask.distributed.TimeoutError`` @@ -5516,20 +5514,20 @@ class as_completed: Examples -------- >>> x, y, z = client.map(inc, [1, 2, 3]) # doctest: +SKIP - >>> for future in as_completed([x, y, z]): # doctest: +SKIP - ... print(future.result()) # doctest: +SKIP + >>> for task in as_completed([x, y, z]): # doctest: +SKIP + ... print(task.result()) # doctest: +SKIP 3 2 4 - Add more futures during computation + Add more tasks during computation >>> x, y, z = client.map(inc, [1, 2, 3]) # doctest: +SKIP >>> ac = as_completed([x, y, z]) # doctest: +SKIP - >>> for future in ac: # doctest: +SKIP - ... print(future.result()) # doctest: +SKIP + >>> for task in ac: # doctest: +SKIP + ... print(task.result()) # doctest: +SKIP ... if random.random() < 0.5: # doctest: +SKIP - ... ac.add(c.submit(double, future)) # doctest: +SKIP + ... ac.add(c.submit(double, task)) # doctest: +SKIP 4 2 8 @@ -5541,7 +5539,7 @@ class as_completed: Optionally wait until the result has been gathered as well >>> ac = as_completed([x, y, z], with_results=True) # doctest: +SKIP - >>> for future, result in ac: # doctest: +SKIP + >>> for task, result in ac: # doctest: +SKIP ... print(result) # doctest: +SKIP 2 4 @@ -5550,16 +5548,16 @@ class as_completed: def __init__( self, - futures=None, + tasks=None, loop=None, with_results=False, raise_errors=True, *, timeout=None, ): - if futures is None: - futures = [] - self.futures = defaultdict(int) + if tasks is None: + tasks = [] + self.tasks = defaultdict(int) self.queue = pyQueue() self.lock = threading.Lock() self.loop = loop or default_client().loop @@ -5568,8 +5566,8 @@ def __init__( self.raise_errors = raise_errors self._deadline = Deadline.after(parse_timedelta(timeout)) - if futures: - self.update(futures) + if tasks: + self.update(tasks) @property def condition(self): @@ -5579,71 +5577,71 @@ def condition(self): self._condition = asyncio.Condition() return self._condition - async def _track_future(self, future): + async def _track_task(self, task): try: - await _wait(future) + await _wait(task) except CancelledError: pass if self.with_results: try: - result = await future._result(raiseit=False) + result = await task._result(raiseit=False) except CancelledError as exc: result = exc with self.lock: - if future in self.futures: - self.futures[future] -= 1 - if not self.futures[future]: - del self.futures[future] + if task in self.tasks: + self.tasks[task] -= 1 + if not self.tasks[task]: + del self.tasks[task] if self.with_results: - self.queue.put_nowait((future, result)) + self.queue.put_nowait((task, result)) else: - self.queue.put_nowait(future) + self.queue.put_nowait(task) async with self.condition: self.condition.notify() with self.thread_condition: self.thread_condition.notify() - def update(self, futures): - """Add multiple futures to the collection. + def update(self, tasks): + """Add multiple tasks to the collection. - The added futures will emit from the iterator once they finish""" - from distributed.actor import BaseActorFuture + The added tasks will emit from the iterator once they finish""" + from distributed.actor import BaseActorTask with self.lock: - for f in futures: - if not isinstance(f, (Future, BaseActorFuture)): - raise TypeError("Input must be a future, got %s" % f) - self.futures[f] += 1 - self.loop.add_callback(self._track_future, f) + for f in tasks: + if not isinstance(f, (Task, BaseActorTask)): + raise TypeError("Input must be a task, got %s" % f) + self.tasks[f] += 1 + self.loop.add_callback(self._track_task, f) - def add(self, future): - """Add a future to the collection + def add(self, task): + """Add a task to the collection - This future will emit from the iterator once it finishes + This task will emit from the iterator once it finishes """ - self.update((future,)) + self.update((task,)) def is_empty(self): - """Returns True if there no completed or computing futures""" + """Returns True if there no completed or computing tasks""" return not self.count() def has_ready(self): - """Returns True if there are completed futures available.""" + """Returns True if there are completed tasks available.""" return not self.queue.empty() def count(self): - """Return the number of futures yet to be returned + """Return the number of tasks yet to be returned - This includes both the number of futures still computing, as well as + This includes both the number of tasks still computing, as well as those that are finished, but have not yet been returned from this iterator. """ with self.lock: - return len(self.futures) + len(self.queue.queue) + return len(self.tasks) + len(self.queue.queue) def __repr__(self): return "".format( - len(self.futures), len(self.queue.queue) + len(self.tasks), len(self.queue.queue) ) def __iter__(self): @@ -5655,12 +5653,12 @@ def __aiter__(self): def _get_and_raise(self): res = self.queue.get() if self.with_results: - future, result = res - if self.raise_errors and future.status == "error": + task, result = res + if self.raise_errors and task.status == "error": typ, exc, tb = result raise exc.with_traceback(tb) - elif future.status == "cancelled": - res = (res[0], CancelledError(future.key)) + elif task.status == "cancelled": + res = (res[0], CancelledError(task.key)) return res def __next__(self): @@ -5679,10 +5677,10 @@ async def __anext__(self): return await wait_for(self._anext(), self._deadline.remaining) async def _anext(self): - if not self.futures and self.queue.empty(): + if not self.tasks and self.queue.empty(): raise StopAsyncIteration while self.queue.empty(): - if not self.futures: + if not self.tasks: raise StopAsyncIteration async with self.condition: await self.condition.wait() @@ -5692,7 +5690,7 @@ async def _anext(self): next = __next__ def next_batch(self, block=True): - """Get the next batch of completed futures. + """Get the next batch of completed tasks. Parameters ---------- @@ -5702,7 +5700,7 @@ def next_batch(self, block=True): Examples -------- - >>> ac = as_completed(futures) # doctest: +SKIP + >>> ac = as_completed(tasks) # doctest: +SKIP >>> client.gather(ac.next_batch()) # doctest: +SKIP [4, 1, 3] @@ -5711,7 +5709,7 @@ def next_batch(self, block=True): Returns ------- - List of futures or (future, result) tuples + List of tasks or (task, result) tuples """ if block: batch = [next(self)] @@ -5723,16 +5721,16 @@ def next_batch(self, block=True): def batches(self): """ - Yield all finished futures at once rather than one-by-one + Yield all finished tasks at once rather than one-by-one - This returns an iterator of lists of futures or lists of - (future, result) tuples rather than individual futures or individual - (future, result) tuples. It will yield these as soon as possible + This returns an iterator of lists of tasks or lists of + (task, result) tuples rather than individual tasks or individual + (task, result) tuples. It will yield these as soon as possible without waiting. Examples -------- - >>> for batch in as_completed(futures).batches(): # doctest: +SKIP + >>> for batch in as_completed(tasks).batches(): # doctest: +SKIP ... results = client.gather(batch) ... print(results) [4, 2] @@ -5747,9 +5745,9 @@ def batches(self): return def clear(self): - """Clear out all submitted futures""" + """Clear out all submitted tasks""" with self.lock: - self.futures.clear() + self.tasks.clear() while not self.queue.empty(): self.queue.get() @@ -5827,8 +5825,8 @@ def redict_collection(c, dsk): return cc -def futures_of(o, client=None): - """Future objects in a collection +def tasks_of(o, client=None): + """Task objects in a collection Parameters ---------- @@ -5839,23 +5837,23 @@ def futures_of(o, client=None): Examples -------- - >>> futures_of(my_dask_dataframe) - [, - ] + >>> tasks_of(my_dask_dataframe) + [, + ] Raises ------ CancelledError - If one of the futures is cancelled a CancelledError is raised + If one of the tasks is cancelled a CancelledError is raised Returns ------- - futures : List[Future] - A list of futures held by those collections + tasks : List[Task] + A list of tasks held by those collections """ stack = [o] seen = set() - futures = list() + tasks = list() while stack: x = stack.pop() if type(x) in (tuple, set, list): @@ -5864,50 +5862,50 @@ def futures_of(o, client=None): stack.extend(x.values()) elif type(x) is SubgraphCallable: stack.extend(x.dsk.values()) - elif isinstance(x, Future): + elif isinstance(x, Task): if x not in seen: seen.add(x) - futures.append(x) + tasks.append(x) elif dask.is_dask_collection(x): stack.extend(x.__dask_graph__().values()) if client is not None: - bad = {f for f in futures if f.cancelled()} + bad = {f for f in tasks if f.cancelled()} if bad: raise CancelledError(bad) - return futures[::-1] + return tasks[::-1] def fire_and_forget(obj): - """Run tasks at least once, even if we release the futures + """Run tasks at least once, even if we release the tasks Under normal operation Dask will not run any tasks for which there is not - an active future (this avoids unnecessary work in many situations). - However sometimes you want to just fire off a task, not track its future, - and expect it to finish eventually. You can use this function on a future - or collection of futures to ask Dask to complete the task even if no active + an active task (this avoids unnecessary work in many situations). + However sometimes you want to just fire off a task, not track its task, + and expect it to finish eventually. You can use this function on a task + or collection of tasks to ask Dask to complete the task even if no active client is tracking it. The results will not be kept in memory after the task completes (unless - there is an active future) so this is only useful for tasks that depend on + there is an active task) so this is only useful for tasks that depend on side effects. Parameters ---------- - obj : Future, list, dict, dask collection - The futures that you want to run at least once + obj : Task, list, dict, dask collection + The tasks that you want to run at least once Examples -------- >>> fire_and_forget(client.submit(func, *args)) # doctest: +SKIP """ - futures = futures_of(obj) - for future in futures: - future.client._send_to_scheduler( + tasks = tasks_of(obj) + for task in tasks: + task.client._send_to_scheduler( { "op": "client-desires-keys", - "keys": [future.key], + "keys": [task.key], "client": "fire-and-forget", } ) diff --git a/distributed/comm/core.py b/distributed/comm/core.py index 5694a344bf..2a9fb03d04 100644 --- a/distributed/comm/core.py +++ b/distributed/comm/core.py @@ -247,9 +247,9 @@ async def __aenter__(self): return self async def __aexit__(self, exc_type, exc_value, traceback): - future = self.stop() - if inspect.isawaitable(future): - await future + task = self.stop() + if inspect.isawaitable(task): + await task def __await__(self): async def _(): diff --git a/distributed/comm/inproc.py b/distributed/comm/inproc.py index 13ee9f757a..7934c16b71 100644 --- a/distributed/comm/inproc.py +++ b/distributed/comm/inproc.py @@ -91,7 +91,7 @@ class QueueEmpty(Exception): def _set_result_unless_cancelled(fut, result): - """Helper setting the result only if the future was not cancelled.""" + """Helper setting the result only if the task was not cancelled.""" if fut.cancelled(): return fut.set_result(result) @@ -104,7 +104,7 @@ class Queue: def __init__(self): self._q = deque() - self._read_future = None + self._read_task = None def get_nowait(self): q = self._q @@ -113,21 +113,21 @@ def get_nowait(self): return q.popleft() def get(self): - assert not self._read_future, "Only one reader allowed" + assert not self._read_task, "Only one reader allowed" fut = TornadoFuture() q = self._q if q: fut.set_result(q.popleft()) else: - self._read_future = fut + self._read_task = fut return fut def put_nowait(self, value): q = self._q - fut = self._read_future + fut = self._read_task if fut is not None: assert len(q) == 0 - self._read_future = None + self._read_task = None _set_result_unless_cancelled(fut, value) else: q.append(value) @@ -302,7 +302,7 @@ def connect_threadsafe(self, conn_req): async def start(self): self.loop = IOLoop.current() - self._listen_future = asyncio.ensure_future(self._listen()) + self._listen_task = asyncio.ensure_future(self._listen()) self.manager.add_listener(self.address, self) def stop(self): diff --git a/distributed/comm/tests/test_comms.py b/distributed/comm/tests/test_comms.py index 55acfd5286..c936f187fd 100644 --- a/distributed/comm/tests/test_comms.py +++ b/distributed/comm/tests/test_comms.py @@ -272,8 +272,8 @@ async def client_communicate(key, delay=0): # Many clients at once N = 100 - futures = [client_communicate(key=i, delay=0.05) for i in range(N)] - await asyncio.gather(*futures) + tasks = [client_communicate(key=i, delay=0.05) for i in range(N)] + await asyncio.gather(*tasks) assert set(l) == {1234} | set(range(N)) @@ -325,8 +325,8 @@ async def client_communicate(key, delay=0): # Many clients at once N = 100 - futures = [client_communicate(key=i, delay=0.05) for i in range(N)] - await asyncio.gather(*futures) + tasks = [client_communicate(key=i, delay=0.05) for i in range(N)] + await asyncio.gather(*tasks) assert set(l) == {1234} | set(range(N)) @@ -352,20 +352,20 @@ async def sleep_for_60ms(): original_thread_count = threading.active_count() # tcp.TCPConnector() - sleep_future = sleep_for_60ms() + sleep_task = sleep_for_60ms() with pytest.raises(IOError): await connect("tcp://localhost:28400", 0.052) - max_thread_count = await sleep_future + max_thread_count = await sleep_task # 2 is the number set by BaseTCPConnector.executor (ThreadPoolExecutor) assert max_thread_count <= 2 + original_thread_count # tcp.TLSConnector() - sleep_future = sleep_for_60ms() + sleep_task = sleep_for_60ms() with pytest.raises(IOError): await connect( "tls://localhost:28400", 0.052, ssl_context=get_client_ssl_context() ) - max_thread_count = await sleep_future + max_thread_count = await sleep_task assert max_thread_count <= 2 + original_thread_count @@ -422,8 +422,8 @@ async def client_communicate(key, delay=0): # Many clients at once N = 20 - futures = [client_communicate(key=i, delay=0.001) for i in range(N)] - await asyncio.gather(*futures) + tasks = [client_communicate(key=i, delay=0.001) for i in range(N)] + await asyncio.gather(*tasks) assert set(l) == {1234} | set(range(N)) assert len(client_addresses) == N + 1 @@ -563,8 +563,8 @@ async def client_communicate(key, delay=0): await client_communicate(key=1234) # Many clients at once - futures = [client_communicate(key=i, delay=0.05) for i in range(20)] - await asyncio.gather(*futures) + tasks = [client_communicate(key=i, delay=0.05) for i in range(20)] + await asyncio.gather(*tasks) assert set(l) == {1234} | set(range(20)) listener.stop() diff --git a/distributed/comm/tests/test_ucx.py b/distributed/comm/tests/test_ucx.py index f2dd3ecb2e..16d808cfa4 100644 --- a/distributed/comm/tests/test_ucx.py +++ b/distributed/comm/tests/test_ucx.py @@ -136,8 +136,8 @@ async def client_communicate(key, delay=0): # Many clients at once N = 2 - futures = [client_communicate(key=i, delay=0.05) for i in range(N)] - await asyncio.gather(*futures) + tasks = [client_communicate(key=i, delay=0.05) for i in range(N)] + await asyncio.gather(*tasks) assert set(l) == {1234} | set(range(N)) listener.stop() diff --git a/distributed/comm/tests/test_ws.py b/distributed/comm/tests/test_ws.py index 2b2c06ead5..977f76a4ba 100644 --- a/distributed/comm/tests/test_ws.py +++ b/distributed/comm/tests/test_ws.py @@ -224,8 +224,8 @@ async def test_quiet_close(): async def test_ws_roundtrip(c, s, a, b): np = pytest.importorskip("numpy") x = np.arange(100) - future = await c.scatter(x) - y = await future + task = await c.scatter(x) + y = await task assert (x == y).all() @@ -241,6 +241,6 @@ async def test_wss_roundtrip(): async with Worker(s.address, security=security) as w: async with Client(s.address, security=security, asynchronous=True) as c: x = np.arange(100) - future = await c.scatter(x) - y = await future + task = await c.scatter(x) + y = await task assert (x == y).all() diff --git a/distributed/core.py b/distributed/core.py index 66470ef224..20c1ea3158 100644 --- a/distributed/core.py +++ b/distributed/core.py @@ -700,15 +700,15 @@ def _stop_listeners(self) -> asyncio.Future: listeners_to_stop: set[Awaitable] = set() for listener in self.listeners: - future = listener.stop() - if inspect.isawaitable(future): + task = listener.stop() + if inspect.isawaitable(task): warnings.warn( f"{type(listener)} is using an asynchronous `stop` method. " "Support for asynchronous `Listener.stop` has been deprecated and " "will be removed in a future version", DeprecationWarning, ) - listeners_to_stop.add(future) + listeners_to_stop.add(task) elif hasattr(listener, "abort_handshaking_comms"): listener.abort_handshaking_comms() @@ -1486,7 +1486,7 @@ def __init__( self._created: weakref.WeakSet[Comm] = weakref.WeakSet() self._instances.add(self) # _n_connecting and _connecting have subtle different semantics. The set - # _connecting contains futures actively trying to establish a connection + # _connecting contains tasks actively trying to establish a connection # while the _n_connecting also accounts for connection attempts which # are waiting due to the connection limit self._connecting: defaultdict[str, set[Callable[[str], None]]] = defaultdict( diff --git a/distributed/dashboard/components/shared.py b/distributed/dashboard/components/shared.py index 689912d4cd..343c5362db 100644 --- a/distributed/dashboard/components/shared.py +++ b/distributed/dashboard/components/shared.py @@ -310,7 +310,7 @@ async def cb(): metadata = await self.server.get_profile_metadata() else: metadata = None - if isinstance(prof, gen.Future): + if isinstance(prof, gen.Task): prof, metadata = await asyncio.gather(prof, metadata) self.doc().add_next_tick_callback(lambda: self.update(prof, metadata)) diff --git a/distributed/dashboard/tests/test_scheduler_bokeh.py b/distributed/dashboard/tests/test_scheduler_bokeh.py index ed61155d9d..684a561f61 100644 --- a/distributed/dashboard/tests/test_scheduler_bokeh.py +++ b/distributed/dashboard/tests/test_scheduler_bokeh.py @@ -82,7 +82,7 @@ async def test_simple(c, s, a, b): port = s.http_server.port ev = Event() - future = c.submit(block_on_event, ev) + task = c.submit(block_on_event, ev) await asyncio.sleep(0.1) http_client = AsyncHTTPClient() @@ -101,7 +101,7 @@ async def test_simple(c, s, a, b): assert response await ev.set() - await future + await task @gen_cluster(client=True, worker_kwargs={"dashboard": True}) @@ -141,11 +141,11 @@ async def test_counters(c, s, a, b): async def test_stealing_events(c, s, a, b): se = StealingEvents(s) - futures = c.map( + tasks = c.map( slowinc, range(10), delay=0.1, workers=a.address, allow_other_workers=True ) - await wait(futures) + await wait(tasks) se.update() assert len(first(se.source.data.values())) assert b.state.tasks @@ -156,7 +156,7 @@ async def test_stealing_events(c, s, a, b): async def test_events(c, s, a, b): e = Events(s, "all") - futures = c.map( + tasks = c.map( slowinc, range(100), delay=0.1, workers=a.address, allow_other_workers=True ) @@ -172,9 +172,9 @@ async def test_events(c, s, a, b): async def test_task_stream(c, s, a, b): ts = TaskStream(s) - futures = c.map(slowinc, range(10), delay=0.001) + tasks = c.map(slowinc, range(10), delay=0.001) - await wait(futures) + await wait(tasks) ts.update() d = dict(ts.source.data) @@ -186,7 +186,7 @@ async def test_task_stream(c, s, a, b): d = dict(ts.source.data) assert all(len(L) == 10 for L in d.values()) - total = c.submit(sum, futures) + total = c.submit(sum, tasks) await wait(total) ts.update() @@ -197,8 +197,8 @@ async def test_task_stream(c, s, a, b): @gen_cluster(client=True) async def test_task_stream_n_rectangles(c, s, a, b): ts = TaskStream(s, n_rectangles=10) - futures = c.map(slowinc, range(10), delay=0.001) - await wait(futures) + tasks = c.map(slowinc, range(10), delay=0.001) + await wait(tasks) ts.update() assert len(ts.source.data["start"]) == 10 @@ -208,8 +208,8 @@ async def test_task_stream_n_rectangles(c, s, a, b): async def test_task_stream_second_plugin(c, s, a, b): ts = TaskStream(s, n_rectangles=10, clear_interval=10) ts.update() - futures = c.map(inc, range(10)) - await wait(futures) + tasks = c.map(inc, range(10)) + await wait(tasks) ts.update() ts2 = TaskStream(s, n_rectangles=5, clear_interval=10) @@ -243,23 +243,23 @@ async def test_task_stream_clear_interval(c, s, a, b): async def test_TaskProgress(c, s, a, b): tp = TaskProgress(s) - futures = c.map(slowinc, range(10), delay=0.001) - await wait(futures) + tasks = c.map(slowinc, range(10), delay=0.001) + await wait(tasks) tp.update() d = dict(tp.source.data) assert all(len(L) == 1 for L in d.values()) assert d["name"] == ["slowinc"] - futures2 = c.map(dec, range(5)) - await wait(futures2) + tasks2 = c.map(dec, range(5)) + await wait(tasks2) tp.update() d = dict(tp.source.data) assert all(len(L) == 2 for L in d.values()) assert d["name"] == ["slowinc", "dec"] - del futures, futures2 + del tasks, tasks2 while s.tasks: await asyncio.sleep(0.01) @@ -273,11 +273,11 @@ async def test_TaskProgress_empty(c, s, a, b): tp = TaskProgress(s) tp.update() - futures = [c.submit(inc, i, key="f-" + "a" * i) for i in range(20)] - await wait(futures) + tasks = [c.submit(inc, i, key="f-" + "a" * i) for i in range(20)] + await wait(tasks) tp.update() - del futures + del tasks while s.tasks: await asyncio.sleep(0.01) tp.update() @@ -289,8 +289,8 @@ async def test_TaskProgress_empty(c, s, a, b): async def test_CurrentLoad(c, s, a, b): cl = CurrentLoad(s) - futures = c.map(slowinc, range(10), delay=0.001) - await wait(futures) + tasks = c.map(slowinc, range(10), delay=0.001) + await wait(tasks) cl.update() d = dict(cl.source.data) @@ -306,7 +306,7 @@ async def test_ProcessingHistogram(c, s, a, b): assert (ph.source.data["top"] != 0).sum() == 1 assert ph.source.data["right"][-1] < 2 - futures = c.map(slowinc, range(10), delay=0.050) + tasks = c.map(slowinc, range(10), delay=0.050) while not s.tasks: await asyncio.sleep(0.01) @@ -318,8 +318,8 @@ async def test_ProcessingHistogram(c, s, a, b): async def test_WorkersMemory(c, s, a, b): cl = WorkersMemory(s) - futures = c.map(slowinc, range(10), delay=0.001) - await wait(futures) + tasks = c.map(slowinc, range(10), delay=0.001) + await wait(tasks) cl.update() d = dict(cl.source.data) @@ -475,8 +475,8 @@ async def test_FinePerformanceMetrics_shuffle(c, s, a, b): async def test_ClusterMemory(c, s, a, b): cl = ClusterMemory(s) - futures = c.map(slowinc, range(10), delay=0.001) - await wait(futures) + tasks = c.map(slowinc, range(10), delay=0.001) + await wait(tasks) cl.update() d = dict(cl.source.data) @@ -537,8 +537,8 @@ async def test_WorkersMemoryHistogram(c, s, a, b): nh.update() assert any(nh.source.data["top"] != 0) - futures = c.map(inc, range(10)) - await wait(futures) + tasks = c.map(inc, range(10)) + await wait(tasks) nh.update() assert nh.source.data["right"][-1] > 5 * 20 @@ -820,8 +820,8 @@ async def test_SystemTimeseries(c, s, a, b): @gen_cluster(client=True) async def test_TaskGraph(c, s, a, b): gp = TaskGraph(s) - futures = c.map(inc, range(5)) - total = c.submit(sum, futures) + tasks = c.map(inc, range(5)) + total = c.submit(sum, tasks) await total gp.update() @@ -843,11 +843,11 @@ async def test_TaskGraph(c, s, a, b): gp.update() - future = c.submit(inc, 10) - future2 = c.submit(inc, future) - await wait(future2) - key = future.key - del future, future2 + task = c.submit(inc, 10) + task2 = c.submit(inc, task) + await wait(task2) + key = task.key + del task, task2 while key in s.tasks: await asyncio.sleep(0.01) @@ -862,13 +862,13 @@ async def test_TaskGraph(c, s, a, b): @gen_cluster(client=True) async def test_TaskGraph_clear(c, s, a, b): gp = TaskGraph(s) - futures = c.map(inc, range(5)) - total = c.submit(sum, futures) + tasks = c.map(inc, range(5)) + total = c.submit(sum, tasks) await total gp.update() - del total, futures + del total, tasks while s.tasks: await asyncio.sleep(0.01) @@ -951,8 +951,8 @@ async def test_TaskGraph_order(c, s, a, b): @gen_cluster(client=True) async def test_TaskGroupGraph(c, s, a, b): tgg = TaskGroupGraph(s) - futures = c.map(inc, range(10)) - await wait(futures) + tasks = c.map(inc, range(10)) + await wait(tasks) tgg.update() assert all(len(L) == 1 for L in tgg.nodes_source.data.values()) @@ -961,15 +961,15 @@ async def test_TaskGroupGraph(c, s, a, b): assert all(len(L) == 0 for L in tgg.arrows_source.data.values()) - futures2 = c.map(dec, range(5)) - await wait(futures2) + tasks2 = c.map(dec, range(5)) + await wait(tasks2) tgg.update() assert all(len(L) == 2 for L in tgg.nodes_source.data.values()) assert tgg.nodes_source.data["name"] == ["inc", "dec"] assert tgg.nodes_source.data["tot_tasks"] == [10, 5] - del futures, futures2 + del tasks, tasks2 while s.task_groups: await asyncio.sleep(0.01) @@ -981,8 +981,8 @@ async def test_TaskGroupGraph(c, s, a, b): async def test_TaskGroupGraph_arrows(c, s, a, b): tgg = TaskGroupGraph(s) - futures = c.map(inc, range(10)) - await wait(futures) + tasks = c.map(inc, range(10)) + await wait(tasks) tgg.update() assert all(len(L) == 1 for L in tgg.nodes_source.data.values()) @@ -991,8 +991,8 @@ async def test_TaskGroupGraph_arrows(c, s, a, b): assert all(len(L) == 0 for L in tgg.arrows_source.data.values()) - futures2 = c.map(dec, futures) - await wait(futures2) + tasks2 = c.map(dec, tasks) + await wait(tasks2) tgg.update() assert all(len(L) == 2 for L in tgg.nodes_source.data.values()) @@ -1001,11 +1001,11 @@ async def test_TaskGroupGraph_arrows(c, s, a, b): assert all(len(L) == 1 for L in tgg.arrows_source.data.values()) - del futures, futures2 + del tasks, tasks2 while s.task_groups: await asyncio.sleep(0.01) - tgg.update() # for some reason after deleting the futures the tgg.node_source.data.values are not clear. + tgg.update() # for some reason after deleting the tasks the tgg.node_source.data.values are not clear. assert not any(tgg.nodes_source.data.values()) assert not any(tgg.arrows_source.data.values()) @@ -1110,8 +1110,8 @@ async def test_lots_of_tasks(c, s, a, b): ts = TaskStream(s) ts.update() - futures = c.map(toolz.identity, range(100)) - await wait(futures) + tasks = c.map(toolz.identity, range(100)) + await wait(tasks) tsp = s.plugins[TaskStreamPlugin.name] assert len(tsp.buffer) == 10 @@ -1119,8 +1119,8 @@ async def test_lots_of_tasks(c, s, a, b): assert len(ts.source.data["start"]) == 10 assert "identity" in str(ts.source.data) - futures = c.map(lambda x: x, range(100), pure=False) - await wait(futures) + tasks = c.map(lambda x: x, range(100), pure=False) + await wait(tasks) ts.update() assert "lambda" in str(ts.source.data) diff --git a/distributed/dashboard/tests/test_worker_bokeh.py b/distributed/dashboard/tests/test_worker_bokeh.py index 6b86cbf6f3..7462e7f1c7 100644 --- a/distributed/dashboard/tests/test_worker_bokeh.py +++ b/distributed/dashboard/tests/test_worker_bokeh.py @@ -30,7 +30,7 @@ async def test_routes(c, s, a, b): port = a.http_server.port - future = c.submit(sleep, 1) + task = c.submit(sleep, 1) await asyncio.sleep(0.1) http_client = AsyncHTTPClient() @@ -52,7 +52,7 @@ async def test_simple(c, s, a, b): assert s.workers[a.address].services == {"dashboard": a.http_server.port} assert s.workers[b.address].services == {"dashboard": b.http_server.port} - future = c.submit(sleep, 1) + task = c.submit(sleep, 1) await asyncio.sleep(0.1) http_client = AsyncHTTPClient() diff --git a/distributed/deploy/cluster.py b/distributed/deploy/cluster.py index 233fa0d969..cd428392c0 100644 --- a/distributed/deploy/cluster.py +++ b/distributed/deploy/cluster.py @@ -290,7 +290,7 @@ def scale(self, n: int) -> None: def _log(self, log): """Log a message. - Output a message to the user and also store for future retrieval. + Output a message to the user and also store for task retrieval. For use in subclasses where initialisation may take a while and it would be beneficial to feed back to the user. diff --git a/distributed/deploy/spec.py b/distributed/deploy/spec.py index 8189cfdd7d..3758bb7a6e 100644 --- a/distributed/deploy/spec.py +++ b/distributed/deploy/spec.py @@ -261,7 +261,7 @@ def __init__( self.workers = {} self._i = 0 self.security = security or Security() - self._futures = set() + self._tasks = set() if silence_logs: stack.enter_context(silence_logging_cmgr(level=silence_logs)) @@ -402,7 +402,7 @@ def f(): for d in self.scheduler_info["workers"].values() ) ): - self._futures.add(asyncio.ensure_future(self.workers[name].close())) + self._tasks.add(asyncio.ensure_future(self.workers[name].close())) del self.workers[name] delay = parse_timedelta( @@ -446,7 +446,7 @@ async def _close(self): if isawaitable(f): await f await self._correct_state() - await asyncio.gather(*self._futures) + await asyncio.gather(*self._tasks) if self.scheduler_comm: async with self._lock: diff --git a/distributed/deploy/tests/test_adaptive.py b/distributed/deploy/tests/test_adaptive.py index 6bbf146085..23e83d32b8 100644 --- a/distributed/deploy/tests/test_adaptive.py +++ b/distributed/deploy/tests/test_adaptive.py @@ -31,14 +31,14 @@ def test_adaptive_local_cluster(loop): alc = cluster.adapt(interval="100 ms") with Client(cluster, loop=loop) as c: assert not cluster.scheduler.workers - future = c.submit(lambda x: x + 1, 1) - assert future.result() == 2 + task = c.submit(lambda x: x + 1, 1) + assert task.result() == 2 assert cluster.scheduler.workers sleep(0.1) assert cluster.scheduler.workers - del future + del task start = time() while cluster.scheduler.workers: @@ -60,13 +60,13 @@ async def test_adaptive_local_cluster_multi_workers(): cluster.scheduler.allowed_failures = 1000 adapt = cluster.adapt(interval="100 ms") async with Client(cluster, asynchronous=True) as c: - futures = c.map(slowinc, range(100), delay=0.01) + tasks = c.map(slowinc, range(100), delay=0.01) while not cluster.scheduler.workers: await asyncio.sleep(0.01) - await c.gather(futures) - del futures + await c.gather(tasks) + del tasks while cluster.scheduler.workers: await asyncio.sleep(0.01) @@ -76,8 +76,8 @@ async def test_adaptive_local_cluster_multi_workers(): assert not cluster.scheduler.workers await asyncio.sleep(0.05) - futures = c.map(slowinc, range(100), delay=0.01) - await c.gather(futures) + tasks = c.map(slowinc, range(100), delay=0.01) + await c.gather(tasks) @pytest.mark.xfail(reason="changed API") @@ -134,7 +134,7 @@ async def test_min_max(): assert len(cluster.scheduler.workers) == 1 assert len(adapt.log) == 1 and adapt.log[-1][1] == {"status": "up", "n": 1} - futures = c.map(slowinc, range(100), delay=0.1) + tasks = c.map(slowinc, range(100), delay=0.1) start = time() while len(cluster.scheduler.workers) < 2: @@ -149,7 +149,7 @@ async def test_min_max(): d["status"] == "up" for _, d in adapt.log ) - del futures + del tasks start = time() while len(cluster.scheduler.workers) != 1: @@ -198,12 +198,12 @@ async def test_adapt_quickly(): threads_per_worker=1, ) as cluster, Client(cluster, asynchronous=True) as client: adapt = cluster.adapt(interval="20 ms", wait_count=5, maximum=10) - future = client.submit(slowinc, 1, delay=0.100) - await wait(future) + task = client.submit(slowinc, 1, delay=0.100) + await wait(task) assert len(adapt.log) == 1 # Scale up when there is plenty of available work - futures = client.map(slowinc, range(1000), delay=0.100) + tasks = client.map(slowinc, range(1000), delay=0.100) while len(adapt.log) == 1: await asyncio.sleep(0.01) assert len(adapt.log) == 2 @@ -214,7 +214,7 @@ async def test_adapt_quickly(): while len(cluster.workers) < adapt.maximum: await asyncio.sleep(0.01) - del futures + del tasks while len(cluster.scheduler.tasks) > 1: await asyncio.sleep(0.01) @@ -249,7 +249,7 @@ async def test_adapt_down(): ) as cluster, Client(cluster, asynchronous=True) as client: cluster.adapt(interval="20ms", maximum=5) - futures = client.map(slowinc, range(1000), delay=0.1) + tasks = client.map(slowinc, range(1000), delay=0.1) while len(cluster.scheduler.workers) < 5: await asyncio.sleep(0.1) @@ -287,8 +287,8 @@ def test_basic_no_loop(cleanup): ) as cluster: with Client(cluster) as client: cluster.adapt() - future = client.submit(lambda x: x + 1, 1) - assert future.result() == 2 + task = client.submit(lambda x: x + 1, 1) + assert task.result() == 2 loop = cluster.loop finally: assert loop is None or not loop.asyncio_loop.is_running() @@ -323,8 +323,8 @@ async def _test(): async with Client(cluster, asynchronous=True) as client: await client.wait_for_workers(2) - futures = client.map(slowinc, range(n_tasks), delay=0.3) - await wait(futures) + tasks = client.map(slowinc, range(n_tasks), delay=0.3) + await wait(tasks) scaleup_recs = [ msg[1]["n"] for msg in adapt.log if msg[1].get("status") == "up" ] @@ -469,13 +469,13 @@ async def _(): n_workers=0, asynchronous=True, dashboard_address=":0" ) as cluster: async with Client(cluster, asynchronous=True) as client: - futures = client.map(slowinc, range(5), delay=0.05) + tasks = client.map(slowinc, range(5), delay=0.05) assert len(cluster.workers) == 0 cluster.adapt() - await client.gather(futures) + await client.gather(tasks) - del futures + del tasks await async_poll_for(lambda: not cluster.workers, 10) @@ -508,13 +508,13 @@ async def test_adaptive_stopped(): ) async def test_scale_up_large_tasks(c, s, saturation): s.WORKER_SATURATION = saturation - futures = c.map(slowinc, range(10)) + tasks = c.map(slowinc, range(10)) while not s.tasks: await asyncio.sleep(0.001) assert s.adaptive_target() == 10 - more_futures = c.map(slowinc, range(200)) + more_tasks = c.map(slowinc, range(200)) while len(s.tasks) != 200: await asyncio.sleep(0.001) @@ -527,13 +527,13 @@ async def test_scale_up_large_tasks(c, s, saturation): config={"distributed.scheduler.default-task-durations": {"slowinc": 1000}}, ) async def test_respect_average_nthreads(c, s, w): - futures = c.map(slowinc, range(10)) + tasks = c.map(slowinc, range(10)) while not s.tasks: await asyncio.sleep(0.001) assert s.adaptive_target() == 2 - more_futures = c.map(slowinc, range(200)) + more_tasks = c.map(slowinc, range(200)) while len(s.tasks) != 200: await asyncio.sleep(0.001) diff --git a/distributed/deploy/tests/test_local.py b/distributed/deploy/tests/test_local.py index 1594bc4db5..e549d579d3 100644 --- a/distributed/deploy/tests/test_local.py +++ b/distributed/deploy/tests/test_local.py @@ -205,8 +205,8 @@ def test_submit(loop): processes=False, loop=loop, ) as cluster, Client(cluster.scheduler_address, loop=loop) as client: - future = client.submit(lambda x: x + 1, 1) - assert future.result() == 2 + task = client.submit(lambda x: x + 1, 1) + assert task.result() == 2 def test_context_manager(loop): @@ -770,7 +770,7 @@ def wait_workers(n): await wait_workers(0) async with Client(cluster, asynchronous=True) as client: - futures = client.map(slowinc, range(1000), delay=0.1) + tasks = client.map(slowinc, range(1000), delay=0.1) await wait_workers(4) cluster._adaptive.stop() diff --git a/distributed/deploy/tests/test_slow_adaptive.py b/distributed/deploy/tests/test_slow_adaptive.py index 2215491a9b..eb999eca52 100644 --- a/distributed/deploy/tests/test_slow_adaptive.py +++ b/distributed/deploy/tests/test_slow_adaptive.py @@ -91,12 +91,12 @@ async def test_adaptive(): ) as cluster: cluster.adapt(minimum=1, maximum=4, target_duration="1s", interval="20ms") async with Client(cluster, asynchronous=True) as client: - futures = client.map(slowinc, range(200), delay=0.1) + tasks = client.map(slowinc, range(200), delay=0.1) while len(cluster.worker_spec) <= 1: await asyncio.sleep(0.05) - del futures + del tasks while len(cluster.worker_spec) > 1: await asyncio.sleep(0.05) diff --git a/distributed/deploy/tests/test_spec_cluster.py b/distributed/deploy/tests/test_spec_cluster.py index 0a30f01508..53e4a8c477 100644 --- a/distributed/deploy/tests/test_spec_cluster.py +++ b/distributed/deploy/tests/test_spec_cluster.py @@ -150,14 +150,14 @@ async def test_adaptive_killed_worker(): while not cluster.workers: await asyncio.sleep(0.01) - future = client.submit(sleep, 0.1) + task = client.submit(sleep, 0.1) # Kill the only worker. [worker_id] = cluster.workers await cluster.workers[worker_id].kill() # Wait for the worker to re-spawn and finish sleeping. - await future + await task @gen_test() @@ -469,8 +469,8 @@ async def test_MultiWorker(): await cluster assert not s.workers - future = client.submit(lambda x: x + 1, 10) - await future + task = client.submit(lambda x: x + 1, 10) + await task assert len(cluster.workers) == 1 diff --git a/distributed/diagnostics/plugin.py b/distributed/diagnostics/plugin.py index 3a866facc3..9731b57ebd 100644 --- a/distributed/diagnostics/plugin.py +++ b/distributed/diagnostics/plugin.py @@ -127,7 +127,7 @@ def update_graph( A mapping that maps a key to its dependencies. **kwargs: It is recommended to allow plugins to accept more parameters to - ensure future compatibility. + ensure task compatibility. """ def restart(self, scheduler: Scheduler) -> None: diff --git a/distributed/diagnostics/progress.py b/distributed/diagnostics/progress.py index 3fd7bbbd07..d6e69b8a9b 100644 --- a/distributed/diagnostics/progress.py +++ b/distributed/diagnostics/progress.py @@ -46,9 +46,9 @@ def dependent_keys(tasks, complete=False): class Progress(SchedulerPlugin): - """Tracks progress of a set of keys or futures + """Tracks progress of a set of keys or tasks - On creation we provide a set of keys or futures that interest us as well as + On creation we provide a set of keys or tasks that interest us as well as a scheduler. We traverse through the scheduler's dependencies to find all relevant keys on which our keys depend. We then plug into the scheduler to learn when our keys become available in memory at which point we record diff --git a/distributed/diagnostics/progressbar.py b/distributed/diagnostics/progressbar.py index e375a08aab..a52dd03514 100644 --- a/distributed/diagnostics/progressbar.py +++ b/distributed/diagnostics/progressbar.py @@ -15,7 +15,7 @@ import dask from dask.utils import key_split -from distributed.client import default_client, futures_of +from distributed.client import default_client, tasks_of from distributed.core import ( CommClosedError, clean_exception, @@ -449,9 +449,9 @@ def _draw_bar(self, remaining, all, status, **kwargs): def progress( - *futures, notebook=None, multi=True, complete=True, group_by="prefix", **kwargs + *tasks, notebook=None, multi=True, complete=True, group_by="prefix", **kwargs ): - """Track progress of futures + """Track progress of tasks This operates differently in the notebook and the console @@ -460,8 +460,8 @@ def progress( Parameters ---------- - futures : Futures - A list of futures or keys to track + tasks : Futures + A list of tasks or keys to track notebook : bool (optional) Running in the notebook or not (defaults to guess) multi : bool (optional) @@ -481,12 +481,12 @@ def progress( Examples -------- - >>> progress(futures) # doctest: +SKIP + >>> progress(tasks) # doctest: +SKIP [########################################] | 100% Completed | 1.7s """ - futures = futures_of(futures) - if not isinstance(futures, (set, list)): - futures = [futures] + tasks = tasks_of(tasks) + if not isinstance(tasks, (set, list)): + tasks = [tasks] if notebook is None: notebook = is_kernel() # often but not always correct assumption if kwargs.get("func", None) is not None: @@ -499,10 +499,10 @@ def progress( if notebook: if multi: bar = MultiProgressWidget( - futures, complete=complete, group_by=group_by, **kwargs + tasks, complete=complete, group_by=group_by, **kwargs ) else: - bar = ProgressWidget(futures, complete=complete, **kwargs) + bar = ProgressWidget(tasks, complete=complete, **kwargs) return bar else: - TextProgressBar(futures, complete=complete, **kwargs) + TextProgressBar(tasks, complete=complete, **kwargs) diff --git a/distributed/diagnostics/tests/test_eventstream.py b/distributed/diagnostics/tests/test_eventstream.py index 566fbdea77..0f4e4d1945 100644 --- a/distributed/diagnostics/tests/test_eventstream.py +++ b/distributed/diagnostics/tests/test_eventstream.py @@ -19,10 +19,10 @@ async def test_eventstream(c, s, *workers): s.add_plugin(es) assert es.buffer == [] - futures = c.map(div, [1] * 10, range(10)) - total = c.submit(sum, futures[1:]) + tasks = c.map(div, [1] * 10, range(10)) + total = c.submit(sum, tasks[1:]) await wait(total) - await wait(futures) + await wait(tasks) assert len(es.buffer) == 11 @@ -53,7 +53,7 @@ async def test_eventstream_remote(c, s, a, b): await asyncio.sleep(0.01) assert time() < start + 5 - futures = c.map(div, [1] * 10, range(10)) + tasks = c.map(div, [1] * 10, range(10)) start = time() total = [] diff --git a/distributed/diagnostics/tests/test_graph_layout.py b/distributed/diagnostics/tests/test_graph_layout.py index d8d3c725b6..d88ada4f3d 100644 --- a/distributed/diagnostics/tests/test_graph_layout.py +++ b/distributed/diagnostics/tests/test_graph_layout.py @@ -12,21 +12,21 @@ async def test_basic(c, s, a, b): gl = GraphLayout(s) s.add_plugin(gl) - futures = c.map(inc, range(5)) - total = c.submit(sum, futures) + tasks = c.map(inc, range(5)) + total = c.submit(sum, tasks) await total assert len(gl.x) == len(gl.y) == 6 - assert all(gl.x[f.key] == 0 for f in futures) + assert all(gl.x[f.key] == 0 for f in tasks) assert gl.x[total.key] == 1 assert min(gl.y.values()) < gl.y[total.key] < max(gl.y.values()) @gen_cluster(client=True) async def test_construct_after_call(c, s, a, b): - futures = c.map(inc, range(5)) - total = c.submit(sum, futures) + tasks = c.map(inc, range(5)) + total = c.submit(sum, tasks) await total @@ -34,7 +34,7 @@ async def test_construct_after_call(c, s, a, b): s.add_plugin(gl) assert len(gl.x) == len(gl.y) == 6 - assert all(gl.x[f.key] == 0 for f in futures) + assert all(gl.x[f.key] == 0 for f in tasks) assert gl.x[total.key] == 1 assert min(gl.y.values()) < gl.y[total.key] < max(gl.y.values()) @@ -56,8 +56,8 @@ async def test_states(c, s, a, b): async def test_release_tasks(c, s, a, b): gl = GraphLayout(s) s.add_plugin(gl) - futures = c.map(inc, range(5)) - total = c.submit(sum, futures) + tasks = c.map(inc, range(5)) + total = c.submit(sum, tasks) await total key = total.key @@ -74,10 +74,10 @@ async def test_forget(c, s, a, b): gl = GraphLayout(s) s.add_plugin(gl) - futures = c.map(inc, range(10)) - futures = c.map(inc, futures) - await wait(futures) - del futures + tasks = c.map(inc, range(10)) + tasks = c.map(inc, tasks) + await wait(tasks) + del tasks while s.tasks: await asyncio.sleep(0.01) @@ -107,6 +107,6 @@ async def test_layout_scatter(c, s, a, b): s.add_plugin(gl) data = await c.scatter([1, 2, 3], broadcast=True) - futures = [c.submit(sum, data) for _ in range(5)] - await wait(futures) + tasks = [c.submit(sum, data) for _ in range(5)] + await wait(tasks) assert len(gl.state_updates) > 0 diff --git a/distributed/diagnostics/tests/test_progress.py b/distributed/diagnostics/tests/test_progress.py index 7bad80da7e..29f85e5008 100644 --- a/distributed/diagnostics/tests/test_progress.py +++ b/distributed/diagnostics/tests/test_progress.py @@ -203,12 +203,12 @@ def f(x): return x for i in range(4): - future = c.submit(f, i) + task = c.submit(f, i) await asyncio.sleep(1) - await wait([future]) - assert p.state["memory"] == {"f": {future.key}} + await wait([task]) + assert p.state["memory"] == {"f": {task.key}} await c._restart() @@ -225,8 +225,8 @@ def f(x): @gen_cluster(client=True, Worker=Nanny) async def test_AllProgress_lost_key(c, s, a, b): p = AllProgress(s) - futures = c.map(inc, range(5)) - await wait(futures) + tasks = c.map(inc, range(5)) + await wait(tasks) assert len(p.state["memory"]["inc"]) == 5 await a.close() @@ -244,9 +244,9 @@ async def test_group_timing(c, s, a, b): assert len(p.time) == 2 assert len(p.nthreads) == 2 - futures1 = c.map(slowinc, range(10), delay=0.3) - futures2 = c.map(slowdec, range(10), delay=0.3) - await wait(futures1 + futures2) + tasks1 = c.map(slowinc, range(10), delay=0.3) + tasks2 = c.map(slowdec, range(10), delay=0.3) + await wait(tasks1 + tasks2) assert len(p.time) > 2 assert len(p.nthreads) == len(p.time) diff --git a/distributed/diagnostics/tests/test_progress_stream.py b/distributed/diagnostics/tests/test_progress_stream.py index 8714d3c874..80867bbc1d 100644 --- a/distributed/diagnostics/tests/test_progress_stream.py +++ b/distributed/diagnostics/tests/test_progress_stream.py @@ -67,14 +67,14 @@ def test_progress_quads_too_many(): @gen_cluster(client=True) async def test_progress_stream(c, s, a, b): - futures = c.map(div, [1] * 10, range(10)) + tasks = c.map(div, [1] * 10, range(10)) x = 1 for _ in range(5): x = delayed(inc)(x) - future = c.compute(x) + task = c.compute(x) - await wait(futures + [future]) + await wait(tasks + [task]) comm = await progress_stream(s.address, interval=0.010) msg = await comm.read() diff --git a/distributed/diagnostics/tests/test_progressbar.py b/distributed/diagnostics/tests/test_progressbar.py index 7874457a2c..6f081d1ba5 100644 --- a/distributed/diagnostics/tests/test_progressbar.py +++ b/distributed/diagnostics/tests/test_progressbar.py @@ -11,9 +11,9 @@ def test_text_progressbar(capsys, client): - futures = client.map(inc, range(10)) - p = TextProgressBar(futures, interval=0.01, complete=True, loop=client.loop) - client.gather(futures) + tasks = client.map(inc, range(10)) + p = TextProgressBar(tasks, interval=0.01, complete=True, loop=client.loop) + client.gather(tasks) start = time() while p.status != "finished": diff --git a/distributed/diagnostics/tests/test_task_stream.py b/distributed/diagnostics/tests/test_task_stream.py index 0e0206952e..b05a20f191 100644 --- a/distributed/diagnostics/tests/test_task_stream.py +++ b/distributed/diagnostics/tests/test_task_stream.py @@ -19,8 +19,8 @@ async def test_TaskStreamPlugin(c, s, *workers): s.add_plugin(es) assert not es.buffer - futures = c.map(div, [1] * 10, range(10)) - total = c.submit(sum, futures[1:]) + tasks = c.map(div, [1] * 10, range(10)) + total = c.submit(sum, tasks[1:]) await wait(total) assert len(es.buffer) == 11 @@ -50,8 +50,8 @@ async def test_TaskStreamPlugin(c, s, *workers): async def test_maxlen(c, s, a, b): tasks = TaskStreamPlugin(s, maxlen=5) s.add_plugin(tasks) - futures = c.map(inc, range(10)) - await wait(futures) + tasks = c.map(inc, range(10)) + await wait(tasks) assert len(tasks.buffer) == 5 @@ -60,22 +60,22 @@ async def test_collect(c, s, a, b): tasks = TaskStreamPlugin(s) s.add_plugin(tasks) start = time() - futures = c.map(slowinc, range(10), delay=0.1) - await wait(futures) + tasks = c.map(slowinc, range(10), delay=0.1) + await wait(tasks) L = tasks.collect() - assert len(L) == len(futures) + assert len(L) == len(tasks) L = tasks.collect(start=start) - assert len(L) == len(futures) + assert len(L) == len(tasks) L = tasks.collect(start=start + 0.2) - assert 4 <= len(L) <= len(futures) + assert 4 <= len(L) <= len(tasks) L = tasks.collect(start="20 s") - assert len(L) == len(futures) + assert len(L) == len(tasks) L = tasks.collect(start="500ms") - assert 0 < len(L) <= len(futures) + assert 0 < len(L) <= len(tasks) L = tasks.collect(count=3) assert len(L) == 3 @@ -90,20 +90,20 @@ async def test_no_startstops(c, s, a, b): tasks = TaskStreamPlugin(s) s.add_plugin(tasks) # just to create the key on the scheduler - future = c.submit(inc, 1) - await wait(future) + task = c.submit(inc, 1) + await wait(task) assert len(tasks.buffer) == 1 - tasks.transition(future.key, "processing", "erred", stimulus_id="s1") + tasks.transition(task.key, "processing", "erred", stimulus_id="s1") # Transition was not recorded because it didn't contain `startstops` assert len(tasks.buffer) == 1 - tasks.transition(future.key, "processing", "erred", stimulus_id="s2", startstops=[]) + tasks.transition(task.key, "processing", "erred", stimulus_id="s2", startstops=[]) # Transition was not recorded because `startstops` was empty assert len(tasks.buffer) == 1 tasks.transition( - future.key, + task.key, "processing", "erred", stimulus_id="s3", @@ -117,8 +117,8 @@ async def test_client(c, s, a, b): L = await c.get_task_stream() assert L == () - futures = c.map(slowinc, range(10), delay=0.1) - await wait(futures) + tasks = c.map(slowinc, range(10), delay=0.1) + await wait(tasks) tasks = s.plugins[TaskStreamPlugin.name] L = await c.get_task_stream() @@ -129,8 +129,8 @@ def test_client_sync(client): with get_task_stream(client=client) as ts: sleep(0.1) # to smooth over time differences on the scheduler # to smooth over time differences on the scheduler - futures = client.map(inc, range(10)) - wait(futures) + tasks = client.map(inc, range(10)) + wait(tasks) assert len(ts.data) == 10 @@ -140,8 +140,8 @@ async def test_get_task_stream_plot(c, s, a, b): bkm = pytest.importorskip("bokeh.models") await c.get_task_stream() - futures = c.map(slowinc, range(10), delay=0.1) - await wait(futures) + tasks = c.map(slowinc, range(10), delay=0.1) + await wait(tasks) data, figure = await c.get_task_stream(plot=True) assert isinstance(figure, bkm.Plot) diff --git a/distributed/distributed-schema.yaml b/distributed/distributed-schema.yaml index 71d3cd1857..287df2f323 100644 --- a/distributed/distributed-schema.yaml +++ b/distributed/distributed-schema.yaml @@ -231,7 +231,7 @@ properties: description: | Settings for Dask's distributed Lock object - See https://docs.dask.org/en/latest/futures.html#locks for more information + See https://docs.dask.org/en/latest/tasks.html#locks for more information properties: lease-validation-interval: type: string diff --git a/distributed/event.py b/distributed/event.py index 145abbf385..3250d22ce6 100644 --- a/distributed/event.py +++ b/distributed/event.py @@ -37,7 +37,7 @@ class EventExtension: 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. + it, as those waiters would then have dangling tasks. 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. """ @@ -68,13 +68,13 @@ async def event_wait(self, name=None, timeout=None): name = self._normalize_name(name) event = self._events[name] - future = event.wait() + task = event.wait() if timeout is not None: - future = wait_for(future, timeout) + task = wait_for(task, timeout) self._waiter_count[name] += 1 try: - await future + await task except TimeoutError: return False finally: diff --git a/distributed/http/scheduler/tests/test_scheduler_http.py b/distributed/http/scheduler/tests/test_scheduler_http.py index d51ddf33ea..d246815652 100644 --- a/distributed/http/scheduler/tests/test_scheduler_http.py +++ b/distributed/http/scheduler/tests/test_scheduler_http.py @@ -36,14 +36,14 @@ async def test_connect(c, s, a, b): lock = Lock() async with lock: - future = c.submit(lambda x: x + 1, 1) + task = c.submit(lambda x: x + 1, 1) x = c.submit(lock_inc, 1, lock=lock, retries=5) - await future + await task http_client = AsyncHTTPClient() for suffix in [ "info/main/workers.html", "info/worker/" + url_escape(a.address) + ".html", - "info/task/" + url_escape(future.key) + ".html", + "info/task/" + url_escape(task.key) + ".html", "info/main/logs.html", "info/logs/" + url_escape(a.address) + ".html", "info/call-stack/" + url_escape(x.key) + ".html", @@ -194,8 +194,8 @@ async def fetch_state_metrics(): assert sum(forgotten_tasks) == 0.0 # submit a task which should show up in the prometheus scraping - future = c.submit(slowinc, 1, delay=0.5) - while not any(future.key in w.state.tasks for w in [a, b]): + task = c.submit(slowinc, 1, delay=0.5) + while not any(task.key in w.state.tasks for w in [a, b]): await asyncio.sleep(0.001) active_metrics, forgotten_tasks = await fetch_state_metrics() @@ -203,12 +203,12 @@ async def fetch_state_metrics(): assert sum(active_metrics.values()) == 1.0 assert sum(forgotten_tasks) == 0.0 - res = await c.gather(future) + res = await c.gather(task) assert res == 2 - future.release() + task.release() - while any(future.key in w.state.tasks for w in [a, b]): + while any(task.key in w.state.tasks for w in [a, b]): await asyncio.sleep(0.001) active_metrics, forgotten_tasks = await fetch_state_metrics() @@ -240,8 +240,8 @@ async def fetch_metrics(): return prefix_state_counts # do some compute and check the counts for each prefix and state - futures = c.map(inc, range(10)) - await c.gather(futures) + tasks = c.map(inc, range(10)) + await c.gather(tasks) prefix_state_counts = await fetch_metrics() assert prefix_state_counts.get(("inc", "memory")) == 10 @@ -379,8 +379,8 @@ async def test_task_page(c, s, a, b, key): response = await http_client.fetch(url, raise_error=False) assert response.code == 404 - future = c.submit(lambda: 1, key=key, workers=a.address) - await future + task = c.submit(lambda: 1, key=key, workers=a.address) + await task response = await http_client.fetch(url) assert response.code == 200 body = response.body.decode() @@ -408,7 +408,7 @@ def f(ev1, ev2): ev1.set() ev2.wait() - future = c.submit(f, ev1, ev2, key=key) + task = c.submit(f, ev1, ev2, key=key) await ev1.wait() response = await http_client.fetch(url) @@ -417,7 +417,7 @@ def f(ev1, ev2): assert "test_scheduler_http.py" in body await ev2.set() - await future + await task response = await http_client.fetch(url) assert response.code == 200 body = response.body.decode() diff --git a/distributed/http/scheduler/tests/test_stealing_http.py b/distributed/http/scheduler/tests/test_stealing_http.py index aa8ffa67e2..76ddefa734 100644 --- a/distributed/http/scheduler/tests/test_stealing_http.py +++ b/distributed/http/scheduler/tests/test_stealing_http.py @@ -43,10 +43,10 @@ async def fetch_metrics_by_cost_multipliers(): expected_metrics = {str(multiplier): 0 for multiplier in stealing.cost_multipliers} assert active_metrics == expected_metrics - futures = c.map( + tasks = c.map( slowinc, range(10), delay=0.1, workers=a.address, allow_other_workers=True ) - await wait(futures) + await wait(tasks) active_metrics = await fetch_metrics_by_cost_multipliers() assert len(active_metrics) == len(stealing.cost_multipliers) @@ -76,10 +76,10 @@ async def fetch_metrics_by_cost_multipliers(): expected_metrics = {str(multiplier): 0 for multiplier in stealing.cost_multipliers} assert active_metrics == expected_metrics - futures = c.map( + tasks = c.map( slowinc, range(10), delay=0.1, workers=a.address, allow_other_workers=True ) - await wait(futures) + await wait(tasks) active_metrics = await fetch_metrics_by_cost_multipliers() assert len(active_metrics) == len(stealing.cost_multipliers) diff --git a/distributed/http/static/js/individual-cluster-map.js b/distributed/http/static/js/individual-cluster-map.js index 57b5d210a6..9744c0109e 100644 --- a/distributed/http/static/js/individual-cluster-map.js +++ b/distributed/http/static/js/individual-cluster-map.js @@ -1,7 +1,7 @@ // TODO Animate workers when performing tasks or swapping to show activity // TODO Add memory usage dial around outside of workers // TODO Add clients -// TODO Show future retrieval +// TODO Show task retrieval // TODO Show graph submission const workerColor = "#ECB172"; diff --git a/distributed/http/templates/task.html b/distributed/http/templates/task.html index 2344c680a9..b6c38d686b 100644 --- a/distributed/http/templates/task.html +++ b/distributed/http/templates/task.html @@ -114,7 +114,7 @@

Workers with data

{% if ts.who_wants %}
-

Clients with future

+

Clients with task

    diff --git a/distributed/lock.py b/distributed/lock.py index 99ec34cd6f..4595362aee 100644 --- a/distributed/lock.py +++ b/distributed/lock.py @@ -41,11 +41,11 @@ async def acquire(self, name=None, id=None, timeout=None): while name in self.ids: event = asyncio.Event() self.events[name].append(event) - future = event.wait() + task = event.wait() if timeout is not None: - future = wait_for(future, timeout) + task = wait_for(task, timeout) try: - await future + await task except TimeoutError: result = False break diff --git a/distributed/multi_lock.py b/distributed/multi_lock.py index c8fe2afe8e..6c6f7bd48c 100644 --- a/distributed/multi_lock.py +++ b/distributed/multi_lock.py @@ -116,11 +116,11 @@ async def acquire(self, locks=None, id=None, timeout=None, num_locks=None): assert id not in self.events event = asyncio.Event() self.events[id] = event - future = event.wait() + task = event.wait() if timeout is not None: - future = wait_for(future, timeout) + task = wait_for(task, timeout) try: - await future + await task except TimeoutError: self._refain_locks(locks, id) return False diff --git a/distributed/preloading.py b/distributed/preloading.py index 96c3889327..0133988d4d 100644 --- a/distributed/preloading.py +++ b/distributed/preloading.py @@ -208,18 +208,18 @@ async def start(self): if inspect.isawaitable(result): await result else: - future = dask_setup(self.dask_object) - if inspect.isawaitable(future): - await future + task = dask_setup(self.dask_object) + if inspect.isawaitable(task): + await task async def teardown(self): """Run when the server starts its close method""" dask_teardown = getattr(self.module, "dask_teardown", None) if dask_teardown: logger.info("Run preload teardown: %s", self.name) - future = dask_teardown(self.dask_object) - if inspect.isawaitable(future): - await future + task = dask_teardown(self.dask_object) + if inspect.isawaitable(task): + await task class PreloadManager(Sequence[Preload]): diff --git a/distributed/process.py b/distributed/process.py index 9b5bee0d32..cc59dfe119 100644 --- a/distributed/process.py +++ b/distributed/process.py @@ -38,27 +38,27 @@ def _loop_add_callback(loop, func, *args): raise -def _future_set_result_unless_cancelled(future, value): - if not future.cancelled(): - future.set_result(value) +def _task_set_result_unless_cancelled(task, value): + if not task.cancelled(): + task.set_result(value) -def _future_set_exception_unless_cancelled(future, exc): - if not future.cancelled(): - future.set_exception(exc) +def _task_set_exception_unless_cancelled(task, exc): + if not task.cancelled(): + task.set_exception(exc) else: - logger.error("Exception after Future was cancelled", exc_info=exc) + logger.error("Exception after Task was cancelled", exc_info=exc) -def _call_and_set_future(loop, future, func, *args, **kwargs): +def _call_and_set_task(loop, task, func, *args, **kwargs): try: res = func(*args, **kwargs) except Exception as exc: - # Tornado futures are not thread-safe, need to + # Tornado tasks are not thread-safe, need to # set_result() / set_exc_info() from the loop's thread - _loop_add_callback(loop, _future_set_exception_unless_cancelled, future, exc) + _loop_add_callback(loop, _task_set_exception_unless_cancelled, task, exc) else: - _loop_add_callback(loop, _future_set_result_unless_cancelled, future, res) + _loop_add_callback(loop, _task_set_result_unless_cancelled, task, res) class _ProcessState: @@ -203,7 +203,7 @@ def _run( @classmethod def _watch_message_queue( # type: ignore[no-untyped-def] - cls, selfref, process: multiprocessing.Process, loop, state, q, exit_future + cls, selfref, process: multiprocessing.Process, loop, state, q, exit_task ): # As multiprocessing.Process is not thread-safe, we run all # blocking operations from this single loop and ship results @@ -231,13 +231,13 @@ def _start(): logger.debug(f"[{r}] got message {msg!r}") op = msg["op"] if op == "start": - _call_and_set_future(loop, msg["future"], _start) + _call_and_set_task(loop, msg["task"], _start) elif op == "terminate": # Send SIGTERM - _call_and_set_future(loop, msg["future"], process.terminate) + _call_and_set_task(loop, msg["task"], process.terminate) elif op == "kill": # Send SIGKILL - _call_and_set_future(loop, msg["future"], process.kill) + _call_and_set_task(loop, msg["task"], process.kill) elif op == "stop": break @@ -279,17 +279,17 @@ def start(self): """ Start the child process. - This method returns a future. + This method returns a task. """ self._check_closed() fut = TornadoFuture() - self._watch_q.put_nowait({"op": "start", "future": fut}) + self._watch_q.put_nowait({"op": "start", "task": fut}) return fut def terminate(self) -> asyncio.Future[None]: """Terminate the child process. - This method returns a future. + This method returns a task. See also -------- @@ -297,14 +297,14 @@ def terminate(self) -> asyncio.Future[None]: """ self._check_closed() fut: TornadoFuture[None] = TornadoFuture() - self._watch_q.put_nowait({"op": "terminate", "future": fut}) + self._watch_q.put_nowait({"op": "terminate", "task": fut}) return fut def kill(self) -> asyncio.Future[None]: """Send SIGKILL to the child process. On Windows, this is the same as terminate(). - This method returns a future. + This method returns a task. See also -------- @@ -312,7 +312,7 @@ def kill(self) -> asyncio.Future[None]: """ self._check_closed() fut: TornadoFuture[None] = TornadoFuture() - self._watch_q.put_nowait({"op": "kill", "future": fut}) + self._watch_q.put_nowait({"op": "kill", "task": fut}) return fut async def join(self, timeout=None): @@ -325,8 +325,8 @@ async def join(self, timeout=None): assert self._state.pid is not None, "can only join a started process" if self._state.exitcode is not None: return - # Shield otherwise the timeout cancels the future and our - # on_exit callback will try to set a result on a canceled future + # Shield otherwise the timeout cancels the task and our + # on_exit callback will try to set a result on a canceled task await wait_for(asyncio.shield(self._exit_future), timeout) def close(self): diff --git a/distributed/protocol/tests/test_compression.py b/distributed/protocol/tests/test_compression.py index 7d8dcc29fb..64e7d8310f 100644 --- a/distributed/protocol/tests/test_compression.py +++ b/distributed/protocol/tests/test_compression.py @@ -180,9 +180,9 @@ def compress_decompress(): assert compressions[rc].decompress(rd) == expect with ThreadPoolExecutor(4) as ex: - futures = [ex.submit(compress_decompress) for _ in range(4)] - for future in futures: - future.result() + tasks = [ex.submit(compress_decompress) for _ in range(4)] + for task in tasks: + task.result() @pytest.mark.slow diff --git a/distributed/protocol/tests/test_serialize.py b/distributed/protocol/tests/test_serialize.py index 8cc85b5db4..7beccde9d8 100644 --- a/distributed/protocol/tests/test_serialize.py +++ b/distributed/protocol/tests/test_serialize.py @@ -204,8 +204,8 @@ async def test_object_in_graph(c, s, a, b): v = delayed(o) v2 = delayed(identity)(v) - future = c.compute(v2) - result = await future + task = c.compute(v2) + result = await task assert isinstance(result, MyObj) assert result.data == 123 @@ -214,9 +214,9 @@ async def test_object_in_graph(c, s, a, b): @gen_cluster(client=True, config=NO_AMM) async def test_scatter(c, s, a, b): o = MyObj(123) - [future] = await c._scatter([o]) + [task] = await c._scatter([o]) await c._replicate(o) - o2 = await c._gather(future) + o2 = await c._gather(task) assert isinstance(o2, MyObj) assert o2.data == 123 @@ -224,9 +224,9 @@ async def test_scatter(c, s, a, b): @gen_cluster(client=True) async def test_inter_worker_comms(c, s, a, b): o = MyObj(123) - [future] = await c._scatter([o], workers=a.address) - future2 = c.submit(identity, future, workers=b.address) - o2 = await c._gather(future2) + [task] = await c._scatter([o], workers=a.address) + task2 = c.submit(identity, task, workers=b.address) + o2 = await c._gather(task2) assert isinstance(o2, MyObj) assert o2.data == 123 diff --git a/distributed/queues.py b/distributed/queues.py index c48616459d..af71ce2dff 100644 --- a/distributed/queues.py +++ b/distributed/queues.py @@ -7,7 +7,7 @@ from dask.utils import parse_timedelta -from distributed.client import Future +from distributed.client import Task from distributed.utils import wait_for from distributed.worker import get_client @@ -30,7 +30,7 @@ def __init__(self, scheduler): self.scheduler = scheduler self.queues = dict() self.client_refcount = dict() - self.future_refcount = defaultdict(int) + self.task_refcount = defaultdict(int) self.scheduler.handlers.update( { @@ -42,7 +42,7 @@ def __init__(self, scheduler): ) self.scheduler.stream_handlers.update( - {"queue-future-release": self.future_release, "queue_release": self.release} + {"queue-task-release": self.task_release, "queue_release": self.release} ) def create(self, name=None, client=None, maxsize=0): @@ -60,31 +60,31 @@ def release(self, name=None, client=None): self.client_refcount[name] -= 1 if self.client_refcount[name] == 0: del self.client_refcount[name] - futures = self.queues[name]._queue + tasks = self.queues[name]._queue del self.queues[name] - keys = [d["value"] for d in futures if d["type"] == "Future"] + keys = [d["value"] for d in tasks if d["type"] == "Task"] if keys: self.scheduler.client_releases_keys(keys=keys, client="queue-%s" % name) async def put(self, name=None, key=None, data=None, client=None, timeout=None): if key is not None: - record = {"type": "Future", "value": key} - self.future_refcount[name, key] += 1 + record = {"type": "Task", "value": key} + self.task_refcount[name, key] += 1 self.scheduler.client_desires_keys(keys=[key], client="queue-%s" % name) else: record = {"type": "msgpack", "value": data} await wait_for(self.queues[name].put(record), timeout=timeout) - def future_release(self, name=None, key=None, client=None): - self.future_refcount[name, key] -= 1 - if self.future_refcount[name, key] == 0: + def task_release(self, name=None, key=None, client=None): + self.task_refcount[name, key] -= 1 + if self.task_refcount[name, key] == 0: self.scheduler.client_releases_keys(keys=[key], client="queue-%s" % name) - del self.future_refcount[name, key] + del self.task_refcount[name, key] async def get(self, name=None, client=None, timeout=None, batch=False): def process(record): """Add task status if known""" - if record["type"] == "Future": + if record["type"] == "Task": record = record.copy() key = record["value"] ts = self.scheduler.tasks.get(key) @@ -128,14 +128,14 @@ def qsize(self, name=None, client=None): class Queue: """Distributed Queue - This allows multiple clients to share futures or small bits of data between + This allows multiple clients to share tasks or small bits of data between each other with a multi-producer/multi-consumer queue. All metadata is sequentialized through the scheduler. Elements of the Queue must be either Futures or msgpack-encodable data (ints, strings, lists, dicts). All data is sent through the scheduler so it is wise not to send large objects. To share large objects scatter the - data and share the future instead. + data and share the task instead. .. warning:: @@ -158,8 +158,8 @@ class Queue: >>> from dask.distributed import Client, Queue # doctest: +SKIP >>> client = Client() # doctest: +SKIP >>> queue = Queue('x') # doctest: +SKIP - >>> future = client.submit(f, x) # doctest: +SKIP - >>> queue.put(future) # doctest: +SKIP + >>> task = client.submit(f, x) # doctest: +SKIP + >>> queue.put(task) # doctest: +SKIP See Also -------- @@ -212,7 +212,7 @@ async def _(): return _().__await__() async def _put(self, value, timeout=None): - if isinstance(value, Future): + if isinstance(value, Task): await self.client.scheduler.queue_put( key=value.key, timeout=timeout, name=self.name ) @@ -264,12 +264,12 @@ async def _get(self, timeout=None, batch=False): ) def process(d): - if d["type"] == "Future": - value = Future(d["value"], self.client, inform=True, state=d["state"]) + if d["type"] == "Task": + value = Task(d["value"], self.client, inform=True, state=d["state"]) if d["state"] == "erred": value._state.set_error(d["exception"], d["traceback"]) self.client._send_to_scheduler( - {"op": "queue-future-release", "name": self.name, "key": d["value"]} + {"op": "queue-task-release", "name": self.name, "key": d["value"]} ) else: value = d["value"] @@ -289,7 +289,7 @@ async def _qsize(self): def close(self): self._verify_running() - if self.client.status == "running": # TODO: can leave zombie futures + if self.client.status == "running": # TODO: can leave zombie tasks self.client._send_to_scheduler({"op": "queue_release", "name": self.name}) def __reduce__(self): diff --git a/distributed/recreate_tasks.py b/distributed/recreate_tasks.py index d3de76f576..3b8858b7d0 100644 --- a/distributed/recreate_tasks.py +++ b/distributed/recreate_tasks.py @@ -4,7 +4,7 @@ from dask.core import validate_key -from distributed.client import Future, futures_of, wait +from distributed.client import Task, tasks_of, wait from distributed.protocol.serialize import ToPickle from distributed.utils import sync from distributed.utils_comm import pack_data @@ -61,12 +61,10 @@ def __init__(self, client): self.client = client self.client.extensions["replay-tasks"] = self # monkey patch - self.client._get_raw_components_from_future = ( - self._get_raw_components_from_future - ) + self.client._get_raw_components_from_task = self._get_raw_components_from_task self.client._prepare_raw_components = self._prepare_raw_components - self.client._get_components_from_future = self._get_components_from_future - self.client._get_errored_future = self._get_errored_future + self.client._get_components_from_task = self._get_components_from_task + self.client._get_errored_task = self._get_errored_task self.client.recreate_task_locally = self.recreate_task_locally self.client.recreate_error_locally = self.recreate_error_locally @@ -74,48 +72,48 @@ def __init__(self, client): def scheduler(self): return self.client.scheduler - async def _get_raw_components_from_future(self, future): + async def _get_raw_components_from_task(self, task): """ - For a given future return the func, args and kwargs and future + For a given task return the func, args and kwargs and task deps that would be executed remotely. """ - if isinstance(future, Future): - await wait(future) - key = future.key + if isinstance(task, Task): + await wait(task) + key = task.key else: - validate_key(future) - key = future + validate_key(task) + key = task spec = await self.scheduler.get_runspec(key=key) return (*spec["task"], spec["deps"]) async def _prepare_raw_components(self, raw_components): """ - Take raw components and resolve future dependencies. + Take raw components and resolve task dependencies. """ function, args, kwargs, deps = raw_components - futures = self.client._graph_to_futures({}, deps) - data = await self.client._gather(futures) + tasks = self.client._graph_to_tasks({}, deps) + data = await self.client._gather(tasks) args = pack_data(args, data) kwargs = pack_data(kwargs, data) return (function, args, kwargs) - async def _get_components_from_future(self, future): + async def _get_components_from_task(self, task): """ - For a given future return the func, args and kwargs that would be - executed remotely. Any args/kwargs that are themselves futures will - be resolved to the return value of those futures. + For a given task return the func, args and kwargs that would be + executed remotely. Any args/kwargs that are themselves tasks will + be resolved to the return value of those tasks. """ - raw_components = await self._get_raw_components_from_future(future) + raw_components = await self._get_raw_components_from_task(task) return await self._prepare_raw_components(raw_components) - def recreate_task_locally(self, future): + def recreate_task_locally(self, task): """ For any calculation, whether it succeeded or failed, perform the task locally for debugging. - This operation should be performed after a future (result of ``gather``, + This operation should be performed after a task (result of ``gather``, ``compute``, etc) comes back with a status other than "pending". Cases - where you might want to debug a successfully completed future could + where you might want to debug a successfully completed task could include a calculation that returns an unexpected results. A common debugging process might include running the task locally in debug mode, with `pdb.runcall`. @@ -123,55 +121,55 @@ def recreate_task_locally(self, future): Examples -------- >>> import pdb # doctest: +SKIP - >>> future = c.submit(div, 1, 1) # doctest: +SKIP - >>> future.status # doctest: +SKIP + >>> task = c.submit(div, 1, 1) # doctest: +SKIP + >>> task.status # doctest: +SKIP 'finished' - >>> pdb.runcall(c.recreate_task_locally, future) # doctest: +SKIP + >>> pdb.runcall(c.recreate_task_locally, task) # doctest: +SKIP Parameters ---------- - future : future + task : task The same thing as was given to ``gather``. Returns ------- - Any; will return the result of the task future. + Any; will return the result of the task task. """ func, args, kwargs = sync( - self.client.loop, self._get_components_from_future, future + self.client.loop, self._get_components_from_task, task ) return func(*args, **kwargs) - async def _get_errored_future(self, future): + async def _get_errored_task(self, task): """ - For a given future collection, return the first future that raised + For a given task collection, return the first task that raised an error. """ - await wait(future) - futures = [f.key for f in futures_of(future) if f.status == "error"] - if not futures: - raise ValueError("No errored futures passed") - cause_key = await self.scheduler.get_error_cause(keys=futures) + await wait(task) + tasks = [f.key for f in tasks_of(task) if f.status == "error"] + if not tasks: + raise ValueError("No errored tasks passed") + cause_key = await self.scheduler.get_error_cause(keys=tasks) return cause_key - def recreate_error_locally(self, future): + def recreate_error_locally(self, task): """ For a failed calculation, perform the blamed task locally for debugging. - This operation should be performed after a future (result of ``gather``, + This operation should be performed after a task (result of ``gather``, ``compute``, etc) comes back with a status of "error", if the stack- trace is not informative enough to diagnose the problem. The specific - task (part of the graph pointing to the future) responsible for the + task (part of the graph pointing to the task) responsible for the error will be fetched from the scheduler, together with the values of its inputs. The function will then be executed, so that ``pdb`` can be used for debugging. Examples -------- - >>> future = c.submit(div, 1, 0) # doctest: +SKIP - >>> future.status # doctest: +SKIP + >>> task = c.submit(div, 1, 0) # doctest: +SKIP + >>> task.status # doctest: +SKIP 'error' - >>> c.recreate_error_locally(future) # doctest: +SKIP + >>> c.recreate_error_locally(task) # doctest: +SKIP ZeroDivisionError: division by zero If you're in IPython you might take this opportunity to use pdb @@ -179,7 +177,7 @@ def recreate_error_locally(self, future): >>> %pdb # doctest: +SKIP Automatic pdb calling has been turned ON - >>> c.recreate_error_locally(future) # doctest: +SKIP + >>> c.recreate_error_locally(task) # doctest: +SKIP ZeroDivisionError: division by zero 1 def div(x, y): ----> 2 return x / y @@ -187,15 +185,15 @@ def recreate_error_locally(self, future): Parameters ---------- - future : future or collection that failed + task : task or collection that failed The same thing as was given to ``gather``, but came back with an exception/stack-trace. Can also be a (persisted) dask collection - containing any errored futures. + containing any errored tasks. Returns ------- Nothing; the function runs and should raise an exception, allowing the debugger to run. """ - errored_future_key = sync(self.client.loop, self._get_errored_future, future) - return self.recreate_task_locally(errored_future_key) + errored_task_key = sync(self.client.loop, self._get_errored_task, task) + return self.recreate_task_locally(errored_task_key) diff --git a/distributed/scheduler.py b/distributed/scheduler.py index 1b51a0e34b..58e4fd507b 100644 --- a/distributed/scheduler.py +++ b/distributed/scheduler.py @@ -203,7 +203,7 @@ class ClientState: #: A set of tasks this client wants to be kept in memory, so that it can download #: its result when desired. This is the reverse mapping of #: :class:`TaskState.who_wants`. Tasks are typically removed from this set when the - #: corresponding object in the client's space (for example a ``Future`` or a Dask + #: corresponding object in the client's space (for example a ``Task`` or a Dask #: collection) gets garbage-collected. wants_what: set[TaskState] @@ -1917,7 +1917,7 @@ def _transition( ------- Tuple of: - - Dictionary of recommendations for future transitions {key: new state} + - Dictionary of recommendations for task transitions {key: new state} - Messages to clients {client address: [msg, msg, ...]} - Messages to workers {worker address: [msg, msg, ...]} @@ -2129,7 +2129,7 @@ def decide_worker_rootish_queuing_disabled( ) -> WorkerState | None: """Pick a worker for a runnable root-ish task, without queuing. - This attempts to schedule sibling tasks on the same worker, reducing future data + This attempts to schedule sibling tasks on the same worker, reducing task data transfer. It does not consider the location of dependencies, since they'll end up on every worker anyway. @@ -3955,7 +3955,7 @@ async def get_cluster_state( ) -> dict: "Produce the state dict used in a cluster state dump" # Kick off state-dumping on workers before we block the event loop in `self._to_dict`. - workers_future = asyncio.gather( + workers_task = asyncio.gather( self.broadcast( msg={"op": "dump_state", "exclude": exclude}, on_error="return", @@ -3968,11 +3968,11 @@ async def get_cluster_state( try: scheduler_state = self._to_dict(exclude=exclude) - worker_states, worker_versions = await workers_future + worker_states, worker_versions = await workers_task finally: # Ensure the tasks aren't left running if anything fails. # Someday (py3.11), use a trio-style TaskGroup for this. - workers_future.cancel() + workers_task.cancel() # Convert any RPC errors to strings worker_states = { @@ -4149,7 +4149,7 @@ async def log_errors(func): ext.teardown() logger.info("Scheduler closing all comms") - futures = [] + tasks = [] for _, comm in list(self.stream_comms.items()): # FIXME use `self.remove_worker()` instead after https://github.com/dask/distributed/issues/6390 if not comm.closed(): @@ -4159,9 +4159,9 @@ async def log_errors(func): comm.send({"op": "close-stream"}) # ^ TODO remove? `Worker.close` will close the stream anyway. with suppress(AttributeError): - futures.append(comm.close()) + tasks.append(comm.close()) - await asyncio.gather(*futures) + await asyncio.gather(*tasks) if self.jupyter: await self._jupyter_server_application._cleanup() @@ -5571,7 +5571,7 @@ async def add_client( ) -> None: """Add client to network - We listen to all future messages from this Comm. + We listen to all task messages from this Comm. """ assert client is not None comm.name = "Scheduler->Client" @@ -7452,10 +7452,10 @@ async def benchmark_hardware(self) -> dict[str, dict[str, float]]: # implementing logic based on IP addresses would not necessarily help. # Randomize the connections to even out the mean measures. random.shuffle(workers) - futures = [ + tasks = [ self.rpc(a).benchmark_network(address=b) for a, b in partition(2, workers) ] - responses = await asyncio.gather(*futures) + responses = await asyncio.gather(*tasks) for d in responses: for size, duration in d.items(): @@ -7601,7 +7601,7 @@ def stop_task_metadata(self, name: str | None = None) -> dict: async def register_worker_plugin( self, comm: None, plugin: bytes, name: str, idempotent: bool | None = None ) -> dict[str, OKMessage]: - """Registers a worker plugin on all running and future workers""" + """Registers a worker plugin on all running and task workers""" logger.info("Registering Worker plugin %s", name) if idempotent is None: warnings.warn( @@ -7636,7 +7636,7 @@ async def unregister_worker_plugin( async def register_nanny_plugin( self, comm: None, plugin: bytes, name: str, idempotent: bool | None = None ) -> dict[str, OKMessage]: - """Registers a nanny plugin on all running and future nannies""" + """Registers a nanny plugin on all running and task nannies""" logger.info("Registering Nanny plugin %s", name) if idempotent is None: @@ -7694,7 +7694,7 @@ def transition( Returns ------- - Dictionary of recommendations for future transitions + Dictionary of recommendations for task transitions See Also -------- @@ -7748,7 +7748,7 @@ def _reschedule( if worker and ts.processing_on and ts.processing_on.address != worker: return # transition_processing_released will immediately suggest an additional - # transition to waiting if the task has any waiters or clients holding a future. + # transition to waiting if the task has any waiters or clients holding a task. self.transitions({key: "released"}, stimulus_id=stimulus_id) ##################### @@ -8720,7 +8720,7 @@ def _materialize_graph( ) dependencies, _ = get_deps(dsk) - # Remove `Future` objects from graph and note any future dependencies + # Remove `Task` objects from graph and note any task dependencies dsk2 = {} fut_deps = {} for k, v in dsk.items(): @@ -8729,9 +8729,9 @@ def _materialize_graph( fut_deps[k] = futs dsk = dsk2 - # - Add in deps for any tasks that depend on futures - for k, futures in fut_deps.items(): - dependencies[k].update(f.key for f in futures) + # - Add in deps for any tasks that depend on tasks + for k, tasks in fut_deps.items(): + dependencies[k].update(f.key for f in tasks) # Remove any self-dependencies (happens on test_publish_bag() and others) for k, v in dependencies.items(): diff --git a/distributed/semaphore.py b/distributed/semaphore.py index 59d6951d7b..11fb64f776 100644 --- a/distributed/semaphore.py +++ b/distributed/semaphore.py @@ -151,9 +151,9 @@ async def acquire(self, name=None, timeout=None, lease_id=None): # If acquiring fails, we wait for the event to be set, i.e. something has # been released and we can try to acquire again (continue loop) if not result: - future = wait_for(self.events[name].wait(), timeout=deadline.remaining) + task = wait_for(self.events[name].wait(), timeout=deadline.remaining) try: - await future + await task continue except TimeoutError: result = False @@ -311,8 +311,8 @@ class Semaphore(SyncMethodMixin): ... with sem: ... pass ... - ... futures = client.map(access_resource, range(10), sem=sem) - ... client.gather(futures) + ... tasks = client.map(access_resource, range(10), sem=sem) + ... client.gather(tasks) ... # Once done, close the semaphore to clean up the state on scheduler side. ... sem.close() diff --git a/distributed/shuffle/_disk.py b/distributed/shuffle/_disk.py index 86a678a845..2668253c8b 100644 --- a/distributed/shuffle/_disk.py +++ b/distributed/shuffle/_disk.py @@ -150,7 +150,7 @@ async def _process(self, id: str, shards: list[Any]) -> None: Most of the logic here is about possibly going back to a separate thread, or about diagnostics. If things don't change much in the - future then we should consider simplifying this considerably and + task then we should consider simplifying this considerably and dropping the write into communicate above. """ frames: Iterable[bytes | bytearray | memoryview] diff --git a/distributed/shuffle/_scheduler_plugin.py b/distributed/shuffle/_scheduler_plugin.py index 09d97fffc9..d82edff774 100644 --- a/distributed/shuffle/_scheduler_plugin.py +++ b/distributed/shuffle/_scheduler_plugin.py @@ -229,7 +229,7 @@ def _calculate_worker_for(self, spec: ShuffleSpec) -> dict[Any, str]: if not (shuffle := self.active_shuffles.get(other_barrier_key)): continue current_worker_for = shuffle.run_spec.worker_for - # This is a fail-safe for future three-ways merges. At the moment there + # This is a fail-safe for task three-ways merges. At the moment there # should only ever be at most one other shuffle that shares output # tasks, so existing will always be empty. if existing: # pragma: nocover diff --git a/distributed/shuffle/tests/test_rechunk.py b/distributed/shuffle/tests/test_rechunk.py index 4744dfa6dc..aa4a1c3289 100644 --- a/distributed/shuffle/tests/test_rechunk.py +++ b/distributed/shuffle/tests/test_rechunk.py @@ -1255,7 +1255,7 @@ def test_pick_worker_homogeneous_distribution(nworkers): @gen_cluster( client=True, nthreads=[("", 1)] * 5, - # Future-proof: disable auto-rebalancing + # Task-proof: disable auto-rebalancing config={"distributed.scheduler.active-memory-manager.start": False}, ) async def test_partial_rechunk_homogeneous_distribution(c, s, *workers): diff --git a/distributed/tests/test_active_memory_manager.py b/distributed/tests/test_active_memory_manager.py index 2b8e56b52d..d436755cfd 100644 --- a/distributed/tests/test_active_memory_manager.py +++ b/distributed/tests/test_active_memory_manager.py @@ -135,7 +135,7 @@ async def test_drop(c, s, *workers): ): s.extensions["amm"].run_once() - futures = await c.scatter({"x": 123}, broadcast=True) + tasks = await c.scatter({"x": 123}, broadcast=True) assert len(s.tasks["x"].who_has) == 4 # Also test the extension handler with assert_amm_log( @@ -181,7 +181,7 @@ async def test_start_stop(c, s, a, b): @gen_cluster(client=True, config=demo_config("drop", start=True, interval=0.1)) async def test_auto_start(c, s, a, b): - futures = await c.scatter({"x": 123}, broadcast=True) + tasks = await c.scatter({"x": 123}, broadcast=True) # The AMM should run within 0.1s of the broadcast. # Add generous extra padding to prevent flakiness. await asyncio.sleep(0.5) @@ -206,7 +206,7 @@ async def test_add_policy(c, s, a, b): m3.add_policy(p3) assert len(m3.policies) == 1 - futures = await c.scatter({"x": 1, "y": 2, "z": 3}, broadcast=True) + tasks = await c.scatter({"x": 1, "y": 2, "z": 3}, broadcast=True) m1.run_once() while len(s.tasks["x"].who_has) == 2: await asyncio.sleep(0.01) @@ -240,7 +240,7 @@ async def test_multi_start(c, s, a, b): assert m2.running assert m3.running - futures = await c.scatter({"x": 1, "y": 2, "z": 3}, broadcast=True) + tasks = await c.scatter({"x": 1, "y": 2, "z": 3}, broadcast=True) # The AMMs should run within 0.1s of the broadcast. # Add generous extra padding to prevent flakiness. @@ -252,7 +252,7 @@ async def test_multi_start(c, s, a, b): @gen_cluster(client=True, config=NO_AMM) async def test_not_registered(c, s, a, b): - futures = await c.scatter({"x": 1}, broadcast=True) + tasks = await c.scatter({"x": 1}, broadcast=True) assert len(s.tasks["x"].who_has) == 2 class Policy(ActiveMemoryManagerPolicy): @@ -327,7 +327,7 @@ async def test_double_drop(c, s, a, b): Test that, in this use case, the last replica of a key is never dropped. """ - futures = await c.scatter({"x": 1}, broadcast=True) + tasks = await c.scatter({"x": 1}, broadcast=True) assert len(s.tasks["x"].who_has) == 2 ws_iter = iter(s.workers.values()) @@ -347,7 +347,7 @@ def run(self): @gen_cluster(client=True, config=demo_config("drop")) async def test_double_drop_stress(c, s, a, b): """AMM runs many times before the recommendations of the first run are enacted""" - futures = await c.scatter({"x": 1}, broadcast=True) + tasks = await c.scatter({"x": 1}, broadcast=True) assert len(s.tasks["x"].who_has) == 2 for _ in range(10): s.extensions["amm"].run_once() @@ -361,7 +361,7 @@ async def test_double_drop_stress(c, s, a, b): async def test_drop_from_worker_with_least_free_memory(c, s, *workers): ws1, ws2, ws3, ws4 = s.workers.values() - futures = await c.scatter({"x": 1}, broadcast=True) + tasks = await c.scatter({"x": 1}, broadcast=True) assert s.tasks["x"].who_has == {ws1, ws2, ws3, ws4} clog = c.submit(lambda: "x" * 100, workers=[ws3.address]) await wait(clog) @@ -378,7 +378,7 @@ async def test_drop_from_worker_with_least_free_memory(c, s, *workers): config=demo_config("drop", n=1, candidates=[5, 6]), ) async def test_drop_with_candidates(c, s, *workers): - futures = await c.scatter({"x": 1}, broadcast=True) + tasks = await c.scatter({"x": 1}, broadcast=True) s.extensions["amm"].run_once() wss = list(s.workers.values()) expect1 = {wss[0], wss[1], wss[2], wss[3], wss[4], wss[6], wss[7]} @@ -392,7 +392,7 @@ async def test_drop_with_empty_candidates(c, s, a, b): """Key is not dropped as the plugin proposes an empty set of candidates, not to be confused with None """ - futures = await c.scatter({"x": 1}, broadcast=True) + tasks = await c.scatter({"x": 1}, broadcast=True) s.extensions["amm"].run_once() await asyncio.sleep(0.2) assert len(s.tasks["x"].who_has) == 2 @@ -567,7 +567,7 @@ def executing() -> bool: @gen_cluster(nthreads=[("", 1)] * 4, client=True, config=demo_config("replicate", n=2)) async def test_replicate(c, s, *workers): - futures = await c.scatter({"x": 123}) + tasks = await c.scatter({"x": 123}) assert len(s.tasks["x"].who_has) == 1 s.extensions["amm"].run_once() @@ -602,7 +602,7 @@ async def test_replicate_not_in_memory(c, s, a, b): @gen_cluster(client=True, config=demo_config("replicate")) async def test_double_replicate_stress(c, s, a, b): """AMM runs many times before the recommendations of the first run are enacted""" - futures = await c.scatter({"x": 1}) + tasks = await c.scatter({"x": 1}) assert len(s.tasks["x"].who_has) == 1 for _ in range(10): s.extensions["amm"].run_once() @@ -631,7 +631,7 @@ async def test_replicate_to_worker_with_most_free_memory(c, s, *workers): ) async def test_replicate_with_candidates(c, s, *workers): wss = list(s.workers.values()) - futures = await c.scatter({"x": 1}, workers=[wss[0].address]) + tasks = await c.scatter({"x": 1}, workers=[wss[0].address]) s.extensions["amm"].run_once() expect1 = {wss[0], wss[5]} expect2 = {wss[0], wss[6]} @@ -644,7 +644,7 @@ async def test_replicate_with_empty_candidates(c, s, a, b): """Key is not replicated as the plugin proposes an empty set of candidates, not to be confused with None """ - futures = await c.scatter({"x": 1}) + tasks = await c.scatter({"x": 1}) s.extensions["amm"].run_once() await asyncio.sleep(0.2) assert len(s.tasks["x"].who_has) == 1 @@ -654,7 +654,7 @@ async def test_replicate_with_empty_candidates(c, s, a, b): async def test_replicate_to_candidates_with_key(c, s, a, b): """Key is not replicated as all candidates already hold replicas""" ws0, ws1 = s.workers.values() # Not necessarily a, b; it could be b, a! - futures = await c.scatter({"x": 1}, workers=[ws0.address]) + tasks = await c.scatter({"x": 1}, workers=[ws0.address]) s.extensions["amm"].run_once() await asyncio.sleep(0.2) assert s.tasks["x"].who_has == {ws0} @@ -666,7 +666,7 @@ async def test_replicate_avoids_paused_workers_1(c, s, w0, w1, w2): while s.workers[w1.address].status != Status.paused: await asyncio.sleep(0.01) - futures = await c.scatter({"x": 1}, workers=[w0.address]) + tasks = await c.scatter({"x": 1}, workers=[w0.address]) s.extensions["amm"].run_once() while "x" not in w2.data: await asyncio.sleep(0.01) @@ -680,7 +680,7 @@ async def test_replicate_avoids_paused_workers_2(c, s, a, b): while s.workers[b.address].status != Status.paused: await asyncio.sleep(0.01) - futures = await c.scatter({"x": 1}, workers=[a.address]) + tasks = await c.scatter({"x": 1}, workers=[a.address]) s.extensions["amm"].run_once() await asyncio.sleep(0.2) assert "x" not in b.data @@ -721,7 +721,7 @@ async def test_ReduceReplicas(c, s, *workers): ): s.extensions["amm"].run_once() - futures = await c.scatter({"x": 123}, broadcast=True) + tasks = await c.scatter({"x": 123}, broadcast=True) assert len(s.tasks["x"].who_has) == 4 with assert_amm_log( @@ -817,7 +817,7 @@ async def test_RetireWorker_amm_on_off(c, s, a, b, start_amm): else: await c.amm.stop() - futures = await c.scatter({"x": 1}, workers=[a.address]) + tasks = await c.scatter({"x": 1}, workers=[a.address]) await c.retire_workers([a.address]) assert a.address not in s.workers assert "x" in b.data @@ -830,7 +830,7 @@ async def test_RetireWorker_amm_on_off(c, s, a, b, start_amm): ) async def test_RetireWorker_no_extension(c, s, a, b): """retire_workers must work when the AMM extension is not loaded""" - futures = await c.scatter({"x": 1}, workers=[a.address]) + tasks = await c.scatter({"x": 1}, workers=[a.address]) await c.retire_workers([a.address]) assert a.address not in s.workers assert "x" in b.data diff --git a/distributed/tests/test_actor.py b/distributed/tests/test_actor.py index c44c0ec7f3..025d07261a 100644 --- a/distributed/tests/test_actor.py +++ b/distributed/tests/test_actor.py @@ -10,10 +10,10 @@ from distributed import ( Actor, - BaseActorFuture, + BaseActorTask, Client, - Future, Nanny, + Task, as_completed, get_client, wait, @@ -72,7 +72,7 @@ async def test_client_actions(s, a, b, direct_to_workers): s.address, asynchronous=True, direct_to_workers=direct_to_workers ) as c: counter = c.submit(Counter, workers=[a.address], actor=True) - assert isinstance(counter, Future) + assert isinstance(counter, Task) counter = await counter assert counter._address assert hasattr(counter, "increment") @@ -106,14 +106,14 @@ def f(counter): assert type(counter) is Actor assert counter._address == a_address - future = counter.increment(separate_thread=separate_thread) - assert isinstance(future, BaseActorFuture) - assert "Future" in type(future).__name__ - end = future.result(timeout=1) + task = counter.increment(separate_thread=separate_thread) + assert isinstance(task, BaseActorTask) + assert "Task" in type(task).__name__ + end = task.result(timeout=1) assert end > start - futures = [c.submit(f, counter, pure=False) for _ in range(10)] - await c.gather(futures) + tasks = [c.submit(f, counter, pure=False) for _ in range(10)] + await c.gather(tasks) counter = await counter assert await counter.n == 10 @@ -139,8 +139,8 @@ async def test_Actor(c, s, a, b): @gen_cluster(client=True) async def test_linear_access(c, s, a, b): start = time() - future = c.submit(sleep, 0.2) - actor = c.submit(List, actor=True, dummy=future) + task = c.submit(sleep, 0.2) + actor = c.submit(List, actor=True, dummy=task) actor = await actor for i in range(100): @@ -208,9 +208,9 @@ async def test_track_dependencies(c, s, a, b): @gen_cluster(client=True) -async def test_future(c, s, a, b): +async def test_task(c, s, a, b): counter = c.submit(Counter, actor=True, workers=[a.address]) - assert isinstance(counter, Future) + assert isinstance(counter, Task) await wait(counter) assert isinstance(a.state.actors[counter.key], Counter) @@ -219,11 +219,11 @@ async def test_future(c, s, a, b): assert counter._address await asyncio.sleep(0.1) - assert counter.key in c.futures # don't lose future + assert counter.key in c.tasks # don't lose task @gen_cluster(client=True) -async def test_future_dependencies(c, s, a, b): +async def test_task_dependencies(c, s, a, b): counter = c.submit(Counter, actor=True, workers=[a.address]) def f(a): @@ -249,15 +249,15 @@ def test_sync(client): assert counter.n == 0 - future = counter.increment() - n = future.result() + task = counter.increment() + n = task.result() assert n == 1 assert counter.n == 1 - assert future.result() == future.result() + assert task.result() == task.result() - assert "ActorFuture" in repr(future) - assert "distributed.actor" not in repr(future) + assert "ActorTask" in repr(task) + assert "distributed.actor" not in repr(task) def test_timeout(client): @@ -272,20 +272,20 @@ async def wait(self): return await self.event.wait() event = client.submit(Waiter, actor=True).result() - future = event.wait() + task = event.wait() with pytest.raises(asyncio.TimeoutError): - future.result(timeout="0.001s") + task.result(timeout="0.001s") event.set().result() - assert future.result() is True + assert task.result() is True @gen_cluster(client=True, config={"distributed.comm.timeouts.connect": "1s"}) async def test_failed_worker(c, s, a, b): - future = c.submit(Counter, actor=True, workers=[a.address]) - await wait(future) - counter = await future + task = c.submit(Counter, actor=True, workers=[a.address]) + await wait(task) + counter = await task await a.close() @@ -359,8 +359,8 @@ def add(n, counter): for _ in range(n): counter.increment().result() - futures = c.map(add, range(10), counter=counter) - done = c.submit(lambda x: None, futures) + tasks = c.map(add, range(10), counter=counter) + done = c.submit(lambda x: None, tasks) while not done.done(): assert ( @@ -389,16 +389,16 @@ def f(self): unsafe = await c.submit(Unsafe, actor=True) - futures = [unsafe.f() for i in range(10)] - await c.gather(futures) + tasks = [unsafe.f() for i in range(10)] + await c.gather(tasks) @gen_cluster(client=True) async def test_Actors_create_dependencies(c, s, a, b): counter = await c.submit(Counter, actor=True) - future = c.submit(lambda x: None, counter) - await wait(future) - assert s.tasks[future.key].dependencies == {s.tasks[counter.key]} + task = c.submit(lambda x: None, counter) + await wait(task) + assert s.tasks[task.key].dependencies == {s.tasks[counter.key]} @gen_cluster(client=True) @@ -565,14 +565,14 @@ async def wait(self): waiter = await c.submit(Waiter, actor=True) - futures = [waiter.wait() for _ in range(5)] # way more than we have actor threads + tasks = [waiter.wait() for _ in range(5)] # way more than we have actor threads await asyncio.sleep(0.1) - assert not any(future.done() for future in futures) + assert not any(task.done() for task in tasks) await waiter.set() - await c.gather(futures) + await c.gather(tasks) @gen_cluster(client=True, client_kwargs=dict(set_as_default=False)) @@ -649,7 +649,7 @@ class UsesCounter: # An actor whose method argument is another actor def do_inc(self, ac): - # ac.increment() returns an EagerActorFuture and so the timeout + # ac.increment() returns an EagerActorTask and so the timeout # cannot expire return ac.increment().result(timeout=0.001) @@ -734,32 +734,32 @@ def prop(self): def test_as_completed(client): ac = client.submit(Counter, actor=True).result() - futures = [ac.increment() for _ in range(10)] + tasks = [ac.increment() for _ in range(10)] max = 0 - for future in as_completed(futures): - value = future.result() + for task in as_completed(tasks): + value = task.result() if value > max: max = value - assert all(future.done() for future in futures) + assert all(task.done() for task in tasks) assert max == 10 @gen_cluster(client=True, timeout=3) -async def test_actor_future_awaitable(client, s, a, b): +async def test_actor_task_awaitable(client, s, a, b): ac = await client.submit(Counter, actor=True) - futures = [ac.increment() for _ in range(10)] + tasks = [ac.increment() for _ in range(10)] - assert all([isinstance(future, BaseActorFuture) for future in futures]) + assert all([isinstance(task, BaseActorTask) for task in tasks]) - out = await asyncio.gather(*futures) - assert all([future.done() for future in futures]) + out = await asyncio.gather(*tasks) + assert all([task.done() for task in tasks]) assert max(out) == 10 @gen_cluster(client=True) -async def test_actor_future_awaitable_deadlock(client, s, a, b): +async def test_actor_task_awaitable_deadlock(client, s, a, b): ac = await client.submit(Counter, actor=True) f = ac.increment() @@ -781,8 +781,8 @@ def __getstate__(self): def __setstate__(self, state): self.actor = state - future = c.submit(Foo, workers=a.address) - foo = await future + task = c.submit(Foo, workers=a.address) + foo = await task assert isinstance(foo.actor, Actor) diff --git a/distributed/tests/test_as_completed.py b/distributed/tests/test_as_completed.py index 27b5b18c94..d4508e98c6 100644 --- a/distributed/tests/test_as_completed.py +++ b/distributed/tests/test_as_completed.py @@ -71,7 +71,7 @@ def test_as_completed_sync(client): assert list(as_completed([])) == [] -def test_as_completed_with_non_futures(client): +def test_as_completed_with_non_tasks(client): with pytest.raises(TypeError): list(as_completed([1, 2, 3])) @@ -79,14 +79,14 @@ def test_as_completed_with_non_futures(client): def test_as_completed_add(client): total = 0 expected = sum(map(inc, range(10))) - futures = client.map(inc, range(10)) - ac = as_completed(futures) - for future in ac: - result = future.result() + tasks = client.map(inc, range(10)) + ac = as_completed(tasks) + for task in ac: + result = task.result() total += result if random.random() < 0.5: - future = client.submit(add, future, 10) - ac.add(future) + task = client.submit(add, task, 10) + ac.add(task) expected += result + 10 assert total == expected @@ -170,8 +170,8 @@ async def _(): @gen_cluster(client=True) async def test_async_for_py2_equivalent(c, s, a, b): - futures = c.map(sleep, [0.01] * 3, pure=False) - seq = as_completed(futures) + tasks = c.map(sleep, [0.01] * 3, pure=False) + seq = as_completed(tasks) x, y, z = [el async for el in seq] assert x.done() assert y.done() @@ -251,8 +251,8 @@ def test_as_completed_with_results_no_raise(client): @gen_cluster(client=True) async def test_str(c, s, a, b): - futures = c.map(inc, range(3)) - ac = as_completed(futures) + tasks = c.map(inc, range(3)) + ac = as_completed(tasks) assert "waiting=3" in str(ac) assert "waiting=3" in repr(ac) assert "done=0" in str(ac) @@ -289,13 +289,13 @@ async def test_as_completed_with_results_no_raise_async(c, s, a, b): @gen_cluster(client=True) async def test_clear(c, s, a, b): - futures = c.map(inc, range(3)) - ac = as_completed(futures) - await wait(futures) + tasks = c.map(inc, range(3)) + ac = as_completed(tasks) + await wait(tasks) ac.clear() with pytest.raises(StopAsyncIteration): await ac.__anext__() - del futures + del tasks while s.tasks: await asyncio.sleep(0.3) diff --git a/distributed/tests/test_cancelled_state.py b/distributed/tests/test_cancelled_state.py index 8daa83bf87..e1b96a9d55 100644 --- a/distributed/tests/test_cancelled_state.py +++ b/distributed/tests/test_cancelled_state.py @@ -221,7 +221,7 @@ async def wait_and_raise(*args, **kwargs): await wait_for_state(f1.key, "cancelled", w) await lock.release() - # At this point we do not fetch the result of the future since the future + # At this point we do not fetch the result of the task since the task # itself would raise a cancelled exception. At this point we're concerned # about the worker. The task should transition over error to be eventually # forgotten since we no longer hold a ref. @@ -466,7 +466,7 @@ def block(x, lock, enter_event, exit_event): await enter_compute.wait() - async def release_all_futures(): + async def release_all_tasks(): futs = [f1, f2, f3, f4] for fut in futs: fut.release() @@ -474,7 +474,7 @@ async def release_all_futures(): while any(fut.key in s.tasks for fut in futs): await asyncio.sleep(0.05) - await release_all_futures() + await release_all_tasks() await wait_for_state(f3.key, "cancelled", b) f1 = c.submit(inc, 1, key="f1", workers=[a.address]) @@ -483,7 +483,7 @@ async def release_all_futures(): f4 = c.submit(sum, [f1, f3], key="f4", workers=[b.address]) await wait_for_state(f3.key, "resumed", b) - await release_all_futures() + await release_all_tasks() if not wait_for_processing: await lock_compute.release() @@ -644,7 +644,7 @@ def block(x, lock, enter_event, exit_event): await enter_compute.wait() - async def release_all_futures(): + async def release_all_tasks(): futs = [f1, f2, f3, f4] for fut in futs: fut.release() @@ -652,7 +652,7 @@ async def release_all_futures(): while any(fut.key in s.tasks for fut in futs): await asyncio.sleep(0.05) - await release_all_futures() + await release_all_tasks() await wait_for_state(f3.key, "cancelled", b) f1 = c.submit(inc, 1, key="f1", workers=[a.address]) @@ -757,7 +757,7 @@ def block(x, lock, enter_event, exit_event, raise_error): await enter_compute_erring.wait() - async def release_all_futures(): + async def release_all_tasks(): futs = [f1, f2, f3, f4] for fut in futs: fut.release() @@ -766,7 +766,7 @@ async def release_all_futures(): await asyncio.sleep(0.05) with freeze_batched_send(s.stream_comms[b.address]): - await release_all_futures() + await release_all_tasks() f1 = c.submit(inc, 1, key="f1", workers=[a.address]) f2 = c.submit(inc, f1, key="f2", workers=[a.address]) diff --git a/distributed/tests/test_client.py b/distributed/tests/test_client.py index 67f79053d9..4d4bfd5203 100644 --- a/distributed/tests/test_client.py +++ b/distributed/tests/test_client.py @@ -62,14 +62,14 @@ ) from distributed.client import ( Client, - Future, + Task, _get_global_client, _global_clients, as_completed, default_client, ensure_default_client, - futures_of, get_task_metadata, + tasks_of, temp_default_client, tokenize, wait, @@ -128,7 +128,7 @@ async def test_submit(c, s, a, b): x = c.submit(inc, 10, key="x") assert not x.done() - assert isinstance(x, Future) + assert isinstance(x, Task) assert x.client is c result = await x @@ -148,7 +148,7 @@ async def test_map(c, s, a, b): L1 = c.map(inc, range(5)) assert len(L1) == 5 assert isdistinct(x.key for x in L1) - assert all(isinstance(x, Future) for x in L1) + assert all(isinstance(x, Task) for x in L1) result = await L1[0] assert result == inc(0) @@ -197,16 +197,16 @@ async def test_map_empty(c, s, a, b): @gen_cluster(client=True) async def test_map_keynames(c, s, a, b): - futures = c.map(inc, range(4), key="INC") - assert all(f.key.startswith("INC") for f in futures) - assert isdistinct(f.key for f in futures) + tasks = c.map(inc, range(4), key="INC") + assert all(f.key.startswith("INC") for f in tasks) + assert isdistinct(f.key for f in tasks) - futures2 = c.map(inc, [5, 6, 7, 8], key="INC") - assert [f.key for f in futures] != [f.key for f in futures2] + tasks2 = c.map(inc, [5, 6, 7, 8], key="INC") + assert [f.key for f in tasks] != [f.key for f in tasks2] keys = ["inc-1", "inc-2", "inc-3", "inc-4"] - futures = c.map(inc, range(4), key=keys) - assert [f.key for f in futures] == keys + tasks = c.map(inc, range(4), key=keys) + assert [f.key for f in tasks] == keys @gen_cluster(client=True) @@ -290,7 +290,7 @@ async def test_compute_retries(c, s, a, b): @gen_cluster(client=True) async def test_compute_retries_annotations(c, s, a, b): - # Per-future retries + # Per-task retries xargs = [ZeroDivisionError("one"), ZeroDivisionError("two"), 30, 40] yargs = [ZeroDivisionError("five"), ZeroDivisionError("six"), 70] zargs = [80, 90, 100] @@ -369,13 +369,13 @@ async def test_persist_retries_annotations(c, s, a, b): async def test_retries_dask_array(c, s, a, b): da = pytest.importorskip("dask.array") x = da.ones((10, 10), chunks=(3, 3)) - future = c.compute(x.sum(), retries=2) - y = await future + task = c.compute(x.sum(), retries=2) + y = await task assert y == 100 @gen_cluster(client=True) -async def test_future_repr(c, s, a, b): +async def test_task_repr(c, s, a, b): pd = pytest.importorskip("pandas") x = c.submit(inc, 10) y = c.submit(pd.DataFrame, {"x": [1, 2, 3]}) @@ -385,7 +385,7 @@ async def test_future_repr(c, s, a, b): for func in [repr, lambda x: x._repr_html_()]: assert str(x.key) in func(x) assert str(x.status) in func(x) - assert str(x.status) in repr(c.futures[x.key]) + assert str(x.status) in repr(c.tasks[x.key]) assert "int" in func(x) assert "pandas" in func(y) @@ -393,10 +393,10 @@ async def test_future_repr(c, s, a, b): @gen_cluster(client=True) -async def test_future_tuple_repr(c, s, a, b): +async def test_task_tuple_repr(c, s, a, b): da = pytest.importorskip("dask.array") y = da.arange(10, chunks=(5,)).persist() - f = futures_of(y)[0] + f = tasks_of(y)[0] for func in [repr, lambda x: x._repr_html_()]: for k in f.key: assert str(k) in func(f) @@ -428,18 +428,18 @@ async def test_Future_release(c, s, a, b): await x x.release() await asyncio.sleep(0) - assert not c.futures + assert not c.tasks x = c.submit(slowinc, 1, delay=0.5) x.release() await asyncio.sleep(0) - assert not c.futures + assert not c.tasks x = c.submit(div, 1, 0) await x.exception() x.release() await asyncio.sleep(0) - assert not c.futures + assert not c.tasks def test_Future_release_sync(c): @@ -447,16 +447,16 @@ def test_Future_release_sync(c): x = c.submit(div, 1, 1) x.result() x.release() - poll_for(lambda: not c.futures, timeout=0.3) + poll_for(lambda: not c.tasks, timeout=0.3) x = c.submit(slowinc, 1, delay=0.8) x.release() - poll_for(lambda: not c.futures, timeout=0.3) + poll_for(lambda: not c.tasks, timeout=0.3) x = c.submit(div, 1, 0) x.exception() x.release() - poll_for(lambda: not c.futures, timeout=0.3) + poll_for(lambda: not c.tasks, timeout=0.3) @pytest.mark.parametrize("method", ["result", "gather"]) @@ -467,12 +467,12 @@ def test_short_tracebacks(c, method): test_short_tracebacks_async dask/tests/test_traceback.py """ - future = c.submit(div, 1, 0) + task = c.submit(div, 1, 0) with pytest.raises(ZeroDivisionError) as e: if method == "result": - future.result() + task.result() else: - c.gather(future) + c.gather(task) frames = list(traceback.walk_tb(e.value.__traceback__)) assert len(frames) < 4 @@ -487,15 +487,15 @@ async def test_short_tracebacks_async(c, s, a, b, method): test_short_tracebacks dask/tests/test_traceback.py """ - future = c.submit(div, 1, 0) + task = c.submit(div, 1, 0) with pytest.raises(ZeroDivisionError) as e: if method == "await": - await future + await task elif method == "result": - await future.result() + await task.result() else: - await c.gather(future) + await c.gather(task) frames = list(traceback.walk_tb(e.value.__traceback__)) assert len(frames) < 4 @@ -653,25 +653,25 @@ async def test_gather_skip(c, s, a): @gen_cluster(client=True) async def test_limit_concurrent_gathering(c, s, a, b): - futures = c.map(inc, range(100)) - await c.gather(futures) + tasks = c.map(inc, range(100)) + await c.gather(tasks) assert len(a.transfer_outgoing_log) + len(b.transfer_outgoing_log) < 100 @gen_cluster(client=True) async def test_get(c, s, a, b): - future = c.get({"x": (inc, 1)}, "x", sync=False) - assert isinstance(future, Future) - result = await future + task = c.get({"x": (inc, 1)}, "x", sync=False) + assert isinstance(task, Task) + result = await task assert result == 2 - futures = c.get({"x": (inc, 1)}, ["x"], sync=False) - assert isinstance(futures[0], Future) - result = await c.gather(futures) + tasks = c.get({"x": (inc, 1)}, ["x"], sync=False) + assert isinstance(tasks[0], Task) + result = await c.gather(tasks) assert result == [2] - futures = c.get({}, [], sync=False) - result = await c.gather(futures) + tasks = c.get({}, [], sync=False) + result = await c.gather(tasks) assert result == [] result = await c.get( @@ -684,14 +684,14 @@ def test_get_sync(c): assert c.get({"x": (inc, 1)}, "x") == 2 -def test_no_future_references(c): - """Test that there are neither global references to Future objects nor circular +def test_no_task_references(c): + """Test that there are neither global references to Task objects nor circular references that need to be collected by gc """ ws = weakref.WeakSet() - futures = c.map(inc, range(10)) - ws.update(futures) - del futures + tasks = c.map(inc, range(10)) + ws.update(tasks) + del tasks with profile.lock: assert not list(ws) @@ -709,12 +709,12 @@ def f(a, b): def g(a, b): raise AttributeError - future_f = c.submit(f, 1, 2) - future_g = c.submit(g, 1, 2) + task_f = c.submit(f, 1, 2) + task_g = c.submit(g, 1, 2) with pytest.raises(TypeError): - await c.gather(future_f) + await c.gather(task_f) with pytest.raises(AttributeError): - await c.gather(future_g) + await c.gather(task_g) await a.close() @@ -751,14 +751,14 @@ async def test_wait_first_completed(c, s, a, b): @gen_cluster(client=True) async def test_wait_timeout(c, s, a, b): - future = c.submit(sleep, 0.3) + task = c.submit(sleep, 0.3) with pytest.raises(TimeoutError): - await wait(future, timeout=0.01) + await wait(task, timeout=0.01) # Ensure timeout can be a string - future = c.submit(sleep, 0.3) + task = c.submit(sleep, 0.3) with pytest.raises(TimeoutError): - await wait(future, timeout="0.01 s") + await wait(task, timeout="0.01 s") def test_wait_sync(c): @@ -770,9 +770,9 @@ def test_wait_sync(c): assert not_done == set() assert x.status == y.status == "finished" - future = c.submit(sleep, 0.3) + task = c.submit(sleep, 0.3) with pytest.raises(TimeoutError): - wait(future, timeout=0.01) + wait(task, timeout=0.01) def test_wait_informative_error_for_timeouts(c): @@ -806,19 +806,19 @@ async def test_garbage_collection(c, s, a, b): ykey = y.key y.__del__() await asyncio.sleep(0) - assert ykey not in c.futures + assert ykey not in c.tasks @gen_cluster(client=True) async def test_garbage_collection_with_scatter(c, s, a, b): - [future] = await c.scatter([1]) - assert future.key in c.futures - assert future.status == "finished" - assert {cs.client_key for cs in s.tasks[future.key].who_wants} == {c.id} + [task] = await c.scatter([1]) + assert task.key in c.tasks + assert task.status == "finished" + assert {cs.client_key for cs in s.tasks[task.key].who_wants} == {c.id} - key = future.key + key = task.key assert c.refcount[key] == 1 - future.__del__() + task.__del__() await asyncio.sleep(0) assert c.refcount[key] == 0 @@ -841,7 +841,7 @@ async def test_recompute_released_key(c, s, a, b): await asyncio.sleep(0.1) x = c.submit(inc, 100) - assert x.key in c.futures + assert x.key in c.tasks result2 = await x assert result1 == result2 @@ -856,14 +856,14 @@ async def test_long_tasks_dont_trigger_timeout(c, s, a, b): @gen_cluster(client=True) -async def test_tokenize_on_futures(c, s, a, b): +async def test_tokenize_on_tasks(c, s, a, b): x = c.submit(inc, 1) y = c.submit(inc, 1) tok = tokenize(x) assert tokenize(x) == tokenize(x) assert tokenize(x) == tokenize(y) - c.futures[x.key].finish() + c.tasks[x.key].finish() assert tok == tokenize(y) @@ -912,8 +912,8 @@ async def test_restrictions_map(c, s, a, b): async def test_restrictions_get(c, s, a, b): dsk = {"x": 1, "y": (inc, "x"), "z": (inc, "y")} - futures = c.get(dsk, ["y", "z"], workers=a.ip, sync=False) - result = await c.gather(futures) + tasks = c.get(dsk, ["y", "z"], workers=a.ip, sync=False) + result = await c.gather(tasks) assert result == [2, 3] assert "y" in a.data assert "z" in a.data @@ -928,8 +928,8 @@ async def test_restrictions_get_annotate(c, s, a, b): with dask.annotate(workers=b.address): z = delayed(inc)(y) - futures = c.get(z.__dask_graph__(), [y.key, z.key], sync=False) - result = await c.gather(futures) + tasks = c.get(z.__dask_graph__(), [y.key, z.key], sync=False) + result = await c.gather(tasks) assert result == [2, 3] assert y.key in a.data assert z.key in b.data @@ -1061,7 +1061,7 @@ async def test_aliases_2(c, s, a, b): @gen_cluster(client=True) async def test_scatter(c, s, a, b): d = await c.scatter({"y": 20}) - assert isinstance(d["y"], Future) + assert isinstance(d["y"], Task) assert a.data.get("y") == 20 or b.data.get("y") == 20 y_who_has = s.get_who_has(keys=["y"])["y"] assert a.address in y_who_has or b.address in y_who_has @@ -1070,7 +1070,7 @@ async def test_scatter(c, s, a, b): assert yy == [20] [x] = await c.scatter([10]) - assert isinstance(x, Future) + assert isinstance(x, Task) assert a.data.get(x.key) == 10 or b.data.get(x.key) == 10 xx = await c.gather([x]) x_who_has = s.get_who_has(keys=[x.key])[x.key] @@ -1082,7 +1082,7 @@ async def test_scatter(c, s, a, b): assert s.get_nbytes(summary=False) == {"y": sizeof(20), x.key: sizeof(10)} assert xx == [10] - z = c.submit(add, x, d["y"]) # submit works on Future + z = c.submit(add, x, d["y"]) # submit works on Task result = await z assert result == 10 + 20 result = await c.gather([z, x]) @@ -1110,7 +1110,7 @@ async def test_scatter_types(c, s, a, b): @gen_cluster(client=True) async def test_scatter_non_list(c, s, a, b): x = await c.scatter(1) - assert isinstance(x, Future) + assert isinstance(x, Task) result = await x assert result == 1 @@ -1131,7 +1131,7 @@ def f(x): obj = MyObj() - future = await c.scatter(obj) + task = await c.scatter(obj) assert L and L[0] is obj @@ -1140,15 +1140,15 @@ async def test_scatter_singletons(c, s, a, b): np = pytest.importorskip("numpy") pd = pytest.importorskip("pandas") for x in [1, np.ones(5), pd.DataFrame({"x": [1, 2, 3]})]: - future = await c.scatter(x) - result = await future + task = await c.scatter(x) + result = await task assert str(result) == str(x) @gen_cluster(client=True) async def test_scatter_typename(c, s, a, b): - future = await c.scatter(123) - assert future.key.startswith("int") + task = await c.scatter(123) + assert task.key.startswith("int") @gen_cluster(client=True) @@ -1492,11 +1492,11 @@ async def test_directed_scatter(c, s, a, b): @gen_cluster(client=True) async def test_scatter_direct(c, s, a, b): - future = await c.scatter(123, direct=True) - assert future.key in a.data or future.key in b.data - assert s.tasks[future.key].who_has - assert future.status == "finished" - result = await future + task = await c.scatter(123, direct=True) + assert task.key in a.data or task.key in b.data + assert s.tasks[task.key].who_has + assert task.status == "finished" + result = await task assert result == 123 assert not s.counters["op"].components[0]["scatter"] @@ -1513,36 +1513,36 @@ async def test_scatter_direct_2(s, a, b): async def test_scatter_direct_numpy(c, s, a, b): np = pytest.importorskip("numpy") x = np.ones(5) - future = await c.scatter(x, direct=True) - result = await future + task = await c.scatter(x, direct=True) + result = await task assert np.allclose(x, result) assert not s.counters["op"].components[0]["scatter"] @gen_cluster(client=True, config=NO_AMM) async def test_scatter_direct_broadcast(c, s, a, b): - future2 = await c.scatter(456, direct=True, broadcast=True) - assert future2.key in a.data - assert future2.key in b.data - assert s.tasks[future2.key].who_has == {s.workers[a.address], s.workers[b.address]} - result = await future2 + task2 = await c.scatter(456, direct=True, broadcast=True) + assert task2.key in a.data + assert task2.key in b.data + assert s.tasks[task2.key].who_has == {s.workers[a.address], s.workers[b.address]} + result = await task2 assert result == 456 assert not s.counters["op"].components[0]["scatter"] @gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 4) async def test_scatter_direct_balanced(c, s, *workers): - futures = await c.scatter([1, 2, 3], direct=True) + tasks = await c.scatter([1, 2, 3], direct=True) assert sorted(len(w.data) for w in workers) == [0, 1, 1, 1] @gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 4, config=NO_AMM) async def test_scatter_direct_broadcast_target(c, s, *workers): - futures = await c.scatter([123, 456], direct=True, workers=workers[0].address) - assert futures[0].key in workers[0].data - assert futures[1].key in workers[0].data + tasks = await c.scatter([123, 456], direct=True, workers=workers[0].address) + assert tasks[0].key in workers[0].data + assert tasks[1].key in workers[0].data - futures = await c.scatter( + tasks = await c.scatter( [123, 456], direct=True, broadcast=True, @@ -1550,7 +1550,7 @@ async def test_scatter_direct_broadcast_target(c, s, *workers): ) assert ( f.key in w.data and w.address in s.tasks[f.key].who_has - for f in futures + for f in tasks for w in workers[:3] ) @@ -1563,10 +1563,10 @@ async def test_scatter_direct_empty(c, s): @gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 5) async def test_scatter_direct_spread_evenly(c, s, *workers): - futures = [] + tasks = [] for i in range(10): - future = await c.scatter(i, direct=True) - futures.append(future) + task = await c.scatter(i, direct=True) + tasks.append(task) assert all(w.data for w in workers) @@ -1574,8 +1574,8 @@ async def test_scatter_direct_spread_evenly(c, s, *workers): @pytest.mark.parametrize("direct", [True, False]) @pytest.mark.parametrize("broadcast", [True, False]) def test_scatter_gather_sync(c, direct, broadcast): - futures = c.scatter([1, 2, 3], direct=direct, broadcast=broadcast) - results = c.gather(futures, direct=direct) + tasks = c.scatter([1, 2, 3], direct=direct, broadcast=broadcast) + results = c.gather(tasks, direct=direct) assert results == [1, 2, 3] delayed(inc)(1).compute(direct=direct) @@ -1583,9 +1583,9 @@ def test_scatter_gather_sync(c, direct, broadcast): @gen_cluster(client=True) async def test_gather_direct(c, s, a, b): - futures = await c.scatter([1, 2, 3]) + tasks = await c.scatter([1, 2, 3]) - data = await c.gather(futures, direct=True) + data = await c.gather(tasks, direct=True) assert data == [1, 2, 3] @@ -1883,14 +1883,14 @@ async def test_async_compute(c, s, a, b): z = delayed(dec)(x) [yy, zz, aa] = c.compute([y, z, 3], sync=False) - assert isinstance(yy, Future) - assert isinstance(zz, Future) + assert isinstance(yy, Task) + assert isinstance(zz, Task) assert aa == 3 result = await c.gather([yy, zz]) assert result == [2, 0] - assert isinstance(c.compute(y), Future) + assert isinstance(c.compute(y), Task) assert isinstance(c.compute([y]), (tuple, list)) @@ -2046,18 +2046,18 @@ def bad(x): @gen_cluster(client=True) -async def test_map_on_futures_with_kwargs(c, s, a, b): +async def test_map_on_tasks_with_kwargs(c, s, a, b): def f(x, y=10): return x + y - futures = c.map(inc, range(10)) - futures2 = c.map(f, futures, y=20) - results = await c.gather(futures2) + tasks = c.map(inc, range(10)) + tasks2 = c.map(f, tasks, y=20) + results = await c.gather(tasks2) assert results == [i + 1 + 20 for i in range(10)] - future = c.submit(inc, 100) - future2 = c.submit(f, future, y=200) - result = await future2 + task = c.submit(inc, 100) + task2 = c.submit(f, task, y=200) + result = await task2 assert result == 100 + 1 + 200 @@ -2073,7 +2073,7 @@ def __setstate__(self, state): @gen_test() async def test_badly_serialized_input_stderr(capsys, c): o = BadlySerializedObject() - future = c.submit(inc, o) + task = c.submit(inc, o) while True: sleep(0.01) @@ -2081,7 +2081,7 @@ async def test_badly_serialized_input_stderr(capsys, c): if "hello!" in err: break - assert future.status == "error" + assert task.status == "error" @pytest.mark.parametrize( @@ -2459,7 +2459,7 @@ def test_cancel_sync(c): @gen_cluster(client=True) -async def test_future_type(c, s, a, b): +async def test_task_type(c, s, a, b): x = c.submit(inc, 1) await wait([x]) assert x.type == int @@ -2541,7 +2541,7 @@ async def test__persist(c, s, a, b): assert len(y.dask) == 6 assert len(yy.dask) == 2 - assert all(isinstance(v, Future) for v in yy.dask.values()) + assert all(isinstance(v, Task) for v in yy.dask.values()) assert yy.__dask_keys__() == y.__dask_keys__() g, h = c.compute([y, yy]) @@ -2560,7 +2560,7 @@ def test_persist(c): yy = c.persist(y) assert len(y.dask) == 6 assert len(yy.dask) == 2 - assert all(isinstance(v, Future) for v in yy.dask.values()) + assert all(isinstance(v, Task) for v in yy.dask.values()) assert yy.__dask_keys__() == y.__dask_keys__() zz = yy.compute() @@ -2580,8 +2580,8 @@ def deep(n): x = c.submit(deep, 200) await wait([x]) - assert len(dumps(c.futures[x.key].traceback)) < 10000 - assert isinstance(c.futures[x.key].exception, ZeroDivisionError) + assert len(dumps(c.tasks[x.key].traceback)) < 10000 + assert isinstance(c.tasks[x.key].exception, ZeroDivisionError) @gen_cluster(client=True) @@ -2594,33 +2594,33 @@ async def test_wait_on_collections(c, s, a, b): @gen_cluster(client=True) -async def test_futures_of_get(c, s, a, b): +async def test_tasks_of_get(c, s, a, b): x, y, z = c.map(inc, [1, 2, 3]) - assert set(futures_of(0)) == set() - assert set(futures_of(x)) == {x} - assert set(futures_of([x, y, z])) == {x, y, z} - assert set(futures_of([x, [y], [[z]]])) == {x, y, z} - assert set(futures_of({"x": x, "y": [y]})) == {x, y} + assert set(tasks_of(0)) == set() + assert set(tasks_of(x)) == {x} + assert set(tasks_of([x, y, z])) == {x, y, z} + assert set(tasks_of([x, [y], [[z]]])) == {x, y, z} + assert set(tasks_of({"x": x, "y": [y]})) == {x, y} b = db.Bag({("b", i): f for i, f in enumerate([x, y, z])}, "b", 3) - assert set(futures_of(b)) == {x, y, z} + assert set(tasks_of(b)) == {x, y, z} sg = SubgraphCallable( {"x": x, "y": y, "z": z, "out": (add, (add, (add, x, y), z), "in")}, "out", ("in",), ) - assert set(futures_of(sg)) == {x, y, z} + assert set(tasks_of(sg)) == {x, y, z} -def test_futures_of_class(): +def test_tasks_of_class(): da = pytest.importorskip("dask.array") - assert futures_of([da.Array]) == [] + assert tasks_of([da.Array]) == [] @gen_cluster(client=True) -async def test_futures_of_cancelled_raises(c, s, a, b): +async def test_tasks_of_cancelled_raises(c, s, a, b): x = c.submit(inc, 1) await c.cancel([x]) @@ -2688,8 +2688,8 @@ async def test_run(c, s, a, b): @gen_cluster(client=True) async def test_run_handles_picklable_data(c, s, a, b): - futures = c.map(inc, range(10)) - await wait(futures) + tasks = c.map(inc, range(10)) + await wait(tasks) def func(): return {}, set(), [], (), 1, "hello", b"100" @@ -2853,11 +2853,11 @@ async def test_worker_aliases(c, s): await asyncio.gather(a, b, w) L = c.map(inc, range(10), workers="alice") - future = await c.scatter(123, workers=3) + task = await c.scatter(123, workers=3) await wait(L) assert len(a.data) == 10 assert len(b.data) == 0 - assert dict(w.data) == {future.key: 123} + assert dict(w.data) == {task.key: 123} for i, alias in enumerate([3, [3], "alice"]): result = await c.submit(lambda x: x + 1, i, workers=alias) @@ -2971,7 +2971,7 @@ async def test_rebalance(c, s, a, b): """Test Client.rebalance(). These are just to test the Client wrapper around Scheduler.rebalance(); for more thorough tests on the latter see test_scheduler.py. """ - futures = await c.scatter(range(100), workers=[a.address]) + tasks = await c.scatter(range(100), workers=[a.address]) assert len(a.data) == 100 assert len(b.data) == 0 await c.rebalance() @@ -2984,7 +2984,7 @@ async def test_rebalance_workers_and_keys(client, s, a, b, c): """Test Client.rebalance(). These are just to test the Client wrapper around Scheduler.rebalance(); for more thorough tests on the latter see test_scheduler.py. """ - futures = await client.scatter(range(100), workers=[a.address]) + tasks = await client.scatter(range(100), workers=[a.address]) assert (len(a.data), len(b.data), len(c.data)) == (100, 0, 0) # Passing empty iterables is not the same as omitting the arguments @@ -2993,7 +2993,7 @@ async def test_rebalance_workers_and_keys(client, s, a, b, c): assert (len(a.data), len(b.data), len(c.data)) == (100, 0, 0) # Limit rebalancing to two arbitrary keys and two arbitrary workers. - await client.rebalance([futures[3], futures[7]], [a.address, b.address]) + await client.rebalance([tasks[3], tasks[7]], [a.address, b.address]) assert (len(a.data), len(b.data), len(c.data)) == (98, 2, 0) with pytest.raises(KeyError): @@ -3008,7 +3008,7 @@ def test_rebalance_sync(loop): s = c.cluster.scheduler a = c.cluster.workers[0] b = c.cluster.workers[1] - futures = c.scatter(range(100), workers=[a.address]) + tasks = c.scatter(range(100), workers=[a.address]) assert len(a.data) == 100 assert len(b.data) == 0 @@ -3019,22 +3019,22 @@ def test_rebalance_sync(loop): @gen_cluster(client=True, config=NO_AMM) async def test_rebalance_unprepared(c, s, a, b): - """Client.rebalance() internally waits for unfinished futures""" - futures = c.map(slowinc, range(10), delay=0.05, workers=a.address) - # Let the futures reach the scheduler + """Client.rebalance() internally waits for unfinished tasks""" + tasks = c.map(slowinc, range(10), delay=0.05, workers=a.address) + # Let the tasks reach the scheduler await asyncio.sleep(0.1) - # We didn't wait enough for futures to complete. However, Client.rebalance() will - # block until all futures are completed before invoking Scheduler.rebalance(). - await c.rebalance(futures) + # We didn't wait enough for tasks to complete. However, Client.rebalance() will + # block until all tasks are completed before invoking Scheduler.rebalance(). + await c.rebalance(tasks) s.validate_state() @gen_cluster(client=True, config=NO_AMM) async def test_rebalance_raises_on_explicit_missing_data(c, s, a, b): - """rebalance() raises KeyError if explicitly listed futures disappear""" - f = Future("x", client=c, state="memory") + """rebalance() raises KeyError if explicitly listed tasks disappear""" + f = Task("x", client=c, state="memory") with pytest.raises(KeyError, match="Could not rebalance keys:"): - await c.rebalance(futures=[f]) + await c.rebalance(tasks=[f]) @gen_cluster(client=True) @@ -3071,9 +3071,9 @@ async def test_unrunnable_task_runs(c, s, a, b): @gen_cluster(client=True, nthreads=[]) async def test_add_worker_after_tasks(c, s): - futures = c.map(inc, range(10)) + tasks = c.map(inc, range(10)) async with Nanny(s.address, nthreads=2): - await c.gather(futures) + await c.gather(tasks) @pytest.mark.skipif(not LINUX, reason="Need 127.0.0.2 to mean localhost") @@ -3089,7 +3089,7 @@ async def test_workers_register_indirect_data(c, s, a, b): @gen_cluster(client=True) -async def test_submit_on_cancelled_future(c, s, a, b): +async def test_submit_on_cancelled_task(c, s, a, b): x = c.submit(inc, 1) await x @@ -3189,10 +3189,10 @@ def __getstate__(self): ) async def test_replicate_tree_branching(c, s, *workers): obj = CountSerialization() - [future] = await c.scatter([obj]) - await s.replicate(keys=[future.key], n=10) + [task] = await c.scatter([obj]) + await s.replicate(keys=[task.key], n=10) - max_count = max(w.data[future.key].n for w in workers) + max_count = max(w.data[task.key].n for w in workers) assert max_count > 1 @@ -3262,12 +3262,12 @@ def test_client_replicate_sync(client_no_amm): @pytest.mark.skipif(WINDOWS, reason="Windows timer too coarse-grained") @gen_cluster(client=True, nthreads=[("127.0.0.1", 4)] * 1) async def test_task_load_adapts_quickly(c, s, a): - future = c.submit(slowinc, 1, delay=0.2) # slow - await wait(future) + task = c.submit(slowinc, 1, delay=0.2) # slow + await wait(task) assert 0.15 < s.task_prefixes["slowinc"].duration_average < 0.4 - futures = c.map(slowinc, range(10), delay=0) # very fast - await wait(futures) + tasks = c.map(slowinc, range(10), delay=0) # very fast + await wait(tasks) assert 0 < s.task_prefixes["slowinc"].duration_average < 0.1 @@ -3278,10 +3278,10 @@ async def test_even_load_after_fast_functions(c, s, a, b): y = c.submit(inc, 2, workers=b.address) # very fast await wait([x, y]) - futures = c.map(inc, range(2, 11)) - await wait(futures) - assert any(f.key in a.data for f in futures) - assert any(f.key in b.data for f in futures) + tasks = c.map(inc, range(2, 11)) + await wait(tasks) + assert any(f.key in a.data for f in tasks) + assert any(f.key in b.data for f in tasks) # assert abs(len(a.data) - len(b.data)) <= 3 @@ -3324,8 +3324,8 @@ async def test_balanced_with_submit_and_resident_data(c, s, *workers): @gen_cluster(client=True, nthreads=[("127.0.0.1", 20)] * 2) async def test_scheduler_saturates_cores(c, s, a, b): for delay in [0, 0.01, 0.1]: - futures = c.map(slowinc, range(100), delay=delay) - futures = c.map(slowinc, futures, delay=delay / 10) + tasks = c.map(slowinc, range(100), delay=delay) + tasks = c.map(slowinc, tasks, delay=delay / 10) while not s.tasks: if s.tasks: assert all( @@ -3338,7 +3338,7 @@ async def test_scheduler_saturates_cores(c, s, a, b): @gen_cluster(client=True, nthreads=[("127.0.0.1", 20)] * 2) async def test_scheduler_saturates_cores_random(c, s, a, b): - futures = c.map(randominc, range(100), scale=0.1) + tasks = c.map(randominc, range(100), scale=0.1) while not s.tasks: if s.tasks: assert all( @@ -3516,23 +3516,23 @@ async def test_set_as_default(s, a, b): @gen_cluster(client=True) async def test_get_foo(c, s, a, b): - futures = c.map(inc, range(10)) - await wait(futures) + tasks = c.map(inc, range(10)) + await wait(tasks) x = await c.scheduler.nbytes(summary=False) assert x == s.get_nbytes(summary=False) - x = await c.scheduler.nbytes(keys=[futures[0].key], summary=False) - assert x == {futures[0].key: s.tasks[futures[0].key].nbytes} + x = await c.scheduler.nbytes(keys=[tasks[0].key], summary=False) + assert x == {tasks[0].key: s.tasks[tasks[0].key].nbytes} x = await c.scheduler.who_has() assert valmap(set, x) == { key: {ws.address for ws in ts.who_has} for key, ts in s.tasks.items() } - x = await c.scheduler.who_has(keys=[futures[0].key]) + x = await c.scheduler.who_has(keys=[tasks[0].key]) assert valmap(set, x) == { - futures[0].key: {ws.address for ws in s.tasks[futures[0].key].who_has} + tasks[0].key: {ws.address for ws in s.tasks[tasks[0].key].who_has} } @@ -3597,7 +3597,7 @@ def test_get_processing_sync(c, s, a, b): processing = c.processing() assert not any(v for v in processing.values()) - futures = c.map( + tasks = c.map( slowinc, range(10), delay=0.1, workers=[a["address"]], allow_other_workers=False ) @@ -3610,7 +3610,7 @@ def test_get_processing_sync(c, s, a, b): assert set(c.processing(aa)) == {aa} assert set(c.processing([aa])) == {aa} - c.cancel(futures) + c.cancel(tasks) def test_close_idempotent(c): @@ -3629,7 +3629,7 @@ def block(ev): result = c.get({"x": (throws, 1), "y": (block, event)}, ["x", "y"]) # Futures should be released and forgotten - poll_for(lambda: not c.futures, timeout=1) + poll_for(lambda: not c.tasks, timeout=1) event.set() poll_for(lambda: not any(c.processing().values()), timeout=3) @@ -3638,7 +3638,7 @@ def block(ev): with pytest.raises(RuntimeError): result = c.get({"x": (throws, 1), x.key: (inc, 1)}, ["x", x.key]) - assert x.key in c.futures + assert x.key in c.tasks @pytest.mark.slow @@ -3646,13 +3646,13 @@ def block(ev): async def test_Client_clears_references_after_restart(c, s, a, b): x = c.submit(inc, 1) assert x.key in c.refcount - assert x.key in c.futures + assert x.key in c.tasks with pytest.raises(TimeoutError): await c.restart(timeout=5) assert x.key not in c.refcount - assert not c.futures + assert not c.tasks key = x.key del x @@ -3687,9 +3687,9 @@ def test_as_completed_results(c): @pytest.mark.parametrize("with_results", [True, False]) def test_as_completed_batches(c, with_results): n = 50 - futures = c.map(slowinc, range(n), delay=0.01) + tasks = c.map(slowinc, range(n), delay=0.01) out = [] - for batch in as_completed(futures, with_results=with_results).batches(): + for batch in as_completed(tasks, with_results=with_results).batches(): assert isinstance(batch, (tuple, list)) sleep(0.05) out.extend(batch) @@ -3698,17 +3698,17 @@ def test_as_completed_batches(c, with_results): if with_results: assert set(pluck(1, out)) == set(range(1, n + 1)) else: - assert set(out) == set(futures) + assert set(out) == set(tasks) def test_as_completed_next_batch(c): - futures = c.map(slowinc, range(2), delay=0.1) - ac = as_completed(futures) + tasks = c.map(slowinc, range(2), delay=0.1) + ac = as_completed(tasks) assert not ac.is_empty() assert ac.next_batch(block=False) == [] - assert set(ac.next_batch(block=True)).issubset(futures) + assert set(ac.next_batch(block=True)).issubset(tasks) while not ac.is_empty(): - assert set(ac.next_batch(block=True)).issubset(futures) + assert set(ac.next_batch(block=True)).issubset(tasks) assert ac.is_empty() assert not ac.has_ready() @@ -4026,8 +4026,8 @@ def test_threaded_get_within_distributed(c): def f(get): return get({"x": (lambda: 1,)}, "x") - future = c.submit(f, get) - assert future.result() == 1 + task = c.submit(f, get) + assert task.result() == 1 @gen_cluster(client=True) @@ -4140,27 +4140,27 @@ async def test_scatter_compute_store_lose_processing(c, s, a, b): @gen_cluster() -async def test_serialize_future(s, a, b): +async def test_serialize_task(s, a, b): async with Client(s.address, asynchronous=True) as c1, Client( s.address, asynchronous=True ) as c2: - future = c1.submit(lambda: 1) - result = await future + task = c1.submit(lambda: 1) + result = await task for ci in (c1, c2): with ensure_no_new_clients(): with ci.as_current(): - future2 = pickle.loads(pickle.dumps(future)) - assert future2.client is ci - assert future2.key in ci.futures - result2 = await future2 + task2 = pickle.loads(pickle.dumps(task)) + assert task2.client is ci + assert task2.key in ci.tasks + result2 = await task2 assert result == result2 with temp_default_client(ci): - future2 = pickle.loads(pickle.dumps(future)) + task2 = pickle.loads(pickle.dumps(task)) @gen_cluster() -async def test_serialize_future_without_client(s, a, b): +async def test_serialize_task_without_client(s, a, b): # Do not use a ctx manager to avoid having this being set as a current and/or default client c1 = await Client(s.address, asynchronous=True, set_as_default=False) try: @@ -4169,8 +4169,8 @@ async def test_serialize_future_without_client(s, a, b): def do_stuff(): return 1 - future = c1.submit(do_stuff) - pickled = pickle.dumps(future) + task = c1.submit(do_stuff) + pickled = pickle.dumps(task) unpickled_fut = pickle.loads(pickled) with pytest.raises(RuntimeError): @@ -4402,8 +4402,8 @@ async def test_compute_nested_containers(c, s, a, b): np = pytest.importorskip("numpy") x = da.ones(10, chunks=(5,)) + 1 - future = c.compute({"x": [x], "y": 123}) - result = await future + task = c.compute({"x": [x], "y": 123}) + result = await task assert isinstance(result, dict) assert (result["x"][0] == np.ones(10) + 1).all() @@ -4412,8 +4412,8 @@ async def test_compute_nested_containers(c, s, a, b): @gen_cluster(client=True) async def test_scatter_type(c, s, a, b): - [future] = await c.scatter([1]) - assert future.type == int + [task] = await c.scatter([1]) + assert task.type == int d = await c.scatter({"x": 1.0}) assert d["x"].type == float @@ -4434,11 +4434,11 @@ async def test_retire_workers_2(c, s, a, b): @gen_cluster(client=True, nthreads=[("", 1)] * 10) async def test_retire_many_workers(c, s, *workers): - futures = await c.scatter(list(range(100))) + tasks = await c.scatter(list(range(100))) await s.retire_workers(workers=[w.address for w in workers[:7]]) - results = await c.gather(futures) + results = await c.gather(tasks) assert results == list(range(100)) while len(s.workers) != 3: @@ -4446,9 +4446,9 @@ async def test_retire_many_workers(c, s, *workers): assert len(s.workers) == 3 - assert all(future.done() for future in futures) - assert all(s.tasks[future.key].state == "memory" for future in futures) - assert await c.gather(futures) == list(range(100)) + assert all(task.done() for task in tasks) + assert all(s.tasks[task.key].state == "memory" for task in tasks) + assert await c.gather(tasks) == list(range(100)) # Don't count how many task landed on each worker. # Normally, tasks would be distributed evenly over the surviving workers. However, @@ -4476,12 +4476,12 @@ def f(x, y=0, z=0): y = await c.scatter([[1, 2, 3, 4]], workers=[a.address]) z = await c.scatter([1], workers=[b.address]) - futures = c.map(f, [1, 2, 3, 4], y=y, z=z) + tasks = c.map(f, [1, 2, 3, 4], y=y, z=z) - await wait(futures) + await wait(tasks) - assert sum(f.key in a.data for f in futures) >= 2 - assert sum(f.key in b.data for f in futures) >= 1 + assert sum(f.key in a.data for f in tasks) >= 2 + assert sum(f.key in b.data for f in tasks) >= 1 @gen_cluster( @@ -4502,9 +4502,9 @@ def f(x, y=0): y = await c.scatter([1], broadcast=True) - futures = c.map(f, range(20), y=y) + tasks = c.map(f, range(20), y=y) - await wait(futures) + await wait(tasks) assert len(b.data) > 2 * len(a.data) @@ -4513,11 +4513,11 @@ def f(x, y=0): async def test_add_done_callback(c, s, a, b): S = set() - def f(future): - future.add_done_callback(g) + def f(task): + task.add_done_callback(g) - def g(future): - S.add((future.key, future.status)) + def g(task): + S.add((task.key, task.status)) u = c.submit(inc, 1, key="u") v = c.submit(throws, "hello", key="v") @@ -4548,7 +4548,7 @@ async def test_normalize_collection(c, s, a, b): zz = c.normalize_collection(z) assert len(z.dask) == len(y.dask) + 1 - assert isinstance(zz.dask[y.key], Future) + assert isinstance(zz.dask[y.key], Task) assert len(zz.dask) < len(z.dask) @@ -4566,7 +4566,7 @@ async def test_normalize_collection_dask_array(c, s, a, b): assert z.dask == zdsk # do not mutate input assert len(z.dask) > len(zz.dask) - assert any(isinstance(v, Future) for v in zz.dask.values()) + assert any(isinstance(v, Task) for v in zz.dask.values()) for k, v in yy.dask.items(): assert zz.dask[k].key == v.key @@ -4577,18 +4577,18 @@ async def test_normalize_collection_dask_array(c, s, a, b): @pytest.mark.slow -def test_normalize_collection_with_released_futures(c): +def test_normalize_collection_with_released_tasks(c): da = pytest.importorskip("dask.array") x = da.arange(2**20, chunks=2**10) y = x.persist() wait(y) sol = y.sum().compute() - # Start releasing futures + # Start releasing tasks del y - # Try to reuse futures. Previously this was a race condition, + # Try to reuse tasks. Previously this was a race condition, # and the call to `.compute()` would error out due to missing - # futures on the scheduler at compute time. + # tasks on the scheduler at compute time. normalized = c.normalize_collection(x) res = normalized.sum().compute() assert res == sol @@ -4602,15 +4602,15 @@ async def test_auto_normalize_collection(c, s, a, b): x = da.ones(10, chunks=5) assert len(x.dask) == 2 - with dask.config.set(optimizations=[c._optimize_insert_futures]): + with dask.config.set(optimizations=[c._optimize_insert_tasks]): y = x.map_blocks(inc, dtype=x.dtype) yy = c.persist(y) await wait(yy) start = time() - future = c.compute(y.sum()) - await future + task = c.compute(y.sum()) + await task end = time() assert end - start < 1 @@ -4631,7 +4631,7 @@ def test_auto_normalize_collection_sync(c): wait(yy) - with dask.config.set(optimizations=[c._optimize_insert_futures]): + with dask.config.set(optimizations=[c._optimize_insert_tasks]): start = time() y.sum().compute() end = time() @@ -4656,7 +4656,7 @@ async def test_interleave_computations(c, s, a, b): total = delayed(sum)(zs) - future = c.compute(total) + task = c.compute(total) done = ("memory", "released") @@ -4741,25 +4741,25 @@ async def run_scheduler_after_2_seconds(): @gen_cluster(client=True) async def test_submit_list_kwargs(c, s, a, b): - futures = await c.scatter([1, 2, 3]) + tasks = await c.scatter([1, 2, 3]) def f(L=None): return sum(L) - future = c.submit(f, L=futures) - result = await future + task = c.submit(f, L=tasks) + result = await task assert result == 1 + 2 + 3 @gen_cluster(client=True) async def test_map_list_kwargs(c, s, a, b): - futures = await c.scatter([1, 2, 3]) + tasks = await c.scatter([1, 2, 3]) def f(i, L=None): return i + sum(L) - futures = c.map(f, range(10), L=futures) - results = await c.gather(futures) + tasks = c.map(f, range(10), L=tasks) + results = await c.gather(tasks) assert results == [i + 6 for i in range(10)] @@ -4775,8 +4775,8 @@ async def test_recreate_error_delayed(c, s, a, b): assert f.status == "pending" - error_f = await c._get_errored_future(f) - function, args, kwargs = await c._get_components_from_future(error_f) + error_f = await c._get_errored_task(f) + function, args, kwargs = await c._get_components_from_task(error_f) assert f.status == "error" assert function.__name__ == "div" assert args == (1, 0) @@ -4785,7 +4785,7 @@ async def test_recreate_error_delayed(c, s, a, b): @gen_cluster(client=True) -async def test_recreate_error_futures(c, s, a, b): +async def test_recreate_error_tasks(c, s, a, b): x0 = c.submit(dec, 2) y0 = c.submit(dec, 1) x = c.submit(div, 1, x0) @@ -4795,8 +4795,8 @@ async def test_recreate_error_futures(c, s, a, b): assert f.status == "pending" - error_f = await c._get_errored_future(f) - function, args, kwargs = await c._get_components_from_future(error_f) + error_f = await c._get_errored_task(f) + function, args, kwargs = await c._get_components_from_task(error_f) assert f.status == "error" assert function.__name__ == "div" assert args == (1, 0) @@ -4811,8 +4811,8 @@ async def test_recreate_error_collection(c, s, a, b): b = b.persist() f = c.compute(b) - error_f = await c._get_errored_future(f) - function, args, kwargs = await c._get_components_from_future(error_f) + error_f = await c._get_errored_task(f) + function, args, kwargs = await c._get_components_from_task(error_f) with pytest.raises(ZeroDivisionError): function(*args, **kwargs) @@ -4829,15 +4829,15 @@ def make_err(x): df2 = df.a.map(make_err) f = c.compute(df2) - error_f = await c._get_errored_future(f) - function, args, kwargs = await c._get_components_from_future(error_f) + error_f = await c._get_errored_task(f) + function, args, kwargs = await c._get_components_from_task(error_f) with pytest.raises(ValueError): function(*args, **kwargs) # with persist df3 = c.persist(df2) - error_f = await c._get_errored_future(df3) - function, args, kwargs = await c._get_components_from_future(error_f) + error_f = await c._get_errored_task(df3) + function, args, kwargs = await c._get_components_from_task(error_f) with pytest.raises(ValueError): function(*args, **kwargs) @@ -4848,8 +4848,8 @@ async def test_recreate_error_array(c, s, a, b): pytest.importorskip("scipy") z = (da.linalg.inv(da.zeros((10, 10), chunks=10)) + 1).sum() zz = z.persist() - error_f = await c._get_errored_future(zz) - function, args, kwargs = await c._get_components_from_future(error_f) + error_f = await c._get_errored_task(zz) + function, args, kwargs = await c._get_components_from_task(error_f) assert "0.,0.,0." in str(args).replace(" ", "") # args contain actual arrays @@ -4868,7 +4868,7 @@ def test_recreate_error_sync(c): def test_recreate_error_not_error(c): f = c.submit(dec, 2) - with pytest.raises(ValueError, match="No errored futures passed"): + with pytest.raises(ValueError, match="No errored tasks passed"): c.recreate_error_locally(f) @@ -4884,7 +4884,7 @@ async def test_recreate_task_delayed(c, s, a, b): assert f.status == "pending" - function, args, kwargs = await c._get_components_from_future(f) + function, args, kwargs = await c._get_components_from_task(f) assert f.status == "finished" assert function.__name__ == "sum" assert args == ([1, 1],) @@ -4892,7 +4892,7 @@ async def test_recreate_task_delayed(c, s, a, b): @gen_cluster(client=True) -async def test_recreate_task_futures(c, s, a, b): +async def test_recreate_task_tasks(c, s, a, b): x0 = c.submit(dec, 2) y0 = c.submit(dec, 2) x = c.submit(div, 1, x0) @@ -4902,7 +4902,7 @@ async def test_recreate_task_futures(c, s, a, b): assert f.status == "pending" - function, args, kwargs = await c._get_components_from_future(f) + function, args, kwargs = await c._get_components_from_task(f) assert f.status == "finished" assert function.__name__ == "sum" assert args == ([1, 1],) @@ -4916,7 +4916,7 @@ async def test_recreate_task_collection(c, s, a, b): b = b.persist() f = c.compute(b) - function, args, kwargs = await c._get_components_from_future(f) + function, args, kwargs = await c._get_components_from_task(f) assert function(*args, **kwargs) == [ 3628800, 1814400, @@ -4938,18 +4938,18 @@ async def test_recreate_task_collection(c, s, a, b): df2 = df.a.map(lambda x: x + 1) f = c.compute(df2) - function, args, kwargs = await c._get_components_from_future(f) + function, args, kwargs = await c._get_components_from_task(f) expected = pd.DataFrame({"a": [1, 2, 3, 4, 5]})["a"] assert function(*args, **kwargs).equals(expected) # with persist df3 = c.persist(df2) - # recreate_task_locally only works with futures + # recreate_task_locally only works with tasks with pytest.raises(TypeError, match="key"): - function, args, kwargs = await c._get_components_from_future(df3) + function, args, kwargs = await c._get_components_from_task(df3) f = c.compute(df3) - function, args, kwargs = await c._get_components_from_future(f) + function, args, kwargs = await c._get_components_from_task(f) assert function(*args, **kwargs).equals(expected) @@ -4958,7 +4958,7 @@ async def test_recreate_task_array(c, s, a, b): da = pytest.importorskip("dask.array") z = (da.zeros((10, 10), chunks=10) + 1).sum() f = c.compute(z) - function, args, kwargs = await c._get_components_from_future(f) + function, args, kwargs = await c._get_components_from_task(f) assert function(*args, **kwargs) == 100 @@ -4997,7 +4997,7 @@ async def test_restart_workers(c, s, a, b): b_start_time = results[b.worker_address] assert set(s.workers) == {a.worker_address, b.worker_address} - # Persist futures and perform a computation + # Persist tasks and perform a computation da = pytest.importorskip("dask.array") size = 100 x = da.ones(size, chunks=10) @@ -5095,10 +5095,10 @@ def __getstate__(self): raise MyException() with pytest.raises(TypeError, match="Could not serialize"): - future = c.submit(identity, Foo()) + task = c.submit(identity, Foo()) - futures = c.map(inc, range(10)) - results = await c.gather(futures) + tasks = c.map(inc, range(10)) + results = await c.gather(tasks) assert results == list(map(inc, range(10))) assert a.data and b.data @@ -5113,14 +5113,14 @@ def __getstate__(self): def __setstate__(self, state): raise MyException("hello") - future = c.submit(identity, Foo()) - await wait(future) - assert future.status == "error" + task = c.submit(identity, Foo()) + await wait(task) + assert task.status == "error" with raises_with_cause(RuntimeError, "deserialization", MyException, "hello"): - await future + await task - futures = c.map(inc, range(10)) - results = await c.gather(futures) + tasks = c.map(inc, range(10)) + results = await c.gather(tasks) assert results == list(map(inc, range(10))) assert a.data and b.data @@ -5138,14 +5138,14 @@ def __setstate__(self, state): def __call__(self, *args): return 1 - future = c.submit(Foo(), 1) - await wait(future) - assert future.status == "error" + task = c.submit(Foo(), 1) + await wait(task) + assert task.status == "error" with raises_with_cause(RuntimeError, "deserialization", MyException, "hello"): - await future + await task - futures = c.map(inc, range(10)) - results = await c.gather(futures) + tasks = c.map(inc, range(10)) + results = await c.gather(tasks) assert results == list(map(inc, range(10))) assert a.data and b.data @@ -5153,14 +5153,14 @@ def __call__(self, *args): @gen_cluster(client=True) async def test_fire_and_forget(c, s, a, b): - future = c.submit(slowinc, 1, delay=0.1) + task = c.submit(slowinc, 1, delay=0.1) import distributed def f(x): distributed.foo = 123 try: - fire_and_forget(c.submit(f, future)) + fire_and_forget(c.submit(f, task)) while not hasattr(distributed, "foo"): await asyncio.sleep(0.01) @@ -5171,8 +5171,8 @@ def f(x): while len(s.tasks) > 1: await asyncio.sleep(0.01) - assert set(s.tasks) == {future.key} - assert s.tasks[future.key].who_wants + assert set(s.tasks) == {task.key} + assert s.tasks[task.key].who_wants @gen_cluster(client=True) @@ -5195,7 +5195,7 @@ def test_quiet_client_close(loop): dashboard_address=":0", threads_per_worker=4, ) as c: - futures = c.map(slowinc, range(1000), delay=0.01) + tasks = c.map(slowinc, range(1000), delay=0.01) sleep(0.200) # stop part-way sleep(0.1) # let things settle @@ -5229,8 +5229,8 @@ def test_quiet_client_close_when_cluster_is_closed_before_client(loop): @gen_cluster() async def test_close(s, a, b): async with Client(s.address, asynchronous=True) as c: - future = c.submit(inc, 1) - await wait(future) + task = c.submit(inc, 1) + await wait(task) assert c.id in s.clients await c.close() @@ -5242,8 +5242,8 @@ def test_threadsafe(c): def f(_): d = deque(maxlen=50) for _ in range(100): - future = c.submit(inc, random.randint(0, 100)) - d.append(future) + task = c.submit(inc, random.randint(0, 100)) + d.append(task) sleep(0.001) c.gather(list(d)) total = c.submit(sum, list(d)) @@ -5284,8 +5284,8 @@ def test_threadsafe_compute(c): def f(_): total = 0 for _ in range(20): - future = c.compute((x + random.randint(0, 20)).sum()) - total += future.result() + task = c.compute((x + random.randint(0, 20)).sum()) + total += task.result() sleep(0.001) return total @@ -5316,15 +5316,15 @@ def f(x): assert not client.asynchronous assert client is distributed.tmp_client - future = client.submit(inc, x) - return future.result() + task = client.submit(inc, x) + return task.result() import distributed distributed.tmp_client = c try: - futures = c.map(f, range(5)) - results = await c.gather(futures) + tasks = c.map(f, range(5)) + results = await c.gather(tasks) assert results == list(map(inc, range(5))) finally: del distributed.tmp_client @@ -5349,8 +5349,8 @@ def f(x): assert isinstance(x, da.Array) return x.sum().compute() - future = c.submit(f, x) - result = await future + task = c.submit(f, x) + result = await task assert result == sum(range(10)) @@ -5373,13 +5373,13 @@ async def test_secede_balances(c, s, a, b): def f(x): client = get_client() secede() - futures = client.map(inc, range(10), pure=False) - total = client.submit(sum, futures).result() + tasks = client.map(inc, range(10), pure=False) + total = client.submit(sum, tasks).result() return total - futures = c.map(f, range(10), workers=[a.address]) + tasks = c.map(f, range(10), workers=[a.address]) - results = await c.gather(futures) + results = await c.gather(tasks) # We dispatch 10 tasks and every task generates 11 more tasks # 10 * 11 + 10 assert a.state.executed_count + b.state.executed_count == 120 @@ -5492,7 +5492,7 @@ def func(): f = client.submit(slowinc, 1, delay=0.5, key="slowinc") client.gather(f) - future = c.submit(func, key="f") + task = c.submit(func, key="f") while len(s.tasks) != 2: await asyncio.sleep(0.001) # lower values schedule first @@ -5519,35 +5519,35 @@ def test_get_client_sync(c, s, a, b): @gen_cluster(client=True) -async def test_serialize_collections_of_futures(c, s, a, b): +async def test_serialize_collections_of_tasks(c, s, a, b): pd = pytest.importorskip("pandas") dd = pytest.importorskip("dask.dataframe") from dask.dataframe.utils import assert_eq df = pd.DataFrame({"x": [1, 2, 3]}) ddf = dd.from_pandas(df, npartitions=2).persist() - future = await c.scatter(ddf) + task = await c.scatter(ddf) - ddf2 = await future + ddf2 = await task df2 = await c.compute(ddf2) assert_eq(df, df2) -def test_serialize_collections_of_futures_sync(c): +def test_serialize_collections_of_tasks_sync(c): pd = pytest.importorskip("pandas") dd = pytest.importorskip("dask.dataframe") from dask.dataframe.utils import assert_eq df = pd.DataFrame({"x": [1, 2, 3]}) ddf = dd.from_pandas(df, npartitions=2).persist() - future = c.scatter(ddf) + task = c.scatter(ddf) - result = future.result() + result = task.result() assert_eq(result.compute(), df) - assert future.type == dd.DataFrame - assert c.submit(lambda x, y: assert_eq(x.compute(), y), future, df).result() + assert task.type == dd.DataFrame + assert c.submit(lambda x, y: assert_eq(x.compute(), y), task, df).result() def _dynamic_workload(x, delay=0.01): @@ -5559,30 +5559,30 @@ def _dynamic_workload(x, delay=0.01): return 4 secede() client = get_client() - futures = client.map( + tasks = client.map( _dynamic_workload, [x + i + 1 for i in range(2)], pure=False, delay=delay ) - total = client.submit(sum, futures) + total = client.submit(sum, tasks) return total.result() def test_dynamic_workloads_sync(c): - future = c.submit(_dynamic_workload, 0, delay=0.02) - assert future.result(timeout=20) == 52 + task = c.submit(_dynamic_workload, 0, delay=0.02) + assert task.result(timeout=20) == 52 @pytest.mark.slow def test_dynamic_workloads_sync_random(c): - future = c.submit(_dynamic_workload, 0, delay="random") - assert future.result(timeout=20) == 52 + task = c.submit(_dynamic_workload, 0, delay="random") + assert task.result(timeout=20) == 52 @gen_cluster(client=True) async def test_bytes_keys(c, s, a, b): key = b"inc-123" - future = c.submit(inc, 1, key=key) - result = await future - assert type(future.key) is bytes + task = c.submit(inc, 1, key=key) + result = await task + assert type(task.key) is bytes assert set(s.tasks) == {key} assert key in a.data or key in b.data assert result == 2 @@ -5592,9 +5592,9 @@ async def test_bytes_keys(c, s, a, b): async def test_unicode_ascii_keys(c, s, a, b): uni_type = str key = "inc-123" - future = c.submit(inc, 1, key=key) - result = await future - assert type(future.key) is uni_type + task = c.submit(inc, 1, key=key) + result = await task + assert type(task.key) is uni_type assert set(s.tasks) == {key} assert key in a.data or key in b.data assert result == 2 @@ -5604,19 +5604,19 @@ async def test_unicode_ascii_keys(c, s, a, b): async def test_unicode_keys(c, s, a, b): uni_type = str key = "inc-123\u03bc" - future = c.submit(inc, 1, key=key) - result = await future - assert type(future.key) is uni_type + task = c.submit(inc, 1, key=key) + result = await task + assert type(task.key) is uni_type assert set(s.tasks) == {key} assert key in a.data or key in b.data assert result == 2 - future2 = c.submit(inc, future) - result2 = await future2 + task2 = c.submit(inc, task) + result2 = await task2 assert result2 == 3 - future3 = await c.scatter({"data-123": 123}) - result3 = await future3["data-123"] + task3 = await c.scatter({"data-123": 123}) + result3 = await task3["data-123"] assert result3 == 123 @@ -5636,7 +5636,7 @@ def test_quiet_quit_when_cluster_leaves(loop_in_thread): with LocalCluster(loop=loop, dashboard_address=":0", silence_logs=False) as cluster: with captured_logger("distributed.comm") as sio: with Client(cluster, loop=loop) as client: - futures = client.map(lambda x: x + 1, range(10)) + tasks = client.map(lambda x: x + 1, range(10)) sleep(0.05) cluster.close() sleep(0.05) @@ -5646,37 +5646,35 @@ def test_quiet_quit_when_cluster_leaves(loop_in_thread): @gen_cluster([("127.0.0.1", 4)] * 2, client=True) -async def test_call_stack_future(c, s, a, b): +async def test_call_stack_task(c, s, a, b): x = c.submit(slowdec, 1, delay=0.5) - future = c.submit(slowinc, 1, delay=0.5) + task = c.submit(slowinc, 1, delay=0.5) await asyncio.sleep(0.1) - results = await asyncio.gather( - c.call_stack(future), c.call_stack(keys=[future.key]) - ) - assert all(list(first(result.values())) == [future.key] for result in results) + results = await asyncio.gather(c.call_stack(task), c.call_stack(keys=[task.key])) + assert all(list(first(result.values())) == [task.key] for result in results) assert results[0] == results[1] result = results[0] - ts = a.state.tasks.get(future.key) + ts = a.state.tasks.get(task.key) if ts is not None and ts.state == "executing": w = a else: w = b assert list(result) == [w.address] - assert list(result[w.address]) == [future.key] + assert list(result[w.address]) == [task.key] assert "slowinc" in str(result) assert "slowdec" not in str(result) @gen_cluster([("127.0.0.1", 4)] * 2, client=True) async def test_call_stack_all(c, s, a, b): - future = c.submit(slowinc, 1, delay=0.8) + task = c.submit(slowinc, 1, delay=0.8) while not a.state.executing_count and not b.state.executing_count: await asyncio.sleep(0.01) result = await c.call_stack() w = a if a.state.executing_count else b assert list(result) == [w.address] - assert list(result[w.address]) == [future.key] + assert list(result[w.address]) == [task.key] assert "slowinc" in str(result) @@ -5709,8 +5707,8 @@ async def test_call_stack_collections_all(c, s, a, b): }, ) async def test_profile(c, s, a, b): - futures = c.map(slowinc, range(10), delay=0.05, workers=a.address) - await wait(futures) + tasks = c.map(slowinc, range(10), delay=0.05, workers=a.address) + await wait(tasks) x = await c.profile(start=time() + 10, stop=time() + 20) assert not x["count"] @@ -5737,8 +5735,8 @@ async def test_profile(c, s, a, b): }, ) async def test_profile_disabled(c, s, a, b): - futures = c.map(slowinc, range(10), delay=0.05, workers=a.address) - await wait(futures) + tasks = c.map(slowinc, range(10), delay=0.05, workers=a.address) + await wait(tasks) x = await c.profile(start=time() + 10, stop=time() + 20) assert x["count"] == 0 @@ -5785,23 +5783,23 @@ async def test_client_with_name(s, a, b): @gen_cluster(client=True) -async def test_future_defaults_to_default_client(c, s, a, b): +async def test_task_defaults_to_default_client(c, s, a, b): x = c.submit(inc, 1) await wait(x) - future = Future(x.key) - assert future.client is c + task = Task(x.key) + assert task.client is c @gen_cluster(client=True) -async def test_future_auto_inform(c, s, a, b): +async def test_task_auto_inform(c, s, a, b): x = c.submit(inc, 1) await wait(x) async with Client(s.address, asynchronous=True) as client: - future = Future(x.key, client) + task = Task(x.key, client) - while future.status != "finished": + while task.status != "finished": await asyncio.sleep(0.01) @@ -5825,8 +5823,8 @@ def fib(x): c = a + b return c.compute() - future = c.submit(fib, 8) - result = await future + task = c.submit(fib, 8) + result = await task assert result == 21 assert len(s.transition_log) > 50 @@ -5856,14 +5854,14 @@ async def test_task_metadata(c, s, a, b): with pytest.raises(TypeError): await c.get_metadata(["x", "y"], None) - future = c.submit(inc, 1) - key = future.key - await wait(future) + task = c.submit(inc, 1) + key = task.key + await wait(task) await c.set_metadata(key, 123) result = await c.get_metadata(key) assert result == 123 - del future + del task while key in s.tasks: await asyncio.sleep(0.01) @@ -5955,10 +5953,10 @@ def on_nanny(dask_worker): @gen_cluster(client=True) async def test_avoid_delayed_finalize(c, s, a, b): x = delayed(inc)(1) - future = c.compute(x) - result = await future + task = c.compute(x) + result = await task assert result == 2 - assert list(s.tasks) == [future.key] == [x.key] + assert list(s.tasks) == [task.key] == [x.key] @gen_cluster() @@ -5974,7 +5972,7 @@ async def test_config_scheduler_address(s, a, b): @gen_cluster(client=True, nthreads=[]) async def test_warn_when_submitting_large_values(c, s): with pytest.warns(UserWarning, match="Sending large graph of size"): - future = c.submit(lambda x: x + 1, b"0" * 10_000_000) + task = c.submit(lambda x: x + 1, b"0" * 10_000_000) @gen_cluster(client=True, nthreads=[]) @@ -6071,12 +6069,12 @@ async def test_turn_off_pickle(c, s, a, b, direct): await c.scatter(inc) # can send complex tasks (this uses pickle regardless) - future = c.submit(lambda x: x, inc) - await wait(future) + task = c.submit(lambda x: x, inc) + await wait(task) # but can't receive complex results with pytest.raises(TypeError): - await c.gather(future, direct=direct) + await c.gather(task, direct=direct) # Run works result = await c.run(lambda: 1) @@ -6102,11 +6100,11 @@ async def test_de_serialization(s, a, b): deserializers=["msgpack"], ) as c: # Can send complex data - future = await c.scatter(np.ones(5)) + task = await c.scatter(np.ones(5)) # But can not retrieve it with pytest.raises(TypeError): - result = await future + result = await task @gen_cluster() @@ -6115,11 +6113,11 @@ async def test_de_serialization_none(s, a, b): async with Client(s.address, asynchronous=True, deserializers=["msgpack"]) as c: # Can send complex data - future = await c.scatter(np.ones(5)) + task = await c.scatter(np.ones(5)) # But can not retrieve it with pytest.raises(TypeError): - result = await future + result = await task @gen_cluster() @@ -6234,8 +6232,8 @@ async def test_mixing_clients_same_scheduler(s, a, b): async with Client(s.address, asynchronous=True) as c1, Client( s.address, asynchronous=True ) as c2: - future = c1.submit(inc, 1) - assert await c2.submit(inc, future) == 3 + task = c1.submit(inc, 1) + assert await c2.submit(inc, task) == 3 assert not s.tasks @@ -6244,9 +6242,9 @@ async def test_mixing_clients_different_scheduler(s, a, b): async with Scheduler(port=open_port()) as s2, Worker(s2.address) as w1, Client( s.address, asynchronous=True ) as c1, Client(s2.address, asynchronous=True) as c2: - future = c1.submit(inc, 1) + task = c1.submit(inc, 1) with pytest.raises(CancelledError): - await c2.submit(inc, future) + await c2.submit(inc, task) @dataclass(frozen=True) @@ -6259,8 +6257,8 @@ class MyHashable: async def test_tuple_keys(c, s, a, b): x = dask.delayed(inc)(1, dask_key_name=("x", 1)) y = dask.delayed(inc)(x, dask_key_name=("y", 1)) - future = c.compute(y) - assert (await future) == 3 + task = c.compute(y) + assert (await task) == 3 z = dask.delayed(inc)(y, dask_key_name=("z", MyHashable(1, 2))) with pytest.raises(TypeError, match="key"): await c.compute(z) @@ -6268,17 +6266,17 @@ async def test_tuple_keys(c, s, a, b): @gen_cluster(client=True) async def test_multiple_scatter(c, s, a, b): - futures = await asyncio.gather(*(c.scatter(1, direct=True) for _ in range(5))) + tasks = await asyncio.gather(*(c.scatter(1, direct=True) for _ in range(5))) - x = await futures[0] - x = await futures[0] + x = await tasks[0] + x = await tasks[0] @gen_cluster(client=True) async def test_map_large_kwargs_in_graph(c, s, a, b): np = pytest.importorskip("numpy") x = np.random.random(100000) - futures = c.map(lambda a, b: a + b, range(100), b=x) + tasks = c.map(lambda a, b: a + b, range(100), b=x) while not s.tasks: await asyncio.sleep(0.01) @@ -6292,13 +6290,13 @@ def f(): assert dask.config.get("foo") with dask.config.set(foo=False): - future = c.submit(f) + task = c.submit(f) with pytest.raises(AssertionError): - await future + await task with dask.config.set(foo=True): - await future.retry() - await future + await task.retry() + await task @gen_cluster(client=True) @@ -6360,16 +6358,16 @@ async def test_profile_bokeh(c, s, a, b): @gen_cluster(client=True) -async def test_get_mix_futures_and_SubgraphCallable(c, s, a, b): - future = c.submit(add, 1, 2) +async def test_get_mix_tasks_and_SubgraphCallable(c, s, a, b): + task = c.submit(add, 1, 2) subgraph = SubgraphCallable( - {"_2": (add, "_0", "_1"), "_3": (add, future, "_2")}, "_3", ("_0", "_1") + {"_2": (add, "_0", "_1"), "_3": (add, task, "_2")}, "_3", ("_0", "_1") ) dsk = {"a": 1, "b": 2, "c": (subgraph, "a", "b"), "d": (subgraph, "c", "b")} - future2 = c.get(dsk, "d", sync=False) - result = await future2 + task2 = c.get(dsk, "d", sync=False) + result = await task2 assert result == 11 # Nested subgraphs @@ -6377,7 +6375,7 @@ async def test_get_mix_futures_and_SubgraphCallable(c, s, a, b): { "_2": (subgraph, "_0", "_1"), "_3": (subgraph, "_2", "_1"), - "_4": (add, "_3", future2), + "_4": (add, "_3", task2), }, "_4", ("_0", "_1"), @@ -6390,7 +6388,7 @@ async def test_get_mix_futures_and_SubgraphCallable(c, s, a, b): @gen_cluster(client=True) -async def test_get_mix_futures_and_SubgraphCallable_dask_dataframe(c, s, a, b): +async def test_get_mix_tasks_and_SubgraphCallable_dask_dataframe(c, s, a, b): dd = pytest.importorskip("dask.dataframe") import pandas as pd @@ -6406,8 +6404,8 @@ async def test_get_mix_futures_and_SubgraphCallable_dask_dataframe(c, s, a, b): def test_direct_to_workers(s, loop): with Client(s["address"], loop=loop, direct_to_workers=True) as client: - future = client.scatter(1) - future.result() + task = client.scatter(1) + task.result() resp = client.run_on_scheduler(lambda dask_scheduler: dask_scheduler.events) assert "gather" not in str(resp) @@ -6421,13 +6419,13 @@ async def test_instances(c, s, a, b): @gen_cluster(client=True) async def test_wait_for_workers(c, s, a, b): - future = asyncio.ensure_future(c.wait_for_workers(n_workers=3)) + task = asyncio.ensure_future(c.wait_for_workers(n_workers=3)) await asyncio.sleep(0.22) # 2 chances - assert not future.done() + assert not task.done() async with Worker(s.address): start = time() - await future + await task assert time() < start + 1 with pytest.raises(TimeoutError) as info: @@ -6551,11 +6549,11 @@ async def test_config_inherited_by_subprocess(): @gen_cluster(client=True) -async def test_futures_of_sorted(c, s, a, b): +async def test_tasks_of_sorted(c, s, a, b): pytest.importorskip("dask.dataframe") df = await dask.datasets.timeseries(dtypes={"x": int}).persist() - futures = futures_of(df) - for k, f in zip(df.__dask_keys__(), futures): + tasks = tasks_of(df) + for k, f in zip(df.__dask_keys__(), tasks): assert str(k) in str(f) @@ -6611,33 +6609,33 @@ async def test_profile_server_disabled(c, s, a, b): @gen_cluster(client=True) -async def test_await_future(c, s, a, b): - future = c.submit(inc, 1) +async def test_await_task(c, s, a, b): + task = c.submit(inc, 1) async def f(): # flake8: noqa - result = await future + result = await task assert result == 2 await f() - future = c.submit(div, 1, 0) + task = c.submit(div, 1, 0) async def f(): with pytest.raises(ZeroDivisionError): - await future + await task await f() @gen_cluster(client=True) async def test_as_completed_async_for(c, s, a, b): - futures = c.map(inc, range(10)) - ac = as_completed(futures) + tasks = c.map(inc, range(10)) + ac = as_completed(tasks) results = [] async def f(): - async for future in ac: - result = await future + async for task in ac: + result = await task results.append(result) await f() @@ -6647,12 +6645,12 @@ async def f(): @gen_cluster(client=True) async def test_as_completed_async_for_results(c, s, a, b): - futures = c.map(inc, range(10)) - ac = as_completed(futures, with_results=True) + tasks = c.map(inc, range(10)) + ac = as_completed(tasks, with_results=True) results = [] async def f(): - async for future, result in ac: + async for task, result in ac: results.append(result) await f() @@ -6670,7 +6668,7 @@ async def test_as_completed_async_for_cancel(c, s, a, b): await x await y.cancel() - futs = [future async for future in ac] + futs = [task async for task in ac] assert futs == [x, y] await ev.set() # Allow for clean teardown @@ -6896,7 +6894,7 @@ async def test_mixed_compression(c, s): assert out.shape == (10000, 10000) -def test_futures_in_subgraphs(loop_in_thread): +def test_tasks_in_subgraphs(loop_in_thread): """Regression test of """ dd = pytest.importorskip("dask.dataframe") @@ -7251,8 +7249,8 @@ async def test_annotations_submit_map(c, s, a, b): async def test_workers_collection_restriction(c, s, a, b): da = pytest.importorskip("dask.array") - future = c.compute(da.arange(10), workers=a.address) - await future + task = c.compute(da.arange(10), workers=a.address) + await task assert a.data and not b.data @@ -7476,8 +7474,8 @@ def fetch_comp_code(dask_scheduler): async def test_computation_object_code_dask_persist(c, s, a, b): da = pytest.importorskip("dask.array") x = da.ones((10, 10), chunks=(3, 3)) - future = x.sum().persist() - await future + task = x.sum().persist() + await task test_function_code = inspect.getsource( test_computation_object_code_dask_persist.__wrapped__ @@ -7583,8 +7581,8 @@ def func(x): async def test_computation_object_code_client_compute(c, s, a, b): da = pytest.importorskip("dask.array") x = da.ones((10, 10), chunks=(3, 3)) - future = c.compute(x.sum(), retries=2) - y = await future + task = c.compute(x.sum(), retries=2) + y = await task test_function_code = inspect.getsource( test_computation_object_code_client_compute.__wrapped__ @@ -7655,10 +7653,10 @@ async def test_exception_text(c, s, a, b): def bad(x): raise Exception(x) - future = c.submit(bad, 123) - await wait(future) + task = c.submit(bad, 123) + await wait(task) - ts = s.tasks[future.key] + ts = s.tasks[task.key] assert isinstance(ts.exception_text, str) assert "123" in ts.exception_text @@ -7671,8 +7669,8 @@ async def test_async_task(c, s, a, b): async def f(x): return x + 1 - future = c.submit(f, 10) - result = await future + task = c.submit(f, 10) + result = await task assert result == 11 @@ -7681,8 +7679,8 @@ async def test_async_task_with_partial(c, s, a, b): async def f(x, y): return x + y + 1 - future = c.submit(functools.partial(f, 1), 10) - result = await future + task = c.submit(functools.partial(f, 1), 10) + result = await task assert result == 12 @@ -8352,7 +8350,7 @@ async def test_wait_for_workers_updates_info(c, s): @pytest.mark.slow # These lines sometimes appear: # Creating scratch directories is taking a surprisingly long time -# Future exception was never retrieved +# Task exception was never retrieved # tornado.util.TimeoutError # Batched Comm Closed @pytest.mark.flaky(reruns=5, reruns_delay=5) @@ -8462,8 +8460,8 @@ async def test_unpacks_remotedata_namedtuple(c, s, a, b, typ, args, kwargs): def identity(x): return x - outer_future = c.submit(identity, typ(*args, **kwargs)) - result = await outer_future + outer_task = c.submit(identity, typ(*args, **kwargs)) + result = await outer_task assert result == typ(*args, **kwargs) @@ -8476,28 +8474,28 @@ def identity(x): ], ) @gen_cluster(client=True) -async def test_resolves_future_in_namedtuple(c, s, a, b, typ, args, kwargs): +async def test_resolves_task_in_namedtuple(c, s, a, b, typ, args, kwargs): def identity(x): return x inner_result = 1 - inner_future = c.submit(identity, inner_result) + inner_task = c.submit(identity, inner_result) kwin, kwout = dict(kwargs), dict(kwargs) if kwargs: - kwin["inner"], kwout["inner"] = inner_future, inner_result - outer_future = c.submit(identity, typ(*args, inner_future, **kwin)) - result = await outer_future + kwin["inner"], kwout["inner"] = inner_task, inner_result + outer_task = c.submit(identity, typ(*args, inner_task, **kwin)) + result = await outer_task assert result == typ(*args, inner_result, **kwout) @gen_cluster(client=True) -async def test_resolves_future_in_dict(c, s, a, b): +async def test_resolves_task_in_dict(c, s, a, b): def identity(x): return x - inner_future = c.submit(identity, 1) - outer_future = c.submit(identity, {"x": inner_future, "y": 2}) - result = await outer_future + inner_task = c.submit(identity, 1) + outer_task = c.submit(identity, {"x": inner_task, "y": 2}) + result = await outer_task assert result == {"x": 1, "y": 2} diff --git a/distributed/tests/test_client_executor.py b/distributed/tests/test_client_executor.py index 4b0b3872dd..3b24db4abf 100644 --- a/distributed/tests/test_client_executor.py +++ b/distributed/tests/test_client_executor.py @@ -1,14 +1,9 @@ from __future__ import annotations import random -from concurrent.futures import ( - FIRST_COMPLETED, - FIRST_EXCEPTION, - Future as ConcurrentFuture, - TimeoutError, - as_completed, - wait, -) +from concurrent.futures import FIRST_COMPLETED, FIRST_EXCEPTION +from concurrent.futures import Future as ConcurrentFuture +from concurrent.futures import TimeoutError, as_completed, wait from time import sleep import pytest @@ -214,23 +209,23 @@ def test_retries(client): args = [ZeroDivisionError("one"), ZeroDivisionError("two"), 42] with client.get_executor(retries=5, pure=False) as e: - future = e.submit(varying(args)) - assert future.result() == 42 + task = e.submit(varying(args)) + assert task.result() == 42 with client.get_executor(retries=4) as e: - future = e.submit(varying(args)) - result = future.result() + task = e.submit(varying(args)) + result = task.result() assert result == 42 with client.get_executor(retries=2) as e: - future = e.submit(varying(args)) + task = e.submit(varying(args)) with pytest.raises(ZeroDivisionError, match="two"): - res = future.result() + res = task.result() with client.get_executor(retries=0) as e: - future = e.submit(varying(args)) + task = e.submit(varying(args)) with pytest.raises(ZeroDivisionError, match="one"): - res = future.result() + res = task.result() def test_shutdown_wait(client): @@ -240,7 +235,7 @@ def test_shutdown_wait(client): fut = e.submit(sleep, 1.0) e.shutdown() assert time() >= start + 1.0 - sleep(0.1) # wait for future outcome to propagate + sleep(0.1) # wait for task outcome to propagate assert fut.done() fut.result() # doesn't raise @@ -255,7 +250,7 @@ def test_shutdown_nowait(client): fut = e.submit(sleep, 5.0) e.shutdown(wait=False) assert time() < start + 2.0 - sleep(0.1) # wait for future outcome to propagate + sleep(0.1) # wait for task outcome to propagate assert fut.cancelled() with pytest.raises(RuntimeError): diff --git a/distributed/tests/test_computations.py b/distributed/tests/test_computations.py index e28a7875a7..64e0208c35 100644 --- a/distributed/tests/test_computations.py +++ b/distributed/tests/test_computations.py @@ -31,9 +31,9 @@ async def test_computations(c, s, a, b): @gen_cluster(client=True) -async def test_computations_futures(c, s, a, b): - futures = [c.submit(inc, i) for i in range(10)] - total = c.submit(sum, futures) +async def test_computations_tasks(c, s, a, b): + tasks = [c.submit(inc, i) for i in range(10)] + total = c.submit(sum, tasks) await total [computation] = s.computations diff --git a/distributed/tests/test_dask_collections.py b/distributed/tests/test_dask_collections.py index 0ed81e849e..cedaae7caa 100644 --- a/distributed/tests/test_dask_collections.py +++ b/distributed/tests/test_dask_collections.py @@ -86,15 +86,15 @@ async def test_dask_array_collections(c, s, a, b): x_dsk = {("x", i, j): np.random.random((3, 3)) for i in range(3) for j in range(2)} y_dsk = {("y", i, j): np.random.random((3, 3)) for i in range(2) for j in range(3)} - x_futures = await c.scatter(x_dsk) - y_futures = await c.scatter(y_dsk) + x_tasks = await c.scatter(x_dsk) + y_tasks = await c.scatter(y_dsk) dt = np.random.random(0).dtype x_local = da.Array(x_dsk, "x", ((3, 3, 3), (3, 3)), dt) y_local = da.Array(y_dsk, "y", ((3, 3), (3, 3, 3)), dt) - x_remote = da.Array(x_futures, "x", ((3, 3, 3), (3, 3)), dt) - y_remote = da.Array(y_futures, "y", ((3, 3), (3, 3, 3)), dt) + x_remote = da.Array(x_tasks, "x", ((3, 3, 3), (3, 3)), dt) + y_remote = da.Array(y_tasks, "y", ((3, 3), (3, 3, 3)), dt) exprs = [ lambda x, y: x.T + y, @@ -183,8 +183,8 @@ def test_rolling_sync(client): async def test_loc(c, s, a, b): df = make_time_dataframe() ddf = dd.from_pandas(df, npartitions=10) - future = c.compute(ddf.loc["2000-01-17":"2000-01-24"]) - await future + task = c.compute(ddf.loc["2000-01-17":"2000-01-24"]) + await task @ignore_single_machine_warning @@ -224,9 +224,9 @@ async def test_sparse_arrays(c, s, a, b): x = da.random.random((100, 10), chunks=(10, 10)) x[x < 0.95] = 0 s = x.map_blocks(sparse.COO) - future = c.compute(s.sum(axis=0)[:10]) + task = c.compute(s.sum(axis=0)[:10]) - await future + await task @gen_cluster(client=True, nthreads=[("127.0.0.1", 1)]) @@ -239,7 +239,7 @@ async def test_delayed_none(c, s, w): @pytest.mark.parametrize("typ", [tuple, list]) -def test_tuple_futures_arg(client, typ): +def test_tuple_tasks_arg(client, typ): x = client.submit( make_time_dataframe, ) diff --git a/distributed/tests/test_events.py b/distributed/tests/test_events.py index b26df7ad39..d1195d7203 100644 --- a/distributed/tests/test_events.py +++ b/distributed/tests/test_events.py @@ -35,20 +35,20 @@ def clear_it(): event = Event("x") event.clear() - wait_futures = c.map(wait_for_it_failing, range(10)) - await c.gather(wait_futures) + wait_tasks = c.map(wait_for_it_failing, range(10)) + await c.gather(wait_tasks) - set_future = c.submit(set_it) - await c.gather(set_future) + set_task = c.submit(set_it) + await c.gather(set_task) - wait_futures = c.map(wait_for_it_ok, range(10)) - await c.gather(wait_futures) + wait_tasks = c.map(wait_for_it_ok, range(10)) + await c.gather(wait_tasks) - clear_future = c.submit(clear_it) - await c.gather(clear_future) + clear_task = c.submit(clear_it) + await c.gather(clear_task) - wait_futures = c.map(wait_for_it_ok, range(10)) - await c.gather(wait_futures) + wait_tasks = c.map(wait_for_it_ok, range(10)) + await c.gather(wait_tasks) assert not s.extensions["events"]._events assert not s.extensions["events"]._waiter_count @@ -149,14 +149,14 @@ def set_it(): event = Event("x") event.set() - wait_futures = client.map(wait_for_it_failing, range(10)) - client.gather(wait_futures) + wait_tasks = client.map(wait_for_it_failing, range(10)) + client.gather(wait_tasks) - set_future = client.submit(set_it) - client.gather(set_future) + set_task = client.submit(set_it) + client.gather(set_task) - wait_futures = client.map(wait_for_it_ok, range(10)) - client.gather(wait_futures) + wait_tasks = client.map(wait_for_it_ok, range(10)) + client.gather(wait_tasks) @gen_cluster(client=True) @@ -183,8 +183,8 @@ def f(x, event=None): return x + 1 event = Event("x") - futures = c.map(f, range(10), event=event) - await c.gather(futures) + tasks = c.map(f, range(10), event=event) + await c.gather(tasks) event2 = pickle.loads(pickle.dumps(event)) assert event2.name == event.name diff --git a/distributed/tests/test_failed_workers.py b/distributed/tests/test_failed_workers.py index 8d7eec4360..f2f3f020dd 100644 --- a/distributed/tests/test_failed_workers.py +++ b/distributed/tests/test_failed_workers.py @@ -206,7 +206,7 @@ def test_restart_sync(loop): def test_worker_doesnt_await_task_completion(loop): with cluster(nanny=True, nworkers=1) as (s, [w]): with Client(s["address"], loop=loop) as c: - future = c.submit(sleep, 100) + task = c.submit(sleep, 100) sleep(0.1) start = time() c.restart(timeout="5s", wait_for_workers=False) @@ -234,8 +234,8 @@ async def test_multiple_clients_restart(s, a, b): await asyncio.sleep(0.01) assert time() < start + 5 - assert not c1.futures - assert not c2.futures + assert not c1.tasks + assert not c2.tasks # Ensure both clients still work after restart. # Reusing a previous key has no effect. @@ -269,7 +269,7 @@ async def test_restart_scheduler(s, a, b): @gen_cluster(Worker=Nanny, client=True, timeout=60) -async def test_forgotten_futures_dont_clean_up_new_futures(c, s, a, b): +async def test_forgotten_tasks_dont_clean_up_new_tasks(c, s, a, b): x = c.submit(inc, 1) await c.restart() y = c.submit(inc, 1) @@ -364,7 +364,7 @@ async def test_worker_who_has_clears_after_failed_connection(c, s, a, b): await asyncio.sleep(0.01) n_worker_address = n.worker_address - futures = c.map( + tasks = c.map( inc, range(20), key=["f%d" % i for i in range(20)], @@ -375,8 +375,8 @@ async def test_worker_who_has_clears_after_failed_connection(c, s, a, b): def sink(*args): pass - await wait(futures) - result_fut = c.submit(sink, futures, workers=a.address) + await wait(tasks) + result_fut = c.submit(sink, tasks, workers=a.address) await n.kill(timeout=1) while len(s.workers) > 2: @@ -406,7 +406,7 @@ def mock_address_host(addr): return "B" with mock.patch("distributed.worker.get_address_host", mock_address_host): - futures = c.map( + tasks = c.map( slowinc, range(20), delay=0.1, @@ -414,13 +414,13 @@ def mock_address_host(addr): workers=[a.address], allow_other_workers=True, ) - await wait(futures) + await wait(tasks) # replicate data to avoid the scheduler retriggering the computation # retriggering cleans up the state nicely but doesn't reflect real world # scenarios where there may be replicas on the cluster, e.g. they are # replicated as a dependency somewhere else - await c.replicate(futures, n=2, workers=[a.address, b.address]) + await c.replicate(tasks, n=2, workers=[a.address, b.address]) def sink(*args): pass @@ -433,7 +433,7 @@ def sink(*args): a.handle_stimulus( FreeKeysEvent(keys=["f1"], stimulus_id="Am I evil?") ) # Yes, I am! - result_fut = c.submit(sink, futures, workers=x.address) + result_fut = c.submit(sink, tasks, workers=x.address) await result_fut @@ -442,7 +442,7 @@ def sink(*args): @gen_cluster(client=True, timeout=60, Worker=Nanny, nthreads=[("127.0.0.1", 1)]) async def test_restart_timeout_on_long_running_task(c, s, a): with captured_logger("distributed.scheduler") as sio: - future = c.submit(sleep, 3600) + task = c.submit(sleep, 3600) await asyncio.sleep(0.1) await c.restart() diff --git a/distributed/tests/test_locks.py b/distributed/tests/test_locks.py index 7477f35460..19d4507e4c 100644 --- a/distributed/tests/test_locks.py +++ b/distributed/tests/test_locks.py @@ -24,8 +24,8 @@ def f(x): assert client.get_metadata("locked") is True client.set_metadata("locked", False) - futures = c.map(f, range(20)) - await c.gather(futures) + tasks = c.map(f, range(20)) + await c.gather(tasks) assert not s.extensions["locks"].events assert not s.extensions["locks"].ids @@ -100,8 +100,8 @@ def f(x): client.set_metadata("locked", False) client.set_metadata("locked", False) - futures = client.map(f, range(10)) - client.gather(futures) + tasks = client.map(f, range(10)) + client.gather(tasks) @gen_cluster(client=True) @@ -124,8 +124,8 @@ def f(x, lock=None): return x + 1 lock = Lock("x") - futures = c.map(f, range(10), lock=lock) - await c.gather(futures) + tasks = c.map(f, range(10), lock=lock) + await c.gather(tasks) lock2 = pickle.loads(pickle.dumps(lock)) assert lock2.name == lock.name diff --git a/distributed/tests/test_multi_locks.py b/distributed/tests/test_multi_locks.py index 8d8d8417dd..03317309b0 100644 --- a/distributed/tests/test_multi_locks.py +++ b/distributed/tests/test_multi_locks.py @@ -24,8 +24,8 @@ def f(_): assert client.get_metadata("locked") is True client.set_metadata("locked", False) - futures = c.map(f, range(20)) - await c.gather(futures) + tasks = c.map(f, range(20)) + await c.gather(tasks) ext = s.extensions["multi_locks"] assert not ext.events assert not ext.requests diff --git a/distributed/tests/test_nanny.py b/distributed/tests/test_nanny.py index 74d049a35d..11f728fe6a 100644 --- a/distributed/tests/test_nanny.py +++ b/distributed/tests/test_nanny.py @@ -652,9 +652,9 @@ async def test_restart_memory(c, s, n): np = pytest.importorskip("numpy") s.allowed_failures = 1 - future = c.submit(np.ones, 300_000_000, dtype="f8") + task = c.submit(np.ones, 300_000_000, dtype="f8") with pytest.raises(KilledWorker): - await future + await task while not s.workers: await asyncio.sleep(0.1) @@ -736,7 +736,7 @@ async def test_malloc_trim_threshold(c, s, a): Note ---- - This test may start failing in a future Python version if CPython switches to + This test may start failing in a task Python version if CPython switches to using mimalloc by default. If it does, a thorough benchmarking exercise is needed. """ da = pytest.importorskip("dask.array") diff --git a/distributed/tests/test_priorities.py b/distributed/tests/test_priorities.py index 8bdc997234..c356f52d34 100644 --- a/distributed/tests/test_priorities.py +++ b/distributed/tests/test_priorities.py @@ -44,10 +44,10 @@ async def block_worker( ntasks_on_scheduler : int, optional Number of tasks that must appear on the scheduler. Defaults to the number of - futures held by the client. + tasks held by the client. ntasks_on_worker : int, optional Number of tasks that must appear on the worker before any task is actually - started. Defaults to the number of futures held by the client. + started. Defaults to the number of tasks held by the client. """ if pause: w.status = Status.paused @@ -62,9 +62,9 @@ async def block_worker( yield if ntasks_on_scheduler is None: - ntasks_on_scheduler = len(c.futures) + ntasks_on_scheduler = len(c.tasks) if ntasks_on_worker is None: - ntasks_on_worker = len(c.futures) + ntasks_on_worker = len(c.tasks) while len(s.tasks) < ntasks_on_scheduler: await asyncio.sleep(0.01) diff --git a/distributed/tests/test_publish.py b/distributed/tests/test_publish.py index f1107b5761..bf80d4f056 100644 --- a/distributed/tests/test_publish.py +++ b/distributed/tests/test_publish.py @@ -7,7 +7,7 @@ from dask import delayed from distributed import Client -from distributed.client import futures_of +from distributed.client import tasks_of from distributed.metrics import time from distributed.protocol import Serialized from distributed.utils_test import gen_cluster, inc @@ -154,14 +154,14 @@ async def test_publish_bag(s, a, b): bag = db.from_sequence([0, 1, 2]) bagp = c.persist(bag) - assert len(futures_of(bagp)) == 3 - keys = {f.key for f in futures_of(bagp)} + assert len(tasks_of(bagp)) == 3 + keys = {f.key for f in tasks_of(bagp)} assert keys == set(bag.dask) await c.publish_dataset(data=bagp) # check that serialization didn't affect original bag's dask - assert len(futures_of(bagp)) == 3 + assert len(tasks_of(bagp)) == 3 result = await f.get_dataset("data") assert set(result.dask.keys()) == set(bagp.dask.keys()) @@ -282,20 +282,20 @@ async def test_deserialize_client(c, s, a, b): - when multiple clients are connected to the same scheduler, test that they don't interfere with each other. - See: test_client.test_serialize_future + See: test_client.test_serialize_task See: https://github.com/dask/distributed/issues/3227 """ - future = await c.scatter("123") - await c.publish_dataset(foo=future) - future = await c.get_dataset("foo") - assert future.client is c + task = await c.scatter("123") + await c.publish_dataset(foo=task) + task = await c.get_dataset("foo") + assert task.client is c for addr in (s.address, "localhost:" + s.address.split(":")[-1]): async with Client(addr, asynchronous=True) as c2: - future = await c.get_dataset("foo") - assert future.client is c - future = await c2.get_dataset("foo") - assert future.client is c2 + task = await c.get_dataset("foo") + assert task.client is c + task = await c2.get_dataset("foo") + assert task.client is c2 # Ensure cleanup from distributed.client import _current_client diff --git a/distributed/tests/test_pubsub.py b/distributed/tests/test_pubsub.py index 664ea4f4ab..93bca694a2 100644 --- a/distributed/tests/test_pubsub.py +++ b/distributed/tests/test_pubsub.py @@ -81,8 +81,8 @@ def f(x): pub = Pub("a") pub.put(x) - futures = c.map(f, range(10)) - await wait(futures) + tasks = c.map(f, range(10)) + await wait(tasks) L = [] for _ in range(10): diff --git a/distributed/tests/test_queues.py b/distributed/tests/test_queues.py index 8a0dd96ca9..400ea03578 100644 --- a/distributed/tests/test_queues.py +++ b/distributed/tests/test_queues.py @@ -20,21 +20,21 @@ async def test_queue(c, s, a, b): xx = await Queue("x") assert x.client is c - future = c.submit(inc, 1) + task = c.submit(inc, 1) - await x.put(future) - await y.put(future) - future2 = await xx.get() - assert future.key == future2.key + await x.put(task) + await y.put(task) + task2 = await xx.get() + assert task.key == task2.key with pytest.raises(TimeoutError): await x.get(timeout=0.1) - del future, future2 + del task, task2 await asyncio.sleep(0.1) - assert s.tasks # future still present in y's queue - await y.get() # burn future + assert s.tasks # task still present in y's queue + await y.get() # burn task start = time() while s.tasks: @@ -58,31 +58,31 @@ async def test_queue_with_data(c, s, a, b): def test_sync(client): - future = client.submit(lambda x: x + 1, 10) + task = client.submit(lambda x: x + 1, 10) x = Queue("x") xx = Queue("x") - x.put(future) + x.put(task) assert x.qsize() == 1 assert xx.qsize() == 1 - future2 = xx.get() + task2 = xx.get() - assert future2.result() == 11 + assert task2.result() == 11 @gen_cluster() -async def test_hold_futures(s, a, b): +async def test_hold_tasks(s, a, b): async with Client(s.address, asynchronous=True) as c1: - future = c1.submit(lambda x: x + 1, 10) + task = c1.submit(lambda x: x + 1, 10) q1 = await Queue("q") - await q1.put(future) + await q1.put(task) del q1 await asyncio.sleep(0.1) async with Client(s.address, asynchronous=True) as c1: q2 = await Queue("q") - future2 = await q2.get() - result = await future2 + task2 = await q2.get() + result = await task2 assert result == 11 @@ -117,8 +117,8 @@ def f(i): with worker_client() as c: q = Queue("x", client=c) for _ in range(100): - future = q.get() - x = future.result() + task = q.get() + x = task.result() y = c.submit(inc, x) q.put(y) sleep(0.01) @@ -127,11 +127,11 @@ def f(i): q = Queue("x", client=c) L = await c.scatter(range(5)) - for future in L: - await q.put(future) + for task in L: + await q.put(task) - futures = c.map(f, range(5)) - results = await c.gather(futures) + tasks = c.map(f, range(5)) + results = await c.gather(tasks) assert all(r > 50 for r in results) assert sum(results) == 510 qsize = await q.qsize() @@ -139,20 +139,20 @@ def f(i): @gen_cluster(client=True) -async def test_same_futures(c, s, a, b): +async def test_same_tasks(c, s, a, b): q = Queue("x") - future = await c.scatter(123) + task = await c.scatter(123) for _ in range(5): - await q.put(future) + await q.put(task) - assert {ts.key for ts in s.clients["queue-x"].wants_what} == {future.key} + assert {ts.key for ts in s.clients["queue-x"].wants_what} == {task.key} for _ in range(4): - future2 = await q.get() - assert {ts.key for ts in s.clients["queue-x"].wants_what} == {future.key} + task2 = await q.get() + assert {ts.key for ts in s.clients["queue-x"].wants_what} == {task.key} await asyncio.sleep(0.05) - assert {ts.key for ts in s.clients["queue-x"].wants_what} == {future.key} + assert {ts.key for ts in s.clients["queue-x"].wants_what} == {task.key} await q.get() @@ -193,22 +193,22 @@ async def test_Future_knows_status_immediately(c, s, a, b): async with Client(s.address, asynchronous=True) as c2: q2 = await Queue("q", client=c2) - future = await q2.get() - assert future.status == "finished" + task = await q2.get() + assert task.status == "finished" x = c.submit(div, 1, 0) await wait(x) await q.put(x) - future2 = await q2.get() - assert future2.status == "error" + task2 = await q2.get() + assert task2.status == "error" with pytest.raises(ZeroDivisionError): - await future2 + await task2 start = time() while True: # we learn about the true error eventually try: - await future2 + await task2 except ZeroDivisionError: break except Exception: @@ -217,16 +217,16 @@ async def test_Future_knows_status_immediately(c, s, a, b): @gen_cluster(client=True) -async def test_erred_future(c, s, a, b): - future = c.submit(div, 1, 0) +async def test_erred_task(c, s, a, b): + task = c.submit(div, 1, 0) q = await Queue() - await q.put(future) + await q.put(task) await asyncio.sleep(0.1) - future2 = await q.get() + task2 = await q.get() with pytest.raises(ZeroDivisionError): - await future2.result() + await task2.result() - exc = await future2.exception() + exc = await task2.exception() assert isinstance(exc, ZeroDivisionError) diff --git a/distributed/tests/test_reschedule.py b/distributed/tests/test_reschedule.py index fe6ef45644..9a1bb7f3c6 100644 --- a/distributed/tests/test_reschedule.py +++ b/distributed/tests/test_reschedule.py @@ -47,17 +47,17 @@ def f(x): if get_worker().address == a_address: raise Reschedule() - futures = c.map(f, range(4), key=["x1", "x2", "x3", "x4"]) - futures2 = c.map(slowinc, range(10), delay=0.1, key="clog", workers=[a.address]) - await wait(futures) + tasks = c.map(f, range(4), key=["x1", "x2", "x3", "x4"]) + tasks2 = c.map(slowinc, range(10), delay=0.1, key="clog", workers=[a.address]) + await wait(tasks) assert any(isinstance(ev, RescheduleEvent) for ev in a.state.stimulus_log) - assert all(f.key in b.data for f in futures) + assert all(f.key in b.data for f in tasks) @pytest.mark.parametrize("state", ["executing", "long-running"]) @gen_cluster(client=True, nthreads=[("", 1)]) async def test_cancelled_reschedule(c, s, a, state): - """A task raises Reschedule(), but the future was released by the client. + """A task raises Reschedule(), but the task was released by the client. Same as test_cancelled_reschedule_worker_state""" ev1 = Event() ev2 = Event() diff --git a/distributed/tests/test_resources.py b/distributed/tests/test_resources.py index 5b1c2d5c35..f8e75ced57 100644 --- a/distributed/tests/test_resources.py +++ b/distributed/tests/test_resources.py @@ -55,8 +55,8 @@ async def test_resource_submit(c, s, a, b): ], ) async def test_submit_many_non_overlapping(c, s, a, b): - futures = [c.submit(inc, i, resources={"A": 1}) for i in range(5)] - await wait(futures) + tasks = [c.submit(inc, i, resources={"A": 1}) for i in range(5)] + await wait(tasks) assert len(a.data) == 5 assert len(b.data) == 0 @@ -70,14 +70,14 @@ async def test_submit_many_non_overlapping(c, s, a, b): ], ) async def test_submit_many_non_overlapping_2(c, s, a, b): - futures = c.map(slowinc, range(100), resources={"A": 1}, delay=0.02) + tasks = c.map(slowinc, range(100), resources={"A": 1}, delay=0.02) while len(a.data) + len(b.data) < 100: await asyncio.sleep(0.01) assert a.state.executing_count <= 2 assert b.state.executing_count <= 1 - await wait(futures) + await wait(tasks) assert a.state.total_resources == a.state.available_resources assert b.state.total_resources == b.state.available_resources @@ -92,10 +92,10 @@ async def test_submit_many_non_overlapping_2(c, s, a, b): async def test_move(c, s, a, b): [x] = await c.scatter([1], workers=b.address) - future = c.submit(inc, x, resources={"A": 1}) + task = c.submit(inc, x, resources={"A": 1}) - await wait(future) - assert a.data[future.key] == 2 + await wait(task) + assert a.data[task.key] == 2 @gen_cluster( @@ -108,12 +108,10 @@ async def test_move(c, s, a, b): async def test_dont_work_steal(c, s, a, b): [x] = await c.scatter([1], workers=a.address) - futures = [ - c.submit(slowadd, x, i, resources={"A": 1}, delay=0.05) for i in range(10) - ] + tasks = [c.submit(slowadd, x, i, resources={"A": 1}, delay=0.05) for i in range(10)] - await wait(futures) - assert all(f.key in a.data for f in futures) + await wait(tasks) + assert all(f.key in a.data for f in tasks) @gen_cluster( @@ -124,9 +122,9 @@ async def test_dont_work_steal(c, s, a, b): ], ) async def test_map(c, s, a, b): - futures = c.map(inc, range(10), resources={"B": 1}) - await wait(futures) - assert set(b.data) == {f.key for f in futures} + tasks = c.map(inc, range(10), resources={"B": 1}) + await wait(tasks) + assert set(b.data) == {f.key for f in tasks} assert not a.data @@ -230,13 +228,13 @@ async def test_resources_str(c, s, a, b): @gen_cluster(client=True, nthreads=[("127.0.0.1", 4, {"resources": {"A": 2, "B": 1}})]) async def test_minimum_resource(c, s, a): - futures = c.map(slowinc, range(30), resources={"A": 1, "B": 1}, delay=0.02) + tasks = c.map(slowinc, range(30), resources={"A": 1, "B": 1}, delay=0.02) while len(a.data) < 30: await asyncio.sleep(0.01) assert a.state.executing_count <= 1 - await wait(futures) + await wait(tasks) assert a.state.total_resources == a.state.available_resources @@ -369,7 +367,7 @@ def test_task_cancelled_and_readded_with_resources(ws): ], ) async def test_balance_resources(c, s, a, b): - futures = c.map(slowinc, range(100), delay=0.1, workers=a.address) + tasks = c.map(slowinc, range(100), delay=0.1, workers=a.address) constrained = c.map(inc, range(2), resources={"A": 1}) await wait(constrained) @@ -385,7 +383,7 @@ async def test_set_resources(c, s, a): assert s.workers[a.address].resources == {"A": 2} lock = Lock() async with lock: - future = c.submit(lock_inc, 1, lock=lock, resources={"A": 1}) + task = c.submit(lock_inc, 1, lock=lock, resources={"A": 1}) while a.state.available_resources["A"] == 2: await asyncio.sleep(0.01) diff --git a/distributed/tests/test_scheduler.py b/distributed/tests/test_scheduler.py index 67595c38d5..e0d5686102 100644 --- a/distributed/tests/test_scheduler.py +++ b/distributed/tests/test_scheduler.py @@ -32,10 +32,10 @@ CancelledError, Client, Event, - Future, Lock, Nanny, SchedulerPlugin, + Task, Worker, fire_and_forget, wait, @@ -174,7 +174,7 @@ def test_decide_worker_coschedule_order_neighbors(ndeps, nthreads): ) async def test_decide_worker_coschedule_order_neighbors_(c, s, *workers): r""" - Ensure that sibling root tasks are scheduled to the same node, reducing future + Ensure that sibling root tasks are scheduled to the same node, reducing task data transfer. We generate a wide layer of "root" tasks (random NumPy arrays). All of those @@ -429,9 +429,9 @@ async def test_graph_execution_width(c, s, *workers): async def test_forget_tasks_while_processing(c, s, a, b): events = [Event() for _ in range(10)] - futures = c.map(Event.wait, events) + tasks = c.map(Event.wait, events) await events[0].set() - await futures[0] + await tasks[0] await c.close() assert not s.tasks @@ -441,9 +441,9 @@ async def test_forget_tasks_while_processing(c, s, a, b): async def test_restart_while_processing(c, s, a, b): events = [Event() for _ in range(10)] - futures = c.map(Event.wait, events) + tasks = c.map(Event.wait, events) await events[0].set() - await futures[0] + await tasks[0] # TODO slow because worker waits a while for the task to finish await c.restart() assert not s.tasks @@ -482,7 +482,7 @@ async def test_queued_release_multiple_workers(c, s, *workers): ] # Cancel the first batch. - # Use `Client.close` instead of `del first_batch` because deleting futures sends cancellation + # Use `Client.close` instead of `del first_batch` because deleting tasks sends cancellation # messages one at a time. We're testing here that when multiple workers have open slots, we don't # recommend the same queued tasks for every worker, so we need a bulk cancellation operation. await c.close() @@ -1087,8 +1087,8 @@ async def test_ready_remove_worker(c, s, a, b, worker_saturation): @gen_cluster(client=True, Worker=Nanny, timeout=60) async def test_restart(c, s, a, b): with captured_logger("distributed.scheduler") as caplog: - futures = c.map(inc, range(20)) - await wait(futures) + tasks = c.map(inc, range(20)) + await wait(tasks) with captured_logger("distributed.nanny") as nanny_logger: await s.restart() assert "Reason: scheduler-restart" in nanny_logger.getvalue() @@ -1105,7 +1105,7 @@ async def test_restart(c, s, a, b): assert not s.tasks - assert all(f.status == "cancelled" for f in futures) + assert all(f.status == "cancelled" for f in tasks) x = c.submit(inc, 1) assert await x == 2 assert "restart" in caplog.getvalue().lower() @@ -1171,7 +1171,7 @@ async def test_restart_nanny_timeout_exceeded(c, s, a, b): assert not s.unrunnable assert not s.tasks - assert not c.futures + assert not c.tasks assert f.status == "cancelled" assert fr.status == "cancelled" finally: @@ -1559,7 +1559,7 @@ async def test_workers_to_close(cl, s, *workers): with dask.config.set( {"distributed.scheduler.default-task-durations": {"a": 4, "b": 4, "c": 1}} ): - futures = cl.map(slowinc, [1, 1, 1], key=["a-4", "b-4", "c-1"]) + tasks = cl.map(slowinc, [1, 1, 1], key=["a-4", "b-4", "c-1"]) while sum(len(w.processing) for w in s.workers.values()) < 3: await asyncio.sleep(0.001) @@ -1583,13 +1583,13 @@ def key(ws): assert set(s.workers_to_close(key=key)) == {w.address for w in workers} # Assert that job in one worker blocks closure of group - future = c.submit(slowinc, 1, delay=0.2, workers=workers[0].address) + task = c.submit(slowinc, 1, delay=0.2, workers=workers[0].address) while not any(ws.processing for ws in s.workers.values()): await asyncio.sleep(0.001) assert set(s.workers_to_close(key=key)) == {workers[2].address, workers[3].address} - del future + del task while any(ws.processing for ws in s.workers.values()): await asyncio.sleep(0.001) @@ -1636,7 +1636,7 @@ def long_running_secede(evt): @gen_cluster(client=True) async def test_retire_workers_no_suspicious_tasks(c, s, a, b): - future = c.submit( + task = c.submit( slowinc, 100, delay=0.5, workers=a.address, allow_other_workers=True ) await asyncio.sleep(0.2) @@ -1703,7 +1703,7 @@ async def test_file_descriptors(c, s): @nodebug @gen_cluster(client=True) async def test_learn_occupancy(c, s, a, b): - futures = c.map(slowinc, range(1000), delay=0.2) + tasks = c.map(slowinc, range(1000), delay=0.2) while sum(len(ts.who_has or ()) for ts in s.tasks.values()) < 10: await asyncio.sleep(0.01) @@ -1720,7 +1720,7 @@ async def test_learn_occupancy(c, s, a, b): @nodebug @gen_cluster(client=True) async def test_learn_occupancy_2(c, s, a, b): - future = c.map(slowinc, range(1000), delay=0.2) + task = c.map(slowinc, range(1000), delay=0.2) while not any(ts.who_has for ts in s.tasks.values()): await asyncio.sleep(0.01) @@ -1731,8 +1731,8 @@ async def test_learn_occupancy_2(c, s, a, b): @nodebug @gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 30) async def test_balance_many_workers(c, s, *workers): - futures = c.map(slowinc, range(20), delay=0.2) - await wait(futures) + tasks = c.map(slowinc, range(20), delay=0.2) + await wait(tasks) assert {len(w.has_what) for w in s.workers.values()} == {0, 1} @@ -1752,8 +1752,8 @@ async def test_balance_many_workers(c, s, *workers): config={"distributed.scheduler.work-stealing": False}, ) async def test_balance_many_workers_2(c, s, *workers): - futures = c.map(slowinc, range(90), delay=0.2) - await wait(futures) + tasks = c.map(slowinc, range(90), delay=0.2) + await wait(tasks) assert {len(w.has_what) for w in s.workers.values()} == {3} @@ -1938,13 +1938,13 @@ async def test_retire_nannies_close(c, s, a, b): @gen_cluster(client=True, nthreads=[("127.0.0.1", 2)]) async def test_fifo_submission(c, s, w): - futures = [] + tasks = [] for i in range(20): - future = c.submit(slowinc, i, delay=0.1, key="inc-%02d" % i, fifo_timeout=0.01) - futures.append(future) + task = c.submit(slowinc, i, delay=0.1, key="inc-%02d" % i, fifo_timeout=0.01) + tasks.append(task) await asyncio.sleep(0.02) - await wait(futures[-1]) - assert futures[10].status == "finished" + await wait(tasks[-1]) + assert tasks[10].status == "finished" @gen_test() @@ -1982,8 +1982,8 @@ async def test_dashboard_host(host, dashboard_address, expect): @gen_cluster(client=True, worker_kwargs={"profile_cycle_interval": "100ms"}) async def test_profile_metadata(c, s, a, b): start = time() - 1 - futures = c.map(slowinc, range(10), delay=0.05, workers=a.address) - await wait(futures) + tasks = c.map(slowinc, range(10), delay=0.05, workers=a.address) + await wait(tasks) await asyncio.sleep(0.200) meta = await s.get_profile_metadata(profile_cycle_interval=0.100) @@ -2009,8 +2009,8 @@ def raise_timeout(*args, **kwargs): b.handlers["profile_metadata"] = raise_timeout - futures = c.map(slowinc, range(10), delay=0.05, workers=a.address) - await wait(futures) + tasks = c.map(slowinc, range(10), delay=0.05, workers=a.address) + await wait(tasks) await asyncio.sleep(0.200) meta = await s.get_profile_metadata(profile_cycle_interval=0.100) @@ -2049,9 +2049,9 @@ async def test_profile_metadata_keys(c, s, a, b): }, ) async def test_statistical_profiling(c, s, a, b): - futures = c.map(slowinc, range(10), delay=0.1) + tasks = c.map(slowinc, range(10), delay=0.1) - await wait(futures) + await wait(tasks) profile = await s.get_profile() assert profile["count"] @@ -2066,13 +2066,13 @@ async def test_statistical_profiling(c, s, a, b): }, ) async def test_statistical_profiling_failure(c, s, a, b): - futures = c.map(slowinc, range(10), delay=0.1) + tasks = c.map(slowinc, range(10), delay=0.1) def raise_timeout(*args, **kwargs): raise TimeoutError b.handlers["profile"] = raise_timeout - await wait(futures) + await wait(tasks) profile = await s.get_profile() assert profile["count"] @@ -2095,16 +2095,16 @@ def g(_, ev1, ev2): x = f(None, dask_key_name="x") y = g(x, ev1, ev2, dask_key_name="y") z = f(y, dask_key_name="z") - future = c.compute(z) + task = c.compute(z) - fire_and_forget(future) + fire_and_forget(task) await ev1.wait() - # Cancel the future for z when + # Cancel the task for z when # - x is in memory # - y is processing # - z is pending - await future.cancel(force=True) - assert future.status == "cancelled" + await task.cancel(force=True) + assert task.status == "cancelled" while s.tasks: await asyncio.sleep(0.01) await ev2.set() @@ -2115,18 +2115,18 @@ def g(_, ev1, ev2): client=True, Worker=Nanny, clean_kwargs={"processes": False, "threads": False} ) async def test_log_tasks_during_restart(c, s, a, b): - future = c.submit(sys.exit, 0) - await wait(future) + task = c.submit(sys.exit, 0) + await wait(task) assert "exit" in str(s.events) @gen_cluster(client=True) async def test_get_task_status(c, s, a, b): - future = c.submit(inc, 1) - await wait(future) + task = c.submit(inc, 1) + await wait(task) - result = await a.scheduler.get_task_status(keys=[future.key]) - assert result == {future.key: "memory"} + result = await a.scheduler.get_task_status(keys=[task.key]) + assert result == {task.key: "memory"} @gen_cluster(nthreads=[]) @@ -2145,26 +2145,26 @@ async def test_deque_handler(s): async def test_retries(c, s, a, b): args = [ZeroDivisionError("one"), ZeroDivisionError("two"), 42] - future = c.submit(varying(args), retries=3) - result = await future + task = c.submit(varying(args), retries=3) + result = await task assert result == 42 - assert s.tasks[future.key].retries == 1 - assert not s.tasks[future.key].exception + assert s.tasks[task.key].retries == 1 + assert not s.tasks[task.key].exception - future = c.submit(varying(args), retries=2, pure=False) - result = await future + task = c.submit(varying(args), retries=2, pure=False) + result = await task assert result == 42 - assert s.tasks[future.key].retries == 0 - assert not s.tasks[future.key].exception + assert s.tasks[task.key].retries == 0 + assert not s.tasks[task.key].exception - future = c.submit(varying(args), retries=1, pure=False) + task = c.submit(varying(args), retries=1, pure=False) with pytest.raises(ZeroDivisionError) as exc_info: - await future + await task exc_info.match("two") - future = c.submit(varying(args), retries=0, pure=False) + task = c.submit(varying(args), retries=0, pure=False) with pytest.raises(ZeroDivisionError) as exc_info: - await future + await task exc_info.match("one") @@ -2312,12 +2312,12 @@ def block(lock): assert s.workers[a.address].used_resources == {"A": 0} assert a.state.available_resources == {"A": 1} - future = c.submit(block, lock, key="x", resources={"A": 1}) + task = c.submit(block, lock, key="x", resources={"A": 1}) await wait_for_state("x", "executing", a) assert s.workers[a.address].used_resources == {"A": 1} assert a.state.available_resources == {"A": 0} - await future.cancel() + await task.cancel() await wait_for_state("x", "cancelled", a) assert s.workers[a.address].used_resources == {"A": 0} assert a.state.available_resources == {"A": 0} @@ -2377,12 +2377,12 @@ async def test_idle_timeout(c, s, a, b): assert s.check_idle() is not None # Repeated calls should still not be None s.idle_timeout = 0.500 pc = PeriodicCallback(s.check_idle, 10) - future = c.submit(slowinc, 1) + task = c.submit(slowinc, 1) while not s.tasks: await asyncio.sleep(0.01) assert s.check_idle() is None pc.start() - await future + await task assert s.idle_since is None or s.idle_since > beginning _idle_since = s.check_idle() assert _idle_since == s.idle_since @@ -2414,7 +2414,7 @@ async def test_idle_timeout_no_workers(c, s): s.periodic_callbacks["idle-timeout"].stop() s.idle_timeout = 0.1 - future = c.submit(inc, 1) + task = c.submit(inc, 1) while not s.tasks: await asyncio.sleep(0.1) @@ -2426,9 +2426,9 @@ async def test_idle_timeout_no_workers(c, s): assert s.tasks async with Worker(s.address): - await future + await task assert not s.check_idle() - del future + del task while s.tasks: await asyncio.sleep(0.1) @@ -2447,7 +2447,7 @@ async def test_idle_timeout_no_workers(c, s): ) async def test_no_workers_timeout_disabled(c, s, a, b): """no-workers-timeout has been disabled""" - future = c.submit(inc, 1, key="x") + task = c.submit(inc, 1, key="x") await wait_for_state("x", ("queued", "no-worker"), s) s._check_no_workers() @@ -2474,7 +2474,7 @@ async def test_no_workers_timeout_without_workers(c, s): assert s.status == Status.running - future = c.submit(inc, 1) + task = c.submit(inc, 1) while s.status != Status.closed: await asyncio.sleep(0.01) @@ -2488,7 +2488,7 @@ async def test_no_workers_timeout_bad_restrictions(c, s, a, b): """Trip no-workers-timeout when there are workers available but none satisfies task restrictions """ - future = c.submit(inc, 1, key="x", workers=["127.0.0.2:1234"]) + task = c.submit(inc, 1, key="x", workers=["127.0.0.2:1234"]) while s.status != Status.closed: await asyncio.sleep(0.01) @@ -2501,7 +2501,7 @@ async def test_no_workers_timeout_bad_restrictions(c, s, a, b): async def test_no_workers_timeout_queued(c, s, a): """Don't trip no-workers-timeout when there are queued tasks AND processing tasks""" ev = Event() - futures = [c.submit(lambda ev: ev.wait(), ev, pure=False) for _ in range(3)] + tasks = [c.submit(lambda ev: ev.wait(), ev, pure=False) for _ in range(3)] while not a.state.tasks: await asyncio.sleep(0.01) assert s.queued or math.isinf(s.WORKER_SATURATION) @@ -2718,12 +2718,12 @@ async def test_finished(): @gen_cluster(nthreads=[], client=True) async def test_retire_names_str(c, s): async with Worker(s.address, name="0") as a, Worker(s.address, name="1") as b: - futures = c.map(inc, range(5), workers=[a.address]) - futures.extend(c.map(inc, range(5, 10), workers=[b.address])) - await wait(futures) + tasks = c.map(inc, range(5), workers=[a.address]) + tasks.extend(c.map(inc, range(5, 10), workers=[b.address])) + await wait(tasks) assert a.data and b.data await s.retire_workers(names=[0]) - assert all(f.done() for f in futures) + assert all(f.done() for f in tasks) assert len(b.data) == 10 @@ -2731,8 +2731,8 @@ async def test_retire_names_str(c, s): client=True, config={"distributed.scheduler.default-task-durations": {"inc": 100}} ) async def test_get_task_duration(c, s, a, b): - future = c.submit(inc, 1) - await future + task = c.submit(inc, 1) + await task assert 10 < s.task_prefixes["inc"].duration_average < 100 ts_pref1 = s.new_task("inc-abcdefab", None, "released") @@ -2981,8 +2981,8 @@ async def test_task_prefix(c, s, a, b): client=True, Worker=Nanny, config={"distributed.scheduler.allowed-failures": 0} ) async def test_failing_task_increments_suspicious(client, s, a, b): - future = client.submit(sys.exit, 0) - await wait(future) + task = client.submit(sys.exit, 0) + await wait(task) assert s.task_prefixes["exit"].suspicious == 1 assert sum(tp.suspicious for tp in s.task_prefixes.values()) == sum( @@ -3160,12 +3160,12 @@ async def test_multiple_listeners(dashboard_link_template, expected_dashboard_li assert b.scheduler.address.startswith("tcp") async with Client(s.address, asynchronous=True) as c: - futures = c.map(inc, range(20)) - await wait(futures) + tasks = c.map(inc, range(20)) + await wait(tasks) # Force inter-worker communication both ways - await c.submit(sum, futures, workers=[a.address]) - await c.submit(len, futures, workers=[b.address]) + await c.submit(sum, tasks, workers=[a.address]) + await c.submit(len, tasks, workers=[b.address]) log = log.getvalue() assert re.search(r"Scheduler at:\s*tcp://", log) @@ -3201,12 +3201,12 @@ async def test_worker_name_collision(s, a): @gen_cluster(client=True, config={"distributed.scheduler.unknown-task-duration": "1h"}) async def test_unknown_task_duration_config(client, s, a, b): - future = client.submit(slowinc, 1) + task = client.submit(slowinc, 1) while not s.tasks: await asyncio.sleep(0.001) assert sum(s.get_task_duration(ts) for ts in s.tasks.values()) == 3600 assert len(s.unknown_durations) == 1 - await wait(future) + await wait(task) assert len(s.unknown_durations) == 0 @@ -3613,10 +3613,10 @@ async def test_rebalance(c, s, a, b): # Generate 500 buffers worth 512 MiB total on worker a. This sends its memory # utilisation slightly above 50% (after counting unmanaged) which is above the # distributed.worker.memory.rebalance.sender-min threshold. - futures = c.map( + tasks = c.map( lambda _: "x" * (2**29 // 500), range(500), workers=[a.worker_address] ) - await wait(futures) + await wait(tasks) # Wait for heartbeats await assert_memory(s, "process", 512, 1024) await assert_ndata(c, {a.worker_address: 500, b.worker_address: 0}) @@ -3648,7 +3648,7 @@ async def test_rebalance(c, s, a, b): @gen_cluster(client=True, config=REBALANCE_MANAGED_CONFIG) async def test_rebalance_managed_memory(c, s, a, b): - futures = await c.scatter(range(100), workers=[a.address]) + tasks = await c.scatter(range(100), workers=[a.address]) assert len(a.data) == 100 assert len(b.data) == 0 await s.rebalance() @@ -3658,7 +3658,7 @@ async def test_rebalance_managed_memory(c, s, a, b): @gen_cluster(nthreads=[("", 1)] * 3, client=True, config=REBALANCE_MANAGED_CONFIG) async def test_rebalance_workers_and_keys(client, s, a, b, c): - futures = await client.scatter(range(100), workers=[a.address]) + tasks = await client.scatter(range(100), workers=[a.address]) assert (len(a.data), len(b.data), len(c.data)) == (100, 0, 0) # Passing empty iterables is not the same as omitting the arguments @@ -3667,9 +3667,7 @@ async def test_rebalance_workers_and_keys(client, s, a, b, c): assert (len(a.data), len(b.data), len(c.data)) == (100, 0, 0) # Limit rebalancing to two arbitrary keys and two arbitrary workers. - await s.rebalance( - keys=[futures[3].key, futures[7].key], workers=[a.address, b.address] - ) + await s.rebalance(keys=[tasks[3].key, tasks[7].key], workers=[a.address, b.address]) assert (len(a.data), len(b.data), len(c.data)) == (98, 2, 0) with pytest.raises(KeyError): @@ -3685,12 +3683,12 @@ async def test_rebalance_missing_data1(s, a, b): @gen_cluster(client=True, config=NO_AMM) async def test_rebalance_missing_data2(c, s, a, b): - """keys exist but belong to unfinished futures. Unlike Client.rebalance(), - Scheduler.rebalance() does not wait for unfinished futures. + """keys exist but belong to unfinished tasks. Unlike Client.rebalance(), + Scheduler.rebalance() does not wait for unfinished tasks. """ - futures = c.map(slowinc, range(10), delay=0.05, workers=a.address) + tasks = c.map(slowinc, range(10), delay=0.05, workers=a.address) await asyncio.sleep(0.1) - out = await s.rebalance(keys=[f.key for f in futures]) + out = await s.rebalance(keys=[f.key for f in tasks]) assert out["status"] == "partial-fail" assert 8 <= len(out["keys"]) <= 10 @@ -3702,7 +3700,7 @@ async def test_rebalance_raises_missing_data3(c, s, a, b, explicit): actual data movement runs. There is an error message only if the keys are explicitly listed in the API call. """ - futures = await c.scatter(range(100), workers=[a.address]) + tasks = await c.scatter(range(100), workers=[a.address]) if explicit: pytest.xfail( @@ -3711,15 +3709,15 @@ async def test_rebalance_raises_missing_data3(c, s, a, b, explicit): partial-fail is very timing sensitive and subject to a race condition. This test assumes that the data is freed before the rebalance get_data requests come in but merely deleting - the futures is not sufficient to guarantee this""" + the tasks is not sufficient to guarantee this""" ) - keys = [f.key for f in futures] - del futures + keys = [f.key for f in tasks] + del tasks out = await s.rebalance(keys=keys) assert out["status"] == "partial-fail" assert 1 <= len(out["keys"]) <= 100 else: - del futures + del tasks out = await s.rebalance() assert out == {"status": "OK"} @@ -3735,7 +3733,7 @@ async def test_rebalance_no_workers(s): config=merge(NO_AMM, {"distributed.worker.memory.rebalance.measure": "managed"}), ) async def test_rebalance_no_limit(c, s, a, b): - futures = await c.scatter(range(100), workers=[a.address]) + tasks = await c.scatter(range(100), workers=[a.address]) assert len(a.data) == 100 assert len(b.data) == 0 await s.rebalance() @@ -3778,11 +3776,11 @@ async def test_rebalance_no_recipients(c, s, a, b): ) async def test_rebalance_skip_recipient(client, s, a, b, c): """A recipient is skipped because it already holds a copy of the key to be sent""" - futures = await client.scatter(range(10), workers=[a.address]) - await client.replicate(futures[0:2], workers=[a.address, b.address]) - await client.replicate(futures[2:4], workers=[a.address, c.address]) + tasks = await client.scatter(range(10), workers=[a.address]) + await client.replicate(tasks[0:2], workers=[a.address, b.address]) + await client.replicate(tasks[2:4], workers=[a.address, c.address]) assert (len(a.data), len(b.data), len(c.data)) == (10, 2, 2) - await client.rebalance(futures[:2]) + await client.rebalance(tasks[:2]) assert (len(a.data), len(b.data), len(c.data)) == (8, 2, 4) @@ -3793,11 +3791,11 @@ async def test_rebalance_skip_recipient(client, s, a, b, c): ) async def test_rebalance_skip_all_recipients(c, s, a, b): """All recipients are skipped because they already hold copies""" - futures = await c.scatter(range(10), workers=[a.address]) - await wait(futures) - await c.replicate([futures[0]]) + tasks = await c.scatter(range(10), workers=[a.address]) + await wait(tasks) + await c.replicate([tasks[0]]) assert (len(a.data), len(b.data)) == (10, 1) - await c.rebalance(futures[:2]) + await c.rebalance(tasks[:2]) assert (len(a.data), len(b.data)) == (9, 2) @@ -3840,21 +3838,21 @@ async def test_rebalance_least_recently_inserted_sender_min(c, s, *_): 2. workers below sender-min are never senders """ a, b = s.workers - small_futures = c.map(lambda _: "x", range(10), workers=[a]) - await wait(small_futures) + small_tasks = c.map(lambda _: "x", range(10), workers=[a]) + await wait(small_tasks) await assert_ndata(c, {a: 10, b: 0}) await s.rebalance() await assert_ndata(c, {a: 10, b: 0}) - large_future = c.submit(lambda: "x" * (300 * 2**20), workers=[a]) - await wait([large_future]) + large_task = c.submit(lambda: "x" * (300 * 2**20), workers=[a]) + await wait([large_task]) await assert_memory(s, "managed", 300, 301) await assert_ndata(c, {a: 11, b: 0}) await s.rebalance() await assert_ndata(c, {a: 1, b: 10}) has_what = await c.has_what() - assert has_what[a] == (large_future.key,) - assert sorted(has_what[b]) == sorted(f.key for f in small_futures) + assert has_what[a] == (large_task.key,) + assert sorted(has_what[b]) == sorted(f.key for f in small_tasks) @gen_cluster(client=True) @@ -4157,8 +4155,8 @@ async def test_avoid_paused_workers(c, s, w1, w2, w3): w2.status = Status.paused while s.workers[w2.address].status != Status.paused: await asyncio.sleep(0.01) - futures = c.map(slowinc, range(8), delay=0.1) - await wait(futures) + tasks = c.map(slowinc, range(8), delay=0.1) + await wait(tasks) assert w1.data assert not w2.data assert w3.data @@ -4431,14 +4429,14 @@ async def test_KilledWorker_informative_message(s, a, b): @gen_cluster(client=True) async def test_count_task_prefix(c, s, a, b): - futures = c.map(inc, range(10)) - await c.gather(futures) + tasks = c.map(inc, range(10)) + await c.gather(tasks) assert s.task_prefixes["inc"].state_counts["memory"] == 10 assert s.task_prefixes["inc"].state_counts["erred"] == 0 - futures = c.map(inc, range(10, 20)) - await c.gather(futures) + tasks = c.map(inc, range(10, 20)) + await c.gather(tasks) assert s.task_prefixes["inc"].state_counts["memory"] == 20 assert s.task_prefixes["inc"].state_counts["erred"] == 0 @@ -4623,7 +4621,7 @@ async def test_client_desires_keys_creates_ts(c, s, a, b): test_scheduler.py::test_scatter_creates_ts test_spans.py::test_client_desires_keys_creates_ts """ - x = Future(key="x") + x = Task(key="x") await wait_for_state("x", "released", s) assert s.tasks["x"].run_spec is None async with Client(s.address, asynchronous=True) as c2: @@ -4677,7 +4675,7 @@ async def test_refuse_to_schedule_huge_task(c, s, *workers, finalize): match += r".* you called client.compute()" else: bg = bg.repartition(npartitions=1).persist() - fut = list(c.futures_of(bg))[0] + fut = list(c.tasks_of(bg))[0] with pytest.raises(MemoryError, match=match): await fut diff --git a/distributed/tests/test_semaphore.py b/distributed/tests/test_semaphore.py index af759832a0..f81355c9ab 100644 --- a/distributed/tests/test_semaphore.py +++ b/distributed/tests/test_semaphore.py @@ -76,8 +76,8 @@ def f(x, semaphore): sem = await Semaphore(max_leases=2, name="x") assert s.extensions["semaphores"]._semaphore_exists("x") - futures = c.map(f, list(range(10)), semaphore=sem) - await c.gather(futures) + tasks = c.map(f, list(range(10)), semaphore=sem) + await c.gather(tasks) @gen_cluster(client=True) @@ -151,10 +151,10 @@ def f(x, sem, kill_address): os.kill(os.getpid(), signal.SIGTERM) return x - futures = client.map( + tasks = client.map( f, range(10), sem=sem, kill_address=workers[0]["address"] ) - results = client.gather(futures) + results = client.gather(tasks) assert sorted(results) == list(range(10)) @@ -171,23 +171,23 @@ def f(x, release=True): return True sem = await Semaphore(name="x") - futures = c.map(f, list(range(10))) - assert all(await c.gather(futures)) + tasks = c.map(f, list(range(10))) + assert all(await c.gather(tasks)) # Clean-up the state, otherwise we would get the same result when calling `f` with the same arguments - del futures + del tasks assert len(s.extensions["semaphores"].leases["x"]) == 0 assert await sem.acquire() assert len(s.extensions["semaphores"].leases["x"]) == 1 - futures = c.map(f, list(range(10))) - assert not any(await c.gather(futures)) + tasks = c.map(f, list(range(10))) + assert not any(await c.gather(tasks)) assert await sem.release() is True - del futures + del tasks - futures = c.map(f, list(range(10)), release=False) - result = await c.gather(futures) + tasks = c.map(f, list(range(10)), release=False) + result = await c.gather(tasks) assert result.count(True) == 1 assert result.count(False) == 9 @@ -264,8 +264,8 @@ def f(x, sem): sem = await Semaphore(max_leases=3, name="x") inpt = list(range(20)) - futures = c.map(f, inpt, sem=sem) - assert sorted(await c.gather(futures)) == inpt + tasks = c.map(f, inpt, sem=sem) + assert sorted(await c.gather(tasks)) == inpt assert not s.extensions["semaphores"].leases["x"] await sem.acquire() @@ -408,13 +408,13 @@ def observe_state(sem): client.set_metadata("release", True) async with Worker(s.address) as observer: - futures = c.map( + tasks = c.map( guaranteed_lease_timeout, range(2), sem=sem, workers=[a.address, b.address] ) fut_observe = c.submit(observe_state, sem=sem, workers=[observer.address]) with captured_logger("distributed.semaphore", level=logging.DEBUG) as caplog: - payload, _ = await c.gather([futures, fut_observe]) + payload, _ = await c.gather([tasks, fut_observe]) logs = caplog.getvalue().split("\n") timeouts = [log for log in logs if "timed out" in log] diff --git a/distributed/tests/test_spans.py b/distributed/tests/test_spans.py index 742f81f70a..4cd1dc94a5 100644 --- a/distributed/tests/test_spans.py +++ b/distributed/tests/test_spans.py @@ -8,7 +8,7 @@ from dask import delayed import distributed -from distributed import Client, Event, Future, Worker, span, wait +from distributed import Client, Event, Task, Worker, span, wait from distributed.diagnostics.plugin import SchedulerPlugin from distributed.utils_test import ( NoSchedulerDelayWorker, @@ -396,7 +396,7 @@ async def test_client_desires_keys_creates_ts(c, s, a, b): test_spans.py::test_scatter_creates_ts test_spans.py::test_scatter_creates_tg """ - x = Future(key="x") + x = Task(key="x") await wait_for_state("x", "released", s) assert s.tasks["x"].group.span_id is None async with Client(s.address, asynchronous=True) as c2: @@ -416,7 +416,7 @@ async def test_client_desires_keys_creates_tg(c, s, a, b): test_spans.py::test_scatter_creates_ts test_spans.py::test_scatter_creates_tg """ - x0 = Future(key="x-0") + x0 = Task(key="x-0") await wait_for_state("x-0", "released", s) assert s.tasks["x-0"].group.span_id is None x1 = c.submit(inc, 1, key="x-1") diff --git a/distributed/tests/test_steal.py b/distributed/tests/test_steal.py index 1153134d10..2ccd56be37 100644 --- a/distributed/tests/test_steal.py +++ b/distributed/tests/test_steal.py @@ -29,7 +29,7 @@ wait, worker_client, ) -from distributed.client import Future +from distributed.client import Task from distributed.compatibility import LINUX from distributed.core import Status from distributed.metrics import time @@ -66,8 +66,8 @@ @gen_cluster(client=True, nthreads=[("", 2), ("", 2)]) async def test_work_stealing(c, s, a, b): [x] = await c._scatter([1], workers=a.address) - futures = c.map(slowadd, range(50), [x] * 50) - await wait(futures) + tasks = c.map(slowadd, range(50), [x] * 50) + await wait(tasks) assert len(a.data) > 10 assert len(b.data) > 10 @@ -78,8 +78,8 @@ async def test_dont_steal_expensive_data_fast_computation(c, s, a, b): x = c.submit(np.arange, 1000000, workers=a.address) await wait([x]) - future = c.submit(np.sum, [1], workers=a.address) # learn that sum is fast - await wait([future]) + task = c.submit(np.sum, [1], workers=a.address) # learn that sum is fast + await wait([task]) cheap = [ c.submit(np.sum, x, pure=False, workers=a.address, allow_other_workers=True) @@ -96,10 +96,10 @@ async def test_steal_cheap_data_slow_computation(c, s, a, b): x = c.submit(slowinc, 100, delay=0.1) # learn that slowinc is slow await wait(x) - futures = c.map( + tasks = c.map( slowinc, range(10), delay=0.1, workers=a.address, allow_other_workers=True ) - await wait(futures) + await wait(tasks) assert abs(len(a.data) - len(b.data)) <= 5 @@ -273,10 +273,10 @@ def blocked_task(x, lock): config={"distributed.scheduler.work-stealing-interval": "10ms"}, ) async def test_eventually_steal_unknown_functions(c, s, a, b): - futures = c.map( + tasks = c.map( slowinc, range(10), delay=0.1, workers=a.address, allow_other_workers=True ) - await wait(futures) + await wait(tasks) assert not s.unknown_durations assert len(a.data) >= 3, [len(a.data), len(b.data)] assert len(b.data) >= 3, [len(a.data), len(b.data)] @@ -285,14 +285,14 @@ async def test_eventually_steal_unknown_functions(c, s, a, b): @pytest.mark.skip(reason="") @gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 3) async def test_steal_related_tasks(e, s, a, b, c): - futures = e.map( + tasks = e.map( slowinc, range(20), delay=0.05, workers=a.address, allow_other_workers=True ) - await wait(futures) + await wait(tasks) nearby = 0 - for f1, f2 in sliding_window(2, futures): + for f1, f2 in sliding_window(2, tasks): if s.tasks[f1.key].who_has == s.tasks[f2.key].who_has: nearby += 1 @@ -307,12 +307,12 @@ def do_nothing(x, y=None): xs = c.map(do_nothing, range(10), workers=workers[0].address) await wait(xs) - futures = c.map(do_nothing, range(100), y=xs) + tasks = c.map(do_nothing, range(100), y=xs) - await wait(futures) + await wait(tasks) assert len(set.union(*(s.tasks[x.key].who_has for x in xs))) == 1 - assert len(s.workers[workers[0].address].has_what) == len(xs) + len(futures) + assert len(s.workers[workers[0].address].has_what) == len(xs) + len(tasks) @gen_cluster(client=True, nthreads=[("", 1)]) @@ -334,7 +334,7 @@ def fast_blocked(i, x): # fast, the standard mechanism shouldn't allow stealing sleep(0.01) - futures = c.map( + tasks = c.map( fast_blocked, range(50), x=x, @@ -348,7 +348,7 @@ def fast_blocked(i, x): while len(s.tasks) < 51: await asyncio.sleep(0.01) b.block_get_data.set() - await wait(futures) + await wait(tasks) # Note: x may now be on a, b, or both, depending if the Active Memory Manager # got to run or not @@ -362,8 +362,8 @@ def fast_blocked(i, x): async def test_new_worker_steals(c, s, a): await wait(c.submit(slowinc, 1, delay=0.01)) - futures = c.map(slowinc, range(100), delay=0.05) - total = c.submit(sum, futures) + tasks = c.map(slowinc, range(100), delay=0.05) + total = c.submit(sum, tasks) while len(a.state.tasks) < 10: await asyncio.sleep(0.01) @@ -384,16 +384,16 @@ async def test_new_worker_steals(c, s, a): async def test_work_steal_no_kwargs(c, s, a, b): await wait(c.submit(slowinc, 1, delay=0.05)) - futures = c.map( + tasks = c.map( slowinc, range(100), workers=a.address, allow_other_workers=True, delay=0.05 ) - await wait(futures) + await wait(tasks) assert 20 < len(a.data) < 80 assert 20 < len(b.data) < 80 - total = c.submit(sum, futures) + total = c.submit(sum, tasks) result = await total assert result == sum(map(inc, range(100))) @@ -401,10 +401,10 @@ async def test_work_steal_no_kwargs(c, s, a, b): @gen_cluster(client=True, nthreads=[("127.0.0.1", 1), ("127.0.0.1", 2)]) async def test_dont_steal_worker_restrictions(c, s, a, b): - future = c.submit(slowinc, 1, delay=0.10, workers=a.address) - await future + task = c.submit(slowinc, 1, delay=0.10, workers=a.address) + await task - futures = c.map(slowinc, range(100), delay=0.1, workers=a.address) + tasks = c.map(slowinc, range(100), delay=0.1, workers=a.address) while len(a.state.tasks) + len(b.state.tasks) < 100: await asyncio.sleep(0.01) @@ -424,11 +424,11 @@ async def test_dont_steal_worker_restrictions(c, s, a, b): client=True, nthreads=[("127.0.0.1", 1), ("127.0.0.1", 2), ("127.0.0.1", 2)] ) async def test_steal_worker_restrictions(c, s, wa, wb, wc): - future = c.submit(slowinc, 1, delay=0.1, workers={wa.address, wb.address}) - await future + task = c.submit(slowinc, 1, delay=0.1, workers={wa.address, wb.address}) + await task ntasks = 100 - futures = c.map(slowinc, range(ntasks), delay=0.1, workers={wa.address, wb.address}) + tasks = c.map(slowinc, range(ntasks), delay=0.1, workers={wa.address, wb.address}) while sum(len(w.state.tasks) for w in [wa, wb, wc]) < ntasks: await asyncio.sleep(0.01) @@ -449,10 +449,10 @@ async def test_steal_worker_restrictions(c, s, wa, wb, wc): @pytest.mark.skipif(not LINUX, reason="Need 127.0.0.2 to mean localhost") @gen_cluster(client=True, nthreads=[("127.0.0.1", 1), ("127.0.0.2", 1)]) async def test_dont_steal_host_restrictions(c, s, a, b): - future = c.submit(slowinc, 1, delay=0.10, workers=a.address) - await future + task = c.submit(slowinc, 1, delay=0.10, workers=a.address) + await task - futures = c.map(slowinc, range(100), delay=0.1, workers="127.0.0.1") + tasks = c.map(slowinc, range(100), delay=0.1, workers="127.0.0.1") while len(a.state.tasks) + len(b.state.tasks) < 100: await asyncio.sleep(0.01) assert len(a.state.tasks) == 100 @@ -468,11 +468,11 @@ async def test_dont_steal_host_restrictions(c, s, a, b): @pytest.mark.skipif(not LINUX, reason="Need 127.0.0.2 to mean localhost") @gen_cluster(client=True, nthreads=[("127.0.0.1", 1), ("127.0.0.2", 2)]) async def test_steal_host_restrictions(c, s, wa, wb): - future = c.submit(slowinc, 1, delay=0.10, workers=wa.address) - await future + task = c.submit(slowinc, 1, delay=0.10, workers=wa.address) + await task ntasks = 100 - futures = c.map(slowinc, range(ntasks), delay=0.1, workers="127.0.0.1") + tasks = c.map(slowinc, range(ntasks), delay=0.1, workers="127.0.0.1") while len(wa.state.tasks) < ntasks: await asyncio.sleep(0.01) assert len(wa.state.tasks) == ntasks @@ -494,10 +494,10 @@ async def test_steal_host_restrictions(c, s, wa, wb): client=True, nthreads=[("127.0.0.1", 1, {"resources": {"A": 2}}), ("127.0.0.1", 1)] ) async def test_dont_steal_resource_restrictions(c, s, a, b): - future = c.submit(slowinc, 1, delay=0.10, workers=a.address) - await future + task = c.submit(slowinc, 1, delay=0.10, workers=a.address) + await task - futures = c.map(slowinc, range(100), delay=0.1, resources={"A": 1}) + tasks = c.map(slowinc, range(100), delay=0.1, resources={"A": 1}) while len(a.state.tasks) + len(b.state.tasks) < 100: await asyncio.sleep(0.01) assert len(a.state.tasks) == 100 @@ -512,10 +512,10 @@ async def test_dont_steal_resource_restrictions(c, s, a, b): @gen_cluster(client=True, nthreads=[("127.0.0.1", 1, {"resources": {"A": 2}})]) async def test_steal_resource_restrictions(c, s, a): - future = c.submit(slowinc, 1, delay=0.10, workers=a.address) - await future + task = c.submit(slowinc, 1, delay=0.10, workers=a.address) + await task - futures = c.map(slowinc, range(100), delay=0.2, resources={"A": 1}) + tasks = c.map(slowinc, range(100), delay=0.2, resources={"A": 1}) while len(a.state.tasks) < 101: await asyncio.sleep(0.01) assert len(a.state.tasks) == 101 @@ -531,10 +531,10 @@ async def test_steal_resource_restrictions(c, s, a): @gen_cluster(client=True, nthreads=[("127.0.0.1", 1, {"resources": {"A": 2, "C": 1}})]) async def test_steal_resource_restrictions_asym_diff(c, s, a): # See https://github.com/dask/distributed/issues/5565 - future = c.submit(slowinc, 1, delay=0.10, workers=a.address) - await future + task = c.submit(slowinc, 1, delay=0.10, workers=a.address) + await task - futures = c.map(slowinc, range(100), delay=0.2, resources={"A": 1}) + tasks = c.map(slowinc, range(100), delay=0.2, resources={"A": 1}) while len(a.state.tasks) < 101: await asyncio.sleep(0.01) assert len(a.state.tasks) == 101 @@ -558,8 +558,8 @@ def slow(x): sleep(y) return y - futures = c.map(slow, range(100)) - await wait(futures) + tasks = c.map(slow, range(100)) + await wait(tasks) durations = [sum(w.data.values()) for w in workers] assert max(durations) / min(durations) < 3 @@ -567,11 +567,11 @@ def slow(x): @gen_cluster(client=True, nthreads=[("127.0.0.1", 4)] * 2) async def test_dont_steal_executing_tasks(c, s, a, b): - futures = c.map( + tasks = c.map( slowinc, range(4), delay=0.1, workers=a.address, allow_other_workers=True ) - await wait(futures) + await wait(tasks) assert len(a.data) == 4 assert len(b.data) == 0 @@ -580,15 +580,15 @@ async def test_dont_steal_executing_tasks(c, s, a, b): async def test_dont_steal_executing_tasks_2(c, s, a, b): steal = s.extensions["stealing"] - future = c.submit(slowinc, 1, delay=0.5, workers=a.address) + task = c.submit(slowinc, 1, delay=0.5, workers=a.address) while not a.state.executing_count: await asyncio.sleep(0.01) steal.move_task_request( - s.tasks[future.key], s.workers[a.address], s.workers[b.address] + s.tasks[task.key], s.workers[a.address], s.workers[b.address] ) await asyncio.sleep(0.1) - assert a.state.tasks[future.key].state == "executing" + assert a.state.tasks[task.key].state == "executing" assert not b.state.executing_count @@ -604,9 +604,9 @@ async def test_dont_steal_few_saturated_tasks_many_workers(c, s, a, *rest): x = c.submit(mul, b"0", 100000000, workers=a.address) # 100 MB await wait(x) - futures = [c.submit(slowidentity, x, pure=False, delay=0.2) for i in range(2)] + tasks = [c.submit(slowidentity, x, pure=False, delay=0.2) for i in range(2)] - await wait(futures) + await wait(tasks) assert len(a.data) == 3 assert not any(w.state.tasks for w in rest) @@ -625,7 +625,7 @@ async def test_steal_when_more_tasks(c, s, a, *rest): x = c.submit(mul, b"0", 50000000, workers=a.address) # 50 MB await wait(x) - futures = [c.submit(slowidentity, x, pure=False, delay=0.2) for i in range(20)] + tasks = [c.submit(slowidentity, x, pure=False, delay=0.2) for i in range(20)] start = time() while not any(w.state.tasks for w in rest): @@ -667,18 +667,18 @@ def slow2(x): # tasks are all on the scheduler. # Related https://github.com/dask/distributed/pull/5443 await ext.stop() - futures = [c.submit(slowidentity, x, pure=False, delay=0.2) for i in range(10)] - future = c.submit(slow2, x, priority=-1) + tasks = [c.submit(slowidentity, x, pure=False, delay=0.2) for i in range(10)] + task = c.submit(slow2, x, priority=-1) - while future.key not in s.tasks: + while task.key not in s.tasks: await asyncio.sleep(0.01) # Now call it once explicitly to move the heavy task ext.balance() while not any(w.state.tasks for w in rest): await asyncio.sleep(0.01) - # good future moves first - assert any(future.key in w.state.tasks for w in rest) + # good task moves first + assert any(task.key in w.state.tasks for w in rest) async def assert_balanced(inp, expected, c, s, *workers): @@ -691,7 +691,7 @@ def block(*args, event, **kwargs): counter = itertools.count() - futures_per_worker = defaultdict(list) + tasks_per_worker = defaultdict(list) for w, tasks in zip(workers, inp): for t in sorted(tasks, reverse=True): if t: @@ -711,13 +711,13 @@ def block(*args, event, **kwargs): pure=False, priority=-i, ) - futures_per_worker[w].append(f) + tasks_per_worker[w].append(f) # Make sure all tasks are scheduled on the workers - # We are relying on the futures not to be rootish (and thus not to remain in the + # We are relying on the tasks not to be rootish (and thus not to remain in the # scheduler-side queue) because they have worker restrictions wait_for_states = [] - for w, fs in futures_per_worker.items(): + for w, fs in tasks_per_worker.items(): for i, f in enumerate(fs): # Make sure the first task is executing, all others are ready state = "executing" if i == 0 else "ready" @@ -732,7 +732,7 @@ def block(*args, event, **kwargs): await steal.stop() await ev.set() - await c.gather([f for fs in futures_per_worker.values() for f in fs]) + await c.gather([f for fs in tasks_per_worker.values() for f in fs]) result = [ sorted( @@ -803,14 +803,14 @@ async def test_balance_(*args, **kwargs): @gen_cluster(client=True, nthreads=[("127.0.0.1", 1)] * 2, Worker=Nanny, timeout=60) async def test_restart(c, s, a, b): - futures = c.map( + tasks = c.map( slowinc, range(100), delay=0.01, workers=a.address, allow_other_workers=True ) while not s.workers[b.worker_address].processing: await asyncio.sleep(0.01) # Unknown tasks are never stolen therefore wait for a measurement - while not any(s.tasks[f.key].state == "memory" for f in futures): + while not any(s.tasks[f.key].state == "memory" for f in tasks): await asyncio.sleep(0.01) steal = s.extensions["stealing"] @@ -826,7 +826,7 @@ async def test_steal_twice(c, s, a, b): x = c.submit(inc, 1, workers=a.address) await wait(x) - futures = [c.submit(slowadd, x, i, delay=0.2) for i in range(100)] + tasks = [c.submit(slowadd, x, i, delay=0.2) for i in range(100)] while len(s.tasks) < 100: # tasks are all allocated await asyncio.sleep(0.01) @@ -845,7 +845,7 @@ async def test_steal_twice(c, s, a, b): workers = [stack.enter_async_context(Worker(s.address)) for _ in range(10)] workers = await asyncio.gather(*workers) - await wait(futures) + await wait(tasks) # Note: this includes a and b empty_workers = [ws for ws in s.workers.values() if not ws.has_what] @@ -875,8 +875,8 @@ async def test_paused_workers_must_not_steal(c, s, w1, w2, w3): x = c.submit(inc, 1, workers=w1.address) await wait(x) - futures = [c.submit(slowadd, x, i, delay=0.1) for i in range(10)] - await wait(futures) + tasks = [c.submit(slowadd, x, i, delay=0.1) for i in range(10)] + await wait(tasks) assert w1.data assert not w2.data @@ -885,12 +885,12 @@ async def test_paused_workers_must_not_steal(c, s, w1, w2, w3): @gen_cluster(client=True) async def test_dont_steal_already_released(c, s, a, b): - future = c.submit(slowinc, 1, delay=0.05, workers=a.address) - key = future.key + task = c.submit(slowinc, 1, delay=0.05, workers=a.address) + key = task.key while key not in a.state.tasks: await asyncio.sleep(0.05) - del future + del task while key in a.state.tasks and a.state.tasks[key].state != "released": await asyncio.sleep(0.05) @@ -983,7 +983,7 @@ async def test_lose_task(c, s, a, b): with captured_logger("distributed.stealing") as log: s.periodic_callbacks["stealing"].interval = 1 for _ in range(100): - futures = c.map( + tasks = c.map( slowinc, range(10), delay=0.01, @@ -992,7 +992,7 @@ async def test_lose_task(c, s, a, b): allow_other_workers=True, ) await asyncio.sleep(0.01) - del futures + del tasks out = log.getvalue() assert "Error" not in out @@ -1318,15 +1318,15 @@ async def test_correct_bad_time_estimate(c, s, *workers): This is done during reevaluate occupancy """ steal = s.extensions["stealing"] - future = c.submit(slowinc, 1, delay=0) - await wait(future) - futures = [c.submit(slowinc, future, delay=0.1, pure=False) for _ in range(20)] + task = c.submit(slowinc, 1, delay=0) + await wait(task) + tasks = [c.submit(slowinc, task, delay=0.1, pure=False) for _ in range(20)] while len(s.tasks) < 21: await asyncio.sleep(0) - assert not any(s.tasks[f.key] in steal.key_stealable for f in futures) + assert not any(s.tasks[f.key] in steal.key_stealable for f in tasks) await asyncio.sleep(0.5) - assert any(s.tasks[f.key] in steal.key_stealable for f in futures) - await wait(futures) + assert any(s.tasks[f.key] in steal.key_stealable for f in tasks) + await wait(tasks) assert all(w.data for w in workers), [sorted(w.data) for w in workers] @@ -1341,12 +1341,12 @@ def blocked(x, lock): # Setup all tasks on worker 0 such that victim/thief relation is the # same for all tasks. - futures = c.map(blocked, range(num_futs), lock=lock, workers=[a.address]) + tasks = c.map(blocked, range(num_futs), lock=lock, workers=[a.address]) # Ensure all tasks are assigned to the worker since otherwise the # move_task_request fails. while len(a.state.tasks) != num_futs: await asyncio.sleep(0.1) - tasks = [s.tasks[f.key] for f in futures] + tasks = [s.tasks[f.key] for f in tasks] w0 = s.workers[a.address] w1 = s.workers[b.address] # Generating the move task requests as fast as possible increases the @@ -1357,7 +1357,7 @@ def blocked(x, lock): # Therefore all stimulus IDs are stored here and must be unique stimulus_ids = {dct["stimulus_id"] for dct in steal.in_flight.values()} assert len(stimulus_ids) == num_futs - await c.cancel(futures) + await c.cancel(tasks) def test_steal_worker_state(ws_with_running_task): @@ -1702,14 +1702,14 @@ async def _dependency_balance_test_permutation( permutated_dependency_placement = [dependency_placement[i] for i in permutation] permutated_task_placement = [task_placement[i] for i in permutation] - dependency_futures = await _place_dependencies( + dependency_tasks = await _place_dependencies( dependencies, permutated_dependency_placement, c, s, workers ) - ev, futures_per_worker = await _place_tasks( + ev, tasks_per_worker = await _place_tasks( permutated_task_placement, permutated_dependency_placement, - dependency_futures, + dependency_tasks, c, s, workers, @@ -1729,7 +1729,7 @@ async def _dependency_balance_test_permutation( await steal.stop() await ev.set() - await c.gather([f for fs in futures_per_worker.values() for f in fs]) + await c.gather([f for fs in tasks_per_worker.values() for f in fs]) permutated_actual_placement = _get_task_placement(s, workers) actual_placement = [permutated_actual_placement[i] for i in inverse] @@ -1743,7 +1743,7 @@ async def _place_dependencies( c: Client, s: Scheduler, workers: Sequence[Worker], -) -> dict[str, Future]: +) -> dict[str, Task]: """Places the dependencies on the workers as specified. Parameters @@ -1756,7 +1756,7 @@ async def _place_dependencies( Returns ------- - Dictionary of futures matching the input dependencies. + Dictionary of tasks matching the input dependencies. See Also -------- @@ -1767,7 +1767,7 @@ async def _place_dependencies( for dependency in placed: dependencies_to_workers[dependency].add(workers[worker_idx].address) - futures = {} + tasks = {} for name, multiplier in dependencies.items(): key = f"dep-{name}" worker_addresses = dependencies_to_workers[name] @@ -1776,21 +1776,21 @@ async def _place_dependencies( workers=worker_addresses, broadcast=True, ) - futures[name] = futs[key] + tasks[name] = futs[key] - await c.gather(futures.values()) + await c.gather(tasks.values()) - return futures + return tasks async def _place_tasks( placement: list[list[list[str]]], dependency_placement: list[list[str]], - dependency_futures: Mapping[str, Future], + dependency_tasks: Mapping[str, Task], c: Client, s: Scheduler, workers: Sequence[Worker], -) -> tuple[Event, dict[Worker, list[Future]]]: +) -> tuple[Event, dict[Worker, list[Task]]]: """Places the tasks on the workers as specified. Parameters @@ -1801,12 +1801,12 @@ async def _place_tasks( dependency_placement List of list of dependencies to be placed on the worker corresponding to the index of the outer list. - dependency_futures - Mapping of dependency names to their corresponding futures. + dependency_tasks + Mapping of dependency names to their corresponding tasks. Returns ------- - Tuple of the event blocking the placed tasks and list of futures matching + Tuple of the event blocking the placed tasks and list of tasks matching the input task placement. See Also @@ -1819,7 +1819,7 @@ def block(*args, event, **kwargs): event.wait() counter = itertools.count() - futures_per_worker = defaultdict(list) + tasks_per_worker = defaultdict(list) for worker, tasks, placed_dependencies in zip( workers, placement, dependency_placement ): @@ -1835,7 +1835,7 @@ def block(*args, event, **kwargs): key = f"{compose_task_prefix(dependencies)}-{i}" f = c.submit( block, - [dependency_futures[dependency] for dependency in dependencies], + [dependency_tasks[dependency] for dependency in dependencies], event=ev, key=key, workers=worker.address, @@ -1843,13 +1843,13 @@ def block(*args, event, **kwargs): pure=False, priority=-i, ) - futures_per_worker[worker].append(f) + tasks_per_worker[worker].append(f) # Make sure all tasks are scheduled on the workers - # We are relying on the futures not to be rootish (and thus not to remain in the + # We are relying on the tasks not to be rootish (and thus not to remain in the # scheduler-side queue) because they have worker restrictions waits_for_state: list[Coroutine] = [] - for w, fs in futures_per_worker.items(): + for w, fs in tasks_per_worker.items(): waits_for_executing_state = [] for f in fs: # Every task should be either ready or executing @@ -1865,7 +1865,7 @@ def block(*args, event, **kwargs): *waits_for_state, ) - return ev, futures_per_worker + return ev, tasks_per_worker def _get_task_placement( diff --git a/distributed/tests/test_stress.py b/distributed/tests/test_stress.py index 53801943cc..96feca4517 100644 --- a/distributed/tests/test_stress.py +++ b/distributed/tests/test_stress.py @@ -149,7 +149,7 @@ async def test_stress_scatter_death(c, s, *workers): for i, (a, b) in enumerate(sliding_window(2, adds)) ] - futures = c.compute(adds) + tasks = c.compute(adds) del L del adds @@ -162,7 +162,7 @@ async def test_stress_scatter_death(c, s, *workers): await w.close() with suppress(CancelledError): - await c.gather(futures) + await c.gather(tasks) def vsum(*args): @@ -192,9 +192,9 @@ async def test_stress_communication(c, s, *workers): ys = [x + x.T for x in xs] z = da.blockwise(vsum, "ij", *concat(zip(ys, ["ij"] * n)), dtype="float64") - future = c.compute(z.sum()) + task = c.compute(z.sum()) - result = await future + result = await task assert isinstance(result, float) @@ -216,9 +216,9 @@ async def test_stress_steal(c, s, *workers): L = [delayed(slowsum)(part, delay=0.005) for part in sliding_window(5, L)] total = delayed(sum)(L) - future = c.compute(total) + task = c.compute(total) - while future.status != "finished": + while task.status != "finished": await asyncio.sleep(0.1) for _ in range(3): a = random.choice(workers) @@ -243,16 +243,16 @@ async def test_close_connections(c, s, *workers): x = x.map_blocks(slowinc, delay=0.1, dtype=x.dtype) x = x.sum() - future = c.compute(x) + task = c.compute(x) n = 0 - while not future.done(): + while not task.done(): n += 1 await asyncio.sleep(0.5) worker = random.choice(list(workers)) for comm in worker._comms: comm.abort() - await future + await task assert n > 5 @@ -282,7 +282,7 @@ async def test_no_delay_during_large_transfer(c, s, w): server._last_tick = time() with ResourceProfiler(dt=0.01) as rprof: - future = await c.scatter(x, direct=True, hash=False) + task = await c.scatter(x, direct=True, hash=False) await asyncio.sleep(0.5) rprof.close() diff --git a/distributed/tests/test_threadpoolexecutor.py b/distributed/tests/test_threadpoolexecutor.py index 163a64acbd..fe8057d467 100644 --- a/distributed/tests/test_threadpoolexecutor.py +++ b/distributed/tests/test_threadpoolexecutor.py @@ -31,7 +31,7 @@ def f(): def test_shutdown_timeout(): e = ThreadPoolExecutor(1) - futures = [e.submit(sleep, 0.1 * i) for i in range(1, 3, 1)] + tasks = [e.submit(sleep, 0.1 * i) for i in range(1, 3, 1)] sleep(0.01) start = time() @@ -42,7 +42,7 @@ def test_shutdown_timeout(): def test_shutdown_timeout_raises(): e = ThreadPoolExecutor(1) - futures = [e.submit(sleep, 0.1 * i) for i in range(1, 3, 1)] + tasks = [e.submit(sleep, 0.1 * i) for i in range(1, 3, 1)] sleep(0.05) start = time() @@ -53,7 +53,7 @@ def test_shutdown_timeout_raises(): def test_shutdown_wait(): e = ThreadPoolExecutor(1) - future = e.submit(sleep, 1) + task = e.submit(sleep, 1) sleep(0.01) start = time() @@ -80,11 +80,11 @@ def f(): assert threading.current_thread() in e._threads return threading.current_thread() - future = e.submit(f) + task = e.submit(f) for _ in range(6): e.submit(sleep, 0.4) start = time() - special_thread = future.result() + special_thread = task.result() stop = time() assert 0.2 < stop - start < 0.6 @@ -96,8 +96,8 @@ def f(): sleep(0.01) return threading.current_thread() - futures = [e.submit(f) for _ in range(10)] - assert special_thread in {future.result() for future in futures} + tasks = [e.submit(f) for _ in range(10)] + assert special_thread in {task.result() for task in tasks} def test_secede_rejoin_quiet(): @@ -113,8 +113,8 @@ def f(): assert threading.current_thread() in e._threads return threading.current_thread() - future = e.submit(f) - result = future.result() + task = e.submit(f) + result = task.result() def test_rejoin_idempotent(): @@ -126,8 +126,8 @@ def f(): rejoin() return 1 - future = e.submit(f) - result = future.result() + task = e.submit(f) + result = task.result() def test_thread_name(): diff --git a/distributed/tests/test_tls_functional.py b/distributed/tests/test_tls_functional.py index ac7a0a17ab..565d321a5e 100644 --- a/distributed/tests/test_tls_functional.py +++ b/distributed/tests/test_tls_functional.py @@ -38,12 +38,12 @@ async def test_Queue(c, s, a, b): size = await x.qsize() assert size == 0 - future = c.submit(inc, 1) + task = c.submit(inc, 1) - await x.put(future) + await x.put(task) - future2 = await x.get() - assert future.key == future2.key + task2 = await x.get() + assert task.key == task2.key @gen_tls_cluster(client=True) @@ -119,7 +119,7 @@ async def test_rebalance(c, s, a, b): """ assert a.address.startswith("tls://") - futures = await c.scatter(range(100), workers=[a.address]) + tasks = await c.scatter(range(100), workers=[a.address]) assert len(a.data) == 100 assert len(b.data) == 0 await c.rebalance() @@ -130,9 +130,9 @@ async def test_rebalance(c, s, a, b): @gen_tls_cluster(client=True, nthreads=[("tls://127.0.0.1", 2)] * 2) async def test_work_stealing(c, s, a, b): [x] = await c._scatter([1], workers=a.address) - futures = c.map(slowadd, range(50), [x] * 50, delay=0.1) + tasks = c.map(slowadd, range(50), [x] * 50, delay=0.1) await asyncio.sleep(0.1) - await wait(futures) + await wait(tasks) assert len(a.data) > 10 assert len(b.data) > 10 @@ -169,8 +169,8 @@ def func(): xx, yy = ee.gather([x, y]) return xx, yy - future = c.submit(func) - result = await future + task = c.submit(func) + result = await task assert result == (2, 3) @@ -182,8 +182,8 @@ def mysum(): with c.get_executor() as e: return sum(e.map(double, range(30))) - future = c.submit(mysum) - result = await future + task = c.submit(mysum) + result = await task assert result == 30 * 29 diff --git a/distributed/tests/test_utils.py b/distributed/tests/test_utils.py index 37e1168cf1..ac44003670 100644 --- a/distributed/tests/test_utils.py +++ b/distributed/tests/test_utils.py @@ -15,7 +15,9 @@ import xml from array import array from collections import deque -from concurrent.futures import Executor, Future as ConcurrentFuture, ThreadPoolExecutor +from concurrent.futures import Executor +from concurrent.futures import Future as ConcurrentFuture +from concurrent.futures import ThreadPoolExecutor from contextvars import ContextVar from time import sleep from unittest import mock diff --git a/distributed/tests/test_utils_test.py b/distributed/tests/test_utils_test.py index 7530b82ce4..fd8647c93b 100755 --- a/distributed/tests/test_utils_test.py +++ b/distributed/tests/test_utils_test.py @@ -170,8 +170,8 @@ async def test_gen_cluster_without_client(s, a, b): assert set(s.workers) == {w.address for w in [a, b]} async with Client(s.address, asynchronous=True) as c: - future = c.submit(lambda x: x + 1, 1) - result = await future + task = c.submit(lambda x: x + 1, 1) + result = await task assert result == 2 diff --git a/distributed/tests/test_variable.py b/distributed/tests/test_variable.py index 00618da70d..98abdfe9fb 100644 --- a/distributed/tests/test_variable.py +++ b/distributed/tests/test_variable.py @@ -21,16 +21,16 @@ async def test_variable(c, s, a, b): xx = Variable("x") assert x.client is c - future = c.submit(inc, 1) + task = c.submit(inc, 1) - await x.set(future) - future2 = await xx.get() - assert future.key == future2.key + await x.set(task) + task2 = await xx.get() + assert task.key == task2.key - del future, future2 + del task, task2 await asyncio.sleep(0.1) - assert s.tasks # future still present + assert s.tasks # task still present x.delete() @@ -84,29 +84,29 @@ async def test_queue_with_data(c, s, a, b): def test_sync(client): - future = client.submit(lambda x: x + 1, 10) + task = client.submit(lambda x: x + 1, 10) x = Variable("x") xx = Variable("x") - x.set(future) - future2 = xx.get() + x.set(task) + task2 = xx.get() - assert future2.result() == 11 + assert task2.result() == 11 @gen_cluster() -async def test_hold_futures(s, a, b): +async def test_hold_tasks(s, a, b): async with Client(s.address, asynchronous=True) as c1: - future = c1.submit(lambda x: x + 1, 10) + task = c1.submit(lambda x: x + 1, 10) x1 = Variable("x") - await x1.set(future) + await x1.set(task) del x1 await asyncio.sleep(0.1) async with Client(s.address, asynchronous=True) as c2: x2 = Variable("x") - future2 = await x2.get() - result = await future2 + task2 = await x2.get() + result = await task2 assert result == 11 @@ -158,13 +158,13 @@ async def test_cleanup(c, s, a, b): del x await asyncio.sleep(0.1) - t_future = xx = asyncio.ensure_future(vv._get()) + t_task = xx = asyncio.ensure_future(vv._get()) await asyncio.sleep(0) asyncio.ensure_future(v.set(y)) - future = await t_future - assert future.key == x_key - result = await future + task = await t_task + assert task.key == x_key + result = await task assert result == 11 @@ -182,12 +182,12 @@ def f(x): async def test_timeout_get(c, s, a, b): v = Variable("v") - tornado_future = v.get() + tornado_task = v.get() vv = Variable("v") await vv.set(1) - result = await tornado_future + result = await tornado_task assert result == 1 @@ -200,8 +200,8 @@ def f(i): with worker_client() as c: v = Variable("x", client=c) for _ in range(NITERS): - future = v.get() - x = future.result() + task = v.get() + x = task.result() y = c.submit(inc, x) v.set(y) sleep(0.01 * random.random()) @@ -213,8 +213,8 @@ def f(i): x = await c.scatter(1) await v.set(x) - futures = c.map(f, range(15)) - results = await c.gather(futures) + tasks = c.map(f, range(15)) + results = await c.gather(tasks) while "variable-x" in s.tasks: await asyncio.sleep(0.01) @@ -228,22 +228,22 @@ async def test_Future_knows_status_immediately(c, s, a, b): async with Client(s.address, asynchronous=True) as c2: v2 = Variable("x", client=c2) - future = await v2.get() - assert future.status == "finished" + task = await v2.get() + assert task.status == "finished" x = c.submit(div, 1, 0) await wait(x) await v.set(x) - future2 = await v2.get() - assert future2.status == "error" + task2 = await v2.get() + assert task2.status == "error" with pytest.raises(ZeroDivisionError): - await future2 + await task2 start = time() while True: # we learn about the true error eventually try: - await future2 + await task2 except ZeroDivisionError: break except Exception: @@ -252,30 +252,30 @@ async def test_Future_knows_status_immediately(c, s, a, b): @gen_cluster(client=True) -async def test_erred_future(c, s, a, b): - future = c.submit(div, 1, 0) +async def test_erred_task(c, s, a, b): + task = c.submit(div, 1, 0) var = Variable() - await var.set(future) + await var.set(task) await asyncio.sleep(0.1) - future2 = await var.get() + task2 = await var.get() with pytest.raises(ZeroDivisionError): - await future2.result() + await task2.result() - exc = await future2.exception() + exc = await task2.exception() assert isinstance(exc, ZeroDivisionError) -def test_future_erred_sync(client): - future = client.submit(div, 1, 0) +def test_task_erred_sync(client): + task = client.submit(div, 1, 0) var = Variable() - var.set(future) + var.set(task) sleep(0.1) - future2 = var.get() + task2 = var.get() with pytest.raises(ZeroDivisionError): - future2.result() + task2.result() @gen_cluster(client=True) @@ -283,10 +283,10 @@ async def test_variables_do_not_leak_client(c, s, a, b): # https://github.com/dask/distributed/issues/3899 clients_pre = set(s.clients) - # setup variable with future + # setup variable with task x = Variable("x") - future = c.submit(inc, 1) - await x.set(future) + task = c.submit(inc, 1) + await x.set(task) # complete teardown x.delete() diff --git a/distributed/tests/test_worker.py b/distributed/tests/test_worker.py index f67cd4c26f..4ee5cb494f 100644 --- a/distributed/tests/test_worker.py +++ b/distributed/tests/test_worker.py @@ -206,8 +206,8 @@ def g(): return foobar.x - future = c.submit(g, workers=a.address) - result = await future + task = c.submit(g, workers=a.address) + result = await task assert result == 123 await s.close() @@ -238,8 +238,8 @@ def g(): return foo.x - future = c.submit(g) - result = await future + task = c.submit(g) + result = await task assert result == 123 finally: sys.path.remove(dirname) @@ -263,8 +263,8 @@ def g(x): return testegg.inc(x) - future = c.submit(g, 10, workers=a.address) - result = await future + task = c.submit(g, 10, workers=a.address) + result = await task assert result == 10 + 1 await c.close() @@ -292,8 +292,8 @@ def g(x): return mytest.inc(x) - future = c.submit(g, 10, workers=a.address) - result = await future + task = c.submit(g, 10, workers=a.address) + result = await task assert result == 10 + 1 await c.close() @@ -609,8 +609,8 @@ def f(i): return thread_state.key - futures = [c.submit(f, i, key="x-%d" % i) for i in range(20)] - results = await c._gather(futures) + tasks = [c.submit(f, i, key="x-%d" % i) for i in range(20)] + results = await c._gather(tasks) assert list(results) == ["x-%d" % i for i in range(20)] @@ -648,8 +648,8 @@ async def test_Executor(c, s): async with Worker(s.address, executor=e) as w: assert w.executor is e - future = c.submit(inc, 1) - result = await future + task = c.submit(inc, 1) + result = await task assert result == 2 assert e._threads # had to do some work @@ -679,8 +679,8 @@ async def test_memory_limit_auto(s): async def test_inter_worker_communication(c, s, a, b): [x, y] = await c._scatter([1, 2], workers=a.address) - future = c.submit(add, x, y, workers=b.address) - result = await future + task = c.submit(add, x, y, workers=b.address) + result = await task assert result == 3 @@ -772,8 +772,8 @@ async def test_clean_nbytes(c, s, a, b): L = [delayed(add)(x, y) for x, y in sliding_window(2, L)] total = delayed(sum)(L) - future = c.compute(total) - await wait(future) + task = c.compute(total) + await wait(task) await asyncio.sleep(1) assert ( @@ -791,17 +791,17 @@ async def test_gather_many_small(c, s, a, *snd_workers, as_deps): scheduled all at once, they will result in a single call to gather_dep. """ a.state.transfer_incoming_count_limit = 2 - futures = await c.scatter( + tasks = await c.scatter( {f"x{i}": i for i in range(100)}, workers=[w.address for w in snd_workers], ) assert all(w.data for w in snd_workers) if as_deps: - future = c.submit(lambda _: None, futures, key="y", workers=[a.address]) - await wait(future) + task = c.submit(lambda _: None, tasks, key="y", workers=[a.address]) + await wait(task) else: - s.request_acquire_replicas(a.address, list(futures), stimulus_id="test") + s.request_acquire_replicas(a.address, list(tasks), stimulus_id="test") while len(a.data) < 100: await asyncio.sleep(0.01) @@ -903,8 +903,8 @@ class C: @gen_cluster(client=True) async def test_log_exception_on_failed_task(c, s, a, b): with captured_logger("distributed.worker") as logger: - future = c.submit(div, 1, 0) - await wait(future) + task = c.submit(div, 1, 0) + await wait(task) await asyncio.sleep(0.1) @@ -957,10 +957,10 @@ async def test_worker_death_timeout(): @gen_cluster(client=True) async def test_stop_doing_unnecessary_work(c, s, a, b): - futures = c.map(slowinc, range(1000), delay=0.01) + tasks = c.map(slowinc, range(1000), delay=0.01) await asyncio.sleep(0.1) - del futures + del tasks await async_poll_for(lambda: a.state.executing_count == 0, timeout=0.5) @@ -976,8 +976,8 @@ async def test_priorities(c, s, w): values.append(a2) values.append(b1) - futures = c.compute(values) - await wait(futures) + tasks = c.compute(values) + await wait(tasks) log = [ t[0] @@ -1032,8 +1032,8 @@ def __init__(self, data): def __sizeof__(self): raise TypeError("Hello") - future = c.submit(BadSize, 123) - result = await future + task = c.submit(BadSize, 123) + result = await task assert result.data == 123 @@ -1046,13 +1046,13 @@ async def test_pid(s, a, b): async def test_get_client(c, s, a, b): def f(x): cc = get_client() - future = cc.submit(inc, x) - return future.result() + task = cc.submit(inc, x) + return task.result() assert default_client() is c - future = c.submit(f, 10, workers=a.address) - result = await future + task = c.submit(f, 10, workers=a.address) + result = await task assert result == 11 assert a._client @@ -1072,11 +1072,11 @@ def f(x): def test_get_client_sync(client): def f(x): cc = get_client() - future = cc.submit(inc, x) - return future.result() + task = cc.submit(inc, x) + return task.result() - future = client.submit(f, 10) - assert future.result() == 11 + task = client.submit(f, 10) + assert task.result() == 11 @gen_cluster(client=True) @@ -1087,8 +1087,8 @@ async def f(): # if you do that. We really don't want users to do that. # https://github.com/dask/distributed/pull/6921/ client = get_client() - future = client.submit(inc, 10) - result = await future + task = client.submit(inc, 10) + result = await task return result results = await c.run(f) @@ -1101,8 +1101,8 @@ async def f(): def test_get_client_coroutine_sync(client, s, a, b): async def f(): client = await get_client() - future = client.submit(inc, 10) - result = await future + task = client.submit(inc, 10) + result = await task return result for w in [a, b]: @@ -1197,8 +1197,8 @@ async def test_scheduler_delay(c, s, a, b): }, ) async def test_statistical_profiling(c, s, a, b): - futures = c.map(slowinc, range(10), delay=0.1) - await wait(futures) + tasks = c.map(slowinc, range(10), delay=0.1) + await wait(tasks) profile = a.profile_keys["slowinc"] assert profile["count"] @@ -1236,8 +1236,8 @@ async def test_statistical_profiling_2(c, s, a, b): }, ) async def test_statistical_profiling_cycle(c, s, a, b): - futures = c.map(slowinc, range(20), delay=0.05) - await wait(futures) + tasks = c.map(slowinc, range(20), delay=0.05) + await wait(tasks) await asyncio.sleep(0.01) end = time() assert len(a.profile_history) > 3 @@ -1290,9 +1290,9 @@ async def test_scheduler_address_config(c, s): async def test_wait_for_outgoing(c, s, a, b): np = pytest.importorskip("numpy") x = np.random.random(10000000) - future = await c.scatter(x, workers=a.address) + task = await c.scatter(x, workers=a.address) - y = c.submit(inc, future, workers=b.address) + y = c.submit(inc, task, workers=b.address) await wait(y) assert len(b.transfer_incoming_log) == len(a.transfer_outgoing_log) == 1 @@ -1329,9 +1329,9 @@ async def test_avoid_oversubscription(c, s, *workers): x = c.submit(np.random.random, 1000000, workers=[workers[0].address]) await wait(x) - futures = [c.submit(len, x, pure=False, workers=[w.address]) for w in workers[1:]] + tasks = [c.submit(len, x, pure=False, workers=[w.address]) for w in workers[1:]] - await wait(futures) + await wait(tasks) # Original worker not responsible for all transfers assert len(workers[0].transfer_outgoing_log) < len(workers) - 2 @@ -1540,7 +1540,7 @@ def assert_amm_transfer_story(key: str, w_from: Worker, w_to: Worker) -> None: @pytest.mark.slow @gen_cluster(client=True) async def test_close_gracefully(c, s, a, b): - futures = c.map(slowinc, range(200), delay=0.1, workers=[b.address]) + tasks = c.map(slowinc, range(200), delay=0.1, workers=[b.address]) # Note: keys will appear in b.data several milliseconds before they switch to # status=memory in s.tasks. It's important to sample the in-memory keys from the @@ -1660,7 +1660,7 @@ async def f(ev): async def test_lifetime(c, s, a): # Note: test was occasionally failing with lifetime="1 seconds" async with Worker(s.address, lifetime="2 seconds") as b: - futures = c.map(slowinc, range(200), delay=0.1, workers=[b.address]) + tasks = c.map(slowinc, range(200), delay=0.1, workers=[b.address]) # Note: keys will appear in b.data several milliseconds before they switch to # status=memory in s.tasks. It's important to sample the in-memory keys from the @@ -1909,9 +1909,9 @@ def f(x): @gen_cluster(client=True, nthreads=[("127.0.0.1", 1)]) async def test_story(c, s, w): - future = c.submit(inc, 1) - await future - ts = w.state.tasks[future.key] + task = c.submit(inc, 1) + await task + ts = w.state.tasks[task.key] assert ts.state in str(w.state.story(ts)) assert w.state.story(ts) == w.state.story(ts.key) @@ -2179,12 +2179,12 @@ def get_thread_name(): nthreads=2, executor={"foo": ThreadPoolExecutor(1, thread_name_prefix="Dask-Foo-Threads")}, ): - futures = [] + tasks = [] with dask.annotate(executor="default"): - futures.append(c.submit(get_thread_name, pure=False)) + tasks.append(c.submit(get_thread_name, pure=False)) with dask.annotate(executor="foo"): - futures.append(c.submit(get_thread_name, pure=False)) - default_result, gpu_result = await c.gather(futures) + tasks.append(c.submit(get_thread_name, pure=False)) + default_result, gpu_result = await c.gather(tasks) assert "Dask-Default-Threads" in default_result assert "Dask-Foo-Threads" in gpu_result @@ -2192,10 +2192,10 @@ def get_thread_name(): @gen_cluster(client=True) async def test_bad_executor_annotation(c, s, a, b): with dask.annotate(executor="bad"): - future = c.submit(inc, 1) + task = c.submit(inc, 1) with pytest.raises(ValueError, match="Invalid executor 'bad'; expected one of: "): - await future - assert future.status == "error" + await task + assert task.status == "error" @gen_cluster(client=True) @@ -2204,13 +2204,13 @@ async def test_process_executor(c, s, a, b): a.executors["processes"] = e b.executors["processes"] = e - future = c.submit(os.getpid, pure=False) - assert (await future) == os.getpid() + task = c.submit(os.getpid, pure=False) + assert (await task) == os.getpid() with dask.annotate(executor="processes"): - future = c.submit(os.getpid, pure=False) + task = c.submit(os.getpid, pure=False) - assert (await future) != os.getpid() + assert (await task) != os.getpid() def kill_process(): @@ -2235,18 +2235,18 @@ async def test_process_executor_kills_process(c, s, a): with ProcessPoolExecutor() as e: a.executors["processes"] = e with dask.annotate(executor="processes", retries=1): - future = c.submit(kill_process) + task = c.submit(kill_process) msg = "A child process terminated abruptly, the process pool is not usable anymore" with pytest.raises(BrokenProcessPool, match=msg): - await future + await task with dask.annotate(executor="processes", retries=1): - future = c.submit(inc, 1) + task = c.submit(inc, 1) # The process pool is now unusable and the worker is effectively dead with pytest.raises(BrokenProcessPool, match=msg): - await future + await task def raise_exc(): @@ -2259,10 +2259,10 @@ async def test_process_executor_raise_exception(c, s, a, b): a.executors["processes"] = e b.executors["processes"] = e with dask.annotate(executor="processes", retries=1): - future = c.submit(raise_exc) + task = c.submit(raise_exc) with pytest.raises(RuntimeError, match="foo"): - await future + await task async def assert_task_states_on_worker( @@ -2335,7 +2335,7 @@ def raise_exc(*args): res.key: "error", } await assert_task_states_on_worker(expected_states, a) - # Expected states after we release references to the futures + # Expected states after we release references to the tasks f.release() g.release() @@ -2403,7 +2403,7 @@ def raise_exc(*args): res.key: "error", } await assert_task_states_on_worker(expected_states, a) - # Expected states after we release references to the futures + # Expected states after we release references to the tasks res.release() # We no longer hold any refs to f or g and B didn't have any errors. It @@ -2467,7 +2467,7 @@ def raise_exc(*args): res.key: "error", } await assert_task_states_on_worker(expected_states, a) - # Expected states after we release references to the futures + # Expected states after we release references to the tasks f.release() res.release() @@ -3334,7 +3334,7 @@ def fn(): await loop.run_in_executor(executor, fn) - async def set_future(): + async def set_task(): while True: try: await loop.run_in_executor(executor, sleep, 0.1) @@ -3347,7 +3347,7 @@ async def close(): # executor_wait is True by default but we want to be explicit here await w.close(executor_wait=True) - await asyncio.gather(block(), close(), set_future()) + await asyncio.gather(block(), close(), set_task()) @gen_cluster(nthreads=[]) diff --git a/distributed/tests/test_worker_client.py b/distributed/tests/test_worker_client.py index f5e72225c3..38d8845b67 100644 --- a/distributed/tests/test_worker_client.py +++ b/distributed/tests/test_worker_client.py @@ -64,35 +64,35 @@ async def func(x): async def test_scatter_from_worker(c, s, a, b): def func(): with worker_client() as c: - futures = c.scatter([1, 2, 3, 4, 5]) - assert isinstance(futures, (list, tuple)) - assert len(futures) == 5 + tasks = c.scatter([1, 2, 3, 4, 5]) + assert isinstance(tasks, (list, tuple)) + assert len(tasks) == 5 x = dict(get_worker().data) - y = {f.key: i for f, i in zip(futures, [1, 2, 3, 4, 5])} + y = {f.key: i for f, i in zip(tasks, [1, 2, 3, 4, 5])} assert x == y - total = c.submit(sum, futures) + total = c.submit(sum, tasks) return total.result() - future = c.submit(func) - result = await future + task = c.submit(func) + result = await task assert result == sum([1, 2, 3, 4, 5]) def func(): with worker_client() as c: correct = True for data in [[1, 2], (1, 2), {1, 2}]: - futures = c.scatter(data) - correct &= type(futures) == type(data) + tasks = c.scatter(data) + correct &= type(tasks) == type(data) o = object() - futures = c.scatter({"x": o}) + tasks = c.scatter({"x": o}) correct &= get_worker().data["x"] is o return correct - future = c.submit(func) - result = await future + task = c.submit(func) + result = await task assert result is True start = time() @@ -108,8 +108,8 @@ async def test_scatter_singleton(c, s, a, b): def func(): with worker_client() as c: x = np.ones(5) - future = c.scatter(x) - assert future.type == np.ndarray + task = c.scatter(x) + assert task.type == np.ndarray await c.submit(func) @@ -128,8 +128,8 @@ def func(): xx, yy = ee.gather([x, y]) return xx, yy - future = c.submit(func) - result = await future + task = c.submit(func) + result = await task assert result == (2, 3) @@ -140,8 +140,8 @@ def f(): with worker_client() as lc: return lc.loop is get_worker().loop - future = c.submit(f) - result = await future + task = c.submit(f) + result = await task assert result @@ -151,8 +151,8 @@ def mysum(): sub_tasks = [delayed(double)(i) for i in range(100)] with worker_client() as lc: - futures = lc.compute(sub_tasks) - for f in as_completed(futures): + tasks = lc.compute(sub_tasks) + for f in as_completed(tasks): result += f.result() return result @@ -166,13 +166,13 @@ def mysum(): sub_tasks = [delayed(double)(i) for i in range(100)] with worker_client() as lc: - futures = lc.compute(sub_tasks) - for f in as_completed(futures): + tasks = lc.compute(sub_tasks) + for f in as_completed(tasks): result += f.result() return result - future = c.compute(delayed(mysum)()) - await future + task = c.compute(delayed(mysum)()) + await task start = time() while len(a.data) + len(b.data) > 1: @@ -193,8 +193,8 @@ def f(i): get_worker().count -= 1 return i - futures = c.map(f, range(20)) - results = await c._gather(futures) + tasks = c.map(f, range(20)) + results = await c._gather(tasks) assert list(results) == list(range(20)) @@ -205,8 +205,8 @@ def mysum(): with c.get_executor() as e: return sum(e.map(double, range(30))) - future = c.submit(mysum) - result = await future + task = c.submit(mysum) + result = await task assert result == 30 * 29 @@ -243,8 +243,8 @@ def func(x): with cmgr as c: return c.submit(inc, x).result() - future = c.submit(func, 10) - result = await future + task = c.submit(func, 10) + result = await task assert result == 11 @@ -264,9 +264,9 @@ def func(): with worker_client(timeout=0) as wc: print("hello") - future = client.submit(func) + task = client.submit(func) with pytest.raises(EnvironmentError): - result = future.result() + result = task.result() def test_secede_without_stealing_issue_1262(): diff --git a/distributed/tests/test_worker_memory.py b/distributed/tests/test_worker_memory.py index fc473ba8d1..4fc69d1304 100644 --- a/distributed/tests/test_worker_memory.py +++ b/distributed/tests/test_worker_memory.py @@ -150,9 +150,9 @@ def __sizeof__(self): return self.reported_size -async def assert_basic_futures(c: Client) -> None: - futures = c.map(inc, range(10)) - results = await c.gather(futures) +async def assert_basic_tasks(c: Client) -> None: + tasks = c.map(inc, range(10)) + results = await c.gather(tasks) assert results == list(map(inc, range(10))) @@ -178,7 +178,7 @@ async def test_fail_to_pickle_execute_1(c, s, a, b): await x assert isinstance(e.value.__cause__.__cause__, CustomError) - await assert_basic_futures(c) + await assert_basic_tasks(c) class FailStoreDict(UserDict): @@ -280,7 +280,7 @@ async def test_fail_to_pickle_execute_2(c, s, a): await wait(y) assert set(a.data.memory) == {"x", "y"} assert not a.data.disk - await assert_basic_futures(c) + await assert_basic_tasks(c) @gen_cluster( @@ -323,7 +323,7 @@ async def test_fail_to_pickle_spill(c, s, a): assert bad.status == "finished" assert bad.key in a.data.fast - await assert_basic_futures(c) + await assert_basic_tasks(c) @gen_cluster( @@ -481,14 +481,14 @@ def __sizeof__(self): a.monitor.get_process_memory = lambda: 50_000_000 * len(a.data.fast) # Add 500MB (reported) process memory. Spilling must not happen. - futures = [c.submit(C, pure=False) for _ in range(10)] - await wait(futures) + tasks = [c.submit(C, pure=False) for _ in range(10)] + await wait(tasks) await asyncio.sleep(0.1) assert not a.data.disk # Add another 250MB unmanaged memory. This must trigger the spilling. - futures += [c.submit(C, pure=False) for _ in range(5)] - await wait(futures) + tasks += [c.submit(C, pure=False) for _ in range(5)] + await wait(tasks) # Wait until spilling starts. Then, wait until it stops. prev_n = 0 @@ -709,8 +709,8 @@ async def test_avoid_memory_monitor_if_zero_limit_worker(c, s, a): assert type(a.data) is dict assert not memory_monitor_running(a) - future = c.submit(inc, 1) - assert await future == 2 + task = c.submit(inc, 1) + assert await task == 2 await asyncio.sleep(0.05) assert await c.submit(inc, 2) == 3 # worker doesn't pause @@ -728,8 +728,8 @@ async def test_avoid_memory_monitor_if_zero_limit_nanny(c, s, nanny): assert not memory_monitor_running(nanny) assert not (await c.run(memory_monitor_running))[nanny.worker_address] - future = c.submit(inc, 1) - assert await future == 2 + task = c.submit(inc, 1) + assert await task == 2 await asyncio.sleep(0.02) assert await c.submit(inc, 2) == 3 # worker doesn't pause @@ -825,7 +825,7 @@ async def test_manual_evict_proto(c, s, a): assert memory_monitor_running(a) assert isinstance(a.data, ManualEvictDict) - futures = await c.scatter({"x": None, "y": None, "z": None}) + tasks = await c.scatter({"x": None, "y": None, "z": None}) while a.data.evicted != {"x", "y", "z"}: await asyncio.sleep(0.01) @@ -842,7 +842,7 @@ def leak(): (addr,) = s.workers pid = (await c.run(os.getpid))[addr] - future = c.submit(leak, key="leak") + task = c.submit(leak, key="leak") # Wait until the worker is restarted while len(s.workers) != 1 or set(s.workers) == {addr}: @@ -853,12 +853,12 @@ def leak(): psutil.Process(pid) with pytest.raises(KilledWorker): - await future + await task assert s.tasks["leak"].suspicious == 1 assert not any( (await c.run(lambda dask_worker: "leak" in dask_worker.state.tasks)).values() ) - future.release() + task.release() while "leak" in s.tasks: await asyncio.sleep(0.01) diff --git a/distributed/tests/test_worker_metrics.py b/distributed/tests/test_worker_metrics.py index e12c4902b1..22eafb6131 100644 --- a/distributed/tests/test_worker_metrics.py +++ b/distributed/tests/test_worker_metrics.py @@ -102,7 +102,7 @@ async def test_task_lifecycle(c, s, a, b): ("execute", span_id(s), "z", "disk-write", "bytes"), # Delta to end-to-end runtime as seen from the worker state machine ("execute", span_id(s), "z", "other", "seconds"), - # a.get_data() (triggered by the client retrieving the Future for z) + # a.get_data() (triggered by the client retrieving the Task for z) # Unspill ("get-data", "disk-read", "seconds"), ("get-data", "disk-read", "count"), @@ -475,10 +475,10 @@ def f(x): if get_worker().address == a_address: raise Reschedule() - futures = c.map(f, range(4), key=["x-1", "x-2", "x-3", "x-4"]) - futures2 = c.map(slowinc, range(10), delay=0.1, key="clog", workers=[a.address]) - await wait(futures) - assert all(f.key in b.data for f in futures) + tasks = c.map(f, range(4), key=["x-1", "x-2", "x-3", "x-4"]) + tasks2 = c.map(slowinc, range(10), delay=0.1, key="clog", workers=[a.address]) + await wait(tasks) + assert all(f.key in b.data for f in tasks) evs = get_digests(a, "x") k = ("execute", span_id(s), "x", "cancelled", "seconds") diff --git a/distributed/utils.py b/distributed/utils.py index 2c0ec8069b..f97586b8d1 100644 --- a/distributed/utils.py +++ b/distributed/utils.py @@ -240,7 +240,7 @@ async def All(args, quiet_exceptions=()): Parameters ---------- - args: futures to wait for + args: tasks to wait for quiet_exceptions: tuple, Exception Exception types to avoid logging if they fail """ @@ -278,7 +278,7 @@ async def Any(args, quiet_exceptions=()): Parameters ---------- - args: futures to wait for + args: tasks to wait for quiet_exceptions: tuple, Exception Exception types to avoid logging if they fail """ @@ -350,10 +350,10 @@ def sync(self, func, *args, asynchronous=None, callback_timeout=None, **kwargs): if asynchronous is None: asynchronous = self.asynchronous if asynchronous: - future = func(*args, **kwargs) + task = func(*args, **kwargs) if callback_timeout is not None: - future = wait_for(future, callback_timeout) - return future + task = wait_for(task, callback_timeout) + return task else: return sync( self.loop, func, *args, callback_timeout=callback_timeout, **kwargs @@ -392,11 +392,11 @@ def sync( # set up non-locals result: T error: BaseException | None = None - future: asyncio.Future[T] + task: asyncio.Future[T] @gen.coroutine def f() -> Generator[AnyType, AnyType, None]: - nonlocal result, error, future + nonlocal result, error, task try: if main_tid == threading.get_ident(): raise RuntimeError("sync() called from thread of running loop") @@ -404,16 +404,16 @@ def f() -> Generator[AnyType, AnyType, None]: awaitable = func(*args, **kwargs) if timeout is not None: awaitable = wait_for(awaitable, timeout) - future = asyncio.ensure_future(awaitable) - result = yield future + task = asyncio.ensure_future(awaitable) + result = yield task except Exception as exception: error = exception finally: e.set() def cancel() -> None: - if future is not None: - future.cancel() + if task is not None: + task.cancel() def wait(timeout: float | None) -> bool: try: @@ -443,7 +443,7 @@ def wait(timeout: float | None) -> bool: # passing a loop kwarg to bind to a loop running in another thread # e.g. calling from Client(asynchronous=False). Instead the loop is bound # as late as possible: when calling any methods that wait on or wake - # Future instances. See: https://bugs.python.org/issue42392 + # Task instances. See: https://bugs.python.org/issue42392 class LateLoopEvent: _event: asyncio.Event | None @@ -947,7 +947,7 @@ def get_traceback(): os.path.join("distributed", "worker"), os.path.join("distributed", "scheduler"), os.path.join("tornado", "gen.py"), - os.path.join("concurrent", "futures"), + os.path.join("concurrent", "tasks"), ] while exc_traceback and any( b in exc_traceback.tb_frame.f_code.co_filename for b in bad diff --git a/distributed/utils_comm.py b/distributed/utils_comm.py index a2e5f1220b..228977c550 100644 --- a/distributed/utils_comm.py +++ b/distributed/utils_comm.py @@ -205,41 +205,41 @@ def _namedtuple_packing(o: Any, handler: Callable[..., Any]) -> Any: def _unpack_remotedata_inner( - o: Any, byte_keys: bool, found_futures: set[WrappedKey] + o: Any, byte_keys: bool, found_tasks: set[WrappedKey] ) -> Any: - """Inner implementation of `unpack_remotedata` that adds found wrapped keys to `found_futures`""" + """Inner implementation of `unpack_remotedata` that adds found wrapped keys to `found_tasks`""" typ = type(o) if typ is tuple: if not o: return o if type(o[0]) is SubgraphCallable: - # Unpack futures within the arguments of the subgraph callable - futures: set[WrappedKey] = set() - args = tuple(_unpack_remotedata_inner(i, byte_keys, futures) for i in o[1:]) - found_futures.update(futures) + # Unpack tasks within the arguments of the subgraph callable + tasks: set[WrappedKey] = set() + args = tuple(_unpack_remotedata_inner(i, byte_keys, tasks) for i in o[1:]) + found_tasks.update(tasks) - # Unpack futures within the subgraph callable itself + # Unpack tasks within the subgraph callable itself sc: SubgraphCallable = o[0] - futures = set() + tasks = set() dsk = { - k: _unpack_remotedata_inner(v, byte_keys, futures) + k: _unpack_remotedata_inner(v, byte_keys, tasks) for k, v in sc.dsk.items() } - future_keys: tuple = () - if futures: # If no futures is in the subgraph, we just use `sc` as-is - found_futures.update(futures) - future_keys = ( - tuple(f.key for f in futures) + task_keys: tuple = () + if tasks: # If no tasks is in the subgraph, we just use `sc` as-is + found_tasks.update(tasks) + task_keys = ( + tuple(f.key for f in tasks) if byte_keys - else tuple(f.key for f in futures) + else tuple(f.key for f in tasks) ) - inkeys = tuple(sc.inkeys) + future_keys + inkeys = tuple(sc.inkeys) + task_keys sc = SubgraphCallable(dsk, sc.outkey, inkeys, sc.name) - return (sc,) + args + future_keys + return (sc,) + args + task_keys else: return tuple( - _unpack_remotedata_inner(item, byte_keys, found_futures) for item in o + _unpack_remotedata_inner(item, byte_keys, found_tasks) for item in o ) elif is_namedtuple_instance(o): return _namedtuple_packing( @@ -247,26 +247,26 @@ def _unpack_remotedata_inner( partial( _unpack_remotedata_inner, byte_keys=byte_keys, - found_futures=found_futures, + found_tasks=found_tasks, ), ) if typ in collection_types: if not o: return o - outs = [_unpack_remotedata_inner(item, byte_keys, found_futures) for item in o] + outs = [_unpack_remotedata_inner(item, byte_keys, found_tasks) for item in o] return typ(outs) elif typ is dict: if o: return { - k: _unpack_remotedata_inner(v, byte_keys, found_futures) + k: _unpack_remotedata_inner(v, byte_keys, found_tasks) for k, v in o.items() } else: return o - elif issubclass(typ, WrappedKey): # TODO use type is Future + elif issubclass(typ, WrappedKey): # TODO use type is Task k = o.key - found_futures.add(o) + found_tasks.add(o) return k else: return o @@ -299,8 +299,8 @@ def unpack_remotedata(o: Any, byte_keys: bool = False) -> tuple[Any, set]: >>> unpack_remotedata(rd, byte_keys=True) ("('x', 1)", {WrappedKey('('x', 1)')}) """ - found_futures: set[WrappedKey] = set() - return _unpack_remotedata_inner(o, byte_keys, found_futures), found_futures + found_tasks: set[WrappedKey] = set() + return _unpack_remotedata_inner(o, byte_keys, found_tasks), found_tasks def pack_data(o, d, key_types=object): diff --git a/distributed/utils_test.py b/distributed/utils_test.py index 9c167d0412..9cc60a08e5 100644 --- a/distributed/utils_test.py +++ b/distributed/utils_test.py @@ -1354,49 +1354,49 @@ async def assert_can_connect_from_everywhere_4_6(port, protocol="tcp", **kwargs) """ Check that the local *port* is reachable from all IPv4 and IPv6 addresses. """ - futures = [ + tasks = [ assert_can_connect("%s://127.0.0.1:%d" % (protocol, port), **kwargs), assert_can_connect("%s://%s:%d" % (protocol, get_ip(), port), **kwargs), ] if has_ipv6(): - futures += [ + tasks += [ assert_can_connect("%s://[::1]:%d" % (protocol, port), **kwargs), assert_can_connect("%s://[%s]:%d" % (protocol, get_ipv6(), port), **kwargs), ] - await asyncio.gather(*futures) + await asyncio.gather(*tasks) async def assert_can_connect_from_everywhere_4(port, protocol="tcp", **kwargs): """ Check that the local *port* is reachable from all IPv4 addresses. """ - futures = [ + tasks = [ assert_can_connect("%s://127.0.0.1:%d" % (protocol, port), **kwargs), assert_can_connect("%s://%s:%d" % (protocol, get_ip(), port), **kwargs), ] if has_ipv6(): - futures += [ + tasks += [ assert_cannot_connect("%s://[::1]:%d" % (protocol, port), **kwargs), assert_cannot_connect( "%s://[%s]:%d" % (protocol, get_ipv6(), port), **kwargs ), ] - await asyncio.gather(*futures) + await asyncio.gather(*tasks) async def assert_can_connect_locally_4(port, **kwargs): """ Check that the local *port* is only reachable from local IPv4 addresses. """ - futures = [assert_can_connect("tcp://127.0.0.1:%d" % port, **kwargs)] + tasks = [assert_can_connect("tcp://127.0.0.1:%d" % port, **kwargs)] if get_ip() != "127.0.0.1": # No outside IPv4 connectivity? - futures += [assert_cannot_connect("tcp://%s:%d" % (get_ip(), port), **kwargs)] + tasks += [assert_cannot_connect("tcp://%s:%d" % (get_ip(), port), **kwargs)] if has_ipv6(): - futures += [ + tasks += [ assert_cannot_connect("tcp://[::1]:%d" % port, **kwargs), assert_cannot_connect("tcp://[%s]:%d" % (get_ipv6(), port), **kwargs), ] - await asyncio.gather(*futures) + await asyncio.gather(*tasks) async def assert_can_connect_from_everywhere_6(port, **kwargs): @@ -1404,13 +1404,13 @@ async def assert_can_connect_from_everywhere_6(port, **kwargs): Check that the local *port* is reachable from all IPv6 addresses. """ assert has_ipv6() - futures = [ + tasks = [ assert_cannot_connect("tcp://127.0.0.1:%d" % port, **kwargs), assert_cannot_connect("tcp://%s:%d" % (get_ip(), port), **kwargs), assert_can_connect("tcp://[::1]:%d" % port, **kwargs), assert_can_connect("tcp://[%s]:%d" % (get_ipv6(), port), **kwargs), ] - await asyncio.gather(*futures) + await asyncio.gather(*tasks) async def assert_can_connect_locally_6(port, **kwargs): @@ -1418,16 +1418,14 @@ async def assert_can_connect_locally_6(port, **kwargs): Check that the local *port* is only reachable from local IPv6 addresses. """ assert has_ipv6() - futures = [ + tasks = [ assert_cannot_connect("tcp://127.0.0.1:%d" % port, **kwargs), assert_cannot_connect("tcp://%s:%d" % (get_ip(), port), **kwargs), assert_can_connect("tcp://[::1]:%d" % port, **kwargs), ] if get_ipv6() != "::1": # No outside IPv6 connectivity? - futures += [ - assert_cannot_connect("tcp://[%s]:%d" % (get_ipv6(), port), **kwargs) - ] - await asyncio.gather(*futures) + tasks += [assert_cannot_connect("tcp://[%s]:%d" % (get_ipv6(), port), **kwargs)] + await asyncio.gather(*tasks) @contextmanager diff --git a/distributed/variable.py b/distributed/variable.py index 3df28ff359..d3b17f7b91 100644 --- a/distributed/variable.py +++ b/distributed/variable.py @@ -10,7 +10,7 @@ from dask.utils import parse_timedelta -from distributed.client import Future +from distributed.client import Task from distributed.metrics import time from distributed.utils import TimeoutError, log_errors, wait_for from distributed.worker import get_client @@ -39,12 +39,12 @@ def __init__(self, scheduler): {"variable_set": self.set, "variable_get": self.get} ) - self.scheduler.stream_handlers["variable-future-release"] = self.future_release + self.scheduler.stream_handlers["variable-task-release"] = self.task_release self.scheduler.stream_handlers["variable_delete"] = self.delete async def set(self, name=None, key=None, data=None, client=None): if key is not None: - record = {"type": "Future", "value": key} + record = {"type": "Task", "value": key} self.scheduler.client_desires_keys(keys=[key], client="variable-%s" % name) else: record = {"type": "msgpack", "value": data} @@ -53,7 +53,7 @@ async def set(self, name=None, key=None, data=None, client=None): except KeyError: pass else: - if old["type"] == "Future" and old["value"] != key: + if old["type"] == "Task" and old["value"] != key: asyncio.ensure_future(self.release(old["value"], name)) if name not in self.variables: async with self.started: @@ -68,7 +68,7 @@ async def release(self, key, name): self.scheduler.client_releases_keys(keys=[key], client="variable-%s" % name) del self.waiting[key, name] - async def future_release(self, name=None, key=None, token=None, client=None): + async def task_release(self, name=None, key=None, token=None, client=None): self.waiting[key, name].remove(token) if not self.waiting[key, name]: async with self.waiting_conditions[name]: @@ -94,7 +94,7 @@ async def _(): # Python 3.6 is odd and requires special help here self.started.release() record = self.variables[name] - if record["type"] == "Future": + if record["type"] == "Task": key = record["value"] token = uuid.uuid4().hex ts = self.scheduler.tasks.get(key) @@ -114,7 +114,7 @@ async def delete(self, name=None, client=None): except KeyError: pass else: - if old["type"] == "Future": + if old["type"] == "Task": await self.release(old["value"], name) with suppress(KeyError): del self.waiting_conditions[name] @@ -127,14 +127,14 @@ async def delete(self, name=None, client=None): class Variable: """Distributed Global Variable - This allows multiple clients to share futures and data between each other + This allows multiple clients to share tasks and data between each other with a single mutable variable. All metadata is sequentialized through the scheduler. Race conditions can occur. Values must be either Futures or msgpack-encodable data (ints, lists, strings, etc..) All data will be kept and sent through the scheduler, so it is wise not to send too much. If you want to share a large amount of - data then ``scatter`` it and share the future instead. + data then ``scatter`` it and share the task instead. Parameters ---------- @@ -153,8 +153,8 @@ class Variable: >>> x.set(123) # docttest: +SKIP >>> x.get() # docttest: +SKIP 123 - >>> future = client.submit(f, x) # doctest: +SKIP - >>> x.set(future) # doctest: +SKIP + >>> task = client.submit(f, x) # doctest: +SKIP + >>> x.set(task) # doctest: +SKIP See Also -------- @@ -183,7 +183,7 @@ def _verify_running(self): ) async def _set(self, value): - if isinstance(value, Future): + if isinstance(value, Task): await self.client.scheduler.variable_set(key=value.key, name=self.name) else: await self.client.scheduler.variable_set(data=value, name=self.name) @@ -193,8 +193,8 @@ def set(self, value, **kwargs): Parameters ---------- - value : Future or object - Must be either a Future or a msgpack-encodable value + value : Task or object + Must be either a Task or a msgpack-encodable value """ self._verify_running() return self.client.sync(self._set, value, **kwargs) @@ -203,13 +203,13 @@ async def _get(self, timeout=None): d = await self.client.scheduler.variable_get( timeout=timeout, name=self.name, client=self.client.id ) - if d["type"] == "Future": - value = Future(d["value"], self.client, inform=True, state=d["state"]) + if d["type"] == "Task": + value = Task(d["value"], self.client, inform=True, state=d["state"]) if d["state"] == "erred": value._state.set_error(d["exception"], d["traceback"]) self.client._send_to_scheduler( { - "op": "variable-future-release", + "op": "variable-task-release", "name": self.name, "key": d["value"], "token": d["token"], @@ -239,7 +239,7 @@ def delete(self): Caution, this affects all clients currently pointing to this variable. """ self._verify_running() - if self.client.status == "running": # TODO: can leave zombie futures + if self.client.status == "running": # TODO: can leave zombie tasks self.client._send_to_scheduler({"op": "variable_delete", "name": self.name}) def __reduce__(self): diff --git a/distributed/widgets/templates/future.html.j2 b/distributed/widgets/templates/future.html.j2 index 46e96b0709..55ef821540 100644 --- a/distributed/widgets/templates/future.html.j2 +++ b/distributed/widgets/templates/future.html.j2 @@ -1,4 +1,4 @@ -Future: {{ key | key_split | html_escape }} +Task: {{ key | key_split | html_escape }} status: {% if status == "error" %} diff --git a/distributed/worker.py b/distributed/worker.py index 55dd5a7724..8aa9c8233f 100644 --- a/distributed/worker.py +++ b/distributed/worker.py @@ -1202,9 +1202,9 @@ async def _register_with_scheduler(self) -> None: ), serializers=["msgpack"], ) - future = comm.read(deserializers=["msgpack"]) + task = comm.read(deserializers=["msgpack"]) - response = await future + response = await task if response.get("warning"): logger.warning(response["warning"]) @@ -1589,7 +1589,7 @@ async def close( # type: ignore ): for c in Worker._initialized_clients: # Regardless of what the client was initialized with - # we'll require the result as a future. This is + # we'll require the result as a task. This is # necessary since the heuristics of asynchronous are not # reliable and we might deadlock here c._asynchronous = True @@ -2616,8 +2616,8 @@ def get_current_task(self) -> Key: >>> def f(): ... return get_worker().get_current_task() - >>> future = client.submit(f) # doctest: +SKIP - >>> future.result() # doctest: +SKIP + >>> task = client.submit(f) # doctest: +SKIP + >>> task.result() # doctest: +SKIP 'f-1234' See Also @@ -2710,8 +2710,8 @@ def get_worker() -> Worker: ... worker = get_worker() # The worker on which this task is running ... return worker.address - >>> future = client.submit(f) # doctest: +SKIP - >>> future.result() # doctest: +SKIP + >>> task = client.submit(f) # doctest: +SKIP + >>> task.result() # doctest: +SKIP 'tcp://127.0.0.1:47373' See Also @@ -2749,12 +2749,12 @@ def get_client(address=None, timeout=None, resolve_address=True) -> Client: -------- >>> def f(): ... client = get_client(timeout="10s") - ... futures = client.map(lambda x: x + 1, range(10)) # spawn many tasks - ... results = client.gather(futures) + ... tasks = client.map(lambda x: x + 1, range(10)) # spawn many tasks + ... results = client.gather(tasks) ... return sum(results) - >>> future = client.submit(f) # doctest: +SKIP - >>> future.result() # doctest: +SKIP + >>> task = client.submit(f) # doctest: +SKIP + >>> task.result() # doctest: +SKIP 55 See Also @@ -2807,9 +2807,9 @@ def secede(): >>> def mytask(x): ... # do some work ... client = get_client() - ... futures = client.map(...) # do some remote work + ... tasks = client.map(...) # do some remote work ... secede() # while that work happens, remove ourself from the pool - ... return client.gather(futures) # return gathered results + ... return client.gather(tasks) # return gathered results See Also -------- @@ -3293,7 +3293,7 @@ def print( >>> def worker_function(): ... print("Hello from worker!") >>> client.submit(worker_function) - + Hello from worker! """ try: diff --git a/distributed/worker_client.py b/distributed/worker_client.py index 355156206d..e8c8fd4bf1 100644 --- a/distributed/worker_client.py +++ b/distributed/worker_client.py @@ -37,7 +37,7 @@ def worker_client(timeout=None, separate_thread=True): ... result = c.gather([a, b]) # and gather results ... return result - >>> future = client.submit(func, 1) # submit func(1) on cluster + >>> task = client.submit(func, 1) # submit func(1) on cluster See Also -------- diff --git a/docs/make.bat b/docs/make.bat index e468da25b8..06775ee82e 100644 --- a/docs/make.bat +++ b/docs/make.bat @@ -1,263 +1,263 @@ -@ECHO OFF - -REM Command file for Sphinx documentation - -if "%SPHINXBUILD%" == "" ( - set SPHINXBUILD=sphinx-build -) -set BUILDDIR=build -set ALLSPHINXOPTS=-d %BUILDDIR%/doctrees %SPHINXOPTS% source -set I18NSPHINXOPTS=%SPHINXOPTS% source -if NOT "%PAPER%" == "" ( - set ALLSPHINXOPTS=-D latex_paper_size=%PAPER% %ALLSPHINXOPTS% - set I18NSPHINXOPTS=-D latex_paper_size=%PAPER% %I18NSPHINXOPTS% -) - -if "%1" == "" goto help - -if "%1" == "help" ( - :help - echo.Please use `make ^` where ^ is one of - echo. html to make standalone HTML files - echo. dirhtml to make HTML files named index.html in directories - echo. singlehtml to make a single large HTML file - echo. pickle to make pickle files - echo. json to make JSON files - echo. htmlhelp to make HTML files and a HTML help project - echo. qthelp to make HTML files and a qthelp project - echo. devhelp to make HTML files and a Devhelp project - echo. epub to make an epub - echo. latex to make LaTeX files, you can set PAPER=a4 or PAPER=letter - echo. text to make text files - echo. man to make manual pages - echo. texinfo to make Texinfo files - echo. gettext to make PO message catalogs - echo. changes to make an overview over all changed/added/deprecated items - echo. xml to make Docutils-native XML files - echo. pseudoxml to make pseudoxml-XML files for display purposes - echo. linkcheck to check all external links for integrity - echo. doctest to run all doctests embedded in the documentation if enabled - echo. coverage to run coverage check of the documentation if enabled - goto end -) - -if "%1" == "clean" ( - for /d %%i in (%BUILDDIR%\*) do rmdir /q /s %%i - del /q /s %BUILDDIR%\* - goto end -) - - -REM Check if sphinx-build is available and fallback to Python version if any -%SPHINXBUILD% 2> nul -if errorlevel 9009 goto sphinx_python -goto sphinx_ok - -:sphinx_python - -set SPHINXBUILD=python -m sphinx.__init__ -%SPHINXBUILD% 2> nul -if errorlevel 9009 ( - echo. - echo.The 'sphinx-build' command was not found. Make sure you have Sphinx - echo.installed, then set the SPHINXBUILD environment variable to point - echo.to the full path of the 'sphinx-build' executable. Alternatively you - echo.may add the Sphinx directory to PATH. - echo. - echo.If you don't have Sphinx installed, grab it from - echo.http://sphinx-doc.org/ - exit /b 1 -) - -:sphinx_ok - - -if "%1" == "html" ( - %SPHINXBUILD% -b html %ALLSPHINXOPTS% %BUILDDIR%/html - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The HTML pages are in %BUILDDIR%/html. - goto end -) - -if "%1" == "dirhtml" ( - %SPHINXBUILD% -b dirhtml %ALLSPHINXOPTS% %BUILDDIR%/dirhtml - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The HTML pages are in %BUILDDIR%/dirhtml. - goto end -) - -if "%1" == "singlehtml" ( - %SPHINXBUILD% -b singlehtml %ALLSPHINXOPTS% %BUILDDIR%/singlehtml - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The HTML pages are in %BUILDDIR%/singlehtml. - goto end -) - -if "%1" == "pickle" ( - %SPHINXBUILD% -b pickle %ALLSPHINXOPTS% %BUILDDIR%/pickle - if errorlevel 1 exit /b 1 - echo. - echo.Build finished; now you can process the pickle files. - goto end -) - -if "%1" == "json" ( - %SPHINXBUILD% -b json %ALLSPHINXOPTS% %BUILDDIR%/json - if errorlevel 1 exit /b 1 - echo. - echo.Build finished; now you can process the JSON files. - goto end -) - -if "%1" == "htmlhelp" ( - %SPHINXBUILD% -b htmlhelp %ALLSPHINXOPTS% %BUILDDIR%/htmlhelp - if errorlevel 1 exit /b 1 - echo. - echo.Build finished; now you can run HTML Help Workshop with the ^ -.hhp project file in %BUILDDIR%/htmlhelp. - goto end -) - -if "%1" == "qthelp" ( - %SPHINXBUILD% -b qthelp %ALLSPHINXOPTS% %BUILDDIR%/qthelp - if errorlevel 1 exit /b 1 - echo. - echo.Build finished; now you can run "qcollectiongenerator" with the ^ -.qhcp project file in %BUILDDIR%/qthelp, like this: - echo.^> qcollectiongenerator %BUILDDIR%\qthelp\distributed.qhcp - echo.To view the help file: - echo.^> assistant -collectionFile %BUILDDIR%\qthelp\distributed.ghc - goto end -) - -if "%1" == "devhelp" ( - %SPHINXBUILD% -b devhelp %ALLSPHINXOPTS% %BUILDDIR%/devhelp - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. - goto end -) - -if "%1" == "epub" ( - %SPHINXBUILD% -b epub %ALLSPHINXOPTS% %BUILDDIR%/epub - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The epub file is in %BUILDDIR%/epub. - goto end -) - -if "%1" == "latex" ( - %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex - if errorlevel 1 exit /b 1 - echo. - echo.Build finished; the LaTeX files are in %BUILDDIR%/latex. - goto end -) - -if "%1" == "latexpdf" ( - %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex - cd %BUILDDIR%/latex - make all-pdf - cd %~dp0 - echo. - echo.Build finished; the PDF files are in %BUILDDIR%/latex. - goto end -) - -if "%1" == "latexpdfja" ( - %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex - cd %BUILDDIR%/latex - make all-pdf-ja - cd %~dp0 - echo. - echo.Build finished; the PDF files are in %BUILDDIR%/latex. - goto end -) - -if "%1" == "text" ( - %SPHINXBUILD% -b text %ALLSPHINXOPTS% %BUILDDIR%/text - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The text files are in %BUILDDIR%/text. - goto end -) - -if "%1" == "man" ( - %SPHINXBUILD% -b man %ALLSPHINXOPTS% %BUILDDIR%/man - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The manual pages are in %BUILDDIR%/man. - goto end -) - -if "%1" == "texinfo" ( - %SPHINXBUILD% -b texinfo %ALLSPHINXOPTS% %BUILDDIR%/texinfo - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The Texinfo files are in %BUILDDIR%/texinfo. - goto end -) - -if "%1" == "gettext" ( - %SPHINXBUILD% -b gettext %I18NSPHINXOPTS% %BUILDDIR%/locale - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The message catalogs are in %BUILDDIR%/locale. - goto end -) - -if "%1" == "changes" ( - %SPHINXBUILD% -b changes %ALLSPHINXOPTS% %BUILDDIR%/changes - if errorlevel 1 exit /b 1 - echo. - echo.The overview file is in %BUILDDIR%/changes. - goto end -) - -if "%1" == "linkcheck" ( - %SPHINXBUILD% -b linkcheck %ALLSPHINXOPTS% %BUILDDIR%/linkcheck - if errorlevel 1 exit /b 1 - echo. - echo.Link check complete; look for any errors in the above output ^ -or in %BUILDDIR%/linkcheck/output.txt. - goto end -) - -if "%1" == "doctest" ( - %SPHINXBUILD% -b doctest %ALLSPHINXOPTS% %BUILDDIR%/doctest - if errorlevel 1 exit /b 1 - echo. - echo.Testing of doctests in the sources finished, look at the ^ -results in %BUILDDIR%/doctest/output.txt. - goto end -) - -if "%1" == "coverage" ( - %SPHINXBUILD% -b coverage %ALLSPHINXOPTS% %BUILDDIR%/coverage - if errorlevel 1 exit /b 1 - echo. - echo.Testing of coverage in the sources finished, look at the ^ -results in %BUILDDIR%/coverage/python.txt. - goto end -) - -if "%1" == "xml" ( - %SPHINXBUILD% -b xml %ALLSPHINXOPTS% %BUILDDIR%/xml - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The XML files are in %BUILDDIR%/xml. - goto end -) - -if "%1" == "pseudoxml" ( - %SPHINXBUILD% -b pseudoxml %ALLSPHINXOPTS% %BUILDDIR%/pseudoxml - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The pseudo-XML files are in %BUILDDIR%/pseudoxml. - goto end -) - -:end +@ECHO OFF + +REM Command file for Sphinx documentation + +if "%SPHINXBUILD%" == "" ( + set SPHINXBUILD=sphinx-build +) +set BUILDDIR=build +set ALLSPHINXOPTS=-d %BUILDDIR%/doctrees %SPHINXOPTS% source +set I18NSPHINXOPTS=%SPHINXOPTS% source +if NOT "%PAPER%" == "" ( + set ALLSPHINXOPTS=-D latex_paper_size=%PAPER% %ALLSPHINXOPTS% + set I18NSPHINXOPTS=-D latex_paper_size=%PAPER% %I18NSPHINXOPTS% +) + +if "%1" == "" goto help + +if "%1" == "help" ( + :help + echo.Please use `make ^` where ^ is one of + echo. html to make standalone HTML files + echo. dirhtml to make HTML files named index.html in directories + echo. singlehtml to make a single large HTML file + echo. pickle to make pickle files + echo. json to make JSON files + echo. htmlhelp to make HTML files and a HTML help project + echo. qthelp to make HTML files and a qthelp project + echo. devhelp to make HTML files and a Devhelp project + echo. epub to make an epub + echo. latex to make LaTeX files, you can set PAPER=a4 or PAPER=letter + echo. text to make text files + echo. man to make manual pages + echo. texinfo to make Texinfo files + echo. gettext to make PO message catalogs + echo. changes to make an overview over all changed/added/deprecated items + echo. xml to make Docutils-native XML files + echo. pseudoxml to make pseudoxml-XML files for display purposes + echo. linkcheck to check all external links for integrity + echo. doctest to run all doctests embedded in the documentation if enabled + echo. coverage to run coverage check of the documentation if enabled + goto end +) + +if "%1" == "clean" ( + for /d %%i in (%BUILDDIR%\*) do rmdir /q /s %%i + del /q /s %BUILDDIR%\* + goto end +) + + +REM Check if sphinx-build is available and fallback to Python version if any +%SPHINXBUILD% 2> nul +if errorlevel 9009 goto sphinx_python +goto sphinx_ok + +:sphinx_python + +set SPHINXBUILD=python -m sphinx.__init__ +%SPHINXBUILD% 2> nul +if errorlevel 9009 ( + echo. + echo.The 'sphinx-build' command was not found. Make sure you have Sphinx + echo.installed, then set the SPHINXBUILD environment variable to point + echo.to the full path of the 'sphinx-build' executable. Alternatively you + echo.may add the Sphinx directory to PATH. + echo. + echo.If you don't have Sphinx installed, grab it from + echo.http://sphinx-doc.org/ + exit /b 1 +) + +:sphinx_ok + + +if "%1" == "html" ( + %SPHINXBUILD% -b html %ALLSPHINXOPTS% %BUILDDIR%/html + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The HTML pages are in %BUILDDIR%/html. + goto end +) + +if "%1" == "dirhtml" ( + %SPHINXBUILD% -b dirhtml %ALLSPHINXOPTS% %BUILDDIR%/dirhtml + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The HTML pages are in %BUILDDIR%/dirhtml. + goto end +) + +if "%1" == "singlehtml" ( + %SPHINXBUILD% -b singlehtml %ALLSPHINXOPTS% %BUILDDIR%/singlehtml + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The HTML pages are in %BUILDDIR%/singlehtml. + goto end +) + +if "%1" == "pickle" ( + %SPHINXBUILD% -b pickle %ALLSPHINXOPTS% %BUILDDIR%/pickle + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; now you can process the pickle files. + goto end +) + +if "%1" == "json" ( + %SPHINXBUILD% -b json %ALLSPHINXOPTS% %BUILDDIR%/json + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; now you can process the JSON files. + goto end +) + +if "%1" == "htmlhelp" ( + %SPHINXBUILD% -b htmlhelp %ALLSPHINXOPTS% %BUILDDIR%/htmlhelp + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; now you can run HTML Help Workshop with the ^ +.hhp project file in %BUILDDIR%/htmlhelp. + goto end +) + +if "%1" == "qthelp" ( + %SPHINXBUILD% -b qthelp %ALLSPHINXOPTS% %BUILDDIR%/qthelp + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; now you can run "qcollectiongenerator" with the ^ +.qhcp project file in %BUILDDIR%/qthelp, like this: + echo.^> qcollectiongenerator %BUILDDIR%\qthelp\distributed.qhcp + echo.To view the help file: + echo.^> assistant -collectionFile %BUILDDIR%\qthelp\distributed.ghc + goto end +) + +if "%1" == "devhelp" ( + %SPHINXBUILD% -b devhelp %ALLSPHINXOPTS% %BUILDDIR%/devhelp + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. + goto end +) + +if "%1" == "epub" ( + %SPHINXBUILD% -b epub %ALLSPHINXOPTS% %BUILDDIR%/epub + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The epub file is in %BUILDDIR%/epub. + goto end +) + +if "%1" == "latex" ( + %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; the LaTeX files are in %BUILDDIR%/latex. + goto end +) + +if "%1" == "latexpdf" ( + %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex + cd %BUILDDIR%/latex + make all-pdf + cd %~dp0 + echo. + echo.Build finished; the PDF files are in %BUILDDIR%/latex. + goto end +) + +if "%1" == "latexpdfja" ( + %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex + cd %BUILDDIR%/latex + make all-pdf-ja + cd %~dp0 + echo. + echo.Build finished; the PDF files are in %BUILDDIR%/latex. + goto end +) + +if "%1" == "text" ( + %SPHINXBUILD% -b text %ALLSPHINXOPTS% %BUILDDIR%/text + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The text files are in %BUILDDIR%/text. + goto end +) + +if "%1" == "man" ( + %SPHINXBUILD% -b man %ALLSPHINXOPTS% %BUILDDIR%/man + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The manual pages are in %BUILDDIR%/man. + goto end +) + +if "%1" == "texinfo" ( + %SPHINXBUILD% -b texinfo %ALLSPHINXOPTS% %BUILDDIR%/texinfo + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The Texinfo files are in %BUILDDIR%/texinfo. + goto end +) + +if "%1" == "gettext" ( + %SPHINXBUILD% -b gettext %I18NSPHINXOPTS% %BUILDDIR%/locale + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The message catalogs are in %BUILDDIR%/locale. + goto end +) + +if "%1" == "changes" ( + %SPHINXBUILD% -b changes %ALLSPHINXOPTS% %BUILDDIR%/changes + if errorlevel 1 exit /b 1 + echo. + echo.The overview file is in %BUILDDIR%/changes. + goto end +) + +if "%1" == "linkcheck" ( + %SPHINXBUILD% -b linkcheck %ALLSPHINXOPTS% %BUILDDIR%/linkcheck + if errorlevel 1 exit /b 1 + echo. + echo.Link check complete; look for any errors in the above output ^ +or in %BUILDDIR%/linkcheck/output.txt. + goto end +) + +if "%1" == "doctest" ( + %SPHINXBUILD% -b doctest %ALLSPHINXOPTS% %BUILDDIR%/doctest + if errorlevel 1 exit /b 1 + echo. + echo.Testing of doctests in the sources finished, look at the ^ +results in %BUILDDIR%/doctest/output.txt. + goto end +) + +if "%1" == "coverage" ( + %SPHINXBUILD% -b coverage %ALLSPHINXOPTS% %BUILDDIR%/coverage + if errorlevel 1 exit /b 1 + echo. + echo.Testing of coverage in the sources finished, look at the ^ +results in %BUILDDIR%/coverage/python.txt. + goto end +) + +if "%1" == "xml" ( + %SPHINXBUILD% -b xml %ALLSPHINXOPTS% %BUILDDIR%/xml + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The XML files are in %BUILDDIR%/xml. + goto end +) + +if "%1" == "pseudoxml" ( + %SPHINXBUILD% -b pseudoxml %ALLSPHINXOPTS% %BUILDDIR%/pseudoxml + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The pseudo-XML files are in %BUILDDIR%/pseudoxml. + goto end +) + +:end diff --git a/docs/source/active_memory_manager.rst b/docs/source/active_memory_manager.rst index 2b4a94b508..9f0ba30cc4 100644 --- a/docs/source/active_memory_manager.rst +++ b/docs/source/active_memory_manager.rst @@ -9,7 +9,7 @@ Memory imbalance and duplication -------------------------------- Whenever a Dask task returns data, it is stored on the worker that executed the task for as long as it's a dependency of other tasks, is referenced by a -:class:`~distributed.Client` through a :class:`~distributed.Future`, or is part of a +:class:`~distributed.Client` through a :class:`~distributed.Task`, or is part of a :doc:`published dataset `. Dask assigns tasks to workers following criteria of CPU occupancy, :doc:`resources`, and diff --git a/docs/source/actors.rst b/docs/source/actors.rst index 8efd027e42..c7efff8c13 100644 --- a/docs/source/actors.rst +++ b/docs/source/actors.rst @@ -34,18 +34,18 @@ and then call methods on that class remotely. from dask.distributed import Client # Start a Dask Client client = Client() - future = client.submit(Counter, actor=True) # Create a Counter on a worker - counter = future.result() # Get back a pointer to that object + task = client.submit(Counter, actor=True) # Create a Counter on a worker + counter = task.result() # Get back a pointer to that object counter # - future = counter.increment() # Call remote method - future.result() # Get back result + task = counter.increment() # Call remote method + task.result() # Get back result # 1 - future = counter.add(10) # Call remote method - future.result() # Get back result + task = counter.add(10) # Call remote method + task.result() # Get back result # 11 Motivation @@ -85,17 +85,17 @@ and using the ``actors=`` keyword (or ``actor=`` on ``submit``). .. code-block:: python - future = client.submit(Counter, actors=True) + task = client.submit(Counter, actors=True) You can use all other keywords to these functions like ``workers=``, ``resources=``, and so on to control where this actor ends up. -This creates a normal Dask future on which you can call ``.result()`` to get +This creates a normal Dask task on which you can call ``.result()`` to get the Actor once it has successfully run on a worker. .. code-block:: python - >>> counter = future.result() + >>> counter = task.result() >>> counter @@ -115,21 +115,21 @@ However accessing an attribute or calling a method will trigger a communication to the remote worker, run the method on the remote worker in a separate thread pool, and then communicate the result back to the calling side. For attribute access these operations block and return when finished, for method calls they -return an ``BaseActorFuture`` immediately. +return an ``BaseActorTask`` immediately. .. code-block:: python - >>> future = counter.increment() # Immediately returns a BaseActorFuture - >>> future.result() # Block until finished and result arrives + >>> task = counter.increment() # Immediately returns a BaseActorTask + >>> task.result() # Block until finished and result arrives 1 -``BaseActorFuture`` are similar to normal Dask ``Future`` objects, but not as fully +``BaseActorTask`` are similar to normal Dask ``Task`` objects, but not as fully featured. They currently *only* support the ``result`` method and nothing else. -They don't currently work with any other Dask functions that expect futures, +They don't currently work with any other Dask functions that expect tasks, like ``as_completed``, ``wait``, or ``client.gather``. They can't be placed into additional submit or map calls to form dependencies. They communicate their results immediately (rather than waiting for result to be called) and -cache the result on the future itself. +cache the result on the task itself. Access Attributes ----------------- @@ -164,17 +164,17 @@ workers are not possible. The appropriate method of the Actor's object is then called in a separate thread, the result captured, and then sent back to the calling side. Currently workers have only a single thread for actors, but this may change in the -future. +task. The result is sent back immediately to the calling side, and is not stored on -the worker with the actor. It is cached on the ``BaseActorFuture`` object. +the worker with the actor. It is cached on the ``BaseActorTask`` object. Calling from coroutines and async/await --------------------------------------- If you use actors within a coroutine or async/await function then actor methods -and attribute access will return Tornado futures +and attribute access will return Tornado tasks .. code-block:: python diff --git a/docs/source/api.rst b/docs/source/api.rst index 461f7a1114..7d11453ffe 100644 --- a/docs/source/api.rst +++ b/docs/source/api.rst @@ -36,12 +36,12 @@ The client connects to and submits computation to a Dask cluster (such as a :cla .. currentmodule:: distributed -**Future** +**Task** .. autosummary:: - Future + Task -.. autoautosummary:: distributed.Future +.. autoautosummary:: distributed.Task :methods: **Synchronization** @@ -64,7 +64,7 @@ The client connects to and submits computation to a Dask cluster (such as a :cla distributed.diagnostics.progressbar.progress wait fire_and_forget - futures_of + tasks_of get_task_stream get_task_metadata performance_report @@ -94,18 +94,18 @@ environment by passing the ``asynchronous=True`` keyword # blocking client = Client() - future = client.submit(func, *args) # immediate, no blocking/async difference - result = client.gather(future) # blocking + task = client.submit(func, *args) # immediate, no blocking/async difference + result = client.gather(task) # blocking # asynchronous Python 2/3 client = yield Client(asynchronous=True) - future = client.submit(func, *args) # immediate, no blocking/async difference - result = yield client.gather(future) # non-blocking/asynchronous + task = client.submit(func, *args) # immediate, no blocking/async difference + result = yield client.gather(task) # non-blocking/asynchronous # asynchronous Python 3 client = await Client(asynchronous=True) - future = client.submit(func, *args) # immediate, no blocking/async difference - result = await client.gather(future) # non-blocking/asynchronous + task = client.submit(func, *args) # immediate, no blocking/async difference + result = await client.gather(task) # non-blocking/asynchronous The asynchronous variants must be run within a Tornado coroutine. See the :doc:`Asynchronous ` documentation for more information. @@ -123,10 +123,10 @@ Client :members: -Future +Task ------ -.. autoclass:: Future +.. autoclass:: Task :members: @@ -179,7 +179,7 @@ Other .. autofunction:: distributed.diagnostics.progressbar.progress .. autofunction:: wait .. autofunction:: fire_and_forget -.. autofunction:: futures_of +.. autofunction:: tasks_of .. currentmodule:: distributed diff --git a/docs/source/asynchronous.rst b/docs/source/asynchronous.rst index 871dce6502..6457f74041 100644 --- a/docs/source/asynchronous.rst +++ b/docs/source/asynchronous.rst @@ -26,13 +26,13 @@ This includes all functions that submit work to the cluster, like ``submit``, .. code-block:: python - future = client.submit(lambda x: x + 1, 10) + task = client.submit(lambda x: x + 1, 10) -You can await futures directly +You can await tasks directly .. code-block:: python - result = await future + result = await task >>> print(result) 11 @@ -42,7 +42,7 @@ received information from the scheduler should now be ``await``'ed. .. code-block:: python - result = await client.gather(future) + result = await client.gather(task) If you want to use an asynchronous function with a synchronous ``Client`` @@ -57,8 +57,8 @@ function to run the asynchronous function: client = Client() # normal blocking client async def f(): - future = client.submit(lambda x: x + 1, 10) - result = await client.gather(future, asynchronous=True) + task = client.submit(lambda x: x + 1, 10) + result = await client.gather(task, asynchronous=True) return result client.sync(f) @@ -92,8 +92,8 @@ Python 3 with Tornado or Asyncio async def f(): client = await Client(asynchronous=True) - future = client.submit(lambda x: x + 1, 10) - result = await future + task = client.submit(lambda x: x + 1, 10) + result = await task await client.close() return result diff --git a/docs/source/client.rst b/docs/source/client.rst index 9b69db9852..e8c08dfe1c 100644 --- a/docs/source/client.rst +++ b/docs/source/client.rst @@ -14,7 +14,7 @@ it to the address of a ``Scheduler``: There are a few different ways to interact with the cluster through the client: 1. The Client satisfies most of the standard concurrent.futures_ - PEP-3148_ - interface with ``.submit``, ``.map`` functions and ``Future`` objects, + interface with ``.submit``, ``.map`` functions and ``Task`` objects, allowing the immediate and direct submission of tasks. 2. The Client registers itself as the default Dask_ scheduler, and so runs all dask collections like dask.array_, dask.bag_, dask.dataframe_ and dask.delayed_ @@ -22,7 +22,7 @@ There are a few different ways to interact with the cluster through the client: full :doc:`API ` for a thorough list. -Concurrent.futures +Concurrent.tasks ------------------ We can submit individual function calls with the ``client.submit`` method or @@ -35,19 +35,19 @@ many function calls with the ``client.map`` method >>> x = client.submit(inc, 10) >>> x - + >>> L = client.map(inc, range(1000)) >>> L - [, - , - , - , ...] + [, + , + , + , ...] These results live on distributed workers. -We gather back the results using either the ``Future.result`` method for single -futures or ``client.gather`` method for many futures at once. +We gather back the results using either the ``Task.result`` method for single +tasks or ``client.gather`` method for many tasks at once. .. code-block:: python @@ -63,12 +63,12 @@ remotely with functions like ``submit``, ``map``, ``get`` and ``compute``. See :doc:`efficiency ` for more information on efficient use of distributed. -We can submit tasks on futures and use futures as inputs. The function will go to -the machine where the futures are stored and run on the result once it has completed. +We can submit tasks on tasks and use tasks as inputs. The function will go to +the machine where the tasks are stored and run on the result once it has completed. .. code-block:: python - >>> y = client.submit(inc, x) # Submit on x, a Future + >>> y = client.submit(inc, x) # Submit on x, a Task >>> total = client.submit(sum, L) # Submit on L, a list of Futures >>> y.result() 12 @@ -123,7 +123,7 @@ By default, ``distributed`` assumes that all functions are pure_. Pure functions If this is not the case, you should use the ``pure=False`` keyword argument in methods like ``Client.map()`` and ``Client.submit()``. The client associates a key to all computations. This key is accessible on -the Future object. +the Task object. .. code-block:: python @@ -168,8 +168,8 @@ functions. async def f(): client = await Client(asynchronous=True) - future = client.submit(func, *args) - result = await future + task = client.submit(func, *args) + result = await task return result If you want to reuse the same client in asynchronous and synchronous @@ -181,8 +181,8 @@ call. client = Client() # normal blocking client async def f(): - futures = client.map(func, L) - results = await client.gather(futures, asynchronous=True) + tasks = client.map(func, L) + results = await client.gather(tasks, asynchronous=True) return results See the :doc:`Asynchronous ` documentation for more information. diff --git a/docs/source/develop.rst b/docs/source/develop.rst index ea729e6180..7da3f148b2 100644 --- a/docs/source/develop.rst +++ b/docs/source/develop.rst @@ -98,7 +98,7 @@ using the ``@gen_cluster`` style of test, e.g. # tests/test_submit.py from distributed.utils_test import gen_cluster, inc - from distributed import Client, Future, Scheduler, Worker + from distributed import Client, Task, Scheduler, Worker @gen_cluster(client=True) async def test_submit(c, s, a, b): @@ -107,16 +107,16 @@ using the ``@gen_cluster`` style of test, e.g. assert isinstance(a, Worker) assert isinstance(b, Worker) - future = c.submit(inc, 1) - assert isinstance(future, Future) - assert future.key in c.futures + task = c.submit(inc, 1) + assert isinstance(task, Task) + assert task.key in c.tasks - # result = future.result() # This synchronous API call would block - result = await future + # result = task.result() # This synchronous API call would block + result = await task assert result == 2 - assert future.key in s.tasks - assert future.key in a.data or future.key in b.data + assert task.key in s.tasks + assert task.key in a.data or task.key in b.data The ``@gen_cluster`` decorator sets up a scheduler, client, and workers for @@ -136,8 +136,8 @@ different forked processes: from distributed.utils_test import client def test_submit(client): - future = client.submit(inc, 10) - assert future.result() == 11 + task = client.submit(inc, 10) + assert task.result() == 11 Additionally, if you want access to the scheduler and worker processes you can also add the ``s, a, b`` fixtures as well. @@ -148,12 +148,12 @@ also add the ``s, a, b`` fixtures as well. from distributed.utils_test import client def test_submit(client, s, a, b): - future = client.submit(inc, 10) - assert future.result() == 11 # use the synchronous/blocking API here + task = client.submit(inc, 10) + assert task.result() == 11 # use the synchronous/blocking API here a['proc'].terminate() # kill one of the workers - result = future.result() # test that future remains valid + result = task.result() # test that task remains valid assert result == 2 In this style of test you do not have access to the scheduler or workers. The diff --git a/docs/source/efficiency.rst b/docs/source/efficiency.rst index 89771548b7..cb1987f585 100644 --- a/docs/source/efficiency.rst +++ b/docs/source/efficiency.rst @@ -25,7 +25,7 @@ shape we might choose one of the following options: >>> x = client.submit(np.random.random, (1000, 1000)) >>> type(x) - Future + Task **Slow** @@ -45,7 +45,7 @@ shape we might choose one of the following options: Use larger tasks ---------------- -The scheduler adds about *one millisecond* of overhead per task or Future +The scheduler adds about *one millisecond* of overhead per task or Task object. While this may sound fast it's quite slow if you run a billion tasks. If your functions run faster than 100ms or so then you might not see any speedup from using distributed computing. @@ -56,8 +56,8 @@ A common solution is to batch your input into larger chunks. .. code-block:: python - >>> futures = client.map(f, seq) - >>> len(futures) # avoid large numbers of futures + >>> tasks = client.map(f, seq) + >>> len(tasks) # avoid large numbers of tasks 1000000000 **Fast** @@ -70,8 +70,8 @@ A common solution is to batch your input into larger chunks. >>> from tlz import partition_all >>> chunks = partition_all(1000000, seq) # Collect into groups of size 1000 - >>> futures = client.map(f_many, chunks) - >>> len(futures) # Compute on larger pieces of your data at once + >>> tasks = client.map(f_many, chunks) + >>> len(tasks) # Compute on larger pieces of your data at once 1000 diff --git a/docs/source/examples/word-count.rst b/docs/source/examples/word-count.rst index b40f333d44..bef6055951 100644 --- a/docs/source/examples/word-count.rst +++ b/docs/source/examples/word-count.rst @@ -117,8 +117,8 @@ worker: .. code-block:: python - >>> future = client.submit(count_words, filenames[0]) - >>> counts = future.result() + >>> task = client.submit(count_words, filenames[0]) + >>> counts = task.result() >>> print(sorted(counts.items(), key=lambda k_v: k_v[1], reverse=True)[:10]) [(b'the', 144873), @@ -139,30 +139,30 @@ are running. .. code-block:: python - >>> futures = client.map(count_words, filenames) + >>> tasks = client.map(count_words, filenames) -We can check the status of some ``futures`` while all of the text files are +We can check the status of some ``tasks`` while all of the text files are being processed: .. code-block:: python - >>> len(futures) + >>> len(tasks) 161 - >>> futures[:5] + >>> tasks[:5] - [, - , - , - , - ] + [, + , + , + , + ] - >>> progress(futures) + >>> progress(tasks) [########################################] | 100% Completed | 3min 0.2s -When the ``futures`` finish reading in all of the text files and counting +When the ``tasks`` finish reading in all of the text files and counting words, the results will exist on each worker. This operation required about 3 minutes to run on a cluster with three worker machines, each with 4 cores and 16 GB RAM. @@ -182,23 +182,23 @@ top 10,000 words from each text file. ... items = sorted(d.items(), key=lambda kv: kv[1], reverse=True)[:10000] ... return dict(items) -We can then ``map`` the futures from the previous step to this culling +We can then ``map`` the tasks from the previous step to this culling function. This is a convenient way to construct a pipeline of computations -using futures: +using tasks: .. code-block:: python - >>> futures2 = client.map(top_items, futures) + >>> tasks2 = client.map(top_items, tasks) We can ``gather`` the resulting culled word count data for each text file to the local process: .. code-block:: python - >>> results = client.gather(iter(futures2)) + >>> results = client.gather(iter(tasks2)) To sum the word counts for all of the text files, we can iterate over the -results in ``futures2`` and update a local dictionary that contains all of the +results in ``tasks2`` and update a local dictionary that contains all of the word counts. .. code-block:: python @@ -259,22 +259,22 @@ The complete Python script for this example is shown below: counts = count_words(filenames[0]) print(sorted(counts.items(), key=lambda k_v: k_v[1], reverse=True)[:10]) - future = client.submit(count_words, filenames[0]) - counts = future.result() + task = client.submit(count_words, filenames[0]) + counts = task.result() print(sorted(counts.items(), key=lambda k_v: k_v[1], reverse=True)[:10]) - futures = client.map(count_words, filenames) - len(futures) - futures[:5] - progress(futures) + tasks = client.map(count_words, filenames) + len(tasks) + tasks[:5] + progress(tasks) def top_items(d): items = sorted(d.items(), key=lambda kv: kv[1], reverse=True)[:10000] return dict(items) - futures2 = client.map(top_items, futures) - results = client.gather(iter(futures2)) + tasks2 = client.map(top_items, tasks) + results = client.gather(iter(tasks2)) all_counts = Counter() for result in results: diff --git a/docs/source/faq.rst b/docs/source/faq.rst index b0882b2ef7..cfb89de86b 100644 --- a/docs/source/faq.rst +++ b/docs/source/faq.rst @@ -47,14 +47,14 @@ Does Dask handle Data Locality? Yes, both data locality in memory and data locality on disk. Often it's *much* cheaper to move computations to where data lives. If one of -your tasks creates a large array and a future task computes the sum of that +your tasks creates a large array and a task task computes the sum of that array, you want to be sure that the sum runs on the same worker that has the array in the first place, otherwise you'll wait for a long while as the data moves between workers. Needless communication can easily dominate costs if we're sloppy. The Dask Scheduler tracks the location and size of every intermediate value -produced by every worker and uses this information when assigning future tasks +produced by every worker and uses this information when assigning task tasks to workers. Dask tries to make computations more efficient by minimizing data movement. diff --git a/docs/source/fine-performance-metrics.rst b/docs/source/fine-performance-metrics.rst index 471fc787ee..7451e39d3e 100644 --- a/docs/source/fine-performance-metrics.rst +++ b/docs/source/fine-performance-metrics.rst @@ -118,7 +118,7 @@ they produce. For example, you may want to separate I/O time from `thread-noncpu def read_some_files(): ... - future = client.submit(read_some_files) + task = client.submit(read_some_files) In the above example, the wall time spent by the custom function ``read_some_files`` will be logged as "I/O", which is a completely arbitrary activity label. diff --git a/docs/source/journey.rst b/docs/source/journey.rst index 1b0c9182c9..9682301e33 100644 --- a/docs/source/journey.rst +++ b/docs/source/journey.rst @@ -30,9 +30,9 @@ message to the ``Scheduler``:: 'tasks': {'z': (add, x, y)}, 'keys': ['z']} -The client then creates a ``Future`` object with the key ``'z'`` and returns +The client then creates a ``Task`` object with the key ``'z'`` and returns that object back to the user. This happens even before the message has been -received by the scheduler. The status of the future says ``'pending'``. +received by the scheduler. The status of the task says ``'pending'``. Step 2: Arrive in the Scheduler @@ -140,7 +140,7 @@ The scheduler receives this message and does a few things: 2. If ``x`` or ``y`` are no longer needed then it sends a message out to relevant workers to delete them from local memory. 3. It sends a message to all of the clients that ``z`` is ready and so all - client ``Future`` objects that are currently waiting should, wake up. In + client ``Task`` objects that are currently waiting should, wake up. In particular, this wakes up the ``z.result()`` command executed by the user originally. @@ -168,13 +168,13 @@ Step 8: Garbage Collection The user leaves this part of their code and the local variable ``z`` goes out of scope. The Python garbage collector cleans it up. This triggers a decremented reference on the client (we didn't mention this, but when we -created the ``Future`` we also started a reference count.) If this is the only -instance of a Future pointing to ``z`` then we send a message up to the +created the ``Task`` we also started a reference count.) If this is the only +instance of a Task pointing to ``z`` then we send a message up to the scheduler that it is OK to release ``z``. The user no longer requires it to persist. The scheduler receives this message and, if there are no computations that -might depend on ``z`` in the immediate future, it removes elements of this key +might depend on ``z`` in the immediate task, it removes elements of this key from local scheduler state and adds the key to a list of keys to be deleted periodically. Every 500 ms a message goes out to relevant workers telling them which keys they can delete from their local memory. The graph/recipe to create diff --git a/docs/source/locality.rst b/docs/source/locality.rst index ddec59f34a..22fd096853 100644 --- a/docs/source/locality.rst +++ b/docs/source/locality.rst @@ -34,7 +34,7 @@ cores and we scatter out the list ``range(10)`` as follows: .. code-block:: python - futures = client.scatter(range(10)) + tasks = client.scatter(range(10)) Then Alice and Bob receive the following data @@ -56,7 +56,7 @@ as follows: .. code-block:: python - future = client.submit(func, *args, workers=['Alice']) + task = client.submit(func, *args, workers=['Alice']) * Alice: ``[0, 1, 4, 5, 8, 9, new_result]`` * Bob: ``[2, 3, 6, 7]`` @@ -69,7 +69,7 @@ used. .. code-block:: python - future = client.submit(func, *args, workers=['Alice'], + task = client.submit(func, *args, workers=['Alice'], allow_other_workers=True) Additionally the ``scatter`` function supports a ``broadcast=`` keyword @@ -79,7 +79,7 @@ data. .. code-block:: python - futures = client.scatter([1, 2, 3], broadcast=True) # send data to all workers + tasks = client.scatter([1, 2, 3], broadcast=True) # send data to all workers * Alice: ``[1, 2, 3]`` * Bob: ``[1, 2, 3]`` @@ -132,7 +132,7 @@ workers. y = delayed(f)(2) z = delayed(g)(x, y) - future = client.compute(z, workers={z: '127.0.0.1', + task = client.compute(z, workers={z: '127.0.0.1', x: '192.168.0.1'}) Here the values of the dictionary are of the same form as before, a host, a @@ -147,35 +147,35 @@ two computations for ``x`` and ``y`` can run anywhere. .. code-block:: python - future = client.compute(z, workers={z: '127.0.0.1'}) + task = client.compute(z, workers={z: '127.0.0.1'}) The computations for both ``z`` and ``x`` must run on ``127.0.0.1`` .. code-block:: python - future = client.compute(z, workers={z: '127.0.0.1', + task = client.compute(z, workers={z: '127.0.0.1', x: '127.0.0.1'}) Use a tuple to group collections. This is shorthand for the above. .. code-block:: python - future = client.compute(z, workers={(x, y): '127.0.0.1'}) + task = client.compute(z, workers={(x, y): '127.0.0.1'}) Recall that all options for ``workers=`` in ``scatter/submit/map`` hold here as well. .. code-block:: python - future = client.compute(z, workers={(x, y): ['192.168.1.100', '192.168.1.101:9999']}) + task = client.compute(z, workers={(x, y): ['192.168.1.100', '192.168.1.101:9999']}) Set ``allow_other_workers=True`` to make these loose restrictions rather than hard requirements. .. code-block:: python - future = client.compute(z, workers={(x, y): '127.0.0.1'}, + task = client.compute(z, workers={(x, y): '127.0.0.1'}, allow_other_workers=True) Provide a collection to ``allow_other_workers=[...]`` to say that @@ -185,7 +185,7 @@ run elsewhere if necessary: .. code-block:: python - future = client.compute(z, workers={(x, y): '127.0.0.1'}, + task = client.compute(z, workers={(x, y): '127.0.0.1'}, allow_other_workers=[x]) This works fine with ``persist`` and with any dask collection (any object with diff --git a/docs/source/logging.rst b/docs/source/logging.rst index a210fe526c..4fa85e8648 100644 --- a/docs/source/logging.rst +++ b/docs/source/logging.rst @@ -28,7 +28,7 @@ To retrieve the transition logs for a given task, pass the task's key to the :me >>> f = client.submit(inc, 123) >>> f - + >>> s.story(f.key) [('inc-aad7bbea25dc61c8e53d929c7ec50bed', 'released', 'waiting', {'inc-aad7bbea25dc61c8e53d929c7ec50bed': 'processing'}, 1605143345.7283862), ('inc-aad7bbea25dc61c8e53d929c7ec50bed', 'waiting', 'processing', {}, 1605143345.7284858), @@ -53,7 +53,7 @@ For example, below we log start and stop times to the ``"runtimes"`` topic using ... ... ... stop = time() ... dask.distributed.get_worker().log_event("runtimes", {"start": start, "stop": stop}) - >>> futures = client.map(myfunc, range(10)) + >>> tasks = client.map(myfunc, range(10)) >>> client.get_events("runtimes") ((1605207481.77175, {'start': 1605207481.769397, 'stop': 1605207481.769397}), (1605207481.772021, {'start': 1605207481.770036, 'stop': 1605207481.770037}), diff --git a/docs/source/manage-computation.rst b/docs/source/manage-computation.rst index e4d2d4298d..3a3061201c 100644 --- a/docs/source/manage-computation.rst +++ b/docs/source/manage-computation.rst @@ -7,7 +7,7 @@ Data and Computation in Dask.distributed are always in one of three states numpy array in the local process. 2. Lazy computations in a dask graph, perhaps stored in a ``dask.delayed`` or ``dask.dataframe`` object. -3. Running computations or remote data, represented by ``Future`` objects +3. Running computations or remote data, represented by ``Task`` objects pointing to computations currently in flight. All three of these forms are important and there are functions that convert @@ -49,32 +49,32 @@ Dask Collections to Futures --------------------------- You can asynchronously submit lazy dask graphs to run on the cluster with the -``client.compute`` and ``client.persist`` methods. These functions return Future objects -immediately. These futures can then be queried to determine the state of the +``client.compute`` and ``client.persist`` methods. These functions return Task objects +immediately. These tasks can then be queried to determine the state of the computation. client.compute ~~~~~~~~~~~~~~ -The ``.compute`` method takes a collection and returns a single future. +The ``.compute`` method takes a collection and returns a single task. .. code-block:: python >>> df = dd.read_csv('s3://...') - >>> total = client.compute(df.sum()) # Return a single future + >>> total = client.compute(df.sum()) # Return a single task >>> total - Future(..., status='pending') + Task(..., status='pending') >>> total.result() # Block until finished 100000000 -Because this is a single future the result must fit on a single worker machine. +Because this is a single task the result must fit on a single worker machine. Like ``dask.compute`` above, the ``client.compute`` method is only appropriate when results are small and should fit in memory. The following would likely fail: .. code-block:: python - >>> future = client.compute(df) # Blows up memory + >>> task = client.compute(df) # Blows up memory Instead, you should use ``client.persist`` @@ -83,11 +83,11 @@ client.persist The ``.persist`` method submits the task graph behind the Dask collection to the scheduler, obtaining Futures for all of the top-most tasks (for example one -Future for each Pandas DataFrame in a Dask DataFrame). It then returns a copy -of the collection pointing to these futures instead of the previous graph. +Task for each Pandas DataFrame in a Dask DataFrame). It then returns a copy +of the collection pointing to these tasks instead of the previous graph. This new collection is semantically equivalent but now points to actively running data rather than a lazy graph. If you look at the dask graph within -the collection you will see the Future objects directly: +the collection you will see the Task objects directly: .. code-block:: python @@ -101,14 +101,14 @@ the collection you will see the Future objects directly: } >>> df = client.persist(df) # Start computation - >>> df.dask # Now points to running futures - {('parse', 0): Future(..., status='finished'), - ('parse', 1): Future(..., status='pending'), + >>> df.dask # Now points to running tasks + {('parse', 0): Task(..., status='finished'), + ('parse', 1): Task(..., status='pending'), ... } The collection is returned immediately and the computation happens in the -background on the cluster. Eventually all of the futures of this collection +background on the cluster. Eventually all of the tasks of this collection will be completed at which point further queries on this collection will likely be very fast. @@ -121,18 +121,18 @@ perform many fast queries off of the resulting collection. Concrete Values to Futures -------------------------- -We obtain futures through a few different ways. One is the mechanism above, by +We obtain tasks through a few different ways. One is the mechanism above, by wrapping Futures within Dask collections. Another is by submitting data or tasks directly to the cluster with ``client.scatter``, ``client.submit`` or ``client.map``. .. code-block:: python - futures = client.scatter(args) # Send data - future = client.submit(function, *args, **kwargs) # Send single task - futures = client.map(function, sequence, **kwargs) # Send many tasks + tasks = client.scatter(args) # Send data + task = client.submit(function, *args, **kwargs) # Send single task + tasks = client.map(function, sequence, **kwargs) # Send many tasks In this case ``*args`` or ``**kwargs`` can be normal Python objects, like ``1`` -or ``'hello'``, or they can be other ``Future`` objects if you want to link +or ``'hello'``, or they can be other ``Task`` objects if you want to link tasks together with dependencies. Unlike Dask collections like dask.delayed these task submissions happen @@ -143,32 +143,32 @@ except that execution is immediate rather than lazy. Futures to Concrete Values -------------------------- -You can turn an individual ``Future`` into a concrete value in the local -process by calling the ``Future.result()`` method. You can convert a -collection of futures into concrete values by calling the ``client.gather`` method. +You can turn an individual ``Task`` into a concrete value in the local +process by calling the ``Task.result()`` method. You can convert a +collection of tasks into concrete values by calling the ``client.gather`` method. .. code-block:: python - >>> future.result() + >>> task.result() 1 - >>> client.gather(futures) + >>> client.gather(tasks) [1, 2, 3, 4, ...] Futures to Dask Collections --------------------------- -As seen in the Collection to futures section it is common to have currently -computing ``Future`` objects within Dask graphs. This lets us build further +As seen in the Collection to tasks section it is common to have currently +computing ``Task`` objects within Dask graphs. This lets us build further computations on top of currently running computations. This is most often done with dask.delayed workflows on custom computations: .. code-block:: python - >>> x = delayed(sum)(futures) - >>> y = delayed(product)(futures) - >>> future = client.compute(x + y) + >>> x = delayed(sum)(tasks) + >>> y = delayed(product)(tasks) + >>> task = client.compute(x + y) Mixing the two forms allow you to build and submit a computation in stages like ``sum(...) + product(...)``. This is often valuable if you want to wait to see diff --git a/docs/source/memory.rst b/docs/source/memory.rst index b207812af4..6676291174 100644 --- a/docs/source/memory.rst +++ b/docs/source/memory.rst @@ -8,15 +8,15 @@ from memory as quickly as possible in order to make room for more computation. The result of a task is kept in memory if either of the following conditions hold: -1. A client holds a future pointing to this task. The data should stay in RAM so that +1. A client holds a task pointing to this task. The data should stay in RAM so that the client can gather the data on demand. 2. The task is necessary for ongoing computations that are working to produce - the final results pointed to by futures. These tasks will be removed once + the final results pointed to by tasks. These tasks will be removed once no ongoing tasks require them. -When users hold Future objects or persisted collections (which contain many such Futures +When users hold Task objects or persisted collections (which contain many such Futures inside their dask graph, typically accessible through their ``.dask`` attribute) they -pin those results to active memory. When the user deletes futures or collections from +pin those results to active memory. When the user deletes tasks or collections from their local Python process, the scheduler removes the associated data from distributed RAM. Because of this relationship, distributed memory reflects the state of local memory. A user may free distributed memory on the cluster by deleting persisted @@ -46,12 +46,12 @@ Persisting Collections ---------------------- Calls to ``Client.compute`` or ``Client.persist`` submit task graphs to the -cluster and return ``Future`` objects that point to particular output tasks. +cluster and return ``Task`` objects that point to particular output tasks. -Compute returns a single future per input; persist returns a copy of the -collection with each block or partition replaced by a single future. In short, +Compute returns a single task per input; persist returns a copy of the +collection with each block or partition replaced by a single task. In short, use ``persist`` to keep full collection on the cluster and use ``compute`` when -you want a small result as a single future. +you want a small result as a single task. Persist is more common and is often used as follows with collections: @@ -76,8 +76,8 @@ then adding a new column. Up until this point all work is lazy; we've just built up a recipe to perform the work as a graph in the ``df`` object. When we call ``df = client.persist(df)``, we cut the graph off the ``df`` object, -send it up to the scheduler, receive ``Future`` objects in return and create a -new dataframe with a very shallow graph that points directly to these futures. +send it up to the scheduler, receive ``Task`` objects in return and create a +new dataframe with a very shallow graph that points directly to these tasks. This happens more or less immediately (as long as it takes to serialize and send the graph) and we can continue working on our new ``df`` object while the cluster works to evaluate the graph in the background. @@ -108,7 +108,7 @@ collections and then use ``df.compute()`` for fast analyses. >>> # df.compute() # This is bad and would likely flood local memory >>> df = client.persist(df) # This is good and asynchronously pins df >>> df.x.sum().compute() # This is good because the result is small - >>> future = client.compute(df.x.sum()) # This is also good but less intuitive + >>> task = client.compute(df.x.sum()) # This is also good but less intuitive Clearing data @@ -131,14 +131,14 @@ we'll have to delete them all. .. code-block:: python >>> df2 = df[df.x < 10] - >>> del df # would not delete data, because df2 still tracks the futures + >>> del df # would not delete data, because df2 still tracks the tasks Aggressively Clearing Data -------------------------- To definitely remove a computation and all computations that depend on it you -can always ``cancel`` the futures/collection. +can always ``cancel`` the tasks/collection. .. code-block:: python @@ -175,8 +175,8 @@ Resilience Results are not intentionally copied unless necessary for computations on other worker nodes. Resilience is achieved through recomputation by maintaining the provenance of any result. If a worker node goes down, the scheduler is able to -recompute all of its results. The complete graph for any desired Future is -maintained until no references to that future exist. +recompute all of its results. The complete graph for any desired Task is +maintained until no references to that task exist. For more information see :doc:`Resilience `. diff --git a/docs/source/priority.rst b/docs/source/priority.rst index 374f04628e..77042a0dee 100644 --- a/docs/source/priority.rst +++ b/docs/source/priority.rst @@ -15,8 +15,8 @@ Dask uses the following priorities, in order: .. code-block:: python - future = client.submit(func, *args, priority=10) # high priority task - future = client.submit(func, *args, priority=-10) # low priority task + task = client.submit(func, *args, priority=10) # high priority task + task = client.submit(func, *args, priority=-10) # low priority task df = df.persist(priority=10) # high priority computation @@ -25,9 +25,9 @@ Dask uses the following priorities, in order: .. code-block:: python with dask.annotate(priority=10): - future = client.submit(func, *args) # high priority task + task = client.submit(func, *args) # high priority task with dask.annotate(priority=-10): - future = client.submit(func, *args) # low priority task + task = client.submit(func, *args) # low priority task with dask.annotate(priority=10): df = df.persist() # high priority computation diff --git a/docs/source/publish.rst b/docs/source/publish.rst index 98e453579e..d291c15aa9 100644 --- a/docs/source/publish.rst +++ b/docs/source/publish.rst @@ -2,7 +2,7 @@ Publish Datasets ================ A *published dataset* is a named reference to a Dask collection or list of -futures that has been published to the cluster. It is available for any client +tasks that has been published to the cluster. It is available for any client to see and persists beyond the scope of an individual session. Publishing datasets is useful in the following cases: diff --git a/docs/source/quickstart.rst b/docs/source/quickstart.rst index 0bb2b14d90..13c8ffd46d 100644 --- a/docs/source/quickstart.rst +++ b/docs/source/quickstart.rst @@ -56,8 +56,8 @@ Map and Submit Functions Use the ``map`` and ``submit`` methods to launch computations on the cluster. The ``map/submit`` functions send the function and arguments to the remote -workers for processing. They return ``Future`` objects that refer to remote -data on the cluster. The ``Future`` returns immediately while the computations +workers for processing. They return ``Task`` objects that refer to remote +data on the cluster. The ``Task`` returns immediately while the computations run remotely in the background. .. code-block:: python @@ -78,27 +78,27 @@ run remotely in the background. Gather ~~~~~~ -The ``map/submit`` functions return ``Future`` objects, lightweight tokens that +The ``map/submit`` functions return ``Task`` objects, lightweight tokens that refer to results on the cluster. By default the results of computations *stay on the cluster*. .. code-block:: python >>> total # Function hasn't yet completed - + >>> total # Function completed, result ready on remote worker - + -Gather results to your local machine either with the ``Future.result`` method -for a single future, or with the ``Client.gather`` method for many futures at +Gather results to your local machine either with the ``Task.result`` method +for a single task, or with the ``Client.gather`` method for many tasks at once. .. code-block:: python - >>> total.result() # result for single future + >>> total.result() # result for single task -285 - >>> client.gather(A) # gather for many futures + >>> client.gather(A) # gather for many tasks [0, 1, 4, 9, 16, 25, 36, 49, 64, 81] diff --git a/docs/source/related-work.rst b/docs/source/related-work.rst index 11aa254d6b..628c46e1db 100644 --- a/docs/source/related-work.rst +++ b/docs/source/related-work.rst @@ -115,7 +115,7 @@ route computations to remote workers >>> view = Client(...)[:] >>> results = view.map(func, sequence) >>> result = view.apply(func, *args, **kwargs) - >>> future = view.apply_async(func, *args, **kwargs) + >>> task = view.apply_async(func, *args, **kwargs) It also provides direct execution of code in the remote process and collection of data from the remote namespace. @@ -158,16 +158,16 @@ The distributed client includes a dynamic task scheduler capable of managing deep data dependencies between tasks. The IPython parallel docs include `a recipe`_ for executing task graphs with data dependencies. This same idea is core to all of ``distributed``, which uses a dynamic task scheduler for all -operations. Notably, ``distributed.Future`` objects can be used within +operations. Notably, ``distributed.Task`` objects can be used within ``submit/map/get`` calls before they have completed. .. code-block:: python - >>> x = client.submit(f, 1) # returns a future - >>> y = client.submit(f, 2) # returns a future - >>> z = client.submit(add, x, y) # consumes futures + >>> x = client.submit(f, 1) # returns a task + >>> y = client.submit(f, 2) # returns a task + >>> z = client.submit(add, x, y) # consumes tasks -The ability to use futures cheaply within ``submit`` and ``map`` methods +The ability to use tasks cheaply within ``submit`` and ``map`` methods enables the construction of very sophisticated data pipelines with simple code. Additionally, distributed can serve as a full dask task scheduler, enabling support for distributed arrays, dataframes, machine learning pipelines, and any @@ -200,13 +200,13 @@ concurrent.futures The :class:`distributed.Client` API is modeled after :mod:`concurrent.futures` and :pep:`3148`. It has a few notable differences: -* ``distributed`` accepts :class:`~distributed.client.Future` objects within +* ``distributed`` accepts :class:`~distributed.client.Task` objects within calls to ``submit/map``. When chaining computations, it is preferable to - submit Future objects directly rather than wait on them before submission. + submit Task objects directly rather than wait on them before submission. * The :meth:`~distributed.client.Client.map` method returns - :class:`~distributed.client.Future` objects, not concrete results. + :class:`~distributed.client.Task` objects, not concrete results. The :meth:`~distributed.client.Client.map` method returns immediately. -* Despite sharing a similar API, ``distributed`` :class:`~distributed.client.Future` +* Despite sharing a similar API, ``distributed`` :class:`~distributed.client.Task` objects cannot always be substituted for :class:`concurrent.futures.Future` objects, especially when using ``wait()`` or ``as_completed()``. * Distributed generally does not support callbacks. diff --git a/docs/source/resilience.rst b/docs/source/resilience.rst index c7219e51a7..e575f3c6ee 100644 --- a/docs/source/resilience.rst +++ b/docs/source/resilience.rst @@ -36,7 +36,7 @@ appropriately raises an ``IOError`` then the scheduler will reroute all pending computations to other workers. If the lost worker was the only worker to hold vital results necessary for -future computations then those results will be recomputed by surviving workers. +task computations then those results will be recomputed by surviving workers. The scheduler maintains a full history of how each result was produced and so is able to reproduce those same computations on other workers. diff --git a/docs/source/resources.rst b/docs/source/resources.rst index 047efe62fe..35f6092742 100644 --- a/docs/source/resources.rst +++ b/docs/source/resources.rst @@ -124,7 +124,7 @@ each task will run in a separate process: from distributed import Client client = Client('scheduler:8786') - futures = [client.submit(non_thread_safe_function, arg, + tasks = [client.submit(non_thread_safe_function, arg, resources={'process': 1}) for arg in args] diff --git a/docs/source/scheduling-policies.rst b/docs/source/scheduling-policies.rst index 278e5064f9..62c3bfc4b9 100644 --- a/docs/source/scheduling-policies.rst +++ b/docs/source/scheduling-policies.rst @@ -48,7 +48,7 @@ good chance those neighbors will be combined in a downstream operation:: X In the above case, we want ``a`` and ``b`` to run on the same worker, -and ``c`` and ``d`` to run on the same worker, reducing future +and ``c`` and ``d`` to run on the same worker, reducing task data transfer. We can also ignore the location of ``X``, because assuming we split the ``a b c d`` group across all workers to maximize parallelism, then ``X`` will eventually get transferred everywhere. @@ -308,7 +308,7 @@ scheduler, and workers at various points in the computation. described above. This per-graph counter encourages a first-in-first-out policy between computations. All tasks from a previous call to compute have a higher priority than all tasks from a subsequent call to compute (or - submit, persist, map, or any operation that generates futures). + submit, persist, map, or any operation that generates tasks). 3. Whenever a task is ready to run (its dependencies, if any, are complete), the scheduler assigns it to a worker. When multiple tasks are ready at once, they are submitted to workers, in priority order. If scheduler-side queuing diff --git a/docs/source/task-launch.rst b/docs/source/task-launch.rst index 80b0c1f8c8..48acbdd36b 100644 --- a/docs/source/task-launch.rst +++ b/docs/source/task-launch.rst @@ -17,7 +17,7 @@ downloads the data and converts it to a list on one of our worker machines: .. code-block:: python - future = client.submit(download_and_convert_to_list, uri) + task = client.submit(download_and_convert_to_list, uri) But now we need to submit new tasks for individual parts of this data. We have three options. @@ -37,12 +37,12 @@ necessary logic on our local machine: .. code-block:: python - >>> data = future.result() # gather data to local process + >>> data = task.result() # gather data to local process >>> data # data is a list [...] - >>> futures = e.map(process_element, data) # submit new tasks on data - >>> analysis = e.submit(aggregate, futures) # submit final aggregation task + >>> tasks = e.map(process_element, data) # submit new tasks on data + >>> analysis = e.submit(aggregate, tasks) # submit final aggregation task This is straightforward and, if ``data`` is small then it is probably the simplest, and therefore correct choice. However, if ``data`` is large then we @@ -64,8 +64,8 @@ on ``data`` remotely and then break up data into its various elements. >>> from operator import getitem >>> elements = [client.submit(getitem, data, i) for i in range(n)] # split data - >>> futures = client.map(process_element, elements) - >>> analysis = client.submit(aggregate, futures) + >>> tasks = client.map(process_element, elements) + >>> analysis = client.submit(aggregate, tasks) We compute the length remotely, gather back this very small result, and then use it to submit more tasks to break up the data and process on the cluster. @@ -159,15 +159,15 @@ to submit, scatter, and gather results. if n < 2: return n client = get_client() - a_future = client.submit(fib, n - 1) - b_future = client.submit(fib, n - 2) - a, b = client.gather([a_future, b_future]) + a_task = client.submit(fib, n - 1) + b_task = client.submit(fib, n - 2) + a, b = client.gather([a_task, b_task]) return a + b if __name__ == "__main__": client = Client() - future = client.submit(fib, 10) - result = future.result() + task = client.submit(fib, 10) + result = task.result() print(result) # prints "55" However, this can deadlock the scheduler if too many tasks request jobs at @@ -186,10 +186,10 @@ respectively. if n < 2: return n client = get_client() - a_future = client.submit(fib, n - 1) - b_future = client.submit(fib, n - 2) + a_task = client.submit(fib, n - 1) + b_task = client.submit(fib, n - 2) secede() - a, b = client.gather([a_future, b_future]) + a, b = client.gather([a_task, b_task]) rejoin() return a + b @@ -211,15 +211,15 @@ worker. if n < 2: return n with worker_client() as client: - a_future = client.submit(fib, n - 1) - b_future = client.submit(fib, n - 2) - a, b = client.gather([a_future, b_future]) + a_task = client.submit(fib, n - 1) + b_task = client.submit(fib, n - 2) + a, b = client.gather([a_task, b_task]) return a + b if __name__ == "__main__": client = Client() - future = client.submit(fib, 10) - result = future.result() + task = client.submit(fib, 10) + result = task.result() print(result) # prints "55" Tasks that invoke :py:func:`worker_client ` are diff --git a/docs/source/worker-state.rst b/docs/source/worker-state.rst index d56a1a4f68..812dbafe69 100644 --- a/docs/source/worker-state.rst +++ b/docs/source/worker-state.rst @@ -193,7 +193,7 @@ There are a few important exceptions to the flow diagrams above: - Scheduler intercession, in which the scheduler reassigns a task that was previously assigned to a separate worker to a new worker. This most commonly occurs when a :doc:`worker dies ` during computation. -- Client intercession, where a client either explicitly releases a Future or descopes +- Client intercession, where a client either explicitly releases a Task or descopes it; alternatively the whole client may shut down or become unresponsive. When there are no more clients holding references to a key or one of its dependents, the Scheduler will release it.