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 1001 1002 1003 1004 1005 1006 1007 1008 1009 1010 1011 1012 1013 1014 1015 1016 1017 1018 1019 1020 1021 1022 1023 1024 1025 1026 1027 1028 1029 1030 1031 1032 1033 1034 1035 1036 1037 1038 1039 1040 1041 1042 1043 1044 1045 1046 1047 1048 1049 1050 1051 1052 1053 1054 1055 1056 1057 1058 1059 1060 1061 1062 1063 1064 1065 1066 1067 1068 1069 1070 1071 1072 1073 1074 1075 1076 1077 1078 1079 1080 1081 1082 1083 1084 1085 1086 1087 1088 1089 1090 1091 1092 1093 1094 1095 1096 1097 1098 1099 1100 1101 1102 1103 1104 1105 1106 1107 1108 1109 1110 1111 1112 1113 1114 1115 1116 1117 1118 1119 1120 1121 1122 1123 1124 1125 1126 1127 1128 1129 1130 1131 1132 1133 1134 1135 1136 1137 1138 1139 1140 1141 1142 1143 1144 1145 1146 1147 1148 1149 1150 1151 1152 1153 1154 1155 1156 1157 1158 1159 1160 1161 1162 1163 1164 1165 1166 1167 1168 1169 1170 1171 1172 1173 1174 1175 1176 1177 1178 1179 1180 1181 1182 1183 1184 1185 1186 1187 1188 1189 1190 1191 1192 1193 1194 1195 1196 1197 1198 1199 1200 1201 1202 1203 1204 1205 1206 1207 1208 1209 1210 1211 1212 1213 1214 1215 1216 1217 1218 1219 1220 1221 1222 1223 1224 1225 1226 1227 1228 1229 1230 1231 1232 1233 1234 1235 1236 1237 1238 1239 1240 1241 1242 1243 1244 1245 1246 1247 1248 1249 1250 1251 1252 1253 1254 1255 1256 1257 1258 1259 1260 1261 1262 1263 1264 1265 1266 1267 1268 1269 1270 1271 1272 1273 1274 1275 1276 1277 1278 1279 1280 1281 1282 1283 1284 1285 1286 1287 1288 1289 1290 1291 1292 1293 1294 1295 1296 1297 1298 1299 1300 1301 1302 1303 1304 1305 1306 1307 1308 1309 1310 1311 1312 1313 1314 1315 1316 1317 1318 1319 1320 1321 1322 1323 1324 1325 1326 1327 1328 1329 1330 1331 1332 1333 1334 1335 1336 1337 1338 1339 1340 1341 1342 1343 1344 1345 1346 1347 1348 1349 1350 1351 1352 1353 1354 1355 1356 1357 1358 1359 1360 1361 1362 1363 1364 1365 1366 1367 1368 1369 1370 1371 1372 1373 1374 1375 1376 1377 1378 1379 1380 1381 1382 1383 1384 1385 1386 1387 1388 1389 1390 1391 1392 1393 1394 1395 1396 1397 1398 1399 1400 1401 1402 1403 1404 1405 1406 1407 1408 1409 1410 1411 1412 1413 1414 1415 1416 1417 1418 1419 1420 1421 1422 1423 1424 1425 1426 1427 1428 1429 1430 1431 1432 1433 1434 1435 1436 1437 1438 1439 1440 1441 1442 1443 1444 1445 1446 1447 1448 1449 1450 1451 1452 1453 1454 1455 1456 1457 1458 1459 1460 1461 1462 1463 1464 1465 1466 1467 1468 1469 1470 1471 1472 1473 1474 1475 1476 1477 1478 1479 1480 1481 1482 1483 1484 1485 1486 1487 1488 1489 1490 1491 1492 1493 1494 1495 1496 1497 1498 1499 1500 1501 1502 1503 1504 1505 1506 1507 1508 1509 1510 1511 1512 1513 1514 1515 1516 1517 1518 1519 1520 1521 1522 1523 1524 1525 1526 1527 1528 1529 1530 1531 1532 1533 1534 1535 1536 1537 1538 1539 1540 1541 1542 1543 1544 1545 1546 1547 1548 1549 1550 1551 1552 1553 1554 1555 1556 1557 1558 1559 1560 1561 1562 1563 1564 1565 1566 1567 1568 1569 1570 1571 1572 1573 1574 1575 1576 1577 1578 1579 1580 1581 1582 1583 1584 1585 1586 1587 1588 1589 1590 1591 1592 1593 1594 1595 1596 1597 1598 1599 1600 1601 1602 1603 1604 1605 1606 1607 1608 1609 1610 1611 1612 1613 1614 1615 1616
|
from __future__ import annotations
import asyncio
import inspect
import logging
import sys
import threading
import traceback
import types
import uuid
import warnings
import weakref
from collections import defaultdict, deque
from collections.abc import Container, Coroutine
from enum import Enum
from functools import partial
from typing import TYPE_CHECKING, Any, Callable, ClassVar, 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.compatibility import PeriodicCallback
from distributed.metrics import time
from distributed.system_monitor import SystemMonitor
from distributed.utils import (
NoOpAwaitable,
get_traceback,
has_keyword,
iscoroutinefunction,
recursive_to_dict,
truncate_exception,
)
if TYPE_CHECKING:
from typing_extensions import ParamSpec
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 = ""
default_port = 0
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,
):
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.counters = None
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()
# Statistics counters for various events
try:
from distributed.counter import Digest
self.digests = defaultdict(partial(Digest, loop=self.io_loop))
except ImportError:
self.digests = None
# In case crick is not installed, also log cumulative totals (reset at server
# restart) and local maximums (reset by prometheus poll)
self.digests_total = defaultdict(float)
self.digests_max = defaultdict(float)
from distributed.counter import Counter
self.counters = defaultdict(partial(Counter, loop=self.io_loop))
self.periodic_callbacks = {}
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
@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()
self.status = Status.failed
self.__startup_exc = exc
try:
await asyncio.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
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(self):
if self.__stopped:
return
self.__stopped = True
_stops = set()
for listener in self.listeners:
future = listener.stop()
if inspect.isawaitable(future):
_stops.add(future)
if _stops:
async def background_stops():
await asyncio.gather(*_stops)
self._ongoing_background_tasks.call_soon(background_stops)
@property
def listener(self):
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
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:
"""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 = 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):
"""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):
"""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 sys.is_finalizing():
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:
await comm.close()
if comm.closed():
break
finally:
del self._comms[comm]
if not sys.is_finalizing() and not comm.closed():
try:
comm.abort()
except Exception as e:
logger.error(
"Failed while closing connection to %r: %s", address, e
)
async def handle_stream(self, comm, extra=None):
extra = extra or {}
logger.info("Starting established connection to %s", comm.peer_address)
closed = False
try:
while not closed:
try:
msgs = await comm.read()
# If another coroutine has closed the comm, stop handling the stream.
except CommClosedError:
closed = True
logger.info(
"Connection to %s has been closed.",
comm.peer_address,
)
break
if not isinstance(msgs, (tuple, list)):
msgs = (msgs,)
for msg in msgs:
if msg == "OK":
break
op = msg.pop("op")
if op:
if op == "close-stream":
closed = True
logger.info(
"Received 'close-stream' from %s; closing.",
comm.peer_address,
)
break
handler = self.stream_handlers[op]
if iscoroutinefunction(handler):
self._ongoing_background_tasks.call_soon(
handler, **merge(extra, msg)
)
await asyncio.sleep(0)
else:
handler(**merge(extra, msg))
else:
logger.error("odd message %s", msg)
await asyncio.sleep(0)
except Exception:
if LOG_PDB:
import pdb
pdb.set_trace()
raise
finally:
await comm.close()
assert comm.closed()
async def close(self, timeout=None):
try:
for pc in self.periodic_callbacks.values():
pc.stop()
if not self.__stopped:
self.__stopped = True
_stops = 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` will be removed in a future version",
PendingDeprecationWarning,
)
_stops.add(future)
if _stops:
await asyncio.gather(*_stops)
# TODO: Deal with exceptions
await self._ongoing_background_tasks.stop()
await self.rpc.close()
await asyncio.gather(*[comm.close() for comm in list(self._comms)])
finally:
self._event_finished.set()
def digest_metric(self, name: str, value: float) -> None:
# Granular data (requires crick)
if self.digests is not None:
self.digests[name].add(value)
# Cumulative data (reset by server restart)
self.digests_total[name] += value
# Local maximums (reset by Prometheus poll)
self.digests_max[name] = max(self.digests_max[name], value)
def pingpong(comm):
return b"pong"
async def send_recv( # type: ignore[no-untyped-def]
comm: Comm,
*,
reply: bool = True,
serializers=None,
deserializers=None,
**kwargs,
):
"""Send and recv with a Comm.
Keyword arguments turn into the message
response = await send_recv(comm, op='ping', reply=True)
"""
msg = kwargs
msg["reply"] = reply
please_close = kwargs.get("close", False)
force_close = False
if deserializers is None:
deserializers = serializers
if deserializers is not None:
msg["serializers"] = deserializers
try:
await comm.write(msg, serializers=serializers, on_error="raise")
if reply:
response = await comm.read(deserializers=deserializers)
else:
response = None
except (asyncio.TimeoutError, OSError):
# On communication errors, we should simply close the communication
# Note that OSError includes CommClosedError and socket timeouts
force_close = True
raise
except asyncio.CancelledError:
# Do not reuse the comm to prevent the next call of send_recv from receiving
# data from this call and/or accidentally putting multiple waiters on read().
# Note that this relies on all Comm implementations to allow a write() in the
# middle of a read().
please_close = True
raise
finally:
if force_close:
comm.abort()
elif please_close:
await comm.close()
if isinstance(response, dict) and response.get("status") == "uncaught-error":
if comm.deserialize:
_, exc, tb = clean_exception(**response)
assert exc
raise exc.with_traceback(tb)
else:
raise Exception(response["exception_text"])
return response
def addr_from_args(addr=None, ip=None, port=None):
if addr is None:
addr = (ip, port)
else:
assert ip is None and port is None
if isinstance(addr, tuple):
addr = unparse_host_port(*addr)
return normalize_address(addr)
class rpc:
"""Conveniently interact with a remote server
>>> remote = rpc(address) # doctest: +SKIP
>>> response = await remote.add(x=10, y=20) # doctest: +SKIP
One rpc object can be reused for several interactions.
Additionally, this object creates and destroys many comms as necessary
and so is safe to use in multiple overlapping communications.
When done, close comms explicitly.
>>> remote.close_comms() # doctest: +SKIP
"""
active: ClassVar[weakref.WeakSet[rpc]] = weakref.WeakSet()
comms = ()
address = None
def __init__(
self,
arg=None,
comm=None,
deserialize=True,
timeout=None,
connection_args=None,
serializers=None,
deserializers=None,
):
self.comms = {}
self.address = coerce_to_address(arg)
self.timeout = timeout
self.status = Status.running
self.deserialize = deserialize
self.serializers = serializers
self.deserializers = deserializers if deserializers is not None else serializers
self.connection_args = connection_args or {}
self._created = weakref.WeakSet()
rpc.active.add(self)
async def live_comm(self):
"""Get an open communication
Some comms to the ip/port target may be in current use by other
coroutines. We track this with the `comms` dict
:: {comm: True/False if open and ready for use}
This function produces an open communication, either by taking one
that we've already made or making a new one if they are all taken.
This also removes comms that have been closed.
When the caller is done with the stream they should set
self.comms[comm] = True
As is done in __getattr__ below.
"""
if self.status == Status.closed:
raise RPCClosed("RPC Closed")
to_clear = set()
open = False
for comm, open in self.comms.items():
if comm.closed():
to_clear.add(comm)
if open:
break
for s in to_clear:
del self.comms[s]
if not open or comm.closed():
comm = await connect(
self.address,
self.timeout,
deserialize=self.deserialize,
**self.connection_args,
)
comm.name = "rpc"
self.comms[comm] = False # mark as taken
return comm
def close_comms(self):
async def _close_comm(comm):
# Make sure we tell the peer to close
try:
if not comm.closed():
await comm.write({"op": "close", "reply": False})
await comm.close()
except OSError:
comm.abort()
tasks = []
for comm in list(self.comms):
if comm and not comm.closed():
task = asyncio.ensure_future(_close_comm(comm))
tasks.append(task)
for comm in list(self._created):
if comm and not comm.closed():
task = asyncio.ensure_future(_close_comm(comm))
tasks.append(task)
self.comms.clear()
return tasks
def __getattr__(self, key):
async def send_recv_from_rpc(**kwargs):
if self.serializers is not None and kwargs.get("serializers") is None:
kwargs["serializers"] = self.serializers
if self.deserializers is not None and kwargs.get("deserializers") is None:
kwargs["deserializers"] = self.deserializers
comm = None
try:
comm = await self.live_comm()
comm.name = "rpc." + key
result = await send_recv(comm=comm, op=key, **kwargs)
except (RPCClosed, CommClosedError) as e:
if comm:
raise type(e)(
f"Exception while trying to call remote method {key!r} before comm was established."
) from e
else:
raise type(e)(
f"Exception while trying to call remote method {key!r} using comm {comm!r}."
) from e
self.comms[comm] = True # mark as open
return result
return send_recv_from_rpc
async def close_rpc(self):
if self.status != Status.closed:
rpc.active.discard(self)
self.status = Status.closed
return await asyncio.gather(*self.close_comms())
def __enter__(self):
warnings.warn(
"the rpc synchronous context manager is deprecated",
DeprecationWarning,
stacklevel=2,
)
return self
def __exit__(self, exc_type, exc_value, traceback):
asyncio.ensure_future(self.close_rpc())
async def __aenter__(self):
return self
async def __aexit__(self, exc_type, exc_value, traceback):
await self.close_rpc()
def __del__(self):
if self.status != Status.closed:
rpc.active.discard(self)
self.status = Status.closed
still_open = [comm for comm in self.comms if not comm.closed()]
if still_open:
logger.warning(
"rpc object %s deleted with %d open comms", self, len(still_open)
)
for comm in still_open:
comm.abort()
def __repr__(self):
return "<rpc to %r, %d comms>" % (self.address, len(self.comms))
class PooledRPCCall:
"""The result of ConnectionPool()('host:port')
See Also:
ConnectionPool
"""
def __init__(self, addr, pool, serializers=None, deserializers=None):
self.addr = addr
self.pool = pool
self.serializers = serializers
self.deserializers = deserializers if deserializers is not None else serializers
@property
def address(self):
return self.addr
def __getattr__(self, key):
async def send_recv_from_rpc(**kwargs):
if self.serializers is not None and kwargs.get("serializers") is None:
kwargs["serializers"] = self.serializers
if self.deserializers is not None and kwargs.get("deserializers") is None:
kwargs["deserializers"] = self.deserializers
comm = await self.pool.connect(self.addr)
prev_name, comm.name = comm.name, "ConnectionPool." + key
try:
return await send_recv(comm=comm, op=key, **kwargs)
finally:
self.pool.reuse(self.addr, comm)
comm.name = prev_name
return send_recv_from_rpc
async def close_rpc(self):
pass
# For compatibility with rpc()
def __enter__(self):
warnings.warn(
"the rpc synchronous context manager is deprecated",
DeprecationWarning,
stacklevel=2,
)
return self
def __exit__(self, exc_type, exc_value, traceback):
pass
async def __aenter__(self):
return self
async def __aexit__(self, *args):
pass
def __repr__(self):
return f"<pooled rpc to {self.addr!r}>"
class ConnectionPool:
"""A maximum sized pool of Comm objects.
This provides a connect method that mirrors the normal distributed.connect
method, but provides connection sharing and tracks connection limits.
This object provides an ``rpc`` like interface::
>>> rpc = ConnectionPool(limit=512)
>>> scheduler = rpc('127.0.0.1:8786')
>>> workers = [rpc(address) for address in ...]
>>> info = await scheduler.identity()
It creates enough comms to satisfy concurrent connections to any
particular address::
>>> a, b = await asyncio.gather(scheduler.who_has(), scheduler.has_what())
It reuses existing comms so that we don't have to continuously reconnect.
It also maintains a comm limit to avoid "too many open file handle"
issues. Whenever this maximum is reached we clear out all idling comms.
If that doesn't do the trick then we wait until one of the occupied comms
closes.
Parameters
----------
limit: int
The number of open comms to maintain at once
deserialize: bool
Whether or not to deserialize data by default or pass it through
"""
_instances: ClassVar[weakref.WeakSet[ConnectionPool]] = weakref.WeakSet()
def __init__(
self,
limit=512,
deserialize=True,
serializers=None,
allow_offload=True,
deserializers=None,
connection_args=None,
timeout=None,
server=None,
):
self.limit = limit # Max number of open comms
# Invariant: len(available) == open - active
self.available = defaultdict(set)
# Invariant: len(occupied) == active
self.occupied = defaultdict(set)
self.allow_offload = allow_offload
self.deserialize = deserialize
self.serializers = serializers
self.deserializers = deserializers if deserializers is not None else serializers
self.connection_args = connection_args or {}
self.timeout = timeout
self.server = weakref.ref(server) if server else None
self._created = weakref.WeakSet()
self._instances.add(self)
# _n_connecting and _connecting have subtle different semantics. The set
# _connecting contains futures actively trying to establish a connection
# while the _n_connecting also accounts for connection attempts which
# are waiting due to the connection limit
self._connecting = set()
self._pending_count = 0
self._connecting_count = 0
self.status = Status.init
def _validate(self):
"""
Validate important invariants of this class
Used only for testing / debugging
"""
assert self.semaphore._value == self.limit - self.open - self._n_connecting
@property
def active(self):
return sum(map(len, self.occupied.values()))
@property
def open(self):
return self.active + sum(map(len, self.available.values()))
def __repr__(self):
return "<ConnectionPool: open=%d, active=%d, connecting=%d>" % (
self.open,
self.active,
len(self._connecting),
)
def __call__(self, addr=None, ip=None, port=None):
"""Cached rpc objects"""
addr = addr_from_args(addr=addr, ip=ip, port=port)
return PooledRPCCall(
addr, self, serializers=self.serializers, deserializers=self.deserializers
)
def __await__(self):
async def _():
await self.start()
return self
return _().__await__()
async def start(self):
# Invariant: semaphore._value == limit - open - _n_connecting
self.semaphore = asyncio.Semaphore(self.limit)
self.status = Status.running
@property
def _n_connecting(self) -> int:
return self._connecting_count
async def _connect(self, addr, timeout=None):
self._pending_count += 1
try:
await self.semaphore.acquire()
try:
self._connecting_count += 1
comm = await connect(
addr,
timeout=timeout or self.timeout,
deserialize=self.deserialize,
**self.connection_args,
)
comm.name = "ConnectionPool"
comm._pool = weakref.ref(self)
comm.allow_offload = self.allow_offload
self._created.add(comm)
self.occupied[addr].add(comm)
return comm
except BaseException:
self.semaphore.release()
raise
finally:
self._connecting_count -= 1
except asyncio.CancelledError:
raise CommClosedError("ConnectionPool closing.")
finally:
self._pending_count -= 1
async def connect(self, addr, timeout=None):
"""
Get a Comm to the given address. For internal use.
"""
available = self.available[addr]
occupied = self.occupied[addr]
while available:
comm = available.pop()
if comm.closed():
self.semaphore.release()
else:
occupied.add(comm)
return comm
if self.semaphore.locked():
self.collect()
# This construction is there to ensure that cancellation requests from
# the outside can be distinguished from cancellations of our own.
# Once the CommPool closes, we'll cancel the connect_attempt which will
# raise an OSError
# If the ``connect`` is cancelled from the outside, the Event.wait will
# be cancelled instead which we'll reraise as a CancelledError and allow
# it to propagate
connect_attempt = asyncio.create_task(self._connect(addr, timeout))
done = asyncio.Event()
self._connecting.add(connect_attempt)
connect_attempt.add_done_callback(lambda _: done.set())
connect_attempt.add_done_callback(self._connecting.discard)
try:
await done.wait()
except asyncio.CancelledError:
# This is an outside cancel attempt
connect_attempt.cancel()
try:
await connect_attempt
except CommClosedError:
pass
raise
return await connect_attempt
def reuse(self, addr, comm):
"""
Reuse an open communication to the given address. For internal use.
"""
# if the pool is asked to re-use a comm it does not know about, ignore
# this comm: just close it.
if comm not in self.occupied[addr]:
IOLoop.current().add_callback(comm.close)
else:
self.occupied[addr].remove(comm)
if comm.closed():
# Either the user passed the close=True parameter to send_recv, or
# the RPC call raised OSError or CancelledError
self.semaphore.release()
else:
self.available[addr].add(comm)
if self.semaphore.locked() and self._pending_count:
self.collect()
def collect(self):
"""
Collect open but unused communications, to allow opening other ones.
"""
logger.info(
"Collecting unused comms. open: %d, active: %d, connecting: %d",
self.open,
self.active,
len(self._connecting),
)
for comms in self.available.values():
for comm in comms:
IOLoop.current().add_callback(comm.close)
self.semaphore.release()
comms.clear()
def remove(self, addr):
"""
Remove all Comms to a given address.
"""
logger.info("Removing comms to %s", addr)
if addr in self.available:
comms = self.available.pop(addr)
for comm in comms:
IOLoop.current().add_callback(comm.close)
self.semaphore.release()
if addr in self.occupied:
comms = self.occupied.pop(addr)
for comm in comms:
IOLoop.current().add_callback(comm.close)
self.semaphore.release()
async def close(self):
"""
Close all communications
"""
self.status = Status.closed
for conn_fut in self._connecting:
conn_fut.cancel()
for d in [self.available, self.occupied]:
comms = set()
while d:
comms.update(d.popitem()[1])
await asyncio.gather(
*(comm.close() for comm in comms), return_exceptions=True
)
for _ in comms:
self.semaphore.release()
while self._connecting:
await asyncio.sleep(0.005)
def coerce_to_address(o):
if isinstance(o, (list, tuple)):
o = unparse_host_port(*o)
return normalize_address(o)
def collect_causes(e: BaseException) -> list[BaseException]:
causes = []
while e.__cause__ is not None:
causes.append(e.__cause__)
e = e.__cause__
return causes
class ErrorMessage(TypedDict):
status: str
exception: protocol.Serialize
traceback: protocol.Serialize | None
exception_text: str
traceback_text: str
def error_message(e: BaseException, status: str = "error") -> ErrorMessage:
"""Produce message to send back given an exception has occurred
This does the following:
1. Gets the traceback
2. Truncates the exception and the traceback
3. Serializes the exception and traceback or
4. If they can't be serialized send string versions
5. Format a message and return
See Also
--------
clean_exception : deserialize and unpack message into exception/traceback
"""
MAX_ERROR_LEN = dask.config.get("distributed.admin.max-error-length")
tblib.pickling_support.install(e, *collect_causes(e))
tb = get_traceback()
tb_text = "".join(traceback.format_tb(tb))
e = truncate_exception(e, MAX_ERROR_LEN)
try:
e_bytes = protocol.pickle.dumps(e)
protocol.pickle.loads(e_bytes)
except Exception:
e_bytes = protocol.pickle.dumps(Exception(repr(e)))
e_serialized = protocol.to_serialize(e_bytes)
try:
tb_bytes = protocol.pickle.dumps(tb)
protocol.pickle.loads(tb_bytes)
except Exception:
tb_bytes = protocol.pickle.dumps(tb_text)
if len(tb_bytes) > MAX_ERROR_LEN:
tb_serialized = None
else:
tb_serialized = protocol.to_serialize(tb_bytes)
return {
"status": status,
"exception": e_serialized,
"traceback": tb_serialized,
"exception_text": repr(e),
"traceback_text": tb_text,
}
def clean_exception(
exception: BaseException | bytes | bytearray | str | None,
traceback: types.TracebackType | bytes | str | None = None,
**kwargs: Any,
) -> tuple[
type[BaseException | None], BaseException | None, types.TracebackType | None
]:
"""Reraise exception and traceback. Deserialize if necessary
See Also
--------
error_message : create and serialize errors into message
"""
if isinstance(exception, (bytes, bytearray)):
try:
exception = protocol.pickle.loads(exception)
except Exception:
exception = Exception(exception)
elif isinstance(exception, str):
exception = Exception(exception)
if isinstance(traceback, bytes):
try:
traceback = protocol.pickle.loads(traceback)
except (TypeError, AttributeError):
traceback = None
elif isinstance(traceback, str):
traceback = None # happens if the traceback failed serializing
assert isinstance(exception, BaseException) or exception is None
assert isinstance(traceback, types.TracebackType) or traceback is None
return type(exception), exception, traceback
|