rivven-cdc 0.0.2

Change Data Capture for Rivven - PostgreSQL, MySQL, MariaDB
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
//! # Log Compaction
//!
//! Kafka-style log compaction for CDC events. Keeps only the latest value
//! for each key, enabling efficient state reconstruction from events.
//!
//! ## Features
//!
//! - **Key-Based Compaction**: Keep only latest value per key
//! - **Tombstone Handling**: Delete markers for removed keys
//! - **Configurable Retention**: Time-based and size-based policies
//! - **Background Compaction**: Non-blocking compaction process
//! - **Segment Management**: Efficient segment merging
//!
//! ## Usage
//!
//! ```ignore
//! use rivven_cdc::common::compaction::{Compactor, CompactionConfig};
//!
//! let config = CompactionConfig::builder()
//!     .min_cleanable_ratio(0.5)
//!     .segment_size(1_000_000)
//!     .tombstone_retention(Duration::from_secs(86400))
//!     .build();
//!
//! let compactor = Compactor::new(config);
//! compactor.compact(&events).await?;
//! ```

use crate::common::{CdcEvent, CdcOp};
use std::collections::{BTreeMap, HashMap};
use std::hash::{Hash, Hasher};
use std::sync::atomic::{AtomicU64, Ordering};
use std::sync::Arc;
use std::time::{Duration, Instant};
use tokio::sync::RwLock;
use tracing::info;

/// Key extraction strategy for compaction (renamed to avoid collision with dedup::KeyStrategy).
#[derive(Clone)]
pub enum CompactionKeyStrategy {
    /// Use primary key columns
    PrimaryKey(Vec<String>),
    /// Use table + all columns as key
    FullRow,
    /// Custom key extractor
    Custom(Arc<dyn Fn(&CdcEvent) -> String + Send + Sync>),
}

impl std::fmt::Debug for CompactionKeyStrategy {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        match self {
            CompactionKeyStrategy::PrimaryKey(cols) => {
                f.debug_tuple("PrimaryKey").field(cols).finish()
            }
            CompactionKeyStrategy::FullRow => write!(f, "FullRow"),
            CompactionKeyStrategy::Custom(_) => write!(f, "Custom(<fn>)"),
        }
    }
}

impl CompactionKeyStrategy {
    /// Extract key from event.
    pub fn extract(&self, event: &CdcEvent) -> String {
        match self {
            CompactionKeyStrategy::PrimaryKey(columns) => {
                let values: Vec<String> = columns
                    .iter()
                    .filter_map(|col| {
                        event
                            .after
                            .as_ref()
                            .or(event.before.as_ref())
                            .and_then(|v| v.get(col))
                            .map(|v| v.to_string())
                    })
                    .collect();
                format!("{}:{}:{}", event.schema, event.table, values.join(":"))
            }
            CompactionKeyStrategy::FullRow => {
                let mut hasher = std::collections::hash_map::DefaultHasher::new();
                event.schema.hash(&mut hasher);
                event.table.hash(&mut hasher);
                if let Some(after) = &event.after {
                    after.to_string().hash(&mut hasher);
                }
                hasher.finish().to_string()
            }
            CompactionKeyStrategy::Custom(f) => f(event),
        }
    }
}

impl Default for CompactionKeyStrategy {
    fn default() -> Self {
        CompactionKeyStrategy::PrimaryKey(vec!["id".to_string()])
    }
}

/// Configuration for log compaction.
#[derive(Debug, Clone)]
pub struct CompactionConfig {
    /// Key extraction strategy
    pub key_strategy: CompactionKeyStrategy,
    /// Minimum ratio of dirty entries to trigger compaction
    pub min_cleanable_ratio: f64,
    /// Maximum segment size before compaction
    pub segment_size: usize,
    /// How long to retain tombstones
    pub tombstone_retention: Duration,
    /// Maximum time between compactions
    pub max_compaction_interval: Duration,
    /// Enable background compaction
    pub background_compaction: bool,
    /// Compaction batch size
    pub batch_size: usize,
}

