-
-
Notifications
You must be signed in to change notification settings - Fork 717
/
worker.py
4449 lines (3845 loc) · 155 KB
/
worker.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 bisect
import builtins
import errno
import heapq
import logging
import os
import random
import sys
import threading
import warnings
import weakref
from collections import defaultdict, deque, namedtuple
from collections.abc import Callable, Iterable, Mapping, MutableMapping
from concurrent.futures import Executor
from contextlib import suppress
from datetime import timedelta
from inspect import isawaitable
from pickle import PicklingError
from typing import TYPE_CHECKING, Any, ClassVar
if TYPE_CHECKING:
from typing_extensions import Literal
from .diagnostics.plugin import WorkerPlugin
from .actor import Actor
from .client import Client
from .nanny import Nanny
from tlz import first, keymap, merge, pluck # noqa: F401
from tornado.ioloop import IOLoop, PeriodicCallback
import dask
from dask.core import istask
from dask.system import CPU_COUNT
from dask.utils import (
apply,
format_bytes,
funcname,
parse_bytes,
parse_timedelta,
stringify,
typename,
)
from . import comm, preloading, profile, system, utils
from .batched import BatchedSend
from .comm import connect, get_address_host
from .comm.addressing import address_from_user_args, parse_address
from .comm.utils import OFFLOAD_THRESHOLD
from .core import (
CommClosedError,
Status,
coerce_to_address,
error_message,
pingpong,
send_recv,
)
from .diagnostics import nvml
from .diagnostics.plugin import _get_plugin_name
from .diskutils import WorkDir, WorkSpace
from .http import get_handlers
from .metrics import time
from .node import ServerNode
from .proctitle import setproctitle
from .protocol import pickle, to_serialize
from .pubsub import PubSubWorkerExtension
from .security import Security
from .sizeof import safe_sizeof as sizeof
from .threadpoolexecutor import ThreadPoolExecutor
from .threadpoolexecutor import secede as tpe_secede
from .utils import (
LRU,
TimeoutError,
_maybe_complex,
get_ip,
has_arg,
import_file,
iscoroutinefunction,
json_load_robust,
key_split,
log_errors,
offload,
parse_ports,
silence_logging,
thread_state,
warn_on_duration,
)
from .utils_comm import gather_from_workers, pack_data, retry_operation
from .utils_perf import ThrottledGC, disable_gc_diagnosis, enable_gc_diagnosis
from .versions import get_versions
logger = logging.getLogger(__name__)
LOG_PDB = dask.config.get("distributed.admin.pdb-on-err")
no_value = "--no-value-sentinel--"
# TaskState.state subsets
PROCESSING = {
"waiting",
"ready",
"constrained",
"executing",
"long-running",
"cancelled",
"resumed",
}
READY = {"ready", "constrained"}
# Worker.status subsets
RUNNING = {Status.running, Status.paused, Status.closing_gracefully}
DEFAULT_EXTENSIONS: list[type] = [PubSubWorkerExtension]
DEFAULT_METRICS: dict[str, Callable[[Worker], Any]] = {}
DEFAULT_STARTUP_INFORMATION: dict[str, Callable[[Worker], Any]] = {}
DEFAULT_DATA_SIZE = parse_bytes(
dask.config.get("distributed.scheduler.default-data-size")
)
SerializedTask = namedtuple("SerializedTask", ["function", "args", "kwargs", "task"])
class InvalidTransition(Exception):
pass
class TaskState:
"""Holds volatile state relating to an individual Dask task
* **dependencies**: ``set(TaskState instances)``
The data needed by this key to run
* **dependents**: ``set(TaskState instances)``
The keys that use this dependency.
* **duration**: ``float``
Expected duration the a task
* **priority**: ``tuple``
The priority this task given by the scheduler. Determines run order.
* **state**: ``str``
The current state of the task. One of ["waiting", "ready", "executing",
"fetch", "memory", "flight", "long-running", "rescheduled", "error"]
* **who_has**: ``set(worker)``
Workers that we believe have this data
* **coming_from**: ``str``
The worker that current task data is coming from if task is in flight
* **waiting_for_data**: ``set(keys of dependencies)``
A dynamic version of dependencies. All dependencies that we still don't
have for a particular key.
* **resource_restrictions**: ``{str: number}``
Abstract resources required to run a task
* **exception**: ``str``
The exception caused by running a task if it erred
* **traceback**: ``str``
The exception caused by running a task if it erred
* **type**: ``type``
The type of a particular piece of data
* **suspicious_count**: ``int``
The number of times a dependency has not been where we expected it
* **startstops**: ``[{startstop}]``
Log of transfer, load, and compute times for a task
* **start_time**: ``float``
Time at which task begins running
* **stop_time**: ``float``
Time at which task finishes running
* **metadata**: ``dict``
Metadata related to task. Stored metadata should be msgpack
serializable (e.g. int, string, list, dict).
* **nbytes**: ``int``
The size of a particular piece of data
* **annotations**: ``dict``
Task annotations
Parameters
----------
key: str
runspec: SerializedTask
A named tuple containing the ``function``, ``args``, ``kwargs`` and
``task`` associated with this `TaskState` instance. This defaults to
``None`` and can remain empty if it is a dependency that this worker
will receive from another worker.
"""
def __init__(self, key, runspec=None):
assert key is not None
self.key = key
self.runspec = runspec
self.dependencies = set()
self.dependents = set()
self.duration = None
self.priority = None
self.state = "released"
self.who_has = set()
self.coming_from = None
self.waiting_for_data = set()
self.waiters = set()
self.resource_restrictions = {}
self.exception = None
self.exception_text = ""
self.traceback = None
self.traceback_text = ""
self.type = None
self.suspicious_count = 0
self.startstops = []
self.start_time = None
self.stop_time = None
self.metadata = {}
self.nbytes = None
self.annotations = None
self.done = False
self._previous = None
self._next = None
def __repr__(self):
return f"<Task {self.key!r} {self.state}>"
def get_nbytes(self) -> int:
nbytes = self.nbytes
return nbytes if nbytes is not None else DEFAULT_DATA_SIZE
def is_protected(self) -> bool:
return self.state in PROCESSING or any(
dep_ts.state in PROCESSING for dep_ts in self.dependents
)
class Worker(ServerNode):
"""Worker node in a Dask distributed cluster
Workers perform two functions:
1. **Serve data** from a local dictionary
2. **Perform computation** on that data and on data from peers
Workers keep the scheduler informed of their data and use that scheduler to
gather data from other workers when necessary to perform a computation.
You can start a worker with the ``dask-worker`` command line application::
$ dask-worker scheduler-ip:port
Use the ``--help`` flag to see more options::
$ dask-worker --help
The rest of this docstring is about the internal state the the worker uses
to manage and track internal computations.
**State**
**Informational State**
These attributes don't change significantly during execution.
* **nthreads:** ``int``:
Number of nthreads used by this worker process
* **executors:** ``dict[str, concurrent.futures.Executor]``:
Executors used to perform computation. Always contains the default
executor.
* **local_directory:** ``path``:
Path on local machine to store temporary files
* **scheduler:** ``rpc``:
Location of scheduler. See ``.ip/.port`` attributes.
* **name:** ``string``:
Alias
* **services:** ``{str: Server}``:
Auxiliary web servers running on this worker
* **service_ports:** ``{str: port}``:
* **total_out_connections**: ``int``
The maximum number of concurrent outgoing requests for data
* **total_in_connections**: ``int``
The maximum number of concurrent incoming requests for data
* **comm_threshold_bytes**: ``int``
As long as the total number of bytes in flight is below this threshold
we will not limit the number of outgoing connections for a single tasks
dependency fetch.
* **batched_stream**: ``BatchedSend``
A batched stream along which we communicate to the scheduler
* **log**: ``[(message)]``
A structured and queryable log. See ``Worker.story``
**Volatile State**
These attributes track the progress of tasks that this worker is trying to
complete. In the descriptions below a ``key`` is the name of a task that
we want to compute and ``dep`` is the name of a piece of dependent data
that we want to collect from others.
* **tasks**: ``{key: TaskState}``
The tasks currently executing on this worker (and any dependencies of those tasks)
* **data:** ``{key: object}``:
Prefer using the **host** attribute instead of this, unless
memory_limit and at least one of memory_target_fraction or
memory_spill_fraction values are defined, in that case, this attribute
is a zict.Buffer, from which information on LRU cache can be queried.
* **data.memory:** ``{key: object}``:
Dictionary mapping keys to actual values stored in memory. Only
available if condition for **data** being a zict.Buffer is met.
* **data.disk:** ``{key: object}``:
Dictionary mapping keys to actual values stored on disk. Only
available if condition for **data** being a zict.Buffer is met.
* **data_needed**: deque(keys)
The keys which still require data in order to execute, arranged in a deque
* **ready**: [keys]
Keys that are ready to run. Stored in a LIFO stack
* **constrained**: [keys]
Keys for which we have the data to run, but are waiting on abstract
resources like GPUs. Stored in a FIFO deque
* **executing_count**: ``int``
A count of tasks currently executing on this worker
* **executed_count**: int
A number of tasks that this worker has run in its lifetime
* **long_running**: {keys}
A set of keys of tasks that are running and have started their own
long-running clients.
* **has_what**: ``{worker: {deps}}``
The data that we care about that we think a worker has
* **pending_data_per_worker**: ``{worker: [dep]}``
The data on each worker that we still want, prioritized as a deque
* **in_flight_tasks**: ``int``
A count of the number of tasks that are coming to us in current
peer-to-peer connections
* **in_flight_workers**: ``{worker: {task}}``
The workers from which we are currently gathering data and the
dependencies we expect from those connections
* **comm_bytes**: ``int``
The total number of bytes in flight
* **threads**: ``{key: int}``
The ID of the thread on which the task ran
* **active_threads**: ``{int: key}``
The keys currently running on active threads
* **waiting_for_data_count**: ``int``
A count of how many tasks are currently waiting for data
Parameters
----------
scheduler_ip: str, optional
scheduler_port: int, optional
scheduler_file: str, optional
ip: str, optional
data: MutableMapping, type, None
The object to use for storage, builds a disk-backed LRU dict by default
nthreads: int, optional
loop: tornado.ioloop.IOLoop
local_directory: str, optional
Directory where we place local resources
name: str, optional
memory_limit: int, float, string
Number of bytes of memory that this worker should use.
Set to zero for no limit. Set to 'auto' to calculate
as system.MEMORY_LIMIT * min(1, nthreads / total_cores)
Use strings or numbers like 5GB or 5e9
memory_target_fraction: float or False
Fraction of memory to try to stay beneath
(default: read from config key distributed.worker.memory.target)
memory_spill_fraction: float or false
Fraction of memory at which we start spilling to disk
(default: read from config key distributed.worker.memory.spill)
memory_pause_fraction: float or False
Fraction of memory at which we stop running new tasks
(default: read from config key distributed.worker.memory.pause)
executor: concurrent.futures.Executor, dict[str, concurrent.futures.Executor], "offload"
The executor(s) to use. Depending on the type, it has the following meanings:
- Executor instance: The default executor.
- Dict[str, Executor]: mapping names to Executor instances. If the
"default" key isn't in the dict, a "default" executor will be created
using ``ThreadPoolExecutor(nthreads)``.
- Str: The string "offload", which refer to the same thread pool used for
offloading communications. This results in the same thread being used
for deserialization and computation.
resources: dict
Resources that this worker has like ``{'GPU': 2}``
nanny: str
Address on which to contact nanny, if it exists
lifetime: str
Amount of time like "1 hour" after which we gracefully shut down the worker.
This defaults to None, meaning no explicit shutdown time.
lifetime_stagger: str
Amount of time like "5 minutes" to stagger the lifetime value
The actual lifetime will be selected uniformly at random between
lifetime +/- lifetime_stagger
lifetime_restart: bool
Whether or not to restart a worker after it has reached its lifetime
Default False
kwargs: optional
Additional parameters to ServerNode constructor
Examples
--------
Use the command line to start a worker::
$ dask-scheduler
Start scheduler at 127.0.0.1:8786
$ dask-worker 127.0.0.1:8786
Start worker at: 127.0.0.1:1234
Registered with scheduler at: 127.0.0.1:8786
See Also
--------
distributed.scheduler.Scheduler
distributed.nanny.Nanny
"""
_instances: ClassVar[weakref.WeakSet[Worker]] = weakref.WeakSet()
_initialized_clients: ClassVar[weakref.WeakSet[Client]] = weakref.WeakSet()
tasks: dict[str, TaskState]
waiting_for_data_count: int
has_what: defaultdict[str, set[str]] # {worker address: {ts.key, ...}
pending_data_per_worker: defaultdict[str, deque[str]]
nanny: Nanny | None
_lock: threading.Lock
data_needed: list[tuple[int, str]] # heap[(ts.priority, ts.key)]
in_flight_workers: dict[str, set[str]] # {worker address: {ts.key, ...}}
total_out_connections: int
total_in_connections: int
comm_threshold_bytes: int
comm_nbytes: int
_missing_dep_flight: set[TaskState]
threads: dict[str, int] # {ts.key: thread ID}
active_threads_lock: threading.Lock
active_threads: dict[int, str] # {thread ID: ts.key}
active_keys: set[str]
profile_keys: defaultdict[str, dict[str, Any]]
profile_keys_history: deque[tuple[float, dict[str, dict[str, Any]]]]
profile_recent: dict[str, Any]
profile_history: deque[tuple[float, dict[str, Any]]]
generation: int
ready: list[str]
constrained: deque[str]
_executing: set[TaskState]
_in_flight_tasks: set[TaskState]
executed_count: int
long_running: set[TaskState]
log: deque[tuple]
incoming_transfer_log: deque[dict[str, Any]]
outgoing_transfer_log: deque[dict[str, Any]]
target_message_size: int
validate: bool
_transitions_table: dict[tuple[str, str], Callable]
_transition_counter: int
incoming_count: int
outgoing_count: int
outgoing_current_count: int
repetitively_busy: int
bandwidth: float
latency: float
profile_cycle_interval: float
workspace: WorkSpace
_workdir: WorkDir
local_directory: str
_client: Client | None
bandwidth_workers: defaultdict[str, tuple[float, int]]
bandwidth_types: defaultdict[type, tuple[float, int]]
preloads: list[preloading.Preload]
contact_address: str | None
_start_port: int | None
_start_host: str | None
_interface: str | None
_protocol: str
_dashboard_address: str | None
_dashboard: bool
_http_prefix: str
nthreads: int
total_resources: dict[str, float]
available_resources: dict[str, float]
death_timeout: float | None
lifetime: float | None
lifetime_stagger: float | None
lifetime_restart: bool
extensions: dict
security: Security
connection_args: dict[str, Any]
memory_limit: int | None
memory_target_fraction: float | Literal[False]
memory_spill_fraction: float | Literal[False]
memory_pause_fraction: float | Literal[False]
data: MutableMapping[str, Any] # {task key: task payload}
actors: dict[str, Actor | None]
loop: IOLoop
reconnect: bool
executors: dict[str, Executor]
batched_stream: BatchedSend
name: Any
scheduler_delay: float
stream_comms: dict[str, BatchedSend]
heartbeat_active: bool
_ipython_kernel: Any | None = None
services: dict[str, Any] = {}
service_specs: dict[str, Any]
metrics: dict[str, Callable[[Worker], Any]]
startup_information: dict[str, Callable[[Worker], Any]]
low_level_profiler: bool
scheduler: Any
execution_state: dict[str, Any]
memory_monitor_interval: float | None
_memory_monitoring: bool
_throttled_gc: ThrottledGC
plugins: dict[str, WorkerPlugin]
_pending_plugins: tuple[WorkerPlugin, ...]
def __init__(
self,
scheduler_ip: str | None = None,
scheduler_port: int | None = None,
*,
scheduler_file: str | None = None,
ncores: None = None, # Deprecated, use nthreads instead
nthreads: int | None = None,
loop: IOLoop | None = None,
local_dir: None = None, # Deprecated, use local_directory instead
local_directory: str | None = None,
services: dict | None = None,
name: Any | None = None,
reconnect: bool = True,
memory_limit: str | float = "auto",
executor: Executor | dict[str, Executor] | Literal["offload"] | None = None,
resources: dict[str, float] | None = None,
silence_logs: int | None = None,
death_timeout: Any | None = None,
preload: list[str] | None = None,
preload_argv: list[str] | list[list[str]] | None = None,
security: Security | dict[str, Any] | None = None,
contact_address: str | None = None,
memory_monitor_interval: Any = "200ms",
memory_target_fraction: float | Literal[False] | None = None,
memory_spill_fraction: float | Literal[False] | None = None,
memory_pause_fraction: float | Literal[False] | None = None,
extensions: list[type] | None = None,
metrics: Mapping[str, Callable[[Worker], Any]] = DEFAULT_METRICS,
startup_information: Mapping[
str, Callable[[Worker], Any]
] = DEFAULT_STARTUP_INFORMATION,
data: (
MutableMapping[str, Any] # pre-initialised
| Callable[[], MutableMapping[str, Any]] # constructor
| tuple[
Callable[..., MutableMapping[str, Any]], dict[str, Any]
] # (constructor, kwargs to constructor)
| None # create internatlly
) = None,
interface: str | None = None,
host: str | None = None,
port: int | None = None,
protocol: str | None = None,
dashboard_address: str | None = None,
dashboard: bool = False,
http_prefix: str = "/",
nanny: Nanny | None = None,
plugins: tuple[WorkerPlugin, ...] = (),
low_level_profiler: bool | None = None,
validate: bool | None = None,
profile_cycle_interval=None,
lifetime: Any | None = None,
lifetime_stagger: Any | None = None,
lifetime_restart: bool | None = None,
**kwargs,
):
self.tasks = {}
self.waiting_for_data_count = 0
self.has_what = defaultdict(set)
self.pending_data_per_worker = defaultdict(deque)
self.nanny = nanny
self._lock = threading.Lock()
self.data_needed = []
self.in_flight_workers = {}
self.total_out_connections = dask.config.get(
"distributed.worker.connections.outgoing"
)
self.total_in_connections = dask.config.get(
"distributed.worker.connections.incoming"
)
self.comm_threshold_bytes = int(10e6)
self.comm_nbytes = 0
self._missing_dep_flight = set()
self.threads = {}
self.active_threads_lock = threading.Lock()
self.active_threads = {}
self.active_keys = set()
self.profile_keys = defaultdict(profile.create)
self.profile_keys_history = deque(maxlen=3600)
self.profile_recent = profile.create()
self.profile_history = deque(maxlen=3600)
self.generation = 0
self.ready = []
self.constrained = deque()
self._executing = set()
self._in_flight_tasks = set()
self.executed_count = 0
self.long_running = set()
self.target_message_size = int(50e6) # 50 MB
self.log = deque(maxlen=100000)
if validate is None:
validate = dask.config.get("distributed.scheduler.validate")
self.validate = validate
self._transitions_table = {
("cancelled", "resumed"): self.transition_cancelled_resumed,
("cancelled", "fetch"): self.transition_cancelled_fetch,
("cancelled", "released"): self.transition_cancelled_released,
("cancelled", "waiting"): self.transition_cancelled_waiting,
("cancelled", "forgotten"): self.transition_cancelled_forgotten,
("cancelled", "memory"): self.transition_cancelled_memory,
("cancelled", "error"): self.transition_generic_error,
("resumed", "memory"): self.transition_generic_memory,
("resumed", "error"): self.transition_generic_error,
("resumed", "released"): self.transition_generic_released,
("resumed", "waiting"): self.transition_rescheduled_next,
("resumed", "fetch"): self.transition_rescheduled_next,
("constrained", "executing"): self.transition_constrained_executing,
("constrained", "released"): self.transition_constrained_released,
("error", "released"): self.transition_generic_released,
("executing", "error"): self.transition_executing_error,
("executing", "long-running"): self.transition_executing_long_running,
("executing", "memory"): self.transition_executing_memory,
("executing", "released"): self.transition_executing_released,
("executing", "rescheduled"): self.transition_executing_rescheduled,
("fetch", "flight"): self.transition_fetch_flight,
("fetch", "missing"): self.transition_fetch_missing,
("fetch", "released"): self.transition_generic_released,
("flight", "error"): self.transition_flight_error,
("flight", "fetch"): self.transition_flight_fetch,
("flight", "memory"): self.transition_flight_memory,
("flight", "released"): self.transition_flight_released,
("long-running", "error"): self.transition_generic_error,
("long-running", "memory"): self.transition_long_running_memory,
("long-running", "rescheduled"): self.transition_executing_rescheduled,
("long-running", "released"): self.transition_executing_released,
("memory", "released"): self.transition_memory_released,
("missing", "fetch"): self.transition_missing_fetch,
("missing", "released"): self.transition_missing_released,
("missing", "error"): self.transition_generic_error,
("ready", "error"): self.transition_generic_error,
("ready", "executing"): self.transition_ready_executing,
("ready", "released"): self.transition_generic_released,
("released", "error"): self.transition_generic_error,
("released", "fetch"): self.transition_released_fetch,
("released", "forgotten"): self.transition_released_forgotten,
("released", "memory"): self.transition_released_memory,
("released", "waiting"): self.transition_released_waiting,
("waiting", "constrained"): self.transition_waiting_constrained,
("waiting", "ready"): self.transition_waiting_ready,
("waiting", "released"): self.transition_generic_released,
}
self._transition_counter = 0
self.incoming_transfer_log = deque(maxlen=100000)
self.incoming_count = 0
self.outgoing_transfer_log = deque(maxlen=100000)
self.outgoing_count = 0
self.outgoing_current_count = 0
self.repetitively_busy = 0
self.bandwidth = parse_bytes(dask.config.get("distributed.scheduler.bandwidth"))
self.bandwidth_workers = defaultdict(
lambda: (0, 0)
) # bw/count recent transfers
self.bandwidth_types = defaultdict(lambda: (0, 0)) # bw/count recent transfers
self.latency = 0.001
self._client = None
if profile_cycle_interval is None:
profile_cycle_interval = dask.config.get("distributed.worker.profile.cycle")
profile_cycle_interval = parse_timedelta(profile_cycle_interval, default="ms")
assert profile_cycle_interval
self._setup_logging(logger)
if local_dir is not None:
warnings.warn("The local_dir keyword has moved to local_directory")
local_directory = local_dir
if not local_directory:
local_directory = dask.config.get("temporary-directory") or os.getcwd()
os.makedirs(local_directory, exist_ok=True)
local_directory = os.path.join(local_directory, "dask-worker-space")
with warn_on_duration(
"1s",
"Creating scratch directories is taking a surprisingly long time. "
"This is often due to running workers on a network file system. "
"Consider specifying a local-directory to point workers to write "
"scratch data to a local disk.",
):
self._workspace = WorkSpace(os.path.abspath(local_directory))
self._workdir = self._workspace.new_work_dir(prefix="worker-")
self.local_directory = self._workdir.dir_path
if not preload:
preload = dask.config.get("distributed.worker.preload")
if not preload_argv:
preload_argv = dask.config.get("distributed.worker.preload-argv")
assert preload is not None
assert preload_argv is not None
self.preloads = preloading.process_preloads(
self, preload, preload_argv, file_dir=self.local_directory
)
if scheduler_file:
cfg = json_load_robust(scheduler_file)
scheduler_addr = cfg["address"]
elif scheduler_ip is None and dask.config.get("scheduler-address", None):
scheduler_addr = dask.config.get("scheduler-address")
elif scheduler_port is None:
scheduler_addr = coerce_to_address(scheduler_ip)
else:
scheduler_addr = coerce_to_address((scheduler_ip, scheduler_port))
self.contact_address = contact_address
if protocol is None:
protocol_address = scheduler_addr.split("://")
if len(protocol_address) == 2:
protocol = protocol_address[0]
assert protocol
self._start_port = port
self._start_host = host
if host:
# Helpful error message if IPv6 specified incorrectly
_, host_address = parse_address(host)
if host_address.count(":") > 1 and not host_address.startswith("["):
raise ValueError(
"Host address with IPv6 must be bracketed like '[::1]'; "
f"got {host_address}"
)
self._interface = interface
self._protocol = protocol
if ncores is not None:
warnings.warn("the ncores= parameter has moved to nthreads=")
nthreads = ncores
self.nthreads = nthreads or CPU_COUNT
if resources is None:
resources = dask.config.get("distributed.worker.resources", None)
assert isinstance(resources, dict)
self.total_resources = resources or {}
self.available_resources = (resources or {}).copy()
self.death_timeout = parse_timedelta(death_timeout)
self.extensions = {}
if silence_logs:
silence_logging(level=silence_logs)
if isinstance(security, dict):
security = Security(**security)
self.security = security or Security()
assert isinstance(self.security, Security)
self.connection_args = self.security.get_connection_args("worker")
self.memory_limit = parse_memory_limit(memory_limit, self.nthreads)
self.memory_target_fraction = (
memory_target_fraction
if memory_target_fraction is not None
else dask.config.get("distributed.worker.memory.target")
)
self.memory_spill_fraction = (
memory_spill_fraction
if memory_spill_fraction is not None
else dask.config.get("distributed.worker.memory.spill")
)
self.memory_pause_fraction = (
memory_pause_fraction
if memory_pause_fraction is not None
else dask.config.get("distributed.worker.memory.pause")
)
if isinstance(data, MutableMapping):
self.data = data
elif callable(data):
self.data = data()
elif isinstance(data, tuple):
self.data = data[0](**data[1])
elif self.memory_limit and (
self.memory_target_fraction or self.memory_spill_fraction
):
from .spill import SpillBuffer
self.data = SpillBuffer(
os.path.join(self.local_directory, "storage"),
target=int(
self.memory_limit
* (self.memory_target_fraction or self.memory_spill_fraction)
)
or sys.maxsize,
)
else:
self.data = {}
self.actors = {}
self.loop = loop or IOLoop.current()
self.reconnect = reconnect
# Common executors always available
self.executors = {
"offload": utils._offload_executor,
"actor": ThreadPoolExecutor(1, thread_name_prefix="Dask-Actor-Threads"),
}
if nvml.device_get_count() > 0:
self.executors["gpu"] = ThreadPoolExecutor(
1, thread_name_prefix="Dask-GPU-Threads"
)
# Find the default executor
if executor == "offload":
self.executors["default"] = self.executors["offload"]
elif isinstance(executor, dict):
self.executors.update(executor)
elif executor is not None:
self.executors["default"] = executor
if "default" not in self.executors:
self.executors["default"] = ThreadPoolExecutor(
self.nthreads, thread_name_prefix="Dask-Default-Threads"
)
self.batched_stream = BatchedSend(interval="2ms", loop=self.loop)
self.name = name
self.scheduler_delay = 0
self.stream_comms = {}
self.heartbeat_active = False
self._ipython_kernel = None
if self.local_directory not in sys.path:
sys.path.insert(0, self.local_directory)
self.services = {}
self.service_specs = services or {}
self._dashboard_address = dashboard_address
self._dashboard = dashboard
self._http_prefix = http_prefix
self.metrics = dict(metrics) if metrics else {}
self.startup_information = (
dict(startup_information) if startup_information else {}
)
if low_level_profiler is None:
low_level_profiler = dask.config.get("distributed.worker.profile.low-level")
self.low_level_profiler = low_level_profiler
handlers = {
"gather": self.gather,
"run": self.run,
"run_coroutine": self.run_coroutine,
"get_data": self.get_data,
"update_data": self.update_data,
"free_keys": self.handle_free_keys,
"terminate": self.close,
"ping": pingpong,
"upload_file": self.upload_file,
"start_ipython": self.start_ipython,
"call_stack": self.get_call_stack,
"profile": self.get_profile,
"profile_metadata": self.get_profile_metadata,
"get_logs": self.get_logs,
"keys": self.keys,
"versions": self.versions,
"actor_execute": self.actor_execute,
"actor_attribute": self.actor_attribute,
"plugin-add": self.plugin_add,
"plugin-remove": self.plugin_remove,
"get_monitor_info": self.get_monitor_info,
}
stream_handlers = {
"close": self.close,
"cancel-compute": self.handle_cancel_compute,
"acquire-replicas": self.handle_acquire_replicas,
"compute-task": self.handle_compute_task,
"free-keys": self.handle_free_keys,
"remove-replicas": self.handle_remove_replicas,
"steal-request": self.handle_steal_request,
}
super().__init__(
handlers=handlers,
stream_handlers=stream_handlers,
io_loop=self.loop,
connection_args=self.connection_args,
**kwargs,
)
self.scheduler = self.rpc(scheduler_addr)
self.execution_state = {
"scheduler": self.scheduler.address,
"ioloop": self.loop,
"worker": self,
}
pc = PeriodicCallback(self.heartbeat, 1000)
self.periodic_callbacks["heartbeat"] = pc
pc = PeriodicCallback(
lambda: self.batched_stream.send({"op": "keep-alive"}), 60000
)
self.periodic_callbacks["keep-alive"] = pc
pc = PeriodicCallback(self.find_missing, 1000)
self.periodic_callbacks["find-missing"] = pc
self._address = contact_address
self.memory_monitor_interval = parse_timedelta(
memory_monitor_interval, default="ms"
)
self._memory_monitoring = False
if self.memory_limit:
assert self.memory_monitor_interval is not None
pc = PeriodicCallback(
self.memory_monitor, self.memory_monitor_interval * 1000
)
self.periodic_callbacks["memory"] = pc
if extensions is None:
extensions = DEFAULT_EXTENSIONS
for ext in extensions:
ext(self)
self._throttled_gc = ThrottledGC(logger=logger)
setproctitle("dask-worker [not started]")
profile_trigger_interval = parse_timedelta(
dask.config.get("distributed.worker.profile.interval"), default="ms"
)
pc = PeriodicCallback(self.trigger_profile, profile_trigger_interval * 1000)
self.periodic_callbacks["profile"] = pc
pc = PeriodicCallback(self.cycle_profile, profile_cycle_interval * 1000)
self.periodic_callbacks["profile-cycle"] = pc
self.plugins = {}
self._pending_plugins = plugins
if lifetime is None:
lifetime = dask.config.get("distributed.worker.lifetime.duration")
self.lifetime = parse_timedelta(lifetime)
if lifetime_stagger is None:
lifetime_stagger = dask.config.get("distributed.worker.lifetime.stagger")
lifetime_stagger = parse_timedelta(lifetime_stagger)
if lifetime_restart is None:
lifetime_restart = dask.config.get("distributed.worker.lifetime.restart")
self.lifetime_restart = lifetime_restart
if self.lifetime:
self.lifetime += (random.random() * 2 - 1) * lifetime_stagger
self.io_loop.call_later(self.lifetime, self.close_gracefully)
Worker._instances.add(self)
##################
# Administrative #
##################
def __repr__(self):
return "<%s: %r, %s, %s, stored: %d, running: %d/%d, ready: %d, comm: %d, waiting: %d>" % (
self.__class__.__name__,
self.address,
self.name,
self.status,
len(self.data),
self.executing_count,
self.nthreads,
len(self.ready),
self.in_flight_tasks,
self.waiting_for_data_count,
)
@property
def logs(self):
return self._deque_handler.deque
def log_event(self, topic, msg):
self.batched_stream.send(
{
"op": "log-event",
"topic": topic,
"msg": msg,
}
)