API

Client

Client([address, loop, timeout, …]) Connect to and submit computation to a Dask cluster
Client.as_current() Thread-local, Task-local context manager that causes the Client.current class method to return self.
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.collections_to_dsk(collections, …) Convert many collections into a single dask graph, after optimization
Client.compute(collections[, sync, …]) Compute dask collections on cluster
Client.current([allow_global]) When running within the context of as_client, return the context-local current client.
Client.futures_of(futures)
Client.gather(futures[, errors, direct, …]) Gather futures from distributed memory
Client.get(dsk, keys[, workers, …]) Compute dask graph
Client.get_dataset(name[, default]) Get named dataset from the scheduler if present.
Client.get_events(topic) Retrieve structured topic logs
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_versions([check, packages]) Return version info for the scheduler, all workers and myself
Client.get_worker_logs([n, workers, nanny]) 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.log_event(topic, msg) Log an event under a given topic
Client.map(func, *iterables[, key, workers, …]) Map a function on a sequence of arguments
Client.nbytes([keys, summary]) The bytes taken up by each key on the cluster
Client.ncores([workers]) The number of threads/cores available on each worker node
Client.normalize_collection(collection) Replace collection’s tasks by already existing futures if they exist
Client.nthreads([workers]) The number of threads/cores available on each worker node
Client.persist(collections[, …]) Persist dask collections on cluster
Client.processing([workers]) The tasks currently running on each worker
Client.profile([key, start, stop, workers, …]) Collect statistical profiling information about recent work
Client.publish_dataset(*args, **kwargs) Publish named datasets to scheduler
Client.rebalance([futures, workers]) Rebalance data within network
Client.register_worker_callbacks([setup]) Registers a setup callback function for all current and future workers.
Client.register_worker_plugin([plugin, name]) Registers a lifecycle worker plugin for all current and future workers.
Client.replicate(futures[, n, workers, …]) Set replication of futures within network
Client.restart(**kwargs) Restart the distributed network
Client.retire_workers([workers, close_workers]) Retire certain workers on the scheduler
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_coroutine(function, *args, **kwargs) Spawn a coroutine on all workers.
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.set_metadata(key, value) Set arbitrary metadata in the scheduler
Client.shutdown() Shut down the connected scheduler and workers
Client.start(**kwargs) Start scheduler running in separate thread
Client.start_ipython(*args, **kwargs) Deprecated - Method moved to start_ipython_workers
Client.start_ipython_scheduler([magic_name, …]) Start IPython kernel on the scheduler
Client.start_ipython_workers([workers, …]) Start IPython kernels on workers
Client.submit(func, *args[, key, workers, …]) Submit a function application to the scheduler
Client.sync(func, *args[, asynchronous, …])
Client.unpublish_dataset(name, **kwargs) Remove named datasets from scheduler
Client.upload_file(filename, **kwargs) Upload local package to workers
Client.wait_for_workers([n_workers, timeout]) Blocking call to wait for n workers before continuing
Client.who_has([futures]) The workers storing each future’s data
Client.write_scheduler_file(scheduler_file) Write the scheduler information to a json file.
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.release([_in_destructor])
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

Event([name, client]) Distributed Centralized Event equivalent to asyncio.Event
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
wait(fs[, timeout, return_when]) Wait until all/any futures are finished
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
get_task_metadata() Collect task metadata 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=None, connection_limit=512, **kwargs)[source]

Connect to and submit computation to a Dask cluster

The Client connects users to a Dask cluster. It provides an asynchronous user interface around functions and futures. This class resembles executors in concurrent.futures but also allows Future objects within submit/map calls. When a Client is instantiated it takes over all dask.compute and dask.persist calls by default.

It is also common to create a Client without specifying the scheduler address , like Client(). In this case the Client creates a LocalCluster in the background and connects to that. Any extra keywords are passed from Client to LocalCluster in this case. See the LocalCluster documentation for more information.

Parameters:
address: string, or Cluster

This can be the address of a Scheduler server like a string '127.0.0.1:8786' or a cluster object like LocalCluster()

timeout: int

Timeout duration for initial connection to the scheduler

set_as_default: bool (True)

Claim this scheduler as the global dask scheduler

scheduler_file: string (optional)

Path to a file with scheduler information if available

security: Security or bool, optional

Optional security information. If creating a local cluster can also pass in True, in which case temporary self-signed credentials will be created automatically.

asynchronous: bool (False by default)

Set to True if using this client within async/await functions or within Tornado gen.coroutines. Otherwise this should remain False for normal use.

name: string (optional)

Gives the client a name that will be included in logs generated on the scheduler for matters relating to this client

direct_to_workers: bool (optional)

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

heartbeat_interval: int

Time in milliseconds between heartbeats to scheduler

**kwargs:

If you do not pass a scheduler address, Client will create a LocalCluster object, passing any extra keyword arguments.

Examples

Provide cluster’s scheduler node address on initialization:

>>> client = Client('127.0.0.1:8786')  # doctest: +SKIP

Use submit method to send individual computations to the cluster

>>> a = client.submit(add, 1, 2)  # doctest: +SKIP
>>> b = client.submit(add, 10, 20)  # doctest: +SKIP

Continue using submit or map on results to build up larger computations

>>> c = client.submit(add, a, b)  # doctest: +SKIP

Gather results with the gather method.

>>> client.gather(c)  # doctest: +SKIP
33

You can also call Client with no arguments in order to create your own local cluster.

>>> client = Client()  # makes your own local "cluster" # doctest: +SKIP

Extra keywords will be passed directly to LocalCluster

>>> client = Client(processes=False, threads_per_worker=1)  # doctest: +SKIP
as_current()[source]

Thread-local, Task-local context manager that causes the Client.current class method to return self. Any Future objects deserialized inside this context manager will be automatically attached to this Client.

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()  # doctest: +SKIP
>>> client.call_stack(df)  # call on collections
>>> client.call_stack()  # Or call with no arguments for all activity  # doctest: +SKIP
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
static collections_to_dsk(collections, *args, **kwargs)[source]