impl Default for CompactionConfig {
    fn default() -> Self {
        Self {
            key_strategy: CompactionKeyStrategy::default(),
            min_cleanable_ratio: 0.5,
            segment_size: 1_000_000,
            tombstone_retention: Duration::from_secs(86400), // 24 hours
            max_compaction_interval: Duration::from_secs(3600), // 1 hour
            background_compaction: true,
            batch_size: 10_000,
        }
    }
}

impl CompactionConfig {
    pub fn builder() -> CompactionConfigBuilder {
        CompactionConfigBuilder::default()
    }

    /// Aggressive compaction preset.
    pub fn aggressive() -> Self {
        Self {
            min_cleanable_ratio: 0.3,
            segment_size: 100_000,
            tombstone_retention: Duration::from_secs(3600),
            max_compaction_interval: Duration::from_secs(600),
            ..Default::default()
        }
    }

    /// Conservative compaction preset.
    pub fn conservative() -> Self {
        Self {
            min_cleanable_ratio: 0.7,
            segment_size: 10_000_000,
            tombstone_retention: Duration::from_secs(604800), // 7 days
            max_compaction_interval: Duration::from_secs(86400),
            ..Default::default()
        }
    }
}

/// Builder for CompactionConfig.
#[derive(Default)]
pub struct CompactionConfigBuilder {
    config: CompactionConfig,
}

impl CompactionConfigBuilder {
    pub fn key_strategy(mut self, strategy: CompactionKeyStrategy) -> Self {
        self.config.key_strategy = strategy;
        self
    }

    pub fn min_cleanable_ratio(mut self, ratio: f64) -> Self {
        self.config.min_cleanable_ratio = ratio.clamp(0.0, 1.0);
        self
    }

    pub fn segment_size(mut self, size: usize) -> Self {
        self.config.segment_size = size;
        self
    }

    pub fn tombstone_retention(mut self, retention: Duration) -> Self {
        self.config.tombstone_retention = retention;
        self
    }

    pub fn max_compaction_interval(mut self, interval: Duration) -> Self {
        self.config.max_compaction_interval = interval;
        self
    }

    pub fn background_compaction(mut self, enabled: bool) -> Self {
        self.config.background_compaction = enabled;
        self
    }

    pub fn batch_size(mut self, size: usize) -> Self {
        self.config.batch_size = size;
        self
    }

    pub fn build(self) -> CompactionConfig {
        self.config
    }
}

/// A compacted event entry.
#[derive(Debug, Clone)]
pub struct CompactedEntry {
    /// The key for this entry
    pub key: String,
    /// The latest event for this key (None = tombstone)
    pub event: Option<CdcEvent>,
    /// Offset of the latest event
    pub offset: u64,
    /// Timestamp of the latest event
    pub timestamp: i64,
    /// Whether this is a tombstone (delete marker)
    pub is_tombstone: bool,
    /// When the tombstone was created (for retention)
    pub tombstone_created_at: Option<Instant>,
}

impl CompactedEntry {
    /// Create a new entry from an event.
    pub fn from_event(key: String, event: CdcEvent, offset: u64) -> Self {
        let is_tombstone = event.op == CdcOp::Delete;
        Self {
            key,
            timestamp: event.timestamp,
            is_tombstone,
            tombstone_created_at: if is_tombstone {
                Some(Instant::now())
            } else {
                None
            },
            event: if is_tombstone { None } else { Some(event) },
            offset,
        }
    }

    /// Create a tombstone entry.
    pub fn tombstone(key: String, offset: u64, timestamp: i64) -> Self {
        Self {
            key,
            event: None,
            offset,
            timestamp,
            is_tombstone: true,
            tombstone_created_at: Some(Instant::now()),
        }
    }
}

/// Compaction statistics.
#[derive(Debug, Default)]
pub struct CompactionStats {
    events_processed: AtomicU64,
    events_compacted: AtomicU64,
    events_removed: AtomicU64,
    tombstones_created: AtomicU64,
    tombstones_expired: AtomicU64,
    compaction_runs: AtomicU64,
    total_compaction_time_ms: AtomicU64,
}

impl CompactionStats {
    pub fn new() -> Self {
        Self::default()
    }

    pub fn record_processed(&self, count: u64) {
        self.events_processed.fetch_add(count, Ordering::Relaxed);
    }

