diff --git a/docs/source/plugins.rst b/docs/source/plugins.rst index 53fd001f2e4..bc02975d939 100644 --- a/docs/source/plugins.rst +++ b/docs/source/plugins.rst @@ -64,10 +64,14 @@ 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: +the scheduler. This can be done in one of two ways. + +Firstly, you can use the ``dask_setup`` function and run the scheduler with +``dask scheduler --preload ``: .. code-block:: python + import click from distributed.diagnostics.plugin import SchedulerPlugin class MyPlugin(SchedulerPlugin): @@ -83,6 +87,34 @@ the scheduler as so: plugin = MyPlugin(scheduler) scheduler.add_plugin(plugin) +You can also use the :meth:`distributed.diagnostics.plugin.SchedulerPlugin.start` method, which will receive a reference to the scheduler: + +.. code-block:: python + + from distributed.diagnostics.plugin import SchedulerPlugin + + class MyPlugin(SchedulerPlugin): + def __init__(self): + self.scheduler = None + + def start(self, scheduler): + self.scheduler = scheduler + + def transition(self, key, start, finish, *args, **kwargs): + # Get full TaskState + ts = self.scheduler.tasks[key] + + if __name__ == '__main__': + # Create and register the plugin in your main code + from distributed import Client, LocalCluster + + cluster = LocalCluster() + client = Client(cluster) + + plugin = MyPlugin() + client.register_plugin(plugin) + + Built-In Scheduler Plugins --------------------------