API

Client

Client([address, loop, timeout, ...]) Connect to and drive computation on a distributed Dask cluster
Client.cancel(futures) Cancel running futures
Client.compute(collections[, sync, ...]) Compute dask collections on cluster
Client.gather(futures[, errors, maxsize, direct]) Gather futures from distributed memory
Client.get(dsk, keys[, restrictions, ...]) Compute dask graph
Client.get_dataset(name) Get named dataset from the scheduler
Client.get_executor(**kwargs) Return a concurrent.futures Executor for submitting tasks on this Client.
Client.has_what([workers]) Which keys are held by which workers
Client.list_datasets() List named datasets available on the scheduler
Client.map(func, *iterables, **kwargs) Map a function on a sequence of arguments
Client.ncores([workers]) The number of threads/cores available on each worker node
Client.persist(collections[, ...]) Persist dask collections on cluster
Client.publish_dataset(**kwargs) Publish named datasets to scheduler
Client.rebalance([futures, workers]) Rebalance data within network
Client.replicate(futures[, n, workers, ...]) Set replication of futures within network
Client.restart() Restart the distributed network
Client.run(function, *args, **kwargs) Run a function on all workers outside of task scheduling system
Client.run_on_scheduler(function, *args, ...) Run a function on the scheduler process
Client.scatter(data[, workers, broadcast, ...]) Scatter data into distributed memory
Client.shutdown([timeout]) Send shutdown signal and wait until scheduler terminates
Client.scheduler_info() Basic information about the workers in the cluster
Client.shutdown([timeout]) Send shutdown signal and wait until scheduler terminates
Client.start_ipython_workers([workers, ...]) Start IPython kernels on workers
Client.start_ipython_scheduler([magic_name, ...]) Start IPython kernel on the scheduler
Client.submit(func, *args, **kwargs) Submit a function application to the scheduler
Client.unpublish_dataset(name) Remove named datasets from scheduler
Client.upload_file(filename) Upload local package to workers
Client.who_has([futures]) The workers storing each future’s data
worker_client(*args, **kwds) Get client for this thread
get_worker() Get the worker currently running this task
ReplayExceptionClient.get_futures_error(future) Ask the scheduler details of the sub-task of the given failed future
ReplayExceptionClient.recreate_error_locally(future) For a failed calculation, perform the blamed task locally for debugging.

Future

Future(key, client[, inform]) A remotely running computation
Future.add_done_callback(fn) Call callback on future when callback has finished
Future.cancel() Returns True if the future has been cancelled
Future.cancelled() Returns True if the future has been cancelled
Future.done() Is the computation complete?
Future.exception([timeout]) Return the exception of a failed task
Future.result([timeout]) Wait until computation completes.
Future.traceback([timeout]) Return the traceback of a failed task

Client Coordination

Queue([name, client, maxsize]) Distributed Queue
Variable([name, client, maxsize]) Distributed Global Variable

Other

as_completed([futures, loop, with_results]) Return futures in the order in which they complete
distributed.diagnostics.progress(*futures, ...) Track progress of futures
wait(fs[, timeout, return_when]) Wait until all futures are complete

Asynchronous methods

Most methods and functions can be used equally well within a blocking or asynchronous environment using Tornado coroutines. If used within a Tornado IOLoop then you should yield or await otherwise blocking operations appropriately.

You must tell the client that you intend to use it within an asynchronous environment by passing the asynchronous=True keyword

# blocking
client = Client()
future = client.submit(func, *args)  # immediate, no blocking/async difference
result = client.gather(future)  # blocking

# asynchronous Python 2/3
client = yield Client(asynchronous=True)
future = client.submit(func, *args)  # immediate, no blocking/async difference
result = yield client.gather(future)  # non-blocking/asynchronous

# asynchronous Python 3
client = await Client(asynchronous=True)
future = client.submit(func, *args)  # immediate, no blocking/async difference
result = await client.gather(future)  # non-blocking/asynchronous

The asynchronous variants must be run within a Tornado coroutine. See the Asynchronous documentation for more information.

Client

class distributed.Client(address=None, loop=None, timeout=5, set_as_default=True, scheduler_file=None, security=None, start=None, asynchronous=False, **kwargs)[source]

Connect to and drive computation on a distributed Dask cluster

The Client connects users to a dask.distributed compute 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.

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

Optional security information

asynchronous: bool (False by default)

Set to True if this client will be used within a Tornado event loop

See also