Convert many collections into a single dask graph, after optimization

compute(collections, sync=False, optimize_graph=True, workers=None, allow_other_workers=False, resources=None, retries=0, priority=0, fifo_timeout='60s', actors=None, traverse=True, **kwargs)[source]

Compute dask collections on cluster

Parameters:
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 : string or iterable of strings

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

allow_other_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 computing a result fails

priority : Number

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

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

Allowed amount of time between calls to consider the same priority

traverse : bool (defaults to True)

By default dask traverses builtin python collections looking for dask objects passed to compute. For large collections this can be expensive. If none of the arguments contain any dask objects, set traverse=False to avoid doing this traversal.

resources : dict (defaults to {})

Defines the resources each instance of this mapped task requires on the worker; e.g. {'GPU': 2}. See worker resources for details on defining resources.

actors : bool or dict (default None)

Whether these tasks should exist on the worker as stateful actors. Specified on a global (True/False) or per-task ({'x': True, 'y': False}) basis. See Actors for additional details.

**kwargs

Options to pass to the graph optimize calls

Returns:
List of Futures if input is a sequence, or a single future otherwise

See also

Client.get
Normal synchronous dask.get function

Examples

>>> from dask import delayed
>>> from operator import add
>>> x = delayed(add)(1, 2)
>>> y = delayed(add)(x, x)
>>> xx, yy = client.compute([x, y])  # doctest: +SKIP
>>> xx  # doctest: +SKIP
<Future: status: finished, key: add-8f6e709446674bad78ea8aeecfee188e>
>>> xx.result()  # doctest: +SKIP
3
>>> yy.result()  # doctest: +SKIP
6

Also support single arguments

>>> xx = client.compute(x)  # doctest: +SKIP
classmethod current(allow_global=True)[source]

When running within the context of as_client, return the context-local current client. Otherwise, return the latest initialised Client. If no Client instances exist, raise ValueError. If allow_global is set to False, raise ValueError if running outside of the as_client context manager.

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

Gather futures from distributed memory

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

Parameters:
futures : Collection of futures

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

errors : string

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

direct : boolean

Whether or not to connect directly to the workers, or to ask the scheduler to serve as intermediary. This can also be set when creating the Client.

Returns:
results: a collection of the same type as the input, but now with
gathered results rather than futures

See also

Client.scatter
Send data out to cluster

Examples

>>> from operator import add  # doctest: +SKIP
>>> c = Client('127.0.0.1:8787')  # doctest: +SKIP
>>> x = c.submit(add, 1, 2)  # doctest: +SKIP
>>> c.gather(x)  # doctest: +SKIP
3
>>> c.gather([x, [x], x])  # support lists and dicts # doctest: +SKIP
[3, [3], 3]
get(dsk, keys, workers=None, allow_other_workers=None, resources=None, sync=True, asynchronous=None, direct=None, retries=None, priority=0, fifo_timeout='60s', actors=None, **kwargs)[source]

Compute dask graph

Parameters:
dsk : dict
keys : object, or nested lists of objects
workers : string or iterable of strings

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

allow_other_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 computing a result fails

priority : Number

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

resources : dict (defaults to {})

Defines the resources each instance of this mapped task requires on the worker; e.g. {'GPU': 2}. See worker resources for details on defining resources.

sync : bool (optional)

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

direct : bool

Whether or not to connect directly to the workers, or to ask the scheduler to serve as intermediary. This can also be set when creating the Client.

See also

Client.compute
Compute asynchronous collections

Examples

>>> from operator import add  # doctest: +SKIP
>>> c = Client('127.0.0.1:8787')  # doctest: +SKIP
>>> c.get({'x': (add, 1, 2)}, 'x')  # doctest: +SKIP
3
get_dataset(name, default='_no_default_', **kwargs)[source]

Get named dataset from the scheduler if present. Return the default or raise a KeyError if not present.

Parameters:
name : name of the dataset to retrieve
default : optional, not set by default

If set, do not raise a KeyError if the name is not present but return this default

kwargs : dict

additional arguments to _get_dataset

get_events(topic: str = None)[source]

Retrieve structured topic logs

Parameters:
topic : str, optional

Name of topic log to retrieve events for. If no topic is provided, then logs for all topics will be returned.

get_executor(**kwargs)[source]

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

Parameters:
**kwargs

Any submit()- or map()- compatible arguments, such as workers or resources.

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

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', bokeh_resources=None)[source]

Get task stream data from scheduler

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

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

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

bokeh_resources : bokeh.resources.Resources (optional)

Specifies if the resource component is INLINE or CDN

Returns:
L: List[Dict]

See also

get_task_stream
a context manager version of this method

Examples

>>> client.get_task_stream()  # prime plugin if not already connected
>>> x.compute()  # do some work
>>> client.get_task_stream()
[{'task': ...,
  'type': ...,
  'thread': ...,
  ...}]

