Futures

Dask supports a real-time task framework that extends Python’s concurrent.futures interface. 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

Client.submit(self, func, \*args[, key, …])

Submit a function application to the scheduler

Client.map(self, func, \*iterables[, key, …])

Map a function on a sequence of arguments

Future.result(self[, timeout])

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

Future.result(self[, timeout])

Wait until computation completes, gather result to local process.

Client.gather(self, futures[, errors, …])

Gather futures from distributed memory

Client.scatter(self, data[, workers, …])

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

Future.cancel(self, \*\*kwargs)

Cancel request to run this future

Future.exception(self[, timeout])

Return the exception of a failed task

Future.traceback(self[, timeout])

Return the traceback of a failed task

Client.cancel(self, futures[, asynchronous, …])

Cancel running futures

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

as_completed([futures, loop, with_results, …])

Return futures in the order in which they complete

wait(fs[, timeout, return_when])

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

Fire and Forget

fire_and_forget(obj)

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 Tasks from Tasks

get_client([address, timeout, resolve_address])

Get a client while within a task.

rejoin()

Have this thread rejoin the ThreadPoolExecutor

secede()

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

Queue([name, client, maxsize])

Distributed Queue

Variable([name, client, maxsize])

Distributed Global Variable

Lock([name, client])

Distributed Centralized Lock

Event

Semaphore

Pub(name[, worker, client])

Publish data with Publish-Subscribe pattern

Sub(name[, worker, client])

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

Queue([name, client, maxsize])

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

Variable([name, client, maxsize])

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

Lock([name, client])

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

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

Semaphore

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

Pub(name[, worker, client])

Publish data with Publish-Subscribe pattern

Sub(name[, worker, client])

Subscribe to a Publish/Subscribe topic

Dask implements the Publish Subscribe pattern, providing an additional channel of communication between ongoing tasks.

class distributed.Pub(name, worker=None, client=None)

Publish data with Publish-Subscribe pattern

This allows clients and workers to directly communicate data between each other with a typical Publish-Subscribe pattern. This involves two components,

Pub objects, into which we put data:

>>> pub = Pub('my-topic')
>>> pub.put(123)

And Sub objects, from which we collect data:

>>> sub = Sub('my-topic')
>>> sub.get()
123

Many Pub and Sub objects can exist for the same topic. All data sent from any Pub will be sent to all Sub objects on that topic that are currently connected. Pub’s and Sub’s find each other using the scheduler, but they communicate directly with each other without coordination from the scheduler.

Pubs and Subs use the central scheduler to find each other, but not to mediate the communication. This means that there is very little additional latency or overhead, and they are appropriate for very frequent data transfers. For context, most data transfer first checks with the scheduler to find which workers should participate, and then does direct worker-to-worker transfers. This checking in with the scheduler provides some stability guarantees, but also adds in a few extra network hops. PubSub doesn’t do this, and so is faster, but also can easily drop messages if Pubs or Subs disappear without notice.

When using a Pub or Sub from a Client all communications will be routed through the scheduler. This can cause some performance degradation. Pubs and Subs only operate at top-speed when they are both on workers.

Parameters
name: object (msgpack serializable)

The name of the group of Pubs and Subs on which to participate.

worker: Worker (optional)

The worker to be used for publishing data. Defaults to the value of `get_worker()`. If given, client must be None.

client: Client (optional)

Client used for communication with the scheduler. Defaults to the value of get_client(). If given, worker must be None.

See also

Sub

Examples

>>> pub = Pub('my-topic')
>>> sub = Sub('my-topic')
>>> pub.put([1, 2, 3])
>>> sub.get()
[1, 2, 3]

You can also use sub within a for loop:

>>> for msg in sub:  
...     print(msg)

or an async for loop

>>> async for msg in sub:  
...     print(msg)

Similarly the .get method will return an awaitable if used by an async client or within the IOLoop thread of a worker

>>> await sub.get()  

You can see the set of connected worker subscribers by looking at the .subscribers attribute:

>>> pub.subscribers
{'tcp://...': {},
 'tcp://...': {}}