distributed.scheduler.Scheduler
Internal scheduler

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
cancel(futures)[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:futures: list of Futures
compute(collections, sync=False, optimize_graph=True, workers=None, allow_other_workers=False, resources=None, **kwargs)[source]

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

**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)  
gather(futures, errors='raise', maxsize=0, direct=False)[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:

futures: Collection of futures

This can be a possibly nested collection of Future objects. Collections can be lists, sets, iterators, queues 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

maxsize: int

If the input is a queue then this produces an output queue with a maximum size.

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]
>>> seq = c.gather(iter([x, x]))  # support iterators 
>>> next(seq)  
3
get(dsk, keys, restrictions=None, loose_restrictions=None, resources=None, sync=True, **kwargs)[source]

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

sync: bool (optional)

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

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)[source]

Get named dataset from the scheduler

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:

An Executor object that’s fully compatible with the concurrent.futures

API.

static get_restrictions(collections, workers, allow_other_workers)[source]

Get restrictions from inputs to compute/persist

get_versions(check=False)[source]

Return version info for the scheduler, all workers and myself

Parameters:

check : boolean, default False

raise ValueError if all required & optional packages do not match

Examples

>>> c.get_versions()  
has_what(workers=None)[source]

Which keys are held by which workers

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()[source]

List named datasets available on the scheduler

map(func, *iterables, **kwargs)[source]

Map a function on a sequence of arguments

Arguments can be normal objects or Futures

Parameters:

func: callable

iterables: Iterables, Iterators, or Queues

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)

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(keys=None, summary=True)[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:

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(workers=None)[source]

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.ncores()  
{'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.

See also

Client.persist
trigger computation of collection’s tasks

Examples

>>> len(x.dask)  # x is a dask collection with 100 tasks
100
>>> set(client.futures).intersection(x.dask)  # some overlap exists
10
>>> x = client.normalize_collection(x)
>>> len(x.dask)  # smaller computational graph
20
persist(collections, optimize_graph=True, workers=None, allow_other_workers=None, resources=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:

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

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(workers=None)[source]

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']}
publish_dataset(**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:

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)  

Receiving client:

>>> c.list_datasets()  
['my_dataset']
>>> df2 = c.get_dataset('my_dataset')  
rebalance(futures=None, workers=None)[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.

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

replicate(futures, n=None, workers=None, branching_factor=2)[source]

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()[source]

Restart the distributed network

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

run(function, *args, **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 if generally used for side effects, such and collecting diagnostic information or installing libraries.

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.

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}
run_coroutine(function, *args, **kwargs)[source]

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

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.task_state)
>>> client.run_on_scheduler(get_number_of_tasks)  
100
scatter(data, workers=None, broadcast=False, direct=False, maxsize=0)[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:

data: list, iterator, dict, Queue, 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 False)

Send data directly to workers, bypassing the central scheduler This avoids burdening the scheduler but assumes that the client is able to talk directly with the workers.

maxsize: int (optional)

Maximum size of queue if using queues, 0 implies infinite

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

Handle streaming sequences of data with iterators or queues

>>> seq = c.scatter(iter([1, 2, 3]))  
>>> next(seq)  
<Future: status: finished, key: c0a8a20f903a4915b94db8de3ea63195>,

Broadcast data to all workers

>>> [future] = c.scatter([element], broadcast=True)  
scheduler_info()[source]

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}}}
shutdown(timeout=10)[source]

Send shutdown signal and wait until scheduler terminates

This cancels all currently running tasks, clears the state of the scheduler, and shuts down all workers and scheduler.

You do not need to call this when you finish your session. You only need to call this if you want to take down the distributed cluster.

See also

Client.restart

stacks(workers=None)[source]

The task queues 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.stacks()  
{'192.168.1.141:46784': ['inc-1c8dd6be1c21646c71f76c16d09304ea',
                         'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b',
                         'inc-1e297fc27658d7b67b3a758f16bcf47a']}
start(asynchronous=None, **kwargs)[source]

Start scheduler running in separate thread

start_ipython_scheduler(magic_name='scheduler_if_ipython', qtconsole=False, qtconsole_args=None)[source]

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(workers=None, magic_names=False, qtconsole=False, qtconsole_args=None)[source]

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(func, *args, **kwargs)[source]

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)

allow_other_workers: bool (defaults to False)

Used with workers. Inidicates whether or not the computations may be performed on workers that are not in the workers set(s).

Returns:

Future

See also

Client.map
Submit on many arguments at once

Examples

>>> c = client.submit(add, a, b)  
unpublish_dataset(name)[source]

Remove named datasets from scheduler

Examples

>>> c.list_datasets()  
['my_dataset']
>>> c.unpublish_datasets('my_dataset')  
>>> c.list_datasets()  
[]
upload_file(filename)[source]

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, .pyc, .egg or .zip files will be importable.

Parameters:

filename: string

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

Examples

>>> client.upload_file('mylibrary.egg')  
>>> from mylibrary import myfunc  
>>> L = c.map(myfunc, seq)  
who_has(futures=None)[source]

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']}
class distributed.recreate_exceptions.ReplayExceptionClient(client)[source]

