slatedb 0.10.0

A cloud native embedded storage engine built on object storage.
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
use std::sync::Arc;

use async_trait::async_trait;
use bytes::Bytes;
use thiserror::Error;

use crate::{
    error::SlateDBError,
    iter::KeyValueIterator,
    types::{RowEntry, ValueDeletable},
    utils::{is_not_expired, merge_options},
};

#[non_exhaustive]
#[derive(Clone, Debug, Error)]
pub enum MergeOperatorError {
    #[error("merge_batch called with empty operands and no existing value")]
    EmptyBatch,
}

/// A trait for implementing custom merge operations in SlateDB.
///
/// The MergeOperator allows applications to bypass the traditional read/modify/update cycle
/// in performance-critical situations where computation can be expressed using an associative operator.
/// This is particularly useful for implementing:
/// - Aggregations (e.g., counters, sums)
/// - Buffering (e.g., append-only lists)
///
/// # Associativity Requirement
/// The merge operation MUST be associative, meaning that for any values a, b, and c:
/// merge(merge(a, b), c) == merge(a, merge(b, c))
///
/// # Examples
/// Here's an example of a counter merge operator:
/// ```
/// use bytes::Bytes;
/// use slatedb::{MergeOperator, MergeOperatorError};
///
/// struct CounterMergeOperator;
///
/// impl MergeOperator for CounterMergeOperator {
///     fn merge(&self, _key: &Bytes, existing_value: Option<Bytes>, operand: Bytes) -> Result<Bytes, MergeOperatorError> {
///         let existing = existing_value
///             .map(|v| u64::from_le_bytes(v.as_ref().try_into().unwrap()))
///             .unwrap_or(0);
///         let increment = u64::from_le_bytes(operand.as_ref().try_into().unwrap());
///         Ok(Bytes::copy_from_slice(&(existing + increment).to_le_bytes()))
///     }
///
///     fn merge_batch(&self, _key: &Bytes, existing_value: Option<Bytes>, operands: &[Bytes]) -> Result<Bytes, MergeOperatorError> {
///         let mut total = existing_value
///             .map(|v| u64::from_le_bytes(v.as_ref().try_into().unwrap()))
///             .unwrap_or(0);
///         
///         for operand in operands {
///             let increment = u64::from_le_bytes(operand.as_ref().try_into().unwrap());
///             total += increment;
///         }
///         
///         Ok(Bytes::copy_from_slice(&total.to_le_bytes()))
///     }
/// }
/// ```
pub trait MergeOperator {
    /// Merges the existing value with a new value to produce a combined result.
    ///
    /// This method is called during reads and compactions to combine multiple merge operands
    /// into a single value. The implementation must be associative to ensure correct behavior.
    ///
    /// # Arguments
    /// * `key` - The key of the entry
    /// * `existing_value` - The current accumulated value
    /// * `value` - The new value to merge with the existing value
    ///
    /// # Returns
    /// * `Ok(Bytes)` - The merged result as bytes
    /// * `Err(MergeOperatorError)` - If the merge operation fails
    fn merge(
        &self,
        key: &Bytes,
        existing_value: Option<Bytes>,
        value: Bytes,
    ) -> Result<Bytes, MergeOperatorError>;

    /// Merges a batch of operands with an optional existing value.
    ///
    /// This method allows for more efficient batch processing of merge operands.
    /// The default implementation applies pairwise merging, but implementations
    /// can override this for better performance (e.g., a counter can sum all values at once).
    ///
    /// # Arguments
    /// * `key` - The key of the entry
    /// * `existing_value` - The current accumulated value (if any)
    /// * `operands` - A slice of operands to merge, ordered from oldest to newest
    ///
    /// # Returns
    /// * `Ok(Bytes)` - The merged result as bytes
    /// * `Err(MergeOperatorError)` - If the merge operation fails
    fn merge_batch(
        &self,
        key: &Bytes,
        existing_value: Option<Bytes>,
        operands: &[Bytes],
    ) -> Result<Bytes, MergeOperatorError> {
        let mut result = existing_value;
        for operand in operands {
            result = Some(self.merge(key, result, operand.clone())?);
        }
        result.ok_or(MergeOperatorError::EmptyBatch)
    }
}

pub(crate) type MergeOperatorType = Arc<dyn MergeOperator + Send + Sync>;

