File: test_worker_state_machine.py

package info (click to toggle)
dask.distributed 2022.12.1%2Bds.1-3
  • links: PTS, VCS
  • area: main
  • in suites: bookworm
  • size: 10,164 kB
  • sloc: python: 81,938; javascript: 1,549; makefile: 228; sh: 100
file content (1722 lines) | stat: -rw-r--r-- 55,944 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
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
from __future__ import annotations

import asyncio
import gc
import pickle
from collections import defaultdict
from collections.abc import Iterator

import pytest
from tlz import first

from dask.sizeof import sizeof

import distributed.profile as profile
from distributed import Nanny, Worker, wait
from distributed.protocol.serialize import Serialize
from distributed.scheduler import TaskState as SchedulerTaskState
from distributed.utils import recursive_to_dict
from distributed.utils_test import (
    NO_AMM,
    _LockedCommPool,
    assert_story,
    freeze_data_fetching,
    gen_cluster,
    inc,
    wait_for_state,
)
from distributed.worker_state_machine import (
    AcquireReplicasEvent,
    AddKeysMsg,
    ComputeTaskEvent,
    Execute,
    ExecuteFailureEvent,
    ExecuteSuccessEvent,
    FreeKeysEvent,
    GatherDep,
    GatherDepFailureEvent,
    GatherDepSuccessEvent,
    Instruction,
    InvalidTaskState,
    InvalidTransition,
    PauseEvent,
    RecommendationsConflict,
    RefreshWhoHasEvent,
    ReleaseWorkerDataMsg,
    RemoveReplicasEvent,
    RescheduleEvent,
    RescheduleMsg,
    SecedeEvent,
    SerializedTask,
    StateMachineEvent,
    TaskErredMsg,
    TaskState,
    TransitionCounterMaxExceeded,
    UnpauseEvent,
    UpdateDataEvent,
    merge_recs_instructions,
)


def test_instruction_match():
    i = ReleaseWorkerDataMsg(key="x", stimulus_id="s1")
    assert i == ReleaseWorkerDataMsg(key="x", stimulus_id="s1")
    assert i != ReleaseWorkerDataMsg(key="y", stimulus_id="s1")
    assert i != ReleaseWorkerDataMsg(key="x", stimulus_id="s2")
    assert i != RescheduleMsg(key="x", stimulus_id="s1")

    assert i == ReleaseWorkerDataMsg.match(key="x")
    assert i == ReleaseWorkerDataMsg.match(stimulus_id="s1")
    assert i != ReleaseWorkerDataMsg.match(key="y")
    assert i != ReleaseWorkerDataMsg.match(stimulus_id="s2")
    assert i != RescheduleMsg.match(key="x")


def test_TaskState_tracking(cleanup):
    gc.collect()
    x = TaskState("x")
    assert len(TaskState._instances) == 1
    assert first(TaskState._instances) == x
    del x
    assert len(TaskState._instances) == 0


def test_TaskState_get_nbytes():
    assert TaskState("x", nbytes=123).get_nbytes() == 123
    # Default to distributed.scheduler.default-data-size
    assert TaskState("y").get_nbytes() == 1024


def test_TaskState_eq():
    """Test that TaskState objects are hashable and that two identical objects compare
    as different. See comment in TaskState.__hash__ for why.
    """
    a = TaskState("x")
    b = TaskState("x")
    assert a != b
    s = {a, b}
    assert len(s) == 2


def test_TaskState__to_dict():
    """Tasks that are listed as dependencies or dependents of other tasks are dumped as
    a short repr and always appear in full directly under Worker.state.tasks.
    Uninteresting fields are omitted.
    """
    x = TaskState("x", state="memory", done=True)
    y = TaskState("y", priority=(0,), dependencies={x})
    x.dependents.add(y)
    actual = recursive_to_dict([x, y])
    assert actual == [
        {
            "key": "x",
            "state": "memory",
            "done": True,
            "dependents": ["<TaskState 'y' released>"],
        },
        {
            "key": "y",
            "state": "released",
            "dependencies": ["<TaskState 'x' memory>"],
            "priority": [0],
        },
    ]


def test_TaskState_repr():
    ts = TaskState("x")
    assert str(ts) == "<TaskState 'x' released>"
    ts.state = "cancelled"
    ts.previous = "flight"
    assert str(ts) == "<TaskState 'x' cancelled(flight)>"
    ts.state = "resumed"
    ts.next = "waiting"
    assert str(ts) == "<TaskState 'x' resumed(flight->waiting)>"


def test_WorkerState__to_dict(ws):
    ws.handle_stimulus(
        AcquireReplicasEvent(
            who_has={"x": ["127.0.0.1:1235"]}, nbytes={"x": 123}, stimulus_id="s1"
        )
    )
    ws.handle_stimulus(
        UpdateDataEvent(data={"y": object()}, report=False, stimulus_id="s2")
    )

    actual = recursive_to_dict(ws)
    # Remove timestamps
    for ev in actual["log"]:
        del ev[-1]
    for stim in actual["stimulus_log"]:
        del stim["handled"]

    expect = {
        "address": "127.0.0.1:1",
        "busy_workers": [],
        "constrained": [],
        "data": {"y": None},
        "data_needed": {},
        "executing": [],
        "has_what": {"127.0.0.1:1235": ["x"]},
        "in_flight_tasks": ["x"],
        "in_flight_workers": {"127.0.0.1:1235": ["x"]},
        "log": [
            ["x", "ensure-task-exists", "released", "s1"],
            ["x", "released", "fetch", "fetch", {}, "s1"],
            ["gather-dependencies", "127.0.0.1:1235", ["x"], "s1"],
            ["x", "fetch", "flight", "flight", {}, "s1"],
            ["y", "put-in-memory", "s2"],
            ["y", "receive-from-scatter", "s2"],
        ],
        "long_running": [],
        "missing_dep_flight": [],
        "nthreads": 1,
        "ready": [],
        "running": True,
        "stimulus_log": [
            {
                "cls": "AcquireReplicasEvent",
                "stimulus_id": "s1",
                "who_has": {"x": ["127.0.0.1:1235"]},
                "nbytes": {"x": 123},
            },
            {
                "cls": "UpdateDataEvent",
                "data": {"y": None},
                "report": False,
                "stimulus_id": "s2",
            },
        ],
        "tasks": {
            "x": {
                "coming_from": "127.0.0.1:1235",
                "key": "x",
                "nbytes": 123,
                "priority": [1],
                "state": "flight",
                "who_has": ["127.0.0.1:1235"],
            },
            "y": {
                "key": "y",
                "nbytes": sizeof(object()),
                "state": "memory",
            },
        },
        "transition_counter": 2,
    }
    assert actual == expect


def test_WorkerState_pickle(ws):
    """Test pickle round-trip.

    Big caveat
    ----------
    WorkerState, on its own, can be serialized with pickle; it doesn't need cloudpickle.
    A WorkerState extracted from a Worker might, as data contents may only be
    serializable with cloudpickle. Some objects created externally and not designed
    for network transfer - namely, the SpillBuffer - may not be serializable at all.
    """
    ws.handle_stimulus(
        AcquireReplicasEvent(
            who_has={"x": ["127.0.0.1:1235"]}, nbytes={"x": 123}, stimulus_id="s1"
        )
    )
    ws.handle_stimulus(UpdateDataEvent(data={"y": 123}, report=False, stimulus_id="s"))
    ws2 = pickle.loads(pickle.dumps(ws))
    assert ws2.tasks.keys() == {"x", "y"}
    assert ws2.data == {"y": 123}