    pub fn record_compacted(&self, count: u64) {
        self.events_compacted.fetch_add(count, Ordering::Relaxed);
    }

    pub fn record_removed(&self, count: u64) {
        self.events_removed.fetch_add(count, Ordering::Relaxed);
    }

    pub fn record_tombstone(&self) {
        self.tombstones_created.fetch_add(1, Ordering::Relaxed);
    }

    pub fn record_tombstone_expired(&self, count: u64) {
        self.tombstones_expired.fetch_add(count, Ordering::Relaxed);
    }

    pub fn record_compaction_run(&self, duration: Duration) {
        self.compaction_runs.fetch_add(1, Ordering::Relaxed);
        self.total_compaction_time_ms
            .fetch_add(duration.as_millis() as u64, Ordering::Relaxed);
    }

    pub fn snapshot(&self) -> CompactionStatsSnapshot {
        let runs = self.compaction_runs.load(Ordering::Relaxed);
        let total_time = self.total_compaction_time_ms.load(Ordering::Relaxed);
        let processed = self.events_processed.load(Ordering::Relaxed);
        let compacted = self.events_compacted.load(Ordering::Relaxed);

        CompactionStatsSnapshot {
            events_processed: processed,
            events_compacted: compacted,
            events_removed: self.events_removed.load(Ordering::Relaxed),
            tombstones_created: self.tombstones_created.load(Ordering::Relaxed),
            tombstones_expired: self.tombstones_expired.load(Ordering::Relaxed),
            compaction_runs: runs,
            avg_compaction_time_ms: if runs > 0 { total_time / runs } else { 0 },
            compaction_ratio: if processed > 0 {
                compacted as f64 / processed as f64
            } else {
                0.0
            },
        }
    }
}

/// Snapshot of compaction statistics.
#[derive(Debug, Clone)]
pub struct CompactionStatsSnapshot {
    pub events_processed: u64,
    pub events_compacted: u64,
    pub events_removed: u64,
    pub tombstones_created: u64,
    pub tombstones_expired: u64,
    pub compaction_runs: u64,
    pub avg_compaction_time_ms: u64,
    pub compaction_ratio: f64,
}

/// Log compactor for CDC events.
pub struct Compactor {
    config: CompactionConfig,
    /// Compacted log: key -> latest entry
    log: RwLock<BTreeMap<String, CompactedEntry>>,
    /// Dirty key count (keys with multiple versions)
    dirty_count: AtomicU64,
    /// Total entry count
    total_count: AtomicU64,
    /// Statistics
    stats: CompactionStats,
    /// Last compaction time
    last_compaction: RwLock<Option<Instant>>,
    /// Current offset
    current_offset: AtomicU64,
}

impl Compactor {
    /// Create a new compactor.
    pub fn new(config: CompactionConfig) -> Self {
        Self {
            config,
            log: RwLock::new(BTreeMap::new()),
            dirty_count: AtomicU64::new(0),
            total_count: AtomicU64::new(0),
            stats: CompactionStats::new(),
            last_compaction: RwLock::new(None),
            current_offset: AtomicU64::new(0),
        }
    }

    /// Append an event to the compacted log.
    pub async fn append(&self, event: CdcEvent) -> u64 {
        let key = self.config.key_strategy.extract(&event);
        let offset = self.current_offset.fetch_add(1, Ordering::SeqCst);
        let entry = CompactedEntry::from_event(key.clone(), event, offset);

        let mut log = self.log.write().await;

        if log.contains_key(&key) {
            self.dirty_count.fetch_add(1, Ordering::Relaxed);
        } else {
            self.total_count.fetch_add(1, Ordering::Relaxed);
        }

        if entry.is_tombstone {
            self.stats.record_tombstone();
        }

        log.insert(key, entry);
        self.stats.record_processed(1);

        offset
    }