/// Maximum number of merge operands to collect before calling `merge_batch`.
///
/// This controls the streaming batch size when merging entries with the same key.
/// The iterator collects up to this many merge operands, then calls `merge_batch`
/// to combine them into a single intermediate result. This process repeats until
/// all operands are processed or a base value is found.
///
/// Benefits of batching:
/// - Reduces function call overhead (N calls → N/BATCH_SIZE calls)
/// - Enables batch-optimized merge implementations (e.g., sum all at once)
/// - Reduces intermediate memory allocations
///
/// Trade-offs:
/// - Larger batches: Better performance, higher peak memory per key
/// - Smaller batches: Lower memory usage, more function calls
///
/// TODO: Make this configurable based on system memory constraints
const MERGE_BATCH_SIZE: usize = 100;

/// An iterator that ensures merge operands are not returned when no merge operator is configured.
pub(crate) struct MergeOperatorRequiredIterator<T: KeyValueIterator> {
    delegate: T,
}

impl<T: KeyValueIterator> MergeOperatorRequiredIterator<T> {
    pub(crate) fn new(delegate: T) -> Self {
        Self { delegate }
    }
}

#[async_trait]
impl<T: KeyValueIterator> KeyValueIterator for MergeOperatorRequiredIterator<T> {
    async fn init(&mut self) -> Result<(), SlateDBError> {
        self.delegate.init().await
    }

    async fn next_entry(&mut self) -> Result<Option<RowEntry>, SlateDBError> {
        let next_entry = self.delegate.next_entry().await?;
        if let Some(entry) = next_entry {
            match &entry.value {
                ValueDeletable::Merge(_) => {
                    return Err(SlateDBError::MergeOperatorMissing);
                }
                _ => return Ok(Some(entry)),
            }
        }
        Ok(None)
    }

    async fn seek(&mut self, next_key: &[u8]) -> Result<(), SlateDBError> {
        self.delegate.seek(next_key).await
    }
}

/// An iterator that merges mergeable entries into a single value.
///
/// It is expected that this is the top level iterator in a merge scan, and therefore
/// return a ValueDeletable::Value entry (instead of a Merge even if the resolved value
/// is a merge operand).
pub(crate) struct MergeOperatorIterator<T: KeyValueIterator> {
    merge_operator: MergeOperatorType,
    delegate: T,
    /// Entry from the delegate that we've peeked ahead and buffered.
    buffered_entry: Option<RowEntry>,
    /// Whether to merge entries with different expire timestamps.
    merge_different_expire_ts: bool,
    now: i64,
    /// A barrier sequence number that supports snapshot reads using this iterator. If not None,
    /// the iterator will not merge entries with sequence number greater than this value.
    snapshot_barrier_seq: Option<u64>,
}

/// Tracks metadata across multiple entries during merge operations.
///
/// When merging entries for the same key, we need to aggregate metadata from all
/// entries to produce the final merged entry. This struct accumulates:
/// - The maximum creation timestamp (for tracking when data was first written)
/// - The minimum expiration timestamp (most restrictive TTL)
/// - The maximum sequence number (for ordering and validation)
#[derive(Debug, Clone)]
struct MergeTracker {
    /// Maximum creation timestamp seen across all merged entries.
    /// Used to track when the data was first created.
    max_create_ts: Option<i64>,

    /// Minimum expiration timestamp seen across all merged entries.
    /// The merged result expires at the earliest expiration time.
    min_expire_ts: Option<i64>,

    /// Maximum sequence number seen across all merged entries.
    /// Also used to validate that entries are processed in descending order.
    seq: u64,
}

impl MergeTracker {
    /// Updates the tracker with metadata from a new entry.
    ///
    /// This method:
    /// - Takes the maximum of creation timestamps (latest creation)
    /// - Takes the minimum of expiration timestamps (earliest expiration)
    /// - Validates that sequence numbers are in descending order
    /// - Updates the maximum sequence number
    ///
    /// # Errors
    ///
    /// Returns `SlateDBError::InvalidSequenceOrder` if the entry's sequence number
    /// is greater than the current maximum, indicating entries are out of order.
    fn update(&mut self, entry: &RowEntry) -> Result<(), SlateDBError> {
        self.max_create_ts = merge_options(self.max_create_ts, entry.create_ts, i64::max);
        self.min_expire_ts = merge_options(self.min_expire_ts, entry.expire_ts, i64::min);

        // sequence numbers should be descending
        if self.seq < entry.seq {
            return Err(SlateDBError::InvalidSequenceOrder {
                current_seq: self.seq,
                next_seq: entry.seq,
            });
        }

        self.seq = std::cmp::max(self.seq, entry.seq);
        Ok(())
    }
}