@pytest.mark.parametrize(
    "cls,kwargs",
    [
        (
            InvalidTransition,
            dict(key="x", start="released", finish="waiting", story=[]),
        ),
        (
            TransitionCounterMaxExceeded,
            dict(key="x", start="released", finish="waiting", story=[]),
        ),
        (InvalidTaskState, dict(key="x", state="released", story=[])),
    ],
)
@pytest.mark.parametrize("positional", [False, True])
def test_pickle_exceptions(cls, kwargs, positional):
    if positional:
        e = cls(*kwargs.values())
    else:
        e = cls(**kwargs)
    e2 = pickle.loads(pickle.dumps(e))
    assert type(e2) is type(e)
    for k, v in kwargs.items():
        assert getattr(e2, k) == v


def traverse_subclasses(cls: type) -> Iterator[type]:
    yield cls
    for subcls in cls.__subclasses__():
        yield from traverse_subclasses(subcls)


@pytest.mark.parametrize(
    "cls",
    [
        *traverse_subclasses(Instruction),
        *traverse_subclasses(StateMachineEvent),
    ],
)
def test_slots(cls):
    params = [
        k
        for k in dir(cls)
        if not k.startswith("_")
        and k not in ("op", "handled")
        and not callable(getattr(cls, k))
    ]
    inst = cls(**dict.fromkeys(params))
    assert not hasattr(inst, "__dict__")


def test_sendmsg_to_dict():
    # Arbitrary sample class
    smsg = ReleaseWorkerDataMsg(key="x", stimulus_id="test")
    assert smsg.to_dict() == {
        "op": "release-worker-data",
        "key": "x",
        "stimulus_id": "test",
    }


def test_merge_recs_instructions():
    x = TaskState("x")
    y = TaskState("y")
    instr1 = RescheduleMsg(key="foo", stimulus_id="test")
    instr2 = RescheduleMsg(key="bar", stimulus_id="test")
    assert merge_recs_instructions(
        ({x: "memory"}, [instr1]),
        ({y: "released"}, [instr2]),
    ) == (
        {x: "memory", y: "released"},
        [instr1, instr2],
    )

    # Identical recommendations are silently ignored; incompatible ones raise
    assert merge_recs_instructions(({x: "memory"}, []), ({x: "memory"}, [])) == (
        {x: "memory"},
        [],
    )
    with pytest.raises(RecommendationsConflict):
        merge_recs_instructions(({x: "memory"}, []), ({x: "released"}, []))


def test_event_to_dict_with_annotations():
    """Test recursive_to_dict(ev), where ev is a subclass of StateMachineEvent that
    defines its own annotations
    """
    ev = RescheduleEvent(stimulus_id="test", key="x")
    ev2 = ev.to_loggable(handled=11.22)
    assert ev2 == ev
    d = recursive_to_dict(ev2)
    assert d == {
        "cls": "RescheduleEvent",
        "stimulus_id": "test",
        "handled": 11.22,
        "key": "x",
    }
    ev3 = StateMachineEvent.from_dict(d)
    assert ev3 == ev


def test_event_to_dict_without_annotations():
    """Test recursive_to_dict(ev), where ev is a subclass of StateMachineEvent that
    does not define its own annotations
    """
    ev = PauseEvent(stimulus_id="test")
    ev2 = ev.to_loggable(handled=11.22)
    assert ev2 == ev
    d = recursive_to_dict(ev2)
    assert d == {
        "cls": "PauseEvent",
        "stimulus_id": "test",
        "handled": 11.22,
    }
    ev3 = StateMachineEvent.from_dict(d)
    assert ev3 == ev


def test_computetask_to_dict():
    """The potentially very large ComputeTaskEvent.run_spec is not stored in the log"""
    ev = ComputeTaskEvent(
        key="x",
        who_has={"y": ["w1"]},
        nbytes={"y": 123},
        priority=(0,),
        duration=123.45,
        run_spec=None,
        resource_restrictions={},
        actor=False,
        annotations={},
        stimulus_id="test",
        function=b"blob",
        args=b"blob",
        kwargs=None,
    )
    assert ev.run_spec == SerializedTask(function=b"blob", args=b"blob")
    ev2 = ev.to_loggable(handled=11.22)
    assert ev2.handled == 11.22
    assert ev2.run_spec == SerializedTask(task=None)
    assert ev.run_spec == SerializedTask(function=b"blob", args=b"blob")
    d = recursive_to_dict(ev2)
    assert d == {
        "cls": "ComputeTaskEvent",
        "key": "x",
        "who_has": {"y": ["w1"]},
        "nbytes": {"y": 123},
        "priority": [0],
        "run_spec": [None, None, None, None],
        "duration": 123.45,
        "resource_restrictions": {},
        "actor": False,
        "annotations": {},
        "stimulus_id": "test",
        "handled": 11.22,
        "function": None,
        "args": None,
        "kwargs": None,
    }
    ev3 = StateMachineEvent.from_dict(d)
    assert isinstance(ev3, ComputeTaskEvent)
    assert ev3.run_spec == SerializedTask(task=None)
    assert ev3.priority == (0,)  # List is automatically converted back to tuple


def test_computetask_dummy():
    ev = ComputeTaskEvent.dummy("x", stimulus_id="s")
    assert ev == ComputeTaskEvent(
        key="x",
        who_has={},
        nbytes={},
        priority=(0,),
        duration=1.0,
        run_spec=None,
        resource_restrictions={},
        actor=False,
        annotations={},
        stimulus_id="s",
        function=None,
        args=None,
        kwargs=None,
    )

    # nbytes is generated from who_has if omitted
    ev2 = ComputeTaskEvent.dummy("x", who_has={"y": "127.0.0.1:2"}, stimulus_id="s")
    assert ev2.nbytes == {"y": 1}


def test_updatedata_to_dict():
    """The potentially very large UpdateDataEvent.data is not stored in the log"""
    ev = UpdateDataEvent(
        data={"x": "foo", "y": "bar"},
        report=True,
        stimulus_id="test",
    )
    ev2 = ev.to_loggable(handled=11.22)
    assert ev2.handled == 11.22
    assert ev2.data == {"x": None, "y": None}
    d = recursive_to_dict(ev2)
    assert d == {
        "cls": "UpdateDataEvent",
        "data": {"x": None, "y": None},
        "report": True,
        "stimulus_id": "test",
        "handled": 11.22,
    }
    ev3 = StateMachineEvent.from_dict(d)
    assert isinstance(ev3, UpdateDataEvent)
    assert ev3.data == {"x": None, "y": None}


def test_executesuccess_to_dict():
    """The potentially very large ExecuteSuccessEvent.value is not stored in the log"""
    ev = ExecuteSuccessEvent(
        stimulus_id="test",
        key="x",
        value=123,
        start=123.4,
        stop=456.7,
        nbytes=890,
        type=int,
    )
    ev2 = ev.to_loggable(handled=11.22)
    assert ev2.value is None
    assert ev.value == 123
    d = recursive_to_dict(ev2)
    assert d == {
        "cls": "ExecuteSuccessEvent",
        "stimulus_id": "test",
        "handled": 11.22,
        "key": "x",
        "value": None,
        "nbytes": 890,
        "start": 123.4,
        "stop": 456.7,
        "type": "<class 'int'>",
    }
    ev3 = StateMachineEvent.from_dict(d)
    assert isinstance(ev3, ExecuteSuccessEvent)
    assert ev3.stimulus_id == "test"
    assert ev3.handled == 11.22
    assert ev3.key == "x"
    assert ev3.value is None
    assert ev3.start == 123.4
    assert ev3.stop == 456.7
    assert ev3.nbytes == 890
    assert ev3.type is None


