|
10 | 10 | from collections.abc import Awaitable, Generator
|
11 | 11 | from contextlib import suppress
|
12 | 12 | from inspect import isawaitable
|
| 13 | +from time import time |
13 | 14 | from typing import TYPE_CHECKING, Any, ClassVar, TypeVar
|
14 | 15 |
|
15 | 16 | from tornado import gen
|
@@ -389,28 +390,64 @@ async def _correct_state_internal(self) -> None:
|
389 | 390 | # proper teardown.
|
390 | 391 | await asyncio.gather(*worker_futs)
|
391 | 392 |
|
392 |
| - def _update_worker_status(self, op, msg): |
| 393 | + def _update_worker_status(self, op, worker_addr): |
393 | 394 | if op == "remove":
|
394 |
| - name = self.scheduler_info["workers"][msg]["name"] |
| 395 | + worker_info = self.scheduler_info["workers"][worker_addr].copy() |
| 396 | + name = worker_info["name"] |
| 397 | + |
| 398 | + from distributed import Nanny, Worker |
395 | 399 |
|
396 | 400 | def f():
|
| 401 | + # FIXME: SpecCluster is tracking workers by `name`` which are |
| 402 | + # not necessarily unique. |
| 403 | + # Clusters with Nannies (default) are susceptible to falsely |
| 404 | + # removing the Nannies on restart due to this logic since the |
| 405 | + # restart emits a op==remove signal on the worker address but |
| 406 | + # the SpecCluster only tracks the names, i.e. after |
| 407 | + # `lost-worker-timeout` the Nanny is still around and this logic |
| 408 | + # could trigger a false close. The below code should handle this |
| 409 | + # but it would be cleaner if the cluster tracked by address |
| 410 | + # instead of name just like the scheduler does |
397 | 411 | if (
|
398 | 412 | name in self.workers
|
399 |
| - and msg not in self.scheduler_info["workers"] |
| 413 | + and worker_addr not in self.scheduler_info["workers"] |
400 | 414 | and not any(
|
401 | 415 | d["name"] == name
|
402 | 416 | for d in self.scheduler_info["workers"].values()
|
403 | 417 | )
|
404 | 418 | ):
|
405 |
| - self._futures.add(asyncio.ensure_future(self.workers[name].close())) |
406 |
| - del self.workers[name] |
| 419 | + w = self.workers[name] |
| 420 | + |
| 421 | + async def remove_worker(): |
| 422 | + await w.close(reason=f"lost-worker-timeout-{time()}") |
| 423 | + self.workers.pop(name, None) |
| 424 | + |
| 425 | + if ( |
| 426 | + worker_info["type"] == "Worker" |
| 427 | + and (isinstance(w, Nanny) and w.worker_address == worker_addr) |
| 428 | + or (isinstance(w, Worker) and w.address == worker_addr) |
| 429 | + ): |
| 430 | + self._futures.add( |
| 431 | + asyncio.create_task( |
| 432 | + remove_worker(), |
| 433 | + name="remove-worker-lost-worker-timeout", |
| 434 | + ) |
| 435 | + ) |
| 436 | + elif worker_info["type"] == "Nanny": |
| 437 | + # This should never happen |
| 438 | + logger.critical( |
| 439 | + "Unespected signal encountered. WorkerStatusPlugin " |
| 440 | + "emitted a op==remove signal for a Nanny which " |
| 441 | + "should not happen. This might cause a lingering " |
| 442 | + "Nanny process." |
| 443 | + ) |
407 | 444 |
|
408 | 445 | delay = parse_timedelta(
|
409 | 446 | dask.config.get("distributed.deploy.lost-worker-timeout")
|
410 | 447 | )
|
411 | 448 |
|
412 | 449 | asyncio.get_running_loop().call_later(delay, f)
|
413 |
| - super()._update_worker_status(op, msg) |
| 450 | + super()._update_worker_status(op, worker_addr) |
414 | 451 |
|
415 | 452 | def __await__(self: Self) -> Generator[Any, Any, Self]:
|
416 | 453 | async def _() -> Self:
|
|
0 commit comments