Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
43 changes: 38 additions & 5 deletions distributed/deploy/cluster.py
Original file line number Diff line number Diff line change
@@ -1,4 +1,6 @@
import asyncio
import contextlib
import copy
import datetime
import logging
import threading
Expand All @@ -13,7 +15,15 @@

from ..core import Status
from ..objects import SchedulerInfo
from ..utils import Log, Logs, format_dashboard_link, log_errors, sync, thread_state
from ..utils import (
Log,
Logs,
format_dashboard_link,
log_errors,
sync,
thread_state,
typename,
)
from .adaptive import Adaptive

logger = logging.getLogger(__name__)
Expand Down Expand Up @@ -51,17 +61,26 @@ def __init__(self, asynchronous, quiet=False, name=None):
self._asynchronous = asynchronous
self._watch_worker_status_comm = None
self._watch_worker_status_task = None
self._sync_cluster_info_task = None
self._cluster_manager_logs = []
self.quiet = quiet
self.scheduler_comm = None
self._adaptive = None

if name is not None:
self.name = name
elif self.name is None:
self.name = str(uuid.uuid4())[:8]
if name is None:
name = str(uuid.uuid4())[:8]

self.cluster_info = {"name": name, "type": typename(type(self))}
self.status = Status.created

@property
def name(self):
return self.cluster_info["name"]

@name.setter
def name(self, name):
self.cluster_info["name"] = name

async def _start(self):
comm = await self.scheduler_comm.live_comm()
await comm.write({"op": "subscribe_worker_status"})
Expand All @@ -70,8 +89,22 @@ async def _start(self):
self._watch_worker_status_task = asyncio.ensure_future(
self._watch_worker_status(comm)
)

with contextlib.suppress(KeyError):
self.cluster_info.update(
(await self.scheduler_comm.get_metadata(keys=["cluster-manager-info"]))
)
self._sync_cluster_info_task = asyncio.ensure_future(self._sync_cluster_info())
self.status = Status.running

async def _sync_cluster_info(self):
while True:
await self.scheduler_comm.set_metadata(
keys=["cluster-manager-info"],
value=copy.copy(self.cluster_info),
)
await asyncio.sleep(1)

async def _close(self):
if self.status == Status.closed:
return
Expand Down
14 changes: 14 additions & 0 deletions distributed/deploy/tests/test_local.py
Original file line number Diff line number Diff line change
Expand Up @@ -1079,3 +1079,17 @@ async def test_local_cluster_redundant_kwarg(nanny):
async with Client(cluster) as c:
f = c.submit(sleep, 0)
await f


@pytest.mark.asyncio
async def test_cluster_info_sync():
async with LocalCluster(processes=False, asynchronous=True) as cluster:
assert cluster.cluster_info["name"] == cluster.name
info = await cluster.scheduler_comm.get_metadata(keys=["cluster-manager-info"])
assert info["name"] == cluster.name
info = cluster.scheduler.get_metadata(keys=["cluster-manager-info"])
assert info["name"] == cluster.name

cluster.cluster_info["foo"] = "bar"
await asyncio.sleep(2)
assert "foo" in cluster.scheduler.get_metadata(keys=["cluster-manager-info"])