def test_executesuccess_dummy():
    ev = ExecuteSuccessEvent.dummy("x", stimulus_id="s")
    assert ev == ExecuteSuccessEvent(
        key="x",
        value=None,
        start=0.0,
        stop=1.0,
        nbytes=1,
        type=None,
        stimulus_id="s",
    )

    ev2 = ExecuteSuccessEvent.dummy("x", 123, stimulus_id="s")
    assert ev2.value == 123


def test_executefailure_to_dict():
    ev = ExecuteFailureEvent(
        stimulus_id="test",
        key="x",
        start=123.4,
        stop=456.7,
        exception=Serialize(ValueError("foo")),
        traceback=Serialize("lose me"),
        exception_text="exc text",
        traceback_text="tb text",
    )
    ev2 = ev.to_loggable(handled=11.22)
    assert ev2 == ev
    d = recursive_to_dict(ev2)
    assert d == {
        "cls": "ExecuteFailureEvent",
        "stimulus_id": "test",
        "handled": 11.22,
        "key": "x",
        "start": 123.4,
        "stop": 456.7,
        "exception": "<Serialize: foo>",
        "traceback": "<Serialize: lose me>",
        "exception_text": "exc text",
        "traceback_text": "tb text",
    }
    ev3 = StateMachineEvent.from_dict(d)
    assert isinstance(ev3, ExecuteFailureEvent)
    assert ev3.stimulus_id == "test"
    assert ev3.handled == 11.22
    assert ev3.key == "x"
    assert ev3.start == 123.4
    assert ev3.stop == 456.7
    assert isinstance(ev3.exception, Serialize)
    assert isinstance(ev3.exception.data, Exception)
    assert ev3.traceback is None
    assert ev3.exception_text == "exc text"
    assert ev3.traceback_text == "tb text"


def test_executefailure_dummy():
    ev = ExecuteFailureEvent.dummy("x", stimulus_id="s")
    assert ev == ExecuteFailureEvent(
        key="x",
        start=None,
        stop=None,
        exception=Serialize(None),
        traceback=None,
        exception_text="",
        traceback_text="",
        stimulus_id="s",
    )


@gen_cluster(client=True)
async def test_fetch_to_compute(c, s, a, b):
    with freeze_data_fetching(b):
        f1 = c.submit(inc, 1, workers=[a.address], key="f1", allow_other_workers=True)
        f2 = c.submit(inc, f1, workers=[b.address], key="f2")
        await wait_for_state(f1.key, "fetch", b)
        await a.close()

    await f2

    assert_story(
        b.state.log,
        # FIXME: This log should be replaced with a StateMachineEvent log
        [
            (f2.key, "compute-task", "released"),
            # This is a "please fetch" request. We don't have anything like
            # this, yet. We don't see the request-dep signal in here because we
            # do not wait for the key to be actually scheduled
            (f1.key, "ensure-task-exists", "released"),
            # After the worker failed, we're instructed to forget f2 before
            # something new comes in
            ("free-keys", (f2.key,)),
            (f1.key, "compute-task", "released"),
            (f1.key, "put-in-memory"),
            (f2.key, "compute-task", "released"),
        ],
    )


@gen_cluster(client=True)
async def test_fetch_via_amm_to_compute(c, s, a, b):
    with freeze_data_fetching(b):
        f1 = c.submit(inc, 1, workers=[a.address], key="f1", allow_other_workers=True)
        await f1
        s.request_acquire_replicas(b.address, [f1.key], stimulus_id="test")
        await wait_for_state(f1.key, "fetch", b)
        await a.close()

    await f1

    assert_story(
        b.state.log,
        # FIXME: This log should be replaced with a StateMachineEvent log
        [
            (f1.key, "ensure-task-exists", "released"),
            (f1.key, "released", "fetch", "fetch", {}),
            (f1.key, "compute-task", "fetch"),
            (f1.key, "put-in-memory"),
        ],
    )


@pytest.mark.parametrize("as_deps", [False, True])
@gen_cluster(
    client=True,
    nthreads=[("", 1)] * 3,
    config=NO_AMM,
)
async def test_lose_replica_during_fetch(c, s, w1, w2, w3, as_deps):
    """
    as_deps=True
        0. task x is a dependency of y1 and y2
        1. scheduler calls handle_compute("y1", who_has={"x": [w2, w3]}) on w1
        2. x transitions released -> fetch
        3. the network stack is busy, so x does not transition to flight yet.
        4. scheduler calls handle_compute("y2", who_has={"x": [w3]}) on w1
        5. when x finally reaches the top of the data_needed heap, w1 will not try
           contacting w2

    as_deps=False
        1. scheduler calls handle_acquire_replicas(who_has={"x": [w2, w3]}) on w1
        2. x transitions released -> fetch
        3. the network stack is busy, so x does not transition to flight yet.
        4. scheduler calls handle_acquire_replicas(who_has={"x": [w3]}) on w1
        5. when x finally reaches the top of the data_needed heap, w1 will not try
           contacting w2
    """
    x = (await c.scatter({"x": 1}, workers=[w2.address, w3.address], broadcast=True))[
        "x"
    ]

    # Make sure find_missing is not involved
    w1.periodic_callbacks["find-missing"].stop()

    with freeze_data_fetching(w1, jump_start=True):
        if as_deps:
            y1 = c.submit(inc, x, key="y1", workers=[w1.address])
        else:
            s.request_acquire_replicas(w1.address, ["x"], stimulus_id="test")

        await wait_for_state("x", "fetch", w1)
        assert w1.state.tasks["x"].who_has == {w2.address, w3.address}

        assert len(s.tasks["x"].who_has) == 2
        await w2.close()
        while len(s.tasks["x"].who_has) > 1:
            await asyncio.sleep(0.01)

        if as_deps:
            y2 = c.submit(inc, x, key="y2", workers=[w1.address])
        else:
            s.request_acquire_replicas(w1.address, ["x"], stimulus_id="test")

        while w1.state.tasks["x"].who_has != {w3.address}:
            await asyncio.sleep(0.01)

    await wait_for_state("x", "memory", w1)
    assert_story(
        w1.state.story("request-dep"),
        [("request-dep", w3.address, {"x"})],
        # This tests that there has been no attempt to contact w2.
        # If the assumption being tested breaks, this will fail 50% of the times.
        strict=True,
    )


@gen_cluster(client=True, nthreads=[("", 1)] * 2)
async def test_fetch_to_missing_on_busy(c, s, a, b):
    """
    1. task x is a dependency of y
    2. scheduler calls handle_compute("y", who_has={"x": [b]}) on a
    3. x transitions released -> fetch -> flight; a connects to b
    4. b responds it's busy. x transitions flight -> fetch
    5. The busy state triggers an RPC call to Scheduler.who_has
    6. the scheduler responds {"x": []}, because w1 in the meantime has lost the key.
    7. x is transitioned fetch -> missing
    """
    # Note: submit and scatter are different. If you lose all workers holding the
    # replicas of a scattered key, the scheduler forgets the task, which in turn would
    # trigger a free-keys response to request-refresh-who-has.
    x = c.submit(inc, 1, key="x", workers=[b.address])
    await x

    b.transfer_outgoing_count_limit = 0
    # Crucially, unlike with `c.submit(inc, x, workers=[a.address])`, the scheduler
    # doesn't keep track of acquire-replicas requests, so it won't proactively inform a
    # when we call remove_worker later on
    s.request_acquire_replicas(a.address, ["x"], stimulus_id="test")

    # state will flip-flop between fetch and flight every 150ms, which is the retry
    # period for busy workers.
    await wait_for_state("x", "fetch", a)
    assert b.address in a.state.busy_workers

    # Sever connection between b and s, but not between b and a.
    # If a tries fetching from b after this, b will keep responding {status: busy}.
    b.periodic_callbacks["heartbeat"].stop()
    await s.remove_worker(b.address, close=False, stimulus_id="test")

    await wait_for_state("x", "missing", a)

    assert_story(
        a.state.story("x"),
        [
            ("x", "ensure-task-exists", "released"),
            ("x", "released", "fetch", "fetch", {}),
            ("gather-dependencies", b.address, {"x"}),
            ("x", "fetch", "flight", "flight", {}),
            ("request-dep", b.address, {"x"}),
            ("busy-gather", b.address, {"x"}),
            ("x", "flight", "fetch", "fetch", {}),
            ("x", "fetch", "missing", "missing", {}),
        ],
        # There may be a round of find_missing() after this.
        # Due to timings, there also may be multiple attempts to connect from a to b.
        strict=False,
    )


