dbsp 0.287.0

Continuous streaming analytics engine
Documentation
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
use crate::circuit::GlobalNodeId;
use crate::circuit::checkpointer::Checkpointer;
use crate::circuit::metrics::{DBSP_STEP, DBSP_STEP_LATENCY_MICROSECONDS};
use crate::circuit::schedule::CommitProgress;
use crate::monitor::visual_graph::Graph;
use crate::operator::dynamic::balance::{BalancerHint, PartitioningPolicy};
use crate::storage::backend::StorageError;
use crate::trace::spine_async::MAX_LEVEL0_BATCH_SIZE_RECORDS;
use crate::{
    Error as DbspError, RootCircuit, Runtime, RuntimeError, circuit::runtime::RuntimeHandle,
    profile::Profiler,
};
use anyhow::Error as AnyError;
use crossbeam::channel::{Receiver, Select, Sender, TryRecvError, bounded};
use feldera_buffer_cache::ThreadType;
use feldera_ir::LirCircuit;
use feldera_storage::{FileCommitter, StorageBackend, StoragePath};
use feldera_types::checkpoint::CheckpointMetadata;
use feldera_types::config::DevTweaks;
use feldera_types::config::dev_tweaks::{BufferCacheAllocationStrategy, BufferCacheStrategy};
pub use feldera_types::config::{StorageCacheConfig, StorageConfig, StorageOptions};
use feldera_types::transaction::CommitProgressSummary;
use itertools::Either;
use std::collections::BTreeMap;
use std::num::NonZeroUsize;
use std::sync::atomic::Ordering;
use std::sync::{Arc, Mutex};
use std::time::Duration;
use std::{
    collections::HashSet,
    error::Error as StdError,
    fmt::{self, Debug, Display, Error as FmtError, Formatter},
    iter::empty,
    net::SocketAddr,
    ops::Range,
    path::{Path, PathBuf},
    thread::Result as ThreadResult,
    time::Instant,
};
use tracing::{debug, info};
use uuid::Uuid;

#[cfg(doc)]
use crate::circuit::circuit_builder::Stream;
use crate::profile::{DbspProfile, GraphProfile, WorkerProfile};

use super::SchedulerError;
use super::circuit_builder::BootstrapInfo;
use super::runtime::WorkerPanicInfo;

/// Default ratio of merger threads to worker threads.
const DEFAULT_MERGER_THREAD_RATIO: usize = 1;

/// A host for some workers in the [`Layout`] for a multi-host DBSP circuit.
#[allow(clippy::manual_non_exhaustive)]
#[derive(Clone, serde::Serialize, serde::Deserialize)]
pub struct Host {
    /// The IP address and TCP port on which the host listens and to which the
    /// other hosts connect.
    pub address: SocketAddr,

    /// The worker thread IDs implemented on this host.  Worker thread IDs start
    /// with 0 in the first host and increase sequentially from there.  A host
    /// has `workers.len()` workers.
    pub workers: Range<usize>,

    /// Prevents `Host` and `Layout::Multihost` from being instantiated without
    /// using the constructor (which checks the invariants).
    _private: (),
}

impl Debug for Host {
    fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result {
        f.debug_struct("Host")
            .field("address", &self.address)
            .field("workers", &self.workers)
            .finish()
    }
}

/// How a DBSP circuit is laid out across one or more machines.
#[derive(Clone, Debug, serde::Serialize, serde::Deserialize)]
pub enum Layout {
    /// A layout whose workers run on a single host.
    Solo {
        /// The number of worker threads.
        n_workers: usize,
    },

    /// A layout across multiple machines.
    Multihost {
        /// Each host in the layout.  There should be two or more, each with a
        /// unique network address.
        hosts: Vec<Host>,

        /// The index within `hosts` of the current host.
        local_host_idx: usize,
    },
}

impl Layout {
    /// Returns a new solo layout with `n_workers` worker threads.
    pub fn new_solo(n_workers: usize) -> Layout {
        assert_ne!(n_workers, 0);
        Layout::Solo { n_workers }
    }

    /// Returns a new multihost layout with as many hosts as specified in
    /// `params`.  Each tuple in `params` specifies a host's unique network
    /// address and the number of workers to run on that host.  `local_address`
    /// must be one of the addresses in `params`.
    ///
    /// To execute such a multihost circuit, one must create a `Runtime` for it
    /// on every host in `params`, passing the same `params` in each case.  Each
    /// host must pass its own `local_address`.  The `Runtime` on each host
    /// listens on its own address and connects to all the other addresses.
    pub fn new_multihost(
        params: &[(SocketAddr, usize)],
        local_address: SocketAddr,
    ) -> Result<Layout, LayoutError> {
        // Check that the addresses are unique.
        let mut uniq = HashSet::new();
        if let Some((duplicate, _)) = params.iter().find(|(address, _)| !uniq.insert(address)) {
            return Err(LayoutError::DuplicateAddress(*duplicate));
        }

        // Find `local_address` in `params`.
        let local_host_idx = params
            .iter()
            .position(|(address, _)| *address == local_address)
            .ok_or(LayoutError::NoSuchAddress(local_address))?;

        if params.len() == 1 {
            Ok(Self::new_solo(params[0].1))
        } else {
            let mut hosts = Vec::with_capacity(params.len());
            let mut total_workers = 0;
            for (address, n_workers) in params {
                assert_ne!(*n_workers, 0);
                hosts.push(Host {
                    address: *address,
                    workers: total_workers..total_workers + *n_workers,
                    _private: (),
                });
                total_workers += *n_workers;
            }
            Ok(Layout::Multihost {
                hosts,
                local_host_idx,
            })
        }
    }

    /// Returns the range of IDs for the workers on the local machine.
    pub fn local_workers(&self) -> Range<usize> {
        match self {
            Self::Solo { n_workers } => 0..*n_workers,
            Self::Multihost {
                hosts,
                local_host_idx,
                ..
            } => hosts[*local_host_idx].workers.clone(),
        }
    }

    /// Returns an iterator over `Host`s in this layout other than this one.  If
    /// this is a single-host layout, this will be an empty iterator.
    pub fn other_hosts(&self) -> impl Iterator<Item = &Host> {
        match self {
            Self::Solo { .. } => Either::Left(empty()),
            Self::Multihost {
                hosts,
                local_host_idx,
            } => Either::Right(
                hosts
                    .iter()
                    .enumerate()
                    .filter_map(|(i, host)| (i != *local_host_idx).then_some(host)),
            ),
        }
    }

    /// Returns the network address for the current machine, or `None` if this
    /// is a solo layout.
    pub fn local_address(&self) -> Option<SocketAddr> {
        match self {
            Self::Solo { .. } => None,
            Self::Multihost {
                hosts,
                local_host_idx,
                ..
            } => Some(hosts[*local_host_idx].address),
        }
    }

    /// Returns the total number of worker threads in this layout.
    pub fn n_workers(&self) -> usize {
        match self {
            Self::Solo { n_workers } => *n_workers,
            Self::Multihost { hosts, .. } => hosts.iter().map(|host| host.workers.len()).sum(),
        }
    }

    pub fn is_multihost(&self) -> bool {
        matches!(self, Self::Multihost { .. })
    }

    pub fn is_solo(&self) -> bool {
        matches!(self, Self::Solo { .. })
    }

    pub fn n_hosts(&self) -> usize {
        match self {
            Layout::Solo { .. } => 1,
            Layout::Multihost { hosts, .. } => hosts.len(),
        }
    }

    pub fn local_host_idx(&self) -> usize {
        match self {
            Layout::Solo { .. } => 0,
            Layout::Multihost { local_host_idx, .. } => *local_host_idx,
        }
    }
}

#[derive(Clone, Debug, Eq, PartialEq, serde::Serialize)]
pub enum LayoutError {
    /// The socket address passed to `new_multihost()` isn't in the list of
    /// hosts.
    NoSuchAddress(SocketAddr),
    /// The list of socket addresses passed to `new_multihost()` contains a
    /// duplicate.
    DuplicateAddress(SocketAddr),
}

impl Display for LayoutError {
    fn fmt(&self, f: &mut Formatter<'_>) -> Result<(), FmtError> {
        match self {
            Self::NoSuchAddress(address) => write!(f, "address {address} not in list of hosts"),
            Self::DuplicateAddress(address) => {
                write!(f, "duplicate address {address} in list of hosts")
            }
        }
    }
}

impl StdError for LayoutError {}

/// DBSP circuit execution mode.
#[derive(Clone, Default, PartialEq, Eq, Debug)]
pub enum Mode {
    /// Operators in the circuit have persistent id's.
    ///
    /// In this mode, operators are assigned persistent id's by the compiler. These id's
    /// are preserved across circuit restarts for parts of the circuit that remain
    /// unmodified.  This allows the circuit to partially or completely restore its state from
    /// a checkpoint.
    ///
    /// In persistent mode, the circuit may fail to start if not all operators are assigned
    /// persistent id's.
    Persistent,

    /// Circuit operators are assigned ephemeral id's.
    ///
    /// In this mode, operators are not assigned persistent id's. The circuit can only
    /// restore its state from a checkpoint if the entire circuit is unmodified.
    #[default]
    Ephemeral,
}

/// A config for instantiating a multithreaded/multihost runtime to execute
/// circuits.
///
/// As opposed to `RuntimeConfig`, this struct stores state about which hosts
/// run the circuit and where they store data, e.g., state typically not
/// tunable/exposed by the user.
#[derive(Clone)]
pub struct CircuitConfig {
    /// How the circuit is laid out across one or multiple machines.
    pub layout: Layout,

    /// The maximum amount of memory, in bytes, that the process is allowed to use.
    /// Used to calculate the memory pressure level.
    pub max_rss_bytes: Option<u64>,

    /// Optionally, CPU numbers for pinning the worker threads.
    pub pin_cpus: Vec<usize>,

    pub mode: Mode,

    /// Storage configuration. If present, then storage is enabled..
    pub storage: Option<CircuitStorageConfig>,

    /// Parsed from `RuntimeConfig` for use by the circuit.
    pub dev_tweaks: DevTweaks,
}

