Source code distributed/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
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
1723
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750
1751
1752
1753
1754
1755
1756
1757
1758
1759
1760
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770
1771
1772
1773
1774
1775
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803
1804
1805
1806
1807
1808
1809
1810
1811
1812
1813
1814
1815
1816
1817
1818
1819
1820
1821
1822
1823
1824
1825
1826
1827
1828
1829
1830
1831
1832
1833
1834
1835
1836
1837
1838
1839
1840
1841
1842
1843
1844
1845
1846
1847
1848
1849
1850
1851
1852
1853
1854
1855
1856
1857
1858
1859
1860
1861
1862
1863
1864
1865
1866
1867
1868
1869
1870
1871
1872
1873
1874
1875
1876
1877
1878
1879
1880
1881
1882
1883
1884
1885
1886
1887
1888
1889
1890
1891
1892
1893
1894
1895
1896
1897
1898
1899
1900
1901
1902
1903
1904
1905
1906
1907
1908
1909
1910
1911
1912
1913
1914
1915
1916
1917
1918
1919
1920
1921
1922
1923
1924
1925
1926
1927
1928
1929
1930
1931
1932
1933
1934
1935
1936
1937
1938
1939
1940
1941
1942
1943
1944
1945
1946
1947
1948
1949
1950
1951
1952
1953
1954
1955
1956
1957
1958
1959
1960
1961
1962
1963
1964
1965
1966
1967
1968
1969
1970
1971
1972
1973
1974
1975
1976
1977
1978
1979
1980
1981
1982
1983
1984
1985
1986
1987
1988
1989
1990
1991
1992
1993
1994
1995
1996
1997
1998
1999
2000
2001
2002
2003
2004
2005
2006
2007
2008
2009
2010
2011
2012
2013
2014
2015
2016
2017
2018
2019
2020
2021
2022
2023
2024
2025
2026
2027
2028
2029
2030
2031
2032
2033
2034
2035
2036
2037
2038
2039
2040
2041
2042
2043
2044
2045
2046
2047
2048
2049
2050
2051
2052
2053
2054
2055
2056
2057
2058
2059
2060
2061
2062
2063
2064
2065
2066
2067
2068
2069
2070
2071
2072
2073
2074
2075
2076
2077
2078
2079
2080
2081
2082
2083
2084
2085
2086
2087
2088
2089
2090
2091
2092
2093
2094
2095
2096
2097
2098
2099
2100
2101
2102
2103
2104
2105
2106
2107
2108
2109
2110
2111
2112
2113
2114
2115
2116
2117
2118
2119
2120
2121
2122
2123
2124
2125
2126
2127
2128
2129
2130
2131
2132
2133
2134
2135
2136
2137
2138
2139
2140
2141
2142
2143
2144
2145
2146
2147
2148
2149
2150
2151
2152
2153
2154
2155
2156
2157
2158
2159
2160
2161
2162
2163
2164
2165
2166
2167
2168
2169
2170
2171
2172
2173
2174
2175
2176
2177
2178
2179
2180
2181
2182
2183
2184
2185
2186
2187
2188
2189
2190
2191
2192
2193
2194
2195
2196
2197
2198
2199
2200
2201
2202
2203
2204
2205
2206
2207
2208
2209
2210
2211
2212
2213
2214
2215
2216
2217
2218
2219
2220
2221
2222
2223
2224
2225
2226
2227
2228
2229
2230
2231
2232
2233
2234
2235
2236
2237
2238
2239
2240
2241
2242
2243
2244
2245
2246
2247
2248
2249
2250
2251
2252
2253
2254
2255
2256
2257
2258
2259
2260
2261
2262
2263
2264
2265
2266
2267
2268
2269
2270
2271
2272
2273
2274
2275
2276
2277
2278
2279
2280
2281
2282
2283
2284
2285
2286
2287
2288
2289
2290
2291
2292
2293
2294
2295
2296
2297
2298
2299
2300
2301
2302
2303
2304
2305
2306
2307
2308
2309
2310
2311
2312
2313
2314
2315
2316
2317
2318
2319
2320
2321
2322
2323
2324
2325
2326
2327
2328
2329
2330
2331
2332
2333
2334
2335
2336
2337
2338
2339
2340
2341
2342
2343
2344
2345
2346
2347
2348
2349
2350
2351
2352
2353
2354
2355
2356
2357
2358
2359
2360
2361
2362
2363
2364
2365
2366
2367
2368
2369
2370
2371
2372
2373
2374
2375
2376
2377
2378
2379
2380
2381
2382
2383
2384
2385
2386
2387
2388
2389
2390
2391
2392
2393
2394
2395
2396
2397
2398
2399
2400
2401
2402
2403
2404
2405
2406
2407
2408
2409
2410
2411
2412
2413
2414
2415
2416
2417
2418
2419
2420
2421
2422
2423
2424
2425
2426
2427
2428
2429
2430
2431
2432
2433
2434
2435
2436
2437
2438
2439
2440
2441
2442
2443
2444
2445
2446
2447
2448
2449
2450
2451
2452
2453
2454
2455
2456
2457
2458
2459
2460
2461
2462
2463
2464
2465
2466
2467
2468
2469
2470
2471
2472
2473
2474
2475
2476
2477
2478
2479
2480
2481
2482
2483
2484
2485
2486
2487
2488
2489
2490
2491
2492
2493
2494
2495
2496
2497
2498
2499
2500
2501
2502
2503
2504
2505
2506
2507
2508
2509
2510
2511
2512
2513
2514
2515
2516
2517
2518
2519
2520
2521
2522
2523
2524
2525
2526
2527
2528
2529
2530
2531
2532
2533
2534
2535
2536
2537
2538
2539
2540
2541
2542
2543
2544
2545
2546
2547
2548
2549
2550
2551
2552
2553
2554
2555
2556
2557
2558
2559
2560
2561
2562
2563
2564
2565
2566
2567
2568
2569
2570
2571
2572
2573
2574
2575
2576
2577
2578
2579
2580
2581
2582
2583
2584
2585
2586
2587
2588
2589
2590
2591
2592
2593
2594
2595
2596
2597
2598
2599
2600
2601
2602
2603
2604
2605
2606
2607
2608
2609
2610
2611
2612
2613
2614
2615
2616
2617
2618
2619
2620
2621
2622
2623
2624
2625
2626
2627
2628
2629
2630
2631
2632
2633
2634
2635
2636
2637
2638
2639
2640
2641
2642
2643
2644
2645
2646
2647
2648
2649
2650
2651
2652
2653
2654
2655
2656
2657
2658
2659
2660
2661
2662
2663
2664
2665
2666
2667
2668
2669
2670
2671
2672
2673
2674
2675
2676
2677
2678
2679
2680
2681
2682
2683
2684
2685
2686
2687
2688
2689
2690
2691
2692
2693
2694
2695
2696
2697
2698
2699
2700
2701
2702
2703
2704
2705
2706
2707
2708
2709
2710
2711
2712
2713
2714
2715
2716
2717
2718
2719
2720
2721
2722
2723
2724
2725
2726
2727
2728
2729
2730
2731
2732
2733
2734
2735
2736
2737
2738
2739
2740
2741
2742
2743
2744
2745
2746
2747
2748
2749
2750
2751
2752
2753
2754
2755
2756
2757
2758
2759
2760
2761
2762
2763
2764
2765
2766
2767
2768
2769
2770
2771
2772
2773
2774
2775
2776
2777
2778
2779
2780
2781
2782
2783
2784
2785
2786
2787
2788
2789
2790
2791
2792
2793
2794
2795
2796
2797
2798
2799
2800
2801
2802
2803
2804
2805
2806
2807
2808
2809
2810
2811
2812
2813
2814
2815
2816
2817
2818
2819
2820
2821
2822
2823
2824
2825
2826
2827
2828
2829
2830
2831
2832
2833
2834
2835
2836
2837
2838
2839
2840
2841
2842
2843
2844
2845
2846
2847
2848
2849
2850
2851
2852
2853
2854
2855
2856
2857
2858
2859
2860
2861
2862
2863
2864
2865
2866
2867
2868
2869
2870
2871
2872
2873
2874
2875
2876
2877
2878
2879
2880
2881
2882
2883
2884
2885
2886
2887
2888
2889
2890
2891
2892
2893
2894
2895
2896
2897
2898
2899
2900
2901
2902
2903
2904
2905
2906
2907
2908
2909
2910
2911
2912
2913
2914
2915
2916
2917
2918
2919
2920
2921
2922
2923
2924
2925
2926
2927
2928
2929
2930
2931
2932
2933
2934
2935
2936
2937
2938
2939
2940
2941
2942
2943
2944
2945
2946
2947
2948
2949
2950
2951
2952
2953
2954
2955
2956
2957
2958
2959
2960
2961
2962
2963
2964
2965
2966
2967
2968
2969
2970
2971
2972
2973
2974
2975
2976
2977
2978
2979
2980
2981
2982
2983
2984
2985
2986
2987
2988
2989
2990
2991
2992
2993
2994
2995
2996
2997
2998
2999
3000
3001
3002
3003
3004
3005
3006
3007
3008
3009
3010
3011
3012
3013
3014
3015
3016
3017
3018
3019
3020
3021
3022
3023
3024
3025
3026
3027
3028
3029
3030
3031
3032
3033
3034
3035
3036
3037
3038
3039
3040
3041
3042
3043
3044
3045
3046
3047
3048
3049
3050
3051
3052
3053
3054
3055
3056
3057
3058
3059
3060
3061
3062
3063
3064
3065
3066
3067
3068
3069
3070
3071
3072
3073
3074
3075
3076
3077
3078
3079
3080
3081
3082
3083
3084
3085
3086
3087
3088
3089
3090
3091
3092
3093
3094
3095
3096
3097
3098
3099
3100
3101
3102
3103
3104
3105
3106
3107
3108
3109
3110
3111
3112
3113
3114
3115
3116
3117
3118
3119
3120
3121
3122
3123
3124
3125
3126
3127
3128
3129
3130
3131
3132
3133
3134
3135
3136
3137
3138
3139
3140
3141
3142
3143
3144
3145
3146
3147
3148
3149
3150
3151
3152
3153
3154
3155
3156
3157
3158
3159
3160
3161
3162
3163
3164
3165
3166
3167
3168
3169
3170
3171
3172
3173
3174
3175
3176
3177
3178
3179
3180
3181
3182
3183
3184
3185
3186
3187
3188
3189
3190
3191
3192
3193
3194
3195
3196
3197
3198
3199
3200
3201
3202
3203
3204
3205
3206
3207
3208
3209
3210
3211
3212
3213
3214
3215
3216
3217
3218
3219
3220
3221
3222
3223
3224
3225
3226
3227
3228
3229
3230
3231
3232
3233
3234
3235
3236
3237
3238
3239
3240
3241
3242
3243
3244
3245
3246
3247
3248
3249
3250
3251
3252
3253
3254
3255
3256
3257
3258
3259
3260
3261
3262
3263
3264
3265
3266
3267
3268
3269
3270
3271
3272
3273
3274
3275
3276
3277
3278
3279
3280
3281
3282
3283
3284
3285
3286
3287
3288
3289
3290
3291
3292
3293
3294
3295
3296
3297
3298
3299
3300
3301
3302
3303
3304
3305
3306
3307
3308
3309
3310
3311
3312
3313
3314
3315
3316
3317
3318
3319
3320
3321
3322
3323
3324
3325
3326
3327
3328
3329
3330
3331
3332
3333
3334
3335
3336
3337
3338
3339
3340
3341
3342
3343
3344
3345
3346
3347
3348
3349
3350
3351
3352
3353
3354
3355
3356
3357
3358
3359
3360
3361
3362
3363
3364
3365
3366
3367
3368
3369
3370
3371
3372
3373
3374
3375
3376
3377
3378
3379
3380
3381
3382
3383
3384
3385
3386
3387
3388
3389
3390
3391
3392
3393
3394
3395
3396
3397
3398
3399
3400
3401
3402
3403
3404
3405
3406
3407
3408
3409
3410
3411
3412
3413
3414
3415
3416
3417
3418
3419
3420
3421
3422
3423
3424
3425
3426
3427
3428
3429
3430
3431
3432
3433
3434
3435
3436
3437
3438
3439
3440
3441
3442
3443
3444
3445
3446
3447
3448
3449
3450
3451
3452
3453
3454
3455
3456
3457
3458
3459
3460
3461
3462
3463
3464
3465
3466
3467
3468
3469
3470
3471
3472
3473
3474
3475
3476
3477
3478
3479
3480
3481
3482
3483
3484
3485
3486
3487
3488
3489
3490
3491
3492
3493
3494
3495
3496
3497
3498
3499
3500
3501
3502
3503
3504
3505
3506
3507
3508
3509
3510
3511
3512
3513
3514
3515
3516
3517
3518
3519
3520
3521
3522
3523
3524
3525
3526
3527
3528
3529
3530
3531
3532
3533
3534
3535
3536
3537
3538
3539
3540
3541
3542
3543
3544
3545
3546
3547
3548
3549
3550
3551
3552
3553
3554
3555
3556
3557
3558
3559
3560
3561
3562
3563
3564
3565
3566
3567
3568
3569
3570
3571
3572
3573
3574
3575
3576
3577
3578
3579
3580
3581
3582
3583
3584
3585
3586
3587
3588
3589
3590
3591
3592
3593
3594
3595
3596
3597
3598
3599
3600
3601
3602
3603
3604
3605
3606
3607
3608
3609
3610
3611
3612
3613
3614
3615
3616
3617
3618
3619
3620
3621
3622
3623
3624
3625
3626
3627
3628
3629
3630
3631
3632
3633
3634
3635
3636
3637
3638
3639
3640
3641
3642
3643
3644
3645
3646
3647
3648
3649
3650
3651
3652
3653
3654
3655
3656
3657
3658
3659
3660
3661
3662
3663
3664
3665
3666
3667
3668
3669
3670
3671
3672
3673
3674
3675
3676
3677
3678
3679
3680
3681
3682
3683
3684
3685
3686
3687
3688
3689
3690
3691
3692
3693
3694
3695
3696
3697
3698
3699
3700
3701
3702
3703
3704
3705
3706
3707
3708
3709
3710
3711
3712
3713
3714
3715
3716
3717
3718
3719
3720
3721
3722
3723
3724
3725
3726
3727
3728
3729
3730
3731
3732
3733
3734
3735
3736
3737
3738
3739
3740
3741
3742
3743
3744
3745
3746
3747
3748
3749
3750
3751
3752
3753
3754
3755
3756
3757
3758
3759
3760
3761
3762
3763
3764
3765
3766
3767
3768
3769
3770
3771
3772
3773
3774
3775
3776
3777
3778
3779
3780
3781
3782
3783
3784
3785
3786
3787
3788
3789
3790
3791
3792
3793
3794
3795
3796
3797
3798
3799
3800
3801
3802
3803
3804
3805
3806
3807
3808
3809
3810
3811
3812
3813
3814
3815
3816
3817
3818
3819
3820
3821
3822
3823
3824
3825
3826
3827
3828
3829
3830
3831
3832
3833
3834
3835
3836
3837
3838
3839
3840
3841
3842
3843
3844
3845
3846
3847
3848
3849
3850
3851
3852
3853
3854
3855
3856
3857
3858
3859
3860
3861
3862
3863
3864
3865
3866
3867
3868
3869
3870
3871
3872
3873
3874
3875
3876
3877
3878
3879
3880
3881
3882
3883
3884
3885
3886
3887
3888
3889
3890
3891
3892
3893
3894
3895
3896
3897
3898
3899
3900
3901
3902
3903
3904
3905
3906
3907
3908
3909
3910
3911
3912
3913
3914
3915
3916
3917
3918
3919
3920
3921
3922
3923
3924
3925
3926
3927
3928
3929
3930
3931
3932
3933
3934
3935
3936
3937
3938
3939
3940
3941
3942
3943
3944
3945
3946
3947
3948
3949
3950
3951
3952
3953
3954
3955
3956
3957
3958
3959
3960
3961
3962
3963
3964
3965
3966
3967
3968
3969
3970
3971
3972
3973
3974
3975
3976
3977
3978
3979
3980
3981
3982
3983
3984
3985
3986
3987
3988
3989
3990
3991
3992
3993
3994
3995
3996
3997
3998
3999
4000
4001
4002
4003
4004
4005
4006
4007
4008
4009
4010
4011
4012
4013
4014
4015
4016
4017
4018
4019
4020
4021
4022
4023
4024
4025
4026
4027
4028
4029
4030
4031
4032
4033
4034
4035
4036
4037
4038
4039
4040
4041
4042
4043
4044
4045
4046
4047
4048
4049
4050
4051
4052
4053
4054
4055
4056
4057
4058
4059
4060
4061
4062
4063
4064
4065
4066
4067
4068
4069
4070
4071
4072
4073
4074
4075
4076
4077
4078
4079
4080
4081
4082
4083
4084
4085
4086
4087
4088
4089
4090
4091
4092
4093
4094
4095
4096
4097
4098
4099
4100
4101
4102
4103
4104
4105
4106
4107
4108
4109
4110
4111
4112
4113
4114
4115
4116
4117
4118
4119
4120
4121
4122
4123
4124
4125
4126
4127
4128
4129
4130
4131
4132
4133
4134
4135
4136
4137
4138
4139
4140
4141
4142
4143
4144
4145
4146
4147
4148
4149
4150
4151
4152
4153
4154
4155
4156
4157
4158
4159
4160
4161
4162
4163
4164
4165
4166
4167
4168
4169
4170
4171
4172
4173
4174
4175
4176
4177
4178
4179
4180
4181
4182
4183
4184
4185
4186
4187
4188
4189
4190
4191
4192
4193
4194
4195
4196
4197
4198
4199
4200
4201
4202
4203
4204
4205
4206
4207
4208
4209
4210
4211
4212
4213
4214
4215
4216
4217
4218
4219
4220
4221
4222
4223
4224
4225
4226
4227
4228
4229
4230
4231
4232
4233
4234
4235
4236
4237
4238
4239
4240
4241
4242
4243
4244
4245
4246
4247
4248
4249
4250
4251
4252
4253
4254
4255
4256
4257
4258
4259
4260
4261
4262
4263
4264
4265
4266
4267
4268
4269
4270
4271
4272
4273
4274
4275
4276
4277
4278
4279
4280
4281
4282
4283
4284
4285
4286
4287
4288
4289
4290
4291
4292
4293
4294
4295
4296
4297
4298
4299
4300
4301
4302
4303
4304
4305
4306
4307
4308
4309
4310
4311
4312
4313
4314
4315
4316
4317
4318
4319
4320
4321
4322
4323
4324
4325
4326
4327
4328
4329
4330
4331
4332
4333
4334
4335
4336
4337
4338
4339
4340
4341
4342
4343
4344
4345
4346
4347
4348
4349
4350
4351
4352
4353
4354
4355
4356
4357
4358
4359
4360
4361
4362
4363
4364
4365
4366
4367
4368
4369
4370
4371
4372
4373
4374
4375
4376
4377
4378
4379
4380
4381
4382
4383
4384
4385
4386
4387
4388
4389
4390
4391
4392
4393
4394
4395
4396
4397
4398
4399
4400
4401
4402
4403
4404
4405
4406
4407
4408
4409
4410
4411
4412
4413
4414
4415
4416
4417
4418
4419
4420
4421
4422
4423
4424
4425
4426
4427
4428
4429
4430
4431
4432
4433
4434
4435
4436
4437
4438
4439
4440
4441
4442
4443
4444
4445
4446
4447
4448
4449
4450
4451
4452
4453
4454
4455
4456
4457
4458
4459
4460
4461
4462
4463
4464
4465
4466
4467
4468
4469
4470
4471
4472
4473
4474
4475
4476
4477
4478
4479
4480
4481
4482
4483
4484
4485
4486
4487
4488
4489
4490
4491
4492
4493
4494
4495
4496
4497
4498
4499
4500
4501
4502
4503
4504
4505
4506
4507
4508
4509
4510
4511
4512
4513
4514
4515
4516
4517
4518
4519
4520
4521
4522
4523
4524
4525
4526
4527
4528
4529
4530
4531
4532
4533
4534
4535
4536
4537
4538
4539
4540
4541
4542
4543
4544
4545
4546
4547
4548
4549
4550
4551
4552
4553
4554
4555
4556
4557
4558
4559
4560
4561
4562
4563
4564
4565
4566
4567
4568
4569
4570
4571
4572
4573
4574
4575
4576
4577
4578
4579
4580
4581
4582
4583
4584
4585
4586
4587
4588
4589
4590
4591
4592
4593
4594
4595
4596
4597
4598
4599
4600
4601
4602
4603
4604
4605
4606
4607
4608
4609
4610
4611
4612
4613
4614
4615
4616
4617
4618
4619
4620
4621
4622
4623
4624
4625
4626
4627
4628
4629
4630
4631
4632
4633
4634
4635
4636
4637
4638
4639
4640
4641
4642
4643
4644
4645
4646
4647
4648
4649
4650
4651
4652
4653
4654
4655
4656
4657
4658
4659
4660
4661
4662
4663
4664
4665
4666
4667
4668
4669
4670
4671
4672
4673
4674
4675
4676
4677
4678
4679
4680
4681
4682
4683
4684
4685
4686
4687
4688
4689
4690
4691
4692
4693
4694
4695
4696
4697
4698
4699
4700
4701
4702
4703
4704
4705
4706
4707
4708
4709
4710
4711
4712
4713
4714
4715
4716
4717
4718
4719
4720
4721
4722
4723
4724
4725
4726
4727
4728
4729
4730
4731
4732
4733
4734
4735
4736
4737
4738
4739
4740
4741
4742
4743
4744
4745
4746
4747
4748
4749
4750
4751
4752
4753
4754
4755
4756
4757
4758
4759
4760
4761
4762
4763
4764
4765
4766
4767
4768
4769
4770
4771
4772
4773
4774
4775
4776
4777
4778
4779
4780
4781
4782
4783
4784
4785
4786
4787
4788
4789
4790
4791
4792
4793
4794
4795
4796
4797
4798
4799
4800
4801
4802
4803
4804
4805
4806
4807
4808
4809
4810
4811
4812
4813
4814
4815
4816
4817
4818
4819
4820
4821
4822
4823
4824
4825
4826
4827
4828
4829
4830
4831
4832
4833
4834
4835
4836
4837
4838
4839
4840
4841
4842
4843
4844
4845
4846
4847
4848
4849
4850
4851
4852
4853
4854
4855
4856
4857
4858
4859
4860
4861
4862
4863
4864
4865
4866
4867
4868
4869
4870
4871
4872
4873
4874
4875
4876
4877
4878
4879
4880
4881
4882
4883
4884
4885
4886
4887
4888
4889
4890
4891
4892
4893
4894
4895
4896
4897
4898
4899
4900
4901
4902
4903
4904
4905
4906
4907
4908
4909
4910
4911
4912
4913
4914
4915
4916
4917
4918
4919
4920
4921
4922
4923
4924
4925
4926
4927
4928
4929
4930
4931
4932
4933
4934
4935
4936
4937
4938
4939
4940
4941
4942
4943
4944
4945
4946
4947
4948
4949
4950
4951
4952
4953
4954
4955
4956
4957
4958
4959
4960
4961
4962
4963
4964
4965
4966
4967
4968
4969
4970
4971
4972
4973
4974
4975
4976
4977
4978
4979
4980
4981
4982
4983
4984
4985
4986
4987
4988
4989
4990
4991
4992
4993
4994
4995
4996
4997
4998
4999
5000
5001
5002
5003
5004
5005
5006
5007
5008
5009
5010
5011
5012
5013
5014
5015
5016
5017
5018
5019
5020
5021
5022
5023
5024
5025
5026
5027
5028
5029
5030
5031
5032
5033
5034
5035
5036
5037
5038
5039
5040
5041
5042
5043
5044
5045
5046
5047
5048
5049
5050
5051
5052
5053
5054
5055
5056
5057
5058
5059
5060
5061
5062
5063
5064
5065
5066
5067
5068
5069
5070
5071
5072
5073
5074
5075
5076
5077
5078
5079
5080
5081
5082
5083
5084
5085
5086
5087
5088
5089
5090
5091
5092
5093
5094
5095
5096
5097
5098
5099
5100
5101
5102
5103
5104
5105
5106
5107
5108
5109
5110
5111
5112
5113
5114
5115
5116
5117
5118
5119
5120
5121
5122
5123
5124
5125
5126
5127
5128
5129
5130
5131
5132
5133
5134
5135
5136
5137
5138
5139
5140
5141
5142
5143
5144
5145
5146
5147
5148
5149
5150
5151
5152
5153
5154
5155
5156
5157
5158
5159
5160
5161
5162
5163
5164
5165
5166
5167
5168
5169
5170
5171
5172
5173
5174
5175
5176
5177
5178
5179
5180
5181
5182
5183
5184
5185
5186
5187
5188
5189
5190
5191
5192
5193
5194
5195
5196
5197
5198
5199
5200
5201
5202
5203
5204
5205
5206
5207
5208
5209
5210
5211
5212
5213
5214
5215
5216
5217
5218
5219
5220
5221
5222
5223
5224
5225
5226
5227
5228
5229
5230
5231
5232
5233
5234
5235
5236
5237
5238
5239
5240
5241
5242
5243
5244
5245
5246
5247
5248
5249
5250
5251
5252
5253
5254
5255
5256
5257
5258
5259
5260
5261
5262
5263
5264
5265
5266
5267
5268
5269
5270
5271
5272
5273
5274
5275
5276
5277
5278
5279
5280
5281
5282
5283
5284
5285
5286
5287
5288
5289
5290
5291
5292
5293
5294
5295
5296
5297
5298
5299
5300
5301
5302
5303
5304
5305
5306
5307
5308
5309
5310
5311
5312
5313
5314
5315
5316
5317
5318
5319
5320
5321
5322
5323
5324
5325
5326
5327
5328
5329
5330
5331
5332
5333
5334
5335
5336
5337
5338
5339
5340
5341
5342
5343
5344
5345
5346
5347
5348
5349
5350
5351
5352
5353
5354
5355
5356
5357
5358
5359
5360
5361
5362
5363
5364
5365
5366
5367
5368
5369
5370
5371
5372
5373
5374
5375
5376
5377
5378
5379
5380
5381
5382
5383
5384
5385
5386
5387
5388
5389
5390
5391
5392
5393
5394
5395
5396
5397
5398
5399
5400
5401
5402
5403
5404
5405
5406
5407
5408
5409
5410
5411
5412
5413
5414
5415
5416
5417
5418
5419
5420
5421
5422
5423
5424
5425
5426
5427
5428
5429
5430
5431
5432
5433
5434
5435
5436
5437
5438
5439
5440
5441
5442
5443
5444
5445
5446
5447
5448
5449
5450
5451
5452
5453
5454
5455
5456
5457
5458
5459
5460
5461
5462
5463
5464
5465
5466
5467
5468
5469
5470
5471
5472
5473
5474
5475
5476
5477
5478
5479
5480
5481
5482
5483
5484
5485
5486
5487
5488
5489
5490
5491
5492
5493
5494
5495
5496
5497
5498
5499
5500
5501
5502
5503
5504
5505
5506
5507
5508
5509
5510
5511
5512
5513
5514
5515
5516
5517
5518
5519
5520
5521
5522
5523
5524
5525
5526
5527
5528
5529
5530
5531
5532
5533
5534
5535
5536
5537
5538
5539
5540
5541
5542
5543
5544
5545
5546
5547
5548
5549
5550
5551
5552
5553
5554
5555
5556
5557
5558
5559
5560
5561
5562
5563
5564
5565
5566
5567
5568
5569
5570
5571
5572
5573
5574
5575
5576
5577
5578
5579
5580
5581
5582
5583
5584
5585
5586
5587
5588
5589
5590
5591
5592
5593
5594
5595
5596
5597
5598
5599
5600
5601
5602
5603
5604
5605
5606
5607
5608
5609
5610
5611
5612
5613
5614
5615
5616
5617
5618
5619
5620
5621
5622
5623
5624
5625
5626
5627
5628
5629
5630
5631
5632
5633
5634
5635
5636
5637
5638
5639
5640
5641
5642
5643
5644
5645
5646
5647
5648
5649
5650
5651
5652
5653
5654
5655
5656
5657
5658
5659
5660
5661
5662
5663
5664
5665
5666
5667
5668
5669
5670
5671
5672
5673
5674
5675
5676
5677
5678
5679
5680
5681
5682
5683
5684
5685
5686
5687
5688
5689
5690
5691
5692
5693
5694
5695
5696
5697
5698
5699
5700
5701
5702
5703
5704
5705
5706
5707
5708
5709
5710
5711
5712
5713
5714
5715
5716
5717
5718
5719
5720
5721
5722
5723
5724
5725
5726
5727
5728
5729
5730
5731
5732
5733
import asyncio
from collections import defaultdict, deque
from collections.abc import Mapping, Set
from contextlib import suppress
from datetime import timedelta
from functools import partial
import inspect
import itertools
import json
import logging
import math
from numbers import Number
import operator
import os
import sys
import random
import warnings
import weakref
import psutil
import sortedcontainers