put(self, msg)

Publish a message to all subscribers of this topic

Actors

Note

This is an advanced feature and is rarely necessary in the common case.

Note

This is an experimental feature and is subject to change without notice.

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:

\[\min_{p\in\mathbb{R}^{1000}} \sum_{i=1}^{1000} (p_i - 1)^2\]

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.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

API

Client

Client([address, loop, timeout, …])

Connect to and submit computation to a Dask cluster

Client.cancel(self, futures[, asynchronous, …])

Cancel running futures

Client.compute(self, collections[, sync, …])

Compute dask collections on cluster

Client.gather(self, futures[, errors, …])

Gather futures from distributed memory

Client.get(self, dsk, keys[, restrictions, …])

Compute dask graph

Client.get_dataset(self, name, \*\*kwargs)

Get named dataset from the scheduler

Client.get_executor(self, \*\*kwargs)

Return a concurrent.futures Executor for submitting tasks on this Client

Client.has_what(self[, workers])

Which keys are held by which workers

Client.list_datasets(self, \*\*kwargs)

List named datasets available on the scheduler

Client.map(self, func, \*iterables[, key, …])

Map a function on a sequence of arguments

Client.ncores(self[, workers])

The number of threads/cores available on each worker node

Client.persist(self, collections[, …])

Persist dask collections on cluster

Client.profile(self[, key, start, stop, …])

Collect statistical profiling information about recent work

Client.publish_dataset(self, \*args, \*\*kwargs)

Publish named datasets to scheduler

Client.rebalance(self[, futures, workers])

Rebalance data within network

Client.replicate(self, futures[, n, …])

Set replication of futures within network

Client.restart(self, \*\*kwargs)

Restart the distributed network

Client.run(self, function, \*args, \*\*kwargs)

Run a function on all workers outside of task scheduling system

Client.run_on_scheduler(self, function, …)

Run a function on the scheduler process

Client.scatter(self, data[, workers, …])

Scatter data into distributed memory

Client.shutdown(self)

Shut down the connected scheduler and workers

Client.scheduler_info(self, \*\*kwargs)

Basic information about the workers in the cluster

Client.shutdown(self)

Shut down the connected scheduler and workers

Client.start_ipython_workers(self[, …])

Start IPython kernels on workers

Client.start_ipython_scheduler(self[, …])

Start IPython kernel on the scheduler

Client.submit(self, func, \*args[, key, …])

Submit a function application to the scheduler

Client.unpublish_dataset(self, name, \*\*kwargs)

Remove named datasets from scheduler

Client.upload_file(self, filename, \*\*kwargs)

Upload local package to workers

Client.who_has(self[, futures])

The workers storing each future’s data

Future

Future(key[, client, inform, state])

A remotely running computation

Future.add_done_callback(self, fn)

Call callback on future when callback has finished

Future.cancel(self, \*\*kwargs)

Cancel request to run this future

Future.cancelled(self)

Returns True if the future has been cancelled

Future.done(self)

Is the computation complete?

Future.exception(self[, timeout])

Return the exception of a failed task

Future.result(self[, timeout])

Wait until computation completes, gather result to local process.

Future.traceback(self[, timeout])

Return the traceback of a failed task

Functions

as_completed([futures, loop, with_results, …])

Return futures in the order in which they complete

fire_and_forget(obj)

Run tasks at least once, even if we release the futures

get_client([address, timeout, resolve_address])

Get a client while within a task.

secede()

Have this task secede from the worker’s thread pool

rejoin()

Have this thread rejoin the ThreadPoolExecutor

wait(fs[, timeout, return_when])

Wait until all/any futures are finished

distributed.as_completed(futures=None, loop=None, with_results=False, raise_errors=True)

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.

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)

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
obj: Future, 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=3, resolve_address=True)

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, default 3

Timeout (in seconds) for getting the Client

resolve_addressbool, default True

Whether to resolve address to its canonical form.

Returns
Client

See also

get_worker
worker_client
secede

Examples

>>> def f():
...     client = get_client()
...     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()

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

get_client
get_worker

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()

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')

Wait until all/any futures are finished