/// Returns the chunk size for splitter operators, in records.
///
/// Operators that split their output into multiple chunks, such as joins, distinct, and aggregation,
/// should attempt to limit their output to this chunk size.
pub fn splitter_output_chunk_size() -> usize {
    Runtime::with_dev_tweaks(|d| d.splitter_chunk_size_records() as usize)
}

pub fn balancer_min_absolute_improvement_threshold() -> u64 {
    Runtime::with_dev_tweaks(|d| d.balancer_min_absolute_improvement_threshold())
}

pub fn balancer_min_relative_improvement_threshold() -> f64 {
    Runtime::with_dev_tweaks(|d| d.balancer_min_relative_improvement_threshold())
}

pub fn balancer_balance_tax() -> f64 {
    Runtime::with_dev_tweaks(|d| d.balancer_balance_tax())
}

pub fn balancer_key_distribution_refresh_threshold() -> f64 {
    Runtime::with_dev_tweaks(|d| d.balancer_key_distribution_refresh_threshold())
}

pub fn adaptive_joins_enabled() -> bool {
    Runtime::with_dev_tweaks(|d| d.adaptive_joins())
}

pub fn max_level0_batch_size_records() -> u16 {
    Runtime::with_dev_tweaks(|d| {
        d.max_level0_batch_size_records
            .unwrap_or(MAX_LEVEL0_BATCH_SIZE_RECORDS)
    })
}

pub fn negative_weight_multiplier() -> u16 {
    Runtime::with_dev_tweaks(|d| d.negative_weight_multiplier())
}

/// Configuration for storage in a [Runtime]-hosted circuit.
#[derive(Clone, derive_more::Debug)]
pub struct CircuitStorageConfig {
    /// Runner configuration.
    pub config: StorageConfig,

    /// User options.
    pub options: StorageOptions,

    /// Storage backend.
    ///
    /// Presumably opened according to `config` and `options`.
    #[debug(skip)]
    pub backend: Arc<dyn StorageBackend>,

    /// The initial checkpoint to start the circuit from, or `None` to start
    /// fresh from a new circuit.
    pub init_checkpoint: Option<Uuid>,
}

impl CircuitStorageConfig {
    /// Opens a backend with `config` and `options` and returns a
    /// [CircuitStorageConfig] with that backend.
    pub fn for_config(
        config: StorageConfig,
        options: StorageOptions,
    ) -> Result<Self, StorageError> {
        let backend = <dyn StorageBackend>::new(&config, &options)?;
        Ok(Self {
            config,
            options,
            backend,
            init_checkpoint: None,
        })
    }

    pub fn with_init_checkpoint(self, init_checkpoint: Option<Uuid>) -> Self {
        Self {
            init_checkpoint,
            ..self
        }
    }
}

impl Default for CircuitConfig {
    fn default() -> Self {
        Self::with_workers(1)
    }
}

impl CircuitConfig {
    pub fn with_workers(n: usize) -> Self {
        Self {
            layout: Layout::new_solo(n),
            max_rss_bytes: None,
            pin_cpus: Vec::new(),
            mode: Mode::Ephemeral,
            storage: None,
            dev_tweaks: DevTweaks::default(),
        }
    }

    pub fn with_max_rss_bytes(mut self, max_rss: Option<u64>) -> Self {
        self.max_rss_bytes = max_rss;
        self
    }

    pub fn with_mode(mut self, mode: Mode) -> Self {
        self.mode = mode;
        self
    }

    pub fn with_storage(mut self, storage: CircuitStorageConfig) -> Self {
        self.storage = Some(storage);
        self
    }

    pub fn with_splitter_chunk_size_records(mut self, records: u64) -> Self {
        self.dev_tweaks.splitter_chunk_size_records = Some(records);
        self
    }

    pub fn with_buffer_cache_strategy(mut self, strategy: BufferCacheStrategy) -> Self {
        self.dev_tweaks.buffer_cache_strategy = Some(strategy);
        self
    }

    pub fn with_buffer_max_buckets(mut self, max_buckets: Option<usize>) -> Self {
        self.dev_tweaks.buffer_max_buckets = max_buckets;
        self
    }

    pub fn with_buffer_cache_allocation_strategy(
        mut self,
        strategy: BufferCacheAllocationStrategy,
    ) -> Self {
        self.dev_tweaks.buffer_cache_allocation_strategy = Some(strategy);
        self
    }

    #[cfg(test)]
    pub fn with_fbuf_slab_bytes_per_class(mut self, bytes_per_class: usize) -> Self {
        self.dev_tweaks.fbuf_slab_bytes_per_class = Some(bytes_per_class);
        self
    }

    pub fn with_balancer_min_relative_improvement_threshold(mut self, threshold: f64) -> Self {
        self.dev_tweaks.balancer_min_relative_improvement_threshold = Some(threshold);
        self
    }

    pub fn with_balancer_min_absolute_improvement_threshold(mut self, threshold: u64) -> Self {
        self.dev_tweaks.balancer_min_absolute_improvement_threshold = Some(threshold);
        self
    }

    pub fn with_balancer_balance_tax(mut self, tax: f64) -> Self {
        self.dev_tweaks.balancer_balance_tax = Some(tax);
        self
    }

    #[cfg(test)]
    pub fn with_temporary_storage(self, path: impl AsRef<Path>) -> Self {
        Self {
            storage: Some(
                CircuitStorageConfig::for_config(
                    StorageConfig {
                        path: path.as_ref().to_string_lossy().into_owned(),
                        cache: Default::default(),
                    },
                    Default::default(),
                )
                .unwrap(),
            ),
            ..self
        }
    }

    /// The number of merger threads per host.
    pub(crate) fn num_merger_threads(&self) -> usize {
        let num_workers = self.layout.local_workers().len();
        match self.dev_tweaks.merger_threads {
            Some(threads) => threads as usize,
            None => num_workers * DEFAULT_MERGER_THREAD_RATIO,
        }
    }
}

impl From<&CircuitConfig> for CircuitConfig {
    fn from(value: &CircuitConfig) -> Self {
        value.clone()
    }
}

impl From<usize> for CircuitConfig {
    fn from(n_workers: usize) -> Self {
        Self::with_workers(n_workers)
    }
}

impl From<NonZeroUsize> for CircuitConfig {
    fn from(n_workers: NonZeroUsize) -> Self {
        Self::with_workers(n_workers.get())
    }
}

impl From<Layout> for CircuitConfig {
    fn from(layout: Layout) -> Self {
        Self {
            layout,
            ..Self::default()
        }
    }
}

