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
//! Implementation of a [`Trace`] which merges batches in the background.
//!
//! This is a "spine", a [`Trace`] that internally consists of a vector of
//! batches. Inserting a new batch appends to the vector, and iterating or
//! searching a spine iterates or searches all of the batches in the vector.
//! The cost of these operations grows with the number of batches in the vector,
//! so it is beneficial to reduce the number by merging batches.

use crate::{
    Error, NumEntries, Runtime,
    circuit::{
        max_level0_batch_size_records,
        metadata::{
            BLOOM_FILTER_BITS_PER_KEY, BLOOM_FILTER_HIT_RATE_PERCENT, BLOOM_FILTER_HITS_COUNT,
            BLOOM_FILTER_MISSES_COUNT, BLOOM_FILTER_SIZE_BYTES, COMPLETED_MERGES,
            LOOSE_BATCHES_COUNT, LOOSE_MEMORY_RECORDS_COUNT, LOOSE_STORAGE_RECORDS_COUNT,
            MERGE_BACKPRESSURE_WAIT_TIME_SECONDS, MERGE_REDUCTION_PERCENT, MERGING_BATCHES_COUNT,
            MERGING_MEMORY_RECORDS_COUNT, MERGING_SIZE_BYTES, MERGING_STORAGE_RECORDS_COUNT,
            MetaItem, MetricId, MetricReading, NEGATIVE_WEIGHT_COUNT, OperatorMeta,
            RANGE_FILTER_HIT_RATE_PERCENT, RANGE_FILTER_HITS_COUNT, RANGE_FILTER_MISSES_COUNT,
            RANGE_FILTER_SIZE_BYTES, SPINE_BATCHES_COUNT, SPINE_STORAGE_SIZE_BYTES,
        },
        metrics::COMPACTION_STALL_TIME_NANOSECONDS,
        negative_weight_multiplier,
        runtime::{TOKIO_BUFFER_CACHE, TOKIO_WORKER_INDEX},
    },
    dynamic::{DynVec, Factory, Weight},
    samply::SamplySpan,
    storage::{
        buffer_cache::{BufferCache, CacheStats},
        filter_stats::FilterStats,
    },
    time::Timestamp,
    trace::{
        Batch, BatchReader, BatchReaderFactories, Builder, Cursor, Filter, GroupFilter, Trace,
        cursor::{CursorList, Position},
        merge_batches,
        ord::fallback::pick_insert_destination,
        spine_async::{
            list_merger::ArcListMerger, push_merger::ArcPushMerger, snapshot::FetchList,
        },
    },
};

use crate::storage::file::{Deserializer, to_bytes};
use crate::trace::CommittedSpine;
use enum_map::EnumMap;
use feldera_buffer_cache::ThreadType;
use feldera_storage::{
    FileCommitter, StoragePath,
    fbuf::slab::{FBufSlabs, TOKIO_FBUF_SLABS},
};
use feldera_types::memory_pressure::MemoryPressure;
use feldera_types::{checkpoint::PSpineBatches, config::dev_tweaks::MergerType};
use ouroboros::self_referencing;
use rand::Rng;
use rkyv::{Archive, Archived, Deserialize, Fallible, Serialize, ser::Serializer};
use size_of::{Context, HumanBytes, SizeOf};
use std::{
    borrow::Cow,
    future::Future,
    sync::{Arc, MutexGuard, atomic::AtomicIsize},
};
use std::{
    collections::BTreeMap,
    time::{Duration, Instant},
};
use std::{collections::VecDeque, sync::atomic::Ordering};
use std::{
    fmt::{self, Debug, Display, Formatter},
    ops::DerefMut,
    sync::Condvar,
};
use std::{ops::RangeInclusive, sync::Mutex};
use textwrap::indent;
use tokio::{sync::Notify, task::yield_now};
mod index_set;
mod list_merger;
mod push_merger;
mod snapshot;
pub use snapshot::{BatchReaderWithSnapshot, SpineSnapshot, WithSnapshot};

use super::{BatchLocation, cursor::CursorFactory};

pub use list_merger::ListMerger;

/// Maximum amount of levels in the spine.
pub(crate) const MAX_LEVELS: usize = 9;

static LEVEL_NAMES: [&str; MAX_LEVELS] = [
    "merge-0", "merge-1", "merge-2", "merge-3", "merge-4", "merge-5", "merge-6", "merge-7",
    "merge-8",
];

/// Default maximum batch size in records for level 0 merges.
///
/// This value must be greater than SPLITTER_OUTPUT_CHUNK_SIZE and DEFAULT_MAX_WORKER_BATCH_SIZE, so that typical batches
/// generated by operators or ingested from connectors fall into level 0.
///
/// This is important because our current merge strategy uses level 0 arrival rate as an estimate of the total arrival
/// rate for the spine. This is something we should fix more systematically.
///
/// Configurable via `dev_tweaks.max_level0_batch_size_records`.
pub(crate) const MAX_LEVEL0_BATCH_SIZE_RECORDS: u16 = 14_999;

fn scope_tokio_merger_locals<F>(
    worker_index: usize,
    buffer_cache: Arc<BufferCache>,
    slab_allocator: Arc<FBufSlabs>,
    future: F,
) -> impl Future<Output = F::Output>
where
    F: Future,
{
    TOKIO_WORKER_INDEX.scope(
        worker_index,
        TOKIO_BUFFER_CACHE.scope(buffer_cache, TOKIO_FBUF_SLABS.scope(slab_allocator, future)),
    )
}

impl<B: Batch + Send + Sync> From<(Vec<String>, &Spine<B>)> for CommittedSpine {
    fn from((batches, spine): (Vec<String>, &Spine<B>)) -> Self {
        CommittedSpine {
            batches,
            merged: Vec::new(),
            effort: 0,
            dirty: spine.dirty,
        }
    }
}

/// A group of batches with similar sizes (as determined by [size_from_level]).
#[derive(Clone, SizeOf)]
struct Slot<B>
where
    B: Batch,
{
    /// Optionally, a list of batches that are currently being merged.  These
    /// batches are not in `loose_batches`.
    ///
    /// Invariant: the batches (if present) must be non-empty.
    merging_batches: Option<Vec<Arc<B>>>,

    /// Zero or more batches not currently being merged.
    ///
    /// Invariant: the batches must be non-empty.
    loose_batches: VecDeque<Arc<B>>,

    /// Amount of time spent merging batches at this level.
    elapsed: Duration,

    /// Number of completed merges at this level.
    n_merged: usize,

    /// Number of batches input to the completed merges.
    n_merged_batches: usize,

    /// Number of merge steps required to complete the merges so far.
    n_steps: usize,

    /// Wake up the task that handles merges at this level.
    #[size_of(skip)]
    notify: Arc<Notify>,
}

impl<B> Default for Slot<B>
where
    B: Batch,
{
    fn default() -> Self {
        Self {
            merging_batches: None,
            loose_batches: VecDeque::new(),
            elapsed: Duration::ZERO,
            n_merged: 0,
            n_merged_batches: 0,
            n_steps: 0,
            notify: Arc::new(Notify::new()),
        }
    }
}