#[allow(unused)]
impl<T: KeyValueIterator> MergeOperatorIterator<T> {
    pub(crate) fn new(
        merge_operator: MergeOperatorType,
        delegate: T,
        merge_different_expire_ts: bool,
        now: i64,
        snapshot_barrier_seq: Option<u64>,
    ) -> Self {
        Self {
            merge_operator,
            delegate,
            buffered_entry: None,
            merge_different_expire_ts,
            now,
            snapshot_barrier_seq,
        }
    }
}

impl<T: KeyValueIterator> MergeOperatorIterator<T> {
    /// Checks if an entry matches the current key and expiration timestamp.
    /// Returns true if keys match and either `merge_different_expire_ts` is enabled
    /// or the expire_ts matches.
    fn is_matching_entry(
        &self,
        entry: &RowEntry,
        key: &Bytes,
        first_expire_ts: Option<i64>,
    ) -> bool {
        entry.key == *key && (self.merge_different_expire_ts || first_expire_ts == entry.expire_ts)
    }

    /// Processes a batch of merge operands into a single intermediate result.
    ///
    /// Reverses the batch (oldest to newest), extracts operand values, updates metadata
    /// tracking, calls `merge_batch`, and clears the batch to free memory.
    fn process_batch(
        &self,
        key: &Bytes,
        batch: &mut Vec<RowEntry>,
        merge_tracker: &mut MergeTracker,
    ) -> Result<Bytes, SlateDBError> {
        batch.reverse();
        let mut operands: Vec<Bytes> = Vec::with_capacity(batch.len());
        for entry in &*batch {
            merge_tracker.update(entry)?;
            if let Some(v) = entry.value.as_bytes() {
                operands.push(v);
            }
        }

        let batch_result = self.merge_operator.merge_batch(key, None, &operands)?;
        batch.clear();
        Ok(batch_result)
    }

    /// Merges a sequence of entries with the same key into a single entry.
    ///
    /// Collects merge operands in batches of `MERGE_BATCH_SIZE`, processes each batch
    /// into intermediate results, and continues until a base value (Value/Tombstone) is
    /// found. Finally combines all intermediate results with the base value.
    ///
    /// Returns `Some(RowEntry)` with merged value and aggregated metadata, or `None` if
    /// only a tombstone with no operands was found.
    async fn merge_with_older_entries(
        &mut self,
        first_entry: RowEntry,
    ) -> Result<Option<RowEntry>, SlateDBError> {
        let key = first_entry.key.clone();
        let first_expire_ts = first_entry.expire_ts;

        let mut merge_tracker = MergeTracker {
            max_create_ts: None,
            min_expire_ts: None,
            seq: first_entry.seq,
        };

        let mut results = Vec::new();
        let mut batch = Vec::with_capacity(MERGE_BATCH_SIZE);

        let mut next = Some(first_entry);

        // this loop returns the "base value" (non merge operand) if it exists
        let base = loop {
            if let Some(entry) = next {
                if !self.is_matching_entry(&entry, &key, first_expire_ts) {
                    self.buffered_entry = Some(entry);
                    break None;
                } else if !matches!(entry.value, ValueDeletable::Merge(_)) {
                    // found a Value or Tombstone, this is the base value
                    break Some(entry);
                } else if is_not_expired(&entry, self.now) {
                    batch.push(entry);
                }

                // if the batch is full, merge it and add the result to the results vector
                if batch.len() >= MERGE_BATCH_SIZE {
                    results.push(self.process_batch(&key, &mut batch, &mut merge_tracker)?);
                }

                next = self.delegate.next_entry().await?;
            } else {
                break None;
            }
        };

        // handle leftovers from the last batch
        if !batch.is_empty() {
            results.push(self.process_batch(&key, &mut batch, &mut merge_tracker)?);
        }

        let base_value = base.as_ref().and_then(|b| b.value.as_bytes());
        let found_base = base.is_some();

        // If we have no results and either no base or a tombstone base, return None
        if results.is_empty() && base_value.is_none() {
            return Ok(None);
        }

        results.reverse();
        let final_result = self
            .merge_operator
            .merge_batch(&key, base_value, &results)?;

        Ok(Some(RowEntry {
            key: key.clone(),
            value: if found_base {
                ValueDeletable::Value(final_result)
            } else {
                ValueDeletable::Merge(final_result)
            },
            seq: merge_tracker.seq,
            create_ts: merge_tracker.max_create_ts,
            expire_ts: merge_tracker.min_expire_ts,
        }))
    }
}

#[async_trait]
impl<T: KeyValueIterator> KeyValueIterator for MergeOperatorIterator<T> {
    async fn init(&mut self) -> Result<(), SlateDBError> {
        self.delegate.init().await
    }

