API

Client

The client connects to and submits computation to a Dask cluster (such as a distributed.LocalCluster)

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

Run a benchmark on the workers for memory, disk, and network bandwidths

Client.call_stack([futures, keys])

The actively running call stack of all relevant keys

Client.cancel(futures[, asynchronous, force])

Cancel running futures This stops future tasks from being scheduled if they have not yet run and deletes them if they have already run.

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.dump_cluster_state([filename, ...])

Extract a dump of the entire cluster state and persist to disk or a URL.

Client.forward_logging([logger_name, level])

Begin forwarding the given logger (by default the root) and all loggers under it from worker tasks to the client process.

Client.futures_of(futures)

Wrapper method of futures_of

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_plugin(plugin[, name, ...])

Register a plugin.

Client.register_scheduler_plugin(plugin[, ...])

Register a scheduler plugin.

Client.register_worker_callbacks([setup])

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

Client.register_worker_plugin(plugin[, ...])

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

Client.replicate(futures[, n, workers, ...])

Set replication of futures within network

Client.restart([timeout, wait_for_workers])

Restart all workers.

Client.restart_workers(workers[, timeout, ...])

Restart a specified set of workers

Client.retire_workers([workers, close_workers])

Retire certain workers on the scheduler

Client.retry(futures[, asynchronous])

Retry failed futures

Client.run(function, *args[, workers, wait, ...])

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.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.story(*keys_or_stimuli[, on_error])

Returns a cluster-wide story for the given keys or stimulus_id's

Client.submit(func, *args[, key, workers, ...])

Submit a function application to the scheduler

Client.subscribe_topic(topic, handler)

Subscribe to a topic and execute a handler for every received event

Client.unforward_logging([logger_name])

Stop forwarding the given logger (default root) from worker tasks to the client process.

Client.unpublish_dataset(name, **kwargs)

Remove named datasets from scheduler

Client.unregister_scheduler_plugin(name)

Unregisters a scheduler plugin

Client.unregister_worker_plugin(name[, nanny])

Unregisters a lifecycle worker plugin

Client.unsubscribe_topic(topic)

Unsubscribe from a topic and remove event handler

Client.upload_file(filename[, load])

Upload local package to scheduler and 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

print(*args[, sep, end, file, flush])

A drop-in replacement of the built-in print function for remote printing from workers to clients.

warn(message[, category, stacklevel, source])

A drop-in replacement of the built-in warnings.warn() function for issuing warnings remotely from workers to clients.

Reschedule

Reschedule this task

ReplayTaskClient.recreate_task_locally(future)

For any calculation, whether it succeeded or failed, perform the task locally for debugging.

ReplayTaskClient.recreate_error_locally(future)

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

Future

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

A remotely running computation

Future.add_done_callback(fn)

Call callback on future when future has finished

Future.cancel(**kwargs)

Cancel the request to run this future

Future.cancelled()

Returns True if the future has been cancelled

Future.done()

Returns whether or not the computation completed.

Future.exception([timeout])

Return the exception of a failed task

Future.make_future(key, id)

Future.release()

Notes

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

Synchronization

Event([name, client])

Distributed Centralized Event equivalent to asyncio.Event

Lock([name, client])

Distributed Centralized Lock

MultiLock([names, client])

Distributed Centralized Lock

Semaphore([max_leases, name, register, ...])

This semaphore will track leases on the scheduler which can be acquired and released by an instance of this class.

Queue([name, client, maxsize])

Distributed Queue

Variable([name, client])

Distributed Global Variable

Other

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

Return futures in the order in which they complete

distributed.diagnostics.progressbar.progress(...)

Track progress of futures

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

performance_report([filename, stacklevel, ...])

Gather performance report

Utilities

distributed.utils.Log

A container for newline-delimited string of log entries

distributed.utils.Logs

A container for a dict mapping names to strings of log entries

distributed.diagnostics.memray.memray_scheduler([...])

Generate a Memray profile on the Scheduler and download the generated report.

distributed.diagnostics.memray.memray_workers([...])

Generate a Memray profile on the workers and download the generated report.

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=_NoDefault.no_default, set_as_default=True, scheduler_file=None, security=None, asynchronous=False, name=None, heartbeat_interval=None, serializers=None, deserializers=None, extensions={'pubsub': <class 'distributed.pubsub.PubSubClientExtension'>}, direct_to_workers=None, connection_limit=512, **kwargs)[source]

Connect to and submit computation to a Dask cluster

The Client connects users to a Dask cluster. It provides an asynchronous user interface around functions and futures. This class resembles executors in concurrent.futures but also 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()

loop

The event loop

timeout: int (defaults to configuration ``distributed.comm.timeouts.connect``)

Timeout duration for initial connection to the scheduler

set_as_default: bool (True)

Use this Client as the global dask scheduler

scheduler_file: string (optional)

Path to a file with scheduler information if available

security: Security or bool, optional

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

asynchronous: bool (False by default)

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

name: string (optional)

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

heartbeat_interval: int (optional)

Time in milliseconds between heartbeats to scheduler

serializers

Iterable of approaches to use when serializing the object. See Serialization for more.

deserializers

Iterable of approaches to use when deserializing the object. See Serialization for more.

extensionslist

The extensions

direct_to_workers: bool (optional)

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

connection_limitint

The number of open comms to maintain at once in the connection pool

**kwargs:

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

Examples

Provide cluster’s scheduler node address on initialization:

>>> client = Client('127.0.0.1:8786')  

Use submit method to send individual computations to the cluster

>>> a = client.submit(add, 1, 2)  
>>> b = client.submit(add, 10, 20)  

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

>>> c = client.submit(add, a, b)  

Gather results with the gather method.

>>> client.gather(c)  
33

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

>>> client = Client()  # makes your own local "cluster" 

Extra keywords will be passed directly to LocalCluster

>>> client = Client(n_workers=2, threads_per_worker=4)  
property amm

Convenience accessors for the Active Memory Manager

as_current()[source]

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

benchmark_hardware() dict[source]

Run a benchmark on the workers for memory, disk, and network bandwidths

Returns
result: dict

A dictionary mapping the names “disk”, “memory”, and “network” to dictionaries mapping sizes to bandwidths. These bandwidths are averaged over many workers running computations across the cluster.

call_stack(futures=None, keys=None)[source]

The actively running call stack of all relevant keys

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

Parameters
futureslist (optional)

List of futures, defaults to all data

keyslist (optional)

List of key names, defaults to all data

Examples

>>> df = dd.read_parquet(...).persist()  
>>> client.call_stack(df)  # call on collections
>>> client.call_stack()  # Or call with no arguments for all activity  
cancel(futures, asynchronous=None, force=False)[source]

Cancel running futures This stops future tasks from being scheduled if they have not yet run and deletes them if they have already run. After calling, this result and all dependent results will no longer be accessible

Parameters
futuresList[Future]

The list of Futures

asynchronous: bool

If True the client is in asynchronous mode

forceboolean (False)

Cancel this future even if other clients desire it

close(timeout=_NoDefault.no_default)[source]

Close this client

Clients will also close automatically when your Python session ends

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

Parameters
timeoutnumber

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

See also

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
collectionsiterable of dask objects or single dask object

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

syncbool (optional)

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

optimize_graphbool

Whether or not to optimize the underlying graphs

workersstring or iterable of strings

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

allow_other_workersbool (defaults to False)

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

retriesint (default to 0)

Number of allowed automatic retries if computing a result fails

priorityNumber

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