impl<B> Slot<B>
where
    B: Batch,
{
    /// If this slot doesn't currently have an ongoing merge, and it does have
    /// at least MERGE_COUNTS[level].start() loose batches, picks an upper limit
    /// of the loose batches and makes them into merging batches, and returns
    /// those batches. Otherwise, returns `None` without changing anything.
    ///
    /// We merge the least recently added batches (ensuring that batches
    /// eventually get merged).
    fn try_start_merge(&mut self, level: usize) -> Option<Vec<Arc<B>>> {
        /// Minimum and maximum numbers of batches to merge at each level.
        ///
        /// The minimum number of batches to merge is key to performance.  The
        /// maximum number seems much less important.
        const MERGE_COUNTS: [RangeInclusive<usize>; MAX_LEVELS] = [
            8..=64,
            8..=64,
            3..=64,
            3..=64,
            3..=64,
            3..=64,
            2..=64,
            2..=64,
            2..=64,
        ];

        let merge_counts = &MERGE_COUNTS[level];

        // Start a merge if there is no ongoing merge and there are either enough loose batches to start a merge,
        // or we are under high memory pressure and there's at least one in-memory batch in this slot.
        if self.merging_batches.is_none()
            && (self.loose_batches.len() >= *merge_counts.start()
                || self.must_relieve_memory_pressure())
        {
            let n = std::cmp::min(*merge_counts.end(), self.loose_batches.len());
            let batches = self.loose_batches.drain(..n).collect::<Vec<_>>();
            self.merging_batches = Some(batches.clone());
            Some(batches)
        } else {
            None
        }
    }

    /// Returns true if the slot must relieve memory pressure, i.e., if the memory pressure level is >=high
    /// and there's at least one in-memory batch in this slot.
    /// In this case, we will initiate a new merge regardless of the number of loose batches.
    fn must_relieve_memory_pressure(&self) -> bool {
        if let Some(memory_pressure) = Runtime::memory_pressure() {
            memory_pressure >= MemoryPressure::High
                && self
                    .loose_batches
                    .iter()
                    .any(|batch| batch.location() == BatchLocation::Memory)
        } else {
            false
        }
    }

    /// Returns the number of batches in the slot, whether loose or merging.
    fn n_batches(&self) -> usize {
        self.all_batches().count()
    }

    /// Returns an iterator over all batches in the slot, whether loose or
    /// merging.
    fn all_batches(&self) -> impl Iterator<Item = &Arc<B>> {
        self.loose_batches
            .iter()
            .chain(self.merging_batches.iter().flatten())
    }
}

/// State shared between all tokio merger tasks and the main thread.
///
/// This shared state is accessed through a `Mutex`, which we try to hold for as
/// short a time as possible.
#[derive(SizeOf)]
struct SharedState<B>
where
    B: Batch,
{
    #[size_of(skip)]
    factories: B::Factories,
    #[size_of(skip)]
    key_filter: Option<Filter<B::Key>>,
    #[size_of(skip)]
    value_filter: Option<GroupFilter<B::Val>>,
    #[size_of(skip)]
    frontier: B::Time,
    slots: [Slot<B>; MAX_LEVELS],
    #[size_of(skip)]
    request_exit: bool,
    #[size_of(skip)]
    spine_stats: SpineStats,
    max_level0_batch_size_records: usize,
}

impl<B> SharedState<B>
where
    B: Batch,
{
    pub fn new(factories: &B::Factories) -> Self {
        let max_level0_batch_size_records = max_level0_batch_size_records() as usize;
        assert!(
            max_level0_batch_size_records > 0,
            "max_level0_batch_size_records must be greater than 0"
        );
        assert!(
            max_level0_batch_size_records <= 99_999,
            "max_level0_batch_size_records must be less than or equal to 99_999"
        );

        Self {
            factories: factories.clone(),
            key_filter: None,
            value_filter: None,
            frontier: B::Time::minimum(),
            slots: std::array::from_fn(|_| Slot::default()),
            request_exit: false,
            spine_stats: SpineStats::default(),
            max_level0_batch_size_records,
        }
    }

    /// Adds all of `batches` as (initially) loose batches.  They will be merged
    /// when the merger thread has a chance (although it might not be awake).
    ///
    /// # Arguments
    ///
    /// * `batches` - The batches to add.
    /// * `merge` - `true` if the batches were produced by a merge; `false` if
    ///   these are freshly added batches.
    fn add_batches(&mut self, batches: impl IntoIterator<Item = Arc<B>>, merge: bool) {
        for batch in batches {
            if !batch.is_empty() {
                self.add_batch(batch, merge);
            }
        }
    }

    /// Add `batch` as an (initially) loose batch, which will be merged when
    /// the merger thread has a chance (although it might not be awake).
    fn add_batch(&mut self, batch: Arc<B>, merge: bool) {
        debug_assert!(!batch.is_empty());
        let level = Spine::<B>::size_to_level(&batch, self.max_level0_batch_size_records, merge);
        self.slots[level].loose_batches.push_back(batch);
        self.slots[level].notify.notify_one();
    }

    fn should_apply_backpressure(&self) -> bool {
        const HIGH_THRESHOLD: usize = 128;
        self.slots
            .iter()
            .map(|s| s.loose_batches.len())
            .sum::<usize>()
            >= HIGH_THRESHOLD
    }

    fn should_relieve_backpressure(&self) -> bool {
        const LOWER_THRESHOLD: usize = 127;
        self.slots
            .iter()
            .map(|s| s.loose_batches.len())
            .sum::<usize>()
            <= LOWER_THRESHOLD
    }

    fn get_filters(&self) -> (Option<Filter<B::Key>>, Option<GroupFilter<B::Val>>) {
        (self.key_filter.clone(), self.value_filter.clone())
    }

    /// Gets a copy of all of the batches (whether loose or being merged).
    fn get_batches(&self) -> Vec<Arc<B>> {
        let mut batches = Vec::with_capacity(self.slots.iter().map(Slot::n_batches).sum());
        for slot in &self.slots {
            batches.extend(slot.all_batches().cloned());
        }
        batches
    }

    fn get_snapshot(&self) -> SpineSnapshot<B> {
        SpineSnapshot::with_batches(&self.factories, self.get_batches())
    }

    /// Removes the loose batches and returns them.  This ensures that the
    /// merger thread will not initiate any more merges.
    fn take_loose_batches(&mut self) -> Vec<Arc<B>> {
        let mut loose_batches =
            Vec::with_capacity(self.slots.iter().map(|slot| slot.loose_batches.len()).sum());
        for slot in &mut self.slots {
            loose_batches.extend(slot.loose_batches.drain(..));
        }
        loose_batches
    }

    /// Returns true if any merging work is currently going on.
    ///
    /// If this returns false, a new merge might still start without any further
    /// batches being submitted if there are enough loose batches.
    fn is_merging(&self) -> bool {
        self.slots.iter().any(|slot| slot.merging_batches.is_some())
    }

    /// Finishes up the ongoing merge at the given `level`, which completed in
    /// `elapsed` time over `n_steps` steps, with `new_batch` as the result.
    fn merge_complete(
        &mut self,
        level: usize,
        new_batch: Arc<B>,
        start: Instant,
        elapsed: Duration,
        n_steps: usize,
    ) {
        let slot = &mut self.slots[level];
        let batches = slot.merging_batches.take().unwrap();
        slot.n_merged += 1;
        slot.n_merged_batches += batches.len();
        slot.elapsed += elapsed;
        slot.n_steps += n_steps;
        let cache_stats = batches.iter().fold(CacheStats::default(), |stats, batch| {
            stats + batch.cache_stats()
        });
        let pre_len = batches.iter().map(|b| b.len()).sum();
        let post_len = new_batch.len();
        self.spine_stats
            .report_merge(pre_len, post_len, cache_stats);
        SamplySpan::new(LEVEL_NAMES[level])
            .with_category("Spine")
            .with_start(start)
            .with_tooltip(|| {
                format!(
                    "Merged {} batches ({pre_len} -> {post_len}) in {n_steps} steps using {:.1} ms CPU",
                    batches.len(),
                    elapsed.as_secs_f64() * 1000.0
                )
            })
            .record();
        self.add_batches([new_batch], true);
    }

    /// Returns a copy of the data that the caller can use to construct a
    /// metadata report.
    ///
    /// This is better than constructing the report here directly, because part
    /// of that is measuring the size of the batches, which can require I/O.
    fn metadata_snapshot(&self) -> ([Slot<B>; MAX_LEVELS], SpineStats) {
        (self.slots.clone(), self.spine_stats.clone())
    }
}

