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
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
//! # CDC Snapshot Support
//!
//! Initial table synchronization before streaming changes.
//!
//! ## Features
//!
//! - Multiple snapshot modes (Debezium-compatible)
//! - Chunked reads with configurable batch size
//! - Progress tracking and resumability
//! - Watermark-based consistency
//! - Parallel table snapshots
//! - Memory-efficient streaming
//!
//! ## Snapshot Modes
//!
//! | Mode | Description |
//! |------|-------------|
//! | `Initial` | Snapshot on first start, then stream (default) |
//! | `Always` | Snapshot on every start |
//! | `InitialOnly` | Snapshot and stop (no streaming) |
//! | `SchemaOnly` | Capture schema, skip data |
//! | `WhenNeeded` | Snapshot if offsets unavailable |
//! | `Recovery` | Rebuild schema from source |
//! | `Custom` | User-defined snapshot logic |
//!
//! ## Example
//!
//! ```rust,ignore
//! use rivven_cdc::common::snapshot::{SnapshotConfig, SnapshotCoordinator, SnapshotMode};
//!
//! let config = SnapshotConfig::builder()
//!     .mode(SnapshotMode::Initial)
//!     .batch_size(10_000)
//!     .parallel_tables(4)
//!     .build();
//!
//! let coordinator = SnapshotCoordinator::new(config, progress_store);
//! let mut stream = coordinator.snapshot_table("users", "id").await?;
//!
//! while let Some(batch) = stream.next().await {
//!     process_batch(batch?).await;
//! }
//! ```

use crate::common::{CdcError, CdcEvent, CdcOp, Result};
use serde::{Deserialize, Serialize};
use std::collections::HashMap;
use std::sync::atomic::{AtomicBool, AtomicU64, Ordering};
use std::sync::Arc;
use std::time::{Duration, Instant};
use tokio::sync::RwLock;

/// Snapshot mode determines when and how snapshots are taken.
///
/// These modes align with Debezium's snapshot.mode configuration for
/// compatibility and familiar behavior.
///
/// # Examples
///
/// ```rust
/// use rivven_cdc::common::SnapshotMode;
///
/// // Default mode: snapshot on first start
/// let mode = SnapshotMode::Initial;
///
/// // Always snapshot on start (useful for debugging)
/// let mode = SnapshotMode::Always;
///
/// // Custom snapshot logic with user-defined function
/// let mode = SnapshotMode::Custom("my_snapshotter".to_string());
/// ```
#[derive(Debug, Clone, PartialEq, Eq, Default, Serialize, Deserialize)]
#[serde(rename_all = "snake_case")]
pub enum SnapshotMode {
    /// Perform a snapshot every time the connector starts.
    ///
    /// After the snapshot completes, the connector begins streaming changes.
    /// Useful when WAL segments may have been deleted.
    Always,

    /// Perform a snapshot only on first start (when no offsets exist).
    ///
    /// This is the default and most common mode. After the initial snapshot
    /// completes, the connector streams changes and never snapshots again
    /// unless offsets are lost.
    #[default]
    Initial,

    /// Perform a snapshot and then stop (no streaming).
    ///
    /// Useful for one-time data migration or backfill scenarios.
    InitialOnly,

    /// Capture schema only, skip data snapshot.
    ///
    /// The connector captures table structures but does not snapshot data.
    /// Only changes occurring after connector start are captured.
    /// Alias: `no_data` (Debezium naming)
    SchemaOnly,

    /// Perform a snapshot only if offsets are unavailable.
    ///
    /// The connector checks for existing offsets:
    /// - If offsets exist: resume streaming from stored position
    /// - If no offsets: perform a full snapshot first
    WhenNeeded,

    /// Recovery mode for corrupted schema history.
    ///
    /// Rebuilds the schema from source tables. Use after schema history
    /// topic corruption or when adding new tables to capture.
    ///
    /// **Warning**: Do not use if schema changes occurred after last shutdown.
    Recovery,

    /// Configuration-based snapshot control.
    ///
    /// Fine-grained control via additional parameters:
    /// - `snapshot_data`: include table data
    /// - `snapshot_schema`: include table schema  
    /// - `start_stream`: begin streaming after snapshot
    ConfigurationBased {
        /// Include table data in snapshot
        snapshot_data: bool,
        /// Include table schema in snapshot
        snapshot_schema: bool,
        /// Start streaming after snapshot completes
        start_stream: bool,
        /// Snapshot if schema history unavailable
        snapshot_on_schema_error: bool,
        /// Snapshot if offsets not found in log
        snapshot_on_data_error: bool,
    },

    /// Custom snapshot implementation.
    ///
    /// Provide a custom snapshotter name that implements the
    /// snapshot logic. The name is used to look up the implementation.
    Custom(String),

