Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
19 changes: 12 additions & 7 deletions distributed/diagnostics/tests/test_worker_plugin.py
Original file line number Diff line number Diff line change
Expand Up @@ -188,18 +188,23 @@ async def test_dependent_tasks(c, s, w):
@gen_cluster(nthreads=[("127.0.0.1", 1)], client=True)
async def test_registering_with_name_arg(c, s, w):
class FooWorkerPlugin:
def setup(self, worker):
if hasattr(worker, "foo"):
raise RuntimeError(f"Worker {worker.address} already has foo!")
def __init__(self, value):
self.value = value

worker.foo = True
def setup(self, worker):
worker.foo = self.value

responses = await c.register_worker_plugin(FooWorkerPlugin(), name="foo")
responses = await c.register_worker_plugin(FooWorkerPlugin(23), name="foo")
assert list(responses.values()) == [{"status": "OK"}]
results = await c.run(lambda dask_worker: dask_worker.foo)
assert results[w.address] == 23

async with Worker(s.address, loop=s.loop):
responses = await c.register_worker_plugin(FooWorkerPlugin(), name="foo")
assert list(responses.values()) == [{"status": "repeat"}] * 2
responses = await c.register_worker_plugin(FooWorkerPlugin(42), name="foo")
assert list(responses.values()) == [{"status": "OK"}] * 2
results = await c.run(lambda dask_worker: dask_worker.foo)
assert len(results) == 2
assert all(v == 42 for v in results.values())


@gen_cluster(nthreads=[("127.0.0.1", 1)], client=True)
Expand Down
25 changes: 11 additions & 14 deletions distributed/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -2721,22 +2721,19 @@ async def plugin_add(self, comm=None, plugin=None, name=None):

assert name

if name in self.plugins:
return {"status": "repeat"}
else:
self.plugins[name] = plugin
self.plugins[name] = plugin

logger.info("Starting Worker plugin %s" % name)
if hasattr(plugin, "setup"):
try:
result = plugin.setup(worker=self)
if isawaitable(result):
result = await result
except Exception as e:
msg = error_message(e)
return msg
logger.info("Starting Worker plugin %s" % name)
if hasattr(plugin, "setup"):
try:
result = plugin.setup(worker=self)
if isawaitable(result):
result = await result
except Exception as e:
msg = error_message(e)
return msg

return {"status": "OK"}
return {"status": "OK"}

async def plugin_remove(self, comm=None, name=None):
with log_errors(pdb=False):
Expand Down