/// A fully asynchronous merger.
///
/// Merging has two benefits:
///
/// 1. To make iteration and searching cheaper because we have fewer batches to
///    iterate and search in our CursorLists.
///
/// 2. In some cases only, to reduce the total amount of data, because weights
///    summarize multiple items or cancel each other out or because filters drop
///    data.
///
/// We only get these benefits when we complete a merge. An ongoing but
/// incomplete merge only has a cost (the CPU we invested in it, plus memory or
/// storage), with no benefits. Therefore, it is to our benefit to both complete
/// as many merges as possible and to have as few ongoing merges as possible.
///
/// Since the smallest merges are cheapest, one might conclude that we should
/// only do one merge at a time and that that should merge a few of the smallest
/// batches. In a static scenario, that would indeed be the right choice. But we
/// tend to be getting a new smallest batch on every step. With that, the
/// strategy of always doing the smallest merge piles up larger batches that
/// never get merged. For example, suppose that we add a new batch with size 1
/// in every step, we start merging the smallest runs whenever we first a merge,
/// and that a merge takes two steps. Then we end up with something like this,
/// where each line is a step that adds a new batch of size 1,` ()` designates
/// that batches are being merged, and `->` shows that a merge was finished or
/// started within the step:
///
/// ```text
/// 1
/// 1 1 -> (1 1)
/// 1 (1 1)
/// 1 1 (1 1) -> 1 1 2 -> 2 (1 1)
/// 1 2 (1 1)
/// 1 1 2 (1 1) -> 1 1 2 2 -> 2 2 (1 1)
/// 1 2 2 (1 1)
/// 1 1 2 2 (1 1) -> 1 1 2 2 2 -> 2 2 2 (1 1)
/// ````
///
/// The result is that we pile up runs that are slightly longer than the
/// shortest and they never get merged.
///
/// So, we still want to complete as many merges as possible and to have as few
/// ongoing merges as possible, but "as few ongoing merges as possible" needs to
/// be more than one and needs to include merges that are bigger than the
/// smallest possible merge.
///
/// The design of this merger does both. We divide batches into categories based
/// on their "level", which is roughly the base-10 log of their size (see
/// [Spine::size_to_level]). We run one merge per level at a time, only merging
/// batches in the same level with each other.  When a level contains more than
/// a pre-defined number of batches, we merge up to 64 batches at that level.
/// The result of a merge might be in the next higher level, ensuring that larger
/// merges eventually happen.
///
/// Merging work is performed in a separate tokio runtime, where we spawn a task
/// for each spine and each merge level. We use fuel to control the amount of work
/// performed during each activation of the task. After spending the fuel, the task
/// yields control to the tokio runtime, which will put it in the end of the run queue.
/// As a result the task's CPU share will be proportional to the amount of fuel allocated
/// to it.
struct AsyncMerger<B>
where
    B: Batch,
{
    /// State shared with the background thread.
    state: Arc<Mutex<SharedState<B>>>,

    /// Allows us to wait for the background worker until we no longer want to
    /// block for backpressure.
    no_backpressure: Arc<Condvar>,

    /// Allows us to wait for the background worker to become idle.
    idle: Arc<Condvar>,
}

/// Return value for a worker function.
enum WorkerStatus {
    /// The worker has more work to do (it only returned to allow other workers
    /// to run).
    Yield,

    /// The worker has no more work to do now, but it might have more later.
    Idle,

    /// The worker has exited.
    Done,
}