Parameters
fs: list of futures
timeout: number, optional

Time in seconds after which to raise a dask.distributed.TimeoutError

return_when: str, optional

One of ALL_COMPLETED or FIRST_COMPLETED

Returns
Named tuple of completed, not completed
class distributed.Client(address=None, loop=None, timeout='__no_default__', set_as_default=True, scheduler_file=None, security=None, asynchronous=False, name=None, heartbeat_interval=None, serializers=None, deserializers=None, extensions=[<class 'distributed.pubsub.PubSubClientExtension'>], direct_to_workers=None, **kwargs)

Connect to and submit computation to a Dask cluster

The Client connects users to a Dask cluster. It provides an asynchronous user interface around functions and futures. This class resembles executors in concurrent.futures but also allows Future objects within submit/map calls. When a Client is instantiated it takes over all dask.compute and dask.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 a LocalCluster 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 like LocalCluster()

timeout: int

Timeout duration for initial connection to the scheduler

set_as_default: bool (True)

Claim this scheduler 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

direct_to_workers: bool (optional)

Whether or not to connect directly to the workers, or to ask the scheduler to serve as intermediary.

heartbeat_interval: int

Time in milliseconds between heartbeats to scheduler

**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.deploy.local.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(processes=False, threads_per_worker=1)  
property asynchronous

Are we running in the event loop?

This is true if the user signaled that we might be when creating the client as in the following:

client = Client(asynchronous=True)

However, we override this expectation if we can definitively tell that we are running from a thread that is not the event loop. This is common when calling get_client() from within a worker task. Even though the client was originally created in asynchronous mode we may find ourselves in contexts when it is better to operate synchronously.

call_stack(self, futures=None, keys=None)

The actively running call stack of all relevant keys

You can specify data of interest either by providing futures or collections in the futures= keyword or a list of explicit keys in the keys= keyword. If neither are provided then all call stacks will be returned.

Parameters
futures: list (optional)

List of futures, defaults to all data

