-
-
Notifications
You must be signed in to change notification settings - Fork 719
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Raise plugin exceptions on Worker.start() #4298
Changes from 13 commits
128c983
3224d42
3eae880
fa81d3d
cc3bec5
f85475f
dde7a09
df4a6cb
51340a5
f93ab5b
5604af5
d323913
7552423
f6a699a
560bb55
dc3aaeb
d44edb4
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -1494,9 +1494,23 @@ async def start(self): | |
|
||
setproctitle("dask-worker [%s]" % self.address) | ||
|
||
await asyncio.gather( | ||
plugins_msgs = await asyncio.gather( | ||
*(self.plugin_add(plugin=plugin) for plugin in self._pending_plugins) | ||
) | ||
plugins_exceptions = [] | ||
for msg in plugins_msgs: | ||
if msg["status"] != "OK": | ||
exc = pickle.loads(msg["exception"].data) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The pickle-unpickle here feels a little cumbersome to me. I'm not sure I love this idea, but what if you add a diff --git a/distributed/worker.py b/distributed/worker.py
index 6e2fc56b..f4e3ed03 100644
--- a/distributed/worker.py
+++ b/distributed/worker.py
@@ -1494,22 +1494,12 @@ class Worker(ServerNode):
setproctitle("dask-worker [%s]" % self.address)
- plugins_msgs = await asyncio.gather(
- *(self.plugin_add(plugin=plugin) for plugin in self._pending_plugins)
+ await asyncio.gather(
+ *(
+ self.plugin_add(plugin=plugin, catch_errors=False)
+ for plugin in self._pending_plugins
+ )
)
- plugins_exceptions = []
- for msg in plugins_msgs:
- if msg["status"] != "OK":
- exc = pickle.loads(msg["exception"].data)
- plugins_exceptions.append(pickle.loads(msg["exception"].data))
- if len(plugins_exceptions) >= 1:
- if len(plugins_exceptions) > 1:
- logger.error(
- "Multiple plugin exceptions raised. All exceptions will be logged, the first is raised."
- )
- for exc in plugins_exceptions:
- logger.error(repr(exc))
- raise plugins_exceptions[0]
self._pending_plugins = ()
@@ -3182,7 +3172,7 @@ class Worker(ServerNode):
def run_coroutine(self, comm, function, args=(), kwargs=None, wait=True):
return run(self, comm, function=function, args=args, kwargs=kwargs, wait=wait)
- async def plugin_add(self, comm=None, plugin=None, name=None):
+ async def plugin_add(self, comm=None, plugin=None, name=None, catch_errors=True):
with log_errors(pdb=False):
if isinstance(plugin, bytes):
plugin = pickle.loads(plugin)
@@ -3204,6 +3194,8 @@ class Worker(ServerNode):
if isawaitable(result):
result = await result
except Exception as e:
+ if not catch_errors:
+ raise
msg = error_message(e)
return msg If you wanted to maintain the behavior you have here where multiple plugin exceptions can be handled and logged, then you could pass There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Thanks for the suggestion, that indeed looks better.
I'm ok with having this or not, it was suggested by @fjetter in #4298 (comment) , for now I'm keeping this but we can remove it if people prefer. Relevant changes applied in 560bb55 . |
||
plugins_exceptions.append(pickle.loads(msg["exception"].data)) | ||
if len(plugins_exceptions) >= 1: | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't think so, that was the case with
and for
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Try having the plugin not raise an exception and see what happens. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, you're absolutely right, thanks for pointing it out. This is fixed now by dc3aaeb. Could you take another look when you have a chance? |
||
if len(plugins_exceptions) > 1: | ||
logger.error( | ||
"Multiple plugin exceptions raised. All exceptions will be logged, the first is raised." | ||
) | ||
for exc in plugins_exceptions: | ||
logger.error(repr(exc)) | ||
raise plugins_exceptions[0] | ||
|
||
self._pending_plugins = () | ||
|
||
await self._register_with_scheduler() | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's also possible for
plugin_add
to raise other exceptions (e.g. unpickling errors). What would happen in these cases? I assume it is failing hard but I would like to confirm. Do we have tests for this?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Added test for that in d323913 .