File: plugins.rst

package info (click to toggle)
dask.distributed 2022.12.1%2Bds.1-3
  • links: PTS, VCS
  • area: main
  • in suites: bookworm
  • size: 10,164 kB
  • sloc: python: 81,938; javascript: 1,549; makefile: 228; sh: 100
file content (126 lines) | stat: -rw-r--r-- 4,100 bytes parent folder | download
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
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
=================

.. autoclass:: distributed.diagnostics.plugin.SchedulerPlugin
   :members:


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 <https://www.rabbitmq.com/tutorials/tutorial-two-python.html>`_
for more information on RabbitMQ and how to consume the messages.

.. code-block:: python

   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 :class:`distributed.scheduler.TaskState`
stored in the scheduler you can do this by passing and storing a reference to
the scheduler as so:

.. code-block:: python

   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` provides a base class
for creating your own worker plugins. In addition, Dask provides some
:ref:`built-in plugins <plugins.builtin>`.

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

.. raw:: html

    <iframe width="560"
            height="315" src="https://www.youtube.com/embed/vF2VItVU5zg?start=468"
            style="margin: 0 auto 20px auto; display: block;" title="YouTube video player" frameborder="0" allow="accelerometer; autoplay; clipboard-write; encrypted-media; gyroscope; picture-in-picture" allowfullscreen></iframe>

.. autoclass:: distributed.diagnostics.plugin.WorkerPlugin
   :members:

.. _plugins.builtin:

Built-In Worker Plugins
-----------------------

.. autoclass:: distributed.diagnostics.plugin.PipInstall
.. autoclass:: distributed.diagnostics.plugin.CondaInstall
.. autoclass:: distributed.diagnostics.plugin.UploadFile


Nanny Plugins
=============

.. autoclass:: distributed.diagnostics.plugin.NannyPlugin
   :members:


Built-In Nanny Plugins
----------------------

.. autoclass:: distributed.diagnostics.plugin.Environ
.. autoclass:: distributed.diagnostics.plugin.UploadDirectory