impl Runtime {
    /// Instantiate a circuit in a multithreaded runtime.
    ///
    /// Creates a multithreaded runtime with the given `layout`, instantiates
    /// an identical circuit in each worker, by calling `constructor` once per
    /// worker.  `init_circuit` passes each call of `constructor` a new
    /// [`RootCircuit`], in which it should create input operators by calling
    /// [`RootCircuit::dyn_add_input_zset`] and related methods.  Each of these
    /// calls returns an input handle and a `Stream`.  The `constructor` can
    /// call [`Stream`] methods to construct more operators, each of which
    /// yields further `Stream`s.  It can also use [`Stream::output`] to obtain
    /// an output handle.
    ///
    /// The `layout` may be specified as a number of worker threads or as a
    /// [`Layout`].
    ///
    /// Returns a [`DBSPHandle`] that the caller can use to control the circuit
    /// and a user-defined value returned by the constructor.  The
    /// `constructor` should use the latter to return the input and output
    /// handles it obtains, because these allow the caller to feed input into
    /// the circuit and read output from the circuit.
    ///
    /// To ensure that the multithreaded runtime has identical input/output
    /// behavior to a single-threaded circuit, the `constructor` closure
    /// must satisfy certain constraints.  Most importantly, it must create
    /// identical circuits in all worker threads, adding and connecting
    /// operators in the same order.  This ensures that operators that shard
    /// their inputs across workers, e.g.,
    /// [`Stream::join`](`crate::Stream::join`), work correctly.
    /// The closure should return the same value in each worker thread; this
    /// function returns one of these values arbitrarily.
    ///
    /// TODO: Document other requirements.  Not all operators are currently
    /// thread-safe.
    pub fn init_circuit<F, T>(
        config: impl Into<CircuitConfig>,
        constructor: F,
    ) -> Result<(DBSPHandle, T), DbspError>
    where
        F: FnOnce(&mut RootCircuit) -> Result<T, AnyError> + Clone + Send + 'static,
        T: Send + 'static,
    {
        let config: CircuitConfig = config.into();
        let nworkers = config.layout.local_workers().len();

        // When a worker finishes building the circuit, it sends completion status back
        // to us via this channel.  The function returns after receiving a
        // notification from each worker.
        let (init_senders, init_receivers): (Vec<_>, Vec<_>) =
            (0..nworkers).map(|_| bounded(0)).unzip();

        // Channels used to send commands to workers.
        let (command_senders, command_receivers): (Vec<_>, Vec<_>) =
            (0..nworkers).map(|_| bounded(1)).unzip();

        // Channels used to signal command completion to the client.
        let (status_senders, status_receivers): (Vec<_>, Vec<_>) =
            (0..nworkers).map(|_| bounded(1)).unzip();

        let runtime = Self::run(&config, move |parker| {
            let worker_index = Runtime::local_worker_offset();

            // Drop all but one channels.  This makes sure that if one of the worker panics
            // or exits, its channel will become disconnected.
            let init_sender = init_senders.into_iter().nth(worker_index).unwrap();
            let status_sender = status_senders.into_iter().nth(worker_index).unwrap();
            let command_receiver = command_receivers.into_iter().nth(worker_index).unwrap();

            let circuit_fn = |circuit: &mut RootCircuit| {
                let profiler = Profiler::new(circuit);
                constructor(circuit).map(|res| (res, profiler))
            };
            let (mut circuit, profiler) = match RootCircuit::build(circuit_fn) {
                Ok((circuit, (res, profiler))) => {
                    if init_sender.send(Ok((res, circuit.fingerprint()))).is_err() {
                        return;
                    }
                    (circuit, profiler)
                }
                Err(e) => {
                    let _ = init_sender.send(Err(e));
                    return;
                }
            };

            while !Runtime::kill_in_progress() {
                // Wait for command.
                match command_receiver.try_recv() {
                    Ok(Command::Transaction) => {
                        let status = circuit.transaction().map(|_| Response::Unit);
                        // Send response.
                        if status_sender.send(status).is_err() {
                            return;
                        }
                    }
                    Ok(Command::StartTransaction) => {
                        let status = circuit.start_transaction().map(|_| Response::Unit);
                        // Send response.
                        if status_sender.send(status).is_err() {
                            return;
                        }
                    }
                    Ok(Command::CommitTransaction) => {
                        let status = circuit.start_commit_transaction().map(|_| Response::Unit);
                        // Send response.
                        if status_sender.send(status).is_err() {
                            return;
                        }
                    }
                    Ok(Command::CommitProgress) => {
                        let status = Ok(Response::CommitProgress(circuit.commit_progress()));
                        // Send response.
                        if status_sender.send(status).is_err() {
                            return;
                        }
                    }
                    Ok(Command::Step) => {
                        let status = circuit
                            .step()
                            .map(|_| Response::CommitComplete(circuit.is_commit_complete()));
                        // Send response.
                        if status_sender.send(status).is_err() {
                            return;
                        }
                    }
                    Ok(Command::BootstrapStep) => {
                        if let Err(e) = circuit.transaction() {
                            if status_sender.send(Err(e)).is_err() {
                                return;
                            }
                        } else if status_sender
                            .send(Ok(Response::BootstrapComplete(
                                circuit.is_replay_complete(),
                            )))
                            .is_err()
                        {
                            return;
                        };
                    }
                    Ok(Command::CompleteBootstrap) => {
                        let status = circuit.complete_replay().map(|_| Response::Unit);
                        if status_sender.send(status).is_err() {
                            return;
                        }
                    }
                    Ok(Command::EnableProfiler) => {
                        profiler.enable_cpu_profiler();
                        // Send response.
                        if status_sender.send(Ok(Response::Unit)).is_err() {
                            return;
                        }
                    }
                    Ok(Command::DumpProfile { runtime_elapsed }) => {
                        if status_sender
                            .send(Ok(Response::ProfileDump(
                                profiler.dump_profile(runtime_elapsed),
                            )))
                            .is_err()
                        {
                            return;
                        }
                    }
                    Ok(Command::RetrieveGraph) => {
                        // This is implemented by just asking the profiler to dump the graph
                        // without any metadata.
                        if status_sender
                            .send(Ok(Response::ProfileDump(profiler.dump_graph())))
                            .is_err()
                        {
                            return;
                        }
                    }
                    Ok(Command::RetrieveProfile { runtime_elapsed }) => {
                        if status_sender
                            .send(Ok(Response::Profile(profiler.profile(runtime_elapsed))))
                            .is_err()
                        {
                            return;
                        }
                    }
                    Ok(Command::Checkpoint(base)) => {
                        let mut files = Vec::new();
                        let response = circuit
                            .checkpoint(&base, &mut files)
                            .map(|_| Response::CheckpointCreated(files));
                        if status_sender.send(response).is_err() {
                            return;
                        }
                    }
                    Ok(Command::Restore(base)) => {
                        let result = circuit.restore(&base).map(Response::CheckpointRestored);
                        if status_sender.send(result).is_err() {
                            return;
                        }
                    }
                    Ok(Command::GetLir) => {
                        let lir = circuit.lir();
                        if status_sender.send(Ok(Response::Lir(lir))).is_err() {
                            return;
                        }
                    }
                    Ok(Command::SetBalancerHints(hints)) => {
                        let results = hints
                            .into_iter()
                            .map(|(global_node_id, hint)| {
                                circuit.set_balancer_hint(&global_node_id, hint)
                            })
                            .collect::<Vec<Result<(), DbspError>>>();
                        if status_sender
                            .send(Ok(Response::SetBalancerHints(results)))
                            .is_err()
                        {
                            return;
                        }
                    }
                    Ok(Command::GetCurrentBalancerPolicy) => {
                        let policy = circuit.get_current_balancer_policy();
                        if status_sender
                            .send(Ok(Response::CurrentBalancerPolicy(policy)))
                            .is_err()
                        {
                            return;
                        }
                    }
                    Ok(Command::Rebalance) => {
                        circuit.rebalance();
                        if status_sender.send(Ok(Response::Unit)).is_err() {
                            return;
                        }
                    }
                    // Nothing to do: do some housekeeping and relinquish the CPU if there's none
                    // left.
                    Err(TryRecvError::Empty) => {
                        parker.park();
                    }
                    Err(_) => {
                        break;
                    }
                }
            }
        })?;

        // Receive initialization status from all workers and take the return
        // value and fingerprint from the first worker, or the first error among
        // all the workers.
        //
        // The return value should return be the same in all workers (we don't
        // check).
        //
        // The fingerprint might differ between workers (even if the circuit is
        // really the same, because of differing effects of e.g. the "gather"
        // operator).
        let result = init_receivers
            .into_iter()
            .map(|receiver| {
                receiver.recv().unwrap_or_else(|_| {
                    Err(DbspError::Runtime(RuntimeError::WorkerPanic {
                        panic_info: runtime.collect_panic_info(),
                    }))
                })
            })
            .reduce(|old, new| {
                if old.is_ok() && new.is_err() {
                    new
                } else {
                    old
                }
            })
            .unwrap();
        let (ret, fingerprint) = match result {
            Err(error) => {
                let _ = runtime.kill();
                return Err(error);
            }
            Ok(result) => result,
        };

        let (backend, init_checkpoint) = config
            .storage
            .map(|storage| (storage.backend.clone(), storage.init_checkpoint))
            .unzip();
        let mut dbsp = DBSPHandle::new(
            backend,
            runtime,
            command_senders,
            status_receivers,
            fingerprint,
        )?;
        if let Some(init_checkpoint) = init_checkpoint.flatten() {
            dbsp.send_restore(init_checkpoint.to_string().into())?;
        }

        Ok((dbsp, ret))
    }
}

#[derive(Clone)]
enum Command {
    StartTransaction,
    Step,
    CommitTransaction,
    CommitProgress,
    Transaction,
    /// Execute a step in bootstrap mode.
    BootstrapStep,
    CompleteBootstrap,
    EnableProfiler,
    DumpProfile {
        runtime_elapsed: Duration,
    },
    /// Retrieve the circuit graph
    RetrieveGraph,
    RetrieveProfile {
        runtime_elapsed: Duration,
    },
    GetLir,
    Checkpoint(StoragePath),
    Restore(StoragePath),
    SetBalancerHints(Vec<(GlobalNodeId, BalancerHint)>),
    GetCurrentBalancerPolicy,
    Rebalance,
}

impl Debug for Command {
    fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result {
        match self {
            Command::StartTransaction => write!(f, "StartTransaction"),
            Command::Step => write!(f, "Step"),
            Command::CommitTransaction => write!(f, "CommitTransaction"),
            Command::CommitProgress => write!(f, "CommitProgress"),
            Command::Transaction => write!(f, "Transaction"),
            Command::BootstrapStep => write!(f, "BootstrapStep"),
            Command::CompleteBootstrap => write!(f, "CompleteBootstrap"),
            Command::EnableProfiler => write!(f, "EnableProfiler"),
            Command::RetrieveGraph => write!(f, "RetrieveGraph"),
            Command::DumpProfile { runtime_elapsed } => f
                .debug_struct("DumpProfile")
                .field("runtime_elapsed", runtime_elapsed)
                .finish(),
            Command::RetrieveProfile { runtime_elapsed } => f
                .debug_struct("RetrieveProfile")
                .field("runtime_elapsed", runtime_elapsed)
                .finish(),
            Command::GetLir => write!(f, "GetLir"),
            Command::Checkpoint(path) => f.debug_tuple("Checkpoint").field(path).finish(),
            Command::Restore(path) => f.debug_tuple("Restore").field(path).finish(),
            Command::SetBalancerHints(hints) => {
                f.debug_tuple("SetBalancerHints").field(hints).finish()
            }
            Command::GetCurrentBalancerPolicy => write!(f, "GetCurrentBalancerPolicy"),
            Command::Rebalance => write!(f, "Rebalance"),
        }
    }
}

#[derive(Debug)]
enum Response {
    Unit,
    CommitComplete(bool),
    BootstrapComplete(bool),
    CommitProgress(CommitProgress),
    ProfileDump(Graph),
    Profile(WorkerProfile),
    CheckpointCreated(Vec<Arc<dyn FileCommitter>>),
    CheckpointRestored(Option<BootstrapInfo>),
    Lir(LirCircuit),
    SetBalancerHints(Vec<Result<(), DbspError>>),
    CurrentBalancerPolicy(BTreeMap<GlobalNodeId, PartitioningPolicy>),
}

/// A handle to control the execution of a circuit in a multithreaded runtime.
#[derive(Debug)]
pub struct DBSPHandle {
    /// Time when the handle was created.
    start_time: Instant,

    /// Time elapsed while the circuit is executing a step, multiplied by the
    /// number of foreground and background threads.
    runtime_elapsed: Duration,

    /// The underlying runtime.
    ///
    /// Normally this will be some runtime, but we take it out if we need to
    /// kill the runtime.
    runtime: Option<RuntimeHandle>,

    /// Channels used to send commands to workers.
    command_senders: Vec<Sender<Command>>,

    /// Channels used to receive command completion status from workers.
    status_receivers: Vec<Receiver<Result<Response, DbspError>>>,

    /// For creating checkpoints, if we can.
    checkpointer: Option<Arc<Mutex<Checkpointer>>>,

    /// Circuit fingerprint.
    fingerprint: u64,

    /// Information about operators that participate in bootstrapping the new parts of the circuit.
    bootstrap_info: Option<BootstrapInfo>,
}
pub struct WorkersCommitProgress(BTreeMap<u16, CommitProgress>);

impl Default for WorkersCommitProgress {
    fn default() -> Self {
        Self::new()
    }
}