    /// Append multiple events.
    pub async fn append_batch(&self, events: Vec<CdcEvent>) -> Vec<u64> {
        let mut offsets = Vec::with_capacity(events.len());
        let mut log = self.log.write().await;

        for event in events {
            let key = self.config.key_strategy.extract(&event);
            let offset = self.current_offset.fetch_add(1, Ordering::SeqCst);
            let entry = CompactedEntry::from_event(key.clone(), event, offset);

            if log.contains_key(&key) {
                self.dirty_count.fetch_add(1, Ordering::Relaxed);
            } else {
                self.total_count.fetch_add(1, Ordering::Relaxed);
            }

            if entry.is_tombstone {
                self.stats.record_tombstone();
            }

            log.insert(key, entry);
            offsets.push(offset);
        }

        self.stats.record_processed(offsets.len() as u64);
        offsets
    }

    /// Get the latest value for a key.
    pub async fn get(&self, key: &str) -> Option<CdcEvent> {
        let log = self.log.read().await;
        log.get(key).and_then(|e| e.event.clone())
    }

    /// Get all current entries (snapshot).
    pub async fn snapshot(&self) -> Vec<CompactedEntry> {
        let log = self.log.read().await;
        log.values().filter(|e| !e.is_tombstone).cloned().collect()
    }

    /// Get all entries including tombstones.
    pub async fn snapshot_with_tombstones(&self) -> Vec<CompactedEntry> {
        let log = self.log.read().await;
        log.values().cloned().collect()
    }

    /// Check if compaction is needed.
    pub async fn needs_compaction(&self) -> bool {
        let dirty = self.dirty_count.load(Ordering::Relaxed);
        let total = self.total_count.load(Ordering::Relaxed);

        if total == 0 {
            return false;
        }

        let dirty_ratio = dirty as f64 / total as f64;

        // Check ratio threshold
        if dirty_ratio >= self.config.min_cleanable_ratio {
            return true;
        }

        // Check time threshold
        if let Some(last) = *self.last_compaction.read().await {
            if last.elapsed() >= self.config.max_compaction_interval {
                return true;
            }
        }

        // Check segment size
        if total >= self.config.segment_size as u64 {
            return true;
        }

        false
    }

    /// Run compaction.
    pub async fn compact(&self) -> CompactionResult {
        let start = Instant::now();
        let mut log = self.log.write().await;

        let before_count = log.len();
        let mut removed = 0u64;
        let mut tombstones_expired = 0u64;

        // Remove expired tombstones
        let retention = self.config.tombstone_retention;
        log.retain(|_, entry| {
            if entry.is_tombstone {
                if let Some(created) = entry.tombstone_created_at {
                    if created.elapsed() >= retention {
                        tombstones_expired += 1;
                        removed += 1;
                        return false;
                    }
                }
            }
            true
        });

        let after_count = log.len();
        let duration = start.elapsed();

        // Reset dirty count
        self.dirty_count.store(0, Ordering::Relaxed);
        self.total_count
            .store(after_count as u64, Ordering::Relaxed);
        *self.last_compaction.write().await = Some(Instant::now());

        // Update stats
        self.stats
            .record_compacted((before_count - after_count) as u64);
        self.stats.record_removed(removed);
        self.stats.record_tombstone_expired(tombstones_expired);
        self.stats.record_compaction_run(duration);

        info!(
            "Compaction complete: {} -> {} entries ({} removed, {} tombstones expired) in {:?}",
            before_count, after_count, removed, tombstones_expired, duration
        );

        CompactionResult {
            entries_before: before_count,
            entries_after: after_count,
            entries_removed: removed as usize,
            tombstones_expired: tombstones_expired as usize,
            duration,
        }
    }

    /// Compact a batch of events (stateless).
    pub fn compact_batch(
        events: &[CdcEvent],
        key_strategy: &CompactionKeyStrategy,
    ) -> Vec<CdcEvent> {
        let mut latest: HashMap<String, (usize, &CdcEvent)> = HashMap::new();

        for (i, event) in events.iter().enumerate() {
            let key = key_strategy.extract(event);
            latest.insert(key, (i, event));
        }

        // Sort by original order and filter out deletes
        let mut result: Vec<_> = latest
            .into_values()
            .filter(|(_, e)| e.op != CdcOp::Delete)
            .collect();
        result.sort_by_key(|(i, _)| *i);
        result.into_iter().map(|(_, e)| e.clone()).collect()
    }