    /// Never perform a snapshot.
    ///
    /// The connector only streams changes. If no offsets exist,
    /// streaming begins from the current position.
    ///
    /// **Warning**: May miss historical data. Use only when certain
    /// all needed data is still in the transaction log.
    #[serde(alias = "never")]
    NoSnapshot,
}

impl SnapshotMode {
    /// Check if this mode includes a data snapshot.
    pub fn includes_data(&self) -> bool {
        match self {
            Self::Always | Self::Initial | Self::InitialOnly | Self::WhenNeeded => true,
            Self::SchemaOnly | Self::Recovery | Self::NoSnapshot => false,
            Self::ConfigurationBased { snapshot_data, .. } => *snapshot_data,
            Self::Custom(_) => true, // Assume custom may include data
        }
    }

    /// Check if streaming should occur after snapshot.
    pub fn should_stream(&self) -> bool {
        match self {
            Self::Always | Self::Initial | Self::WhenNeeded | Self::SchemaOnly | Self::Recovery => {
                true
            }
            Self::InitialOnly | Self::NoSnapshot => false,
            Self::ConfigurationBased { start_stream, .. } => *start_stream,
            Self::Custom(_) => true, // Assume custom may stream
        }
    }

    /// Check if schema should be captured.
    pub fn includes_schema(&self) -> bool {
        match self {
            Self::NoSnapshot => false,
            Self::ConfigurationBased {
                snapshot_schema, ..
            } => *snapshot_schema,
            _ => true,
        }
    }

    /// Parse from Debezium-compatible string.
    pub fn from_str_debezium(s: &str) -> Option<Self> {
        match s.to_lowercase().as_str() {
            "always" => Some(Self::Always),
            "initial" => Some(Self::Initial),
            "initial_only" => Some(Self::InitialOnly),
            "schema_only" | "no_data" => Some(Self::SchemaOnly),
            "when_needed" => Some(Self::WhenNeeded),
            "recovery" | "schema_only_recovery" => Some(Self::Recovery),
            "never" => Some(Self::NoSnapshot),
            "configuration_based" => Some(Self::ConfigurationBased {
                snapshot_data: false,
                snapshot_schema: false,
                start_stream: false,
                snapshot_on_schema_error: false,
                snapshot_on_data_error: false,
            }),
            _ if s.starts_with("custom:") => Some(Self::Custom(s[7..].to_string())),
            _ => None,
        }
    }
}

/// Snapshot state for a table.
#[derive(Debug, Clone, Copy, PartialEq, Eq, Serialize, Deserialize, Default)]
pub enum SnapshotState {
    /// Not started
    #[default]
    Pending,
    /// Currently running
    InProgress,
    /// Completed successfully
    Completed,
    /// Failed with error
    Failed,
    /// Paused (can resume)
    Paused,
}

/// Progress information for a table snapshot.
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct SnapshotProgress {
    /// Table being snapshotted
    pub table: String,
    /// Schema name
    pub schema: String,
    /// Current state
    pub state: SnapshotState,
    /// Rows processed so far
    pub rows_processed: u64,
    /// Estimated total rows (if known)
    pub total_rows: Option<u64>,
    /// Last processed key (for resumption)
    pub last_key: Option<String>,
    /// Start timestamp
    pub started_at: Option<i64>,
    /// Completion timestamp
    pub completed_at: Option<i64>,
    /// Error message if failed
    pub error: Option<String>,
    /// Watermark LSN/position at snapshot start
    pub watermark: Option<String>,
}

impl SnapshotProgress {
    /// Create new progress for a table.
    pub fn new(schema: impl Into<String>, table: impl Into<String>) -> Self {
        Self {
            table: table.into(),
            schema: schema.into(),
            state: SnapshotState::Pending,
            rows_processed: 0,
            total_rows: None,
            last_key: None,
            started_at: None,
            completed_at: None,
            error: None,
            watermark: None,
        }
    }

    /// Start the snapshot.
    pub fn start(&mut self, watermark: Option<String>) {
        self.state = SnapshotState::InProgress;
        self.started_at = Some(chrono::Utc::now().timestamp());
        self.watermark = watermark;
    }

    /// Update progress.
    pub fn update(&mut self, rows: u64, last_key: Option<String>) {
        self.rows_processed += rows;
        if last_key.is_some() {
            self.last_key = last_key;
        }
    }

    /// Mark as completed.
    pub fn complete(&mut self) {
        self.state = SnapshotState::Completed;
        self.completed_at = Some(chrono::Utc::now().timestamp());
    }

    /// Mark as failed.
    pub fn fail(&mut self, error: impl Into<String>) {
        self.state = SnapshotState::Failed;
        self.error = Some(error.into());
        self.completed_at = Some(chrono::Utc::now().timestamp());
    }

