Plugins

Dask’s plugin system enables you to run custom Python code for certain events. You can use plugins that are specific to schedulers, workers, or nannies. A worker plugin, for example, allows you to run custom Python code on all your workers at certain event in the worker’s lifecycle (e.g. when the worker process is started). In each section below, you’ll see how to create your own plugin or use a Dask-provided built-in plugin.

Scheduler Plugins

class distributed.diagnostics.plugin.SchedulerPlugin[source]

Interface to extend the Scheduler

A plugin enables custom hooks to run when specific events occur. The scheduler will run the methods of this plugin whenever the corresponding method of the scheduler is run. This runs user code within the scheduler thread that can perform arbitrary operations in synchrony with the scheduler itself.

Plugins are often used for diagnostics and measurement, but have full access to the scheduler and could in principle affect core scheduling.

To implement a plugin:

  1. inherit from this class

  2. override some of its methods

  3. register the plugin using Client.register_plugin.

The idempotent attribute is used to control whether or not the plugin should be ignored upon registration if a scheduler plugin with the same name already exists. If True, the plugin is ignored, otherwise the existing plugin is replaced. Defaults to False.

Examples

>>> class Counter(SchedulerPlugin):
...     def __init__(self):
...         self.counter = 0
...
...     def transition(self, key, start, finish, *args, **kwargs):
...         if start == 'processing' and finish == 'memory':
...             self.counter += 1
...
...     def restart(self, scheduler):
...         self.counter = 0
>>> plugin = Counter()
>>> scheduler.add_plugin(plugin)  
add_client(scheduler: Scheduler, client: str) None[source]

Run when a new client connects

add_worker(scheduler: Scheduler, worker: str) None | Awaitable[None][source]

Run when a new worker enters the cluster

If this method is synchronous, it is immediately and synchronously executed without Scheduler.add_worker ever yielding to the event loop. If it is asynchronous, it will be awaited after all synchronous SchedulerPlugin.add_worker hooks have executed.

Warning

There are no guarantees about the execution order between individual SchedulerPlugin.add_worker hooks and the ordering may be subject to change without deprecation cycle.

async before_close() None[source]

Runs prior to any Scheduler shutdown logic

async close() None[source]

Run when the scheduler closes down

This runs at the beginning of the Scheduler shutdown process, but after workers have been asked to shut down gracefully

log_event(topic: str, msg: Any) None[source]

Run when an event is logged

remove_client(scheduler: Scheduler, client: str) None[source]

Run when a client disconnects

remove_worker(scheduler: Scheduler, worker: str, *, stimulus_id: str, **kwargs: Any) None | Awaitable[None][source]

Run when a worker leaves the cluster

If this method is synchronous, it is immediately and synchronously executed without Scheduler.remove_worker ever yielding to the event loop. If it is asynchronous, it will be awaited after all synchronous SchedulerPlugin.remove_worker hooks have executed.

Warning

There are no guarantees about the execution order between individual SchedulerPlugin.remove_worker hooks and the ordering may be subject to change without deprecation cycle.

restart(scheduler: Scheduler) None[source]

Run when the scheduler restarts itself

async start(scheduler: Scheduler) None[source]

Run when the scheduler starts up

This runs at the end of the Scheduler startup process

transition(key: Key, start: SchedulerTaskStateState, finish: SchedulerTaskStateState, *args: Any, stimulus_id: str, **kwargs: Any) None[source]

Run whenever a task changes state

For a description of the transition mechanism and the available states, see Scheduler task states.

Warning

This is an advanced feature and the transition mechanism and details of task states are subject to change without deprecation cycle.

Parameters
key
start

Start state of the transition. One of released, waiting, processing, memory, error.

finish

Final state of the transition.

stimulus_id

ID of stimulus causing the transition.

*args, **kwargs

More options passed when transitioning This may include worker ID, compute time, etc.

update_graph(scheduler: Scheduler, *, client: str, keys: set[Key], tasks: list[Key], annotations: dict[str, dict[Key, Any]], priority: dict[Key, tuple[int | float, ...]], dependencies: dict[Key, set[Key]], **kwargs: Any) None[source]

Run when a new graph / tasks enter the scheduler

Parameters
scheduler:

The Scheduler instance.

client:

The unique Client id.

keys:

The keys the Client is interested in when calling update_graph.

tasks:

The

annotations:

Fully resolved annotations as applied to the tasks in the format:

{
    "annotation": {
        "key": "value,
        ...
    },
    ...
}
priority:

Task calculated priorities as assigned to the tasks.

dependencies:

A mapping that maps a key to its dependencies.

