API

Client

Client([address, loop, timeout, …]) Connect to and drive computation on a distributed Dask cluster
Client.call_stack([futures, keys]) The actively running call stack of all relevant keys
Client.cancel(futures[, asynchronous, force]) Cancel running futures
Client.close([timeout]) Close this client
Client.compute(collections[, sync, …]) Compute dask collections on cluster
Client.gather(futures[, errors, maxsize, …]) Gather futures from distributed memory
Client.get(dsk, keys[, restrictions, …]) Compute dask graph
Client.get_dataset(name, **kwargs) Get named dataset from the scheduler
Client.get_executor(**kwargs) Return a concurrent.futures Executor for submitting tasks on this Client
Client.get_metadata(keys[, default]) Get arbitrary metadata from scheduler
Client.get_scheduler_logs([n]) Get logs from scheduler
Client.get_task_stream([start, stop, count, …]) Get task stream data from scheduler
Client.get_worker_logs([n, workers]) Get logs from workers
Client.has_what([workers]) Which keys are held by which workers
Client.list_datasets(**kwargs) 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(*args, **kwargs) Publish named datasets to scheduler
Client.profile([key, start, stop, workers, …]) Collect statistical profiling information about recent work
Client.rebalance([futures, workers]) Rebalance data within network
Client.replicate(futures[, n, workers, …]) Set replication of futures within network
Client.restart(**kwargs) Restart the distributed network
Client.retry(futures[, asynchronous]) Retry failed futures
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.scheduler_info(**kwargs) Basic information about the workers in the cluster
Client.write_scheduler_file(scheduler_file) Write the scheduler information to a json file.
Client.set_metadata(key, value) Set arbitrary metadata in the scheduler
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, **kwargs) Remove named datasets from scheduler
Client.upload_file(filename, **kwargs) Upload local package to workers
Client.who_has([futures]) The workers storing each future’s data
worker_client([timeout, separate_thread]) Get client for this thread
get_worker() Get the worker currently running this task
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
Reschedule Reschedule 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, state]) A remotely running computation
Future.add_done_callback(fn) Call callback on future when callback has finished
Future.cancel(**kwargs) Cancel request to run this future
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, gather result to local process.
Future.retry(**kwargs) Retry this future if it has failed
Future.traceback([timeout]) Return the traceback of a failed task

Client Coordination

Lock([name, client]) Distributed Centralized Lock
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
fire_and_forget(obj) Run tasks at least once, even if we release the futures
futures_of(o[, client]) Future objects in a collection
get_task_stream([client, plot, filename]) Collect task stream within a context block

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

Can this client connect directly to workers or should it proxy through the scheduler?

heartbeat_interval: int

Time in milliseconds between heartbeats to scheduler

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
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(futures=None, keys=None)[source]

The actively running call stack of all relevant keys