    async fn next_entry(&mut self) -> Result<Option<RowEntry>, SlateDBError> {
        let next_entry = match self.buffered_entry.take() {
            Some(entry) => Some(entry),
            None => self.delegate.next_entry().await?,
        };
        if let Some(entry) = next_entry {
            match &entry.value {
                ValueDeletable::Merge(_) => {
                    if let Some(snapshot_barrier_seq) = self.snapshot_barrier_seq {
                        if entry.seq > snapshot_barrier_seq {
                            return Ok(Some(entry));
                        }
                    }
                    // A mergeable entry, we need to accumulate all mergeable entries
                    // ahead for the same key and merge them into a single value.
                    return self.merge_with_older_entries(entry).await;
                }
                // Not a mergeable entry, just return it.
                _ => return Ok(Some(entry)),
            }
        }
        Ok(None)
    }

    async fn seek(&mut self, next_key: &[u8]) -> Result<(), SlateDBError> {
        self.delegate.seek(next_key).await
    }
}

#[cfg(test)]
mod tests {
    use std::{cmp::Ordering, collections::VecDeque, fmt::Debug};

    use rstest::rstest;

    use crate::test_utils::assert_iterator;

    use super::*;

    struct MockMergeOperator;

    impl MergeOperator for MockMergeOperator {
        fn merge(
            &self,
            _key: &Bytes,
            existing_value: Option<Bytes>,
            value: Bytes,
        ) -> Result<Bytes, MergeOperatorError> {
            match existing_value {
                Some(existing) => {
                    let mut merged = existing.to_vec();
                    merged.extend_from_slice(&value);
                    Ok(Bytes::from(merged))
                }
                None => Ok(value),
            }
        }
    }

    /// Mock merge operator that tracks whether merge_batch is called
    struct MockBatchedMergeOperator {
        merge_batch_call_count: std::sync::Arc<std::sync::atomic::AtomicUsize>,
    }

    impl MockBatchedMergeOperator {
        fn new() -> (Self, std::sync::Arc<std::sync::atomic::AtomicUsize>) {
            let counter = std::sync::Arc::new(std::sync::atomic::AtomicUsize::new(0));
            (
                Self {
                    merge_batch_call_count: counter.clone(),
                },
                counter,
            )
        }
    }

    impl MergeOperator for MockBatchedMergeOperator {
        fn merge(
            &self,
            _key: &Bytes,
            existing_value: Option<Bytes>,
            value: Bytes,
        ) -> Result<Bytes, MergeOperatorError> {
            // Same as MockMergeOperator - concatenate bytes
            match existing_value {
                Some(existing) => {
                    let mut merged = existing.to_vec();
                    merged.extend_from_slice(&value);
                    Ok(Bytes::from(merged))
                }
                None => Ok(value),
            }
        }

        fn merge_batch(
            &self,
            _key: &Bytes,
            existing_value: Option<Bytes>,
            operands: &[Bytes],
        ) -> Result<Bytes, MergeOperatorError> {
            // Increment counter to track that merge_batch was called
            self.merge_batch_call_count
                .fetch_add(1, std::sync::atomic::Ordering::SeqCst);

            // Efficiently concatenate all operands at once
            let mut result = existing_value.unwrap_or_default().to_vec();
            for operand in operands {
                result.extend_from_slice(operand);
            }
            Ok(Bytes::from(result))
        }
    }

    #[tokio::test]
    async fn test_merge_operator_iterator() {
        let merge_operator = Arc::new(MockMergeOperator {});
        let data = vec![
            RowEntry::new_merge(b"key1", b"1", 1),
            RowEntry::new_merge(b"key1", b"2", 2),
            RowEntry::new_merge(b"key1", b"3", 3),
            RowEntry::new_merge(b"key1", b"4", 4),
            RowEntry::new_value(b"key2", b"1", 5),
            RowEntry::new_value(b"key3", b"1", 6),
            RowEntry::new_merge(b"key3", b"2", 7),
            RowEntry::new_merge(b"key3", b"3", 8),
        ];
        let mut iterator = MergeOperatorIterator::<MockKeyValueIterator>::new(
            merge_operator,
            data.into(),
            true,
            0,
            None,
        );
        assert_iterator(
            &mut iterator,
            vec![
                RowEntry::new_merge(b"key1", b"1234", 4),
                RowEntry::new_value(b"key2", b"1", 5),
                RowEntry::new_value(b"key3", b"123", 8),
            ],
        )
        .await;
    }

    #[derive(Debug)]
    struct TestCase {
        unsorted_data: Vec<RowEntry>,
        expected: Vec<RowEntry>,
        merge_different_expire_ts: bool,
        snapshot_barrier_seq: Option<u64>,
    }