fifo_timeouttimedelta str (defaults to ’60s’)

Allowed amount of time between calls to consider the same priority

traversebool (defaults to True)

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

resourcesdict (defaults to {})

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

actorsbool or dict (default None)

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

**kwargs

Options to pass to the graph optimize calls

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

See also

Client.get

Normal synchronous dask.get function

Examples

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

Also support single arguments

>>> xx = client.compute(x)  
classmethod current(allow_global=True)[source]

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

Parameters
allow_globalbool

If True returns the default client

Returns
Client

The current client

Raises
ValueError

If there is no client set, a ValueError is raised

See also

default_client

Link to the scheduler’s dashboard.

Returns
str

Dashboard URL.

Examples

Opening the dashboard in your default web browser:

>>> import webbrowser
>>> from distributed import Client
>>> client = Client()
>>> webbrowser.open(client.dashboard_link)
dump_cluster_state(filename: str = 'dask-cluster-dump', write_from_scheduler: bool | None = None, exclude: collections.abc.Collection[str] = ('run_spec',), format: Literal['msgpack', 'yaml'] = 'msgpack', **storage_options)[source]

Extract a dump of the entire cluster state and persist to disk or a URL. This is intended for debugging purposes only.

Warning: Memory usage on the scheduler (and client, if writing the dump locally) can be large. On a large or long-running cluster, this can take several minutes. The scheduler may be unresponsive while the dump is processed.

Results will be stored in a dict:

{
    "scheduler": {...},  # scheduler state
    "workers": {
        worker_addr: {...},  # worker state
        ...
    }
    "versions": {
        "scheduler": {...},
        "workers": {
            worker_addr: {...},
            ...
        }
    }
}
Parameters
filename:

The path or URL to write to. The appropriate file suffix (.msgpack.gz or .yaml) will be appended automatically.