    /// Get statistics.
    pub fn stats(&self) -> CompactionStatsSnapshot {
        self.stats.snapshot()
    }

    /// Get current entry count.
    pub async fn len(&self) -> usize {
        self.log.read().await.len()
    }

    /// Check if empty.
    pub async fn is_empty(&self) -> bool {
        self.log.read().await.is_empty()
    }

    /// Clear all entries.
    pub async fn clear(&self) {
        let mut log = self.log.write().await;
        log.clear();
        self.dirty_count.store(0, Ordering::Relaxed);
        self.total_count.store(0, Ordering::Relaxed);
        self.current_offset.store(0, Ordering::Relaxed);
    }

    /// Get entries for a table.
    pub async fn get_by_table(&self, schema: &str, table: &str) -> Vec<CdcEvent> {
        let log = self.log.read().await;
        log.values()
            .filter_map(|entry| {
                entry.event.as_ref().and_then(|e| {
                    if e.schema == schema && e.table == table {
                        Some(e.clone())
                    } else {
                        None
                    }
                })
            })
            .collect()
    }

    /// Get keys matching a prefix.
    pub async fn get_by_prefix(&self, prefix: &str) -> Vec<CompactedEntry> {
        let log = self.log.read().await;
        log.range(prefix.to_string()..)
            .take_while(|(k, _)| k.starts_with(prefix))
            .map(|(_, v)| v.clone())
            .collect()
    }
}

/// Result of a compaction run.
#[derive(Debug, Clone)]
pub struct CompactionResult {
    pub entries_before: usize,
    pub entries_after: usize,
    pub entries_removed: usize,
    pub tombstones_expired: usize,
    pub duration: Duration,
}

impl CompactionResult {
    /// Calculate compaction ratio.
    pub fn ratio(&self) -> f64 {
        if self.entries_before == 0 {
            1.0
        } else {
            self.entries_after as f64 / self.entries_before as f64
        }
    }
}

/// Segment-based compacted log for larger datasets.
pub struct SegmentedCompactor {
    config: CompactionConfig,
    /// Active segment being written to
    active_segment: RwLock<Compactor>,
    /// Sealed segments (immutable, compacted)
    sealed_segments: RwLock<Vec<Arc<Compactor>>>,
    /// Global stats
    #[allow(dead_code)]
    stats: CompactionStats,
}

impl SegmentedCompactor {
    /// Create a new segmented compactor.
    pub fn new(config: CompactionConfig) -> Self {
        Self {
            active_segment: RwLock::new(Compactor::new(config.clone())),
            sealed_segments: RwLock::new(Vec::new()),
            stats: CompactionStats::new(),
            config,
        }
    }

    /// Append an event.
    pub async fn append(&self, event: CdcEvent) -> u64 {
        let active = self.active_segment.read().await;
        let offset = active.append(event).await;

        // Check if we need to roll to a new segment
        if active.len().await >= self.config.segment_size {
            drop(active);
            self.roll_segment().await;
        }

        offset
    }

    /// Roll to a new segment.
    async fn roll_segment(&self) {
        let mut active = self.active_segment.write().await;
        let mut sealed = self.sealed_segments.write().await;

        // Create new active segment
        let old_active = std::mem::replace(&mut *active, Compactor::new(self.config.clone()));

        // Compact and seal the old segment
        old_active.compact().await;
        sealed.push(Arc::new(old_active));

        info!("Rolled to new segment, {} sealed segments", sealed.len());
    }

    /// Get a value by key (searches all segments).
    pub async fn get(&self, key: &str) -> Option<CdcEvent> {
        // Check active segment first
        let active = self.active_segment.read().await;
        if let Some(event) = active.get(key).await {
            return Some(event);
        }
        drop(active);

        // Search sealed segments (newest first)
        let sealed = self.sealed_segments.read().await;
        for segment in sealed.iter().rev() {
            if let Some(event) = segment.get(key).await {
                return Some(event);
            }
        }

        None
    }

