-
-
Notifications
You must be signed in to change notification settings - Fork 710
/
core.py
1863 lines (1559 loc) · 60.1 KB
/
core.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
from __future__ import annotations
import asyncio
import inspect
import logging
import math
import os
import sys
import tempfile
import threading
import traceback
import types
import uuid
import warnings
import weakref
from collections import defaultdict, deque
from collections.abc import (
Awaitable,
Callable,
Container,
Coroutine,
Generator,
Hashable,
)
from enum import Enum
from functools import wraps
from typing import TYPE_CHECKING, Any, ClassVar, Literal, TypedDict, TypeVar, final
import tblib
from tlz import merge
from tornado.ioloop import IOLoop
import dask
from dask.utils import parse_timedelta
from distributed import profile, protocol
from distributed.comm import (
Comm,
CommClosedError,
connect,
get_address_host_port,
listen,
normalize_address,
unparse_host_port,
)
from distributed.comm.core import Listener
from distributed.compatibility import PeriodicCallback
from distributed.counter import Counter
from distributed.diskutils import WorkDir, WorkSpace
from distributed.metrics import context_meter, time
from distributed.system_monitor import SystemMonitor
from distributed.utils import (
NoOpAwaitable,
get_traceback,
has_keyword,
import_file,
is_python_shutting_down,
iscoroutinefunction,
offload,
recursive_to_dict,
truncate_exception,
wait_for,
warn_on_duration,
)
if TYPE_CHECKING:
from typing_extensions import ParamSpec, Self
from distributed.counter import Digest
P = ParamSpec("P")
R = TypeVar("R")
T = TypeVar("T")
Coro = Coroutine[Any, Any, T]
class Status(Enum):
"""
This Enum contains the various states a cluster, worker, scheduler and nanny can be
in. Some of the status can only be observed in one of cluster, nanny, scheduler or
worker but we put them in the same Enum as they are compared with each
other.
"""
undefined = "undefined"
created = "created"
init = "init"
starting = "starting"
running = "running"
paused = "paused"
stopping = "stopping"
stopped = "stopped"
closing = "closing"
closing_gracefully = "closing_gracefully"
closed = "closed"
failed = "failed"
dont_reply = "dont_reply"
Status.lookup = {s.name: s for s in Status} # type: ignore
class RPCClosed(IOError):
pass
logger = logging.getLogger(__name__)
def raise_later(exc):
def _raise(*args, **kwargs):
raise exc
return _raise
tick_maximum_delay = parse_timedelta(
dask.config.get("distributed.admin.tick.limit"), default="ms"
)
LOG_PDB = dask.config.get("distributed.admin.pdb-on-err")
def _expects_comm(func: Callable) -> bool:
sig = inspect.signature(func)
params = list(sig.parameters)
if params and params[0] == "comm":
return True
if params and params[0] == "stream":
warnings.warn(
"Calling the first argument of a RPC handler `stream` is "
"deprecated. Defining this argument is optional. Either remove the "
f"argument or rename it to `comm` in {func}.",
FutureWarning,
)
return True
return False
class _LoopBoundMixin:
"""Backport of the private asyncio.mixins._LoopBoundMixin from 3.11"""
_global_lock = threading.Lock()
_loop = None
def _get_loop(self):
loop = asyncio.get_running_loop()
if self._loop is None:
with self._global_lock:
if self._loop is None:
self._loop = loop
if loop is not self._loop:
raise RuntimeError(f"{self!r} is bound to a different event loop")
return loop
class AsyncTaskGroupClosedError(RuntimeError):
pass
def _delayed(corofunc: Callable[P, Coro[T]], delay: float) -> Callable[P, Coro[T]]:
"""Decorator to delay the evaluation of a coroutine function by the given delay in seconds."""
async def wrapper(*args: P.args, **kwargs: P.kwargs) -> T:
await asyncio.sleep(delay)
return await corofunc(*args, **kwargs)
return wrapper
class AsyncTaskGroup(_LoopBoundMixin):
"""Collection tracking all currently running asynchronous tasks within a group"""
#: If True, the group is closed and does not allow adding new tasks.
closed: bool
def __init__(self) -> None:
self.closed = False
self._ongoing_tasks: set[asyncio.Task[None]] = set()
def call_soon(
self, afunc: Callable[P, Coro[None]], /, *args: P.args, **kwargs: P.kwargs
) -> None:
"""Schedule a coroutine function to be executed as an `asyncio.Task`.
The coroutine function `afunc` is scheduled with `args` arguments and `kwargs` keyword arguments
as an `asyncio.Task`.
Parameters
----------
afunc
Coroutine function to schedule.
*args
Arguments to be passed to `afunc`.
**kwargs
Keyword arguments to be passed to `afunc`
Returns
-------
None
Raises
------
AsyncTaskGroupClosedError
If the task group is closed.
"""
if self.closed: # Avoid creating a coroutine
raise AsyncTaskGroupClosedError(
"Cannot schedule a new coroutine function as the group is already closed."
)
task = self._get_loop().create_task(afunc(*args, **kwargs))
task.add_done_callback(self._ongoing_tasks.remove)
self._ongoing_tasks.add(task)
return None
def call_later(
self,
delay: float,
afunc: Callable[P, Coro[None]],
/,
*args: P.args,
**kwargs: P.kwargs,
) -> None:
"""Schedule a coroutine function to be executed after `delay` seconds as an `asyncio.Task`.
The coroutine function `afunc` is scheduled with `args` arguments and `kwargs` keyword arguments
as an `asyncio.Task` that is executed after `delay` seconds.
Parameters
----------
delay
Delay in seconds.
afunc
Coroutine function to schedule.
*args
Arguments to be passed to `afunc`.
**kwargs
Keyword arguments to be passed to `afunc`
Returns
-------
The None
Raises
------
AsyncTaskGroupClosedError
If the task group is closed.
"""
self.call_soon(_delayed(afunc, delay), *args, **kwargs)
def close(self) -> None:
"""Closes the task group so that no new tasks can be scheduled.
Existing tasks continue to run.
"""
self.closed = True
async def stop(self) -> None:
"""Close the group and stop all currently running tasks.
Closes the task group and cancels all tasks. All tasks are cancelled
an additional time for each time this task is cancelled.
"""
self.close()
current_task = asyncio.current_task(self._get_loop())
err = None
while tasks_to_stop := (self._ongoing_tasks - {current_task}):
for task in tasks_to_stop:
task.cancel()
try:
await asyncio.wait(tasks_to_stop)
except asyncio.CancelledError as e:
err = e
if err is not None:
raise err
def __len__(self):
return len(self._ongoing_tasks)
class Server:
"""Dask Distributed Server
Superclass for endpoints in a distributed cluster, such as Worker
and Scheduler objects.
**Handlers**
Servers define operations with a ``handlers`` dict mapping operation names
to functions. The first argument of a handler function will be a ``Comm``
for the communication established with the client. Other arguments
will receive inputs from the keys of the incoming message which will
always be a dictionary.
>>> def pingpong(comm):
... return b'pong'
>>> def add(comm, x, y):
... return x + y
>>> handlers = {'ping': pingpong, 'add': add}
>>> server = Server(handlers) # doctest: +SKIP
>>> server.listen('tcp://0.0.0.0:8000') # doctest: +SKIP
**Message Format**
The server expects messages to be dictionaries with a special key, `'op'`
that corresponds to the name of the operation, and other key-value pairs as
required by the function.
So in the example above the following would be good messages.
* ``{'op': 'ping'}``
* ``{'op': 'add', 'x': 10, 'y': 20}``
"""
default_ip: ClassVar[str] = ""
default_port: ClassVar[int] = 0
id: str
blocked_handlers: list[str]
handlers: dict[str, Callable]
stream_handlers: dict[str, Callable]
listeners: list[Listener]
counters: defaultdict[str, Counter]
deserialize: bool
local_directory: str
monitor: SystemMonitor
io_loop: IOLoop
thread_id: int
periodic_callbacks: dict[str, PeriodicCallback]
digests: defaultdict[Hashable, Digest] | None
digests_total: defaultdict[Hashable, float]
digests_total_since_heartbeat: defaultdict[Hashable, float]
digests_max: defaultdict[Hashable, float]
_last_tick: float
_tick_counter: int
_last_tick_counter: int
_tick_interval: float
_tick_interval_observed: float
_status: Status
_address: str | None
_listen_address: str | None
_host: str | None
_port: int | None
_comms: dict[Comm, str | None]
_ongoing_background_tasks: AsyncTaskGroup
_event_finished: asyncio.Event
_original_local_dir: str
_updated_sys_path: bool
_workspace: WorkSpace
_workdir: None | WorkDir
_startup_lock: asyncio.Lock
__startup_exc: Exception | None
def __init__(
self,
handlers,
blocked_handlers=None,
stream_handlers=None,
connection_limit=512,
deserialize=True,
serializers=None,
deserializers=None,
connection_args=None,
timeout=None,
io_loop=None,
local_directory=None,
needs_workdir=True,
):
if local_directory is None:
local_directory = (
dask.config.get("temporary-directory") or tempfile.gettempdir()
)
if "dask-scratch-space" not in str(local_directory):
local_directory = os.path.join(local_directory, "dask-scratch-space")
self._original_local_dir = local_directory
with warn_on_duration(
"1s",
"Creating scratch directories is taking a surprisingly long time. ({duration:.2f}s) "
"This is often due to running workers on a network file system. "
"Consider specifying a local-directory to point workers to write "
"scratch data to a local disk.",
):
self._workspace = WorkSpace(local_directory)
if not needs_workdir: # eg. Nanny will not need a WorkDir
self._workdir = None
self.local_directory = self._workspace.base_dir
else:
name = type(self).__name__.lower()
self._workdir = self._workspace.new_work_dir(prefix=f"{name}-")
self.local_directory = self._workdir.dir_path
self._updated_sys_path = False
if self.local_directory not in sys.path:
sys.path.insert(0, self.local_directory)
self._updated_sys_path = True
if io_loop is not None:
warnings.warn(
"The io_loop kwarg to Server is ignored and will be deprecated",
DeprecationWarning,
stacklevel=2,
)
self._status = Status.init
self.handlers = {
"identity": self.identity,
"echo": self.echo,
"connection_stream": self.handle_stream,
"dump_state": self._to_dict,
}
self.handlers.update(handlers)
if blocked_handlers is None:
blocked_handlers = dask.config.get(
"distributed.%s.blocked-handlers" % type(self).__name__.lower(), []
)
self.blocked_handlers = blocked_handlers
self.stream_handlers = {}
self.stream_handlers.update(stream_handlers or {})
self.id = type(self).__name__ + "-" + str(uuid.uuid4())
self._address = None
self._listen_address = None
self._port = None
self._host = None
self._comms = {}
self.deserialize = deserialize
self.monitor = SystemMonitor()
self._ongoing_background_tasks = AsyncTaskGroup()
self._event_finished = asyncio.Event()
self.listeners = []
self.io_loop = self.loop = IOLoop.current()
if not hasattr(self.io_loop, "profile"):
if dask.config.get("distributed.worker.profile.enabled"):
ref = weakref.ref(self.io_loop)
def stop() -> bool:
loop = ref()
return loop is None or loop.asyncio_loop.is_closed()
self.io_loop.profile = profile.watch(
omit=("profile.py", "selectors.py"),
interval=dask.config.get("distributed.worker.profile.interval"),
cycle=dask.config.get("distributed.worker.profile.cycle"),
stop=stop,
)
else:
self.io_loop.profile = deque()
self.periodic_callbacks = {}
# Statistics counters for various events
try:
from distributed.counter import Digest
self.digests = defaultdict(Digest)
except ImportError:
self.digests = None
# Also log cumulative totals (reset at server restart)
# and local maximums (reset by prometheus poll)
# Don't cast int metrics to float
self.digests_total = defaultdict(int)
self.digests_total_since_heartbeat = defaultdict(int)
self.digests_max = defaultdict(int)
self.counters = defaultdict(Counter)
pc = PeriodicCallback(self._shift_counters, 5000)
self.periodic_callbacks["shift_counters"] = pc
pc = PeriodicCallback(
self.monitor.update,
parse_timedelta(
dask.config.get("distributed.admin.system-monitor.interval")
)
* 1000,
)
self.periodic_callbacks["monitor"] = pc
self._last_tick = time()
self._tick_counter = 0
self._last_tick_counter = 0
self._last_tick_cycle = time()
self._tick_interval = parse_timedelta(
dask.config.get("distributed.admin.tick.interval"), default="ms"
)
self._tick_interval_observed = self._tick_interval
self.periodic_callbacks["tick"] = PeriodicCallback(
self._measure_tick, self._tick_interval * 1000
)
self.periodic_callbacks["ticks"] = PeriodicCallback(
self._cycle_ticks,
parse_timedelta(dask.config.get("distributed.admin.tick.cycle")) * 1000,
)
self.thread_id = 0
def set_thread_ident():
self.thread_id = threading.get_ident()
self.io_loop.add_callback(set_thread_ident)
self._startup_lock = asyncio.Lock()
self.__startup_exc = None
self.rpc = ConnectionPool(
limit=connection_limit,
deserialize=deserialize,
serializers=serializers,
deserializers=deserializers,
connection_args=connection_args,
timeout=timeout,
server=self,
)
self.__stopped = False
async def upload_file(
self, filename: str, data: str | bytes, load: bool = True
) -> dict[str, Any]:
out_filename = os.path.join(self.local_directory, filename)
def func(data):
if isinstance(data, str):
data = data.encode()
with open(out_filename, "wb") as f:
f.write(data)
f.flush()
os.fsync(f.fileno())
return data
if len(data) < 10000:
data = func(data)
else:
data = await offload(func, data)
if load:
try:
import_file(out_filename)
except Exception as e:
logger.exception(e)
raise e
return {"status": "OK", "nbytes": len(data)}
def _shift_counters(self):
for counter in self.counters.values():
counter.shift()
if self.digests is not None:
for digest in self.digests.values():
digest.shift()
@property
def status(self) -> Status:
try:
return self._status
except AttributeError:
return Status.undefined
@status.setter
def status(self, value: Status) -> None:
if not isinstance(value, Status):
raise TypeError(f"Expected Status; got {value!r}")
self._status = value
@property
def incoming_comms_open(self) -> int:
"""The number of total incoming connections listening to remote RPCs"""
return len(self._comms)
@property
def incoming_comms_active(self) -> int:
"""The number of connections currently handling a remote RPC"""
return len([c for c, op in self._comms.items() if op is not None])
@property
def outgoing_comms_open(self) -> int:
"""The number of connections currently open and waiting for a remote RPC"""
return self.rpc.open
@property
def outgoing_comms_active(self) -> int:
"""The number of outgoing connections that are currently used to
execute a RPC"""
return self.rpc.active
def get_connection_counters(self) -> dict[str, int]:
"""A dict with various connection counters
See also
--------
Server.incoming_comms_open
Server.incoming_comms_active
Server.outgoing_comms_open
Server.outgoing_comms_active
"""
return {
attr: getattr(self, attr)
for attr in [
"incoming_comms_open",
"incoming_comms_active",
"outgoing_comms_open",
"outgoing_comms_active",
]
}
async def finished(self):
"""Wait until the server has finished"""
await self._event_finished.wait()
def __await__(self):
return self.start().__await__()
async def start_unsafe(self):
"""Attempt to start the server. This is not idempotent and not protected against concurrent startup attempts.
This is intended to be overwritten or called by subclasses. For a safe
startup, please use ``Server.start`` instead.
If ``death_timeout`` is configured, we will require this coroutine to
finish before this timeout is reached. If the timeout is reached we will
close the instance and raise an ``asyncio.TimeoutError``
"""
await self.rpc.start()
return self
@final
async def start(self):
async with self._startup_lock:
if self.status == Status.failed:
assert self.__startup_exc is not None
raise self.__startup_exc
elif self.status != Status.init:
return self
timeout = getattr(self, "death_timeout", None)
async def _close_on_failure(exc: Exception) -> None:
await self.close(reason=f"failure-to-start-{str(type(exc))}")
self.status = Status.failed
self.__startup_exc = exc
try:
await wait_for(self.start_unsafe(), timeout=timeout)
except asyncio.TimeoutError as exc:
await _close_on_failure(exc)
raise asyncio.TimeoutError(
f"{type(self).__name__} start timed out after {timeout}s."
) from exc
except Exception as exc:
await _close_on_failure(exc)
raise RuntimeError(f"{type(self).__name__} failed to start.") from exc
if self.status == Status.init:
self.status = Status.running
return self
async def __aenter__(self):
await self
return self
async def __aexit__(self, exc_type, exc_value, traceback):
await self.close()
def start_periodic_callbacks(self):
"""Start Periodic Callbacks consistently
This starts all PeriodicCallbacks stored in self.periodic_callbacks if
they are not yet running. It does this safely by checking that it is using the
correct event loop.
"""
if self.io_loop.asyncio_loop is not asyncio.get_running_loop():
raise RuntimeError(f"{self!r} is bound to a different event loop")
self._last_tick = time()
for pc in self.periodic_callbacks.values():
if not pc.is_running():
pc.start()
def _stop_listeners(self) -> asyncio.Future:
listeners_to_stop: set[Awaitable] = set()
for listener in self.listeners:
future = listener.stop()
if inspect.isawaitable(future):
warnings.warn(
f"{type(listener)} is using an asynchronous `stop` method. "
"Support for asynchronous `Listener.stop` has been deprecated and "
"will be removed in a future version",
DeprecationWarning,
)
listeners_to_stop.add(future)
elif hasattr(listener, "abort_handshaking_comms"):
listener.abort_handshaking_comms()
return asyncio.gather(*listeners_to_stop)
def stop(self) -> None:
if self.__stopped:
return
self.__stopped = True
self.monitor.close()
if not (stop_listeners := self._stop_listeners()).done():
self._ongoing_background_tasks.call_soon(
asyncio.wait_for(stop_listeners, timeout=None) # type: ignore[arg-type]
)
if self._workdir is not None:
self._workdir.release()
@property
def listener(self) -> Listener | None:
if self.listeners:
return self.listeners[0]
else:
return None
def _measure_tick(self):
now = time()
tick_duration = now - self._last_tick
self._last_tick = now
self._tick_counter += 1
# This metric is exposed in Prometheus and is reset there during
# collection
if tick_duration > tick_maximum_delay:
logger.info(
"Event loop was unresponsive in %s for %.2fs. "
"This is often caused by long-running GIL-holding "
"functions or moving large chunks of data. "
"This can cause timeouts and instability.",
type(self).__name__,
tick_duration,
)
self.digest_metric("tick-duration", tick_duration)
def _cycle_ticks(self):
if not self._tick_counter:
return
now = time()
last_tick_cycle, self._last_tick_cycle = self._last_tick_cycle, now
count = self._tick_counter - self._last_tick_counter
self._last_tick_counter = self._tick_counter
self._tick_interval_observed = (now - last_tick_cycle) / (count or 1)
@property
def address(self) -> str:
"""
The address this Server can be contacted on.
If the server is not up, yet, this raises a ValueError.
"""
if not self._address:
if self.listener is None:
raise ValueError("cannot get address of non-running Server")
self._address = self.listener.contact_address
assert self._address
return self._address
@property
def address_safe(self) -> str:
"""
The address this Server can be contacted on.
If the server is not up, yet, this returns a ``"not-running"``.
"""
try:
return self.address
except ValueError:
return "not-running"
@property
def listen_address(self):
"""
The address this Server is listening on. This may be a wildcard
address such as `tcp://0.0.0.0:1234`.
"""
if not self._listen_address:
if self.listener is None:
raise ValueError("cannot get listen address of non-running Server")
self._listen_address = self.listener.listen_address
return self._listen_address
@property
def host(self):
"""
The host this Server is running on.
This will raise ValueError if the Server is listening on a
non-IP based protocol.
"""
if not self._host:
self._host, self._port = get_address_host_port(self.address)
return self._host
@property
def port(self):
"""
The port number this Server is listening on.
This will raise ValueError if the Server is listening on a
non-IP based protocol.
"""
if not self._port:
self._host, self._port = get_address_host_port(self.address)
return self._port
def identity(self) -> dict[str, str]:
return {"type": type(self).__name__, "id": self.id}
def _to_dict(self, *, exclude: Container[str] = ()) -> dict[str, Any]:
"""Dictionary representation for debugging purposes.
Not type stable and not intended for roundtrips.
See also
--------
Server.identity
Client.dump_cluster_state
distributed.utils.recursive_to_dict
"""
info: dict[str, Any] = self.identity()
extra = {
"address": self.address,
"status": self.status.name,
"thread_id": self.thread_id,
}
info.update(extra)
info = {k: v for k, v in info.items() if k not in exclude}
return recursive_to_dict(info, exclude=exclude)
def echo(self, data=None):
return data
async def listen(self, port_or_addr=None, allow_offload=True, **kwargs):
if port_or_addr is None:
port_or_addr = self.default_port
if isinstance(port_or_addr, int):
addr = unparse_host_port(self.default_ip, port_or_addr)
elif isinstance(port_or_addr, tuple):
addr = unparse_host_port(*port_or_addr)
else:
addr = port_or_addr
assert isinstance(addr, str)
listener = await listen(
addr,
self.handle_comm,
deserialize=self.deserialize,
allow_offload=allow_offload,
**kwargs,
)
self.listeners.append(listener)
def handle_comm(self, comm: Comm) -> NoOpAwaitable:
"""Start a background task that dispatches new communications to coroutine-handlers"""
try:
self._ongoing_background_tasks.call_soon(self._handle_comm, comm)
except AsyncTaskGroupClosedError:
comm.abort()
return NoOpAwaitable()
async def _handle_comm(self, comm: Comm) -> None:
"""Dispatch new communications to coroutine-handlers
Handlers is a dictionary mapping operation names to functions or
coroutines.
{'get_data': get_data,
'ping': pingpong}
Coroutines should expect a single Comm object.
"""
if self.__stopped:
comm.abort()
return
address = comm.peer_address
op = None
logger.debug("Connection from %r to %s", address, type(self).__name__)
self._comms[comm] = op
await self
try:
while not self.__stopped:
try:
msg = await comm.read()
logger.debug("Message from %r: %s", address, msg)
except OSError as e:
if not is_python_shutting_down():
logger.debug(
"Lost connection to %r while reading message: %s."
" Last operation: %s",
address,
e,
op,
)
break
except Exception as e:
logger.exception("Exception while reading from %s", address)
if comm.closed():
raise
else:
await comm.write(error_message(e, status="uncaught-error"))
continue
if not isinstance(msg, dict):
raise TypeError(
"Bad message type. Expected dict, got\n " + str(msg)
)
try:
op = msg.pop("op")
except KeyError as e:
raise ValueError(
"Received unexpected message without 'op' key: " + str(msg)
) from e
if self.counters is not None:
self.counters["op"].add(op)
self._comms[comm] = op
serializers = msg.pop("serializers", None)
close_desired = msg.pop("close", False)
reply = msg.pop("reply", True)
if op == "close":
if reply:
await comm.write("OK")
break
result = None
try:
if op in self.blocked_handlers:
_msg = (
"The '{op}' handler has been explicitly disallowed "
"in {obj}, possibly due to security concerns."
)
exc = ValueError(_msg.format(op=op, obj=type(self).__name__))
handler = raise_later(exc)
else:
handler = self.handlers[op]
except KeyError:
logger.warning(
"No handler %s found in %s",
op,
type(self).__name__,
exc_info=True,
)
else:
if serializers is not None and has_keyword(handler, "serializers"):
msg["serializers"] = serializers # add back in
logger.debug("Calling into handler %s", handler.__name__)
try:
if _expects_comm(handler):
result = handler(comm, **msg)
else:
result = handler(**msg)
if inspect.iscoroutine(result):
result = await result
elif inspect.isawaitable(result):
raise RuntimeError(
f"Comm handler returned unknown awaitable. Expected coroutine, instead got {type(result)}"
)
except CommClosedError:
if self.status == Status.running:
logger.info("Lost connection to %r", address, exc_info=True)
break
except Exception as e:
logger.exception("Exception while handling op %s", op)
if comm.closed():
raise
else:
result = error_message(e, status="uncaught-error")
if reply and result != Status.dont_reply:
try:
await comm.write(result, serializers=serializers)
except (OSError, TypeError) as e:
logger.debug(
"Lost connection to %r while sending result for op %r: %s",
address,
op,
e,
)
break
self._comms[comm] = None
msg = result = None
if close_desired: