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
// 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;
use std::time::Instant;

use chrono::{DateTime, Duration, DurationRound, Utc};
use differential_dataflow::lattice::Lattice;
use futures::stream::FuturesUnordered;
use futures::{future, StreamExt};
use mz_build_info::BuildInfo;
use mz_cluster_client::client::{ClusterStartupEpoch, TimelyConfig};
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, PersistSinkConnection};
use mz_compute_types::sources::SourceInstanceDesc;
use mz_dyncfg::ConfigSet;
use mz_expr::RowSetFinishing;
use mz_ore::cast::CastFrom;
use mz_ore::tracing::OpenTelemetryContext;
use mz_repr::{Datum, Diff, GlobalId, Row};
use mz_storage_client::controller::{IntrospectionType, StorageController};
use mz_storage_types::read_holds::ReadHoldError;
use mz_storage_types::read_policy::ReadPolicy;
use serde::Serialize;
use thiserror::Error;
use timely::progress::{Antichain, ChangeBatch};
use timely::{Container, PartialOrder};
use uuid::Uuid;

use crate::controller::error::CollectionMissing;
use crate::controller::replica::{ReplicaClient, ReplicaConfig};
use crate::controller::{
    CollectionState, ComputeControllerResponse, ComputeControllerTimestamp, IntrospectionUpdates,
    ReplicaId,
};
use crate::logging::LogVariant;
use crate::metrics::{InstanceMetrics, ReplicaMetrics};
use crate::metrics::{ReplicaCollectionMetrics, 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("dataflow definition lacks an as_of value")]
    MissingAsOf,
    #[error("dataflow has an as_of not beyond the since of collection: {0}")]
    SinceViolation(GlobalId),
    #[error("subscribe dataflow has an empty as_of")]
    EmptyAsOfForSubscribe,
    #[error("copy to dataflow has an empty as_of")]
    EmptyAsOfForCopyTo,
}

impl From<CollectionMissing> for DataflowCreationError {
    fn from(error: CollectionMissing) -> Self {
        Self::CollectionMissing(error.0)
    }
}

impl From<ReadHoldError> for DataflowCreationError {
    fn from(error: ReadHoldError) -> Self {
        match error {
            ReadHoldError::CollectionMissing(id) => DataflowCreationError::CollectionMissing(id),
            ReadHoldError::SinceViolation(id) => DataflowCreationError::SinceViolation(id),
        }
    }
}

#[derive(Error, Debug)]
pub(super) enum PeekError {
    #[error("collection does not exist: {0}")]
    CollectionMissing(GlobalId),
    #[error("replica does not exist: {0}")]
    ReplicaMissing(ReplicaId),
    #[error("peek timestamp is not beyond the since of collection: {0}")]
    SinceViolation(GlobalId),
}

impl From<CollectionMissing> for PeekError {
    fn from(error: CollectionMissing) -> Self {
        Self::CollectionMissing(error.0)
    }
}

impl From<ReadHoldError> for PeekError {
    fn from(error: ReadHoldError) -> Self {
        match error {
            ReadHoldError::CollectionMissing(id) => PeekError::CollectionMissing(id),
            ReadHoldError::SinceViolation(id) => PeekError::SinceViolation(id),
        }
    }
}

#[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)
    }
}

#[derive(Error, Debug)]
pub(super) enum SubscribeTargetError {
    #[error("subscribe does not exist: {0}")]
    SubscribeMissing(GlobalId),
    #[error("replica does not exist: {0}")]
    ReplicaMissing(ReplicaId),
    #[error("subscribe has already produced output")]
    SubscribeAlreadyStarted,
}

/// The state we keep for a compute instance.
#[derive(Debug)]
pub(super) struct Instance<T> {
    /// Build info for spawning replicas
    build_info: &'static BuildInfo,
    /// Whether instance initialization has been completed.
    initialized: 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
    /// [`ActiveInstance::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 [`ActiveInstance::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. #16641 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
    /// [`ActiveInstance::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 [`ActiveInstance::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>,
    /// Sender for responses to be delivered.
    response_tx: crossbeam_channel::Sender<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>,
}

impl<T: ComputeControllerTimestamp> Instance<T> {
    /// Acquire a handle to the collection state associated with `id`.
    pub 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.
    pub 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")
    }

    pub fn collections_iter(&self) -> impl Iterator<Item = (&GlobalId, &CollectionState<T>)> {
        self.collections.iter()
    }

    /// 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>,
        storage_dependencies: Vec<GlobalId>,
        compute_dependencies: Vec<GlobalId>,
        write_only: bool,
    ) {
        // Add global collection state.
        let mut state =
            CollectionState::new(as_of.clone(), storage_dependencies, compute_dependencies);
        // 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());
        }

        // 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,
    ) {
        let metrics = self.metrics.for_replica(id);
        let mut replica =
            ReplicaState::new(id, client, config, metrics, self.introspection_tx.clone());

        // Add per-replica collection state.
        for (collection_id, collection) in &self.collections {
            let as_of = collection.read_frontier().to_owned();
            replica.add_collection(*collection_id, as_of);
        }

        self.replicas.insert(id, replica);
    }

    fn remove_replica_state(&mut self, id: ReplicaId) -> Option<ReplicaState<T>> {
        let Some(replica) = self.replicas.remove(&id) else {
            return None;
        };

        if let Some(time) = replica.last_heartbeat {
            let row = Row::pack_slice(&[
                Datum::String(&id.to_string()),
                Datum::TimestampTz(time.try_into().expect("must fit")),
            ]);
            self.deliver_introspection_updates(
                IntrospectionType::ComputeReplicaHeartbeats,
                vec![(row, -1)],
            );
        }

        Some(replica)
    }

    /// Enqueue the given response for delivery to the controller clients.
    fn deliver_response(&mut self, response: ComputeControllerResponse<T>) {
        self.response_tx
            .send(response)
            .expect("global controller never drops");
    }

    /// Enqueue the given introspection updates for recording.
    fn deliver_introspection_updates(
        &mut self,
        type_: IntrospectionType,
        updates: Vec<(Row, Diff)>,
    ) {
        self.introspection_tx
            .send((type_, updates))
            .expect("global controller never drops");
    }

    /// Acquire an [`ActiveInstance`] by providing a storage controller.
    pub fn activate<'a>(
        &'a mut self,
        storage_controller: &'a mut dyn StorageController<Timestamp = T>,
    ) -> ActiveInstance<'a, T> {
        ActiveInstance {
            compute: self,
            storage_controller,
        }
    }

    /// Returns whether the identified replica exists.
    pub fn replica_exists(&self, id: ReplicaId) -> bool {
        self.replicas.contains_key(&id)
    }

    /// Returns the ids of all replicas of this instance.
    pub fn replica_ids(&self) -> impl Iterator<Item = ReplicaId> + '_ {
        self.replicas.keys().copied()
    }

    /// 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)
        })
    }

    /// 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 `ActiveComputeController::process`, which we expect to
    /// be periodically called 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()));
    }

    /// Report updates (inserts or retractions) to the identified collection's dependencies.
    ///
    /// # Panics
    ///
    /// Panics if the identified collection does not exist.
    fn report_dependency_updates(&mut 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 the given collection and replica according to an
    /// observed frontier update.
    fn update_hydration_status(
        &mut self,
        id: GlobalId,
        replica_id: ReplicaId,
        frontier: &Antichain<T>,
    ) {
        let Some(replica) = self.replicas.get_mut(&replica_id) else {
            tracing::error!(
                %id, %replica_id, frontier = ?frontier.elements(),
                "frontier update for an unknown replica"
            );
            return;
        };
        let Some(collection) = replica.collections.get_mut(&id) else {
            tracing::error!(
                %id, %replica_id, frontier = ?frontier.elements(),
                "frontier update for an unknown collection"
            );
            return;
        };

        // We may have already reported successful hydration before, in which case we have nothing
        // left to do.
        if collection.hydrated() {
            return;
        }

        // If the observed frontier is greater than the collection's as-of, the collection has
        // produced some output and is therefore hydrated now.
        if PartialOrder::less_than(&collection.as_of, frontier) {
            collection.set_hydrated();
        }
    }

    /// 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.hydration_state.operator_hydrated(
            status.lir_id,
            status.worker_id,
            status.hydrated,
        );
    }

    /// 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 write frontiers for the collection must have advanced to the empty
    ///     frontier. Advancement to the empty frontier 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.read_frontier().is_empty()
                    && collection
                        .replica_write_frontiers
                        .values()
                        .all(|frontier| frontier.is_empty())
            })
            .map(|(id, _collection)| *id)
            .collect();

        for id in to_remove {
            self.remove_collection(id);
        }
    }

    /// List compute collections that depend on the given collection.
    pub fn collection_reverse_dependencies(&self, id: GlobalId) -> impl Iterator<Item = &GlobalId> {
        self.collections_iter().filter_map(move |(id2, state)| {
            if state.compute_dependencies.contains(&id) {
                Some(id2)
            } else {
                None
            }
        })
    }

    /// 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.
    pub(crate) 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: _,
            initialized,
            replicas,
            collections,
            log_sources: _,
            peeks,
            subscribes,
            copy_tos,
            history: _,
            response_tx: _,
            introspection_tx: _,
            envd_epoch,
            replica_epochs,
            metrics: _,
            dyncfg: _,
        } = 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 map = serde_json::Map::from_iter([
            field("initialized", initialized)?,
            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)?,
        ]);
        Ok(serde_json::Value::Object(map))
    }
}

impl<T> Instance<T>
where
    T: ComputeControllerTimestamp,
    ComputeGrpcClient: ComputeClient<T>,
{
    pub fn new(
        build_info: &'static BuildInfo,
        arranged_logs: BTreeMap<LogVariant, GlobalId>,
        envd_epoch: NonZeroI64,
        metrics: InstanceMetrics,
        dyncfg: Arc<ConfigSet>,
        response_tx: crossbeam_channel::Sender<ComputeControllerResponse<T>>,
        introspection_tx: crossbeam_channel::Sender<IntrospectionUpdates>,
    ) -> Self {
        let collections = arranged_logs
            .iter()
            .map(|(_, id)| {
                let state = CollectionState::new_log_collection();
                (*id, state)
            })
            .collect();
        let history = ComputeCommandHistory::new(metrics.for_history());

        let mut instance = Self {
            build_info,
            initialized: false,
            replicas: Default::default(),
            collections,
            log_sources: arranged_logs,
            peeks: Default::default(),
            subscribes: Default::default(),
            copy_tos: Default::default(),
            history,
            response_tx,
            introspection_tx,
            envd_epoch,
            replica_epochs: Default::default(),
            metrics,
            dyncfg,
        };

        instance.send(ComputeCommand::CreateTimely {
            config: TimelyConfig::default(),
            epoch: ClusterStartupEpoch::new(envd_epoch, 0),
        });

        let dummy_logging_config = Default::default();
        instance.send(ComputeCommand::CreateInstance(InstanceConfig {
            logging: dummy_logging_config,
        }));

        instance
    }

    /// Update instance configuration.
    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.
    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;
        }
    }

    /// Drop this compute instance.
    ///
    /// # Panics
    ///
    /// Panics if the compute instance still has active replicas.
    /// Panics if the compute instance still has collections installed.
    pub fn drop(mut self) {
        // Collections might have been dropped but not cleaned up yet.
        self.cleanup_collections();

        assert!(
            self.replicas.is_empty(),
            "cannot drop instances with provisioned replicas"
        );
        assert!(
            self.collections.values().all(|c| c.log_collection),
            "cannot drop instances with installed collections"
        );
    }

    /// Sends a command to all replicas of this instance.
    #[mz_ore::instrument(level = "debug")]
    pub 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() {
            // If sending the command fails, the replica requires rehydration.
            if replica.client.send(cmd.clone()).is_err() {
                replica.failed = true;
            }
        }
    }

    /// Receives the next response from any replica of this instance.
    ///
    /// Returns `Err` if receiving from a replica has failed, to signal that it is in need of
    /// rehydration.
    ///
    /// This method is cancellation safe.
    pub async fn recv(&mut self) -> Result<(ReplicaId, ComputeResponse<T>), ReplicaId> {
        // Receive responses from any of the replicas, and take appropriate
        // action.
        let response = self
            .replicas
            .iter_mut()
            .map(|(id, replica)| async { (*id, replica.client.recv().await) })
            .collect::<FuturesUnordered<_>>()
            .next()
            .await;

        match response {
            None => {
                // There were no replicas in the set. Block forever to
                // communicate that no response is ready.
                future::pending().await
            }
            Some((replica_id, None)) => {
                // A replica has failed and requires rehydration.
                let replica = self.replicas.get_mut(&replica_id).unwrap();
                replica.failed = true;
                Err(replica_id)
            }
            Some((replica_id, Some(response))) => {
                // A replica has produced a response. Return it.
                self.register_replica_heartbeat(replica_id);
                Ok((replica_id, response))
            }
        }
    }

    /// Register a heartbeat from the given replica.
    ///
    /// # Panics
    ///
    /// Panics if the specified replica does not exist.
    fn register_replica_heartbeat(&mut self, replica_id: ReplicaId) {
        let replica = self
            .replicas
            .get_mut(&replica_id)
            .expect("replica must exist");

        let now = Utc::now()
            .duration_trunc(Duration::try_seconds(60).unwrap())
            .expect("cannot fail");

        let mut updates = Vec::new();
        if let Some(old) = replica.last_heartbeat {
            if old == now {
                return; // nothing new to report
            }

            let retraction = Row::pack_slice(&[
                Datum::String(&replica_id.to_string()),
                Datum::TimestampTz(old.try_into().expect("must fit")),
            ]);
            updates.push((retraction, -1));
        }

        replica.last_heartbeat = Some(now);

        let insertion = Row::pack_slice(&[
            Datum::String(&replica_id.to_string()),
            Datum::TimestampTz(now.try_into().expect("must fit")),
        ]);
        updates.push((insertion, 1));

        self.deliver_introspection_updates(IntrospectionType::ComputeReplicaHeartbeats, updates);
    }

    /// Assign a target replica to the identified subscribe.
    ///
    /// If a subscribe has a target replica assigned, only subscribe responses
    /// sent by that replica are considered.
    pub fn set_subscribe_target_replica(
        &mut self,
        id: GlobalId,
        target_replica: ReplicaId,
    ) -> Result<(), SubscribeTargetError> {
        if !self.replica_exists(target_replica) {
            return Err(SubscribeTargetError::ReplicaMissing(target_replica));
        }

        let Some(subscribe) = self.subscribes.get_mut(&id) else {
            return Err(SubscribeTargetError::SubscribeMissing(id));
        };

        // For sanity reasons, we don't allow re-targeting a subscribe for which we have already
        // produced output.
        if !subscribe.frontier.less_equal(&T::minimum()) {
            return Err(SubscribeTargetError::SubscribeAlreadyStarted);
        }

        subscribe.target_replica = Some(target_replica);
        Ok(())
    }
}