impl<B> AsyncMerger<B>
where
    B: Batch,
{
    fn new(factories: &B::Factories) -> Self {
        let idle = Arc::new(Condvar::new());
        let no_backpressure = Arc::new(Condvar::new());
        let state = Arc::new(Mutex::new(SharedState::new(factories)));
        let worker_state = Arc::new(WorkerState::default());

        for level in 0..MAX_LEVELS {
            let worker_state = worker_state.clone();
            let state = state.clone();
            let idle = idle.clone();
            let no_backpressure = no_backpressure.clone();

            let worker_index = Runtime::worker_index();
            let runtime = Runtime::runtime()
                .expect("Attempting to create a spine merger outside of a DBSP runtime");

            let buffer_cache =
                runtime.get_buffer_cache(Runtime::local_worker_offset(), ThreadType::Background);
            let slab_allocator = runtime
                .get_fbuf_slab_allocator(Runtime::local_worker_offset(), ThreadType::Background);
            let memory_pressure_notify = runtime.memory_pressure_notify();

            runtime.tokio_merger_runtime().spawn(async move {
                // Setup task-local variables for the tokio merger runtime.
                scope_tokio_merger_locals(worker_index, buffer_cache, slab_allocator, async move {
                    let state = Arc::clone(&state);
                    let idle = Arc::clone(&idle);
                    let no_backpressure = Arc::clone(&no_backpressure);
                    let mut merger = None;
                    let merger_type = Runtime::with_dev_tweaks(|tweaks| tweaks.merger());
                    let notify = state.lock().unwrap().slots[level].notify.clone();

                    loop {
                        let status = Self::run(
                            &worker_state,
                            level,
                            &mut merger,
                            merger_type,
                            &state,
                            &idle,
                            &no_backpressure,
                        );
                        match status {
                            // Fuel has been spent, but there's still work to do for the ongoing merge.
                            // Yield control to the tokio runtime, which will put the task in the end of the run queue.
                            WorkerStatus::Yield => yield_now().await,
                            // No more merge work currently available -- wait for the next merges to be started
                            // of for a global memory pressure notification.
                            WorkerStatus::Idle => {
                                tokio::select! {
                                    _ = notify.notified() => {}
                                    _ = memory_pressure_notify.notified() => {}
                                }
                            }
                            // The spine is being dropped -- exit the task.
                            WorkerStatus::Done => {
                                break;
                            }
                        }
                    }
                })
                .await;
            });
        }

        Self {
            state,
            idle,
            no_backpressure,
        }
    }
    fn set_key_filter(&self, key_filter: &Filter<B::Key>) {
        self.state.lock().unwrap().key_filter = Some(key_filter.clone());
    }
    fn set_value_filter(&self, value_filter: &GroupFilter<B::Val>) {
        self.state.lock().unwrap().value_filter = Some(value_filter.clone());
    }

    fn set_frontier(&self, frontier: &B::Time) {
        self.state.lock().unwrap().frontier = frontier.clone();
    }

    /// Adds `batch` to the shared merging state and wakes up the merger.
    fn add_batch(&self, batch: Arc<B>, merge: bool) {
        debug_assert!(!batch.is_empty());
        let mut state = self.state.lock().unwrap();
        state.add_batch(batch, merge);
        if state.should_apply_backpressure() {
            let start = Instant::now();
            let mut state = self.no_backpressure.wait(state).unwrap();
            state.spine_stats.backpressure_wait += start.elapsed();
            COMPACTION_STALL_TIME_NANOSECONDS
                .fetch_add(start.elapsed().as_nanos() as u64, Ordering::Relaxed);
            SamplySpan::new("backpressure-wait")
                .with_category("Spine")
                .with_start(start)
                .record();
        }
    }

    /// Adds `batches` to the shared merging state and wakes up the merger.
    fn add_batches(&self, batches: impl IntoIterator<Item = Arc<B>>, merge: bool) {
        self.state.lock().unwrap().add_batches(batches, merge);
    }

    /// Gets the complete set of batches to include in the spine.
    fn get_batches(&self) -> Vec<Arc<B>> {
        self.state.lock().unwrap().get_batches()
    }

    /// Pauses merging, by stopping the initiation of new merges and waiting for
    /// ongoing merges to finish.  Removes all of the batches from the merger
    /// and returns them. The caller can resume merging by passing those batches
    /// back to [Self::resume].
    fn pause(&self) -> Vec<Arc<B>> {
        let mut state = self.state.lock().unwrap();
        let mut batches = state.take_loose_batches();
        let mut state = self
            .idle
            .wait_while(state, |state| state.is_merging())
            .unwrap();
        batches.extend(state.take_loose_batches());
        batches
    }

    /// Stops the initiation of new merges.  Returns `(not_merging, merging)`,
    /// where `not_merging` is the batches that were not being merged and
    /// `merging` is the ones that were. Merging of batches in `merging` will
    /// continue, and further merges of the results of those merges could happen
    /// as well. The caller can re-insert `not_merging` later by passing it to
    /// [Self::resume].
    fn pause_new_merges(&self) -> (Vec<Arc<B>>, Vec<Arc<B>>) {
        let mut state = self.state.lock().unwrap();
        let not_merging = state.take_loose_batches();
        let merging = state.get_batches();
        (not_merging, merging)
    }

    /// Starts merging again with `batches`, which are presumably what
    /// [Self::pause] or [Self::pause_new_merges] returned.
    fn resume(&self, batches: impl IntoIterator<Item = Arc<B>>) {
        self.add_batches(batches, false);
    }

    fn metadata(&self, meta: &mut OperatorMeta) {
        fn class_batch_count_label(class: &str) -> MetricId {
            match class {
                "loose" => LOOSE_BATCHES_COUNT,
                "merging" => MERGING_BATCHES_COUNT,
                _ => panic!("invalid class: {class}"),
            }
        }

        fn class_tuple_count_label(class: &str, location: BatchLocation) -> MetricId {
            match (class, location) {
                ("loose", BatchLocation::Memory) => LOOSE_MEMORY_RECORDS_COUNT,
                ("loose", BatchLocation::Storage) => LOOSE_STORAGE_RECORDS_COUNT,
                ("merging", BatchLocation::Memory) => MERGING_MEMORY_RECORDS_COUNT,
                ("merging", BatchLocation::Storage) => MERGING_STORAGE_RECORDS_COUNT,
                _ => panic!("invalid class: {class} and location: {location:?}"),
            }
        }

        let (mut slots, spine_stats) = self.state.lock().unwrap().metadata_snapshot();

        // Construct per-slot occupancy description.
        for (index, slot) in slots.iter_mut().enumerate() {
            for (class, batches) in [
                ("loose", slot.loose_batches.make_contiguous() as &_),
                (
                    "merging",
                    slot.merging_batches
                        .as_ref()
                        .unwrap_or(&Vec::new())
                        .as_slice(),
                ),
            ] {
                if !batches.is_empty() {
                    let mut tuple_counts = EnumMap::<BatchLocation, usize>::default();
                    for batch in batches {
                        tuple_counts[batch.location()] += batch.len();
                    }

                    let mut facts = Vec::with_capacity(3);
                    facts.push(MetricReading::new(
                        class_batch_count_label(class),
                        vec![(Cow::Borrowed("slot"), index.to_string().into())],
                        MetaItem::Count(batches.len()),
                    ));
                    for (location, count) in tuple_counts {
                        if count > 0 {
                            facts.push(MetricReading::new(
                                class_tuple_count_label(class, location),
                                vec![(Cow::Borrowed("slot"), index.to_string().into())],
                                MetaItem::Count(count),
                            ));
                        }
                    }
                    meta.extend(facts);
                }
            }
            if slot.n_merged > 0 {
                meta.extend([MetricReading::new(
                    COMPLETED_MERGES,
                    vec![(Cow::Borrowed("slot"), index.to_string().into())],
                    MetaItem::Map(BTreeMap::from([
                        (Cow::Borrowed("merges"), MetaItem::Count(slot.n_merged)),
                        (
                            Cow::Borrowed("batches"),
                            MetaItem::Count(slot.n_merged_batches),
                        ),
                        (Cow::Borrowed("steps"), MetaItem::Count(slot.n_steps)),
                        (
                            Cow::Borrowed("avg_step_time"),
                            MetaItem::Duration(slot.elapsed / slot.n_steps as u32),
                        ),
                    ])),
                )]);
            }

            let mut negative_weight_count = 0;
            let mut has_negative_weight_counts = false;

            for batch in slot.all_batches() {
                if let Some(count) = batch.negative_weight_count() {
                    negative_weight_count += count;
                    has_negative_weight_counts = true;
                }
            }

            if has_negative_weight_counts {
                meta.extend([MetricReading::new(
                    NEGATIVE_WEIGHT_COUNT,
                    vec![(Cow::Borrowed("slot"), index.to_string().into())],
                    MetaItem::Count(negative_weight_count as usize),
                )]);
            }
        }

        // Extract all the batches from `slots`, annotating with whether they're
        // merging.
        let mut batches = Vec::new();
        for slot in slots {
            batches.extend(slot.loose_batches.into_iter().map(|b| (b, false)));
            if let Some(merging_batches) = slot.merging_batches {
                batches.extend(merging_batches.into_iter().map(|b| (b, true)));
            }
        }

        // Then summarize the batches.
        let n_batches = batches.len();
        let n_merging = batches.iter().filter(|(_batch, merging)| *merging).count();
        let mut cache_stats = spine_stats.cache_stats;
        let mut storage_size = 0;
        let mut merging_size = 0;
        let mut membership_filter_stats = FilterStats::default();
        let mut range_filter_stats = FilterStats::default();
        let mut storage_records = 0;
        for (batch, merging) in batches {
            cache_stats += batch.cache_stats();
            membership_filter_stats += batch.membership_filter_stats();
            range_filter_stats += batch.range_filter_stats();
            let on_storage = batch.location() == BatchLocation::Storage;
            if on_storage || merging {
                let size = batch.approximate_byte_size();
                if on_storage {
                    storage_size += size;
                    storage_records += batch.key_count();
                }
                if merging {
                    merging_size += size;
                }
            }
        }

        if storage_records > 0 {
            let bits_per_key =
                membership_filter_stats.size_byte as f64 * 8.0 / storage_records as f64;
            let bits_per_key = bits_per_key as usize;
            meta.extend(metadata! {
                BLOOM_FILTER_BITS_PER_KEY => MetaItem::Int(bits_per_key)
            })
        }

        meta.extend([
            MetricReading::new(SPINE_BATCHES_COUNT, Vec::new(), MetaItem::Count(n_batches)),
            MetricReading::new(
                SPINE_STORAGE_SIZE_BYTES,
                Vec::new(),
                MetaItem::bytes(storage_size),
            ),
            MetricReading::new(
                MERGING_BATCHES_COUNT,
                Vec::new(),
                MetaItem::Count(n_merging),
            ),
            MetricReading::new(
                MERGING_SIZE_BYTES,
                Vec::new(),
                MetaItem::bytes(merging_size),
            ),
            MetricReading::new(
                MERGE_REDUCTION_PERCENT,
                Vec::new(),
                spine_stats.merge_reduction(),
            ),
            MetricReading::new(
                MERGE_BACKPRESSURE_WAIT_TIME_SECONDS,
                Vec::new(),
                MetaItem::Duration(spine_stats.backpressure_wait),
            ),
            MetricReading::new(
                BLOOM_FILTER_SIZE_BYTES,
                Vec::new(),
                MetaItem::bytes(membership_filter_stats.size_byte),
            ),
            MetricReading::new(
                BLOOM_FILTER_HITS_COUNT,
                Vec::new(),
                MetaItem::Count(membership_filter_stats.hits),
            ),
            MetricReading::new(
                BLOOM_FILTER_MISSES_COUNT,
                Vec::new(),
                MetaItem::Count(membership_filter_stats.misses),
            ),
            MetricReading::new(
                BLOOM_FILTER_HIT_RATE_PERCENT,
                Vec::new(),
                MetaItem::Percent {
                    numerator: membership_filter_stats.hits as u64,
                    denominator: membership_filter_stats.hits as u64
                        + membership_filter_stats.misses as u64,
                },
            ),
            MetricReading::new(
                RANGE_FILTER_SIZE_BYTES,
                Vec::new(),
                MetaItem::bytes(range_filter_stats.size_byte),
            ),
            MetricReading::new(
                RANGE_FILTER_HITS_COUNT,
                Vec::new(),
                MetaItem::Count(range_filter_stats.hits),
            ),
            MetricReading::new(
                RANGE_FILTER_MISSES_COUNT,
                Vec::new(),
                MetaItem::Count(range_filter_stats.misses),
            ),
            MetricReading::new(
                RANGE_FILTER_HIT_RATE_PERCENT,
                Vec::new(),
                MetaItem::Percent {
                    numerator: range_filter_stats.hits as u64,
                    denominator: range_filter_stats.hits as u64 + range_filter_stats.misses as u64,
                },
            ),
        ]);

        cache_stats.metadata(meta);
    }

    fn maybe_relieve_backpressure(
        no_backpressure: &Arc<Condvar>,
        state: &MutexGuard<SharedState<B>>,
    ) {
        if state.should_relieve_backpressure() {
            Self::relieve_backpressure(no_backpressure);
        }
    }

    fn relieve_backpressure(no_backpressure: &Arc<Condvar>) {
        no_backpressure.notify_all();
    }

    fn run(
        worker_state: &Arc<WorkerState>,
        level: usize,
        opt_merger: &mut Option<Merge<B>>,
        merger_type: MergerType,
        state: &Arc<Mutex<SharedState<B>>>,
        idle: &Arc<Condvar>,
        no_backpressure: &Arc<Condvar>,
    ) -> WorkerStatus {
        // Run in-progress merges.
        let ((key_filter, value_filter), frontier) = {
            let shared = state.lock().unwrap();
            (shared.get_filters(), shared.frontier.clone())
        };

        if let Some(merger) = opt_merger.as_mut() {
            // Run level-0 merges to completion.  For other levels, we
            // supply as much fuel as the average level-0 merge.  Along with
            // round-robinning between levels, this means that we invest
            // about the same amount of effort into merges at each level,
            // which should ensure that the higher-level merges complete in
            // time to keep batches from piling up.
            let fuel = if level == 0 {
                isize::MAX
            } else {
                worker_state.avg_slot0_merge_fuel()
            };
            merger.merge(&frontier, fuel);
            if merger.done {
                if level == 0 {
                    worker_state.report_slot0_merge(merger.fuel);
                }
                let merger = opt_merger.take().unwrap();
                let new_batch = Arc::new(merger.builder.done());
                state.lock().unwrap().merge_complete(
                    level,
                    new_batch,
                    merger.start,
                    merger.elapsed,
                    merger.n_steps,
                );
            }
        }

        // Start new merges out of loose batches.
        //
        // Figuring out what merges to start requires the lock. Then we drop
        // the lock to actually start them, in case that's expensive (it
        // might require creating a file, for example).
        let start_merge = state.lock().unwrap().slots[level].try_start_merge(level);

        let snapshot = if value_filter
            .as_ref()
            .map(|f| f.requires_snapshot())
            .unwrap_or(false)
        {
            Some(Arc::new(state.lock().unwrap().get_snapshot()))
        } else {
            None
        };
        if let Some(batches) = start_merge {
            *opt_merger = Some(Merge::new(
                merger_type,
                batches,
                &key_filter,
                &value_filter,
                snapshot.clone(),
            ));
        }

        let state = state.lock().unwrap();
        if state.request_exit {
            Self::relieve_backpressure(no_backpressure);
            WorkerStatus::Done
        } else if opt_merger.is_none() {
            Self::maybe_relieve_backpressure(no_backpressure, &state);

            idle.notify_all(); // XXX is there a race here?
            WorkerStatus::Idle
        } else {
            Self::maybe_relieve_backpressure(no_backpressure, &state);
            WorkerStatus::Yield
        }
    }
}