    /// Pause the snapshot.
    pub fn pause(&mut self) {
        self.state = SnapshotState::Paused;
    }

    /// Get completion percentage.
    pub fn percentage(&self) -> Option<f64> {
        self.total_rows.map(|total| {
            if total == 0 {
                100.0
            } else {
                (self.rows_processed as f64 / total as f64) * 100.0
            }
        })
    }

    /// Check if can resume.
    pub fn can_resume(&self) -> bool {
        matches!(self.state, SnapshotState::Paused | SnapshotState::Failed)
            && self.last_key.is_some()
    }
}

/// Configuration for snapshots.
#[derive(Debug, Clone)]
pub struct SnapshotConfig {
    /// Snapshot mode (when to snapshot)
    pub mode: SnapshotMode,
    /// Rows per batch (Debezium: snapshot.fetch.size)
    pub batch_size: usize,
    /// Number of tables to snapshot in parallel (Debezium: snapshot.max.threads)
    pub parallel_tables: usize,
    /// Query timeout
    pub query_timeout: Duration,
    /// Progress save interval
    pub progress_interval: Duration,
    /// Use consistent read (snapshot isolation)
    pub consistent_read: bool,
    /// Include row count estimate
    pub estimate_rows: bool,
    /// Maximum retries per batch
    pub max_retries: u32,
    /// Throttle delay between batches
    pub throttle_delay: Option<Duration>,
    /// Delay before starting snapshot (Debezium: snapshot.delay.ms)
    pub snapshot_delay: Option<Duration>,
    /// Delay before streaming after snapshot (Debezium: streaming.delay.ms)
    pub streaming_delay: Option<Duration>,
    /// Lock timeout for acquiring table locks (Debezium: snapshot.lock.timeout.ms)
    pub lock_timeout: Option<Duration>,
}

impl Default for SnapshotConfig {
    fn default() -> Self {
        Self {
            mode: SnapshotMode::Initial,
            batch_size: 10_000,
            parallel_tables: 4,
            query_timeout: Duration::from_secs(300),
            progress_interval: Duration::from_secs(10),
            consistent_read: true,
            estimate_rows: true,
            max_retries: 3,
            throttle_delay: None,
            snapshot_delay: None,
            streaming_delay: None,
            lock_timeout: Some(Duration::from_secs(10)),
        }
    }
}

impl SnapshotConfig {
    pub fn builder() -> SnapshotConfigBuilder {
        SnapshotConfigBuilder::default()
    }

    /// High-throughput preset.
    pub fn high_throughput() -> Self {
        Self {
            mode: SnapshotMode::Initial,
            batch_size: 50_000,
            parallel_tables: 8,
            query_timeout: Duration::from_secs(600),
            progress_interval: Duration::from_secs(30),
            consistent_read: false,
            estimate_rows: false,
            max_retries: 5,
            throttle_delay: None,
            snapshot_delay: None,
            streaming_delay: None,
            lock_timeout: None,
        }
    }

    /// Memory-efficient preset.
    pub fn low_memory() -> Self {
        Self {
            mode: SnapshotMode::Initial,
            batch_size: 1_000,
            parallel_tables: 2,
            query_timeout: Duration::from_secs(120),
            progress_interval: Duration::from_secs(5),
            consistent_read: true,
            estimate_rows: true,
            max_retries: 3,
            throttle_delay: Some(Duration::from_millis(100)),
            snapshot_delay: None,
            streaming_delay: None,
            lock_timeout: Some(Duration::from_secs(10)),
        }
    }

    /// Check if a snapshot should be performed.
    pub fn should_snapshot(&self, has_offsets: bool) -> bool {
        match self.mode {
            SnapshotMode::Always => true,
            SnapshotMode::Initial => !has_offsets,
            SnapshotMode::InitialOnly => !has_offsets,
            SnapshotMode::SchemaOnly => !has_offsets,
            SnapshotMode::WhenNeeded => !has_offsets,
            SnapshotMode::Recovery => true,
            SnapshotMode::ConfigurationBased { snapshot_data, .. } => snapshot_data && !has_offsets,
            SnapshotMode::Custom(_) => true, // Custom decides
            SnapshotMode::NoSnapshot => false,
        }
    }
}

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

impl SnapshotConfigBuilder {
    /// Set snapshot mode.
    pub fn mode(mut self, mode: SnapshotMode) -> Self {
        self.config.mode = mode;
        self
    }
    pub fn batch_size(mut self, size: usize) -> Self {
        self.config.batch_size = size.max(1);
        self
    }

    pub fn parallel_tables(mut self, count: usize) -> Self {
        self.config.parallel_tables = count.max(1);
        self
    }

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

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

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

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