**kwargs:

It is recommended to allow plugins to accept more parameters to ensure future compatibility.

RabbitMQ Example

RabbitMQ is a distributed messaging queue that we can use to post updates about task transitions. By posting transitions to RabbitMQ, we allow other machines to do the processing of transitions and keep scheduler processing to a minimum. See the RabbitMQ tutorial for more information on RabbitMQ and how to consume the messages.

import json
from distributed.diagnostics.plugin import SchedulerPlugin
import pika

class RabbitMQPlugin(SchedulerPlugin):
    def __init__(self):
        # Update host to be your RabbitMQ host
        self.connection = pika.BlockingConnection(
            pika.ConnectionParameters(host='localhost'))
        self.channel = self.connection.channel()
        self.channel.queue_declare(queue='dask_task_status', durable=True)

    def transition(self, key, start, finish, *args, **kwargs):
        message = dict(
            key=key,
            start=start,
            finish=finish,
        )
        self.channel.basic_publish(
            exchange='',
            routing_key='dask_task_status',
            body=json.dumps(message),
            properties=pika.BasicProperties(
                delivery_mode=2,  # make message persistent
            ))

@click.command()
def dask_setup(scheduler):
    plugin = RabbitMQPlugin()
    scheduler.add_plugin(plugin)

Run with: dask scheduler --preload <filename.py>

Accessing Full Task State

If you would like to access the full distributed.scheduler.TaskState stored in the scheduler you can do this by passing and storing a reference to the scheduler as so:

from distributed.diagnostics.plugin import SchedulerPlugin

class MyPlugin(SchedulerPlugin):
    def __init__(self, scheduler):
         self.scheduler = scheduler

    def transition(self, key, start, finish, *args, **kwargs):
         # Get full TaskState
         ts = self.scheduler.tasks[key]

@click.command()
def dask_setup(scheduler):
    plugin = MyPlugin(scheduler)
    scheduler.add_plugin(plugin)

Built-In Scheduler Plugins

class distributed.diagnostics.plugin.PipInstall(packages: list[str], pip_options: list[str] | None = None, restart_workers: bool = False)[source]

A plugin to pip install a set of packages

This accepts a set of packages to install on all workers as well as options to use when installing. You can also optionally ask for the worker to restart itself after performing this installation.

Note

This will increase the time it takes to start up each worker. If possible, we recommend including the libraries in the worker environment or image. This is primarily intended for experimentation and debugging.

Parameters
packages

A list of packages to install using pip. Packages should follow the structure defined for requirement files. Packages also may include environment variables.

pip_options

Additional options to pass to pip

restart_workers

Whether or not to restart the worker after installing the packages; only functions if the worker has an attached nanny process.

Examples

>>> from dask.distributed import PipInstall
>>> plugin = PipInstall(packages=["scikit-learn"], pip_options=["--upgrade"])
>>> client.register_plugin(plugin)

Install package from a private repository using a TOKEN environment variable.

