-
-
Notifications
You must be signed in to change notification settings - Fork 723
/
Copy pathscheduler.py
8379 lines (7073 loc) · 276 KB
/
scheduler.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
import asyncio
import heapq
import inspect
import itertools
import json
import logging
import math
import operator
import os
import random
import sys
import uuid
import warnings
import weakref
from collections import defaultdict, deque
from collections.abc import (
Callable,
Collection,
Hashable,
Iterable,
Iterator,
Mapping,
Set,
)
from contextlib import suppress
from datetime import timedelta
from functools import partial
from numbers import Number
from typing import ClassVar, Container
from typing import cast as pep484_cast
import psutil
from sortedcontainers import SortedDict, SortedSet
from tlz import (
compose,
first,
groupby,
merge,
merge_sorted,
merge_with,
pluck,
second,
valmap,
)
from tornado.ioloop import IOLoop, PeriodicCallback
import dask
from dask.highlevelgraph import HighLevelGraph
from dask.utils import format_bytes, format_time, parse_bytes, parse_timedelta, tmpfile
from dask.widgets import get_template
from distributed.utils import recursive_to_dict
from . import preloading, profile
from . import versions as version_module
from .active_memory_manager import ActiveMemoryManagerExtension
from .batched import BatchedSend
from .comm import (
Comm,
get_address_host,
normalize_address,
resolve_address,
unparse_host_port,
)
from .comm.addressing import addresses_from_user_args
from .core import CommClosedError, Status, clean_exception, rpc, send_recv
from .diagnostics.memory_sampler import MemorySamplerExtension
from .diagnostics.plugin import SchedulerPlugin, _get_plugin_name
from .event import EventExtension
from .http import get_handlers
from .lock import LockExtension
from .metrics import time
from .multi_lock import MultiLockExtension
from .node import ServerNode
from .proctitle import setproctitle
from .protocol.pickle import loads
from .publish import PublishExtension
from .pubsub import PubSubSchedulerExtension
from .queues import QueueExtension
from .recreate_tasks import ReplayTaskScheduler
from .security import Security
from .semaphore import SemaphoreExtension
from .stealing import WorkStealing
from .utils import (
All,
TimeoutError,
empty_context,
get_fileno_limit,
key_split,
key_split_group,
log_errors,
no_default,
validate_key,
)
from .utils_comm import gather_from_workers, retry_operation, scatter_to_workers
from .utils_perf import disable_gc_diagnosis, enable_gc_diagnosis
from .variable import VariableExtension
try:
from cython import compiled
except ImportError:
compiled = False
if compiled:
from cython import (
Py_hash_t,
Py_ssize_t,
bint,
cast,
ccall,
cclass,
cfunc,
declare,
double,
exceptval,
final,
inline,
nogil,
)
else:
from ctypes import c_double as double
from ctypes import c_ssize_t as Py_hash_t
from ctypes import c_ssize_t as Py_ssize_t
bint = bool
def cast(T, v, *a, **k):
return v
def ccall(func):
return func
def cclass(cls):
return cls
def cfunc(func):
return func
def declare(*a, **k):
if len(a) == 2:
return a[1]
else:
pass
def exceptval(*a, **k):
def wrapper(func):
return func
return wrapper
def final(cls):
return cls
def inline(func):
return func
def nogil(func):
return func
if sys.version_info < (3, 8):
try:
import pickle5 as pickle
except ImportError:
import pickle
else:
import pickle
logger = logging.getLogger(__name__)
LOG_PDB = dask.config.get("distributed.admin.pdb-on-err")
DEFAULT_DATA_SIZE = declare(
Py_ssize_t, parse_bytes(dask.config.get("distributed.scheduler.default-data-size"))
)
DEFAULT_EXTENSIONS = [
LockExtension,
MultiLockExtension,
PublishExtension,
ReplayTaskScheduler,
QueueExtension,
VariableExtension,
PubSubSchedulerExtension,
SemaphoreExtension,
EventExtension,
ActiveMemoryManagerExtension,
MemorySamplerExtension,
]
ALL_TASK_STATES = declare(
set, {"released", "waiting", "no-worker", "processing", "erred", "memory"}
)
globals()["ALL_TASK_STATES"] = ALL_TASK_STATES
COMPILED = declare(bint, compiled)
globals()["COMPILED"] = COMPILED
@final
@cclass
class ClientState:
"""
A simple object holding information about a client.
.. attribute:: client_key: str
A unique identifier for this client. This is generally an opaque
string generated by the client itself.
.. attribute:: wants_what: {TaskState}
A set of tasks this client wants kept in memory, so that it can
download its result when desired. This is the reverse mapping of
:class:`TaskState.who_wants`.
Tasks are typically removed from this set when the corresponding
object in the client's space (for example a ``Future`` or a Dask
collection) gets garbage-collected.
"""
_client_key: str
_hash: Py_hash_t
_wants_what: set
_last_seen: double
_versions: dict
__slots__ = ("_client_key", "_hash", "_wants_what", "_last_seen", "_versions")
def __init__(self, client: str, versions: dict = None):
self._client_key = client
self._hash = hash(client)
self._wants_what = set()
self._last_seen = time()
self._versions = versions or {}
def __hash__(self):
return self._hash
def __eq__(self, other):
typ_self: type = type(self)
typ_other: type = type(other)
if typ_self == typ_other:
other_cs: ClientState = other
return self._client_key == other_cs._client_key
else:
return False
def __repr__(self):
return "<Client '%s'>" % self._client_key
def __str__(self):
return self._client_key
@property
def client_key(self):
return self._client_key
@property
def wants_what(self):
return self._wants_what
@property
def last_seen(self):
return self._last_seen
@property
def versions(self):
return self._versions
@final
@cclass
class MemoryState:
"""Memory readings on a worker or on the whole cluster.
managed
Sum of the output of sizeof() for all dask keys held by the worker, both in
memory and spilled to disk
managed_in_memory
Sum of the output of sizeof() for the dask keys held in RAM
managed_spilled
Sum of the output of sizeof() for the dask keys spilled to the hard drive.
Note that this is the size in memory; serialized size may be different.
process
Total RSS memory measured by the OS on the worker process.
This is always exactly equal to managed_in_memory + unmanaged.
unmanaged
process - managed_in_memory. This is the sum of
- Python interpreter and modules
- global variables
- memory temporarily allocated by the dask tasks that are currently running
- memory fragmentation
- memory leaks
- memory not yet garbage collected
- memory not yet free()'d by the Python memory manager to the OS
unmanaged_old
Minimum of the 'unmanaged' measures over the last
``distributed.memory.recent-to-old-time`` seconds
unmanaged_recent
unmanaged - unmanaged_old; in other words process memory that has been recently
allocated but is not accounted for by dask; hopefully it's mostly a temporary
spike.
optimistic
managed_in_memory + unmanaged_old; in other words the memory held long-term by
the process under the hopeful assumption that all unmanaged_recent memory is a
temporary spike
"""
__slots__ = ("_process", "_managed_in_memory", "_managed_spilled", "_unmanaged_old")
_process: Py_ssize_t
_managed_in_memory: Py_ssize_t
_managed_spilled: Py_ssize_t
_unmanaged_old: Py_ssize_t
def __init__(
self,
*,
process: Py_ssize_t,
unmanaged_old: Py_ssize_t,
managed: Py_ssize_t,
managed_spilled: Py_ssize_t,
):
# Some data arrives with the heartbeat, some other arrives in realtime as the
# tasks progress. Also, sizeof() is not guaranteed to return correct results.
# This can cause glitches where a partial measure is larger than the whole, so
# we need to force all numbers to add up exactly by definition.
self._process = process
self._managed_spilled = min(managed_spilled, managed)
# Subtractions between unsigned ints guaranteed by construction to be >= 0
self._managed_in_memory = min(managed - self._managed_spilled, process)
self._unmanaged_old = min(unmanaged_old, process - self._managed_in_memory)
@property
def process(self) -> Py_ssize_t:
return self._process
@property
def managed_in_memory(self) -> Py_ssize_t:
return self._managed_in_memory
@property
def managed_spilled(self) -> Py_ssize_t:
return self._managed_spilled
@property
def unmanaged_old(self) -> Py_ssize_t:
return self._unmanaged_old
@classmethod
def sum(cls, *infos: "MemoryState") -> "MemoryState":
out = MemoryState(process=0, unmanaged_old=0, managed=0, managed_spilled=0)
ms: MemoryState
for ms in infos:
out._process += ms._process
out._managed_spilled += ms._managed_spilled
out._managed_in_memory += ms._managed_in_memory
out._unmanaged_old += ms._unmanaged_old
return out
@property
def managed(self) -> Py_ssize_t:
return self._managed_in_memory + self._managed_spilled
@property
def unmanaged(self) -> Py_ssize_t:
# This is never negative thanks to __init__
return self._process - self._managed_in_memory
@property
def unmanaged_recent(self) -> Py_ssize_t:
# This is never negative thanks to __init__
return self._process - self._managed_in_memory - self._unmanaged_old
@property
def optimistic(self) -> Py_ssize_t:
return self._managed_in_memory + self._unmanaged_old
def __repr__(self) -> str:
return (
f"Process memory (RSS) : {format_bytes(self._process)}\n"
f" - managed by Dask : {format_bytes(self._managed_in_memory)}\n"
f" - unmanaged (old) : {format_bytes(self._unmanaged_old)}\n"
f" - unmanaged (recent): {format_bytes(self.unmanaged_recent)}\n"
f"Spilled to disk : {format_bytes(self._managed_spilled)}\n"
)
@final
@cclass
class WorkerState:
"""
A simple object holding information about a worker.
.. attribute:: address: str
This worker's unique key. This can be its connected address
(such as ``'tcp://127.0.0.1:8891'``) or an alias (such as ``'alice'``).
.. attribute:: processing: {TaskState: cost}
A dictionary of tasks that have been submitted to this worker.
Each task state is associated with the expected cost in seconds
of running that task, summing both the task's expected computation
time and the expected communication time of its result.
If a task is already executing on the worker and the excecution time is
twice the learned average TaskGroup duration, this will be set to twice
the current executing time. If the task is unknown, the default task
duration is used instead of the TaskGroup average.
Multiple tasks may be submitted to a worker in advance and the worker
will run them eventually, depending on its execution resources
(but see :doc:`work-stealing`).
All the tasks here are in the "processing" state.
This attribute is kept in sync with :attr:`TaskState.processing_on`.
.. attribute:: executing: {TaskState: duration}
A dictionary of tasks that are currently being run on this worker.
Each task state is asssociated with the duration in seconds which
the task has been running.
.. attribute:: has_what: {TaskState}
An insertion-sorted set-like of tasks which currently reside on this worker.
All the tasks here are in the "memory" state.
This is the reverse mapping of :class:`TaskState.who_has`.
.. attribute:: nbytes: int
The total memory size, in bytes, used by the tasks this worker
holds in memory (i.e. the tasks in this worker's :attr:`has_what`).
.. attribute:: nthreads: int
The number of CPU threads made available on this worker.
.. attribute:: resources: {str: Number}
The available resources on this worker like ``{'gpu': 2}``.
These are abstract quantities that constrain certain tasks from
running at the same time on this worker.
.. attribute:: used_resources: {str: Number}
The sum of each resource used by all tasks allocated to this worker.
The numbers in this dictionary can only be less or equal than
those in this worker's :attr:`resources`.
.. attribute:: occupancy: double
The total expected runtime, in seconds, of all tasks currently
processing on this worker. This is the sum of all the costs in
this worker's :attr:`processing` dictionary.
.. attribute:: status: Status
Read-only worker status, synced one way from the remote Worker object
.. attribute:: nanny: str
Address of the associated Nanny, if present
.. attribute:: last_seen: Py_ssize_t
The last time we received a heartbeat from this worker, in local
scheduler time.
.. attribute:: actors: {TaskState}
A set of all TaskStates on this worker that are actors. This only
includes those actors whose state actually lives on this worker, not
actors to which this worker has a reference.
"""
# XXX need a state field to signal active/removed?
_actors: set
_address: str
_bandwidth: double
_executing: dict
_extra: dict
# _has_what is a dict with all values set to None as rebalance() relies on the
# property of Python >=3.7 dicts to be insertion-sorted.
_has_what: dict
_hash: Py_hash_t
_last_seen: double
_local_directory: str
_memory_limit: Py_ssize_t
_memory_other_history: "deque[tuple[float, Py_ssize_t]]"
_memory_unmanaged_old: Py_ssize_t
_metrics: dict
_name: object
_nanny: str
_nbytes: Py_ssize_t
_nthreads: Py_ssize_t
_occupancy: double
_pid: Py_ssize_t
_processing: dict
_resources: dict
_services: dict
_status: Status
_time_delay: double
_used_resources: dict
_versions: dict
__slots__ = (
"_actors",
"_address",
"_bandwidth",
"_extra",
"_executing",
"_has_what",
"_hash",
"_last_seen",
"_local_directory",
"_memory_limit",
"_memory_other_history",
"_memory_unmanaged_old",
"_metrics",
"_name",
"_nanny",
"_nbytes",
"_nthreads",
"_occupancy",
"_pid",
"_processing",
"_resources",
"_services",
"_status",
"_time_delay",
"_used_resources",
"_versions",
)
def __init__(
self,
*,
address: str,
status: Status,
pid: Py_ssize_t,
name: object,
nthreads: Py_ssize_t = 0,
memory_limit: Py_ssize_t,
local_directory: str,
nanny: str,
services: "dict | None" = None,
versions: "dict | None" = None,
extra: "dict | None" = None,
):
self._address = address
self._pid = pid
self._name = name
self._nthreads = nthreads
self._memory_limit = memory_limit
self._local_directory = local_directory
self._services = services or {}
self._versions = versions or {}
self._nanny = nanny
self._status = status
self._hash = hash(address)
self._nbytes = 0
self._occupancy = 0
self._memory_unmanaged_old = 0
self._memory_other_history = deque()
self._metrics = {}
self._last_seen = 0
self._time_delay = 0
self._bandwidth = float(
parse_bytes(dask.config.get("distributed.scheduler.bandwidth"))
)
self._actors = set()
self._has_what = {}
self._processing = {}
self._executing = {}
self._resources = {}
self._used_resources = {}
self._extra = extra or {}
def __hash__(self):
return self._hash
def __eq__(self, other):
typ_self: type = type(self)
typ_other: type = type(other)
if typ_self == typ_other:
other_ws: WorkerState = other
return self._address == other_ws._address
else:
return False
@property
def actors(self):
return self._actors
@property
def address(self) -> str:
return self._address
@property
def bandwidth(self):
return self._bandwidth
@property
def executing(self):
return self._executing
@property
def extra(self):
return self._extra
@property
def has_what(self) -> "Set[TaskState]":
return self._has_what.keys()
@property
def host(self):
return get_address_host(self._address)
@property
def last_seen(self):
return self._last_seen
@property
def local_directory(self):
return self._local_directory
@property
def memory_limit(self):
return self._memory_limit
@property
def metrics(self):
return self._metrics
@property
def memory(self) -> MemoryState:
return MemoryState(
# metrics["memory"] is None if the worker sent a heartbeat before its
# SystemMonitor ever had a chance to run
process=self._metrics["memory"] or 0,
managed=self._nbytes,
managed_spilled=self._metrics["spilled_nbytes"],
unmanaged_old=self._memory_unmanaged_old,
)
@property
def name(self):
return self._name
@property
def nanny(self):
return self._nanny
@property
def nbytes(self):
return self._nbytes
@nbytes.setter
def nbytes(self, v: Py_ssize_t):
self._nbytes = v
@property
def nthreads(self):
return self._nthreads
@property
def occupancy(self):
return self._occupancy
@occupancy.setter
def occupancy(self, v: double):
self._occupancy = v
@property
def pid(self):
return self._pid
@property
def processing(self):
return self._processing
@property
def resources(self):
return self._resources
@property
def services(self):
return self._services
@property
def status(self):
return self._status
@status.setter
def status(self, new_status):
if not isinstance(new_status, Status):
raise TypeError(f"Expected Status; got {new_status!r}")
self._status = new_status
@property
def time_delay(self):
return self._time_delay
@property
def used_resources(self):
return self._used_resources
@property
def versions(self):
return self._versions
@ccall
def clean(self):
"""Return a version of this object that is appropriate for serialization"""
ws: WorkerState = WorkerState(
address=self._address,
status=self._status,
pid=self._pid,
name=self._name,
nthreads=self._nthreads,
memory_limit=self._memory_limit,
local_directory=self._local_directory,
services=self._services,
nanny=self._nanny,
extra=self._extra,
)
ts: TaskState
ws._processing = {ts._key: cost for ts, cost in self._processing.items()}
ws._executing = {ts._key: duration for ts, duration in self._executing.items()}
return ws
def __repr__(self):
return "<WorkerState %r, name: %s, status: %s, memory: %d, processing: %d>" % (
self._address,
self._name,
self._status.name,
len(self._has_what),
len(self._processing),
)
def _repr_html_(self):
return get_template("worker_state.html.j2").render(
address=self.address,
name=self.name,
status=self.status.name,
has_what=self._has_what,
processing=self.processing,
)
@ccall
@exceptval(check=False)
def identity(self) -> dict:
return {
"type": "Worker",
"id": self._name,
"host": self.host,
"resources": self._resources,
"local_directory": self._local_directory,
"name": self._name,
"nthreads": self._nthreads,
"memory_limit": self._memory_limit,
"last_seen": self._last_seen,
"services": self._services,
"metrics": self._metrics,
"nanny": self._nanny,
**self._extra,
}
@property
def ncores(self):
warnings.warn("WorkerState.ncores has moved to WorkerState.nthreads")
return self._nthreads
@final
@cclass
class Computation:
"""
Collection tracking a single compute or persist call
See also
--------
TaskPrefix
TaskGroup
TaskState
"""
_start: double
_groups: set
_code: object
_id: object
def __init__(self):
self._start = time()
self._groups = set()
self._code = SortedSet()
self._id = uuid.uuid4()
@property
def code(self):
return self._code
@property
def start(self):
return self._start
@property
def stop(self):
if self.groups:
return max(tg.stop for tg in self.groups)
else:
return -1
@property
def states(self):
tg: TaskGroup
return merge_with(sum, [tg._states for tg in self._groups])
@property
def groups(self):
return self._groups
def __repr__(self):
return (
f"<Computation {self._id}: "
+ "Tasks: "
+ ", ".join(
"%s: %d" % (k, v) for (k, v) in sorted(self.states.items()) if v
)
+ ">"
)
def _repr_html_(self):
return get_template("computation.html.j2").render(
id=self._id,
start=self.start,
stop=self.stop,
groups=self.groups,
states=self.states,
code=self.code,
)
@final
@cclass
class TaskPrefix:
"""Collection tracking all tasks within a group
Keys often have a structure like ``("x-123", 0)``
A group takes the first section, like ``"x"``
.. attribute:: name: str
The name of a group of tasks.
For a task like ``("x-123", 0)`` this is the text ``"x"``
.. attribute:: states: Dict[str, int]
The number of tasks in each state,
like ``{"memory": 10, "processing": 3, "released": 4, ...}``
.. attribute:: duration_average: float
An exponentially weighted moving average duration of all tasks with this prefix
.. attribute:: suspicious: int
Numbers of times a task was marked as suspicious with this prefix
See Also
--------
TaskGroup
"""
_name: str
_all_durations: "defaultdict[str, float]"
_duration_average: double
_suspicious: Py_ssize_t
_groups: list
def __init__(self, name: str):
self._name = name
self._groups = []
# store timings for each prefix-action
self._all_durations = defaultdict(float)
task_durations = dask.config.get("distributed.scheduler.default-task-durations")
if self._name in task_durations:
self._duration_average = parse_timedelta(task_durations[self._name])
else:
self._duration_average = -1
self._suspicious = 0
@property
def name(self) -> str:
return self._name
@property
def all_durations(self) -> "defaultdict[str, float]":
return self._all_durations
@ccall
@exceptval(check=False)
def add_duration(self, action: str, start: double, stop: double):
duration = stop - start
self._all_durations[action] += duration
if action == "compute":
old = self._duration_average
if old < 0:
self._duration_average = duration
else:
self._duration_average = 0.5 * duration + 0.5 * old
@property
def duration_average(self) -> double:
return self._duration_average
@property
def suspicious(self) -> Py_ssize_t:
return self._suspicious
@property
def groups(self):
return self._groups
@property
def states(self):
tg: TaskGroup
return merge_with(sum, [tg._states for tg in self._groups])
@property
def active(self) -> "list[TaskGroup]":
tg: TaskGroup
return [
tg
for tg in self._groups
if any([v != 0 for k, v in tg._states.items() if k != "forgotten"])
]
@property
def active_states(self):
tg: TaskGroup
return merge_with(sum, [tg._states for tg in self.active])
def __repr__(self):
return (
"<"
+ self._name
+ ": "
+ ", ".join(
"%s: %d" % (k, v) for (k, v) in sorted(self.states.items()) if v
)
+ ">"
)
@property
def nbytes_total(self):
tg: TaskGroup
return sum([tg._nbytes_total for tg in self._groups])
def __len__(self):
return sum(map(len, self._groups))
@property
def duration(self):
tg: TaskGroup
return sum([tg._duration for tg in self._groups])
@property
def types(self):
tg: TaskGroup
return set().union(*[tg._types for tg in self._groups])
@final
@cclass
class TaskGroup:
"""Collection tracking all tasks within a group
Keys often have a structure like ``("x-123", 0)``
A group takes the first section, like ``"x-123"``
.. attribute:: name: str
The name of a group of tasks.