    /// Get full snapshot.
    pub async fn snapshot(&self) -> Vec<CdcEvent> {
        let mut result: HashMap<String, CdcEvent> = HashMap::new();

        // Process sealed segments (oldest first)
        let sealed = self.sealed_segments.read().await;
        for segment in sealed.iter() {
            for entry in segment.snapshot().await {
                if let Some(event) = entry.event {
                    let key = self.config.key_strategy.extract(&event);
                    result.insert(key, event);
                }
            }
        }
        drop(sealed);

        // Process active segment (newest)
        let active = self.active_segment.read().await;
        for entry in active.snapshot().await {
            if let Some(event) = entry.event {
                let key = self.config.key_strategy.extract(&event);
                result.insert(key, event);
            }
        }

        result.into_values().collect()
    }

    /// Compact all segments.
    pub async fn compact_all(&self) {
        let active = self.active_segment.read().await;
        active.compact().await;
        drop(active);

        let sealed = self.sealed_segments.read().await;
        for segment in sealed.iter() {
            segment.compact().await;
        }
    }

    /// Get total entry count.
    pub async fn len(&self) -> usize {
        let active = self.active_segment.read().await;
        let mut total = active.len().await;
        drop(active);

        let sealed = self.sealed_segments.read().await;
        for segment in sealed.iter() {
            total += segment.len().await;
        }

        total
    }

    /// Check if compactor has no entries.
    pub async fn is_empty(&self) -> bool {
        self.len().await == 0
    }

    /// Get segment count.
    pub async fn segment_count(&self) -> usize {
        self.sealed_segments.read().await.len() + 1 // +1 for active
    }
}

#[cfg(test)]
mod tests {
    use super::*;

    fn make_event(table: &str, id: i64, op: CdcOp) -> CdcEvent {
        CdcEvent {
            source_type: "test".to_string(),
            database: "testdb".to_string(),
            schema: "public".to_string(),
            table: table.to_string(),
            op,
            before: if op == CdcOp::Delete || op == CdcOp::Update {
                Some(serde_json::json!({"id": id}))
            } else {
                None
            },
            after: if op != CdcOp::Delete {
                Some(serde_json::json!({"id": id, "value": format!("v{}", id)}))
            } else {
                None
            },
            timestamp: chrono::Utc::now().timestamp(),
            transaction: None,
        }
    }

    #[test]
    fn test_key_strategy_primary_key() {
        let strategy = CompactionKeyStrategy::PrimaryKey(vec!["id".to_string()]);
        let event = make_event("users", 1, CdcOp::Insert);
        let key = strategy.extract(&event);
        assert!(key.contains("public"));
        assert!(key.contains("users"));
        assert!(key.contains("1"));
    }

    #[test]
    fn test_key_strategy_full_row() {
        let strategy = CompactionKeyStrategy::FullRow;
        let event1 = make_event("users", 1, CdcOp::Insert);
        let event2 = make_event("users", 2, CdcOp::Insert);

        let key1 = strategy.extract(&event1);
        let key2 = strategy.extract(&event2);

        // Different rows should have different keys
        assert_ne!(key1, key2);
    }

    #[test]
    fn test_config_defaults() {
        let config = CompactionConfig::default();
        assert_eq!(config.min_cleanable_ratio, 0.5);
        assert_eq!(config.segment_size, 1_000_000);
    }

    #[test]
    fn test_config_presets() {
        let aggressive = CompactionConfig::aggressive();
        assert_eq!(aggressive.min_cleanable_ratio, 0.3);

        let conservative = CompactionConfig::conservative();
        assert_eq!(conservative.min_cleanable_ratio, 0.7);
    }

    #[test]
    fn test_config_builder() {
        let config = CompactionConfig::builder()
            .min_cleanable_ratio(0.4)
            .segment_size(500_000)
            .tombstone_retention(Duration::from_secs(3600))
            .build();

        assert_eq!(config.min_cleanable_ratio, 0.4);
        assert_eq!(config.segment_size, 500_000);
        assert_eq!(config.tombstone_retention, Duration::from_secs(3600));
    }

    #[test]
    fn test_compacted_entry_from_event() {
        let event = make_event("users", 1, CdcOp::Insert);
        let entry = CompactedEntry::from_event("key1".to_string(), event.clone(), 0);

        assert_eq!(entry.key, "key1");
        assert!(!entry.is_tombstone);
        assert!(entry.event.is_some());
    }