/// A wrapper around [`Instance`] with a live storage controller.
#[derive(Debug)]
pub(super) struct ActiveInstance<'a, T> {
    compute: &'a mut Instance<T>,
    storage_controller: &'a mut dyn StorageController<Timestamp = T>,
}

impl<'a, T> ActiveInstance<'a, T>
where
    T: ComputeControllerTimestamp,
    ComputeGrpcClient: ComputeClient<T>,
{
    /// Add a new instance replica, by ID.
    pub fn add_replica(
        &mut self,
        id: ReplicaId,
        mut config: ReplicaConfig,
    ) -> Result<(), ReplicaExists> {
        if self.compute.replica_exists(id) {
            return Err(ReplicaExists(id));
        }

        config.logging.index_logs = self.compute.log_sources.clone();
        let log_ids: BTreeSet<_> = config.logging.index_logs.values().collect();

        // Initialize frontier tracking for the new replica.
        let mut updates = BTreeMap::new();
        for (compute_id, collection) in &mut self.compute.collections {
            // Skip log collections not maintained by this replica.
            if collection.log_collection && !log_ids.contains(compute_id) {
                continue;
            }

            let read_frontier = collection.read_frontier();
            updates.insert(*compute_id, read_frontier.to_owned());
        }
        self.update_replica_write_frontiers(id, &updates);
        self.update_replica_input_frontiers(id, &updates);

        let replica_epoch = self.compute.replica_epochs.entry(id).or_default();
        *replica_epoch += 1;
        let metrics = self.compute.metrics.for_replica(id);
        let client = ReplicaClient::spawn(
            id,
            self.compute.build_info,
            config.clone(),
            ClusterStartupEpoch::new(self.compute.envd_epoch, *replica_epoch),
            metrics.clone(),
            Arc::clone(&self.compute.dyncfg),
        );

        // Take this opportunity to clean up the history we should present.
        self.compute.history.reduce();

        // Replay the commands at the client, creating new dataflow identifiers.
        for command in self.compute.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.compute.add_replica_state(id, client, config);

        Ok(())
    }

    /// Remove an existing instance replica, by ID.
    pub fn remove_replica(&mut self, id: ReplicaId) -> Result<(), ReplicaMissing> {
        self.compute
            .remove_replica_state(id)
            .ok_or(ReplicaMissing(id))?;

        // Remove frontier tracking for this replica.
        self.remove_replica_frontiers(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.compute.subscribes_targeting(id).collect();
        for subscribe_id in to_drop {
            let subscribe = self.compute.subscribes.remove(&subscribe_id).unwrap();
            let response = ComputeControllerResponse::SubscribeResponse(
                subscribe_id,
                SubscribeBatch {
                    lower: subscribe.frontier.clone(),
                    upper: subscribe.frontier,
                    updates: Err("target replica failed or was dropped".into()),
                },
            );
            self.compute.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.compute.peeks_targeting(id) {
            peek_responses.push(ComputeControllerResponse::PeekResponse(
                uuid,
                PeekResponse::Error("target replica failed or was dropped".into()),
                peek.otel_ctx.clone(),
            ));
            to_drop.push(uuid);
        }
        for response in peek_responses {
            self.compute.deliver_response(response);
        }
        to_drop.into_iter().for_each(|uuid| self.remove_peek(uuid));

        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.compute.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.compute.replicas.iter();
        let failed_replicas: Vec<_> = replicas
            .filter_map(|(id, replica)| replica.failed.then_some(*id))
            .collect();

        for replica_id in failed_replicas {
            self.rehydrate_replica(replica_id);
        }
    }

    /// Create the described dataflows and initializes state for their output.
    pub fn create_dataflow(
        &mut self,
        dataflow: DataflowDescription<mz_compute_types::plan::Plan<T>, (), T>,
    ) -> Result<(), DataflowCreationError> {
        // Simple sanity checks around `as_of`
        let as_of = dataflow
            .as_of
            .as_ref()
            .ok_or(DataflowCreationError::MissingAsOf)?;
        if as_of.is_empty() && dataflow.subscribe_ids().next().is_some() {
            return Err(DataflowCreationError::EmptyAsOfForSubscribe);
        }
        if as_of.is_empty() && dataflow.copy_to_ids().next().is_some() {
            return Err(DataflowCreationError::EmptyAsOfForCopyTo);
        }

        // Validate the dataflow as having inputs whose `since` is less or equal to the dataflow's `as_of`.
        // Start tracking frontiers for each dataflow, using its `as_of` for each index and sink.

        // When we initialize per-replica input frontiers (and thereby the per-replica read
        // capabilities), 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 initialize the per-replica write frontiers with the smallest possible value that
        // is a valid read capability for all inputs, which is the join of all input `since`s.
        let mut replica_input_frontier = Antichain::from_elem(T::minimum());

        // Record all transitive dependencies of the outputs.
        let mut storage_dependencies = Vec::new();
        let mut compute_dependencies = Vec::new();

        // Any potentially acquired STORAGE read holds. We acquire them and
        // check whether our as_of is valid. They are dropped once we installed
        // read capabilities manually.
        //
        // TODO: Instead of acquiring these and then dropping later, we should
        // instead store them and don't "manually" acquire read holds using
        // `update_read_capabilities`.
        let mut storage_read_holds = Vec::new();

        // Validate sources have `since.less_equal(as_of)`.
        for source_id in dataflow.source_imports.keys() {
            let storage_read_hold = self
                .storage_controller
                .acquire_read_hold_at_time(*source_id, as_of.clone())?;

            storage_dependencies.push(*source_id);
            replica_input_frontier.join_assign(storage_read_hold.since());

            storage_read_holds.push(storage_read_hold);
        }

        // Validate indexes have `since.less_equal(as_of)`.
        // TODO(mcsherry): Instead, return an error from the constructing method.
        for index_id in dataflow.index_imports.keys() {
            let collection = self.compute.collection(*index_id)?;
            let since = collection.read_capabilities.frontier();
            if !(timely::order::PartialOrder::less_equal(&since, &as_of.borrow())) {
                Err(DataflowCreationError::SinceViolation(*index_id))?;
            }

            compute_dependencies.push(*index_id);
            replica_input_frontier.join_assign(&since.to_owned());
        }

        // 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_frontier = Antichain::new();
        }

        // Canonicalize dependencies.
        // Probably redundant based on key structure, but doing for sanity.
        storage_dependencies.sort();
        storage_dependencies.dedup();
        compute_dependencies.sort();
        compute_dependencies.dedup();

        // We will bump the internals of each input by the number of dependents (outputs).
        let outputs = dataflow.sink_exports.len() + dataflow.index_exports.len();
        let mut changes = ChangeBatch::new();
        for time in as_of.iter() {
            // TODO(benesch): fix this dangerous use of `as`.
            #[allow(clippy::as_conversions)]
            changes.update(time.clone(), outputs as i64);
        }
        // Update storage read capabilities for inputs.
        let mut storage_read_updates = storage_dependencies
            .iter()
            .map(|id| (*id, changes.clone()))
            .collect();
        self.storage_controller
            .update_read_capabilities(&mut storage_read_updates);
        // Drop the acquired read holds after we installed our old-style, manual
        // read capabilities.
        drop(storage_read_holds);

        // Update compute read capabilities for inputs.
        let compute_read_updates = compute_dependencies
            .iter()
            .map(|id| (*id, changes.clone()))
            .collect();
        self.update_read_capabilities(compute_read_updates);

        // Install collection state for each of the exports.
        for export_id in dataflow.export_ids() {
            let write_only = dataflow.sink_exports.contains_key(&export_id);
            self.compute.add_collection(
                export_id,
                as_of.clone(),
                storage_dependencies.clone(),
                compute_dependencies.clone(),
                write_only,
            );
        }

        // Initialize tracking of replica frontiers.
        let replica_write_frontier_updates = dataflow
            .export_ids()
            .map(|id| (id, as_of.clone()))
            .collect();
        let replica_input_frontier_updates = dataflow
            .export_ids()
            .map(|id| (id, replica_input_frontier.clone()))
            .collect();
        let replica_ids: Vec<_> = self.compute.replica_ids().collect();
        for replica_id in replica_ids {
            self.update_replica_write_frontiers(replica_id, &replica_write_frontier_updates);
            self.update_replica_input_frontiers(replica_id, &replica_input_frontier_updates);
        }

        // Initialize tracking of subscribes.
        for subscribe_id in dataflow.subscribe_ids() {
            self.compute
                .subscribes
                .insert(subscribe_id, ActiveSubscribe::new());
        }

        // Initialize tracking of copy tos.
        for copy_to_id in dataflow.copy_to_ids() {
            self.compute.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_controller
                .collection_metadata(id)
                .map_err(|_| DataflowCreationError::CollectionMissing(id))?;

            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::Persist(conn) => {
                    let metadata = self
                        .storage_controller
                        .collection_metadata(id)
                        .map_err(|_| DataflowCreationError::CollectionMissing(id))?
                        .clone();
                    let conn = PersistSinkConnection {
                        value_desc: conn.value_desc,
                        storage_metadata: metadata,
                    };
                    ComputeSinkConnection::Persist(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,
            debug_name: dataflow.debug_name,
        };

        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.compute
                .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.compute.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 {
            // 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_frontiers = collection.compute_dependencies.iter().map(|id| {
                let dep = &self.compute.expect_collection(*id);
                &dep.write_frontier
            });

            let storage_frontiers = self
                .storage_controller
                .collections_frontiers(collection.storage_dependencies.clone())
                .expect("must exist");
            let storage_frontiers = storage_frontiers.iter().map(|(_id, _since, upper)| upper);

            let ready = compute_frontiers
                .chain(storage_frontiers)
                .all(|frontier| PartialOrder::less_than(&as_of, &frontier.borrow()));

            ready
        };

        if ready {
            self.compute.send(ComputeCommand::Schedule(id));
            let collection = self.compute.expect_collection_mut(id);
            collection.scheduled = true;
        }
    }

    /// Schedule any unscheduled collections that are ready.
    fn schedule_collections(&mut self) {
        let ids: Vec<_> = self.compute.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.
    pub fn drop_collections(&mut self, ids: Vec<GlobalId>) -> Result<(), CollectionMissing> {
        let mut read_capability_updates = BTreeMap::new();

        for id in &ids {
            let collection = self.compute.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 capabilities to announce that clients are not
            // interested in the collection anymore.
            let implied_capability = std::mem::take(&mut collection.implied_capability);
            let warmup_capability = std::mem::take(&mut collection.warmup_capability);
            let mut update = ChangeBatch::new();
            update.extend(implied_capability.iter().map(|t| (t.clone(), -1)));
            update.extend(warmup_capability.iter().map(|t| (t.clone(), -1)));
            read_capability_updates.insert(*id, update);

            // If the collection is a subscribe, stop tracking it. This ensures that the controller
            // ceases to produce `SubscribeResponse`s for this subscribe.
            self.compute.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.compute.copy_tos.remove(id);
        }

        if !read_capability_updates.is_empty() {
            self.update_read_capabilities(read_capability_updates);
        }

        Ok(())
    }

    /// Initiate a peek request for the contents of `id` at `timestamp`.
    #[mz_ore::instrument(level = "debug")]
    pub fn peek(
        &mut self,
        id: GlobalId,
        literal_constraints: Option<Vec<Row>>,
        uuid: Uuid,
        timestamp: T,
        finishing: RowSetFinishing,
        map_filter_project: mz_expr::SafeMfpPlan,
        target_replica: Option<ReplicaId>,
        peek_target: PeekTarget,
    ) -> Result<(), PeekError> {
        // When querying persist directly, we acquire read holds and verify that
        // we can actually acquire them at the right time.
        let mut maybe_storage_read_hold = None;
        match &peek_target {
            PeekTarget::Index { .. } => {
                let since = self.compute.collection(id)?.read_capabilities.frontier();
                if !since.less_equal(&timestamp) {
                    return Err(PeekError::SinceViolation(id));
                }
            }

            PeekTarget::Persist { .. } => {
                let storage_read_hold = self
                    .storage_controller
                    .acquire_read_hold_at_time(id, Antichain::from_elem(timestamp.clone()))?;

                maybe_storage_read_hold = Some(storage_read_hold);
            }
        }

        if let Some(target) = target_replica {
            if !self.compute.replica_exists(target) {
                return Err(PeekError::ReplicaMissing(target));
            }
        }

        // Install a compaction hold on `id` at `timestamp`.
        let mut updates = BTreeMap::new();
        updates.insert(id, ChangeBatch::new_from(timestamp.clone(), 1));
        match &peek_target {
            PeekTarget::Index { .. } => self.update_read_capabilities(updates),
            PeekTarget::Persist { .. } => self
                .storage_controller
                .update_read_capabilities(&mut updates),
        };

        // Drop the acquired read hold after we installed our old-style, manual
        // read capabilities.
        drop(maybe_storage_read_hold);

        let otel_ctx = OpenTelemetryContext::obtain();
        self.compute.peeks.insert(
            uuid,
            PendingPeek {
                target: peek_target.clone(),
                time: timestamp.clone(),
                target_replica,
                // TODO(guswynn): can we just hold the `tracing::Span` here instead?
                otel_ctx: otel_ctx.clone(),
                requested_at: Instant::now(),
            },
        );

        self.compute.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.
    pub fn cancel_peek(&mut self, uuid: Uuid) {
        let Some(peek) = self.compute.peeks.get_mut(&uuid) else {
            tracing::warn!("did not find pending peek for {uuid}");
            return;
        };

        let response = PeekResponse::Canceled;
        let duration = peek.requested_at.elapsed();
        self.compute
            .metrics
            .observe_peek_response(&response, duration);

        // Enqueue the response to the cancellation.
        let otel_ctx = peek.otel_ctx.clone();
        self.compute
            .deliver_response(ComputeControllerResponse::PeekResponse(
                uuid, response, otel_ctx,
            ));

        // Remove the peek.
        // This will also propagate the cancellation to the replicas.
        self.remove_peek(uuid);
    }

    /// 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.compute.collection(*id)?;
            if collection.read_policy.is_none() {
                return Err(ReadPolicyError::WriteOnlyCollection(*id));
            }
        }

        let mut read_capability_changes = BTreeMap::default();
        for (id, new_policy) in policies {
            let collection = self.compute.expect_collection_mut(id);

            let old_capability = &collection.implied_capability;
            let new_capability = new_policy.frontier(collection.write_frontier.borrow());
            if PartialOrder::less_than(old_capability, &new_capability) {
                let entry = read_capability_changes
                    .entry(id)
                    .or_insert_with(ChangeBatch::new);
                entry.extend(old_capability.iter().map(|t| (t.clone(), -1)));
                entry.extend(new_capability.iter().map(|t| (t.clone(), 1)));
                collection.implied_capability = new_capability;
            }

            collection.read_policy = Some(new_policy);
        }
        if !read_capability_changes.is_empty() {
            self.update_read_capabilities(read_capability_changes);
        }
        Ok(())
    }

    /// Accept write frontier updates from the compute layer.
    ///
    /// # Panics
    ///
    /// Panics if any of the `updates` references an absent collection.
    /// Panics if any of the `updates` regresses an existing write frontier.
    #[mz_ore::instrument(level = "debug")]
    fn update_write_frontiers(
        &mut self,
        replica_id: ReplicaId,
        updates: &BTreeMap<GlobalId, Antichain<T>>,
    ) {
        // Apply advancements of replica frontiers.
        self.update_replica_write_frontiers(replica_id, updates);

        // Apply advancements of global collection frontiers.
        self.maybe_update_global_write_frontiers(updates);
    }

    /// Apply replica write frontier updates.
    ///
    /// # Panics
    ///
    /// Panics if any of the `updates` references an absent collection.
    /// Panics if any of the `updates` regresses an existing replica write frontier.
    #[mz_ore::instrument(level = "debug")]
    fn update_replica_write_frontiers(
        &mut self,
        replica_id: ReplicaId,
        updates: &BTreeMap<GlobalId, Antichain<T>>,
    ) {
        for (id, new_upper) in updates {
            let collection = self.compute.expect_collection_mut(*id);

            let old_upper = collection
                .replica_write_frontiers
                .insert(replica_id, new_upper.clone());

            // Safety check against frontier regressions.
            if let Some(old) = &old_upper {
                assert!(
                    PartialOrder::less_equal(old, new_upper),
                    "replica frontier regression: {old:?} -> {new_upper:?}, \
                     collection={id}, replica={replica_id}",
                );
            }
        }
    }

    /// Apply replica input frontier updates.
    ///
    /// # Panics
    ///
    /// Panics if any of the `updates` references an absent collection.
    /// Panics if any of the `updates` regresses an existing replica input frontier.
    #[mz_ore::instrument(level = "debug")]
    fn update_replica_input_frontiers(
        &mut self,
        replica_id: ReplicaId,
        updates: &BTreeMap<GlobalId, Antichain<T>>,
    ) {
        // Compute and apply read hold downgrades on storage dependencies that result from
        // input frontier advancements.
        let mut storage_read_capability_changes = BTreeMap::default();
        for (id, new_cap) in updates {
            let collection = self.compute.expect_collection_mut(*id);

            let old_cap = collection
                .replica_input_frontiers
                .insert(replica_id, new_cap.clone());

            // Safety check against frontier regressions.
            if let Some(old) = &old_cap {
                assert!(
                    PartialOrder::less_equal(old, new_cap),
                    "replica input frontier regression: {old:?} -> {new_cap:?}, \
                     collection={id}, replica={replica_id}",
                );
            }

            // Update per-replica read holds on storage dependencies.
            for storage_id in &collection.storage_dependencies {
                let update = storage_read_capability_changes
                    .entry(*storage_id)
                    .or_insert_with(|| ChangeBatch::new());
                if let Some(old) = &old_cap {
                    update.extend(old.iter().map(|time| (time.clone(), -1)));
                }
                update.extend(new_cap.iter().map(|time| (time.clone(), 1)));
            }
        }

        // Prune empty changes. We might end up with empty changes for dependencies that have been
        // dropped already, which is fine but might be confusing if we reported them.
        storage_read_capability_changes.retain(|_key, update| !update.is_empty());

        if !storage_read_capability_changes.is_empty() {
            self.storage_controller
                .update_read_capabilities(&mut storage_read_capability_changes);
        }
    }

    /// Remove frontier tracking state for the given replica.
    #[mz_ore::instrument(level = "debug")]
    fn remove_replica_frontiers(&mut self, replica_id: ReplicaId) {
        let mut storage_read_capability_changes = BTreeMap::default();
        for collection in self.compute.collections.values_mut() {
            // Remove the tracked write frontier.
            collection.replica_write_frontiers.remove(&replica_id);

            // Remove the tracked input frontier and release any corresponding read holds on
            // storage dependencies.
            let last_cap = collection.replica_input_frontiers.remove(&replica_id);
            if let Some(frontier) = last_cap {
                if !frontier.is_empty() {
                    for storage_id in &collection.storage_dependencies {
                        let update = storage_read_capability_changes
                            .entry(*storage_id)
                            .or_insert_with(ChangeBatch::new);
                        update.extend(frontier.iter().map(|time| (time.clone(), -1)));
                    }
                }
            }
        }

        if !storage_read_capability_changes.is_empty() {
            self.storage_controller
                .update_read_capabilities(&mut storage_read_capability_changes);
        }
    }

    /// Apply global write frontier updates.
    ///
    /// Frontier regressions are gracefully ignored.
    ///
    /// # Panics
    ///
    /// Panics if any of the `updates` references an absent collection.
    #[mz_ore::instrument(level = "debug")]
    fn maybe_update_global_write_frontiers(&mut self, updates: &BTreeMap<GlobalId, Antichain<T>>) {
        // Compute and apply read capability downgrades that result from collection frontier
        // advancements.
        let mut read_capability_changes = BTreeMap::new();
        for (id, new_upper) in updates {
            let collection = self.compute.expect_collection_mut(*id);

            if !PartialOrder::less_than(&collection.write_frontier, new_upper) {
                continue; // frontier has not advanced
            }

            collection.write_frontier = new_upper.clone();

            let old_since = &collection.implied_capability;
            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_upper.borrow())
                }
                None => {
                    // Write-only collections cannot be read within the context of the compute
                    // controller, so we can immediately advance their read frontier to the new write
                    // frontier.
                    new_upper.clone()
                }
            };

            if PartialOrder::less_than(old_since, &new_since) {
                let mut update = ChangeBatch::new();
                update.extend(old_since.iter().map(|t| (t.clone(), -1)));
                update.extend(new_since.iter().map(|t| (t.clone(), 1)));
                read_capability_changes.insert(*id, update);
                collection.implied_capability = new_since;
            }
        }
        if !read_capability_changes.is_empty() {
            self.update_read_capabilities(read_capability_changes);
        }

        // Tell the storage controller about new write frontiers for storage collections that are
        // advanced by compute sinks.
        let storage_updates: Vec<_> = updates
            .iter()
            .filter_map(|(&id, upper)| {
                let found = self.storage_controller.check_exists(id).is_ok();
                found.then_some((id, upper.clone()))
            })
            .collect();
        self.storage_controller
            .update_write_frontiers(&storage_updates);
    }

    /// Applies `updates`, propagates consequences through other read capabilities, and sends
    /// appropriate compaction commands.
    #[mz_ore::instrument(level = "debug")]
    fn update_read_capabilities(&mut self, mut updates: BTreeMap<GlobalId, ChangeBatch<T>>) {
        // Records storage read capability updates.
        let mut storage_updates = BTreeMap::default();
        // Records compute collections with downgraded read frontiers.
        let mut compute_downgraded = BTreeSet::default();

        // We must not rely on any specific relative ordering of `GlobalId`s.
        // That said, it is reasonable to assume that collections generally have greater IDs than
        // their dependencies, so starting with the largest is a useful optimization.
        while let Some((id, mut update)) = updates.pop_last() {
            let Some(collection) = self.compute.collections.get_mut(&id) else {
                tracing::error!(
                    %id, ?update,
                    "received read capability update for an unknown collection",
                );
                continue;
            };

            // Sanity check to prevent corrupted `read_capabilities`, which can cause hard-to-debug
            // issues (usually stuck read frontiers).
            let read_frontier = collection.read_capabilities.frontier();
            for (time, diff) in update.iter() {
                let count = collection.read_capabilities.count_for(time) + diff;
                if count < 0 {
                    panic!(
                        "invalid read capabilities update for collection {id}: negative capability \
                         (read_capabilities={:?}, update={update:?}",
                        collection.read_capabilities
                    );
                } else if count > 0 && !read_frontier.less_equal(time) {
                    panic!(
                        "invalid read capabilities update for collection {id}: frontier regression \
                         (read_capabilities={:?}, update={update:?}",
                        collection.read_capabilities
                    );
                }
            }

            // Apply read capability updates and learn about resulting changes to the read
            // frontier.
            let changes = collection.read_capabilities.update_iter(update.drain());
            update.extend(changes);

            if update.is_empty() {
                continue; // read frontier did not change
            }

            compute_downgraded.insert(id);

            // Propagate read frontier updates to dependencies.
            for dep_id in &collection.compute_dependencies {
                updates
                    .entry(*dep_id)
                    .or_insert_with(ChangeBatch::new)
                    .extend(update.iter().cloned());
            }
            for dep_id in &collection.storage_dependencies {
                storage_updates
                    .entry(*dep_id)
                    .or_insert_with(ChangeBatch::new)
                    .extend(update.iter().cloned());
            }
        }

        // Produce `AllowCompaction` commands for collections that had read frontier downgrades.
        for id in compute_downgraded {
            let collection = self.compute.expect_collection(id);
            let frontier = collection.read_frontier().to_owned();
            self.compute
                .send(ComputeCommand::AllowCompaction { id, frontier });
        }

        // Report storage read capability updates.
        if !storage_updates.is_empty() {
            self.storage_controller
                .update_read_capabilities(&mut storage_updates);
        }
    }

    /// Removes a registered peek and clean up associated state.
    ///
    /// As part of this we:
    ///  * 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 remove_peek(&mut self, uuid: Uuid) {
        let Some(peek) = self.compute.peeks.remove(&uuid) else {
            return;
        };

        // NOTE: We need to send the `CancelPeek` command _before_ we release the peek's read hold,
        // to avoid the edge case that caused #16615.
        self.compute.send(ComputeCommand::CancelPeek { uuid });

        let update = (peek.target.id(), ChangeBatch::new_from(peek.time, -1));
        let mut updates = [update].into();
        match &peek.target {
            PeekTarget::Index { .. } => self.update_read_capabilities(updates),
            PeekTarget::Persist { .. } => self
                .storage_controller
                .update_read_capabilities(&mut updates),
        }
    }

    pub fn handle_response(
        &mut self,
        response: ComputeResponse<T>,
        replica_id: ReplicaId,
    ) -> Option<ComputeControllerResponse<T>> {
        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);
                None
            }
        }
    }

    /// 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,
    ) -> Option<ComputeControllerResponse<T>> {
        let mut response = None;

        // According to the compute protocol, replicas are not allowed to send `Frontiers`
        // responses that regress frontiers they have reported previously. We still perform a check
        // here, rather than risking the controller becoming confused trying to handle regressions.
        let Ok(coll) = self.compute.collection(id) else {
            tracing::error!(
               %id, %replica_id, ?frontiers,
               "frontiers update for unknown collection",
            );
            return None;
        };

        // Apply a write frontier advancement.
        if let Some(new_frontier) = frontiers.write_frontier {
            if let Some(old_frontier) = coll.replica_write_frontiers.get(&replica_id) {
                if !PartialOrder::less_equal(old_frontier, &new_frontier) {
                    tracing::error!(
                       %id, %replica_id, ?old_frontier, ?new_frontier,
                       "collection write frontier regression",
                    );
                    return None;
                }
            }

            let old_global_frontier = coll.write_frontier.clone();

            self.compute
                .update_hydration_status(id, replica_id, &new_frontier);
            self.update_write_frontiers(replica_id, &[(id, new_frontier.clone())].into());

            if let Ok(coll) = self.compute.collection(id) {
                if coll.write_frontier != old_global_frontier {
                    response = Some(ComputeControllerResponse::FrontierUpper {
                        id,
                        upper: coll.write_frontier.clone(),
                    });
                }
            }
        }

        // Apply an input frontier advancement.
        if let Some(new_frontier) = frontiers.input_frontier {
            self.update_replica_input_frontiers(replica_id, &[(id, new_frontier.clone())].into());
        }

        response
    }

    fn handle_peek_response(
        &mut self,
        uuid: Uuid,
        response: PeekResponse,
        otel_ctx: OpenTelemetryContext,
        replica_id: ReplicaId,
    ) -> Option<ComputeControllerResponse<T>> {
        // 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 peek = self.compute.peeks.get(&uuid)?;

        // 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 None;
        }

        let duration = peek.requested_at.elapsed();
        self.compute
            .metrics
            .observe_peek_response(&response, duration);

        self.remove_peek(uuid);

        // 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.
        Some(ComputeControllerResponse::PeekResponse(
            uuid, response, otel_ctx,
        ))
    }

    fn handle_copy_to_response(
        &mut self,
        sink_id: GlobalId,
        response: CopyToResponse,
        replica_id: ReplicaId,
    ) -> Option<ComputeControllerResponse<T>> {
        // 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.compute.copy_tos.remove(&sink_id) {
            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 None;
                }
            };
            Some(ComputeControllerResponse::CopyToResponse(sink_id, result))
        } else {
            None
        }
    }

    fn handle_subscribe_response(
        &mut self,
        subscribe_id: GlobalId,
        response: SubscribeResponse<T>,
        replica_id: ReplicaId,
    ) -> Option<ComputeControllerResponse<T>> {
        if !self.compute.collections.contains_key(&subscribe_id) {
            tracing::warn!(?replica_id, "Response for unknown subscribe {subscribe_id}",);
            tracing::error!("Replica sent a response for an unknown subscribe");
            return None;
        }

        // 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(),
        };

        self.compute
            .update_hydration_status(subscribe_id, replica_id, &write_frontier);

        let write_frontier_updates = [(subscribe_id, write_frontier)].into();
        self.update_replica_write_frontiers(replica_id, &write_frontier_updates);

        // For subscribes we downgrade replica input frontiers based on write frontiers. This
        // should be fine because subscribes can't jump their write frontiers ahead of the times
        // they have read from their inputs currently.
        // TODO(#16274): report subscribe input frontiers through `Frontiers` responses
        self.update_replica_input_frontiers(replica_id, &write_frontier_updates);

        // If the subscribe is not tracked, or targets a different replica, there is nothing to do.
        let mut subscribe = self.compute.subscribes.get(&subscribe_id)?.clone();
        let replica_targeted = subscribe.target_replica.unwrap_or(replica_id) == replica_id;
        if !replica_targeted {
            return None;
        }

        // 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_frontiers(&write_frontier_updates);

        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.compute.subscribes.remove(&subscribe_id);
                    } else {
                        // This subscribe can produce more data. Update our tracking of it.
                        self.compute.subscribes.insert(subscribe_id, subscribe);
                    }

                    if let Ok(updates) = updates.as_mut() {
                        updates.retain(|(time, _data, _diff)| lower.less_equal(time));
                    }
                    Some(ComputeControllerResponse::SubscribeResponse(
                        subscribe_id,
                        SubscribeBatch {
                            lower,
                            upper,
                            updates,
                        },
                    ))
                } else {
                    None
                }
            }
            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.compute.subscribes.remove(&subscribe_id);
                None
            }
        }
    }

    fn handle_status_response(&mut self, response: StatusResponse, replica_id: ReplicaId) {
        match response {
            StatusResponse::OperatorHydration(status) => self
                .compute
                .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.compute.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.write_frontier.is_empty()
                && !collection.warmup_capability.is_empty()
            {
                new_capabilities.insert(*id, Antichain::new());
                continue;
            }

            let compute_frontiers = collection.compute_dependencies.iter().flat_map(|dep_id| {
                let collection = self.compute.collections.get(dep_id);
                collection.map(|c| c.write_frontier.clone())
            });

            let existing_storage_dependencies = collection
                .storage_dependencies
                .iter()
                .filter(|id| self.storage_controller.check_exists(**id).is_ok())
                .copied()
                .collect::<Vec<_>>();
            let storage_frontiers = self
                .storage_controller
                .collections_frontiers(existing_storage_dependencies)
                .expect("missing storage collections")
                .into_iter()
                .map(|(_id, _since, upper)| upper);

            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()));
                }
            }

            if PartialOrder::less_than(&collection.warmup_capability, &new_capability) {
                new_capabilities.insert(*id, new_capability);
            }
        }

        let mut read_capability_changes = BTreeMap::new();
        for (id, new_capability) in new_capabilities {
            let collection = self.compute.expect_collection_mut(id);
            let old_capability = &collection.warmup_capability;

            let mut update = ChangeBatch::new();
            update.extend(old_capability.iter().map(|t| (t.clone(), -1)));
            update.extend(new_capability.iter().map(|t| (t.clone(), 1)));
            read_capability_changes.insert(id, update);
            collection.warmup_capability = new_capability;
        }

        if !read_capability_changes.is_empty() {
            self.update_read_capabilities(read_capability_changes);
        }
    }

    /// 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.
    pub fn maintain(&mut self) {
        self.rehydrate_failed_replicas();
        self.downgrade_warmup_capabilities();
        self.schedule_collections();
        self.compute.cleanup_collections();
        self.compute.refresh_state_metrics();
    }
}