from tlz import (
    frequencies,
    merge,
    pluck,
    merge_sorted,
    first,
    merge_with,
    valmap,
    second,
    compose,
    groupby,
    concat,
)
from tornado.ioloop import IOLoop, PeriodicCallback

import dask

from . import profile
from .batched import BatchedSend
from .comm import (
    normalize_address,
    resolve_address,
    get_address_host,
    unparse_host_port,
)
from .comm.addressing import addresses_from_user_args
from .core import rpc, send_recv, clean_exception, CommClosedError, Status
from .diagnostics.plugin import SchedulerPlugin

from .http import get_handlers
from .metrics import time
from .node import ServerNode
from . import preloading
from .proctitle import setproctitle
from .security import Security
from .utils import (
    All,
    get_fileno_limit,
    log_errors,
    key_split,
    validate_key,
    no_default,
    parse_timedelta,
    parse_bytes,
    shutting_down,
    key_split_group,
    empty_context,
    tmpfile,
    format_bytes,
    format_time,
    TimeoutError,
)
from .utils_comm import scatter_to_workers, gather_from_workers, retry_operation
from .utils_perf import enable_gc_diagnosis, disable_gc_diagnosis
from . import versions as version_module

from .publish import PublishExtension
from .queues import QueueExtension
from .semaphore import SemaphoreExtension
from .recreate_exceptions import ReplayExceptionScheduler
from .lock import LockExtension
from .event import EventExtension
from .pubsub import PubSubSchedulerExtension
from .stealing import WorkStealing
from .variable import VariableExtension
from .protocol.highlevelgraph import highlevelgraph_unpack

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 = parse_bytes(
    dask.config.get("distributed.scheduler.default-data-size")
)

DEFAULT_EXTENSIONS = [
    LockExtension,
    PublishExtension,
    ReplayExceptionScheduler,
    QueueExtension,
    VariableExtension,
    PubSubSchedulerExtension,
    SemaphoreExtension,
    EventExtension,
]