    pub fn max_retries(mut self, retries: u32) -> Self {
        self.config.max_retries = retries;
        self
    }

    pub fn throttle_delay(mut self, delay: Duration) -> Self {
        self.config.throttle_delay = Some(delay);
        self
    }

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

/// A batch of snapshot rows.
#[derive(Debug, Clone)]
pub struct SnapshotBatch {
    /// Events in this batch
    pub events: Vec<CdcEvent>,
    /// Batch sequence number
    pub sequence: u64,
    /// Is this the last batch?
    pub is_last: bool,
    /// Last key in batch (for pagination)
    pub last_key: Option<String>,
}

impl SnapshotBatch {
    /// Create a new batch.
    pub fn new(events: Vec<CdcEvent>, sequence: u64) -> Self {
        Self {
            events,
            sequence,
            is_last: false,
            last_key: None,
        }
    }

    /// Mark as last batch.
    pub fn mark_last(mut self) -> Self {
        self.is_last = true;
        self
    }

    /// Set last key for pagination.
    pub fn with_last_key(mut self, key: impl Into<String>) -> Self {
        self.last_key = Some(key.into());
        self
    }

    /// Get number of events.
    pub fn len(&self) -> usize {
        self.events.len()
    }

    /// Check if batch is empty.
    pub fn is_empty(&self) -> bool {
        self.events.is_empty()
    }
}

/// Trait for snapshot progress persistence.
#[async_trait::async_trait]
pub trait ProgressStore: Send + Sync {
    /// Save progress for a table.
    async fn save(&self, progress: &SnapshotProgress) -> Result<()>;

    /// Load progress for a table.
    async fn load(&self, schema: &str, table: &str) -> Result<Option<SnapshotProgress>>;

    /// Delete progress for a table.
    async fn delete(&self, schema: &str, table: &str) -> Result<()>;

    /// List all progress entries.
    async fn list(&self) -> Result<Vec<SnapshotProgress>>;
}

/// In-memory progress store for testing.
#[derive(Default)]
pub struct MemoryProgressStore {
    progress: RwLock<HashMap<String, SnapshotProgress>>,
}

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

    fn key(schema: &str, table: &str) -> String {
        format!("{}.{}", schema, table)
    }
}

#[async_trait::async_trait]
impl ProgressStore for MemoryProgressStore {
    async fn save(&self, progress: &SnapshotProgress) -> Result<()> {
        let key = Self::key(&progress.schema, &progress.table);
        self.progress.write().await.insert(key, progress.clone());
        Ok(())
    }

    async fn load(&self, schema: &str, table: &str) -> Result<Option<SnapshotProgress>> {
        let key = Self::key(schema, table);
        Ok(self.progress.read().await.get(&key).cloned())
    }

    async fn delete(&self, schema: &str, table: &str) -> Result<()> {
        let key = Self::key(schema, table);
        self.progress.write().await.remove(&key);
        Ok(())
    }

    async fn list(&self) -> Result<Vec<SnapshotProgress>> {
        Ok(self.progress.read().await.values().cloned().collect())
    }
}

/// Trait for snapshot data source.
#[async_trait::async_trait]
pub trait SnapshotSource: Send + Sync {
    /// Get current position/watermark.
    async fn get_watermark(&self) -> Result<String>;

    /// Estimate row count for a table.
    async fn estimate_row_count(&self, schema: &str, table: &str) -> Result<Option<u64>>;

    /// Fetch a batch of rows.
    async fn fetch_batch(
        &self,
        schema: &str,
        table: &str,
        key_column: &str,
        last_key: Option<&str>,
        batch_size: usize,
    ) -> Result<SnapshotBatch>;
}

/// Snapshot statistics.
#[derive(Debug, Default)]
pub struct SnapshotStats {
    tables_completed: AtomicU64,
    tables_failed: AtomicU64,
    rows_processed: AtomicU64,
    batches_processed: AtomicU64,
    bytes_processed: AtomicU64,
}

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

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

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

    pub fn record_bytes(&self, bytes: u64) {
        self.bytes_processed.fetch_add(bytes, Ordering::Relaxed);
    }

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

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

    pub fn snapshot(&self) -> SnapshotStatsSnapshot {
        SnapshotStatsSnapshot {
            tables_completed: self.tables_completed.load(Ordering::Relaxed),
            tables_failed: self.tables_failed.load(Ordering::Relaxed),
            rows_processed: self.rows_processed.load(Ordering::Relaxed),
            batches_processed: self.batches_processed.load(Ordering::Relaxed),
            bytes_processed: self.bytes_processed.load(Ordering::Relaxed),
        }
    }
}