impl WorkersCommitProgress {
    pub fn new() -> Self {
        WorkersCommitProgress(BTreeMap::new())
    }

    pub fn insert(&mut self, worker_id: u16, progress: CommitProgress) {
        debug_assert!(!self.0.contains_key(&worker_id));
        self.0.insert(worker_id, progress);
    }

    pub fn summary(&self) -> CommitProgressSummary {
        let mut result = CommitProgressSummary::new();

        for worker_progress in self.0.values() {
            // println!(
            //     "{worker} in progress: {}",
            //     worker_progress
            //         .get_in_progress()
            //         .keys()
            //         .map(|k| k.to_string())
            //         .join(", ")
            // );
            result.merge(&worker_progress.summary());
        }

        result
    }
}

impl DBSPHandle {
    fn new(
        backend: Option<Arc<dyn StorageBackend>>,
        runtime: RuntimeHandle,
        command_senders: Vec<Sender<Command>>,
        status_receivers: Vec<Receiver<Result<Response, DbspError>>>,
        fingerprint: u64,
    ) -> Result<Self, DbspError> {
        // TODO: We allow the circuit to change between suspend and resume in Persistent mode;
        // we therefore only validate the fingerprint in ephemeral mode; however it can sometimes
        // be useful to validate it in persistent mode too, e.g., when recovering after a crash.
        // In that case, do we need to change the checkpointer logic to only validate the
        // fingerprint of the last checkpoint (it currently checks all checkpoints in the backend
        // directory)?
        let checkpointer = backend
            .map(|backend| {
                let checkpointer = Checkpointer::new(backend)?;
                if runtime.runtime().get_mode() == Mode::Ephemeral {
                    checkpointer.verify_fingerprint(fingerprint)?;
                };
                Ok::<_, DbspError>(checkpointer)
            })
            .transpose()?
            .map(|checkpointer| Arc::new(Mutex::new(checkpointer)));
        Ok(Self {
            start_time: Instant::now(),
            runtime: Some(runtime),
            command_senders,
            status_receivers,
            checkpointer,
            fingerprint,
            runtime_elapsed: Duration::ZERO,
            bootstrap_info: None,
        })
    }

    pub fn runtime(&self) -> &Runtime {
        self.runtime.as_ref().unwrap().runtime()
    }

    fn kill_inner(&mut self) -> ThreadResult<()> {
        self.command_senders.clear();
        self.status_receivers.clear();
        self.runtime.take().unwrap().kill()
    }

    fn kill_async(&mut self) {
        self.command_senders.clear();
        self.status_receivers.clear();
        self.runtime.take().unwrap().kill_async()
    }

    fn collect_panic_info(&self) -> Option<Vec<(usize, ThreadType, WorkerPanicInfo)>> {
        self.runtime
            .as_ref()
            .map(|runtime| runtime.collect_panic_info())
    }

    fn panicked(&self) -> bool {
        self.runtime
            .as_ref()
            .is_some_and(|runtime| runtime.panicked())
    }

    fn broadcast_command<F>(&mut self, command: Command, mut handler: F) -> Result<(), DbspError>
    where
        F: FnMut(usize, Response),
    {
        if self.runtime.is_none() {
            return Err(DbspError::Runtime(RuntimeError::Terminated));
        }

        // Send command.
        for (worker, sender) in self.command_senders.iter().enumerate() {
            if sender.send(command.clone()).is_err() {
                let panic_info = self.collect_panic_info().unwrap_or_default();

                // Worker thread panicked. Exit without waiting for all workers to exit
                // to avoid deadlocks due to workers waiting for each other.
                self.kill_async();
                return Err(DbspError::Runtime(RuntimeError::WorkerPanic { panic_info }));
            }
            self.runtime.as_ref().unwrap().unpark_worker(worker);
        }

        // Use `Select` to wait for responses from all workers simultaneously.
        // This way if one of the workers panics, leaving other workers waiting
        // for it in exchange operators, we won't deadlock waiting for these
        // workers.
        let mut select = Select::new();
        for receiver in self.status_receivers.iter() {
            select.recv(receiver);
        }

        fn handle_panic(this: &mut DBSPHandle) -> Result<(), DbspError> {
            // Retrieve panic info before killing the circuit.
            let panic_info = this.collect_panic_info().unwrap_or_default();
            this.kill_async();

            Err(DbspError::Runtime(RuntimeError::WorkerPanic { panic_info }))
        }

        // Receive responses.
        for _ in 0..self.status_receivers.len() {
            let ready = select.select();
            let worker = ready.index();

            match ready.recv(&self.status_receivers[worker]) {
                Err(_) => return handle_panic(self),
                Ok(Err(e)) => {
                    let _ = self.kill_inner();
                    return Err(e);
                }
                Ok(Ok(resp)) => handler(worker, resp),
            }
        }
        if self.panicked() {
            return handle_panic(self);
        }

        Ok(())
    }

    fn unicast_command(&mut self, worker: usize, command: Command) -> Result<Response, DbspError> {
        if self.runtime.is_none() {
            return Err(DbspError::Runtime(RuntimeError::Terminated));
        }

        // Send command.
        if self.command_senders[worker].send(command.clone()).is_err() {
            let panic_info = self.collect_panic_info().unwrap_or_default();

            // Worker thread panicked. Exit without waiting for all workers to exit
            // to avoid deadlocks due to workers waiting for each other.
            self.kill_async();
            return Err(DbspError::Runtime(RuntimeError::WorkerPanic { panic_info }));
        }
        self.runtime.as_ref().unwrap().unpark_worker(worker);

        let reply = match self.status_receivers[worker].recv() {
            Err(_) => return handle_panic(self),
            Ok(Err(e)) => {
                let _ = self.kill_inner();
                return Err(e);
            }
            Ok(Ok(resp)) => resp,
        };

        // Receive responses.
        fn handle_panic(this: &mut DBSPHandle) -> Result<Response, DbspError> {
            // Retrieve panic info before killing the circuit.
            let panic_info = this.collect_panic_info().unwrap_or_default();
            this.kill_async();

            Err(DbspError::Runtime(RuntimeError::WorkerPanic { panic_info }))
        }
        if self.panicked() {
            return handle_panic(self);
        }

        Ok(reply)
    }

    /// Start and instantly commit a transaction, waiting for the commit to complete.
    pub fn transaction(&mut self) -> Result<(), DbspError> {
        if self.bootstrap_in_progress() {
            self.step_bootstrap()
        } else {
            self.transaction_regular()
        }
    }

    /// Start a transaction.
    ///
    /// A transaction consists of a sequence of steps that evaluate a set of inputs for a single logical
    /// clock tick.
    ///
    /// Transaction lifecycle:
    ///
    /// ```text
    ///                              is_commit_complete() = true
    ///    ┌────────────────────────────────────────────────────────────────────────────────────┐
    ///    ▼                                                                                    │
    /// ┌───────┐      start_transaction()      ┌───────────┐ start_commit_transaction()  ┌─────┴────┐
    /// │ idle  ├──────────────────────────────►│in progress├────────────────────────────►│committing│
    /// └───────┘                               └────────┬──┘                             └─────────┬┘
    ///                                           ▲      │                                    ▲     │
    ///                                           └──────┘                                    └─────┘
    ///                                            step()                                      step()
    /// ```
    ///
    /// The value of the circuit's logical clock remains unchanged during the transaction.
    /// The clock advances between transactions.
    pub fn start_transaction(&mut self) -> Result<(), DbspError> {
        let start = Instant::now();
        let result = self.broadcast_command(Command::StartTransaction, |_, _| {});
        if let Some(handle) = self.runtime.as_ref() {
            self.runtime_elapsed +=
                start.elapsed() * handle.runtime().layout().local_workers().len() as u32 * 2;
        }
        result
    }

    /// Evaluate the circuit for a single step.
    ///
    /// In the `in progress` state of the transaction, this method always returns `false`.
    ///
    /// In the `committing` state, this method returns `true` when the commit is complete and
    /// the circuit has produced all outputs for the inputs received during the transaction.
    pub fn step(&mut self) -> Result<bool, DbspError> {
        let start = Instant::now();
        let mut commit_complete = Vec::with_capacity(self.status_receivers.len());

        let result = self.broadcast_command(Command::Step, |_worker, response| {
            let Response::CommitComplete(complete) = response else {
                panic!("Expected CommitComplete response, got {response:?}");
            };
            commit_complete.push(complete);
        });
        if let Some(handle) = self.runtime.as_ref() {
            self.runtime_elapsed +=
                start.elapsed() * handle.runtime().layout().local_workers().len() as u32 * 2;
        }

        result?;

        let commit_complete = commit_complete.iter().any(|complete| *complete);

        if commit_complete {
            debug!("Commit complete");
        }

        Ok(commit_complete)
    }

    /// Start committing the current transaction by forcing all operators to process
    /// their inputs to completion.
    ///
    /// The caller must invoke `step` repeatedly until the commit is complete.
    pub fn start_commit_transaction(&mut self) -> Result<(), DbspError> {
        let start = Instant::now();
        let result = self.broadcast_command(Command::CommitTransaction, |_, _| {});
        if let Some(handle) = self.runtime.as_ref() {
            self.runtime_elapsed +=
                start.elapsed() * handle.runtime().layout().local_workers().len() as u32 * 2;
        }
        result
    }

    /// Convenience method that calls `start_commit_transaction` and then repeatedly calls `step`
    /// until the commit is complete.
    pub fn commit_transaction(&mut self) -> Result<(), DbspError> {
        self.start_commit_transaction()?;

        loop {
            let commit_complete = self.step()?;
            if commit_complete {
                return Ok(());
            }
        }
    }

    /// Estimated commit progress.
    pub fn commit_progress(&mut self) -> Result<WorkersCommitProgress, DbspError> {
        let mut progress = WorkersCommitProgress::new();

        self.broadcast_command(Command::CommitProgress, |worker, response| {
            let Response::CommitProgress(worker_progress) = response else {
                panic!("Expected CommitProgress response, got {response:?}");
            };
            progress.insert(worker as u16, worker_progress);
        })?;

        Ok(progress)
    }