ALL_TASK_STATES = {"released", "waiting", "no-worker", "processing", "erred", "memory"}


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.

    """

    __slots__ = ("client_key", "wants_what", "last_seen", "versions")

    def __init__(self, client, versions=None):
        self.client_key = client
        self.wants_what = set()
        self.last_seen = time()
        self.versions = versions or {}

    def __repr__(self):
        return "<Client %r>" % (self.client_key,)

    def __str__(self):
        return self.client_key


class WorkerState:
    """
    A simple object holding information about a worker.

    .. attribute:: address

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

       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:: has_what: {TaskState}

       The set 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: Number

       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: str

       The current status of the worker, either ``'running'`` or ``'closed'``

    .. attribute:: nanny: str

       Address of the associated Nanny, if present

    .. attribute:: last_seen: Number

       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?

    __slots__ = (
        "actors",
        "address",
        "bandwidth",
        "extra",
        "has_what",
        "last_seen",
        "local_directory",
        "memory_limit",
        "metrics",
        "name",
        "nanny",
        "nbytes",
        "nthreads",
        "occupancy",
        "pid",
        "processing",
        "resources",
        "services",
        "_status",
        "time_delay",
        "used_resources",
        "versions",
    )

    def __init__(
        self,
        address=None,
        pid=0,
        name=None,
        nthreads=0,
        memory_limit=0,
        local_directory=None,
        services=None,
        versions=None,
        nanny=None,
        extra=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.running
        self.nbytes = 0
        self.occupancy = 0
        self.metrics = {}
        self.last_seen = 0
        self.time_delay = 0
        self.bandwidth = parse_bytes(dask.config.get("distributed.scheduler.bandwidth"))

        self.actors = set()
        self.has_what = set()
        self.processing = {}
        self.resources = {}
        self.used_resources = {}

        self.extra = extra or {}

    def __hash__(self):
        return hash(self.address)

    def __eq__(self, other):
        return type(self) == type(other) and self.address == other.address

    @property
    def status(self):
        return self._status

    @status.setter
    def status(self, new_status):
        if isinstance(new_status, Status):
            self._status = new_status
        elif isinstance(new_status, str) or new_status is None:
            corresponding_enum_variants = [s for s in Status if s.value == new_status]
            assert len(corresponding_enum_variants) == 1
            self._status = corresponding_enum_variants[0]
        else:
            raise TypeError(f"expected Status or str, got {new_status}")

    @property
    def host(self):
        return get_address_host(self.address)

    def clean(self):
        """ Return a version of this object that is appropriate for serialization """
        ws = WorkerState(
            address=self.address,
            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,
        )
        ws.processing = {ts.key for ts in self.processing}
        return ws

    def __repr__(self):
        return "<Worker %r, name: %s, memory: %d, processing: %d>" % (
            self.address,
            self.name,
            len(self.has_what),
            len(self.processing),
        )

    def identity(self):
        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


class TaskState:
    """
        A simple object holding information about a task.

        .. attribute:: key: str

           The key is the unique identifier of a task, generally formed
           from the name of the function, followed by a hash of the function
           and arguments, like ``'inc-ab31c010444977004d656610d2d421ec'``.

        .. attribute:: prefix: TaskPrefix

           The broad class of tasks to which this task belongs like "inc" or
           "read_csv"

        .. attribute:: run_spec: object

           A specification of how to run the task.  The type and meaning of this
           value is opaque to the scheduler, as it is only interpreted by the
           worker to which the task is sent for executing.

           As a special case, this attribute may also be ``None``, in which case
           the task is "pure data" (such as, for example, a piece of data loaded
           in the scheduler using :meth:`Client.scatter`).  A "pure data" task
           cannot be computed again if its value is lost.

        .. attribute:: priority: tuple

           The priority provides each task with a relative ranking which is used
           to break ties when many tasks are being considered for execution.

           This ranking is generally a 2-item tuple.  The first (and dominant)
           item corresponds to when it was submitted.  Generally, earlier tasks
           take precedence.  The second item is determined by the client, and is
           a way to prioritize tasks within a large graph that may be important,
           such as if they are on the critical path, or good to run in order to
           release many dependencies.  This is explained further in
           :doc:`Scheduling Policy <scheduling-policies>`.

        .. attribute:: state: str

           This task's current state.  Valid states include ``released``,
           ``waiting``, ``no-worker``, ``processing``, ``memory``, ``erred``
           and ``forgotten``.  If it is ``forgotten``, the task isn't stored
           in the ``tasks`` dictionary anymore and will probably disappear
           soon from memory.

        .. attribute:: dependencies: {TaskState}

           The set of tasks this task depends on for proper execution.  Only
           tasks still alive are listed in this set.  If, for whatever reason,
           this task also depends on a forgotten task, the
           :attr:`has_lost_dependencies` flag is set.

           A task can only be executed once all its dependencies have already
           been successfully executed and have their result stored on at least
           one worker.  This is tracked by progressively draining the
           :attr:`waiting_on` set.

        .. attribute:: dependents: {TaskState}

           The set of tasks which depend on this task.  Only tasks still alive
           are listed in this set.

           This is the reverse mapping of :attr:`dependencies`.

        .. attribute:: has_lost_dependencies: bool

           Whether any of the dependencies of this task has been forgotten.
           For memory consumption reasons, forgotten tasks are not kept in
           memory even though they may have dependent tasks.  When a task is
           forgotten, therefore, each of its dependents has their
           :attr:`has_lost_dependencies` attribute set to ``True``.

           If :attr:`has_lost_dependencies` is true, this task cannot go
           into the "processing" state anymore.

        .. attribute:: waiting_on: {TaskState}

           The set of tasks this task is waiting on *before* it can be executed.
           This is always a subset of :attr:`dependencies`.  Each time one of the
           dependencies has finished processing, it is removed from the
           :attr:`waiting_on` set.

           Once :attr:`waiting_on` becomes empty, this task can move from the
           "waiting" state to the "processing" state (unless one of the
           dependencies errored out, in which case this task is instead
           marked "erred").

        .. attribute:: waiters: {TaskState}

           The set of tasks which need this task to remain alive.  This is always
           a subset of :attr:`dependents`.  Each time one of the dependents
           has finished processing, it is removed from the :attr:`waiters`
           set.

           Once both :attr:`waiters` and :attr:`who_wants` become empty, this
           task can be released (if it has a non-empty :attr:`run_spec`) or
           forgotten (otherwise) by the scheduler, and by any workers
           in :attr:`who_has`.

           .. note:: Counter-intuitively, :attr:`waiting_on` and
              :attr:`waiters` are not reverse mappings of each other.

        .. attribute:: who_wants: {ClientState}

           The set of clients who want this task's result to remain alive.
           This is the reverse mapping of :attr:`ClientState.wants_what`.

           When a client submits a graph to the scheduler it also specifies
           which output tasks it desires, such that their results are not released
           from memory.

           Once a task has finished executing (i.e. moves into the "memory"
           or "erred" state), the clients in :attr:`who_wants` are notified.

           Once both :attr:`waiters` and :attr:`who_wants` become empty, this
           task can be released (if it has a non-empty :attr:`run_spec`) or
           forgotten (otherwise) by the scheduler, and by any workers
           in :attr:`who_has`.

        .. attribute:: who_has: {WorkerState}

           The set of workers who have this task's result in memory.
           It is non-empty iff the task is in the "memory" state.  There can be
           more than one worker in this set if, for example, :meth:`Client.scatter`
           or :meth:`Client.replicate` was used.

           This is the reverse mapping of :attr:`WorkerState.has_what`.

        .. attribute:: processing_on: WorkerState (or None)

           If this task is in the "processing" state, which worker is currently
           processing it.  Otherwise this is ``None``.

           This attribute is kept in sync with :attr:`WorkerState.processing`.

        .. attribute:: retries: int

           The number of times this task can automatically be retried in case
           of failure.  If a task fails executing (the worker returns with
           an error), its :attr:`retries` attribute is checked.  If it is
           equal to 0, the task is marked "erred".  If it is greater than 0,
           the :attr:`retries` attribute is decremented and execution is
           attempted again.

        .. attribute:: nbytes: int (or None)

           The number of bytes, as determined by ``sizeof``, of the result
           of a finished task.  This number is used for diagnostics and to
           help prioritize work.

        .. attribute:: type: str

           The type of the object as a string.  Only present for tasks that have
           been computed.

        .. attribute:: exception: object

           If this task failed executing, the exception object is stored here.
           Otherwise this is ``None``.

        .. attribute:: traceback: object

           If this task failed executing, the traceback object is stored here.
           Otherwise this is ``None``.

        .. attribute:: exception_blame: TaskState (or None)

           If this task or one of its dependencies failed executing, the
           failed task is stored here (possibly itself).  Otherwise this
           is ``None``.

        .. attribute:: suspicious: int

           The number of times this task has been involved in a worker death.

           Some tasks may cause workers to die (such as calling ``os._exit(0)``).
           When a worker dies, all of the tasks on that worker are reassigned
           to others.  This combination of behaviors can cause a bad task to
           catastrophically destroy all workers on the cluster, one after
           another.  Whenever a worker dies, we mark each task currently
           processing on that worker (as recorded by
           :attr:`WorkerState.processing`) as suspicious.

           If a task is involved in three deaths (or some other fixed constant)
           then we mark the task as ``erred``.

        .. attribute:: host_restrictions: {hostnames}

           A set of hostnames where this task can be run (or ``None`` if empty).
           Usually this is empty unless the task has been specifically restricted
           to only run on certain hosts.  A hostname may correspond to one or
           several connected workers.

        .. attribute:: worker_restrictions: {worker addresses}

           A set of complete worker addresses where this can be run (or ``None``
           if empty).  Usually this is empty unless the task has been specifically
           restricted to only run on certain workers.

           Note this is tracking worker addresses, not worker states, since
           the specific workers may not be connected at this time.

        .. attribute:: resource_restrictions: {resource: quantity}

           Resources required by this task, such as ``{'gpu': 1}`` or
           ``{'memory': 1e9}`` (or ``None`` if empty).  These are user-defined
           names and are matched against the contents of each
           :attr:`WorkerState.resources` dictionary.

        .. attribute:: loose_restrictions: bool

           If ``False``, each of :attr:`host_restrictions`,
           :attr:`worker_restrictions` and :attr:`resource_restrictions` is
           a hard constraint: if no worker is available satisfying those
           restrictions, the task cannot go into the "processing" state and
           will instead go into the "no-worker" state.

           If ``True``, the above restrictions are mere preferences: if no worker
           is available satisfying those restrictions, the task can still go
           into the "processing" state and be sent for execution to another
           connected worker.

        .. attribute: metadata: dict

           Metadata related to task.

        .. attribute: actor: bool

           Whether or not this task is an Actor.

        .. attribute: group: TaskGroup

    :      The group of tasks to which this one belongs.
    """

    __slots__ = (
        # === General description ===
        "actor",
        # Key name
        "key",
        # Key prefix (see key_split())
        "prefix",
        # How to run the task (None if pure data)
        "run_spec",
        # Alive dependents and dependencies
        "dependencies",
        "dependents",
        # Compute priority
        "priority",
        # Restrictions
        "host_restrictions",
        "worker_restrictions",  # not WorkerStates but addresses
        "resource_restrictions",
        "loose_restrictions",
        # === Task state ===
        "_state",
        # Whether some dependencies were forgotten
        "has_lost_dependencies",
        # If in 'waiting' state, which tasks need to complete
        # before we can run
        "waiting_on",
        # If in 'waiting' or 'processing' state, which tasks needs us
        # to complete before they can run
        "waiters",
        # In in 'processing' state, which worker we are processing on
        "processing_on",
        # If in 'memory' state, Which workers have us
        "who_has",
        # Which clients want us
        "who_wants",
        "exception",
        "traceback",
        "exception_blame",
        "suspicious",
        "retries",
        "nbytes",
        "type",
        "group_key",
        "group",
        "metadata",
    )

    def __init__(self, key, run_spec):
        self.key = key
        self.run_spec = run_spec
        self._state = None
        self.exception = self.traceback = self.exception_blame = None
        self.suspicious = self.retries = 0
        self.nbytes = None
        self.priority = None
        self.who_wants = set()
        self.dependencies = set()
        self.dependents = set()
        self.waiting_on = set()
        self.waiters = set()
        self.who_has = set()
        self.processing_on = None
        self.has_lost_dependencies = False
        self.host_restrictions = None
        self.worker_restrictions = None
        self.resource_restrictions = None
        self.loose_restrictions = False
        self.actor = None
        self.type = None
        self.group_key = key_split_group(key)
        self.group = None
        self.metadata = {}

    @property
    def state(self) -> str:
        return self._state

    @property
    def prefix_key(self):
        return self.prefix.name

    @state.setter
    def state(self, value: str):
        self.group.states[self._state] -= 1
        self.group.states[value] += 1
        self._state = value

    def add_dependency(self, other: "TaskState"):
        """ Add another task as a dependency of this task """
        self.dependencies.add(other)
        self.group.dependencies.add(other.group)
        other.dependents.add(self)

    def get_nbytes(self) -> int:
        nbytes = self.nbytes
        return nbytes if nbytes is not None else DEFAULT_DATA_SIZE

    def set_nbytes(self, nbytes: int):
        old_nbytes = self.nbytes
        diff = nbytes - (old_nbytes or 0)
        self.group.nbytes_total += diff
        self.group.nbytes_in_memory += diff
        for ws in self.who_has:
            ws.nbytes += diff
        self.nbytes = nbytes

    def __repr__(self):
        return "<Task %r %s>" % (self.key, self.state)

    def validate(self):
        try:
            for cs in self.who_wants:
                assert isinstance(cs, ClientState), (repr(cs), self.who_wants)
            for ws in self.who_has:
                assert isinstance(ws, WorkerState), (repr(ws), self.who_has)
            for ts in self.dependencies:
                assert isinstance(ts, TaskState), (repr(ts), self.dependencies)
            for ts in self.dependents:
                assert isinstance(ts, TaskState), (repr(ts), self.dependents)
            validate_task_state(self)
        except Exception as e:
            logger.exception(e)
            if LOG_PDB:
                import pdb

                pdb.set_trace()


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.
       For a task like ``("x-123", 0)`` this is the text ``"x-123"``

    .. attribute:: states: Dict[str, int]

       The number of tasks in each state,
       like ``{"memory": 10, "processing": 3, "released": 4, ...}``

    .. attribute:: dependencies: Set[TaskGroup]

       The other TaskGroups on which this one depends

    .. attribute:: nbytes_total: int

       The total number of bytes that this task group has produced

    .. attribute:: nbytes_in_memory: int

       The number of bytes currently stored by this TaskGroup

    .. attribute:: duration: float

       The total amount of time spent on all tasks in this TaskGroup

    .. attribute:: types: Set[str]

       The result types of this TaskGroup

    See also
    --------
    TaskPrefix
    """

    def __init__(self, name):
        self.name = name
        self.states = {state: 0 for state in ALL_TASK_STATES}
        self.states["forgotten"] = 0
        self.dependencies = set()
        self.nbytes_total = 0
        self.nbytes_in_memory = 0
        self.duration = 0
        self.types = set()

    def add(self, ts):
        self.states[ts.state] += 1
        ts.group = self

    def __repr__(self):
        return (
            "<"
            + (self.name or "no-group")
            + ": "
            + ", ".join(
                "%s: %d" % (k, v) for (k, v) in sorted(self.states.items()) if v
            )
            + ">"
        )

    def __len__(self):
        return sum(self.states.values())


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

    def __init__(self, name):
        self.name = name
        self.groups = []

        # store timings for each prefix-action
        self.all_durations = defaultdict(float)

        if self.name in dask.config.get("distributed.scheduler.default-task-durations"):
            self.duration_average = parse_timedelta(
                dask.config.get("distributed.scheduler.default-task-durations")[
                    self.name
                ]
            )
        else:
            self.duration_average = None
        self.suspicious = 0

    @property
    def states(self):
        return merge_with(sum, [g.states for g in self.groups])

    @property
    def active(self):
        return [
            g
            for g in self.groups
            if any(v != 0 for k, v in g.states.items() if k != "forgotten")
        ]

    @property
    def active_states(self):
        return merge_with(sum, [g.states for g 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_in_memory(self):
        return sum(tg.nbytes_in_memory for tg in self.groups)

    @property
    def nbytes_total(self):
        return sum(tg.nbytes_total for tg in self.groups)

    def __len__(self):
        return sum(map(len, self.groups))

    @property
    def duration(self):
        return sum(tg.duration for tg in self.groups)

    @property
    def types(self):
        return set().union(*[tg.types for tg in self.groups])


class _StateLegacyMapping(Mapping):
    """
    A mapping interface mimicking the former Scheduler state dictionaries.
    """

    def __init__(self, states, accessor):
        self._states = states
        self._accessor = accessor

    def __iter__(self):
        return iter(self._states)

    def __len__(self):
        return len(self._states)

    def __getitem__(self, key):
        return self._accessor(self._states[key])

    def __repr__(self):
        return "%s(%s)" % (self.__class__, dict(self))


class _OptionalStateLegacyMapping(_StateLegacyMapping):
    """
    Similar to _StateLegacyMapping, but a false-y value is interpreted
    as a missing key.
    """

    # For tasks etc.

    def __iter__(self):
        accessor = self._accessor
        for k, v in self._states.items():
            if accessor(v):
                yield k

    def __len__(self):
        accessor = self._accessor
        return sum(bool(accessor(v)) for v in self._states.values())

    def __getitem__(self, key):
        v = self._accessor(self._states[key])
        if v:
            return v
        else:
            raise KeyError


class _StateLegacySet(Set):
    """
    Similar to _StateLegacyMapping, but exposes a set containing
    all values with a true value.
    """

    # For loose_restrictions

    def __init__(self, states, accessor):
        self._states = states
        self._accessor = accessor

    def __iter__(self):
        return (k for k, v in self._states.items() if self._accessor(v))

    def __len__(self):
        return sum(map(bool, map(self._accessor, self._states.values())))

    def __contains__(self, k):
        st = self._states.get(k)
        return st is not None and bool(self._accessor(st))

    def __repr__(self):
        return "%s(%s)" % (self.__class__, set(self))


def _legacy_task_key_set(tasks):
    """
    Transform a set of task states into a set of task keys.
    """
    return {ts.key for ts in tasks}


def _legacy_client_key_set(clients):
    """
    Transform a set of client states into a set of client keys.
    """
    return {cs.client_key for cs in clients}


def _legacy_worker_key_set(workers):
    """
    Transform a set of worker states into a set of worker keys.
    """
    return {ws.address for ws in workers}


def _legacy_task_key_dict(task_dict):
    """
    Transform a dict of {task state: value} into a dict of {task key: value}.
    """
    return {ts.key: value for ts, value in task_dict.items()}


def _task_key_or_none(task):
    return task.key if task is not None else None


class Scheduler(ServerNode):
    """Dynamic distributed task scheduler

    The scheduler tracks the current state of workers, data, and computations.
    The scheduler listens for events and responds by controlling workers
    appropriately.  It continuously tries to use the workers to execute an ever
    growing dask graph.

    All events are handled quickly, in linear time with respect to their input
    (which is often of constant size) and generally within a millisecond.  To
    accomplish this the scheduler tracks a lot of state.  Every operation
    maintains the consistency of this state.

    The scheduler communicates with the outside world through Comm objects.
    It maintains a consistent and valid view of the world even when listening
    to several clients at once.

    A Scheduler is typically started either with the ``dask-scheduler``
    executable::

         $ dask-scheduler
         Scheduler started at 127.0.0.1:8786

    Or within a LocalCluster a Client starts up without connection
    information::

        >>> c = Client()  # doctest: +SKIP
        >>> c.cluster.scheduler  # doctest: +SKIP
        Scheduler(...)

    Users typically do not interact with the scheduler directly but rather with
    the client object ``Client``.

    **State**

    The scheduler contains the following state variables.  Each variable is
    listed along with what it stores and a brief description.

    * **tasks:** ``{task key: TaskState}``
        Tasks currently known to the scheduler
    * **unrunnable:** ``{TaskState}``
        Tasks in the "no-worker" state

    * **workers:** ``{worker key: WorkerState}``
        Workers currently connected to the scheduler
    * **idle:** ``{WorkerState}``:
        Set of workers that are not fully utilized
    * **saturated:** ``{WorkerState}``:
        Set of workers that are not over-utilized

    * **host_info:** ``{hostname: dict}``:
        Information about each worker host

    * **clients:** ``{client key: ClientState}``
        Clients currently connected to the scheduler

    * **services:** ``{str: port}``:
        Other services running on this scheduler, like Bokeh
    * **loop:** ``IOLoop``:
        The running Tornado IOLoop
    * **client_comms:** ``{client key: Comm}``
        For each client, a Comm object used to receive task requests and
        report task status updates.
    * **stream_comms:** ``{worker key: Comm}``
        For each worker, a Comm object from which we both accept stimuli and
        report results
    * **task_duration:** ``{key-prefix: time}``
        Time we expect certain functions to take, e.g. ``{'sum': 0.25}``
    """

    default_port = 8786
    _instances = weakref.WeakSet()

    def __init__(
        self,
        loop=None,
        delete_interval="500ms",
        synchronize_worker_interval="60s",
        services=None,
        service_kwargs=None,
        allowed_failures=None,
        extensions=None,
        validate=None,
        scheduler_file=None,
        security=None,
        worker_ttl=None,
        idle_timeout=None,
        interface=None,
        host=None,
        port=0,
        protocol=None,
        dashboard_address=None,
        dashboard=None,
        http_prefix="/",
        preload=None,
        preload_argv=(),
        plugins=(),
        **kwargs,
    ):
        self._setup_logging(logger)

        # Attributes
        if allowed_failures is None:
            allowed_failures = dask.config.get("distributed.scheduler.allowed-failures")
        self.allowed_failures = allowed_failures
        if validate is None:
            validate = dask.config.get("distributed.scheduler.validate")
        self.validate = validate
        self.proc = psutil.Process()
        self.delete_interval = parse_timedelta(delete_interval, default="ms")
        self.synchronize_worker_interval = parse_timedelta(
            synchronize_worker_interval, default="ms"
        )
        self.digests = None
        self.service_specs = services or {}
        self.service_kwargs = service_kwargs or {}
        self.services = {}
        self.scheduler_file = scheduler_file
        worker_ttl = worker_ttl or dask.config.get("distributed.scheduler.worker-ttl")
        self.worker_ttl = parse_timedelta(worker_ttl) if worker_ttl else None
        idle_timeout = idle_timeout or dask.config.get(
            "distributed.scheduler.idle-timeout"
        )
        if idle_timeout:
            self.idle_timeout = parse_timedelta(idle_timeout)
        else:
            self.idle_timeout = None
        self.idle_since = time()
        self.time_started = self.idle_since  # compatibility for dask-gateway
        self._lock = asyncio.Lock()
        self.bandwidth = parse_bytes(dask.config.get("distributed.scheduler.bandwidth"))
        self.bandwidth_workers = defaultdict(float)
        self.bandwidth_types = defaultdict(float)

        if not preload:
            preload = dask.config.get("distributed.scheduler.preload")
        if not preload_argv:
            preload_argv = dask.config.get("distributed.scheduler.preload-argv")
        self.preloads = preloading.process_preloads(self, preload, preload_argv)

        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("scheduler")
        self.connection_args["handshake_overrides"] = {  # common denominator
            "pickle-protocol": 4
        }

        self._start_address = addresses_from_user_args(
            host=host,
            port=port,
            interface=interface,
            protocol=protocol,
            security=security,
            default_port=self.default_port,
        )

        http_server_modules = dask.config.get("distributed.scheduler.http.routes")
        show_dashboard = dashboard or (dashboard is None and dashboard_address)
        missing_bokeh = False
        # install vanilla route if show_dashboard but bokeh is not installed
        if show_dashboard:
            try:
                import distributed.dashboard.scheduler
            except ImportError:
                missing_bokeh = True
                http_server_modules.append("distributed.http.scheduler.missing_bokeh")
        routes = get_handlers(
            server=self, modules=http_server_modules, prefix=http_prefix
        )
        self.start_http_server(routes, dashboard_address, default_port=8787)
        if show_dashboard and not missing_bokeh:
            distributed.dashboard.scheduler.connect(
                self.http_application, self.http_server, self, prefix=http_prefix
            )

        # Communication state
        self.loop = loop or IOLoop.current()
        self.client_comms = dict()
        self.stream_comms = dict()
        self._worker_coroutines = []
        self._ipython_kernel = None

        # Task state
        self.tasks = dict()
        self.task_groups = dict()
        self.task_prefixes = dict()
        for old_attr, new_attr, wrap in [
            ("priority", "priority", None),
            ("dependencies", "dependencies", _legacy_task_key_set),
            ("dependents", "dependents", _legacy_task_key_set),
            ("retries", "retries", None),
        ]:
            func = operator.attrgetter(new_attr)
            if wrap is not None:
                func = compose(wrap, func)
            setattr(self, old_attr, _StateLegacyMapping(self.tasks, func))

        for old_attr, new_attr, wrap in [
            ("nbytes", "nbytes", None),
            ("who_wants", "who_wants", _legacy_client_key_set),
            ("who_has", "who_has", _legacy_worker_key_set),
            ("waiting", "waiting_on", _legacy_task_key_set),
            ("waiting_data", "waiters", _legacy_task_key_set),
            ("rprocessing", "processing_on", None),
            ("host_restrictions", "host_restrictions", None),
            ("worker_restrictions", "worker_restrictions", None),
            ("resource_restrictions", "resource_restrictions", None),
            ("suspicious_tasks", "suspicious", None),
            ("exceptions", "exception", None),
            ("tracebacks", "traceback", None),
            ("exceptions_blame", "exception_blame", _task_key_or_none),
        ]:
            func = operator.attrgetter(new_attr)
            if wrap is not None:
                func = compose(wrap, func)
            setattr(self, old_attr, _OptionalStateLegacyMapping(self.tasks, func))

        for old_attr, new_attr, wrap in [
            ("loose_restrictions", "loose_restrictions", None)
        ]:
            func = operator.attrgetter(new_attr)
            if wrap is not None:
                func = compose(wrap, func)
            setattr(self, old_attr, _StateLegacySet(self.tasks, func))

        self.generation = 0
        self._last_client = None
        self._last_time = 0
        self.unrunnable = set()

        self.n_tasks = 0
        self.task_metadata = dict()
        self.datasets = dict()

        # Prefix-keyed containers
        self.unknown_durations = defaultdict(set)

        # Client state
        self.clients = dict()
        for old_attr, new_attr, wrap in [
            ("wants_what", "wants_what", _legacy_task_key_set)
        ]:
            func = operator.attrgetter(new_attr)
            if wrap is not None:
                func = compose(wrap, func)
            setattr(self, old_attr, _StateLegacyMapping(self.clients, func))
        self.clients["fire-and-forget"] = ClientState("fire-and-forget")

        # Worker state
        self.workers = sortedcontainers.SortedDict()
        for old_attr, new_attr, wrap in [
            ("nthreads", "nthreads", None),
            ("worker_bytes", "nbytes", None),
            ("worker_resources", "resources", None),
            ("used_resources", "used_resources", None),
            ("occupancy", "occupancy", None),
            ("worker_info", "metrics", None),
            ("processing", "processing", _legacy_task_key_dict),
            ("has_what", "has_what", _legacy_task_key_set),
        ]:
            func = operator.attrgetter(new_attr)
            if wrap is not None:
                func = compose(wrap, func)
            setattr(self, old_attr, _StateLegacyMapping(self.workers, func))

        self.idle = sortedcontainers.SortedSet(key=operator.attrgetter("address"))
        self.saturated = set()

        self.total_nthreads = 0
        self.total_occupancy = 0
        self.host_info = defaultdict(dict)
        self.resources = defaultdict(dict)
        self.aliases = dict()

        self._task_state_collections = [self.unrunnable]

        self._worker_collections = [
            self.workers,
            self.host_info,
            self.resources,
            self.aliases,
        ]

        self.extensions = {}
        self.plugins = list(plugins)
        self.transition_log = deque(
            maxlen=dask.config.get("distributed.scheduler.transition-log-length")
        )
        self.log = deque(
            maxlen=dask.config.get("distributed.scheduler.transition-log-length")
        )
        self.worker_plugins = []

        worker_handlers = {
            "task-finished": self.handle_task_finished,
            "task-erred": self.handle_task_erred,
            "release": self.handle_release_data,
            "release-worker-data": self.release_worker_data,
            "add-keys": self.add_keys,
            "missing-data": self.handle_missing_data,
            "long-running": self.handle_long_running,
            "reschedule": self.reschedule,
            "keep-alive": lambda *args, **kwargs: None,
        }

        client_handlers = {
            "update-graph": self.update_graph,
            "update-graph-hlg": self.update_graph_hlg,
            "client-desires-keys": self.client_desires_keys,
            "update-data": self.update_data,
            "report-key": self.report_on_key,
            "client-releases-keys": self.client_releases_keys,
            "heartbeat-client": self.client_heartbeat,
            "close-client": self.remove_client,
            "restart": self.restart,
        }

        self.handlers = {
            "register-client": self.add_client,
            "scatter": self.scatter,
            "register-worker": self.add_worker,
            "unregister": self.remove_worker,
            "gather": self.gather,
            "cancel": self.stimulus_cancel,
            "retry": self.stimulus_retry,
            "feed": self.feed,
            "terminate": self.close,
            "broadcast": self.broadcast,
            "proxy": self.proxy,
            "ncores": self.get_ncores,
            "has_what": self.get_has_what,
            "who_has": self.get_who_has,
            "processing": self.get_processing,
            "call_stack": self.get_call_stack,
            "profile": self.get_profile,
            "performance_report": self.performance_report,
            "get_logs": self.get_logs,
            "logs": self.get_logs,
            "worker_logs": self.get_worker_logs,
            "nbytes": self.get_nbytes,
            "versions": self.versions,
            "add_keys": self.add_keys,
            "rebalance": self.rebalance,
            "replicate": self.replicate,
            "start_ipython": self.start_ipython,
            "run_function": self.run_function,
            "update_data": self.update_data,
            "set_resources": self.add_resources,
            "retire_workers": self.retire_workers,
            "get_metadata": self.get_metadata,
            "set_metadata": self.set_metadata,
            "heartbeat_worker": self.heartbeat_worker,
            "get_task_status": self.get_task_status,
            "get_task_stream": self.get_task_stream,
            "register_worker_plugin": self.register_worker_plugin,
            "adaptive_target": self.adaptive_target,
            "workers_to_close": self.workers_to_close,
            "subscribe_worker_status": self.subscribe_worker_status,
            "start_task_metadata": self.start_task_metadata,
            "stop_task_metadata": self.stop_task_metadata,
        }

        self._transitions = {
            ("released", "waiting"): self.transition_released_waiting,
            ("waiting", "released"): self.transition_waiting_released,
            ("waiting", "processing"): self.transition_waiting_processing,
            ("waiting", "memory"): self.transition_waiting_memory,
            ("processing", "released"): self.transition_processing_released,
            ("processing", "memory"): self.transition_processing_memory,
            ("processing", "erred"): self.transition_processing_erred,
            ("no-worker", "released"): self.transition_no_worker_released,
            ("no-worker", "waiting"): self.transition_no_worker_waiting,
            ("released", "forgotten"): self.transition_released_forgotten,
            ("memory", "forgotten"): self.transition_memory_forgotten,
            ("erred", "forgotten"): self.transition_released_forgotten,
            ("erred", "released"): self.transition_erred_released,
            ("memory", "released"): self.transition_memory_released,
            ("released", "erred"): self.transition_released_erred,
        }

        connection_limit = get_fileno_limit() / 2

        super().__init__(
            handlers=self.handlers,
            stream_handlers=merge(worker_handlers, client_handlers),
            io_loop=self.loop,
            connection_limit=connection_limit,
            deserialize=False,
            connection_args=self.connection_args,
            **kwargs,
        )

        if self.worker_ttl:
            pc = PeriodicCallback(self.check_worker_ttl, self.worker_ttl)
            self.periodic_callbacks["worker-ttl"] = pc

        if self.idle_timeout:
            pc = PeriodicCallback(self.check_idle, self.idle_timeout / 4)
            self.periodic_callbacks["idle-timeout"] = pc

        if extensions is None:
            extensions = list(DEFAULT_EXTENSIONS)
            if dask.config.get("distributed.scheduler.work-stealing"):
                extensions.append(WorkStealing)
        for ext in extensions:
            ext(self)

        setproctitle("dask-scheduler [not started]")
        Scheduler._instances.add(self)
        self.rpc.allow_offload = False
        self.status = Status.undefined

    ##################
    # Administration #
    ##################

    def __repr__(self):
        return '<Scheduler: "%s" processes: %d cores: %d>' % (
            self.address,
            len(self.workers),
            self.total_nthreads,
        )

    def identity(self, comm=None):
        """ Basic information about ourselves and our cluster """
        d = {
            "type": type(self).__name__,
            "id": str(self.id),
            "address": self.address,
            "services": {key: v.port for (key, v) in self.services.items()},
            "workers": {
                worker.address: worker.identity() for worker in self.workers.values()
            },
        }
        return d

    def get_worker_service_addr(self, worker, service_name, protocol=False):
        """
        Get the (host, port) address of the named service on the *worker*.
        Returns None if the service doesn't exist.

        Parameters
        ----------
        worker : address
        service_name : str
            Common services include 'bokeh' and 'nanny'
        protocol : boolean
            Whether or not to include a full address with protocol (True)
            or just a (host, port) pair
        """
        ws = self.workers[worker]
        port = ws.services.get(service_name)
        if port is None:
            return None
        elif protocol:
            return "%(protocol)s://%(host)s:%(port)d" % {
                "protocol": ws.address.split("://")[0],
                "host": ws.host,
                "port": port,
            }
        else:
            return ws.host, port

    async def start(self):
        """ Clear out old state and restart all running coroutines """
        await super().start()
        assert self.status != Status.running

        enable_gc_diagnosis()

        self.clear_task_state()

        with suppress(AttributeError):
            for c in self._worker_coroutines:
                c.cancel()

        for addr in self._start_address:
            await self.listen(
                addr,
                allow_offload=False,
                handshake_overrides={"pickle-protocol": 4, "compression": None},
                **self.security.get_listen_args("scheduler"),
            )
            self.ip = get_address_host(self.listen_address)
            listen_ip = self.ip

            if listen_ip == "0.0.0.0":
                listen_ip = ""

        if self.address.startswith("inproc://"):
            listen_ip = "localhost"

        # Services listen on all addresses
        self.start_services(listen_ip)

        for listener in self.listeners:
            logger.info("  Scheduler at: %25s", listener.contact_address)
        for k, v in self.services.items():
            logger.info("%11s at: %25s", k, "%s:%d" % (listen_ip, v.port))

        self.loop.add_callback(self.reevaluate_occupancy)

        if self.scheduler_file:
            with open(self.scheduler_file, "w") as f:
                json.dump(self.identity(), f, indent=2)

            fn = self.scheduler_file  # remove file when we close the process

            def del_scheduler_file():
                if os.path.exists(fn):
                    os.remove(fn)

            weakref.finalize(self, del_scheduler_file)

        for preload in self.preloads:
            await preload.start()

        await asyncio.gather(*[plugin.start(self) for plugin in self.plugins])

        self.start_periodic_callbacks()

        setproctitle("dask-scheduler [%s]" % (self.address,))
        return self

    async def close(self, comm=None, fast=False, close_workers=False):
        """Send cleanup signal to all coroutines then wait until finished

        See Also
        --------
        Scheduler.cleanup
        """
        if self.status in (Status.closing, Status.closed, Status.closing_gracefully):
            await self.finished()
            return
        self.status = Status.closing

        logger.info("Scheduler closing...")
        setproctitle("dask-scheduler [closing]")

        for preload in self.preloads:
            await preload.teardown()

        if close_workers:
            await self.broadcast(msg={"op": "close_gracefully"}, nanny=True)
            for worker in self.workers:
                self.worker_send(worker, {"op": "close"})
            for i in range(20):  # wait a second for send signals to clear
                if self.workers:
                    await asyncio.sleep(0.05)
                else:
                    break

        await asyncio.gather(*[plugin.close() for plugin in self.plugins])

        for pc in self.periodic_callbacks.values():
            pc.stop()
        self.periodic_callbacks.clear()

        self.stop_services()

        for ext in self.extensions.values():
            with suppress(AttributeError):
                ext.teardown()
        logger.info("Scheduler closing all comms")

        futures = []
        for w, comm in list(self.stream_comms.items()):
            if not comm.closed():
                comm.send({"op": "close", "report": False})
                comm.send({"op": "close-stream"})
            with suppress(AttributeError):
                futures.append(comm.close())

        for future in futures:  # TODO: do all at once
            await future

        for comm in self.client_comms.values():
            comm.abort()

        await self.rpc.close()

        self.status = Status.closed
        self.stop()
        await super().close()

        setproctitle("dask-scheduler [closed]")
        disable_gc_diagnosis()

    async def close_worker(self, comm=None, worker=None, safe=None):
        """Remove a worker from the cluster

        This both removes the worker from our local state and also sends a
        signal to the worker to shut down.  This works regardless of whether or
        not the worker has a nanny process restarting it
        """
        logger.info("Closing worker %s", worker)
        with log_errors():
            self.log_event(worker, {"action": "close-worker"})
            nanny_addr = self.workers[worker].nanny
            address = nanny_addr or worker

            self.worker_send(worker, {"op": "close", "report": False})
            await self.remove_worker(address=worker, safe=safe)

    ###########
    # Stimuli #
    ###########

    def heartbeat_worker(
        self,
        comm=None,
        address=None,
        resolve_address=True,
        now=None,
        resources=None,
        host_info=None,
        metrics=None,
    ):
        address = self.coerce_address(address, resolve_address)
        address = normalize_address(address)
        if address not in self.workers:
            return {"status": "missing"}

        host = get_address_host(address)
        local_now = time()
        now = now or time()
        assert metrics
        host_info = host_info or {}

        self.host_info[host]["last-seen"] = local_now
        frac = 1 / len(self.workers)
        self.bandwidth = (
            self.bandwidth * (1 - frac) + metrics["bandwidth"]["total"] * frac
        )
        for other, (bw, count) in metrics["bandwidth"]["workers"].items():
            if (address, other) not in self.bandwidth_workers:
                self.bandwidth_workers[address, other] = bw / count
            else:
                alpha = (1 - frac) ** count
                self.bandwidth_workers[address, other] = self.bandwidth_workers[
                    address, other
                ] * alpha + bw * (1 - alpha)
        for typ, (bw, count) in metrics["bandwidth"]["types"].items():
            if typ not in self.bandwidth_types:
                self.bandwidth_types[typ] = bw / count
            else:
                alpha = (1 - frac) ** count
                self.bandwidth_types[typ] = self.bandwidth_types[typ] * alpha + bw * (
                    1 - alpha
                )

        ws = self.workers[address]

        ws.last_seen = time()

        if metrics:
            ws.metrics = metrics

        if host_info:
            self.host_info[host].update(host_info)

        delay = time() - now
        ws.time_delay = delay

        if resources:
            self.add_resources(worker=address, resources=resources)

        self.log_event(address, merge({"action": "heartbeat"}, metrics))

        return {
            "status": "OK",
            "time": time(),
            "heartbeat-interval": heartbeat_interval(len(self.workers)),
        }

    async def add_worker(
        self,
        comm=None,
        address=None,
        keys=(),
        nthreads=None,
        name=None,
        resolve_address=True,
        nbytes=None,
        types=None,
        now=None,
        resources=None,
        host_info=None,
        memory_limit=None,
        metrics=None,
        pid=0,
        services=None,
        local_directory=None,
        versions=None,
        nanny=None,
        extra=None,
    ):
        """ Add a new worker to the cluster """
        with log_errors():
            address = self.coerce_address(address, resolve_address)
            address = normalize_address(address)
            host = get_address_host(address)

            ws = self.workers.get(address)
            if ws is not None:
                raise ValueError("Worker already exists %s" % ws)

            if name in self.aliases:
                msg = {
                    "status": "error",
                    "message": "name taken, %s" % name,
                    "time": time(),
                }
                if comm:
                    await comm.write(msg)
                return

            self.workers[address] = ws = WorkerState(
                address=address,
                pid=pid,
                nthreads=nthreads,
                memory_limit=memory_limit,
                name=name,
                local_directory=local_directory,
                services=services,
                versions=versions,
                nanny=nanny,
                extra=extra,
            )

            if "addresses" not in self.host_info[host]:
                self.host_info[host].update({"addresses": set(), "nthreads": 0})

            self.host_info[host]["addresses"].add(address)
            self.host_info[host]["nthreads"] += nthreads

            self.total_nthreads += nthreads
            self.aliases[name] = address

            response = self.heartbeat_worker(
                address=address,
                resolve_address=resolve_address,
                now=now,
                resources=resources,
                host_info=host_info,
                metrics=metrics,
            )

            # Do not need to adjust self.total_occupancy as self.occupancy[ws] cannot exist before this.
            self.check_idle_saturated(ws)

            # for key in keys:  # TODO
            #     self.mark_key_in_memory(key, [address])

            self.stream_comms[address] = BatchedSend(interval="5ms", loop=self.loop)

            if ws.nthreads > len(ws.processing):
                self.idle.add(ws)

            for plugin in self.plugins[:]:
                try:
                    result = plugin.add_worker(scheduler=self, worker=address)
                    if inspect.isawaitable(result):
                        await result
                except Exception as e:
                    logger.exception(e)

            if nbytes:
                for key in nbytes:
                    ts = self.tasks.get(key)
                    if ts is not None and ts.state in ("processing", "waiting"):
                        recommendations = self.transition(
                            key,
                            "memory",
                            worker=address,
                            nbytes=nbytes[key],
                            typename=types[key],
                        )
                        self.transitions(recommendations)

            recommendations = {}
            for ts in list(self.unrunnable):
                valid = self.valid_workers(ts)
                if valid is True or ws in valid:
                    recommendations[ts.key] = "waiting"

            if recommendations:
                self.transitions(recommendations)

            self.log_event(address, {"action": "add-worker"})
            self.log_event("all", {"action": "add-worker", "worker": address})
            logger.info("Register worker %s", ws)

            msg = {
                "status": "OK",
                "time": time(),
                "heartbeat-interval": heartbeat_interval(len(self.workers)),
                "worker-plugins": self.worker_plugins,
            }

            version_warning = version_module.error_message(
                version_module.get_versions(),
                merge(
                    {w: ws.versions for w, ws in self.workers.items()},
                    {c: cs.versions for c, cs in self.clients.items() if cs.versions},
                ),
                versions,
                client_name="This Worker",
            )
            msg.update(version_warning)

            if comm:
                await comm.write(msg)
            await self.handle_worker(comm=comm, worker=address)

    def update_graph_hlg(
        self,
        client=None,
        hlg=None,
        keys=None,
        dependencies=None,
        restrictions=None,
        priority=None,
        loose_restrictions=None,
        resources=None,
        submitting_task=None,
        retries=None,
        user_priority=0,
        actors=None,
        fifo_timeout=0,
    ):

        dsk, dependencies = highlevelgraph_unpack(hlg)

        # Remove any self-dependencies (happens on test_publish_bag() and others)
        for k, v in dependencies.items():
            deps = set(v)
            if k in deps:
                deps.remove(k)
            dependencies[k] = deps

        if priority is None:
            # Removing all non-local keys before calling order()
            dsk_keys = set(dsk)  # intersection() of sets is much faster than dict_keys
            stripped_deps = {
                k: v.intersection(dsk_keys)
                for k, v in dependencies.items()
                if k in dsk_keys
            }
            priority = dask.order.order(dsk, dependencies=stripped_deps)

        return self.update_graph(
            client,
            dsk,
            keys,
            dependencies,
            restrictions,
            priority,
            loose_restrictions,
            resources,
            submitting_task,
            retries,
            user_priority,
            actors,
            fifo_timeout,
        )

    def update_graph(
        self,
        client=None,
        tasks=None,
        keys=None,
        dependencies=None,
        restrictions=None,
        priority=None,
        loose_restrictions=None,
        resources=None,
        submitting_task=None,
        retries=None,
        user_priority=0,
        actors=None,
        fifo_timeout=0,
    ):
        """
        Add new computations to the internal dask graph

        This happens whenever the Client calls submit, map, get, or compute.
        """
        start = time()
        fifo_timeout = parse_timedelta(fifo_timeout)
        keys = set(keys)
        if len(tasks) > 1:
            self.log_event(
                ["all", client], {"action": "update_graph", "count": len(tasks)}
            )

        # Remove aliases
        for k in list(tasks):
            if tasks[k] is k:
                del tasks[k]

        dependencies = dependencies or {}

        n = 0
        while len(tasks) != n:  # walk through new tasks, cancel any bad deps
            n = len(tasks)
            for k, deps in list(dependencies.items()):
                if any(
                    dep not in self.tasks and dep not in tasks for dep in deps
                ):  # bad key
                    logger.info("User asked for computation on lost data, %s", k)
                    del tasks[k]
                    del dependencies[k]
                    if k in keys:
                        keys.remove(k)
                    self.report({"op": "cancelled-key", "key": k}, client=client)
                    self.client_releases_keys(keys=[k], client=client)

        # Avoid computation that is already finished
        already_in_memory = set()  # tasks that are already done
        for k, v in dependencies.items():
            if v and k in self.tasks and self.tasks[k].state in ("memory", "erred"):
                already_in_memory.add(k)

        if already_in_memory:
            dependents = dask.core.reverse_dict(dependencies)
            stack = list(already_in_memory)
            done = set(already_in_memory)
            while stack:  # remove unnecessary dependencies
                key = stack.pop()
                ts = self.tasks[key]
                try:
                    deps = dependencies[key]
                except KeyError:
                    deps = self.dependencies[key]
                for dep in deps:
                    if dep in dependents:
                        child_deps = dependents[dep]
                    else:
                        child_deps = self.dependencies[dep]
                    if all(d in done for d in child_deps):
                        if dep in self.tasks and dep not in done:
                            done.add(dep)
                            stack.append(dep)

            for d in done:
                tasks.pop(d, None)
                dependencies.pop(d, None)

        # Get or create task states
        stack = list(keys)
        touched_keys = set()
        touched_tasks = []
        while stack:
            k = stack.pop()
            if k in touched_keys:
                continue
            # XXX Have a method get_task_state(self, k) ?
            ts = self.tasks.get(k)
            if ts is None:
                ts = self.new_task(k, tasks.get(k), "released")
            elif not ts.run_spec:
                ts.run_spec = tasks.get(k)

            touched_keys.add(k)
            touched_tasks.append(ts)
            stack.extend(dependencies.get(k, ()))

        self.client_desires_keys(keys=keys, client=client)

        # Add dependencies
        for key, deps in dependencies.items():
            ts = self.tasks.get(key)
            if ts is None or ts.dependencies:
                continue
            for dep in deps:
                dts = self.tasks[dep]
                ts.add_dependency(dts)

        # Compute priorities
        if isinstance(user_priority, Number):
            user_priority = {k: user_priority for k in tasks}

        # Add actors
        if actors is True:
            actors = list(keys)
        for actor in actors or []:
            self.tasks[actor].actor = True

        priority = priority or dask.order.order(
            tasks
        )  # TODO: define order wrt old graph

        if submitting_task:  # sub-tasks get better priority than parent tasks
            ts = self.tasks.get(submitting_task)
            if ts is not None:
                generation = ts.priority[0] - 0.01
            else:  # super-task already cleaned up
                generation = self.generation
        elif self._last_time + fifo_timeout < start:
            self.generation += 1  # older graph generations take precedence
            generation = self.generation
            self._last_time = start
        else:
            generation = self.generation

        for key in set(priority) & touched_keys:
            ts = self.tasks[key]
            if ts.priority is None:
                ts.priority = (-(user_priority.get(key, 0)), generation, priority[key])

        # Ensure all runnables have a priority
        runnables = [ts for ts in touched_tasks if ts.run_spec]
        for ts in runnables:
            if ts.priority is None and ts.run_spec:
                ts.priority = (self.generation, 0)

        if restrictions:
            # *restrictions* is a dict keying task ids to lists of
            # restriction specifications (either worker names or addresses)
            for k, v in restrictions.items():
                if v is None:
                    continue
                ts = self.tasks.get(k)
                if ts is None:
                    continue
                ts.host_restrictions = set()
                ts.worker_restrictions = set()
                for w in v:
                    try:
                        w = self.coerce_address(w)
                    except ValueError:
                        # Not a valid address, but perhaps it's a hostname
                        ts.host_restrictions.add(w)
                    else:
                        ts.worker_restrictions.add(w)

            if loose_restrictions:
                for k in loose_restrictions:
                    ts = self.tasks[k]
                    ts.loose_restrictions = True

        if resources:
            for k, v in resources.items():
                if v is None:
                    continue
                assert isinstance(v, dict)
                ts = self.tasks.get(k)
                if ts is None:
                    continue
                ts.resource_restrictions = v

        if retries:
            for k, v in retries.items():
                assert isinstance(v, int)
                ts = self.tasks.get(k)
                if ts is None:
                    continue
                ts.retries = v

        # Compute recommendations
        recommendations = {}

        for ts in sorted(runnables, key=operator.attrgetter("priority"), reverse=True):
            if ts.state == "released" and ts.run_spec:
                recommendations[ts.key] = "waiting"

        for ts in touched_tasks:
            for dts in ts.dependencies:
                if dts.exception_blame:
                    ts.exception_blame = dts.exception_blame
                    recommendations[ts.key] = "erred"
                    break

        for plugin in self.plugins[:]:
            try:
                plugin.update_graph(
                    self,
                    client=client,
                    tasks=tasks,
                    keys=keys,
                    restrictions=restrictions or {},
                    dependencies=dependencies,
                    priority=priority,
                    loose_restrictions=loose_restrictions,
                    resources=resources,
                )
            except Exception as e:
                logger.exception(e)

        self.transitions(recommendations)

        for ts in touched_tasks:
            if ts.state in ("memory", "erred"):
                self.report_on_key(ts.key, client=client)

        end = time()
        if self.digests is not None:
            self.digests["update-graph-duration"].add(end - start)

        # TODO: balance workers

    def new_task(self, key, spec, state):
        """ Create a new task, and associated states """
        ts = TaskState(key, spec)
        ts._state = state
        prefix_key = key_split(key)
        try:
            tp = self.task_prefixes[prefix_key]
        except KeyError:
            tp = self.task_prefixes[prefix_key] = TaskPrefix(prefix_key)
        ts.prefix = tp

        group_key = ts.group_key
        try:
            tg = self.task_groups[group_key]
        except KeyError:
            tg = self.task_groups[group_key] = TaskGroup(group_key)
            tg.prefix = tp
            tp.groups.append(tg)
        tg.add(ts)
        self.tasks[key] = ts
        return ts

    def stimulus_task_finished(self, key=None, worker=None, **kwargs):
        """ Mark that a task has finished execution on a particular worker """
        logger.debug("Stimulus task finished %s, %s", key, worker)

        ts = self.tasks.get(key)
        if ts is None:
            return {}
        ws = self.workers[worker]
        ts.metadata.update(kwargs["metadata"])

        if ts.state == "processing":
            recommendations = self.transition(key, "memory", worker=worker, **kwargs)

            if ts.state == "memory":
                assert ws in ts.who_has
        else:
            logger.debug(
                "Received already computed task, worker: %s, state: %s"
                ", key: %s, who_has: %s",
                worker,
                ts.state,
                key,
                ts.who_has,
            )
            if ws not in ts.who_has:
                self.worker_send(worker, {"op": "release-task", "key": key})
            recommendations = {}

        return recommendations

    def stimulus_task_erred(
        self, key=None, worker=None, exception=None, traceback=None, **kwargs
    ):
        """ Mark that a task has erred on a particular worker """
        logger.debug("Stimulus task erred %s, %s", key, worker)

        ts = self.tasks.get(key)
        if ts is None:
            return {}

        if ts.state == "processing":
            retries = ts.retries
            if retries > 0:
                ts.retries = retries - 1
                recommendations = self.transition(key, "waiting")
            else:
                recommendations = self.transition(
                    key,
                    "erred",
                    cause=key,
                    exception=exception,
                    traceback=traceback,
                    worker=worker,
                    **kwargs,
                )
        else:
            recommendations = {}

        return recommendations

    def stimulus_missing_data(
        self, cause=None, key=None, worker=None, ensure=True, **kwargs
    ):
        """ Mark that certain keys have gone missing.  Recover. """
        with log_errors():
            logger.debug("Stimulus missing data %s, %s", key, worker)

            ts = self.tasks.get(key)
            if ts is None or ts.state == "memory":
                return {}
            cts = self.tasks.get(cause)

            recommendations = {}

            if cts is not None and cts.state == "memory":  # couldn't find this
                for ws in cts.who_has:  # TODO: this behavior is extreme
                    ws.has_what.remove(cts)
                    ws.nbytes -= cts.get_nbytes()
                cts.who_has.clear()
                recommendations[cause] = "released"

            if key:
                recommendations[key] = "released"

            self.transitions(recommendations)

            if self.validate:
                assert cause not in self.who_has

            return {}

    def stimulus_retry(self, comm=None, keys=None, client=None):
        logger.info("Client %s requests to retry %d keys", client, len(keys))
        if client:
            self.log_event(client, {"action": "retry", "count": len(keys)})

        stack = list(keys)
        seen = set()
        roots = []
        while stack:
            key = stack.pop()
            seen.add(key)
            erred_deps = [
                dts.key for dts in self.tasks[key].dependencies if dts.state == "erred"
            ]
            if erred_deps:
                stack.extend(erred_deps)
            else:
                roots.append(key)

        recommendations = {key: "waiting" for key in roots}
        self.transitions(recommendations)

        if self.validate:
            for key in seen:
                assert not self.tasks[key].exception_blame

        return tuple(seen)

    async def remove_worker(self, comm=None, address=None, safe=False, close=True):
        """
        Remove worker from cluster

        We do this when a worker reports that it plans to leave or when it
        appears to be unresponsive.  This may send its tasks back to a released
        state.
        """
        with log_errors():
            if self.status == Status.closed:
                return

            address = self.coerce_address(address)

            if address not in self.workers:
                return "already-removed"

            host = get_address_host(address)

            ws = self.workers[address]

            self.log_event(
                ["all", address],
                {
                    "action": "remove-worker",
                    "worker": address,
                    "processing-tasks": dict(ws.processing),
                },
            )
            logger.info("Remove worker %s", ws)
            if close:
                with suppress(AttributeError, CommClosedError):
                    self.stream_comms[address].send({"op": "close", "report": False})

            self.remove_resources(address)

            self.host_info[host]["nthreads"] -= ws.nthreads
            self.host_info[host]["addresses"].remove(address)
            self.total_nthreads -= ws.nthreads

            if not self.host_info[host]["addresses"]:
                del self.host_info[host]

            self.rpc.remove(address)
            del self.stream_comms[address]
            del self.aliases[ws.name]
            self.idle.discard(ws)
            self.saturated.discard(ws)
            del self.workers[address]
            ws.status = "closed"
            self.total_occupancy -= ws.occupancy

            recommendations = {}

            for ts in list(ws.processing):
                k = ts.key
                recommendations[k] = "released"
                if not safe:
                    ts.suspicious += 1
                    ts.prefix.suspicious += 1
                    if ts.suspicious > self.allowed_failures:
                        del recommendations[k]
                        e = pickle.dumps(
                            KilledWorker(task=k, last_worker=ws.clean()), protocol=4
                        )
                        r = self.transition(k, "erred", exception=e, cause=k)
                        recommendations.update(r)
                        logger.info(
                            "Task %s marked as failed because %d workers died"
                            " while trying to run it",
                            ts.key,
                            self.allowed_failures,
                        )

            for ts in ws.has_what:
                ts.who_has.remove(ws)
                if not ts.who_has:
                    if ts.run_spec:
                        recommendations[ts.key] = "released"
                    else:  # pure data
                        recommendations[ts.key] = "forgotten"
            ws.has_what.clear()

            self.transitions(recommendations)

            for plugin in self.plugins[:]:
                try:
                    result = plugin.remove_worker(scheduler=self, worker=address)
                    if inspect.isawaitable(result):
                        await result
                except Exception as e:
                    logger.exception(e)

            if not self.workers:
                logger.info("Lost all workers")

            for w in self.workers:
                self.bandwidth_workers.pop((address, w), None)
                self.bandwidth_workers.pop((w, address), None)

            def remove_worker_from_events():
                # If the worker isn't registered anymore after the delay, remove from events
                if address not in self.workers and address in self.events:
                    del self.events[address]

            cleanup_delay = parse_timedelta(
                dask.config.get("distributed.scheduler.events-cleanup-delay")
            )
            self.loop.call_later(cleanup_delay, remove_worker_from_events)
            logger.debug("Removed worker %s", ws)

        return "OK"

    def stimulus_cancel(self, comm, keys=None, client=None, force=False):
        """ Stop execution on a list of keys """
        logger.info("Client %s requests to cancel %d keys", client, len(keys))
        if client:
            self.log_event(
                client, {"action": "cancel", "count": len(keys), "force": force}
            )
        for key in keys:
            self.cancel_key(key, client, force=force)

    def cancel_key(self, key, client, retries=5, force=False):
        """ Cancel a particular key and all dependents """
        # TODO: this should be converted to use the transition mechanism
        ts = self.tasks.get(key)
        try:
            cs = self.clients[client]
        except KeyError:
            return
        if ts is None or not ts.who_wants:  # no key yet, lets try again in a moment
            if retries:
                self.loop.call_later(
                    0.2, lambda: self.cancel_key(key, client, retries - 1)
                )
            return
        if force or ts.who_wants == {cs}:  # no one else wants this key
            for dts in list(ts.dependents):
                self.cancel_key(dts.key, client, force=force)
        logger.info("Scheduler cancels key %s.  Force=%s", key, force)
        self.report({"op": "cancelled-key", "key": key})
        clients = list(ts.who_wants) if force else [cs]
        for c in clients:
            self.client_releases_keys(keys=[key], client=c.client_key)

    def client_desires_keys(self, keys=None, client=None):
        cs = self.clients.get(client)
        if cs is None:
            # For publish, queues etc.
            cs = self.clients[client] = ClientState(client)
        for k in keys:
            ts = self.tasks.get(k)
            if ts is None:
                # For publish, queues etc.
                ts = self.new_task(k, None, "released")
            ts.who_wants.add(cs)
            cs.wants_what.add(ts)

            if ts.state in ("memory", "erred"):
                self.report_on_key(k, client=client)

    def client_releases_keys(self, keys=None, client=None):
        """ Remove keys from client desired list """
        logger.debug("Client %s releases keys: %s", client, keys)
        cs = self.clients[client]
        tasks2 = set()
        for key in list(keys):
            ts = self.tasks.get(key)
            if ts is not None and ts in cs.wants_what:
                cs.wants_what.remove(ts)
                s = ts.who_wants
                s.remove(cs)
                if not s:
                    tasks2.add(ts)

        recommendations = {}
        for ts in tasks2:
            if not ts.dependents:
                # No live dependents, can forget
                recommendations[ts.key] = "forgotten"
            elif ts.state != "erred" and not ts.waiters:
                recommendations[ts.key] = "released"

        self.transitions(recommendations)

    def client_heartbeat(self, client=None):
        """ Handle heartbeats from Client """
        self.clients[client].last_seen = time()

    ###################
    # Task Validation #
    ###################

    def validate_released(self, key):
        ts = self.tasks[key]
        assert ts.state == "released"
        assert not ts.waiters
        assert not ts.waiting_on
        assert not ts.who_has
        assert not ts.processing_on
        assert not any(ts in dts.waiters for dts in ts.dependencies)
        assert ts not in self.unrunnable

    def validate_waiting(self, key):
        ts = self.tasks[key]
        assert ts.waiting_on
        assert not ts.who_has
        assert not ts.processing_on
        assert ts not in self.unrunnable
        for dts in ts.dependencies:
            # We are waiting on a dependency iff it's not stored
            assert bool(dts.who_has) + (dts in ts.waiting_on) == 1
            assert ts in dts.waiters  # XXX even if dts.who_has?

    def validate_processing(self, key):
        ts = self.tasks[key]
        assert not ts.waiting_on
        ws = ts.processing_on
        assert ws
        assert ts in ws.processing
        assert not ts.who_has
        for dts in ts.dependencies:
            assert dts.who_has
            assert ts in dts.waiters

    def validate_memory(self, key):
        ts = self.tasks[key]
        assert ts.who_has
        assert not ts.processing_on
        assert not ts.waiting_on
        assert ts not in self.unrunnable
        for dts in ts.dependents:
            assert (dts in ts.waiters) == (dts.state in ("waiting", "processing"))
            assert ts not in dts.waiting_on

    def validate_no_worker(self, key):
        ts = self.tasks[key]
        assert ts in self.unrunnable
        assert not ts.waiting_on
        assert ts in self.unrunnable
        assert not ts.processing_on
        assert not ts.who_has
        for dts in ts.dependencies:
            assert dts.who_has

    def validate_erred(self, key):
        ts = self.tasks[key]
        assert ts.exception_blame
        assert not ts.who_has

    def validate_key(self, key, ts=None):
        try:
            if ts is None:
                ts = self.tasks.get(key)
            if ts is None:
                logger.debug("Key lost: %s", key)
            else:
                ts.validate()
                try:
                    func = getattr(self, "validate_" + ts.state.replace("-", "_"))
                except AttributeError:
                    logger.error(
                        "self.validate_%s not found", ts.state.replace("-", "_")
                    )
                else:
                    func(key)
        except Exception as e:
            logger.exception(e)
            if LOG_PDB:
                import pdb

                pdb.set_trace()
            raise

    def validate_state(self, allow_overlap=False):
        validate_state(self.tasks, self.workers, self.clients)

        if not (set(self.workers) == set(self.stream_comms)):
            raise ValueError("Workers not the same in all collections")

        for w, ws in self.workers.items():
            assert isinstance(w, str), (type(w), w)
            assert isinstance(ws, WorkerState), (type(ws), ws)
            assert ws.address == w
            if not ws.processing:
                assert not ws.occupancy
                assert ws in self.idle

        for k, ts in self.tasks.items():
            assert isinstance(ts, TaskState), (type(ts), ts)
            assert ts.key == k
            self.validate_key(k, ts)

        for c, cs in self.clients.items():
            # client=None is often used in tests...
            assert c is None or isinstance(c, str), (type(c), c)
            assert isinstance(cs, ClientState), (type(cs), cs)
            assert cs.client_key == c

        a = {w: ws.nbytes for w, ws in self.workers.items()}
        b = {
            w: sum(ts.get_nbytes() for ts in ws.has_what)
            for w, ws in self.workers.items()
        }
        assert a == b, (a, b)

        actual_total_occupancy = 0
        for worker, ws in self.workers.items():
            assert abs(sum(ws.processing.values()) - ws.occupancy) < 1e-8
            actual_total_occupancy += ws.occupancy

        assert abs(actual_total_occupancy - self.total_occupancy) < 1e-8, (
            actual_total_occupancy,
            self.total_occupancy,
        )

    ###################
    # Manage Messages #
    ###################

    def report(self, msg, ts=None, client=None):
        """
        Publish updates to all listening Queues and Comms

        If the message contains a key then we only send the message to those
        comms that care about the key.
        """
        comms = set()
        if client is not None:
            try:
                comms.add(self.client_comms[client])
            except KeyError:
                pass

        if ts is None and "key" in msg:
            ts = self.tasks.get(msg["key"])
        if ts is None:
            # Notify all clients
            comms |= set(self.client_comms.values())
        else:
            # Notify clients interested in key
            comms |= {
                self.client_comms[c.client_key]
                for c in ts.who_wants
                if c.client_key in self.client_comms
            }
        for c in comms:
            try:
                c.send(msg)
                # logger.debug("Scheduler sends message to client %s", msg)
            except CommClosedError:
                if self.status == Status.running:
                    logger.critical("Tried writing to closed comm: %s", msg)

    async def add_client(self, comm, client=None, versions=None):
        """Add client to network

        We listen to all future messages from this Comm.
        """
        assert client is not None
        comm.name = "Scheduler->Client"
        logger.info("Receive client connection: %s", client)
        self.log_event(["all", client], {"action": "add-client", "client": client})
        self.clients[client] = ClientState(client, versions=versions)

        for plugin in self.plugins[:]:
            try:
                plugin.add_client(scheduler=self, client=client)
            except Exception as e:
                logger.exception(e)

        try:
            bcomm = BatchedSend(interval="2ms", loop=self.loop)
            bcomm.start(comm)
            self.client_comms[client] = bcomm
            msg = {"op": "stream-start"}
            version_warning = version_module.error_message(
                version_module.get_versions(),
                {w: ws.versions for w, ws in self.workers.items()},
                versions,
            )
            msg.update(version_warning)
            bcomm.send(msg)

            try:
                await self.handle_stream(comm=comm, extra={"client": client})
            finally:
                self.remove_client(client=client)
                logger.debug("Finished handling client %s", client)
        finally:
            if not comm.closed():
                self.client_comms[client].send({"op": "stream-closed"})
            try:
                if not shutting_down():
                    await self.client_comms[client].close()
                    del self.client_comms[client]
                    if self.status == Status.running:
                        logger.info("Close client connection: %s", client)
            except TypeError:  # comm becomes None during GC
                pass

    def remove_client(self, client=None):
        """ Remove client from network """
        if self.status == Status.running:
            logger.info("Remove client %s", client)
        self.log_event(["all", client], {"action": "remove-client", "client": client})
        try:
            cs = self.clients[client]
        except KeyError:
            # XXX is this a legitimate condition?
            pass
        else:
            self.client_releases_keys(
                keys=[ts.key for ts in cs.wants_what], client=cs.client_key
            )
            del self.clients[client]

            for plugin in self.plugins[:]:
                try:
                    plugin.remove_client(scheduler=self, client=client)
                except Exception as e:
                    logger.exception(e)

        def remove_client_from_events():
            # If the client isn't registered anymore after the delay, remove from events
            if client not in self.clients and client in self.events:
                del self.events[client]

        cleanup_delay = parse_timedelta(
            dask.config.get("distributed.scheduler.events-cleanup-delay")
        )
        self.loop.call_later(cleanup_delay, remove_client_from_events)

    def send_task_to_worker(self, worker, key):
        """ Send a single computational task to a worker """
        try:
            ts = self.tasks[key]

            msg = {
                "op": "compute-task",
                "key": key,
                "priority": ts.priority,
                "duration": self.get_task_duration(ts),
            }
            if ts.resource_restrictions:
                msg["resource_restrictions"] = ts.resource_restrictions
            if ts.actor:
                msg["actor"] = True

            deps = ts.dependencies
            if deps:
                msg["who_has"] = {
                    dep.key: [ws.address for ws in dep.who_has] for dep in deps
                }
                msg["nbytes"] = {dep.key: dep.nbytes for dep in deps}

            if self.validate and deps:
                assert all(msg["who_has"].values())

            task = ts.run_spec
            if type(task) is dict:
                msg.update(task)
            else:
                msg["task"] = task

            self.worker_send(worker, msg)
        except Exception as e:
            logger.exception(e)
            if LOG_PDB:
                import pdb

                pdb.set_trace()
            raise

    def handle_uncaught_error(self, **msg):
        logger.exception(clean_exception(**msg)[1])

    def handle_task_finished(self, key=None, worker=None, **msg):
        if worker not in self.workers:
            return
        validate_key(key)
        r = self.stimulus_task_finished(key=key, worker=worker, **msg)
        self.transitions(r)

    def handle_task_erred(self, key=None, **msg):
        r = self.stimulus_task_erred(key=key, **msg)
        self.transitions(r)

    def handle_release_data(self, key=None, worker=None, client=None, **msg):
        ts = self.tasks.get(key)
        if ts is None:
            return
        ws = self.workers[worker]
        if ts.processing_on != ws:
            return
        r = self.stimulus_missing_data(key=key, ensure=False, **msg)
        self.transitions(r)

    def handle_missing_data(self, key=None, errant_worker=None, **kwargs):
        logger.debug("handle missing data key=%s worker=%s", key, errant_worker)
        self.log.append(("missing", key, errant_worker))

        ts = self.tasks.get(key)
        if ts is None or not ts.who_has:
            return
        if errant_worker in self.workers:
            ws = self.workers[errant_worker]
            if ws in ts.who_has:
                ts.who_has.remove(ws)
                ws.has_what.remove(ts)
                ws.nbytes -= ts.get_nbytes()
        if not ts.who_has:
            if ts.run_spec:
                self.transitions({key: "released"})
            else:
                self.transitions({key: "forgotten"})

    def release_worker_data(self, comm=None, keys=None, worker=None):
        ws = self.workers[worker]
        tasks = {self.tasks[k] for k in keys}
        removed_tasks = tasks & ws.has_what
        ws.has_what -= removed_tasks

        recommendations = {}
        for ts in removed_tasks:
            ws.nbytes -= ts.get_nbytes()
            wh = ts.who_has
            wh.remove(ws)
            if not wh:
                recommendations[ts.key] = "released"
        if recommendations:
            self.transitions(recommendations)

    def handle_long_running(self, key=None, worker=None, compute_duration=None):
        """A task has seceded from the thread pool

        We stop the task from being stolen in the future, and change task
        duration accounting as if the task has stopped.
        """
        ts = self.tasks[key]
        if "stealing" in self.extensions:
            self.extensions["stealing"].remove_key_from_stealable(ts)

        ws = ts.processing_on
        if ws is None:
            logger.debug("Received long-running signal from duplicate task. Ignoring.")
            return

        if compute_duration:
            old_duration = ts.prefix.duration_average or 0
            new_duration = compute_duration
            if not old_duration:
                avg_duration = new_duration
            else:
                avg_duration = 0.5 * old_duration + 0.5 * new_duration

            ts.prefix.duration_average = avg_duration

        ws.occupancy -= ws.processing[ts]
        self.total_occupancy -= ws.processing[ts]
        ws.processing[ts] = 0
        self.check_idle_saturated(ws)

    async def handle_worker(self, comm=None, worker=None):
        """
        Listen to responses from a single worker

        This is the main loop for scheduler-worker interaction

        See Also
        --------
        Scheduler.handle_client: Equivalent coroutine for clients
        """
        comm.name = "Scheduler connection to worker"
        worker_comm = self.stream_comms[worker]
        worker_comm.start(comm)
        logger.info("Starting worker compute stream, %s", worker)
        try:
            await self.handle_stream(comm=comm, extra={"worker": worker})
        finally:
            if worker in self.stream_comms:
                worker_comm.abort()
                await self.remove_worker(address=worker)

    def add_plugin(self, plugin=None, idempotent=False, **kwargs):
        """
        Add external plugin to scheduler

        See https://distributed.readthedocs.io/en/latest/plugins.html
        """
        if isinstance(plugin, type):
            plugin = plugin(self, **kwargs)

        if idempotent and any(isinstance(p, type(plugin)) for p in self.plugins):
            return

        self.plugins.append(plugin)

    def remove_plugin(self, plugin):
        """ Remove external plugin from scheduler """
        self.plugins.remove(plugin)

    def worker_send(self, worker, msg):
        """Send message to worker

        This also handles connection failures by adding a callback to remove
        the worker on the next cycle.
        """
        try:
            self.stream_comms[worker].send(msg)
        except (CommClosedError, AttributeError):
            self.loop.add_callback(self.remove_worker, address=worker)

    ############################
    # Less common interactions #
    ############################

    async def scatter(
        self,
        comm=None,
        data=None,
        workers=None,
        client=None,
        broadcast=False,
        timeout=2,
    ):
        """Send data out to workers

        See also
        --------
        Scheduler.broadcast:
        """
        start = time()
        while not self.workers:
            await asyncio.sleep(0.2)
            if time() > start + timeout:
                raise TimeoutError("No workers found")

        if workers is None:
            nthreads = {w: ws.nthreads for w, ws in self.workers.items()}
        else:
            workers = [self.coerce_address(w) for w in workers]
            nthreads = {w: self.workers[w].nthreads for w in workers}

        assert isinstance(data, dict)

        keys, who_has, nbytes = await scatter_to_workers(
            nthreads, data, rpc=self.rpc, report=False
        )

        self.update_data(who_has=who_has, nbytes=nbytes, client=client)

        if broadcast:
            if broadcast == True:  # noqa: E712
                n = len(nthreads)
            else:
                n = broadcast
            await self.replicate(keys=keys, workers=workers, n=n)

        self.log_event(
            [client, "all"], {"action": "scatter", "client": client, "count": len(data)}
        )
        return keys

    async def gather(self, comm=None, keys=None, serializers=None):
        """ Collect data in from workers """
        keys = list(keys)
        who_has = {}
        for key in keys:
            ts = self.tasks.get(key)
            if ts is not None:
                who_has[key] = [ws.address for ws in ts.who_has]
            else:
                who_has[key] = []

        data, missing_keys, missing_workers = await gather_from_workers(
            who_has, rpc=self.rpc, close=False, serializers=serializers
        )
        if not missing_keys:
            result = {"status": "OK", "data": data}
        else:
            missing_states = [
                (self.tasks[key].state if key in self.tasks else None)
                for key in missing_keys
            ]
            logger.exception(
                "Couldn't gather keys %s state: %s workers: %s",
                missing_keys,
                missing_states,
                missing_workers,
            )
            result = {"status": "error", "keys": missing_keys}
            with log_errors():
                # Remove suspicious workers from the scheduler but allow them to
                # reconnect.
                await asyncio.gather(
                    *[
                        self.remove_worker(address=worker, close=False)
                        for worker in missing_workers
                    ]
                )
                for key, workers in missing_keys.items():
                    # Task may already be gone if it was held by a
                    # `missing_worker`
                    ts = self.tasks.get(key)
                    logger.exception(
                        "Workers don't have promised key: %s, %s",
                        str(workers),
                        str(key),
                    )
                    if not workers or ts is None:
                        continue
                    for worker in workers:
                        ws = self.workers.get(worker)
                        if ws is not None and ts in ws.has_what:
                            ws.has_what.remove(ts)
                            ts.who_has.remove(ws)
                            ws.nbytes -= ts.get_nbytes()
                            self.transitions({key: "released"})

        self.log_event("all", {"action": "gather", "count": len(keys)})
        return result

    def clear_task_state(self):
        # XXX what about nested state such as ClientState.wants_what
        # (see also fire-and-forget...)
        logger.info("Clear task state")
        for collection in self._task_state_collections:
            collection.clear()

    async def restart(self, client=None, timeout=3):
        """ Restart all workers.  Reset local state. """
        with log_errors():

            n_workers = len(self.workers)

            logger.info("Send lost future signal to clients")
            for cs in self.clients.values():
                self.client_releases_keys(
                    keys=[ts.key for ts in cs.wants_what], client=cs.client_key
                )

            nannies = {addr: ws.nanny for addr, ws in self.workers.items()}

            for addr in list(self.workers):
                try:
                    # Ask the worker to close if it doesn't have a nanny,
                    # otherwise the nanny will kill it anyway
                    await self.remove_worker(address=addr, close=addr not in nannies)
                except Exception as e:
                    logger.info(
                        "Exception while restarting.  This is normal", exc_info=True
                    )

            self.clear_task_state()

            for plugin in self.plugins[:]:
                try:
                    plugin.restart(self)
                except Exception as e:
                    logger.exception(e)

            logger.debug("Send kill signal to nannies: %s", nannies)

            nannies = [
                rpc(nanny_address, connection_args=self.connection_args)
                for nanny_address in nannies.values()
                if nanny_address is not None
            ]

            resps = All(
                [
                    nanny.restart(
                        close=True, timeout=timeout * 0.8, executor_wait=False
                    )
                    for nanny in nannies
                ]
            )
            try:
                resps = await asyncio.wait_for(resps, timeout)
            except TimeoutError:
                logger.error(
                    "Nannies didn't report back restarted within "
                    "timeout.  Continuuing with restart process"
                )
            else:
                if not all(resp == "OK" for resp in resps):
                    logger.error(
                        "Not all workers responded positively: %s", resps, exc_info=True
                    )
            finally:
                await asyncio.gather(*[nanny.close_rpc() for nanny in nannies])

            self.clear_task_state()

            with suppress(AttributeError):
                for c in self._worker_coroutines:
                    c.cancel()

            self.log_event([client, "all"], {"action": "restart", "client": client})
            start = time()
            while time() < start + 10 and len(self.workers) < n_workers:
                await asyncio.sleep(0.01)

            self.report({"op": "restart"})

    async def broadcast(
        self,
        comm=None,
        msg=None,
        workers=None,
        hosts=None,
        nanny=False,
        serializers=None,
    ):
        """ Broadcast message to workers, return all results """
        if workers is None or workers is True:
            if hosts is None:
                workers = list(self.workers)
            else:
                workers = []
        if hosts is not None:
            for host in hosts:
                if host in self.host_info:
                    workers.extend(self.host_info[host]["addresses"])
        # TODO replace with worker_list

        if nanny:
            addresses = [self.workers[w].nanny for w in workers]
        else:
            addresses = workers

        async def send_message(addr):
            comm = await self.rpc.connect(addr)
            comm.name = "Scheduler Broadcast"
            try:
                resp = await send_recv(comm, close=True, serializers=serializers, **msg)
            finally:
                self.rpc.reuse(addr, comm)
            return resp

        results = await All(
            [send_message(address) for address in addresses if address is not None]
        )

        return dict(zip(workers, results))

    async def proxy(self, comm=None, msg=None, worker=None, serializers=None):
        """ Proxy a communication through the scheduler to some other worker """
        d = await self.broadcast(
            comm=comm, msg=msg, workers=[worker], serializers=serializers
        )
        return d[worker]

    async def _delete_worker_data(self, worker_address, keys):
        """Delete data from a worker and update the corresponding worker/task states

        Parameters
        ----------
        worker_address: str
            Worker address to delete keys from
        keys: List[str]
            List of keys to delete on the specified worker
        """
        await retry_operation(
            self.rpc(addr=worker_address).delete_data, keys=list(keys), report=False
        )

        ws = self.workers[worker_address]
        tasks = {self.tasks[key] for key in keys}
        ws.has_what -= tasks
        for ts in tasks:
            ts.who_has.remove(ws)
            ws.nbytes -= ts.get_nbytes()
        self.log_event(ws.address, {"action": "remove-worker-data", "keys": keys})

    async def rebalance(self, comm=None, keys=None, workers=None):
        """Rebalance keys so that each worker stores roughly equal bytes

        **Policy**

        This orders the workers by what fraction of bytes of the existing keys
        they have.  It walks down this list from most-to-least.  At each worker
        it sends the largest results it can find and sends them to the least
        occupied worker until either the sender or the recipient are at the
        average expected load.
        """
        with log_errors():
            async with self._lock:
                if keys:
                    tasks = {self.tasks[k] for k in keys}
                    missing_data = [ts.key for ts in tasks if not ts.who_has]
                    if missing_data:
                        return {"status": "missing-data", "keys": missing_data}
                else:
                    tasks = set(self.tasks.values())

                if workers:
                    workers = {self.workers[w] for w in workers}
                    workers_by_task = {ts: ts.who_has & workers for ts in tasks}
                else:
                    workers = set(self.workers.values())
                    workers_by_task = {ts: ts.who_has for ts in tasks}

                tasks_by_worker = {ws: set() for ws in workers}

                for k, v in workers_by_task.items():
                    for vv in v:
                        tasks_by_worker[vv].add(k)

                worker_bytes = {
                    ws: sum(ts.get_nbytes() for ts in v)
                    for ws, v in tasks_by_worker.items()
                }

                avg = sum(worker_bytes.values()) / len(worker_bytes)

                sorted_workers = list(
                    map(first, sorted(worker_bytes.items(), key=second, reverse=True))
                )

                recipients = iter(reversed(sorted_workers))
                recipient = next(recipients)
                msgs = []  # (sender, recipient, key)
                for sender in sorted_workers[: len(workers) // 2]:
                    sender_keys = {
                        ts: ts.get_nbytes() for ts in tasks_by_worker[sender]
                    }
                    sender_keys = iter(
                        sorted(sender_keys.items(), key=second, reverse=True)
                    )

                    try:
                        while worker_bytes[sender] > avg:
                            while (
                                worker_bytes[recipient] < avg
                                and worker_bytes[sender] > avg
                            ):
                                ts, nb = next(sender_keys)
                                if ts not in tasks_by_worker[recipient]:
                                    tasks_by_worker[recipient].add(ts)
                                    # tasks_by_worker[sender].remove(ts)
                                    msgs.append((sender, recipient, ts))
                                    worker_bytes[sender] -= nb
                                    worker_bytes[recipient] += nb
                            if worker_bytes[sender] > avg:
                                recipient = next(recipients)
                    except StopIteration:
                        break

                to_recipients = defaultdict(lambda: defaultdict(list))
                to_senders = defaultdict(list)
                for sender, recipient, ts in msgs:
                    to_recipients[recipient.address][ts.key].append(sender.address)
                    to_senders[sender.address].append(ts.key)

                result = await asyncio.gather(
                    *(
                        retry_operation(self.rpc(addr=r).gather, who_has=v)
                        for r, v in to_recipients.items()
                    )
                )
                for r, v in to_recipients.items():
                    self.log_event(r, {"action": "rebalance", "who_has": v})

                self.log_event(
                    "all",
                    {
                        "action": "rebalance",
                        "total-keys": len(tasks),
                        "senders": valmap(len, to_senders),
                        "recipients": valmap(len, to_recipients),
                        "moved_keys": len(msgs),
                    },
                )

                if not all(r["status"] == "OK" for r in result):
                    return {
                        "status": "missing-data",
                        "keys": tuple(
                            concat(
                                r["keys"].keys()
                                for r in result
                                if r["status"] == "missing-data"
                            )
                        ),
                    }

                for sender, recipient, ts in msgs:
                    assert ts.state == "memory"
                    ts.who_has.add(recipient)
                    recipient.has_what.add(ts)
                    recipient.nbytes += ts.get_nbytes()
                    self.log.append(
                        ("rebalance", ts.key, time(), sender.address, recipient.address)
                    )

                await asyncio.gather(
                    *(self._delete_worker_data(r, v) for r, v in to_senders.items())
                )

                return {"status": "OK"}

    async def replicate(
        self,
        comm=None,
        keys=None,
        n=None,
        workers=None,
        branching_factor=2,
        delete=True,
        lock=True,
    ):
        """Replicate data throughout cluster

        This performs a tree copy of the data throughout the network
        individually on each piece of data.

        Parameters
        ----------
        keys: Iterable
            list of keys to replicate
        n: int
            Number of replications we expect to see within the cluster
        branching_factor: int, optional
            The number of workers that can copy data in each generation.
            The larger the branching factor, the more data we copy in
            a single step, but the more a given worker risks being
            swamped by data requests.

        See also
        --------
        Scheduler.rebalance
        """
        assert branching_factor > 0
        async with self._lock if lock else empty_context:
            workers = {self.workers[w] for w in self.workers_list(workers)}
            if n is None:
                n = len(workers)
            else:
                n = min(n, len(workers))
            if n == 0:
                raise ValueError("Can not use replicate to delete data")

            tasks = {self.tasks[k] for k in keys}
            missing_data = [ts.key for ts in tasks if not ts.who_has]
            if missing_data:
                return {"status": "missing-data", "keys": missing_data}

            # Delete extraneous data
            if delete:
                del_worker_tasks = defaultdict(set)
                for ts in tasks:
                    del_candidates = ts.who_has & workers
                    if len(del_candidates) > n:
                        for ws in random.sample(
                            del_candidates, len(del_candidates) - n
                        ):
                            del_worker_tasks[ws].add(ts)

                await asyncio.gather(
                    *(
                        self._delete_worker_data(ws.address, [t.key for t in tasks])
                        for ws, tasks in del_worker_tasks.items()
                    )
                )

            # Copy not-yet-filled data
            while tasks:
                gathers = defaultdict(dict)
                for ts in list(tasks):
                    if ts.state == "forgotten":
                        # task is no longer needed by any client or dependant task
                        tasks.remove(ts)
                        continue
                    n_missing = n - len(ts.who_has & workers)
                    if n_missing <= 0:
                        # Already replicated enough
                        tasks.remove(ts)
                        continue

                    count = min(n_missing, branching_factor * len(ts.who_has))
                    assert count > 0

                    for ws in random.sample(workers - ts.who_has, count):
                        gathers[ws.address][ts.key] = [
                            wws.address for wws in ts.who_has
                        ]

                results = await asyncio.gather(
                    *(
                        retry_operation(self.rpc(addr=w).gather, who_has=who_has)
                        for w, who_has in gathers.items()
                    )
                )
                for w, v in zip(gathers, results):
                    if v["status"] == "OK":
                        self.add_keys(worker=w, keys=list(gathers[w]))
                    else:
                        logger.warning("Communication failed during replication: %s", v)

                    self.log_event(w, {"action": "replicate-add", "keys": gathers[w]})

            self.log_event(
                "all",
                {
                    "action": "replicate",
                    "workers": list(workers),
                    "key-count": len(keys),
                    "branching-factor": branching_factor,
                },
            )

    def workers_to_close(
        self,
        comm=None,
        memory_ratio=None,
        n=None,
        key=None,
        minimum=None,
        target=None,
        attribute="address",
    ):
        """
        Find workers that we can close with low cost

        This returns a list of workers that are good candidates to retire.
        These workers are not running anything and are storing
        relatively little data relative to their peers.  If all workers are
        idle then we still maintain enough workers to have enough RAM to store
        our data, with a comfortable buffer.

        This is for use with systems like ``distributed.deploy.adaptive``.

        Parameters
        ----------
        memory_factor: Number
            Amount of extra space we want to have for our stored data.
            Defaults two 2, or that we want to have twice as much memory as we
            currently have data.
        n: int
            Number of workers to close
        minimum: int
            Minimum number of workers to keep around
        key: Callable(WorkerState)
            An optional callable mapping a WorkerState object to a group
            affiliation.  Groups will be closed together.  This is useful when
            closing workers must be done collectively, such as by hostname.
        target: int
            Target number of workers to have after we close
        attribute : str
            The attribute of the WorkerState object to return, like "address"
            or "name".  Defaults to "address".

        Examples
        --------
        >>> scheduler.workers_to_close()
        ['tcp://192.168.0.1:1234', 'tcp://192.168.0.2:1234']

        Group workers by hostname prior to closing

        >>> scheduler.workers_to_close(key=lambda ws: ws.host)
        ['tcp://192.168.0.1:1234', 'tcp://192.168.0.1:4567']

        Remove two workers

        >>> scheduler.workers_to_close(n=2)

        Keep enough workers to have twice as much memory as we we need.

        >>> scheduler.workers_to_close(memory_ratio=2)

        Returns
        -------
        to_close: list of worker addresses that are OK to close

        See Also
        --------
        Scheduler.retire_workers
        """
        if target is not None and n is None:
            n = len(self.workers) - target
        if n is not None:
            if n < 0:
                n = 0
            target = len(self.workers) - n

        if n is None and memory_ratio is None:
            memory_ratio = 2

        with log_errors():
            if not n and all(ws.processing for ws in self.workers.values()):
                return []

            if key is None:
                key = lambda ws: ws.address
            if isinstance(key, bytes) and dask.config.get(
                "distributed.scheduler.pickle"
            ):
                key = pickle.loads(key)

            groups = groupby(key, self.workers.values())

            limit_bytes = {
                k: sum(ws.memory_limit for ws in v) for k, v in groups.items()
            }
            group_bytes = {k: sum(ws.nbytes for ws in v) for k, v in groups.items()}

            limit = sum(limit_bytes.values())
            total = sum(group_bytes.values())

            def _key(group):
                is_idle = not any(ws.processing for ws in groups[group])
                bytes = -group_bytes[group]
                return (is_idle, bytes)

            idle = sorted(groups, key=_key)

            to_close = []
            n_remain = len(self.workers)

            while idle:
                group = idle.pop()
                if n is None and any(ws.processing for ws in groups[group]):
                    break

                if minimum and n_remain - len(groups[group]) < minimum:
                    break

                limit -= limit_bytes[group]

                if (n is not None and n_remain - len(groups[group]) >= target) or (
                    memory_ratio is not None and limit >= memory_ratio * total
                ):
                    to_close.append(group)
                    n_remain -= len(groups[group])

                else:
                    break

            result = [getattr(ws, attribute) for g in to_close for ws in groups[g]]
            if result:
                logger.debug("Suggest closing workers: %s", result)

            return result

    async def retire_workers(
        self,
        comm=None,
        workers=None,
        remove=True,
        close_workers=False,
        names=None,
        lock=True,
        **kwargs,
    ):
        """Gracefully retire workers from cluster

        Parameters
        ----------
        workers: list (optional)
            List of worker addresses to retire.
            If not provided we call ``workers_to_close`` which finds a good set
        workers_names: list (optional)
            List of worker names to retire.
        remove: bool (defaults to True)
            Whether or not to remove the worker metadata immediately or else
            wait for the worker to contact us
        close_workers: bool (defaults to False)
            Whether or not to actually close the worker explicitly from here.
            Otherwise we expect some external job scheduler to finish off the
            worker.
        **kwargs: dict
            Extra options to pass to workers_to_close to determine which
            workers we should drop

        Returns
        -------
        Dictionary mapping worker ID/address to dictionary of information about
        that worker for each retired worker.

        See Also
        --------
        Scheduler.workers_to_close
        """
        with log_errors():
            async with self._lock if lock else empty_context:
                if names is not None:
                    if names:
                        logger.info("Retire worker names %s", names)
                    names = set(map(str, names))
                    workers = [
                        ws.address
                        for ws in self.workers.values()
                        if str(ws.name) in names
                    ]
                if workers is None:
                    while True:
                        try:
                            workers = self.workers_to_close(**kwargs)
                            if workers:
                                workers = await self.retire_workers(
                                    workers=workers,
                                    remove=remove,
                                    close_workers=close_workers,
                                    lock=False,
                                )
                            return workers
                        except KeyError:  # keys left during replicate
                            pass
                workers = {self.workers[w] for w in workers if w in self.workers}
                if not workers:
                    return []
                logger.info("Retire workers %s", workers)

                # Keys orphaned by retiring those workers
                keys = set.union(*[w.has_what for w in workers])
                keys = {ts.key for ts in keys if ts.who_has.issubset(workers)}

                other_workers = set(self.workers.values()) - workers
                if keys:
                    if other_workers:
                        logger.info("Moving %d keys to other workers", len(keys))
                        await self.replicate(
                            keys=keys,
                            workers=[ws.address for ws in other_workers],
                            n=1,
                            delete=False,
                            lock=False,
                        )
                    else:
                        return []

                worker_keys = {ws.address: ws.identity() for ws in workers}
                if close_workers and worker_keys:
                    await asyncio.gather(
                        *[self.close_worker(worker=w, safe=True) for w in worker_keys]
                    )
                if remove:
                    await asyncio.gather(
                        *[self.remove_worker(address=w, safe=True) for w in worker_keys]
                    )

                self.log_event(
                    "all",
                    {
                        "action": "retire-workers",
                        "workers": worker_keys,
                        "moved-keys": len(keys),
                    },
                )
                self.log_event(list(worker_keys), {"action": "retired"})

                return worker_keys

    def add_keys(self, comm=None, worker=None, keys=()):
        """
        Learn that a worker has certain keys

        This should not be used in practice and is mostly here for legacy
        reasons.  However, it is sent by workers from time to time.
        """
        if worker not in self.workers:
            return "not found"
        ws = self.workers[worker]
        for key in keys:
            ts = self.tasks.get(key)
            if ts is not None and ts.state == "memory":
                if ts not in ws.has_what:
                    ws.nbytes += ts.get_nbytes()
                    ws.has_what.add(ts)
                    ts.who_has.add(ws)
            else:
                self.worker_send(
                    worker, {"op": "delete-data", "keys": [key], "report": False}
                )

        return "OK"

    def update_data(
        self, comm=None, who_has=None, nbytes=None, client=None, serializers=None
    ):
        """
        Learn that new data has entered the network from an external source

        See Also
        --------
        Scheduler.mark_key_in_memory
        """
        with log_errors():
            who_has = {
                k: [self.coerce_address(vv) for vv in v] for k, v in who_has.items()
            }
            logger.debug("Update data %s", who_has)

            for key, workers in who_has.items():
                ts = self.tasks.get(key)
                if ts is None:
                    ts = self.new_task(key, None, "memory")
                ts.state = "memory"
                if key in nbytes:
                    ts.set_nbytes(nbytes[key])
                for w in workers:
                    ws = self.workers[w]
                    if ts not in ws.has_what:
                        ws.nbytes += ts.get_nbytes()
                        ws.has_what.add(ts)
                        ts.who_has.add(ws)
                self.report(
                    {"op": "key-in-memory", "key": key, "workers": list(workers)}
                )

            if client:
                self.client_desires_keys(keys=list(who_has), client=client)

    def report_on_key(self, key=None, ts=None, client=None):
        assert (key is None) + (ts is None) == 1, (key, ts)
        if ts is None:
            try:
                ts = self.tasks[key]
            except KeyError:
                self.report({"op": "cancelled-key", "key": key}, client=client)
                return
        else:
            key = ts.key
        if ts.state == "forgotten":
            self.report({"op": "cancelled-key", "key": key}, ts=ts, client=client)
        elif ts.state == "memory":
            self.report({"op": "key-in-memory", "key": key}, ts=ts, client=client)
        elif ts.state == "erred":
            failing_ts = ts.exception_blame
            self.report(
                {
                    "op": "task-erred",
                    "key": key,
                    "exception": failing_ts.exception,
                    "traceback": failing_ts.traceback,
                },
                ts=ts,
                client=client,
            )

    async def feed(
        self, comm, function=None, setup=None, teardown=None, interval="1s", **kwargs
    ):
        """
        Provides a data Comm to external requester

        Caution: this runs arbitrary Python code on the scheduler.  This should
        eventually be phased out.  It is mostly used by diagnostics.
        """
        if not dask.config.get("distributed.scheduler.pickle"):
            logger.warn(
                "Tried to call 'feed' route with custom functions, but "
                "pickle is disallowed.  Set the 'distributed.scheduler.pickle'"
                "config value to True to use the 'feed' route (this is mostly "
                "commonly used with progress bars)"
            )
            return

        interval = parse_timedelta(interval)
        with log_errors():
            if function:
                function = pickle.loads(function)
            if setup:
                setup = pickle.loads(setup)
            if teardown:
                teardown = pickle.loads(teardown)
            state = setup(self) if setup else None
            if inspect.isawaitable(state):
                state = await state
            try:
                while self.status == Status.running:
                    if state is None:
                        response = function(self)
                    else:
                        response = function(self, state)
                    await comm.write(response)
                    await asyncio.sleep(interval)
            except (EnvironmentError, CommClosedError):
                pass
            finally:
                if teardown:
                    teardown(self, state)

    def subscribe_worker_status(self, comm=None):
        WorkerStatusPlugin(self, comm)
        ident = self.identity()
        for v in ident["workers"].values():
            del v["metrics"]
            del v["last_seen"]
        return ident

    def get_processing(self, comm=None, workers=None):
        if workers is not None:
            workers = set(map(self.coerce_address, workers))
            return {w: [ts.key for ts in self.workers[w].processing] for w in workers}
        else:
            return {
                w: [ts.key for ts in ws.processing] for w, ws in self.workers.items()
            }

    def get_who_has(self, comm=None, keys=None):
        if keys is not None:
            return {
                k: [ws.address for ws in self.tasks[k].who_has]
                if k in self.tasks
                else []
                for k in keys
            }
        else:
            return {
                key: [ws.address for ws in ts.who_has] for key, ts in self.tasks.items()
            }

    def get_has_what(self, comm=None, workers=None):
        if workers is not None:
            workers = map(self.coerce_address, workers)
            return {
                w: [ts.key for ts in self.workers[w].has_what]
                if w in self.workers
                else []
                for w in workers
            }
        else:
            return {w: [ts.key for ts in ws.has_what] for w, ws in self.workers.items()}

    def get_ncores(self, comm=None, workers=None):
        if workers is not None:
            workers = map(self.coerce_address, workers)
            return {w: self.workers[w].nthreads for w in workers if w in self.workers}
        else:
            return {w: ws.nthreads for w, ws in self.workers.items()}

    async def get_call_stack(self, comm=None, keys=None):
        if keys is not None:
            stack = list(keys)
            processing = set()
            while stack:
                key = stack.pop()
                ts = self.tasks[key]
                if ts.state == "waiting":
                    stack.extend(dts.key for dts in ts.dependencies)
                elif ts.state == "processing":
                    processing.add(ts)

            workers = defaultdict(list)
            for ts in processing:
                if ts.processing_on:
                    workers[ts.processing_on.address].append(ts.key)
        else:
            workers = {w: None for w in self.workers}

        if not workers:
            return {}

        results = await asyncio.gather(
            *(self.rpc(w).call_stack(keys=v) for w, v in workers.items())
        )
        response = {w: r for w, r in zip(workers, results) if r}
        return response

    def get_nbytes(self, comm=None, keys=None, summary=True):
        with log_errors():
            if keys is not None:
                result = {k: self.tasks[k].nbytes for k in keys}
            else:
                result = {
                    k: ts.nbytes
                    for k, ts in self.tasks.items()
                    if ts.nbytes is not None
                }

            if summary:
                out = defaultdict(lambda: 0)
                for k, v in result.items():
                    out[key_split(k)] += v
                result = dict(out)

            return result

    def get_comm_cost(self, ts, ws):
        """
        Get the estimated communication cost (in s.) to compute the task
        on the given worker.
        """
        return sum(dts.nbytes for dts in ts.dependencies - ws.has_what) / self.bandwidth

    def get_task_duration(self, ts, default=None):
        """
        Get the estimated computation cost of the given task
        (not including any communication cost).
        """
        duration = ts.prefix.duration_average
        if duration is None:
            self.unknown_durations[ts.prefix.name].add(ts)
            if default is None:
                default = parse_timedelta(
                    dask.config.get("distributed.scheduler.unknown-task-duration")
                )
            return default

        return duration

    def run_function(self, stream, function, args=(), kwargs={}, wait=True):
        """Run a function within this process

        See Also
        --------
        Client.run_on_scheduler:
        """
        from .worker import run

        self.log_event("all", {"action": "run-function", "function": function})
        return run(self, stream, function=function, args=args, kwargs=kwargs, wait=wait)

    def set_metadata(self, comm=None, keys=None, value=None):
        try:
            metadata = self.task_metadata
            for key in keys[:-1]:
                if key not in metadata or not isinstance(metadata[key], (dict, list)):
                    metadata[key] = dict()
                metadata = metadata[key]
            metadata[keys[-1]] = value
        except Exception as e:
            import pdb

            pdb.set_trace()

    def get_metadata(self, comm=None, keys=None, default=no_default):
        metadata = self.task_metadata
        for key in keys[:-1]:
            metadata = metadata[key]
        try:
            return metadata[keys[-1]]
        except KeyError:
            if default != no_default:
                return default
            else:
                raise

    def get_task_status(self, comm=None, keys=None):
        return {
            key: (self.tasks[key].state if key in self.tasks else None) for key in keys
        }

    def get_task_stream(self, comm=None, start=None, stop=None, count=None):
        from distributed.diagnostics.task_stream import TaskStreamPlugin

        self.add_plugin(TaskStreamPlugin, idempotent=True)
        ts = [p for p in self.plugins if isinstance(p, TaskStreamPlugin)][0]
        return ts.collect(start=start, stop=stop, count=count)

    def start_task_metadata(self, comm=None, name=None):
        plugin = CollectTaskMetaDataPlugin(scheduler=self, name=name)

        self.add_plugin(plugin)

    def stop_task_metadata(self, comm=None, name=None):
        plugins = [
            p
            for p in self.plugins
            if isinstance(p, CollectTaskMetaDataPlugin) and p.name == name
        ]
        if len(plugins) != 1:
            raise ValueError(
                "Expected to find exactly one CollectTaskMetaDataPlugin "
                f"with name {name} but found {len(plugins)}."
            )

        plugin = plugins[0]
        self.remove_plugin(plugin)
        return {"metadata": plugin.metadata, "state": plugin.state}

    async def register_worker_plugin(self, comm, plugin, name=None):
        """ Registers a setup function, and call it on every worker """
        self.worker_plugins.append({"plugin": plugin, "name": name})

        responses = await self.broadcast(
            msg=dict(op="plugin-add", plugin=plugin, name=name)
        )
        return responses

    #####################
    # State Transitions #
    #####################

    def _remove_from_processing(self, ts, send_worker_msg=None):
        """
        Remove *ts* from the set of processing tasks.
        """
        ws = ts.processing_on
        ts.processing_on = None
        w = ws.address
        if w in self.workers:  # may have been removed
            duration = ws.processing.pop(ts)
            if not ws.processing:
                self.total_occupancy -= ws.occupancy
                ws.occupancy = 0
            else:
                self.total_occupancy -= duration
                ws.occupancy -= duration
            self.check_idle_saturated(ws)
            self.release_resources(ts, ws)
            if send_worker_msg:
                self.worker_send(w, send_worker_msg)

    def _add_to_memory(
        self, ts, ws, recommendations, type=None, typename=None, **kwargs
    ):
        """
        Add *ts* to the set of in-memory tasks.
        """
        if self.validate:
            assert ts not in ws.has_what

        ts.who_has.add(ws)
        ws.has_what.add(ts)
        ws.nbytes += ts.get_nbytes()

        deps = ts.dependents
        if len(deps) > 1:
            deps = sorted(deps, key=operator.attrgetter("priority"), reverse=True)
        for dts in deps:
            s = dts.waiting_on
            if ts in s:
                s.discard(ts)
                if not s:  # new task ready to run
                    recommendations[dts.key] = "processing"

        for dts in ts.dependencies:
            s = dts.waiters
            s.discard(ts)
            if not s and not dts.who_wants:
                recommendations[dts.key] = "released"

        if not ts.waiters and not ts.who_wants:
            recommendations[ts.key] = "released"
        else:
            msg = {"op": "key-in-memory", "key": ts.key}
            if type is not None:
                msg["type"] = type
            self.report(msg)

        ts.state = "memory"
        ts.type = typename
        ts.group.types.add(typename)

        cs = self.clients["fire-and-forget"]
        if ts in cs.wants_what:
            self.client_releases_keys(client="fire-and-forget", keys=[ts.key])

    def transition_released_waiting(self, key):
        try:
            ts = self.tasks[key]

            if self.validate:
                assert ts.run_spec
                assert not ts.waiting_on
                assert not ts.who_has
                assert not ts.processing_on
                assert not any(dts.state == "forgotten" for dts in ts.dependencies)

            if ts.has_lost_dependencies:
                return {key: "forgotten"}

            ts.state = "waiting"

            recommendations = {}

            for dts in ts.dependencies:
                if dts.exception_blame:
                    ts.exception_blame = dts.exception_blame
                    recommendations[key] = "erred"
                    return recommendations

            for dts in ts.dependencies:
                dep = dts.key
                if not dts.who_has:
                    ts.waiting_on.add(dts)
                if dts.state == "released":
                    recommendations[dep] = "waiting"
                else:
                    dts.waiters.add(ts)

            ts.waiters = {dts for dts in ts.dependents if dts.state == "waiting"}

            if not ts.waiting_on:
                if self.workers:
                    recommendations[key] = "processing"
                else:
                    self.unrunnable.add(ts)
                    ts.state = "no-worker"

            return recommendations
        except Exception as e:
            logger.exception(e)
            if LOG_PDB:
                import pdb

                pdb.set_trace()
            raise

    def transition_no_worker_waiting(self, key):
        try:
            ts = self.tasks[key]

            if self.validate:
                assert ts in self.unrunnable
                assert not ts.waiting_on
                assert not ts.who_has
                assert not ts.processing_on

            self.unrunnable.remove(ts)

            if ts.has_lost_dependencies:
                return {key: "forgotten"}

            recommendations = {}

            for dts in ts.dependencies:
                dep = dts.key
                if not dts.who_has:
                    ts.waiting_on.add(dts)
                if dts.state == "released":
                    recommendations[dep] = "waiting"
                else:
                    dts.waiters.add(ts)

            ts.state = "waiting"

            if not ts.waiting_on:
                if self.workers:
                    recommendations[key] = "processing"
                else:
                    self.unrunnable.add(ts)
                    ts.state = "no-worker"

            return recommendations
        except Exception as e:
            logger.exception(e)
            if LOG_PDB:
                import pdb

                pdb.set_trace()
            raise

    def decide_worker(self, ts):
        """
        Decide on a worker for task *ts*.  Return a WorkerState.
        """
        valid_workers = self.valid_workers(ts)

        if not valid_workers and not ts.loose_restrictions and self.workers:
            self.unrunnable.add(ts)
            ts.state = "no-worker"
            return None

        if ts.dependencies or valid_workers is not True:
            worker = decide_worker(
                ts,
                self.workers.values(),
                valid_workers,
                partial(self.worker_objective, ts),
            )
        elif self.idle:
            if len(self.idle) < 20:  # smart but linear in small case
                worker = min(self.idle, key=operator.attrgetter("occupancy"))
            else:  # dumb but fast in large case
                worker = self.idle[self.n_tasks % len(self.idle)]
        else:
            if len(self.workers) < 20:  # smart but linear in small case
                worker = min(
                    self.workers.values(), key=operator.attrgetter("occupancy")
                )
            else:  # dumb but fast in large case
                worker = self.workers.values()[self.n_tasks % len(self.workers)]

        if self.validate:
            assert worker is None or isinstance(worker, WorkerState), (
                type(worker),
                worker,
            )
            assert worker.address in self.workers

        return worker

    def transition_waiting_processing(self, key):
        try:
            ts = self.tasks[key]

            if self.validate:
                assert not ts.waiting_on
                assert not ts.who_has
                assert not ts.exception_blame
                assert not ts.processing_on
                assert not ts.has_lost_dependencies
                assert ts not in self.unrunnable
                assert all(dts.who_has for dts in ts.dependencies)

            ws = self.decide_worker(ts)
            if ws is None:
                return {}
            worker = ws.address

            duration = self.get_task_duration(ts)
            comm = self.get_comm_cost(ts, ws)

            ws.processing[ts] = duration + comm
            ts.processing_on = ws
            ws.occupancy += duration + comm
            self.total_occupancy += duration + comm
            ts.state = "processing"
            self.consume_resources(ts, ws)
            self.check_idle_saturated(ws)
            self.n_tasks += 1

            if ts.actor:
                ws.actors.add(ts)

            # logger.debug("Send job to worker: %s, %s", worker, key)

            self.send_task_to_worker(worker, key)

            return {}
        except Exception as e:
            logger.exception(e)
            if LOG_PDB:
                import pdb

                pdb.set_trace()
            raise

    def transition_waiting_memory(self, key, nbytes=None, worker=None, **kwargs):
        try:
            ws = self.workers[worker]
            ts = self.tasks[key]

            if self.validate:
                assert not ts.processing_on
                assert ts.waiting_on
                assert ts.state == "waiting"

            ts.waiting_on.clear()

            if nbytes is not None:
                ts.set_nbytes(nbytes)

            self.check_idle_saturated(ws)

            recommendations = {}

            self._add_to_memory(ts, ws, recommendations, **kwargs)

            if self.validate:
                assert not ts.processing_on
                assert not ts.waiting_on
                assert ts.who_has

            return recommendations
        except Exception as e:
            logger.exception(e)
            if LOG_PDB:
                import pdb

                pdb.set_trace()
            raise

    def transition_processing_memory(
        self,
        key,
        nbytes=None,
        type=None,
        typename=None,
        worker=None,
        startstops=None,
        **kwargs,
    ):
        try:
            ts = self.tasks[key]
            assert worker
            assert isinstance(worker, str)

            if self.validate:
                assert ts.processing_on
                ws = ts.processing_on
                assert ts in ws.processing
                assert not ts.waiting_on
                assert not ts.who_has, (ts, ts.who_has)
                assert not ts.exception_blame
                assert ts.state == "processing"

            ws = self.workers.get(worker)
            if ws is None:
                return {key: "released"}

            if ws != ts.processing_on:  # someone else has this task
                logger.info(
                    "Unexpected worker completed task, likely due to"
                    " work stealing.  Expected: %s, Got: %s, Key: %s",
                    ts.processing_on,
                    ws,
                    key,
                )
                return {}

            if startstops:
                L = list()
                for startstop in startstops:
                    stop = startstop["stop"]
                    start = startstop["start"]
                    action = startstop["action"]
                    if action == "compute":
                        L.append((start, stop))

                    # record timings of all actions -- a cheaper way of
                    # getting timing info compared with get_task_stream()
                    ts.prefix.all_durations[action] += stop - start

                if len(L) > 0:
                    compute_start, compute_stop = L[0]
                else:  # This is very rare
                    compute_start = compute_stop = None
            else:
                compute_start = compute_stop = None

            #############################
            # Update Timing Information #
            #############################
            if compute_start and ws.processing.get(ts, True):
                # Update average task duration for worker
                old_duration = ts.prefix.duration_average or 0
                new_duration = compute_stop - compute_start
                if not old_duration:
                    avg_duration = new_duration
                else:
                    avg_duration = 0.5 * old_duration + 0.5 * new_duration

                ts.prefix.duration_average = avg_duration
                ts.group.duration += new_duration

                for tts in self.unknown_durations.pop(ts.prefix.name, ()):
                    if tts.processing_on:
                        wws = tts.processing_on
                        old = wws.processing[tts]
                        comm = self.get_comm_cost(tts, wws)
                        wws.processing[tts] = avg_duration + comm
                        wws.occupancy += avg_duration + comm - old
                        self.total_occupancy += avg_duration + comm - old

            ############################
            # Update State Information #
            ############################
            if nbytes is not None:
                ts.set_nbytes(nbytes)

            recommendations = {}

            self._remove_from_processing(ts)

            self._add_to_memory(ts, ws, recommendations, type=type, typename=typename)

            if self.validate:
                assert not ts.processing_on
                assert not ts.waiting_on

            return recommendations
        except Exception as e:
            logger.exception(e)
            if LOG_PDB:
                import pdb

                pdb.set_trace()
            raise

    def transition_memory_released(self, key, safe=False):
        try:
            ts = self.tasks[key]

            if self.validate:
                assert not ts.waiting_on
                assert not ts.processing_on
                if safe:
                    assert not ts.waiters

            if ts.actor:
                for ws in ts.who_has:
                    ws.actors.discard(ts)
                if ts.who_wants:
                    ts.exception_blame = ts
                    ts.exception = "Worker holding Actor was lost"
                    return {ts.key: "erred"}  # don't try to recreate

            recommendations = {}

            for dts in ts.waiters:
                if dts.state in ("no-worker", "processing"):
                    recommendations[dts.key] = "waiting"
                elif dts.state == "waiting":
                    dts.waiting_on.add(ts)

            # XXX factor this out?
            for ws in ts.who_has:
                ws.has_what.remove(ts)
                ws.nbytes -= ts.get_nbytes()
                ts.group.nbytes_in_memory -= ts.get_nbytes()
                self.worker_send(
                    ws.address, {"op": "delete-data", "keys": [key], "report": False}
                )
            ts.who_has.clear()

            ts.state = "released"

            self.report({"op": "lost-data", "key": key})

            if not ts.run_spec:  # pure data
                recommendations[key] = "forgotten"
            elif ts.has_lost_dependencies:
                recommendations[key] = "forgotten"
            elif ts.who_wants or ts.waiters:
                recommendations[key] = "waiting"

            if self.validate:
                assert not ts.waiting_on

            return recommendations
        except Exception as e:
            logger.exception(e)
            if LOG_PDB:
                import pdb

                pdb.set_trace()
            raise

    def transition_released_erred(self, key):
        try:
            ts = self.tasks[key]

            if self.validate:
                with log_errors(pdb=LOG_PDB):
                    assert ts.exception_blame
                    assert not ts.who_has
                    assert not ts.waiting_on
                    assert not ts.waiters

            recommendations = {}

            failing_ts = ts.exception_blame

            for dts in ts.dependents:
                dts.exception_blame = failing_ts
                if not dts.who_has:
                    recommendations[dts.key] = "erred"

            self.report(
                {
                    "op": "task-erred",
                    "key": key,
                    "exception": failing_ts.exception,
                    "traceback": failing_ts.traceback,
                }
            )

            ts.state = "erred"

            # TODO: waiting data?
            return recommendations
        except Exception as e:
            logger.exception(e)
            if LOG_PDB:
                import pdb

                pdb.set_trace()
            raise

    def transition_erred_released(self, key):
        try:
            ts = self.tasks[key]

            if self.validate:
                with log_errors(pdb=LOG_PDB):
                    assert all(dts.state != "erred" for dts in ts.dependencies)
                    assert ts.exception_blame
                    assert not ts.who_has
                    assert not ts.waiting_on
                    assert not ts.waiters

            recommendations = {}

            ts.exception = None
            ts.exception_blame = None
            ts.traceback = None

            for dep in ts.dependents:
                if dep.state == "erred":
                    recommendations[dep.key] = "waiting"

            self.report({"op": "task-retried", "key": key})
            ts.state = "released"

            return recommendations
        except Exception as e:
            logger.exception(e)
            if LOG_PDB:
                import pdb

                pdb.set_trace()
            raise

    def transition_waiting_released(self, key):
        try:
            ts = self.tasks[key]

            if self.validate:
                assert not ts.who_has
                assert not ts.processing_on

            recommendations = {}

            for dts in ts.dependencies:
                s = dts.waiters
                if ts in s:
                    s.discard(ts)
                    if not s and not dts.who_wants:
                        recommendations[dts.key] = "released"
            ts.waiting_on.clear()

            ts.state = "released"

            if ts.has_lost_dependencies:
                recommendations[key] = "forgotten"
            elif not ts.exception_blame and (ts.who_wants or ts.waiters):
                recommendations[key] = "waiting"
            else:
                ts.waiters.clear()

            return recommendations
        except Exception as e:
            logger.exception(e)
            if LOG_PDB:
                import pdb

                pdb.set_trace()
            raise

    def transition_processing_released(self, key):
        try:
            ts = self.tasks[key]

            if self.validate:
                assert ts.processing_on
                assert not ts.who_has
                assert not ts.waiting_on
                assert self.tasks[key].state == "processing"

            self._remove_from_processing(
                ts, send_worker_msg={"op": "release-task", "key": key}
            )

            ts.state = "released"

            recommendations = {}

            if ts.has_lost_dependencies:
                recommendations[key] = "forgotten"
            elif ts.waiters or ts.who_wants:
                recommendations[key] = "waiting"

            if recommendations.get(key) != "waiting":
                for dts in ts.dependencies:
                    if dts.state != "released":
                        s = dts.waiters
                        s.discard(ts)
                        if not s and not dts.who_wants:
                            recommendations[dts.key] = "released"
                ts.waiters.clear()

            if self.validate:
                assert not ts.processing_on

            return recommendations
        except Exception as e:
            logger.exception(e)
            if LOG_PDB:
                import pdb

                pdb.set_trace()
            raise

    def transition_processing_erred(
        self, key, cause=None, exception=None, traceback=None, **kwargs
    ):
        try:
            ts = self.tasks[key]

            if self.validate:
                assert cause or ts.exception_blame
                assert ts.processing_on
                assert not ts.who_has
                assert not ts.waiting_on

            if ts.actor:
                ws = ts.processing_on
                ws.actors.remove(ts)

            self._remove_from_processing(ts)

            if exception is not None:
                ts.exception = exception
            if traceback is not None:
                ts.traceback = traceback
            if cause is not None:
                failing_ts = self.tasks[cause]
                ts.exception_blame = failing_ts
            else:
                failing_ts = ts.exception_blame

            recommendations = {}

            for dts in ts.dependents:
                dts.exception_blame = failing_ts
                recommendations[dts.key] = "erred"

            for dts in ts.dependencies:
                s = dts.waiters
                s.discard(ts)
                if not s and not dts.who_wants:
                    recommendations[dts.key] = "released"

            ts.waiters.clear()  # do anything with this?

            ts.state = "erred"

            self.report(
                {
                    "op": "task-erred",
                    "key": key,
                    "exception": failing_ts.exception,
                    "traceback": failing_ts.traceback,
                }
            )

            cs = self.clients["fire-and-forget"]
            if ts in cs.wants_what:
                self.client_releases_keys(client="fire-and-forget", keys=[key])

            if self.validate:
                assert not ts.processing_on

            return recommendations
        except Exception as e:
            logger.exception(e)
            if LOG_PDB:
                import pdb

                pdb.set_trace()
            raise

    def transition_no_worker_released(self, key):
        try:
            ts = self.tasks[key]

            if self.validate:
                assert self.tasks[key].state == "no-worker"
                assert not ts.who_has
                assert not ts.waiting_on

            self.unrunnable.remove(ts)
            ts.state = "released"

            for dts in ts.dependencies:
                dts.waiters.discard(ts)

            ts.waiters.clear()

            return {}
        except Exception as e:
            logger.exception(e)
            if LOG_PDB:
                import pdb

                pdb.set_trace()
            raise

    def remove_key(self, key):
        ts = self.tasks.pop(key)
        assert ts.state == "forgotten"
        self.unrunnable.discard(ts)
        for cs in ts.who_wants:
            cs.wants_what.remove(ts)
        ts.who_wants.clear()
        ts.processing_on = None
        ts.exception_blame = ts.exception = ts.traceback = None

        if key in self.task_metadata:
            del self.task_metadata[key]

    def _propagate_forgotten(self, ts, recommendations):
        ts.state = "forgotten"
        key = ts.key
        for dts in ts.dependents:
            dts.has_lost_dependencies = True
            dts.dependencies.remove(ts)
            dts.waiting_on.discard(ts)
            if dts.state not in ("memory", "erred"):
                # Cannot compute task anymore
                recommendations[dts.key] = "forgotten"
        ts.dependents.clear()
        ts.waiters.clear()

        for dts in ts.dependencies:
            dts.dependents.remove(ts)
            s = dts.waiters
            s.discard(ts)
            if not dts.dependents and not dts.who_wants:
                # Task not needed anymore
                assert dts is not ts
                recommendations[dts.key] = "forgotten"
        ts.dependencies.clear()
        ts.waiting_on.clear()

        if ts.who_has:
            ts.group.nbytes_in_memory -= ts.get_nbytes()

        for ws in ts.who_has:
            ws.has_what.remove(ts)
            ws.nbytes -= ts.get_nbytes()
            w = ws.address
            if w in self.workers:  # in case worker has died
                self.worker_send(
                    w, {"op": "delete-data", "keys": [key], "report": False}
                )
        ts.who_has.clear()

    def transition_memory_forgotten(self, key):
        try:
            ts = self.tasks[key]

            if self.validate:
                assert ts.state == "memory"
                assert not ts.processing_on
                assert not ts.waiting_on
                if not ts.run_spec:
                    # It's ok to forget a pure data task
                    pass
                elif ts.has_lost_dependencies:
                    # It's ok to forget a task with forgotten dependencies
                    pass
                elif not ts.who_wants and not ts.waiters and not ts.dependents:
                    # It's ok to forget a task that nobody needs
                    pass
                else:
                    assert 0, (ts,)

            recommendations = {}

            if ts.actor:
                for ws in ts.who_has:
                    ws.actors.discard(ts)

            self._propagate_forgotten(ts, recommendations)

            self.report_on_key(ts=ts)
            self.remove_key(key)

            return recommendations
        except Exception as e:
            logger.exception(e)
            if LOG_PDB:
                import pdb

                pdb.set_trace()
            raise

    def transition_released_forgotten(self, key):
        try:
            ts = self.tasks[key]

            if self.validate:
                assert ts.state in ("released", "erred")
                assert not ts.who_has
                assert not ts.processing_on
                assert not ts.waiting_on, (ts, ts.waiting_on)
                if not ts.run_spec:
                    # It's ok to forget a pure data task
                    pass
                elif ts.has_lost_dependencies:
                    # It's ok to forget a task with forgotten dependencies
                    pass
                elif not ts.who_wants and not ts.waiters and not ts.dependents:
                    # It's ok to forget a task that nobody needs
                    pass
                else:
                    assert 0, (ts,)

            recommendations = {}
            self._propagate_forgotten(ts, recommendations)

            self.report_on_key(ts=ts)
            self.remove_key(key)

            return recommendations
        except Exception as e:
            logger.exception(e)
            if LOG_PDB:
                import pdb

                pdb.set_trace()
            raise

    def transition(self, key, finish, *args, **kwargs):
        """Transition a key from its current state to the finish state

        Examples
        --------
        >>> self.transition('x', 'waiting')
        {'x': 'processing'}

        Returns
        -------
        Dictionary of recommendations for future transitions

        See Also
        --------
        Scheduler.transitions: transitive version of this function
        """
        try:
            try:
                ts = self.tasks[key]
            except KeyError:
                return {}
            start = ts.state
            if start == finish:
                return {}

            if self.plugins:
                dependents = set(ts.dependents)
                dependencies = set(ts.dependencies)

            if (start, finish) in self._transitions:
                func = self._transitions[start, finish]
                recommendations = func(key, *args, **kwargs)
            elif "released" not in (start, finish):
                func = self._transitions["released", finish]
                assert not args and not kwargs
                a = self.transition(key, "released")
                if key in a:
                    func = self._transitions["released", a[key]]
                b = func(key)
                a = a.copy()
                a.update(b)
                recommendations = a
                start = "released"
            else:
                raise RuntimeError(
                    "Impossible transition from %r to %r" % (start, finish)
                )

            finish2 = ts.state
            self.transition_log.append((key, start, finish2, recommendations, time()))
            if self.validate:
                logger.debug(
                    "Transitioned %r %s->%s (actual: %s).  Consequence: %s",
                    key,
                    start,
                    finish2,
                    ts.state,
                    dict(recommendations),
                )
            if self.plugins:
                # Temporarily put back forgotten key for plugin to retrieve it
                if ts.state == "forgotten":
                    try:
                        ts.dependents = dependents
                        ts.dependencies = dependencies
                    except KeyError:
                        pass
                    self.tasks[ts.key] = ts
                for plugin in list(self.plugins):
                    try:
                        plugin.transition(key, start, finish2, *args, **kwargs)
                    except Exception:
                        logger.info("Plugin failed with exception", exc_info=True)
                if ts.state == "forgotten":
                    del self.tasks[ts.key]

            if ts.state == "forgotten" and ts.group.name in self.task_groups:
                # Remove TaskGroup if all tasks are in the forgotten state
                tg = ts.group
                if not any(tg.states.get(s) for s in ALL_TASK_STATES):
                    ts.prefix.groups.remove(tg)
                    del self.task_groups[tg.name]

            return recommendations
        except Exception as e:
            logger.exception("Error transitioning %r from %r to %r", key, start, finish)
            if LOG_PDB:
                import pdb

                pdb.set_trace()
            raise

    def transitions(self, recommendations):
        """Process transitions until none are left

        This includes feedback from previous transitions and continues until we
        reach a steady state
        """
        keys = set()
        recommendations = recommendations.copy()
        while recommendations:
            key, finish = recommendations.popitem()
            keys.add(key)
            new = self.transition(key, finish)
            recommendations.update(new)

        if self.validate:
            for key in keys:
                self.validate_key(key)

    def story(self, *keys):
        """ Get all transitions that touch one of the input keys """
        keys = set(keys)
        return [
            t for t in self.transition_log if t[0] in keys or keys.intersection(t[3])
        ]

    transition_story = story

    def reschedule(self, key=None, worker=None):
        """Reschedule a task

        Things may have shifted and this task may now be better suited to run
        elsewhere
        """
        try:
            ts = self.tasks[key]
        except KeyError:
            logger.warning(
                "Attempting to reschedule task {}, which was not "
                "found on the scheduler. Aborting reschedule.".format(key)
            )
            return
        if ts.state != "processing":
            return
        if worker and ts.processing_on.address != worker:
            return
        self.transitions({key: "released"})

    ##############################
    # Assigning Tasks to Workers #
    ##############################

    def check_idle_saturated(self, ws, occ=None):
        """Update the status of the idle and saturated state

        The scheduler keeps track of workers that are ..

        -  Saturated: have enough work to stay busy
        -  Idle: do not have enough work to stay busy

        They are considered saturated if they both have enough tasks to occupy
        all of their threads, and if the expected runtime of those tasks is
        large enough.

        This is useful for load balancing and adaptivity.
        """
        if self.total_nthreads == 0 or ws.status == Status.closed:
            return
        if occ is None:
            occ = ws.occupancy
        nc = ws.nthreads
        p = len(ws.processing)

        avg = self.total_occupancy / self.total_nthreads

        if p < nc or occ / nc < avg / 2:
            self.idle.add(ws)
            self.saturated.discard(ws)
        else:
            self.idle.discard(ws)

            pending = occ * (p - nc) / p / nc
            if p > nc and pending > 0.4 and pending > 1.9 * avg:
                self.saturated.add(ws)
            else:
                self.saturated.discard(ws)

    def valid_workers(self, ts):
        """Return set of currently valid workers for key

        If all workers are valid then this returns ``True``.
        This checks tracks the following state:

        *  worker_restrictions
        *  host_restrictions
        *  resource_restrictions
        """
        s = True

        if ts.worker_restrictions:
            s = {w for w in ts.worker_restrictions if w in self.workers}

        if ts.host_restrictions:
            # Resolve the alias here rather than early, for the worker
            # may not be connected when host_restrictions is populated
            hr = [self.coerce_hostname(h) for h in ts.host_restrictions]
            # XXX need HostState?
            ss = [self.host_info[h]["addresses"] for h in hr if h in self.host_info]
            ss = set.union(*ss) if ss else set()
            if s is True:
                s = ss
            else:
                s |= ss

        if ts.resource_restrictions:
            w = {
                resource: {
                    w
                    for w, supplied in self.resources[resource].items()
                    if supplied >= required
                }
                for resource, required in ts.resource_restrictions.items()
            }

            ww = set.intersection(*w.values())

            if s is True:
                s = ww
            else:
                s &= ww

        if s is True:
            return s
        else:
            return {self.workers[w] for w in s}

    def consume_resources(self, ts, ws):
        if ts.resource_restrictions:
            for r, required in ts.resource_restrictions.items():
                ws.used_resources[r] += required

    def release_resources(self, ts, ws):
        if ts.resource_restrictions:
            for r, required in ts.resource_restrictions.items():
                ws.used_resources[r] -= required

    #####################
    # Utility functions #
    #####################

    def add_resources(self, comm=None, worker=None, resources=None):
        ws = self.workers[worker]
        if resources:
            ws.resources.update(resources)
        ws.used_resources = {}
        for resource, quantity in ws.resources.items():
            ws.used_resources[resource] = 0
            self.resources[resource][worker] = quantity
        return "OK"

    def remove_resources(self, worker):
        ws = self.workers[worker]
        for resource, quantity in ws.resources.items():
            del self.resources[resource][worker]

    def coerce_address(self, addr, resolve=True):
        """
        Coerce possible input addresses to canonical form.
        *resolve* can be disabled for testing with fake hostnames.

        Handles strings, tuples, or aliases.
        """
        # XXX how many address-parsing routines do we have?
        if addr in self.aliases:
            addr = self.aliases[addr]
        if isinstance(addr, tuple):
            addr = unparse_host_port(*addr)
        if not isinstance(addr, str):
            raise TypeError("addresses should be strings or tuples, got %r" % (addr,))

        if resolve:
            addr = resolve_address(addr)
        else:
            addr = normalize_address(addr)

        return addr

    def coerce_hostname(self, host):
        """
        Coerce the hostname of a worker.
        """
        if host in self.aliases:
            return self.workers[self.aliases[host]].host
        else:
            return host

    def workers_list(self, workers):
        """
        List of qualifying workers

        Takes a list of worker addresses or hostnames.
        Returns a list of all worker addresses that match
        """
        if workers is None:
            return list(self.workers)

        out = set()
        for w in workers:
            if ":" in w:
                out.add(w)
            else:
                out.update({ww for ww in self.workers if w in ww})  # TODO: quadratic
        return list(out)

    def start_ipython(self, comm=None):
        """Start an IPython kernel

        Returns Jupyter connection info dictionary.
        """
        from ._ipython_utils import start_ipython

        if self._ipython_kernel is None:
            self._ipython_kernel = start_ipython(
                ip=self.ip, ns={"scheduler": self}, log=logger
            )
        return self._ipython_kernel.get_connection_info()

    def worker_objective(self, ts, ws):
        """
        Objective function to determine which worker should get the task

        Minimize expected start time.  If a tie then break with data storage.
        """
        comm_bytes = sum(
            [dts.get_nbytes() for dts in ts.dependencies if ws not in dts.who_has]
        )
        stack_time = ws.occupancy / ws.nthreads
        start_time = comm_bytes / self.bandwidth + stack_time

        if ts.actor:
            return (len(ws.actors), start_time, ws.nbytes)
        else:
            return (start_time, ws.nbytes)

    async def get_profile(
        self,
        comm=None,
        workers=None,
        scheduler=False,
        server=False,
        merge_workers=True,
        start=None,
        stop=None,
        key=None,
    ):
        if workers is None:
            workers = self.workers
        else:
            workers = set(self.workers) & set(workers)

        if scheduler:
            return profile.get_profile(self.io_loop.profile, start=start, stop=stop)

        results = await asyncio.gather(
            *(
                self.rpc(w).profile(start=start, stop=stop, key=key, server=server)
                for w in workers
            ),
            return_exceptions=True,
        )

        results = [r for r in results if not isinstance(r, Exception)]

        if merge_workers:
            response = profile.merge(*results)
        else:
            response = dict(zip(workers, results))
        return response

    async def get_profile_metadata(
        self,
        comm=None,
        workers=None,
        merge_workers=True,
        start=None,
        stop=None,
        profile_cycle_interval=None,
    ):
        dt = profile_cycle_interval or dask.config.get(
            "distributed.worker.profile.cycle"
        )
        dt = parse_timedelta(dt, default="ms")

        if workers is None:
            workers = self.workers
        else:
            workers = set(self.workers) & set(workers)
        results = await asyncio.gather(
            *(self.rpc(w).profile_metadata(start=start, stop=stop) for w in workers),
            return_exceptions=True,
        )

        results = [r for r in results if not isinstance(r, Exception)]
        counts = [v["counts"] for v in results]
        counts = itertools.groupby(merge_sorted(*counts), lambda t: t[0] // dt * dt)
        counts = [(time, sum(pluck(1, group))) for time, group in counts]

        keys = set()
        for v in results:
            for t, d in v["keys"]:
                for k in d:
                    keys.add(k)
        keys = {k: [] for k in keys}

        groups1 = [v["keys"] for v in results]
        groups2 = list(merge_sorted(*groups1, key=first))

        last = 0
        for t, d in groups2:
            tt = t // dt * dt
            if tt > last:
                last = tt
                for k, v in keys.items():
                    v.append([tt, 0])
            for k, v in d.items():
                keys[k][-1][1] += v

        return {"counts": counts, "keys": keys}

    async def performance_report(self, comm=None, start=None, code=""):
        stop = time()
        # Profiles
        compute, scheduler, workers = await asyncio.gather(
            *[
                self.get_profile(start=start),
                self.get_profile(scheduler=True, start=start),
                self.get_profile(server=True, start=start),
            ]
        )
        from . import profile

        def profile_to_figure(state):
            data = profile.plot_data(state)
            figure, source = profile.plot_figure(data, sizing_mode="stretch_both")
            return figure

        compute, scheduler, workers = map(
            profile_to_figure, (compute, scheduler, workers)
        )

        # Task stream
        task_stream = self.get_task_stream(start=start)
        total_tasks = len(task_stream)
        timespent = defaultdict(int)
        for d in task_stream:
            for x in d.get("startstops", []):
                timespent[x["action"]] += x["stop"] - x["start"]
        tasks_timings = ""
        for k in sorted(timespent.keys()):
            tasks_timings += f"\n<li> {k} time: {format_time(timespent[k])} </li>"

        from .diagnostics.task_stream import rectangles
        from .dashboard.components.scheduler import task_stream_figure

        rects = rectangles(task_stream)
        source, task_stream = task_stream_figure(sizing_mode="stretch_both")
        source.data.update(rects)

        from distributed.dashboard.components.scheduler import (
            BandwidthWorkers,
            BandwidthTypes,
        )

        bandwidth_workers = BandwidthWorkers(self, sizing_mode="stretch_both")
        bandwidth_workers.update()
        bandwidth_types = BandwidthTypes(self, sizing_mode="stretch_both")
        bandwidth_types.update()

        from bokeh.models import Panel, Tabs, Div
        import distributed

        # HTML
        html = """
        <h1> Dask Performance Report </h1>

        <i> Select different tabs on the top for additional information </i>

        <h2> Duration: {time} </h2>
        <h2> Tasks Information </h2>
        <ul>
         <li> number of tasks: {ntasks} </li>
         {tasks_timings}
        </ul>

        <h2> Scheduler Information </h2>
        <ul>
          <li> Address: {address} </li>
          <li> Workers: {nworkers} </li>
          <li> Threads: {threads} </li>
          <li> Memory: {memory} </li>
          <li> Dask Version: {dask_version} </li>
          <li> Dask.Distributed Version: {distributed_version} </li>
        </ul>

        <h2> Calling Code </h2>
        <pre>
{code}
        </pre>
        """.format(
            time=format_time(stop - start),
            ntasks=total_tasks,
            tasks_timings=tasks_timings,
            address=self.address,
            nworkers=len(self.workers),
            threads=sum(w.nthreads for w in self.workers.values()),
            memory=format_bytes(sum(w.memory_limit for w in self.workers.values())),
            code=code,
            dask_version=dask.__version__,
            distributed_version=distributed.__version__,
        )
        html = Div(text=html)

        html = Panel(child=html, title="Summary")
        compute = Panel(child=compute, title="Worker Profile (compute)")
        workers = Panel(child=workers, title="Worker Profile (administrative)")
        scheduler = Panel(child=scheduler, title="Scheduler Profile (administrative)")
        task_stream = Panel(child=task_stream, title="Task Stream")
        bandwidth_workers = Panel(
            child=bandwidth_workers.fig, title="Bandwidth (Workers)"
        )
        bandwidth_types = Panel(child=bandwidth_types.fig, title="Bandwidth (Types)")

        tabs = Tabs(
            tabs=[
                html,
                task_stream,
                compute,
                workers,
                scheduler,
                bandwidth_workers,
                bandwidth_types,
            ]
        )

        from bokeh.plotting import save, output_file
        from bokeh.core.templates import get_env

        with tmpfile(extension=".html") as fn:
            output_file(filename=fn, title="Dask Performance Report")
            template_directory = os.path.join(
                os.path.dirname(os.path.abspath(__file__)), "dashboard", "templates"
            )
            template_environment = get_env()
            template_environment.loader.searchpath.append(template_directory)
            template = template_environment.get_template("performance_report.html")
            save(tabs, filename=fn, template=template)

            with open(fn) as f:
                data = f.read()

        return data

    async def get_worker_logs(self, comm=None, n=None, workers=None, nanny=False):
        results = await self.broadcast(
            msg={"op": "get_logs", "n": n}, workers=workers, nanny=nanny
        )
        return results

    ###########
    # Cleanup #
    ###########

    def reevaluate_occupancy(self, worker_index=0):
        """Periodically reassess task duration time

        The expected duration of a task can change over time.  Unfortunately we
        don't have a good constant-time way to propagate the effects of these
        changes out to the summaries that they affect, like the total expected
        runtime of each of the workers, or what tasks are stealable.

        In this coroutine we walk through all of the workers and re-align their
        estimates with the current state of tasks.  We do this periodically
        rather than at every transition, and we only do it if the scheduler
        process isn't under load (using psutil.Process.cpu_percent()).  This
        lets us avoid this fringe optimization when we have better things to
        think about.
        """
        DELAY = 0.1
        try:
            if self.status == Status.closed:
                return

            last = time()
            next_time = timedelta(seconds=DELAY)

            if self.proc.cpu_percent() < 50:
                workers = list(self.workers.values())
                for i in range(len(workers)):
                    ws = workers[worker_index % len(workers)]
                    worker_index += 1
                    try:
                        if ws is None or not ws.processing:
                            continue
                        self._reevaluate_occupancy_worker(ws)
                    finally:
                        del ws  # lose ref

                    duration = time() - last
                    if duration > 0.005:  # 5ms since last release
                        next_time = timedelta(seconds=duration * 5)  # 25ms gap
                        break

            self.loop.add_timeout(
                next_time, self.reevaluate_occupancy, worker_index=worker_index
            )

        except Exception:
            logger.error("Error in reevaluate occupancy", exc_info=True)
            raise

    def _reevaluate_occupancy_worker(self, ws):
        """ See reevaluate_occupancy """
        old = ws.occupancy

        new = 0
        nbytes = 0
        for ts in ws.processing:
            duration = self.get_task_duration(ts)
            comm = self.get_comm_cost(ts, ws)
            ws.processing[ts] = duration + comm
            new += duration + comm

        ws.occupancy = new
        self.total_occupancy += new - old
        self.check_idle_saturated(ws)

        # significant increase in duration
        if (new > old * 1.3) and ("stealing" in self.extensions):
            steal = self.extensions["stealing"]
            for ts in ws.processing:
                steal.remove_key_from_stealable(ts)
                steal.put_key_in_stealable(ts)

    async def check_worker_ttl(self):
        now = time()
        for ws in self.workers.values():
            if (ws.last_seen < now - self.worker_ttl) and (
                ws.last_seen < now - 10 * heartbeat_interval(len(self.workers))
            ):
                logger.warning(
                    "Worker failed to heartbeat within %s seconds. Closing: %s",
                    self.worker_ttl,
                    ws,
                )
                await self.remove_worker(address=ws.address)

    def check_idle(self):
        if any(ws.processing for ws in self.workers.values()) or self.unrunnable:
            self.idle_since = None
            return
        elif not self.idle_since:
            self.idle_since = time()

        if time() > self.idle_since + self.idle_timeout:
            logger.info(
                "Scheduler closing after being idle for %s",
                format_time(self.idle_timeout),
            )
            self.loop.add_callback(self.close)

    def adaptive_target(self, comm=None, target_duration=None):
        """Desired number of workers based on the current workload

        This looks at the current running tasks and memory use, and returns a
        number of desired workers.  This is often used by adaptive scheduling.

        Parameters
        ----------
        target_duration: str
            A desired duration of time for computations to take.  This affects
            how rapidly the scheduler will ask to scale.

        See Also
        --------
        distributed.deploy.Adaptive
        """
        if target_duration is None:
            target_duration = dask.config.get("distributed.adaptive.target-duration")
        target_duration = parse_timedelta(target_duration)

        # CPU
        cpu = math.ceil(
            self.total_occupancy / target_duration
        )  # TODO: threads per worker

        # Avoid a few long tasks from asking for many cores
        tasks_processing = 0
        for ws in self.workers.values():
            tasks_processing += len(ws.processing)

            if tasks_processing > cpu:
                break
        else:
            cpu = min(tasks_processing, cpu)

        if self.unrunnable and not self.workers:
            cpu = max(1, cpu)

        # Memory
        limit_bytes = {addr: ws.memory_limit for addr, ws in self.workers.items()}
        worker_bytes = [ws.nbytes for ws in self.workers.values()]
        limit = sum(limit_bytes.values())
        total = sum(worker_bytes)
        if total > 0.6 * limit:
            memory = 2 * len(self.workers)
        else:
            memory = 0

        target = max(memory, cpu)
        if target >= len(self.workers):
            return target
        else:  # Scale down?
            to_close = self.workers_to_close()
            return len(self.workers) - len(to_close)


def decide_worker(ts, all_workers, valid_workers, objective):
    """
    Decide which worker should take task *ts*.

    We choose the worker that has the data on which *ts* depends.

    If several workers have dependencies then we choose the less-busy worker.

    Optionally provide *valid_workers* of where jobs are allowed to occur
    (if all workers are allowed to take the task, pass True instead).

    If the task requires data communication because no eligible worker has
    all the dependencies already, then we choose to minimize the number
    of bytes sent between workers.  This is determined by calling the
    *objective* function.
    """
    deps = ts.dependencies
    assert all(dts.who_has for dts in deps)
    if ts.actor:
        candidates = all_workers
    else:
        candidates = frequencies([ws for dts in deps for ws in dts.who_has])
    if valid_workers is True:
        if not candidates:
            candidates = all_workers
    else:
        candidates = valid_workers & set(candidates)
        if not candidates:
            candidates = valid_workers
            if not candidates:
                if ts.loose_restrictions:
                    return decide_worker(ts, all_workers, True, objective)
                else:
                    return None
    if not candidates:
        return None

    if len(candidates) == 1:
        return first(candidates)

    return min(candidates, key=objective)


def validate_task_state(ts):
    """
    Validate the given TaskState.
    """
    assert ts.state in ALL_TASK_STATES or ts.state == "forgotten", ts

    if ts.waiting_on:
        assert ts.waiting_on.issubset(ts.dependencies), (
            "waiting not subset of dependencies",
            str(ts.waiting_on),
            str(ts.dependencies),
        )
    if ts.waiters:
        assert ts.waiters.issubset(ts.dependents), (
            "waiters not subset of dependents",
            str(ts.waiters),
            str(ts.dependents),
        )

    for dts in ts.waiting_on:
        assert not dts.who_has, ("waiting on in-memory dep", str(ts), str(dts))
        assert dts.state != "released", ("waiting on released dep", str(ts), str(dts))
    for dts in ts.dependencies:
        assert ts in dts.dependents, (
            "not in dependency's dependents",
            str(ts),
            str(dts),
            str(dts.dependents),
        )
        if ts.state in ("waiting", "processing"):
            assert dts in ts.waiting_on or dts.who_has, (
                "dep missing",
                str(ts),
                str(dts),
            )
        assert dts.state != "forgotten"

    for dts in ts.waiters:
        assert dts.state in ("waiting", "processing"), (
            "waiter not in play",
            str(ts),
            str(dts),
        )
    for dts in ts.dependents:
        assert ts in dts.dependencies, (
            "not in dependent's dependencies",
            str(ts),
            str(dts),
            str(dts.dependencies),
        )
        assert dts.state != "forgotten"

    assert (ts.processing_on is not None) == (ts.state == "processing")
    assert bool(ts.who_has) == (ts.state == "memory"), (ts, ts.who_has)

    if ts.state == "processing":
        assert all(dts.who_has for dts in ts.dependencies), (
            "task processing without all deps",
            str(ts),
            str(ts.dependencies),
        )
        assert not ts.waiting_on

    if ts.who_has:
        assert ts.waiters or ts.who_wants, (
            "unneeded task in memory",
            str(ts),
            str(ts.who_has),
        )
        if ts.run_spec:  # was computed
            assert ts.type
            assert isinstance(ts.type, str)
        assert not any(ts in dts.waiting_on for dts in ts.dependents)
        for ws in ts.who_has:
            assert ts in ws.has_what, (
                "not in who_has' has_what",
                str(ts),
                str(ws),
                str(ws.has_what),
            )

    if ts.who_wants:
        for cs in ts.who_wants:
            assert ts in cs.wants_what, (
                "not in who_wants' wants_what",
                str(ts),
                str(cs),
                str(cs.wants_what),
            )

    if ts.actor:
        if ts.state == "memory":
            assert sum([ts in ws.actors for ws in ts.who_has]) == 1
        if ts.state == "processing":
            assert ts in ts.processing_on.actors


def validate_worker_state(ws):
    for ts in ws.has_what:
        assert ws in ts.who_has, (
            "not in has_what' who_has",
            str(ws),
            str(ts),
            str(ts.who_has),
        )

    for ts in ws.actors:
        assert ts.state in ("memory", "processing")


def validate_state(tasks, workers, clients):
    """
    Validate a current runtime state

    This performs a sequence of checks on the entire graph, running in about
    linear time.  This raises assert errors if anything doesn't check out.
    """
    for ts in tasks.values():
        validate_task_state(ts)

    for ws in workers.values():
        validate_worker_state(ws)

    for cs in clients.values():
        for ts in cs.wants_what:
            assert cs in ts.who_wants, (
                "not in wants_what' who_wants",
                str(cs),
                str(ts),
                str(ts.who_wants),
            )


_round_robin = [0]


def heartbeat_interval(n):
    """
    Interval in seconds that we desire heartbeats based on number of workers
    """
    if n <= 10:
        return 0.5
    elif n < 50:
        return 1
    elif n < 200:
        return 2
    else:
        # no more than 200 hearbeats a second scaled by workers
        return n / 200 + 1


class KilledWorker(Exception):
    def __init__(self, task, last_worker):
        super().__init__(task, last_worker)
        self.task = task
        self.last_worker = last_worker


class WorkerStatusPlugin(SchedulerPlugin):
    """
    An plugin to share worker status with a remote observer

    This is used in cluster managers to keep updated about the status of the
    scheduler.
    """

    def __init__(self, scheduler, comm):
        self.bcomm = BatchedSend(interval="5ms")
        self.bcomm.start(comm)

        self.scheduler = scheduler
        self.scheduler.add_plugin(self)

    def add_worker(self, worker=None, **kwargs):
        ident = self.scheduler.workers[worker].identity()
        del ident["metrics"]
        del ident["last_seen"]
        try:
            self.bcomm.send(["add", {"workers": {worker: ident}}])
        except CommClosedError:
            self.scheduler.remove_plugin(self)

    def remove_worker(self, worker=None, **kwargs):
        try:
            self.bcomm.send(["remove", worker])
        except CommClosedError:
            self.scheduler.remove_plugin(self)

    def teardown(self):
        self.bcomm.close()


class CollectTaskMetaDataPlugin(SchedulerPlugin):
    def __init__(self, scheduler, name):
        self.scheduler = scheduler
        self.name = name
        self.keys = set()
        self.metadata = {}
        self.state = {}

    def update_graph(self, scheduler, dsk=None, keys=None, restrictions=None, **kwargs):
        self.keys.update(keys)

    def transition(self, key, start, finish, *args, **kwargs):
        if finish == "memory" or finish == "erred":
            ts = self.scheduler.tasks.get(key)
            if ts is not None and ts.key in self.keys:
                self.metadata[key] = ts.metadata
                self.state[key] = finish
                self.keys.discard(key)