File: core.py

package info (click to toggle)
dask.distributed 2024.12.1%2Bds-1
  • links: PTS, VCS
  • area: main
  • in suites: forky, sid, trixie
  • size: 12,588 kB
  • sloc: python: 96,954; javascript: 1,549; sh: 390; makefile: 220
file content (411 lines) | stat: -rw-r--r-- 12,721 bytes parent folder | download
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
from __future__ import annotations

import asyncio
import inspect
import logging
import random
import sys
import weakref
from abc import ABC, abstractmethod
from typing import Any, ClassVar

import dask
from dask.utils import parse_timedelta

from distributed.comm import registry
from distributed.comm.addressing import get_address_host, parse_address, resolve_address
from distributed.metrics import time
from distributed.protocol.compression import get_compression_settings
from distributed.protocol.pickle import HIGHEST_PROTOCOL
from distributed.utils import wait_for

logger = logging.getLogger(__name__)


class CommClosedError(IOError):
    pass


class FatalCommClosedError(CommClosedError):
    pass


class Comm(ABC):
    """
    A message-oriented communication object, representing an established
    communication channel.  There should be only one reader and one
    writer at a time: to manage current communications, even with a
    single peer, you must create distinct ``Comm`` objects.

    Messages are arbitrary Python objects.  Concrete implementations
    of this class can implement different serialization mechanisms
    depending on the underlying transport's characteristics.
    """

    _instances: ClassVar[weakref.WeakSet[Comm]] = weakref.WeakSet()
    name: str | None
    local_info: dict
    remote_info: dict
    handshake_options: dict
    deserialize: bool

    def __init__(self, deserialize: bool = True):
        self._instances.add(self)
        self.allow_offload = True  # for deserialization in utils.from_frames
        self.name = None
        self.local_info = {}
        self.remote_info = {}
        self.handshake_options = {}
        self.deserialize = deserialize

    # XXX add set_close_callback()?

    @abstractmethod
    async def read(self, deserializers=None):
        """
        Read and return a message (a Python object).

        This method returns a coroutine.

        Parameters
        ----------
        deserializers : dict[str, tuple[Callable, Callable, bool]] | None
            An optional dict appropriate for distributed.protocol.deserialize.
            See :ref:`serialization` for more.
        """

    @abstractmethod
    async def write(self, msg, serializers=None, on_error=None):
        """
        Write a message (a Python object).

        This method returns a coroutine.

        Parameters
        ----------
        msg
        on_error : str | None
            The behavior when serialization fails. See
            ``distributed.protocol.core.dumps`` for valid values.
        """

    @abstractmethod
    async def close(self):
        """
        Close the communication cleanly.  This will attempt to flush
        outgoing buffers before actually closing the underlying transport.

        This method returns a coroutine.
        """

    @abstractmethod
    def abort(self):
        """
        Close the communication immediately and abruptly.
        Useful in destructors or generators' ``finally`` blocks.
        """

    @abstractmethod
    def closed(self):
        """Return whether the stream is closed."""

    @property
    @abstractmethod
    def local_address(self) -> str:
        """The local address"""

    @property
    @abstractmethod
    def peer_address(self) -> str:
        """The peer's address"""

    @property
    def same_host(self) -> bool:
        """Return True if the peer is on localhost; False otherwise"""
        local_ipaddr = get_address_host(resolve_address(self.local_address))
        peer_ipaddr = get_address_host(resolve_address(self.peer_address))

        # Note: this is not the same as testing `peer_ipaddr == "127.0.0.1"`.
        # When you start a Server, by default it starts listening on the LAN interface,
        # so its advertised address will be 10.x or 192.168.x.
        return local_ipaddr == peer_ipaddr

    @property
    def extra_info(self):
        """
        Return backend-specific information about the communication,
        as a dict.  Typically, this is information which is initialized
        when the communication is established and doesn't vary afterwards.
        """
        return {}

    def handshake_info(self) -> dict[str, Any]:
        """Share environment information with the peer that may differ, i.e. compression
        settings.

        Notes
        -----
        By the time this method runs, the "auto" compression setting has been updated to
        an actual compression algorithm. This matters if both peers had compression set
        to 'auto' but only one has lz4 installed. See
        distributed.protocol.compression._update_and_check_compression_settings()

        See also
        --------
        handshake_configuration
        """
        if self.same_host:
            compression = None
        else:
            compression = get_compression_settings("distributed.comm.compression")

        return {
            "compression": compression,
            "python": tuple(sys.version_info)[:3],
            "pickle-protocol": HIGHEST_PROTOCOL,
        }

    @staticmethod
    def handshake_configuration(
        local: dict[str, Any], remote: dict[str, Any]
    ) -> dict[str, Any]:
        """Find a configuration that is suitable for both local and remote

        Parameters
        ----------
        local
            Output of handshake_info() in this process
        remote
            Output of handshake_info() on the remote host

        See also
        --------
        handshake_info
        """
        try:
            out = {
                "pickle-protocol": min(
                    local["pickle-protocol"], remote["pickle-protocol"]
                )
            }
        except KeyError as e:
            raise ValueError(
                "Your Dask versions may not be in sync. "
                "Please ensure that you have the same version of dask "
                "and distributed on your client, scheduler, and worker machines"
            ) from e

        if local["compression"] == remote["compression"]:
            out["compression"] = local["compression"]
        else:
            out["compression"] = None

        return out

    def __repr__(self):
        return "<{}{} {} local={} remote={}>".format(
            self.__class__.__name__,
            " (closed)" if self.closed() else "",
            self.name or "",
            self.local_address,
            self.peer_address,
        )