You can specify data of interest either by providing futures or collections in the futures= keyword or a list of explicit keys in 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(futures, asynchronous=None, force=False)[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
force: boolean (False)

Cancel this future even if other clients desire it

close(timeout='__no_default__')[source]

Close this client

Clients will also close automatically when your Python session ends

If you started a client without arguments like Client() then this will also close the local cluster that was started at the same time.

See also

Client.restart

compute(collections, sync=False, optimize_graph=True, workers=None, allow_other_workers=False, resources=None, retries=0, priority=0, fifo_timeout='60s', actors=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

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

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

Return global client if one exists, otherwise raise ValueError

gather(futures, errors='raise', maxsize=0, direct=None, asynchronous=None)[source]

Gather futures from distributed memory

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

Parameters:
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, asynchronous=None, direct=None, retries=None, priority=0, fifo_timeout='60s', **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

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

Gather results directly from workers

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, **kwargs)[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.
get_metadata(keys, default='__no_default__')[source]

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

Get restrictions from inputs to compute/persist

get_scheduler_logs(n=None)[source]

Get logs from scheduler

Parameters:
n: int

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(start=None, stop=None, count=None, plot=False, filename='task-stream.html')[source]

Get task stream data from scheduler

This collects the data present in the diagnostic “Task Stream” plot on the dashboard. It includes the start, stop, transfer, and deserialization time of every task for a particular duration.

Note that the task stream diagnostic does not run by default. You may wish to call this function once before you start work to ensure that things start recording, and then again after you have completed.

Parameters:
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(check=False, packages=[])[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

packages : List[str]

Extra package names to check

Examples

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

Get logs from workers

Parameters:
n: int

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

workers: iterable

List of worker addresses to retrive. Gets all workers by default.

Returns:
Dictionary mapping worker address to logs.
Logs are returned in reversed order (newest first)
has_what(workers=None, **kwargs)[source]

Which keys are held by which workers

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

Parameters:
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(**kwargs)[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)

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

**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(keys=None, summary=True, **kwargs)[source]

The bytes taken up by each key on the cluster

This is as measured by sys.getsizeof which may not accurately reflect the true cost.

Parameters:
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, **kwargs)[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_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
persist(collections, optimize_graph=True, workers=None, allow_other_workers=None, resources=None, retries=None, priority=0, fifo_timeout='60s', actors=None, **kwargs)[source]

Persist dask collections on cluster

Starts computation of the collection on the cluster in the background. Provides a new dask collection that is semantically identical to the previous one, but now based off of futures currently in execution.

Parameters:
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

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']}
profile(key=None, start=None, stop=None, workers=None, merge_workers=True)[source]

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

Examples

>>> client.profile()  # call on collections
publish_dataset(*args, **kwargs)[source]

Publish named datasets to scheduler

This stores a named reference to a dask collection or list of futures on the scheduler. These references are available to other Clients which can download the collection or futures with get_dataset.

Datasets are not immediately computed. You may wish to call Client.persist prior to publishing a dataset.

Parameters:
args : list of objects to publish as name
name : optional 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(futures=None, workers=None, **kwargs)[source]

Rebalance data within network

Move data between workers to roughly balance memory burden. This either affects a subset of the keys/workers or the entire network, depending on keyword arguments.

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

Registers a setup callback function for all current and future workers.

This registers a new setup function for workers in this cluster. The function will run immediately on all currently connected workers. It will also be run upon connection by any workers that are added in the future. Multiple setup functions can be registered - these will be called in the order they were added.

If the function takes an input argument named dask_worker then that variable will be populated with the worker itself.

Parameters:
setup : callable(dask_worker: Worker) -> None

Function to register and run on all workers

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

Set replication of futures within network

Copy data onto many workers. This helps to broadcast frequently accessed data and 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(**kwargs)[source]

Restart the distributed network

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

retire_workers(workers=None, close_workers=True, **kwargs)[source]

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(futures, asynchronous=None)[source]

Retry failed futures

Parameters:
futures: list of Futures
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.

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.

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_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.tasks)
>>> client.run_on_scheduler(get_number_of_tasks)  
100
scatter(data, workers=None, broadcast=False, direct=None, hash=True, maxsize=0, timeout='__no_default__', asynchronous=None)[source]

Scatter data into distributed memory

This moves data from the local client process into the workers of the distributed scheduler. Note that it is often better to submit jobs to your workers to have them load the data rather than loading data locally and then scattering it out to them.

Parameters:
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 automatically check)

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

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

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)  

Send scattered data to parallelized function using client futures interface

>>> data = c.scatter(data, broadcast=True)  
>>> res = [c.submit(func, data, i) for i in range(100)]
scheduler_info(**kwargs)[source]

Basic information about the workers in the cluster

Examples

>>> c.scheduler_info()  
{'id': '2de2b6da-69ee-11e6-ab6a-e82aea155996',
 'services': {},
 'type': 'Scheduler',
 'workers': {'127.0.0.1:40575': {'active': 0,
                                 'last-seen': 1472038237.4845693,
                                 'name': '127.0.0.1:40575',
                                 'services': {},
                                 'stored': 0,
                                 'time-delay': 0.0061032772064208984}}}
set_metadata(key, value)[source]

