Futures
Contents
Futures¶
Dask supports a real-time task framework that extends Python’s concurrent.futures interface. Dask futures allow you to scale generic Python workflows across a Dask cluster with minimal code changes.
This interface is good for arbitrary task scheduling like dask.delayed, but is immediate rather than lazy, which provides some more flexibility in situations where the computations may evolve over time. These features depend on the second generation task scheduler found in dask.distributed (which, despite its name, runs very well on a single machine).
Examples¶
Visit https://examples.dask.org/futures.html to see and run examples using futures with Dask.
Start Dask Client¶
You must start a Client
to use the futures interface. This tracks state
among the various worker processes or threads:
from dask.distributed import Client
client = Client() # start local workers as processes
# or
client = Client(processes=False) # start local workers as threads
If you have Bokeh installed, then this starts up a
diagnostic dashboard at http://localhost:8787
.
Submit Tasks¶
|
Submit a function application to the scheduler |
|
Map a function on a sequence of arguments |
|
Wait until computation completes, gather result to local process. |
You can submit individual tasks using the submit
method:
def inc(x):
return x + 1
def add(x, y):
return x + y
a = client.submit(inc, 10) # calls inc(10) in background thread or process
b = client.submit(inc, 20) # calls inc(20) in background thread or process
The submit
function returns a Future
, which refers to a remote result. This result may
not yet be completed:
>>> a
<Future: status: pending, key: inc-b8aaf26b99466a7a1980efa1ade6701d>
Eventually it will complete. The result stays in the remote thread/process/worker until you ask for it back explicitly:
>>> a
<Future: status: finished, type: int, key: inc-b8aaf26b99466a7a1980efa1ade6701d>
>>> a.result() # blocks until task completes and data arrives
11
You can pass futures as inputs to submit. Dask automatically handles dependency tracking; once all input futures have completed, they will be moved onto a single worker (if necessary), and then the computation that depends on them will be started. You do not need to wait for inputs to finish before submitting a new task; Dask will handle this automatically:
c = client.submit(add, a, b) # calls add on the results of a and b
Similar to Python’s map
, you can use Client.map
to call the same
function and many inputs:
futures = client.map(inc, range(1000))
However, note that each task comes with about 1ms of overhead. If you want to map a function over a large number of inputs, then you might consider dask.bag or dask.dataframe instead.
Move Data¶
|
Wait until computation completes, gather result to local process. |
|
Gather futures from distributed memory |
|
Scatter data into distributed memory |
Given any future, you can call the .result
method to gather the result.
This will block until the future is done computing and then transfer the result
back to your local process if necessary:
>>> c.result()
32
You can gather many results concurrently using the Client.gather
method.
This can be more efficient than calling .result()
on each future
sequentially:
>>> # results = [future.result() for future in futures]
>>> results = client.gather(futures) # this can be faster
If you have important local data that you want to include in your computation, you can either include it as a normal input to a submit or map call:
>>> df = pd.read_csv('training-data.csv')
>>> future = client.submit(my_function, df)
Or you can scatter
it explicitly. Scattering moves your data to a worker
and returns a future pointing to that data:
>>> remote_df = client.scatter(df)
>>> remote_df
<Future: status: finished, type: DataFrame, key: bbd0ca93589c56ea14af49cba470006e>
>>> future = client.submit(my_function, remote_df)
Both of these accomplish the same result, but using scatter can sometimes be
faster. This is especially true if you use processes or distributed workers
(where data transfer is necessary) and you want to use df
in many
computations. Scattering the data beforehand avoids excessive data movement.
Calling scatter on a list scatters all elements individually. Dask will spread these elements evenly throughout workers in a round-robin fashion:
>>> client.scatter([1, 2, 3])
[<Future: status: finished, type: int, key: c0a8a20f903a4915b94db8de3ea63195>,
<Future: status: finished, type: int, key: 58e78e1b34eb49a68c65b54815d1b158>,
<Future: status: finished, type: int, key: d3395e15f605bc35ab1bac6341a285e2>]
References, Cancellation, and Exceptions¶
|
Cancel the request to run this future |
|
Return the exception of a failed task |
|
Return the traceback of a failed task |
|
Cancel running futures This stops future tasks from being scheduled if they have not yet run and deletes them if they have already run. |
Dask will only compute and hold onto results for which there are active futures. In this way, your local variables define what is active in Dask. When a future is garbage collected by your local Python session, Dask will feel free to delete that data or stop ongoing computations that were trying to produce it:
>>> del future # deletes remote data once future is garbage collected
You can also explicitly cancel a task using the Future.cancel
or
Client.cancel
methods:
>>> future.cancel() # deletes data even if other futures point to it
If a future fails, then Dask will raise the remote exceptions and tracebacks if you try to get the result:
def div(x, y):
return x / y
>>> a = client.submit(div, 1, 0) # 1 / 0 raises a ZeroDivisionError
>>> a
<Future: status: error, key: div-3601743182196fb56339e584a2bf1039>
>>> a.result()
1 def div(x, y):
----> 2 return x / y
ZeroDivisionError: division by zero
All futures that depend on an erred future also err with the same exception:
>>> b = client.submit(inc, a)
>>> b
<Future: status: error, key: inc-15e2e4450a0227fa38ede4d6b1a952db>
You can collect the exception or traceback explicitly with the
Future.exception
or Future.traceback
methods.
Waiting on Futures¶
|
Return futures in the order in which they complete |
|
Wait until all/any futures are finished |
You can wait on a future or collection of futures using the wait
function:
from dask.distributed import wait
>>> wait(futures)
This blocks until all futures are finished or have erred.
You can also iterate over the futures as they complete using the
as_completed
function:
from dask.distributed import as_completed
futures = client.map(score, x_values)
best = -1
for future in as_completed(futures):
y = future.result()
if y > best:
best = y
For greater efficiency, you can also ask as_completed
to gather the results
in the background:
for future, result in as_completed(futures, with_results=True):
# y = future.result() # don't need this
...
Or collect all futures in batches that had arrived since the last iteration:
for batch in as_completed(futures, with_results=True).batches():
for future, result in batch:
...
Additionally, for iterative algorithms, you can add more futures into the
as_completed
iterator during iteration:
seq = as_completed(futures)
for future in seq:
y = future.result()
if condition(y):
new_future = client.submit(...)
seq.add(new_future) # add back into the loop
or use seq.update(futures)
to add multiple futures at once.
Fire and Forget¶
|
Run tasks at least once, even if we release the futures |
Sometimes we don’t care about gathering the result of a task, and only care about side effects that it might have like writing a result to a file:
>>> a = client.submit(load, filename)
>>> b = client.submit(process, a)
>>> c = client.submit(write, b, out_filename)
As noted above, Dask will stop work that doesn’t have any active futures. It
thinks that because no one has a pointer to this data that no one cares. You
can tell Dask to compute a task anyway, even if there are no active futures,
using the fire_and_forget
function:
from dask.distributed import fire_and_forget
>>> fire_and_forget(c)
This is particularly useful when a future may go out of scope, for example, as part of a function:
def process(filename):
out_filename = 'out-' + filename
a = client.submit(load, filename)
b = client.submit(process, a)
c = client.submit(write, b, out_filename)
fire_and_forget(c)
return # here we lose the reference to c, but that's now ok
for filename in filenames:
process(filename)
Submit task and retrieve results from a different process¶
Sometimes we care about retrieving a result but not necessarily from the same process.
from distributed import Variable
var = Variable("my-result")
fut = client.submit(...)
var.set(fut)
Using a Variable
instructs dask to remember the result of this task under
the given name so that it can be retrieved later without having to keep the
Client alive in the meantime.
var = Variable("my-result")
fut = var.get()
result = fut.result()
Submit Tasks from Tasks¶
|
Get a client while within a task. |
|
Have this thread rejoin the ThreadPoolExecutor |
|
Have this task secede from the worker's thread pool |
This is an advanced feature and is rarely necessary in the common case.
Tasks can launch other tasks by getting their own client. This enables complex and highly dynamic workloads:
from dask.distributed import get_client
def my_function(x):
...
# Get locally created client
client = get_client()
# Do normal client operations, asking cluster for computation
a = client.submit(...)
b = client.submit(...)
a, b = client.gather([a, b])
return a + b
It also allows you to set up long running tasks that watch other resources like sockets or physical sensors:
def monitor(device):
client = get_client()
while True:
data = device.read_data()
future = client.submit(process, data)
fire_and_forget(future)
for device in devices:
fire_and_forget(client.submit(monitor))
However, each running task takes up a single thread, and so if you launch many
tasks that launch other tasks, then it is possible to deadlock the system if you
are not careful. You can call the secede
function from within a task to
have it remove itself from the dedicated thread pool into an administrative
thread that does not take up a slot within the Dask worker:
from dask.distributed import get_client, secede
def monitor(device):
client = get_client()
secede() # remove this task from the thread pool
while True:
data = device.read_data()
future = client.submit(process, data)
fire_and_forget(future)
If you intend to do more work in the same thread after waiting on client work, you may want to explicitly block until the thread is able to rejoin the thread pool. This allows some control over the number of threads that are created and stops too many threads from being active at once, over-saturating your hardware:
def f(n): # assume that this runs as a task
client = get_client()
secede() # secede while we wait for results to come back
futures = client.map(func, range(n))
results = client.gather(futures)
rejoin() # block until a slot is open in the thread pool
result = analyze(results)
return result
Alternatively, you can just use the normal compute
function within a
task. This will automatically call secede
and rejoin
appropriately:
def f(name, fn):
df = dd.read_csv(fn) # note that this is a dask collection
result = df[df.name == name].count()
# This calls secede
# Then runs the computation on the cluster (including this worker)
# Then blocks on rejoin, and finally delivers the answer
result = result.compute()
return result
Coordination Primitives¶
|
Distributed Queue |
|
Distributed Global Variable |
|
Distributed Centralized Lock |
|
Distributed Centralized Event equivalent to asyncio.Event |
|
This semaphore will track leases on the scheduler which can be acquired and released by an instance of this class. |
|
Publish data with Publish-Subscribe pattern |
|
Subscribe to a Publish/Subscribe topic |
Sometimes situations arise where tasks, workers, or clients need to coordinate with each other in ways beyond normal task scheduling with futures. In these cases Dask provides additional primitives to help in complex situations.
Dask provides distributed versions of coordination primitives like locks, events, queues, global variables, and pub-sub systems that, where appropriate, match their in-memory counterparts. These can be used to control access to external resources, track progress of ongoing computations, or share data in side-channels between many workers, clients, and tasks sensibly.
These features are rarely necessary for common use of Dask. We recommend that
beginning users stick with using the simpler futures found above (like
Client.submit
and Client.gather
) rather than embracing needlessly
complex techniques.
Queues¶
|
Distributed Queue |
Dask queues follow the API for the standard Python Queue, but now move futures or small messages between clients. Queues serialize sensibly and reconnect themselves on remote clients if necessary:
from dask.distributed import Queue
def load_and_submit(filename):
data = load(filename)
client = get_client()
future = client.submit(process, data)
queue.put(future)
client = Client()
queue = Queue()
for filename in filenames:
future = client.submit(load_and_submit, filename)
fire_and_forget(future)
while True:
future = queue.get()
print(future.result())
Queues can also send small pieces of information, anything that is msgpack encodable (ints, strings, bools, lists, dicts, etc.). This can be useful to send back small scores or administrative messages:
def func(x):
try:
...
except Exception as e:
error_queue.put(str(e))
error_queue = Queue()
Queues are mediated by the central scheduler, and so they are not ideal for sending large amounts of data (everything you send will be routed through a central point). They are well suited to move around small bits of metadata, or futures. These futures may point to much larger pieces of data safely:
>>> x = ... # my large numpy array
# Don't do this!
>>> q.put(x)
# Do this instead
>>> future = client.scatter(x)
>>> q.put(future)
# Or use futures for metadata
>>> q.put({'status': 'OK', 'stage=': 1234})
If you’re looking to move large amounts of data between workers, then you might also want to consider the Pub/Sub system described a few sections below.
Global Variables¶
|
Distributed Global Variable |
Variables are like Queues in that they communicate futures and small data between clients. However, variables hold only a single value. You can get or set that value at any time:
>>> var = Variable('stopping-criterion')
>>> var.set(False)
>>> var.get()
False
This is often used to signal stopping criteria or current parameters between clients.
If you want to share large pieces of information, then scatter the data first:
>>> parameters = np.array(...)
>>> future = client.scatter(parameters)
>>> var.set(future)
Locks¶
|
Distributed Centralized Lock |
You can also hold onto cluster-wide locks using the Lock
object.
Dask Locks have the same API as normal threading.Lock
objects, except that
they work across the cluster:
from dask.distributed import Lock
lock = Lock()
with lock:
# access protected resource
You can manage several locks at the same time. Lock can either be given a consistent name or you can pass the lock object around itself.
Using a consistent name is convenient when you want to lock some known named resource:
from dask.distributed import Lock
def load(fn):
with Lock('the-production-database'):
# read data from filename using some sensitive source
return ...
futures = client.map(load, filenames)
Passing around a lock works as well and is easier when you want to create short-term locks for a particular situation:
from dask.distributed import Lock
lock = Lock()
def load(fn, lock=None):
with lock:
# read data from filename using some sensitive source
return ...
futures = client.map(load, filenames, lock=lock)
This can be useful if you want to control concurrent access to some external resource like a database or un-thread-safe library.
Events¶
|
Distributed Centralized Event equivalent to asyncio.Event |
Dask Events mimic asyncio.Event
objects, but on a cluster scope.
They hold a single flag which can be set or cleared.
Clients can wait until the event flag is set.
Different from a Lock
, every client can set or clear the flag and there
is no “ownership” of an event.
You can use events to e.g. synchronize multiple clients:
# One one client
from dask.distributed import Event
event = Event("my-event-1")
event.wait()
The call to wait will block until the event is set, e.g. in another client
# In another client
from dask.distributed import Event
event = Event("my-event-1")
# do some work
event.set()
Events can be set, cleared and waited on multiple times. Every waiter referencing the same event name will be notified on event set (and not only the first one as in the case of a lock):
from dask.distributed import Event
def wait_for_event(x):
event = Event("my-event")
event.wait()
# at this point, all function calls
# are in sync once the event is set
futures = client.map(wait_for_event, range(10))
Event("my-event").set()
client.gather(futures)
Semaphore¶
|
This semaphore will track leases on the scheduler which can be acquired and released by an instance of this class. |
Similar to the single-valued Lock
it is also possible to use a cluster-wide
semaphore to coordinate and limit access to a sensitive resource like a
database.
from dask.distributed import Semaphore
sem = Semaphore(max_leases=2, name="database")
def access_limited(val, sem):
with sem:
# Interact with the DB
return
futures = client.map(access_limited, range(10), sem=sem)
client.gather(futures)
sem.close()
Publish-Subscribe¶
|
Publish data with Publish-Subscribe pattern |
|
Subscribe to a Publish/Subscribe topic |
Dask implements the Publish Subscribe pattern, providing an additional channel of communication between ongoing tasks.
Actors¶
Actors allow workers to manage rapidly changing state without coordinating with the central scheduler. This has the advantage of reducing latency (worker-to-worker roundtrip latency is around 1ms), reducing pressure on the centralized scheduler (workers can coordinate actors entirely among each other), and also enabling workflows that require stateful or in-place memory manipulation.
However, these benefits come at a cost. The scheduler is unaware of actors and so they don’t benefit from diagnostics, load balancing, or resilience. Once an actor is running on a worker it is forever tied to that worker. If that worker becomes overburdened or dies, then there is no opportunity to recover the workload.
Because Actors avoid the central scheduler they can be high-performing, but not resilient.
Example: Counter¶
An actor is a class containing both state and methods that is submitted to a worker:
class Counter:
n = 0
def __init__(self):
self.n = 0
def increment(self):
self.n += 1
return self.n
from dask.distributed import Client
client = Client()
future = client.submit(Counter, actor=True)
counter = future.result()
>>> counter
<Actor: Counter, key=Counter-afa1cdfb6b4761e616fa2cfab42398c8>
Method calls on this object produce ActorFutures
, which are similar to
normal Futures, but interact only with the worker holding the Actor:
>>> future = counter.increment()
>>> future
<ActorFuture>
>>> future.result()
1
Attribute access is synchronous and blocking:
>>> counter.n
1
Example: Parameter Server¶
This example will perform the following minimization with a parameter server:
This is a simple minimization that will serve as an illustrative example.
The Dask Actor will serve as the parameter server that will hold the model. The client will calculate the gradient of the loss function above.
import numpy as np
from dask.distributed import Client
client = Client(processes=False)
class ParameterServer:
def __init__(self):
self.data = dict()
def put(self, key, value):
self.data[key] = value
def get(self, key):
return self.data[key]
def train(params, lr=0.1):
grad = 2 * (params - 1) # gradient of (params - 1)**2
new_params = params - lr * grad
return new_params
ps_future = client.submit(ParameterServer, actor=True)
ps = ps_future.result()
ps.put('parameters', np.random.default_rng().random(1000))
for k in range(20):
params = ps.get('parameters').result()
new_params = train(params)
ps.put('parameters', new_params)
print(new_params.mean())
# k=0: "0.5988202981316124"
# k=10: "0.9569236575164062"
This example works, and the loss function is minimized. The (simple) equation above is minimize, so each \(p_i\) converges to 1. If desired, this example could be adapted to machine learning with a more complex function to minimize.
Asynchronous Operation¶
All operations that require talking to the remote worker are awaitable:
async def f():
future = client.submit(Counter, actor=True)
counter = await future # gather actor object locally
counter.increment() # send off a request asynchronously
await counter.increment() # or wait until it was received
n = await counter.n # attribute access also must be awaited
Generally, all I/O operations that trigger computations (e.g. to_parquet
) should be done using the compute=False
parameter to avoid asynchronous blocking:
await client.compute(ddf.to_parquet('/tmp/some.parquet', compute=False))
API¶
Client
|
Connect to and submit computation to a Dask cluster |
|
Cancel running futures This stops future tasks from being scheduled if they have not yet run and deletes them if they have already run. |
|
Compute dask collections on cluster |
|
Gather futures from distributed memory |
|
Compute dask graph |
|
Get named dataset from the scheduler if present. |
|
Return a concurrent.futures Executor for submitting tasks on this Client |
|
Which keys are held by which workers |
|
List named datasets available on the scheduler |
|
Map a function on a sequence of arguments |
|
The number of threads/cores available on each worker node |
|
Persist dask collections on cluster |
|
Collect statistical profiling information about recent work |
|
Publish named datasets to scheduler |
|
Rebalance data within network |
|
Set replication of futures within network |
|
Restart all workers. |
|
Run a function on all workers outside of task scheduling system |
|
Run a function on the scheduler process |
|
Scatter data into distributed memory |
Shut down the connected scheduler and workers |
|
|
Basic information about the workers in the cluster |
|
Submit a function application to the scheduler |
|
Remove named datasets from scheduler |
|
Upload local package to scheduler and workers |
|
The workers storing each future's data |
Future
|
A remotely running computation |
Call callback on future when future has finished |
|
|
Cancel the request to run this future |
Returns True if the future has been cancelled |
|
Returns whether or not the computation completed. |
|
|
Return the exception of a failed task |
|
Wait until computation completes, gather result to local process. |
|
Return the traceback of a failed task |
Functions
|
Return futures in the order in which they complete |
|
Run tasks at least once, even if we release the futures |
|
Get a client while within a task. |
|
Have this task secede from the worker's thread pool |
|
Have this thread rejoin the ThreadPoolExecutor |
|
Wait until all/any futures are finished |
|
A drop-in replacement of the built-in |
|
A drop-in replacement of the built-in |
- distributed.as_completed(futures=None, loop=None, with_results=False, raise_errors=True, *, timeout=None)[source]¶
Return futures in the order in which they complete
This returns an iterator that yields the input future objects in the order in which they complete. Calling
next
on the iterator will block until the next future completes, irrespective of order.Additionally, you can also add more futures 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 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)- raise_errors: bool (True)
Whether we should raise when the result of a future raises an exception; only affects behavior when
with_results=True
.- timeout: int (optional)
The returned iterator raises a
dask.distributed.TimeoutError
if__next__()
or__anext__()
is called and the result isn’t available after timeout seconds from the original call toas_completed()
. If timeout is not specified orNone
, there is no limit to the wait time.
Examples
>>> x, y, z = client.map(inc, [1, 2, 3]) >>> for future in as_completed([x, y, z]): ... print(future.result()) 3 2 4
Add more futures during computation
>>> x, y, z = client.map(inc, [1, 2, 3]) >>> ac = as_completed([x, y, z]) >>> for future in ac: ... print(future.result()) ... if random.random() < 0.5: ... ac.add(c.submit(double, future)) 4 2 8 3 6 12 24
Optionally wait until the result has been gathered as well
>>> ac = as_completed([x, y, z], with_results=True) >>> for future, result in ac: ... print(result) 2 4 3
- distributed.fire_and_forget(obj)[source]¶
Run tasks at least once, even if we release the futures
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 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 side effects.
- Parameters
- objFuture, list, dict, dask collection
The futures that you want to run at least once
Examples
>>> fire_and_forget(client.submit(func, *args))
- distributed.get_client(address=None, timeout=None, resolve_address=True) Client [source]¶
Get a client while within a task.
This client connects to the same scheduler to which the worker is connected
- Parameters
- addressstr, optional
The address of the scheduler to connect to. Defaults to the scheduler the worker is connected to.
- timeoutint or str
Timeout (in seconds) for getting the Client. Defaults to the
distributed.comm.timeouts.connect
configuration value.- resolve_addressbool, default True
Whether to resolve address to its canonical form.
- Returns
- Client
See also
Examples
>>> def f(): ... client = get_client(timeout="10s") ... futures = client.map(lambda x: x + 1, range(10)) # spawn many tasks ... results = client.gather(futures) ... return sum(results)
>>> future = client.submit(f) >>> future.result() 55
- distributed.secede()[source]¶
Have this task secede from the worker’s thread pool
This opens up a new scheduling slot and a new thread for a new task. This enables the client to schedule tasks on this node, which is especially useful while waiting for other jobs to finish (e.g., with
client.gather
).See also
Examples
>>> def mytask(x): ... # do some work ... client = get_client() ... futures = client.map(...) # do some remote work ... secede() # while that work happens, remove ourself from the pool ... return client.gather(futures) # return gathered results
- distributed.rejoin()[source]¶
Have this thread rejoin the ThreadPoolExecutor
This will block until a new slot opens up in the executor. The next thread to finish a task will leave the pool to allow this one to join.
See also
secede
leave the thread pool
- distributed.wait(fs, timeout=None, return_when='ALL_COMPLETED')[source]¶
Wait until all/any futures are finished
- Parameters
- fsList[Future]
- timeoutnumber, 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.- return_whenstr, optional
One of ALL_COMPLETED or FIRST_COMPLETED
- Returns
- Named tuple of completed, not completed
- distributed.print(*args, sep: str | None = ' ', end: str | None = '\n', file: TextIO | None = None, flush: bool = False) None [source]¶
A drop-in replacement of the built-in
print
function for remote printing from workers to clients. If called from outside a dask worker, its arguments are passed directly tobuiltins.print()
. If called by code running on a worker, then in addition to printing locally, any clients connected (possibly remotely) to the scheduler managing this worker will receive an event instructing them to print the same output to their own standard output or standard error streams. For example, the user can perform simple debugging of remote computations by including calls to thisprint
function in the submitted code and inspecting the output in a local Jupyter notebook or interpreter session.All arguments behave the same as those of
builtins.print()
, with the exception that thefile
keyword argument, if specified, must either besys.stdout
orsys.stderr
; arbitrary file-like objects are not allowed.All non-keyword arguments are converted to strings using
str()
and written to the stream, separated bysep
and followed byend
. Bothsep
andend
must be strings; they can also beNone
, which means to use the default values. If no objects are given,print()
will just writeend
.- Parameters
- sepstr, optional
String inserted between values, default a space.
- endstr, optional
String appended after the last value, default a newline.
- file
sys.stdout
orsys.stderr
, optional Defaults to the current sys.stdout.
- flushbool, default False
Whether to forcibly flush the stream.
Examples
>>> from dask.distributed import Client, print >>> client = distributed.Client(...) >>> def worker_function(): ... print("Hello from worker!") >>> client.submit(worker_function) <Future: finished, type: NoneType, key: worker_function-...> Hello from worker!
- distributed.warn(message: str | Warning, category: type[Warning] | None = <class 'UserWarning'>, stacklevel: int = 1, source: typing.Any = None) None [source]¶
A drop-in replacement of the built-in
warnings.warn()
function for issuing warnings remotely from workers to clients.If called from outside a dask worker, its arguments are passed directly to
warnings.warn()
. If called by code running on a worker, then in addition to emitting a warning locally, any clients connected (possibly remotely) to the scheduler managing this worker will receive an event instructing them to emit the same warning (subject to their own local filters, etc.). When implementing computations that may run on a worker, the user can call thiswarn
function to ensure that any remote client sessions will see their warnings, for example in a Jupyter output cell.While all of the arguments are respected by the locally emitted warning (with same meanings as in
warnings.warn()
),stacklevel
andsource
are ignored by clients because they would not be meaningful in the client’s thread.Examples
>>> from dask.distributed import Client, warn >>> client = Client() >>> def do_warn(): ... warn("A warning from a worker.") >>> client.submit(do_warn).result() /path/to/distributed/client.py:678: UserWarning: A warning from a worker.
- class distributed.Client(address=None, loop=None, timeout=_NoDefault.no_default, set_as_default=True, scheduler_file=None, security=None, asynchronous=False, name=None, heartbeat_interval=None, serializers=None, deserializers=None, extensions={'pubsub': <class 'distributed.pubsub.PubSubClientExtension'>}, direct_to_workers=None, connection_limit=512, **kwargs)[source]¶
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 allowsFuture
objects withinsubmit/map
calls. When a Client is instantiated it takes over alldask.compute
anddask.persist
calls by default.It is also common to create a Client without specifying the scheduler address , like
Client()
. In this case the Client creates aLocalCluster
in the background and connects to that. Any extra keywords are passed from Client to LocalCluster in this case. See the LocalCluster documentation for more information.- Parameters
- address: string, or Cluster
This can be the address of a
Scheduler
server like a string'127.0.0.1:8786'
or a cluster object likeLocalCluster()
- loop
The event loop
- timeout: int (defaults to configuration ``distributed.comm.timeouts.connect``)
Timeout duration for initial connection to the scheduler
- set_as_default: bool (True)
Use this Client as the global dask scheduler
- scheduler_file: string (optional)
Path to a file with scheduler information if available
- security: Security or bool, optional
Optional security information. If creating a local cluster can also pass in
True
, in which case temporary self-signed credentials will be created automatically.- asynchronous: bool (False by default)
Set to True if using this client within async/await functions or within Tornado gen.coroutines. Otherwise this should remain False for normal use.
- name: string (optional)
Gives the client a name that will be included in logs generated on the scheduler for matters relating to this client
- heartbeat_interval: int (optional)
Time in milliseconds between heartbeats to scheduler
- serializers
Iterable of approaches to use when serializing the object. See Serialization for more.
- deserializers
Iterable of approaches to use when deserializing the object. See Serialization for more.
- extensionslist
The extensions
- direct_to_workers: bool (optional)
Whether or not to connect directly to the workers, or to ask the scheduler to serve as intermediary.
- connection_limitint
The number of open comms to maintain at once in the connection pool
- **kwargs:
If you do not pass a scheduler address, Client will create a
LocalCluster
object, passing any extra keyword arguments.
See also
distributed.scheduler.Scheduler
Internal scheduler
distributed.LocalCluster
Examples
Provide cluster’s scheduler node address on initialization:
>>> client = Client('127.0.0.1:8786')
Use
submit
method to send individual computations to the cluster>>> a = client.submit(add, 1, 2) >>> b = client.submit(add, 10, 20)
Continue using submit or map on results to build up larger computations
>>> c = client.submit(add, a, b)
Gather results with the
gather
method.>>> client.gather(c) 33
You can also call Client with no arguments in order to create your own local cluster.
>>> client = Client() # makes your own local "cluster"
Extra keywords will be passed directly to LocalCluster
>>> client = Client(n_workers=2, threads_per_worker=4)
- property amm¶
Convenience accessors for the Active Memory Manager
- as_current()[source]¶
Thread-local, Task-local context manager that causes the Client.current class method to return self. Any Future objects deserialized inside this context manager will be automatically attached to this Client.
- benchmark_hardware() dict [source]¶
Run a benchmark on the workers for memory, disk, and network bandwidths
- Returns
- result: dict
A dictionary mapping the names “disk”, “memory”, and “network” to dictionaries mapping sizes to bandwidths. These bandwidths are averaged over many workers running computations across the cluster.
- call_stack(futures=None, keys=None)[source]¶
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 thekeys=
keyword. If neither are provided then all call stacks will be returned.- Parameters
- futureslist (optional)
List of futures, defaults to all data
- keyslist (optional)
List of key names, defaults to all data
Examples
>>> df = dd.read_parquet(...).persist() >>> client.call_stack(df) # call on collections
>>> client.call_stack() # Or call with no arguments for all activity
- cancel(futures, asynchronous=None, force=False, reason=None, msg=None)[source]¶
Cancel running futures This stops future 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
- futuresList[Future]
The list of Futures
- asynchronous: bool
If True the client is in asynchronous mode
- forceboolean (False)
Cancel this future even if other clients desire it
- reason: str
Reason for cancelling the futures
- msgstr
Message that will be attached to the cancelled future
- close(timeout=_NoDefault.no_default)[source]¶
Close this client
Clients will also close automatically when your Python session ends
If you started a client without arguments like
Client()
then this will also close the local cluster that was started at the same time.- Parameters
- timeoutnumber
Time in seconds after which to raise a
dask.distributed.TimeoutError
See also
- static collections_to_dsk(collections, *args, **kwargs)[source]¶
Convert many collections into a single dask graph, after optimization
- compute(collections, sync=False, optimize_graph=True, workers=None, allow_other_workers=False, resources=None, retries=0, priority=0, fifo_timeout='60s', actors=None, traverse=True, **kwargs)[source]¶
Compute dask collections on cluster
- Parameters
- collectionsiterable of dask objects or single dask object
Collections like dask.array or dataframe or dask.value objects
- syncbool (optional)
Returns Futures if False (default) or concrete values if True
- optimize_graphbool
Whether or not to optimize the underlying graphs
- workersstring or iterable of strings
A set of worker hostnames on which computations may be performed. Leave empty to default to all workers (common case)
- allow_other_workersbool (defaults to False)
Used with workers. Indicates whether or not the computations may be performed on workers that are not in the workers set(s).
- retriesint (default to 0)
Number of allowed automatic retries if computing a result fails
- priorityNumber
Optional prioritization of task. Zero is default. Higher priorities take precedence
- fifo_timeouttimedelta str (defaults to ’60s’)
Allowed amount of time between calls to consider the same priority
- traversebool (defaults to True)
By default dask traverses builtin python collections looking for dask objects passed to
compute
. For large collections this can be expensive. If none of the arguments contain any dask objects, settraverse=False
to avoid doing this traversal.- resourcesdict (defaults to {})
Defines the resources each instance of this mapped task requires on the worker; e.g.
{'GPU': 2}
. See worker resources for details on defining resources.- actorsbool or dict (default None)
Whether these tasks should exist on the worker as stateful actors. Specified on a global (True/False) or per-task (
{'x': True, 'y': False}
) basis. See Actors for additional details.- **kwargs
Options to pass to the graph optimize calls
- Returns
- List of Futures if input is a sequence, or a single future otherwise
See also
Client.get
Normal synchronous dask.get function
Examples
>>> from dask import delayed >>> from operator import add >>> x = delayed(add)(1, 2) >>> y = delayed(add)(x, x) >>> xx, yy = client.compute([x, y]) >>> xx <Future: status: finished, key: add-8f6e709446674bad78ea8aeecfee188e> >>> xx.result() 3 >>> yy.result() 6
Also support single arguments
>>> xx = client.compute(x)
- classmethod current(allow_global=True)[source]¶
When running within the context of as_client, return the context-local current client. Otherwise, return the latest initialised Client. If no Client instances exist, raise ValueError. If allow_global is set to False, raise ValueError if running outside of the as_client context manager.
- Parameters
- allow_globalbool
If True returns the default client
- Returns
- Client
The current client
- Raises
- ValueError
If there is no client set, a ValueError is raised
See also
default_client
- property dashboard_link¶
Link to the scheduler’s dashboard.
- Returns
- str
Dashboard URL.
Examples
Opening the dashboard in your default web browser:
>>> import webbrowser >>> from distributed import Client >>> client = Client() >>> webbrowser.open(client.dashboard_link)
- dump_cluster_state(filename: str = 'dask-cluster-dump', write_from_scheduler: bool | None = None, exclude: collections.abc.Collection[str] = ('run_spec',), format: Literal['msgpack', 'yaml'] = 'msgpack', **storage_options)[source]¶
Extract a dump of the entire cluster state and persist to disk or a URL. This is intended for debugging purposes only.
Warning: Memory usage on the scheduler (and client, if writing the dump locally) can be large. On a large or long-running cluster, this can take several minutes. The scheduler may be unresponsive while the dump is processed.
Results will be stored in a dict:
{ "scheduler": {...}, # scheduler state "workers": { worker_addr: {...}, # worker state ... } "versions": { "scheduler": {...}, "workers": { worker_addr: {...}, ... } } }
- Parameters
- filename:
The path or URL to write to. The appropriate file suffix (
.msgpack.gz
or.yaml
) will be appended automatically.Must be a path supported by
fsspec.open()
(likes3://my-bucket/cluster-dump
, orcluster-dumps/dump
). Seewrite_from_scheduler
to control whether the dump is written directly tofilename
from the scheduler, or sent back to the client over the network, then written locally.- write_from_scheduler:
If None (default), infer based on whether
filename
looks like a URL or a local path: True if the filename contains://
(likes3://my-bucket/cluster-dump
), False otherwise (likelocal_dir/cluster-dump
).If True, write cluster state directly to
filename
from the scheduler. Iffilename
is a local path, the dump will be written to that path on the scheduler’s filesystem, so be careful if the scheduler is running on ephemeral hardware. Useful when the scheduler is attached to a network filesystem or persistent disk, or for writing to buckets.If False, transfer cluster state from the scheduler back to the client over the network, then write it to
filename
. This is much less efficient for large dumps, but useful when the scheduler doesn’t have access to any persistent storage.- exclude:
A collection of attribute names which are supposed to be excluded from the dump, e.g. to exclude code, tracebacks, logs, etc.
Defaults to exclude
run_spec
, which is the serialized user code. This is typically not required for debugging. To allow serialization of this, pass an empty tuple.- format:
Either
"msgpack"
or"yaml"
. If msgpack is used (default), the output will be stored in a gzipped file as msgpack.To read:
import gzip, msgpack with gzip.open("filename") as fd: state = msgpack.unpack(fd)
or:
import yaml try: from yaml import CLoader as Loader except ImportError: from yaml import Loader with open("filename") as fd: state = yaml.load(fd, Loader=Loader)
- **storage_options:
Any additional arguments to
fsspec.open()
when writing to a URL.
- forward_logging(logger_name=None, level=0)[source]¶
Begin forwarding the given logger (by default the root) and all loggers under it from worker tasks to the client process. Whenever the named logger handles a LogRecord on the worker-side, the record will be serialized, sent to the client, and handled by the logger with the same name on the client-side.
Note that worker-side loggers will only handle LogRecords if their level is set appropriately, and the client-side logger will only emit the forwarded LogRecord if its own level is likewise set appropriately. For example, if your submitted task logs a DEBUG message to logger “foo”, then in order for
forward_logging()
to cause that message to be emitted in your client session, you must ensure that the logger “foo” have its level set to DEBUG (or lower) in the worker process and in the client process.- Parameters
- logger_namestr, optional
The name of the logger to begin forwarding. The usual rules of the
logging
module’s hierarchical naming system apply. For example, ifname
is"foo"
, then not only"foo"
, but also"foo.bar"
,"foo.baz"
, etc. will be forwarded. Ifname
isNone
, this indicates the root logger, and so all loggers will be forwarded.Note that a logger will only forward a given LogRecord if the logger’s level is sufficient for the LogRecord to be handled at all.
- levelstr | int, optional
Optionally restrict forwarding to LogRecords of this level or higher, even if the forwarded logger’s own level is lower.
Examples
For purposes of the examples, suppose we configure client-side logging as a user might: with a single StreamHandler attached to the root logger with an output level of INFO and a simple output format:
import logging import distributed import io, yaml TYPICAL_LOGGING_CONFIG = ''' version: 1 handlers: console: class : logging.StreamHandler formatter: default level : INFO formatters: default: format: '%(asctime)s %(levelname)-8s [worker %(worker)s] %(name)-15s %(message)s' datefmt: '%Y-%m-%d %H:%M:%S' root: handlers: - console ''' config = yaml.safe_load(io.StringIO(TYPICAL_LOGGING_CONFIG)) logging.config.dictConfig(config)
Now create a client and begin forwarding the root logger from workers back to our local client process.
>>> client = distributed.Client() >>> client.forward_logging() # forward the root logger at any handled level
Then submit a task that does some error logging on a worker. We see output from the client-side StreamHandler.
>>> def do_error(): ... logging.getLogger("user.module").error("Hello error") ... return 42 >>> client.submit(do_error).result() 2022-11-09 03:43:25 ERROR [worker tcp://127.0.0.1:34783] user.module Hello error 42
Note how an attribute
"worker"
is also added by dask to the forwarded LogRecord, which our custom formatter uses. This is useful for identifying exactly which worker logged the error.One nuance worth highlighting: even though our client-side root logger is configured with a level of INFO, the worker-side root loggers still have their default level of ERROR because we haven’t done any explicit logging configuration on the workers. Therefore worker-side INFO logs will not be forwarded because they never even get handled in the first place.
>>> def do_info_1(): ... # no output on the client side ... logging.getLogger("user.module").info("Hello info the first time") ... return 84 >>> client.submit(do_info_1).result() 84
It is necessary to set the client-side logger’s level to INFO before the info message will be handled and forwarded to the client. In other words, the “effective” level of the client-side forwarded logging is the maximum of each logger’s client-side and worker-side levels.
>>> def do_info_2(): ... logger = logging.getLogger("user.module") ... logger.setLevel(logging.INFO) ... # now produces output on the client side ... logger.info("Hello info the second time") ... return 84 >>> client.submit(do_info_2).result() 2022-11-09 03:57:39 INFO [worker tcp://127.0.0.1:42815] user.module Hello info the second time 84
- gather(futures, errors='raise', direct=None, asynchronous=None)[source]¶
Gather futures from distributed memory
Accepts a future, nested container of futures, iterator, or queue. The return type will match the input type.
- Parameters
- futuresCollection of futures
This can be a possibly nested collection of Future objects. Collections can be lists, sets, or dictionaries
- errorsstring
Either ‘raise’ or ‘skip’ if we should raise if a future has erred or skip its inclusion in the output collection
- directboolean
Whether or not to connect directly to the workers, or to ask the scheduler to serve as intermediary. This can also be set when creating the Client.
- asynchronous: bool
If True the client is in asynchronous mode
- Returns
- results: a collection of the same type as the input, but now with
- gathered results rather than futures
See also
Client.scatter
Send data out to cluster
Examples
>>> from operator import add >>> c = Client('127.0.0.1:8787') >>> x = c.submit(add, 1, 2) >>> c.gather(x) 3 >>> c.gather([x, [x], x]) # support lists and dicts [3, [3], 3]
- get(dsk, keys, workers=None, allow_other_workers=None, resources=None, sync=True, asynchronous=None, direct=None, retries=None, priority=0, fifo_timeout='60s', actors=None, **kwargs)[source]¶
Compute dask graph
- Parameters
- dskdict
- keysobject, or nested lists of objects
- workersstring or iterable of strings
A set of worker addresses or hostnames on which computations may be performed. Leave empty to default to all workers (common case)
- allow_other_workersbool (defaults to False)
Used with
workers
. Indicates whether or not the computations may be performed on workers that are not in the workers set(s).- resourcesdict (defaults to {})
Defines the
resources
each instance of this mapped task requires on the worker; e.g.{'GPU': 2}
. See worker resources for details on defining resources.- syncbool (optional)
Returns Futures if False or concrete values if True (default).
- asynchronous: bool
If True the client is in asynchronous mode
- directbool
Whether or not to connect directly to the workers, or to ask the scheduler to serve as intermediary. This can also be set when creating the Client.
- retriesint (default to 0)
Number of allowed automatic retries if computing a result fails
- priorityNumber
Optional prioritization of task. Zero is default. Higher priorities take precedence
- fifo_timeouttimedelta str (defaults to ’60s’)
Allowed amount of time between calls to consider the same priority
- actorsbool or dict (default None)
Whether these tasks should exist on the worker as stateful actors. Specified on a global (True/False) or per-task (
{'x': True, 'y': False}
) basis. See Actors for additional details.
- Returns
- results
If ‘sync’ is True, returns the results. Otherwise, returns the known data packed If ‘sync’ is False, returns the known data. Otherwise, returns the results
See also
Client.compute
Compute asynchronous collections
Examples
>>> from operator import add >>> c = Client('127.0.0.1:8787') >>> c.get({'x': (add, 1, 2)}, 'x') 3
- get_dataset(name, default=_NoDefault.no_default, **kwargs)[source]¶
Get named dataset from the scheduler if present. Return the default or raise a KeyError if not present.
- Parameters
- namestr
name of the dataset to retrieve
- defaultstr
optional, not set by default If set, do not raise a KeyError if the name is not present but return this default
- kwargsdict
additional keyword arguments to _get_dataset
- Returns
- The dataset from the scheduler, if present
- get_events(topic: str | None = None)[source]¶
Retrieve structured topic logs
- Parameters
- topicstr, optional
Name of topic log to retrieve events for. If no
topic
is provided, then logs for all topics will be returned.
- get_executor(**kwargs)[source]¶
Return a concurrent.futures Executor for submitting tasks on this Client
- Parameters
- **kwargs
Any submit()- or map()- compatible arguments, such as workers or resources.
- Returns
- ClientExecutor
An Executor object that’s fully compatible with the concurrent.futures API.
- get_metadata(keys, default=_NoDefault.no_default)[source]¶
Get arbitrary metadata from scheduler
See set_metadata for the full docstring with examples
- Parameters
- keyskey or list
Key to access. If a list then gets within a nested collection
- defaultoptional
If the key does not exist then return this value instead. If not provided then this raises a KeyError if the key is not present
See also
- get_scheduler_logs(n=None)[source]¶
Get logs from scheduler
- Parameters
- nint
Number of logs to retrieve. Maxes out at 10000 by default, configurable via the
distributed.admin.log-length
configuration value.
- Returns
- Logs in reversed order (newest first)
- get_task_stream(start=None, stop=None, count=None, plot=False, filename='task-stream.html', bokeh_resources=None)[source]¶
Get task stream data from scheduler
This collects the data present in the diagnostic “Task Stream” plot on the dashboard. It includes the start, stop, transfer, and deserialization time of every task for a particular duration.
Note that the task stream diagnostic does not run by default. You may wish to call this function once before you start work to ensure that things start recording, and then again after you have completed.
- Parameters
- startNumber or string
When you want to start recording If a number it should be the result of calling time() If a string then it should be a time difference before now, like ’60s’ or ‘500 ms’
- stopNumber or string
When you want to stop recording
- countint
The number of desired records, ignored if both start and stop are specified
- plotboolean, str
If true then also return a Bokeh figure If plot == ‘save’ then save the figure to a file
- filenamestr (optional)
The filename to save to if you set
plot='save'
- bokeh_resourcesbokeh.resources.Resources (optional)
Specifies if the resource component is INLINE or CDN
- Returns
- L: List[Dict]
See also
get_task_stream
a context manager version of this method
Examples
>>> client.get_task_stream() # prime plugin if not already connected >>> x.compute() # do some work >>> client.get_task_stream() [{'task': ..., 'type': ..., 'thread': ..., ...}]
Pass the
plot=True
orplot='save'
keywords to get back a Bokeh figure>>> data, figure = client.get_task_stream(plot='save', filename='myfile.html')
Alternatively consider the context manager
>>> from dask.distributed import get_task_stream >>> with get_task_stream() as ts: ... x.compute() >>> ts.data [...]
- get_versions(check: bool = False, packages: collections.abc.Sequence[str] | None = None) distributed.client.VersionsDict | collections.abc.Coroutine[Any, Any, distributed.client.VersionsDict] [source]¶
Return version info for the scheduler, all workers and myself
- Parameters
- check
raise ValueError if all required & optional packages do not match
- packages
Extra package names to check
Examples
>>> c.get_versions()
>>> c.get_versions(packages=['sklearn', 'geopandas'])
- get_worker_logs(n=None, workers=None, nanny=False)[source]¶
Get logs from workers
- Parameters
- nint
Number of logs to retrieve. Maxes out at 10000 by default, configurable via the
distributed.admin.log-length
configuration value.- workersiterable
List of worker addresses to retrieve. Gets all workers by default.
- nannybool, default False
Whether to get the logs from the workers (False) or the nannies (True). If specified, the addresses in workers should still be the worker addresses, not the nanny addresses.
- Returns
- Dictionary mapping worker address to logs.
- Logs are returned in reversed order (newest first)
- has_what(workers=None, **kwargs)[source]¶
Which keys are held by which workers
This returns the keys of the data that are held in each worker’s memory.
- Parameters
- workerslist (optional)
A list of worker addresses, defaults to all
- **kwargsdict
Optional keyword arguments for the remote function
Examples
>>> x, y, z = c.map(inc, [1, 2, 3]) >>> wait([x, y, z]) >>> c.has_what() {'192.168.1.141:46784': ['inc-1c8dd6be1c21646c71f76c16d09304ea', 'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b', 'inc-1e297fc27658d7b67b3a758f16bcf47a']}
- log_event(topic: str | collections.abc.Collection[str], msg: Any)[source]¶
Log an event under a given topic
- Parameters
- topicstr, list[str]
Name of the topic under which to log an event. To log the same event under multiple topics, pass a list of topic names.
- msg
Event message to log. Note this must be msgpack serializable.
Examples
>>> from time import time >>> client.log_event("current-time", time())
- map(func: Callable, *iterables: collections.abc.Collection, key: str | list | None = None, workers: str | collections.abc.Iterable[str] | None = None, retries: int | None = None, resources: dict[str, Any] | None = None, priority: int = 0, allow_other_workers: bool = False, fifo_timeout: str = '100 ms', actor: bool = False, actors: bool = False, pure: bool = True, batch_size=None, **kwargs)[source]¶
Map a function on a sequence of arguments
Arguments can be normal objects or Futures
- Parameters
- funccallable
Callable to be scheduled for execution. If
func
returns a coroutine, it will be run on the main event loop of a worker. Otherwisefunc
will be run in a worker’s task executor pool (seeWorker.executors
for more information.)- iterablesIterables
List-like objects to map over. They should have the same length.
- keystr, list
Prefix for task names if string. Explicit names if list.
- workersstring or iterable of strings
A set of worker hostnames on which computations may be performed. Leave empty to default to all workers (common case)
- retriesint (default to 0)
Number of allowed automatic retries if a task fails
- resourcesdict (defaults to {})
Defines the resources each instance of this mapped task requires on the worker; e.g.
{'GPU': 2}
. See worker resources for details on defining resources.- priorityNumber
Optional prioritization of task. Zero is default. Higher priorities take precedence
- allow_other_workersbool (defaults to False)
Used with workers. Indicates whether or not the computations may be performed on workers that are not in the workers set(s).
- fifo_timeoutstr timedelta (default ‘100ms’)
Allowed amount of time between calls to consider the same priority
- actorbool (default False)
Whether these tasks should exist on the worker as stateful actors. See Actors for additional details.
- actorsbool (default False)
Alias for actor
- purebool (defaults to True)
Whether or not the function is pure. Set
pure=False
for impure functions likenp.random.random
. Note that if bothactor
andpure
kwargs are set to True, then the value ofpure
will be reverted to False, since an actor is stateful. See Pure Functions by Default for more details.- batch_sizeint, optional (default: just one batch whose size is the entire iterable)
Submit tasks to the scheduler in batches of (at most)
batch_size
. The tradeoff in batch size is that large batches avoid more per-batch overhead, but batches that are too big can take a long time to submit and unreasonably delay the cluster from starting its processing.- **kwargsdict
Extra keyword arguments to send to the function. Large values will be included explicitly in the task graph.
- Returns
- List, iterator, or Queue of futures, depending on the type of the
- inputs.
See also
Client.submit
Submit a single function
Notes
The current implementation of a task graph resolution searches for occurrences of
key
and replaces it with a correspondingFuture
result. That can lead to unwanted substitution of strings passed as arguments to a task if these strings match somekey
that already exists on a cluster. To avoid these situations it is required to use unique values if akey
is set manually. See https://github.com/dask/dask/issues/9969 to track progress on resolving this issue.Examples
>>> L = client.map(func, sequence)
- nbytes(keys=None, summary=True, **kwargs)[source]¶
The bytes taken up by each key on the cluster
This is as measured by
sys.getsizeof
which may not accurately reflect the true cost.- Parameters
- keyslist (optional)
A list of keys, defaults to all keys
- summaryboolean, (optional)
Summarize keys into key types
- **kwargsdict
Optional keyword arguments for the remote function
See also
Examples
>>> x, y, z = c.map(inc, [1, 2, 3]) >>> c.nbytes(summary=False) {'inc-1c8dd6be1c21646c71f76c16d09304ea': 28, 'inc-1e297fc27658d7b67b3a758f16bcf47a': 28, 'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b': 28}
>>> c.nbytes(summary=True) {'inc': 84}
- ncores(workers=None, **kwargs)¶
The number of threads/cores available on each worker node
- Parameters
- workerslist (optional)
A list of workers that we care about specifically. Leave empty to receive information about all workers.
- **kwargsdict
Optional keyword arguments for the remote function
See also
Examples
>>> c.nthreads() {'192.168.1.141:46784': 8, '192.167.1.142:47548': 8, '192.167.1.143:47329': 8, '192.167.1.144:37297': 8}
- normalize_collection(collection)[source]¶
Replace collection’s tasks by already existing futures 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.
- Parameters
- collectiondask object
Collection like dask.array or dataframe or dask.value objects
- Returns
- collectiondask object
Collection with its tasks replaced with any existing futures.
See also
Client.persist
trigger computation of collection’s tasks
Examples
>>> len(x.__dask_graph__()) # x is a dask collection with 100 tasks 100 >>> set(client.futures).intersection(x.__dask_graph__()) # some overlap exists 10
>>> x = client.normalize_collection(x) >>> len(x.__dask_graph__()) # smaller computational graph 20
- nthreads(workers=None, **kwargs)[source]¶
The number of threads/cores available on each worker node
- Parameters
- workerslist (optional)
A list of workers that we care about specifically. Leave empty to receive information about all workers.
- **kwargsdict
Optional keyword arguments for the remote function
See also
Examples
>>> c.nthreads() {'192.168.1.141:46784': 8, '192.167.1.142:47548': 8, '192.167.1.143:47329': 8, '192.167.1.144:37297': 8}
- persist(collections, optimize_graph=True, workers=None, allow_other_workers=None, resources=None, retries=None, priority=0, fifo_timeout='60s', actors=None, **kwargs)[source]¶
Persist dask collections on cluster
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.
- Parameters
- collectionssequence or single dask object
Collections like dask.array or dataframe or dask.value objects
- optimize_graphbool
Whether or not to optimize the underlying graphs
- workersstring or iterable of strings
A set of worker hostnames on which computations may be performed. Leave empty to default to all workers (common case)
- allow_other_workersbool (defaults to False)
Used with workers. Indicates whether or not the computations may be performed on workers that are not in the workers set(s).
- retriesint (default to 0)
Number of allowed automatic retries if computing a result fails
- priorityNumber
Optional prioritization of task. Zero is default. Higher priorities take precedence
- fifo_timeouttimedelta str (defaults to ’60s’)
Allowed amount of time between calls to consider the same priority
- resourcesdict (defaults to {})
Defines the resources each instance of this mapped task requires on the worker; e.g.
{'GPU': 2}
. See worker resources for details on defining resources.- actorsbool or dict (default None)
Whether these tasks should exist on the worker as stateful actors. Specified on a global (True/False) or per-task (
{'x': True, 'y': False}
) basis. See Actors for additional details.- **kwargs
Options to pass to the graph optimize calls
- Returns
- List of collections, or single collection, depending on type of input.
See also
Examples
>>> xx = client.persist(x) >>> xx, yy = client.persist([x, y])
- processing(workers=None)[source]¶
The tasks currently running on each worker
- Parameters
- workerslist (optional)
A list of worker addresses, defaults to all
See also
Examples
>>> x, y, z = c.map(inc, [1, 2, 3]) >>> c.processing() {'192.168.1.141:46784': ['inc-1c8dd6be1c21646c71f76c16d09304ea', 'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b', 'inc-1e297fc27658d7b67b3a758f16bcf47a']}
- profile(key=None, start=None, stop=None, workers=None, merge_workers=True, plot=False, filename=None, server=False, scheduler=False)[source]¶
Collect statistical profiling information about recent work
- Parameters
- keystr
Key prefix to select, this is typically a function name like ‘inc’ Leave as None to collect all data
- starttime
- stoptime
- workerslist
List of workers to restrict profile information
- serverbool
If true, return the profile of the worker’s administrative thread rather than the worker threads. This is useful when profiling Dask itself, rather than user code.
- schedulerbool
If true, return the profile information from the scheduler’s administrative thread rather than the workers. This is useful when profiling Dask’s scheduling itself.
- plotboolean or string
Whether or not to return a plot object
- filenamestr
Filename to save the plot
Examples
>>> client.profile() # call on collections >>> client.profile(filename='dask-profile.html') # save to html file
- publish_dataset(*args, **kwargs)[source]¶
Publish named datasets to scheduler
This stores a named reference to a dask collection or list of futures on the scheduler. These references are available to other Clients which can download the collection or futures with
get_dataset
.Datasets are not immediately computed. You may wish to call
Client.persist
prior to publishing a dataset.- Parameters
- argslist of objects to publish as name
- kwargsdict
named collections to publish on the scheduler
- Returns
- None
Examples
Publishing client:
>>> df = dd.read_csv('s3://...') >>> df = c.persist(df) >>> c.publish_dataset(my_dataset=df)
Alternative invocation >>> c.publish_dataset(df, name=’my_dataset’)
Receiving client:
>>> c.list_datasets() ['my_dataset'] >>> df2 = c.get_dataset('my_dataset')
- rebalance(futures=None, workers=None, **kwargs)[source]¶
Rebalance data within network
Move data between workers to roughly balance memory burden. This either affects a subset of the keys/workers or the entire network, depending on keyword arguments.
For details on the algorithm and configuration options, refer to the matching scheduler-side method
rebalance()
.Warning
This operation is generally not well tested against normal operation of the scheduler. It is not recommended to use it while waiting on computations.
- Parameters
- futureslist, optional
A list of futures to balance, defaults all data
- workerslist, optional
A list of workers on which to balance, defaults to all workers
- **kwargsdict
Optional keyword arguments for the function
- register_plugin(plugin: distributed.diagnostics.plugin.NannyPlugin | distributed.diagnostics.plugin.SchedulerPlugin | distributed.diagnostics.plugin.WorkerPlugin, name: str | None = None, idempotent: bool | None = None)[source]¶
Register a plugin.
See https://distributed.readthedocs.io/en/latest/plugins.html
- Parameters
- plugin
A nanny, scheduler, or worker plugin to register.
- name
Name for the plugin; if None, a name is taken from the plugin instance or automatically generated if not present.
- idempotent
Do not re-register if a plugin of the given name already exists. If None,
plugin.idempotent
is taken if defined, False otherwise.
- register_scheduler_plugin(plugin: distributed.diagnostics.plugin.SchedulerPlugin, name: str | None = None, idempotent: bool | None = None)[source]¶
Register a scheduler plugin.
Deprecated since version 2023.9.2: Use
Client.register_plugin()
instead.See https://distributed.readthedocs.io/en/latest/plugins.html#scheduler-plugins
- Parameters
- pluginSchedulerPlugin
SchedulerPlugin instance to pass to the scheduler.
- namestr
Name for the plugin; if None, a name is taken from the plugin instance or automatically generated if not present.
- idempotentbool
Do not re-register if a plugin of the given name already exists.
- register_worker_callbacks(setup=None)[source]¶
Registers a setup callback function for all current and future 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 called in the order they were added.
If the function takes an input argument named
dask_worker
then that variable will be populated with the worker itself.- Parameters
- setupcallable(dask_worker: Worker) -> None
Function to register and run on all workers
- register_worker_plugin(plugin: distributed.diagnostics.plugin.NannyPlugin | distributed.diagnostics.plugin.WorkerPlugin, name: str | None = None, nanny: bool | None = None)[source]¶
Registers a lifecycle worker plugin for all current and future workers.
Deprecated since version 2023.9.2: Use
Client.register_plugin()
instead.This registers a new object to handle setup, task state transitions and teardown for workers in this cluster. The plugin will instantiate itself on all currently connected workers. It will also be run on any worker that connects in the future.
The plugin may include methods
setup
,teardown
,transition
, andrelease_key
. See thedask.distributed.WorkerPlugin
class or the examples below for the interface and docstrings. It must be serializable with the pickle or cloudpickle modules.If the plugin has a
name
attribute, or if thename=
keyword is used then that will control idempotency. If a plugin with that name has already been registered, then it will be removed and replaced by the new one.For alternatives to plugins, you may also wish to look into preload scripts.
- Parameters
- pluginWorkerPlugin or NannyPlugin
WorkerPlugin or NannyPlugin instance to register.
- namestr, optional
A name for the plugin. Registering a plugin with the same name will have no effect. If plugin has no name attribute a random name is used.
- nannybool, optional
Whether to register the plugin with workers or nannies.
See also
distributed.WorkerPlugin
unregister_worker_plugin
Examples
>>> class MyPlugin(WorkerPlugin): ... def __init__(self, *args, **kwargs): ... pass # the constructor is up to you ... def setup(self, worker: dask.distributed.Worker): ... pass ... def teardown(self, worker: dask.distributed.Worker): ... pass ... def transition(self, key: str, start: str, finish: str, ... **kwargs): ... pass ... def release_key(self, key: str, state: str, cause: str | None, reason: None, report: bool): ... pass
>>> plugin = MyPlugin(1, 2, 3) >>> client.register_plugin(plugin)
You can get access to the plugin with the
get_worker
function>>> client.register_plugin(other_plugin, name='my-plugin') >>> def f(): ... worker = get_worker() ... plugin = worker.plugins['my-plugin'] ... return plugin.my_state
>>> future = client.run(f)
- replicate(futures, n=None, workers=None, branching_factor=2, **kwargs)[source]¶
Set replication of futures 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.
Note
This method is incompatible with the Active Memory Manager’s ReduceReplicas policy. If you wish to use it, you must first disable the policy or disable the AMM entirely.
- Parameters
- futureslist of futures
Futures we wish to replicate
- nint, optional
Number of processes on the cluster on which to replicate the data. Defaults to all.
- workerslist of worker addresses
Workers on which we want to restrict the replication. Defaults to all.
- branching_factorint, optional
The number of workers that can copy data in each generation
- **kwargsdict
Optional keyword arguments for the remote function
See also
Examples
>>> x = c.submit(func, *args) >>> c.replicate([x]) # send to all workers >>> c.replicate([x], n=3) # send to three workers >>> c.replicate([x], workers=['alice', 'bob']) # send to specific >>> c.replicate([x], n=1, workers=['alice', 'bob']) # send to one of specific workers >>> c.replicate([x], n=1) # reduce replications
- restart(timeout: typing.Union[str, int, float, typing.Literal[<no_default>]] = _NoDefault.no_default, wait_for_workers: bool = True)[source]¶
Restart all workers. Reset local state. Optionally wait for workers to return.
Workers without nannies are shut down, hoping an external deployment system will restart them. Therefore, if not using nannies and your deployment system does not automatically restart workers,
restart
will just shut down all workers, then time out!After
restart
, all connected workers are new, regardless of whetherTimeoutError
was raised. Any workers that failed to shut down in time are removed, and may or may not shut down on their own in the future.- Parameters
- timeout:
How long to wait for workers to shut down and come back, if
wait_for_workers
is True, otherwise just how long to wait for workers to shut down. Raisesasyncio.TimeoutError
if this is exceeded.- wait_for_workers:
Whether to wait for all workers to reconnect, or just for them to shut down (default True). Use
restart(wait_for_workers=False)
combined withClient.wait_for_workers()
for granular control over how many workers to wait for.
See also
- restart_workers(workers: list[str], timeout: typing.Union[str, int, float, typing.Literal[<no_default>]] = _NoDefault.no_default, raise_for_error: bool = True)[source]¶
Restart a specified set of workers
Note
Only workers being monitored by a
distributed.Nanny
can be restarted. SeeNanny.restart
for more details.- Parameters
- workerslist[str]
Workers to restart. This can be a list of worker addresses, names, or a both.
- timeoutint | float | None
Number of seconds to wait
- raise_for_error: bool (default True)
Whether to raise a
TimeoutError
if restarting worker(s) doesn’t finish withintimeout
, or another exception caused from restarting worker(s).
- Returns
- dict[str, “OK” | “removed” | “timed out”]
Mapping of worker and restart status, the keys will match the original values passed in via
workers
.
See also
Notes
This method differs from
Client.restart()
in that this method simply restarts the specified set of workers, whileClient.restart
will restart all workers and also reset local state on the cluster (e.g. all keys are released).Additionally, this method does not gracefully handle tasks that are being executed when a worker is restarted. These tasks may fail or have their suspicious count incremented.
Examples
You can get information about active workers using the following:
>>> workers = client.scheduler_info()['workers']
From that list you may want to select some workers to restart
>>> client.restart_workers(workers=['tcp://address:port', ...])
- retire_workers(workers: list[str] | None = None, close_workers: bool = True, **kwargs)[source]¶
Retire certain workers on the scheduler
See
distributed.Scheduler.retire_workers()
for the full docstring.- Parameters
- workers
- close_workers
- **kwargsdict
Optional keyword arguments for the remote function
See also
dask.distributed.Scheduler.retire_workers
Examples
You can get information about active workers using the following:
>>> workers = client.scheduler_info()['workers']
From that list you may want to select some workers to close
>>> client.retire_workers(workers=['tcp://address:port', ...])
- retry(futures, asynchronous=None)[source]¶
Retry failed futures
- Parameters
- futureslist of Futures
The list of Futures
- asynchronous: bool
If True the client is in asynchronous mode
- run(function, *args, workers: list[str] | None = None, wait: bool = True, nanny: bool = False, on_error: Literal['raise', 'return', 'ignore'] = 'raise', **kwargs)[source]¶
Run a function on all workers outside of task scheduling system
This calls a function on all currently known workers immediately, blocks until those results come back, and returns the results asynchronously as a dictionary keyed by worker address. This method is generally used for side effects such as collecting diagnostic information or installing libraries.
If your function takes an input argument named
dask_worker
then that variable will be populated with the worker itself.- Parameters
- functioncallable
The function to run
- *argstuple
Optional arguments for the remote function
- **kwargsdict
Optional keyword arguments for the remote function
- workerslist
Workers on which to run the function. Defaults to all known workers.
- waitboolean (optional)
If the function is asynchronous whether or not to wait until that function finishes.
- nannybool, default False
Whether to run
function
on the nanny. By default, the function is run on the worker process. If specified, the addresses inworkers
should still be the worker addresses, not the nanny addresses.- on_error: “raise” | “return” | “ignore”
If the function raises an error on a worker:
- raise
(default) Re-raise the exception on the client. The output from other workers will be lost.
- return
Return the Exception object instead of the function output for the worker
- ignore
Ignore the exception and remove the worker from the result dict
Examples
>>> c.run(os.getpid) {'192.168.0.100:9000': 1234, '192.168.0.101:9000': 4321, '192.168.0.102:9000': 5555}
Restrict computation to particular workers with the
workers=
keyword argument.>>> c.run(os.getpid, workers=['192.168.0.100:9000', ... '192.168.0.101:9000']) {'192.168.0.100:9000': 1234, '192.168.0.101:9000': 4321}
>>> def get_status(dask_worker): ... return dask_worker.status
>>> c.run(get_status) {'192.168.0.100:9000': 'running', '192.168.0.101:9000': 'running}
Run asynchronous functions in the background:
>>> async def print_state(dask_worker): ... while True: ... print(dask_worker.status) ... await asyncio.sleep(1)
>>> c.run(print_state, wait=False)
- run_on_scheduler(function, *args, **kwargs)[source]¶
Run a function on the scheduler process
This is typically used for live debugging. The function should take a keyword argument
dask_scheduler=
, which will be given the scheduler object itself.- Parameters
- functioncallable
The function to run on the scheduler process
- *argstuple
Optional arguments for the function
- **kwargsdict
Optional keyword arguments for the function
See also
Client.run
Run a function on all workers
Examples
>>> def get_number_of_tasks(dask_scheduler=None): ... return len(dask_scheduler.tasks)
>>> client.run_on_scheduler(get_number_of_tasks) 100
Run asynchronous functions in the background:
>>> async def print_state(dask_scheduler): ... while True: ... print(dask_scheduler.status) ... await asyncio.sleep(1)
>>> c.run(print_state, wait=False)
- scatter(data, workers=None, broadcast=False, direct=None, hash=True, timeout=_NoDefault.no_default, asynchronous=None)[source]¶
Scatter data into distributed memory
This moves data from the local client process into the workers of the distributed scheduler. Note that it is often better to submit jobs to your workers to have them load the data rather than loading data locally and then scattering it out to them.
- Parameters
- datalist, dict, or object
Data to scatter out to workers. Output type matches input type.
- workerslist of tuples (optional)
Optionally constrain locations of data. Specify workers as hostname/port pairs, e.g.
('127.0.0.1', 8787)
.- broadcastbool (defaults to False)
Whether to send each data element to all workers. By default we round-robin based on number of cores.
Note
Setting this flag to True is incompatible with the Active Memory Manager’s ReduceReplicas policy. If you wish to use it, you must first disable the policy or disable the AMM entirely.
- directbool (defaults to automatically check)
Whether or not to connect directly to the workers, or to ask the scheduler to serve as intermediary. This can also be set when creating the Client.
- hashbool (optional)
Whether or not to hash data to determine key. If False then this uses a random key
- timeoutnumber, optional
Time in seconds after which to raise a
dask.distributed.TimeoutError
- asynchronous: bool
If True the client is in asynchronous mode
- Returns
- List, dict, iterator, or queue of futures matching the type of input.
See also
Client.gather
Gather data back to local process
Notes
Scattering a dictionary uses
dict
keys to createFuture
keys. The current implementation of a task graph resolution searches for occurrences ofkey
and replaces it with a correspondingFuture
result. That can lead to unwanted substitution of strings passed as arguments to a task if these strings match somekey
that already exists on a cluster. To avoid these situations it is required to use unique values if akey
is set manually. See https://github.com/dask/dask/issues/9969 to track progress on resolving this issue.Examples
>>> c = Client('127.0.0.1:8787') >>> c.scatter(1) <Future: status: finished, key: c0a8a20f903a4915b94db8de3ea63195>
>>> c.scatter([1, 2, 3]) [<Future: status: finished, key: c0a8a20f903a4915b94db8de3ea63195>, <Future: status: finished, key: 58e78e1b34eb49a68c65b54815d1b158>, <Future: status: finished, key: d3395e15f605bc35ab1bac6341a285e2>]
>>> c.scatter({'x': 1, 'y': 2, 'z': 3}) {'x': <Future: status: finished, key: x>, 'y': <Future: status: finished, key: y>, 'z': <Future: status: finished, key: z>}
Constrain location of data to subset of workers
>>> c.scatter([1, 2, 3], workers=[('hostname', 8788)])
Broadcast data to all workers
>>> [future] = c.scatter([element], broadcast=True)
Send scattered data to parallelized function using client futures interface
>>> data = c.scatter(data, broadcast=True) >>> res = [c.submit(func, data, i) for i in range(100)]
- scheduler_info(**kwargs)[source]¶
Basic information about the workers in the cluster
- Parameters
- **kwargsdict
Optional keyword arguments for the remote function
Examples
>>> c.scheduler_info() {'id': '2de2b6da-69ee-11e6-ab6a-e82aea155996', 'services': {}, 'type': 'Scheduler', 'workers': {'127.0.0.1:40575': {'active': 0, 'last-seen': 1472038237.4845693, 'name': '127.0.0.1:40575', 'services': {}, 'stored': 0, 'time-delay': 0.0061032772064208984}}}
- set_metadata(key, value)[source]¶
Set arbitrary metadata in the scheduler
This allows you to store small amounts of data on the central scheduler process for administrative purposes. Data should be msgpack serializable (ints, strings, lists, dicts)
If the key corresponds to a task then that key will be cleaned up when the task is forgotten by the scheduler.
If the key is a list then it will be assumed that you want to index into a nested dictionary structure using those keys. For example if you call the following:
>>> client.set_metadata(['a', 'b', 'c'], 123)
Then this is the same as setting
>>> scheduler.task_metadata['a']['b']['c'] = 123
The lower level dictionaries will be created on demand.
See also
Examples
>>> client.set_metadata('x', 123) >>> client.get_metadata('x') 123
>>> client.set_metadata(['x', 'y'], 123) >>> client.get_metadata('x') {'y': 123}
>>> client.set_metadata(['x', 'w', 'z'], 456) >>> client.get_metadata('x') {'y': 123, 'w': {'z': 456}}
>>> client.get_metadata(['x', 'w']) {'z': 456}
- shutdown()[source]¶
Shut down the connected scheduler and workers
Note, this may disrupt other clients that may be using the same scheduler and workers.
See also
Client.close
close only this client
- story(*keys_or_stimuli, on_error='raise')[source]¶
Returns a cluster-wide story for the given keys or stimulus_id’s
- submit(func, *args, key=None, workers=None, resources=None, retries=None, priority=0, fifo_timeout='100 ms', allow_other_workers=False, actor=False, actors=False, pure=True, **kwargs)[source]¶
Submit a function application to the scheduler
- Parameters
- funccallable
Callable to be scheduled as
func(*args **kwargs)
. Iffunc
returns a coroutine, it will be run on the main event loop of a worker. Otherwisefunc
will be run in a worker’s task executor pool (seeWorker.executors
for more information.)- *argstuple
Optional positional arguments
- keystr
Unique identifier for the task. Defaults to function-name and hash
- workersstring or iterable of strings
A set of worker addresses or hostnames on which computations may be performed. Leave empty to default to all workers (common case)
- resourcesdict (defaults to {})
Defines the
resources
each instance of this mapped task requires on the worker; e.g.{'GPU': 2}
. See worker resources for details on defining resources.- retriesint (default to 0)
Number of allowed automatic retries if the task fails
- priorityNumber
Optional prioritization of task. Zero is default. Higher priorities take precedence
- fifo_timeoutstr timedelta (default ‘100ms’)
Allowed amount of time between calls to consider the same priority
- allow_other_workersbool (defaults to False)
Used with
workers
. Indicates whether or not the computations may be performed on workers that are not in the workers set(s).- actorbool (default False)
Whether this task should exist on the worker as a stateful actor. See Actors for additional details.
- actorsbool (default False)
Alias for actor
- purebool (defaults to True)
Whether or not the function is pure. Set
pure=False
for impure functions likenp.random.random
. Note that if bothactor
andpure
kwargs are set to True, then the value ofpure
will be reverted to False, since an actor is stateful. See Pure Functions by Default for more details.- **kwargs
- Returns
- Future
If running in asynchronous mode, returns the future. Otherwise returns the concrete value
- Raises
- TypeError
If ‘func’ is not callable, a TypeError is raised
- ValueError
If ‘allow_other_workers’is True and ‘workers’ is None, a ValueError is raised
See also
Client.map
Submit on many arguments at once
Notes
The current implementation of a task graph resolution searches for occurrences of
key
and replaces it with a correspondingFuture
result. That can lead to unwanted substitution of strings passed as arguments to a task if these strings match somekey
that already exists on a cluster. To avoid these situations it is required to use unique values if akey
is set manually. See https://github.com/dask/dask/issues/9969 to track progress on resolving this issue.Examples
>>> c = client.submit(add, a, b)
- subscribe_topic(topic, handler)[source]¶
Subscribe to a topic and execute a handler for every received event
- Parameters
- topic: str
The topic name
- handler: callable or coroutine function
A handler called for every received event. The handler must accept a single argument event which is a tuple (timestamp, msg) where timestamp refers to the clock on the scheduler.
See also
dask.distributed.Client.unsubscribe_topic
dask.distributed.Client.get_events
dask.distributed.Client.log_event
Examples
>>> import logging >>> logger = logging.getLogger("myLogger") # Log config not shown >>> client.subscribe_topic("topic-name", lambda: logger.info)
- unforward_logging(logger_name=None)[source]¶
Stop forwarding the given logger (default root) from worker tasks to the client process.
- unpublish_dataset(name, **kwargs)[source]¶
Remove named datasets from scheduler
- Parameters
- namestr
The name of the dataset to unpublish
See also
Examples
>>> c.list_datasets() ['my_dataset'] >>> c.unpublish_dataset('my_dataset') >>> c.list_datasets() []
- unregister_scheduler_plugin(name)[source]¶
Unregisters a scheduler plugin
See https://distributed.readthedocs.io/en/latest/plugins.html#scheduler-plugins
- Parameters
- namestr
Name of the plugin to unregister. See the
Client.register_scheduler_plugin()
docstring for more information.
See also
Examples
>>> class MyPlugin(SchedulerPlugin): ... def __init__(self, *args, **kwargs): ... pass # the constructor is up to you ... async def start(self, scheduler: Scheduler) -> None: ... pass ... async def before_close(self) -> None: ... pass ... async def close(self) -> None: ... pass ... def restart(self, scheduler: Scheduler) -> None: ... pass
>>> plugin = MyPlugin(1, 2, 3) >>> client.register_plugin(plugin, name='foo') >>> client.unregister_scheduler_plugin(name='foo')
- unregister_worker_plugin(name, nanny=None)[source]¶
Unregisters a lifecycle worker plugin
This unregisters an existing worker plugin. As part of the unregistration process the plugin’s
teardown
method will be called.- Parameters
- namestr
Name of the plugin to unregister. See the
Client.register_plugin()
docstring for more information.
See also
Examples
>>> class MyPlugin(WorkerPlugin): ... def __init__(self, *args, **kwargs): ... pass # the constructor is up to you ... def setup(self, worker: dask.distributed.Worker): ... pass ... def teardown(self, worker: dask.distributed.Worker): ... pass ... def transition(self, key: str, start: str, finish: str, **kwargs): ... pass ... def release_key(self, key: str, state: str, cause: str | None, reason: None, report: bool): ... pass
>>> plugin = MyPlugin(1, 2, 3) >>> client.register_plugin(plugin, name='foo') >>> client.unregister_worker_plugin(name='foo')
- unsubscribe_topic(topic)[source]¶
Unsubscribe from a topic and remove event handler
See also
dask.distributed.Client.subscribe_topic
dask.distributed.Client.get_events
dask.distributed.Client.log_event
- upload_file(filename, load: bool = True)[source]¶
Upload local package to scheduler and workers
This sends a local file up to the scheduler and all worker nodes. This file is placed into the working directory of each node, see config option
temporary-directory
(defaults totempfile.gettempdir()
).This directory will be added to the Python’s system path so any
.py
,.egg
or.zip
files will be importable.- Parameters
- filenamestring
Filename of
.py
,.egg
, or.zip
file to send to workers- loadbool, optional
Whether or not to import the module as part of the upload process. Defaults to
True
.
Examples
>>> client.upload_file('mylibrary.egg') >>> from mylibrary import myfunc >>> L = client.map(myfunc, seq) >>> >>> # Where did that file go? Use `dask_worker.local_directory`. >>> def where_is_mylibrary(dask_worker): >>> path = pathlib.Path(dask_worker.local_directory) / 'mylibrary.egg' >>> assert path.exists() >>> return str(path) >>> >>> client.run(where_is_mylibrary)
- wait_for_workers(n_workers: int, timeout: float | None = None) None [source]¶
Blocking call to wait for n workers before continuing
- Parameters
- n_workersint
The number of workers
- timeoutnumber, optional
Time in seconds after which to raise a
dask.distributed.TimeoutError
- who_has(futures=None, **kwargs)[source]¶
The workers storing each future’s data
- Parameters
- futureslist (optional)
A list of futures, defaults to all data
- **kwargsdict
Optional keyword arguments for the remote function
See also
Examples
>>> x, y, z = c.map(inc, [1, 2, 3]) >>> wait([x, y, z]) >>> c.who_has() {'inc-1c8dd6be1c21646c71f76c16d09304ea': ['192.168.1.141:46784'], 'inc-1e297fc27658d7b67b3a758f16bcf47a': ['192.168.1.141:46784'], 'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b': ['192.168.1.141:46784']}
>>> c.who_has([x, y]) {'inc-1c8dd6be1c21646c71f76c16d09304ea': ['192.168.1.141:46784'], 'inc-1e297fc27658d7b67b3a758f16bcf47a': ['192.168.1.141:46784']}
- write_scheduler_file(scheduler_file)[source]¶
Write the scheduler information to a json file.
This facilitates easy sharing of scheduler information using a file system. The scheduler file can be used to instantiate a second Client using the same scheduler.
- Parameters
- scheduler_filestr
Path to a write the scheduler file.
Examples
>>> client = Client() >>> client.write_scheduler_file('scheduler.json') # connect to previous client's scheduler >>> client2 = Client(scheduler_file='scheduler.json')
- class distributed.Future(key, client=None, state=None, _id=None)[source]¶
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 happens in the larger cluster.
Note
Users should not instantiate futures manually. This can lead to state corruption and deadlocking clusters.
- Parameters
- key: str, or tuple
Key of remote data to which this future refers
- client: Client
Client that should own this future. 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
See also
Client
Creates futures
Examples
Futures typically emerge from Client computations
>>> my_future = client.submit(add, 1, 2)
We can track the progress and results of a future
>>> my_future <Future: status: finished, key: add-8f6e709446674bad78ea8aeecfee188e>
We can get the result or the exception and traceback from the future
>>> my_future.result()
- add_done_callback(fn)[source]¶
Call callback on future when future has finished
The callback
fn
should take the future as its only argument. This will be called regardless of if the future completes successfully, errs, or is cancelledThe callback is executed in a separate thread.
- Parameters
- fncallable
The method or function to be called
- cancelled()[source]¶
Returns True if the future has been cancelled
- Returns
- bool
True if the future was ‘cancelled’, otherwise False
- done()[source]¶
Returns whether or not the computation completed.
- Returns
- bool
True if the computation is complete, otherwise False
- exception(timeout=None, **kwargs)[source]¶
Return the exception of a failed task
- Parameters
- timeoutnumber, optional
Time in seconds after which to raise a
dask.distributed.TimeoutError
- **kwargsdict
Optional keyword arguments for the function
- Returns
- Exception
The exception that was raised If timeout seconds are elapsed before returning, a
dask.distributed.TimeoutError
is raised.
See also
- property executor¶
Returns the executor, which is the client.
- Returns
- Client
The executor
- release()[source]¶
Notes
This method can be called from different threads (see e.g. Client.get() or Future.__del__())
- result(timeout=None)[source]¶
Wait until computation completes, gather result to local process.
- Parameters
- timeoutnumber, optional
Time in seconds after which to raise a
dask.distributed.TimeoutError
- Returns
- result
The result of the computation. Or a coroutine if the client is asynchronous.
- Raises
- dask.distributed.TimeoutError
If timeout seconds are elapsed before returning, a
dask.distributed.TimeoutError
is raised.
- property status¶
Returns the status
- Returns
- str
The status
- traceback(timeout=None, **kwargs)[source]¶
Return the traceback of a failed task
This returns a traceback object. You can inspect this object using the
traceback
module. Alternatively if you callfuture.result()
this traceback will accompany the raised exception.- Parameters
- timeoutnumber, optional
Time in seconds after which to raise a
dask.distributed.TimeoutError
If timeout seconds are elapsed before returning, adask.distributed.TimeoutError
is raised.
- Returns
- traceback
The traceback object. Or a coroutine if the client is asynchronous.
See also
Examples
>>> import traceback >>> tb = future.traceback() >>> traceback.format_tb(tb) [...]
- property type¶
Returns the type
- class distributed.Queue(name=None, client=None, maxsize=0)[source]¶
Distributed Queue
This allows multiple clients to share futures 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.
Warning
This object is experimental
- Parameters
- name: string (optional)
Name used by other clients and the scheduler to identify the queue. If not given, a random name will be generated.
- client: Client (optional)
Client used for communication with the scheduler. If not given, the default global client will be used.
- maxsize: int (optional)
Number of items allowed in the queue. If 0 (the default), the queue size is unbounded.
See also
Variable
shared variable between clients
Examples
>>> from dask.distributed import Client, Queue >>> client = Client() >>> queue = Queue('x') >>> future = client.submit(f, x) >>> queue.put(future)
- get(timeout=None, batch=False, **kwargs)[source]¶
Get data from the queue
- Parameters
- timeoutnumber or string or timedelta, optional
Time in seconds to wait before timing out. Instead of number of seconds, it is also possible to specify a timedelta in string format, e.g. “200ms”.
- batchboolean, int (optional)
If True then return all elements currently waiting in the queue. If an integer than return that many elements from the queue If False (default) then return one item at a time
- class distributed.Variable(name=None, client=None)[source]¶
Distributed Global Variable
This allows multiple clients to share futures 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.- Parameters
- name: string (optional)
Name used by other clients and the scheduler to identify the variable. If not given, a random name will be generated.
- client: Client (optional)
Client used for communication with the scheduler. If not given, the default global client will be used.
See also
Queue
shared multi-producer/multi-consumer queue between clients
Examples
>>> from dask.distributed import Client, Variable >>> client = Client() >>> x = Variable('x') >>> x.set(123) # docttest: +SKIP >>> x.get() # docttest: +SKIP 123 >>> future = client.submit(f, x) >>> x.set(future)
- delete()[source]¶
Delete this variable
Caution, this affects all clients currently pointing to this variable.
- class distributed.Lock(name=None, client=<object object>, scheduler_rpc=None, loop=None)[source]¶
Distributed Centralized Lock
Warning
This is using the
distributed.Semaphore
as a backend, which is susceptible to lease overbooking. For the Lock this means that if a lease is timing out, two or more instances could acquire the lock at the same time. To disable lease timeouts, setdistributed.scheduler.locks.lease-timeout
to inf, e.g.with dask.config.set({"distributed.scheduler.locks.lease-timeout": "inf"}): lock = Lock("x") ...
Note, that without lease timeouts, the Lock may deadlock in case of cluster downscaling or worker failures.
- Parameters
- name: string (optional)
Name of the lock to acquire. Choosing the same name allows two disconnected processes to coordinate a lock. If not given, a random name will be generated.
- client: Client (optional)
Client to use for communication with the scheduler. If not given, the default global client will be used.
Examples
>>> lock = Lock('x') >>> lock.acquire(timeout=1) >>> # do things with protected resource >>> lock.release()
- acquire(blocking=True, timeout=None)[source]¶
Acquire the lock
- Parameters
- blockingbool, optional
If false, don’t wait on the lock in the scheduler at all.
- timeoutstring or number or timedelta, optional
Seconds to wait on the lock in the scheduler. This does not include local coroutine time, network transfer time, etc.. It is forbidden to specify a timeout when blocking is false. Instead of number of seconds, it is also possible to specify a timedelta in string format, e.g. “200ms”.
- Returns
- True or False whether or not it successfully acquired the lock
Examples
>>> lock = Lock('x') >>> lock.acquire(timeout="1s")
- class distributed.Event(name=None, client=None)[source]¶
Distributed Centralized Event equivalent to asyncio.Event
An event stores a single flag, which is set to false on start. The flag can be set to true (using the set() call) or back to false (with the clear() call). Every call to wait() blocks until the event flag is set to true.
- Parameters
- name: string (optional)
Name of the event. Choosing the same name allows two disconnected processes to coordinate an event. If not given, a random name will be generated.
- client: Client (optional)
Client to use for communication with the scheduler. If not given, the default global client will be used.
Examples
>>> event_1 = Event('a') >>> event_1.wait(timeout=1) >>> # in another process >>> event_2 = Event('a') >>> event_2.set() >>> # now event_1 will stop waiting
- wait(timeout=None)[source]¶
Wait until the event is set.
- Parameters
- timeoutnumber or string or timedelta, optional
Seconds to wait on the event in the scheduler. This does not include local coroutine time, network transfer time, etc.. Instead of number of seconds, it is also possible to specify a timedelta in string format, e.g. “200ms”.
- Returns
- True if the event was set of false, if a timeout happened
Examples
>>> event = Event('a') >>> event.wait(timeout="1s")
- class distributed.Semaphore(max_leases=1, name=None, scheduler_rpc=None, loop=None)[source]¶
This semaphore will track leases on the scheduler which can be acquired and released by an instance of this class. If the maximum amount of leases are already acquired, it is not possible to acquire more and the caller waits until another lease has been released.
The lifetime or leases are controlled using a timeout. This timeout is refreshed in regular intervals by the
Client
of this instance and provides protection from deadlocks or resource starvation in case of worker failure. The timeout can be controlled using the configuration optiondistributed.scheduler.locks.lease-timeout
and the interval in which the scheduler verifies the timeout is set using the optiondistributed.scheduler.locks.lease-validation-interval
.A noticeable difference to the Semaphore of the python standard library is that this implementation does not allow to release more often than it was acquired. If this happens, a warning is emitted but the internal state is not modified.
Warning
This implementation is susceptible to lease overbooking in case of lease timeouts. It is advised to monitor log information and adjust above configuration options to suitable values for the user application.
- Parameters
- max_leases: int (optional)
The maximum amount of leases that may be granted at the same time. This effectively sets an upper limit to the amount of parallel access to a specific resource. Defaults to 1.
- name: string (optional)
Name of the semaphore to acquire. Choosing the same name allows two disconnected processes to coordinate. If not given, a random name will be generated.
- register: bool
If True, register the semaphore with the scheduler. This needs to be done before any leases can be acquired. If not done during initialization, this can also be done by calling the register method of this class. When registering, this needs to be awaited.
- scheduler_rpc: ConnectionPool
The ConnectionPool to connect to the scheduler. If None is provided, it uses the worker or client pool. This parameter is mostly used for testing.
- loop: IOLoop
The event loop this instance is using. If None is provided, reuse the loop of the active worker or client.
Notes
If a client attempts to release the semaphore but doesn’t have a lease acquired, this will raise an exception.
dask executes functions by default assuming they are pure, when using semaphore acquire/releases inside such a function, it must be noted that there are in fact side-effects, thus, the function can no longer be considered pure. If this is not taken into account, this may lead to unexpected behavior.
Examples
>>> from distributed import Semaphore ... sem = Semaphore(max_leases=2, name='my_database') ... ... def access_resource(s, sem): ... # This automatically acquires a lease from the semaphore (if available) which will be ... # released when leaving the context manager. ... with sem: ... pass ... ... futures = client.map(access_resource, range(10), sem=sem) ... client.gather(futures) ... # Once done, close the semaphore to clean up the state on scheduler side. ... sem.close()
- acquire(timeout=None)[source]¶
Acquire a semaphore.
If the internal counter is greater than zero, decrement it by one and return True immediately. If it is zero, wait until a release() is called and return True.
- Parameters
- timeoutnumber or string or timedelta, optional
Seconds to wait on acquiring the semaphore. This does not include local coroutine time, network transfer time, etc.. Instead of number of seconds, it is also possible to specify a timedelta in string format, e.g. “200ms”.
- release()[source]¶
Release the semaphore.
- Returns
- bool
This value indicates whether a lease was released immediately or not. Note that a user should not retry this operation. Under certain circumstances (e.g. scheduler overload) the lease may not be released immediately, but it will always be automatically released after a specific interval configured using “distributed.scheduler.locks.lease-validation-interval” and “distributed.scheduler.locks.lease-timeout”.