Pass the plot=True 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()  # doctest: +SKIP
>>> c.get_versions(packages=['sklearn', 'geopandas'])  # doctest: +SKIP
get_worker_logs(n=None, workers=None, nanny=False)[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 retrieve. Gets all workers by default.

nanny : bool, default False

Whether to get the logs from the workers (False) or the nannies (True). If specified, the addresses in workers should still be the worker addresses, not the nanny addresses.

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

Which keys are held by which workers

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

Parameters:
workers : list (optional)

A list of worker addresses, defaults to all

Examples

>>> x, y, z = c.map(inc, [1, 2, 3])  # doctest: +SKIP
>>> wait([x, y, z])  # doctest: +SKIP
>>> c.has_what()  # doctest: +SKIP
{'192.168.1.141:46784': ['inc-1c8dd6be1c21646c71f76c16d09304ea',
                         'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b',
                         'inc-1e297fc27658d7b67b3a758f16bcf47a']}
list_datasets(**kwargs)[source]

List named datasets available on the scheduler

log_event(topic, msg)[source]

Log an event under a given topic

Parameters:
topic : str, list

Name of the topic under which to log an event. To log the same event under multiple topics, pass a list of topic names.

msg

Event message to log. Note this must be msgpack serializable.

Examples

>>> from time import time
>>> client.log_event("current-time", time())
map(func, *iterables, key=None, workers=None, retries=None, resources=None, priority=0, allow_other_workers=False, fifo_timeout='100 ms', actor=False, actors=False, pure=None, batch_size=None, **kwargs)[source]

Map a function on a sequence of arguments

Arguments can be normal objects or Futures

Parameters:
func : callable
iterables : Iterables

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

key : str, list

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

pure : bool (defaults to True)

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

workers : string or iterable of strings

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

allow_other_workers : bool (defaults to False)

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

retries : int (default to 0)

Number of allowed automatic retries if a task fails

priority : Number

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

fifo_timeout : str timedelta (default ‘100ms’)

Allowed amount of time between calls to consider the same priority

resources : dict (defaults to {})

Defines the resources each instance of this mapped task requires on the worker; e.g. {'GPU': 2}. See worker resources for details on defining resources.

actor : bool (default False)

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

actors : bool (default False)

Alias for actor

batch_size : int, optional

Submit tasks to the scheduler in batches of (at most) batch_size. Larger batch sizes can be useful for very large iterables, as the cluster can start processing tasks while later ones are submitted asynchronously.

**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)  # doctest: +SKIP
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])  # doctest: +SKIP
>>> c.nbytes(summary=False)  # doctest: +SKIP
{'inc-1c8dd6be1c21646c71f76c16d09304ea': 28,
 'inc-1e297fc27658d7b67b3a758f16bcf47a': 28,
 'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b': 28}
>>> c.nbytes(summary=True)  # doctest: +SKIP
{'inc': 84}
ncores(workers=None, **kwargs)

The number of threads/cores available on each worker node

Parameters:
workers : list (optional)

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

Examples

>>> c.threads()  # doctest: +SKIP
{'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  # doctest: +SKIP
100
>>> set(client.futures).intersection(x.__dask_graph__())  # some overlap exists  # doctest: +SKIP
10
>>> x = client.normalize_collection(x)  # doctest: +SKIP
>>> len(x.__dask_graph__())  # smaller computational graph  # doctest: +SKIP
20
nthreads(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.threads()  # doctest: +SKIP
{'192.168.1.141:46784': 8,
 '192.167.1.142:47548': 8,
 '192.167.1.143:47329': 8,
 '192.167.1.144:37297': 8}
persist(collections, optimize_graph=True, workers=None, allow_other_workers=None, resources=None, retries=None, priority=0, fifo_timeout='60s', actors=None, **kwargs)[source]

Persist dask collections on cluster

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

Parameters:
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 : string or iterable of strings

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

allow_other_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 computing a result fails

priority : Number

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

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

Allowed amount of time between calls to consider the same priority

resources : dict (defaults to {})

Defines the resources each instance of this mapped task requires on the worker; e.g. {'GPU': 2}. See worker resources for details on defining resources.

actors : bool or dict (default None)

Whether these tasks should exist on the worker as stateful actors. Specified on a global (True/False) or per-task ({'x': True, 'y': False}) basis. See Actors for additional details.

**kwargs

Options to pass to the graph optimize calls

Returns:
List of collections, or single collection, depending on type of input.

See also

Client.compute

Examples

>>> xx = client.persist(x)  # doctest: +SKIP
>>> xx, yy = client.persist([x, y])  # doctest: +SKIP
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])  # doctest: +SKIP
>>> c.processing()  # doctest: +SKIP
{'192.168.1.141:46784': ['inc-1c8dd6be1c21646c71f76c16d09304ea',
                         'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b',
                         'inc-1e297fc27658d7b67b3a758f16bcf47a']}
profile(key=None, start=None, stop=None, workers=None, merge_workers=True, plot=False, filename=None, server=False, scheduler=False)[source]

Collect statistical profiling information about recent work

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

server : bool

If true, return the profile of the worker’s administrative thread rather than the worker threads. This is useful when profiling Dask itself, rather than user code.

scheduler : bool

If true, return the profile information from the scheduler’s administrative thread rather than the workers. This is useful when profiling Dask’s scheduling itself.

plot : boolean or string

Whether or not to return a plot object

filename : str

Filename to save the plot

Examples