Set arbitrary metadata in the scheduler

This allows you to store small amounts of data on the central scheduler process for administrative purposes. Data should be msgpack serializable (ints, strings, lists, dicts)

If the key corresponds to a task then that key will be cleaned up when the task is forgotten by the scheduler.

If the key is a list then it will be assumed that you want to index into a nested dictionary structure using those keys. For example if you call the following:

>>> client.set_metadata(['a', 'b', 'c'], 123)

Then this is the same as setting

>>> scheduler.task_metadata['a']['b']['c'] = 123

The lower level dictionaries will be created on demand.

See also

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

Deprecated, see close instead

This was deprecated because “shutdown” was sometimes confusingly thought to refer to the cluster rather than the client

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

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

Returns:
Future

See also

Client.map
Submit on many arguments at once

Examples

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

Remove named datasets from scheduler

Examples

>>> c.list_datasets()  
['my_dataset']
>>> c.unpublish_datasets('my_dataset')  
>>> c.list_datasets()  
[]
upload_file(filename, **kwargs)[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, .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)  
who_has(futures=None, **kwargs)[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']}
write_scheduler_file(scheduler_file)[source]

Write the scheduler information to a json file.

This facilitates easy sharing of scheduler information using a file system. The scheduler file can be used to instantiate a second Client using the same scheduler.

Parameters:
scheduler_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.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
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

class distributed.Future(key, client=None, inform=True, state=None)[source]

A remotely running computation

A Future is a local proxy to a result running on a remote worker. A user manages future objects in the local Python process to determine what happens in the larger cluster.

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

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

Cancel request to run this future

See also

Client.cancel

cancelled()[source]

Returns True if the future has been cancelled

done()[source]

Is the computation complete?

exception(timeout=None, **kwargs)[source]

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

Wait until computation completes, gather result to local process.

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

retry(**kwargs)[source]

Retry this future if it has failed

See also

Client.retry

traceback(timeout=None, **kwargs)[source]

Return the traceback of a failed task

This returns a traceback object. You can inspect this object using the traceback module. Alternatively if you 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)  
[...]

Other

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

Return futures in the order in which they complete

This returns an iterator that yields the input future objects in the order in which they complete. Calling next on the iterator will block until the next future completes, irrespective of order.

Additionally, you can also add more futures to this object during computation with the .add method

Parameters:
futures: Collection of futures

A list of Future objects to be iterated over in the order in which they complete

with_results: bool (False)

Whether to wait and include results of futures as well; in this case as_completed yields a tuple of (future, result)

raise_errors: bool (True)

Whether we should raise when the result of a future raises an exception; only affects behavior when with_results=True.

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.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 dask.distributed.TimeoutError

——-
Named tuple of completed, not completed
distributed.fire_and_forget(obj)[source]

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

Under normal operation Dask will not run any tasks for which there is not an active future (this avoids unnecessary work in many situations). However sometimes you want to just fire off a task, not track its future, and expect it to finish eventually. You can use this function on a future or collection of futures to ask Dask to complete the task even if no active client is tracking it.

The results will not be kept in memory after the task completes (unless there is an active future) so this is only useful for tasks that depend on side effects.

Parameters:
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.futures_of(o, client=None)[source]

Future objects in a collection

