Skip to content

Commit

Permalink
Log rather than raise exceptions in preload.teardown
Browse files Browse the repository at this point in the history
  • Loading branch information
mrocklin committed May 26, 2022
1 parent 5e9e97f commit 440219f
Show file tree
Hide file tree
Showing 3 changed files with 19 additions and 2 deletions.
5 changes: 4 additions & 1 deletion distributed/scheduler.py
Original file line number Diff line number Diff line change
Expand Up @@ -3368,7 +3368,10 @@ async def close(self):
setproctitle("dask-scheduler [closing]")

for preload in self.preloads:
await preload.teardown()
try:
await preload.teardown()
except Exception as e:
logger.exception(e)

await asyncio.gather(
*[plugin.close() for plugin in list(self.plugins.values())]
Expand Down
11 changes: 11 additions & 0 deletions distributed/tests/test_preload.py
Original file line number Diff line number Diff line change
Expand Up @@ -302,3 +302,14 @@ def dask_setup(client, value):
):
async with Client(address=s.address, asynchronous=True) as c:
assert c.foo == value


@gen_test()
async def test_teardown_failure_doesnt_crash_scheduler():
text = """
def dask_teardown(worker):
raise Exception(123)
"""
async with Scheduler(dashboard_address=":0", preload=text) as s:
async with Worker(s.address, preload=[text]) as w:
pass
5 changes: 4 additions & 1 deletion distributed/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -1489,7 +1489,10 @@ async def close(
)

for preload in self.preloads:
await preload.teardown()
try:
await preload.teardown()
except Exception as e:
logger.exception(e)

for extension in self.extensions.values():
if hasattr(extension, "close"):
Expand Down

0 comments on commit 440219f

Please sign in to comment.