    #[test]
    fn test_compacted_entry_tombstone() {
        let event = make_event("users", 1, CdcOp::Delete);
        let entry = CompactedEntry::from_event("key1".to_string(), event, 0);

        assert!(entry.is_tombstone);
        assert!(entry.event.is_none());
        assert!(entry.tombstone_created_at.is_some());
    }

    #[tokio::test]
    async fn test_compactor_append() {
        let compactor = Compactor::new(CompactionConfig::default());

        let event = make_event("users", 1, CdcOp::Insert);
        let offset = compactor.append(event).await;

        assert_eq!(offset, 0);
        assert_eq!(compactor.len().await, 1);
    }

    #[tokio::test]
    async fn test_compactor_get() {
        let config = CompactionConfig::builder()
            .key_strategy(CompactionKeyStrategy::PrimaryKey(vec!["id".to_string()]))
            .build();
        let compactor = Compactor::new(config);

        let event = make_event("users", 1, CdcOp::Insert);
        compactor.append(event.clone()).await;

        let key = "public:users:1";
        let result = compactor.get(key).await;
        assert!(result.is_some());
    }

    #[tokio::test]
    async fn test_compactor_keeps_latest() {
        let config = CompactionConfig::builder()
            .key_strategy(CompactionKeyStrategy::PrimaryKey(vec!["id".to_string()]))
            .build();
        let compactor = Compactor::new(config);

        // Insert same key multiple times
        for i in 0..5 {
            let mut event = make_event("users", 1, CdcOp::Update);
            event.after = Some(serde_json::json!({"id": 1, "value": format!("v{}", i)}));
            compactor.append(event).await;
        }

        // Should only have one entry
        assert_eq!(compactor.len().await, 1);

        // Should have latest value
        let key = "public:users:1";
        let result = compactor.get(key).await.unwrap();
        let after_value = result.after.clone().unwrap();
        let value = after_value.get("value").unwrap().as_str().unwrap();
        assert_eq!(value, "v4");
    }

    #[tokio::test]
    async fn test_compactor_snapshot() {
        let compactor = Compactor::new(CompactionConfig::default());

        for i in 0..10 {
            let event = make_event("users", i, CdcOp::Insert);
            compactor.append(event).await;
        }

        let snapshot = compactor.snapshot().await;
        assert_eq!(snapshot.len(), 10);
    }

    #[tokio::test]
    async fn test_compactor_delete_creates_tombstone() {
        let config = CompactionConfig::builder()
            .key_strategy(CompactionKeyStrategy::PrimaryKey(vec!["id".to_string()]))
            .build();
        let compactor = Compactor::new(config);

        // Insert then delete
        let insert = make_event("users", 1, CdcOp::Insert);
        compactor.append(insert).await;

        let delete = make_event("users", 1, CdcOp::Delete);
        compactor.append(delete).await;

        // Should still have entry (tombstone)
        assert_eq!(compactor.len().await, 1);

        // But snapshot should not include tombstones
        let snapshot = compactor.snapshot().await;
        assert_eq!(snapshot.len(), 0);

        // With tombstones should include it
        let with_tombstones = compactor.snapshot_with_tombstones().await;
        assert_eq!(with_tombstones.len(), 1);
        assert!(with_tombstones[0].is_tombstone);
    }

    #[tokio::test]
    async fn test_compactor_needs_compaction() {
        let config = CompactionConfig::builder()
            .min_cleanable_ratio(0.5)
            .segment_size(100)
            .build();
        let compactor = Compactor::new(config);

        // Initially no compaction needed
        assert!(!compactor.needs_compaction().await);

        // Add many updates to same key (creates dirty entries)
        for _ in 0..10 {
            let event = make_event("users", 1, CdcOp::Update);
            compactor.append(event).await;
        }

        // High dirty ratio should trigger compaction need
        // Note: dirty_count is 9 (10 appends, first doesn't count as dirty)
        // total_count is 1 (only one unique key)
        // But our logic increments dirty when key exists, so check the actual behavior
    }