impl<B> Drop for AsyncMerger<B>
where
    B: Batch,
{
    fn drop(&mut self) {
        self.state.lock().unwrap().request_exit = true;

        for level in 0..MAX_LEVELS {
            self.state.lock().unwrap().slots[level].notify.notify_one();
        }
    }
}

/// State shared among all of the workers in a background thread.
#[derive(Debug)]
struct WorkerState {
    /// Average amount of fuel used for slot-0 merges.
    avg_slot0_merge_fuel: AtomicIsize,
}

impl Default for WorkerState {
    fn default() -> Self {
        Self {
            avg_slot0_merge_fuel: AtomicIsize::new(10_000),
        }
    }
}

impl WorkerState {
    fn report_slot0_merge(&self, fuel: isize) {
        // Maintains an exponentially weighted moving average of the amount of
        // fuel used to merge batches in slot 0.
        self.avg_slot0_merge_fuel.store(
            ((127 * self.avg_slot0_merge_fuel.load(Ordering::Relaxed) + fuel + 64) / 128).max(1),
            Ordering::Relaxed,
        );
    }

    fn avg_slot0_merge_fuel(&self) -> isize {
        self.avg_slot0_merge_fuel.load(Ordering::Relaxed)
    }
}

/// A single merge in progress in an [AsyncMerger].
struct Merge<B>
where
    B: Batch,
{
    /// Builder for merge output.
    builder: B::Builder,

    /// Total fuel consumed by this merge.
    fuel: isize,

    /// Done?
    done: bool,

    start: Instant,
    elapsed: Duration,

    n_steps: usize,

    /// The merger itself.
    inner: MergeInner<B>,
}

enum MergeInner<B>
where
    B: Batch,
{
    ListMerger(ArcListMerger<B>),
    PushMerger(ArcPushMerger<B>),
}

impl<B> Merge<B>
where
    B: Batch,
{
    fn new(
        merger_type: MergerType,
        batches: Vec<Arc<B>>,
        key_filter: &Option<Filter<B::Key>>,
        value_filter: &Option<GroupFilter<B::Val>>,
        snapshot: Option<Arc<SpineSnapshot<B>>>,
    ) -> Self {
        let factories = batches[0].factories();
        let builder = B::Builder::for_merge(&factories, &batches, None);
        Self {
            builder,
            fuel: 0,
            start: Instant::now(),
            elapsed: Duration::ZERO,
            n_steps: 0,
            done: false,
            inner: match merger_type {
                MergerType::ListMerger => MergeInner::ListMerger(ArcListMerger::new(
                    &factories,
                    batches,
                    key_filter,
                    value_filter,
                    snapshot,
                )),
                MergerType::PushMerger => {
                    let mut inner =
                        ArcPushMerger::new(&factories, batches, key_filter, value_filter);
                    inner.run();
                    MergeInner::PushMerger(inner)
                }
            },
        }
    }

    fn merge(&mut self, frontier: &B::Time, mut fuel: isize) -> isize {
        debug_assert!(fuel > 0);
        let supplied_fuel = fuel;
        let start = Instant::now();
        match &mut self.inner {
            MergeInner::ListMerger(merger) => {
                merger.work(&mut self.builder, frontier, &mut fuel);
                self.done = fuel > 0;
            }
            MergeInner::PushMerger(merger) => {
                self.done =
                    merger.merge(&mut self.builder, frontier, &mut fuel).is_ok() && fuel > 0;
                if !self.done {
                    merger.run();
                }
            }
        };
        self.elapsed += start.elapsed();
        self.n_steps += 1;
        let consumed_fuel = supplied_fuel - fuel;
        self.fuel += consumed_fuel;
        consumed_fuel
    }
}

/// Statistics about merges that a [Spine] has performed.
///
/// The difference between `post_len` and `pre_len` reflects updates that were
/// dropped because weights added to zero or because of key or value filters.
#[derive(Clone, Default)]
struct SpineStats {
    /// Number of updates before merging.
    pre_len: u64,
    /// Number of updates after merging.
    post_len: u64,
    /// Cache statistics, only for the batches that have already been merged and
    /// discarded.
    cache_stats: CacheStats,
    /// Time spent waiting for backpressure.
    backpressure_wait: Duration,
}