>>> from dask.distributed import PipInstall
>>> plugin = PipInstall(packages=["private_package@git+https://${TOKEN}@github.com/dask/private_package.git])
>>> client.register_plugin(plugin)
class distributed.diagnostics.plugin.CondaInstall(packages: list[str], conda_options: list[str] | None = None, restart_workers: bool = False)[source]

A plugin to conda install a set of packages

This accepts a set of packages to install on the scheduler and all workers as well as options to use when installing.

You can also optionally ask for the workers to restart after performing this installation.

Note

This will increase the time it takes to start up the cluster. If possible, we recommend including the libraries in the cluster environment or image. This is primarily intended for experimentation and debugging.

Parameters
packages

A list of packages (with optional versions) to install using conda

conda_options

Additional options to pass to conda

restart_workers

Whether or not to restart the worker after installing the packages Only functions if the workers have an attached nanny process

Examples

>>> from dask.distributed import CondaInstall
>>> plugin = CondaInstall(packages=["scikit-learn"], conda_options=["--update-deps"])
>>> client.register_plugin(plugin)
class distributed.diagnostics.plugin.InstallPlugin(install_fn: Callable[[], None], restart_workers: bool)[source]

Scheduler plugin to install software on the cluster

This accepts an function that installs software on the scheduler and all workers. You can also optionally ask for the worker to restart after performing this installation.

Note

This will increase the time it takes to start up each worker. If possible, we recommend including the software in the worker environment or image. This is primarily intended for experimentation and debugging.

Parameters
install_fn

Callable used to install the software; must be idempotent.

restart_workers

Whether or not to restart the worker after installing the packages Only functions if the worker has an attached nanny process

class distributed.diagnostics.plugin.SchedulerUploadFile(filepath: str, load: bool = True)[source]

Worker Plugins

distributed.diagnostics.plugin.WorkerPlugin provides a base class for creating your own worker plugins. In addition, Dask provides some built-in plugins.

Watch the video below for an example using a WorkerPlugin to add a concurrent.futures.ProcessPoolExecutor:

class distributed.diagnostics.plugin.WorkerPlugin[source]

Interface to extend the Worker

A worker plugin enables custom code to run at different stages of the Workers’ lifecycle.

A plugin enables custom code to run at each of step of a Workers’s life. Whenever such an event happens, the corresponding method on this class will be called. Note that the user code always runs within the Worker’s main thread.

To implement a plugin:

  1. inherit from this class

  2. override some of its methods

  3. register the plugin using Client.register_plugin.

The idempotent attribute is used to control whether or not the plugin should be ignored upon registration if a worker plugin with the same name already exists. If True, the plugin is ignored, otherwise the existing plugin is replaced. Defaults to False.

Examples

>>> class ErrorLogger(WorkerPlugin):
...     def __init__(self, logger):
...         self.logger = logger
...
...     def setup(self, worker):
...         self.worker = worker
...
...     def transition(self, key, start, finish, *args, **kwargs):
...         if finish == 'error':
...             ts = self.worker.tasks[key]
...             exc_info = (type(ts.exception), ts.exception, ts.traceback)
...             self.logger.error(
...                 "Error during computation of '%s'.", key,
...                 exc_info=exc_info
...             )
>>> import logging
>>> plugin = ErrorLogger(logging)
>>> client.register_plugin(plugin)  
setup(worker: Worker) None | Awaitable[None][source]

Run when the plugin is attached to a worker. This happens when the plugin is registered and attached to existing workers, or when a worker is created after the plugin has been registered.

teardown(worker: Worker) None | Awaitable[None][source]

Run when the worker to which the plugin is attached is closed, or when the plugin is removed.

transition(key: Key, start: WorkerTaskStateState, finish: WorkerTaskStateState, **kwargs: Any) None[source]

Throughout the lifecycle of a task (see Worker State), Workers are instructed by the scheduler to compute certain tasks, resulting in transitions in the state of each task. The Worker owning the task is then notified of this state transition.

Whenever a task changes its state, this method will be called.

Warning

This is an advanced feature and the transition mechanism and details of task states are subject to change without deprecation cycle.

Parameters
key
start

Start state of the transition. One of waiting, ready, executing, long-running, memory, error.

finish

Final state of the transition.

kwargs

More options passed when transitioning

Built-In Worker Plugins

class distributed.diagnostics.plugin.UploadFile(filepath: str, load: bool = True)[source]

A WorkerPlugin to upload a local file to workers.

Parameters
filepath: str

A path to the file (.py, egg, or zip) to upload

Examples

>>> from distributed.diagnostics.plugin import UploadFile
>>> client.register_plugin(UploadFile("/path/to/file.py"))  

Nanny Plugins

class distributed.diagnostics.plugin.NannyPlugin[source]

Interface to extend the Nanny

A worker plugin enables custom code to run at different stages of the Workers’ lifecycle. A nanny plugin does the same thing, but benefits from being able to run code before the worker is started, or to restart the worker if necessary.

To implement a plugin:

  1. inherit from this class

  2. override some of its methods

  3. register the plugin using Client.register_plugin.

The idempotent attribute is used to control whether or not the plugin should be ignored upon registration if a nanny plugin with the same name already exists. If True, the plugin is ignored, otherwise the existing plugin is replaced. Defaults to False.

The restart attribute is used to control whether or not a running Worker needs to be restarted when registering the plugin.

setup(nanny)[source]

Run when the plugin is attached to a nanny. This happens when the plugin is registered and attached to existing nannies, or when a nanny is created after the plugin has been registered.

teardown(nanny)[source]

Run when the nanny to which the plugin is attached to is closed

Built-In Nanny Plugins

class distributed.diagnostics.plugin.Environ(environ: dict | None = None)[source]
class distributed.diagnostics.plugin.UploadDirectory(path, restart=False, update_path=False, skip_words=('.git', '.github', '.pytest_cache', 'tests', 'docs'), skip=(<function UploadDirectory.<lambda>>, ))[source]

A NannyPlugin to upload a local file to workers.

Parameters
path: str

A path to the directory to upload

Examples

>>> from distributed.diagnostics.plugin import UploadDirectory
>>> client.register_plugin(UploadDirectory("/path/to/directory"), nanny=True)