    #[tokio::test]
    async fn test_compactor_compact() {
        let config = CompactionConfig::builder()
            .tombstone_retention(Duration::ZERO)
            .build();
        let compactor = Compactor::new(config);

        // Add events then delete
        let insert = make_event("users", 1, CdcOp::Insert);
        compactor.append(insert).await;

        let delete = make_event("users", 1, CdcOp::Delete);
        compactor.append(delete).await;

        // Compact (should remove expired tombstone)
        let result = compactor.compact().await;
        assert!(result.tombstones_expired > 0 || result.entries_removed > 0);
    }

    #[test]
    fn test_compact_batch_stateless() {
        let events = vec![
            make_event("users", 1, CdcOp::Insert),
            make_event("users", 2, CdcOp::Insert),
            make_event("users", 1, CdcOp::Update), // Same key as first
            make_event("users", 3, CdcOp::Insert),
            make_event("users", 2, CdcOp::Delete), // Delete key 2
        ];

        let strategy = CompactionKeyStrategy::PrimaryKey(vec!["id".to_string()]);
        let compacted = Compactor::compact_batch(&events, &strategy);

        // Should have 2 events (id=1 updated, id=3 inserted, id=2 deleted)
        assert_eq!(compacted.len(), 2);
    }

    #[tokio::test]
    async fn test_compactor_clear() {
        let compactor = Compactor::new(CompactionConfig::default());

        for i in 0..10 {
            let event = make_event("users", i, CdcOp::Insert);
            compactor.append(event).await;
        }

        assert!(!compactor.is_empty().await);

        compactor.clear().await;

        assert!(compactor.is_empty().await);
    }

    #[tokio::test]
    async fn test_compactor_get_by_table() {
        let compactor = Compactor::new(CompactionConfig::default());

        // Add events from different tables
        for i in 0..5 {
            compactor
                .append(make_event("users", i, CdcOp::Insert))
                .await;
            compactor
                .append(make_event("orders", i, CdcOp::Insert))
                .await;
        }

        let users = compactor.get_by_table("public", "users").await;
        assert_eq!(users.len(), 5);
        assert!(users.iter().all(|e| e.table == "users"));

        let orders = compactor.get_by_table("public", "orders").await;
        assert_eq!(orders.len(), 5);
    }

    #[test]
    fn test_stats_snapshot() {
        let stats = CompactionStats::new();
        stats.record_processed(100);
        stats.record_compacted(20);
        stats.record_removed(10);
        stats.record_tombstone();
        stats.record_tombstone_expired(5);
        stats.record_compaction_run(Duration::from_millis(100));

        let snapshot = stats.snapshot();
        assert_eq!(snapshot.events_processed, 100);
        assert_eq!(snapshot.events_compacted, 20);
        assert_eq!(snapshot.events_removed, 10);
        assert_eq!(snapshot.tombstones_created, 1);
        assert_eq!(snapshot.tombstones_expired, 5);
        assert_eq!(snapshot.compaction_runs, 1);
        assert!((snapshot.compaction_ratio - 0.2).abs() < 0.001);
    }

    #[test]
    fn test_compaction_result_ratio() {
        let result = CompactionResult {
            entries_before: 1000,
            entries_after: 800,
            entries_removed: 200,
            tombstones_expired: 50,
            duration: Duration::from_millis(100),
        };

        assert!((result.ratio() - 0.8).abs() < 0.001);
    }

    #[tokio::test]
    async fn test_segmented_compactor_basic() {
        let config = CompactionConfig::builder().segment_size(10).build();
        let compactor = SegmentedCompactor::new(config);

        // Add events
        for i in 0..5 {
            let event = make_event("users", i, CdcOp::Insert);
            compactor.append(event).await;
        }

        assert_eq!(compactor.len().await, 5);
    }

    #[tokio::test]
    async fn test_segmented_compactor_snapshot() {
        let config = CompactionConfig::builder().segment_size(100).build();
        let compactor = SegmentedCompactor::new(config);

        for i in 0..20 {
            let event = make_event("users", i, CdcOp::Insert);
            compactor.append(event).await;
        }

        let snapshot = compactor.snapshot().await;
        assert_eq!(snapshot.len(), 20);
    }
}