    impl Default for TestCase {
        fn default() -> Self {
            Self {
                unsorted_data: vec![],
                expected: vec![],
                merge_different_expire_ts: true,
                snapshot_barrier_seq: None,
            }
        }
    }

    #[rstest]
    #[case::different_expire_ts_read_path(TestCase {
        unsorted_data: vec![
            RowEntry::new_merge(b"key1", b"1", 1).with_expire_ts(1),
            RowEntry::new_merge(b"key1", b"2", 2).with_expire_ts(2),
            RowEntry::new_merge(b"key1", b"3", 3).with_expire_ts(3),
            RowEntry::new_value(b"key2", b"1", 4),
            RowEntry::new_merge(b"key3", b"1", 5).with_expire_ts(1),
            RowEntry::new_merge(b"key3", b"2", 6).with_expire_ts(1),
            RowEntry::new_merge(b"key3", b"3", 7).with_expire_ts(2),
        ],
        expected: vec![
            RowEntry::new_merge(b"key1", b"123", 3).with_expire_ts(1),
            RowEntry::new_value(b"key2", b"1", 4),
            RowEntry::new_merge(b"key3", b"123", 7).with_expire_ts(1),
        ],
        ..TestCase::default()
    })]
    #[case::different_expire_ts_write_path(TestCase {
        unsorted_data: vec![
            RowEntry::new_merge(b"key1", b"1", 1).with_expire_ts(1),
            RowEntry::new_merge(b"key1", b"2", 2).with_expire_ts(2),
            RowEntry::new_merge(b"key1", b"3", 3).with_expire_ts(3),
            RowEntry::new_value(b"key2", b"1", 4),
            RowEntry::new_merge(b"key3", b"1", 5).with_expire_ts(1),
            RowEntry::new_merge(b"key3", b"2", 6).with_expire_ts(1),
            RowEntry::new_merge(b"key3", b"3", 7).with_expire_ts(2),
        ],
        expected: vec![
            RowEntry::new_merge(b"key1", b"3", 3).with_expire_ts(3),
            RowEntry::new_merge(b"key1", b"2", 2).with_expire_ts(2),
            RowEntry::new_merge(b"key1", b"1", 1).with_expire_ts(1),
            RowEntry::new_value(b"key2", b"1", 4),
            RowEntry::new_merge(b"key3", b"3", 7).with_expire_ts(2),
            RowEntry::new_merge(b"key3", b"12", 6).with_expire_ts(1),
        ],
        // On write path (compaction, memtable), we don't merge entries
        // with different expire timestamps to allow per-element expiration.
        merge_different_expire_ts: false,
        ..TestCase::default()
    })]
    #[case::merge_with_tombstone(TestCase {
        unsorted_data: vec![
            RowEntry::new_merge(b"key1", b"1", 1),
            RowEntry::new_merge(b"key1", b"2", 2),
            RowEntry::new_tombstone(b"key1", 3),
            RowEntry::new_merge(b"key1", b"3", 4),
            RowEntry::new_value(b"key2", b"1", 5)
        ],
        expected: vec![
            // Merge + Tombstone becomes a value to invalidate older entries.
            RowEntry::new_value(b"key1", b"3", 4),
            RowEntry::new_merge(b"key1", b"12", 2),
            RowEntry::new_value(b"key2", b"1", 5)
        ],
        ..TestCase::default()
    })]
    #[case::multiple_values(TestCase {
        unsorted_data: vec![
            RowEntry::new_value(b"key1", b"1", 1),
            RowEntry::new_value(b"key1", b"2", 2),
        ],
        expected: vec![
            RowEntry::new_value(b"key1", b"2", 2),
            RowEntry::new_value(b"key1", b"1", 1),
        ],
        ..TestCase::default()
    })]
    #[case::merge_with_snapshot_barrier(TestCase {
        unsorted_data: vec![
            RowEntry::new_merge(b"key1", b"1", 1),
            RowEntry::new_merge(b"key1", b"2", 2),
            RowEntry::new_merge(b"key1", b"3", 3),
            RowEntry::new_merge(b"key1", b"4", 4),
            RowEntry::new_value(b"key1", b"5", 5)
        ],
        expected: vec![
            RowEntry::new_value(b"key1", b"5", 5),
            RowEntry::new_merge(b"key1", b"4", 4),
            RowEntry::new_merge(b"key1", b"123", 3),
        ],
        snapshot_barrier_seq: Some(3),
        ..TestCase::default()
    })]
    #[tokio::test]
    async fn test(#[case] test_case: TestCase) {
        let merge_operator = Arc::new(MockMergeOperator {});
        let mut iterator = MergeOperatorIterator::<MockKeyValueIterator>::new(
            merge_operator,
            test_case.unsorted_data.into(),
            test_case.merge_different_expire_ts,
            0,
            test_case.snapshot_barrier_seq,
        );
        assert_iterator(&mut iterator, test_case.expected).await;
    }

    struct MockKeyValueIterator {
        values: VecDeque<RowEntry>,
    }

    #[async_trait]
    impl KeyValueIterator for MockKeyValueIterator {
        async fn init(&mut self) -> Result<(), SlateDBError> {
            Ok(())
        }

        async fn next_entry(&mut self) -> Result<Option<RowEntry>, SlateDBError> {
            Ok(self.values.pop_front())
        }

        async fn seek(&mut self, next_key: &[u8]) -> Result<(), SlateDBError> {
            self.values.retain(|entry| entry.key == next_key);
            Ok(())
        }
    }

    impl From<Vec<RowEntry>> for MockKeyValueIterator {
        /// Converts a vector of RowEntries into a MockKeyValueIterator. The vector is sorted
        /// by key and reverse sequence number.
        fn from(values: Vec<RowEntry>) -> Self {
            let mut sorted_values = values;
            sorted_values.sort_by(|left, right| {
                let ord = left.key.cmp(&right.key);
                if ord == Ordering::Equal {
                    right.seq.cmp(&left.seq)
                } else {
                    ord
                }
            });
            Self {
                values: sorted_values.into(),
            }
        }
    }

    /// A merge operator that routes to different merge strategies based on key prefix
    struct KeyPrefixMergeOperator;

    impl MergeOperator for KeyPrefixMergeOperator {
        fn merge(
            &self,
            key: &Bytes,
            existing_value: Option<Bytes>,
            value: Bytes,
        ) -> Result<Bytes, MergeOperatorError> {
            if key.starts_with(b"sum:") {
                // Sum merge for sum keys
                match existing_value {
                    Some(existing) => {
                        let existing_num =
                            u64::from_le_bytes(existing.as_ref().try_into().unwrap());
                        let new_num = u64::from_le_bytes(value.as_ref().try_into().unwrap());
                        Ok(Bytes::copy_from_slice(
                            &(existing_num + new_num).to_le_bytes(),
                        ))
                    }
                    None => Ok(value),
                }
            } else if key.starts_with(b"max:") {
                // Max merge for max keys
                match existing_value {
                    Some(existing) => {
                        let existing_num =
                            u64::from_le_bytes(existing.as_ref().try_into().unwrap());
                        let new_num = u64::from_le_bytes(value.as_ref().try_into().unwrap());
                        Ok(Bytes::copy_from_slice(
                            &existing_num.max(new_num).to_le_bytes(),
                        ))
                    }
                    None => Ok(value),
                }
            } else {
                // Default to concat for unknown prefixes
                match existing_value {
                    Some(existing) => {
                        let mut merged = existing.to_vec();
                        merged.extend_from_slice(&value);
                        Ok(Bytes::from(merged))
                    }
                    None => Ok(value),
                }
            }
        }

        // Override merge_batch to handle batches efficiently with key-based routing
        fn merge_batch(
            &self,
            key: &Bytes,
            existing_value: Option<Bytes>,
            operands: &[Bytes],
        ) -> Result<Bytes, MergeOperatorError> {
            if key.starts_with(b"max:") {
                // For max operator, find the maximum value across all operands
                let mut max_val = existing_value
                    .map(|v| u64::from_le_bytes(v.as_ref().try_into().unwrap()))
                    .unwrap_or(0);

                for operand in operands {
                    let val = u64::from_le_bytes(operand.as_ref().try_into().unwrap());
                    max_val = max_val.max(val);
                }

                Ok(Bytes::copy_from_slice(&max_val.to_le_bytes()))
            } else {
                // For other prefixes, use pairwise merge
                let mut result = existing_value;
                for operand in operands {
                    result = Some(self.merge(key, result, operand.clone())?);
                }
                result.ok_or(MergeOperatorError::EmptyBatch)
            }
        }
    }

    #[tokio::test]
    async fn should_route_merge_based_on_key_prefix() {
        let merge_operator = Arc::new(KeyPrefixMergeOperator {});

        let data = vec![
            // Sum key
            RowEntry::new_merge(b"sum:counter", &5u64.to_le_bytes(), 1),
            RowEntry::new_merge(b"sum:counter", &3u64.to_le_bytes(), 2),
            RowEntry::new_merge(b"sum:counter", &7u64.to_le_bytes(), 3),
            // Max key
            RowEntry::new_merge(b"max:score", &5u64.to_le_bytes(), 4),
            RowEntry::new_merge(b"max:score", &10u64.to_le_bytes(), 5),
            RowEntry::new_merge(b"max:score", &3u64.to_le_bytes(), 6),
        ];

        let mut iterator = MergeOperatorIterator::<MockKeyValueIterator>::new(
            merge_operator,
            data.into(),
            true,
            0,
            None,
        );

        // Expected: max should return 10, sum should return 15
        let max_expected = 10u64.to_le_bytes();
        let sum_expected = 15u64.to_le_bytes();

        assert_iterator(
            &mut iterator,
            vec![
                RowEntry::new_merge(b"max:score", &max_expected, 6),
                RowEntry::new_merge(b"sum:counter", &sum_expected, 3),
            ],
        )
        .await;
    }

    #[tokio::test]
    async fn test_batched_merge_with_many_operands() {
        let merge_operator = Arc::new(MockMergeOperator {});

        let mut data = vec![];
        for i in 1..=250 {
            data.push(RowEntry::new_merge(b"key1", &[i as u8], i));
        }

        let mut iterator = MergeOperatorIterator::<MockKeyValueIterator>::new(
            merge_operator,
            data.into(),
            true,
            0,
            None,
        );

        let expected_bytes: Vec<u8> = (1..=250).map(|i| i as u8).collect();
        let expected = vec![RowEntry::new_merge(b"key1", &expected_bytes, 250)];

        assert_iterator(&mut iterator, expected).await;
    }

    #[tokio::test]
    async fn test_batched_merge_with_base_value() {
        let merge_operator = Arc::new(MockMergeOperator {});

        let mut data = vec![];
        data.push(RowEntry::new_value(b"key1", b"BASE", 0));
        for i in 1..=150 {
            data.push(RowEntry::new_merge(b"key1", &[i as u8], i));
        }

        let mut iterator = MergeOperatorIterator::<MockKeyValueIterator>::new(
            merge_operator,
            data.into(),
            true,
            0,
            None,
        );

        let mut expected_bytes = b"BASE".to_vec();
        expected_bytes.extend((1..=150).map(|i| i as u8));
        let expected = vec![RowEntry::new_value(b"key1", &expected_bytes, 150)];

        assert_iterator(&mut iterator, expected).await;
    }

    #[tokio::test]
    async fn test_merge_batch_is_actually_called() {
        let (merge_operator, call_count) = MockBatchedMergeOperator::new();
        let merge_operator = Arc::new(merge_operator);

        let mut data = vec![];
        for i in 1..=250 {
            data.push(RowEntry::new_merge(b"key1", &[i as u8], i));
        }

        let mut iterator = MergeOperatorIterator::<MockKeyValueIterator>::new(
            merge_operator,
            data.into(),
            true,
            0,
            None,
        );

        let expected_bytes: Vec<u8> = (1..=250).map(|i| i as u8).collect();
        let expected = vec![RowEntry::new_merge(b"key1", &expected_bytes, 250)];
        assert_iterator(&mut iterator, expected).await;

        let actual_calls = call_count.load(std::sync::atomic::Ordering::SeqCst);
        assert_eq!(
            actual_calls, 4,
            "Expected merge_batch to be called 4 times for 250 operands (3 batches + 1 final merge), but was called {} times",
            actual_calls
        );
    }

    #[tokio::test]
    async fn test_merge_batch_with_base_value_call_count() {
        let (merge_operator, call_count) = MockBatchedMergeOperator::new();
        let merge_operator = Arc::new(merge_operator);

        // Create base value + 150 merge operands (will require 2 batches: 100 + 50)
        let mut data = vec![];
        data.push(RowEntry::new_value(b"key1", b"BASE", 0));
        for i in 1..=150 {
            data.push(RowEntry::new_merge(b"key1", &[i as u8], i));
        }

        let mut iterator = MergeOperatorIterator::<MockKeyValueIterator>::new(
            merge_operator,
            data.into(),
            true,
            0,
            None,
        );

        let mut expected_bytes = b"BASE".to_vec();
        expected_bytes.extend((1..=150).map(|i| i as u8));
        let expected = vec![RowEntry::new_value(b"key1", &expected_bytes, 150)];
        assert_iterator(&mut iterator, expected).await;

        let actual_calls = call_count.load(std::sync::atomic::Ordering::SeqCst);
        assert_eq!(
            actual_calls, 3,
            "Expected merge_batch to be called 3 times for 150 operands (2 batches + 1 final merge), but was called {} times",
            actual_calls
        );
    }

    #[tokio::test]
    async fn test_merge_operator_filters_expired_entries() {
        let merge_operator = Arc::new(MockMergeOperator {});

        // Create entries with different expiration times
        // now = 100, so entries with expire_ts <= 100 are expired
        // Entries are sorted by reverse seq, so we need the first entry (highest seq) to be non-expired
        // to properly initialize the tracker
        let data = vec![
            // Non-expired merge operands (expire_ts > 100) - highest seq first
            RowEntry::new_merge(b"key1", b"4", 4).with_expire_ts(300),
            RowEntry::new_merge(b"key1", b"2", 2).with_expire_ts(150),
            RowEntry::new_merge(b"key1", b"1", 1).with_expire_ts(200),
            // Expired merge operands (expire_ts <= 100) - should be filtered out
            RowEntry::new_merge(b"key1", b"5", 5).with_expire_ts(100),
            RowEntry::new_merge(b"key1", b"3", 3).with_expire_ts(50),
        ];

        let mut iterator = MergeOperatorIterator::<MockKeyValueIterator>::new(
            merge_operator,
            data.into(),
            true,
            100, // now = 100
            None,
        );

        // Only non-expired entries (4, 2, 1) should be merged
        // Entries with expire_ts 50 and 100 should be filtered out
        // Since entries are sorted by reverse seq, first entry is seq=5 (expired)
        // Tracker initializes with seq=5, but max_create_ts and min_expire_ts are None
        // Seq=5 is filtered out (expired), then seq=4, 2, 1 are added to batch
        // process_batch updates tracker: min_expire_ts becomes 150 (min of 300, 150, 200)
        // Batch is [4, 2, 1], reversed to [1, 2, 4], merged to "124"
        // Final seq is 5 (from first entry initialization), min_expire_ts is 150
        assert_iterator(
            &mut iterator,
            vec![RowEntry::new_merge(b"key1", b"124", 5).with_expire_ts(150)], // seq=5 from first entry, min_expire_ts=150 from non-expired entries
        )
        .await;
    }

    #[tokio::test]
    async fn test_merge_operator_filters_expired_entries_with_base_value() {
        let merge_operator = Arc::new(MockMergeOperator {});

        // Create entries with a base value and mixed expired/non-expired merge operands
        // Entries are sorted by reverse seq, so base value (seq=0) comes last
        // We need non-expired merge operands with higher seq to come first
        let data = vec![
            // Non-expired merge operands (higher seq first)
            RowEntry::new_merge(b"key1", b"4", 4).with_expire_ts(300),
            RowEntry::new_merge(b"key1", b"2", 2).with_expire_ts(250),
            RowEntry::new_merge(b"key1", b"1", 1).with_expire_ts(150),
            // Expired merge operand - should be filtered out
            RowEntry::new_merge(b"key1", b"3", 3).with_expire_ts(50),
            // Base value (non-expired) - comes last due to seq=0
            RowEntry::new_value(b"key1", b"BASE", 0).with_expire_ts(200),
        ];

        let mut iterator = MergeOperatorIterator::<MockKeyValueIterator>::new(
            merge_operator,
            data.into(),
            true,
            100, // now = 100,
            None,
        );

        // Base value + non-expired entries (4, 2, 1) should be merged
        // Entry with expire_ts 50 should be filtered out
        // First entry is seq=4 (non-expired), tracker starts with seq=4, but max_create_ts and min_expire_ts are None
        // Seq=4, 2, 1 are added to batch (all non-expired)
        // process_batch updates tracker: min_expire_ts becomes 150 (min of 300, 250, 150)
        // Batch is [4, 2, 1], reversed to [1, 2, 4], merged to "124", then merged with BASE to "BASE124"
        // Final seq is 4 (from first entry initialization), min_expire_ts is 150
        assert_iterator(
            &mut iterator,
            vec![RowEntry::new_value(b"key1", b"BASE124", 4).with_expire_ts(150)], // seq=4 from first entry, min_expire_ts=150
        )
        .await;
    }

    #[tokio::test]
    async fn test_merge_operator_handles_all_expired_entries() {
        let merge_operator = Arc::new(MockMergeOperator {});

        // All merge operands are expired
        let data = vec![
            RowEntry::new_merge(b"key1", b"1", 1).with_expire_ts(50),
            RowEntry::new_merge(b"key1", b"2", 2).with_expire_ts(80),
            RowEntry::new_merge(b"key1", b"3", 3).with_expire_ts(90),
        ];

        let mut iterator = MergeOperatorIterator::<MockKeyValueIterator>::new(
            merge_operator,
            data.into(),
            true,
            100, // now = 100, all entries are expired,
            None,
        );

        // All entries are expired, so nothing should be returned
        assert_iterator(&mut iterator, vec![]).await;
    }
}