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
// Copyright Materialize, Inc. and contributors. All rights reserved.
//
// Use of this software is governed by the Business Source License
// included in the LICENSE file.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0.
//! A controller for a compute instance.
use std::collections::{BTreeMap, BTreeSet};
use std::fmt::Debug;
use std::num::NonZeroI64;
use std::sync::{Arc, Mutex};
use std::time::{Duration, Instant};
use mz_build_info::BuildInfo;
use mz_cluster_client::client::{ClusterStartupEpoch, TimelyConfig};
use mz_cluster_client::WallclockLagFn;
use mz_compute_types::dataflows::{BuildDesc, DataflowDescription};
use mz_compute_types::plan::flat_plan::FlatPlan;
use mz_compute_types::plan::LirId;
use mz_compute_types::sinks::{
ComputeSinkConnection, ComputeSinkDesc, ContinualTaskConnection, MaterializedViewSinkConnection,
};
use mz_compute_types::sources::SourceInstanceDesc;
use mz_compute_types::ComputeInstanceId;
use mz_controller_types::dyncfgs::WALLCLOCK_LAG_REFRESH_INTERVAL;
use mz_dyncfg::ConfigSet;
use mz_expr::RowSetFinishing;
use mz_ore::cast::CastFrom;
use mz_ore::channel::instrumented_unbounded_channel;
use mz_ore::now::NowFn;
use mz_ore::tracing::OpenTelemetryContext;
use mz_ore::{soft_assert_or_log, soft_panic_or_log};
use mz_repr::adt::interval::Interval;
use mz_repr::refresh_schedule::RefreshSchedule;
use mz_repr::{Datum, Diff, GlobalId, Row};
use mz_storage_client::controller::IntrospectionType;
use mz_storage_types::read_holds::ReadHold;
use mz_storage_types::read_policy::ReadPolicy;
use serde::Serialize;
use thiserror::Error;
use timely::progress::frontier::MutableAntichain;
use timely::progress::{Antichain, ChangeBatch, Timestamp};
use timely::PartialOrder;
use tokio::sync::{mpsc, oneshot};
use uuid::Uuid;
use crate::controller::error::{
CollectionLookupError, CollectionMissing, HydrationCheckBadTarget, ERROR_TARGET_REPLICA_FAILED,
};
use crate::controller::replica::{ReplicaClient, ReplicaConfig};
use crate::controller::{
ComputeControllerResponse, ComputeControllerTimestamp, IntrospectionUpdates, PeekNotification,
ReplicaId, StorageCollections,
};
use crate::logging::LogVariant;
use crate::metrics::IntCounter;
use crate::metrics::{InstanceMetrics, ReplicaCollectionMetrics, ReplicaMetrics, UIntGauge};
use crate::protocol::command::{
ComputeCommand, ComputeParameters, InstanceConfig, Peek, PeekTarget,
};
use crate::protocol::history::ComputeCommandHistory;
use crate::protocol::response::{
ComputeResponse, CopyToResponse, FrontiersResponse, OperatorHydrationStatus, PeekResponse,
StatusResponse, SubscribeBatch, SubscribeResponse,
};
use crate::service::{ComputeClient, ComputeGrpcClient};
#[derive(Error, Debug)]
#[error("replica exists already: {0}")]
pub(super) struct ReplicaExists(pub ReplicaId);
#[derive(Error, Debug)]
#[error("replica does not exist: {0}")]
pub(super) struct ReplicaMissing(pub ReplicaId);
#[derive(Error, Debug)]
pub(super) enum DataflowCreationError {
#[error("collection does not exist: {0}")]
CollectionMissing(GlobalId),
#[error("replica does not exist: {0}")]
ReplicaMissing(ReplicaId),
#[error("dataflow definition lacks an as_of value")]
MissingAsOf,
#[error("subscribe dataflow has an empty as_of")]
EmptyAsOfForSubscribe,
#[error("copy to dataflow has an empty as_of")]
EmptyAsOfForCopyTo,
#[error("no read hold provided for dataflow import: {0}")]
ReadHoldMissing(GlobalId),
#[error("insufficient read hold provided for dataflow import: {0}")]
ReadHoldInsufficient(GlobalId),
}
impl From<CollectionMissing> for DataflowCreationError {
fn from(error: CollectionMissing) -> Self {
Self::CollectionMissing(error.0)
}
}
#[derive(Error, Debug)]
pub(super) enum PeekError {
#[error("replica does not exist: {0}")]
ReplicaMissing(ReplicaId),
#[error("read hold ID does not match peeked collection: {0}")]
ReadHoldIdMismatch(GlobalId),
#[error("insufficient read hold provided: {0}")]
ReadHoldInsufficient(GlobalId),
}
#[derive(Error, Debug)]
pub(super) enum ReadPolicyError {
#[error("collection does not exist: {0}")]
CollectionMissing(GlobalId),
#[error("collection is write-only: {0}")]
WriteOnlyCollection(GlobalId),
}
impl From<CollectionMissing> for ReadPolicyError {
fn from(error: CollectionMissing) -> Self {
Self::CollectionMissing(error.0)
}
}
/// A command sent to an [`Instance`] task.
pub type Command<T> = Box<dyn FnOnce(&mut Instance<T>) + Send>;
/// A client for an [`Instance`] task.
#[derive(Debug)]
pub(super) struct Client<T: ComputeControllerTimestamp> {
/// A sender for commands for the instance.
command_tx: mpsc::UnboundedSender<Command<T>>,
}
impl<T: ComputeControllerTimestamp> Client<T> {
pub fn send(&self, command: Command<T>) {
self.command_tx.send(command).expect("instance not dropped");
}
}
impl<T> Client<T>
where
T: ComputeControllerTimestamp,
ComputeGrpcClient: ComputeClient<T>,
{
pub fn spawn(
id: ComputeInstanceId,
build_info: &'static BuildInfo,
storage: StorageCollections<T>,
arranged_logs: Vec<(LogVariant, GlobalId, SharedCollectionState<T>)>,
envd_epoch: NonZeroI64,
metrics: InstanceMetrics,
now: NowFn,
wallclock_lag: WallclockLagFn<T>,
dyncfg: Arc<ConfigSet>,
response_tx: mpsc::UnboundedSender<ComputeControllerResponse<T>>,
introspection_tx: crossbeam_channel::Sender<IntrospectionUpdates>,
read_holds_tx: mpsc::UnboundedSender<(GlobalId, ChangeBatch<T>)>,
read_holds_rx: mpsc::UnboundedReceiver<(GlobalId, ChangeBatch<T>)>,
) -> Self {
let (command_tx, command_rx) = mpsc::unbounded_channel();
mz_ore::task::spawn(
|| format!("compute-instance-{id}"),
Instance::new(
build_info,
storage,
arranged_logs,
envd_epoch,
metrics,
now,
wallclock_lag,
dyncfg,
command_rx,
response_tx,
introspection_tx,
read_holds_tx,
read_holds_rx,
)
.run(),
);
Self { command_tx }
}
}
/// A response from a replica, composed of a replica ID, the replica's current epoch, and the
/// compute response itself.
pub(super) type ReplicaResponse<T> = (ReplicaId, u64, ComputeResponse<T>);
/// The state we keep for a compute instance.
pub(super) struct Instance<T: ComputeControllerTimestamp> {
/// Build info for spawning replicas
build_info: &'static BuildInfo,
/// A handle providing access to storage collections.
storage_collections: StorageCollections<T>,
/// Whether instance initialization has been completed.
initialized: bool,
/// Whether or not this instance is in read-only mode.
///
/// When in read-only mode, neither the controller nor the instances
/// controlled by it are allowed to affect changes to external systems
/// (largely persist).
read_only: bool,
/// The replicas of this compute instance.
replicas: BTreeMap<ReplicaId, ReplicaState<T>>,
/// Currently installed compute collections.
///
/// New entries are added for all collections exported from dataflows created through
/// [`Instance::create_dataflow`].
///
/// Entries are removed by [`Instance::cleanup_collections`]. See that method's documentation
/// about the conditions for removing collection state.
collections: BTreeMap<GlobalId, CollectionState<T>>,
/// IDs of log sources maintained by this compute instance.
log_sources: BTreeMap<LogVariant, GlobalId>,
/// Currently outstanding peeks.
///
/// New entries are added for all peeks initiated through [`Instance::peek`].
///
/// The entry for a peek is only removed once all replicas have responded to the peek. This is
/// currently required to ensure all replicas have stopped reading from the peeked collection's
/// inputs before we allow them to compact. database-issues#4822 tracks changing this so we only have to wait
/// for the first peek response.
peeks: BTreeMap<Uuid, PendingPeek<T>>,
/// Currently in-progress subscribes.
///
/// New entries are added for all subscribes exported from dataflows created through
/// [`Instance::create_dataflow`].
///
/// The entry for a subscribe is removed once at least one replica has reported the subscribe
/// to have advanced to the empty frontier or to have been dropped, implying that no further
/// updates will be emitted for this subscribe.
///
/// Note that subscribes are tracked both in `collections` and `subscribes`. `collections`
/// keeps track of the subscribe's upper and since frontiers and ensures appropriate read holds
/// on the subscribe's input. `subscribes` is only used to track which updates have been
/// emitted, to decide if new ones should be emitted or suppressed.
subscribes: BTreeMap<GlobalId, ActiveSubscribe<T>>,
/// Tracks all in-progress COPY TOs.
///
/// New entries are added for all s3 oneshot sinks (corresponding to a COPY TO) exported from
/// dataflows created through [`Instance::create_dataflow`].
///
/// The entry for a copy to is removed once at least one replica has finished
/// or the exporting collection is dropped.
copy_tos: BTreeSet<GlobalId>,
/// The command history, used when introducing new replicas or restarting existing replicas.
history: ComputeCommandHistory<UIntGauge, T>,
/// Receiver for commands to be executed.
command_rx: mpsc::UnboundedReceiver<Command<T>>,
/// Sender for responses to be delivered.
response_tx: mpsc::UnboundedSender<ComputeControllerResponse<T>>,
/// Sender for introspection updates to be recorded.
introspection_tx: crossbeam_channel::Sender<IntrospectionUpdates>,
/// A number that increases with each restart of `environmentd`.
envd_epoch: NonZeroI64,
/// Numbers that increase with each restart of a replica.
replica_epochs: BTreeMap<ReplicaId, u64>,
/// The registry the controller uses to report metrics.
metrics: InstanceMetrics,
/// Dynamic system configuration.
dyncfg: Arc<ConfigSet>,
/// A function that produces the current wallclock time.
now: NowFn,
/// A function that computes the lag between the given time and wallclock time.
wallclock_lag: WallclockLagFn<T>,
/// The last time wallclock lag introspection was refreshed.
wallclock_lag_last_refresh: Instant,
/// Sender for updates to collection read holds.
///
/// Copies of this sender are given to [`ReadHold`]s that are created in
/// [`CollectionState::new`].
read_holds_tx: mpsc::UnboundedSender<(GlobalId, ChangeBatch<T>)>,
/// Receiver for updates to collection read holds.
///
/// Received updates are applied by [`Instance::apply_read_hold_changes`].
read_holds_rx: mpsc::UnboundedReceiver<(GlobalId, ChangeBatch<T>)>,
/// Stashed read hold changes.
///
/// Used by [`Instance::apply_read_hold_changes`] to stash read hold changes that cannot be
/// applied immediately until they can be applied.
stashed_read_hold_changes: BTreeMap<GlobalId, ChangeBatch<T>>,
/// A sender for responses from replicas.
replica_tx: mz_ore::channel::InstrumentedUnboundedSender<ReplicaResponse<T>, IntCounter>,
/// A receiver for responses from replicas.
replica_rx: mz_ore::channel::InstrumentedUnboundedReceiver<ReplicaResponse<T>, IntCounter>,
}
impl<T: ComputeControllerTimestamp> Instance<T> {
/// Acquire a handle to the collection state associated with `id`.
fn collection(&self, id: GlobalId) -> Result<&CollectionState<T>, CollectionMissing> {
self.collections.get(&id).ok_or(CollectionMissing(id))
}
/// Acquire a mutable handle to the collection state associated with `id`.
fn collection_mut(
&mut self,
id: GlobalId,
) -> Result<&mut CollectionState<T>, CollectionMissing> {
self.collections.get_mut(&id).ok_or(CollectionMissing(id))
}
/// Acquire a handle to the collection state associated with `id`.
///
/// # Panics
///
/// Panics if the identified collection does not exist.
fn expect_collection(&self, id: GlobalId) -> &CollectionState<T> {
self.collections.get(&id).expect("collection must exist")
}
/// Acquire a mutable handle to the collection state associated with `id`.
///
/// # Panics
///
/// Panics if the identified collection does not exist.
fn expect_collection_mut(&mut self, id: GlobalId) -> &mut CollectionState<T> {
self.collections
.get_mut(&id)
.expect("collection must exist")
}
fn collections_iter(&self) -> impl Iterator<Item = (GlobalId, &CollectionState<T>)> {
self.collections.iter().map(|(id, coll)| (*id, coll))
}
/// Add a collection to the instance state.
///
/// # Panics
///
/// Panics if a collection with the same ID exists already.
fn add_collection(
&mut self,
id: GlobalId,
as_of: Antichain<T>,
shared: SharedCollectionState<T>,
storage_dependencies: BTreeMap<GlobalId, ReadHold<T>>,
compute_dependencies: BTreeMap<GlobalId, ReadHold<T>>,
replica_input_read_holds: Vec<ReadHold<T>>,
write_only: bool,
storage_sink: bool,
initial_as_of: Option<Antichain<T>>,
refresh_schedule: Option<RefreshSchedule>,
) {
// Add global collection state.
let introspection = CollectionIntrospection::new(
id,
self.introspection_tx.clone(),
as_of.clone(),
storage_sink,
initial_as_of,
refresh_schedule,
);
let mut state = CollectionState::new(
id,
as_of.clone(),
shared,
storage_dependencies,
compute_dependencies,
self.read_holds_tx.clone(),
introspection,
);
// If the collection is write-only, clear its read policy to reflect that.
if write_only {
state.read_policy = None;
}
if let Some(previous) = self.collections.insert(id, state) {
panic!("attempt to add a collection with existing ID {id} (previous={previous:?}");
}
// Add per-replica collection state.
for replica in self.replicas.values_mut() {
replica.add_collection(id, as_of.clone(), replica_input_read_holds.clone());
}
// Update introspection.
self.report_dependency_updates(id, 1);
}
fn remove_collection(&mut self, id: GlobalId) {
// Update introspection.
self.report_dependency_updates(id, -1);
// Remove per-replica collection state.
for replica in self.replicas.values_mut() {
replica.remove_collection(id);
}
// Remove global collection state.
self.collections.remove(&id);
}
fn add_replica_state(
&mut self,
id: ReplicaId,
client: ReplicaClient<T>,
config: ReplicaConfig,
epoch: ClusterStartupEpoch,
) {
let log_ids: BTreeSet<_> = config.logging.index_logs.values().copied().collect();
let metrics = self.metrics.for_replica(id);
let mut replica = ReplicaState::new(
id,
client,
config,
metrics,
self.introspection_tx.clone(),
epoch,
);
// Add per-replica collection state.
for (collection_id, collection) in &self.collections {
// Skip log collections not maintained by this replica.
if collection.log_collection && !log_ids.contains(collection_id) {
continue;
}
let as_of = collection.read_frontier().to_owned();
let input_read_holds = collection.storage_dependencies.values().cloned().collect();
replica.add_collection(*collection_id, as_of, input_read_holds);
}
self.replicas.insert(id, replica);
}
/// Enqueue the given response for delivery to the controller clients.
fn deliver_response(&self, response: ComputeControllerResponse<T>) {
// Failure to send means the `ComputeController` has been dropped and doesn't care about
// responses anymore.
let _ = self.response_tx.send(response);
}
/// Enqueue the given introspection updates for recording.
fn deliver_introspection_updates(&self, type_: IntrospectionType, updates: Vec<(Row, Diff)>) {
// Failure to send means the `ComputeController` has been dropped and doesn't care about
// introspection updates anymore.
let _ = self.introspection_tx.send((type_, updates));
}
/// Returns whether the identified replica exists.
fn replica_exists(&self, id: ReplicaId) -> bool {
self.replicas.contains_key(&id)
}
/// Return the IDs of pending peeks targeting the specified replica.
fn peeks_targeting(
&self,
replica_id: ReplicaId,
) -> impl Iterator<Item = (Uuid, &PendingPeek<T>)> {
self.peeks.iter().filter_map(move |(uuid, peek)| {
if peek.target_replica == Some(replica_id) {
Some((*uuid, peek))
} else {
None
}
})
}
/// Return the IDs of in-progress subscribes targeting the specified replica.
fn subscribes_targeting(&self, replica_id: ReplicaId) -> impl Iterator<Item = GlobalId> + '_ {
self.subscribes.iter().filter_map(move |(id, subscribe)| {
let targeting = subscribe.target_replica == Some(replica_id);
targeting.then_some(*id)
})
}
/// Update introspection with the current collection frontiers.
///
/// We could also do this directly in response to frontier changes, but doing it periodically
/// lets us avoid emitting some introspection updates that can be consolidated (e.g. a write
/// frontier updated immediately followed by a read frontier update).
///
/// This method is invoked by `ComputeController::maintain`, which we expect to be called once
/// per second during normal operation.
fn update_frontier_introspection(&mut self) {
for collection in self.collections.values_mut() {
collection
.introspection
.observe_frontiers(&collection.read_frontier(), &collection.write_frontier());
}
for replica in self.replicas.values_mut() {
for collection in replica.collections.values_mut() {
collection
.introspection
.observe_frontier(&collection.write_frontier);
}
}
}
/// Refresh the controller state metrics for this instance.
///
/// We could also do state metric updates directly in response to state changes, but that would
/// mean littering the code with metric update calls. Encapsulating state metric maintenance in
/// a single method is less noisy.
///
/// This method is invoked by `ComputeController::maintain`, which we expect to be called once
/// per second during normal operation.
fn refresh_state_metrics(&self) {
let unscheduled_collections_count =
self.collections.values().filter(|c| !c.scheduled).count();
self.metrics
.replica_count
.set(u64::cast_from(self.replicas.len()));
self.metrics
.collection_count
.set(u64::cast_from(self.collections.len()));
self.metrics
.collection_unscheduled_count
.set(u64::cast_from(unscheduled_collections_count));
self.metrics
.peek_count
.set(u64::cast_from(self.peeks.len()));
self.metrics
.subscribe_count
.set(u64::cast_from(self.subscribes.len()));
self.metrics
.copy_to_count
.set(u64::cast_from(self.copy_tos.len()));
}
/// Refresh the `WallclockLagHistory` introspection and the `wallclock_lag_*_seconds` metrics
/// with the current lag values.
///
/// This method is invoked by `ComputeController::maintain`, which we expect to be called once
/// per second during normal operation.
fn refresh_wallclock_lag(&mut self) {
let refresh_introspection = !self.read_only
&& self.wallclock_lag_last_refresh.elapsed()
>= WALLCLOCK_LAG_REFRESH_INTERVAL.get(&self.dyncfg);
let mut introspection_updates = refresh_introspection.then(Vec::new);
let now = mz_ore::now::to_datetime((self.now)());
let now_tz = now.try_into().expect("must fit");
for (replica_id, replica) in &mut self.replicas {
for (collection_id, collection) in &mut replica.collections {
let lag = match collection.write_frontier.as_option() {
Some(ts) => (self.wallclock_lag)(ts),
None => Duration::ZERO,
};
collection.wallclock_lag_max = std::cmp::max(collection.wallclock_lag_max, lag);
if let Some(updates) = &mut introspection_updates {
let max_lag = std::mem::take(&mut collection.wallclock_lag_max);
let max_lag_us = i64::try_from(max_lag.as_micros()).expect("must fit");
let row = Row::pack_slice(&[
Datum::String(&collection_id.to_string()),
Datum::String(&replica_id.to_string()),
Datum::Interval(Interval::new(0, 0, max_lag_us)),
Datum::TimestampTz(now_tz),
]);
updates.push((row, 1));
}
if let Some(metrics) = &mut collection.metrics {
metrics.observe_wallclock_lag(lag);
};
}
}
if let Some(updates) = introspection_updates {
self.deliver_introspection_updates(IntrospectionType::WallclockLagHistory, updates);
self.wallclock_lag_last_refresh = Instant::now();
}
}
/// Report updates (inserts or retractions) to the identified collection's dependencies.
///
/// # Panics
///
/// Panics if the identified collection does not exist.
fn report_dependency_updates(&self, id: GlobalId, diff: i64) {
let collection = self.expect_collection(id);
let dependencies = collection.dependency_ids();
let updates = dependencies
.map(|dependency_id| {
let row = Row::pack_slice(&[
Datum::String(&id.to_string()),
Datum::String(&dependency_id.to_string()),
]);
(row, diff)
})
.collect();
self.deliver_introspection_updates(IntrospectionType::ComputeDependencies, updates);
}
/// Update the tracked hydration status for an operator according to a received status update.
fn update_operator_hydration_status(
&mut self,
replica_id: ReplicaId,
status: OperatorHydrationStatus,
) {
let Some(replica) = self.replicas.get_mut(&replica_id) else {
tracing::error!(
%replica_id, ?status,
"status update for an unknown replica"
);
return;
};
let Some(collection) = replica.collections.get_mut(&status.collection_id) else {
tracing::error!(
%replica_id, ?status,
"status update for an unknown collection"
);
return;
};
collection.introspection.operator_hydrated(
status.lir_id,
status.worker_id,
status.hydrated,
);
}
/// Returns `true` if the given collection is hydrated on at least one
/// replica.
///
/// This also returns `true` in case this cluster does not have any
/// replicas.
#[mz_ore::instrument(level = "debug")]
pub fn collection_hydrated(
&self,
collection_id: GlobalId,
) -> Result<bool, CollectionLookupError> {
if self.replicas.is_empty() {
return Ok(true);
}
for replica_state in self.replicas.values() {
let collection_state = replica_state
.collections
.get(&collection_id)
.ok_or(CollectionLookupError::CollectionMissing(collection_id))?;
if collection_state.hydrated() {
return Ok(true);
}
}
Ok(false)
}
/// Returns `true` if each non-transient, non-excluded collection is hydrated on at
/// least one replica.
///
/// This also returns `true` in case this cluster does not have any
/// replicas.
#[mz_ore::instrument(level = "debug")]
pub fn collections_hydrated_on_replicas(
&self,
target_replica_ids: Option<Vec<ReplicaId>>,
exclude_collections: &BTreeSet<GlobalId>,
) -> Result<bool, HydrationCheckBadTarget> {
if self.replicas.is_empty() {
return Ok(true);
}
let mut all_hydrated = true;
let target_replicas: BTreeSet<ReplicaId> = self
.replicas
.keys()
.filter_map(|id| match target_replica_ids {
None => Some(id.clone()),
Some(ref ids) if ids.contains(id) => Some(id.clone()),
Some(_) => None,
})
.collect();
if let Some(targets) = target_replica_ids {
if target_replicas.is_empty() {
return Err(HydrationCheckBadTarget(targets));
}
}
for (id, _collection) in self.collections_iter() {
if id.is_transient() || exclude_collections.contains(&id) {
continue;
}
let mut collection_hydrated = false;
for replica_state in self.replicas.values() {
if !target_replicas.contains(&replica_state.id) {
continue;
}
let collection_state = replica_state
.collections
.get(&id)
.expect("missing collection state");
if collection_state.hydrated() {
collection_hydrated = true;
break;
}
}
if !collection_hydrated {
tracing::info!("collection {id} is not hydrated on any replica");
all_hydrated = false;
// We continue with our loop instead of breaking out early, so
// that we log all non-hydrated replicas.
}
}
Ok(all_hydrated)
}
/// Returns `true` if all non-transient, non-excluded collections are hydrated on at least one
/// replica.
///
/// This also returns `true` in case this cluster does not have any
/// replicas.
#[mz_ore::instrument(level = "debug")]
pub fn collections_hydrated(&self, exclude_collections: &BTreeSet<GlobalId>) -> bool {
self.collections_hydrated_on_replicas(None, exclude_collections)
.expect("Cannot error if target_replica_ids is None")
}
/// Clean up collection state that is not needed anymore.
///
/// Three conditions need to be true before we can remove state for a collection:
///
/// 1. A client must have explicitly dropped the collection. If that is not the case, clients
/// can still reasonably assume that the controller knows about the collection and can
/// answer queries about it.
/// 2. There must be no outstanding read capabilities on the collection. As long as someone
/// still holds read capabilities on a collection, we need to keep it around to be able
/// to properly handle downgrading of said capabilities.
/// 3. All replica frontiers for the collection must have advanced to the empty frontier.
/// Advancement to the empty frontiers signals that replicas are done computing the
/// collection and that they won't send more `ComputeResponse`s for it. As long as we might
/// receive responses for a collection we want to keep it around to be able to validate and
/// handle these responses.
fn cleanup_collections(&mut self) {
let to_remove: Vec<_> = self
.collections_iter()
.filter(|(id, collection)| {
collection.dropped
&& collection.shared.lock_read_capabilities(|c| c.is_empty())
&& self
.replicas
.values()
.all(|r| r.collection_frontiers_empty(*id))
})
.map(|(id, _collection)| id)
.collect();
for id in to_remove {
self.remove_collection(id);
}
}
/// Returns the state of the [`Instance`] formatted as JSON.
///
/// The returned value is not guaranteed to be stable and may change at any point in time.
#[mz_ore::instrument(level = "debug")]
pub fn dump(&self) -> Result<serde_json::Value, anyhow::Error> {
// Note: We purposefully use the `Debug` formatting for the value of all fields in the
// returned object as a tradeoff between usability and stability. `serde_json` will fail
// to serialize an object if the keys aren't strings, so `Debug` formatting the values
// prevents a future unrelated change from silently breaking this method.
// Destructure `self` here so we don't forget to consider dumping newly added fields.
let Self {
build_info: _,
storage_collections: _,
initialized,
read_only,
replicas,
collections,
log_sources: _,
peeks,
subscribes,
copy_tos,
history: _,
command_rx: _,
response_tx: _,
introspection_tx: _,
envd_epoch,
replica_epochs,
metrics: _,
dyncfg: _,
now: _,
wallclock_lag: _,
wallclock_lag_last_refresh,
read_holds_tx: _,
read_holds_rx: _,
stashed_read_hold_changes,
replica_tx: _,
replica_rx: _,
} = self;
fn field(
key: &str,
value: impl Serialize,
) -> Result<(String, serde_json::Value), anyhow::Error> {
let value = serde_json::to_value(value)?;
Ok((key.to_string(), value))
}
let replicas: BTreeMap<_, _> = replicas
.iter()
.map(|(id, replica)| Ok((id.to_string(), replica.dump()?)))
.collect::<Result<_, anyhow::Error>>()?;
let collections: BTreeMap<_, _> = collections
.iter()
.map(|(id, collection)| (id.to_string(), format!("{collection:?}")))
.collect();
let peeks: BTreeMap<_, _> = peeks
.iter()
.map(|(uuid, peek)| (uuid.to_string(), format!("{peek:?}")))
.collect();
let subscribes: BTreeMap<_, _> = subscribes
.iter()
.map(|(id, subscribe)| (id.to_string(), format!("{subscribe:?}")))
.collect();
let copy_tos: Vec<_> = copy_tos.iter().map(|id| id.to_string()).collect();
let replica_epochs: BTreeMap<_, _> = replica_epochs
.iter()
.map(|(id, epoch)| (id.to_string(), epoch))
.collect();
let wallclock_lag_last_refresh = format!("{wallclock_lag_last_refresh:?}");
let stashed_read_hold_changes: BTreeMap<_, _> = stashed_read_hold_changes
.iter()
.map(|(id, changes)| (id.to_string(), changes))
.collect();
let map = serde_json::Map::from_iter([
field("initialized", initialized)?,
field("read_only", read_only)?,
field("replicas", replicas)?,
field("collections", collections)?,
field("peeks", peeks)?,
field("subscribes", subscribes)?,
field("copy_tos", copy_tos)?,
field("envd_epoch", envd_epoch)?,
field("replica_epochs", replica_epochs)?,
field("wallclock_lag_last_refresh", wallclock_lag_last_refresh)?,
field("stashed_read_hold_changes", stashed_read_hold_changes)?,
]);
Ok(serde_json::Value::Object(map))
}
}
impl<T> Instance<T>
where
T: ComputeControllerTimestamp,
ComputeGrpcClient: ComputeClient<T>,
{
fn new(
build_info: &'static BuildInfo,
storage: StorageCollections<T>,
arranged_logs: Vec<(LogVariant, GlobalId, SharedCollectionState<T>)>,
envd_epoch: NonZeroI64,
metrics: InstanceMetrics,
now: NowFn,
wallclock_lag: WallclockLagFn<T>,
dyncfg: Arc<ConfigSet>,
command_rx: mpsc::UnboundedReceiver<Command<T>>,
response_tx: mpsc::UnboundedSender<ComputeControllerResponse<T>>,
introspection_tx: crossbeam_channel::Sender<IntrospectionUpdates>,
read_holds_tx: mpsc::UnboundedSender<(GlobalId, ChangeBatch<T>)>,
read_holds_rx: mpsc::UnboundedReceiver<(GlobalId, ChangeBatch<T>)>,
) -> Self {
let mut collections = BTreeMap::new();
let mut log_sources = BTreeMap::new();
for (log, id, shared) in arranged_logs {
let collection = CollectionState::new_log_collection(
id,
shared,
read_holds_tx.clone(),
introspection_tx.clone(),
);
collections.insert(id, collection);
log_sources.insert(log, id);
}
let history = ComputeCommandHistory::new(metrics.for_history());
let send_count = metrics.response_send_count.clone();
let recv_count = metrics.response_recv_count.clone();
let (replica_tx, replica_rx) = instrumented_unbounded_channel(send_count, recv_count);
Self {
build_info,
storage_collections: storage,
initialized: false,
read_only: true,
replicas: Default::default(),
collections,
log_sources,
peeks: Default::default(),
subscribes: Default::default(),
copy_tos: Default::default(),
history,
command_rx,
response_tx,
introspection_tx,
envd_epoch,
replica_epochs: Default::default(),
metrics,
dyncfg,
now,
wallclock_lag,
wallclock_lag_last_refresh: Instant::now(),
read_holds_tx,
read_holds_rx,
stashed_read_hold_changes: Default::default(),
replica_tx,
replica_rx,
}
}
async fn run(mut self) {
self.send(ComputeCommand::CreateTimely {
config: TimelyConfig::default(),
epoch: ClusterStartupEpoch::new(self.envd_epoch, 0),
});
// Send a placeholder instance configuration for the replica task to fill in.
let dummy_logging_config = Default::default();
self.send(ComputeCommand::CreateInstance(InstanceConfig {
logging: dummy_logging_config,
expiration_offset: None,
}));
loop {
tokio::select! {
command = self.command_rx.recv() => match command {
Some(cmd) => cmd(&mut self),
None => break,
},
response = self.replica_rx.recv() => match response {
Some(response) => self.handle_response(response),
None => unreachable!("self owns a sender side of the channel"),
}
}
}
}
/// Update instance configuration.
#[mz_ore::instrument(level = "debug")]
pub fn update_configuration(&mut self, config_params: ComputeParameters) {
self.send(ComputeCommand::UpdateConfiguration(config_params));
}
/// Marks the end of any initialization commands.
///
/// Intended to be called by `Controller`, rather than by other code.
/// Calling this method repeatedly has no effect.
#[mz_ore::instrument(level = "debug")]
pub fn initialization_complete(&mut self) {
// The compute protocol requires that `InitializationComplete` is sent only once.
if !self.initialized {
self.send(ComputeCommand::InitializationComplete);
self.initialized = true;
}
}
/// Allows this instance to affect writes to external systems (persist).
///
/// Calling this method repeatedly has no effect.
#[mz_ore::instrument(level = "debug")]
pub fn allow_writes(&mut self) {
if self.read_only {
self.read_only = false;
self.send(ComputeCommand::AllowWrites);
}
}
/// Check that the current instance is empty.
///
/// This method exists to help us find bugs where the client drops a compute instance that
/// still has replicas or collections installed, and later assumes that said
/// replicas/collections still exists.
///
/// # Panics
///
/// Panics if the compute instance still has active replicas.
/// Panics if the compute instance still has collections installed.
#[mz_ore::instrument(level = "debug")]
pub fn check_empty(&mut self) {
// Collections might have been dropped but not cleaned up yet.
self.apply_read_hold_changes();
self.cleanup_collections();
let stray_replicas: Vec<_> = self.replicas.keys().collect();
soft_assert_or_log!(
stray_replicas.is_empty(),
"dropped instance still has provisioned replicas: {stray_replicas:?}",
);
let collections = self.collections.iter();
let stray_collections: Vec<_> = collections
.filter(|(_, c)| !c.log_collection)
.map(|(id, _)| id)
.collect();
soft_assert_or_log!(
stray_collections.is_empty(),
"dropped instance still has installed collections: {stray_collections:?}",
);
}
/// Sends a command to all replicas of this instance.
#[mz_ore::instrument(level = "debug")]
fn send(&mut self, cmd: ComputeCommand<T>) {
// Record the command so that new replicas can be brought up to speed.
self.history.push(cmd.clone());
// Clone the command for each active replica.
for replica in self.replicas.values_mut() {
// Swallow error, we'll notice because the replica task has stopped.
let _ = replica.client.send(cmd.clone());
}
}
/// Add a new instance replica, by ID.
#[mz_ore::instrument(level = "debug")]
pub fn add_replica(
&mut self,
id: ReplicaId,
mut config: ReplicaConfig,
) -> Result<(), ReplicaExists> {
if self.replica_exists(id) {
return Err(ReplicaExists(id));
}
config.logging.index_logs = self.log_sources.clone();
let replica_epoch = self.replica_epochs.entry(id).or_default();
*replica_epoch += 1;
let metrics = self.metrics.for_replica(id);
let epoch = ClusterStartupEpoch::new(self.envd_epoch, *replica_epoch);
let client = ReplicaClient::spawn(
id,
self.build_info,
config.clone(),
epoch,
metrics.clone(),
Arc::clone(&self.dyncfg),
self.replica_tx.clone(),
);
// Take this opportunity to clean up the history we should present.
self.history.reduce();
// Replay the commands at the client, creating new dataflow identifiers.
for command in self.history.iter() {
if client.send(command.clone()).is_err() {
// We swallow the error here. On the next send, we will fail again, and
// restart the connection as well as this rehydration.
tracing::warn!("Replica {:?} connection terminated during hydration", id);
break;
}
}
// Add replica to tracked state.
self.add_replica_state(id, client, config, epoch);
Ok(())
}
/// Remove an existing instance replica, by ID.
#[mz_ore::instrument(level = "debug")]
pub fn remove_replica(&mut self, id: ReplicaId) -> Result<(), ReplicaMissing> {
self.replicas.remove(&id).ok_or(ReplicaMissing(id))?;
// Subscribes targeting this replica either won't be served anymore (if the replica is
// dropped) or might produce inconsistent output (if the target collection is an
// introspection index). We produce an error to inform upstream.
let to_drop: Vec<_> = self.subscribes_targeting(id).collect();
for subscribe_id in to_drop {
let subscribe = self.subscribes.remove(&subscribe_id).unwrap();
let response = ComputeControllerResponse::SubscribeResponse(
subscribe_id,
SubscribeBatch {
lower: subscribe.frontier.clone(),
upper: subscribe.frontier,
updates: Err(ERROR_TARGET_REPLICA_FAILED.into()),
},
);
self.deliver_response(response);
}
// Peeks targeting this replica might not be served anymore (if the replica is dropped).
// If the replica has failed it might come back and respond to the peek later, but it still
// seems like a good idea to cancel the peek to inform the caller about the failure. This
// is consistent with how we handle targeted subscribes above.
let mut peek_responses = Vec::new();
let mut to_drop = Vec::new();
for (uuid, peek) in self.peeks_targeting(id) {
peek_responses.push(ComputeControllerResponse::PeekNotification(
uuid,
PeekNotification::Error(ERROR_TARGET_REPLICA_FAILED.into()),
peek.otel_ctx.clone(),
));
to_drop.push(uuid);
}
for response in peek_responses {
self.deliver_response(response);
}
for uuid in to_drop {
let response = PeekResponse::Error(ERROR_TARGET_REPLICA_FAILED.into());
self.finish_peek(uuid, response);
}
Ok(())
}
/// Rehydrate the given instance replica.
///
/// # Panics
///
/// Panics if the specified replica does not exist.
fn rehydrate_replica(&mut self, id: ReplicaId) {
let config = self.replicas[&id].config.clone();
self.remove_replica(id).expect("replica must exist");
let result = self.add_replica(id, config);
match result {
Ok(()) => (),
Err(ReplicaExists(_)) => unreachable!("replica was removed"),
}
}
/// Rehydrate any failed replicas of this instance.
fn rehydrate_failed_replicas(&mut self) {
let replicas = self.replicas.iter();
let failed_replicas: Vec<_> = replicas
.filter_map(|(id, replica)| replica.client.is_failed().then_some(*id))
.collect();
for replica_id in failed_replicas {
self.rehydrate_replica(replica_id);
}
}
/// Creates the described dataflow and initializes state for its output.
///
/// This method expects a `DataflowDescription` with an `as_of` frontier specified, as well as
/// for each imported collection a read hold in `import_read_holds` at at least the `as_of`.
///
/// If a `subscribe_target_replica` is given, any subscribes exported by the dataflow are
/// configured to target that replica, i.e., only subscribe responses sent by that replica are
/// considered.
#[mz_ore::instrument(level = "debug")]
pub fn create_dataflow(
&mut self,
dataflow: DataflowDescription<mz_compute_types::plan::Plan<T>, (), T>,
import_read_holds: Vec<ReadHold<T>>,
subscribe_target_replica: Option<ReplicaId>,
mut shared_collection_state: BTreeMap<GlobalId, SharedCollectionState<T>>,
) -> Result<(), DataflowCreationError> {
use DataflowCreationError::*;
if let Some(replica_id) = subscribe_target_replica {
if !self.replica_exists(replica_id) {
return Err(ReplicaMissing(replica_id));
}
}
// Simple sanity checks around `as_of`
let as_of = dataflow.as_of.as_ref().ok_or(MissingAsOf)?;
if as_of.is_empty() && dataflow.subscribe_ids().next().is_some() {
return Err(EmptyAsOfForSubscribe);
}
if as_of.is_empty() && dataflow.copy_to_ids().next().is_some() {
return Err(EmptyAsOfForCopyTo);
}
// Collect all dependencies of the dataflow, and read holds on them at the `as_of`.
let mut storage_dependencies = BTreeMap::new();
let mut compute_dependencies = BTreeMap::new();
// When we install per-replica input read holds, we cannot use the `as_of` because of
// reconciliation: Existing slow replicas might be reading from the inputs at times before
// the `as_of` and we would rather not crash them by allowing their inputs to compact too
// far. So instead we take read holds at the least time available.
let mut replica_input_read_holds = Vec::new();
let mut import_read_holds: BTreeMap<_, _> =
import_read_holds.into_iter().map(|r| (r.id(), r)).collect();
for &id in dataflow.source_imports.keys() {
let mut read_hold = import_read_holds.remove(&id).ok_or(ReadHoldMissing(id))?;
replica_input_read_holds.push(read_hold.clone());
read_hold
.try_downgrade(as_of.clone())
.map_err(|_| ReadHoldInsufficient(id))?;
storage_dependencies.insert(id, read_hold);
}
for &id in dataflow.index_imports.keys() {
let mut read_hold = import_read_holds.remove(&id).ok_or(ReadHoldMissing(id))?;
read_hold
.try_downgrade(as_of.clone())
.map_err(|_| ReadHoldInsufficient(id))?;
compute_dependencies.insert(id, read_hold);
}
// If the `as_of` is empty, we are not going to create a dataflow, so replicas won't read
// from the inputs.
if as_of.is_empty() {
replica_input_read_holds = Default::default();
}
// Install collection state for each of the exports.
for export_id in dataflow.export_ids() {
let shared = shared_collection_state
.remove(&export_id)
.unwrap_or_else(|| SharedCollectionState::new(as_of.clone()));
let write_only = dataflow.sink_exports.contains_key(&export_id);
let storage_sink = dataflow.persist_sink_ids().any(|id| id == export_id);
self.add_collection(
export_id,
as_of.clone(),
shared,
storage_dependencies.clone(),
compute_dependencies.clone(),
replica_input_read_holds.clone(),
write_only,
storage_sink,
dataflow.initial_storage_as_of.clone(),
dataflow.refresh_schedule.clone(),
);
// If the export is a storage sink, we can advance its write frontier to the write
// frontier of the target storage collection.
if let Ok(frontiers) = self.storage_collections.collection_frontiers(export_id) {
self.maybe_update_global_write_frontier(export_id, frontiers.write_frontier);
}
}
// Initialize tracking of subscribes.
for subscribe_id in dataflow.subscribe_ids() {
self.subscribes
.insert(subscribe_id, ActiveSubscribe::new(subscribe_target_replica));
}
// Initialize tracking of copy tos.
for copy_to_id in dataflow.copy_to_ids() {
self.copy_tos.insert(copy_to_id);
}
// Here we augment all imported sources and all exported sinks with the appropriate
// storage metadata needed by the compute instance.
let mut source_imports = BTreeMap::new();
for (id, (si, monotonic)) in dataflow.source_imports {
let collection_metadata = self
.storage_collections
.collection_metadata(id)
.expect("we have a read hold on this collection");
let desc = SourceInstanceDesc {
storage_metadata: collection_metadata.clone(),
arguments: si.arguments,
typ: si.typ.clone(),
};
source_imports.insert(id, (desc, monotonic));
}
let mut sink_exports = BTreeMap::new();
for (id, se) in dataflow.sink_exports {
let connection = match se.connection {
ComputeSinkConnection::MaterializedView(conn) => {
let metadata = self
.storage_collections
.collection_metadata(id)
.map_err(|_| CollectionMissing(id))?
.clone();
let conn = MaterializedViewSinkConnection {
value_desc: conn.value_desc,
storage_metadata: metadata,
};
ComputeSinkConnection::MaterializedView(conn)
}
ComputeSinkConnection::ContinualTask(conn) => {
let metadata = self
.storage_collections
.collection_metadata(id)
.map_err(|_| DataflowCreationError::CollectionMissing(id))?
.clone();
let conn = ContinualTaskConnection {
input_id: conn.input_id,
storage_metadata: metadata,
};
ComputeSinkConnection::ContinualTask(conn)
}
ComputeSinkConnection::Subscribe(conn) => ComputeSinkConnection::Subscribe(conn),
ComputeSinkConnection::CopyToS3Oneshot(conn) => {
ComputeSinkConnection::CopyToS3Oneshot(conn)
}
};
let desc = ComputeSinkDesc {
from: se.from,
from_desc: se.from_desc,
connection,
with_snapshot: se.with_snapshot,
up_to: se.up_to,
non_null_assertions: se.non_null_assertions,
refresh_schedule: se.refresh_schedule,
};
sink_exports.insert(id, desc);
}
// Flatten the dataflow plans into the representation expected by replicas.
let objects_to_build = dataflow
.objects_to_build
.into_iter()
.map(|object| BuildDesc {
id: object.id,
plan: FlatPlan::from(object.plan),
})
.collect();
let augmented_dataflow = DataflowDescription {
source_imports,
sink_exports,
objects_to_build,
// The rest of the fields are identical
index_imports: dataflow.index_imports,
index_exports: dataflow.index_exports,
as_of: dataflow.as_of.clone(),
until: dataflow.until,
initial_storage_as_of: dataflow.initial_storage_as_of,
refresh_schedule: dataflow.refresh_schedule,
debug_name: dataflow.debug_name,
time_dependence: dataflow.time_dependence,
};
if augmented_dataflow.is_transient() {
tracing::debug!(
name = %augmented_dataflow.debug_name,
import_ids = %augmented_dataflow.display_import_ids(),
export_ids = %augmented_dataflow.display_export_ids(),
as_of = ?augmented_dataflow.as_of.as_ref().unwrap().elements(),
until = ?augmented_dataflow.until.elements(),
"creating dataflow",
);
} else {
tracing::info!(
name = %augmented_dataflow.debug_name,
import_ids = %augmented_dataflow.display_import_ids(),
export_ids = %augmented_dataflow.display_export_ids(),
as_of = ?augmented_dataflow.as_of.as_ref().unwrap().elements(),
until = ?augmented_dataflow.until.elements(),
"creating dataflow",
);
}
// Skip the actual dataflow creation for an empty `as_of`. (Happens e.g. for the
// bootstrapping of a REFRESH AT mat view that is past its last refresh.)
if as_of.is_empty() {
tracing::info!(
name = %augmented_dataflow.debug_name,
"not sending `CreateDataflow`, because of empty `as_of`",
);
} else {
let collections: Vec<_> = augmented_dataflow.export_ids().collect();
self.send(ComputeCommand::CreateDataflow(augmented_dataflow));
for id in collections {
self.maybe_schedule_collection(id);
}
}
Ok(())
}
/// Schedule the identified collection if all its inputs are available.
///
/// # Panics
///
/// Panics if the identified collection does not exist.
fn maybe_schedule_collection(&mut self, id: GlobalId) {
let collection = self.expect_collection(id);
// Don't schedule collections twice.
if collection.scheduled {
return;
}
let as_of = collection.read_frontier();
// If the collection has an empty `as_of`, it was either never installed on the replica or
// has since been dropped. In either case the replica does not expect any commands for it.
if as_of.is_empty() {
return;
}
let ready = if id.is_transient() {
// Always schedule transient collections immediately. The assumption is that those are
// created by interactive user commands and we want to schedule them as quickly as
// possible. Inputs might not yet be available, but when they become available, we
// don't need to wait for the controller to become aware and for the scheduling check
// to run again.
true
} else {
// Ignore self-dependencies. Any self-dependencies do not need to be
// available at the as_of for the dataflow to make progress, so we
// can ignore them here. At the moment, only continual tasks have
// self-dependencies, but this logic is correct for any dataflow, so
// we don't special case it to CTs.
let not_self_dep = |x: &GlobalId| *x != id;
// Check dependency frontiers to determine if all inputs are
// available. An input is available when its frontier is greater
// than the `as_of`, i.e., all input data up to and including the
// `as_of` has been sealed.
let compute_deps = collection.compute_dependency_ids().filter(not_self_dep);
let compute_frontiers = compute_deps.map(|id| {
let dep = &self.expect_collection(id);
dep.write_frontier()
});
let storage_deps = collection.storage_dependency_ids().filter(not_self_dep);
let storage_frontiers = self
.storage_collections
.collections_frontiers(storage_deps.collect())
.expect("must exist");
let storage_frontiers = storage_frontiers.into_iter().map(|f| f.write_frontier);
let ready = compute_frontiers
.chain(storage_frontiers)
.all(|frontier| PartialOrder::less_than(&as_of, &frontier));
ready
};
if ready {
self.send(ComputeCommand::Schedule(id));
let collection = self.expect_collection_mut(id);
collection.scheduled = true;
}
}
/// Schedule any unscheduled collections that are ready.
fn schedule_collections(&mut self) {
let ids: Vec<_> = self.collections.keys().copied().collect();
for id in ids {
self.maybe_schedule_collection(id);
}
}
/// Drops the read capability for the given collections and allows their resources to be
/// reclaimed.
#[mz_ore::instrument(level = "debug")]
pub fn drop_collections(&mut self, ids: Vec<GlobalId>) -> Result<(), CollectionMissing> {
for id in &ids {
let collection = self.collection_mut(*id)?;
// Mark the collection as dropped to allow it to be removed from the controller state.
collection.dropped = true;
// Drop the implied and warmup read holds to announce that clients are not
// interested in the collection anymore.
collection.implied_read_hold.release();
collection.warmup_read_hold.release();
// If the collection is a subscribe, stop tracking it. This ensures that the controller
// ceases to produce `SubscribeResponse`s for this subscribe.
self.subscribes.remove(id);
// If the collection is a copy to, stop tracking it. This ensures that the controller
// ceases to produce `CopyToResponse`s` for this copy to.
self.copy_tos.remove(id);
}
Ok(())
}
/// Initiate a peek request for the contents of `id` at `timestamp`.
#[mz_ore::instrument(level = "debug")]
pub fn peek(
&mut self,
peek_target: PeekTarget,
literal_constraints: Option<Vec<Row>>,
uuid: Uuid,
timestamp: T,
finishing: RowSetFinishing,
map_filter_project: mz_expr::SafeMfpPlan,
mut read_hold: ReadHold<T>,
target_replica: Option<ReplicaId>,
peek_response_tx: oneshot::Sender<PeekResponse>,
) -> Result<(), PeekError> {
use PeekError::*;
// Downgrade the provided read hold to the peek time.
let target_id = peek_target.id();
if read_hold.id() != target_id {
return Err(ReadHoldIdMismatch(read_hold.id()));
}
read_hold
.try_downgrade(Antichain::from_elem(timestamp.clone()))
.map_err(|_| ReadHoldInsufficient(target_id))?;
if let Some(target) = target_replica {
if !self.replica_exists(target) {
return Err(ReplicaMissing(target));
}
}
let otel_ctx = OpenTelemetryContext::obtain();
self.peeks.insert(
uuid,
PendingPeek {
target_replica,
// TODO(guswynn): can we just hold the `tracing::Span` here instead?
otel_ctx: otel_ctx.clone(),
requested_at: Instant::now(),
read_hold,
peek_response_tx,
limit: finishing.limit.map(usize::cast_from),
offset: finishing.offset,
},
);
self.send(ComputeCommand::Peek(Peek {
literal_constraints,
uuid,
timestamp,
finishing,
map_filter_project,
// Obtain an `OpenTelemetryContext` from the thread-local tracing
// tree to forward it on to the compute worker.
otel_ctx,
target: peek_target,
}));
Ok(())
}
/// Cancels an existing peek request.
#[mz_ore::instrument(level = "debug")]
pub fn cancel_peek(&mut self, uuid: Uuid, reason: PeekResponse) {
let Some(peek) = self.peeks.get_mut(&uuid) else {
tracing::warn!("did not find pending peek for {uuid}");
return;
};
let duration = peek.requested_at.elapsed();
self.metrics
.observe_peek_response(&PeekResponse::Canceled, duration);
// Enqueue a notification for the cancellation.
let otel_ctx = peek.otel_ctx.clone();
otel_ctx.attach_as_parent();
self.deliver_response(ComputeControllerResponse::PeekNotification(
uuid,
PeekNotification::Canceled,
otel_ctx,
));
// Finish the peek.
// This will also propagate the cancellation to the replicas.
self.finish_peek(uuid, reason);
}
/// Assigns a read policy to specific identifiers.
///
/// The policies are assigned in the order presented, and repeated identifiers should
/// conclude with the last policy. Changing a policy will immediately downgrade the read
/// capability if appropriate, but it will not "recover" the read capability if the prior
/// capability is already ahead of it.
///
/// Identifiers not present in `policies` retain their existing read policies.
///
/// It is an error to attempt to set a read policy for a collection that is not readable in the
/// context of compute. At this time, only indexes are readable compute collections.
#[mz_ore::instrument(level = "debug")]
pub fn set_read_policy(
&mut self,
policies: Vec<(GlobalId, ReadPolicy<T>)>,
) -> Result<(), ReadPolicyError> {
// Do error checking upfront, to avoid introducing inconsistencies between a collection's
// `implied_capability` and `read_capabilities`.
for (id, _policy) in &policies {
let collection = self.collection(*id)?;
if collection.read_policy.is_none() {
return Err(ReadPolicyError::WriteOnlyCollection(*id));
}
}
for (id, new_policy) in policies {
let collection = self.expect_collection_mut(id);
let new_since = new_policy.frontier(collection.write_frontier().borrow());
let _ = collection.implied_read_hold.try_downgrade(new_since);
collection.read_policy = Some(new_policy);
}
Ok(())
}
/// Advance the global write frontier of the given collection.
///
/// Frontier regressions are gracefully ignored.
///
/// # Panics
///
/// Panics if the identified collection does not exist.
#[mz_ore::instrument(level = "debug")]
fn maybe_update_global_write_frontier(&mut self, id: GlobalId, new_frontier: Antichain<T>) {
let collection = self.expect_collection_mut(id);
let advanced = collection.shared.lock_write_frontier(|f| {
let advanced = PartialOrder::less_than(f, &new_frontier);
if advanced {
f.clone_from(&new_frontier);
}
advanced
});
if !advanced {
return;
}
// Relax the implied read hold according to the read policy.
let new_since = match &collection.read_policy {
Some(read_policy) => {
// For readable collections the read frontier is determined by applying the
// client-provided read policy to the write frontier.
read_policy.frontier(new_frontier.borrow())
}
None => {
// Write-only collections cannot be read within the context of the compute
// controller, so their read frontier only controls the read holds taken on their
// inputs. We can safely downgrade the input read holds to any time less than the
// write frontier.
//
// Note that some write-only collections (continual tasks) need to observe changes
// at their current write frontier during hydration. Thus, we cannot downgrade the
// read frontier to the write frontier and instead step it back by one.
Antichain::from_iter(
new_frontier
.iter()
.map(|t| t.step_back().unwrap_or(T::minimum())),
)
}
};
let _ = collection.implied_read_hold.try_downgrade(new_since);
// Report the frontier advancement.
self.deliver_response(ComputeControllerResponse::FrontierUpper {
id,
upper: new_frontier,
});
}
/// Apply collection read hold changes pending in `read_holds_rx`.
fn apply_read_hold_changes(&mut self) {
let mut allowed_compaction = BTreeMap::new();
let mut stashed_changes = std::mem::take(&mut self.stashed_read_hold_changes);
// It's more efficient to apply updates for greater IDs before updates for smaller IDs,
// since ID order usually matches dependency order and downgrading read holds on a
// collection can cause downgrades on its dependencies. So instead of processing changes as
// they come in, we batch them up as much as we can and process them in reverse ID order.
let mut recv_batch = || {
let mut batch = std::mem::take(&mut stashed_changes);
while let Ok((id, mut update)) = self.read_holds_rx.try_recv() {
batch
.entry(id)
.and_modify(|e| e.extend(update.drain()))
.or_insert(update);
}
let has_updates = !batch.is_empty();
has_updates.then_some(batch)
};
while let Some(batch) = recv_batch() {
for (id, mut update) in batch.into_iter().rev() {
let Some(collection) = self.collections.get_mut(&id) else {
// The `ComputeController` provides a sync API for creating collections and
// taking out read holds on them, without waiting for the collection to be
// created in the `Instance`. Thus we might see read hold changes for
// collections that haven't been created yet. Stash them for later application.
self.stashed_read_hold_changes
.entry(id)
.and_modify(|e| e.extend(update.drain()))
.or_insert(update);
continue;
};
let new_since = collection.shared.lock_read_capabilities(|caps| {
// Sanity check to prevent corrupted `read_capabilities`, which can cause hard-to-debug
// issues (usually stuck read frontiers).
let read_frontier = caps.frontier();
for (time, diff) in update.iter() {
let count = caps.count_for(time) + diff;
assert!(
count >= 0,
"invalid read capabilities update: negative capability \
(id={id:?}, read_capabilities={caps:?}, update={update:?})",
);
assert!(
count == 0 || read_frontier.less_equal(time),
"invalid read capabilities update: frontier regression \
(id={id:?}, read_capabilities={caps:?}, update={update:?})",
);
}
// Apply read capability updates and learn about resulting changes to the read
// frontier.
let changes = caps.update_iter(update.drain());
let changed = changes.count() > 0;
changed.then(|| caps.frontier().to_owned())
});
let Some(new_since) = new_since else {
continue; // read frontier did not change
};
// Propagate read frontier update to dependencies.
for read_hold in collection.compute_dependencies.values_mut() {
read_hold
.try_downgrade(new_since.clone())
.expect("frontiers don't regress");
}
for read_hold in collection.storage_dependencies.values_mut() {
read_hold
.try_downgrade(new_since.clone())
.expect("frontiers don't regress");
}
allowed_compaction.insert(id, new_since);
}
}
// Produce `AllowCompaction` commands.
for (id, frontier) in allowed_compaction {
self.send(ComputeCommand::AllowCompaction { id, frontier });
}
}
/// Fulfills a registered peek and cleans up associated state.
///
/// As part of this we:
/// * Send a `PeekResponse` through the peek's response channel.
/// * Emit a `CancelPeek` command to instruct replicas to stop spending resources on this
/// peek, and to allow the `ComputeCommandHistory` to reduce away the corresponding `Peek`
/// command.
/// * Remove the read hold for this peek, unblocking compaction that might have waited on it.
fn finish_peek(&mut self, uuid: Uuid, response: PeekResponse) {
let Some(peek) = self.peeks.remove(&uuid) else {
return;
};
// The recipient might not be interested in the peek response anymore, which is fine.
let _ = peek.peek_response_tx.send(response);
// NOTE: We need to send the `CancelPeek` command _before_ we release the peek's read hold
// (by dropping it), to avoid the edge case that caused database-issues#4812.
self.send(ComputeCommand::CancelPeek { uuid });
drop(peek.read_hold);
}
/// Handles a response from a replica. Replica IDs are re-used across replica restarts, so we
/// use the replica incarnation to drop stale responses.
fn handle_response(&mut self, (replica_id, incarnation, response): ReplicaResponse<T>) {
// Filter responses from non-existing or stale replicas.
if self
.replicas
.get(&replica_id)
.filter(|replica| replica.epoch.replica() == incarnation)
.is_none()
{
return;
}
// Invariant: the replica exists and has the expected incarnation.
match response {
ComputeResponse::Frontiers(id, frontiers) => {
self.handle_frontiers_response(id, frontiers, replica_id);
}
ComputeResponse::PeekResponse(uuid, peek_response, otel_ctx) => {
self.handle_peek_response(uuid, peek_response, otel_ctx, replica_id);
}
ComputeResponse::CopyToResponse(id, response) => {
self.handle_copy_to_response(id, response, replica_id);
}
ComputeResponse::SubscribeResponse(id, response) => {
self.handle_subscribe_response(id, response, replica_id);
}
ComputeResponse::Status(response) => {
self.handle_status_response(response, replica_id);
}
}
}
/// Handle new frontiers, returning any compute response that needs to
/// be sent to the client.
fn handle_frontiers_response(
&mut self,
id: GlobalId,
frontiers: FrontiersResponse<T>,
replica_id: ReplicaId,
) {
if !self.collections.contains_key(&id) {
soft_panic_or_log!(
"frontiers update for an unknown collection \
(id={id}, replica_id={replica_id}, frontiers={frontiers:?})"
);
return;
}
let Some(replica) = self.replicas.get_mut(&replica_id) else {
soft_panic_or_log!(
"frontiers update for an unknown replica \
(replica_id={replica_id}, frontiers={frontiers:?})"
);
return;
};
let Some(replica_collection) = replica.collections.get_mut(&id) else {
soft_panic_or_log!(
"frontiers update for an unknown replica collection \
(id={id}, replica_id={replica_id}, frontiers={frontiers:?})"
);
return;
};
if let Some(new_frontier) = frontiers.input_frontier {
replica_collection.update_input_frontier(new_frontier.clone());
}
if let Some(new_frontier) = frontiers.output_frontier {
replica_collection.update_output_frontier(new_frontier.clone());
}
if let Some(new_frontier) = frontiers.write_frontier {
replica_collection.update_write_frontier(new_frontier.clone());
self.maybe_update_global_write_frontier(id, new_frontier);
}
}
#[mz_ore::instrument(level = "debug")]
fn handle_peek_response(
&mut self,
uuid: Uuid,
response: PeekResponse,
otel_ctx: OpenTelemetryContext,
replica_id: ReplicaId,
) {
otel_ctx.attach_as_parent();
// We might not be tracking this peek anymore, because we have served a response already or
// because it was canceled. If this is the case, we ignore the response.
let Some(peek) = self.peeks.get(&uuid) else {
return;
};
// If the peek is targeting a replica, ignore responses from other replicas.
let target_replica = peek.target_replica.unwrap_or(replica_id);
if target_replica != replica_id {
return;
}
let duration = peek.requested_at.elapsed();
self.metrics.observe_peek_response(&response, duration);
let notification = PeekNotification::new(&response, peek.offset, peek.limit);
// NOTE: We use the `otel_ctx` from the response, not the pending peek, because we
// currently want the parent to be whatever the compute worker did with this peek.
self.deliver_response(ComputeControllerResponse::PeekNotification(
uuid,
notification,
otel_ctx,
));
self.finish_peek(uuid, response)
}
fn handle_copy_to_response(
&mut self,
sink_id: GlobalId,
response: CopyToResponse,
replica_id: ReplicaId,
) {
// We might not be tracking this COPY TO because we have already returned a response
// from one of the replicas. In that case, we ignore the response.
if !self.copy_tos.remove(&sink_id) {
return;
}
let result = match response {
CopyToResponse::RowCount(count) => Ok(count),
CopyToResponse::Error(error) => Err(anyhow::anyhow!(error)),
// We should never get here: Replicas only drop copy to collections in response
// to the controller allowing them to do so, and when the controller drops a
// copy to it also removes it from the list of tracked copy_tos (see
// [`Instance::drop_collections`]).
CopyToResponse::Dropped => {
tracing::error!(
%sink_id, %replica_id,
"received `Dropped` response for a tracked copy to",
);
return;
}
};
self.deliver_response(ComputeControllerResponse::CopyToResponse(sink_id, result));
}
fn handle_subscribe_response(
&mut self,
subscribe_id: GlobalId,
response: SubscribeResponse<T>,
replica_id: ReplicaId,
) {
if !self.collections.contains_key(&subscribe_id) {
soft_panic_or_log!(
"received response for an unknown subscribe \
(subscribe_id={subscribe_id}, replica_id={replica_id})",
);
return;
}
let Some(replica) = self.replicas.get_mut(&replica_id) else {
soft_panic_or_log!(
"subscribe response for an unknown replica (replica_id={replica_id})"
);
return;
};
let Some(replica_collection) = replica.collections.get_mut(&subscribe_id) else {
soft_panic_or_log!(
"subscribe response for an unknown replica collection \
(subscribe_id={subscribe_id}, replica_id={replica_id})"
);
return;
};
// Always apply replica write frontier updates. Even if the subscribe is not tracked
// anymore, there might still be replicas reading from its inputs, so we need to track the
// frontiers until all replicas have advanced to the empty one.
let write_frontier = match &response {
SubscribeResponse::Batch(batch) => batch.upper.clone(),
SubscribeResponse::DroppedAt(_) => Antichain::new(),
};
// For subscribes we downgrade all replica frontiers based on write frontiers. This should
// be fine because the input and output frontier of a subscribe track its write frontier.
// TODO(database-issues#4701): report subscribe frontiers through `Frontiers` responses
replica_collection.update_write_frontier(write_frontier.clone());
replica_collection.update_input_frontier(write_frontier.clone());
replica_collection.update_output_frontier(write_frontier.clone());
// If the subscribe is not tracked, or targets a different replica, there is nothing to do.
let Some(mut subscribe) = self.subscribes.get(&subscribe_id).cloned() else {
return;
};
let replica_targeted = subscribe.target_replica.unwrap_or(replica_id) == replica_id;
if !replica_targeted {
return;
}
// Apply a global frontier update.
// If this is a replica-targeted subscribe, it is important that we advance the global
// frontier only based on responses from the targeted replica. Otherwise, another replica
// could advance to the empty frontier, making us drop the subscribe on the targeted
// replica prematurely.
self.maybe_update_global_write_frontier(subscribe_id, write_frontier);
match response {
SubscribeResponse::Batch(batch) => {
let upper = batch.upper;
let mut updates = batch.updates;
// If this batch advances the subscribe's frontier, we emit all updates at times
// greater or equal to the last frontier (to avoid emitting duplicate updates).
if PartialOrder::less_than(&subscribe.frontier, &upper) {
let lower = std::mem::replace(&mut subscribe.frontier, upper.clone());
if upper.is_empty() {
// This subscribe cannot produce more data. Stop tracking it.
self.subscribes.remove(&subscribe_id);
} else {
// This subscribe can produce more data. Update our tracking of it.
self.subscribes.insert(subscribe_id, subscribe);
}
if let Ok(updates) = updates.as_mut() {
updates.retain(|(time, _data, _diff)| lower.less_equal(time));
}
self.deliver_response(ComputeControllerResponse::SubscribeResponse(
subscribe_id,
SubscribeBatch {
lower,
upper,
updates,
},
));
}
}
SubscribeResponse::DroppedAt(frontier) => {
// We should never get here: Replicas only drop subscribe collections in response
// to the controller allowing them to do so, and when the controller drops a
// subscribe it also removes it from the list of tracked subscribes (see
// [`Instance::drop_collections`]).
tracing::error!(
%subscribe_id,
%replica_id,
frontier = ?frontier.elements(),
"received `DroppedAt` response for a tracked subscribe",
);
self.subscribes.remove(&subscribe_id);
}
}
}
fn handle_status_response(&mut self, response: StatusResponse, replica_id: ReplicaId) {
match response {
StatusResponse::OperatorHydration(status) => {
self.update_operator_hydration_status(replica_id, status)
}
}
}
/// Downgrade the warmup capabilities of collections as much as possible.
///
/// The only requirement we have for a collection's warmup capability is that it is for a time
/// that is available in all of the collection's inputs. For each input the latest time that is
/// the case for is `write_frontier - 1`. So the farthest we can downgrade a collection's
/// warmup capability is the minimum of `write_frontier - 1` of all its inputs.
///
/// This method expects to be periodically called as part of instance maintenance work.
/// We would like to instead update the warmup capabilities synchronously in response to
/// frontier updates of dependency collections, but that is not generally possible because we
/// don't learn about frontier updates of storage collections synchronously. We could do
/// synchronous updates for compute dependencies, but we refrain from doing for simplicity.
fn downgrade_warmup_capabilities(&mut self) {
let mut new_capabilities = BTreeMap::new();
for (id, collection) in &self.collections {
// For write-only collections that have advanced to the empty frontier, we can drop the
// warmup capability entirely. There is no reason why we would need to hydrate those
// collections again, so being able to warm them up is not useful.
if collection.read_policy.is_none()
&& collection.shared.lock_write_frontier(|f| f.is_empty())
{
new_capabilities.insert(*id, Antichain::new());
continue;
}
let compute_frontiers = collection.compute_dependency_ids().flat_map(|dep_id| {
let collection = self.collections.get(&dep_id);
collection.map(|c| c.write_frontier())
});
let existing_storage_dependencies = collection
.storage_dependency_ids()
.filter(|id| self.storage_collections.check_exists(*id).is_ok())
.collect::<Vec<_>>();
let storage_frontiers = self
.storage_collections
.collections_frontiers(existing_storage_dependencies)
.expect("missing storage collections")
.into_iter()
.map(|f| f.write_frontier);
let mut new_capability = Antichain::new();
for frontier in compute_frontiers.chain(storage_frontiers) {
for time in frontier.iter() {
new_capability.insert(time.step_back().unwrap_or(time.clone()));
}
}
new_capabilities.insert(*id, new_capability);
}
for (id, new_capability) in new_capabilities {
let collection = self.expect_collection_mut(id);
let _ = collection.warmup_read_hold.try_downgrade(new_capability);
}
}
/// Process pending maintenance work.
///
/// This method is invoked periodically by the global controller.
/// It is a good place to perform maintenance work that arises from various controller state
/// changes and that cannot conveniently be handled synchronously with those state changes.
#[mz_ore::instrument(level = "debug")]
pub fn maintain(&mut self) {
self.rehydrate_failed_replicas();
self.downgrade_warmup_capabilities();
self.apply_read_hold_changes();
self.schedule_collections();
self.cleanup_collections();
self.update_frontier_introspection();
self.refresh_state_metrics();
self.refresh_wallclock_lag();
}
}
/// State maintained about individual compute collections.
///
/// A compute collection is either an index, or a storage sink, or a subscribe, exported by a
/// compute dataflow.
#[derive(Debug)]
struct CollectionState<T: ComputeControllerTimestamp> {
/// Whether this collection is a log collection.
///
/// Log collections are special in that they are only maintained by a subset of all replicas.
log_collection: bool,
/// Whether this collection has been dropped by a controller client.
///
/// The controller is allowed to remove the `CollectionState` for a collection only when
/// `dropped == true`. Otherwise, clients might still expect to be able to query information
/// about this collection.
dropped: bool,
/// Whether this collection has been scheduled, i.e., the controller has sent a `Schedule`
/// command for it.
scheduled: bool,
/// State shared with the `ComputeController`.
shared: SharedCollectionState<T>,
/// A read hold maintaining the implicit capability of the collection.
///
/// This capability is kept to ensure that the collection remains readable according to its
/// `read_policy`. It also ensures that read holds on the collection's dependencies are kept at
/// some time not greater than the collection's `write_frontier`, guaranteeing that the
/// collection's next outputs can always be computed without skipping times.
implied_read_hold: ReadHold<T>,
/// A read hold held to enable dataflow warmup.
///
/// Dataflow warmup is an optimization that allows dataflows to immediately start hydrating
/// even when their next output time (as implied by the `write_frontier`) is in the future.
/// By installing a read capability derived from the write frontiers of the collection's
/// inputs, we ensure that the as-of of new dataflows installed for the collection is at a time
/// that is immediately available, so hydration can begin immediately too.
warmup_read_hold: ReadHold<T>,
/// The policy to use to downgrade `self.implied_read_hold`.
///
/// If `None`, the collection is a write-only collection (i.e. a sink). For write-only
/// collections, the `implied_read_hold` is only required for maintaining read holds on the
/// inputs, so we can immediately downgrade it to the `write_frontier`.
read_policy: Option<ReadPolicy<T>>,
/// Storage identifiers on which this collection depends, and read holds this collection
/// requires on them.
storage_dependencies: BTreeMap<GlobalId, ReadHold<T>>,
/// Compute identifiers on which this collection depends, and read holds this collection
/// requires on them.
compute_dependencies: BTreeMap<GlobalId, ReadHold<T>>,
/// Introspection state associated with this collection.
introspection: CollectionIntrospection<T>,
}
impl<T: ComputeControllerTimestamp> CollectionState<T> {
/// Creates a new collection state, with an initial read policy valid from `since`.
fn new(
collection_id: GlobalId,
as_of: Antichain<T>,
shared: SharedCollectionState<T>,
storage_dependencies: BTreeMap<GlobalId, ReadHold<T>>,
compute_dependencies: BTreeMap<GlobalId, ReadHold<T>>,
read_holds_tx: mpsc::UnboundedSender<(GlobalId, ChangeBatch<T>)>,
introspection: CollectionIntrospection<T>,
) -> Self {
// A collection is not readable before the `as_of`.
let since = as_of.clone();
// A collection won't produce updates for times before the `as_of`.
let upper = as_of;
// Ensure that the provided `shared` is valid for the given `as_of`.
assert!(shared.lock_read_capabilities(|c| c.frontier() == since.borrow()));
assert!(shared.lock_write_frontier(|f| f == &upper));
// Initialize collection read holds.
// Note that the implied read hold was already added to the `read_capabilities` when
// `shared` was created, so we only need to add the warmup read hold here.
let implied_read_hold = ReadHold::new(collection_id, since.clone(), read_holds_tx.clone());
let warmup_read_hold = ReadHold::new(collection_id, since.clone(), read_holds_tx);
let updates = warmup_read_hold.since().iter().map(|t| (t.clone(), 1));
shared.lock_read_capabilities(|c| {
c.update_iter(updates);
});
Self {
log_collection: false,
dropped: false,
scheduled: false,
shared,
implied_read_hold,
warmup_read_hold,
read_policy: Some(ReadPolicy::ValidFrom(since)),
storage_dependencies,
compute_dependencies,
introspection,
}
}
/// Creates a new collection state for a log collection.
fn new_log_collection(
id: GlobalId,
shared: SharedCollectionState<T>,
read_holds_tx: mpsc::UnboundedSender<(GlobalId, ChangeBatch<T>)>,
introspection_tx: crossbeam_channel::Sender<IntrospectionUpdates>,
) -> Self {
let since = Antichain::from_elem(T::minimum());
let introspection =
CollectionIntrospection::new(id, introspection_tx, since.clone(), false, None, None);
let mut state = Self::new(
id,
since,
shared,
Default::default(),
Default::default(),
read_holds_tx,
introspection,
);
state.log_collection = true;
// Log collections are created and scheduled implicitly as part of replica initialization.
state.scheduled = true;
state
}
/// Reports the current read frontier.
fn read_frontier(&self) -> Antichain<T> {
self.shared
.lock_read_capabilities(|c| c.frontier().to_owned())
}
/// Reports the current write frontier.
fn write_frontier(&self) -> Antichain<T> {
self.shared.lock_write_frontier(|f| f.clone())
}
fn storage_dependency_ids(&self) -> impl Iterator<Item = GlobalId> + '_ {
self.storage_dependencies.keys().copied()
}
fn compute_dependency_ids(&self) -> impl Iterator<Item = GlobalId> + '_ {
self.compute_dependencies.keys().copied()
}
/// Reports the IDs of the dependencies of this collection.
fn dependency_ids(&self) -> impl Iterator<Item = GlobalId> + '_ {
self.compute_dependency_ids()
.chain(self.storage_dependency_ids())
}
}
/// Collection state shared with the `ComputeController`.
///
/// Having this allows certain controller APIs, such as `ComputeController::collection_frontiers`
/// and `ComputeController::acquire_read_hold` to be non-`async`. This comes at the cost of
/// complexity (by introducing shared mutable state) and performance (by introducing locking). We
/// should aim to reduce the amount of shared state over time, rather than expand it.
///
/// Note that [`SharedCollectionState`]s are initialized by the `ComputeController` prior to the
/// collection's creation in the [`Instance`]. This is to allow compute clients to query frontiers
/// and take new read holds immediately, without having to wait for the [`Instance`] to update.
#[derive(Clone, Debug)]
pub(super) struct SharedCollectionState<T> {
/// Accumulation of read capabilities for the collection.
///
/// This accumulation contains the capabilities held by all [`ReadHold`]s given out for the
/// collection, including `implied_read_hold` and `warmup_read_hold`.
///
/// NOTE: This field may only be modified by [`Instance::apply_read_hold_changes`] and
/// `ComputeController::acquire_read_hold`. Nobody else should modify read capabilities
/// directly. Instead, collection users should manage read holds through [`ReadHold`] objects
/// acquired through `ComputeController::acquire_read_hold`.
///
/// TODO(teskje): Restructure the code to enforce the above in the type system.
read_capabilities: Arc<Mutex<MutableAntichain<T>>>,
/// The write frontier of this collection.
write_frontier: Arc<Mutex<Antichain<T>>>,
}
impl<T: Timestamp> SharedCollectionState<T> {
pub fn new(as_of: Antichain<T>) -> Self {
// A collection is not readable before the `as_of`.
let since = as_of.clone();
// A collection won't produce updates for times before the `as_of`.
let upper = as_of;
// Initialize read capabilities to the `since`.
// The is the implied read capability. The corresponding [`ReadHold`] is created in
// [`CollectionState::new`].
let mut read_capabilities = MutableAntichain::new();
read_capabilities.update_iter(since.iter().map(|time| (time.clone(), 1)));
Self {
read_capabilities: Arc::new(Mutex::new(read_capabilities)),
write_frontier: Arc::new(Mutex::new(upper)),
}
}
pub fn lock_read_capabilities<F, R>(&self, f: F) -> R
where
F: FnOnce(&mut MutableAntichain<T>) -> R,
{
let mut caps = self.read_capabilities.lock().expect("poisoned");
f(&mut *caps)
}
pub fn lock_write_frontier<F, R>(&self, f: F) -> R
where
F: FnOnce(&mut Antichain<T>) -> R,
{
let mut frontier = self.write_frontier.lock().expect("poisoned");
f(&mut *frontier)
}
}
/// Manages certain introspection relations associated with a collection. Upon creation, it adds
/// rows to introspection relations. When dropped, it retracts its managed rows.
///
/// TODO: `ComputeDependencies` could be moved under this.
#[derive(Debug)]
struct CollectionIntrospection<T: ComputeControllerTimestamp> {
/// The ID of the compute collection.
collection_id: GlobalId,
/// A channel through which introspection updates are delivered.
introspection_tx: crossbeam_channel::Sender<IntrospectionUpdates>,
/// Introspection state for `IntrospectionType::Frontiers`.
///
/// `Some` if the collection does _not_ sink into a storage collection (i.e. is not an MV). If
/// the collection sinks into storage, the storage controller reports its frontiers instead.
frontiers: Option<FrontiersIntrospectionState<T>>,
/// Introspection state for `IntrospectionType::ComputeMaterializedViewRefreshes`.
///
/// `Some` if the collection is a REFRESH MV.
refresh: Option<RefreshIntrospectionState<T>>,
}
impl<T: ComputeControllerTimestamp> CollectionIntrospection<T> {
fn new(
collection_id: GlobalId,
introspection_tx: crossbeam_channel::Sender<IntrospectionUpdates>,
as_of: Antichain<T>,
storage_sink: bool,
initial_as_of: Option<Antichain<T>>,
refresh_schedule: Option<RefreshSchedule>,
) -> Self {
let refresh =
match (refresh_schedule, initial_as_of) {
(Some(refresh_schedule), Some(initial_as_of)) => Some(
RefreshIntrospectionState::new(refresh_schedule, initial_as_of, &as_of),
),
(refresh_schedule, _) => {
// If we have a `refresh_schedule`, then the collection is a MV, so we should also have
// an `initial_as_of`.
soft_assert_or_log!(
refresh_schedule.is_none(),
"`refresh_schedule` without an `initial_as_of`: {collection_id}"
);
None
}
};
let frontiers = (!storage_sink).then(|| FrontiersIntrospectionState::new(as_of));
let self_ = Self {
collection_id,
introspection_tx,
frontiers,
refresh,
};
self_.report_initial_state();
self_
}
/// Reports the initial introspection state.
fn report_initial_state(&self) {
if let Some(frontiers) = &self.frontiers {
let row = frontiers.row_for_collection(self.collection_id);
let updates = vec![(row, 1)];
self.send(IntrospectionType::Frontiers, updates);
}
if let Some(refresh) = &self.refresh {
let row = refresh.row_for_collection(self.collection_id);
let updates = vec![(row, 1)];
self.send(IntrospectionType::ComputeMaterializedViewRefreshes, updates);
}
}
/// Observe the given current collection frontiers and update the introspection state as
/// necessary.
fn observe_frontiers(&mut self, read_frontier: &Antichain<T>, write_frontier: &Antichain<T>) {
self.update_frontier_introspection(read_frontier, write_frontier);
self.update_refresh_introspection(write_frontier);
}
fn update_frontier_introspection(
&mut self,
read_frontier: &Antichain<T>,
write_frontier: &Antichain<T>,
) {
let Some(frontiers) = &mut self.frontiers else {
return;
};
if &frontiers.read_frontier == read_frontier && &frontiers.write_frontier == write_frontier
{
return; // no change
};
let retraction = frontiers.row_for_collection(self.collection_id);
frontiers.update(read_frontier, write_frontier);
let insertion = frontiers.row_for_collection(self.collection_id);
let updates = vec![(retraction, -1), (insertion, 1)];
self.send(IntrospectionType::Frontiers, updates);
}
fn update_refresh_introspection(&mut self, write_frontier: &Antichain<T>) {
let Some(refresh) = &mut self.refresh else {
return;
};
let retraction = refresh.row_for_collection(self.collection_id);
refresh.frontier_update(write_frontier);
let insertion = refresh.row_for_collection(self.collection_id);
if retraction == insertion {
return; // no change
}
let updates = vec![(retraction, -1), (insertion, 1)];
self.send(IntrospectionType::ComputeMaterializedViewRefreshes, updates);
}
fn send(&self, introspection_type: IntrospectionType, updates: Vec<(Row, Diff)>) {
// Failure to send means the `ComputeController` has been dropped and doesn't care about
// introspection updates anymore.
let _ = self.introspection_tx.send((introspection_type, updates));
}
}
impl<T: ComputeControllerTimestamp> Drop for CollectionIntrospection<T> {
fn drop(&mut self) {
// Retract collection frontiers.
if let Some(frontiers) = &self.frontiers {
let row = frontiers.row_for_collection(self.collection_id);
let updates = vec![(row, -1)];
self.send(IntrospectionType::Frontiers, updates);
}
// Retract MV refresh state.
if let Some(refresh) = &self.refresh {
let retraction = refresh.row_for_collection(self.collection_id);
let updates = vec![(retraction, -1)];
self.send(IntrospectionType::ComputeMaterializedViewRefreshes, updates);
}
}
}
#[derive(Debug)]
struct FrontiersIntrospectionState<T> {
read_frontier: Antichain<T>,
write_frontier: Antichain<T>,
}
impl<T: ComputeControllerTimestamp> FrontiersIntrospectionState<T> {
fn new(as_of: Antichain<T>) -> Self {
Self {
read_frontier: as_of.clone(),
write_frontier: as_of,
}
}
/// Return a `Row` reflecting the current collection frontiers.
fn row_for_collection(&self, collection_id: GlobalId) -> Row {
let read_frontier = self
.read_frontier
.as_option()
.map_or(Datum::Null, |ts| ts.clone().into());
let write_frontier = self
.write_frontier
.as_option()
.map_or(Datum::Null, |ts| ts.clone().into());
Row::pack_slice(&[
Datum::String(&collection_id.to_string()),
read_frontier,
write_frontier,
])
}
/// Update the introspection state with the given new frontiers.
fn update(&mut self, read_frontier: &Antichain<T>, write_frontier: &Antichain<T>) {
if read_frontier != &self.read_frontier {
self.read_frontier.clone_from(read_frontier);
}
if write_frontier != &self.write_frontier {
self.write_frontier.clone_from(write_frontier);
}
}
}
/// Information needed to compute introspection updates for a REFRESH materialized view when the
/// write frontier advances.
#[derive(Debug)]
struct RefreshIntrospectionState<T> {
// Immutable properties of the MV
refresh_schedule: RefreshSchedule,
initial_as_of: Antichain<T>,
// Refresh state
next_refresh: Datum<'static>, // Null or an MzTimestamp
last_completed_refresh: Datum<'static>, // Null or an MzTimestamp
}
impl<T> RefreshIntrospectionState<T> {
/// Return a `Row` reflecting the current refresh introspection state.
fn row_for_collection(&self, collection_id: GlobalId) -> Row {
Row::pack_slice(&[
Datum::String(&collection_id.to_string()),
self.last_completed_refresh,
self.next_refresh,
])
}
}
impl<T: ComputeControllerTimestamp> RefreshIntrospectionState<T> {
/// Construct a new [`RefreshIntrospectionState`], and apply an initial `frontier_update()` at
/// the `upper`.
fn new(
refresh_schedule: RefreshSchedule,
initial_as_of: Antichain<T>,
upper: &Antichain<T>,
) -> Self {
let mut self_ = Self {
refresh_schedule: refresh_schedule.clone(),
initial_as_of: initial_as_of.clone(),
next_refresh: Datum::Null,
last_completed_refresh: Datum::Null,
};
self_.frontier_update(upper);
self_
}
/// Should be called whenever the write frontier of the collection advances. It updates the
/// state that should be recorded in introspection relations, but doesn't send the updates yet.
fn frontier_update(&mut self, write_frontier: &Antichain<T>) {
if write_frontier.is_empty() {
self.last_completed_refresh =
if let Some(last_refresh) = self.refresh_schedule.last_refresh() {
last_refresh.into()
} else {
// If there is no last refresh, then we have a `REFRESH EVERY`, in which case
// the saturating roundup puts a refresh at the maximum possible timestamp.
T::maximum().into()
};
self.next_refresh = Datum::Null;
} else {
if PartialOrder::less_equal(write_frontier, &self.initial_as_of) {
// We are before the first refresh.
self.last_completed_refresh = Datum::Null;
let initial_as_of = self.initial_as_of.as_option().expect(
"initial_as_of can't be [], because then there would be no refreshes at all",
);
let first_refresh = initial_as_of
.round_up(&self.refresh_schedule)
.expect("sequencing makes sure that REFRESH MVs always have a first refresh");
soft_assert_or_log!(
first_refresh == *initial_as_of,
"initial_as_of should be set to the first refresh"
);
self.next_refresh = first_refresh.into();
} else {
// The first refresh has already happened.
let write_frontier = write_frontier.as_option().expect("checked above");
self.last_completed_refresh = write_frontier
.round_down_minus_1(&self.refresh_schedule)
.map_or_else(
|| {
soft_panic_or_log!(
"rounding down should have returned the first refresh or later"
);
Datum::Null
},
|last_completed_refresh| last_completed_refresh.into(),
);
self.next_refresh = write_frontier.clone().into();
}
}
}
}
/// A note of an outstanding peek response.
#[derive(Debug)]
struct PendingPeek<T: Timestamp> {
/// For replica-targeted peeks, this specifies the replica whose response we should pass on.
///
/// If this value is `None`, we pass on the first response.
target_replica: Option<ReplicaId>,
/// The OpenTelemetry context for this peek.
otel_ctx: OpenTelemetryContext,
/// The time at which the peek was requested.
///
/// Used to track peek durations.
requested_at: Instant,
/// The read hold installed to serve this peek.
read_hold: ReadHold<T>,
/// The channel to send peek results.
peek_response_tx: oneshot::Sender<PeekResponse>,
/// An optional limit of the peek's result size.
limit: Option<usize>,
/// The offset into the peek's result.
offset: usize,
}
#[derive(Debug, Clone)]
struct ActiveSubscribe<T> {
/// Current upper frontier of this subscribe.
frontier: Antichain<T>,
/// For replica-targeted subscribes, this specifies the replica whose responses we should pass on.
///
/// If this value is `None`, we pass on the first response for each time slice.
target_replica: Option<ReplicaId>,
}
impl<T: ComputeControllerTimestamp> ActiveSubscribe<T> {
fn new(target_replica: Option<ReplicaId>) -> Self {
Self {
frontier: Antichain::from_elem(T::minimum()),
target_replica,
}
}
}
/// State maintained about individual replicas.
#[derive(Debug)]
struct ReplicaState<T: ComputeControllerTimestamp> {
/// The ID of the replica.
id: ReplicaId,
/// Client for the running replica task.
client: ReplicaClient<T>,
/// The replica configuration.
config: ReplicaConfig,
/// Replica metrics.
metrics: ReplicaMetrics,
/// A channel through which introspection updates are delivered.
introspection_tx: crossbeam_channel::Sender<IntrospectionUpdates>,
/// Per-replica collection state.
collections: BTreeMap<GlobalId, ReplicaCollectionState<T>>,
/// The epoch of the replica.
epoch: ClusterStartupEpoch,
}
impl<T: ComputeControllerTimestamp> ReplicaState<T> {
fn new(
id: ReplicaId,
client: ReplicaClient<T>,
config: ReplicaConfig,
metrics: ReplicaMetrics,
introspection_tx: crossbeam_channel::Sender<IntrospectionUpdates>,
epoch: ClusterStartupEpoch,
) -> Self {
Self {
id,
client,
config,
metrics,
introspection_tx,
epoch,
collections: Default::default(),
}
}
/// Add a collection to the replica state.
///
/// # Panics
///
/// Panics if a collection with the same ID exists already.
fn add_collection(
&mut self,
id: GlobalId,
as_of: Antichain<T>,
input_read_holds: Vec<ReadHold<T>>,
) {
let metrics = self.metrics.for_collection(id);
let introspection = ReplicaCollectionIntrospection::new(
self.id,
id,
self.introspection_tx.clone(),
as_of.clone(),
);
let mut state =
ReplicaCollectionState::new(metrics, as_of, introspection, input_read_holds);
// We need to consider the edge case where the as-of is the empty frontier. Such an as-of
// is not useful for indexes, because they wouldn't be readable. For write-only
// collections, an empty as-of means that the collection has been fully written and no new
// dataflow needs to be created for it. Consequently, no hydration will happen either.
//
// Based on this, we could set the hydration flag in two ways:
// * `false`, as in "the dataflow was never created"
// * `true`, as in "the dataflow completed immediately"
//
// Since hydration is often used as a measure of dataflow progress and we don't want to
// give the impression that certain dataflows are somehow stuck when they are not, we go
// go with the second interpretation here.
if state.as_of.is_empty() {
state.set_hydrated();
}
if let Some(previous) = self.collections.insert(id, state) {
panic!("attempt to add a collection with existing ID {id} (previous={previous:?}");
}
}
/// Remove state for a collection.
fn remove_collection(&mut self, id: GlobalId) -> Option<ReplicaCollectionState<T>> {
self.collections.remove(&id)
}
/// Returns whether all replica frontiers of the given collection are empty.
fn collection_frontiers_empty(&self, id: GlobalId) -> bool {
self.collections.get(&id).map_or(true, |c| {
c.write_frontier.is_empty()
&& c.input_frontier.is_empty()
&& c.output_frontier.is_empty()
})
}
/// Returns the state of the [`ReplicaState`] formatted as JSON.
///
/// The returned value is not guaranteed to be stable and may change at any point in time.
#[mz_ore::instrument(level = "debug")]
pub fn dump(&self) -> Result<serde_json::Value, anyhow::Error> {
// Note: We purposefully use the `Debug` formatting for the value of all fields in the
// returned object as a tradeoff between usability and stability. `serde_json` will fail
// to serialize an object if the keys aren't strings, so `Debug` formatting the values
// prevents a future unrelated change from silently breaking this method.
// Destructure `self` here so we don't forget to consider dumping newly added fields.
let Self {
id,
client: _,
config: _,
metrics: _,
introspection_tx: _,
epoch,
collections,
} = self;
fn field(
key: &str,
value: impl Serialize,
) -> Result<(String, serde_json::Value), anyhow::Error> {
let value = serde_json::to_value(value)?;
Ok((key.to_string(), value))
}
let collections: BTreeMap<_, _> = collections
.iter()
.map(|(id, collection)| (id.to_string(), format!("{collection:?}")))
.collect();
let map = serde_json::Map::from_iter([
field("id", id.to_string())?,
field("collections", collections)?,
field("epoch", epoch)?,
]);
Ok(serde_json::Value::Object(map))
}
}
#[derive(Debug)]
struct ReplicaCollectionState<T: ComputeControllerTimestamp> {
/// The replica write frontier of this collection.
///
/// See [`FrontiersResponse::write_frontier`].
write_frontier: Antichain<T>,
/// The replica input frontier of this collection.
///
/// See [`FrontiersResponse::input_frontier`].
input_frontier: Antichain<T>,
/// The replica output frontier of this collection.
///
/// See [`FrontiersResponse::output_frontier`].
output_frontier: Antichain<T>,
/// Metrics tracked for this collection.
///
/// If this is `None`, no metrics are collected.
metrics: Option<ReplicaCollectionMetrics>,
/// Time at which this collection was installed.
created_at: Instant,
/// As-of frontier with which this collection was installed on the replica.
as_of: Antichain<T>,
/// Whether the collection is hydrated.
hydrated: bool,
/// Tracks introspection state for this collection.
introspection: ReplicaCollectionIntrospection<T>,
/// Read holds on storage inputs to this collection.
///
/// These read holds are kept to ensure that the replica is able to read from storage inputs at
/// all times it hasn't read yet. We only need to install read holds for storage inputs since
/// compaction of compute inputs is implicitly held back by Timely/DD.
input_read_holds: Vec<ReadHold<T>>,
/// Maximum frontier wallclock lag since the last introspection update.
wallclock_lag_max: Duration,
}
impl<T: ComputeControllerTimestamp> ReplicaCollectionState<T> {
fn new(
metrics: Option<ReplicaCollectionMetrics>,
as_of: Antichain<T>,
introspection: ReplicaCollectionIntrospection<T>,
input_read_holds: Vec<ReadHold<T>>,
) -> Self {
Self {
write_frontier: as_of.clone(),
input_frontier: as_of.clone(),
output_frontier: as_of.clone(),
metrics,
created_at: Instant::now(),
as_of,
hydrated: false,
introspection,
input_read_holds,
wallclock_lag_max: Default::default(),
}
}
/// Returns whether this collection is hydrated.
fn hydrated(&self) -> bool {
self.hydrated
}
/// Marks the collection as hydrated and updates metrics and introspection accordingly.
fn set_hydrated(&mut self) {
if let Some(metrics) = &self.metrics {
let duration = self.created_at.elapsed().as_secs_f64();
metrics.initial_output_duration_seconds.set(duration);
}
self.hydrated = true;
}
/// Updates the replica write frontier of this collection.
fn update_write_frontier(&mut self, new_frontier: Antichain<T>) {
if PartialOrder::less_than(&new_frontier, &self.write_frontier) {
soft_panic_or_log!(
"replica collection write frontier regression (old={:?}, new={new_frontier:?})",
self.write_frontier,
);
return;
} else if new_frontier == self.write_frontier {
return;
}
self.write_frontier = new_frontier;
}
/// Updates the replica input frontier of this collection.
fn update_input_frontier(&mut self, new_frontier: Antichain<T>) {
if PartialOrder::less_than(&new_frontier, &self.input_frontier) {
soft_panic_or_log!(
"replica collection input frontier regression (old={:?}, new={new_frontier:?})",
self.input_frontier,
);
return;
} else if new_frontier == self.input_frontier {
return;
}
self.input_frontier = new_frontier;
// Relax our read holds on collection inputs.
for read_hold in &mut self.input_read_holds {
let result = read_hold.try_downgrade(self.input_frontier.clone());
soft_assert_or_log!(
result.is_ok(),
"read hold downgrade failed (read_hold={read_hold:?}, new_since={:?})",
self.input_frontier,
);
}
}
/// Updates the replica output frontier of this collection.
fn update_output_frontier(&mut self, new_frontier: Antichain<T>) {
if PartialOrder::less_than(&new_frontier, &self.output_frontier) {
soft_panic_or_log!(
"replica collection output frontier regression (old={:?}, new={new_frontier:?})",
self.output_frontier,
);
return;
} else if new_frontier == self.output_frontier {
return;
}
self.output_frontier = new_frontier;
// If the observed frontier is greater than the collection's as-of, the collection has
// produced some output and is therefore hydrated now.
if !self.hydrated() && PartialOrder::less_than(&self.as_of, &self.output_frontier) {
self.set_hydrated();
}
}
}
/// Maintains the introspection state for a given replica and collection, and ensures that reported
/// introspection data is retracted when the collection is dropped.
#[derive(Debug)]
struct ReplicaCollectionIntrospection<T: ComputeControllerTimestamp> {
/// The ID of the replica.
replica_id: ReplicaId,
/// The ID of the compute collection.
collection_id: GlobalId,
/// Operator-level hydration state.
/// (lir_id, worker_id) -> hydrated
operators: BTreeMap<(LirId, usize), bool>,
/// The collection's reported replica write frontier.
write_frontier: Antichain<T>,
/// A channel through which introspection updates are delivered.
introspection_tx: crossbeam_channel::Sender<IntrospectionUpdates>,
}
impl<T: ComputeControllerTimestamp> ReplicaCollectionIntrospection<T> {
/// Create a new `HydrationState` and initialize introspection.
fn new(
replica_id: ReplicaId,
collection_id: GlobalId,
introspection_tx: crossbeam_channel::Sender<IntrospectionUpdates>,
as_of: Antichain<T>,
) -> Self {
let self_ = Self {
replica_id,
collection_id,
operators: Default::default(),
write_frontier: as_of,
introspection_tx,
};
self_.report_initial_state();
self_
}
/// Reports the initial introspection state.
fn report_initial_state(&self) {
let row = self.write_frontier_row();
let updates = vec![(row, 1)];
self.send(IntrospectionType::ReplicaFrontiers, updates);
}
/// Update the given (lir_id, worker_id) pair as hydrated.
fn operator_hydrated(&mut self, lir_id: LirId, worker_id: usize, hydrated: bool) {
let retraction = self.operator_hydration_row(lir_id, worker_id);
self.operators.insert((lir_id, worker_id), hydrated);
let insertion = self.operator_hydration_row(lir_id, worker_id);
if retraction == insertion {
return; // no change
}
let updates = retraction
.map(|r| (r, -1))
.into_iter()
.chain(insertion.map(|r| (r, 1)))
.collect();
self.send(IntrospectionType::ComputeOperatorHydrationStatus, updates);
}
/// Return a `Row` reflecting the current hydration status of the identified operator.
///
/// Returns `None` if the identified operator is not tracked.
fn operator_hydration_row(&self, lir_id: LirId, worker_id: usize) -> Option<Row> {
self.operators.get(&(lir_id, worker_id)).map(|hydrated| {
Row::pack_slice(&[
Datum::String(&self.collection_id.to_string()),
Datum::UInt64(lir_id.into()),
Datum::String(&self.replica_id.to_string()),
Datum::UInt64(u64::cast_from(worker_id)),
Datum::from(*hydrated),
])
})
}
/// Observe the given current write frontier and update the introspection state as necessary.
fn observe_frontier(&mut self, write_frontier: &Antichain<T>) {
if self.write_frontier == *write_frontier {
return; // no change
}
let retraction = self.write_frontier_row();
self.write_frontier.clone_from(write_frontier);
let insertion = self.write_frontier_row();
let updates = vec![(retraction, -1), (insertion, 1)];
self.send(IntrospectionType::ReplicaFrontiers, updates);
}
/// Return a `Row` reflecting the current replica write frontier.
fn write_frontier_row(&self) -> Row {
let write_frontier = self
.write_frontier
.as_option()
.map_or(Datum::Null, |ts| ts.clone().into());
Row::pack_slice(&[
Datum::String(&self.collection_id.to_string()),
Datum::String(&self.replica_id.to_string()),
write_frontier,
])
}
fn send(&self, introspection_type: IntrospectionType, updates: Vec<(Row, Diff)>) {
// Failure to send means the `ComputeController` has been dropped and doesn't care about
// introspection updates anymore.
let _ = self.introspection_tx.send((introspection_type, updates));
}
}
impl<T: ComputeControllerTimestamp> Drop for ReplicaCollectionIntrospection<T> {
fn drop(&mut self) {
// Retract operator hydration status.
let operators: Vec<_> = self.operators.keys().collect();
let updates: Vec<_> = operators
.into_iter()
.flat_map(|(lir_id, worker_id)| self.operator_hydration_row(*lir_id, *worker_id))
.map(|r| (r, -1))
.collect();
if !updates.is_empty() {
self.send(IntrospectionType::ComputeOperatorHydrationStatus, updates);
}
// Retract the write frontier.
let row = self.write_frontier_row();
let updates = vec![(row, -1)];
self.send(IntrospectionType::ReplicaFrontiers, updates);
}
}