impl SpineStats {
    /// Adds `pre_len`, `post_len`, and `cache_stats` to the statistics.
    fn report_merge(&mut self, pre_len: usize, post_len: usize, cache_stats: CacheStats) {
        self.pre_len += pre_len as u64;
        self.post_len += post_len as u64;
        self.cache_stats += cache_stats;
    }

    /// Reports the percentage (in range `0..=100`) of updates that merging
    /// eliminated.
    fn merge_reduction(&self) -> MetaItem {
        MetaItem::Percent {
            numerator: self.pre_len - self.post_len,
            denominator: self.pre_len,
        }
    }
}

/// Persistence optimized [trace][crate::trace::Trace] implementation based on
/// collection and merging immutable batches of updates.
///
/// This spine works asynchronously.  The batches exposed to cursors are
/// maintained separately from the batches currently being merged by an
/// asynchronous thread. When one or more merges complete, the spine fetches the
/// new (smaller) collection of batches from the thread in the next step. (It
/// could fetch them earlier, but it might be unfriendly to expose potentially
/// one form of data to a given cursor and then a different form to the next one
/// within a single step.)
pub struct Spine<B>
where
    B: Batch,
{
    factories: B::Factories,

    dirty: bool,
    key_filter: Option<Filter<B::Key>>,
    value_filter: Option<GroupFilter<B::Val>>,

    /// The asynchronous merger.
    merger: AsyncMerger<B>,
}

impl<B> Spine<B>
where
    B: Batch,
{
    pub fn get_batches(&self) -> Vec<Arc<B>> {
        self.merger.get_batches()
    }
}

impl<B> SizeOf for Spine<B>
where
    B: Batch,
{
    fn size_of_children(&self, context: &mut Context) {
        self.merger.get_batches().size_of_with_context(context);
    }
}

impl<B> Display for Spine<B>
where
    B: Batch + Display,
{
    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
        for batch in self.merger.get_batches() {
            writeln!(f, "batch:\n{}", indent(&batch.to_string(), "    "))?
        }
        Ok(())
    }
}

impl<B> Debug for Spine<B>
where
    B: Batch,
{
    fn fmt(&self, f: &mut Formatter<'_>) -> Result<(), std::fmt::Error> {
        let mut cursor = self.cursor();
        writeln!(f, "spine:")?;
        while cursor.key_valid() {
            writeln!(f, "{:?}:", cursor.key())?;
            while cursor.val_valid() {
                writeln!(f, "    {:?}:", cursor.val())?;

                cursor.map_times(&mut |t, w| {
                    writeln!(f, "        {t:?} -> {w:?}").unwrap();
                });

                cursor.step_val();
            }
            cursor.step_key();
        }
        writeln!(f)?;
        Ok(())
    }
}

// TODO.
impl<B> Clone for Spine<B>
where
    B: Batch,
{
    fn clone(&self) -> Self {
        unimplemented!()
    }
}

impl<B> Archive for Spine<B>
where
    B: Batch,
{
    type Archived = ();
    type Resolver = ();

    unsafe fn resolve(&self, _pos: usize, _resolver: Self::Resolver, _out: *mut Self::Archived) {
        unimplemented!();
    }
}

impl<B: Batch, S: Serializer + ?Sized> Serialize<S> for Spine<B> {
    fn serialize(&self, _serializer: &mut S) -> Result<Self::Resolver, S::Error> {
        unimplemented!();
    }
}

impl<B: Batch, D: Fallible> Deserialize<Spine<B>, D> for Archived<Spine<B>> {
    fn deserialize(&self, _deserializer: &mut D) -> Result<Spine<B>, D::Error> {
        unimplemented!();
    }
}

impl<B> NumEntries for Spine<B>
where
    B: Batch,
{
    const CONST_NUM_ENTRIES: Option<usize> = None;

    fn num_entries_shallow(&self) -> usize {
        self.merger
            .get_batches()
            .iter()
            .map(|batch| batch.len())
            .sum()
    }

    fn num_entries_deep(&self) -> usize {
        self.num_entries_shallow()
    }
}

/// Samples `sample_size` keys from a set of batches.
///
/// See [`BatchReader::sample_keys`](`crate::trace::BatchReader::sample_keys`) for more details.
pub(crate) fn sample_keys_from_batches<B, RG>(
    factories: &B::Factories,
    batches: &[Arc<B>],
    rng: &mut RG,
    sample_size: usize,
    sample: &mut DynVec<B::Key>,
) where
    B: Batch,
    B::Time: PartialEq<()>,
    RG: Rng,
{
    let total_keys = batches.iter().map(|batch| batch.key_count()).sum::<usize>();

    if sample_size == 0 || total_keys == 0 {
        // Avoid division by zero.
        return;
    }

    // Sample each batch, picking the number of keys proportional to
    // batch size.
    let mut intermediate = factories.keys_factory().default_box();
    intermediate.reserve(sample_size);

    for batch in batches {
        batch.sample_keys(
            rng,
            ((batch.key_count() as u128) * (sample_size as u128) / (total_keys as u128)) as usize,
            intermediate.as_mut(),
        );
    }

    // Drop duplicate keys and keys that appear with 0 weight, i.e.,
    // get canceled out across multiple batches.
    intermediate.deref_mut().sort_unstable();
    intermediate.dedup();

    let mut cursor = SpineCursor::new_cursor(factories, batches.to_vec());
    for key in intermediate.dyn_iter_mut() {
        cursor.seek_key(key);
        if let Some(current_key) = cursor.get_key()
            && current_key == key
        {
            debug_assert!(cursor.val_valid() && !cursor.weight().is_zero());
            sample.push_ref(key);
        }
    }
}

impl<B> BatchReader for Spine<B>
where
    B: Batch,
{
    type Key = B::Key;
    type Val = B::Val;
    type Time = B::Time;
    type R = B::R;
    type Factories = B::Factories;

    type Cursor<'s> = SpineCursor<B>;

    fn factories(&self) -> Self::Factories {
        self.factories.clone()
    }

    fn key_count(&self) -> usize {
        self.merger
            .get_batches()
            .iter()
            .map(|batch| batch.key_count())
            .sum()
    }

    fn len(&self) -> usize {
        self.merger
            .get_batches()
            .iter()
            .map(|batch| batch.len())
            .sum()
    }

    fn approximate_byte_size(&self) -> usize {
        self.merger
            .get_batches()
            .iter()
            .map(|batch| batch.approximate_byte_size())
            .sum()
    }

    fn membership_filter_stats(&self) -> FilterStats {
        self.merger
            .get_batches()
            .iter()
            .map(|batch| batch.membership_filter_stats())
            .sum()
    }

    fn range_filter_stats(&self) -> FilterStats {
        self.merger
            .get_batches()
            .iter()
            .map(|batch| batch.range_filter_stats())
            .sum()
    }

    fn cursor(&self) -> Self::Cursor<'_> {
        SpineCursor::new_cursor(&self.factories, self.merger.get_batches())
    }

    fn sample_keys<RG>(&self, rng: &mut RG, sample_size: usize, sample: &mut DynVec<Self::Key>)
    where
        Self::Time: PartialEq<()>,
        RG: Rng,
    {
        sample_keys_from_batches(
            &self.factories,
            &self.merger.get_batches(),
            rng,
            sample_size,
            sample,
        );
    }

    async fn fetch<KR>(
        &self,
        keys: &KR,
    ) -> Option<Box<dyn CursorFactory<Self::Key, Self::Val, Self::Time, Self::R>>>
    where
        KR: BatchReader<Key = Self::Key, Time = ()>,
    {
        Some(Box::new(
            FetchList::new(
                self.merger.get_batches(),
                keys,
                self.factories.weight_factory(),
            )
            .await,
        ))
    }
}