>>> client.profile()  # call on collections
>>> client.profile(filename='dask-profile.html')  # save to html file
publish_dataset(*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
override : bool (optional, default False)

if true, override any already present dataset with the same name

kwargs : dict

named collections to publish on the scheduler

Returns:
None

Examples

Publishing client:

>>> df = dd.read_csv('s3://...')  # doctest: +SKIP
>>> df = c.persist(df) # doctest: +SKIP
>>> c.publish_dataset(my_dataset=df)  # doctest: +SKIP

Alternative invocation >>> c.publish_dataset(df, name=’my_dataset’)

Receiving client:

>>> c.list_datasets()  # doctest: +SKIP
['my_dataset']
>>> df2 = c.get_dataset('my_dataset')  # doctest: +SKIP
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 is 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

register_worker_plugin(plugin=None, name=None, **kwargs)[source]

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

This registers a new object to handle setup, task state transitions and teardown for workers in this cluster. The plugin will instantiate itself on all currently connected workers. It will also be run on any worker that connects in the future.

The plugin may include methods setup, teardown, transition, release_key, and release_dep. See the dask.distributed.WorkerPlugin class or the examples below for the interface and docstrings. It must be serializable with the pickle or cloudpickle modules.

If the plugin has a name attribute, or if the name= keyword is used then that will control idempotency. If a plugin with that name has already been registered then any future plugins will not run.

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

Parameters:
plugin : WorkerPlugin

The plugin object to pass to the workers

name : str, optional

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

**kwargs : optional

If you pass a class as the plugin, instead of a class instance, then the class will be instantiated with any extra keyword arguments.

See also

distributed.WorkerPlugin

Examples

>>> class MyPlugin(WorkerPlugin):
...     def __init__(self, *args, **kwargs):
...         pass  # the constructor is up to you
...     def setup(self, worker: dask.distributed.Worker):
...         pass
...     def teardown(self, worker: dask.distributed.Worker):
...         pass
...     def transition(self, key: str, start: str, finish: str, **kwargs):
...         pass
...     def release_key(self, key: str, state: str, cause: Optional[str], reason: None, report: bool):
...         pass
...     def release_dep(self, dep: str, state: str, report: bool):
...         pass
>>> plugin = MyPlugin(1, 2, 3)
>>> client.register_worker_plugin(plugin)

You can get access to the plugin with the get_worker function

>>> client.register_worker_plugin(other_plugin, name='my-plugin')
>>> def f():
...    worker = get_worker()
...    plugin = worker.plugins['my-plugin']
...    return plugin.my_state
>>> future = client.run(f)
replicate(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)  # doctest: +SKIP
>>> c.replicate([x])  # send to all workers  # doctest: +SKIP
>>> c.replicate([x], n=3)  # send to three workers  # doctest: +SKIP
>>> c.replicate([x], workers=['alice', 'bob'])  # send to specific  # doctest: +SKIP
>>> c.replicate([x], n=1, workers=['alice', 'bob'])  # send to one of specific workers  # doctest: +SKIP
>>> c.replicate([x], n=1)  # reduce replications # doctest: +SKIP
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.

wait : boolean (optional)

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

nanny : bool, defualt False

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

Examples

>>> c.run(os.getpid)  # doctest: +SKIP
{'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'])  # doctest: +SKIP
{'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)  # doctest: +SKIP
{'192.168.0.100:9000': 'running',
 '192.168.0.101:9000': 'running}

Run asynchronous functions in the background:

>>> async def print_state(dask_worker):  # doctest: +SKIP
...    while True:
...        print(dask_worker.status)
...        await asyncio.sleep(1)
>>> c.run(print_state, wait=False)  # doctest: +SKIP
run_coroutine(function, *args, **kwargs)[source]

Spawn a coroutine on all workers.

This spawns 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)  # doctest: +SKIP
100

Run asynchronous functions in the background:

>>> async def print_state(dask_scheduler):  # doctest: +SKIP
...    while True:
...        print(dask_scheduler.status)
...        await asyncio.sleep(1)
>>> c.run(print_state, wait=False)  # doctest: +SKIP
scatter(data, workers=None, broadcast=False, direct=None, hash=True, 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, dict, or object

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

workers : list of tuples (optional)

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

broadcast : bool (defaults to False)

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

direct : bool (defaults to automatically check)

Whether or not to connect directly to the workers, or to ask the scheduler to serve as intermediary. This can also be set when creating the Client.

hash : bool (optional)

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

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

See also

Client.gather
Gather data back to local process

Examples

>>> c = Client('127.0.0.1:8787')  # doctest: +SKIP
>>> c.scatter(1) # doctest: +SKIP
<Future: status: finished, key: c0a8a20f903a4915b94db8de3ea63195>
>>> c.scatter([1, 2, 3])  # doctest: +SKIP
[<Future: status: finished, key: c0a8a20f903a4915b94db8de3ea63195>,
 <Future: status: finished, key: 58e78e1b34eb49a68c65b54815d1b158>,
 <Future: status: finished, key: d3395e15f605bc35ab1bac6341a285e2>]
>>> c.scatter({'x': 1, 'y': 2, 'z': 3})  # doctest: +SKIP
{'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)])   # doctest: +SKIP

Broadcast data to all workers

>>> [future] = c.scatter([element], broadcast=True)  # doctest: +SKIP

Send scattered data to parallelized function using client futures interface

>>> data = c.scatter(data, broadcast=True)  # doctest: +SKIP
>>> 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()  # doctest: +SKIP
{'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)  # doctest: +SKIP
>>> client.get_metadata('x')  # doctest: +SKIP
123
>>> client.set_metadata(['x', 'y'], 123)  # doctest: +SKIP
>>> client.get_metadata('x')  # doctest: +SKIP
{'y': 123}
>>> client.set_metadata(['x', 'w', 'z'], 456)  # doctest: +SKIP
>>> client.get_metadata('x')  # doctest: +SKIP
{'y': 123, 'w': {'z': 456}}
>>> client.get_metadata(['x', 'w'])  # doctest: +SKIP
{'z': 456}
shutdown()[source]

Shut down the connected scheduler and workers

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

See also

Client.close
close only this client
start(**kwargs)[source]

Start scheduler running in separate thread

start_ipython(*args, **kwargs)[source]

Deprecated - Method moved to start_ipython_workers

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() # doctest: +SKIP
>>> %scheduler scheduler.processing  # doctest: +SKIP
{'127.0.0.1:3595': {'inc-1', 'inc-2'},
 '127.0.0.1:53589': {'inc-2', 'add-5'}}
>>> c.start_ipython_scheduler(qtconsole=True) # doctest: +SKIP
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() # doctest: +SKIP
>>> %remote info['192.168.1.101:5752'] worker.data  # doctest: +SKIP
{'x': 1, 'y': 100}
>>> c.start_ipython_workers('192.168.1.101:5752', magic_names='w') # doctest: +SKIP
>>> %w worker.data  # doctest: +SKIP
{'x': 1, 'y': 100}
>>> c.start_ipython_workers('192.168.1.101:5752', qtconsole=True) # doctest: +SKIP

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

>>> c.start_ipython_workers(magic_names='w_*') # doctest: +SKIP
>>> %w_0 worker.data  # doctest: +SKIP
{'x': 1, 'y': 100}
>>> %w_1 worker.data  # doctest: +SKIP
{'z': 5}
submit(func, *args, key=None, workers=None, resources=None, retries=None, priority=0, fifo_timeout='100 ms', allow_other_workers=False, actor=False, actors=False, pure=None, **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 : string or iterable of strings

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

key : str

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

allow_other_workers : bool (defaults to False)

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

retries : int (default to 0)

Number of allowed automatic retries if the task fails

priority : Number

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

fifo_timeout : str timedelta (default ‘100ms’)

Allowed amount of time between calls to consider the same priority

resources : dict (defaults to {})

Defines the resources each instance of this mapped task requires on the worker; e.g. {'GPU': 2}. See worker resources for details on defining resources.

actor : bool (default False)

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

actors : bool (default False)

Alias for actor

Returns:
Future

See also

Client.map
Submit on many arguments at once

Examples

>>> c = client.submit(add, a, b)  # doctest: +SKIP
unpublish_dataset(name, **kwargs)[source]

Remove named datasets from scheduler

Examples

>>> c.list_datasets()  # doctest: +SKIP
['my_dataset']
>>> c.unpublish_datasets('my_dataset')  # doctest: +SKIP
>>> c.list_datasets()  # doctest: +SKIP
[]
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')  # doctest: +SKIP
>>> from mylibrary import myfunc  # doctest: +SKIP
>>> L = client.map(myfunc, seq)  # doctest: +SKIP
wait_for_workers(n_workers=0, timeout=None)[source]

Blocking call to wait for n workers before continuing

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])  # doctest: +SKIP
>>> wait([x, y, z])  # doctest: +SKIP
>>> c.who_has()  # doctest: +SKIP
{'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])  # doctest: +SKIP
{'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()  # doctest: +SKIP
>>> client.write_scheduler_file('scheduler.json')  # doctest: +SKIP
# connect to previous client's scheduler
>>> client2 = Client(scheduler_file='scheduler.json')  # doctest: +SKIP
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)         # doctest: +SKIP
>>> future.status                        # doctest: +SKIP
'error'
>>> c.recreate_error_locally(future)     # doctest: +SKIP
ZeroDivisionError: division by zero

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

>>> %pdb                                 # doctest: +SKIP
Automatic pdb calling has been turned ON
>>> c.recreate_error_locally(future)     # doctest: +SKIP
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)  # doctest: +SKIP

We can track the progress and results of a future

>>> my_future  # doctest: +SKIP
<Future: status: finished, key: add-8f6e709446674bad78ea8aeecfee188e>

We can get the result or the exception and traceback from the future

>>> my_future.result()  # doctest: +SKIP
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  # doctest: +SKIP
>>> tb = future.traceback()  # doctest: +SKIP
>>> traceback.format_tb(tb)  # doctest: +SKIP
[...]

Cluster

Classes relevant for cluster creation and management. Other libraries (like dask-jobqueue, dask-gateway, dask-kubernetes, dask-yarn etc.) provide additional cluster objects.

LocalCluster([name, n_workers, …]) Create local Scheduler and Workers
SpecCluster([workers, scheduler, worker, …]) Cluster that requires a full specification of workers
class distributed.LocalCluster(name=None, n_workers=None, threads_per_worker=None, processes=True, loop=None, start=None, host=None, ip=None, scheduler_port=0, silence_logs=30, dashboard_address=':8787', worker_dashboard_address=None, diagnostics_port=None, services=None, worker_services=None, service_kwargs=None, asynchronous=False, security=None, protocol=None, blocked_handlers=None, interface=None, worker_class=None, scheduler_kwargs=None, **worker_kwargs)[source]

Create local Scheduler and Workers

This creates a “cluster” of a scheduler and workers running on the local machine.

Parameters:
n_workers: int

Number of workers to start

processes: bool

Whether to use processes (True) or threads (False). Defaults to True

threads_per_worker: int

Number of threads per each worker

scheduler_port: int

Port of the scheduler. 8786 by default, use 0 to choose a random port

silence_logs: logging level

Level of logs to print out to stdout. logging.WARN by default. Use a falsey value like False or None for no change.

host: string

Host address on which the scheduler will listen, defaults to only localhost

ip: string

Deprecated. See host above.

dashboard_address: str

Address on which to listen for the Bokeh diagnostics server like ‘localhost:8787’ or ‘0.0.0.0:8787’. Defaults to ‘:8787’. Set to None to disable the dashboard. Use ‘:0’ for a random port.

worker_dashboard_address: str

Address on which to listen for the Bokeh worker diagnostics server like ‘localhost:8787’ or ‘0.0.0.0:8787’. Defaults to None which disables the dashboard. Use ‘:0’ for a random port.

diagnostics_port: int

Deprecated. See dashboard_address.

asynchronous: bool (False by default)

Set to True if using this cluster within async/await functions or within Tornado gen.coroutines. This should remain False for normal use.

blocked_handlers: List[str]

A list of strings specifying a blacklist of handlers to disallow on the Scheduler, like ['feed', 'run_function']

service_kwargs: Dict[str, Dict]

Extra keywords to hand to the running services

security : Security or bool, optional

Configures communication security in this cluster. Can be a security object, or True. If True, temporary self-signed credentials will be created automatically.

protocol: str (optional)

Protocol to use like tcp://, tls://, inproc:// This defaults to sensible choice given other keyword arguments like processes and security

interface: str (optional)

Network interface to use. Defaults to lo/localhost

worker_class: Worker

Worker class used to instantiate workers from.

**worker_kwargs:

Extra worker arguments. Any additional keyword arguments will be passed to the Worker class constructor.

Examples

>>> cluster = LocalCluster()  # Create a local cluster with as many workers as cores  # doctest: +SKIP
>>> cluster  # doctest: +SKIP
LocalCluster("127.0.0.1:8786", workers=8, threads=8)
>>> c = Client(cluster)  # connect to local cluster  # doctest: +SKIP

Scale the cluster to three workers

>>> cluster.scale(3)  # doctest: +SKIP

Pass extra keyword arguments to Bokeh

>>> LocalCluster(service_kwargs={'dashboard': {'prefix': '/foo'}})  # doctest: +SKIP
class distributed.SpecCluster(workers=None, scheduler=None, worker=None, asynchronous=False, loop=None, security=None, silence_logs=False, name=None, shutdown_on_close=True)[source]

Cluster that requires a full specification of workers

The SpecCluster class expects a full specification of the Scheduler and Workers to use. It removes any handling of user inputs (like threads vs processes, number of cores, and so on) and any handling of cluster resource managers (like pods, jobs, and so on). Instead, it expects this information to be passed in scheduler and worker specifications. This class does handle all of the logic around asynchronously cleanly setting up and tearing things down at the right times. Hopefully it can form a base for other more user-centric classes.

Parameters:
workers: dict

A dictionary mapping names to worker classes and their specifications See example below

scheduler: dict, optional

A similar mapping for a scheduler

worker: dict

A specification of a single worker. This is used for any new workers that are created.

asynchronous: bool

If this is intended to be used directly within an event loop with async/await

silence_logs: bool

Whether or not we should silence logging when setting up the cluster.

name: str, optional

A name to use when printing out the cluster, defaults to type name

Examples

To create a SpecCluster you specify how to set up a Scheduler and Workers

>>> from dask.distributed import Scheduler, Worker, Nanny
>>> scheduler = {'cls': Scheduler, 'options': {"dashboard_address": ':8787'}}
>>> workers = {
...     'my-worker': {"cls": Worker, "options": {"nthreads": 1}},
...     'my-nanny': {"cls": Nanny, "options": {"nthreads": 2}},
... }
>>> cluster = SpecCluster(scheduler=scheduler, workers=workers)

The worker spec is stored as the .worker_spec attribute

>>> cluster.worker_spec
{
   'my-worker': {"cls": Worker, "options": {"nthreads": 1}},
   'my-nanny': {"cls": Nanny, "options": {"nthreads": 2}},
}

While the instantiation of this spec is stored in the .workers attribute

>>> cluster.workers
{
    'my-worker': <Worker ...>
    'my-nanny': <Nanny ...>
}

Should the spec change, we can await the cluster or call the ._correct_state method to align the actual state to the specified state.

We can also .scale(...) the cluster, which adds new workers of a given form.

>>> worker = {'cls': Worker, 'options': {}}
>>> cluster = SpecCluster(scheduler=scheduler, worker=worker)
>>> cluster.worker_spec
{}
>>> cluster.scale(3)
>>> cluster.worker_spec
{
    0: {'cls': Worker, 'options': {}},
    1: {'cls': Worker, 'options': {}},
    2: {'cls': Worker, 'options': {}},
}

Note that above we are using the standard Worker and Nanny classes, however in practice other classes could be used that handle resource management like KubernetesPod or SLURMJob. The spec does not need to conform to the expectations of the standard Dask Worker class. It just needs to be called with the provided options, support __await__ and close methods and the worker_address property..

Also note that uniformity of the specification is not required. Other API could be added externally (in subclasses) that adds workers of different specifications into the same dictionary.

If a single entry in the spec will generate multiple dask workers then please provide a “group” element to the spec, that includes the suffixes that will be added to each name (this should be handled by your worker class).

>>> cluster.worker_spec
{
    0: {"cls": MultiWorker, "options": {"processes": 3}, "group": ["-0", "-1", -2"]}
    1: {"cls": MultiWorker, "options": {"processes": 2}, "group": ["-0", "-1"]}
}

These suffixes should correspond to the names used by the workers when they deploy.

>>> [ws.name for ws in cluster.scheduler.workers.values()]
["0-0", "0-1", "0-2", "1-0", "1-1"]
adapt(*args, minimum=0, maximum=inf, minimum_cores: int = None, maximum_cores: int = None, minimum_memory: str = None, maximum_memory: str = None, **kwargs) → distributed.deploy.adaptive.Adaptive[source]

Turn on adaptivity

This scales Dask clusters automatically based on scheduler activity.

Parameters:
minimum : int

Minimum number of workers

maximum : int

Maximum number of workers

minimum_cores : int

Minimum number of cores/threads to keep around in the cluster

maximum_cores : int

Maximum number of cores/threads to keep around in the cluster

minimum_memory : str

Minimum amount of memory to keep around in the cluster Expressed as a string like “100 GiB”

maximum_memory : str

Maximum amount of memory to keep around in the cluster Expressed as a string like “100 GiB”

See also

dask.distributed.Adaptive
for more keyword arguments

Examples

>>> cluster.adapt(minimum=0, maximum_memory="100 GiB", interval='500ms')
classmethod from_name(name: str)[source]

Create an instance of this class to represent an existing cluster by name.

new_worker_spec()[source]

Return name and spec for the next worker

Returns:
d: dict mapping names to worker specs

See also

scale
scale(n=0, memory=None, cores=None)[source]

Scale cluster to n workers

Parameters:
n : int

Target number of workers

Examples

>>> cluster.scale(10)  # scale cluster to ten workers
scale_up(n=0, memory=None, cores=None)

Scale cluster to n workers

Parameters:
n : int

Target number of workers

Examples

>>> cluster.scale(10)  # scale cluster to ten workers

Other

class 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])  # doctest: +SKIP
>>> for future in as_completed([x, y, z]):  # doctest: +SKIP
...     print(future.result())  # doctest: +SKIP
3
2
4

Add more futures during computation

>>> x, y, z = client.map(inc, [1, 2, 3])  # doctest: +SKIP
>>> ac = as_completed([x, y, z])  # doctest: +SKIP
>>> for future in ac:  # doctest: +SKIP
...     print(future.result())  # doctest: +SKIP
...     if random.random() < 0.5:  # doctest: +SKIP
...         ac.add(c.submit(double, future))  # doctest: +SKIP
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)  # doctest: +SKIP
>>> for future, result in ac:  # doctest: +SKIP
...     print(result)  # doctest: +SKIP
2
4
3
add(future)[source]

Add a future to the collection

This future will emit from the iterator once it finishes

batches()[source]

Yield all finished futures at once rather than one-by-one

This returns an iterator of lists of futures or lists of (future, result) tuples rather than individual futures or individual (future, result) tuples. It will yield these as soon as possible without waiting.

Examples

>>> for batch in as_completed(futures).batches():  # doctest: +SKIP
...     results = client.gather(batch)
...     print(results)
[4, 2]
[1, 3, 7]
[5]
[6]
clear()[source]

Clear out all submitted futures

count()[source]

Return the number of futures yet to be returned

This includes both the number of futures still computing, as well as those that are finished, but have not yet been returned from this iterator.

has_ready()[source]

Returns True if there are completed futures available.

is_empty()[source]

Returns True if there no completed or computing futures

next_batch(block=True)[source]

Get the next batch of completed futures.

Parameters:
block : bool, optional

If True then wait until we have some result, otherwise return immediately, even with an empty list. Defaults to True.

Returns:
List of futures or (future, result) tuples

Examples

>>> ac = as_completed(futures)  # doctest: +SKIP
>>> client.gather(ac.next_batch())  # doctest: +SKIP
[4, 1, 3]
>>> client.gather(ac.next_batch(block=False))  # doctest: +SKIP
[]
update(futures)[source]

Add multiple futures to the collection.

The added futures will emit from the iterator once they finish

distributed.diagnostics.progress()
distributed.wait(fs, timeout=None, return_when='ALL_COMPLETED')[source]

Wait until all/any futures are finished

Parameters:
fs : list of futures
timeout : number, optional

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

return_when : str, optional

One of ALL_COMPLETED or FIRST_COMPLETED

Returns:
Named tuple of completed, not completed
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))  # doctest: +SKIP
distributed.futures_of(o, client=None)[source]