#[derive(Debug)]
struct PendingPeek<T> {
    /// Information about the collection targeted by the peek.
    target: PeekTarget,
    /// The peek time.
    time: T,
    /// 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,
}

#[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() -> Self {
        Self {
            frontier: Antichain::from_elem(timely::progress::Timestamp::minimum()),
            target_replica: None,
        }
    }
}

/// State maintained about individual replicas.
#[derive(Debug)]
pub struct ReplicaState<T> {
    /// 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>>,
    /// Whether the replica has failed and requires rehydration.
    failed: bool,
    /// The time of the last reported heartbeat.
    last_heartbeat: Option<DateTime<Utc>>,
}

impl<T: Debug> ReplicaState<T> {
    fn new(
        id: ReplicaId,
        client: ReplicaClient<T>,
        config: ReplicaConfig,
        metrics: ReplicaMetrics,
        introspection_tx: crossbeam_channel::Sender<IntrospectionUpdates>,
    ) -> Self {
        Self {
            id,
            client,
            config,
            metrics,
            introspection_tx,
            collections: Default::default(),
            failed: false,
            last_heartbeat: None,
        }
    }

    /// 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>) {
        let metrics = self.metrics.for_collection(id);
        let hydration_state = HydrationState::new(self.id, id, self.introspection_tx.clone());
        let mut state = ReplicaCollectionState {
            metrics,
            created_at: Instant::now(),
            as_of,
            hydration_state,
        };