keys: list (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(self, futures, asynchronous=None, force=False)

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
futures: list of Futures
force: boolean (False)

Cancel this future even if other clients desire it

close(self, timeout='__no_default__')

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.

See also

Client.restart
compute(self, 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)

Compute dask collections on cluster

Parameters
collections: iterable of dask objects or single dask object

Collections like dask.array or dataframe or dask.value objects

sync: bool (optional)

Returns Futures if False (default) or concrete values if True

optimize_graph: bool

Whether or not to optimize the underlying graphs

workers: str, list, dict

Which workers can run which parts of the computation If a string a list then the output collections will run on the listed workers, but other sub-computations can run anywhere If a dict then keys should be (tuples of) collections and values should be addresses or lists.

allow_other_workers: bool, list

If True then all restrictions in workers= are considered loose If a list then only the keys for the listed collections are loose

retries: int (default to 0)

Number of allowed automatic retries if computing a result fails

priority: Number

Optional prioritization of task. Zero is default. Higher priorities take precedence

fifo_timeout: timedelta str (defaults to ’60s’)

Allowed amount of time between calls to consider the same priority

traverse: bool (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, set traverse=False to avoid doing this traversal.

resources: dict (defaults to {})

Defines the resources these tasks require on the worker. Can specify global resources ({'GPU': 2}), or per-task resources ({'x': {'GPU': 1}, 'y': {'SSD': 4}}), but not both. See worker resources for details on defining resources.

actors: bool 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()

Return global client if one exists, otherwise raise ValueError

gather(self, futures, errors='raise', direct=None, asynchronous=None)

Gather futures from distributed memory

Accepts a future, nested container of futures, iterator, or queue. The return type will match the input type.

Parameters
futures: Collection of futures

This can be a possibly nested collection of Future objects. Collections can be lists, sets, or dictionaries

errors: string

Either ‘raise’ or ‘skip’ if we should raise if a future has erred or skip its inclusion in the output collection

direct: boolean

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.

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(self, dsk, keys, restrictions=None, loose_restrictions=None, resources=None, sync=True, asynchronous=None, direct=None, retries=None, priority=0, fifo_timeout='60s', actors=None, **kwargs)

Compute dask graph

Parameters
dsk: dict
keys: object, or nested lists of objects
restrictions: dict (optional)

A mapping of {key: {set of worker hostnames}} that restricts where jobs can take place

retries: int (default to 0)

Number of allowed automatic retries if computing a result fails

priority: Number

Optional prioritization of task. Zero is default. Higher priorities take precedence

sync: bool (optional)

Returns Futures if False or concrete values if True (default).

direct: bool

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.

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(self, name, **kwargs)

Get named dataset from the scheduler

get_executor(self, **kwargs)

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
An Executor object that’s fully compatible with the concurrent.futures
API.
get_metadata(self, keys, default='__no_default__')

Get arbitrary metadata from scheduler

See set_metadata for the full docstring with examples

Parameters
keys: key or list

Key to access. If a list then gets within a nested collection

default: optional

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

classmethod get_restrictions(collections, workers, allow_other_workers)

Get restrictions from inputs to compute/persist

get_scheduler_logs(self, n=None)

Get logs from scheduler

Parameters
nint

Number of logs to retrive. Maxes out at 10000 by default, confiruable in config.yaml::log-length

Returns
Logs in reversed order (newest first)
get_task_stream(self, start=None, stop=None, count=None, plot=False, filename='task-stream.html')

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
start: Number 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’

stop: Number or string

When you want to stop recording

count: int

The number of desired records, ignored if both start and stop are specified

plot: boolean, str

If true then also return a Bokeh figure If plot == ‘save’ then save the figure to a file

filename: str (optional)

The filename to save to if you set plot='save'

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 or plot='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(self, check=False, packages=[])

Return version info for the scheduler, all workers and myself

Parameters
checkboolean, default False

raise ValueError if all required & optional packages do not match

packagesList[str]

Extra package names to check

Examples

>>> c.get_versions()  
>>> c.get_versions(packages=['sklearn', 'geopandas'])  
get_worker_logs(self, n=None, workers=None, nanny=False)

Get logs from workers

Parameters
nint

Number of logs to retrive. Maxes out at 10000 by default, confiruable in config.yaml::log-length

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(self, workers=None, **kwargs)

Which keys are held by which workers

This returns the keys of the data that are held in each worker’s memory.

Parameters
workers: list (optional)

A list of worker addresses, defaults to all

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']}
list_datasets(self, **kwargs)

List named datasets available on the scheduler

map(self, func, *iterables, key=None, workers=None, retries=None, resources=None, priority=0, allow_other_workers=False, fifo_timeout='100 ms', actor=False, actors=False, pure=None, **kwargs)

Map a function on a sequence of arguments

Arguments can be normal objects or Futures

Parameters
func: callable
iterables: Iterables

List-like objects to map over. They should have the same length.

key: str, list

Prefix for task names if string. Explicit names if list.

pure: bool (defaults to True)

Whether or not the function is pure. Set pure=False for impure functions like np.random.random.

workers: set, iterable of sets

A set of worker hostnames on which computations may be performed. Leave empty to default to all workers (common case)

allow_other_workers: bool (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).

retries: int (default to 0)

Number of allowed automatic retries if a task fails

priority: Number

Optional prioritization of task. Zero is default. Higher priorities take precedence

fifo_timeout: str timedelta (default ‘100ms’)

Allowed amount of time between calls to consider the same priority

resources: dict (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.

actor: bool (default False)

Whether these tasks should exist on the worker as stateful actors. See actors for additional details.

actors: bool (default False)

Alias for actor

**kwargs: dict

Extra keywords 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

Examples

>>> L = client.map(func, sequence)  
nbytes(self, keys=None, summary=True, **kwargs)

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
keys: list (optional)

A list of keys, defaults to all keys

summary: boolean, (optional)

Summarize keys into key types

See also

Client.who_has

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(self, workers=None, **kwargs)

The number of threads/cores available on each worker node

Parameters
workers: list (optional)

A list of workers that we care about specifically. Leave empty to receive information about all workers.

Examples

>>> c.threads()  
{'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(self, collection)

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.

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(self, workers=None, **kwargs)

The number of threads/cores available on each worker node

Parameters
workers: list (optional)

A list of workers that we care about specifically. Leave empty to receive information about all workers.

Examples

>>> c.threads()  
{'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(self, collections, optimize_graph=True, workers=None, allow_other_workers=None, resources=None, retries=None, priority=0, fifo_timeout='60s', actors=None, **kwargs)

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
collections: sequence or single dask object

Collections like dask.array or dataframe or dask.value objects

optimize_graph: bool

Whether or not to optimize the underlying graphs

workers: str, list, dict

Which workers can run which parts of the computation If a string a list then the output collections will run on the listed workers, but other sub-computations can run anywhere If a dict then keys should be (tuples of) collections and values should be addresses or lists.

allow_other_workers: bool, list

If True then all restrictions in workers= are considered loose If a list then only the keys for the listed collections are loose

retries: int (default to 0)

Number of allowed automatic retries if computing a result fails

priority: Number

Optional prioritization of task. Zero is default. Higher priorities take precedence

fifo_timeout: timedelta str (defaults to ’60s’)

Allowed amount of time between calls to consider the same priority

resources: dict (defaults to {})

Defines the resources these tasks require on the worker. Can specify global resources ({'GPU': 2}), or per-task resources ({'x': {'GPU': 1}, 'y': {'SSD': 4}}), but not both. See worker resources for details on defining resources.

actors: bool 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

Client.compute

Examples

>>> xx = client.persist(x)  
>>> xx, yy = client.persist([x, y])  
processing(self, workers=None)

The tasks currently running on each worker

Parameters
workers: list (optional)

A list of worker addresses, defaults to all

Examples

>>> x, y, z = c.map(inc, [1, 2, 3])  
>>> c.processing()  
{'192.168.1.141:46784': ['inc-1c8dd6be1c21646c71f76c16d09304ea',
                         'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b',
                         'inc-1e297fc27658d7b67b3a758f16bcf47a']}
profile(self, key=None, start=None, stop=None, workers=None, merge_workers=True, plot=False, filename=None, server=False, scheduler=False)

Collect statistical profiling information about recent work

Parameters
key: str

Key prefix to select, this is typically a function name like ‘inc’ Leave as None to collect all data

start: time
stop: time
workers: list

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.

scheduler: bool

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.

plot: boolean or string

Whether or not to return a plot object

filename: str

Filename to save the plot

Examples

>>> client.profile()  # call on collections
>>> client.profile(filename='dask-profile.html')  # save to html file
publish_dataset(self, *args, **kwargs)

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
nameoptional name of the dataset to publish
kwargs: dict

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(self, futures=None, workers=None, **kwargs)

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.

This operation is generally not well tested against normal operation of the scheduler. It it not recommended to use it while waiting on computations.

Parameters
futures: list, optional

A list of futures to balance, defaults all data

workers: list, optional

A list of workers on which to balance, defaults to all workers

register_worker_callbacks(self, setup=None)

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(self, plugin=None, name=None)

Registers a lifecycle worker plugin for all current and future workers.

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, and transition. See the dask.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 the name= keyword is used then that will control idempotency. A a plugin with that name has already registered then any future plugins will not run.

For alternatives to plugins, you may also wish to look into preload scripts.

Parameters
plugin: WorkerPlugin

The plugin object to pass to the workers

name: str, optional

A name for the plugin. Registering a plugin with the same name will have no effect.

See also

distributed.WorkerPlugin

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
>>> plugin = MyPlugin(1, 2, 3)
>>> client.register_worker_plugin(plugin)

You can get access to the plugin with the get_worker function

>>> client.register_worker_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(self, futures, n=None, workers=None, branching_factor=2, **kwargs)

Set replication of futures within network

Copy data onto many workers. This helps to broadcast frequently accessed data and it helps to 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.

Parameters
futures: list of futures

Futures we wish to replicate

n: int, optional

Number of processes on the cluster on which to replicate the data. Defaults to all.

workers: list of worker addresses

Workers on which we want to restrict the replication. Defaults to all.

branching_factor: int, optional

The number of workers that can copy data in each generation

See also

Client.rebalance

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(self, **kwargs)

Restart the distributed network

This kills all active work, deletes all data on the network, and restarts the worker processes.

retire_workers(self, workers=None, close_workers=True, **kwargs)

Retire certain workers on the scheduler

See dask.distributed.Scheduler.retire_workers for the full docstring.

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(self, futures, asynchronous=None)

Retry failed futures

Parameters
futures: list of Futures
run(self, function, *args, **kwargs)

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 if generally used for side effects, such and 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
function: callable
*args: arguments for remote function
**kwargs: keyword arguments for remote function
workers: list

Workers on which to run the function. Defaults to all known workers.

wait: boolean (optional)

If the function is asynchronous whether or not to wait until that function finishes.

nannybool, defualt False

Whether to run function on the nanny. By default, the function is run on the worker process. If specified, the addresses in workers should still be the worker addresses, not the nanny addresses.

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_hostname)  
{'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_coroutine(self, function, *args, **kwargs)

Spawn a coroutine on all workers.

This spaws a coroutine on all currently known workers and then waits for the coroutine on each worker. The coroutines’ results are returned as a dictionary keyed by worker address.

Parameters
function: a coroutine function
(typically a function wrapped in gen.coroutine or

a Python 3.5+ async function)

*args: arguments for remote function
**kwargs: keyword arguments for remote function
wait: boolean (default True)

Whether to wait for coroutines to end.

workers: list

Workers on which to run the function. Defaults to all known workers.

run_on_scheduler(self, function, *args, **kwargs)

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.

See also

Client.run

Run a function on all workers

Client.start_ipython_scheduler

Start an IPython session on scheduler

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(self, data, workers=None, broadcast=False, direct=None, hash=True, timeout='__no_default__', asynchronous=None)

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
data: list, dict, or object

Data to scatter out to workers. Output type matches input type.

workers: list of tuples (optional)

Optionally constrain locations of data. Specify workers as hostname/port pairs, e.g. ('127.0.0.1', 8787).

broadcast: bool (defaults to False)

Whether to send each data element to all workers. By default we round-robin based on number of cores.

direct: bool (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.

hash: bool (optional)

Whether or not to hash data to determine key. If False then this uses a random key

Returns
List, dict, iterator, or queue of futures matching the type of input.

See also

Client.gather

Gather data back to local process

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(self, **kwargs)

Basic information about the workers in the cluster

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(self, key, value)

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

get_metadata

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(self)

Shut down the connected scheduler and workers

Note, this may disrupt other clients that may be using the same scheudler and workers.

See also

Client.close

close only this client

start(self, **kwargs)

Start scheduler running in separate thread

start_ipython_scheduler(self, magic_name='scheduler_if_ipython', qtconsole=False, qtconsole_args=None)

Start IPython kernel on the scheduler

Parameters
magic_name: str or None (optional)

If defined, register IPython magic with this name for executing code on the scheduler. If not defined, register %scheduler magic if IPython is running.

qtconsole: bool (optional)

If True, launch a Jupyter QtConsole connected to the worker(s).

qtconsole_args: list(str) (optional)

Additional arguments to pass to the qtconsole on startup.

Returns
connection_info: dict

connection_info dict containing info necessary to connect Jupyter clients to the scheduler.

See also

Client.start_ipython_workers

Start IPython on the workers

Examples

>>> c.start_ipython_scheduler() 
>>> %scheduler scheduler.processing  
{'127.0.0.1:3595': {'inc-1', 'inc-2'},
 '127.0.0.1:53589': {'inc-2', 'add-5'}}
>>> c.start_ipython_scheduler(qtconsole=True) 
start_ipython_workers(self, workers=None, magic_names=False, qtconsole=False, qtconsole_args=None)

Start IPython kernels on workers

Parameters
workers: list (optional)

A list of worker addresses, defaults to all

magic_names: str or list(str) (optional)

If defined, register IPython magics with these names for executing code on the workers. If string has asterix then expand asterix into 0, 1, …, n for n workers

qtconsole: bool (optional)

If True, launch a Jupyter QtConsole connected to the worker(s).

qtconsole_args: list(str) (optional)

Additional arguments to pass to the qtconsole on startup.

Returns
iter_connection_info: list

List of connection_info dicts containing info necessary to connect Jupyter clients to the workers.

See also

Client.start_ipython_scheduler

start ipython on the scheduler

Examples

>>> info = c.start_ipython_workers() 
>>> %remote info['192.168.1.101:5752'] worker.data  
{'x': 1, 'y': 100}
>>> c.start_ipython_workers('192.168.1.101:5752', magic_names='w') 
>>> %w worker.data  
{'x': 1, 'y': 100}
>>> c.start_ipython_workers('192.168.1.101:5752', qtconsole=True) 

Add asterix * in magic names to add one magic per worker

>>> c.start_ipython_workers(magic_names='w_*') 
>>> %w_0 worker.data  
{'x': 1, 'y': 100}
>>> %w_1 worker.data  
{'z': 5}
submit(self, 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=None, **kwargs)

Submit a function application to the scheduler

Parameters
func: callable
*args:
**kwargs:
pure: bool (defaults to True)

Whether or not the function is pure. Set pure=False for impure functions like np.random.random.

workers: set, iterable of sets

A set of worker hostnames on which computations may be performed. Leave empty to default to all workers (common case)

key: str

Unique identifier for the task. Defaults to function-name and hash

allow_other_workers: bool (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).

retries: int (default to 0)

Number of allowed automatic retries if the task fails

priority: Number

Optional prioritization of task. Zero is default. Higher priorities take precedence

fifo_timeout: str timedelta (default ‘100ms’)

Allowed amount of time between calls to consider the same priority

resources: dict (defaults to {})

Defines the resources this job requires on the worker; e.g. {'GPU': 2}. See worker resources for details on defining resources.

actor: bool (default False)

Whether this task should exist on the worker as a stateful actor. See actors for additional details.

actors: bool (default False)

Alias for actor

Returns
Future

See also

Client.map

Submit on many arguments at once

Examples

>>> c = client.submit(add, a, b)  
unpublish_dataset(self, name, **kwargs)

Remove named datasets from scheduler

Examples

>>> c.list_datasets()  
['my_dataset']
>>> c.unpublish_datasets('my_dataset')  
>>> c.list_datasets()  
[]
upload_file(self, filename, **kwargs)

Upload local package to workers

This sends a local file up to all worker nodes. This file is placed into a temporary directory on Python’s system path so any .py, .egg or .zip files will be importable.

Parameters
filename: string

Filename of .py, .egg or .zip file to send to workers

Examples

>>> client.upload_file('mylibrary.egg')  
>>> from mylibrary import myfunc  
>>> L = c.map(myfunc, seq)  
wait_for_workers(self, n_workers=0)

Blocking call to wait for n workers before continuing

who_has(self, futures=None, **kwargs)

The workers storing each future’s data

Parameters
futures: list (optional)

A list of futures, defaults to all data

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(self, scheduler_file)

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_file: str

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, inform=True, state=None)

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.

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

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(self, fn)

Call callback on future when callback 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 cancelled

The callback is executed in a separate thread.

cancel(self, **kwargs)

Cancel request to run this future

See also

Client.cancel
cancelled(self)

Returns True if the future has been cancelled

done(self)

Is the computation complete?

exception(self, timeout=None, **kwargs)

Return the exception of a failed task

If timeout seconds are elapsed before returning, a dask.distributed.TimeoutError is raised.

See also

Future.traceback
result(self, timeout=None)

Wait until computation completes, gather result to local process.

If timeout seconds are elapsed before returning, a dask.distributed.TimeoutError is raised.

retry(self, **kwargs)

Retry this future if it has failed

See also

Client.retry
traceback(self, timeout=None, **kwargs)

Return the traceback of a failed task

This returns a traceback object. You can inspect this object using the traceback module. Alternatively if you call future.result() this traceback will accompany the raised exception.

If timeout seconds are elapsed before returning, a dask.distributed.TimeoutError is raised.

See also

Future.exception

Examples

>>> import traceback  
>>> tb = future.traceback()  
>>> traceback.format_tb(tb)  
[...]
class distributed.Queue(name=None, client=None, maxsize=0)

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 and has known issues in Python 2

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. Defaults to the value of _get_global_client().

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(self, timeout=None, batch=False, **kwargs)

Get data from the queue

Parameters
timeout: Number (optional)

Time in seconds to wait before timing out

batch: boolean, 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

put(self, value, timeout=None, **kwargs)

Put data into the queue

qsize(self, **kwargs)

Current number of elements in the queue

class distributed.Variable(name=None, client=None, maxsize=0)

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.

Warning

This object is experimental and has known issues in Python 2

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. Defaults to the value of _get_global_client().

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(self)

Delete this variable

Caution, this affects all clients currently pointing to this variable.

get(self, timeout=None, **kwargs)

Get the value of this variable

set(self, value, **kwargs)

Set the value of this variable

Parameters
value: Future or object

Must be either a Future or a msgpack-encodable value

class distributed.Lock(name=None, client=None)

Distributed Centralized Lock

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(self, blocking=True, timeout=None)

Acquire the lock

Parameters
blockingbool, optional

If false, don’t wait on the lock in the scheduler at all.

timeoutnumber, 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.

Returns
True or False whether or not it sucessfully acquired the lock

Examples

>>> lock = Lock('x')  
>>> lock.acquire(timeout=1)  
release(self)

Release the lock if already acquired

class distributed.Pub(name, worker=None, client=None)

Publish data with Publish-Subscribe pattern

This allows clients and workers to directly communicate data between each other with a typical Publish-Subscribe pattern. This involves two components,

Pub objects, into which we put data:

>>> pub = Pub('my-topic')
>>> pub.put(123)

And Sub objects, from which we collect data:

>>> sub = Sub('my-topic')
>>> sub.get()
123

Many Pub and Sub objects can exist for the same topic. All data sent from any Pub will be sent to all Sub objects on that topic that are currently connected. Pub’s and Sub’s find each other using the scheduler, but they communicate directly with each other without coordination from the scheduler.

Pubs and Subs use the central scheduler to find each other, but not to mediate the communication. This means that there is very little additional latency or overhead, and they are appropriate for very frequent data transfers. For context, most data transfer first checks with the scheduler to find which workers should participate, and then does direct worker-to-worker transfers. This checking in with the scheduler provides some stability guarantees, but also adds in a few extra network hops. PubSub doesn’t do this, and so is faster, but also can easily drop messages if Pubs or Subs disappear without notice.

When using a Pub or Sub from a Client all communications will be routed through the scheduler. This can cause some performance degradation. Pubs and Subs only operate at top-speed when they are both on workers.

Parameters
name: object (msgpack serializable)

The name of the group of Pubs and Subs on which to participate.

worker: Worker (optional)

The worker to be used for publishing data. Defaults to the value of `get_worker()`. If given, client must be None.

client: Client (optional)

Client used for communication with the scheduler. Defaults to the value of get_client(). If given, worker must be None.

See also

Sub

Examples

>>> pub = Pub('my-topic')
>>> sub = Sub('my-topic')
>>> pub.put([1, 2, 3])
>>> sub.get()
[1, 2, 3]

You can also use sub within a for loop:

>>> for msg in sub:  
...     print(msg)

or an async for loop

>>> async for msg in sub:  
...     print(msg)

Similarly the .get method will return an awaitable if used by an async client or within the IOLoop thread of a worker

>>> await sub.get()  

You can see the set of connected worker subscribers by looking at the .subscribers attribute:

>>> pub.subscribers
{'tcp://...': {},
 'tcp://...': {}}
put(self, msg)

Publish a message to all subscribers of this topic

class distributed.Sub(name, worker=None, client=None)

Subscribe to a Publish/Subscribe topic

See also

Pub

for full docstring

get(self, timeout=None)

Get a single message

next(self, timeout=None)

Get a single message