Skip to content

Commit

Permalink
Restore ordering of worker update in _correct_state_internal
Browse files Browse the repository at this point in the history
Prior to #8233, when correcting state after calling Cluster.scale, we
would wait until all futures had completed before updating the mapping
of workers that we knew about. This meant that failure to boot a
worker would propagate from a message on the worker side to an
exception on the cluster side. With #8233 this order was changed, so
that the workers we know about are updated before checking if the
worker successfully booted. With this change, any exception is not
propagated from the worker to the cluster, and so we cannot easily
tell if scaling our cluster was successful. While _correct_state has
issues (see #5919) until we can fix this properly, at least restore
the old behaviour of propagating any raised exceptions to the cluster.
  • Loading branch information
wence- committed Oct 30, 2023
1 parent b62b700 commit cfe5906
Show file tree
Hide file tree
Showing 2 changed files with 24 additions and 1 deletion.
5 changes: 4 additions & 1 deletion distributed/deploy/spec.py
Original file line number Diff line number Diff line change
Expand Up @@ -381,13 +381,16 @@ async def _correct_state_internal(self) -> None:
if workers:
worker_futs = [asyncio.ensure_future(w) for w in workers]
await asyncio.wait(worker_futs)
self.workers.update(dict(zip(to_open, workers)))
for w in workers:
w._cluster = weakref.ref(self)
# Collect exceptions from failed workers. This must happen after all
# *other* workers have finished initialising, so that we can have a
# proper teardown.
await asyncio.gather(*worker_futs)
# And remember the new workers, note this doesn't
# handle partial failure since the gather above would
# raise in that case. See also #5919.
self.workers.update(dict(zip(to_open, workers)))

Check warning on line 393 in distributed/deploy/spec.py

View check run for this annotation

Codecov / codecov/patch

distributed/deploy/spec.py#L393

Added line #L393 was not covered by tests

def _update_worker_status(self, op, msg):
if op == "remove":
Expand Down
20 changes: 20 additions & 0 deletions distributed/deploy/tests/test_local.py
Original file line number Diff line number Diff line change
Expand Up @@ -1275,6 +1275,26 @@ def test_localcluster_start_exception(loop):
pass


def test_localcluster_scale_exception(loop):
with raises_with_cause(
RuntimeError,
"Nanny failed to start",
RuntimeError,
"Worker failed to start",
ImportError,
"my_nonexistent_library",
):
with LocalCluster(
n_workers=0,
threads_per_worker=1,
processes=True,
plugins={MyPlugin()},
loop=loop,
) as cluster:
cluster.scale(1)
cluster.sync(cluster._correct_state)


def test_localcluster_get_client(loop):
with LocalCluster(
n_workers=0, asynchronous=False, dashboard_address=":0", loop=loop
Expand Down

0 comments on commit cfe5906

Please sign in to comment.