        // 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 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.
    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: _,
            collections,
            failed,
            last_heartbeat,
        } = 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("failed", failed)?,
            field("last_heartbeat", format!("{last_heartbeat:?}"))?,
        ]);
        Ok(serde_json::Value::Object(map))
    }
}

#[derive(Debug)]
struct ReplicaCollectionState<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>,
    /// Tracks hydration state for this collection.
    hydration_state: HydrationState,
}

impl<T> ReplicaCollectionState<T> {
    /// Returns whether this collection is hydrated.
    fn hydrated(&self) -> bool {
        self.hydration_state.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.hydration_state.collection_hydrated();
    }
}

/// Maintains both global and operator-level hydration introspection for a given replica and
/// collection, and ensures that reported introspection data is retracted when the flag is dropped.
#[derive(Debug)]
struct HydrationState {
    /// The ID of the replica.
    replica_id: ReplicaId,
    /// The ID of the compute collection.
    collection_id: GlobalId,
    /// Whether the collection is hydrated.
    hydrated: bool,
    /// Operator-level hydration state.
    /// (lir_id, worker_id) -> hydrated
    operators: BTreeMap<(LirId, usize), bool>,
    /// A channel through which introspection updates are delivered.
    introspection_tx: crossbeam_channel::Sender<IntrospectionUpdates>,
}