    pub fn set_replay_step_size(&mut self, step_size: usize) {
        if let Some(handle) = self.runtime.as_ref() {
            handle.runtime().set_replay_step_size(step_size);
        }
    }

    pub fn get_replay_step_size(&self) -> usize {
        if let Some(handle) = self.runtime.as_ref() {
            handle.runtime().get_replay_step_size()
        } else {
            0
        }
    }

    fn transaction_regular(&mut self) -> Result<(), DbspError> {
        DBSP_STEP.fetch_add(1, Ordering::Relaxed);
        let start = Instant::now();
        let result = self.broadcast_command(Command::Transaction, |_, _| {});
        DBSP_STEP_LATENCY_MICROSECONDS
            .lock()
            .unwrap()
            .record_elapsed(start);
        if let Some(handle) = self.runtime.as_ref() {
            self.runtime_elapsed +=
                start.elapsed() * handle.runtime().layout().local_workers().len() as u32 * 2;
        }
        result
    }

    /// In the bootstrap mode, after performing a step, check if all workers have finished bootstrapping.
    /// If so, notify all workers to exit the bootstrap phase and start normal operation.
    fn step_bootstrap(&mut self) -> Result<(), DbspError> {
        DBSP_STEP.fetch_add(1, Ordering::Relaxed);
        let start = Instant::now();

        let mut replay_complete = Vec::with_capacity(self.status_receivers.len());

        let result = self.broadcast_command(Command::BootstrapStep, |_worker, response| {
            let Response::BootstrapComplete(complete) = response else {
                panic!("Expected BootstrapComplete response, got {response:?}");
            };
            replay_complete.push(complete);
        });

        DBSP_STEP_LATENCY_MICROSECONDS
            .lock()
            .unwrap()
            .record_elapsed(start);
        if let Some(handle) = self.runtime.as_ref() {
            self.runtime_elapsed +=
                start.elapsed() * handle.runtime().layout().local_workers().len() as u32 * 2;
        }

        result?;

        if replay_complete.iter().all(|complete| *complete) {
            info!("Bootstrap complete");
            self.send_complete_bootstrap()?;
        }

        Ok(())
    }

    /// The circuit has been resumed from a checkpoint and is currently bootstrapping the modified part of the circuit.
    pub fn bootstrap_in_progress(&self) -> bool {
        self.bootstrap_info.is_some()
    }

    /// In the bootstrap mode, returns information about operators involved in bootstrapping.
    pub fn bootstrap_info(&self) -> &Option<BootstrapInfo> {
        &self.bootstrap_info
    }

    /// Returns the time elapsed while the circuit is executing a step,
    /// multiplied by the number of foreground and background threads.
    pub fn runtime_elapsed(&self) -> Duration {
        self.runtime_elapsed
    }

    /// Fingerprint of this circuit.
    pub fn fingerprint(&self) -> u64 {
        self.fingerprint
    }

    /// Reset circuit state to the point of the given Commit.
    ///
    /// If the circuit needs bootstrapping new operators, put it in the bootstrap mode.
    fn send_restore(&mut self, base: StoragePath) -> Result<(), DbspError> {
        let mut worker_replay_info = BTreeMap::<usize, Option<BootstrapInfo>>::new();

        self.broadcast_command(Command::Restore(base), |worker, resp| {
            let Response::CheckpointRestored(replay_info) = resp else {
                panic!("Expected checkpoint restore response, got {resp:?}");
            };
            worker_replay_info.insert(worker, replay_info);
        })?;

        // All workers should have the same replay info.
        for i in 1..worker_replay_info.len() {
            if worker_replay_info[&i] != worker_replay_info[&0] {
                let mut info = Vec::new();
                for j in 0..worker_replay_info.len() {
                    info.push(format!(
                        "  worker {j} replay info: {:?}",
                        worker_replay_info[&j]
                    ));
                }
                let info = info.join("\n");
                return Err(DbspError::Scheduler(SchedulerError::ReplayInfoConflict {
                    error: format!(
                        "worker 0 and worker {i} returned different replay info after restarting from a checkpoint; this can be caused by a bug or data corruption; replay info\n{info}"
                    ),
                }));
            }
        }

        self.bootstrap_info = worker_replay_info[&0].clone();

        if let Some(bootstrap_info) = &self.bootstrap_info {
            info!(
                "Circuit restored from checkpoint, bootstrapping new parts of the circuit: {bootstrap_info:?}"
            );
        }

        Ok(())
    }

    /// Notifies workers to exit the replay phase and start normal operation
    /// (during the replay phase only the operators involved in replay are scheduled).
    fn send_complete_bootstrap(&mut self) -> Result<(), DbspError> {
        self.broadcast_command(Command::CompleteBootstrap, |_, _| {})?;

        self.bootstrap_info = None;

        Ok(())
    }

    fn checkpointer(&self) -> Result<&Arc<Mutex<Checkpointer>>, DbspError> {
        self.checkpointer
            .as_ref()
            .ok_or(DbspError::Storage(StorageError::StorageDisabled))
    }

    /// Allows creating a new checkpoint by taking a consistent snapshot of the
    /// state in dbsp.
    pub fn checkpoint(&mut self) -> CheckpointBuilder<'_> {
        CheckpointBuilder::new(self)
    }

    /// List all currently available checkpoints.
    pub fn list_checkpoints(&mut self) -> Result<Vec<CheckpointMetadata>, DbspError> {
        self.checkpointer()?.lock().unwrap().list_checkpoints()
    }

    /// Remove the oldest checkpoint from the list.
    /// - Prevents removing checkpoints whose UUID is in `except`.
    ///
    /// # Returns
    /// - Uuid of the removed checkpoints, if any were removed.
    /// - An empty set if there were no checkpoints to remove.
    pub fn gc_checkpoint(
        &mut self,
        except: HashSet<uuid::Uuid>,
    ) -> Result<HashSet<uuid::Uuid>, DbspError> {
        self.checkpointer()?.lock().unwrap().gc_checkpoint(except)
    }

    /// Enable CPU profiler.
    ///
    /// Enable recording of CPU usage info.  When CPU profiling is enabled,
    /// [`Self::dump_profile`] outputs CPU usage info along with memory
    /// usage and other circuit metadata.  CPU profiling introduces small
    /// runtime overhead.
    pub fn enable_cpu_profiler(&mut self) -> Result<(), DbspError> {
        self.broadcast_command(Command::EnableProfiler, |_, _| {})
    }

    /// Dump profiling information to the specified directory.
    ///
    /// Creates `dir_path` if it doesn't exist.  For each worker thread, creates
    /// `dir_path/<timestamp>/<worker>.dot` file containing worker profile in
    /// the graphviz format.  If CPU profiling was enabled (see
    /// [`Self::enable_cpu_profiler`]), the profile will contain both CPU and
    /// memory usage information; otherwise only memory usage details are
    /// reported.
    pub fn dump_profile<P: AsRef<Path>>(&mut self, dir_path: P) -> Result<PathBuf, DbspError> {
        Ok(self.graph_profile()?.dump(dir_path)?)
    }

    /// Returns an array of worker profiles in graphviz `.dot` format.  Each
    /// array element corresponds to the profile of the corresponding worker.
    /// If CPU profiling was enabled (see [`Self::enable_cpu_profiler`]), the
    /// profile will contain both CPU and memory usage information; otherwise
    /// only memory usage details are reported.
    pub fn graph_profile(&mut self) -> Result<GraphProfile, DbspError> {
        let mut worker_graphs = vec![Default::default(); self.status_receivers.len()];
        self.broadcast_command(
            Command::DumpProfile {
                runtime_elapsed: self.runtime_elapsed(),
            },
            |worker, resp| {
                if let Response::ProfileDump(prof) = resp {
                    worker_graphs[worker] = prof;
                }
            },
        )?;
        Ok(GraphProfile {
            elapsed_time: self.start_time.elapsed(),
            worker_offset: self.runtime().layout().local_workers().start,
            worker_graphs,
        })
    }

    pub fn retrieve_profile(&mut self) -> Result<DbspProfile, DbspError> {
        let mut profiles = vec![Default::default(); self.status_receivers.len()];
        let mut graphs = vec![Default::default(); self.status_receivers.len()];

        self.broadcast_command(
            Command::RetrieveProfile {
                runtime_elapsed: self.runtime_elapsed(),
            },
            |worker, resp| {
                if let Response::Profile(prof) = resp {
                    profiles[worker] = prof;
                }
            },
        )?;

        self.broadcast_command(Command::RetrieveGraph, |worker, resp| {
            // They are all identical, but the command returns one per worker.
            if let Response::ProfileDump(graph) = resp {
                graphs[worker] = graph;
            }
        })?;

        Ok(DbspProfile::new(profiles, graphs.pop()))
    }

    pub fn lir(&mut self) -> Result<LirCircuit, DbspError> {
        let mut lirs = vec![Default::default(); self.status_receivers.len()];

        self.broadcast_command(Command::GetLir, |worker, resp| {
            if let Response::Lir(lir) = resp {
                lirs[worker] = lir;
            }
        })?;

        Ok(lirs.remove(0))
    }

    /// Terminate the execution of the circuit, exiting all worker threads.
    ///
    /// If one or more of the worker threads panics, returns the argument the
    /// `panic!` macro was called with (see `std::thread::Result`).
    ///
    /// This is the preferred way of killing a circuit.  Simply dropping the
    /// handle will have the same effect, but without reporting the error
    /// status.
    pub fn kill(mut self) -> ThreadResult<()> {
        if self.runtime.is_none() {
            return Ok(());
        }

        self.kill_inner()
    }

    pub fn set_balancer_hint(
        &mut self,
        global_node_id: &GlobalNodeId,
        hint: BalancerHint,
    ) -> Result<(), DbspError> {
        let mut result = self.set_balancer_hints(vec![(global_node_id.clone(), hint)])?;
        result.pop().unwrap()
    }

    pub fn set_balancer_hints(
        &mut self,
        hints: Vec<(GlobalNodeId, BalancerHint)>,
    ) -> Result<Vec<Result<(), DbspError>>, DbspError> {
        let mut results = Vec::new();

        self.broadcast_command(Command::SetBalancerHints(hints), |_, resp| {
            let Response::SetBalancerHints(worker_results) = resp else {
                panic!("Expected SetBalancerHints response, got {resp:?}");
            };
            results = worker_results;
        })?;

        Ok(results)
    }

    pub fn get_current_balancer_policy(
        &mut self,
    ) -> Result<BTreeMap<GlobalNodeId, PartitioningPolicy>, DbspError> {
        let resp = self.unicast_command(0, Command::GetCurrentBalancerPolicy)?;

        let Response::CurrentBalancerPolicy(policy) = resp else {
            panic!("Expected GetCurrentBalancerPolicy policy response, got {resp:?}");
        };
        Ok(policy)
    }

    pub fn rebalance(&mut self) -> Result<(), DbspError> {
        self.broadcast_command(Command::Rebalance, |_, _| {})?;
        Ok(())
    }
}