A plugin for the client allowing replay of remote exceptions locally

Adds the following methods (and their async variants)to the given client:

  • recreate_error_locally: main user method
  • get_futures_error: gets the task, its details and dependencies,
    responsible for failure of the given future.
get_futures_error(future)[source]

Ask the scheduler details of the sub-task of the given failed future

When a future evaluates to a status of “error”, i.e., an exception was raised in a task within its graph, we an get information from the scheduler. This function gets the details of the specific task that raised the exception and led to the error, but does not fetch data from the cluster or execute the function.

Parameters:

future : future that failed, having status=="error", typically

after an attempt to gather() shows a stack-stace.

Returns:

Tuple:

  • the function that raised an exception
  • argument list (a tuple), may include values and keys
  • keyword arguments (a dictionary), may include values and keys
  • list of keys that the function requires to be fetched to run

See also

ReplayExceptionClient.recreate_error_locally

recreate_error_locally(future)[source]

For a failed calculation, perform the blamed task locally for debugging.

This operation should be performed after a future (result of gather, compute, etc) comes back with a status of “error”, if the stack- trace is not informative enough to diagnose the problem. The specific task (part of the graph pointing to the future) responsible for the error will be fetched from the scheduler, together with the values of its inputs. The function will then be executed, so that pdb can be used for debugging.

Parameters:

future : future or collection that failed

The same thing as was given to gather, but came back with an exception/stack-trace. Can also be a (persisted) dask collection containing any errored futures.

Returns:

Nothing; the function runs and should raise an exception, allowing

the debugger to run.

Examples

>>> future = c.submit(div, 1, 0)         
>>> future.status                        
'error'
>>> c.recreate_error_locally(future)     
ZeroDivisionError: division by zero

If you’re in IPython you might take this opportunity to use pdb

>>> %pdb                                 
Automatic pdb calling has been turned ON
>>> c.recreate_error_locally(future)     
ZeroDivisionError: division by zero
      1 def div(x, y):
----> 2     return x / y
ipdb>

Future

Other

distributed.as_completed(futures=None, loop=None, with_results=False)[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

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], results=True)  
>>> for future, result in ac:  
...     print(result)  
2
4
3
distributed.diagnostics.progress(*futures, **kwargs)[source]

Track progress of futures

This operates differently in the notebook and the console

  • Notebook: This returns immediately, leaving an IPython widget on screen
  • Console: This blocks until the computation completes
Parameters:

futures: Futures

A list of futures or keys to track

notebook: bool (optional)

Running in the notebook or not (defaults to guess)

multi: bool (optional)

Track different functions independently (defaults to True)

complete: bool (optional)

Track all keys (True) or only keys that have not yet run (False) (defaults to True)

Notes

In the notebook, the output of progress must be the last statement in the cell. Typically, this means calling progress at the end of a cell.

Examples

>>> progress(futures)  
[########################################] | 100% Completed |  1.7s
distributed.wait(fs, timeout=None, return_when='ALL_COMPLETED')[source]

Wait until all futures are complete

Parameters:

fs: list of futures

timeout: number, optional

Time in seconds after which to raise a gen.TimeoutError

Returns:

Named tuple of completed, not completed

distributed.worker_client(*args, **kwds)[source]

Get client for this thread

This context manager is intended to be called within functions that we run on workers. When run as a context manager it delivers a client Client object that can submit other tasks directly from that worker.

Parameters:

timeout: Number

Timeout after which to err

separate_thread: bool, optional

Whether to run this function outside of the normal thread pool defaults to True

See also

get_worker

Examples

>>> def func(x):
...     with worker_client() as c:  # connect from worker back to scheduler
...         a = c.submit(inc, x)     # this task can submit more tasks
...         b = c.submit(dec, x)
...         result = c.gather([a, b])  # and gather results
...     return result
>>> future = client.submit(func, 1)  # submit func(1) on cluster
distributed.get_worker()[source]

Get the worker currently running this task

See also

worker_client

Examples

>>> def f():
...     worker = get_worker()  # The worker on which this task is running
...     return worker.address
>>> future = client.submit(f)  
>>> future.result()  
'tcp://127.0.0.1:47373'
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 and has known issues in Python 2

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)[source]

Get data from the queue

put(value, timeout=None)[source]

Put data into the queue

qsize()[source]

Current number of elements in the queue

class distributed.Variable(name=None, client=None, maxsize=0)[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.

Warning

This object is experimental and has known issues in Python 2

See also

Queue

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.

get(timeout=None)[source]

Get the value of this variable

set(value)[source]

Set the value of this variable

Parameters:

value: Future or object

Must be either a Future or a msgpack-encodable value

Asyncio Client