Future objects in a collection

Parameters:
o : collection

A possibly nested collection of Dask objects

Returns:
futures : List[Future]

A list of futures held by those collections

Examples

>>> futures_of(my_dask_dataframe)
[<Future: finished key: ...>,
 <Future: pending  key: ...>]
distributed.worker_client(timeout=None, 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 or String

Timeout after which to error out. Defaults to the distributed.comm.timeouts.connect configuration value.

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(timeout="10s") 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)  # doctest: +SKIP
>>> future.result()  # doctest: +SKIP
'tcp://127.0.0.1:47373'
distributed.get_client(address=None, timeout=None, 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 or str

Timeout (in seconds) for getting the Client. Defaults to the distributed.comm.timeouts.connect configuration value.

resolve_address : bool, default True

Whether to resolve address to its canonical form.

Returns:
Client

Examples

>>> def f():
...     client = get_client(timeout="10s")
...     futures = client.map(lambda x: x + 1, range(10))  # spawn many tasks
...     results = client.gather(futures)
...     return sum(results)
>>> future = client.submit(f)  # doctest: +SKIP
>>> future.result()  # doctest: +SKIP
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.Reschedule[source]

Reschedule this task

Raising this exception will stop the current execution of the task and ask the scheduler to reschedule this task, possibly on a different machine.

This does not guarantee that the task will move onto a different machine. The scheduler will proceed through its normal heuristics to determine the optimal machine to accept this task. The machine will likely change if the load across the cluster has significantly changed since first scheduling the task.

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://raw.githack.com

$ python -m 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://raw.githack.com . This process should provide a sharable link that others can use to see your task stream plot.

class distributed.get_task_metadata[source]

Collect task metadata within a context block

This gathers TaskState metadata and final state from the scheduler for tasks which are submitted and finished within the scope of this context manager.

Examples

>>> with get_task_metadata() as tasks:
...     x.compute()
>>> tasks.metadata
{...}
>>> tasks.state
{...}
class distributed.Event(name=None, client=None)[source]

Distributed Centralized Event equivalent to asyncio.Event

An event stores a single flag, which is set to false on start. The flag can be set to true (using the set() call) or back to false (with the clear() call). Every call to wait() blocks until the event flag is set to true.

Parameters:
name: string (optional)

Name of the event. Choosing the same name allows two disconnected processes to coordinate an event. If not given, a random name will be generated.

client: Client (optional)

Client to use for communication with the scheduler. If not given, the default global client will be used.

Examples

>>> event_1 = Event('a')  # doctest: +SKIP
>>> event_1.wait(timeout=1)  # doctest: +SKIP
>>> # in another process
>>> event_2 = Event('a')  # doctest: +SKIP
>>> event_2.set() # doctest: +SKIP
>>> # now event_1 will stop waiting
clear()[source]

Clear the event (set its flag to false).

All waiters will now block.

is_set()[source]

Check if the event is set

set()[source]

Set the event (set its flag to false).

All waiters will now be released.

wait(timeout=None)[source]

Wait until the event is set.

Parameters:
timeout : number or string or timedelta, optional

Seconds to wait on the event in the scheduler. This does not include local coroutine time, network transfer time, etc.. Instead of number of seconds, it is also possible to specify a timedelta in string format, e.g. “200ms”.

Returns:
True if the event was set of false, if a timeout happend

Examples

>>> event = Event('a')  # doctest: +SKIP
>>> event.wait(timeout="1s")  # doctest: +SKIP
class distributed.Lock(name=None, client=None)[source]

Distributed Centralized Lock

Parameters:
name: string (optional)

Name of the lock to acquire. Choosing the same name allows two disconnected processes to coordinate a lock. If not given, a random name will be generated.

client: Client (optional)

Client to use for communication with the scheduler. If not given, the default global client will be used.

Examples

>>> lock = Lock('x')  # doctest: +SKIP
>>> lock.acquire(timeout=1)  # doctest: +SKIP
>>> # do things with protected resource
>>> lock.release()  # doctest: +SKIP
acquire(blocking=True, timeout=None)[source]

Acquire the lock

Parameters:
blocking : bool, optional

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

timeout : string or number or timedelta, optional

Seconds to wait on the lock in the scheduler. This does not include local coroutine time, network transfer time, etc.. It is forbidden to specify a timeout when blocking is false. Instead of number of seconds, it is also possible to specify a timedelta in string format, e.g. “200ms”.

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

Examples

>>> lock = Lock('x')  # doctest: +SKIP
>>> lock.acquire(timeout="1s")  # doctest: +SKIP
release()[source]

Release the lock if already acquired

class distributed.Semaphore(max_leases=1, name=None, register=True, scheduler_rpc=None, loop=None)[source]

This semaphore will track leases on the scheduler which can be acquired and released by an instance of this class. If the maximum amount of leases are already acquired, it is not possible to acquire more and the caller waits until another lease has been released.

The lifetime or leases are controlled using a timeout. This timeout is refreshed in regular intervals by the Client of this instance and provides protection from deadlocks or resource starvation in case of worker failure. The timeout can be controlled using the configuration option distributed.scheduler.locks.lease-timeout and the interval in which the scheduler verifies the timeout is set using the option distributed.scheduler.locks.lease-validation-interval.

A noticeable difference to the Semaphore of the python standard library is that this implementation does not allow to release more often than it was acquired. If this happens, a warning is emitted but the internal state is not modified.

Warning

This implementation is still in an experimental state and subtle changes in behavior may occur without any change in the major version of this library.

Warning

This implementation is susceptible to lease overbooking in case of lease timeouts. It is advised to monitor log information and adjust above configuration options to suitable values for the user application.

Parameters:
max_leases: int (optional)

The maximum amount of leases that may be granted at the same time. This effectively sets an upper limit to the amount of parallel access to a specific resource. Defaults to 1.

name: string (optional)

Name of the semaphore to acquire. Choosing the same name allows two disconnected processes to coordinate. If not given, a random name will be generated.

register: bool

If True, register the semaphore with the scheduler. This needs to be done before any leases can be acquired. If not done during initialization, this can also be done by calling the register method of this class. When registering, this needs to be awaited.

scheduler_rpc: ConnectionPool

The ConnectionPool to connect to the scheduler. If None is provided, it uses the worker or client pool. This paramter is mostly used for testing.

loop: IOLoop

The event loop this instance is using. If None is provided, reuse the loop of the active worker or client.

Notes

If a client attempts to release the semaphore but doesn’t have a lease acquired, this will raise an exception.

When a semaphore is closed, if, for that closed semaphore, a client attempts to:

  • Acquire a lease: an exception will be raised.
  • Release: a warning will be logged.
  • Close: nothing will happen.

dask executes functions by default assuming they are pure, when using semaphore acquire/releases inside such a function, it must be noted that there are in fact side-effects, thus, the function can no longer be considered pure. If this is not taken into account, this may lead to unexpected behavior.

Examples

>>> from distributed import Semaphore
... sem = Semaphore(max_leases=2, name='my_database')
...
... def access_resource(s, sem):
...     # This automatically acquires a lease from the semaphore (if available) which will be
...     # released when leaving the context manager.
...     with sem:
...         pass
...
... futures = client.map(access_resource, range(10), sem=sem)
... client.gather(futures)
... # Once done, close the semaphore to clean up the state on scheduler side.
... sem.close()
acquire(timeout=None)[source]

Acquire a semaphore.

If the internal counter is greater than zero, decrement it by one and return True immediately. If it is zero, wait until a release() is called and return True.

Parameters:
timeout : number or string or timedelta, optional

Seconds to wait on acquiring the semaphore. This does not include local coroutine time, network transfer time, etc.. Instead of number of seconds, it is also possible to specify a timedelta in string format, e.g. “200ms”.

get_value()[source]

Return the number of currently registered leases.

release()[source]

Release the semaphore.

Returns:
bool

This value indicates whether a lease was released immediately or not. Note that a user should not retry this operation. Under certain circumstances (e.g. scheduler overload) the lease may not be released immediately, but it will always be automatically released after a specific interval configured using “distributed.scheduler.locks.lease-validation-interval” and “distributed.scheduler.locks.lease-timeout”.

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

Parameters:
name: string (optional)

Name used by other clients and the scheduler to identify the queue. If not given, a random name will be generated.

client: Client (optional)

Client used for communication with the scheduler. If not given, the default global client will be used.

maxsize: int (optional)

Number of items allowed in the queue. If 0 (the default), the queue size is unbounded.

See also

Variable
shared variable between clients

Examples

>>> from dask.distributed import Client, Queue  # doctest: +SKIP
>>> client = Client()  # doctest: +SKIP
>>> queue = Queue('x')  # doctest: +SKIP
>>> future = client.submit(f, x)  # doctest: +SKIP
>>> queue.put(future)  # doctest: +SKIP
get(timeout=None, batch=False, **kwargs)[source]

Get data from the queue

Parameters:
timeout : number or string or timedelta, optional

Time in seconds to wait before timing out. Instead of number of seconds, it is also possible to specify a timedelta in string format, e.g. “200ms”.

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

Parameters:
timeout : number or string or timedelta, optional

Time in seconds to wait before timing out. Instead of number of seconds, it is also possible to specify a timedelta in string format, e.g. “200ms”.

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

Parameters:
name: string (optional)

Name used by other clients and the scheduler to identify the variable. If not given, a random name will be generated.

client: Client (optional)

Client used for communication with the scheduler. If not given, the default global client will be used.

See also

Queue
shared multi-producer/multi-consumer queue between clients

Examples

>>> from dask.distributed import Client, Variable # doctest: +SKIP
>>> client = Client()  # doctest: +SKIP
>>> x = Variable('x')  # doctest: +SKIP
>>> x.set(123)  # docttest: +SKIP
>>> x.get()  # docttest: +SKIP
123
>>> future = client.submit(f, x)  # doctest: +SKIP
>>> x.set(future)  # doctest: +SKIP
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

Parameters:
timeout : number or string or timedelta, optional

Time in seconds to wait before timing out. Instead of number of seconds, it is also possible to specify a timedelta in string format, e.g. “200ms”.

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

Adaptive

class distributed.deploy.Adaptive(cluster=None, interval=None, minimum=None, maximum=None, wait_count=None, target_duration=None, worker_key=None, **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:
cluster: object

Must have scale and scale_down methods/coroutines

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.

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.target() 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 Scheduler.adaptive_target()). The values for interval, min, max, wait_count and target_duration can be specified in the dask config under the distributed.adaptive key.

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)
recommendations(target: int) → dict[source]

Make scale up/down recommendations based on current state and target

target()[source]

Determine target number of workers that should exist.

Returns:
Target number of workers

See also

Scheduler.adaptive_target

Notes

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

workers_to_close(target: int)[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.