Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add synced dict between cluster and scheduler to store cluster info #5033

Merged
merged 14 commits into from
Sep 9, 2021
Merged
Show file tree
Hide file tree
Changes from 5 commits
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
58 changes: 48 additions & 10 deletions distributed/deploy/cluster.py
Original file line number Diff line number Diff line change
@@ -1,19 +1,29 @@
import asyncio
import contextlib
import copy
import datetime
import logging
import threading
import uuid
from contextlib import suppress
from inspect import isawaitable

from tornado.ioloop import PeriodicCallback
from tornado.ioloop import IOLoop, PeriodicCallback

import dask.config
from dask.utils import _deprecated, format_bytes, parse_timedelta

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,
jacobtomlinson marked this conversation as resolved.
Show resolved Hide resolved
)
from .adaptive import Adaptive

logger = logging.getLogger(__name__)
Expand Down Expand Up @@ -45,23 +55,35 @@ class Cluster:
_supports_scaling = True
name = None
jacobtomlinson marked this conversation as resolved.
Show resolved Hide resolved

def __init__(self, asynchronous, quiet=False, name=None):
def __init__(self, asynchronous, quiet=False, name=None, scheduler_sync_interval=1):
self.scheduler_info = {"workers": {}}
self.periodic_callbacks = {}
self._asynchronous = asynchronous
self._watch_worker_status_comm = None
self._watch_worker_status_task = None
self._sync_cluster_info_task = None
jacobtomlinson marked this conversation as resolved.
Show resolved Hide resolved
self._cluster_manager_logs = []
self.quiet = quiet
self.scheduler_comm = None
self._adaptive = None
self._sync_interval = parse_timedelta(
scheduler_sync_interval, default="seconds"
)

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

if name is not None:
self.name = name
elif self.name is None:
self.name = str(uuid.uuid4())[:8]
self.cluster_info = {"name": name, "type": typename(type(self))}
jacobtomlinson marked this conversation as resolved.
Show resolved Hide resolved
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 +92,24 @@ async def _start(self):
self._watch_worker_status_task = asyncio.ensure_future(
self._watch_worker_status(comm)
)

with contextlib.suppress(KeyError): # The scheduler might not have any info
self.cluster_info.update(
(await self.scheduler_comm.get_metadata(keys=["cluster-manager-info"]))
)
self.loop = IOLoop.current()
self.periodic_callbacks["sync-cluster-info"] = pc = PeriodicCallback(
self._sync_cluster_info, self._sync_interval * 1000
)
self.loop.add_callback(pc.start)
jacobtomlinson marked this conversation as resolved.
Show resolved Hide resolved
self.status = Status.running

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

async def _close(self):
if self.status == Status.closed:
return
Expand All @@ -84,12 +122,12 @@ async def _close(self):
if self._watch_worker_status_task:
await self._watch_worker_status_task

for pc in self.periodic_callbacks.values():
pc.stop()

if self.scheduler_comm:
await self.scheduler_comm.close_rpc()

for pc in self.periodic_callbacks.values():
pc.stop()

self.status = Status.closed

def close(self, timeout=None):
Expand Down
2 changes: 2 additions & 0 deletions distributed/deploy/local.py
Original file line number Diff line number Diff line change
Expand Up @@ -120,6 +120,7 @@ def __init__(
interface=None,
worker_class=None,
scheduler_kwargs=None,
scheduler_sync_interval=1,
**worker_kwargs,
):
if ip is not None:
Expand Down Expand Up @@ -240,6 +241,7 @@ def __init__(
asynchronous=asynchronous,
silence_logs=silence_logs,
security=security,
scheduler_sync_interval=scheduler_sync_interval,
)

def start_worker(self, *args, **kwargs):
Expand Down
2 changes: 2 additions & 0 deletions distributed/deploy/spec.py
Original file line number Diff line number Diff line change
Expand Up @@ -241,6 +241,7 @@ def __init__(
silence_logs=False,
name=None,
shutdown_on_close=True,
scheduler_sync_interval=1,
):
self._created = weakref.WeakSet()

Expand Down Expand Up @@ -270,6 +271,7 @@ def __init__(
super().__init__(
asynchronous=asynchronous,
name=name,
scheduler_sync_interval=scheduler_sync_interval,
)

if not self.asynchronous:
Expand Down
17 changes: 17 additions & 0 deletions distributed/deploy/tests/test_local.py
Original file line number Diff line number Diff line change
Expand Up @@ -1079,3 +1079,20 @@ 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, scheduler_sync_interval="2ms"
) as cluster:
assert cluster.cluster_info["name"] == cluster.name
await asyncio.sleep(0.005)
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(0.005)
assert "foo" in cluster.scheduler.get_metadata(keys=["cluster-manager-info"])