-
-
Notifications
You must be signed in to change notification settings - Fork 719
/
Copy pathclient.py
5396 lines (4559 loc) · 178 KB
/
client.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
from __future__ import annotations
import asyncio
import atexit
import copy
import inspect
import json
import logging
import os
import re
import sys
import threading
import traceback
import uuid
import warnings
import weakref
from collections import defaultdict
from collections.abc import Collection, Iterator
from concurrent.futures import ThreadPoolExecutor
from concurrent.futures._base import DoneAndNotDoneFutures
from contextlib import contextmanager, suppress
from contextvars import ContextVar
from functools import partial
from numbers import Number
from queue import Queue as pyQueue
from typing import Any, ClassVar, Coroutine, Literal, Sequence, TypedDict
from tlz import first, groupby, keymap, merge, partition_all, valmap
import dask
from dask.base import collections_to_dsk, normalize_token, tokenize
from dask.core import flatten
from dask.highlevelgraph import HighLevelGraph
from dask.optimization import SubgraphCallable
from dask.utils import (
apply,
ensure_dict,
format_bytes,
funcname,
parse_timedelta,
stringify,
typename,
)
from dask.widgets import get_template
try:
from dask.delayed import single_key
except ImportError:
single_key = first
from tornado import gen
from tornado.ioloop import IOLoop, PeriodicCallback
import distributed.utils
from distributed import cluster_dump, preloading
from distributed import versions as version_module
from distributed.batched import BatchedSend
from distributed.cfexecutor import ClientExecutor
from distributed.core import (
CommClosedError,
ConnectionPool,
PooledRPCCall,
Status,
clean_exception,
connect,
rpc,
)
from distributed.diagnostics.plugin import (
NannyPlugin,
UploadFile,
WorkerPlugin,
_get_plugin_name,
)
from distributed.metrics import time
from distributed.objects import HasWhat, SchedulerInfo, WhoHas
from distributed.protocol import to_serialize
from distributed.protocol.pickle import dumps, loads
from distributed.publish import Datasets
from distributed.pubsub import PubSubClientExtension
from distributed.security import Security
from distributed.sizeof import sizeof
from distributed.threadpoolexecutor import rejoin
from distributed.utils import (
CancelledError,
LoopRunner,
NoOpAwaitable,
SyncMethodMixin,
TimeoutError,
format_dashboard_link,
has_keyword,
import_term,
is_python_shutting_down,
log_errors,
no_default,
sync,
thread_state,
)
from distributed.utils_comm import (
WrappedKey,
gather_from_workers,
pack_data,
retry_operation,
scatter_to_workers,
unpack_remotedata,
)
from distributed.worker import get_client, get_worker, secede
logger = logging.getLogger(__name__)
_global_clients: weakref.WeakValueDictionary[
int, Client
] = weakref.WeakValueDictionary()
_global_client_index = [0]
_current_client = ContextVar("_current_client", default=None)
DEFAULT_EXTENSIONS = {
"pubsub": PubSubClientExtension,
}
def _get_global_client() -> Client | None:
L = sorted(list(_global_clients), reverse=True)
for k in L:
c = _global_clients[k]
if c.status != "closed":
return c
else:
del _global_clients[k]
return None
def _set_global_client(c: Client | None) -> None:
if c is not None:
_global_clients[_global_client_index[0]] = c
_global_client_index[0] += 1
def _del_global_client(c: Client) -> None:
for k in list(_global_clients):
try:
if _global_clients[k] is c:
del _global_clients[k]
except KeyError: # pragma: no cover
pass
class Future(WrappedKey):
"""A remotely running computation
A Future is a local proxy to a result running on a remote worker. A user
manages future objects in the local Python process to determine what
happens in the larger cluster.
Parameters
----------
key: str, or tuple
Key of remote data to which this future refers
client: Client
Client that should own this future. Defaults to _get_global_client()
inform: bool
Do we inform the scheduler that we need an update on this future
state: FutureState
The state of the future
Examples
--------
Futures typically emerge from Client computations
>>> my_future = client.submit(add, 1, 2) # doctest: +SKIP
We can track the progress and results of a future
>>> my_future # doctest: +SKIP
<Future: status: finished, key: add-8f6e709446674bad78ea8aeecfee188e>
We can get the result or the exception and traceback from the future
>>> my_future.result() # doctest: +SKIP
See Also
--------
Client: Creates futures
"""
_cb_executor = None
_cb_executor_pid = None
def __init__(self, key, client=None, inform=True, state=None):
self.key = key
self._cleared = False
tkey = stringify(key)
self.client = client or Client.current()
self.client._inc_ref(tkey)
self._generation = self.client.generation
if tkey in self.client.futures:
self._state = self.client.futures[tkey]
else:
self._state = self.client.futures[tkey] = FutureState()
if inform:
self.client._send_to_scheduler(
{
"op": "client-desires-keys",
"keys": [stringify(key)],
"client": self.client.id,
}
)
if state is not None:
try:
handler = self.client._state_handlers[state]
except KeyError:
pass
else:
handler(key=key)
@property
def executor(self):
"""Returns the executor, which is the client.
Returns
-------
Client
The executor
"""
return self.client
@property
def status(self):
"""Returns the status
Returns
-------
str
The status
"""
return self._state.status
def done(self):
"""Returns whether or not the computation completed.
Returns
-------
bool
True if the computation is complete, otherwise False
"""
return self._state.done()
def result(self, timeout=None):
"""Wait until computation completes, gather result to local process.
Parameters
----------
timeout : number, optional
Time in seconds after which to raise a
``dask.distributed.TimeoutError``
Raises
------
dask.distributed.TimeoutError
If *timeout* seconds are elapsed before returning, a
``dask.distributed.TimeoutError`` is raised.
Returns
-------
result
The result of the computation. Or a coroutine if the client is asynchronous.
"""
if self.client.asynchronous:
return self.client.sync(self._result, callback_timeout=timeout)
# shorten error traceback
result = self.client.sync(self._result, callback_timeout=timeout, raiseit=False)
if self.status == "error":
typ, exc, tb = result
raise exc.with_traceback(tb)
elif self.status == "cancelled":
raise result
else:
return result
async def _result(self, raiseit=True):
await self._state.wait()
if self.status == "error":
exc = clean_exception(self._state.exception, self._state.traceback)
if raiseit:
typ, exc, tb = exc
raise exc.with_traceback(tb)
else:
return exc
elif self.status == "cancelled":
exception = CancelledError(self.key)
if raiseit:
raise exception
else:
return exception
else:
result = await self.client._gather([self])
return result[0]
async def _exception(self):
await self._state.wait()
if self.status == "error":
return self._state.exception
else:
return None
def exception(self, timeout=None, **kwargs):
"""Return the exception of a failed task
Parameters
----------
timeout : number, optional
Time in seconds after which to raise a
``dask.distributed.TimeoutError``
**kwargs : dict
Optional keyword arguments for the function
Returns
-------
Exception
The exception that was raised
If *timeout* seconds are elapsed before returning, a
``dask.distributed.TimeoutError`` is raised.
See Also
--------
Future.traceback
"""
return self.client.sync(self._exception, callback_timeout=timeout, **kwargs)
def add_done_callback(self, fn):
"""Call callback on future when callback has finished
The callback ``fn`` should take the future as its only argument. This
will be called regardless of if the future completes successfully,
errs, or is cancelled
The callback is executed in a separate thread.
Parameters
----------
fn : callable
The method or function to be called
"""
cls = Future
if cls._cb_executor is None or cls._cb_executor_pid != os.getpid():
try:
cls._cb_executor = ThreadPoolExecutor(
1, thread_name_prefix="Dask-Callback-Thread"
)
except TypeError:
cls._cb_executor = ThreadPoolExecutor(1)
cls._cb_executor_pid = os.getpid()
def execute_callback(fut):
try:
fn(fut)
except BaseException:
logger.exception("Error in callback %s of %s:", fn, fut)
self.client.loop.add_callback(
done_callback, self, partial(cls._cb_executor.submit, execute_callback)
)
def cancel(self, **kwargs):
"""Cancel the request to run this future
See Also
--------
Client.cancel
"""
return self.client.cancel([self], **kwargs)
def retry(self, **kwargs):
"""Retry this future if it has failed
See Also
--------
Client.retry
"""
return self.client.retry([self], **kwargs)
def cancelled(self):
"""Returns True if the future has been cancelled
Returns
-------
bool
True if the future was 'cancelled', otherwise False
"""
return self._state.status == "cancelled"
async def _traceback(self):
await self._state.wait()
if self.status == "error":
return self._state.traceback
else:
return None
def traceback(self, timeout=None, **kwargs):
"""Return the traceback of a failed task
This returns a traceback object. You can inspect this object using the
``traceback`` module. Alternatively if you call ``future.result()``
this traceback will accompany the raised exception.
Parameters
----------
timeout : number, optional
Time in seconds after which to raise a
``dask.distributed.TimeoutError``
If *timeout* seconds are elapsed before returning, a
``dask.distributed.TimeoutError`` is raised.
Examples
--------
>>> import traceback # doctest: +SKIP
>>> tb = future.traceback() # doctest: +SKIP
>>> traceback.format_tb(tb) # doctest: +SKIP
[...]
Returns
-------
traceback
The traceback object. Or a coroutine if the client is asynchronous.
See Also
--------
Future.exception
"""
return self.client.sync(self._traceback, callback_timeout=timeout, **kwargs)
@property
def type(self):
"""Returns the type"""
return self._state.type
def release(self):
"""
Notes
-----
This method can be called from different threads
(see e.g. Client.get() or Future.__del__())
"""
if not self._cleared and self.client.generation == self._generation:
self._cleared = True
try:
self.client.loop.add_callback(self.client._dec_ref, stringify(self.key))
except TypeError: # pragma: no cover
pass # Shutting down, add_callback may be None
def __getstate__(self):
return self.key, self.client.scheduler.address
def __setstate__(self, state):
key, address = state
try:
c = Client.current(allow_global=False)
except ValueError:
c = get_client(address)
self.__init__(key, c)
c._send_to_scheduler(
{
"op": "update-graph",
"tasks": {},
"keys": [stringify(self.key)],
"client": c.id,
}
)
def __del__(self):
try:
self.release()
except AttributeError:
# Ocassionally we see this error when shutting down the client
# https://github.com/dask/distributed/issues/4305
if not sys.is_finalizing():
raise
except RuntimeError: # closed event loop
pass
def __repr__(self):
if self.type:
return (
f"<Future: {self.status}, type: {typename(self.type)}, key: {self.key}>"
)
else:
return f"<Future: {self.status}, key: {self.key}>"
def _repr_html_(self):
return get_template("future.html.j2").render(
key=str(self.key),
type=typename(self.type),
status=self.status,
)
def __await__(self):
return self.result().__await__()
class FutureState:
"""A Future's internal state.
This is shared between all Futures with the same key and client.
"""
__slots__ = ("_event", "status", "type", "exception", "traceback")
def __init__(self):
self._event = None
self.status = "pending"
self.type = None
def _get_event(self):
# Can't create Event eagerly in constructor as it can fetch
# its IOLoop from the wrong thread
# (https://github.com/tornadoweb/tornado/issues/2189)
event = self._event
if event is None:
event = self._event = asyncio.Event()
return event
def cancel(self):
"""Cancels the operation"""
self.status = "cancelled"
self.exception = CancelledError()
self._get_event().set()
def finish(self, type=None):
"""Sets the status to 'finished' and sets the event
Parameters
----------
type : any
The type
"""
self.status = "finished"
self._get_event().set()
if type is not None:
self.type = type
def lose(self):
"""Sets the status to 'lost' and clears the event"""
self.status = "lost"
self._get_event().clear()
def retry(self):
"""Sets the status to 'pending' and clears the event"""
self.status = "pending"
self._get_event().clear()
def set_error(self, exception, traceback):
"""Sets the error data
Sets the status to 'error'. Sets the exception, the traceback,
and the event
Parameters
----------
exception: Exception
The exception
traceback: Exception
The traceback
"""
_, exception, traceback = clean_exception(exception, traceback)
self.status = "error"
self.exception = exception
self.traceback = traceback
self._get_event().set()
def done(self):
"""Returns 'True' if the event is not None and the event is set"""
return self._event is not None and self._event.is_set()
def reset(self):
"""Sets the status to 'pending' and clears the event"""
self.status = "pending"
if self._event is not None:
self._event.clear()
async def wait(self, timeout=None):
"""Wait for the awaitable to complete with a timeout.
Parameters
----------
timeout : number, optional
Time in seconds after which to raise a
``dask.distributed.TimeoutError``
"""
await asyncio.wait_for(self._get_event().wait(), timeout)
def __repr__(self):
return f"<{self.__class__.__name__}: {self.status}>"
async def done_callback(future, callback):
"""Coroutine that waits on the future, then calls the callback
Parameters
----------
future : asyncio.Future
The future
callback : callable
The callback
"""
while future.status == "pending":
await future._state.wait()
callback(future)
@partial(normalize_token.register, Future)
def normalize_future(f):
"""Returns the key and the type as a list
Parameters
----------
list
The key and the type
"""
return [f.key, type(f)]
class AllExit(Exception):
"""Custom exception class to exit All(...) early."""
def _handle_print(event):
_, msg = event
if isinstance(msg, dict) and "args" in msg and "kwargs" in msg:
print(*msg["args"], **msg["kwargs"])
else:
print(msg)
def _handle_warn(event):
_, msg = event
if isinstance(msg, dict) and "args" in msg and "kwargs" in msg:
warnings.warn(*msg["args"], **msg["kwargs"])
else:
warnings.warn(msg)
def _maybe_call_security_loader(address):
security_loader_term = dask.config.get("distributed.client.security-loader")
if security_loader_term:
try:
security_loader = import_term(security_loader_term)
except Exception as exc:
raise ImportError(
f"Failed to import `{security_loader_term}` configured at "
f"`distributed.client.security-loader` - is this module "
f"installed?"
) from exc
return security_loader({"address": address})
return None
class VersionsDict(TypedDict):
scheduler: dict[str, dict[str, Any]]
workers: dict[str, dict[str, dict[str, Any]]]
client: dict[str, dict[str, Any]]
class Client(SyncMethodMixin):
"""Connect to and submit computation to a Dask cluster
The Client connects users to a Dask cluster. It provides an asynchronous
user interface around functions and futures. This class resembles
executors in ``concurrent.futures`` but also allows ``Future`` objects
within ``submit/map`` calls. When a Client is instantiated it takes over
all ``dask.compute`` and ``dask.persist`` calls by default.
It is also common to create a Client without specifying the scheduler
address , like ``Client()``. In this case the Client creates a
:class:`LocalCluster` in the background and connects to that. Any extra
keywords are passed from Client to LocalCluster in this case. See the
LocalCluster documentation for more information.
Parameters
----------
address: string, or Cluster
This can be the address of a ``Scheduler`` server like a string
``'127.0.0.1:8786'`` or a cluster object like ``LocalCluster()``
loop
The event loop
timeout: int (defaults to configuration ``distributed.comm.timeouts.connect``)
Timeout duration for initial connection to the scheduler
set_as_default: bool (True)
Use this Client as the global dask scheduler
scheduler_file: string (optional)
Path to a file with scheduler information if available
security: Security or bool, optional
Optional security information. If creating a local cluster can also
pass in ``True``, in which case temporary self-signed credentials will
be created automatically.
asynchronous: bool (False by default)
Set to True if using this client within async/await functions or within
Tornado gen.coroutines. Otherwise this should remain False for normal
use.
name: string (optional)
Gives the client a name that will be included in logs generated on
the scheduler for matters relating to this client
heartbeat_interval: int (optional)
Time in milliseconds between heartbeats to scheduler
serializers
Iterable of approaches to use when serializing the object.
See :ref:`serialization` for more.
deserializers
Iterable of approaches to use when deserializing the object.
See :ref:`serialization` for more.
extensions : list
The extensions
direct_to_workers: bool (optional)
Whether or not to connect directly to the workers, or to ask
the scheduler to serve as intermediary.
connection_limit : int
The number of open comms to maintain at once in the connection pool
**kwargs:
If you do not pass a scheduler address, Client will create a
``LocalCluster`` object, passing any extra keyword arguments.
Examples
--------
Provide cluster's scheduler node address on initialization:
>>> client = Client('127.0.0.1:8786') # doctest: +SKIP
Use ``submit`` method to send individual computations to the cluster
>>> a = client.submit(add, 1, 2) # doctest: +SKIP
>>> b = client.submit(add, 10, 20) # doctest: +SKIP
Continue using submit or map on results to build up larger computations
>>> c = client.submit(add, a, b) # doctest: +SKIP
Gather results with the ``gather`` method.
>>> client.gather(c) # doctest: +SKIP
33
You can also call Client with no arguments in order to create your own
local cluster.
>>> client = Client() # makes your own local "cluster" # doctest: +SKIP
Extra keywords will be passed directly to LocalCluster
>>> client = Client(n_workers=2, threads_per_worker=4) # doctest: +SKIP
See Also
--------
distributed.scheduler.Scheduler: Internal scheduler
distributed.LocalCluster:
"""
_instances: ClassVar[weakref.WeakSet[Client]] = weakref.WeakSet()
_default_event_handlers = {"print": _handle_print, "warn": _handle_warn}
preloads: list[preloading.Preload]
__loop: IOLoop | None = None
def __init__(
self,
address=None,
loop=None,
timeout=no_default,
set_as_default=True,
scheduler_file=None,
security=None,
asynchronous=False,
name=None,
heartbeat_interval=None,
serializers=None,
deserializers=None,
extensions=DEFAULT_EXTENSIONS,
direct_to_workers=None,
connection_limit=512,
**kwargs,
):
if timeout == no_default:
timeout = dask.config.get("distributed.comm.timeouts.connect")
if timeout is not None:
timeout = parse_timedelta(timeout, "s")
self._timeout = timeout
self.futures = dict()
self.refcount = defaultdict(lambda: 0)
self._handle_report_task = None
if name is None:
name = dask.config.get("client-name", None)
self.id = (
type(self).__name__
+ ("-" + name + "-" if name else "-")
+ str(uuid.uuid1(clock_seq=os.getpid()))
)
self.generation = 0
self.status = "newly-created"
self._pending_msg_buffer = []
self.extensions = {}
self.scheduler_file = scheduler_file
self._startup_kwargs = kwargs
self.cluster = None
self.scheduler = None
self._scheduler_identity = {}
# A reentrant-lock on the refcounts for futures associated with this
# client. Should be held by individual operations modifying refcounts,
# or any bulk operation that needs to ensure the set of futures doesn't
# change during operation.
self._refcount_lock = threading.RLock()
self.datasets = Datasets(self)
self._serializers = serializers
if deserializers is None:
deserializers = serializers
self._deserializers = deserializers
self.direct_to_workers = direct_to_workers
# Communication
self.scheduler_comm = None
if address is None:
address = dask.config.get("scheduler-address", None)
if address:
logger.info("Config value `scheduler-address` found: %s", address)
if address is not None and kwargs:
raise ValueError(f"Unexpected keyword arguments: {sorted(kwargs)}")
if isinstance(address, (rpc, PooledRPCCall)):
self.scheduler = address
elif isinstance(getattr(address, "scheduler_address", None), str):
# It's a LocalCluster or LocalCluster-compatible object
self.cluster = address
status = self.cluster.status
if status in (Status.closed, Status.closing):
raise RuntimeError(
f"Trying to connect to an already closed or closing Cluster {self.cluster}."
)
with suppress(AttributeError):
loop = address.loop
if security is None:
security = getattr(self.cluster, "security", None)
elif address is not None and not isinstance(address, str):
raise TypeError(
"Scheduler address must be a string or a Cluster instance, got {}".format(
type(address)
)
)
# If connecting to an address and no explicit security is configured, attempt
# to load security credentials with a security loader (if configured).
if security is None and isinstance(address, str):
security = _maybe_call_security_loader(address)
if security is None:
security = Security()
elif isinstance(security, dict):
security = Security(**security)
elif security is True:
security = Security.temporary()
self._startup_kwargs["security"] = security
elif not isinstance(security, Security): # pragma: no cover
raise TypeError("security must be a Security object")
self.security = security
if name == "worker":
self.connection_args = self.security.get_connection_args("worker")
else:
self.connection_args = self.security.get_connection_args("client")
self._asynchronous = asynchronous
self._loop_runner = LoopRunner(loop=loop, asynchronous=asynchronous)
self._connecting_to_scheduler = False
self._gather_keys = None
self._gather_future = None
if heartbeat_interval is None:
heartbeat_interval = dask.config.get("distributed.client.heartbeat")
heartbeat_interval = parse_timedelta(heartbeat_interval, default="ms")
scheduler_info_interval = parse_timedelta(
dask.config.get("distributed.client.scheduler-info-interval", default="ms")
)
self._periodic_callbacks = dict()
self._periodic_callbacks["scheduler-info"] = PeriodicCallback(
self._update_scheduler_info, scheduler_info_interval * 1000
)
self._periodic_callbacks["heartbeat"] = PeriodicCallback(
self._heartbeat, heartbeat_interval * 1000
)
self._start_arg = address
self._set_as_default = set_as_default
if set_as_default:
self._set_config = dask.config.set(
scheduler="dask.distributed", shuffle="tasks"
)
self._event_handlers = {}
self._stream_handlers = {
"key-in-memory": self._handle_key_in_memory,
"lost-data": self._handle_lost_data,
"cancelled-key": self._handle_cancelled_key,
"task-retried": self._handle_retried_key,
"task-erred": self._handle_task_erred,
"restart": self._handle_restart,
"error": self._handle_error,
"event": self._handle_event,
}
self._state_handlers = {
"memory": self._handle_key_in_memory,
"lost": self._handle_lost_data,
"erred": self._handle_task_erred,
}
self.rpc = ConnectionPool(
limit=connection_limit,
serializers=serializers,
deserializers=deserializers,
deserialize=True,
connection_args=self.connection_args,
timeout=timeout,
server=self,
)
self.extensions = {
name: extension(self) for name, extension in extensions.items()
}
preload = dask.config.get("distributed.client.preload")
preload_argv = dask.config.get("distributed.client.preload-argv")
self.preloads = preloading.process_preloads(self, preload, preload_argv)
self.start(timeout=timeout)
Client._instances.add(self)
from distributed.recreate_tasks import ReplayTaskClient
ReplayTaskClient(self)
@property
def io_loop(self) -> IOLoop | None:
warnings.warn(
"The io_loop property is deprecated", DeprecationWarning, stacklevel=2
)
return self.loop
@io_loop.setter
def io_loop(self, value: IOLoop) -> None:
warnings.warn(
"The io_loop property is deprecated", DeprecationWarning, stacklevel=2
)
self.loop = value
@property
def loop(self) -> IOLoop | None:
loop = self.__loop
if loop is None:
# If the loop is not running when this is called, the LoopRunner.loop
# property will raise a DeprecationWarning
# However subsequent calls might occur - eg atexit, where a stopped
# loop is still acceptable - so we cache access to the loop.
self.__loop = loop = self._loop_runner.loop
return loop
@loop.setter
def loop(self, value: IOLoop) -> None:
warnings.warn(
"setting the loop property is deprecated", DeprecationWarning, stacklevel=2
)
self.__loop = value
@contextmanager
def as_current(self):
"""Thread-local, Task-local context manager that causes the Client.current
class method to return self. Any Future objects deserialized inside this
context manager will be automatically attached to this Client.
"""
tok = _current_client.set(self)
try:
yield
finally:
_current_client.reset(tok)
@classmethod
def current(cls, allow_global=True):
"""When running within the context of `as_client`, return the context-local
current client. Otherwise, return the latest initialised Client.
If no Client instances exist, raise ValueError.
If allow_global is set to False, raise ValueError if running outside of
the `as_client` context manager.