File: core.py

package info (click to toggle)
dask.distributed 2021.01.0%2Bds.1-2.1%2Bdeb11u1
  • links: PTS, VCS
  • area: main
  • in suites: bullseye
  • size: 6,668 kB
  • sloc: python: 54,131; javascript: 1,549; makefile: 207; sh: 100
file content (1160 lines) | stat: -rw-r--r-- 37,565 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
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
import asyncio
from collections import defaultdict
from contextlib import suppress
from enum import Enum
from functools import partial
import inspect
import logging
import threading
import traceback
import uuid
import weakref
import warnings

import dask
import tblib
from tlz import merge
from tornado import gen
from tornado.ioloop import IOLoop, PeriodicCallback

from .comm import (
    connect,
    listen,
    CommClosedError,
    normalize_address,
    unparse_host_port,
    get_address_host_port,
)
from .metrics import time
from . import profile
from .system_monitor import SystemMonitor
from .utils import (
    is_coroutine_function,
    get_traceback,
    truncate_exception,
    shutting_down,
    parse_timedelta,
    has_keyword,
    CancelledError,
    TimeoutError,
)
from . import protocol


class Status(Enum):
    """
    This Enum contains the various states a worker, scheduler and nanny can be
    in. Some of the status can only be observed in one of nanny, scheduler or
    worker but we put them in the same Enum as they are compared with each
    other.
    """

    closed = "closed"
    closing = "closing"
    closing_gracefully = "closing-gracefully"
    init = "init"
    created = "created"
    running = "running"
    starting = "starting"
    stopped = "stopped"
    stopping = "stopping"
    undefined = None
    dont_reply = "dont-reply"


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")


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,
    ):
        self.handlers = {
            "identity": self.identity,
            "connection_stream": self.handle_stream,
        }
        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._comms = {}
        self.deserialize = deserialize
        self.monitor = SystemMonitor()
        self.counters = None
        self.digests = None
        self._ongoing_coroutines = weakref.WeakSet()
        self._event_finished = asyncio.Event()

        self.listeners = []
        self.io_loop = io_loop or IOLoop.current()
        self.loop = self.io_loop

        if not hasattr(self.io_loop, "profile"):
            ref = weakref.ref(self.io_loop)

            if hasattr(self.io_loop, "asyncio_loop"):

                def stop():
                    loop = ref()
                    return loop is None or loop.asyncio_loop.is_closed()

            else:

                def stop():
                    loop = ref()
                    return loop is None or loop._closing

            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,
            )

        # Statistics counters for various events
        with suppress(ImportError):
            from .counter import Digest

            self.digests = defaultdict(partial(Digest, loop=self.io_loop))

        from .counter import Counter

        self.counters = defaultdict(partial(Counter, loop=self.io_loop))

        self.periodic_callbacks = dict()

        pc = PeriodicCallback(self.monitor.update, 500)
        self.periodic_callbacks["monitor"] = pc

        self._last_tick = time()
        measure_tick_interval = parse_timedelta(
            dask.config.get("distributed.admin.tick.interval"), default="ms"
        )
        pc = PeriodicCallback(self._measure_tick, measure_tick_interval * 1000)
        self.periodic_callbacks["tick"] = pc

        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.status = Status.undefined

        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):
        return self._status

    @status.setter
    def status(self, new_status):
        if isinstance(new_status, Status):
            self._status = new_status
        elif isinstance(new_status, str) or new_status is None:
            warnings.warn(
                f"Since distributed 2.23 `.status` is now an Enum, please assign `Status.{new_status}`",
                PendingDeprecationWarning,
                stacklevel=1,
            )
            corresponding_enum_variants = [s for s in Status if s.value == new_status]
            assert len(corresponding_enum_variants) == 1
            self._status = corresponding_enum_variants[0]
        else:
            raise TypeError(f"expected Status or str, got {new_status}")

    async def finished(self):
        """ Wait until the server has finished """
        await self._event_finished.wait()

    def __await__(self):
        async def _():
            timeout = getattr(self, "death_timeout", 0)
            async with self._startup_lock:
                if self.status == Status.running:
                    return self
                if timeout:
                    try:
                        await asyncio.wait_for(self.start(), timeout=timeout)
                        self.status = Status.running
                    except Exception:
                        await self.close(timeout=1)
                        raise TimeoutError(
                            "{} failed to start in {} seconds".format(
                                type(self).__name__, timeout
                            )
                        )
                else:
                    await self.start()
                    self.status = Status.running
            return self

        return _().__await__()

    async def start(self):
        await self.rpc.start()

    async def __aenter__(self):
        await self
        return self

    async def __aexit__(self, typ, 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 on the IOLoop.
        """
        self._last_tick = time()

        def start_pcs():
            for pc in self.periodic_callbacks.values():
                if not pc.is_running():
                    pc.start()

        self.io_loop.add_callback(start_pcs)

    def stop(self):
        if not self.__stopped:
            self.__stopped = True
            for listener in self.listeners:
                # Delay closing the server socket until the next IO loop tick.
                # Otherwise race conditions can appear if an event handler
                # for an accept() call is already scheduled by the IO loop,
                # raising EBADF.
                # The demonstrator for this is Worker.terminate(), which
                # closes the server socket in response to an incoming message.
                # See https://github.com/tornadoweb/tornado/issues/2069
                self.io_loop.add_callback(listener.stop)

    @property
    def listener(self):
        if self.listeners:
            return self.listeners[0]
        else:
            return None

    def _measure_tick(self):
        now = time()
        diff = now - self._last_tick
        self._last_tick = now
        if diff > 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__,
                diff,
            )
        if self.digests is not None:
            self.digests["tick-duration"].add(diff)

    @property
    def address(self):
        """
        The address this Server can be contacted on.
        """
        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 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 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._port = get_address_host_port(self.address)
        return self._port

    def identity(self, comm=None):
        return {"type": type(self).__name__, "id": self.id}

    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)

    async def handle_comm(self, comm, shutting_down=shutting_down):
        """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 True:
                try:
                    msg = await comm.read()
                    logger.debug("Message from %r: %s", address, msg)
                except EnvironmentError as e:
                    if not 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:
                        result = handler(comm, **msg)
                        if inspect.isawaitable(result):
                            result = asyncio.ensure_future(result)
                            self._ongoing_coroutines.add(result)
                            result = await result
                    except (CommClosedError, CancelledError) as e:
                        if self.status == Status.running:
                            logger.info("Lost connection to %r: %s", address, e)
                        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")

                # result is not type stable:
                # when LHS is not Status then RHS must not be Status or it raises.
                # when LHS is Status then RHS must be status or it raises in tests
                is_dont_reply = False
                if isinstance(result, Status) and (result == Status.dont_reply):
                    is_dont_reply = True

                if reply and not is_dont_reply:
                    try:
                        await comm.write(result, serializers=serializers)
                    except (EnvironmentError, TypeError) as e:
                        logger.debug(
                            "Lost connection to %r while sending result for op %r: %s",
                            address,
                            op,
                            e,
                        )
                        break
                msg = result = None
                if close_desired:
                    await comm.close()
                if comm.closed():
                    break

        finally:
            del self._comms[comm]
            if not shutting_down() 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, every_cycle=[]):
        extra = extra or {}
        logger.info("Starting established connection")

        io_error = None
        closed = False
        try:
            while not closed:
                msgs = await comm.read()
                if not isinstance(msgs, (tuple, list)):
                    msgs = (msgs,)

                if not comm.closed():
                    for msg in msgs:
                        if msg == "OK":  # from close
                            break
                        op = msg.pop("op")
                        if op:
                            if op == "close-stream":
                                closed = True
                                break
                            handler = self.stream_handlers[op]
                            if is_coroutine_function(handler):
                                self.loop.add_callback(handler, **merge(extra, msg))
                                await gen.sleep(0)
                            else:
                                handler(**merge(extra, msg))
                        else:
                            logger.error("odd message %s", msg)
                    await asyncio.sleep(0)

                for func in every_cycle:
                    if is_coroutine_function(func):
                        self.loop.add_callback(func)
                    else:
                        func()

        except (CommClosedError, EnvironmentError) as e:
            io_error = e
        except Exception as e:
            logger.exception(e)
            if LOG_PDB:
                import pdb

                pdb.set_trace()
            raise
        finally:
            await comm.close()
            assert comm.closed()

    @gen.coroutine
    def close(self):
        for pc in self.periodic_callbacks.values():
            pc.stop()
        for listener in self.listeners:
            future = self.listener.stop()
            if inspect.isawaitable(future):
                yield future
        for i in range(20):  # let comms close naturally for a second
            if not self._comms:
                break
            else:
                yield asyncio.sleep(0.05)
        yield [comm.close() for comm in list(self._comms)]  # then forcefully close
        for cb in self._ongoing_coroutines:
            cb.cancel()
        for i in range(10):
            if all(cb.cancelled() for c in self._ongoing_coroutines):
                break
            else:
                yield asyncio.sleep(0.01)

        self._event_finished.set()