Must be a path supported by fsspec.open() (like s3://my-bucket/cluster-dump, or cluster-dumps/dump). See write_from_scheduler to control whether the dump is written directly to filename from the scheduler, or sent back to the client over the network, then written locally.

write_from_scheduler:

If None (default), infer based on whether filename looks like a URL or a local path: True if the filename contains :// (like s3://my-bucket/cluster-dump), False otherwise (like local_dir/cluster-dump).

If True, write cluster state directly to filename from the scheduler. If filename is a local path, the dump will be written to that path on the scheduler’s filesystem, so be careful if the scheduler is running on ephemeral hardware. Useful when the scheduler is attached to a network filesystem or persistent disk, or for writing to buckets.

If False, transfer cluster state from the scheduler back to the client over the network, then write it to filename. This is much less efficient for large dumps, but useful when the scheduler doesn’t have access to any persistent storage.

exclude:

A collection of attribute names which are supposed to be excluded from the dump, e.g. to exclude code, tracebacks, logs, etc.

Defaults to exclude run_spec, which is the serialized user code. This is typically not required for debugging. To allow serialization of this, pass an empty tuple.

format:

Either "msgpack" or "yaml". If msgpack is used (default), the output will be stored in a gzipped file as msgpack.

To read:

import gzip, msgpack
with gzip.open("filename") as fd:
    state = msgpack.unpack(fd)

or:

import yaml
try:
    from yaml import CLoader as Loader
except ImportError:
    from yaml import Loader
with open("filename") as fd:
    state = yaml.load(fd, Loader=Loader)
**storage_options:

Any additional arguments to fsspec.open() when writing to a URL.

forward_logging(logger_name=None, level=0)[source]

Begin forwarding the given logger (by default the root) and all loggers under it from worker tasks to the client process. Whenever the named logger handles a LogRecord on the worker-side, the record will be serialized, sent to the client, and handled by the logger with the same name on the client-side.

Note that worker-side loggers will only handle LogRecords if their level is set appropriately, and the client-side logger will only emit the forwarded LogRecord if its own level is likewise set appropriately. For example, if your submitted task logs a DEBUG message to logger “foo”, then in order for forward_logging() to cause that message to be emitted in your client session, you must ensure that the logger “foo” have its level set to DEBUG (or lower) in the worker process and in the client process.

Parameters
logger_namestr, optional

The name of the logger to begin forwarding. The usual rules of the logging module’s hierarchical naming system apply. For example, if name is "foo", then not only "foo", but also "foo.bar", "foo.baz", etc. will be forwarded. If name is None, this indicates the root logger, and so all loggers will be forwarded.

Note that a logger will only forward a given LogRecord if the logger’s level is sufficient for the LogRecord to be handled at all.

levelstr | int, optional

Optionally restrict forwarding to LogRecords of this level or higher, even if the forwarded logger’s own level is lower.

Examples

For purposes of the examples, suppose we configure client-side logging as a user might: with a single StreamHandler attached to the root logger with an output level of INFO and a simple output format:

import logging
import distributed
import io, yaml

TYPICAL_LOGGING_CONFIG = '''
version: 1
handlers:
  console:
    class : logging.StreamHandler
    formatter: default
    level   : INFO
formatters:
  default:
    format: '%(asctime)s %(levelname)-8s [worker %(worker)s] %(name)-15s %(message)s'
    datefmt: '%Y-%m-%d %H:%M:%S'
root:
  handlers:
    - console
'''
config = yaml.safe_load(io.StringIO(TYPICAL_LOGGING_CONFIG))
logging.config.dictConfig(config)

Now create a client and begin forwarding the root logger from workers back to our local client process.

>>> client = distributed.Client()
>>> client.forward_logging()  # forward the root logger at any handled level

Then submit a task that does some error logging on a worker. We see output from the client-side StreamHandler.

>>> def do_error():
...     logging.getLogger("user.module").error("Hello error")
...     return 42
>>> client.submit(do_error).result()
2022-11-09 03:43:25 ERROR    [worker tcp://127.0.0.1:34783] user.module     Hello error
42

Note how an attribute "worker" is also added by dask to the forwarded LogRecord, which our custom formatter uses. This is useful for identifying exactly which worker logged the error.

One nuance worth highlighting: even though our client-side root logger is configured with a level of INFO, the worker-side root loggers still have their default level of ERROR because we haven’t done any explicit logging configuration on the workers. Therefore worker-side INFO logs will not be forwarded because they never even get handled in the first place.

>>> def do_info_1():
...     # no output on the client side
...     logging.getLogger("user.module").info("Hello info the first time")
...     return 84
>>> client.submit(do_info_1).result()
84

It is necessary to set the client-side logger’s level to INFO before the info message will be handled and forwarded to the client. In other words, the “effective” level of the client-side forwarded logging is the maximum of each logger’s client-side and worker-side levels.

>>> def do_info_2():
...     logger = logging.getLogger("user.module")
...     logger.setLevel(logging.INFO)
...     # now produces output on the client side
...     logger.info("Hello info the second time")
...     return 84
>>> client.submit(do_info_2).result()
2022-11-09 03:57:39 INFO     [worker tcp://127.0.0.1:42815] user.module     Hello info the second time
84
futures_of(futures)[source]

Wrapper method of futures_of

Parameters
futurestuple

The futures

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

Gather futures from distributed memory

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

Parameters
futuresCollection of futures

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

errorsstring

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

directboolean

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

asynchronous: bool

If True the client is in asynchronous mode

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

See also

Client.scatter

Send data out to cluster

Examples

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

Compute dask graph

Parameters
dskdict
keysobject, or nested lists of objects
workersstring or iterable of strings

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

allow_other_workersbool (defaults to False)

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

resourcesdict (defaults to {})

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

syncbool (optional)

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

asynchronous: bool

If True the client is in asynchronous mode

directbool

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

retriesint (default to 0)

Number of allowed automatic retries if computing a result fails

priorityNumber

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

fifo_timeouttimedelta str (defaults to ’60s’)

Allowed amount of time between calls to consider the same priority

actorsbool or dict (default None)

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

Returns
results

If ‘sync’ is True, returns the results. Otherwise, returns the known data packed If ‘sync’ is False, returns the known data. Otherwise, returns the results

See also

Client.compute

Compute asynchronous collections

Examples

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

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

Parameters
namestr

name of the dataset to retrieve

defaultstr

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

kwargsdict

additional keyword arguments to _get_dataset

Returns
The dataset from the scheduler, if present
get_events(topic: str | None = None)[source]

Retrieve structured topic logs

Parameters
topicstr, optional

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

get_executor(**kwargs)[source]

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

Parameters
**kwargs

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

Returns
ClientExecutor

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

get_metadata(keys, default=_NoDefault.no_default)[source]

Get arbitrary metadata from scheduler

See set_metadata for the full docstring with examples

Parameters
keyskey or list

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

defaultoptional

If the key does not exist then return this value instead. If not provided then this raises a KeyError if the key is not present

get_scheduler_logs(n=None)[source]

Get logs from scheduler

Parameters
nint

Number of logs to retrieve. Maxes out at 10000 by default, configurable via the distributed.admin.log-length configuration value.

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

Get task stream data from scheduler

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

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

Parameters
startNumber or string

When you want to start recording If a number it should be the result of calling time() If a string then it should be a time difference before now, like ’60s’ or ‘500 ms’

stopNumber or string

When you want to stop recording

countint

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

plotboolean, str

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

filenamestr (optional)

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

bokeh_resourcesbokeh.resources.Resources (optional)

Specifies if the resource component is INLINE or CDN

Returns
L: List[Dict]

See also

get_task_stream

a context manager version of this method

Examples

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

Pass the plot=True 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: bool = False, packages: collections.abc.Sequence[str] | None = None) distributed.client.VersionsDict | collections.abc.Coroutine[Any, Any, distributed.client.VersionsDict][source]

Return version info for the scheduler, all workers and myself

Parameters
check

raise ValueError if all required & optional packages do not match

packages

Extra package names to check

Examples

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

Get logs from workers

Parameters
nint

Number of logs to retrieve. Maxes out at 10000 by default, configurable via the distributed.admin.log-length configuration value.

workersiterable

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

nannybool, default False

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

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

Which keys are held by which workers

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

Parameters
workerslist (optional)

A list of worker addresses, defaults to all

**kwargsdict

Optional keyword arguments for the remote function

Examples

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

List named datasets available on the scheduler

log_event(topic: str | collections.abc.Collection[str], msg: Any)[source]

Log an event under a given topic

Parameters
topicstr, list[str]

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

msg

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

Examples

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

Map a function on a sequence of arguments

Arguments can be normal objects or Futures

Parameters
funccallable

Callable to be scheduled for execution. If func returns a coroutine, it will be run on the main event loop of a worker. Otherwise func will be run in a worker’s task executor pool (see Worker.executors for more information.)

iterablesIterables

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

keystr, list

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

workersstring or iterable of strings

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

retriesint (default to 0)

Number of allowed automatic retries if a task fails

resourcesdict (defaults to {})

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

priorityNumber

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

allow_other_workersbool (defaults to False)

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

fifo_timeoutstr timedelta (default ‘100ms’)

Allowed amount of time between calls to consider the same priority

actorbool (default False)

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

actorsbool (default False)

Alias for actor

purebool (defaults to True)

Whether or not the function is pure. Set pure=False for impure functions like np.random.random. Note that if both actor and pure kwargs are set to True, then the value of pure will be reverted to False, since an actor is stateful. See Pure Functions by Default for more details.

batch_sizeint, optional (default: just one batch whose size is the entire iterable)

Submit tasks to the scheduler in batches of (at most) batch_size. The tradeoff in batch size is that large batches avoid more per-batch overhead, but batches that are too big can take a long time to submit and unreasonably delay the cluster from starting its processing.

**kwargsdict

Extra keyword arguments to send to the function. Large values will be included explicitly in the task graph.

Returns
List, iterator, or Queue of futures, depending on the type of the
inputs.

See also

Client.submit

Submit a single function

Notes

The current implementation of a task graph resolution searches for occurrences of key and replaces it with a corresponding Future result. That can lead to unwanted substitution of strings passed as arguments to a task if these strings match some key that already exists on a cluster. To avoid these situations it is required to use unique values if a key is set manually. See https://github.com/dask/dask/issues/9969 to track progress on resolving this issue.

Examples

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

The bytes taken up by each key on the cluster

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

Parameters
keyslist (optional)

A list of keys, defaults to all keys

summaryboolean, (optional)

Summarize keys into key types

**kwargsdict

Optional keyword arguments for the remote function

See also

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)

The number of threads/cores available on each worker node

Parameters
workerslist (optional)

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

**kwargsdict

Optional keyword arguments for the remote function

Examples

>>> c.nthreads()  
{'192.168.1.141:46784': 8,
 '192.167.1.142:47548': 8,
 '192.167.1.143:47329': 8,
 '192.167.1.144:37297': 8}
normalize_collection(collection)[source]

Replace collection’s tasks by already existing futures if they exist

This normalizes the tasks within a collections task graph against the known futures within the scheduler. It returns a copy of the collection with a task graph that includes the overlapping futures.

Parameters
collectiondask object

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

Returns
collectiondask object

Collection with its tasks replaced with any existing futures.

See also

Client.persist

trigger computation of collection’s tasks

Examples

>>> len(x.__dask_graph__())  # x is a dask collection with 100 tasks  
100
>>> set(client.futures).intersection(x.__dask_graph__())  # some overlap exists  
10
>>> x = client.normalize_collection(x)  
>>> len(x.__dask_graph__())  # smaller computational graph  
20
nthreads(workers=None, **kwargs)[source]

The number of threads/cores available on each worker node

Parameters
workerslist (optional)

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

**kwargsdict

Optional keyword arguments for the remote function

Examples

>>> c.nthreads()  
{'192.168.1.141:46784': 8,
 '192.167.1.142:47548': 8,
 '192.167.1.143:47329': 8,
 '192.167.1.144:37297': 8}
persist(collections, optimize_graph=True, workers=None, allow_other_workers=None, resources=None, retries=None, priority=0, fifo_timeout='60s', actors=None, **kwargs)[source]

Persist dask collections on cluster

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

Parameters
collectionssequence or single dask object

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

optimize_graphbool

Whether or not to optimize the underlying graphs

workersstring or iterable of strings

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

allow_other_workersbool (defaults to False)

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

retriesint (default to 0)

Number of allowed automatic retries if computing a result fails

priorityNumber

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

fifo_timeouttimedelta str (defaults to ’60s’)

Allowed amount of time between calls to consider the same priority

resourcesdict (defaults to {})

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

actorsbool or dict (default None)

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

**kwargs

Options to pass to the graph optimize calls

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

See also

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
workerslist (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, plot=False, filename=None, server=False, scheduler=False)[source]

Collect statistical profiling information about recent work

Parameters
keystr

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

starttime
stoptime
workerslist

List of workers to restrict profile information

serverbool

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

schedulerbool

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

plotboolean or string

Whether or not to return a plot object

filenamestr

Filename to save the plot

Examples

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

Publish named datasets to scheduler

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

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

Parameters
argslist of objects to publish as name
kwargsdict

named collections to publish on the scheduler

Returns
None

Examples

Publishing client:

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

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

Receiving client:

>>> c.list_datasets()  
['my_dataset']
>>> df2 = c.get_dataset('my_dataset')  
rebalance(futures=None, workers=None, **kwargs)[source]

Rebalance data within network

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

For details on the algorithm and configuration options, refer to the matching scheduler-side method rebalance().

Warning

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

Parameters
futureslist, optional

A list of futures to balance, defaults all data

workerslist, optional

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

**kwargsdict

Optional keyword arguments for the function

register_plugin(plugin: distributed.diagnostics.plugin.NannyPlugin | distributed.diagnostics.plugin.SchedulerPlugin | distributed.diagnostics.plugin.WorkerPlugin, name: str | None = None, idempotent: bool | None = None)[source]

Register a plugin.

See https://distributed.readthedocs.io/en/latest/plugins.html

Parameters
plugin

A nanny, scheduler, or worker plugin to register.

name

Name for the plugin; if None, a name is taken from the plugin instance or automatically generated if not present.

idempotent

Do not re-register if a plugin of the given name already exists. If None, plugin.idempotent is taken if defined, False otherwise.

register_scheduler_plugin(plugin: distributed.diagnostics.plugin.SchedulerPlugin, name: str | None = None, idempotent: bool | None = None)[source]

Register a scheduler plugin.

Deprecated since version 2023.9.2: Use Client.register_plugin() instead.

See https://distributed.readthedocs.io/en/latest/plugins.html#scheduler-plugins

Parameters
pluginSchedulerPlugin

SchedulerPlugin instance to pass to the scheduler.

namestr

Name for the plugin; if None, a name is taken from the plugin instance or automatically generated if not present.

idempotentbool

Do not re-register if a plugin of the given name already exists.

register_worker_callbacks(setup=None)[source]

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

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

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

Parameters
setupcallable(dask_worker: Worker) -> None

Function to register and run on all workers

register_worker_plugin(plugin: distributed.diagnostics.plugin.NannyPlugin | distributed.diagnostics.plugin.WorkerPlugin, name: str | None = None, nanny: bool | None = None)[source]

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

Deprecated since version 2023.9.2: Use Client.register_plugin() instead.

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

The plugin may include methods setup, teardown, transition, and release_key. 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 it will be removed and replaced by the new one.

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

Parameters
pluginWorkerPlugin or NannyPlugin

WorkerPlugin or NannyPlugin instance to register.

namestr, optional

A name for the plugin. Registering a plugin with the same name will have no effect. If plugin has no name attribute a random name is used.

nannybool, optional

Whether to register the plugin with workers or nannies.

See also

distributed.WorkerPlugin
unregister_worker_plugin

Examples

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

You can get access to the plugin with the get_worker function

>>> client.register_plugin(other_plugin, name='my-plugin')
>>> def f():
...    worker = get_worker()
...    plugin = worker.plugins['my-plugin']
...    return plugin.my_state
>>> future = client.run(f)
replicate(futures, n=None, workers=None, branching_factor=2, **kwargs)[source]

Set replication of futures within network

Copy data onto many workers. This helps to broadcast frequently accessed data and can improve resilience.

This performs a tree copy of the data throughout the network individually on each piece of data. This operation blocks until complete. It does not guarantee replication of data to future workers.

Note

This method is incompatible with the Active Memory Manager’s ReduceReplicas policy. If you wish to use it, you must first disable the policy or disable the AMM entirely.

Parameters
futureslist of futures

Futures we wish to replicate

nint, optional

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

workerslist of worker addresses

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

branching_factorint, optional

The number of workers that can copy data in each generation

**kwargsdict

Optional keyword arguments for the remote function

See also

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(timeout=_NoDefault.no_default, wait_for_workers=True)[source]

Restart all workers. Reset local state. Optionally wait for workers to return.

Workers without nannies are shut down, hoping an external deployment system will restart them. Therefore, if not using nannies and your deployment system does not automatically restart workers, restart will just shut down all workers, then time out!

After restart, all connected workers are new, regardless of whether TimeoutError was raised. Any workers that failed to shut down in time are removed, and may or may not shut down on their own in the future.

Parameters
timeout:

How long to wait for workers to shut down and come back, if wait_for_workers is True, otherwise just how long to wait for workers to shut down. Raises asyncio.TimeoutError if this is exceeded.

wait_for_workers:

Whether to wait for all workers to reconnect, or just for them to shut down (default True). Use restart(wait_for_workers=False) combined with Client.wait_for_workers() for granular control over how many workers to wait for.

See also

Scheduler.restart
Client.restart_workers
restart_workers(workers: list[str], timeout: int | float | None = None, raise_for_error: bool = True) dict[str, str][source]

Restart a specified set of workers

Note

Only workers being monitored by a distributed.Nanny can be restarted.

See Nanny.restart for more details.

Parameters
workerslist[str]

Workers to restart. This can be a list of worker addresses, names, or a both.

timeoutint | float | None

Number of seconds to wait

raise_for_error: bool (default True)

Whether to raise a TimeoutError if restarting worker(s) doesn’t finish within timeout, or another exception caused from restarting worker(s).

Returns
dict[str, str]

Mapping of worker and restart status, the keys will match the original values passed in via workers.

See also

Client.restart

Notes

This method differs from Client.restart() in that this method simply restarts the specified set of workers, while Client.restart will restart all workers and also reset local state on the cluster (e.g. all keys are released).

Additionally, this method does not gracefully handle tasks that are being executed when a worker is restarted. These tasks may fail or have their suspicious count incremented.

Examples

You can get information about active workers using the following:

>>> workers = client.scheduler_info()['workers']

From that list you may want to select some workers to restart

>>> client.restart_workers(workers=['tcp://address:port', ...])
retire_workers(workers: list[str] | None = None, close_workers: bool = True, **kwargs)[source]

Retire certain workers on the scheduler

See distributed.Scheduler.retire_workers() for the full docstring.

Parameters
workers
close_workers
**kwargsdict

Optional keyword arguments for the remote function

See also

dask.distributed.Scheduler.retire_workers

Examples

You can get information about active workers using the following:

>>> workers = client.scheduler_info()['workers']

From that list you may want to select some workers to close

>>> client.retire_workers(workers=['tcp://address:port', ...])
retry(futures, asynchronous=None)[source]

Retry failed futures

Parameters
futureslist of Futures

The list of Futures

asynchronous: bool

If True the client is in asynchronous mode

run(function, *args, workers: list[str] | None = None, wait: bool = True, nanny: bool = False, on_error: Literal['raise', 'return', 'ignore'] = 'raise', **kwargs)[source]

Run a function on all workers outside of task scheduling system

This calls a function on all currently known workers immediately, blocks until those results come back, and returns the results asynchronously as a dictionary keyed by worker address. This method is generally used for side effects such as collecting diagnostic information or installing libraries.

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

Parameters
functioncallable

The function to run

*argstuple

Optional arguments for the remote function

**kwargsdict

Optional keyword arguments for the remote function

workerslist

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

waitboolean (optional)

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

nannybool, default False

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

on_error: “raise” | “return” | “ignore”

If the function raises an error on a worker:

raise

(default) Re-raise the exception on the client. The output from other workers will be lost.

return

Return the Exception object instead of the function output for the worker

ignore

Ignore the exception and remove the worker from the result dict

Examples

>>> c.run(os.getpid)  
{'192.168.0.100:9000': 1234,
 '192.168.0.101:9000': 4321,
 '192.168.0.102:9000': 5555}

Restrict computation to particular workers with the workers= keyword argument.

>>> c.run(os.getpid, workers=['192.168.0.100:9000',
...                           '192.168.0.101:9000'])  
{'192.168.0.100:9000': 1234,
 '192.168.0.101:9000': 4321}
>>> def get_status(dask_worker):
...     return dask_worker.status
>>> c.run(get_status)  
{'192.168.0.100:9000': 'running',
 '192.168.0.101:9000': 'running}

Run asynchronous functions in the background:

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

Run a function on the scheduler process

This is typically used for live debugging. The function should take a keyword argument dask_scheduler=, which will be given the scheduler object itself.

Parameters
functioncallable

The function to run on the scheduler process

*argstuple

Optional arguments for the function

**kwargsdict

Optional keyword arguments for the function

See also

Client.run

Run a function on all workers

Examples

>>> def get_number_of_tasks(dask_scheduler=None):
...     return len(dask_scheduler.tasks)
>>> client.run_on_scheduler(get_number_of_tasks)  
100

Run asynchronous functions in the background:

>>> async def print_state(dask_scheduler):  
...    while True:
...        print(dask_scheduler.status)
...        await asyncio.sleep(1)
>>> c.run(print_state, wait=False)  
scatter(data, workers=None, broadcast=False, direct=None, hash=True, timeout=_NoDefault.no_default, asynchronous=None)[source]

Scatter data into distributed memory

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

Parameters
datalist, dict, or object

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

workerslist of tuples (optional)

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

broadcastbool (defaults to False)

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

Note

Setting this flag to True is incompatible with the Active Memory Manager’s ReduceReplicas policy. If you wish to use it, you must first disable the policy or disable the AMM entirely.

directbool (defaults to automatically check)

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

hashbool (optional)

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

timeoutnumber, optional

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

asynchronous: bool

If True the client is in asynchronous mode

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

See also

Client.gather

Gather data back to local process

Notes

Scattering a dictionary uses dict keys to create Future keys. The current implementation of a task graph resolution searches for occurrences of key and replaces it with a corresponding Future result. That can lead to unwanted substitution of strings passed as arguments to a task if these strings match some key that already exists on a cluster. To avoid these situations it is required to use unique values if a key is set manually. See https://github.com/dask/dask/issues/9969 to track progress on resolving this issue.

Examples

>>> c = Client('127.0.0.1:8787')  
>>> c.scatter(1) 
<Future: status: finished, key: c0a8a20f903a4915b94db8de3ea63195>
>>> c.scatter([1, 2, 3])  
[<Future: status: finished, key: c0a8a20f903a4915b94db8de3ea63195>,
 <Future: status: finished, key: 58e78e1b34eb49a68c65b54815d1b158>,
 <Future: status: finished, key: d3395e15f605bc35ab1bac6341a285e2>]
>>> c.scatter({'x': 1, 'y': 2, 'z': 3})  
{'x': <Future: status: finished, key: x>,
 'y': <Future: status: finished, key: y>,
 'z': <Future: status: finished, key: z>}

Constrain location of data to subset of workers

>>> c.scatter([1, 2, 3], workers=[('hostname', 8788)])   

Broadcast data to all workers

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

Send scattered data to parallelized function using client futures interface

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

Basic information about the workers in the cluster

Parameters
**kwargsdict

Optional keyword arguments for the remote function

Examples

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

Set arbitrary metadata in the scheduler

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

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

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

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

Then this is the same as setting

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

The lower level dictionaries will be created on demand.

See also

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

story(*keys_or_stimuli, on_error='raise')[source]

Returns a cluster-wide story for the given keys or stimulus_id’s

submit(func, *args, key=None, workers=None, resources=None, retries=None, priority=0, fifo_timeout='100 ms', allow_other_workers=False, actor=False, actors=False, pure=True, **kwargs)[source]

Submit a function application to the scheduler

Parameters
funccallable

Callable to be scheduled as func(*args **kwargs). If func returns a coroutine, it will be run on the main event loop of a worker. Otherwise func will be run in a worker’s task executor pool (see Worker.executors for more information.)

*argstuple

Optional positional arguments

keystr

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

workersstring or iterable of strings

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

resourcesdict (defaults to {})

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

retriesint (default to 0)

Number of allowed automatic retries if the task fails

priorityNumber

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

fifo_timeoutstr timedelta (default ‘100ms’)

Allowed amount of time between calls to consider the same priority

allow_other_workersbool (defaults to False)

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

actorbool (default False)

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

actorsbool (default False)

Alias for actor

purebool (defaults to True)

Whether or not the function is pure. Set pure=False for impure functions like np.random.random. Note that if both actor and pure kwargs are set to True, then the value of pure will be reverted to False, since an actor is stateful. See Pure Functions by Default for more details.

**kwargs
Returns
Future

If running in asynchronous mode, returns the future. Otherwise returns the concrete value

Raises
TypeError

If ‘func’ is not callable, a TypeError is raised

ValueError

If ‘allow_other_workers’is True and ‘workers’ is None, a ValueError is raised

See also

Client.map

Submit on many arguments at once

Notes

The current implementation of a task graph resolution searches for occurrences of key and replaces it with a corresponding Future result. That can lead to unwanted substitution of strings passed as arguments to a task if these strings match some key that already exists on a cluster. To avoid these situations it is required to use unique values if a key is set manually. See https://github.com/dask/dask/issues/9969 to track progress on resolving this issue.

Examples

>>> c = client.submit(add, a, b)  
subscribe_topic(topic, handler)[source]

Subscribe to a topic and execute a handler for every received event

Parameters
topic: str

The topic name

handler: callable or coroutine function

A handler called for every received event. The handler must accept a single argument event which is a tuple (timestamp, msg) where timestamp refers to the clock on the scheduler.

See also

dask.distributed.Client.unsubscribe_topic
dask.distributed.Client.get_events
dask.distributed.Client.log_event

Examples

>>> import logging
>>> logger = logging.getLogger("myLogger")  # Log config not shown
>>> client.subscribe_topic("topic-name", lambda: logger.info)
unforward_logging(logger_name=None)[source]

Stop forwarding the given logger (default root) from worker tasks to the client process.

unpublish_dataset(name, **kwargs)[source]

Remove named datasets from scheduler

Parameters
namestr

The name of the dataset to unpublish

Examples

>>> c.list_datasets()  
['my_dataset']
>>> c.unpublish_dataset('my_dataset')  
>>> c.list_datasets()  
[]
unregister_scheduler_plugin(name)[source]

Unregisters a scheduler plugin

See https://distributed.readthedocs.io/en/latest/plugins.html#scheduler-plugins

Parameters
namestr

Name of the plugin to unregister. See the Client.register_scheduler_plugin() docstring for more information.

Examples

>>> class MyPlugin(SchedulerPlugin):
...     def __init__(self, *args, **kwargs):
...         pass  # the constructor is up to you
...     async def start(self, scheduler: Scheduler) -> None:
...         pass
...     async def before_close(self) -> None:
...         pass
...     async def close(self) -> None:
...         pass
...     def restart(self, scheduler: Scheduler) -> None:
...         pass
>>> plugin = MyPlugin(1, 2, 3)
>>> client.register_plugin(plugin, name='foo')
>>> client.unregister_scheduler_plugin(name='foo')
unregister_worker_plugin(name, nanny=None)[source]

Unregisters a lifecycle worker plugin

This unregisters an existing worker plugin. As part of the unregistration process the plugin’s teardown method will be called.

Parameters
namestr

Name of the plugin to unregister. See the Client.register_plugin() docstring for more information.

See also

register_plugin

Examples

>>> class MyPlugin(WorkerPlugin):
...     def __init__(self, *args, **kwargs):
...         pass  # the constructor is up to you
...     def setup(self, worker: dask.distributed.Worker):
...         pass
...     def teardown(self, worker: dask.distributed.Worker):
...         pass
...     def transition(self, key: str, start: str, finish: str, **kwargs):
...         pass
...     def release_key(self, key: str, state: str, cause: str | None, reason: None, report: bool):
...         pass
>>> plugin = MyPlugin(1, 2, 3)
>>> client.register_plugin(plugin, name='foo')
>>> client.unregister_worker_plugin(name='foo')
unsubscribe_topic(topic)[source]

Unsubscribe from a topic and remove event handler

See also

dask.distributed.Client.subscribe_topic
dask.distributed.Client.get_events
dask.distributed.Client.log_event
upload_file(filename, load: bool = True)[source]

Upload local package to scheduler and workers

This sends a local file up to the scheduler and all worker nodes. This file is placed into the working directory of each node, see config option temporary-directory (defaults to tempfile.gettempdir()).

This directory will be added to the Python’s system path so any .py, .egg or .zip files will be importable.

Parameters
filenamestring

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

loadbool, optional

Whether or not to import the module as part of the upload process. Defaults to True.

Examples

>>> client.upload_file('mylibrary.egg')  
>>> from mylibrary import myfunc  
>>> L = client.map(myfunc, seq)  
>>>
>>> # Where did that file go? Use `dask_worker.local_directory`.
>>> def where_is_mylibrary(dask_worker):
>>>     path = pathlib.Path(dask_worker.local_directory) / 'mylibrary.egg'
>>>     assert path.exists()
>>>     return str(path)
>>>
>>> client.run(where_is_mylibrary)  
wait_for_workers(n_workers: int, timeout: float | None = None) None[source]

Blocking call to wait for n workers before continuing

Parameters
n_workersint

The number of workers

timeoutnumber, optional

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

who_has(futures=None, **kwargs)[source]

The workers storing each future’s data

Parameters
futureslist (optional)

A list of futures, defaults to all data

**kwargsdict

Optional keyword arguments for the remote function

Examples

>>> x, y, z = c.map(inc, [1, 2, 3])  
>>> wait([x, y, z])  
>>> c.who_has()  
{'inc-1c8dd6be1c21646c71f76c16d09304ea': ['192.168.1.141:46784'],
 'inc-1e297fc27658d7b67b3a758f16bcf47a': ['192.168.1.141:46784'],
 'inc-fd65c238a7ea60f6a01bf4c8a5fcf44b': ['192.168.1.141:46784']}
>>> c.who_has([x, y])  
{'inc-1c8dd6be1c21646c71f76c16d09304ea': ['192.168.1.141:46784'],
 'inc-1e297fc27658d7b67b3a758f16bcf47a': ['192.168.1.141:46784']}
write_scheduler_file(scheduler_file)[source]

Write the scheduler information to a json file.

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

Parameters
scheduler_filestr

Path to a write the scheduler file.

Examples

>>> client = Client()  
>>> client.write_scheduler_file('scheduler.json')  
# connect to previous client's scheduler
>>> client2 = Client(scheduler_file='scheduler.json')  
class distributed.recreate_tasks.ReplayTaskClient(client)[source]

A plugin for the client allowing replay of remote tasks locally

Adds the following methods to the given client:

  • recreate_error_locally: main user method for replaying failed tasks

  • recreate_task_locally: main user method for replaying any task

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

For any calculation, whether it succeeded or failed, perform the task locally for debugging.

This operation should be performed after a future (result of gather, compute, etc) comes back with a status other than “pending”. Cases where you might want to debug a successfully completed future could include a calculation that returns an unexpected results. A common debugging process might include running the task locally in debug mode, with pdb.runcall.

Parameters
futurefuture

The same thing as was given to gather.

Returns
Any; will return the result of the task future.

Examples

>>> import pdb                                    
>>> future = c.submit(div, 1, 1)                  
>>> future.status                                 
'finished'
>>> pdb.runcall(c.recreate_task_locally, future)  

Future

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

A remotely running computation

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

Parameters
key: str, or tuple

Key of remote data to which this future refers

client: Client

Client that should own this future. Defaults to _get_global_client()

inform: bool

Do we inform the scheduler that we need an update on this future

state: FutureState

The state of the future

See also

Client

Creates futures

Examples

Futures typically emerge from Client computations

>>> my_future = client.submit(add, 1, 2)  

We can track the progress and results of a future

>>> my_future  
<Future: status: finished, key: add-8f6e709446674bad78ea8aeecfee188e>

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

>>> my_future.result()  
add_done_callback(fn)[source]

Call callback on future when future has finished

The callback fn should take the future as its only argument. This will be called regardless of if the future completes successfully, errs, or is cancelled

The callback is executed in a separate thread.

Parameters
fncallable

The method or function to be called

cancel(**kwargs)[source]

Cancel the request to run this future

See also

Client.cancel
cancelled()[source]

Returns True if the future has been cancelled

Returns
bool

True if the future was ‘cancelled’, otherwise False

done()[source]

Returns whether or not the computation completed.

Returns
bool

True if the computation is complete, otherwise False

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

Return the exception of a failed task

Parameters
timeoutnumber, optional

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

**kwargsdict

Optional keyword arguments for the function

Returns
Exception

The exception that was raised If timeout seconds are elapsed before returning, a dask.distributed.TimeoutError is raised.

See also

Future.traceback
property executor

Returns the executor, which is the client.

Returns
Client

The executor

release()[source]

Notes

This method can be called from different threads (see e.g. Client.get() or Future.__del__())

result(timeout=None)[source]

Wait until computation completes, gather result to local process.

Parameters
timeoutnumber, optional

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

Returns
result

The result of the computation. Or a coroutine if the client is asynchronous.

Raises
dask.distributed.TimeoutError

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

retry(**kwargs)[source]

Retry this future if it has failed

See also

Client.retry
property status

Returns the status

Returns
str

The status

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

Return the traceback of a failed task

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

Parameters
timeoutnumber, optional

Time in seconds after which to raise a dask.distributed.TimeoutError If timeout seconds are elapsed before returning, a dask.distributed.TimeoutError is raised.

Returns
traceback

The traceback object. Or a coroutine if the client is asynchronous.

See also

Future.exception

Examples

>>> import traceback  
>>> tb = future.traceback()  
>>> traceback.format_tb(tb)  
[...]
property type

Returns the type

Synchronization

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

Distributed Centralized Event equivalent to asyncio.Event

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

Parameters
name: string (optional)

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

client: Client (optional)

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

Examples

>>> event_1 = Event('a')  
>>> event_1.wait(timeout=1)  
>>> # in another process
>>> event_2 = Event('a')  
>>> event_2.set() 
>>> # now event_1 will stop waiting
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
timeoutnumber or string or timedelta, optional

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

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

Examples

>>> event = Event('a')  
>>> event.wait(timeout="1s")  
class distributed.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')  
>>> lock.acquire(timeout=1)  
>>> # do things with protected resource
>>> lock.release()  
acquire(blocking=True, timeout=None)[source]

Acquire the lock

Parameters
blockingbool, optional

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

timeoutstring or number or timedelta, optional

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

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

Examples

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

Release the lock if already acquired

class distributed.MultiLock(names: list[str] | None = None, client: distributed.client.Client | None = None)[source]

Distributed Centralized Lock

Parameters
names

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

client

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

Examples

>>> lock = MultiLock(['x', 'y'])  
>>> lock.acquire(timeout=1)  
>>> # do things with protected resource 'x' and 'y'
>>> lock.release()  
acquire(blocking=True, timeout=None, num_locks=None)[source]

Acquire the lock

Parameters
blockingbool, optional

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

timeoutstring or number or timedelta, optional

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

num_locksint, optional

Number of locks needed. If None, all locks are needed

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

Examples

>>> lock = MultiLock(['x', 'y'])  
>>> lock.acquire(timeout="1s")  
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 susceptible to lease overbooking in case of lease timeouts. It is advised to monitor log information and adjust above configuration options to suitable values for the user application.

Parameters
max_leases: int (optional)

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

name: string (optional)

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

register: bool

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

scheduler_rpc: ConnectionPool

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

loop: IOLoop

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

Notes

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

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
timeoutnumber or string or timedelta, optional

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

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

Parameters
name: string (optional)

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

client: Client (optional)

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

maxsize: int (optional)

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

See also

Variable

shared variable between clients

Examples

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

Get data from the queue

Parameters
timeoutnumber or string or timedelta, optional

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

batchboolean, int (optional)

If True then return all elements currently waiting in the queue. If an integer than return that many elements from the queue If False (default) then return one item at a time

put(value, timeout=None, **kwargs)[source]

Put data into the queue

Parameters
timeoutnumber or string or timedelta, optional

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

qsize(**kwargs)[source]

Current number of elements in the queue

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

Distributed Global Variable

This allows multiple clients to share futures and data between each other with a single mutable variable. All metadata is sequentialized through the scheduler. Race conditions can occur.

Values must be either Futures or msgpack-encodable data (ints, lists, strings, etc..) All data will be kept and sent through the scheduler, so it is wise not to send too much. If you want to share a large amount of data then scatter it and share the future instead.

Parameters
name: string (optional)

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

client: Client (optional)

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

See also

Queue

shared multi-producer/multi-consumer queue between clients

Examples

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

Delete this variable

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

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

Get the value of this variable

Parameters
timeoutnumber or string or timedelta, optional

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

set(value, **kwargs)[source]

Set the value of this variable

Parameters
valueFuture or object

Must be either a Future or a msgpack-encodable value

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=None, 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, scheduler_sync_interval=1, **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

memory_limit: str, float, int, or None, default “auto”

Sets the memory limit per worker.

Notes regarding argument data type:

  • If None or 0, no limit is applied.

  • If “auto”, the total system memory is split evenly between the workers.

  • If a float, that fraction of the system memory is used per worker.

  • If a string giving a number of bytes (like "1GiB"), that amount is used per worker.

  • If an int, that number of bytes is used per worker.

Note that the limit will only be enforced when processes=True, and the limit is only enforced on a best-effort basis — it’s still possible for workers to exceed this limit.

processes: bool

Whether to use processes (True) or threads (False). Defaults to True, unless worker_class=Worker, in which case it defaults to False.

threads_per_worker: int

Number of threads per each worker

scheduler_port: int

Port of the scheduler. Use 0 to choose a random port (default). 8786 is a common choice.

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 blocklist of handlers to disallow on the Scheduler, like ['feed', 'run_function']

service_kwargs: Dict[str, Dict]

Extra keywords to hand to the running services

securitySecurity 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. Defaults to Worker if processes=False and Nanny if processes=True or omitted.

**worker_kwargs:

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

Examples

>>> cluster = LocalCluster()  # Create a local cluster  
>>> cluster  
LocalCluster("127.0.0.1:8786", workers=8, threads=8)
>>> c = Client(cluster)  # connect to local cluster  

Scale the cluster to three workers

>>> cluster.scale(3)  

Pass extra keyword arguments to Bokeh

>>> LocalCluster(service_kwargs={'dashboard': {'prefix': '/foo'}})  
class distributed.SpecCluster(workers=None, scheduler=None, worker=None, asynchronous=False, loop=None, security=None, silence_logs=False, name=None, shutdown_on_close=True, scheduler_sync_interval=1)[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(Adaptive: type[distributed.deploy.adaptive.Adaptive] = <class 'distributed.deploy.adaptive.Adaptive'>, minimum: float = 0, maximum: float = inf, minimum_cores: int | None = None, maximum_cores: int | None = None, minimum_memory: str | None = None, maximum_memory: str | None = None, **kwargs: typing.Any) distributed.deploy.adaptive.Adaptive[source]

Turn on adaptivity

This scales Dask clusters automatically based on scheduler activity.

Parameters
minimumint

Minimum number of workers

maximumint

Maximum number of workers

minimum_coresint

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

maximum_coresint

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

minimum_memorystr

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

maximum_memorystr

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) distributed.deploy.spec.ProcessInterface[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
nint

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
nint

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

Return futures in the order in which they complete

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

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

Parameters
futures: Collection of futures

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

with_results: bool (False)

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

raise_errors: bool (True)

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

timeout: int (optional)

The returned iterator raises a dask.distributed.TimeoutError if __next__() or __anext__() is called and the result isn’t available after timeout seconds from the original call to as_completed(). If timeout is not specified or None, there is no limit to the wait time.

Examples

>>> x, y, z = client.map(inc, [1, 2, 3])  
>>> for future in as_completed([x, y, z]):  
...     print(future.result())  
3
2
4

Add more futures during computation

>>> x, y, z = client.map(inc, [1, 2, 3])  
>>> ac = as_completed([x, y, z])  
>>> for future in ac:  
...     print(future.result())  
...     if random.random() < 0.5:  
...         ac.add(c.submit(double, future))  
4
2
8
3
6
12
24

Optionally wait until the result has been gathered as well

>>> ac = as_completed([x, y, z], with_results=True)  
>>> for future, result in ac:  
...     print(result)  
2
4
3
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():  
...     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
blockbool, 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)  
>>> client.gather(ac.next_batch())  
[4, 1, 3]
>>> client.gather(ac.next_batch(block=False))  
[]
update(futures)[source]

Add multiple futures to the collection.

The added futures will emit from the iterator once they finish

distributed.diagnostics.progressbar.progress(*futures, notebook=None, multi=True, complete=True, group_by='prefix', **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
futuresFutures

A list of futures or keys to track

notebookbool (optional)

Running in the notebook or not (defaults to guess)

multibool (optional)

Track different functions independently (defaults to True)

completebool (optional)

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

group_byCallable | Literal[“spans”] | Literal[“prefix”]

Use spans instead of task key names for grouping tasks (defaults to “prefix”)

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/any futures are finished

Parameters
fsList[Future]
timeoutnumber, string, optional

Time after which to raise a dask.distributed.TimeoutError. Can be a string like "10 minutes" or a number of seconds to wait.

return_whenstr, optional

One of ALL_COMPLETED or FIRST_COMPLETED

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

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

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

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

Parameters
objFuture, list, dict, dask collection

The futures that you want to run at least once

Examples

>>> fire_and_forget(client.submit(func, *args))  
distributed.futures_of(o, client=None)[source]

Future objects in a collection

Parameters
ocollection

A possibly nested collection of Dask objects

clientClient, optional

The client

Returns
futuresList[Future]

A list of futures held by those collections

Raises
CancelledError

If one of the futures is cancelled a CancelledError is raised

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
timeoutNumber or String

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

separate_threadbool, 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() distributed.worker.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=None, resolve_address=True) Client[source]

Get a client while within a task.

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

Parameters
addressstr, optional

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

timeoutint or str

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

resolve_addressbool, default True

Whether to resolve address to its canonical form.

Returns
Client

Examples

>>> def f():
...     client = get_client(timeout="10s")
...     futures = client.map(lambda x: x + 1, range(10))  # spawn many tasks
...     results = client.gather(futures)
...     return sum(results)
>>> future = client.submit(f)  
>>> future.result()  
55
distributed.secede()[source]

Have this task secede from the worker’s thread pool

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

Examples

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

Have this thread rejoin the ThreadPoolExecutor

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

See also

secede

leave the thread pool

distributed.print(*args, sep: str | None = ' ', end: str | None = '\n', file: TextIO | None = None, flush: bool = False) None[source]

A drop-in replacement of the built-in print function for remote printing from workers to clients. If called from outside a dask worker, its arguments are passed directly to builtins.print(). If called by code running on a worker, then in addition to printing locally, any clients connected (possibly remotely) to the scheduler managing this worker will receive an event instructing them to print the same output to their own standard output or standard error streams. For example, the user can perform simple debugging of remote computations by including calls to this print function in the submitted code and inspecting the output in a local Jupyter notebook or interpreter session.

All arguments behave the same as those of builtins.print(), with the exception that the file keyword argument, if specified, must either be sys.stdout or sys.stderr; arbitrary file-like objects are not allowed.

All non-keyword arguments are converted to strings using str() and written to the stream, separated by sep and followed by end. Both sep and end must be strings; they can also be None, which means to use the default values. If no objects are given, print() will just write end.

Parameters
sepstr, optional

String inserted between values, default a space.

endstr, optional

String appended after the last value, default a newline.

filesys.stdout or sys.stderr, optional

Defaults to the current sys.stdout.

flushbool, default False

Whether to forcibly flush the stream.

Examples

>>> from dask.distributed import Client, print
>>> client = distributed.Client(...)
>>> def worker_function():
...     print("Hello from worker!")
>>> client.submit(worker_function)
<Future: finished, type: NoneType, key: worker_function-...>
Hello from worker!
distributed.warn(message: str | Warning, category: type[Warning] | None = <class 'UserWarning'>, stacklevel: int = 1, source: typing.Any = None) None[source]

A drop-in replacement of the built-in warnings.warn() function for issuing warnings remotely from workers to clients.

If called from outside a dask worker, its arguments are passed directly to warnings.warn(). If called by code running on a worker, then in addition to emitting a warning locally, any clients connected (possibly remotely) to the scheduler managing this worker will receive an event instructing them to emit the same warning (subject to their own local filters, etc.). When implementing computations that may run on a worker, the user can call this warn function to ensure that any remote client sessions will see their warnings, for example in a Jupyter output cell.

While all of the arguments are respected by the locally emitted warning (with same meanings as in warnings.warn()), stacklevel and source are ignored by clients because they would not be meaningful in the client’s thread.

Examples

>>> from dask.distributed import Client, warn
>>> client = Client()
>>> def do_warn():
...    warn("A warning from a worker.")
>>> client.submit(do_warn).result()
/path/to/distributed/client.py:678: UserWarning: A warning from a worker.
class distributed.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.performance_report(filename='dask-report.html', stacklevel=1, mode=None, storage_options=None)[source]

Gather performance report

This creates a static HTML file that includes many of the same plots of the dashboard for later viewing.

The resulting file uses JavaScript, and so must be viewed with a web browser. Locally we recommend using python -m http.server or hosting the file live online.

Parameters
filename: str, optional

The filename to save the performance report locally

stacklevel: int, optional

The code execution frame utilized for populating the Calling Code section of the report. Defaults to 1 which is the frame calling performance_report

mode: str, optional

Mode parameter to pass to bokeh.io.output.output_file(). Defaults to None.

storage_options: dict, optional

Any additional arguments to fsspec.open() when writing to a URL.

Examples

>>> with performance_report(filename="myfile.html", stacklevel=1):
...     x.compute()

Utilities

class distributed.utils.Log[source]

A container for newline-delimited string of log entries

class distributed.utils.Logs[source]

A container for a dict mapping names to strings of log entries

distributed.diagnostics.memray.memray_scheduler(directory: str | pathlib.Path = 'memray-profiles', report_args: Union[collections.abc.Sequence[str], Literal[False]] = ('flamegraph', '--temporal', '--leaks'), **memray_kwargs: Any) collections.abc.Iterator[None][source]

Generate a Memray profile on the Scheduler and download the generated report.

Example:

with memray_scheduler():
    client.submit(my_function).result()

# Or even while the computation is already running

fut = client.submit(my_function)

with memray_scheduler():
    time.sleep(10)

fut.result()
Parameters
directorystr

The directory to save the reports to.

report_argstuple[str]

Particularly for native_traces=True, the reports have to be generated on the same host using the same Python interpreter as the profile was generated. Otherwise, native traces will yield unusable results. Therefore, we’re generating the reports on the Scheduler and download them afterwards. You can modify the report generation by providing additional arguments and we will generate the reports as:

memray *report_args -f <filename> -o <filename>.html

If the raw data should be fetched instead of the report, set this to False.

**memray_kwargs

Keyword arguments to be passed to memray.Tracker, e.g. {“native_traces”: True}

distributed.diagnostics.memray.memray_workers(directory: str | pathlib.Path = 'memray-profiles', workers: int | None | list[str] = None, report_args: Union[collections.abc.Sequence[str], Literal[False]] = ('flamegraph', '--temporal', '--leaks'), fetch_reports_parallel: bool | int = True, **memray_kwargs: Any) collections.abc.Iterator[None][source]

Generate a Memray profile on the workers and download the generated report.

Example:

with memray_workers():
    client.submit(my_function).result()

# Or even while the computation is already running

fut = client.submit(my_function)

with memray_workers():
    time.sleep(10)

fut.result()
Parameters
directorystr

The directory to save the reports to.

workersint | None | list[str]

The workers to profile. If int, the first n workers will be used. If None, all workers will be used. If list[str], the workers with the given addresses will be used.

report_argstuple[str]

Particularly for native_traces=True, the reports have to be generated on the same host using the same Python interpreter as the profile was generated. Otherwise, native traces will yield unusable results. Therefore, we’re generating the reports on the workers and download them afterwards. You can modify the report generation by providing additional arguments and we will generate the reports as:

memray *report_args -f <filename> -o <filename>.html

If the raw data should be fetched instead of the report, set this to False.

fetch_reports_parallelbool | int

Fetching results is sometimes slow and it’s sometimes not desired to wait for all workers to finish before receiving the first reports. This controls how many workers are fetched concurrently.

int: Number of workers to fetch concurrently True: All workers concurrently False: One worker at a time

**memray_kwargs

Keyword arguments to be passed to memray.Tracker, e.g. {“native_traces”: True}

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

intervaltimedelta 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 distributed.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)
property loop: tornado.ioloop.IOLoop

Override Adaptive.loop

async recommendations(target: int) dict[source]

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

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

async workers_to_close(target: int) list[str][source]

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

Returns
List of worker names 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.