/// Snapshot for stats.
#[derive(Debug, Clone)]
pub struct SnapshotStatsSnapshot {
    pub tables_completed: u64,
    pub tables_failed: u64,
    pub rows_processed: u64,
    pub batches_processed: u64,
    pub bytes_processed: u64,
}

/// Table specification for snapshot.
#[derive(Debug, Clone)]
pub struct TableSpec {
    /// Schema name
    pub schema: String,
    /// Table name
    pub table: String,
    /// Primary key column for pagination
    pub key_column: String,
    /// Optional WHERE clause
    pub filter: Option<String>,
}

impl TableSpec {
    pub fn new(
        schema: impl Into<String>,
        table: impl Into<String>,
        key_column: impl Into<String>,
    ) -> Self {
        Self {
            schema: schema.into(),
            table: table.into(),
            key_column: key_column.into(),
            filter: None,
        }
    }

    pub fn with_filter(mut self, filter: impl Into<String>) -> Self {
        self.filter = Some(filter.into());
        self
    }
}

/// Coordinator for multi-table snapshots.
pub struct SnapshotCoordinator<S: SnapshotSource, P: ProgressStore> {
    config: SnapshotConfig,
    source: Arc<S>,
    progress_store: Arc<P>,
    stats: Arc<SnapshotStats>,
    cancelled: AtomicBool,
}

impl<S: SnapshotSource, P: ProgressStore> SnapshotCoordinator<S, P> {
    /// Create a new coordinator.
    pub fn new(config: SnapshotConfig, source: S, progress_store: P) -> Self {
        Self {
            config,
            source: Arc::new(source),
            progress_store: Arc::new(progress_store),
            stats: Arc::new(SnapshotStats::new()),
            cancelled: AtomicBool::new(false),
        }
    }

    /// Cancel all running snapshots.
    pub fn cancel(&self) {
        self.cancelled.store(true, Ordering::Relaxed);
    }

    /// Check if cancelled.
    pub fn is_cancelled(&self) -> bool {
        self.cancelled.load(Ordering::Relaxed)
    }

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

    /// Snapshot a single table.
    pub async fn snapshot_table(&self, spec: &TableSpec) -> Result<Vec<SnapshotBatch>> {
        let mut batches = Vec::new();
        let mut progress = self
            .progress_store
            .load(&spec.schema, &spec.table)
            .await?
            .unwrap_or_else(|| SnapshotProgress::new(&spec.schema, &spec.table));

        // Get watermark
        let watermark = self.source.get_watermark().await?;

        // Start or resume
        if progress.state == SnapshotState::Pending {
            progress.start(Some(watermark));

            // Estimate rows if configured
            if self.config.estimate_rows {
                progress.total_rows = self
                    .source
                    .estimate_row_count(&spec.schema, &spec.table)
                    .await?;
            }
        }

        let mut last_key = progress.last_key.clone();
        let mut sequence = 0u64;
        let mut last_save = Instant::now();

        loop {
            if self.is_cancelled() {
                progress.pause();
                self.progress_store.save(&progress).await?;
                return Err(CdcError::replication("Snapshot cancelled"));
            }

            // Fetch batch with retry
            let batch = self.fetch_with_retry(spec, last_key.as_deref()).await?;

            let batch_len = batch.len() as u64;
            let is_last = batch.is_last || batch.is_empty();

            if !batch.is_empty() {
                last_key = batch.last_key.clone();
                progress.update(batch_len, last_key.clone());
                self.stats.record_rows(batch_len);
                self.stats.record_batch();

                batches.push(SnapshotBatch {
                    events: batch.events,
                    sequence,
                    is_last,
                    last_key: last_key.clone(),
                });
                sequence += 1;
            }

            // Save progress periodically
            if last_save.elapsed() >= self.config.progress_interval {
                self.progress_store.save(&progress).await?;
                last_save = Instant::now();
            }

            if is_last {
                break;
            }

            // Throttle if configured
            if let Some(delay) = self.config.throttle_delay {
                tokio::time::sleep(delay).await;
            }
        }

        // Mark complete
        progress.complete();
        self.progress_store.save(&progress).await?;
        self.stats.record_table_completed();

        Ok(batches)
    }

    /// Fetch batch with retry.
    async fn fetch_with_retry(
        &self,
        spec: &TableSpec,
        last_key: Option<&str>,
    ) -> Result<SnapshotBatch> {
        let mut attempts = 0;
        loop {
            match self
                .source
                .fetch_batch(
                    &spec.schema,
                    &spec.table,
                    &spec.key_column,
                    last_key,
                    self.config.batch_size,
                )
                .await
            {
                Ok(batch) => return Ok(batch),
                Err(e) => {
                    attempts += 1;
                    if attempts >= self.config.max_retries {
                        return Err(e);
                    }
                    let delay = Duration::from_millis(100 * 2u64.pow(attempts));
                    tokio::time::sleep(delay).await;
                }
            }
        }
    }

