11import asyncio
2+ import copy
23import datetime
34import logging
45import threading
910from tornado .ioloop import PeriodicCallback
1011
1112import dask .config
12- from dask .utils import _deprecated , format_bytes , parse_timedelta
13+ from dask .utils import _deprecated , format_bytes , parse_timedelta , typename
1314from dask .widgets import get_template
1415
1516from ..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 ):
0 commit comments