Plugins
Contents
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:
inherit from this class
override some of its methods
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. IfTrue
, the plugin is ignored, otherwise the existing plugin is replaced. Defaults toFalse
.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_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 synchronousSchedulerPlugin.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 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
- 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 synchronousSchedulerPlugin.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.
- 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]], stimulus_id: str, **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.
- stimulus_id:
ID of the stimulus causing the graph update
- **kwargs:
It is recommended to allow plugins to accept more parameters to ensure future compatibility.
- valid_workers_downscaling(scheduler: Scheduler, workers: list[scheduler_module.WorkerState]) list[scheduler_module.WorkerState] [source]¶
Determine which workers can be removed from the cluster
This method is called when the scheduler is about to downscale the cluster by removing workers. The method should return a set of worker states that can be removed from the cluster.
- Parameters
- workerslist
The list of worker states that are candidates for removal.
- stimulus_idstr
ID of stimulus causing the downscaling.
- Returns
- list
The list of worker states that can be removed from the cluster.
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.
See also
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
See also
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
See also
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:
inherit from this class
override some of its methods
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. IfTrue
, the plugin is ignored, otherwise the existing plugin is replaced. Defaults toFalse
.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:
inherit from this class
override some of its methods
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. IfTrue
, the plugin is ignored, otherwise the existing plugin is replaced. Defaults toFalse
.The
restart
attribute is used to control whether or not a runningWorker
needs to be restarted when registering the plugin.See also
Built-In Nanny Plugins¶
- class distributed.diagnostics.plugin.UploadDirectory(path, restart_workers=False, update_path=False, skip_words=('.git', '.github', '.pytest_cache', 'tests', 'docs'), skip=(<function UploadDirectory.<lambda>>, ), mode='workers')[source]¶
Scheduler to upload a local directory to the cluster.
- Parameters
- path:
Path to the directory to upload
- scheduler:
Whether to upload the directory to the scheduler
Examples
>>> from distributed.diagnostics.plugin import UploadDirectory >>> client.register_plugin(UploadDirectory("/path/to/directory"))