    /// Snapshot multiple tables.
    pub async fn snapshot_tables(
        &self,
        specs: Vec<TableSpec>,
    ) -> Result<HashMap<String, Vec<SnapshotBatch>>> {
        let mut results = HashMap::new();

        // Process tables (could be parallelized with semaphore)
        for spec in specs {
            let key = format!("{}.{}", spec.schema, spec.table);
            match self.snapshot_table(&spec).await {
                Ok(batches) => {
                    results.insert(key, batches);
                }
                Err(e) => {
                    self.stats.record_table_failed();
                    // Continue with other tables or fail fast based on config
                    return Err(e);
                }
            }
        }

        Ok(results)
    }

    /// Get progress for all tables.
    pub async fn get_all_progress(&self) -> Result<Vec<SnapshotProgress>> {
        self.progress_store.list().await
    }

    /// Reset progress for a table.
    pub async fn reset_table(&self, schema: &str, table: &str) -> Result<()> {
        self.progress_store.delete(schema, table).await
    }
}

/// Mock snapshot source for testing.
pub struct MockSnapshotSource {
    rows: RwLock<HashMap<String, Vec<serde_json::Value>>>,
}

impl MockSnapshotSource {
    pub fn new() -> Self {
        Self {
            rows: RwLock::new(HashMap::new()),
        }
    }

    /// Add test rows.
    pub async fn add_rows(&self, schema: &str, table: &str, rows: Vec<serde_json::Value>) {
        let key = format!("{}.{}", schema, table);
        self.rows.write().await.insert(key, rows);
    }
}

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

#[async_trait::async_trait]
impl SnapshotSource for MockSnapshotSource {
    async fn get_watermark(&self) -> Result<String> {
        Ok("0/0".to_string())
    }

    async fn estimate_row_count(&self, schema: &str, table: &str) -> Result<Option<u64>> {
        let key = format!("{}.{}", schema, table);
        let rows = self.rows.read().await;
        Ok(rows.get(&key).map(|r| r.len() as u64))
    }