impl HydrationState {
    /// Create a new `HydrationState` and initialize introspection.
    fn new(
        replica_id: ReplicaId,
        collection_id: GlobalId,
        introspection_tx: crossbeam_channel::Sender<IntrospectionUpdates>,
    ) -> Self {
        let self_ = Self {
            replica_id,
            collection_id,
            hydrated: false,
            operators: Default::default(),
            introspection_tx,
        };

        let insertion = self_.row_for_collection();
        self_.send(
            IntrospectionType::ComputeHydrationStatus,
            vec![(insertion, 1)],
        );

        self_
    }

    /// Update the collection as hydrated.
    fn collection_hydrated(&mut self) {
        if self.hydrated {
            return; // nothing to do
        }

        let retraction = self.row_for_collection();
        self.hydrated = true;
        let insertion = self.row_for_collection();

        self.send(
            IntrospectionType::ComputeHydrationStatus,
            vec![(retraction, -1), (insertion, 1)],
        );
    }

    /// 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.row_for_operator(lir_id, worker_id);
        self.operators.insert((lir_id, worker_id), hydrated);
        let insertion = self.row_for_operator(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 collection hydration status.
    fn row_for_collection(&self) -> Row {
        Row::pack_slice(&[
            Datum::String(&self.collection_id.to_string()),
            Datum::String(&self.replica_id.to_string()),
            Datum::from(self.hydrated),
        ])
    }

    /// Return a `Row` reflecting the current hydration status of the identified operator.
    ///
    /// Returns `None` if the identified operator is not tracked.
    fn row_for_operator(&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),
                Datum::String(&self.replica_id.to_string()),
                Datum::UInt64(u64::cast_from(worker_id)),
                Datum::from(*hydrated),
            ])
        })
    }

    fn send(&self, introspection_type: IntrospectionType, updates: Vec<(Row, Diff)>) {
        let result = self.introspection_tx.send((introspection_type, updates));

        if result.is_err() {
            // The global controller holds on to the `introspection_rx`. So when we get here that
            // probably means that the controller was dropped and the process is shutting down, in
            // which case we don't care about introspection updates anymore.
            tracing::info!(
                ?introspection_type,
                "discarding introspection update because the receiver disconnected"
            );
        }
    }
}

impl Drop for HydrationState {
    fn drop(&mut self) {
        // Retract collection hydration status.
        let retraction = self.row_for_collection();
        self.send(
            IntrospectionType::ComputeHydrationStatus,
            vec![(retraction, -1)],
        );

        // Retract operator-level hydration status.
        let operators: Vec<_> = self.operators.keys().collect();
        let updates: Vec<_> = operators
            .into_iter()
            .flat_map(|(lir_id, worker_id)| self.row_for_operator(*lir_id, *worker_id))
            .map(|r| (r, -1))
            .collect();
        if !updates.is_empty() {
            self.send(IntrospectionType::ComputeOperatorHydrationStatus, updates)
        }
    }
}