def test_new_replica_while_all_workers_in_flight(ws):
    """A task is stuck in 'fetch' state because all workers that hold a replica are in
    flight. While in this state, a new replica appears on a different worker and the
    scheduler informs the waiting worker through a new acquire-replicas or
    compute-task op.

    In real life, this will typically happen when the Active Memory Manager replicates a
    key to multiple workers and some workers are much faster than others to acquire it,
    due to unrelated tasks being in flight, so 2 seconds later the AMM reiterates the
    request, passing a larger who_has.

    Test that, when this happens, the task is immediately acquired from the new worker,
    without waiting for the original replica holders to get out of flight.
    """
    ws2 = "127.0.0.1:2"
    ws3 = "127.0.0.1:3"
    instructions = ws.handle_stimulus(
        AcquireReplicasEvent(
            who_has={"x": [ws2]},
            nbytes={"x": 1},
            stimulus_id="s1",
        ),
        AcquireReplicasEvent(
            who_has={"y": [ws2]},
            nbytes={"y": 1},
            stimulus_id="s2",
        ),
        AcquireReplicasEvent(
            who_has={"y": [ws2, ws3]},
            nbytes={"y": 1},
            stimulus_id="s3",
        ),
    )
    assert instructions == [
        GatherDep(
            worker=ws2,
            to_gather={"x"},
            total_nbytes=1,
            stimulus_id="s1",
        ),
        GatherDep(
            worker=ws3,
            to_gather={"y"},
            total_nbytes=1,
            stimulus_id="s3",
        ),
    ]
    assert ws.tasks["x"].state == "flight"
    assert ws.tasks["y"].state == "flight"


@gen_cluster(client=True)
async def test_cancelled_while_in_flight(c, s, a, b):
    event = asyncio.Event()
    a.rpc = _LockedCommPool(a.rpc, write_event=event)

    x = c.submit(inc, 1, key="x", workers=[b.address])
    y = c.submit(inc, x, key="y", workers=[a.address])
    await wait_for_state("x", "flight", a)
    y.release()
    await wait_for_state("x", "cancelled", a)

    # Let the comm from b to a return the result
    event.set()
    # upon reception, x transitions cancelled->forgotten
    while a.state.tasks:
        await asyncio.sleep(0.01)


@gen_cluster(client=True, config=NO_AMM)
async def test_in_memory_while_in_flight(c, s, a, b):
    """
    1. A client scatters x to a
    2. The scheduler does not know about scattered keys until the three-way round-trip
       between client, worker, and scheduler has been completed (see Scheduler.scatter)
    3. In the middle of that handshake, a client (not necessarily the same client) calls
       ``{op: compute-task, key: x}`` on b and then
       ``{op: compute-task, key: y, who_has: {x: [b]}`` on a, which triggers a
       gather_dep call to copy x key from b to a.
    4. while x is in flight from b to a, the scatter finishes, which triggers
       update_data, which in turn transitions x from flight to memory.
    5. later on, gather_dep finishes, but the key is already in memory.
    """
    event = asyncio.Event()
    a.rpc = _LockedCommPool(a.rpc, write_event=event)

    x = c.submit(inc, 1, key="x", workers=[b.address])
    y = c.submit(inc, x, key="y", workers=[a.address])
    await wait_for_state("x", "flight", a)
    a.update_data({"x": 3})
    await wait_for_state("x", "memory", a)

    # Let the comm from b to a return the result
    event.set()
    assert await y == 4  # Data in flight from b has been discarded


@gen_cluster(client=True)
async def test_forget_data_needed(c, s, a, b):
    """
    1. A task transitions to fetch and is added to data_needed
    2. _ensure_communicating runs, but the network is saturated so the task is not
       popped from data_needed
    3. Task is forgotten
    4. Task is recreated from scratch and transitioned to fetch again
    5. BUG: at the moment of writing this test, adding to data_needed silently did
       nothing, because it still contained the forgotten task, which is a different
       TaskState instance which will be no longer updated.
    6. _ensure_communicating runs. It pops the forgotten task and discards it.
    7. We now have a task stuck in fetch state.
    """
    x = c.submit(inc, 1, key="x", workers=[a.address])
    with freeze_data_fetching(b):
        y = c.submit(inc, x, key="y", workers=[b.address])
        await wait_for_state("x", "fetch", b)
        x.release()
        y.release()
        while s.tasks or a.state.tasks or b.state.tasks:
            await asyncio.sleep(0.01)

    x = c.submit(inc, 2, key="x", workers=[a.address])
    y = c.submit(inc, x, key="y", workers=[b.address])
    assert await y == 4


@gen_cluster(client=True, nthreads=[("", 1)] * 3)
async def test_missing_handle_compute_dependency(c, s, w1, w2, w3):
    """Test that it is OK for a dependency to be in state missing if a dependent is
    asked to be computed
    """
    w3.periodic_callbacks["find-missing"].stop()

    f1 = c.submit(inc, 1, key="f1", workers=[w1.address])
    f2 = c.submit(inc, 2, key="f2", workers=[w1.address])
    await wait_for_state(f1.key, "memory", w1)

    w3.handle_stimulus(
        AcquireReplicasEvent(
            who_has={f1.key: [w2.address]}, nbytes={f1.key: 1}, stimulus_id="acquire"
        )
    )
    await wait_for_state(f1.key, "missing", w3)

    f3 = c.submit(sum, [f1, f2], key="f3", workers=[w3.address])

    await f3


@gen_cluster(client=True, nthreads=[("", 1)] * 3)
async def test_missing_to_waiting(c, s, w1, w2, w3):
    w3.periodic_callbacks["find-missing"].stop()

    f1 = c.submit(inc, 1, key="f1", workers=[w1.address], allow_other_workers=True)
    await wait_for_state(f1.key, "memory", w1)

    w3.handle_stimulus(
        AcquireReplicasEvent(
            who_has={f1.key: [w2.address]}, nbytes={f1.key: 1}, stimulus_id="acquire"
        )
    )
    await wait_for_state(f1.key, "missing", w3)

    await w2.close()
    await w1.close()

    await f1


@gen_cluster(client=True, Worker=Nanny)
async def test_task_state_instance_are_garbage_collected(c, s, a, b):
    futs = c.map(inc, range(10))
    red = c.submit(sum, futs)
    f1 = c.submit(inc, red, pure=False)
    f2 = c.submit(inc, red, pure=False)

    async def check(dask_worker):
        while dask_worker.tasks:
            await asyncio.sleep(0.01)
        with profile.lock:
            gc.collect()
        assert not TaskState._instances

    await c.gather([f2, f1])
    del futs, red, f1, f2
    await c.run(check)

    async def check(dask_scheduler):
        while dask_scheduler.tasks:
            await asyncio.sleep(0.01)
        with profile.lock:
            gc.collect()
        assert not SchedulerTaskState._instances

    await c.run_on_scheduler(check)