distributed.worker_client(timeout=3, separate_thread=True)[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

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

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'
distributed.get_client(address=None, timeout=3, resolve_address=True)[source]

Get a client while within a task.

This client connects to the same scheduler to which the worker is connected

Parameters:
address : str, optional

The address of the scheduler to connect to. Defaults to the scheduler the worker is connected to.

timeout : int, default 3

Timeout (in seconds) for getting the Client

resolve_address : bool, default True

Whether to resolve address to its canonical form.

Returns:
Client

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

Have this task secede from the worker’s thread pool

This opens up a new scheduling slot and a new thread for a new task. This enables the client to schedule tasks on this node, which is especially useful while waiting for other jobs to finish (e.g., with client.gather).

Examples

>>> def mytask(x):
...     # do some work
...     client = get_client()
...     futures = client.map(...)  # do some remote work
...     secede()  # while that work happens, remove ourself from the pool
...     return client.gather(futures)  # return gathered results
distributed.rejoin()[source]

Have this thread rejoin the ThreadPoolExecutor

This will block until a new slot opens up in the executor. The next thread to finish a task will leave the pool to allow this one to join.

See also

secede
leave the thread pool
class distributed.get_task_stream(client=None, plot=False, filename='task-stream.html')[source]

Collect task stream within a context block

This provides diagnostic information about every task that was run during the time when this block was active.

This must be used as a context manager.

Parameters:
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'

See also

Client.get_task_stream
Function version of this context manager

Examples

>>> with get_task_stream() as ts:
...     x.compute()
>>> ts.data
[...]

Get back a Bokeh figure and optionally save to a file

>>> with get_task_stream(plot='save', filename='task-stream.html') as ts:
...    x.compute()
>>> ts.figure
<Bokeh Figure>

To share this file with others you may wish to upload and serve it online. A common way to do this is to upload the file as a gist, and then serve it on https://rawgit.com

$ pip install gist
$ gist task-stream.html
https://gist.github.com/8a5b3c74b10b413f612bb5e250856ceb

You can then navigate to that site, click the “Raw” button to the right of the task-stream.html file, and then provide that URL to https://rawgit.com . This process should provide a sharable link that others can use to see your task stream plot.

class distributed.Lock(name=None, client=None)[source]

Distributed Centralized Lock

Parameters:
name: string

Name of the lock to acquire. Choosing the same name allows two disconnected processes to coordinate a lock.

Examples

>>> lock = Lock('x')  
>>> lock.acquire(timeout=1)  
>>> # do things with protected resource
>>> lock.release()  
acquire(timeout=None)[source]

Acquire the lock

Parameters:
timeout: number

Seconds to wait on the lock in the scheduler. This does not include local coroutine time, network transfer time, etc..

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

Examples

>>> lock = Lock('x')  
>>> lock.acquire(timeout=1)  
release()[source]

Release the lock if already acquired

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, batch=False, **kwargs)[source]

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(value, timeout=None, **kwargs)[source]

Put data into the queue