impl Drop for DBSPHandle {
    fn drop(&mut self) {
        if self.runtime.is_some() {
            let _ = self.kill_inner();
        }
    }
}

/// Checkpoint builder.
#[derive(Debug)]
pub struct CheckpointBuilder<'a> {
    handle: &'a mut DBSPHandle,
    name: Option<String>,
    steps: Option<u64>,
    processed_records: Option<u64>,
}

impl<'a> CheckpointBuilder<'a> {
    fn new(handle: &'a mut DBSPHandle) -> Self {
        Self {
            handle,
            name: None,
            steps: None,
            processed_records: None,
        }
    }

    /// Gives the checkpoint a name.
    pub fn with_name(self, name: impl Into<String>) -> Self {
        Self {
            name: Some(name.into()),
            ..self
        }
    }

    /// Adds `steps` to the checkpoint to be created.
    pub fn with_steps(self, steps: u64) -> Self {
        Self {
            steps: Some(steps),
            ..self
        }
    }

    /// Adds `processed_records` to the checkpoint to be created.
    pub fn with_processed_records(self, processed_records: u64) -> Self {
        Self {
            processed_records: Some(processed_records),
            ..self
        }
    }

    /// Prepares and commits the checkpoint.
    pub fn run(self) -> Result<CheckpointMetadata, DbspError> {
        self.prepare().and_then(CheckpointCommitter::commit)
    }

    /// Prepares the checkpoint and returns a committer that can be used to
    /// commit it later.
    pub fn prepare(self) -> Result<CheckpointCommitter, DbspError> {
        let checkpointer = self.handle.checkpointer()?.clone();
        let uuid = Uuid::now_v7();
        let checkpoint_dir = Checkpointer::checkpoint_dir(uuid);
        let mut readers = Vec::new();
        self.handle
            .broadcast_command(Command::Checkpoint(checkpoint_dir), |_worker, resp| {
                let Response::CheckpointCreated(r) = resp else {
                    panic!("Expected checkpoint response, got {resp:?}");
                };
                readers.push(r);
            })?;
        Ok(CheckpointCommitter {
            checkpointer,
            uuid,
            readers,
            fingerprint: self.handle.fingerprint,
            name: self.name,
            steps: self.steps,
            processed_records: self.processed_records,
        })
    }
}

/// Committer for a checkpoint.
pub struct CheckpointCommitter {
    checkpointer: Arc<Mutex<Checkpointer>>,
    uuid: Uuid,
    readers: Vec<Vec<Arc<dyn FileCommitter>>>,
    fingerprint: u64,
    name: Option<String>,
    steps: Option<u64>,
    processed_records: Option<u64>,
}

impl CheckpointCommitter {
    /// Commits the checkpoint.
    ///
    /// Committing a checkpoint ensures that its data is on stable storage.  It
    /// can run in the background while the circuit processes more steps.
    pub fn commit(self) -> Result<CheckpointMetadata, DbspError> {
        for reader in self.readers.into_iter().flatten() {
            reader.commit()?;
        }
        self.checkpointer.lock().unwrap().commit(
            self.uuid,
            self.fingerprint,
            self.name,
            self.steps,
            self.processed_records,
        )
    }
}

#[cfg(test)]
pub(crate) mod tests {
    use std::fs::{File, create_dir_all};
    use std::io;
    use std::path::Path;
    use std::time::Duration;
    use std::{fs, vec};

    use super::{CircuitStorageConfig, Mode};
    use crate::circuit::checkpointer::Checkpointer;
    use crate::circuit::dbsp_handle::DevTweaks;
    use crate::circuit::runtime::TOKIO_WORKER_INDEX;
    use crate::circuit::{CircuitConfig, Layout};
    use crate::dynamic::{ClonableTrait, DowncastTrait, DynData, Erase};
    use crate::operator::Generator;
    use crate::operator::TraceBound;
    use crate::storage::backend::StorageError;
    use crate::trace::BatchReaderFactories;
    use crate::utils::Tup2;
    use crate::{
        Circuit, DBSPHandle, Error as DbspError, IndexedZSetHandle, InputHandle, OrdZSet,
        OutputHandle, Runtime, RuntimeError, ZSetHandle, ZWeight,
    };
    use anyhow::anyhow;
    use feldera_buffer_cache::ThreadType;
    use feldera_types::config::{StorageCacheConfig, StorageConfig, StorageOptions};
    use tempfile::{TempDir, tempdir};
    use uuid::Uuid;

    // Panic during initialization in worker thread.
    #[test]
    fn test_panic_in_worker1() {
        test_panic_in_worker(1);
    }

    #[test]
    fn test_panic_in_worker4() {
        test_panic_in_worker(4);
    }

    fn test_panic_in_worker(nworkers: usize) {
        let res = Runtime::init_circuit(nworkers, |circuit| {
            if Runtime::worker_index() == 0 {
                panic!();
            }

            circuit.add_source(Generator::new(|| 5usize));
            Ok(())
        });

        if let DbspError::Runtime(err) = res.unwrap_err() {
            // println!("error: {err}");
            assert!(matches!(err, RuntimeError::WorkerPanic { .. }));
        } else {
            panic!();
        }
    }

    // TODO: initialization error in worker thread (the `constructor` closure
    // currently does not return an error).
    // TODO: panic/error during GC.

    // Panic in `Circuit::step`.
    #[test]
    fn test_step_panic1() {
        test_step_panic(1);
    }

    #[test]
    fn test_step_panic4() {
        test_step_panic(4);
    }

    fn test_step_panic(nworkers: usize) {
        let (mut handle, _) = Runtime::init_circuit(nworkers, |circuit| {
            circuit.add_source(Generator::new(|| {
                if Runtime::worker_index() == 0 {
                    panic!()
                } else {
                    5usize
                }
            }));
            Ok(())
        })
        .unwrap();

        if let DbspError::Runtime(err) = handle.transaction().unwrap_err() {
            // println!("error: {err}");
            matches!(err, RuntimeError::WorkerPanic { .. });
        } else {
            panic!();
        }
    }

    #[test]
    fn test_panic_no_deadlock() {
        let (mut handle, _) = Runtime::init_circuit(4, |circuit| {
            circuit.add_source(Generator::new(|| {
                if Runtime::worker_index() == 1 {
                    panic!()
                } else {
                    std::thread::sleep(Duration::MAX)
                }
            }));
            Ok(())
        })
        .unwrap();

        if let DbspError::Runtime(err) = handle.transaction().unwrap_err() {
            // println!("error: {err}");
            matches!(err, RuntimeError::WorkerPanic { .. });
        } else {
            panic!();
        }
    }

    /// Check that a panic in the tokio merger runtime is propagated to the client.
    #[test]
    fn test_panic_in_tokio_merger_runtime() {
        let (panic_tx, panic_rx) = std::sync::mpsc::channel();
        let (mut handle, _) = Runtime::init_circuit(1, move |circuit| {
            let (_stream, _input_handle) = circuit.add_input_map::<u64, u64, i64, _>(|v, u| {
                *v = ((*v as i64) + *u) as u64;
            });

            if Runtime::worker_index() == 0 {
                let runtime = Runtime::runtime().unwrap();
                let panic_tx = panic_tx.clone();
                runtime.tokio_merger_runtime().spawn(async move {
                    TOKIO_WORKER_INDEX
                        .scope(0, async move {
                            let _ = std::panic::catch_unwind(|| {
                                panic!("panic from tokio merger runtime task");
                            });
                            let _ = panic_tx.send(());
                        })
                        .await;
                });
            }

            Ok(())
        })
        .unwrap();

        panic_rx
            .recv_timeout(Duration::from_secs(5))
            .expect("timed out waiting for panic task to complete");

        if let DbspError::Runtime(err) = handle.transaction().unwrap_err() {
            println!("error: {err}");
            match err {
                RuntimeError::WorkerPanic { panic_info } => {
                    assert!(
                        panic_info
                            .iter()
                            .any(|(_worker, thread_type, _info)| *thread_type
                                == ThreadType::Background),
                        "expected WorkerPanic to include background worker panic info"
                    );
                }
                _ => panic!(),
            }
        } else {
            panic!();
        }
    }

    // Kill the runtime.
    #[test]
    fn test_kill1() {
        test_kill(1);
    }

    #[test]
    fn test_kill4() {
        test_kill(4);
    }

    fn test_kill(nworkers: usize) {
        let (mut handle, _) = Runtime::init_circuit(nworkers, |circuit| {
            circuit.add_source(Generator::new(|| 5usize));
            Ok(())
        })
        .unwrap();

        handle.enable_cpu_profiler().unwrap();
        handle.transaction().unwrap();
        handle
            .dump_profile(std::env::temp_dir().join("test_kill"))
            .unwrap();
        handle.kill().unwrap();
    }

    // Drop the runtime.
    #[test]
    fn test_drop1() {
        test_drop(1);
    }

    #[test]
    fn test_drop4() {
        test_drop(4);
    }

    fn test_drop(nworkers: usize) {
        let (mut handle, _) = Runtime::init_circuit(nworkers, |circuit| {
            circuit.add_source(Generator::new(|| 5usize));
            Ok(())
        })
        .unwrap();

        handle.transaction().unwrap();
    }