impl<B> Spine<B>
where
    B: Batch,
{
    /// Return the absolute path of the file for this Spine checkpoint.
    ///
    /// # Arguments
    /// - `cid`: The checkpoint id.
    /// - `persistent_id`: The persistent id that identifies the spine within
    ///   the circuit for a given checkpoint.
    fn checkpoint_file(base: &StoragePath, persistent_id: &str) -> StoragePath {
        base.child(format!("pspine-{}.dat", persistent_id))
    }

    /// Return the absolute path of the file for this Spine's batchlist.
    fn batchlist_file(&self, base: &StoragePath, persistent_id: &str) -> StoragePath {
        base.child(format!("pspine-batches-{}.dat", persistent_id))
    }
}

#[self_referencing]
pub struct SpineCursor<B: Batch> {
    batches: Vec<Arc<B>>,
    #[borrows(batches)]
    #[not_covariant]
    cursor: CursorList<B::Key, B::Val, B::Time, B::R, B::Cursor<'this>>,
}

impl<B: Batch> Clone for SpineCursor<B> {
    fn clone(&self) -> Self {
        let batches = self.borrow_batches().clone();
        let weight_factory = self.with_cursor(|cursor| cursor.weight_factory());
        SpineCursorBuilder {
            batches,
            cursor_builder: |batches| {
                CursorList::new(
                    weight_factory,
                    batches.iter().map(|batch| batch.cursor()).collect(),
                )
            },
        }
        .build()
    }
}

impl<B: Batch> SpineCursor<B> {
    pub fn new_cursor(factories: &B::Factories, batches: Vec<Arc<B>>) -> Self {
        SpineCursorBuilder {
            batches,
            cursor_builder: |batches| {
                CursorList::new(
                    factories.weight_factory(),
                    batches.iter().map(|batch| batch.cursor()).collect(),
                )
            },
        }
        .build()
    }
}

impl<B: Batch> Cursor<B::Key, B::Val, B::Time, B::R> for SpineCursor<B> {
    // fn key_vtable(&self) -> &'static VTable<B::Key> {
    //     self.cursor.key_vtable()
    // }

    // fn val_vtable(&self) -> &'static VTable<B::Val> {
    //     self.cursor.val_vtable()
    // }

    fn weight_factory(&self) -> &'static dyn Factory<B::R> {
        self.with_cursor(|cursor| cursor.weight_factory())
    }

    fn key_valid(&self) -> bool {
        self.with_cursor(|cursor| cursor.key_valid())
    }

    fn val_valid(&self) -> bool {
        self.with_cursor(|cursor| cursor.val_valid())
    }

    fn key(&self) -> &B::Key {
        self.with_cursor(|cursor| cursor.key())
    }

    fn val(&self) -> &B::Val {
        self.with_cursor(|cursor| cursor.val())
    }

    fn map_times(&mut self, logic: &mut dyn FnMut(&B::Time, &B::R)) {
        self.with_cursor_mut(|cursor| cursor.map_times(logic));
    }

    fn map_times_through(&mut self, upper: &B::Time, logic: &mut dyn FnMut(&B::Time, &B::R)) {
        self.with_cursor_mut(|cursor| cursor.map_times_through(upper, logic));
    }

    fn weight(&mut self) -> &B::R
    where
        B::Time: PartialEq<()>,
    {
        self.with_cursor_mut(|cursor| cursor.weight())
    }

    fn weight_checked(&mut self) -> &B::R {
        self.with_cursor_mut(|cursor| cursor.weight_checked())
    }

    fn map_values(&mut self, logic: &mut dyn FnMut(&B::Val, &B::R))
    where
        B::Time: PartialEq<()>,
    {
        self.with_cursor_mut(|cursor| cursor.map_values(logic))
    }

    fn step_key(&mut self) {
        self.with_cursor_mut(|cursor| cursor.step_key());
    }

    fn step_key_reverse(&mut self) {
        self.with_cursor_mut(|cursor| cursor.step_key_reverse());
    }

    fn seek_key(&mut self, key: &B::Key) {
        self.with_cursor_mut(|cursor| cursor.seek_key(key));
    }

    fn seek_key_exact(&mut self, key: &B::Key, hash: Option<u64>) -> bool {
        self.with_cursor_mut(|cursor| cursor.seek_key_exact(key, hash))
    }

    fn seek_key_with(&mut self, predicate: &dyn Fn(&B::Key) -> bool) {
        self.with_cursor_mut(|cursor| cursor.seek_key_with(predicate));
    }

    fn seek_key_with_reverse(&mut self, predicate: &dyn Fn(&B::Key) -> bool) {
        self.with_cursor_mut(|cursor| cursor.seek_key_with_reverse(predicate));
    }

    fn seek_key_reverse(&mut self, key: &B::Key) {
        self.with_cursor_mut(|cursor| cursor.seek_key_reverse(key));
    }

    fn step_val(&mut self) {
        self.with_cursor_mut(|cursor| cursor.step_val());
    }

    fn seek_val(&mut self, val: &B::Val) {
        self.with_cursor_mut(|cursor| cursor.seek_val(val));
    }

    fn seek_val_with(&mut self, predicate: &dyn Fn(&B::Val) -> bool) {
        self.with_cursor_mut(|cursor| cursor.seek_val_with(predicate));
    }

    fn rewind_keys(&mut self) {
        self.with_cursor_mut(|cursor| cursor.rewind_keys());
    }

    fn fast_forward_keys(&mut self) {
        self.with_cursor_mut(|cursor| cursor.fast_forward_keys());
    }

    fn rewind_vals(&mut self) {
        self.with_cursor_mut(|cursor| cursor.rewind_vals());
    }

    fn step_val_reverse(&mut self) {
        self.with_cursor_mut(|cursor| cursor.step_val_reverse());
    }

    fn seek_val_reverse(&mut self, val: &B::Val) {
        self.with_cursor_mut(|cursor| cursor.seek_val_reverse(val));
    }

    fn seek_val_with_reverse(&mut self, predicate: &dyn Fn(&B::Val) -> bool) {
        self.with_cursor_mut(|cursor| cursor.seek_val_with_reverse(predicate));
    }

    fn fast_forward_vals(&mut self) {
        self.with_cursor_mut(|cursor| cursor.fast_forward_vals());
    }

    fn position(&self) -> Option<Position> {
        self.with_cursor(|cursor| cursor.position())
    }
}