class Listener(ABC):
    @abstractmethod
    async def start(self):
        """
        Start listening for incoming connections.
        """

    @abstractmethod
    def stop(self):
        """
        Stop listening.  This does not shutdown already established
        communications, but prevents accepting new ones.
        """

    @property
    @abstractmethod
    def listen_address(self):
        """
        The listening address as a URI string.
        """

    @property
    @abstractmethod
    def contact_address(self):
        """
        An address this listener can be contacted on.  This can be
        different from `listen_address` if the latter is some wildcard
        address such as 'tcp://0.0.0.0:123'.
        """

    async def __aenter__(self):
        await self.start()
        return self

    async def __aexit__(self, exc_type, exc_value, traceback):
        future = self.stop()
        if inspect.isawaitable(future):
            await future

    def __await__(self):
        async def _():
            await self.start()
            return self

        return _().__await__()

    async def on_connection(
        self, comm: Comm, handshake_overrides: dict[str, Any] | None = None
    ) -> None:
        local_info = {**comm.handshake_info(), **(handshake_overrides or {})}

        await comm.write(local_info)
        handshake = await comm.read()

        comm.remote_info = handshake
        comm.remote_info["address"] = comm.peer_address
        comm.local_info = local_info
        comm.local_info["address"] = comm.local_address

        comm.handshake_options = comm.handshake_configuration(
            comm.local_info, comm.remote_info
        )


class BaseListener(Listener):
    def __init__(self) -> None:
        self.__comms: set[Comm] = set()

    async def on_connection(
        self, comm: Comm, handshake_overrides: dict[str, Any] | None = None
    ) -> None:
        self.__comms.add(comm)
        try:
            return await super().on_connection(comm, handshake_overrides)
        finally:
            self.__comms.discard(comm)

    def abort_handshaking_comms(self) -> None:
        comms, self.__comms = self.__comms, set()
        for comm in comms:
            comm.abort()


class Connector(ABC):
    @abstractmethod
    async def connect(self, address, deserialize=True):
        """
        Connect to the given address and return a Comm object.
        This function returns a coroutine. It may raise EnvironmentError
        if the other endpoint is unreachable or unavailable.  It
        may raise ValueError if the address is malformed.
        """


async def connect(
    addr, timeout=None, deserialize=True, handshake_overrides=None, **connection_args
):
    """
    Connect to the given address (a URI such as ``tcp://127.0.0.1:1234``)
    and yield a ``Comm`` object.  If the connection attempt fails, it is
    retried until the *timeout* is expired.
    """
    if timeout is None:
        timeout = dask.config.get("distributed.comm.timeouts.connect")
    timeout = parse_timedelta(timeout, default="seconds")

    scheme, loc = parse_address(addr)
    backend = registry.get_backend(scheme)
    connector = backend.get_connector()
    comm = None

    start = time()

    def time_left():
        deadline = start + timeout
        return max(0, deadline - time())

    backoff_base = 0.01
    attempt = 0
    logger.debug("Establishing connection to %s", loc)
    # Prefer multiple small attempts than one long attempt. This should protect
    # primarily from DNS race conditions
    # gh3104, gh4176, gh4167
    intermediate_cap = timeout / 5
    active_exception = None
    while time_left() > 0:
        try:
            comm = await wait_for(
                connector.connect(loc, deserialize=deserialize, **connection_args),
                timeout=min(intermediate_cap, time_left()),
            )
            break
        except FatalCommClosedError:
            raise
        # Note: CommClosed inherits from OSError
        except (asyncio.TimeoutError, OSError) as exc:
            active_exception = exc

            # As described above, the intermediate timeout is used to distributed
            # initial, bulk connect attempts homogeneously. In particular with
            # the jitter upon retries we should not be worred about overloading
            # any more DNS servers
            intermediate_cap = timeout
            # FullJitter see https://aws.amazon.com/blogs/architecture/exponential-backoff-and-jitter/

            upper_cap = min(time_left(), backoff_base * (2**attempt))
            backoff = random.uniform(0, upper_cap)
            attempt += 1
            logger.debug(
                "Could not connect to %s, waiting for %s before retrying", loc, backoff
            )
            await asyncio.sleep(backoff)
    else:
        raise OSError(
            f"Timed out trying to connect to {addr} after {timeout} s"
        ) from active_exception

    local_info = {
        **comm.handshake_info(),
        **(handshake_overrides or {}),
    }
    await comm.write(local_info)
    handshake = await comm.read()

    comm.remote_info = handshake
    comm.remote_info["address"] = comm._peer_addr
    comm.local_info = local_info
    comm.local_info["address"] = comm._local_addr

    comm.handshake_options = comm.handshake_configuration(
        comm.local_info, comm.remote_info
    )
    logger.debug("Connection to %s established", loc)
    return comm


def listen(addr, handle_comm, deserialize=True, **kwargs):
    """
    Create a listener object with the given parameters.  When its ``start()``
    method is called, the listener will listen on the given address
    (a URI such as ``tcp://0.0.0.0``) and call *handle_comm* with a
    ``Comm`` object for each incoming connection.

    *handle_comm* can be a regular function or a coroutine.
    """
    try:
        scheme, loc = parse_address(addr, strict=True)
    except ValueError:
        if kwargs.get("ssl_context"):
            addr = "tls://" + addr
        else:
            addr = "tcp://" + addr
        scheme, loc = parse_address(addr, strict=True)

    backend = registry.get_backend(scheme)

    return backend.get_listener(loc, handle_comm, deserialize, **kwargs)