@gen_cluster(client=True, nthreads=[("", 1)] * 3)
async def test_fetch_to_missing_on_refresh_who_has(c, s, w1, w2, w3):
    """
    1. Two tasks, x and y, are only available on a busy worker.
       The worker sends request-refresh-who-has to the scheduler.
    2. The scheduler responds that x has become missing, while y has gained an
       additional replica
    3. The handler for RefreshWhoHasEvent empties x.who_has and recommends a transition
       to missing.
    4. Before the recommendation can be implemented, the same event invokes
       _ensure_communicating to let y to transition to flight. This in turn pops x from
       data_needed - but x has an empty who_has, which is an exceptional situation.
    5. The transition fetch->missing is executed, but x is no longer in
       data_needed - another exceptional situation.
    """
    x = c.submit(inc, 1, key="x", workers=[w1.address])
    y = c.submit(inc, 2, key="y", workers=[w1.address])
    await wait([x, y])
    w1.transfer_outgoing_count_limit = 0
    s.request_acquire_replicas(w3.address, ["x", "y"], stimulus_id="test1")

    # The tasks will now flip-flop between fetch and flight every 150ms
    # (see Worker.retry_busy_worker_later)
    await wait_for_state("x", "fetch", w3)
    await wait_for_state("y", "fetch", w3)
    assert w1.address in w3.state.busy_workers
    # w3 sent {op: request-refresh-who-has, keys: [x, y]}
    # There also may have been enough time for a refresh-who-has message to come back,
    # which reiterated what w3 already knew:
    # {op: refresh-who-has, who_has={x: [w1.address], y: [w1.address]}}

    # Let's instead simulate that, while request-refresh-who-has was in transit,
    # w2 gained a replica of y and w1 closed down.
    # When request-refresh-who-has lands, the scheduler will respond:
    # {op: refresh-who-has, who_has={x: [], y: [w2.address]}}
    w3.handle_stimulus(
        RefreshWhoHasEvent(who_has={"x": [], "y": [w2.address]}, stimulus_id="test2")
    )
    assert w3.state.tasks["x"].state == "missing"
    assert w3.state.tasks["y"].state == "flight"
    assert w3.state.tasks["y"].who_has == {w2.address}


@gen_cluster(client=True, nthreads=[("", 1)])
async def test_fetch_to_missing_on_network_failure(c, s, a):
    """
    1. Two tasks, x and y, are respectively in flight and fetch state from the same
       worker, which holds the only replica of both.
    2. gather_dep for x returns GatherDepNetworkFailureEvent
    3. The event empties has_what, x.who_has, and y.who_has.
    4. The same event invokes _ensure_communicating, which pops y from data_needed
       - but y has an empty who_has, which is an exceptional situation.
       _ensure_communicating recommends a transition to missing for x.
    5. The fetch->missing transition is executed, but y is no longer in data_needed -
       another exceptional situation.
    """
    block_get_data = asyncio.Event()

    class BlockedBreakingWorker(Worker):
        async def get_data(self, comm, *args, **kwargs):
            await block_get_data.wait()
            raise OSError("fake error")

    async with BlockedBreakingWorker(s.address) as b:
        x = c.submit(inc, 1, key="x", workers=[b.address])
        y = c.submit(inc, 2, key="y", workers=[b.address])
        await wait([x, y])
        s.request_acquire_replicas(a.address, ["x"], stimulus_id="test_x")
        await wait_for_state("x", "flight", a)
        s.request_acquire_replicas(a.address, ["y"], stimulus_id="test_y")
        await wait_for_state("y", "fetch", a)

        block_get_data.set()

        await wait_for_state("x", "missing", a)
        await wait_for_state("y", "missing", a)


@gen_cluster()
async def test_deprecated_worker_attributes(s, a, b):
    n = a.state.generation
    msg = (
        "The `Worker.generation` attribute has been moved to "
        "`Worker.state.generation`"
    )
    with pytest.warns(FutureWarning, match=msg):
        assert a.generation == n
    with pytest.warns(FutureWarning, match=msg):
        a.generation -= 1
        assert a.generation == n - 1
    assert a.state.generation == n - 1

    # Old and new names differ
    msg = (
        "The `Worker.in_flight_tasks` attribute has been moved to "
        "`Worker.state.in_flight_tasks_count`"
    )
    with pytest.warns(FutureWarning, match=msg):
        assert a.in_flight_tasks == 0

    with pytest.warns(FutureWarning, match="attribute has been removed"):
        assert a.data_needed == set()
    with pytest.warns(FutureWarning, match="attribute has been removed"):
        assert a.waiting_for_data_count == 0