impl<B> Trace for Spine<B>
where
    B: Batch,
{
    type Batch = B;

    fn new(factories: &B::Factories) -> Self {
        Self::with_effort(factories, 1)
    }

    fn set_frontier(&mut self, frontier: &B::Time) {
        self.merger.set_frontier(frontier)
    }

    fn exert(&mut self, _effort: &mut isize) {}

    fn consolidate(self) -> Option<B> {
        let batches = self
            .merger
            .pause()
            .into_iter()
            .map(|batch| Arc::into_inner(batch).unwrap());
        let result = merge_batches(
            &self.factories,
            batches,
            &self.key_filter,
            &self.value_filter,
        );
        if result.is_empty() {
            None
        } else {
            Some(result)
        }
    }

    fn insert(&mut self, mut batch: Self::Batch) {
        if !batch.is_empty() {
            // Push in-memory batches to storage if they exceed the user-configured `min_storage_bytes` or
            // we're under high memory pressure.
            let batch = if batch.location() == BatchLocation::Memory
                && pick_insert_destination(&batch) == BatchLocation::Storage
            {
                let _span = SamplySpan::new("eager spill")
                    .with_category("Spine")
                    .with_tooltip(|| {
                        format!(
                            "Eagerly spilling {} batch with {} keys and {} values",
                            HumanBytes::from(batch.approximate_byte_size()),
                            batch.key_count(),
                            batch.len()
                        )
                    });
                let factories = batch.factories();
                let builder =
                    B::Builder::for_merge(&factories, [&batch], Some(BatchLocation::Storage));
                let (key_filter, value_filter) = self.merger.state.lock().unwrap().get_filters();
                ListMerger::merge(
                    &factories,
                    builder,
                    vec![batch.consuming_cursor(key_filter, value_filter)],
                )
            } else {
                batch
            };

            self.dirty = true;
            self.merger.add_batch(Arc::new(batch), false);
        }
    }

    fn insert_arc(&mut self, batch: Arc<Self::Batch>) {
        if !batch.is_empty() {
            let batch = if batch.location() == BatchLocation::Memory
                && pick_insert_destination(&batch) == BatchLocation::Storage
            {
                let factories = batch.factories();
                let builder =
                    B::Builder::for_merge(&factories, [&batch], Some(BatchLocation::Storage));
                let (key_filter, value_filter) = self.merger.state.lock().unwrap().get_filters();
                Arc::new(ListMerger::merge(
                    &factories,
                    builder,
                    vec![batch.merge_cursor(key_filter, value_filter)],
                ))
            } else {
                batch
            };

            self.dirty = true;
            self.merger.add_batch(batch, false);
        }
    }

    fn clear_dirty_flag(&mut self) {
        self.dirty = false;
    }

    fn dirty(&self) -> bool {
        self.dirty
    }

    fn retain_keys(&mut self, filter: Filter<Self::Key>) {
        self.merger.set_key_filter(&filter);
        self.key_filter = Some(filter);
    }

    fn retain_values(&mut self, filter: GroupFilter<Self::Val>) {
        self.merger.set_value_filter(&filter);
        self.value_filter = Some(filter);
    }

    fn key_filter(&self) -> &Option<Filter<Self::Key>> {
        &self.key_filter
    }

    fn value_filter(&self) -> &Option<GroupFilter<Self::Val>> {
        &self.value_filter
    }

    fn save(
        &mut self,
        base: &StoragePath,
        persistent_id: &str,
        files: &mut Vec<Arc<dyn FileCommitter>>,
    ) -> Result<(), Error> {
        fn persist_batches<B>(batches: Vec<Arc<B>>) -> Vec<Arc<B>>
        where
            B: Batch,
        {
            batches
                .into_iter()
                .map(|batch| {
                    if let Some(persisted) = batch.persisted() {
                        Arc::new(persisted)
                    } else {
                        batch
                    }
                })
                .collect::<Vec<_>>()
        }

        // Persist all the batches, and stick the not-merging batches back into
        // the merger.  (Putting the persisted batches into the merger means
        // that we don't have to persist them again for the next checkpoint,
        // saving time then. On the other hand, we do have to read them back
        // from disk to use them: no free lunch.)
        let (not_merging, merging) = self.merger.pause_new_merges();
        let not_merging = persist_batches(not_merging);
        self.merger.resume(not_merging.iter().cloned());
        let merging = persist_batches(merging);

        // Get the persistent IDs.
        let ids = not_merging
            .iter()
            .chain(merging.iter())
            .map(|batch| {
                let file = batch
                    .file_reader()
                    .expect("The batch should have been persisted");
                let path = file.path().to_string();
                files.push(file);
                path
            })
            .collect::<Vec<_>>();

        let backend = Runtime::storage_backend().unwrap();
        let committed: CommittedSpine = (ids, self as &Self).into();
        let as_bytes = to_bytes(&committed).expect("Serializing CommittedSpine should work.");
        backend.write(&Self::checkpoint_file(base, persistent_id), as_bytes)?;

        // Write the batches as a separate file, this allows to parse it
        // in `Checkpointer` without the need to know the exact Spine type.
        let pspine_batches = PSpineBatches {
            files: committed.batches,
        };
        backend
            .write_json(&self.batchlist_file(base, persistent_id), &pspine_batches)
            .and_then(|reader| reader.commit())?;

        Ok(())
    }

    fn restore(&mut self, base: &StoragePath, persistent_id: &str) -> Result<(), Error> {
        let pspine_path = Self::checkpoint_file(base, persistent_id);

        let content = Runtime::storage_backend().unwrap().read(&pspine_path)?;
        let archived = unsafe { rkyv::archived_root::<CommittedSpine>(&content) };

        let committed: CommittedSpine = archived.deserialize(&mut Deserializer::default()).unwrap();
        self.dirty = committed.dirty;
        self.key_filter = None;
        self.value_filter = None;
        for batch in committed.batches {
            let batch = B::from_path(&self.factories.clone(), &batch.clone().into())
                .unwrap_or_else(|error| {
                    panic!("Failed to read batch {batch} for checkpoint ({error}).")
                });
            self.insert(batch);
        }

        Ok(())
    }

    fn metadata(&self, meta: &mut OperatorMeta) {
        self.merger.metadata(meta);
    }
}

impl<B> Spine<B>
where
    B: Batch,
{
    /// Given a batch size figure out which level it should reside in.
    fn size_to_level(batch: &B, max_level0_batch_size_records: usize, merge: bool) -> usize {
        debug_assert_eq!(MAX_LEVELS, 9);
        debug_assert!(max_level0_batch_size_records > 0 && max_level0_batch_size_records <= 99_999);

        let len = batch.len();

        let effective_len = if merge {
            // Merge batches with many negative weights more aggressively. Negative updates are likely to cancel
            // out during merging. Every time this happens, two records are eliminated from the spine, speeding
            // up lookups and reducing the spine's storage footprint. Furthermore, this avoids performance anomalies
            // where an operator spends a lot of time skipping over 0-weight records.
            //
            // We implement this heuristic by accounting each record with a negative weight as N records when calculating
            // the batch's effective length (N is equal to negative_weight_multiplier() + 1). This should push such batches
            // to the next level more eagerly, until they get merged with batches that contain matching positive-weight
            // records.
            //
            // We only apply this heuristic to merged batches to avoid reordering insertions and deletions by pushing
            // deletions to higher levels than the insertions they correspond to.
            let negative_weight_count = batch.negative_weight_count().unwrap_or(0) as usize;
            len + negative_weight_count * (negative_weight_multiplier() as usize)
        } else {
            len
        };

        if effective_len <= max_level0_batch_size_records {
            return 0;
        }
        match effective_len {
            0..=99_999 => 1,
            100_000..=999_999 => 2,
            1_000_000..=9_999_999 => 3,
            10_000_000..=99_999_999 => 4,
            100_000_000..=999_999_999 => 5,
            1_000_000_000..=9_999_999_999 => 6,
            10_000_000_000..=99_999_999_999 => 7,
            _ => 8, // For reference: 100 bln * 8 bytes ~= 750 GiB
        }
    }

    /// Allocates a fueled `Spine` with a specified effort multiplier.
    ///
    /// This trace will merge batches progressively, with each inserted batch
    /// applying a multiple of the batch's length in effort to each merge.
    /// The `effort` parameter is that multiplier. This value should be at
    /// least one for the merging to happen; a value of zero is not helpful.
    pub fn with_effort(factories: &B::Factories, _effort: usize) -> Self {
        Spine {
            factories: factories.clone(),
            dirty: false,
            key_filter: None,
            value_filter: None,
            merger: AsyncMerger::new(factories),
        }
    }

    pub fn complete_merges(&mut self) {
        let batches = self.merger.pause();
        let batch = merge_batches(
            &self.factories,
            batches.into_iter().map(|b| Arc::unwrap_or_clone(b)),
            &self.key_filter,
            &self.value_filter,
        );
        self.merger.resume(vec![Arc::new(batch)]);
    }
}

impl<B: Batch> WithSnapshot for Spine<B>
where
    B: Batch,
{
    type Batch = B;

    fn ro_snapshot(&self) -> SpineSnapshot<B> {
        self.into()
    }
}