    #[test]
    fn test_failing_constructor() {
        match Runtime::init_circuit(4, |_circuit| Err::<(), _>(anyhow!("constructor failed"))) {
            Err(DbspError::Constructor(msg)) => assert_eq!(msg.to_string(), "constructor failed"),
            _ => panic!(),
        }
    }

    type CircuitHandle = (
        IndexedZSetHandle<i32, i32>,
        OutputHandle<OrdZSet<Tup2<i32, i32>>>,
        InputHandle<usize>,
    );

    fn mkcircuit(cconf: &CircuitConfig) -> Result<(DBSPHandle, CircuitHandle), DbspError> {
        Runtime::init_circuit(cconf, move |circuit| {
            let (stream, handle) = circuit.add_input_indexed_zset::<i32, i32>();
            let (sample_size_stream, sample_size_handle) = circuit.add_input_stream::<usize>();
            let map_factories = BatchReaderFactories::new::<Tup2<i32, i32>, (), ZWeight>();
            let sample_handle = stream
                .integrate_trace()
                .stream_sample_unique_key_vals(&sample_size_stream)
                .inner()
                .dyn_map(
                    &map_factories,
                    Box::new(|kinput, kv| {
                        let kinput: &DynData =
                            unsafe { kinput.downcast::<Tup2<i32, i32>>() }.erase();
                        kinput.clone_to(kv.split_mut().0);
                    }),
                )
                .typed()
                .output();
            Ok((handle, sample_handle, sample_size_handle))
        })
    }

    fn mkcircuit_different(
        cconf: &CircuitConfig,
    ) -> Result<(DBSPHandle, CircuitHandle), DbspError> {
        Runtime::init_circuit(cconf, move |circuit| {
            let map_factories = BatchReaderFactories::new::<Tup2<i32, i32>, (), ZWeight>();
            let (stream, handle) = circuit.add_input_indexed_zset::<i32, i32>();
            let (sample_size_stream, sample_size_handle) = circuit.add_input_stream::<usize>();
            let sample_handle = stream
                .integrate_trace()
                .stream_sample_unique_key_vals(&sample_size_stream)
                .inner()
                .dyn_map(
                    &map_factories,
                    Box::new(|kinput, kv| {
                        let kinput: &DynData =
                            unsafe { kinput.downcast::<Tup2<i32, i32>>() }.erase();
                        kinput.clone_to(kv.split_mut().0);
                    }),
                )
                .typed()
                .output();
            let _sample_handle2: OutputHandle<OrdZSet<Tup2<i32, i32>>> = stream
                .integrate_trace()
                .stream_sample_unique_key_vals(&sample_size_stream)
                .inner()
                .dyn_map(
                    &map_factories,
                    Box::new(|kinput, kv| {
                        let kinput: &DynData =
                            unsafe { kinput.downcast::<Tup2<i32, i32>>() }.erase();
                        kinput.clone_to(kv.split_mut().0);
                    }),
                )
                .typed()
                .output();
            Ok((handle, sample_handle, sample_size_handle))
        })
    }

    #[allow(clippy::type_complexity)]
    fn mkcircuit_with_bounds(
        cconf: &CircuitConfig,
    ) -> Result<(DBSPHandle, CircuitHandle), DbspError> {
        Runtime::init_circuit(cconf, move |circuit| {
            let (stream, handle) = circuit.add_input_indexed_zset::<i32, i32>();
            let (sample_size_stream, sample_size_handle) = circuit.add_input_stream::<usize>();
            let map_factories = BatchReaderFactories::new::<Tup2<i32, i32>, (), ZWeight>();
            let tb = TraceBound::new();
            tb.set(Box::new(10).erase_box());
            let sample_handle = stream
                .integrate_trace_with_bound(tb.clone(), tb)
                .stream_sample_unique_key_vals(&sample_size_stream)
                .inner()
                .dyn_map(
                    &map_factories,
                    Box::new(|kinput, kv| {
                        let kinput: &DynData =
                            unsafe { kinput.downcast::<Tup2<i32, i32>>() }.erase();
                        kinput.clone_to(kv.split_mut().0);
                    }),
                )
                .typed()
                .output();
            Ok((handle, sample_handle, sample_size_handle))
        })
    }

    pub(crate) fn mkconfig() -> (TempDir, CircuitConfig) {
        let temp = tempdir().expect("Can't create temp dir for storage");
        let cconf = CircuitConfig {
            layout: Layout::new_solo(1),
            max_rss_bytes: None,
            mode: Mode::Ephemeral,
            pin_cpus: Vec::new(),
            storage: Some(
                CircuitStorageConfig::for_config(
                    StorageConfig {
                        path: temp.path().to_string_lossy().into_owned(),
                        cache: StorageCacheConfig::default(),
                    },
                    StorageOptions {
                        min_storage_bytes: Some(0),
                        ..StorageOptions::default()
                    },
                )
                .unwrap(),
            ),
            dev_tweaks: DevTweaks::default(),
        };
        (temp, cconf)
    }

    /// Utility function that runs a circuit and takes a checkpoint at every
    /// step. It then restores the circuit to every checkpoint and checks that
    /// the state is consistent with what we would expect it to be at that
    /// point.
    fn generic_checkpoint_restore(
        input: Vec<Vec<Tup2<i32, Tup2<i32, i64>>>>,
        circuit_fun: fn(&CircuitConfig) -> Result<(DBSPHandle, CircuitHandle), DbspError>,
    ) {
        const SAMPLE_SIZE: usize = 25; // should be bigger than #keys
        assert!(input.len() < SAMPLE_SIZE, "input should be <SAMPLE_SIZE");
        let (_temp, mut cconf) = mkconfig();

        let mut committed = vec![];
        let mut checkpoints = vec![];

        // We create a circuit and push data into it, we also take a checkpoint at every
        // step.
        {
            let (mut dbsp, (input_handle, output_handle, sample_size_handle)) =
                circuit_fun(&cconf).unwrap();
            for mut batch in input.clone() {
                let cpm = dbsp.checkpoint().run().expect("commit shouldn't fail");
                checkpoints.push(cpm);

                sample_size_handle.set_for_all(SAMPLE_SIZE);
                input_handle.append(&mut batch);
                dbsp.transaction().unwrap();

                let res = output_handle.take_from_all();
                committed.push(res[0].clone());
            }
        }
        assert_eq!(committed.len(), input.len());

        // Next, we instantiate every checkpoint and make sure the circuit state is
        // what we would expect it to be at the given point we restored it to
        let mut batches_to_insert = input.clone();
        for (i, cpm) in checkpoints.iter().enumerate() {
            cconf.storage.as_mut().unwrap().init_checkpoint = Some(cpm.uuid);
            let (mut dbsp, (input_handle, output_handle, sample_size_handle)) =
                mkcircuit(&cconf).unwrap();
            sample_size_handle.set_for_all(SAMPLE_SIZE);
            input_handle.append(&mut batches_to_insert[i]);
            dbsp.transaction().unwrap();

            let res = output_handle.take_from_all();
            let expected_zset = committed[i].clone();
            assert_eq!(expected_zset, res[0]);
        }
    }

    /// Smoke test for `gather_batches_for_checkpoint`.
    #[test]
    fn can_find_batches_for_checkpoint() {
        let (_temp, cconf) = mkconfig();
        let (mut dbsp, (input_handle, _, _)) = mkcircuit(&cconf).unwrap();
        let mut batch = vec![Tup2(1, Tup2(2, 1))];
        input_handle.append(&mut batch);
        dbsp.transaction().unwrap();
        let cpm = dbsp.checkpoint().run().expect("commit failed");
        let batchfiles = dbsp
            .checkpointer
            .as_ref()
            .unwrap()
            .lock()
            .unwrap()
            .gather_batches_for_checkpoint(&cpm)
            .expect("failed to gather batches");
        assert_eq!(batchfiles.len(), 1);
    }

    /// If we call commit, we should preserve the checkpoint list across circuit
    /// restarts.
    #[test]
    fn checkpoint_file() {
        let (_temp, cconf) = mkconfig();

        {
            let (mut dbsp, (_input_handle, _output_handle, sample_size_handle)) =
                mkcircuit(&cconf).unwrap();
            sample_size_handle.set_for_all(2);
            dbsp.transaction().unwrap();
            dbsp.checkpoint()
                .with_name("test-commit")
                .run()
                .expect("commit failed");
            dbsp.transaction().unwrap();
            dbsp.checkpoint().run().expect("commit failed");
        }

        {
            let (dbsp, _) = mkcircuit(&cconf).unwrap();
            let cpm = &dbsp
                .checkpointer
                .as_ref()
                .unwrap()
                .lock()
                .unwrap()
                .list_checkpoints()
                .unwrap()[0];
            assert_ne!(cpm.uuid, Uuid::nil());
            assert_eq!(cpm.identifier, Some(String::from("test-commit")));

            let cpm2 = &dbsp
                .checkpointer
                .as_ref()
                .unwrap()
                .lock()
                .unwrap()
                .list_checkpoints()
                .unwrap()[1];
            assert_ne!(cpm2.uuid, Uuid::nil());
            assert_ne!(cpm2.uuid, cpm.uuid);
            assert_eq!(cpm2.identifier, None);
        }
    }

    /// We should fail if we instantiate a circuit with the same storage
    /// directory twice because the directory is locked.
    #[test]
    fn circuit_takes_ownership_of_storage_dir() {
        let (_temp, cconf) = mkconfig();
        let (_dbsp, _) = mkcircuit(&cconf).unwrap();

        let r = Runtime::init_circuit(cconf, |_circuit| Ok(()));
        assert!(matches!(
            r,
            Err(DbspError::Storage(StorageError::StorageLocked(_, _)))
        ));
    }