    async fn fetch_batch(
        &self,
        schema: &str,
        table: &str,
        key_column: &str,
        last_key: Option<&str>,
        batch_size: usize,
    ) -> Result<SnapshotBatch> {
        let key = format!("{}.{}", schema, table);
        let rows = self.rows.read().await;

        let table_rows = rows.get(&key).cloned().unwrap_or_default();

        // Find start position
        let start_idx = if let Some(last) = last_key {
            let last_val: i64 = last.parse().unwrap_or(0);
            table_rows
                .iter()
                .position(|r| {
                    r.get(key_column)
                        .and_then(|v| v.as_i64())
                        .map(|v| v > last_val)
                        .unwrap_or(false)
                })
                .unwrap_or(table_rows.len())
        } else {
            0
        };

        let end_idx = (start_idx + batch_size).min(table_rows.len());
        let batch_rows: Vec<_> = table_rows[start_idx..end_idx].to_vec();
        let is_last = end_idx >= table_rows.len();

        let events: Vec<CdcEvent> = batch_rows
            .iter()
            .map(|row| {
                CdcEvent {
                    source_type: "snapshot".to_string(),
                    database: "test".to_string(),
                    schema: schema.to_string(),
                    table: table.to_string(),
                    op: CdcOp::Snapshot, // Snapshot events use Snapshot op
                    before: None,
                    after: Some(row.clone()),
                    timestamp: chrono::Utc::now().timestamp(),
                    transaction: None,
                }
            })
            .collect();

        let last_key = batch_rows
            .last()
            .and_then(|r| r.get(key_column))
            .and_then(|v| v.as_i64())
            .map(|v| v.to_string());

        Ok(SnapshotBatch {
            events,
            sequence: 0,
            is_last,
            last_key,
        })
    }
}

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

    #[test]
    fn test_snapshot_progress_new() {
        let progress = SnapshotProgress::new("public", "users");
        assert_eq!(progress.schema, "public");
        assert_eq!(progress.table, "users");
        assert_eq!(progress.state, SnapshotState::Pending);
        assert_eq!(progress.rows_processed, 0);
    }

    #[test]
    fn test_snapshot_progress_lifecycle() {
        let mut progress = SnapshotProgress::new("public", "users");

        // Start
        progress.start(Some("0/1234".to_string()));
        assert_eq!(progress.state, SnapshotState::InProgress);
        assert!(progress.started_at.is_some());
        assert_eq!(progress.watermark, Some("0/1234".to_string()));

        // Update
        progress.update(100, Some("100".to_string()));
        assert_eq!(progress.rows_processed, 100);
        assert_eq!(progress.last_key, Some("100".to_string()));

        // Complete
        progress.complete();
        assert_eq!(progress.state, SnapshotState::Completed);
        assert!(progress.completed_at.is_some());
    }

    #[test]
    fn test_snapshot_progress_failure() {
        let mut progress = SnapshotProgress::new("public", "users");
        progress.start(None);
        progress.fail("Connection lost");

        assert_eq!(progress.state, SnapshotState::Failed);
        assert_eq!(progress.error, Some("Connection lost".to_string()));
    }

    #[test]
    fn test_snapshot_progress_percentage() {
        let mut progress = SnapshotProgress::new("public", "users");
        progress.total_rows = Some(1000);
        progress.rows_processed = 500;

        assert_eq!(progress.percentage(), Some(50.0));
    }

    #[test]
    fn test_snapshot_progress_can_resume() {
        let mut progress = SnapshotProgress::new("public", "users");
        progress.start(None);
        progress.update(100, Some("100".to_string()));
        progress.pause();

        assert!(progress.can_resume());
    }

    #[test]
    fn test_snapshot_config_defaults() {
        let config = SnapshotConfig::default();
        assert_eq!(config.batch_size, 10_000);
        assert_eq!(config.parallel_tables, 4);
        assert!(config.consistent_read);
    }

    #[test]
    fn test_snapshot_config_builder() {
        let config = SnapshotConfig::builder()
            .batch_size(5000)
            .parallel_tables(2)
            .consistent_read(false)
            .max_retries(5)
            .build();

        assert_eq!(config.batch_size, 5000);
        assert_eq!(config.parallel_tables, 2);
        assert!(!config.consistent_read);
        assert_eq!(config.max_retries, 5);
    }

    #[test]
    fn test_snapshot_config_presets() {
        let high = SnapshotConfig::high_throughput();
        assert_eq!(high.batch_size, 50_000);
        assert!(!high.consistent_read);

        let low = SnapshotConfig::low_memory();
        assert_eq!(low.batch_size, 1_000);
        assert!(low.throttle_delay.is_some());
    }

    #[test]
    fn test_snapshot_batch() {
        let events = vec![CdcEvent {
            source_type: "snapshot".to_string(),
            database: "test".to_string(),
            schema: "public".to_string(),
            table: "users".to_string(),
            op: CdcOp::Snapshot,
            before: None,
            after: Some(serde_json::json!({"id": 1})),
            timestamp: 0,
            transaction: None,
        }];

        let batch = SnapshotBatch::new(events, 0).with_last_key("1").mark_last();

        assert_eq!(batch.len(), 1);
        assert!(batch.is_last);
        assert_eq!(batch.last_key, Some("1".to_string()));
    }

    #[test]
    fn test_table_spec() {
        let spec = TableSpec::new("public", "users", "id").with_filter("active = true");

        assert_eq!(spec.schema, "public");
        assert_eq!(spec.table, "users");
        assert_eq!(spec.key_column, "id");
        assert_eq!(spec.filter, Some("active = true".to_string()));
    }

    #[tokio::test]
    async fn test_memory_progress_store() {
        let store = MemoryProgressStore::new();

        let mut progress = SnapshotProgress::new("public", "users");
        progress.start(None);
        progress.update(100, Some("100".to_string()));

        store.save(&progress).await.unwrap();

        let loaded = store.load("public", "users").await.unwrap();
        assert!(loaded.is_some());
        let loaded = loaded.unwrap();
        assert_eq!(loaded.rows_processed, 100);

        let list = store.list().await.unwrap();
        assert_eq!(list.len(), 1);

        store.delete("public", "users").await.unwrap();
        let deleted = store.load("public", "users").await.unwrap();
        assert!(deleted.is_none());
    }

    #[tokio::test]
    async fn test_mock_snapshot_source() {
        let source = MockSnapshotSource::new();

        let rows = vec![
            serde_json::json!({"id": 1, "name": "Alice"}),
            serde_json::json!({"id": 2, "name": "Bob"}),
            serde_json::json!({"id": 3, "name": "Charlie"}),
        ];
        source.add_rows("public", "users", rows).await;

        // First batch
        let batch = source
            .fetch_batch("public", "users", "id", None, 2)
            .await
            .unwrap();
        assert_eq!(batch.events.len(), 2);
        assert!(!batch.is_last);
        assert_eq!(batch.last_key, Some("2".to_string()));

        // Second batch (from last key)
        let batch = source
            .fetch_batch("public", "users", "id", Some("2"), 2)
            .await
            .unwrap();
        assert_eq!(batch.events.len(), 1);
        assert!(batch.is_last);
    }

    #[tokio::test]
    async fn test_mock_snapshot_source_estimate() {
        let source = MockSnapshotSource::new();

        let rows = vec![serde_json::json!({"id": 1}), serde_json::json!({"id": 2})];
        source.add_rows("public", "users", rows).await;

        let count = source.estimate_row_count("public", "users").await.unwrap();
        assert_eq!(count, Some(2));
    }

    #[tokio::test]
    async fn test_snapshot_coordinator_single_table() {
        let source = MockSnapshotSource::new();
        let rows = vec![
            serde_json::json!({"id": 1, "name": "Alice"}),
            serde_json::json!({"id": 2, "name": "Bob"}),
        ];
        source.add_rows("public", "users", rows).await;

        let progress_store = MemoryProgressStore::new();
        let config = SnapshotConfig::builder().batch_size(10).build();

        let coordinator = SnapshotCoordinator::new(config, source, progress_store);
        let spec = TableSpec::new("public", "users", "id");

        let batches = coordinator.snapshot_table(&spec).await.unwrap();

        assert_eq!(batches.len(), 1);
        assert_eq!(batches[0].events.len(), 2);
        assert!(batches[0].is_last);
    }

    #[tokio::test]
    async fn test_snapshot_coordinator_stats() {
        let source = MockSnapshotSource::new();
        let rows = vec![serde_json::json!({"id": 1}), serde_json::json!({"id": 2})];
        source.add_rows("public", "users", rows).await;

        let progress_store = MemoryProgressStore::new();
        let config = SnapshotConfig::default();

        let coordinator = SnapshotCoordinator::new(config, source, progress_store);
        let spec = TableSpec::new("public", "users", "id");

        coordinator.snapshot_table(&spec).await.unwrap();

        let stats = coordinator.stats();
        assert_eq!(stats.tables_completed, 1);
        assert_eq!(stats.rows_processed, 2);
        assert_eq!(stats.batches_processed, 1);
    }

    #[tokio::test]
    async fn test_snapshot_coordinator_cancel() {
        let source = MockSnapshotSource::new();
        let rows: Vec<_> = (1..=100).map(|i| serde_json::json!({"id": i})).collect();
        source.add_rows("public", "users", rows).await;

        let progress_store = MemoryProgressStore::new();
        let config = SnapshotConfig::builder().batch_size(10).build();

        let coordinator = SnapshotCoordinator::new(config, source, progress_store);
        coordinator.cancel();

        let spec = TableSpec::new("public", "users", "id");
        let result = coordinator.snapshot_table(&spec).await;

        assert!(result.is_err());
    }

    #[tokio::test]
    async fn test_snapshot_coordinator_progress_persistence() {
        let source = MockSnapshotSource::new();
        let rows = vec![serde_json::json!({"id": 1}), serde_json::json!({"id": 2})];
        source.add_rows("public", "users", rows).await;

        let progress_store = MemoryProgressStore::new();
        let config = SnapshotConfig::default();

        let coordinator = SnapshotCoordinator::new(config, source, progress_store);
        let spec = TableSpec::new("public", "users", "id");

        coordinator.snapshot_table(&spec).await.unwrap();

        // Progress should be saved after completion
        let all_progress = coordinator.get_all_progress().await.unwrap();
        assert_eq!(all_progress.len(), 1);
        assert_eq!(all_progress[0].state, SnapshotState::Completed);
    }

    #[test]
    fn test_snapshot_stats() {
        let stats = SnapshotStats::new();

        stats.record_rows(100);
        stats.record_batch();
        stats.record_bytes(1024);
        stats.record_table_completed();

        let snapshot = stats.snapshot();
        assert_eq!(snapshot.rows_processed, 100);
        assert_eq!(snapshot.batches_processed, 1);
        assert_eq!(snapshot.bytes_processed, 1024);
        assert_eq!(snapshot.tables_completed, 1);
    }

    #[tokio::test]
    async fn test_snapshot_multiple_tables() {
        let source = MockSnapshotSource::new();
        source
            .add_rows("public", "users", vec![serde_json::json!({"id": 1})])
            .await;
        source
            .add_rows("public", "orders", vec![serde_json::json!({"id": 1})])
            .await;

        let progress_store = MemoryProgressStore::new();
        let config = SnapshotConfig::default();

        let coordinator = SnapshotCoordinator::new(config, source, progress_store);

        let specs = vec![
            TableSpec::new("public", "users", "id"),
            TableSpec::new("public", "orders", "id"),
        ];

        let results = coordinator.snapshot_tables(specs).await.unwrap();

        assert_eq!(results.len(), 2);
        assert!(results.contains_key("public.users"));
        assert!(results.contains_key("public.orders"));
    }

    #[test]
    fn test_snapshot_state_default() {
        let state = SnapshotState::default();
        assert_eq!(state, SnapshotState::Pending);
    }
}