def pingpong(comm):
    return b"pong"


async def send_recv(comm, reply=True, serializers=None, deserializers=None, **kwargs):
    """Send and recv with a Comm.

    Keyword arguments turn into the message

    response = yield send_recv(comm, op='ping', reply=True)
    """
    msg = kwargs
    msg["reply"] = reply
    please_close = kwargs.get("close")
    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 EnvironmentError:
        # On communication errors, we should simply close the communication
        force_close = True
        raise
    finally:
        if please_close:
            await comm.close()
        elif force_close:
            comm.abort()

    if isinstance(response, dict) and response.get("status") == "uncaught-error":
        if comm.deserialize:
            typ, exc, tb = clean_exception(**response)
            raise exc.with_traceback(tb)
        else:
            raise Exception(response["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 = yield 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 = 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 EnvironmentError:
                comm.abort()

        tasks = []
        for comm in list(self.comms):
            if comm and not comm.closed():
                # IOLoop.current().add_callback(_close_comm, comm)
                task = asyncio.ensure_future(_close_comm(comm))
                tasks.append(task)
        for comm in list(self._created):
            if comm and not comm.closed():
                # IOLoop.current().add_callback(_close_comm, comm)
                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
            try:
                comm = await self.live_comm()
                comm.name = "rpc." + key
                result = await send_recv(comm=comm, op=key, **kwargs)
            except (RPCClosed, CommClosedError) as e:
                raise e.__class__(
                    "%s: while trying to call remote method %r" % (e, key)
                )

            self.comms[comm] = True  # mark as open
            return result

        return send_recv_from_rpc

    def close_rpc(self):
        if self.status != Status.closed:
            rpc.active.discard(self)
        self.status = Status.closed
        return asyncio.gather(*self.close_comms())

    def __enter__(self):
        return self

    def __exit__(self, *args):
        asyncio.ensure_future(self.close_rpc())

    async def __aenter__(self):
        return self

    async def __aexit__(self, *args):
        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)
            name, comm.name = comm.name, "ConnectionPool." + key
            try:
                result = await send_recv(comm=comm, op=key, **kwargs)
            finally:
                self.pool.reuse(self.addr, comm)
                comm.name = name

            return result

        return send_recv_from_rpc

    async def close_rpc(self):
        pass

    # For compatibility with rpc()
    def __enter__(self):
        return self

    def __exit__(self, *args):
        pass

    def __repr__(self):
        return "<pooled rpc to %r>" % (self.addr,)


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 ...]

        >>> info = yield scheduler.identity()

    It creates enough comms to satisfy concurrent connections to any
    particular address::

        >>> a, b = yield [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 = 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._n_connecting = 0
        self.server = weakref.ref(server) if server else None
        self._created = weakref.WeakSet()
        self._instances.add(self)

    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,
            self._n_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)

    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()

        self._n_connecting += 1
        await self.semaphore.acquire()

        try:
            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)
        except Exception:
            self.semaphore.release()
            raise
        finally:
            self._n_connecting -= 1

        occupied.add(comm)

        return comm

    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():
                self.semaphore.release()
            else:
                self.available[addr].add(comm)
                if self.semaphore.locked() and self._n_connecting > 0:
                    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,
            self._n_connecting,
        )
        for addr, comms in self.available.items():
            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
        """
        for d in [self.available, self.occupied]:
            comms = [comm for comms in d.values() for comm in comms]
            await asyncio.gather(
                *[comm.close() for comm in comms], return_exceptions=True
            )
            for _ in comms:
                self.semaphore.release()

        for comm in self._created:
            IOLoop.current().add_callback(comm.abort)


def coerce_to_address(o):
    if isinstance(o, (list, tuple)):
        o = unparse_host_port(*o)

    return normalize_address(o)


def collect_causes(e):
    causes = []
    while e.__cause__ is not None:
        causes.append(e.__cause__)
        e = e.__cause__
    return causes


def error_message(e, status="error"):
    """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()
    e2 = truncate_exception(e, MAX_ERROR_LEN)
    try:
        e3 = protocol.pickle.dumps(e2, protocol=4)
        protocol.pickle.loads(e3)
    except Exception:
        e2 = Exception(str(e2))
    e4 = protocol.to_serialize(e2)
    try:
        tb2 = protocol.pickle.dumps(tb, protocol=4)
        protocol.pickle.loads(tb2)
    except Exception:
        tb = tb2 = "".join(traceback.format_tb(tb))

    if len(tb2) > MAX_ERROR_LEN:
        tb_result = None
    else:
        tb_result = protocol.to_serialize(tb)

    return {"status": status, "exception": e4, "traceback": tb_result, "text": str(e2)}


def clean_exception(exception, traceback, **kwargs):
    """Reraise exception and traceback. Deserialize if necessary

    See Also
    --------
    error_message: create and serialize errors into message
    """
    if isinstance(exception, bytes) or isinstance(exception, 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
    return type(exception), exception, traceback