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
|
from __future__ import annotations
import abc
import asyncio
import heapq
import logging
import math
import operator
import random
import sys
import warnings
import weakref
from collections import defaultdict, deque
from collections.abc import (
Callable,
Collection,
Container,
Iterator,
Mapping,
MutableMapping,
)
from copy import copy
from dataclasses import dataclass, field
from functools import lru_cache, partial, singledispatchmethod
from itertools import chain
from typing import TYPE_CHECKING, Any, ClassVar, Literal, NamedTuple, TypedDict, cast
from tlz import peekn
import dask
from dask.utils import parse_bytes, typename
from distributed._stories import worker_story
from distributed.collections import HeapSet
from distributed.comm import get_address_host
from distributed.core import ErrorMessage, error_message
from distributed.metrics import time
from distributed.protocol import pickle
from distributed.protocol.serialize import Serialize
from distributed.sizeof import safe_sizeof as sizeof
from distributed.utils import recursive_to_dict
logger = logging.getLogger("distributed.worker.state_machine")
if TYPE_CHECKING:
# TODO import from typing (requires Python >=3.10)
from typing_extensions import TypeAlias
# Circular imports
from distributed.diagnostics.plugin import WorkerPlugin
from distributed.worker import Worker
# TODO move out of TYPE_CHECKING (requires Python >=3.10)
# Not to be confused with distributed.scheduler.TaskStateState
TaskStateState: TypeAlias = Literal[
"cancelled",
"constrained",
"error",
"executing",
"fetch",
"flight",
"forgotten",
"long-running",
"memory",
"missing",
"ready",
"released",
"rescheduled",
"resumed",
"waiting",
]
else:
TaskStateState = str
# TaskState.state subsets
PROCESSING: set[TaskStateState] = {
"waiting",
"ready",
"constrained",
"executing",
"long-running",
"cancelled",
"resumed",
}
READY: set[TaskStateState] = {"ready", "constrained"}
# Valid states for a task that is found in TaskState.waiting_for_data
WAITING_FOR_DATA: set[TaskStateState] = {
"constrained",
"executing",
"fetch",
"flight",
"long-running",
"missing",
"ready",
"resumed",
"waiting",
}
NO_VALUE = "--no-value-sentinel--"
class SerializedTask(NamedTuple):
"""Info from distributed.scheduler.TaskState.run_spec
Input to distributed.worker._deserialize
(function, args kwargs) and task are mutually exclusive
"""
function: bytes | None = None
args: bytes | tuple | list | None = None
kwargs: bytes | dict[str, Any] | None = None
task: object = NO_VALUE
class StartStop(TypedDict, total=False):
action: str
start: float
stop: float
source: str # optional
class InvalidTransition(Exception):
def __init__(
self,
key: str,
start: TaskStateState,
finish: TaskStateState,
story: list[tuple],
):
self.key = key
self.start = start
self.finish = finish
self.story = story
def __reduce__(self) -> tuple[Callable, tuple]:
return type(self), (self.key, self.start, self.finish, self.story)
def __repr__(self) -> str:
return (
f"{self.__class__.__name__}: {self.key} :: {self.start}->{self.finish}"
+ "\n"
+ " Story:\n "
+ "\n ".join(map(str, self.story))
)
__str__ = __repr__
def to_event(self) -> tuple[str, dict[str, Any]]:
return (
"invalid-worker-transition",
{
"key": self.key,
"start": self.start,
"finish": self.finish,
"story": self.story,
},
)
class TransitionCounterMaxExceeded(InvalidTransition):
def to_event(self) -> tuple[str, dict[str, Any]]:
topic, msg = super().to_event()
return "transition-counter-max-exceeded", msg
class InvalidTaskState(Exception):
def __init__(
self,
key: str,
state: TaskStateState,
story: list[tuple],
):
self.key = key
self.state = state
self.story = story
def __reduce__(self) -> tuple[Callable, tuple]:
return type(self), (self.key, self.state, self.story)
def __repr__(self) -> str:
return (
f"{self.__class__.__name__}: {self.key} :: {self.state}"
+ "\n"
+ " Story:\n "
+ "\n ".join(map(str, self.story))
)
__str__ = __repr__
def to_event(self) -> tuple[str, dict[str, Any]]:
return (
"invalid-worker-task-state",
{
"key": self.key,
"state": self.state,
"story": self.story,
},
)
class RecommendationsConflict(Exception):
"""Two or more recommendations for the same task suggested different finish states"""
@lru_cache
def _default_data_size() -> int:
return parse_bytes(dask.config.get("distributed.scheduler.default-data-size"))
# Note: can't specify __slots__ manually to enable slots in Python <3.10 in a @dataclass
# that defines any default values
dc_slots = {"slots": True} if sys.version_info >= (3, 10) else {}
@dataclass(repr=False, eq=False, **dc_slots)
class TaskState:
"""Holds volatile state relating to an individual Dask task.
Not to be confused with :class:`distributed.scheduler.TaskState`, which holds
similar information on the scheduler side.
"""
#: Task key. Mandatory.
key: str
#: A named tuple containing the ``function``, ``args``, ``kwargs`` and ``task``
#: associated with this `TaskState` instance. This defaults to ``None`` and can
#: remain empty if it is a dependency that this worker will receive from another
#: worker.
run_spec: SerializedTask | None = None
#: The data needed by this key to run
dependencies: set[TaskState] = field(default_factory=set)
#: The keys that use this dependency
dependents: set[TaskState] = field(default_factory=set)
#: Subset of dependencies that are not in memory
waiting_for_data: set[TaskState] = field(default_factory=set)
#: Subset of dependents that are not in memory
waiters: set[TaskState] = field(default_factory=set)
#: The current state of the task
state: TaskStateState = "released"
#: The previous state of the task. It is not None iff :attr:`state` in
#: (cancelled, resumed).
previous: Literal["executing", "long-running", "flight", None] = None
#: The next state of the task. It is not None iff :attr:`state` == resumed.
next: Literal["fetch", "waiting", None] = None
#: Expected duration of the task
duration: float | None = None
#: The priority this task given by the scheduler. Determines run order.
priority: tuple[int, ...] | None = None
#: Addresses of workers that we believe have this data
who_has: set[str] = field(default_factory=set)
#: The worker that current task data is coming from if task is in flight
coming_from: str | None = None
#: Abstract resources required to run a task
resource_restrictions: dict[str, float] = field(default_factory=dict)
#: The exception caused by running a task if it erred (serialized)
exception: Serialize | None = None
#: The traceback caused by running a task if it erred (serialized)
traceback: Serialize | None = None
#: string representation of exception
exception_text: str = ""
#: string representation of traceback
traceback_text: str = ""
#: The type of a particular piece of data
type: type | None = None
#: The number of times a dependency has not been where we expected it
suspicious_count: int = 0
#: Log of transfer, load, and compute times for a task
startstops: list[StartStop] = field(default_factory=list)
#: Time at which task begins running
start_time: float | None = None
#: Time at which task finishes running
stop_time: float | None = None
#: Metadata related to the task.
#: Stored metadata should be msgpack serializable (e.g. int, string, list, dict).
metadata: dict = field(default_factory=dict)
#: The size of the value of the task, if in memory
nbytes: int | None = None
#: Arbitrary task annotations
annotations: dict | None = None
#: True if the :meth:`~WorkerBase.execute` or :meth:`~WorkerBase.gather_dep`
#: coroutine servicing this task completed; False otherwise. This flag changes
#: the behaviour of transitions out of the ``executing``, ``flight`` etc. states.
done: bool = False
_instances: ClassVar[weakref.WeakSet[TaskState]] = weakref.WeakSet()
# Support for weakrefs to a class with __slots__
__weakref__: Any = field(init=False)
def __post_init__(self) -> None:
TaskState._instances.add(self)
def __repr__(self) -> str:
if self.state == "cancelled":
state = f"cancelled({self.previous})"
elif self.state == "resumed":
state = f"resumed({self.previous}->{self.next})"
else:
state = self.state
return f"<TaskState {self.key!r} {state}>"
def __hash__(self) -> int:
"""Override dataclass __hash__, reverting to the default behaviour
hash(o) == id(o).
Note that we also defined @dataclass(eq=False), which reverts to the default
behaviour (a == b) == (a is b).
On first thought, it would make sense to use TaskState.key for equality and
hashing. However, a task may be forgotten and a new TaskState object with the
same key may be created in its place later on. In the Worker state, you should
never have multiple TaskState objects with the same key; see
WorkerState.validate_state for relevant checks. We can't assert the same thing
in __eq__ though, as multiple objects with the same key may appear in
TaskState._instances for a brief period of time.
"""
return id(self)
def get_nbytes(self) -> int:
nbytes = self.nbytes
return nbytes if nbytes is not None else _default_data_size()
def _to_dict_no_nest(self, *, exclude: Container[str] = ()) -> dict:
"""Dictionary representation for debugging purposes.
Not type stable and not intended for roundtrips.
See also
--------
Client.dump_cluster_state
distributed.utils.recursive_to_dict
Notes
-----
This class uses ``_to_dict_no_nest`` instead of ``_to_dict``.
When a task references another task, just print the task repr. All tasks
should neatly appear under Worker.tasks. This also prevents a RecursionError
during particularly heavy loads, which have been observed to happen whenever
there's an acyclic dependency chain of ~200+ tasks.
"""
out = recursive_to_dict(self, exclude=exclude, members=True)
# Remove all Nones and empty containers
return {k: v for k, v in out.items() if v}
def is_protected(self) -> bool:
return self.state in PROCESSING or any(
dep_ts.state in PROCESSING for dep_ts in self.dependents
)
@dataclass
class Instruction:
"""Command from the worker state machine to the Worker, in response to an event"""
__slots__ = ("stimulus_id",)
stimulus_id: str
@classmethod
def match(cls, **kwargs: Any) -> _InstructionMatch:
"""Generate a partial match to compare against an Instruction instance.
The typical usage is to compare a list of instructions returned by
:meth:`WorkerState.handle_stimulus` or in :attr:`WorkerState.stimulus_log` vs.
an expected list of matches.
Examples
--------
.. code-block:: python
instructions = ws.handle_stimulus(...)
assert instructions == [
TaskFinishedMsg.match(key="x"),
...
]
"""
return _InstructionMatch(cls, **kwargs)
def __eq__(self, other: object) -> bool:
if isinstance(other, _InstructionMatch):
return other == self
else:
# Revert to default dataclass behaviour
return super().__eq__(other)
class _InstructionMatch:
"""Utility class, to be used to test an instructions list.
See :meth:`Instruction.match`.
"""
cls: type[Instruction]
kwargs: dict[str, Any]
def __init__(self, cls: type[Instruction], **kwargs: Any):
self.cls = cls
self.kwargs = kwargs
def __repr__(self) -> str:
cls_str = self.cls.__name__
kwargs_str = ", ".join(f"{k}={v}" for k, v in self.kwargs.items())
return f"{cls_str}({kwargs_str}) (partial match)"
def __eq__(self, other: object) -> bool:
if type(other) is not self.cls:
return False
return all(getattr(other, k) == v for k, v in self.kwargs.items())
@dataclass
class GatherDep(Instruction):
__slots__ = ("worker", "to_gather", "total_nbytes")
worker: str
to_gather: set[str]
total_nbytes: int
@dataclass
class Execute(Instruction):
__slots__ = ("key",)
key: str
@dataclass
class RetryBusyWorkerLater(Instruction):
__slots__ = ("worker",)
worker: str
@dataclass
class DigestMetric(Instruction):
__slots__ = ("name", "value")
name: str
value: float
class SendMessageToScheduler(Instruction):
#: Matches a key in Scheduler.stream_handlers
op: ClassVar[str]
__slots__ = ()
def to_dict(self) -> dict[str, Any]:
"""Convert object to dict so that it can be serialized with msgpack"""
d = {k: getattr(self, k) for k in self.__annotations__}
d["op"] = self.op
d["stimulus_id"] = self.stimulus_id
return d
@dataclass
class TaskFinishedMsg(SendMessageToScheduler):
op = "task-finished"
key: str
nbytes: int | None
type: bytes # serialized class
typename: str
metadata: dict
thread: int | None
startstops: list[StartStop]
__slots__ = tuple(__annotations__)
def to_dict(self) -> dict[str, Any]:
d = super().to_dict()
d["status"] = "OK"
return d
@dataclass
class TaskErredMsg(SendMessageToScheduler):
op = "task-erred"
key: str
exception: Serialize
traceback: Serialize | None
exception_text: str
traceback_text: str
thread: int | None
startstops: list[StartStop]
__slots__ = tuple(__annotations__)
def to_dict(self) -> dict[str, Any]:
d = super().to_dict()
d["status"] = "error"
return d
@staticmethod
def from_task(
ts: TaskState, stimulus_id: str, thread: int | None = None
) -> TaskErredMsg:
assert ts.exception
return TaskErredMsg(
key=ts.key,
exception=ts.exception,
traceback=ts.traceback,
exception_text=ts.exception_text,
traceback_text=ts.traceback_text,
thread=thread,
startstops=ts.startstops,
stimulus_id=stimulus_id,
)
@dataclass
class ReleaseWorkerDataMsg(SendMessageToScheduler):
op = "release-worker-data"
__slots__ = ("key",)
key: str
# Not to be confused with RescheduleEvent below or the distributed.Reschedule Exception
@dataclass
class RescheduleMsg(SendMessageToScheduler):
op = "reschedule"
__slots__ = ("key",)
key: str
@dataclass
class LongRunningMsg(SendMessageToScheduler):
op = "long-running"
__slots__ = ("key", "compute_duration")
key: str
compute_duration: float | None
@dataclass
class AddKeysMsg(SendMessageToScheduler):
op = "add-keys"
__slots__ = ("keys",)
keys: Collection[str]
@dataclass
class RequestRefreshWhoHasMsg(SendMessageToScheduler):
"""Worker -> Scheduler asynchronous request for updated who_has information.
Not to be confused with the scheduler.who_has synchronous RPC call, which is used
by the Client.
See also
--------
RefreshWhoHasEvent
distributed.scheduler.Scheduler.request_refresh_who_has
distributed.client.Client.who_has
distributed.scheduler.Scheduler.get_who_has
"""
op = "request-refresh-who-has"
__slots__ = ("keys",)
keys: Collection[str]
@dataclass
class StealResponseMsg(SendMessageToScheduler):
"""Worker->Scheduler response to ``{op: steal-request}``
See also
--------
StealRequestEvent
"""
op = "steal-response"
__slots__ = ("key", "state")
key: str
state: TaskStateState | None
@dataclass
class StateMachineEvent:
"""Base abstract class for all stimuli that can modify the worker state"""
__slots__ = ("stimulus_id", "handled")
#: Unique ID of the event
stimulus_id: str
#: timestamp of when the event was handled by the worker
# TODO Switch to @dataclass(slots=True), uncomment the line below, and remove the
# __new__ method (requires Python >=3.10)
# handled: float | None = field(init=False, default=None)
_classes: ClassVar[dict[str, type[StateMachineEvent]]] = {}
def __new__(cls, *args: Any, **kwargs: Any) -> StateMachineEvent:
"""Hack to initialize the ``handled`` attribute in Python <3.10"""
self = object.__new__(cls)
self.handled = None
return self
def __init_subclass__(cls) -> None:
StateMachineEvent._classes[cls.__name__] = cls
def to_loggable(self, *, handled: float) -> StateMachineEvent:
"""Produce a variant version of self that is small enough to be stored in memory
in the medium term and contains meaningful information for debugging
"""
self.handled: float | None = handled
return self
def _to_dict(self, *, exclude: Container[str] = ()) -> dict:
"""Dictionary representation for debugging purposes.
See also
--------
distributed.utils.recursive_to_dict
"""
info = {"cls": type(self).__name__}
for k in dir(self):
if k in exclude or k.startswith("_"):
continue
v = getattr(self, k)
if not callable(v):
info[k] = v
return recursive_to_dict(info, exclude=exclude)
@staticmethod
def from_dict(d: dict) -> StateMachineEvent:
"""Convert the output of ``recursive_to_dict`` back into the original object.
The output object is meaningful for the purpose of rebuilding the state machine,
but not necessarily identical to the original.
"""
kwargs = d.copy()
cls = StateMachineEvent._classes[kwargs.pop("cls")]
handled = kwargs.pop("handled")
inst = cls(**kwargs)
inst.handled = handled
inst._after_from_dict()
return inst
def _after_from_dict(self) -> None:
"""Optional post-processing after an instance is created by ``from_dict``"""
@dataclass
class PauseEvent(StateMachineEvent):
__slots__ = ()
@dataclass
class UnpauseEvent(StateMachineEvent):
__slots__ = ()
@dataclass
class RetryBusyWorkerEvent(StateMachineEvent):
__slots__ = ("worker",)
worker: str
@dataclass
class GatherDepDoneEvent(StateMachineEvent):
""":class:`GatherDep` instruction terminated (abstract base class)"""
__slots__ = ("worker", "total_nbytes")
worker: str
total_nbytes: int # Must be the same as in GatherDep instruction
@dataclass
class GatherDepSuccessEvent(GatherDepDoneEvent):
""":class:`GatherDep` instruction terminated:
remote worker fetched successfully
"""
__slots__ = ("data",)
data: dict[str, object] # There may be less keys than in GatherDep
def to_loggable(self, *, handled: float) -> StateMachineEvent:
out = copy(self)
out.handled = handled
out.data = {k: None for k in self.data}
return out
def _after_from_dict(self) -> None:
self.data = {k: None for k in self.data}
@dataclass
class GatherDepBusyEvent(GatherDepDoneEvent):
""":class:`GatherDep` instruction terminated:
remote worker is busy
"""
__slots__ = ()
@dataclass
class GatherDepNetworkFailureEvent(GatherDepDoneEvent):
""":class:`GatherDep` instruction terminated:
network failure while trying to communicate with remote worker
"""
__slots__ = ()
@dataclass
class GatherDepFailureEvent(GatherDepDoneEvent):
"""class:`GatherDep` instruction terminated:
generic error raised (not a network failure); e.g. data failed to deserialize.
"""
exception: Serialize
traceback: Serialize | None
exception_text: str
traceback_text: str
__slots__ = tuple(__annotations__)
def _after_from_dict(self) -> None:
self.exception = Serialize(Exception())
self.traceback = None
@classmethod
def from_exception(
cls,
err: BaseException,
*,
worker: str,
total_nbytes: int,
stimulus_id: str,
) -> GatherDepFailureEvent:
msg = error_message(err)
return cls(
worker=worker,
total_nbytes=total_nbytes,
exception=msg["exception"],
traceback=msg["traceback"],
exception_text=msg["exception_text"],
traceback_text=msg["traceback_text"],
stimulus_id=stimulus_id,
)
@dataclass
class ComputeTaskEvent(StateMachineEvent):
key: str
who_has: dict[str, Collection[str]]
nbytes: dict[str, int]
priority: tuple[int, ...]
duration: float
run_spec: SerializedTask | None
function: bytes | None
args: bytes | tuple | list | None | None
kwargs: bytes | dict[str, Any] | None
resource_restrictions: dict[str, float]
actor: bool
annotations: dict
__slots__ = tuple(__annotations__)
def __post_init__(self) -> None:
# Fixes after msgpack decode
if isinstance(self.priority, list): # type: ignore[unreachable]
self.priority = tuple(self.priority) # type: ignore[unreachable]
if self.function is not None:
assert self.run_spec is None
self.run_spec = SerializedTask(
function=self.function, args=self.args, kwargs=self.kwargs
)
elif not isinstance(self.run_spec, SerializedTask):
self.run_spec = SerializedTask(task=self.run_spec)
def _to_dict(self, *, exclude: Container[str] = ()) -> dict:
return StateMachineEvent._to_dict(self._clean(), exclude=exclude)
def _clean(self) -> StateMachineEvent:
out = copy(self)
out.function = None
out.kwargs = None
out.args = None
out.run_spec = SerializedTask(task=None, function=None, args=None, kwargs=None)
return out
def to_loggable(self, *, handled: float) -> StateMachineEvent:
out = self._clean()
out.handled = handled
return out
def _after_from_dict(self) -> None:
self.run_spec = SerializedTask(task=None, function=None, args=None, kwargs=None)
@staticmethod
def dummy(
key: str,
*,
who_has: dict[str, Collection[str]] | None = None,
nbytes: dict[str, int] | None = None,
priority: tuple[int, ...] = (0,),
duration: float = 1.0,
resource_restrictions: dict[str, float] | None = None,
actor: bool = False,
annotations: dict | None = None,
stimulus_id: str,
) -> ComputeTaskEvent:
"""Build a dummy event, with most attributes set to a reasonable default.
This is a convenience method to be used in unit testing only.
"""
return ComputeTaskEvent(
key=key,
who_has=who_has or {},
nbytes=nbytes or {k: 1 for k in who_has or ()},
priority=priority,
duration=duration,
run_spec=None,
function=None,
args=None,
kwargs=None,
resource_restrictions=resource_restrictions or {},
actor=actor,
annotations=annotations or {},
stimulus_id=stimulus_id,
)
@dataclass
class ExecuteDoneEvent(StateMachineEvent):
"""Abstract base event for all the possible outcomes of a :class:`Compute`
instruction
"""
key: str
__slots__ = ("key",)
@dataclass
class ExecuteSuccessEvent(ExecuteDoneEvent):
value: object
start: float
stop: float
nbytes: int
type: type | None
__slots__ = tuple(__annotations__)
def to_loggable(self, *, handled: float) -> StateMachineEvent:
out = copy(self)
out.handled = handled
out.value = None
return out
def _to_dict(self, *, exclude: Container[str] = ()) -> dict:
d = super()._to_dict(exclude=exclude)
# This is excluded by the parent class as it is a callable
if "type" not in exclude:
d["type"] = str(self.type)
return d
def _after_from_dict(self) -> None:
self.value = None
self.type = None
@staticmethod
def dummy(
key: str,
value: object = None,
*,
nbytes: int = 1,
stimulus_id: str,
) -> ExecuteSuccessEvent:
"""Build a dummy event, with most attributes set to a reasonable default.
This is a convenience method to be used in unit testing only.
"""
return ExecuteSuccessEvent(
key=key,
value=value,
start=0.0,
stop=1.0,
nbytes=nbytes,
type=None,
stimulus_id=stimulus_id,
)
@dataclass
class ExecuteFailureEvent(ExecuteDoneEvent):
start: float | None
stop: float | None
exception: Serialize
traceback: Serialize | None
exception_text: str
traceback_text: str
__slots__ = tuple(__annotations__)
def _after_from_dict(self) -> None:
self.exception = Serialize(Exception())
self.traceback = None
@classmethod
def from_exception(
cls,
err_or_msg: BaseException | ErrorMessage,
*,
key: str,
start: float | None = None,
stop: float | None = None,
stimulus_id: str,
) -> ExecuteFailureEvent:
if isinstance(err_or_msg, dict):
msg = err_or_msg
else:
msg = error_message(err_or_msg)
return cls(
key=key,
start=start,
stop=stop,
exception=msg["exception"],
traceback=msg["traceback"],
exception_text=msg["exception_text"],
traceback_text=msg["traceback_text"],
stimulus_id=stimulus_id,
)
@staticmethod
def dummy(
key: str,
*,
stimulus_id: str,
) -> ExecuteFailureEvent:
"""Build a dummy event, with most attributes set to a reasonable default.
This is a convenience method to be used in unit testing only.
"""
return ExecuteFailureEvent(
key=key,
start=None,
stop=None,
exception=Serialize(None),
traceback=None,
exception_text="",
traceback_text="",
stimulus_id=stimulus_id,
)
# Not to be confused with RescheduleMsg above or the distributed.Reschedule Exception
@dataclass
class RescheduleEvent(ExecuteDoneEvent):
__slots__ = ()
@staticmethod
def dummy(key: str, *, stimulus_id: str) -> RescheduleEvent:
"""Build an event. This method exists for compatibility with the other
ExecuteDoneEvent subclasses.
"""
return RescheduleEvent(key=key, stimulus_id=stimulus_id)
@dataclass
class CancelComputeEvent(StateMachineEvent):
__slots__ = ("key",)
key: str
@dataclass
class FindMissingEvent(StateMachineEvent):
__slots__ = ()
@dataclass
class RefreshWhoHasEvent(StateMachineEvent):
"""Scheduler -> Worker message containing updated who_has information.
See also
--------
RequestRefreshWhoHasMsg
"""
__slots__ = ("who_has",)
# {key: [worker address, ...]}
who_has: dict[str, Collection[str]]
@dataclass
class AcquireReplicasEvent(StateMachineEvent):
__slots__ = ("who_has", "nbytes")
who_has: dict[str, Collection[str]]
nbytes: dict[str, int]
@dataclass
class RemoveReplicasEvent(StateMachineEvent):
__slots__ = ("keys",)
keys: Collection[str]
@dataclass
class FreeKeysEvent(StateMachineEvent):
__slots__ = ("keys",)
keys: Collection[str]
@dataclass
class StealRequestEvent(StateMachineEvent):
"""Event that requests a worker to release a key because it's now being computed
somewhere else.
See also
--------
StealResponseMsg
"""
__slots__ = ("key",)
key: str
@dataclass
class UpdateDataEvent(StateMachineEvent):
__slots__ = ("data", "report")
data: dict[str, object]
report: bool
def to_loggable(self, *, handled: float) -> StateMachineEvent:
out = copy(self)
out.handled = handled
out.data = dict.fromkeys(self.data)
return out
@dataclass
class SecedeEvent(StateMachineEvent):
__slots__ = ("key", "compute_duration")
key: str
compute_duration: float
if TYPE_CHECKING:
# TODO remove quotes (requires Python >=3.9)
# TODO get out of TYPE_CHECKING (requires Python >=3.10)
# {TaskState -> finish: TaskStateState | (finish: TaskStateState, transition *args)}
# Not to be confused with distributed.scheduler.Recs
Recs: TypeAlias = "dict[TaskState, TaskStateState | tuple]"
Instructions: TypeAlias = "list[Instruction]"
RecsInstrs: TypeAlias = "tuple[Recs, Instructions]"
else:
Recs = dict
Instructions = list
RecsInstrs = tuple
def merge_recs_instructions(*args: RecsInstrs) -> RecsInstrs:
"""Merge multiple (recommendations, instructions) tuples.
Collisions in recommendations are only allowed if identical.
"""
recs: Recs = {}
instr: Instructions = []
for recs_i, instr_i in args:
for ts, finish in recs_i.items():
if ts in recs and recs[ts] != finish:
raise RecommendationsConflict(
f"Mismatched recommendations for {ts.key}: {recs[ts]} vs. {finish}"
)
recs[ts] = finish
instr += instr_i
return recs, instr
class WorkerState:
"""State machine encapsulating the lifetime of all tasks on a worker.
Not to be confused with :class:`distributed.scheduler.WorkerState`.
.. note::
The data attributes of this class are implementation details and may be
changed without a deprecation cycle.
.. warning::
The attributes of this class are all heavily correlated with each other.
*Do not* modify them directly, *ever*, as it is extremely easy to obtain a broken
state this way, which in turn will likely result in cluster-wide deadlocks.
The state should be exclusively mutated through :meth:`handle_stimulus`.
"""
#: Worker <IP address>:<port>. This is used in decision-making by the state machine,
#: e.g. to determine if a peer worker is running on the same host or not.
#: This attribute may not be known when the WorkerState is initialised. It *must* be
#: set before the first call to :meth:`handle_stimulus`.
address: str
#: ``{key: TaskState}``. The tasks currently executing on this worker (and any
#: dependencies of those tasks)
tasks: dict[str, TaskState]
#: ``{ts.key: thread ID}``. This collection is shared by reference between
#: :class:`~distributed.worker.Worker` and this class. While the WorkerState is
#: thread-agnostic, it still needs access to this information in some cases.
#: This collection is populated by :meth:`distributed.worker.Worker.execute`.
#: It does not *need* to be populated for the WorkerState to work.
threads: dict[str, int]
#: In-memory tasks data. This collection is shared by reference between
#: :class:`~distributed.worker.Worker`,
#: :class:`~distributed.worker_memory.WorkerMemoryManager`, and this class.
data: MutableMapping[str, object]
#: ``{name: worker plugin}``. This collection is shared by reference between
#: :class:`~distributed.worker.Worker` and this class. The Worker managed adding and
#: removing plugins, while the WorkerState invokes the ``WorkerPlugin.transition``
#: method, is available.
plugins: dict[str, WorkerPlugin]
#: Priority heap of tasks that are ready to run and have no resource constrains.
#: Mutually exclusive with :attr:`constrained`.
ready: HeapSet[TaskState]
#: Priority heap of tasks that are ready to run, but are waiting on abstract
#: resources like GPUs. Mutually exclusive with :attr:`ready`.
#: See :attr:`available_resources` and :doc:`resources`.
constrained: HeapSet[TaskState]
#: Number of tasks that can be executing in parallel.
#: At any given time, :meth:`executing_count` <= nthreads.
nthreads: int
#: True if the state machine should start executing more tasks and fetch
#: dependencies whenever a slot is available. This property must be kept aligned
#: with the Worker: ``WorkerState.running == (Worker.status is Status.running)``.
running: bool
#: Tasks that are currently waiting for data
waiting: set[TaskState]
#: ``{worker address: {ts.key, ...}``.
#: The data that we care about that we think a worker has
has_what: defaultdict[str, set[str]]
#: The tasks which still require data in order to execute and are in memory on at
#: least another worker, prioritized as per-worker heaps. All and only tasks with
#: ``TaskState.state == 'fetch'`` are in this collection. A :class:`TaskState` with
#: multiple entries in :attr:`~TaskState.who_has` will appear multiple times here.
data_needed: defaultdict[str, HeapSet[TaskState]]
#: Total number of tasks in fetch state. If a task is in more than one data_needed
#: heap, it's only counted once.
fetch_count: int
#: Number of bytes to gather from the same worker in a single call to
#: :meth:`BaseWorker.gather_dep`. Multiple small tasks that can be gathered from the
#: same worker will be batched in a single instruction as long as their combined
#: size doesn't exceed this value. If the first task to be gathered exceeds this
# limit, it will still be gathered to ensure progress. Hence, this limit is not absolute.
transfer_message_bytes_limit: float
#: All and only tasks with ``TaskState.state == 'missing'``.
missing_dep_flight: set[TaskState]
#: Tasks that are coming to us in current peer-to-peer connections.
#:
#: This set includes exclusively tasks with :attr:`~TaskState.state` == 'flight' as
#: well as tasks with :attr:`~TaskState.state` in ('cancelled', 'resumed') and
#: :attr:`~TaskState.previous` == 'flight`.
#:
#: See also :meth:`in_flight_tasks_count`.
in_flight_tasks: set[TaskState]
#: ``{worker address: {ts.key, ...}}``
#: The workers from which we are currently gathering data and the dependencies we
#: expect from those connections. Workers in this dict won't be asked for additional
#: dependencies until the current query returns.
in_flight_workers: dict[str, set[str]]
#: Current total size of open data transfers from other workers
transfer_incoming_bytes: int
#: Maximum number of concurrent incoming data transfers from other workers.
#: See also :attr:`distributed.worker.Worker.transfer_outgoing_count_limit`.
transfer_incoming_count_limit: int
#: Total number of data transfers from other workers since the worker was started.
transfer_incoming_count_total: int
#: Ignore :attr:`transfer_incoming_count_limit` as long as :attr:`transfer_incoming_bytes` is
#: less than this value.
transfer_incoming_bytes_throttle_threshold: int
#: Peer workers that recently returned a busy status. Workers in this set won't be
#: asked for additional dependencies for some time.
busy_workers: set[str]
#: Counter that decreases every time the compute-task handler is invoked by the
#: Scheduler. It is appended to :attr:`TaskState.priority` and acts as a
#: tie-breaker between tasks that have the same priority on the Scheduler,
#: determining a last-in-first-out order between them.
generation: int
#: ``{resource name: amount}``. Total resources available for task execution.
#: See :doc: `resources`.
total_resources: dict[str, float]
#: ``{resource name: amount}``. Current resources that aren't being currently
#: consumed by task execution. Always less or equal to :attr:`total_resources`.
#: See :doc:`resources`.
available_resources: dict[str, float]
#: Set of tasks that are currently running.
#:
#: This set includes exclusively tasks with :attr:`~TaskState.state` == 'executing'
#: as well as tasks with :attr:`~TaskState.state` in ('cancelled', 'resumed') and
#: :attr:`~TaskState.previous` == 'executing`.
#:
#: See also :meth:`executing_count` and :attr:`long_running`.
executing: set[TaskState]
#: Set of tasks that are currently running and have called
#: :func:`~distributed.secede`, so they no longer count towards the maximum number
#: of concurrent tasks (nthreads).
#: These tasks do not appear in the :attr:`executing` set.
#:
#: This set includes exclusively tasks with
#: :attr:`~TaskState.state` == 'long-running' as well as tasks with
#: :attr:`~TaskState.state` in ('cancelled', 'resumed') and
#: :attr:`~TaskState.previous` == 'long-running`.
long_running: set[TaskState]
#: A number of tasks that this worker has run in its lifetime; this includes failed
#: and cancelled tasks. See also :meth:`executing_count`.
executed_count: int
#: Total size of all tasks in memory
nbytes: int
#: Actor tasks. See :doc:`actors`.
actors: dict[str, object]
#: Transition log: ``[(..., stimulus_id: str | None, timestamp: float), ...]``
#: The number of stimuli logged is capped.
#: See also :meth:`story` and :attr:`stimulus_log`.
log: deque[tuple]
#: Log of all stimuli received by :meth:`handle_stimulus`.
#: The number of events logged is capped.
#: See also :attr:`log` and :meth:`stimulus_story`.
stimulus_log: deque[StateMachineEvent]
#: If True, enable expensive internal consistency check.
#: Typically disabled in production.
validate: bool
#: Total number of state transitions so far.
#: See also :attr:`log` and :attr:`transition_counter_max`.
transition_counter: int
#: Raise an error if the :attr:`transition_counter` ever reaches this value.
#: This is meant for debugging only, to catch infinite recursion loops.
#: In production, it should always be set to False.
transition_counter_max: int | Literal[False]
#: Limit of bytes for incoming data transfers; this is used for throttling.
transfer_incoming_bytes_limit: float
#: Statically-seeded random state, used to guarantee determinism whenever a
#: pseudo-random choice is required
rng: random.Random
__slots__ = tuple(__annotations__)
def __init__(
self,
*,
nthreads: int = 1,
address: str | None = None,
data: MutableMapping[str, object] | None = None,
threads: dict[str, int] | None = None,
plugins: dict[str, WorkerPlugin] | None = None,
resources: Mapping[str, float] | None = None,
transfer_incoming_count_limit: int = 9999,
validate: bool = True,
transition_counter_max: int | Literal[False] = False,
transfer_incoming_bytes_limit: float = math.inf,
transfer_message_bytes_limit: float = math.inf,
):
self.nthreads = nthreads
# address may not be known yet when the State Machine is initialised.
# Raise AttributeError if a method tries reading it before it's been set.
if address:
self.address = address
# These collections are normally passed by reference by the Worker.
# For the sake of convenience, create independent ones during unit tests.
self.data = data if data is not None else {}
self.threads = threads if threads is not None else {}
self.plugins = plugins if plugins is not None else {}
self.total_resources = dict(resources) if resources is not None else {}
self.available_resources = self.total_resources.copy()
self.validate = validate
self.tasks = {}
self.running = True
self.waiting = set()
self.has_what = defaultdict(set)
self.data_needed = defaultdict(
partial(HeapSet[TaskState], key=operator.attrgetter("priority"))
)
self.fetch_count = 0
self.in_flight_workers = {}
self.busy_workers = set()
self.transfer_incoming_count_limit = transfer_incoming_count_limit
self.transfer_incoming_count_total = 0
self.transfer_incoming_bytes_throttle_threshold = int(10e6)
self.transfer_incoming_bytes = 0
self.missing_dep_flight = set()
self.generation = 0
self.ready = HeapSet(key=operator.attrgetter("priority"))
self.constrained = HeapSet(key=operator.attrgetter("priority"))
self.executing = set()
self.in_flight_tasks = set()
self.nbytes = 0
self.executed_count = 0
self.long_running = set()
self.transfer_message_bytes_limit = transfer_message_bytes_limit
self.log = deque(maxlen=100_000)
self.stimulus_log = deque(maxlen=10_000)
self.transition_counter = 0
self.transition_counter_max = transition_counter_max
self.transfer_incoming_bytes_limit = transfer_incoming_bytes_limit
self.actors = {}
self.rng = random.Random(0)
def handle_stimulus(self, *stims: StateMachineEvent) -> Instructions:
"""Process one or more external events, transition relevant tasks to new states,
and return a list of instructions to be executed as a consequence.
See also
--------
BaseWorker.handle_stimulus
"""
instructions = []
handled = time()
for stim in stims:
if not isinstance(stim, FindMissingEvent):
self.stimulus_log.append(stim.to_loggable(handled=handled))
recs, instr = self._handle_event(stim)
instructions += instr
instructions += self._transitions(recs, stimulus_id=stim.stimulus_id)
return instructions
#############
# Accessors #
#############
@property
def executing_count(self) -> int:
"""Count of tasks currently executing on this worker and counting towards the
maximum number of threads.
It includes cancelled tasks, but does not include long running (a.k.a. seceded)
tasks.
See also
--------
WorkerState.executing
WorkerState.executed_count
WorkerState.nthreads
WorkerState.all_running_tasks
"""
return len(self.executing)
@property
def all_running_tasks(self) -> set[TaskState]:
"""All tasks that are currently occupying a thread. They may or may not count
towards the maximum number of threads.
These are:
- ts.status in (executing, long-running)
- ts.status in (cancelled, resumed) and ts.previous in (executing, long-running)
See also
--------
WorkerState.executing_count
"""
# Note: cancelled and resumed tasks are still in either of these sets
return self.executing | self.long_running
@property
def in_flight_tasks_count(self) -> int:
"""Number of tasks currently being replicated from other workers to this one.
See also
--------
WorkerState.in_flight_tasks
"""
return len(self.in_flight_tasks)
@property
def transfer_incoming_count(self) -> int:
"""Current number of open data transfers from other workers.
See also
--------
WorkerState.in_flight_workers
"""
return len(self.in_flight_workers)
#########################
# Shared helper methods #
#########################
def _ensure_task_exists(
self, key: str, *, priority: tuple[int, ...], stimulus_id: str
) -> TaskState:
try:
ts = self.tasks[key]
logger.debug("Data task %s already known (stimulus_id=%s)", ts, stimulus_id)
except KeyError:
self.tasks[key] = ts = TaskState(key)
if not ts.priority:
assert priority
ts.priority = priority
self.log.append((key, "ensure-task-exists", ts.state, stimulus_id, time()))
return ts
def _update_who_has(self, who_has: Mapping[str, Collection[str]]) -> None:
for key, workers in who_has.items():
ts = self.tasks.get(key)
if not ts:
# The worker sent a refresh-who-has request to the scheduler but, by the
# time the answer comes back, some of the keys have been forgotten.
continue
workers = set(workers)
if self.address in workers:
workers.remove(self.address)
# This can only happen if rebalance() recently asked to release a key,
# but the RPC call hasn't returned yet. rebalance() is flagged as not
# being safe to run while the cluster is not at rest and has already
# been penned in to be redesigned on top of the AMM.
# It is not necessary to send a message back to the
# scheduler here, because it is guaranteed that there's already a
# release-worker-data message in transit to it.
if ts.state != "memory":
logger.debug( # pragma: nocover
"Scheduler claims worker %s holds data for task %s, "
"which is not true.",
self.address,
ts,
)
if ts.who_has == workers:
continue
for worker in ts.who_has - workers:
self.has_what[worker].remove(key)
if ts.state == "fetch":
self.data_needed[worker].remove(ts)
for worker in workers - ts.who_has:
self.has_what[worker].add(key)
if ts.state == "fetch":
self.data_needed[worker].add(ts)
ts.who_has = workers
def _purge_state(self, ts: TaskState) -> None:
"""Ensure that TaskState attributes are reset to a neutral default and
Worker-level state associated to the provided key is cleared (e.g.
who_has)
This is idempotent
"""
logger.debug("Purge task: %s", ts)
key = ts.key
self.data.pop(key, None)
self.actors.pop(key, None)
for worker in ts.who_has:
self.has_what[worker].discard(ts.key)
self.data_needed[worker].discard(ts)
ts.who_has.clear()
self.threads.pop(key, None)
for d in ts.dependencies:
ts.waiting_for_data.discard(d)
d.waiters.discard(ts)
ts.waiting_for_data.clear()
ts.nbytes = None
ts.previous = None
ts.next = None
ts.done = False
ts.coming_from = None
ts.exception = None
ts.traceback = None
ts.traceback_text = ""
ts.traceback_text = ""
self.missing_dep_flight.discard(ts)
self.ready.discard(ts)
self.constrained.discard(ts)
self.executing.discard(ts)
self.long_running.discard(ts)
self.in_flight_tasks.discard(ts)
self.waiting.discard(ts)
def _should_throttle_incoming_transfers(self) -> bool:
"""Decides whether the WorkerState should throttle data transfers from other workers.
Returns
-------
* True if the number of incoming data transfers reached its limit
and the size of incoming data transfers reached the minimum threshold for throttling
* True if the size of incoming data transfers reached its limit
* False otherwise
"""
reached_count_limit = (
self.transfer_incoming_count >= self.transfer_incoming_count_limit
)
reached_throttle_threshold = (
self.transfer_incoming_bytes
>= self.transfer_incoming_bytes_throttle_threshold
)
reached_bytes_limit = (
self.transfer_incoming_bytes >= self.transfer_incoming_bytes_limit
)
return reached_count_limit and reached_throttle_threshold or reached_bytes_limit
def _ensure_communicating(self, *, stimulus_id: str) -> RecsInstrs:
"""Transition tasks from fetch to flight, until there are no more tasks in fetch
state or a threshold has been reached.
"""
if not self.running or not self.data_needed:
return {}, []
if self._should_throttle_incoming_transfers():
return {}, []
recommendations: Recs = {}
instructions: Instructions = []
for worker, available_tasks in self._select_workers_for_gather():
assert worker != self.address
to_gather_tasks, message_nbytes = self._select_keys_for_gather(
available_tasks
)
# We always load at least one task
assert to_gather_tasks or self.transfer_incoming_bytes
# ...but that task might be selected in the previous iteration of the loop
if not to_gather_tasks:
break
to_gather_keys = {ts.key for ts in to_gather_tasks}
logger.debug(
"Gathering %d tasks from %s; %d more remain. "
"Pending workers: %d; connections: %d/%d; busy: %d",
len(to_gather_tasks),
worker,
len(available_tasks),
len(self.data_needed),
self.transfer_incoming_count,
self.transfer_incoming_count_limit,
len(self.busy_workers),
)
self.log.append(
("gather-dependencies", worker, to_gather_keys, stimulus_id, time())
)
for ts in to_gather_tasks:
if self.validate:
assert ts.state == "fetch"
assert worker in ts.who_has
assert ts not in recommendations
recommendations[ts] = ("flight", worker)
# A single invocation of _ensure_communicating may generate up to one
# GatherDep instruction per worker. Multiple tasks from the same worker may
# be batched in the same instruction by _select_keys_for_gather. But once
# a worker has been selected for a GatherDep and added to in_flight_workers,
# it won't be selected again until the gather completes.
instructions.append(
GatherDep(
worker=worker,
to_gather=to_gather_keys,
total_nbytes=message_nbytes,
stimulus_id=stimulus_id,
)
)
self.in_flight_workers[worker] = to_gather_keys
self.transfer_incoming_count_total += 1
self.transfer_incoming_bytes += message_nbytes
if self._should_throttle_incoming_transfers():
break
return recommendations, instructions
def _select_workers_for_gather(self) -> Iterator[tuple[str, HeapSet[TaskState]]]:
"""Helper of _ensure_communicating.
Yield the peer workers and tasks in data_needed, sorted by:
1. By highest-priority task available across all workers
2. If tied, first by local peer workers, then remote. Note that, if a task is
replicated across multiple host, it may go in a tie with itself.
3. If still tied, by number of tasks available to be fetched from the host
(see note below)
4. If still tied, by a random element. This is statically seeded to guarantee
reproducibility.
FIXME https://github.com/dask/distributed/issues/6620
You won't get determinism when a single task is replicated on multiple
workers, because TaskState.who_has changes order at every interpreter
restart.
Omit workers that are either busy or in flight.
Remove peer workers with no tasks from data_needed.
Note
----
Instead of number of tasks, we could've measured total nbytes and/or number of
tasks that only exist on the worker. Raw number of tasks is cruder but simpler.
"""
host = get_address_host(self.address)
heap = []
for worker, tasks in list(self.data_needed.items()):
if not tasks:
del self.data_needed[worker]
continue
if worker in self.in_flight_workers or worker in self.busy_workers:
continue
heap.append(
(
tasks.peek().priority,
get_address_host(worker) != host, # False < True
-len(tasks),
self.rng.random(),
worker,
tasks,
)
)
heapq.heapify(heap)
while heap:
_, is_remote, ntasks_neg, rnd, worker, tasks = heapq.heappop(heap)
# The number of tasks and possibly the top priority task may have changed
# since the last sort, since _select_keys_for_gather may have removed tasks
# that are also replicated on a higher-priority worker.
if not tasks:
del self.data_needed[worker]
elif -ntasks_neg != len(tasks):
heapq.heappush(
heap,
(tasks.peek().priority, is_remote, -len(tasks), rnd, worker, tasks),
)
else:
yield worker, tasks
if not tasks: # _select_keys_for_gather just emptied it
del self.data_needed[worker]
def _select_keys_for_gather(
self, available: HeapSet[TaskState]
) -> tuple[list[TaskState], int]:
"""Helper of _ensure_communicating.
Fetch all tasks that are replicated on the target worker within a single
message, up to transfer_message_bytes_limit or until we reach the limit
for the size of incoming data transfers.
"""
to_gather: list[TaskState] = []
message_nbytes = 0
while available:
ts = available.peek()
if self._task_exceeds_transfer_limits(ts, message_nbytes):
break
for worker in ts.who_has:
# This also effectively pops from available
self.data_needed[worker].remove(ts)
to_gather.append(ts)
message_nbytes += ts.get_nbytes()
return to_gather, message_nbytes
def _task_exceeds_transfer_limits(self, ts: TaskState, message_nbytes: int) -> bool:
"""Would asking to gather this task exceed transfer limits?
Parameters
----------
ts
Candidate task for gathering
message_nbytes
Total number of bytes already scheduled for gathering in this message
Returns
-------
exceeds_limit
True if gathering the task would exceed limits, False otherwise
(in which case the task can be gathered).
"""
if self.transfer_incoming_bytes == 0 and message_nbytes == 0:
# When there is no other traffic, the top-priority task is fetched
# regardless of its size to ensure progress
return False
incoming_bytes_allowance = (
self.transfer_incoming_bytes_limit - self.transfer_incoming_bytes
)
# If message_nbytes == 0, i.e., this is the first task to gather in this
# message, ignore `self.transfer_message_bytes_limit` for the top-priority
# task to ensure progress. Otherwise:
if message_nbytes != 0:
incoming_bytes_allowance = (
min(
incoming_bytes_allowance,
self.transfer_message_bytes_limit,
)
- message_nbytes
)
return ts.get_nbytes() > incoming_bytes_allowance
def _ensure_computing(self) -> RecsInstrs:
if not self.running:
return {}, []
recs: Recs = {}
while len(self.executing) < self.nthreads:
ts = self._next_ready_task()
if not ts:
break
if self.validate:
assert ts.state in READY
assert ts not in recs
recs[ts] = "executing"
self._acquire_resources(ts)
self.executing.add(ts)
return recs, []
def _next_ready_task(self) -> TaskState | None:
"""Pop the top-priority task from self.ready or self.constrained"""
if self.ready and self.constrained:
tsr = self.ready.peek()
tsc = self.constrained.peek()
assert tsr.priority
assert tsc.priority
if tsc.priority < tsr.priority and self._resource_restrictions_satisfied(
tsc
):
return self.constrained.pop()
else:
return self.ready.pop()
elif self.ready:
return self.ready.pop()
elif self.constrained:
tsc = self.constrained.peek()
if self._resource_restrictions_satisfied(tsc):
return self.constrained.pop()
return None
def _get_task_finished_msg(
self, ts: TaskState, stimulus_id: str
) -> TaskFinishedMsg:
if ts.key not in self.data and ts.key not in self.actors:
raise RuntimeError(f"Task {ts} not ready")
typ = ts.type
if ts.nbytes is None or typ is None:
try:
value = self.data[ts.key]
except KeyError:
value = self.actors[ts.key]
ts.nbytes = sizeof(value)
typ = ts.type = type(value)
del value
try:
typ_serialized = pickle.dumps(typ)
except Exception:
# Some types fail pickling (example: _thread.lock objects),
# send their name as a best effort.
typ_serialized = pickle.dumps(typ.__name__)
return TaskFinishedMsg(
key=ts.key,
nbytes=ts.nbytes,
type=typ_serialized,
typename=typename(typ),
metadata=ts.metadata,
thread=self.threads.get(ts.key),
startstops=ts.startstops,
stimulus_id=stimulus_id,
)
def _put_key_in_memory(
self, ts: TaskState, value: object, *, stimulus_id: str
) -> RecsInstrs:
"""
Put a key into memory and set data related task state attributes.
On success, generate recommendations for dependents.
This method does not generate any scheduler messages since this method
cannot distinguish whether it has to be an `add-task` or a
`task-finished` signal. The caller is required to generate this message
on success.
Raises
------
Exception:
In case the data is put into the in-memory buffer and a serialization error
occurs during spilling, this re-raises that error. This has to be handled by
the caller since most callers generate scheduler messages on success (see
comment above) but we need to signal that this was not successful.
Can only trigger if distributed.worker.memory.target is enabled, the value
is individually larger than target * memory_limit, and the task is not an
actor.
"""
if ts.key in self.data:
ts.state = "memory"
return {}, []
recommendations: Recs = {}
instructions: Instructions = []
if ts.key in self.actors:
self.actors[ts.key] = value
else:
start = time()
self.data[ts.key] = value
stop = time()
if stop - start > 0.005:
ts.startstops.append(
{"action": "disk-write", "start": start, "stop": stop}
)
instructions.append(
DigestMetric(
# See metrics:
# - disk-load-duration
# - get-data-load-duration
# - disk-write-target-duration
# - disk-write-spill-duration
name="disk-write-target-duration",
value=stop - start,
stimulus_id=stimulus_id,
)
)
ts.state = "memory"
if ts.nbytes is None:
ts.nbytes = sizeof(value)
self.nbytes += ts.nbytes
ts.type = type(value)
for dep in ts.dependents:
dep.waiting_for_data.discard(ts)
if not dep.waiting_for_data and dep.state == "waiting":
recommendations[dep] = "ready"
self.log.append((ts.key, "put-in-memory", stimulus_id, time()))
return recommendations, instructions
###############
# Transitions #
###############
def _transition_generic_fetch(self, ts: TaskState, stimulus_id: str) -> RecsInstrs:
if not ts.who_has:
return {ts: "missing"}, []
ts.state = "fetch"
ts.done = False
self.fetch_count += 1
assert ts.priority
for w in ts.who_has:
self.data_needed[w].add(ts)
return {}, []
def _transition_missing_waiting(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
self.missing_dep_flight.discard(ts)
self._purge_state(ts)
return self._transition_released_waiting(ts, stimulus_id=stimulus_id)
def _transition_missing_fetch(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
if self.validate:
assert ts.state == "missing"
if not ts.who_has:
return {}, []
self.missing_dep_flight.discard(ts)
return self._transition_generic_fetch(ts, stimulus_id=stimulus_id)
def _transition_missing_released(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
self.missing_dep_flight.discard(ts)
recs, instructions = self._transition_generic_released(
ts, stimulus_id=stimulus_id
)
assert ts.key in self.tasks
return recs, instructions
def _transition_flight_missing(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
assert ts.done
return self._transition_generic_missing(ts, stimulus_id=stimulus_id)
def _transition_generic_missing(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
if self.validate:
assert not ts.who_has
ts.state = "missing"
self.missing_dep_flight.add(ts)
ts.done = False
return {}, []
def _transition_released_fetch(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
if self.validate:
assert ts.state == "released"
return self._transition_generic_fetch(ts, stimulus_id=stimulus_id)
def _transition_generic_released(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
self._purge_state(ts)
recs: Recs = {}
for dependency in ts.dependencies:
if (
not dependency.waiters
and dependency.state not in READY | PROCESSING | {"memory"}
):
recs[dependency] = "released"
ts.state = "released"
if not ts.dependents:
recs[ts] = "forgotten"
return recs, []
def _transition_released_waiting(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
if self.validate:
assert all(d.key in self.tasks for d in ts.dependencies)
recommendations: Recs = {}
ts.waiting_for_data.clear()
for dep_ts in ts.dependencies:
if dep_ts.state != "memory":
ts.waiting_for_data.add(dep_ts)
dep_ts.waiters.add(ts)
recommendations[dep_ts] = "fetch"
if not ts.waiting_for_data:
recommendations[ts] = "ready"
ts.state = "waiting"
self.waiting.add(ts)
return recommendations, []
def _transition_fetch_flight(
self, ts: TaskState, worker: str, *, stimulus_id: str
) -> RecsInstrs:
if self.validate:
assert ts.state == "fetch"
assert ts.who_has
# The task has already been removed by _ensure_communicating
for w in ts.who_has:
assert ts not in self.data_needed[w]
ts.done = False
ts.state = "flight"
ts.coming_from = worker
self.in_flight_tasks.add(ts)
self.fetch_count -= 1
return {}, []
def _transition_fetch_missing(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
self.fetch_count -= 1
return self._transition_generic_missing(ts, stimulus_id=stimulus_id)
def _transition_fetch_released(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
self.fetch_count -= 1
return self._transition_generic_released(ts, stimulus_id=stimulus_id)
def _transition_memory_released(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
assert ts.nbytes is not None
self.nbytes -= ts.nbytes
recs, instructions = self._transition_generic_released(
ts, stimulus_id=stimulus_id
)
instructions.append(ReleaseWorkerDataMsg(key=ts.key, stimulus_id=stimulus_id))
return recs, instructions
def _transition_waiting_constrained(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
if self.validate:
assert ts.state == "waiting"
assert not ts.waiting_for_data
assert all(
dep.key in self.data or dep.key in self.actors
for dep in ts.dependencies
)
assert all(dep.state == "memory" for dep in ts.dependencies)
assert ts not in self.ready
assert ts not in self.constrained
ts.state = "constrained"
self.waiting.remove(ts)
self.constrained.add(ts)
return self._ensure_computing()
def _transition_executing_rescheduled(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
"""Note: this transition is triggered exclusively by a task raising the
Reschedule() Exception; it is not involved in work stealing.
"""
assert ts.done
return merge_recs_instructions(
({}, [RescheduleMsg(key=ts.key, stimulus_id=stimulus_id)]),
# Note: this is not the same as recommending {ts: "released"} on the
# previous line, as it would instead run the ("executing", "released")
# transition, which would need special code for ts.done=True.
self._transition_generic_released(ts, stimulus_id=stimulus_id),
)
def _transition_waiting_ready(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
if self.validate:
assert ts.state == "waiting"
assert ts not in self.ready
assert ts not in self.constrained
assert not ts.waiting_for_data
for dep in ts.dependencies:
assert dep.key in self.data or dep.key in self.actors
assert dep.state == "memory"
if ts.resource_restrictions:
return {ts: "constrained"}, []
ts.state = "ready"
assert ts.priority is not None
self.waiting.remove(ts)
self.ready.add(ts)
return self._ensure_computing()
def _transition_generic_error(
self,
ts: TaskState,
exception: Serialize,
traceback: Serialize | None,
exception_text: str,
traceback_text: str,
*,
stimulus_id: str,
) -> RecsInstrs:
ts.exception = exception
ts.traceback = traceback
ts.exception_text = exception_text
ts.traceback_text = traceback_text
ts.state = "error"
smsg = TaskErredMsg.from_task(
ts,
stimulus_id=stimulus_id,
thread=self.threads.get(ts.key),
)
return {}, [smsg]
def _transition_resumed_error(
self,
ts: TaskState,
exception: Serialize,
traceback: Serialize | None,
exception_text: str,
traceback_text: str,
*,
stimulus_id: str,
) -> RecsInstrs:
"""In case of failure of the previous state, discard the error and kick off the
next state without informing the scheduler
"""
assert ts.done
if ts.previous in ("executing", "long-running"):
assert ts.next == "fetch"
recs: Recs = {ts: "fetch"}
else:
assert ts.previous == "flight"
assert ts.next == "waiting"
recs = {ts: "waiting"}
ts.state = "released"
ts.done = False
ts.previous = None
ts.next = None
return recs, []
def _transition_resumed_rescheduled(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
"""If the task raises the Reschedule() exception, but the scheduler already told
the worker to fetch it somewhere else, silently transition to fetch.
Note that this transition effectively duplicates the logic of
_transition_resumed_error.
"""
assert ts.done
assert ts.previous in ("executing", "long-running")
assert ts.next == "fetch"
ts.state = "released"
ts.done = False
ts.previous = None
ts.next = None
return {ts: "fetch"}, []
def _transition_resumed_fetch(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
"""
See also
--------
_transition_cancelled_fetch
_transition_cancelled_waiting
_transition_resumed_waiting
_transition_flight_fetch
"""
if ts.previous == "flight":
if self.validate:
assert ts.next == "waiting"
if ts.done:
# We arrived here either from GatherDepNetworkFailureEvent or from
# GatherDepSuccessEvent but without the key in the data attribute.
# We would now normally try to fetch the task from another peer worker
# or transition it to missing if none are left; here instead we're going
# to compute the task as we had been asked by the scheduler.
ts.state = "released"
ts.done = False
ts.previous = None
ts.next = None
return {ts: "waiting"}, []
else:
# We're back where we started. We should forget about the entire
# cancellation attempt
ts.state = "flight"
ts.previous = None
ts.next = None
elif self.validate:
assert ts.previous in ("executing", "long-running")
assert ts.next == "fetch"
# None of the exit events of execute recommend a transition to fetch
assert not ts.done
return {}, []
def _transition_resumed_missing(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
return {ts: "fetch"}, []
def _transition_resumed_released(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
# None of the exit events of execute or gather_dep recommend a transition to
# released
assert not ts.done
ts.state = "cancelled"
ts.next = None
return {}, []
def _transition_resumed_waiting(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
"""
See also
--------
_transition_cancelled_fetch
_transition_cancelled_or_resumed_long_running
_transition_cancelled_waiting
_transition_resumed_fetch
"""
# None of the exit events of execute or gather_dep recommend a transition to
# waiting
assert not ts.done
if ts.previous == "executing":
assert ts.next == "fetch"
# We're back where we started. We should forget about the entire
# cancellation attempt
ts.state = "executing"
ts.next = None
ts.previous = None
return {}, []
elif ts.previous == "long-running":
assert ts.next == "fetch"
# Same as executing, and in addition send the LongRunningMsg in arrears
# Note that, if the task seceded before it was cancelled, this will cause
# the message to be sent twice.
ts.state = "long-running"
ts.next = None
ts.previous = None
smsg = LongRunningMsg(
key=ts.key, compute_duration=None, stimulus_id=stimulus_id
)
return {}, [smsg]
else:
assert ts.previous == "flight"
assert ts.next == "waiting"
return {}, []
def _transition_cancelled_fetch(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
"""
See also
--------
_transition_cancelled_waiting
_transition_resumed_fetch
_transition_resumed_waiting
"""
if ts.previous == "flight":
if ts.done:
# gather_dep just completed for a cancelled task.
# Discard output and possibly forget
return {ts: "released"}, []
else:
# Forget the task was cancelled to begin with
ts.state = "flight"
ts.previous = None
return {}, []
else:
assert ts.previous in ("executing", "long-running")
# None of the exit events of execute recommend a transition to fetch
assert not ts.done
ts.state = "resumed"
ts.next = "fetch"
return {}, []
def _transition_cancelled_waiting(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
"""
See also
--------
_transition_cancelled_fetch
_transition_cancelled_or_resumed_long_running
_transition_resumed_fetch
_transition_resumed_waiting
"""
# None of the exit events of gather_dep or execute recommend a transition to
# waiting
assert not ts.done
if ts.previous == "executing":
# Forget the task was cancelled to begin with
ts.state = "executing"
ts.previous = None
return {}, []
elif ts.previous == "long-running":
# Forget the task was cancelled to begin with, and inform the scheduler
# in arrears that it has seceded.
# Note that, if the task seceded before it was cancelled, this will cause
# the message to be sent twice.
ts.state = "long-running"
ts.previous = None
smsg = LongRunningMsg(
key=ts.key, compute_duration=None, stimulus_id=stimulus_id
)
return {}, [smsg]
else:
assert ts.previous == "flight"
ts.state = "resumed"
ts.next = "waiting"
return {}, []
def _transition_cancelled_released(
self,
ts: TaskState,
*args: Any, # extra arguments of transitions to memory or error - ignored
stimulus_id: str,
) -> RecsInstrs:
if not ts.done:
return {}, []
ts.previous = None
ts.done = False
return self._transition_generic_released(ts, stimulus_id=stimulus_id)
def _transition_executing_released(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
"""We can't stop executing a task just because the scheduler asked us to,
so we're entering cancelled state and waiting until it completes.
"""
if self.validate:
assert ts.state in ("executing", "long-running")
assert not ts.next
assert not ts.done
ts.previous = cast(Literal["executing", "long-running"], ts.state)
ts.state = "cancelled"
return {}, []
def _transition_constrained_executing(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
if self.validate:
assert ts.state == "constrained"
assert not ts.waiting_for_data
assert ts.key not in self.data
assert ts not in self.ready
assert ts not in self.constrained
for dep in ts.dependencies:
assert dep.key in self.data or dep.key in self.actors
ts.state = "executing"
instr = Execute(key=ts.key, stimulus_id=stimulus_id)
return {}, [instr]
def _transition_ready_executing(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
if self.validate:
assert ts.state == "ready"
assert not ts.waiting_for_data
assert ts.key not in self.data
assert ts not in self.ready
assert ts not in self.constrained
assert all(
dep.key in self.data or dep.key in self.actors
for dep in ts.dependencies
)
ts.state = "executing"
instr = Execute(key=ts.key, stimulus_id=stimulus_id)
return {}, [instr]
def _transition_flight_fetch(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
# If this transition is called after the flight coroutine has finished,
# we can reset the task and transition to fetch again. If it is not yet
# finished, this should be a no-op
if not ts.done:
return {}, []
return self._transition_generic_fetch(ts, stimulus_id=stimulus_id)
def _transition_flight_released(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
# None of the exit events of gather_dep recommend a transition to released
assert not ts.done
ts.previous = "flight"
ts.next = None
# See https://github.com/dask/distributed/pull/5046#discussion_r685093940
ts.state = "cancelled"
return {}, []
def _transition_executing_long_running(
self, ts: TaskState, compute_duration: float, *, stimulus_id: str
) -> RecsInstrs:
"""
See also
--------
_transition_cancelled_or_resumed_long_running
"""
ts.state = "long-running"
self.executing.discard(ts)
self.long_running.add(ts)
smsg = LongRunningMsg(
key=ts.key, compute_duration=compute_duration, stimulus_id=stimulus_id
)
return merge_recs_instructions(
({}, [smsg]),
self._ensure_computing(),
)
def _transition_cancelled_or_resumed_long_running(
self, ts: TaskState, compute_duration: float, *, stimulus_id: str
) -> RecsInstrs:
"""Handles transitions:
- cancelled(executing) -> long-running
- cancelled(long-running) -> long-running (user called secede() twice)
- resumed(executing->fetch) -> long-running
- resumed(long-running->fetch) -> long-running (user called secede() twice)
Unlike in the executing->long_running transition, do not send LongRunningMsg.
From the scheduler's perspective, this task no longer exists (cancelled) or is
in memory on another worker (resumed). So it shouldn't hear about it.
Instead, we're going to send the LongRunningMsg when and if the task
transitions back to waiting.
See also
--------
_transition_executing_long_running
_transition_cancelled_waiting
_transition_resumed_waiting
"""
assert ts.previous in ("executing", "long-running")
ts.previous = "long-running"
self.executing.discard(ts)
self.long_running.add(ts)
return self._ensure_computing()
def _transition_executing_memory(
self, ts: TaskState, value: object, *, stimulus_id: str
) -> RecsInstrs:
"""This transition is *normally* triggered by ExecuteSuccessEvent.
However, beware that it can also be triggered by scatter().
"""
return self._transition_to_memory(
ts, value, "task-finished", stimulus_id=stimulus_id
)
def _transition_released_memory(
self, ts: TaskState, value: object, *, stimulus_id: str
) -> RecsInstrs:
"""This transition is triggered by scatter()"""
return self._transition_to_memory(
ts, value, "add-keys", stimulus_id=stimulus_id
)
def _transition_flight_memory(
self, ts: TaskState, value: object, *, stimulus_id: str
) -> RecsInstrs:
"""This transition is *normally* triggered by GatherDepSuccessEvent.
However, beware that it can also be triggered by scatter().
"""
return self._transition_to_memory(
ts, value, "add-keys", stimulus_id=stimulus_id
)
def _transition_resumed_memory(
self, ts: TaskState, value: object, *, stimulus_id: str
) -> RecsInstrs:
"""Normally, we send to the scheduler a 'task-finished' message for a completed
execution and 'add-data' for a completed replication from another worker. The
scheduler's reaction to the two messages is fundamentally different; namely,
add-data is only admissible for tasks that are already in memory on another
worker, and won't trigger transitions.
In the case of resumed tasks, the scheduler's expectation is set by ts.next -
which means, the opposite of what the worker actually just completed.
"""
msg_type: Literal["add-keys", "task-finished"]
if ts.previous in ("executing", "long-running"):
assert ts.next == "fetch"
msg_type = "add-keys"
else:
assert ts.previous == "flight"
assert ts.next == "waiting"
msg_type = "task-finished"
ts.previous = None
ts.next = None
return self._transition_to_memory(ts, value, msg_type, stimulus_id=stimulus_id)
def _transition_to_memory(
self,
ts: TaskState,
value: object,
msg_type: Literal["add-keys", "task-finished"],
*,
stimulus_id: str,
) -> RecsInstrs:
try:
recs, instrs = self._put_key_in_memory(ts, value, stimulus_id=stimulus_id)
except Exception as e:
msg = error_message(e)
return {ts: tuple(msg.values())}, []
# NOTE: The scheduler's reaction to these two messages is fundamentally
# different. Namely, add-keys is only admissible for tasks that are already in
# memory on another worker, and won't trigger transitions.
if msg_type == "add-keys":
instrs.append(AddKeysMsg(keys=[ts.key], stimulus_id=stimulus_id))
else:
assert msg_type == "task-finished"
instrs.append(self._get_task_finished_msg(ts, stimulus_id=stimulus_id))
return recs, instrs
def _transition_released_forgotten(
self, ts: TaskState, *, stimulus_id: str
) -> RecsInstrs:
recommendations: Recs = {}
# Dependents _should_ be released by the scheduler before this
if self.validate:
assert not any(d.state != "forgotten" for d in ts.dependents)
for dep in ts.dependencies:
dep.dependents.discard(ts)
if dep.state == "released" and not dep.dependents:
recommendations[dep] = "forgotten"
self._purge_state(ts)
# Mark state as forgotten in case it is still referenced
ts.state = "forgotten"
self.tasks.pop(ts.key, None)
return recommendations, []
# {
# (start, finish):
# transition_<start>_<finish>(
# self, ts: TaskState, *args, stimulus_id: str
# ) -> (recommendations, instructions)
# }
_TRANSITIONS_TABLE: ClassVar[
Mapping[tuple[TaskStateState, TaskStateState], Callable[..., RecsInstrs]]
] = {
("cancelled", "error"): _transition_cancelled_released,
("cancelled", "fetch"): _transition_cancelled_fetch,
("cancelled", "long-running"): _transition_cancelled_or_resumed_long_running,
("cancelled", "memory"): _transition_cancelled_released,
("cancelled", "missing"): _transition_cancelled_released,
("cancelled", "released"): _transition_cancelled_released,
("cancelled", "rescheduled"): _transition_cancelled_released,
("cancelled", "waiting"): _transition_cancelled_waiting,
("resumed", "error"): _transition_resumed_error,
("resumed", "fetch"): _transition_resumed_fetch,
("resumed", "long-running"): _transition_cancelled_or_resumed_long_running,
("resumed", "memory"): _transition_resumed_memory,
("resumed", "released"): _transition_resumed_released,
("resumed", "rescheduled"): _transition_resumed_rescheduled,
("resumed", "waiting"): _transition_resumed_waiting,
("constrained", "executing"): _transition_constrained_executing,
("constrained", "released"): _transition_generic_released,
("error", "released"): _transition_generic_released,
("executing", "error"): _transition_generic_error,
("executing", "long-running"): _transition_executing_long_running,
("executing", "memory"): _transition_executing_memory,
("executing", "released"): _transition_executing_released,
("executing", "rescheduled"): _transition_executing_rescheduled,
("fetch", "flight"): _transition_fetch_flight,
("fetch", "missing"): _transition_fetch_missing,
("fetch", "released"): _transition_fetch_released,
("flight", "error"): _transition_generic_error,
("flight", "fetch"): _transition_flight_fetch,
("flight", "memory"): _transition_flight_memory,
("flight", "missing"): _transition_flight_missing,
("flight", "released"): _transition_flight_released,
("long-running", "error"): _transition_generic_error,
("long-running", "memory"): _transition_executing_memory,
("long-running", "rescheduled"): _transition_executing_rescheduled,
("long-running", "released"): _transition_executing_released,
("memory", "released"): _transition_memory_released,
("missing", "error"): _transition_generic_error,
("missing", "fetch"): _transition_missing_fetch,
("missing", "released"): _transition_missing_released,
("missing", "waiting"): _transition_missing_waiting,
("ready", "executing"): _transition_ready_executing,
("ready", "released"): _transition_generic_released,
("released", "error"): _transition_generic_error,
("released", "fetch"): _transition_released_fetch,
("released", "forgotten"): _transition_released_forgotten,
("released", "memory"): _transition_released_memory,
("released", "missing"): _transition_generic_missing,
("released", "waiting"): _transition_released_waiting,
("waiting", "constrained"): _transition_waiting_constrained,
("waiting", "ready"): _transition_waiting_ready,
("waiting", "released"): _transition_generic_released,
}
def _notify_plugins(self, method_name: str, *args: Any, **kwargs: Any) -> None:
for name, plugin in self.plugins.items():
if hasattr(plugin, method_name):
try:
getattr(plugin, method_name)(*args, **kwargs)
except Exception:
logger.info(
"Plugin '%s' failed with exception", name, exc_info=True
)
def _transition(
self,
ts: TaskState,
finish: TaskStateState | tuple,
*args: Any,
stimulus_id: str,
) -> RecsInstrs:
"""Transition a key from its current state to the finish state
See Also
--------
Worker.transitions: wrapper around this method
"""
if isinstance(finish, tuple):
# the concatenated transition path might need to access the tuple
assert not args
args = finish[1:]
finish = cast(TaskStateState, finish[0])
if ts.state == finish:
return {}, []
start = ts.state
func = self._TRANSITIONS_TABLE.get((start, finish))
# Notes:
# - in case of transition through released, this counter is incremented by 2
# - this increase happens before the actual transitions, so that it can
# catch potential infinite recursions
self.transition_counter += 1
if (
self.transition_counter_max
and self.transition_counter >= self.transition_counter_max
):
raise TransitionCounterMaxExceeded(ts.key, start, finish, self.story(ts))
if func is not None:
recs, instructions = func(self, ts, *args, stimulus_id=stimulus_id)
self._notify_plugins("transition", ts.key, start, finish)
elif "released" not in (start, finish):
# start -> "released" -> finish
try:
recs, instructions = self._transition(
ts, "released", stimulus_id=stimulus_id
)
v_state: TaskStateState
v_args: list | tuple
while v := recs.pop(ts, None):
if isinstance(v, tuple):
v_state, *v_args = v
else:
v_state, v_args = v, ()
if v_state == "forgotten":
# We do not want to forget. The purpose of this
# transition path is to get to `finish`
continue
recs, instructions = merge_recs_instructions(
(recs, instructions),
self._transition(ts, v_state, *v_args, stimulus_id=stimulus_id),
)
recs, instructions = merge_recs_instructions(
(recs, instructions),
self._transition(ts, finish, *args, stimulus_id=stimulus_id),
)
except (InvalidTransition, RecommendationsConflict) as e:
raise InvalidTransition(ts.key, start, finish, self.story(ts)) from e
else:
raise InvalidTransition(ts.key, start, finish, self.story(ts))
self.log.append(
(
# key
ts.key,
# initial
start,
# recommended
finish,
# final
ts.state,
# new recommendations
{
ts.key: new[0] if isinstance(new, tuple) else new
for ts, new in recs.items()
},
stimulus_id,
time(),
)
)
return recs, instructions
def _resource_restrictions_satisfied(self, ts: TaskState) -> bool:
return all(
self.available_resources[resource] >= needed
for resource, needed in ts.resource_restrictions.items()
)
def _acquire_resources(self, ts: TaskState) -> None:
for resource, needed in ts.resource_restrictions.items():
self.available_resources[resource] -= needed
def _release_resources(self, ts: TaskState) -> None:
for resource, needed in ts.resource_restrictions.items():
self.available_resources[resource] += needed
def _transitions(self, recommendations: Recs, *, stimulus_id: str) -> Instructions:
"""Process transitions until none are left
This includes feedback from previous transitions and continues until we
reach a steady state
"""
instructions = []
tasks = set()
def process_recs(recs: Recs) -> None:
while recs:
ts, finish = recs.popitem()
tasks.add(ts)
a_recs, a_instructions = self._transition(
ts, finish, stimulus_id=stimulus_id
)
recs.update(a_recs)
instructions.extend(a_instructions)
process_recs(recommendations.copy())
# We could call _ensure_communicating after we change something that could
# trigger a new call to gather_dep (e.g. on transitions to fetch,
# GatherDepDoneEvent, or RetryBusyWorkerEvent). However, doing so we'd
# potentially call it too early, before all tasks have transitioned to fetch.
# This in turn would hurt aggregation of multiple tasks into a single GatherDep
# instruction.
# Read: https://github.com/dask/distributed/issues/6497
a_recs, a_instructions = self._ensure_communicating(stimulus_id=stimulus_id)
instructions += a_instructions
process_recs(a_recs)
if self.validate:
# Full state validation is very expensive
for ts in tasks:
self.validate_task(ts)
return instructions
##########
# Events #
##########
@singledispatchmethod
def _handle_event(self, ev: StateMachineEvent) -> RecsInstrs:
raise TypeError(ev) # pragma: nocover
@_handle_event.register
def _handle_update_data(self, ev: UpdateDataEvent) -> RecsInstrs:
recommendations: Recs = {}
instructions: Instructions = []
for key, value in ev.data.items():
try:
ts = self.tasks[key]
recommendations[ts] = ("memory", value)
except KeyError:
self.tasks[key] = ts = TaskState(key)
try:
recs, instrs = self._put_key_in_memory(
ts, value, stimulus_id=ev.stimulus_id
)
except Exception as e:
recs = {ts: tuple(error_message(e).values())}
instrs = []
recommendations.update(recs)
instructions.extend(instrs)
self.log.append((key, "receive-from-scatter", ev.stimulus_id, time()))
if ev.report:
instructions.append(
AddKeysMsg(keys=list(ev.data), stimulus_id=ev.stimulus_id)
)
return recommendations, instructions
@_handle_event.register
def _handle_free_keys(self, ev: FreeKeysEvent) -> RecsInstrs:
"""Handler to be called by the scheduler.
The given keys are no longer referred to and required by the scheduler.
The worker is now allowed to release the key, if applicable.
This does not guarantee that the memory is released since the worker may
still decide to hold on to the data and task since it is required by an
upstream dependency.
"""
self.log.append(("free-keys", ev.keys, ev.stimulus_id, time()))
recommendations: Recs = {}
for key in ev.keys:
ts = self.tasks.get(key)
if ts:
recommendations[ts] = "released"
return recommendations, []
@_handle_event.register
def _handle_remove_replicas(self, ev: RemoveReplicasEvent) -> RecsInstrs:
"""Stream handler notifying the worker that it might be holding unreferenced,
superfluous data.
This should not actually happen during ordinary operations and is only intended
to correct any erroneous state. An example where this is necessary is if a
worker fetches data for a downstream task but that task is released before the
data arrives. In this case, the scheduler will notify the worker that it may be
holding this unnecessary data, if the worker hasn't released the data itself,
already.
This handler does not guarantee the task nor the data to be actually
released but only asks the worker to release the data on a best effort
guarantee. This protects from race conditions where the given keys may
already have been rescheduled for compute in which case the compute
would win and this handler is ignored.
For stronger guarantees, see handler free_keys
"""
recommendations: Recs = {}
instructions: Instructions = []
rejected = []
for key in ev.keys:
ts = self.tasks.get(key)
if ts is None or ts.state != "memory":
continue
if not ts.is_protected():
self.log.append(
(ts.key, "remove-replica-confirmed", ev.stimulus_id, time())
)
recommendations[ts] = "released"
else:
rejected.append(key)
if rejected:
self.log.append(
("remove-replica-rejected", rejected, ev.stimulus_id, time())
)
instructions.append(AddKeysMsg(keys=rejected, stimulus_id=ev.stimulus_id))
return recommendations, instructions
@_handle_event.register
def _handle_acquire_replicas(self, ev: AcquireReplicasEvent) -> RecsInstrs:
if self.validate:
assert ev.who_has.keys() == ev.nbytes.keys()
assert all(ev.who_has.values())
recommendations: Recs = {}
for key, nbytes in ev.nbytes.items():
ts = self._ensure_task_exists(
key=key,
# Transfer this data after all dependency tasks of computations with
# default or explicitly high (>0) user priority and before all
# computations with low priority (<0). Note that the priority= parameter
# of compute() is multiplied by -1 before it reaches TaskState.priority.
priority=(1,),
stimulus_id=ev.stimulus_id,
)
if ts.state != "memory":
ts.nbytes = nbytes
recommendations[ts] = "fetch"
self._update_who_has(ev.who_has)
return recommendations, []
@_handle_event.register
def _handle_compute_task(self, ev: ComputeTaskEvent) -> RecsInstrs:
try:
ts = self.tasks[ev.key]
logger.debug(
"Asked to compute an already known task %s",
{"task": ts, "stimulus_id": ev.stimulus_id},
)
except KeyError:
self.tasks[ev.key] = ts = TaskState(ev.key)
self.log.append((ev.key, "compute-task", ts.state, ev.stimulus_id, time()))
recommendations: Recs = {}
instructions: Instructions = []
if ts.state in READY | {
"executing",
"long-running",
"waiting",
}:
pass
elif ts.state == "memory":
instructions.append(
self._get_task_finished_msg(ts, stimulus_id=ev.stimulus_id)
)
elif ts.state == "error":
instructions.append(TaskErredMsg.from_task(ts, stimulus_id=ev.stimulus_id))
elif ts.state in {
"released",
"fetch",
"flight",
"missing",
"cancelled",
"resumed",
}:
recommendations[ts] = "waiting"
ts.run_spec = ev.run_spec
priority = ev.priority + (self.generation,)
self.generation -= 1
if ev.actor:
self.actors[ts.key] = None
ts.exception = None
ts.traceback = None
ts.exception_text = ""
ts.traceback_text = ""
ts.priority = priority
ts.duration = ev.duration
ts.annotations = ev.annotations
# If we receive ComputeTaskEvent twice for the same task, resources may have
# changed, but the task is still running. Preserve the previous resource
# restrictions so that they can be properly released when it eventually
# completes.
if not (
ts.state in ("cancelled", "resumed")
and ts.previous in ("executing", "long-running")
):
ts.resource_restrictions = ev.resource_restrictions
if self.validate:
assert ev.who_has.keys() == ev.nbytes.keys()
for dep_workers in ev.who_has.values():
assert dep_workers
assert len(dep_workers) == len(set(dep_workers))
for dep_key, nbytes in ev.nbytes.items():
dep_ts = self._ensure_task_exists(
key=dep_key,
priority=priority,
stimulus_id=ev.stimulus_id,
)
if dep_ts.state != "memory":
dep_ts.nbytes = nbytes
# link up to child / parents
ts.dependencies.add(dep_ts)
dep_ts.dependents.add(ts)
self._update_who_has(ev.who_has)
else:
raise RuntimeError( # pragma: nocover
f"Unexpected task state encountered for {ts}; "
f"stimulus_id={ev.stimulus_id}; story={self.story(ts)}"
)
return recommendations, instructions
def _gather_dep_done_common(self, ev: GatherDepDoneEvent) -> Iterator[TaskState]:
"""Common code for the handlers of all subclasses of GatherDepDoneEvent.
Yields the tasks that need to transition out of flight.
The task states can be flight, cancelled, or resumed, but in case of scatter()
they can also be in memory or error states.
See also
--------
_execute_done_common
"""
self.transfer_incoming_bytes -= ev.total_nbytes
keys = self.in_flight_workers.pop(ev.worker)
for key in keys:
ts = self.tasks[key]
ts.done = True
ts.coming_from = None
self.in_flight_tasks.remove(ts)
yield ts
@_handle_event.register
def _handle_gather_dep_success(self, ev: GatherDepSuccessEvent) -> RecsInstrs:
"""gather_dep terminated successfully.
The response may contain fewer keys than the request.
"""
recommendations: Recs = {}
for ts in self._gather_dep_done_common(ev):
if ts.key in ev.data:
recommendations[ts] = ("memory", ev.data[ts.key])
else:
self.log.append((ts.key, "missing-dep", ev.stimulus_id, time()))
if self.validate:
assert ts.state != "fetch"
assert ts not in self.data_needed[ev.worker]
ts.who_has.discard(ev.worker)
self.has_what[ev.worker].discard(ts.key)
recommendations[ts] = "fetch"
return recommendations, []
@_handle_event.register
def _handle_gather_dep_busy(self, ev: GatherDepBusyEvent) -> RecsInstrs:
"""gather_dep terminated: remote worker is busy"""
# Avoid hammering the worker. If there are multiple replicas
# available, immediately try fetching from a different worker.
self.busy_workers.add(ev.worker)
recommendations: Recs = {}
refresh_who_has = []
for ts in self._gather_dep_done_common(ev):
recommendations[ts] = "fetch"
if not ts.who_has - self.busy_workers:
refresh_who_has.append(ts.key)
instructions: Instructions = [
RetryBusyWorkerLater(worker=ev.worker, stimulus_id=ev.stimulus_id),
]
if refresh_who_has:
# All workers that hold known replicas of our tasks are busy.
# Try querying the scheduler for unknown ones.
instructions.append(
RequestRefreshWhoHasMsg(
keys=refresh_who_has, stimulus_id=ev.stimulus_id
)
)
return recommendations, instructions
@_handle_event.register
def _handle_gather_dep_network_failure(
self, ev: GatherDepNetworkFailureEvent
) -> RecsInstrs:
"""gather_dep terminated: network failure while trying to
communicate with remote worker
Though the network failure could be transient, we assume it is not, and
preemptively act as though the other worker has died (including removing all
keys from it, even ones we did not fetch).
This optimization leads to faster completion of the fetch, since we immediately
either retry a different worker, or ask the scheduler to inform us of a new
worker if no other worker is available.
"""
recommendations: Recs = {}
for ts in self._gather_dep_done_common(ev):
self.log.append((ts.key, "missing-dep", ev.stimulus_id, time()))
recommendations[ts] = "fetch"
for ts in self.data_needed.pop(ev.worker, ()):
if self.validate:
assert ts.state == "fetch"
assert ev.worker in ts.who_has
if ts.who_has == {ev.worker}:
# This can override a recommendation from the previous for loop
recommendations[ts] = "missing"
for key in self.has_what.pop(ev.worker):
ts = self.tasks[key]
ts.who_has.remove(ev.worker)
return recommendations, []
@_handle_event.register
def _handle_gather_dep_failure(self, ev: GatherDepFailureEvent) -> RecsInstrs:
"""gather_dep terminated: generic error raised (not a network failure);
e.g. data failed to deserialize.
"""
recommendations: Recs = {
ts: (
"error",
ev.exception,
ev.traceback,
ev.exception_text,
ev.traceback_text,
)
for ts in self._gather_dep_done_common(ev)
}
return recommendations, []
@_handle_event.register
def _handle_secede(self, ev: SecedeEvent) -> RecsInstrs:
ts = self.tasks.get(ev.key)
if not ts:
return {}, []
return {ts: ("long-running", ev.compute_duration)}, []
@_handle_event.register
def _handle_steal_request(self, ev: StealRequestEvent) -> RecsInstrs:
# There may be a race condition between stealing and releasing a task.
# In this case the self.tasks is already cleared. The `None` will be
# registered as `already-computing` on the other end
ts = self.tasks.get(ev.key)
state = ts.state if ts is not None else None
smsg = StealResponseMsg(key=ev.key, state=state, stimulus_id=ev.stimulus_id)
if state in READY | {"waiting"}:
# If task is marked as "constrained" we haven't yet assigned it an
# `available_resources` to run on, that happens in
# `_transition_constrained_executing`
assert ts
return {ts: "released"}, [smsg]
else:
return {}, [smsg]
@_handle_event.register
def _handle_pause(self, ev: PauseEvent) -> RecsInstrs:
"""Prevent any further tasks to be executed or gathered. Tasks that are
currently executing or in flight will continue to progress.
"""
self.running = False
return {}, []
@_handle_event.register
def _handle_unpause(self, ev: UnpauseEvent) -> RecsInstrs:
"""Emerge from paused status"""
self.running = True
return self._ensure_computing()
@_handle_event.register
def _handle_retry_busy_worker(self, ev: RetryBusyWorkerEvent) -> RecsInstrs:
self.busy_workers.discard(ev.worker)
return {}, []
@_handle_event.register
def _handle_cancel_compute(self, ev: CancelComputeEvent) -> RecsInstrs:
"""Cancel a task on a best-effort basis. This is only possible while a task
is in state `waiting` or `ready`; nothing will happen otherwise.
"""
ts = self.tasks.get(ev.key)
if not ts or ts.state not in READY | {"waiting"}:
return {}, []
self.log.append((ev.key, "cancel-compute", ev.stimulus_id, time()))
# All possible dependents of ts should not be in state Processing on
# scheduler side and therefore should not be assigned to a worker, yet.
assert not ts.dependents
return {ts: "released"}, []
def _execute_done_common(
self, ev: ExecuteDoneEvent
) -> tuple[TaskState, Recs, Instructions]:
"""Common code for the handlers of all subclasses of ExecuteDoneEvent.
The task state can be executing, cancelled, or resumed, but in case of scatter()
it can also be in memory or error state.
See also
--------
_gather_dep_done_common
"""
# key *must* be still in tasks - see _transition_released_forgotten
ts = self.tasks.get(ev.key)
assert ts, self.story(ev.key)
if self.validate:
assert (ts in self.executing) != (ts in self.long_running) # XOR
ts.done = True
self.executed_count += 1
self._release_resources(ts)
self.executing.discard(ts)
self.long_running.discard(ts)
recs, instr = self._ensure_computing()
assert ts not in recs
return ts, recs, instr
@_handle_event.register
def _handle_execute_success(self, ev: ExecuteSuccessEvent) -> RecsInstrs:
"""Task completed successfully"""
ts, recs, instr = self._execute_done_common(ev)
ts.startstops.append({"action": "compute", "start": ev.start, "stop": ev.stop})
instr.append(
DigestMetric(
name="compute-duration",
value=ev.stop - ev.start,
stimulus_id=ev.stimulus_id,
)
)
ts.nbytes = ev.nbytes
ts.type = ev.type
recs[ts] = ("memory", ev.value)
return recs, instr
@_handle_event.register
def _handle_execute_failure(self, ev: ExecuteFailureEvent) -> RecsInstrs:
"""Task execution failed"""
ts, recs, instr = self._execute_done_common(ev)
if ev.start is not None and ev.stop is not None:
ts.startstops.append(
{"action": "compute", "start": ev.start, "stop": ev.stop}
)
recs[ts] = (
"error",
ev.exception,
ev.traceback,
ev.exception_text,
ev.traceback_text,
)
return recs, instr
@_handle_event.register
def _handle_reschedule(self, ev: RescheduleEvent) -> RecsInstrs:
"""Task raised Reschedule() exception while it was running.
Note: this has nothing to do with work stealing, which instead causes a
FreeKeysEvent.
"""
ts, recs, instr = self._execute_done_common(ev)
recs[ts] = "rescheduled"
return recs, instr
@_handle_event.register
def _handle_find_missing(self, ev: FindMissingEvent) -> RecsInstrs:
if not self.missing_dep_flight:
return {}, []
if self.validate:
for ts in self.missing_dep_flight:
assert not ts.who_has, self.story(ts)
smsg = RequestRefreshWhoHasMsg(
keys=[ts.key for ts in self.missing_dep_flight],
stimulus_id=ev.stimulus_id,
)
return {}, [smsg]
@_handle_event.register
def _handle_refresh_who_has(self, ev: RefreshWhoHasEvent) -> RecsInstrs:
self._update_who_has(ev.who_has)
recommendations: Recs = {}
instructions: Instructions = []
for key in ev.who_has:
ts = self.tasks.get(key)
if not ts:
continue
if ts.who_has and ts.state == "missing":
recommendations[ts] = "fetch"
elif not ts.who_has and ts.state == "fetch":
recommendations[ts] = "missing"
# Note: if ts.who_has and ts.state == "fetch", we may have just acquired new
# replicas whereas all previously known workers are in flight or busy. We
# rely on _transitions to call _ensure_communicating every time, even in
# absence of recommendations, to potentially kick off a new call to
# gather_dep.
return recommendations, instructions
###############
# Diagnostics #
###############
def story(self, *keys_or_tasks_or_stimuli: str | TaskState) -> list[tuple]:
"""Return all records from the transitions log involving one or more tasks or
stimulus_id's
"""
keys_or_stimuli = {
e.key if isinstance(e, TaskState) else e for e in keys_or_tasks_or_stimuli
}
return worker_story(keys_or_stimuli, self.log)
def stimulus_story(
self, *keys_or_tasks: str | TaskState
) -> list[StateMachineEvent]:
"""Return all state machine events involving one or more tasks"""
keys = {e.key if isinstance(e, TaskState) else e for e in keys_or_tasks}
return [ev for ev in self.stimulus_log if getattr(ev, "key", None) in keys]
def _to_dict(self, *, exclude: Container[str] = ()) -> dict:
"""Dictionary representation for debugging purposes.
Not type stable and not intended for roundtrips.
See also
--------
Client.dump_cluster_state
distributed.utils.recursive_to_dict
"""
info = {
"address": self.address,
"nthreads": self.nthreads,
"running": self.running,
"ready": [ts.key for ts in self.ready.sorted()],
"constrained": [ts.key for ts in self.constrained.sorted()],
"data": dict.fromkeys(self.data),
"data_needed": {
w: [ts.key for ts in tss.sorted()]
for w, tss in self.data_needed.items()
},
"executing": {ts.key for ts in self.executing},
"has_what": dict(self.has_what),
"long_running": {ts.key for ts in self.long_running},
"in_flight_tasks": {ts.key for ts in self.in_flight_tasks},
"in_flight_workers": self.in_flight_workers,
"missing_dep_flight": [ts.key for ts in self.missing_dep_flight],
"busy_workers": self.busy_workers,
"log": self.log,
"stimulus_log": self.stimulus_log,
"transition_counter": self.transition_counter,
"tasks": self.tasks,
}
info = {k: v for k, v in info.items() if k not in exclude}
return recursive_to_dict(info, exclude=exclude)
@property
def task_counts(self) -> dict[TaskStateState | Literal["other"], int]:
# Actors can be in any state other than {fetch, flight, missing}
n_actors_in_memory = sum(
self.tasks[key].state == "memory" for key in self.actors
)
out: dict[TaskStateState | Literal["other"], int] = {
# Key measure for occupancy.
# Also includes cancelled(executing) and resumed(executing->fetch)
"executing": len(self.executing),
# Also includes cancelled(long-running) and resumed(long-running->fetch)
"long-running": len(self.long_running),
"memory": len(self.data) + n_actors_in_memory,
"ready": len(self.ready),
"constrained": len(self.constrained),
"waiting": len(self.waiting),
"fetch": self.fetch_count,
"missing": len(self.missing_dep_flight),
# Also includes cancelled(flight) and resumed(flight->waiting)
"flight": len(self.in_flight_tasks),
}
# released | error
out["other"] = other = len(self.tasks) - sum(out.values())
assert other >= 0
return out
##############
# Validation #
##############
def _validate_task_memory(self, ts: TaskState) -> None:
assert ts.key in self.data or ts.key in self.actors
assert isinstance(ts.nbytes, int)
assert not ts.waiting_for_data
def _validate_task_executing(self, ts: TaskState) -> None:
"""Validate tasks:
- ts.state == executing
- ts.state == long-running
- ts.state == cancelled, ts.previous == executing
- ts.state == cancelled, ts.previous == long-running
- ts.state == resumed, ts.previous == executing, ts.next == fetch
- ts.state == resumed, ts.previous == long-running, ts.next == fetch
"""
if ts.state == "executing" or ts.previous == "executing":
assert ts in self.executing
assert ts not in self.long_running
else:
assert ts.state == "long-running" or ts.previous == "long-running"
assert ts not in self.executing
assert ts in self.long_running
assert ts.run_spec is not None
assert ts.key not in self.data
assert not ts.waiting_for_data
for dep in ts.dependents:
assert dep not in self.ready
assert dep not in self.constrained
# FIXME https://github.com/dask/distributed/issues/6893
# This assertion can be false for
# - cancelled or resumed tasks
# - executing tasks which used to be cancelled in the past
# for dep in ts.dependencies:
# assert dep.state == "memory", self.story(dep)
# assert dep.key in self.data or dep.key in self.actors
def _validate_task_ready(self, ts: TaskState) -> None:
"""Validate tasks:
- ts.state == ready
- ts.state == constrained
"""
if ts.state == "ready":
assert not ts.resource_restrictions
assert ts in self.ready
assert ts not in self.constrained
else:
assert ts.resource_restrictions
assert ts.state == "constrained"
assert ts not in self.ready
assert ts in self.constrained
assert ts.key not in self.data
assert not ts.done
assert not ts.waiting_for_data
assert all(
dep.key in self.data or dep.key in self.actors for dep in ts.dependencies
)
def _validate_task_waiting(self, ts: TaskState) -> None:
assert ts.key not in self.data
assert not ts.done
assert ts in self.waiting
assert ts.waiting_for_data
assert ts.waiting_for_data == {
dep
for dep in ts.dependencies
if dep.key not in self.data and dep.key not in self.actors
}
for dep in ts.dependents:
assert dep not in self.ready
assert dep not in self.constrained
def _validate_task_flight(self, ts: TaskState) -> None:
"""Validate tasks:
- ts.state == flight
- ts.state == cancelled, ts.previous == flight
- ts.state == resumed, ts.previous == flight, ts.next == waiting
"""
assert ts.key not in self.data
assert ts.key not in self.actors
assert ts in self.in_flight_tasks
for dep in ts.dependents:
assert dep not in self.ready
assert dep not in self.constrained
assert ts.coming_from
assert ts.coming_from in self.in_flight_workers
assert ts.key in self.in_flight_workers[ts.coming_from]
def _validate_task_fetch(self, ts: TaskState) -> None:
assert ts.key not in self.data
assert ts.key not in self.actors
assert self.address not in ts.who_has
assert not ts.done
assert ts.who_has
for w in ts.who_has:
assert ts.key in self.has_what[w]
assert ts in self.data_needed[w]
for dep in ts.dependents:
assert dep not in self.ready
assert dep not in self.constrained
def _validate_task_missing(self, ts: TaskState) -> None:
assert ts.key not in self.data
assert ts.key not in self.actors
assert not ts.who_has
assert not ts.done
assert not any(ts.key in has_what for has_what in self.has_what.values())
assert ts in self.missing_dep_flight
for dep in ts.dependents:
assert dep not in self.ready
assert dep not in self.constrained
def _validate_task_cancelled(self, ts: TaskState) -> None:
assert ts.next is None
if ts.previous in ("executing", "long-running"):
self._validate_task_executing(ts)
else:
assert ts.previous == "flight"
self._validate_task_flight(ts)
def _validate_task_resumed(self, ts: TaskState) -> None:
if ts.previous in ("executing", "long-running"):
assert ts.next == "fetch"
self._validate_task_executing(ts)
else:
assert ts.previous == "flight"
assert ts.next == "waiting"
self._validate_task_flight(ts)
for dep in ts.dependents:
assert dep not in self.ready
assert dep not in self.constrained
def _validate_task_released(self, ts: TaskState) -> None:
assert ts.key not in self.data
assert ts.key not in self.actors
assert not ts.next
assert not ts.previous
for tss in self.data_needed.values():
assert ts not in tss
assert ts not in self.executing
assert ts not in self.in_flight_tasks
assert ts not in self.missing_dep_flight
# The below assert statement is true most of the time. If a task performs the
# transition flight->cancel->waiting, its dependencies are normally in released
# state. However, the compute-task call for their previous dependent provided
# them with who_has, such that this assert is no longer true.
#
# assert not any(ts.key in has_what for has_what in self.has_what.values())
assert not ts.waiting_for_data
assert not ts.done
assert not ts.exception
assert not ts.traceback
def validate_task(self, ts: TaskState) -> None:
try:
if ts.key in self.tasks:
assert self.tasks[ts.key] is ts
if ts.state == "memory":
self._validate_task_memory(ts)
elif ts.state == "waiting":
self._validate_task_waiting(ts)
elif ts.state == "missing":
self._validate_task_missing(ts)
elif ts.state == "cancelled":
self._validate_task_cancelled(ts)
elif ts.state == "resumed":
self._validate_task_resumed(ts)
elif ts.state in ("ready", "constrained"):
self._validate_task_ready(ts)
elif ts.state in ("executing", "long-running"):
self._validate_task_executing(ts)
elif ts.state == "flight":
self._validate_task_flight(ts)
elif ts.state == "fetch":
self._validate_task_fetch(ts)
elif ts.state == "released":
self._validate_task_released(ts)
except Exception as e:
logger.exception(e)
raise InvalidTaskState(
key=ts.key, state=ts.state, story=self.story(ts)
) from e
def validate_state(self) -> None:
for ts in self.tasks.values():
# check that worker has task
for worker in ts.who_has:
assert worker != self.address
assert ts.key in self.has_what[worker]
# check that deps have a set state and that dependency<->dependent links
# are there
for dep in ts.dependencies:
# self.tasks was just a dict of tasks
# and this check was originally that the key was in `task_state`
# so we may have popped the key out of `self.tasks` but the
# dependency can still be in `memory` before GC grabs it...?
# Might need better bookkeeping
assert self.tasks[dep.key] is dep
assert ts in dep.dependents, ts
for ts_wait in ts.waiting_for_data:
assert self.tasks[ts_wait.key] is ts_wait
assert ts_wait.state in WAITING_FOR_DATA, ts_wait
for worker, keys in self.has_what.items():
assert worker != self.address
for k in keys:
assert k in self.tasks, self.story(k)
assert worker in self.tasks[k].who_has
# Test contents of the various sets of TaskState objects
fetch_tss = set()
for worker, tss in self.data_needed.items():
for ts in tss:
fetch_tss.add(ts)
assert ts.state == "fetch"
assert worker in ts.who_has
assert len(fetch_tss) == self.fetch_count
for ts in self.missing_dep_flight:
assert ts.state == "missing"
for ts in self.ready:
assert ts.state == "ready"
for ts in self.constrained:
assert ts.state == "constrained"
for ts in self.executing:
assert ts.state == "executing" or (
ts.state in ("cancelled", "resumed") and ts.previous == "executing"
), ts
for ts in self.long_running:
assert ts.state == "long-running" or (
ts.state in ("cancelled", "resumed") and ts.previous == "long-running"
), ts
for ts in self.in_flight_tasks:
assert ts.state == "flight" or (
ts.state in ("cancelled", "resumed") and ts.previous == "flight"
), ts
for ts in self.waiting:
assert ts.state == "waiting"
# Test that there aren't multiple TaskState objects with the same key in any
# Set[TaskState]. See note in TaskState.__hash__.
for ts in chain(
*self.data_needed.values(),
self.missing_dep_flight,
self.ready,
self.constrained,
self.in_flight_tasks,
self.executing,
self.long_running,
self.waiting,
):
assert self.tasks[ts.key] is ts
expect_nbytes = sum(
self.tasks[key].nbytes or 0 for key in chain(self.data, self.actors)
)
assert self.nbytes == expect_nbytes, f"{self.nbytes=}; expected {expect_nbytes}"
for key in self.data:
assert key in self.tasks, key
for key in self.actors:
assert key in self.tasks, key
for ts in self.tasks.values():
self.validate_task(ts)
if self.transition_counter_max:
assert self.transition_counter < self.transition_counter_max
self._validate_resources()
def _validate_resources(self) -> None:
"""Assert that available_resources + resources held by tasks = total_resources"""
assert self.total_resources.keys() == self.available_resources.keys()
total = self.total_resources.copy()
for k, v in self.available_resources.items():
assert v > -1e-9, self.available_resources
total[k] -= v
for ts in self.all_running_tasks:
for k, v in ts.resource_restrictions.items():
assert v >= 0, (ts, ts.resource_restrictions)
total[k] -= v
assert all((abs(v) < 1e-9) for v in total.values()), total
class BaseWorker(abc.ABC):
"""Wrapper around the :class:`WorkerState` that implements instructions handling.
This is an abstract class with several ``@abc.abstractmethod`` methods, to be
subclassed by :class:`~distributed.worker.Worker` and by unit test mock-ups.
"""
state: WorkerState
_async_instructions: set[asyncio.Task]
def __init__(self, state: WorkerState):
self.state = state
self._async_instructions = set()
def _handle_stimulus_from_task(self, task: asyncio.Task[StateMachineEvent]) -> None:
"""An asynchronous instruction just completed; process the returned stimulus."""
self._async_instructions.remove(task)
try:
# This *should* never raise any other exceptions
stim = task.result()
except asyncio.CancelledError:
# This should exclusively happen in Worker.close()
return
self.handle_stimulus(stim)
def handle_stimulus(self, *stims: StateMachineEvent) -> None:
"""Forward one or more external stimuli to :meth:`WorkerState.handle_stimulus`
and process the returned instructions, invoking the relevant Worker callbacks
(``@abc.abstractmethod`` methods below).
Spawn asyncio tasks for all asynchronous instructions and start tracking them.
See also
--------
WorkerState.handle_stimulus
"""
instructions = self.state.handle_stimulus(*stims)
for inst in instructions:
task: asyncio.Task | None = None
if isinstance(inst, SendMessageToScheduler):
self.batched_send(inst.to_dict())
elif isinstance(inst, DigestMetric):
self.digest_metric(inst.name, inst.value)
elif isinstance(inst, GatherDep):
assert inst.to_gather
keys_str = ", ".join(peekn(27, inst.to_gather)[0])
if len(keys_str) > 80:
keys_str = keys_str[:77] + "..."
task = asyncio.create_task(
self.gather_dep(
inst.worker,
inst.to_gather,
total_nbytes=inst.total_nbytes,
stimulus_id=inst.stimulus_id,
),
name=f"gather_dep({inst.worker}, {{{keys_str}}})",
)
elif isinstance(inst, Execute):
task = asyncio.create_task(
self.execute(inst.key, stimulus_id=inst.stimulus_id),
name=f"execute({inst.key})",
)
elif isinstance(inst, RetryBusyWorkerLater):
task = asyncio.create_task(
self.retry_busy_worker_later(inst.worker),
name=f"retry_busy_worker_later({inst.worker})",
)
else:
raise TypeError(inst) # pragma: nocover
if task is not None:
self._async_instructions.add(task)
task.add_done_callback(self._handle_stimulus_from_task)
async def close(self, timeout: float = 30) -> None:
"""Cancel all asynchronous instructions"""
if not self._async_instructions:
return
for task in self._async_instructions:
task.cancel()
# async tasks can handle cancellation and could take an arbitrary amount
# of time to terminate
_, pending = await asyncio.wait(self._async_instructions, timeout=timeout)
for task in pending:
logger.error(
f"Failed to cancel asyncio task after {timeout} seconds: {task}"
)
@abc.abstractmethod
def batched_send(self, msg: dict[str, Any]) -> None:
"""Send a fire-and-forget message to the scheduler through bulk comms.
Parameters
----------
msg: dict
msgpack-serializable message to send to the scheduler.
Must have a 'op' key which is registered in Scheduler.stream_handlers.
"""
@abc.abstractmethod
async def gather_dep(
self,
worker: str,
to_gather: Collection[str],
total_nbytes: int,
*,
stimulus_id: str,
) -> StateMachineEvent:
"""Gather dependencies for a task from a worker who has them
Parameters
----------
worker : str
Address of worker to gather dependencies from
to_gather : list
Keys of dependencies to gather from worker -- this is not
necessarily equivalent to the full list of dependencies of ``dep``
as some dependencies may already be present on this worker.
total_nbytes : int
Total number of bytes for all the dependencies in to_gather combined
"""
@abc.abstractmethod
async def execute(self, key: str, *, stimulus_id: str) -> StateMachineEvent:
"""Execute a task"""
@abc.abstractmethod
async def retry_busy_worker_later(self, worker: str) -> StateMachineEvent:
"""Wait some time, then take a peer worker out of busy state"""
@abc.abstractmethod
def digest_metric(self, name: str, value: float) -> None:
"""Log an arbitrary numerical metric"""
class DeprecatedWorkerStateAttribute:
name: str
target: str | None
def __init__(self, target: str | None = None):
self.target = target
def __set_name__(self, owner: type, name: str) -> None:
self.name = name
def _warn_deprecated(self) -> None:
warnings.warn(
f"The `Worker.{self.name}` attribute has been moved to "
f"`Worker.state.{self.target or self.name}`",
FutureWarning,
)
def __get__(self, instance: Worker | None, owner: type[Worker]) -> Any:
if instance is None:
# This is triggered by Sphinx
return None # pragma: nocover
self._warn_deprecated()
return getattr(instance.state, self.target or self.name)
def __set__(self, instance: Worker, value: Any) -> None:
self._warn_deprecated()
setattr(instance.state, self.target or self.name, value)
|