Scheduler Plugins

class distributed.diagnostics.plugin.SchedulerPlugin[source]

Interface to extend the Scheduler

The scheduler operates by triggering and responding to events like task_finished, update_graph, task_erred, etc..

A plugin enables custom code to run at each of those same events. The scheduler will run the analogous methods on this class when each event is triggered. 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 implement some of the methods of this class and add the plugin to the scheduler with Scheduler.add_plugin(myplugin).

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(self, scheduler=None, client=None, **kwargs)[source]

Run when a new client connects

add_worker(self, scheduler=None, worker=None, **kwargs)[source]

Run when a new worker enters the cluster

async close(self)[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_client(self, scheduler=None, client=None, **kwargs)[source]

Run when a client disconnects

remove_worker(self, scheduler=None, worker=None, **kwargs)[source]

Run when a worker leaves the cluster

restart(self, scheduler, **kwargs)[source]

Run when the scheduler restarts itself

async start(self, scheduler)[source]

Run when the scheduler starts up

This runs at the end of the Scheduler startup process

transition(self, key, start, finish, *args, **kwargs)[source]

Run whenever a task changes state

Parameters
key: string
start: string

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

finish: string

Final state of the transition.

*args, **kwargs: More options passed when transitioning

This may include worker ID, compute time, etc.

update_graph(self, scheduler, dsk=None, keys=None, restrictions=None, **kwargs)[source]

Run when a new graph / tasks enter the scheduler

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)

Worker Plugins

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: at setup, during task state transitions and at teardown.

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 implement some of the methods of this class and register the plugin to your client in order to have it attached to every existing and future workers with Client.register_worker_plugin.

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':
...             exc = self.worker.exceptions[key]
...             self.logger.error("Task '%s' has failed with exception: %s" % (key, str(exc)))
>>> plugin = ErrorLogger()
>>> client.register_worker_plugin(plugin)  
setup(self, worker)[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(self, worker)[source]

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

transition(self, key, start, finish, **kwargs)[source]

Throughout the lifecycle of a task (see Worker), 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.

Parameters
key: string
start: string

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

finish: string

Final state of the transition.

kwargs: More options passed when transitioning