Skip to content

Commit 5bf60e3

Browse files
Add synced dict between cluster and scheduler to store cluster info (#5033)
Adds a `cluster_info` attribute to all `Cluster` objects which is a dictionary that is synced to the scheduler periodically. Any info already on the scheduler during `_start` is merged into the dict in `Cluster` and then that dict is synced back to the scheduler every second.
1 parent 3fba8f2 commit 5bf60e3

File tree

4 files changed

+69
-10
lines changed

4 files changed

+69
-10
lines changed

distributed/deploy/cluster.py

Lines changed: 38 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,5 @@
11
import asyncio
2+
import copy
23
import datetime
34
import logging
45
import threading
@@ -9,7 +10,7 @@
910
from tornado.ioloop import PeriodicCallback
1011

1112
import dask.config
12-
from dask.utils import _deprecated, format_bytes, parse_timedelta
13+
from dask.utils import _deprecated, format_bytes, parse_timedelta, typename
1314
from dask.widgets import get_template
1415

1516
from ..core import Status
@@ -44,9 +45,8 @@ class Cluster:
4445
"""
4546

4647
_supports_scaling = True
47-
name = None
4848

49-
def __init__(self, asynchronous, quiet=False, name=None):
49+
def __init__(self, asynchronous, quiet=False, name=None, scheduler_sync_interval=1):
5050
self.scheduler_info = {"workers": {}}
5151
self.periodic_callbacks = {}
5252
self._asynchronous = asynchronous
@@ -56,13 +56,24 @@ def __init__(self, asynchronous, quiet=False, name=None):
5656
self.quiet = quiet
5757
self.scheduler_comm = None
5858
self._adaptive = None
59+
self._sync_interval = parse_timedelta(
60+
scheduler_sync_interval, default="seconds"
61+
)
62+
63+
if name is None:
64+
name = str(uuid.uuid4())[:8]
5965

60-
if name is not None:
61-
self.name = name
62-
elif self.name is None:
63-
self.name = str(uuid.uuid4())[:8]
66+
self._cluster_info = {"name": name, "type": typename(type(self))}
6467
self.status = Status.created
6568

69+
@property
70+
def name(self):
71+
return self._cluster_info["name"]
72+
73+
@name.setter
74+
def name(self, name):
75+
self._cluster_info["name"] = name
76+
6677
async def _start(self):
6778
comm = await self.scheduler_comm.live_comm()
6879
await comm.write({"op": "subscribe_worker_status"})
@@ -71,8 +82,25 @@ async def _start(self):
7182
self._watch_worker_status_task = asyncio.ensure_future(
7283
self._watch_worker_status(comm)
7384
)
85+
86+
info = await self.scheduler_comm.get_metadata(
87+
keys=["cluster-manager-info"], default={}
88+
)
89+
self._cluster_info.update(info)
90+
91+
self.periodic_callbacks["sync-cluster-info"] = PeriodicCallback(
92+
self._sync_cluster_info, self._sync_interval * 1000
93+
)
94+
for pc in self.periodic_callbacks.values():
95+
pc.start()
7496
self.status = Status.running
7597

98+
async def _sync_cluster_info(self):
99+
await self.scheduler_comm.set_metadata(
100+
keys=["cluster-manager-info"],
101+
value=copy.copy(self._cluster_info),
102+
)
103+
76104
async def _close(self):
77105
if self.status == Status.closed:
78106
return
@@ -85,12 +113,12 @@ async def _close(self):
85113
if self._watch_worker_status_task:
86114
await self._watch_worker_status_task
87115

88-
for pc in self.periodic_callbacks.values():
89-
pc.stop()
90-
91116
if self.scheduler_comm:
92117
await self.scheduler_comm.close_rpc()
93118

119+
for pc in self.periodic_callbacks.values():
120+
pc.stop()
121+
94122
self.status = Status.closed
95123

96124
def close(self, timeout=None):

distributed/deploy/local.py

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -121,6 +121,7 @@ def __init__(
121121
interface=None,
122122
worker_class=None,
123123
scheduler_kwargs=None,
124+
scheduler_sync_interval=1,
124125
**worker_kwargs,
125126
):
126127
if ip is not None:
@@ -241,6 +242,7 @@ def __init__(
241242
asynchronous=asynchronous,
242243
silence_logs=silence_logs,
243244
security=security,
245+
scheduler_sync_interval=scheduler_sync_interval,
244246
)
245247

246248
def start_worker(self, *args, **kwargs):

distributed/deploy/spec.py

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -245,6 +245,7 @@ def __init__(
245245
silence_logs=False,
246246
name=None,
247247
shutdown_on_close=True,
248+
scheduler_sync_interval=1,
248249
):
249250
self._created = weakref.WeakSet()
250251

@@ -274,6 +275,7 @@ def __init__(
274275
super().__init__(
275276
asynchronous=asynchronous,
276277
name=name,
278+
scheduler_sync_interval=scheduler_sync_interval,
277279
)
278280

279281
if not self.asynchronous:

distributed/deploy/tests/test_local.py

Lines changed: 27 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1073,3 +1073,30 @@ async def test_local_cluster_redundant_kwarg(nanny):
10731073
async with Client(cluster) as c:
10741074
f = c.submit(sleep, 0)
10751075
await f
1076+
1077+
1078+
@pytest.mark.asyncio
1079+
async def test_cluster_info_sync():
1080+
async with LocalCluster(
1081+
processes=False, asynchronous=True, scheduler_sync_interval="1ms"
1082+
) as cluster:
1083+
assert cluster._cluster_info["name"] == cluster.name
1084+
1085+
while "name" not in cluster.scheduler.get_metadata(
1086+
keys=["cluster-manager-info"]
1087+
):
1088+
await asyncio.sleep(0.01)
1089+
1090+
info = await cluster.scheduler_comm.get_metadata(keys=["cluster-manager-info"])
1091+
assert info["name"] == cluster.name
1092+
info = cluster.scheduler.get_metadata(keys=["cluster-manager-info"])
1093+
assert info["name"] == cluster.name
1094+
1095+
cluster._cluster_info["foo"] = "bar"
1096+
while "foo" not in cluster.scheduler.get_metadata(
1097+
keys=["cluster-manager-info"]
1098+
):
1099+
await asyncio.sleep(0.01)
1100+
1101+
info = cluster.scheduler.get_metadata(keys=["cluster-manager-info"])
1102+
assert info["foo"] == "bar"

0 commit comments

Comments
 (0)