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
|
from __future__ import annotations
import asyncio
import bisect
import builtins
import errno
import functools
import logging
import math
import os
import pathlib
import random
import sys
import tempfile
import threading
import warnings
import weakref
from collections import defaultdict, deque
from collections.abc import (
Callable,
Collection,
Container,
Iterable,
Mapping,
MutableMapping,
)
from concurrent.futures import Executor
from contextlib import suppress
from datetime import timedelta
from inspect import isawaitable
from typing import TYPE_CHECKING, Any, ClassVar, Literal, TextIO, TypeVar, cast
from tlz import first, keymap, pluck
from tornado.ioloop import IOLoop
import dask
from dask.core import istask
from dask.system import CPU_COUNT
from dask.utils import (
apply,
format_bytes,
funcname,
key_split,
parse_bytes,
parse_timedelta,
tmpdir,
typename,
)
from distributed import preloading, profile, utils
from distributed.batched import BatchedSend
from distributed.collections import LRU
from distributed.comm import Comm, connect, get_address_host, parse_address
from distributed.comm import resolve_address as comm_resolve_address
from distributed.comm.addressing import address_from_user_args
from distributed.comm.utils import OFFLOAD_THRESHOLD
from distributed.compatibility import PeriodicCallback, randbytes, to_thread
from distributed.core import (
ConnectionPool,
Status,
coerce_to_address,
error_message,
pingpong,
)
from distributed.core import rpc as RPCType
from distributed.core import send_recv
from distributed.diagnostics import nvml
from distributed.diagnostics.plugin import _get_plugin_name
from distributed.diskutils import WorkDir, WorkSpace
from distributed.http import get_handlers
from distributed.metrics import time
from distributed.node import ServerNode
from distributed.proctitle import setproctitle
from distributed.protocol import pickle, to_serialize
from distributed.pubsub import PubSubWorkerExtension
from distributed.security import Security
from distributed.shuffle import ShuffleWorkerExtension
from distributed.sizeof import safe_sizeof as sizeof
from distributed.threadpoolexecutor import ThreadPoolExecutor
from distributed.threadpoolexecutor import secede as tpe_secede
from distributed.utils import (
TimeoutError,
_maybe_complex,
get_ip,
has_arg,
import_file,
in_async_call,
is_python_shutting_down,
iscoroutinefunction,
json_load_robust,
log_errors,
offload,
parse_ports,
recursive_to_dict,
silence_logging,
thread_state,
warn_on_duration,
)
from distributed.utils_comm import gather_from_workers, pack_data, retry_operation
from distributed.utils_perf import disable_gc_diagnosis, enable_gc_diagnosis
from distributed.versions import get_versions
from distributed.worker_memory import (
DeprecatedMemoryManagerAttribute,
DeprecatedMemoryMonitor,
WorkerMemoryManager,
)
from distributed.worker_state_machine import (
NO_VALUE,
AcquireReplicasEvent,
BaseWorker,
CancelComputeEvent,
ComputeTaskEvent,
DeprecatedWorkerStateAttribute,
ExecuteFailureEvent,
ExecuteSuccessEvent,
FindMissingEvent,
FreeKeysEvent,
GatherDepBusyEvent,
GatherDepFailureEvent,
GatherDepNetworkFailureEvent,
GatherDepSuccessEvent,
PauseEvent,
RefreshWhoHasEvent,
RemoveReplicasEvent,
RescheduleEvent,
RetryBusyWorkerEvent,
SecedeEvent,
StateMachineEvent,
StealRequestEvent,
TaskState,
UnpauseEvent,
UpdateDataEvent,
WorkerState,
)
if TYPE_CHECKING:
# FIXME import from typing (needs Python >=3.10)
from typing_extensions import ParamSpec
# Circular imports
from distributed.client import Client
from distributed.diagnostics.plugin import WorkerPlugin
from distributed.nanny import Nanny
P = ParamSpec("P")
T = TypeVar("T")
logger = logging.getLogger(__name__)
LOG_PDB = dask.config.get("distributed.admin.pdb-on-err")
DEFAULT_EXTENSIONS: dict[str, type] = {
"pubsub": PubSubWorkerExtension,
"shuffle": ShuffleWorkerExtension,
}
DEFAULT_METRICS: dict[str, Callable[[Worker], Any]] = {}
DEFAULT_STARTUP_INFORMATION: dict[str, Callable[[Worker], Any]] = {}
WORKER_ANY_RUNNING = {
Status.running,
Status.paused,
Status.closing_gracefully,
}
def fail_hard(method: Callable[P, T]) -> Callable[P, T]:
"""
Decorator to close the worker if this method encounters an exception.
"""
reason = f"worker-{method.__name__}-fail-hard"
if iscoroutinefunction(method):
@functools.wraps(method)
async def wrapper(self, *args: P.args, **kwargs: P.kwargs) -> Any:
try:
return await method(self, *args, **kwargs) # type: ignore
except Exception as e:
if self.status not in (Status.closed, Status.closing):
self.log_event("worker-fail-hard", error_message(e))
logger.exception(e)
await _force_close(self, reason)
raise
else:
@functools.wraps(method)
def wrapper(self, *args: P.args, **kwargs: P.kwargs) -> T:
try:
return method(self, *args, **kwargs)
except Exception as e:
if self.status not in (Status.closed, Status.closing):
self.log_event("worker-fail-hard", error_message(e))
logger.exception(e)
self.loop.add_callback(_force_close, self, reason)
raise
return wrapper # type: ignore
async def _force_close(self, reason: str):
"""
Used with the fail_hard decorator defined above
1. Wait for a worker to close
2. If it doesn't, log and kill the process
"""
try:
await asyncio.wait_for(
self.close(nanny=False, executor_wait=False, reason=reason),
30,
)
except (KeyboardInterrupt, SystemExit): # pragma: nocover
raise
except BaseException: # pragma: nocover
# Worker is in a very broken state if closing fails. We need to shut down
# immediately, to ensure things don't get even worse and this worker potentially
# deadlocks the cluster.
if self.state.validate and not self.nanny:
# We're likely in a unit test. Don't kill the whole test suite!
raise
logger.critical(
"Error trying close worker in response to broken internal state. "
"Forcibly exiting worker NOW",
exc_info=True,
)
# use `os._exit` instead of `sys.exit` because of uncertainty
# around propagating `SystemExit` from asyncio callbacks
os._exit(1)
class Worker(BaseWorker, ServerNode):
"""Worker node in a Dask distributed cluster
Workers perform two functions:
1. **Serve data** from a local dictionary
2. **Perform computation** on that data and on data from peers
Workers keep the scheduler informed of their data and use that scheduler to
gather data from other workers when necessary to perform a computation.
You can start a worker with the ``dask worker`` command line application::
$ dask worker scheduler-ip:port
Use the ``--help`` flag to see more options::
$ dask worker --help
The rest of this docstring is about the internal state that the worker uses
to manage and track internal computations.
**State**
**Informational State**
These attributes don't change significantly during execution.
* **nthreads:** ``int``:
Number of nthreads used by this worker process
* **executors:** ``dict[str, concurrent.futures.Executor]``:
Executors used to perform computation. Always contains the default
executor.
* **local_directory:** ``path``:
Path on local machine to store temporary files
* **scheduler:** ``rpc``:
Location of scheduler. See ``.ip/.port`` attributes.
* **name:** ``string``:
Alias
* **services:** ``{str: Server}``:
Auxiliary web servers running on this worker
* **service_ports:** ``{str: port}``:
* **transfer_outgoing_count_limit**: ``int``
The maximum number of concurrent outgoing data transfers.
See also
:attr:`distributed.worker_state_machine.WorkerState.transfer_incoming_count_limit`.
* **batched_stream**: ``BatchedSend``
A batched stream along which we communicate to the scheduler
* **log**: ``[(message)]``
A structured and queryable log. See ``Worker.story``
**Volatile State**
These attributes track the progress of tasks that this worker is trying to
complete. In the descriptions below a ``key`` is the name of a task that
we want to compute and ``dep`` is the name of a piece of dependent data
that we want to collect from others.
* **threads**: ``{key: int}``
The ID of the thread on which the task ran
* **active_threads**: ``{int: key}``
The keys currently running on active threads
* **state**: ``WorkerState``
Encapsulated state machine. See
:class:`~distributed.worker_state_machine.BaseWorker` and
:class:`~distributed.worker_state_machine.WorkerState`
Parameters
----------
scheduler_ip: str, optional
scheduler_port: int, optional
scheduler_file: str, optional
host: str, optional
data: MutableMapping, type, None
The object to use for storage, builds a disk-backed LRU dict by default.
If a callable to construct the storage object is provided, it
will receive the worker's attr:``local_directory`` as an
argument if the calling signature has an argument named
``worker_local_directory``.
nthreads: int, optional
local_directory: str, optional
Directory where we place local resources
name: str, optional
memory_limit: int, float, string
Number of bytes of memory that this worker should use.
Set to zero for no limit. Set to 'auto' to calculate
as system.MEMORY_LIMIT * min(1, nthreads / total_cores)
Use strings or numbers like 5GB or 5e9
memory_target_fraction: float or False
Fraction of memory to try to stay beneath
(default: read from config key distributed.worker.memory.target)
memory_spill_fraction: float or False
Fraction of memory at which we start spilling to disk
(default: read from config key distributed.worker.memory.spill)
memory_pause_fraction: float or False
Fraction of memory at which we stop running new tasks
(default: read from config key distributed.worker.memory.pause)
max_spill: int, string or False
Limit of number of bytes to be spilled on disk.
(default: read from config key distributed.worker.memory.max-spill)
executor: concurrent.futures.Executor, dict[str, concurrent.futures.Executor], "offload"
The executor(s) to use. Depending on the type, it has the following meanings:
- Executor instance: The default executor.
- Dict[str, Executor]: mapping names to Executor instances. If the
"default" key isn't in the dict, a "default" executor will be created
using ``ThreadPoolExecutor(nthreads)``.
- Str: The string "offload", which refer to the same thread pool used for
offloading communications. This results in the same thread being used
for deserialization and computation.
resources: dict
Resources that this worker has like ``{'GPU': 2}``
nanny: str
Address on which to contact nanny, if it exists
lifetime: str
Amount of time like "1 hour" after which we gracefully shut down the worker.
This defaults to None, meaning no explicit shutdown time.
lifetime_stagger: str
Amount of time like "5 minutes" to stagger the lifetime value
The actual lifetime will be selected uniformly at random between
lifetime +/- lifetime_stagger
lifetime_restart: bool
Whether or not to restart a worker after it has reached its lifetime
Default False
kwargs: optional
Additional parameters to ServerNode constructor
Examples
--------
Use the command line to start a worker::
$ dask scheduler
Start scheduler at 127.0.0.1:8786
$ dask worker 127.0.0.1:8786
Start worker at: 127.0.0.1:1234
Registered with scheduler at: 127.0.0.1:8786
See Also
--------
distributed.scheduler.Scheduler
distributed.nanny.Nanny
"""
_instances: ClassVar[weakref.WeakSet[Worker]] = weakref.WeakSet()
_initialized_clients: ClassVar[weakref.WeakSet[Client]] = weakref.WeakSet()
nanny: Nanny | None
_lock: threading.Lock
transfer_outgoing_count_limit: int
threads: dict[str, int] # {ts.key: thread ID}
active_threads_lock: threading.Lock
active_threads: dict[int, str] # {thread ID: ts.key}
active_keys: set[str]
profile_keys: defaultdict[str, dict[str, Any]]
profile_keys_history: deque[tuple[float, dict[str, dict[str, Any]]]]
profile_recent: dict[str, Any]
profile_history: deque[tuple[float, dict[str, Any]]]
transfer_incoming_log: deque[dict[str, Any]]
transfer_outgoing_log: deque[dict[str, Any]]
#: Total number of data transfers to other workers since the worker was started
transfer_outgoing_count_total: int
#: Total size of data transfers to other workers (including in-progress and failed transfers)
transfer_outgoing_bytes_total: int
#: Current total size of open data transfers to other workers
transfer_outgoing_bytes: int
#: Current number of open data transfers to other workers
transfer_outgoing_count: int
bandwidth: float
latency: float
profile_cycle_interval: float
workspace: WorkSpace
_workdir: WorkDir
local_directory: str
_client: Client | None
bandwidth_workers: defaultdict[str, tuple[float, int]]
bandwidth_types: defaultdict[type, tuple[float, int]]
preloads: list[preloading.Preload]
contact_address: str | None
_start_port: int | str | Collection[int] | None = None
_start_host: str | None
_interface: str | None
_protocol: str
_dashboard_address: str | None
_dashboard: bool
_http_prefix: str
death_timeout: float | None
lifetime: float | None
lifetime_stagger: float | None
lifetime_restart: bool
extensions: dict
security: Security
connection_args: dict[str, Any]
loop: IOLoop
executors: dict[str, Executor]
batched_stream: BatchedSend
name: Any
scheduler_delay: float
stream_comms: dict[str, BatchedSend]
heartbeat_interval: float
services: dict[str, Any] = {}
service_specs: dict[str, Any]
metrics: dict[str, Callable[[Worker], Any]]
startup_information: dict[str, Callable[[Worker], Any]]
low_level_profiler: bool
scheduler: Any
execution_state: dict[str, Any]
plugins: dict[str, WorkerPlugin]
_pending_plugins: tuple[WorkerPlugin, ...]
def __init__(
self,
scheduler_ip: str | None = None,
scheduler_port: int | None = None,
*,
scheduler_file: str | None = None,
nthreads: int | None = None,
loop: IOLoop | None = None, # Deprecated
local_directory: str | None = None,
services: dict | None = None,
name: Any | None = None,
reconnect: bool | None = None,
executor: Executor | dict[str, Executor] | Literal["offload"] | None = None,
resources: dict[str, float] | None = None,
silence_logs: int | None = None,
death_timeout: Any | None = None,
preload: list[str] | None = None,
preload_argv: list[str] | list[list[str]] | None = None,
security: Security | dict[str, Any] | None = None,
contact_address: str | None = None,
heartbeat_interval: Any = "1s",
extensions: dict[str, type] | None = None,
metrics: Mapping[str, Callable[[Worker], Any]] = DEFAULT_METRICS,
startup_information: Mapping[
str, Callable[[Worker], Any]
] = DEFAULT_STARTUP_INFORMATION,
interface: str | None = None,
host: str | None = None,
port: int | str | Collection[int] | None = None,
protocol: str | None = None,
dashboard_address: str | None = None,
dashboard: bool = False,
http_prefix: str = "/",
nanny: Nanny | None = None,
plugins: tuple[WorkerPlugin, ...] = (),
low_level_profiler: bool | None = None,
validate: bool | None = None,
profile_cycle_interval=None,
lifetime: Any | None = None,
lifetime_stagger: Any | None = None,
lifetime_restart: bool | None = None,
transition_counter_max: int | Literal[False] = False,
###################################
# Parameters to WorkerMemoryManager
memory_limit: str | float = "auto",
# Allow overriding the dict-like that stores the task outputs.
# This is meant for power users only. See WorkerMemoryManager for details.
data: (
MutableMapping[str, Any] # pre-initialised
| Callable[[], MutableMapping[str, Any]] # constructor
# constructor receiving self.local_directory
| Callable[[str], MutableMapping[str, Any]]
| tuple[
Callable[..., MutableMapping[str, Any]], dict[str, Any]
] # (constructor, kwargs to constructor)
| None # create internally
) = None,
# Deprecated parameters; please use dask config instead.
memory_target_fraction: float | Literal[False] | None = None,
memory_spill_fraction: float | Literal[False] | None = None,
memory_pause_fraction: float | Literal[False] | None = None,
###################################
# Parameters to Server
scheduler_sni: str | None = None,
**kwargs,
):
if reconnect is not None:
if reconnect:
raise ValueError(
"The `reconnect=True` option for `Worker` has been removed. "
"To improve cluster stability, workers now always shut down in the face of network disconnects. "
"For details, or if this is an issue for you, see https://github.com/dask/distributed/issues/6350."
)
else:
warnings.warn(
"The `reconnect` argument to `Worker` is deprecated, and will be removed in a future release. "
"Worker reconnection is now always disabled, so passing `reconnect=False` is unnecessary. "
"See https://github.com/dask/distributed/issues/6350 for details.",
DeprecationWarning,
stacklevel=2,
)
if loop is not None:
warnings.warn(
"The `loop` argument to `Worker` is ignored, and will be removed in a future release. "
"The Worker always binds to the current loop",
DeprecationWarning,
stacklevel=2,
)
self.nanny = nanny
self._lock = threading.Lock()
transfer_incoming_count_limit = dask.config.get(
"distributed.worker.connections.outgoing"
)
self.transfer_outgoing_count_limit = dask.config.get(
"distributed.worker.connections.incoming"
)
transfer_message_bytes_limit = parse_bytes(
dask.config.get("distributed.worker.transfer.message-bytes-limit")
)
self.threads = {}
self.active_threads_lock = threading.Lock()
self.active_threads = {}
self.active_keys = set()
self.profile_keys = defaultdict(profile.create)
self.profile_keys_history = deque(maxlen=3600)
self.profile_recent = profile.create()
self.profile_history = deque(maxlen=3600)
if validate is None:
validate = dask.config.get("distributed.scheduler.validate")
self.transfer_incoming_log = deque(maxlen=100000)
self.transfer_outgoing_log = deque(maxlen=100000)
self.transfer_outgoing_count_total = 0
self.transfer_outgoing_bytes_total = 0
self.transfer_outgoing_bytes = 0
self.transfer_outgoing_count = 0
self.bandwidth = parse_bytes(dask.config.get("distributed.scheduler.bandwidth"))
self.bandwidth_workers = defaultdict(
lambda: (0, 0)
) # bw/count recent transfers
self.bandwidth_types = defaultdict(lambda: (0, 0)) # bw/count recent transfers
self.latency = 0.001
self._client = None
if profile_cycle_interval is None:
profile_cycle_interval = dask.config.get("distributed.worker.profile.cycle")
profile_cycle_interval = parse_timedelta(profile_cycle_interval, default="ms")
assert profile_cycle_interval
self._setup_logging(logger)
if not local_directory:
local_directory = (
dask.config.get("temporary-directory") or tempfile.gettempdir()
)
local_directory = os.path.join(local_directory, "dask-worker-space")
with warn_on_duration(
"1s",
"Creating scratch directories is taking a surprisingly long time. ({duration:.2f}s) "
"This is often due to running workers on a network file system. "
"Consider specifying a local-directory to point workers to write "
"scratch data to a local disk.",
):
self._workspace = WorkSpace(local_directory)
self._workdir = self._workspace.new_work_dir(prefix="worker-")
self.local_directory = self._workdir.dir_path
if not preload:
preload = dask.config.get("distributed.worker.preload")
if not preload_argv:
preload_argv = dask.config.get("distributed.worker.preload-argv")
assert preload is not None
assert preload_argv is not None
self.preloads = preloading.process_preloads(
self, preload, preload_argv, file_dir=self.local_directory
)
self.death_timeout = parse_timedelta(death_timeout)
if scheduler_file:
cfg = json_load_robust(scheduler_file, timeout=self.death_timeout)
scheduler_addr = cfg["address"]
elif scheduler_ip is None and dask.config.get("scheduler-address", None):
scheduler_addr = dask.config.get("scheduler-address")
elif scheduler_port is None:
scheduler_addr = coerce_to_address(scheduler_ip)
else:
scheduler_addr = coerce_to_address((scheduler_ip, scheduler_port))
self.contact_address = contact_address
if protocol is None:
protocol_address = scheduler_addr.split("://")
if len(protocol_address) == 2:
protocol = protocol_address[0]
assert protocol
self._start_port = port
self._start_host = host
if host:
# Helpful error message if IPv6 specified incorrectly
_, host_address = parse_address(host)
if host_address.count(":") > 1 and not host_address.startswith("["):
raise ValueError(
"Host address with IPv6 must be bracketed like '[::1]'; "
f"got {host_address}"
)
self._interface = interface
self._protocol = protocol
nthreads = nthreads or CPU_COUNT
if resources is None:
resources = dask.config.get("distributed.worker.resources")
assert isinstance(resources, dict)
self.extensions = {}
if silence_logs:
silence_logging(level=silence_logs)
if isinstance(security, dict):
security = Security(**security)
self.security = security or Security()
assert isinstance(self.security, Security)
self.connection_args = self.security.get_connection_args("worker")
self.loop = self.io_loop = IOLoop.current()
if scheduler_sni:
self.connection_args["server_hostname"] = scheduler_sni
# Common executors always available
self.executors = {
"offload": utils._offload_executor,
"actor": ThreadPoolExecutor(1, thread_name_prefix="Dask-Actor-Threads"),
}
if nvml.device_get_count() > 0:
self.executors["gpu"] = ThreadPoolExecutor(
1, thread_name_prefix="Dask-GPU-Threads"
)
# Find the default executor
if executor == "offload":
self.executors["default"] = self.executors["offload"]
elif isinstance(executor, dict):
self.executors.update(executor)
elif executor is not None:
self.executors["default"] = executor
if "default" not in self.executors:
self.executors["default"] = ThreadPoolExecutor(
nthreads, thread_name_prefix="Dask-Default-Threads"
)
self.batched_stream = BatchedSend(interval="2ms", loop=self.loop)
self.name = name
self.scheduler_delay = 0
self.stream_comms = {}
if self.local_directory not in sys.path:
sys.path.insert(0, self.local_directory)
self.plugins = {}
self._pending_plugins = plugins
self.services = {}
self.service_specs = services or {}
self._dashboard_address = dashboard_address
self._dashboard = dashboard
self._http_prefix = http_prefix
self.metrics = dict(metrics) if metrics else {}
self.startup_information = (
dict(startup_information) if startup_information else {}
)
if low_level_profiler is None:
low_level_profiler = dask.config.get("distributed.worker.profile.low-level")
self.low_level_profiler = low_level_profiler
handlers = {
"gather": self.gather,
"run": self.run,
"run_coroutine": self.run_coroutine,
"get_data": self.get_data,
"update_data": self.update_data,
"free_keys": self._handle_remote_stimulus(FreeKeysEvent),
"terminate": self.close,
"ping": pingpong,
"upload_file": self.upload_file,
"call_stack": self.get_call_stack,
"profile": self.get_profile,
"profile_metadata": self.get_profile_metadata,
"get_logs": self.get_logs,
"keys": self.keys,
"versions": self.versions,
"actor_execute": self.actor_execute,
"actor_attribute": self.actor_attribute,
"plugin-add": self.plugin_add,
"plugin-remove": self.plugin_remove,
"get_monitor_info": self.get_monitor_info,
"benchmark_disk": self.benchmark_disk,
"benchmark_memory": self.benchmark_memory,
"benchmark_network": self.benchmark_network,
"get_story": self.get_story,
}
stream_handlers = {
"close": self.close,
"cancel-compute": self._handle_remote_stimulus(CancelComputeEvent),
"acquire-replicas": self._handle_remote_stimulus(AcquireReplicasEvent),
"compute-task": self._handle_remote_stimulus(ComputeTaskEvent),
"free-keys": self._handle_remote_stimulus(FreeKeysEvent),
"remove-replicas": self._handle_remote_stimulus(RemoveReplicasEvent),
"steal-request": self._handle_remote_stimulus(StealRequestEvent),
"refresh-who-has": self._handle_remote_stimulus(RefreshWhoHasEvent),
"worker-status-change": self.handle_worker_status_change,
}
ServerNode.__init__(
self,
handlers=handlers,
stream_handlers=stream_handlers,
connection_args=self.connection_args,
**kwargs,
)
self.memory_manager = WorkerMemoryManager(
self,
data=data,
nthreads=nthreads,
memory_limit=memory_limit,
memory_target_fraction=memory_target_fraction,
memory_spill_fraction=memory_spill_fraction,
memory_pause_fraction=memory_pause_fraction,
)
transfer_incoming_bytes_limit = math.inf
transfer_incoming_bytes_fraction = dask.config.get(
"distributed.worker.memory.transfer"
)
if (
self.memory_manager.memory_limit is not None
and transfer_incoming_bytes_fraction is not False
):
transfer_incoming_bytes_limit = int(
self.memory_manager.memory_limit * transfer_incoming_bytes_fraction
)
state = WorkerState(
nthreads=nthreads,
data=self.memory_manager.data,
threads=self.threads,
plugins=self.plugins,
resources=resources,
transfer_incoming_count_limit=transfer_incoming_count_limit,
validate=validate,
transition_counter_max=transition_counter_max,
transfer_incoming_bytes_limit=transfer_incoming_bytes_limit,
transfer_message_bytes_limit=transfer_message_bytes_limit,
)
BaseWorker.__init__(self, state)
self.scheduler = self.rpc(scheduler_addr)
self.execution_state = {
"scheduler": self.scheduler.address,
"ioloop": self.loop,
"worker": self,
}
self.heartbeat_interval = parse_timedelta(heartbeat_interval, default="ms")
pc = PeriodicCallback(self.heartbeat, self.heartbeat_interval * 1000)
self.periodic_callbacks["heartbeat"] = pc
pc = PeriodicCallback(lambda: self.batched_send({"op": "keep-alive"}), 60000)
self.periodic_callbacks["keep-alive"] = pc
pc = PeriodicCallback(self.find_missing, 1000)
self.periodic_callbacks["find-missing"] = pc
self._address = contact_address
if extensions is None:
extensions = DEFAULT_EXTENSIONS
self.extensions = {
name: extension(self) for name, extension in extensions.items()
}
setproctitle("dask worker [not started]")
if dask.config.get("distributed.worker.profile.enabled"):
profile_trigger_interval = parse_timedelta(
dask.config.get("distributed.worker.profile.interval"), default="ms"
)
pc = PeriodicCallback(self.trigger_profile, profile_trigger_interval * 1000)
self.periodic_callbacks["profile"] = pc
pc = PeriodicCallback(self.cycle_profile, profile_cycle_interval * 1000)
self.periodic_callbacks["profile-cycle"] = pc
if lifetime is None:
lifetime = dask.config.get("distributed.worker.lifetime.duration")
lifetime = parse_timedelta(lifetime)
if lifetime_stagger is None:
lifetime_stagger = dask.config.get("distributed.worker.lifetime.stagger")
lifetime_stagger = parse_timedelta(lifetime_stagger)
if lifetime_restart is None:
lifetime_restart = dask.config.get("distributed.worker.lifetime.restart")
self.lifetime_restart = lifetime_restart
if lifetime:
lifetime += (random.random() * 2 - 1) * lifetime_stagger
self.io_loop.call_later(
lifetime, self.close_gracefully, reason="worker-lifetime-reached"
)
self.lifetime = lifetime
Worker._instances.add(self)
################
# Memory manager
################
memory_manager: WorkerMemoryManager
@property
def data(self) -> MutableMapping[str, Any]:
"""{task key: task payload} of all completed tasks, whether they were computed
on this Worker or computed somewhere else and then transferred here over the
network.
When using the default configuration, this is a zict buffer that automatically
spills to disk whenever the target threshold is exceeded.
If spilling is disabled, it is a plain dict instead.
It could also be a user-defined arbitrary dict-like passed when initialising
the Worker or the Nanny.
Worker logic should treat this opaquely and stick to the MutableMapping API.
.. note::
This same collection is also available at ``self.state.data`` and
``self.memory_manager.data``.
"""
return self.memory_manager.data
# Deprecated attributes moved to self.memory_manager.<name>
memory_limit = DeprecatedMemoryManagerAttribute()
memory_target_fraction = DeprecatedMemoryManagerAttribute()
memory_spill_fraction = DeprecatedMemoryManagerAttribute()
memory_pause_fraction = DeprecatedMemoryManagerAttribute()
memory_monitor = DeprecatedMemoryMonitor()
###########################
# State machine accessors #
###########################
# Deprecated attributes moved to self.state.<name>
actors = DeprecatedWorkerStateAttribute()
available_resources = DeprecatedWorkerStateAttribute()
busy_workers = DeprecatedWorkerStateAttribute()
comm_nbytes = DeprecatedWorkerStateAttribute(target="transfer_incoming_bytes")
comm_threshold_bytes = DeprecatedWorkerStateAttribute(
target="transfer_incoming_bytes_throttle_threshold"
)
constrained = DeprecatedWorkerStateAttribute()
data_needed_per_worker = DeprecatedWorkerStateAttribute(target="data_needed")
executed_count = DeprecatedWorkerStateAttribute()
executing_count = DeprecatedWorkerStateAttribute()
generation = DeprecatedWorkerStateAttribute()
has_what = DeprecatedWorkerStateAttribute()
incoming_count = DeprecatedWorkerStateAttribute(
target="transfer_incoming_count_total"
)
in_flight_tasks = DeprecatedWorkerStateAttribute(target="in_flight_tasks_count")
in_flight_workers = DeprecatedWorkerStateAttribute()
log = DeprecatedWorkerStateAttribute()
long_running = DeprecatedWorkerStateAttribute()
nthreads = DeprecatedWorkerStateAttribute()
stimulus_log = DeprecatedWorkerStateAttribute()
stimulus_story = DeprecatedWorkerStateAttribute()
story = DeprecatedWorkerStateAttribute()
ready = DeprecatedWorkerStateAttribute()
tasks = DeprecatedWorkerStateAttribute()
target_message_size = DeprecatedWorkerStateAttribute(
target="transfer_message_bytes_limit"
)
total_out_connections = DeprecatedWorkerStateAttribute(
target="transfer_incoming_count_limit"
)
total_resources = DeprecatedWorkerStateAttribute()
transition_counter = DeprecatedWorkerStateAttribute()
transition_counter_max = DeprecatedWorkerStateAttribute()
validate = DeprecatedWorkerStateAttribute()
validate_task = DeprecatedWorkerStateAttribute()
@property
def data_needed(self) -> set[TaskState]:
warnings.warn(
"The `Worker.data_needed` attribute has been removed; "
"use `Worker.state.data_needed[address]`",
FutureWarning,
)
return {ts for tss in self.state.data_needed.values() for ts in tss}
@property
def waiting_for_data_count(self) -> int:
warnings.warn(
"The `Worker.waiting_for_data_count` attribute has been removed; "
"use `len(Worker.state.waiting)`",
FutureWarning,
)
return len(self.state.waiting)
##################
# Administrative #
##################
def __repr__(self) -> str:
name = f", name: {self.name}" if self.name != self.address_safe else ""
return (
f"<{self.__class__.__name__} {self.address_safe!r}{name}, "
f"status: {self.status.name}, "
f"stored: {len(self.data)}, "
f"running: {self.state.executing_count}/{self.state.nthreads}, "
f"ready: {len(self.state.ready)}, "
f"comm: {self.state.in_flight_tasks_count}, "
f"waiting: {len(self.state.waiting)}>"
)
@property
def logs(self):
return self._deque_handler.deque
def log_event(self, topic: str | Collection[str], msg: Any) -> None:
full_msg = {
"op": "log-event",
"topic": topic,
"msg": msg,
}
if self.thread_id == threading.get_ident():
self.batched_send(full_msg)
else:
self.loop.add_callback(self.batched_send, full_msg)
@property
def worker_address(self):
"""For API compatibility with Nanny"""
return self.address
@property
def executor(self):
return self.executors["default"]
@ServerNode.status.setter # type: ignore
def status(self, value: Status) -> None:
"""Override Server.status to notify the Scheduler of status changes.
Also handles pausing/unpausing.
"""
prev_status = self.status
ServerNode.status.__set__(self, value) # type: ignore
stimulus_id = f"worker-status-change-{time()}"
self._send_worker_status_change(stimulus_id)
if prev_status == Status.running and value != Status.running:
self.handle_stimulus(PauseEvent(stimulus_id=stimulus_id))
elif value == Status.running and prev_status in (
Status.paused,
Status.closing_gracefully,
):
self.handle_stimulus(UnpauseEvent(stimulus_id=stimulus_id))
def _send_worker_status_change(self, stimulus_id: str) -> None:
self.batched_send(
{
"op": "worker-status-change",
"status": self._status.name,
"stimulus_id": stimulus_id,
},
)
async def get_metrics(self) -> dict:
try:
spilled_memory, spilled_disk = self.data.spilled_total # type: ignore
except AttributeError:
# spilling is disabled
spilled_memory, spilled_disk = 0, 0
out = dict(
task_counts=self.state.task_counts,
bandwidth={
"total": self.bandwidth,
"workers": dict(self.bandwidth_workers),
"types": keymap(typename, self.bandwidth_types),
},
managed_bytes=self.state.nbytes,
spilled_bytes={
"memory": spilled_memory,
"disk": spilled_disk,
},
transfer={
"incoming_bytes": self.state.transfer_incoming_bytes,
"incoming_count": self.state.transfer_incoming_count,
"incoming_count_total": self.state.transfer_incoming_count_total,
"outgoing_bytes": self.transfer_outgoing_bytes,
"outgoing_count": self.transfer_outgoing_count,
"outgoing_count_total": self.transfer_outgoing_count_total,
},
event_loop_interval=self._tick_interval_observed,
)
monitor_recent = self.monitor.recent()
# Convert {foo.bar: 123} to {foo: {bar: 123}}
for k, v in monitor_recent.items():
if "." in k:
k0, _, k1 = k.partition(".")
out.setdefault(k0, {})[k1] = v
else:
out[k] = v
for k, metric in self.metrics.items():
try:
result = metric(self)
if isawaitable(result):
result = await result
# In case of collision, prefer core metrics
out.setdefault(k, result)
except Exception: # TODO: log error once
pass
return out
async def get_startup_information(self):
result = {}
for k, f in self.startup_information.items():
try:
v = f(self)
if isawaitable(v):
v = await v
result[k] = v
except Exception: # TODO: log error once
pass
return result
def identity(self):
return {
"type": type(self).__name__,
"id": self.id,
"scheduler": self.scheduler.address,
"nthreads": self.state.nthreads,
"memory_limit": self.memory_manager.memory_limit,
}
def _to_dict(self, *, exclude: Container[str] = ()) -> dict:
"""Dictionary representation for debugging purposes.
Not type stable and not intended for roundtrips.
See also
--------
Worker.identity
Client.dump_cluster_state
distributed.utils.recursive_to_dict
"""
info = super()._to_dict(exclude=exclude)
extra = {
"status": self.status,
"logs": self.get_logs(),
"config": dask.config.config,
"transfer_incoming_log": self.transfer_incoming_log,
"transfer_outgoing_log": self.transfer_outgoing_log,
}
extra = {k: v for k, v in extra.items() if k not in exclude}
info.update(extra)
info.update(self.state._to_dict(exclude=exclude))
info.update(self.memory_manager._to_dict(exclude=exclude))
return recursive_to_dict(info, exclude=exclude)
#####################
# External Services #
#####################
def batched_send(self, msg: dict[str, Any]) -> None:
"""Implements BaseWorker abstract method.
Send a fire-and-forget message to the scheduler through bulk comms.
If we're not currently connected to the scheduler, the message will be silently
dropped!
See also
--------
distributed.worker_state_machine.BaseWorker.batched_send
"""
if (
self.batched_stream
and self.batched_stream.comm
and not self.batched_stream.comm.closed()
):
self.batched_stream.send(msg)
async def _register_with_scheduler(self) -> None:
self.periodic_callbacks["keep-alive"].stop()
self.periodic_callbacks["heartbeat"].stop()
start = time()
if self.contact_address is None:
self.contact_address = self.address
logger.info("-" * 49)
while True:
try:
_start = time()
comm = await connect(self.scheduler.address, **self.connection_args)
comm.name = "Worker->Scheduler"
comm._server = weakref.ref(self)
await comm.write(
dict(
op="register-worker",
reply=False,
address=self.contact_address,
status=self.status.name,
keys=list(self.data),
nthreads=self.state.nthreads,
name=self.name,
nbytes={
ts.key: ts.get_nbytes()
for ts in self.state.tasks.values()
# Only if the task is in memory this is a sensible
# result since otherwise it simply submits the
# default value
if ts.state == "memory"
},
types={k: typename(v) for k, v in self.data.items()},
now=time(),
resources=self.state.total_resources,
memory_limit=self.memory_manager.memory_limit,
local_directory=self.local_directory,
services=self.service_ports,
nanny=self.nanny,
pid=os.getpid(),
versions=get_versions(),
metrics=await self.get_metrics(),
extra=await self.get_startup_information(),
stimulus_id=f"worker-connect-{time()}",
server_id=self.id,
),
serializers=["msgpack"],
)
future = comm.read(deserializers=["msgpack"])
response = await future
if response.get("warning"):
logger.warning(response["warning"])
_end = time()
middle = (_start + _end) / 2
self._update_latency(_end - start)
self.scheduler_delay = response["time"] - middle
self.status = Status.running
break
except OSError:
logger.info("Waiting to connect to: %26s", self.scheduler.address)
await asyncio.sleep(0.1)
except TimeoutError: # pragma: no cover
logger.info("Timed out when connecting to scheduler")
if response["status"] != "OK":
msg = response["message"] if "message" in response else repr(response)
logger.error(f"Unable to connect to scheduler: {msg}")
raise ValueError(f"Unexpected response from register: {response!r}")
else:
await asyncio.gather(
*(
self.plugin_add(name=name, plugin=plugin)
for name, plugin in response["worker-plugins"].items()
)
)
logger.info(" Registered to: %26s", self.scheduler.address)
logger.info("-" * 49)
self.batched_stream.start(comm)
self.periodic_callbacks["keep-alive"].start()
self.periodic_callbacks["heartbeat"].start()
self.loop.add_callback(self.handle_scheduler, comm)
def _update_latency(self, latency: float) -> None:
self.latency = latency * 0.05 + self.latency * 0.95
self.digest_metric("latency", latency)
async def heartbeat(self) -> None:
logger.debug("Heartbeat: %s", self.address)
try:
start = time()
response = await retry_operation(
self.scheduler.heartbeat_worker,
address=self.contact_address,
now=start,
metrics=await self.get_metrics(),
executing={
key: start - self.state.tasks[key].start_time
for key in self.active_keys
if key in self.state.tasks
},
extensions={
name: extension.heartbeat()
for name, extension in self.extensions.items()
if hasattr(extension, "heartbeat")
},
)
end = time()
middle = (start + end) / 2
self._update_latency(end - start)
if response["status"] == "missing":
# Scheduler thought we left. Reconnection is not supported, so just shut down.
logger.error(
f"Scheduler was unaware of this worker {self.address!r}. Shutting down."
)
# Something is out of sync; have the nanny restart us if possible.
await self.close(nanny=False)
return
self.scheduler_delay = response["time"] - middle
self.periodic_callbacks["heartbeat"].callback_time = (
response["heartbeat-interval"] * 1000
)
self.bandwidth_workers.clear()
self.bandwidth_types.clear()
except OSError:
logger.exception("Failed to communicate with scheduler during heartbeat.")
except Exception:
logger.exception("Unexpected exception during heartbeat. Closing worker.")
await self.close()
raise
@fail_hard
async def handle_scheduler(self, comm: Comm) -> None:
try:
await self.handle_stream(comm)
finally:
await self.close(reason="worker-handle-scheduler-connection-broken")
async def upload_file(
self, filename: str, data: str | bytes, load: bool = True
) -> dict[str, Any]:
out_filename = os.path.join(self.local_directory, filename)
def func(data):
if isinstance(data, str):
data = data.encode()
with open(out_filename, "wb") as f:
f.write(data)
f.flush()
os.fsync(f.fileno())
return data
if len(data) < 10000:
data = func(data)
else:
data = await offload(func, data)
if load:
try:
import_file(out_filename)
cache_loads.data.clear()
except Exception as e:
logger.exception(e)
raise e
return {"status": "OK", "nbytes": len(data)}
def keys(self) -> list[str]:
return list(self.data)
async def gather(self, who_has: dict[str, list[str]]) -> dict[str, Any]:
who_has = {
k: [coerce_to_address(addr) for addr in v]
for k, v in who_has.items()
if k not in self.data
}
result, missing_keys, missing_workers = await gather_from_workers(
who_has, rpc=self.rpc, who=self.address
)
self.update_data(data=result, report=False)
if missing_keys:
logger.warning(
"Could not find data: %s on workers: %s (who_has: %s)",
missing_keys,
missing_workers,
who_has,
)
return {"status": "partial-fail", "keys": missing_keys}
else:
return {"status": "OK"}
def get_monitor_info(
self, recent: bool = False, start: int = 0
) -> dict[str, float]:
result = dict(
range_query=(
self.monitor.recent()
if recent
else self.monitor.range_query(start=start)
),
count=self.monitor.count,
last_time=self.monitor.last_time,
)
if nvml.device_get_count() > 0:
result["gpu_name"] = self.monitor.gpu_name
result["gpu_memory_total"] = self.monitor.gpu_memory_total
return result
#############
# Lifecycle #
#############
async def start_unsafe(self):
await super().start_unsafe()
enable_gc_diagnosis()
ports = parse_ports(self._start_port)
for port in ports:
start_address = address_from_user_args(
host=self._start_host,
port=port,
interface=self._interface,
protocol=self._protocol,
security=self.security,
)
kwargs = self.security.get_listen_args("worker")
if self._protocol in ("tcp", "tls"):
kwargs = kwargs.copy()
kwargs["default_host"] = get_ip(
get_address_host(self.scheduler.address)
)
try:
await self.listen(start_address, **kwargs)
except OSError as e:
if len(ports) > 1 and e.errno == errno.EADDRINUSE:
continue
else:
raise
else:
self._start_address = start_address
break
else:
raise ValueError(
f"Could not start Worker on host {self._start_host} "
f"with port {self._start_port}"
)
# Start HTTP server associated with this Worker node
routes = get_handlers(
server=self,
modules=dask.config.get("distributed.worker.http.routes"),
prefix=self._http_prefix,
)
self.start_http_server(routes, self._dashboard_address)
if self._dashboard:
try:
import distributed.dashboard.worker
except ImportError:
logger.debug("To start diagnostics web server please install Bokeh")
else:
distributed.dashboard.worker.connect(
self.http_application,
self.http_server,
self,
prefix=self._http_prefix,
)
self.ip = get_address_host(self.address)
if self.name is None:
self.name = self.address
for preload in self.preloads:
try:
await preload.start()
except Exception:
logger.exception("Failed to start preload")
# Services listen on all addresses
# Note Nanny is not a "real" service, just some metadata
# passed in service_ports...
self.start_services(self.ip)
try:
listening_address = "%s%s:%d" % (self.listener.prefix, self.ip, self.port)
except Exception:
listening_address = f"{self.listener.prefix}{self.ip}"
logger.info(" Start worker at: %26s", self.address)
logger.info(" Listening to: %26s", listening_address)
if self.name != self.address_safe:
# only if name was not None
logger.info(" Worker name: %26s", self.name)
for k, v in self.service_ports.items():
logger.info(" {:>16} at: {:>26}".format(k, self.ip + ":" + str(v)))
logger.info("Waiting to connect to: %26s", self.scheduler.address)
logger.info("-" * 49)
logger.info(" Threads: %26d", self.state.nthreads)
if self.memory_manager.memory_limit:
logger.info(
" Memory: %26s",
format_bytes(self.memory_manager.memory_limit),
)
logger.info(" Local Directory: %26s", self.local_directory)
setproctitle("dask worker [%s]" % self.address)
plugins_msgs = await asyncio.gather(
*(
self.plugin_add(plugin=plugin, catch_errors=False)
for plugin in self._pending_plugins
),
return_exceptions=True,
)
plugins_exceptions = [msg for msg in plugins_msgs if isinstance(msg, Exception)]
if len(plugins_exceptions) >= 1:
if len(plugins_exceptions) > 1:
logger.error(
"Multiple plugin exceptions raised. All exceptions will be logged, the first is raised."
)
for exc in plugins_exceptions:
logger.error(repr(exc))
raise plugins_exceptions[0]
self._pending_plugins = ()
self.state.address = self.address
await self._register_with_scheduler()
self.start_periodic_callbacks()
return self
@log_errors
async def close( # type: ignore
self,
timeout: float = 30,
executor_wait: bool = True,
nanny: bool = True,
reason: str = "worker-close",
) -> str | None:
"""Close the worker
Close asynchronous operations running on the worker, stop all executors and
comms. If requested, this also closes the nanny.
Parameters
----------
timeout
Timeout in seconds for shutting down individual instructions
executor_wait
If True, shut down executors synchronously, otherwise asynchronously
nanny
If True, close the nanny
reason
Reason for closing the worker
Returns
-------
str | None
None if worker already in closing state or failed, "OK" otherwise
"""
# FIXME: The worker should not be allowed to close the nanny. Ownership
# is the other way round. If an external caller wants to close
# nanny+worker, the nanny must be notified first. ==> Remove kwarg
# nanny, see also Scheduler.retire_workers
if self.status in (Status.closed, Status.closing, Status.failed):
logger.debug(
"Attempted to close worker that is already %s. Reason: %s",
self.status,
reason,
)
await self.finished()
return None
if self.status == Status.init:
# If the worker is still in startup/init and is started by a nanny,
# this means the nanny itself is not up, yet. If the Nanny isn't up,
# yet, it's server will not accept any incoming RPC requests and
# will block until the startup is finished.
# Therefore, this worker trying to communicate with the Nanny during
# startup is not possible and we cannot close it.
# In this case, the Nanny will automatically close after inspecting
# the worker status
nanny = False
disable_gc_diagnosis()
try:
logger.info("Stopping worker at %s. Reason: %s", self.address, reason)
except ValueError: # address not available if already closed
logger.info("Stopping worker. Reason: %s", reason)
if self.status not in WORKER_ANY_RUNNING:
logger.info("Closed worker has not yet started: %s", self.status)
if not executor_wait:
logger.info("Not waiting on executor to close")
self.status = Status.closing
# Stop callbacks before giving up control in any `await`.
# We don't want to heartbeat while closing.
for pc in self.periodic_callbacks.values():
pc.stop()
# Cancel async instructions
await BaseWorker.close(self, timeout=timeout)
for preload in self.preloads:
try:
await preload.teardown()
except Exception:
logger.exception("Failed to tear down preload")
for extension in self.extensions.values():
if hasattr(extension, "close"):
result = extension.close()
if isawaitable(result):
result = await result
if nanny and self.nanny:
with self.rpc(self.nanny) as r:
await r.close_gracefully(reason=reason)
setproctitle("dask worker [closing]")
teardowns = [
plugin.teardown(self)
for plugin in self.plugins.values()
if hasattr(plugin, "teardown")
]
await asyncio.gather(*(td for td in teardowns if isawaitable(td)))
if self._client:
# If this worker is the last one alive, clean up the worker
# initialized clients
if not any(
w
for w in Worker._instances
if w != self and w.status in WORKER_ANY_RUNNING
):
for c in Worker._initialized_clients:
# Regardless of what the client was initialized with
# we'll require the result as a future. This is
# necessary since the heuristics of asynchronous are not
# reliable and we might deadlock here
c._asynchronous = True
if c.asynchronous:
await c.close()
else:
# There is still the chance that even with us
# telling the client to be async, itself will decide
# otherwise
c.close()
await self.scheduler.close_rpc()
self._workdir.release()
self.stop_services()
# Give some time for a UCX scheduler to complete closing endpoints
# before closing self.batched_stream, otherwise the local endpoint
# may be closed too early and errors be raised on the scheduler when
# trying to send closing message.
if self._protocol == "ucx": # pragma: no cover
await asyncio.sleep(0.2)
self.batched_send({"op": "close-stream"})
if self.batched_stream:
with suppress(TimeoutError):
await self.batched_stream.close(timedelta(seconds=timeout))
for executor in self.executors.values():
if executor is utils._offload_executor:
continue # Never shutdown the offload executor
def _close(executor, wait):
if isinstance(executor, ThreadPoolExecutor):
executor._work_queue.queue.clear()
executor.shutdown(wait=wait, timeout=timeout)
else:
executor.shutdown(wait=wait)
# Waiting for the shutdown can block the event loop causing
# weird deadlocks particularly if the task that is executing in
# the thread is waiting for a server reply, e.g. when using
# worker clients, semaphores, etc.
if is_python_shutting_down():
# If we're shutting down there is no need to wait for daemon
# threads to finish
_close(executor=executor, wait=False)
else:
try:
await to_thread(_close, executor=executor, wait=executor_wait)
except RuntimeError: # Are we shutting down the process?
logger.error(
"Could not close executor %r by dispatching to thread. Trying synchronously.",
executor,
exc_info=True,
)
_close(
executor=executor, wait=executor_wait
) # Just run it directly
self.stop()
await self.rpc.close()
self.status = Status.closed
await ServerNode.close(self)
setproctitle("dask worker [closed]")
return "OK"
async def close_gracefully(
self, restart=None, reason: str = "worker-close-gracefully"
):
"""Gracefully shut down a worker
This first informs the scheduler that we're shutting down, and asks it
to move our data elsewhere. Afterwards, we close as normal
"""
if self.status in (Status.closing, Status.closing_gracefully):
await self.finished()
if self.status == Status.closed:
return
logger.info("Closing worker gracefully: %s. Reason: %s", self.address, reason)
# Wait for all tasks to leave the worker and don't accept any new ones.
# Scheduler.retire_workers will set the status to closing_gracefully and push it
# back to this worker.
await self.scheduler.retire_workers(
workers=[self.address],
close_workers=False,
remove=False,
stimulus_id=f"worker-close-gracefully-{time()}",
)
if restart is None:
restart = self.lifetime_restart
await self.close(nanny=not restart, reason=reason)
async def wait_until_closed(self):
warnings.warn("wait_until_closed has moved to finished()")
await self.finished()
assert self.status == Status.closed
################
# Worker Peers #
################
def send_to_worker(self, address, msg):
if address not in self.stream_comms:
bcomm = BatchedSend(interval="1ms", loop=self.loop)
self.stream_comms[address] = bcomm
async def batched_send_connect():
comm = await connect(
address, **self.connection_args # TODO, serialization
)
comm.name = "Worker->Worker"
await comm.write({"op": "connection_stream"})
bcomm.start(comm)
self._ongoing_background_tasks.call_soon(batched_send_connect)
self.stream_comms[address].send(msg)
async def get_data(
self, comm, keys=None, who=None, serializers=None, max_connections=None
) -> dict | Status:
start = time()
if max_connections is None:
max_connections = self.transfer_outgoing_count_limit
# Allow same-host connections more liberally
if (
max_connections
and comm
and get_address_host(comm.peer_address) == get_address_host(self.address)
):
max_connections = max_connections * 2
if self.status == Status.paused:
max_connections = 1
throttle_msg = (
" Throttling outgoing data transfers because worker is paused."
)
else:
throttle_msg = ""
if (
max_connections is not False
and self.transfer_outgoing_count >= max_connections
):
logger.debug(
"Worker %s has too many open connections to respond to data request "
"from %s (%d/%d).%s",
self.address,
who,
self.transfer_outgoing_count,
max_connections,
throttle_msg,
)
return {"status": "busy"}
self.transfer_outgoing_count += 1
self.transfer_outgoing_count_total += 1
data = {k: self.data[k] for k in keys if k in self.data}
if len(data) < len(keys):
for k in set(keys) - set(data):
if k in self.state.actors:
from distributed.actor import Actor
data[k] = Actor(
type(self.state.actors[k]), self.address, k, worker=self
)
msg = {"status": "OK", "data": {k: to_serialize(v) for k, v in data.items()}}
# Note: `if k in self.data` above guarantees that
# k is in self.state.tasks too and that nbytes is non-None
bytes_per_task = {k: self.state.tasks[k].nbytes or 0 for k in data}
total_bytes = sum(bytes_per_task.values())
self.transfer_outgoing_bytes += total_bytes
self.transfer_outgoing_bytes_total += total_bytes
stop = time()
# Don't log metrics if all keys are in memory
if stop - start > 0.005:
# See metrics:
# - disk-load-duration
# - get-data-load-duration
# - disk-write-target-duration
# - disk-write-spill-duration
self.digest_metric("get-data-load-duration", stop - start)
start = time()
try:
compressed = await comm.write(msg, serializers=serializers)
response = await comm.read(deserializers=serializers)
assert response == "OK", response
except OSError:
logger.exception(
"failed during get data with %s -> %s",
self.address,
who,
)
comm.abort()
raise
finally:
self.transfer_outgoing_bytes -= total_bytes
self.transfer_outgoing_count -= 1
stop = time()
self.digest_metric("get-data-send-duration", stop - start)
duration = (stop - start) or 0.5 # windows
self.transfer_outgoing_log.append(
{
"start": start + self.scheduler_delay,
"stop": stop + self.scheduler_delay,
"middle": (start + stop) / 2,
"duration": duration,
"who": who,
"keys": bytes_per_task,
"total": total_bytes,
"compressed": compressed,
"bandwidth": total_bytes / duration,
}
)
return Status.dont_reply
###################
# Local Execution #
###################
def update_data(
self,
data: dict[str, object],
report: bool = True,
stimulus_id: str | None = None,
) -> dict[str, Any]:
self.handle_stimulus(
UpdateDataEvent(
data=data,
report=report,
stimulus_id=stimulus_id or f"update-data-{time()}",
)
)
return {"nbytes": {k: sizeof(v) for k, v in data.items()}, "status": "OK"}
async def set_resources(self, **resources: float) -> None:
for r, quantity in resources.items():
if r in self.state.total_resources:
self.state.available_resources[r] += (
quantity - self.state.total_resources[r]
)
else:
self.state.available_resources[r] = quantity
self.state.total_resources[r] = quantity
await retry_operation(
self.scheduler.set_resources,
resources=self.state.total_resources,
worker=self.contact_address,
)
@log_errors
async def plugin_add(
self,
plugin: WorkerPlugin | bytes,
name: str | None = None,
catch_errors: bool = True,
) -> dict[str, Any]:
if isinstance(plugin, bytes):
# Note: historically we have accepted duck-typed classes that don't
# inherit from WorkerPlugin. Don't do `assert isinstance`.
plugin = cast("WorkerPlugin", pickle.loads(plugin))
if name is None:
name = _get_plugin_name(plugin)
assert name
if name in self.plugins:
await self.plugin_remove(name=name)
self.plugins[name] = plugin
logger.info("Starting Worker plugin %s" % name)
if hasattr(plugin, "setup"):
try:
result = plugin.setup(worker=self)
if isawaitable(result):
result = await result
except Exception as e:
if not catch_errors:
raise
msg = error_message(e)
return cast("dict[str, Any]", msg)
return {"status": "OK"}
@log_errors
async def plugin_remove(self, name: str) -> dict[str, Any]:
logger.info(f"Removing Worker plugin {name}")
try:
plugin = self.plugins.pop(name)
if hasattr(plugin, "teardown"):
result = plugin.teardown(worker=self)
if isawaitable(result):
result = await result
except Exception as e:
msg = error_message(e)
return cast("dict[str, Any]", msg)
return {"status": "OK"}
def handle_worker_status_change(self, status: str, stimulus_id: str) -> None:
new_status = Status.lookup[status] # type: ignore
if (
new_status == Status.closing_gracefully
and self._status not in WORKER_ANY_RUNNING
):
logger.error(
"Invalid Worker.status transition: %s -> %s", self._status, new_status
)
# Reiterate the current status to the scheduler to restore sync
self._send_worker_status_change(stimulus_id)
else:
# Update status and send confirmation to the Scheduler (see status.setter)
self.status = new_status
###################
# Task Management #
###################
def _handle_remote_stimulus(
self, cls: type[StateMachineEvent]
) -> Callable[..., None]:
def _(**kwargs):
event = cls(**kwargs)
self.handle_stimulus(event)
_.__name__ = f"_handle_remote_stimulus({cls.__name__})"
return _
@fail_hard
def _handle_stimulus_from_task(self, task: asyncio.Task[StateMachineEvent]) -> None:
"""Override BaseWorker method for added validation
See also
--------
distributed.worker_state_machine.BaseWorker._handle_stimulus_from_task
"""
super()._handle_stimulus_from_task(task)
@fail_hard
def handle_stimulus(self, *stims: StateMachineEvent) -> None:
"""Override BaseWorker method for added validation
See also
--------
distributed.worker_state_machine.BaseWorker.handle_stimulus
distributed.worker_state_machine.WorkerState.handle_stimulus
"""
try:
super().handle_stimulus(*stims)
except Exception as e:
if hasattr(e, "to_event"):
topic, msg = e.to_event()
self.log_event(topic, msg)
raise
def stateof(self, key: str) -> dict[str, Any]:
ts = self.state.tasks[key]
return {
"executing": ts.state == "executing",
"waiting_for_data": bool(ts.waiting_for_data),
"heap": ts in self.state.ready or ts in self.state.constrained,
"data": key in self.data,
}
async def get_story(self, keys_or_stimuli: Iterable[str]) -> list[tuple]:
return self.state.story(*keys_or_stimuli)
##########################
# Dependencies gathering #
##########################
def _get_cause(self, keys: Iterable[str]) -> TaskState:
"""For diagnostics, we want to attach a transfer to a single task. This task is
typically the next to be executed but since we're fetching tasks for potentially
many dependents, an exact match is not possible. Additionally, if a key was
fetched through acquire-replicas, dependents may not be known at all.
Returns
-------
The task to attach startstops of this transfer to
"""
cause = None
for key in keys:
ts = self.state.tasks[key]
if ts.dependents:
return next(iter(ts.dependents))
cause = ts
assert cause # Always at least one key
return cause
def _update_metrics_received_data(
self,
start: float,
stop: float,
data: dict[str, object],
cause: TaskState,
worker: str,
) -> None:
total_bytes = sum(self.state.tasks[key].get_nbytes() for key in data)
cause.startstops.append(
{
"action": "transfer",
"start": start + self.scheduler_delay,
"stop": stop + self.scheduler_delay,
"source": worker,
}
)
duration = (stop - start) or 0.010
bandwidth = total_bytes / duration
self.transfer_incoming_log.append(
{
"start": start + self.scheduler_delay,
"stop": stop + self.scheduler_delay,
"middle": (start + stop) / 2.0 + self.scheduler_delay,
"duration": duration,
"keys": {key: self.state.tasks[key].nbytes for key in data},
"total": total_bytes,
"bandwidth": bandwidth,
"who": worker,
}
)
if total_bytes > 1_000_000:
self.bandwidth = self.bandwidth * 0.95 + bandwidth * 0.05
bw, cnt = self.bandwidth_workers[worker]
self.bandwidth_workers[worker] = (bw + bandwidth, cnt + 1)
types = set(map(type, data.values()))
if len(types) == 1:
[typ] = types
bw, cnt = self.bandwidth_types[typ]
self.bandwidth_types[typ] = (bw + bandwidth, cnt + 1)
self.digest_metric("transfer-bandwidth", total_bytes / duration)
self.digest_metric("transfer-duration", duration)
self.counters["transfer-count"].add(len(data))
@fail_hard
async def gather_dep(
self,
worker: str,
to_gather: Collection[str],
total_nbytes: int,
*,
stimulus_id: str,
) -> StateMachineEvent:
"""Implements BaseWorker abstract method
See also
--------
distributed.worker_state_machine.BaseWorker.gather_dep
"""
if self.status not in WORKER_ANY_RUNNING:
# This is only for the sake of coherence of the WorkerState;
# it should never actually reach the scheduler.
return GatherDepFailureEvent.from_exception(
RuntimeError("Worker is shutting down"),
worker=worker,
total_nbytes=total_nbytes,
stimulus_id=f"worker-closing-{time()}",
)
try:
self.state.log.append(
("request-dep", worker, to_gather, stimulus_id, time())
)
logger.debug("Request %d keys from %s", len(to_gather), worker)
start = time()
response = await get_data_from_worker(
self.rpc, to_gather, worker, who=self.address
)
stop = time()
if response["status"] == "busy":
self.state.log.append(
("busy-gather", worker, to_gather, stimulus_id, time())
)
return GatherDepBusyEvent(
worker=worker,
total_nbytes=total_nbytes,
stimulus_id=f"gather-dep-busy-{time()}",
)
assert response["status"] == "OK"
cause = self._get_cause(to_gather)
self._update_metrics_received_data(
start=start,
stop=stop,
data=response["data"],
cause=cause,
worker=worker,
)
self.state.log.append(
("receive-dep", worker, set(response["data"]), stimulus_id, time())
)
return GatherDepSuccessEvent(
worker=worker,
total_nbytes=total_nbytes,
data=response["data"],
stimulus_id=f"gather-dep-success-{time()}",
)
except OSError:
logger.exception("Worker stream died during communication: %s", worker)
self.state.log.append(
("receive-dep-failed", worker, to_gather, stimulus_id, time())
)
return GatherDepNetworkFailureEvent(
worker=worker,
total_nbytes=total_nbytes,
stimulus_id=f"gather-dep-network-failure-{time()}",
)
except Exception as e:
# e.g. data failed to deserialize
logger.exception(e)
if self.batched_stream and LOG_PDB:
import pdb
pdb.set_trace()
return GatherDepFailureEvent.from_exception(
e,
worker=worker,
total_nbytes=total_nbytes,
stimulus_id=f"gather-dep-failure-{time()}",
)
async def retry_busy_worker_later(self, worker: str) -> StateMachineEvent:
"""Wait some time, then take a peer worker out of busy state.
Implements BaseWorker abstract method.
See Also
--------
distributed.worker_state_machine.BaseWorker.retry_busy_worker_later
"""
await asyncio.sleep(0.15)
return RetryBusyWorkerEvent(
worker=worker, stimulus_id=f"retry-busy-worker-{time()}"
)
def digest_metric(self, name: str, value: float) -> None:
"""Implement BaseWorker.digest_metric by calling Server.digest_metric"""
ServerNode.digest_metric(self, name, value)
@log_errors
def find_missing(self) -> None:
self.handle_stimulus(FindMissingEvent(stimulus_id=f"find-missing-{time()}"))
# This is quite arbitrary but the heartbeat has scaling implemented
self.periodic_callbacks["find-missing"].callback_time = self.periodic_callbacks[
"heartbeat"
].callback_time
################
# Execute Task #
################
def run(self, comm, function, args=(), wait=True, kwargs=None):
return run(self, comm, function=function, args=args, kwargs=kwargs, wait=wait)
def run_coroutine(self, comm, function, args=(), kwargs=None, wait=True):
return run(self, comm, function=function, args=args, kwargs=kwargs, wait=wait)
async def actor_execute(
self,
actor=None,
function=None,
args=(),
kwargs: dict | None = None,
) -> dict[str, Any]:
kwargs = kwargs or {}
separate_thread = kwargs.pop("separate_thread", True)
key = actor
actor = self.state.actors[key]
func = getattr(actor, function)
name = key_split(key) + "." + function
try:
if iscoroutinefunction(func):
result = await func(*args, **kwargs)
elif separate_thread:
result = await self.loop.run_in_executor(
self.executors["actor"],
apply_function_actor,
func,
args,
kwargs,
self.execution_state,
name,
self.active_threads,
self.active_threads_lock,
)
else:
result = func(*args, **kwargs)
return {"status": "OK", "result": to_serialize(result)}
except Exception as ex:
return {"status": "error", "exception": to_serialize(ex)}
def actor_attribute(self, actor=None, attribute=None) -> dict[str, Any]:
try:
value = getattr(self.state.actors[actor], attribute)
return {"status": "OK", "result": to_serialize(value)}
except Exception as ex:
return {"status": "error", "exception": to_serialize(ex)}
async def _maybe_deserialize_task(
self, ts: TaskState
) -> tuple[Callable, tuple, dict[str, Any]]:
assert ts.run_spec is not None
start = time()
# Offload deserializing large tasks
if sizeof(ts.run_spec) > OFFLOAD_THRESHOLD:
function, args, kwargs = await offload(_deserialize, *ts.run_spec)
else:
function, args, kwargs = _deserialize(*ts.run_spec)
stop = time()
if stop - start > 0.010:
ts.startstops.append(
{"action": "deserialize", "start": start, "stop": stop}
)
return function, args, kwargs
@fail_hard
async def execute(self, key: str, *, stimulus_id: str) -> StateMachineEvent:
"""Execute a task. Implements BaseWorker abstract method.
See also
--------
distributed.worker_state_machine.BaseWorker.execute
"""
if self.status not in WORKER_ANY_RUNNING:
# This is just for internal coherence of the WorkerState; the reschedule
# message should not ever reach the Scheduler.
# It is still OK if it does though.
return RescheduleEvent(key=key, stimulus_id=f"worker-closing-{time()}")
# The key *must* be in the worker state thanks to the cancelled state
ts = self.state.tasks[key]
try:
function, args, kwargs = await self._maybe_deserialize_task(ts)
except Exception as exc:
logger.error("Could not deserialize task %s", key, exc_info=True)
return ExecuteFailureEvent.from_exception(
exc,
key=key,
stimulus_id=f"run-spec-deserialize-failed-{time()}",
)
try:
if self.state.validate:
assert not ts.waiting_for_data
assert ts.state in ("executing", "cancelled", "resumed"), ts
assert ts.run_spec is not None
args2, kwargs2 = self._prepare_args_for_execution(ts, args, kwargs)
assert ts.annotations is not None
executor = ts.annotations.get("executor", "default")
try:
e = self.executors[executor]
except KeyError:
raise ValueError(
f"Invalid executor {executor!r}; "
f"expected one of: {sorted(self.executors)}"
)
self.active_keys.add(key)
try:
ts.start_time = time()
if iscoroutinefunction(function):
result = await apply_function_async(
function,
args2,
kwargs2,
self.scheduler_delay,
)
elif "ThreadPoolExecutor" in str(type(e)):
result = await self.loop.run_in_executor(
e,
apply_function,
function,
args2,
kwargs2,
self.execution_state,
key,
self.active_threads,
self.active_threads_lock,
self.scheduler_delay,
)
else:
result = await self.loop.run_in_executor(
e,
apply_function_simple,
function,
args2,
kwargs2,
self.scheduler_delay,
)
finally:
self.active_keys.discard(key)
self.threads[key] = result["thread"]
if result["op"] == "task-finished":
if self.digests is not None:
self.digests["task-duration"].add(result["stop"] - result["start"])
return ExecuteSuccessEvent(
key=key,
value=result["result"],
start=result["start"],
stop=result["stop"],
nbytes=result["nbytes"],
type=result["type"],
stimulus_id=f"task-finished-{time()}",
)
task_exc = result["actual-exception"]
if isinstance(task_exc, Reschedule):
return RescheduleEvent(key=ts.key, stimulus_id=f"reschedule-{time()}")
if (
self.status == Status.closing
and isinstance(task_exc, asyncio.CancelledError)
and iscoroutinefunction(function)
):
# `Worker.cancel` will cause async user tasks to raise `CancelledError`.
# Since we cancelled those tasks, we shouldn't treat them as failures.
# This is just a heuristic; it's _possible_ the task happened to
# fail independently with `CancelledError`.
logger.info(
f"Async task {key!r} cancelled during worker close; rescheduling."
)
return RescheduleEvent(
key=ts.key, stimulus_id=f"cancelled-by-worker-close-{time()}"
)
logger.warning(
"Compute Failed\n"
"Key: %s\n"
"Function: %s\n"
"args: %s\n"
"kwargs: %s\n"
"Exception: %r\n",
key,
str(funcname(function))[:1000],
convert_args_to_str(args2, max_len=1000),
convert_kwargs_to_str(kwargs2, max_len=1000),
result["exception_text"],
)
return ExecuteFailureEvent.from_exception(
result,
key=key,
start=result["start"],
stop=result["stop"],
stimulus_id=f"task-erred-{time()}",
)
except Exception as exc:
logger.error("Exception during execution of task %s.", key, exc_info=True)
return ExecuteFailureEvent.from_exception(
exc,
key=key,
stimulus_id=f"execute-unknown-error-{time()}",
)
def _prepare_args_for_execution(
self, ts: TaskState, args: tuple, kwargs: dict[str, Any]
) -> tuple[tuple, dict[str, Any]]:
start = time()
data = {}
for dep in ts.dependencies:
k = dep.key
try:
data[k] = self.data[k]
except KeyError:
from distributed.actor import Actor # TODO: create local actor
data[k] = Actor(type(self.state.actors[k]), self.address, k, self)
args2 = pack_data(args, data, key_types=(bytes, str))
kwargs2 = pack_data(kwargs, data, key_types=(bytes, str))
stop = time()
if stop - start > 0.005:
ts.startstops.append({"action": "disk-read", "start": start, "stop": stop})
# See metrics:
# - disk-load-duration
# - get-data-load-duration
# - disk-write-target-duration
# - disk-write-spill-duration
self.digest_metric("disk-load-duration", stop - start)
return args2, kwargs2
##################
# Administrative #
##################
def cycle_profile(self) -> None:
now = time() + self.scheduler_delay
prof, self.profile_recent = self.profile_recent, profile.create()
self.profile_history.append((now, prof))
self.profile_keys_history.append((now, dict(self.profile_keys)))
self.profile_keys.clear()
def trigger_profile(self) -> None:
"""
Get a frame from all actively computing threads
Merge these frames into existing profile counts
"""
if not self.active_threads: # hope that this is thread-atomic?
return
start = time()
with self.active_threads_lock:
active_threads = self.active_threads.copy()
frames = sys._current_frames()
frames = {ident: frames[ident] for ident in active_threads}
llframes = {}
if self.low_level_profiler:
llframes = {ident: profile.ll_get_stack(ident) for ident in active_threads}
for ident, frame in frames.items():
if frame is not None:
key = key_split(active_threads[ident])
llframe = llframes.get(ident)
state = profile.process(
frame, True, self.profile_recent, stop="distributed/worker.py"
)
profile.llprocess(llframe, None, state)
profile.process(
frame, True, self.profile_keys[key], stop="distributed/worker.py"
)
stop = time()
self.digest_metric("profile-duration", stop - start)
async def get_profile(
self,
start=None,
stop=None,
key=None,
server: bool = False,
):
now = time() + self.scheduler_delay
if server:
history = self.io_loop.profile
elif key is None:
history = self.profile_history
else:
history = [(t, d[key]) for t, d in self.profile_keys_history if key in d]
if start is None:
istart = 0
else:
istart = bisect.bisect_left(history, (start,))
if stop is None:
istop = None
else:
istop = bisect.bisect_right(history, (stop,)) + 1
if istop >= len(history):
istop = None # include end
if istart == 0 and istop is None:
history = list(history)
else:
iistop = len(history) if istop is None else istop
history = [history[i] for i in range(istart, iistop)]
prof = profile.merge(*pluck(1, history))
if not history:
return profile.create()
if istop is None and (start is None or start < now):
if key is None:
recent = self.profile_recent
else:
recent = self.profile_keys[key]
prof = profile.merge(prof, recent)
return prof
async def get_profile_metadata(
self, start: float = 0, stop: float | None = None
) -> dict[str, Any]:
add_recent = stop is None
now = time() + self.scheduler_delay
stop = stop or now
result = {
"counts": [
(t, d["count"]) for t, d in self.profile_history if start < t < stop
],
"keys": [
(t, {k: d["count"] for k, d in v.items()})
for t, v in self.profile_keys_history
if start < t < stop
],
}
if add_recent:
result["counts"].append((now, self.profile_recent["count"]))
result["keys"].append(
(now, {k: v["count"] for k, v in self.profile_keys.items()})
)
return result
def get_call_stack(self, keys: Collection[str] | None = None) -> dict[str, Any]:
with self.active_threads_lock:
sys_frames = sys._current_frames()
frames = {key: sys_frames[tid] for tid, key in self.active_threads.items()}
if keys is not None:
frames = {key: frames[key] for key in keys if key in frames}
return {key: profile.call_stack(frame) for key, frame in frames.items()}
async def benchmark_disk(self) -> dict[str, float]:
return await self.loop.run_in_executor(
self.executor, benchmark_disk, self.local_directory
)
async def benchmark_memory(self) -> dict[str, float]:
return await self.loop.run_in_executor(self.executor, benchmark_memory)
async def benchmark_network(self, address: str) -> dict[str, float]:
return await benchmark_network(rpc=self.rpc, address=address)
#######################################
# Worker Clients (advanced workloads) #
#######################################
@property
def client(self) -> Client:
with self._lock:
if self._client:
return self._client
else:
return self._get_client()
def _get_client(self, timeout: float | None = None) -> Client:
"""Get local client attached to this worker
If no such client exists, create one
See Also
--------
get_client
"""
if timeout is None:
timeout = dask.config.get("distributed.comm.timeouts.connect")
timeout = parse_timedelta(timeout, "s")
try:
from distributed.client import default_client
client = default_client()
except ValueError: # no clients found, need to make a new one
pass
else:
# must be lazy import otherwise cyclic import
from distributed.deploy.cluster import Cluster
if (
client.scheduler
and client.scheduler.address == self.scheduler.address
# The below conditions should only happen in case a second
# cluster is alive, e.g. if a submitted task spawned its onwn
# LocalCluster, see gh4565
or (
isinstance(client._start_arg, str)
and client._start_arg == self.scheduler.address
or isinstance(client._start_arg, Cluster)
and client._start_arg.scheduler_address == self.scheduler.address
)
):
self._client = client
if not self._client:
from distributed.client import Client
asynchronous = in_async_call(self.loop)
self._client = Client(
self.scheduler,
loop=self.loop,
security=self.security,
set_as_default=True,
asynchronous=asynchronous,
direct_to_workers=True,
name="worker",
timeout=timeout,
)
Worker._initialized_clients.add(self._client)
if not asynchronous:
assert self._client.status == "running"
return self._client
def get_current_task(self) -> str:
"""Get the key of the task we are currently running
This only makes sense to run within a task
Examples
--------
>>> from dask.distributed import get_worker
>>> def f():
... return get_worker().get_current_task()
>>> future = client.submit(f) # doctest: +SKIP
>>> future.result() # doctest: +SKIP
'f-1234'
See Also
--------
get_worker
"""
return self.active_threads[threading.get_ident()]
def validate_state(self) -> None:
try:
self.state.validate_state()
except Exception as e:
logger.error("Validate state failed", exc_info=e)
logger.exception(e)
if LOG_PDB:
import pdb
pdb.set_trace()
if hasattr(e, "to_event"):
topic, msg = e.to_event()
self.log_event(topic, msg)
raise
@property
def incoming_transfer_log(self):
warnings.warn(
"The `Worker.incoming_transfer_log` attribute has been renamed to "
"`Worker.transfer_incoming_log`",
DeprecationWarning,
stacklevel=2,
)
return self.transfer_incoming_log
@property
def outgoing_count(self):
warnings.warn(
"The `Worker.outgoing_count` attribute has been renamed to "
"`Worker.transfer_outgoing_count_total`",
DeprecationWarning,
stacklevel=2,
)
return self.transfer_outgoing_count_total
@property
def outgoing_current_count(self):
warnings.warn(
"The `Worker.outgoing_current_count` attribute has been renamed to "
"`Worker.transfer_outgoing_count`",
DeprecationWarning,
stacklevel=2,
)
return self.transfer_outgoing_count
@property
def outgoing_transfer_log(self):
warnings.warn(
"The `Worker.outgoing_transfer_log` attribute has been renamed to "
"`Worker.transfer_outgoing_log`",
DeprecationWarning,
stacklevel=2,
)
return self.transfer_outgoing_log
@property
def total_in_connections(self):
warnings.warn(
"The `Worker.total_in_connections` attribute has been renamed to "
"`Worker.transfer_outgoing_count_limit`",
DeprecationWarning,
stacklevel=2,
)
return self.transfer_outgoing_count_limit
def get_worker() -> Worker:
"""Get the worker currently running this task
Examples
--------
>>> def f():
... worker = get_worker() # The worker on which this task is running
... return worker.address
>>> future = client.submit(f) # doctest: +SKIP
>>> future.result() # doctest: +SKIP
'tcp://127.0.0.1:47373'
See Also
--------
get_client
worker_client
"""
try:
return thread_state.execution_state["worker"]
except AttributeError:
try:
return first(w for w in Worker._instances if w.status in WORKER_ANY_RUNNING)
except StopIteration:
raise ValueError("No workers found")
def get_client(address=None, timeout=None, resolve_address=True) -> Client:
"""Get a client while within a task.
This client connects to the same scheduler to which the worker is connected
Parameters
----------
address : str, optional
The address of the scheduler to connect to. Defaults to the scheduler
the worker is connected to.
timeout : int or str
Timeout (in seconds) for getting the Client. Defaults to the
``distributed.comm.timeouts.connect`` configuration value.
resolve_address : bool, default True
Whether to resolve `address` to its canonical form.
Returns
-------
Client
Examples
--------
>>> def f():
... client = get_client(timeout="10s")
... futures = client.map(lambda x: x + 1, range(10)) # spawn many tasks
... results = client.gather(futures)
... return sum(results)
>>> future = client.submit(f) # doctest: +SKIP
>>> future.result() # doctest: +SKIP
55
See Also
--------
get_worker
worker_client
secede
"""
if timeout is None:
timeout = dask.config.get("distributed.comm.timeouts.connect")
timeout = parse_timedelta(timeout, "s")
if address and resolve_address:
address = comm_resolve_address(address)
try:
worker = get_worker()
except ValueError: # could not find worker
pass
else:
if not address or worker.scheduler.address == address:
return worker._get_client(timeout=timeout)
from distributed.client import Client
try:
client = Client.current() # TODO: assumes the same scheduler
except ValueError:
client = None
if client and (not address or client.scheduler.address == address):
return client
elif address:
return Client(address, timeout=timeout)
else:
raise ValueError("No global client found and no address provided")
def secede():
"""
Have this task secede from the worker's thread pool
This opens up a new scheduling slot and a new thread for a new task. This
enables the client to schedule tasks on this node, which is
especially useful while waiting for other jobs to finish (e.g., with
``client.gather``).
Examples
--------
>>> def mytask(x):
... # do some work
... client = get_client()
... futures = client.map(...) # do some remote work
... secede() # while that work happens, remove ourself from the pool
... return client.gather(futures) # return gathered results
See Also
--------
get_client
get_worker
"""
worker = get_worker()
tpe_secede() # have this thread secede from the thread pool
duration = time() - thread_state.start_time
worker.loop.add_callback(
worker.handle_stimulus,
SecedeEvent(
key=thread_state.key,
compute_duration=duration,
stimulus_id=f"secede-{time()}",
),
)
class Reschedule(Exception):
"""Reschedule this task
Raising this exception will stop the current execution of the task and ask
the scheduler to reschedule this task, possibly on a different machine.
This does not guarantee that the task will move onto a different machine.
The scheduler will proceed through its normal heuristics to determine the
optimal machine to accept this task. The machine will likely change if the
load across the cluster has significantly changed since first scheduling
the task.
"""
async def get_data_from_worker(
rpc,
keys,
worker,
who=None,
max_connections=None,
serializers=None,
deserializers=None,
):
"""Get keys from worker
The worker has a two step handshake to acknowledge when data has been fully
delivered. This function implements that handshake.
See Also
--------
Worker.get_data
Worker.gather_dep
utils_comm.gather_data_from_workers
"""
if serializers is None:
serializers = rpc.serializers
if deserializers is None:
deserializers = rpc.deserializers
async def _get_data():
comm = await rpc.connect(worker)
comm.name = "Ephemeral Worker->Worker for gather"
try:
response = await send_recv(
comm,
serializers=serializers,
deserializers=deserializers,
op="get_data",
keys=keys,
who=who,
max_connections=max_connections,
)
try:
status = response["status"]
except KeyError: # pragma: no cover
raise ValueError("Unexpected response", response)
else:
if status == "OK":
await comm.write("OK")
return response
finally:
rpc.reuse(worker, comm)
return await retry_operation(_get_data, operation="get_data_from_worker")
job_counter = [0]
cache_loads = LRU(maxsize=100)
def loads_function(bytes_object):
"""Load a function from bytes, cache bytes"""
if len(bytes_object) < 100000:
try:
result = cache_loads[bytes_object]
except KeyError:
result = pickle.loads(bytes_object)
cache_loads[bytes_object] = result
return result
return pickle.loads(bytes_object)
def _deserialize(function=None, args=None, kwargs=None, task=NO_VALUE):
"""Deserialize task inputs and regularize to func, args, kwargs"""
if function is not None:
function = loads_function(function)
if args and isinstance(args, bytes):
args = pickle.loads(args)
if kwargs and isinstance(kwargs, bytes):
kwargs = pickle.loads(kwargs)
if task is not NO_VALUE:
assert not function and not args and not kwargs
function = execute_task
args = (task,)
return function, args or (), kwargs or {}
def execute_task(task):
"""Evaluate a nested task
>>> inc = lambda x: x + 1
>>> execute_task((inc, 1))
2
>>> execute_task((sum, [1, 2, (inc, 3)]))
7
"""
if istask(task):
func, args = task[0], task[1:]
return func(*map(execute_task, args))
elif isinstance(task, list):
return list(map(execute_task, task))
else:
return task
cache_dumps = LRU(maxsize=100)
_cache_lock = threading.Lock()
def dumps_function(func) -> bytes:
"""Dump a function to bytes, cache functions"""
try:
with _cache_lock:
result = cache_dumps[func]
except KeyError:
result = pickle.dumps(func)
if len(result) < 100000:
with _cache_lock:
cache_dumps[func] = result
except TypeError: # Unhashable function
result = pickle.dumps(func)
return result
def dumps_task(task):
"""Serialize a dask task
Returns a dict of bytestrings that can each be loaded with ``loads``
Examples
--------
Either returns a task as a function, args, kwargs dict
>>> from operator import add
>>> dumps_task((add, 1)) # doctest: +SKIP
{'function': b'\x80\x04\x95\x00\x8c\t_operator\x94\x8c\x03add\x94\x93\x94.'
'args': b'\x80\x04\x95\x07\x00\x00\x00K\x01K\x02\x86\x94.'}
Or as a single task blob if it can't easily decompose the result. This
happens either if the task is highly nested, or if it isn't a task at all
>>> dumps_task(1) # doctest: +SKIP
{'task': b'\x80\x04\x95\x03\x00\x00\x00\x00\x00\x00\x00K\x01.'}
"""
if istask(task):
if task[0] is apply and not any(map(_maybe_complex, task[2:])):
d = {"function": dumps_function(task[1]), "args": warn_dumps(task[2])}
if len(task) == 4:
d["kwargs"] = warn_dumps(task[3])
return d
elif not any(map(_maybe_complex, task[1:])):
return {"function": dumps_function(task[0]), "args": warn_dumps(task[1:])}
return to_serialize(task)
_warn_dumps_warned = [False]
def warn_dumps(obj, dumps=pickle.dumps, limit=1e6):
"""Dump an object to bytes, warn if those bytes are large"""
b = dumps(obj)
if not _warn_dumps_warned[0] and len(b) > limit:
_warn_dumps_warned[0] = True
s = str(obj)
if len(s) > 70:
s = s[:50] + " ... " + s[-15:]
warnings.warn(
"Large object of size %s detected in task graph: \n"
" %s\n"
"Consider scattering large objects ahead of time\n"
"with client.scatter to reduce scheduler burden and \n"
"keep data on workers\n\n"
" future = client.submit(func, big_data) # bad\n\n"
" big_future = client.scatter(big_data) # good\n"
" future = client.submit(func, big_future) # good"
% (format_bytes(len(b)), s)
)
return b
def apply_function(
function,
args,
kwargs,
execution_state,
key,
active_threads,
active_threads_lock,
time_delay,
):
"""Run a function, collect information
Returns
-------
msg: dictionary with status, result/error, timings, etc..
"""
ident = threading.get_ident()
with active_threads_lock:
active_threads[ident] = key
thread_state.start_time = time()
thread_state.execution_state = execution_state
thread_state.key = key
msg = apply_function_simple(function, args, kwargs, time_delay)
with active_threads_lock:
del active_threads[ident]
return msg
def apply_function_simple(
function,
args,
kwargs,
time_delay,
):
"""Run a function, collect information
Returns
-------
msg: dictionary with status, result/error, timings, etc..
"""
ident = threading.get_ident()
start = time()
try:
result = function(*args, **kwargs)
except (SystemExit, KeyboardInterrupt):
# Special-case these, just like asyncio does all over the place. They will pass
# through `fail_hard` and `_handle_stimulus_from_task`, and eventually be caught
# by special-case logic in asyncio:
# https://github.com/python/cpython/blob/v3.9.4/Lib/asyncio/events.py#L81-L82
# Any other `BaseException` types would ultimately be ignored by asyncio if
# raised here, after messing up the worker state machine along their way.
raise
except BaseException as e:
# Users _shouldn't_ use `BaseException`s, but if they do, we can assume they
# aren't a reason to shut down the whole system (since we allow the
# system-shutting-down `SystemExit` and `KeyboardInterrupt` to pass through)
msg = error_message(e)
msg["op"] = "task-erred"
msg["actual-exception"] = e
else:
msg = {
"op": "task-finished",
"status": "OK",
"result": result,
"nbytes": sizeof(result),
"type": type(result) if result is not None else None,
}
finally:
end = time()
msg["start"] = start + time_delay
msg["stop"] = end + time_delay
msg["thread"] = ident
return msg
async def apply_function_async(
function,
args,
kwargs,
time_delay,
):
"""Run a function, collect information
Returns
-------
msg: dictionary with status, result/error, timings, etc..
"""
ident = threading.get_ident()
start = time()
try:
result = await function(*args, **kwargs)
except (SystemExit, KeyboardInterrupt):
# Special-case these, just like asyncio does all over the place. They will pass
# through `fail_hard` and `_handle_stimulus_from_task`, and eventually be caught
# by special-case logic in asyncio:
# https://github.com/python/cpython/blob/v3.9.4/Lib/asyncio/events.py#L81-L82
# Any other `BaseException` types would ultimately be ignored by asyncio if
# raised here, after messing up the worker state machine along their way.
raise
except BaseException as e:
# NOTE: this includes `CancelledError`! Since it's a user task, that's _not_ a
# reason to shut down the worker.
# Users _shouldn't_ use `BaseException`s, but if they do, we can assume they
# aren't a reason to shut down the whole system (since we allow the
# system-shutting-down `SystemExit` and `KeyboardInterrupt` to pass through)
msg = error_message(e)
msg["op"] = "task-erred"
msg["actual-exception"] = e
else:
msg = {
"op": "task-finished",
"status": "OK",
"result": result,
"nbytes": sizeof(result),
"type": type(result) if result is not None else None,
}
finally:
end = time()
msg["start"] = start + time_delay
msg["stop"] = end + time_delay
msg["thread"] = ident
return msg
def apply_function_actor(
function, args, kwargs, execution_state, key, active_threads, active_threads_lock
):
"""Run a function, collect information
Returns
-------
msg: dictionary with status, result/error, timings, etc..
"""
ident = threading.get_ident()
with active_threads_lock:
active_threads[ident] = key
thread_state.execution_state = execution_state
thread_state.key = key
thread_state.actor = True
result = function(*args, **kwargs)
with active_threads_lock:
del active_threads[ident]
return result
def get_msg_safe_str(msg):
"""Make a worker msg, which contains args and kwargs, safe to cast to str:
allowing for some arguments to raise exceptions during conversion and
ignoring them.
"""
class Repr:
def __init__(self, f, val):
self._f = f
self._val = val
def __repr__(self):
return self._f(self._val)
msg = msg.copy()
if "args" in msg:
msg["args"] = Repr(convert_args_to_str, msg["args"])
if "kwargs" in msg:
msg["kwargs"] = Repr(convert_kwargs_to_str, msg["kwargs"])
return msg
def convert_args_to_str(args, max_len: int | None = None) -> str:
"""Convert args to a string, allowing for some arguments to raise
exceptions during conversion and ignoring them.
"""
length = 0
strs = ["" for i in range(len(args))]
for i, arg in enumerate(args):
try:
sarg = repr(arg)
except Exception:
sarg = "< could not convert arg to str >"
strs[i] = sarg
length += len(sarg) + 2
if max_len is not None and length > max_len:
return "({}".format(", ".join(strs[: i + 1]))[:max_len]
else:
return "({})".format(", ".join(strs))
def convert_kwargs_to_str(kwargs: dict, max_len: int | None = None) -> str:
"""Convert kwargs to a string, allowing for some arguments to raise
exceptions during conversion and ignoring them.
"""
length = 0
strs = ["" for i in range(len(kwargs))]
for i, (argname, arg) in enumerate(kwargs.items()):
try:
sarg = repr(arg)
except Exception:
sarg = "< could not convert arg to str >"
skwarg = repr(argname) + ": " + sarg
strs[i] = skwarg
length += len(skwarg) + 2
if max_len is not None and length > max_len:
return "{{{}".format(", ".join(strs[: i + 1]))[:max_len]
else:
return "{{{}}}".format(", ".join(strs))
async def run(server, comm, function, args=(), kwargs=None, wait=True):
kwargs = kwargs or {}
function = pickle.loads(function)
is_coro = iscoroutinefunction(function)
assert wait or is_coro, "Combination not supported"
if args:
args = pickle.loads(args)
if kwargs:
kwargs = pickle.loads(kwargs)
if has_arg(function, "dask_worker"):
kwargs["dask_worker"] = server
if has_arg(function, "dask_scheduler"):
kwargs["dask_scheduler"] = server
logger.info("Run out-of-band function %r", funcname(function))
try:
if not is_coro:
result = function(*args, **kwargs)
else:
if wait:
result = await function(*args, **kwargs)
else:
server._ongoing_background_tasks.call_soon(function, *args, **kwargs)
result = None
except Exception as e:
logger.warning(
"Run Failed\nFunction: %s\nargs: %s\nkwargs: %s\n",
str(funcname(function))[:1000],
convert_args_to_str(args, max_len=1000),
convert_kwargs_to_str(kwargs, max_len=1000),
exc_info=True,
)
response = error_message(e)
else:
response = {"status": "OK", "result": to_serialize(result)}
return response
_global_workers = Worker._instances
def add_gpu_metrics():
async def gpu_metric(worker):
result = await offload(nvml.real_time)
return result
DEFAULT_METRICS["gpu"] = gpu_metric
def gpu_startup(worker):
return nvml.one_time()
DEFAULT_STARTUP_INFORMATION["gpu"] = gpu_startup
def print(
*args,
sep: str | None = " ",
end: str | None = "\n",
file: TextIO | None = None,
flush: bool = False,
) -> None:
"""
A drop-in replacement of the built-in ``print`` function for remote printing
from workers to clients. If called from outside a dask worker, its arguments
are passed directly to ``builtins.print()``. If called by code running on a
worker, then in addition to printing locally, any clients connected
(possibly remotely) to the scheduler managing this worker will receive an
event instructing them to print the same output to their own standard output
or standard error streams. For example, the user can perform simple
debugging of remote computations by including calls to this ``print``
function in the submitted code and inspecting the output in a local Jupyter
notebook or interpreter session.
All arguments behave the same as those of ``builtins.print()``, with the
exception that the ``file`` keyword argument, if specified, must either be
``sys.stdout`` or ``sys.stderr``; arbitrary file-like objects are not
allowed.
All non-keyword arguments are converted to strings using ``str()`` and
written to the stream, separated by ``sep`` and followed by ``end``. Both
``sep`` and ``end`` must be strings; they can also be ``None``, which means
to use the default values. If no objects are given, ``print()`` will just
write ``end``.
Parameters
----------
sep : str, optional
String inserted between values, default a space.
end : str, optional
String appended after the last value, default a newline.
file : ``sys.stdout`` or ``sys.stderr``, optional
Defaults to the current sys.stdout.
flush : bool, default False
Whether to forcibly flush the stream.
Examples
--------
>>> from dask.distributed import Client, print
>>> client = distributed.Client(...)
>>> def worker_function():
... print("Hello from worker!")
>>> client.submit(worker_function)
<Future: finished, type: NoneType, key: worker_function-...>
Hello from worker!
"""
try:
worker = get_worker()
except ValueError:
pass
else:
# We are in a worker: prepare all of the print args and kwargs to be
# serialized over the wire to the client.
msg = {
# According to the Python stdlib docs, builtin print() simply calls
# str() on each positional argument, so we do the same here.
"args": tuple(map(str, args)),
"sep": sep,
"end": end,
"flush": flush,
}
if file == sys.stdout:
msg["file"] = 1 # type: ignore
elif file == sys.stderr:
msg["file"] = 2 # type: ignore
elif file is not None:
raise TypeError(
f"Remote printing to arbitrary file objects is not supported. file "
f"kwarg must be one of None, sys.stdout, or sys.stderr; got: {file!r}"
)
worker.log_event("print", msg)
builtins.print(*args, sep=sep, end=end, file=file, flush=flush)
def warn(
message: str | Warning,
category: type[Warning] | None = UserWarning,
stacklevel: int = 1,
source: Any = None,
) -> None:
"""
A drop-in replacement of the built-in ``warnings.warn()`` function for
issuing warnings remotely from workers to clients.
If called from outside a dask worker, its arguments are passed directly to
``warnings.warn()``. If called by code running on a worker, then in addition
to emitting a warning locally, any clients connected (possibly remotely) to
the scheduler managing this worker will receive an event instructing them to
emit the same warning (subject to their own local filters, etc.). When
implementing computations that may run on a worker, the user can call this
``warn`` function to ensure that any remote client sessions will see their
warnings, for example in a Jupyter output cell.
While all of the arguments are respected by the locally emitted warning
(with same meanings as in ``warnings.warn()``), ``stacklevel`` and
``source`` are ignored by clients because they would not be meaningful in
the client's thread.
Examples
--------
>>> from dask.distributed import Client, warn
>>> client = Client()
>>> def do_warn():
... warn("A warning from a worker.")
>>> client.submit(do_warn).result()
/path/to/distributed/client.py:678: UserWarning: A warning from a worker.
"""
try:
worker = get_worker()
except ValueError: # pragma: no cover
pass
else:
# We are in a worker: log a warn event with args serialized to the
# client. We have to pickle message and category into bytes ourselves
# because msgpack cannot handle them. The expectations is that these are
# always small objects.
worker.log_event(
"warn",
{
"message": pickle.dumps(message),
"category": pickle.dumps(category),
# We ignore stacklevel because it will be meaningless in the
# client's thread/process.
# We ignore source because we don't want to serialize arbitrary
# objects.
},
)
# add 1 to stacklevel so that, at least in the worker's local stderr, we'll
# see the source line that called us
warnings.warn(message, category, stacklevel + 1, source)
def benchmark_disk(
rootdir: str | None = None,
sizes: Iterable[str] = ("1 kiB", "100 kiB", "1 MiB", "10 MiB", "100 MiB"),
duration="1 s",
) -> dict[str, float]:
"""
Benchmark disk bandwidth
Returns
-------
out: dict
Maps sizes of outputs to measured bandwidths
"""
duration = parse_timedelta(duration)
out = {}
for size_str in sizes:
with tmpdir(dir=rootdir) as dir:
dir = pathlib.Path(dir)
names = list(map(str, range(100)))
size = parse_bytes(size_str)
data = randbytes(size)
start = time()
total = 0
while time() < start + duration:
with open(dir / random.choice(names), mode="ab") as f:
f.write(data)
f.flush()
os.fsync(f.fileno())
total += size
out[size_str] = total / (time() - start)
return out
def benchmark_memory(
sizes: Iterable[str] = ("2 kiB", "10 kiB", "100 kiB", "1 MiB", "10 MiB"),
duration="200 ms",
) -> dict[str, float]:
"""
Benchmark memory bandwidth
Returns
-------
out: dict
Maps sizes of outputs to measured bandwidths
"""
duration = parse_timedelta(duration)
out = {}
for size_str in sizes:
size = parse_bytes(size_str)
data = randbytes(size)
start = time()
total = 0
while time() < start + duration:
_ = data[:-1]
del _
total += size
out[size_str] = total / (time() - start)
return out
async def benchmark_network(
address: str,
rpc: ConnectionPool | Callable[[str], RPCType],
sizes: Iterable[str] = ("1 kiB", "10 kiB", "100 kiB", "1 MiB", "10 MiB", "50 MiB"),
duration="1 s",
) -> dict[str, float]:
"""
Benchmark network communications to another worker
Returns
-------
out: dict
Maps sizes of outputs to measured bandwidths
"""
duration = parse_timedelta(duration)
out = {}
async with rpc(address) as r:
for size_str in sizes:
size = parse_bytes(size_str)
data = to_serialize(randbytes(size))
start = time()
total = 0
while time() < start + duration:
await r.echo(data=data)
total += size * 2
out[size_str] = total / (time() - start)
return out
|