    /// We should fail if we revert to a checkpoint that doesn't exist.
    #[test]
    fn revert_to_unknown_checkpoint() {
        let (_temp, mut cconf) = mkconfig();
        let (dbsp, _) = mkcircuit(&cconf).unwrap();
        drop(dbsp); // makes sure we can take ownership of storage dir again

        cconf.storage.as_mut().unwrap().init_checkpoint = Some(Uuid::now_v7()); // this checkpoint doesn't exist

        let res = mkcircuit(&cconf);
        let Err(err) = res else {
            panic!("revert_to_unknown_checkpoint is supposed to fail");
        };

        assert!(matches!(
            err,
            DbspError::Storage(StorageError::CheckpointNotFound(_))
        ));
    }

    /// We panic if we initialize to a partially incomplete checkpoint.
    #[test]
    #[should_panic]
    fn revert_to_partial_checkpoint() {
        let (temp, mut cconf) = mkconfig();
        let (dbsp, _) = mkcircuit(&cconf).unwrap();
        drop(dbsp); // makes sure we can take ownership of storage dir again

        let init_checkpoint = Uuid::now_v7(); // A made-up checkpoint, that does not have the necessary files
        cconf.storage.as_mut().unwrap().init_checkpoint = Some(init_checkpoint);
        let checkpoint_dir = temp.path().join(init_checkpoint.to_string());
        create_dir_all(checkpoint_dir).expect("can't create checkpoint dir");

        // Initializing this circuit again will panic because it won't find the
        // necessary files in the checkpoint directory.
        mkcircuit(&cconf).unwrap();
    }

    fn init_test_tracing() {
        let _ = tracing_subscriber::fmt::try_init();
    }

    /// Checks that we end up cleaning old checkpoints on disk after calling
    /// `gc_checkpoint`.
    #[test]
    fn gc_commits() {
        init_test_tracing();
        let (temp, cconf) = mkconfig();

        fn count_directory_entries<P: AsRef<Path>>(path: P) -> io::Result<usize> {
            let mut file_count = 0;
            let entries = fs::read_dir(path)?;
            for entry in entries {
                let _entry = entry?;
                file_count += 1;
            }
            Ok(file_count)
        }

        let (mut dbsp, (input_handle, _, _)) = mkcircuit(&cconf).unwrap();

        let _cpm = dbsp.checkpoint().run().expect("commit failed");
        let mut batches: Vec<Vec<Tup2<i32, Tup2<i32, i64>>>> = vec![
            vec![Tup2(1, Tup2(2, 1))],
            vec![Tup2(2, Tup2(3, 1))],
            vec![Tup2(3, Tup2(4, 1))],
            vec![Tup2(3, Tup2(4, 1))],
            vec![Tup2(1, Tup2(2, 1))],
            vec![Tup2(2, Tup2(3, 1))],
            vec![Tup2(3, Tup2(4, 1))],
            vec![Tup2(3, Tup2(4, 1))],
        ];
        for chunk in batches.chunks_mut(2) {
            input_handle.append(&mut chunk[0]);
            input_handle.append(&mut chunk[1]);
            dbsp.transaction().unwrap();
            let _cpm = dbsp.checkpoint().run().expect("commit failed");
        }

        let prev_count = count_directory_entries(temp.path()).unwrap();
        let num_checkpoints = dbsp.list_checkpoints().unwrap().len();

        assert!(num_checkpoints > Checkpointer::MIN_CHECKPOINT_THRESHOLD);

        // Only MIN_CHECKPONT_THRESHOLD checkpoints will be kept.
        let _r = dbsp.gc_checkpoint(std::collections::HashSet::new());
        let count = count_directory_entries(temp.path()).unwrap();
        assert!(count < prev_count);
        assert!(dbsp.list_checkpoints().unwrap().len() <= Checkpointer::MIN_CHECKPOINT_THRESHOLD);
    }

    /// Make sure that leftover files from uncompleted checkpoints that were
    /// written during a previous run are cleaned up when we start a new
    /// circuit with this storage directory.
    #[test]
    fn gc_on_startup() {
        init_test_tracing();

        let (temp, cconf) = mkconfig();
        let (mut dbsp, (input_handle, _, _)) = mkcircuit(&cconf).unwrap();

        let mut batch: Vec<Tup2<i32, Tup2<i32, i64>>> = vec![Tup2(1, Tup2(2, 1))];
        input_handle.append(&mut batch);
        dbsp.checkpoint().run().expect("commit shouldn't fail");
        drop(dbsp);

        let incomplete_batch_path = temp.path().join("incomplete_batch.mut");
        let _ = File::create(&incomplete_batch_path).expect("can't create file");

        let incomplete_checkpoint_dir = temp.path().join(Uuid::now_v7().to_string());
        fs::create_dir(&incomplete_checkpoint_dir).expect("can't create checkpoint dir");
        let _ = File::create(incomplete_checkpoint_dir.join("filename.feldera"))
            .expect("can't create file");

        let complete_batch_unused = temp.path().join("complete_batch.feldera");
        let _ = File::create(&complete_batch_unused).expect("can't create file");

        // Initializing this circuit again will remove the
        // unnecessary files in the checkpoint directory.
        let (_dbsp, _) = mkcircuit(&cconf).unwrap();

        assert!(!incomplete_checkpoint_dir.exists());
        assert!(!incomplete_batch_path.exists());
        assert!(!complete_batch_unused.exists());
    }

    /// Make sure we can take checkpoints of a simple spine and restore them.
    #[test]
    fn commit_restore() {
        init_test_tracing();
        let batches: Vec<Vec<Tup2<i32, Tup2<i32, i64>>>> = vec![
            vec![Tup2(1, Tup2(2, 1))],
            vec![Tup2(3, Tup2(4, 1))],
            vec![Tup2(5, Tup2(6, 1))],
            vec![Tup2(7, Tup2(8, 1))],
            vec![Tup2(9, Tup2(10, 1))],
        ];

        generic_checkpoint_restore(batches, mkcircuit);
    }

    // TODO: fix this test. The circuit uses integrate_trace_with_bounds,
    // which applied bounds lazily; however the correctness check assumes
    // that bounds are applied instantly.
    /// Make sure we can take checkpoints of a spine with a trace bound and
    /// restore them.
    #[test]
    #[ignore]
    fn commit_restore_bounds() {
        init_test_tracing();
        let batches: Vec<Vec<Tup2<i32, Tup2<i32, i64>>>> = vec![
            vec![Tup2(1, Tup2(2, 1))],
            vec![Tup2(7, Tup2(8, 1))],
            vec![Tup2(9, Tup2(10, 1))],
            vec![Tup2(12, Tup2(12, 1))],
            vec![Tup2(13, Tup2(13, 1))],
        ];

        generic_checkpoint_restore(batches, mkcircuit_with_bounds);
    }

    /// Make sure two circuits end up with a different fingerprint.
    #[test]
    fn fingerprint_is_different() {
        let (_tempdir, cconf) = mkconfig();
        let fid1 = mkcircuit(&cconf).unwrap().0.fingerprint();
        let fid2 = mkcircuit_different(&cconf).unwrap().0.fingerprint();
        assert_ne!(fid1, fid2);

        // Unfortunately, the fingerprint isn't perfect, e.g., it thinks these two
        // circuits are the same:
        let fid3 = mkcircuit_with_bounds(&cconf).unwrap().0.fingerprint();
        assert_eq!(fid1, fid3); // Ideally, should be assert_ne
    }

    /// Make sure if we create a new circuit with a different fingerprint in the
    /// same storage directory we don't allow it to start.
    #[test]
    #[should_panic]
    fn reject_different_fingerprint() {
        let (_temp, mut cconf) = mkconfig();
        let (mut dbsp, (input_handle, _, _)) = mkcircuit(&cconf).unwrap();
        let mut batch: Vec<Tup2<i32, Tup2<i32, i64>>> = vec![Tup2(1, Tup2(2, 1))];
        input_handle.append(&mut batch);
        let cpi = dbsp.checkpoint().run().expect("commit shouldn't fail");
        drop(dbsp);

        cconf.storage.as_mut().unwrap().init_checkpoint = Some(cpi.uuid);
        let (dbsp_different, (_input_handle, _, _sample_size_handle)) =
            mkcircuit_different(&cconf).unwrap();
        drop(dbsp_different);
    }

    /// This test exercises the checkpoint/restore path of the Z1 operator.
    #[test]
    #[allow(clippy::borrowed_box)]
    fn test_z1_checkpointing() {
        let (_temp, mut cconf) = mkconfig();

        //let expected_waterlines = vec![115, 115, 125, 145];
        let expected_waterlines = vec![115, 115, 125, 145];
        fn mkcircuit(
            cconf: &CircuitConfig,
            mut expected_waterline: vec::IntoIter<i32>,
        ) -> (DBSPHandle, ZSetHandle<i32>) {
            Runtime::init_circuit(cconf, move |circuit| {
                let (stream, handle) = circuit.add_input_zset();
                stream
                    .waterline_monotonic(|| 0, |ts| ts + 5)
                    .inner_data()
                    .inspect(move |waterline: &Box<DynData>| {
                        if Runtime::worker_index() == 0 {
                            assert_eq!(
                                waterline.downcast_checked::<i32>(),
                                &expected_waterline.next().unwrap()
                            );
                        }
                    });
                Ok(handle)
            })
            .unwrap()
        }

        let batches = vec![
            vec![Tup2(100, 1), Tup2(110, 1), Tup2(50, 1)],
            vec![Tup2(90, 1), Tup2(90, 1), Tup2(50, 1)],
            vec![Tup2(110, 1), Tup2(120, 1), Tup2(100, 1)],
            vec![Tup2(130, 1), Tup2(140, 1), Tup2(0, 1)],
        ];

        for (idx, mut batch) in batches.into_iter().enumerate() {
            let expected_waterlines = expected_waterlines.clone();
            let expected_waterlines: Vec<i32> = expected_waterlines[idx..].into();
            let (mut dbsp, input_handle) = mkcircuit(&cconf, expected_waterlines.into_iter());
            input_handle.append(&mut batch);
            dbsp.transaction().unwrap();
            let cpm = dbsp.checkpoint().run().unwrap();
            cconf.storage.as_mut().unwrap().init_checkpoint = Some(cpm.uuid);
            dbsp.kill().unwrap();
        }
    }
}