@pytest.mark.parametrize("n_remote_workers", [1, 2])
@pytest.mark.parametrize(
    "nbytes,n_in_flight_per_worker",
    [
        (int(10e6), 3),
        (int(20e6), 2),
        (int(30e6), 1),
        (int(60e6), 1),
    ],
)
def test_aggregate_gather_deps(ws, nbytes, n_in_flight_per_worker, n_remote_workers):
    ws.transfer_message_bytes_limit = int(50e6)
    wss = [f"127.0.0.1:{2 + i}" for i in range(n_remote_workers)]
    who_has = {f"x{i}": [wss[i // 3]] for i in range(3 * n_remote_workers)}
    instructions = ws.handle_stimulus(
        AcquireReplicasEvent(
            who_has=who_has,
            nbytes={task: nbytes for task in who_has.keys()},
            stimulus_id="s1",
        )
    )
    assert instructions == [
        GatherDep.match(worker=remote, stimulus_id="s1") for remote in wss
    ]
    assert all(
        len(instruction.to_gather) == n_in_flight_per_worker
        for instruction in instructions
    )
    assert len(ws.in_flight_tasks) == n_in_flight_per_worker * n_remote_workers
    assert (
        ws.transfer_incoming_bytes == nbytes * n_in_flight_per_worker * n_remote_workers
    )
    assert ws.transfer_incoming_count == n_remote_workers
    assert ws.transfer_incoming_count_total == n_remote_workers


def test_gather_priority(ws):
    """Test that tasks are fetched in the following order:

    1. by task priority
    2. in case of tie, from local workers first
    3. in case of tie, from the worker with the most tasks queued
    4. in case of tie, from a random worker (which is actually deterministic).
    """
    ws.transfer_incoming_count_limit = 4

    instructions = ws.handle_stimulus(
        PauseEvent(stimulus_id="pause"),
        # Note: tasks fetched by acquire-replicas always have priority=(1, )
        AcquireReplicasEvent(
            who_has={
                # Remote + local
                "x1": ["127.0.0.2:1", "127.0.0.1:2"],
                # Remote. After getting x11 from .1, .2  will have less tasks than .3
                "x2": ["127.0.0.2:1"],
                "x3": ["127.0.0.3:1"],
                "x4": ["127.0.0.3:1"],
                # It will be a random choice between .2, .4, .5, .6, and .7
                "x5": ["127.0.0.4:1"],
                "x6": ["127.0.0.5:1"],
                "x7": ["127.0.0.6:1"],
                # This will be fetched first because it's on the same worker as y
                "x8": ["127.0.0.7:1"],
            },
            # Substantial nbytes prevents transfer_incoming_count_limit to be
            # overridden by transfer_incoming_bytes_throttle_threshold,
            # but it's less than transfer_message_bytes_limit
            nbytes={f"x{i}": 4 * 2**20 for i in range(1, 9)},
            stimulus_id="compute1",
        ),
        # A higher-priority task, even if scheduled later, is fetched first
        ComputeTaskEvent.dummy(
            key="z",
            who_has={"y": ["127.0.0.7:1"]},
            priority=(0,),
            stimulus_id="compute2",
        ),
        UnpauseEvent(stimulus_id="unpause"),
    )

    assert instructions == [
        # Highest-priority task first. Lower priority tasks from the same worker are
        # shoved into the same instruction (up to 50MB worth)
        GatherDep(
            stimulus_id="unpause",
            worker="127.0.0.7:1",
            to_gather={"y", "x8"},
            total_nbytes=1 + 4 * 2**20,
        ),
        # Followed by local workers
        GatherDep(
            stimulus_id="unpause",
            worker="127.0.0.1:2",
            to_gather={"x1"},
            total_nbytes=4 * 2**20,
        ),
        # Followed by remote workers with the most tasks
        GatherDep(
            stimulus_id="unpause",
            worker="127.0.0.3:1",
            to_gather={"x3", "x4"},
            total_nbytes=8 * 2**20,
        ),
        # Followed by other remote workers, randomly.
        # Determinism is guaranteed by a statically-seeded random number generator.
        # FIXME It would have not been deterministic if we instead of multiple keys we
        #       had used a single key with multiple workers, because sets
        #       (like TaskState.who_has) change order at every interpreter restart.
        GatherDep(
            stimulus_id="unpause",
            worker="127.0.0.4:1",
            to_gather={"x5"},
            total_nbytes=4 * 2**20,
        ),
    ]
    expected_bytes = 1 + 4 * 2**20 + 4 * 2**20 + 8 * 2**20 + 4 * 2**20
    assert ws.transfer_incoming_bytes == expected_bytes
    assert ws.transfer_incoming_count == 4
    assert ws.transfer_incoming_count_total == 4


@pytest.mark.parametrize("state", ["executing", "long-running"])
def test_task_acquires_resources(ws, state):
    ws.available_resources = {"R": 1}
    ws.total_resources = {"R": 1}

    ws.handle_stimulus(
        ComputeTaskEvent.dummy(
            key="x", resource_restrictions={"R": 1}, stimulus_id="compute"
        )
    )
    if state == "long-running":
        ws.handle_stimulus(
            SecedeEvent(key="x", compute_duration=1.0, stimulus_id="secede")
        )
    assert ws.tasks["x"].state == state
    assert ws.available_resources == {"R": 0}


@pytest.mark.parametrize(
    "done_ev_cls", [ExecuteSuccessEvent, ExecuteFailureEvent, RescheduleEvent]
)
def test_task_releases_resources(ws_with_running_task, done_ev_cls):
    ws = ws_with_running_task
    assert ws.available_resources == {"R": 0}

    ws.handle_stimulus(done_ev_cls.dummy("x", stimulus_id="success"))
    assert ws.available_resources == {"R": 1}


def test_task_with_dependencies_acquires_resources(ws):
    ws.available_resources = {"R": 1}
    ws.total_resources = {"R": 1}
    ws2 = "127.0.0.1:2"
    ws.handle_stimulus(
        ComputeTaskEvent.dummy(
            "y", who_has={"x": [ws2]}, resource_restrictions={"R": 1}, stimulus_id="s1"
        )
    )
    assert ws.tasks["x"].state == "flight"
    assert ws.tasks["y"].state == "waiting"
    assert ws.available_resources == {"R": 1}

    instructions = ws.handle_stimulus(
        GatherDepSuccessEvent(
            worker=ws2, data={"x": 123}, total_nbytes=8, stimulus_id="s2"
        )
    )
    assert instructions == [
        AddKeysMsg(keys=["x"], stimulus_id="s2"),
        Execute(key="y", stimulus_id="s2"),
    ]
    assert ws.tasks["y"].state == "executing"
    assert ws.available_resources == {"R": 0}


@pytest.mark.parametrize(
    "done_ev_cls", [ExecuteSuccessEvent, ExecuteFailureEvent, RescheduleEvent]
)
def test_resumed_task_releases_resources(ws_with_running_task, done_ev_cls):
    ws = ws_with_running_task
    assert ws.available_resources == {"R": 0}
    ws2 = "127.0.0.1:2"

    ws.handle_stimulus(FreeKeysEvent("cancel", ["x"]))
    assert ws.tasks["x"].state == "cancelled"
    assert ws.available_resources == {"R": 0}

    instructions = ws.handle_stimulus(
        ComputeTaskEvent.dummy("y", who_has={"x": [ws2]}, stimulus_id="compute")
    )
    assert not instructions
    assert ws.tasks["x"].state == "resumed"
    assert ws.available_resources == {"R": 0}

    ws.handle_stimulus(done_ev_cls.dummy("x", stimulus_id="s2"))
    assert ws.available_resources == {"R": 1}


@gen_cluster()
async def test_clean_log(s, a, b):
    """Test that brand new workers start with a clean log"""
    assert not a.state.log
    assert not a.state.stimulus_log


def test_running_task_in_all_running_tasks(ws_with_running_task):
    ws = ws_with_running_task
    ws2 = "127.0.0.1:2"
    ts = ws.tasks["x"]
    assert ts in ws.all_running_tasks

    ws.handle_stimulus(FreeKeysEvent(keys=["x"], stimulus_id="s1"))
    assert ts.state == "cancelled"
    assert ts in ws.all_running_tasks

    ws.handle_stimulus(
        ComputeTaskEvent.dummy("y", who_has={"x": [ws2]}, stimulus_id="s2")
    )
    assert ts.state == "resumed"
    assert ts in ws.all_running_tasks


@pytest.mark.parametrize(
    "done_ev_cls", [ExecuteSuccessEvent, ExecuteFailureEvent, RescheduleEvent]
)
def test_done_task_not_in_all_running_tasks(ws_with_running_task, done_ev_cls):
    ws = ws_with_running_task
    ts = ws.tasks["x"]
    assert ts in ws.all_running_tasks

    ws.handle_stimulus(done_ev_cls.dummy("x", stimulus_id="s1"))
    assert ts not in ws.all_running_tasks


@pytest.mark.parametrize(
    "done_ev_cls", [ExecuteSuccessEvent, ExecuteFailureEvent, RescheduleEvent]
)
def test_done_resumed_task_not_in_all_running_tasks(ws_with_running_task, done_ev_cls):
    ws = ws_with_running_task
    ws2 = "127.0.0.1:2"

    ws.handle_stimulus(
        FreeKeysEvent(keys=["x"], stimulus_id="s1"),
        ComputeTaskEvent.dummy("y", who_has={"x": [ws2]}, stimulus_id="s2"),
        done_ev_cls.dummy("x", stimulus_id="s3"),
    )
    ts = ws.tasks["x"]
    assert ts not in ws.all_running_tasks


@pytest.mark.xfail(reason="https://github.com/dask/distributed/issues/6705")
def test_gather_dep_failure(ws):
    """Simulate a task failing to unpickle when it reaches the destination worker after
    a flight.

    See also test_worker_memory.py::test_workerstate_fail_to_pickle_flight,
    where the task instead is gathered successfully, but fails to spill.
    """
    ws2 = "127.0.0.1:2"
    instructions = ws.handle_stimulus(
        ComputeTaskEvent.dummy("y", who_has={"x": [ws2]}, stimulus_id="s1"),
        GatherDepFailureEvent.from_exception(
            Exception(), worker=ws2, total_nbytes=1, stimulus_id="s2"
        ),
    )
    assert instructions == [
        GatherDep(worker=ws2, to_gather={"x"}, total_nbytes=1, stimulus_id="s1"),
        TaskErredMsg.match(key="x", stimulus_id="s2"),
    ]
    assert ws.tasks["x"].state == "error"
    assert ws.tasks["y"].state == "waiting"  # Not ready
    assert ws.transfer_incoming_bytes == 0
    assert ws.transfer_incoming_count == 0
    assert ws.transfer_incoming_count_total == 1


def test_transfer_incoming_metrics(ws):
    assert ws.transfer_incoming_bytes == 0
    assert ws.transfer_incoming_count == 0
    assert ws.transfer_incoming_count_total == 0

    ws2 = "127.0.0.1:2"
    ws.handle_stimulus(
        ComputeTaskEvent.dummy(
            "b", who_has={"a": [ws2]}, nbytes={"a": 7}, stimulus_id="s1"
        )
    )
    assert ws.transfer_incoming_bytes == 7
    assert ws.transfer_incoming_count == 1
    assert ws.transfer_incoming_count_total == 1

    ws.handle_stimulus(
        GatherDepSuccessEvent(
            worker=ws2, data={"a": 123}, total_nbytes=7, stimulus_id="s2"
        )
    )
    assert ws.transfer_incoming_bytes == 0
    assert ws.transfer_incoming_count == 0
    assert ws.transfer_incoming_count_total == 1

    ws.handle_stimulus(
        ComputeTaskEvent.dummy(
            "e",
            who_has={"c": [ws2], "d": [ws2]},
            nbytes={"c": 11, "d": 13},
            stimulus_id="s2",
        )
    )
    assert ws.transfer_incoming_bytes == 24
    assert ws.transfer_incoming_count == 1
    assert ws.transfer_incoming_count_total == 2

    ws.handle_stimulus(
        GatherDepSuccessEvent(
            worker=ws2, data={"c": 123, "d": 234}, total_nbytes=24, stimulus_id="s3"
        )
    )
    assert ws.transfer_incoming_bytes == 0
    assert ws.transfer_incoming_count == 0
    assert ws.transfer_incoming_count_total == 2

    ws3 = "127.0.0.1:3"
    ws.handle_stimulus(
        ComputeTaskEvent.dummy(
            "h",
            who_has={"f": [ws2], "g": [ws3]},
            nbytes={"f": 17, "g": 19},
            stimulus_id="s4",
        )
    )
    assert ws.transfer_incoming_bytes == 36
    assert ws.transfer_incoming_count == 2
    assert ws.transfer_incoming_count_total == 4

    ws.handle_stimulus(
        GatherDepSuccessEvent(
            worker=ws3, data={"g": 345}, total_nbytes=19, stimulus_id="s5"
        )
    )
    assert ws.transfer_incoming_bytes == 17
    assert ws.transfer_incoming_count == 1
    assert ws.transfer_incoming_count_total == 4

    ws.handle_stimulus(
        GatherDepSuccessEvent(
            worker=ws2, data={"g": 456}, total_nbytes=17, stimulus_id="s6"
        )
    )
    assert ws.transfer_incoming_bytes == 0
    assert ws.transfer_incoming_count == 0
    assert ws.transfer_incoming_count_total == 4


def test_throttling_does_not_affect_first_transfer(ws):
    ws.transfer_incoming_count_limit = 100
    ws.transfer_incoming_bytes_limit = 100
    ws.transfer_incoming_bytes_throttle_threshold = 1
    ws.transfer_message_bytes_limit = 100
    ws2 = "127.0.0.1:2"
    ws.handle_stimulus(
        ComputeTaskEvent.dummy(
            "c",
            who_has={"a": [ws2]},
            nbytes={"a": 200},
            stimulus_id="s1",
        )
    )
    assert ws.tasks["a"].state == "flight"


def test_message_target_does_not_affect_first_transfer_on_different_worker(ws):
    ws.transfer_incoming_count_limit = 100
    ws.transfer_incoming_bytes_limit = 600
    ws.transfer_message_bytes_limit = 100
    ws.transfer_incoming_bytes_throttle_threshold = 1
    ws2 = "127.0.0.1:2"
    ws3 = "127.0.0.1:3"
    ws.handle_stimulus(
        ComputeTaskEvent.dummy(
            "c",
            who_has={"a": [ws2], "b": [ws3]},
            nbytes={"a": 200, "b": 200},
            stimulus_id="s1",
        )
    )
    assert ws.tasks["a"].state == "flight"
    assert ws.tasks["b"].state == "flight"


def test_throttle_incoming_transfers_on_count_limit(ws):
    ws.transfer_incoming_count_limit = 1
    ws.transfer_incoming_bytes_limit = 100_000
    ws.transfer_incoming_bytes_throttle_threshold = 1
    ws2 = "127.0.0.1:2"
    ws3 = "127.0.0.1:3"
    who_has = {"a": [ws2], "b": [ws3]}
    ws.handle_stimulus(
        ComputeTaskEvent.dummy(
            "c",
            who_has=who_has,
            nbytes={"a": 100, "b": 100},
            stimulus_id="s1",
        )
    )
    tasks_by_state = defaultdict(list)
    for ts in ws.tasks.values():
        tasks_by_state[ts.state].append(ts)
    assert len(tasks_by_state["flight"]) == 1
    assert len(tasks_by_state["fetch"]) == 1
    assert ws.transfer_incoming_bytes == 100

    in_flight_task = tasks_by_state["flight"][0]
    ws.handle_stimulus(
        GatherDepSuccessEvent(
            worker=who_has[in_flight_task.key][0],
            data={in_flight_task.key: 123},
            total_nbytes=100,
            stimulus_id="s2",
        )
    )
    assert tasks_by_state["flight"][0].state == "memory"
    assert tasks_by_state["fetch"][0].state == "flight"
    assert ws.transfer_incoming_bytes == 100


def test_throttling_incoming_transfer_on_transfer_bytes_same_worker(ws):
    ws.transfer_incoming_count_limit = 100
    ws.transfer_incoming_bytes_limit = 250
    ws.transfer_incoming_bytes_throttle_threshold = 1
    ws2 = "127.0.0.1:2"
    ws.handle_stimulus(
        ComputeTaskEvent.dummy(
            "d",
            who_has={"a": [ws2], "b": [ws2], "c": [ws2]},
            nbytes={"a": 100, "b": 100, "c": 100},
            stimulus_id="s1",
        )
    )
    tasks_by_state = defaultdict(list)
    for ts in ws.tasks.values():
        tasks_by_state[ts.state].append(ts)
    assert ws.transfer_incoming_bytes == 200
    assert len(tasks_by_state["flight"]) == 2
    assert len(tasks_by_state["fetch"]) == 1

    ws.handle_stimulus(
        GatherDepSuccessEvent(
            worker=ws2,
            data={ts.key: 123 for ts in tasks_by_state["flight"]},
            total_nbytes=200,
            stimulus_id="s2",
        )
    )
    assert all(ts.state == "memory" for ts in tasks_by_state["flight"])
    assert all(ts.state == "flight" for ts in tasks_by_state["fetch"])


def test_throttling_incoming_transfer_on_transfer_bytes_different_workers(ws):
    ws.transfer_incoming_count_limit = 100
    ws.transfer_incoming_bytes_limit = 150
    ws.transfer_incoming_bytes_throttle_threshold = 1
    ws2 = "127.0.0.1:2"
    ws3 = "127.0.0.1:3"
    who_has = {"a": [ws2], "b": [ws3]}
    ws.handle_stimulus(
        ComputeTaskEvent.dummy(
            "c",
            who_has=who_has,
            nbytes={"a": 100, "b": 100},
            stimulus_id="s1",
        )
    )
    tasks_by_state = defaultdict(list)
    for ts in ws.tasks.values():
        tasks_by_state[ts.state].append(ts)
    assert ws.transfer_incoming_bytes == 100
    assert len(tasks_by_state["flight"]) == 1
    assert len(tasks_by_state["fetch"]) == 1

    in_flight_task = tasks_by_state["flight"][0]
    ws.handle_stimulus(
        GatherDepSuccessEvent(
            worker=who_has[in_flight_task.key][0],
            data={in_flight_task.key: 123},
            total_nbytes=100,
            stimulus_id="s2",
        )
    )
    assert tasks_by_state["flight"][0].state == "memory"
    assert tasks_by_state["fetch"][0].state == "flight"


def test_do_not_throttle_connections_while_below_threshold(ws):
    ws.transfer_incoming_count_limit = 1
    ws.transfer_incoming_bytes_limit = 200
    ws.transfer_incoming_bytes_throttle_threshold = 50
    ws2 = "127.0.0.1:2"
    ws3 = "127.0.0.1:3"
    ws4 = "127.0.0.1:4"
    ws.handle_stimulus(
        ComputeTaskEvent.dummy(
            "b",
            who_has={"a": [ws2]},
            nbytes={"a": 1},
            stimulus_id="s1",
        )
    )
    assert ws.tasks["a"].state == "flight"

    ws.handle_stimulus(
        ComputeTaskEvent.dummy(
            "d",
            who_has={"c": [ws3]},
            nbytes={"c": 1},
            stimulus_id="s2",
        )
    )
    assert ws.tasks["c"].state == "flight"

    ws.handle_stimulus(
        ComputeTaskEvent.dummy(
            "f",
            who_has={"e": [ws4]},
            nbytes={"e": 100},
            stimulus_id="s3",
        )
    )
    assert ws.tasks["e"].state == "flight"
    assert ws.transfer_incoming_bytes == 102


def test_throttle_on_transfer_bytes_regardless_of_threshold(ws):
    ws.transfer_incoming_count_limit = 1
    ws.transfer_incoming_bytes_limit = 100
    ws.transfer_incoming_bytes_throttle_threshold = 50
    ws2 = "127.0.0.1:2"
    ws3 = "127.0.0.1:3"
    ws.handle_stimulus(
        ComputeTaskEvent.dummy(
            "b",
            who_has={"a": [ws2]},
            nbytes={"a": 1},
            stimulus_id="s1",
        )
    )
    assert ws.tasks["a"].state == "flight"

    ws.handle_stimulus(
        ComputeTaskEvent.dummy(
            "d",
            who_has={"c": [ws3]},
            nbytes={"c": 100},
            stimulus_id="s2",
        )
    )
    assert ws.tasks["c"].state == "fetch"
    assert ws.transfer_incoming_bytes == 1


def test_worker_nbytes(ws_with_running_task):
    ws = ws_with_running_task
    ws2 = "127.0.0.1:2"
    assert ws.nbytes == 0

    # executing->memory
    ws.handle_stimulus(ExecuteSuccessEvent.dummy("x", nbytes=12, stimulus_id="s1"))
    assert ws.nbytes == 12

    # flight->memory
    ws.handle_stimulus(
        AcquireReplicasEvent(who_has={"y": [ws2]}, nbytes={"y": 13}, stimulus_id="s2")
    )
    assert ws.nbytes == 12
    ws.handle_stimulus(
        GatherDepSuccessEvent(
            worker=ws2,
            data={"y": "foo"},
            total_nbytes=13,
            stimulus_id="s3",
        )
    )
    assert ws.nbytes == 12 + 13

    # released -> memory (scatter)
    ws.handle_stimulus(
        UpdateDataEvent(data={"z": "bar"}, report=False, stimulus_id="s3")
    )
    assert ws.nbytes == 12 + 13 + sizeof("bar")

    # actors
    ws.handle_stimulus(
        ComputeTaskEvent.dummy("w", actor=True, stimulus_id="s4"),
        ExecuteSuccessEvent.dummy("w", nbytes=14, stimulus_id="s5"),
    )
    assert ws.nbytes == 12 + 13 + sizeof("bar") + 14

    # memory -> released by FreeKeysEvent
    ws.handle_stimulus(FreeKeysEvent(keys=["z"], stimulus_id="s6"))
    assert ws.nbytes == 12 + 13 + 14

    # memory -> released by RemoveReplicasEvent
    ws.handle_stimulus(RemoveReplicasEvent(keys=["x", "y", "w"], stimulus_id="s7"))
    assert ws.nbytes == 0


def test_fetch_count(ws):
    ws.transfer_incoming_count_limit = 0
    ws2 = "127.0.0.1:2"
    ws3 = "127.0.0.1:3"
    assert ws.fetch_count == 0
    # Saturate comms
    # released->fetch->flight
    ws.handle_stimulus(
        AcquireReplicasEvent(who_has={"a": [ws2]}, nbytes={"a": 1}, stimulus_id="s1"),
        AcquireReplicasEvent(
            who_has={"b": [ws2, ws3]}, nbytes={"b": 1}, stimulus_id="s2"
        ),
    )
    assert ws.tasks["b"].coming_from == ws3
    assert ws.fetch_count == 0

    # released->fetch
    # d is in two data_needed heaps
    ws.handle_stimulus(
        AcquireReplicasEvent(
            who_has={"c": [ws2], "d": [ws2, ws3]},
            nbytes={"c": 1, "d": 1},
            stimulus_id="s3",
        )
    )
    assert ws.fetch_count == 2

    # fetch->released
    ws.handle_stimulus(FreeKeysEvent(keys={"c", "d"}, stimulus_id="s4"))
    assert ws.fetch_count == 0

    # flight->missing
    ws.handle_stimulus(
        GatherDepSuccessEvent(worker=ws2, data={}, total_nbytes=0, stimulus_id="s5")
    )
    assert ws.tasks["a"].state == "missing"
    print(ws.tasks)
    assert ws.fetch_count == 0
    assert len(ws.missing_dep_flight) == 1

    # flight->fetch
    ws.handle_stimulus(
        ComputeTaskEvent.dummy(
            "clog", who_has={"clog_dep": [ws2]}, priority=(-1,), stimulus_id="s6"
        ),
        GatherDepSuccessEvent(worker=ws3, data={}, total_nbytes=0, stimulus_id="s7"),
    )
    assert ws.tasks["b"].state == "fetch"
    assert ws.fetch_count == 1
    assert len(ws.missing_dep_flight) == 1


def test_task_counts(ws):
    assert ws.task_counts == {
        "constrained": 0,
        "executing": 0,
        "fetch": 0,
        "flight": 0,
        "long-running": 0,
        "memory": 0,
        "missing": 0,
        "other": 0,
        "ready": 0,
        "waiting": 0,
    }


def test_task_counts_with_actors(ws):
    ws.handle_stimulus(ComputeTaskEvent.dummy("x", actor=True, stimulus_id="s1"))
    assert ws.actors == {"x": None}
    assert ws.task_counts == {
        "constrained": 0,
        "executing": 1,
        "fetch": 0,
        "flight": 0,
        "long-running": 0,
        "memory": 0,
        "missing": 0,
        "other": 0,
        "ready": 0,
        "waiting": 0,
    }
    ws.handle_stimulus(ExecuteSuccessEvent.dummy("x", value=123, stimulus_id="s2"))
    assert ws.actors == {"x": 123}
    assert ws.task_counts == {
        "constrained": 0,
        "executing": 0,
        "fetch": 0,
        "flight": 0,
        "long-running": 0,
        "memory": 1,
        "missing": 0,
        "other": 0,
        "ready": 0,
        "waiting": 0,
    }