Scheduler Plugins

  • class distributed.diagnostics.plugin.SchedulerPlugin[source]
  • Interface to extend the Scheduler

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

A plugin enables custom code to run at each of those same events. Thescheduler will run the analogous methods on this class when each event istriggered. This runs user code within the scheduler thread that canperform arbitrary operations in synchrony with the scheduler itself.

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

To implement a plugin implement some of the methods of this class and addthe plugin to the scheduler with Scheduler.add_plugin(myplugin).

Examples

  1. >>> class Counter(SchedulerPlugin):
  2. ... def __init__(self):
  3. ... self.counter = 0
  4. ...
  5. ... def transition(self, key, start, finish, *args, **kwargs):
  6. ... if start == 'processing' and finish == 'memory':
  7. ... self.counter += 1
  8. ...
  9. ... def restart(self, scheduler):
  10. ... self.counter = 0
  1. >>> plugin = Counter()
  2. >>> scheduler.add_plugin(plugin) # doctest: +SKIP
  • addworker(_self, scheduler=None, worker=None, **kwargs)[source]
  • Run when a new worker enters the cluster

  • removeworker(_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

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

  • Run whenever a task changes state

Parameters:

  1. - **key: string**
  2. -
  3. - **start: string**
  4. -

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

  1. - **finish: string**
  2. -

Final state of the transition.

  1. - ***args, **kwargs: More options passed when transitioning**
  2. -

This may include worker ID, compute time, etc.

  • updategraph(_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 abouttask transitions. By posting transitions to RabbitMQ, we allow other machinesto do the processing of transitions and keep scheduler processing to a minimum.See theRabbitMQ tutorialfor more information on RabbitMQ and how to consume the messages.

  1. import json
  2. from distributed.diagnostics.plugin import SchedulerPlugin
  3. import pika
  4.  
  5. class RabbitMQPlugin(SchedulerPlugin):
  6. def __init__(self):
  7. # Update host to be your RabbitMQ host
  8. self.connection = pika.BlockingConnection(
  9. pika.ConnectionParameters(host='localhost'))
  10. self.channel = self.connection.channel()
  11. self.channel.queue_declare(queue='dask_task_status', durable=True)
  12.  
  13. def transition(self, key, start, finish, *args, **kwargs):
  14. message = dict(
  15. key=key,
  16. start=start,
  17. finish=finish,
  18. )
  19. self.channel.basic_publish(
  20. exchange='',
  21. routing_key='dask_task_status',
  22. body=json.dumps(message),
  23. properties=pika.BasicProperties(
  24. delivery_mode=2, # make message persistent
  25. ))
  26.  
  27. @click.command()
  28. def dask_setup(scheduler):
  29. plugin = RabbitMQPlugin()
  30. 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.TaskStatestored in the scheduler you can do this by passing and storing a reference tothe scheduler as so:

  1. from distributed.diagnostics.plugin import SchedulerPlugin
  2.  
  3. class MyPlugin(SchedulerPlugin):
  4. def __init__(self, scheduler):
  5. self.scheduler = scheduler
  6.  
  7. def transition(self, key, start, finish, *args, **kwargs):
  8. # Get full TaskState
  9. ts = self.scheduler.tasks[key]
  10.  
  11. @click.command()
  12. def dask_setup(scheduler):
  13. plugin = MyPlugin(scheduler)
  14. 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 suchan event happens, the corresponding method on this class will be called. Note that theuser code always runs within the Worker’s main thread.

To implement a plugin implement some of the methods of this class and registerthe plugin to your client in order to have it attached to every existing andfuture workers with Client.register_worker_plugin.

Examples

  1. >>> class ErrorLogger(WorkerPlugin):
  2. ... def __init__(self, logger):
  3. ... self.logger = logger
  4. ...
  5. ... def setup(self, worker):
  6. ... self.worker = worker
  7. ...
  8. ... def transition(self, key, start, finish, *args, **kwargs):
  9. ... if finish == 'error':
  10. ... exc = self.worker.exceptions[key]
  11. ... self.logger.error("Task '%s' has failed with exception: %s" % (key, str(exc)))
  1. >>> plugin = ErrorLogger()
  2. >>> client.register_worker_plugin(plugin) # doctest: +SKIP
  • setup(self, worker)[source]
  • Run when the plugin is attached to a worker. This happens when the plugin is registeredand attached to existing workers, or when a worker is created after the plugin has beenregistered.

  • 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 areinstructed by the scheduler to compute certain tasks, resulting in transitionsin the state of each task. The Worker owning the task is then notified of thisstate transition.

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

Parameters:

  1. - **key: string**
  2. -
  3. - **start: string**
  4. -

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

  1. - **finish: string**
  2. -

Final state of the transition.

  1. - **kwargs: More options passed when transitioning**
  2. -