qsize(**kwargs)[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
shared multi-producer/multi-consumer queue between clients

Examples

>>> from dask.distributed import Client, Variable 
>>> client = Client()  
>>> x = Variable('x')  
>>> x.set(123)  # docttest: +SKIP
>>> x.get()  # docttest: +SKIP
123
>>> future = client.submit(f, x)  
>>> x.set(future)  
delete()[source]

Delete this variable

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

get(timeout=None, **kwargs)[source]

Get the value of this variable

set(value, **kwargs)[source]

Set the value of this variable

Parameters:
value: Future or object

Must be either a Future or a msgpack-encodable value

Asyncio Client

class distributed.asyncio.AioClient(*args, **kwargs)[source]

Connect to and drive computation on a distributed Dask cluster

This class provides an asyncio compatible async/await interface for dask.distributed.

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.

AioClient is an experimental interface for distributed and may disappear without warning!

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

See also

distributed.client.Client
Blocking Client
distributed.scheduler.Scheduler
Internal scheduler

Examples

Provide cluster’s scheduler address on initialization:

client = AioClient('127.0.0.1:8786')

Start the client:

async def start_the_client():
    client = await AioClient()

    # Use the client....

    await client.close()

An async with statement is a more convenient way to start and shut down the client:

async def start_the_client():
    async with AioClient() as client:
        # Use the client within this block.
        pass

Use the submit method to send individual computations to the cluster, and await the returned future to retrieve the result:

async def add_two_numbers():
    async with AioClient() as client:
        a = client.submit(add, 1, 2)
        result = await a

Continue using submit or map on results to build up larger computations, and gather results with the gather method:

async def gather_some_results():
    async with AioClient() as client:
        a = client.submit(add, 1, 2)
        b = client.submit(add, 10, 20)
        c = client.submit(add, a, b)
        result = await client.gather([c])
close(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

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

Gather results directly from workers

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

Deprecated, see close instead

This was deprecated because “shutdown” was sometimes confusingly thought to refer to the cluster rather than the client

Adaptive

class distributed.deploy.Adaptive(scheduler, cluster=None, interval='1s', startup_cost='1s', scale_factor=2, minimum=0, maximum=None, wait_count=3, target_duration='5s', worker_key=<function Adaptive.<lambda>>, **kwargs)[source]

Adaptively allocate workers based on scheduler load. A superclass.

Contains logic to dynamically resize a Dask cluster based on current use. This class needs to be paired with a system that can create and destroy Dask workers using a cluster resource manager. Typically it is built into already existing solutions, rather than used directly by users. It is most commonly used from the .adapt(...) method of various Dask cluster classes.

Parameters:
scheduler: distributed.Scheduler
cluster: object

Must have scale_up and scale_down methods/coroutines

startup_cost : timedelta or str, default “1s”

Estimate of the number of seconds for nnFactor representing how costly it is to start an additional worker. Affects quickly to adapt to high tasks per worker loads

interval : timedelta or str, default “1000 ms”

Milliseconds between checks

wait_count: int, default 3

Number of consecutive times that a worker should be suggested for removal before we remove it.

scale_factor : int, default 2

Factor to scale by when it’s determined additional workers are needed

target_duration: timedelta or str, default “5s”

Amount of time we want a computation to take. This affects how aggressively we scale up.

worker_key: Callable[WorkerState]

Function to group workers together when scaling down See Scheduler.workers_to_close for more information

minimum: int

Minimum number of workers to keep around

maximum: int

Maximum number of workers to keep around

**kwargs:

Extra parameters to pass to Scheduler.workers_to_close

Notes

Subclasses can override Adaptive.should_scale_up() and Adaptive.workers_to_close() to control when the cluster should be resized. The default implementation checks if there are too many tasks per worker or too little memory available (see Adaptive.needs_cpu() and Adaptive.needs_memory()).

Adaptive.get_scale_up_kwargs() method controls the arguments passed to the cluster’s scale_up method.

Examples

This is commonly used from existing Dask classes, like KubeCluster

>>> from dask_kubernetes import KubeCluster
>>> cluster = KubeCluster()
>>> cluster.adapt(minimum=10, maximum=100)

Alternatively you can use it from your own Cluster class by subclassing from Dask’s Cluster superclass

>>> from distributed.deploy import Cluster
>>> class MyCluster(Cluster):
...     def scale_up(self, n):
...         """ Bring worker count up to n """
...     def scale_down(self, workers):
...        """ Remove worker addresses from cluster """
>>> cluster = MyCluster()
>>> cluster.adapt(minimum=10, maximum=100)
get_scale_up_kwargs()[source]

Get the arguments to be passed to self.cluster.scale_up.

See also

LocalCluster.scale_up

Notes

By default the desired number of total workers is returned (n). Subclasses should ensure that the return dictionary includes a key- value pair for n, either by implementing it or by calling the parent’s get_scale_up_kwargs.

needs_cpu()[source]

Check if the cluster is CPU constrained (too many tasks per core)

Notes

Returns True if the occupancy per core is some factor larger than startup_cost.

needs_memory()[source]

Check if the cluster is RAM constrained

Notes

Returns True if the required bytes in distributed memory is some factor larger than the actual distributed memory available.

should_scale_up()[source]

Determine whether additional workers should be added to the cluster

Returns:
scale_up : bool
Notes
—-
Additional workers are added whenever
1. There are unrunnable tasks and no workers
2. The cluster is CPU constrained
3. The cluster is RAM constrained
4. There are fewer workers than our minimum
workers_to_close(**kwargs)[source]

Determine which, if any, workers should potentially be removed from the cluster.

Returns:
List of worker addresses to close, if any

See also

Scheduler.workers_to_close

Notes

Adaptive.workers_to_close dispatches to Scheduler.workers_to_close(), but may be overridden in subclasses.