rhei-sidecar 1.5.0

Sidecar CDC consumer for Rhei — polls external databases by timestamp columns
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
//! The [`TimestampCdcConsumer`] — poll-based CDC consumer generic over
//! [`crate::SourceConnector`].
//!
//! The consumer implements [`rhei_core::CdcConsumer`] so it can be plugged
//! directly into `CdcSyncEngine` in place of the trigger-based producer.
//! Each call to [`rhei_core::CdcConsumer::poll`] issues one SQL query per
//! configured table, converts the returned rows to [`rhei_core::CdcEvent`]s,
//! advances the per-table [`crate::Watermark`]s, and persists them via the
//! configured [`crate::WatermarkStore`].

use std::collections::HashMap;
use std::sync::atomic::{AtomicI64, Ordering};
use std::sync::Arc;

use arrow::array::Array;
use arrow::record_batch::RecordBatch;
use tokio::sync::Mutex;
use tracing::{debug, warn};

use rhei_core::types::{CdcEvent, CdcOperation};

use crate::config::{DeleteDetection, TimestampCdcConfig, TimestampTableConfig};
use crate::error::SidecarError;
use crate::source::SourceConnector;
use crate::watermark::{NullWatermarkStore, Watermark, WatermarkStore};

// ---------------------------------------------------------------------------
// Internal snapshot used for rollback on persistence failure
// ---------------------------------------------------------------------------

struct PollSnapshot {
    watermarks: HashMap<String, Watermark>,
    global_seq: i64,
}

/// CDC consumer that polls an external database by timestamp columns.
///
/// Generic over `S: SourceConnector` so it works with SQLite, PostgreSQL, or
/// any backend that implements the trait.
///
/// Limitations:
/// - Cannot detect hard deletes (see `DeleteDetection` for workarounds)
/// - Does not provide `old_data` (`CdcEvent.old_data` is always `None`)
/// - Timestamp ties are resolved using `(updated_at, pk1, pk2, ...)` compound ordering
pub struct TimestampCdcConsumer<S: SourceConnector> {
    conn: Arc<Mutex<S>>,
    config: TimestampCdcConfig,
    watermarks: Mutex<HashMap<String, Watermark>>,
    global_seq: AtomicI64,
    watermark_store: Box<dyn WatermarkStore>,
}

impl<S: SourceConnector> TimestampCdcConsumer<S> {
    /// Create a new consumer from any `SourceConnector` implementation.
    ///
    /// Watermarks start from zero (in-memory only). Use [`Self::with_watermark_store`]
    /// for persistence across restarts.
    pub fn new(conn: S, config: TimestampCdcConfig) -> Self {
        Self::with_watermark_store(conn, config, Box::new(NullWatermarkStore))
    }

    /// Create a consumer with a persistent watermark store.
    ///
    /// On construction, watermarks and the global sequence counter are loaded
    /// from the store so polling resumes from the last persisted position.
    ///
    /// **Load errors are treated permissively**: if the store returns an error
    /// for any table or for the global sequence counter, a warning is logged and
    /// the default (zero watermark / seq=1) is used. This preserves backward
    /// compatibility but may cause duplicate CDC events after a corrupted
    /// restart. Prefer [`Self::try_with_watermark_store`] when you want strict
    /// error propagation.
    pub fn with_watermark_store(
        conn: S,
        config: TimestampCdcConfig,
        store: Box<dyn WatermarkStore>,
    ) -> Self {
        // Load persisted global_seq — log and fall back on error
        let persisted_seq = match store.load_global_seq() {
            Ok(seq) => seq,
            Err(e) => {
                warn!(error = %e, "failed to load global_seq from watermark store; resetting to 1 (may cause duplicate events)");
                1
            }
        };

        // Load persisted watermarks for each configured table
        let mut watermarks = HashMap::new();
        for table in &config.tables {
            let wm = match store.load(&table.table_name) {
                Ok(Some(wm)) => wm,
                Ok(None) => Watermark::new(),
                Err(e) => {
                    warn!(
                        table = %table.table_name,
                        error = %e,
                        "failed to load watermark from store; resetting to zero (may cause duplicate events)"
                    );
                    Watermark::new()
                }
            };
            watermarks.insert(table.table_name.clone(), wm);
        }

        Self {
            conn: Arc::new(Mutex::new(conn)),
            config,
            watermarks: Mutex::new(watermarks),
            global_seq: AtomicI64::new(persisted_seq),
            watermark_store: store,
        }
    }

    /// Create a consumer with a persistent watermark store, propagating load errors.
    ///
    /// Unlike [`Self::with_watermark_store`], this constructor returns an error
    /// if the store fails to load any watermark or the global sequence counter
    /// (e.g. corrupt RocksDB, permission denied, partial write). This prevents
    /// silent resets that would cause duplicate CDC events after a corrupted
    /// restart.
    ///
    /// Use this variant in production. The permissive
    /// [`Self::with_watermark_store`] is provided for backward compatibility.
    pub fn try_with_watermark_store(
        conn: S,
        config: TimestampCdcConfig,
        store: Box<dyn WatermarkStore>,
    ) -> Result<Self, SidecarError> {
        // Strict load — propagate errors
        let persisted_seq = store.load_global_seq()?;

        let mut watermarks = HashMap::new();
        for table in &config.tables {
            let wm = store
                .load(&table.table_name)?
                .unwrap_or_else(Watermark::new);
            watermarks.insert(table.table_name.clone(), wm);
        }

        Ok(Self {
            conn: Arc::new(Mutex::new(conn)),
            config,
            watermarks: Mutex::new(watermarks),
            global_seq: AtomicI64::new(persisted_seq),
            watermark_store: store,
        })
    }

    /// Build the poll query for a table given its config and current watermark.
    ///
    /// Supports composite primary keys: ORDER BY and tie-breaking use all PK columns
    /// with SQL tuple comparison for correct lexicographic ordering.
    pub(crate) fn build_poll_query(
        table_config: &TimestampTableConfig,
        wm: &Watermark,
        limit: u32,
    ) -> String {
        let columns = if table_config.columns.is_empty() {
            "*".to_string()
        } else {
            table_config.columns.join(", ")
        };

        let updated_at = &table_config.updated_at_column;
        let pk_cols = &table_config.primary_key;

        let where_clause = match &wm.last_pk {
            Some(last_pk_values) if last_pk_values.len() == pk_cols.len() => {
                // Tuple comparison: (pk1, pk2) > (val1, val2)
                let pk_tuple = pk_cols.join(", ");
                let val_tuple: String = last_pk_values
                    .iter()
                    .map(|v| pk_to_sql_literal(v))
                    .collect::<Vec<_>>()
                    .join(", ");
                format!(
                    "WHERE {updated_at} > {ts} OR ({updated_at} = {ts} AND ({pk_tuple}) > ({val_tuple}))",
                    ts = wm.timestamp,
                )
            }
            _ => format!("WHERE {updated_at} > {}", wm.timestamp),
        };

        // ORDER BY updated_at ASC, pk1 ASC, pk2 ASC, ...
        let order_cols: Vec<String> = std::iter::once(format!("{updated_at} ASC"))
            .chain(pk_cols.iter().map(|pk| format!("{pk} ASC")))
            .collect();
        let order_by = order_cols.join(", ");

        format!(
            "SELECT {columns} FROM {} {where_clause} ORDER BY {order_by} LIMIT {limit}",
            table_config.table_name,
        )
    }

    /// Build the soft-delete detection query.
    pub(crate) fn build_soft_delete_query(
        table_config: &TimestampTableConfig,
        delete_col: &str,
        wm: &Watermark,
        limit: u32,
    ) -> String {
        let columns = if table_config.columns.is_empty() {
            "*".to_string()
        } else {
            table_config.columns.join(", ")
        };

        let pk_cols = &table_config.primary_key;

        // ORDER BY delete_col ASC, pk1 ASC, pk2 ASC, ...
        let order_cols: Vec<String> = std::iter::once(format!("{delete_col} ASC"))
            .chain(pk_cols.iter().map(|pk| format!("{pk} ASC")))
            .collect();
        let order_by = order_cols.join(", ");

        format!(
            "SELECT {columns} FROM {} WHERE {delete_col} IS NOT NULL AND {delete_col} > {} ORDER BY {order_by} LIMIT {limit}",
            table_config.table_name,
            wm.timestamp,
        )
    }

    /// Extract the primary key values from a row as a vector of stringified values.
    fn extract_pk_values(
        row_data: &serde_json::Map<String, serde_json::Value>,
        pk_cols: &[String],
    ) -> Vec<String> {
        pk_cols
            .iter()
            .map(|col| {
                row_data
                    .get(col)
                    .map(|v| v.to_string())
                    .unwrap_or_else(|| "null".to_string())
            })
            .collect()
    }

    /// Convert a RecordBatch row into a CdcEvent.
    fn row_to_event(
        batch: &RecordBatch,
        row_idx: usize,
        table_config: &TimestampTableConfig,
        seq: i64,
    ) -> Result<CdcEvent, SidecarError> {
        let schema = batch.schema();

        let mut row_data = serde_json::Map::new();
        for (col_idx, field) in schema.fields().iter().enumerate() {
            let col = batch.column(col_idx);
            let val = array_value_to_json(col, row_idx);
            row_data.insert(field.name().clone(), val);
        }

        let ts_val = row_data
            .get(&table_config.updated_at_column)
            .and_then(|v| v.as_i64())
            .unwrap_or(0);

        let created_at_val = row_data
            .get(&table_config.created_at_column)
            .and_then(|v| v.as_i64())
            .unwrap_or(0);

        // Heuristic: if created_at == updated_at, it's an INSERT; otherwise UPDATE
        let operation = if created_at_val == ts_val {
            CdcOperation::Insert
        } else {
            CdcOperation::Update
        };

        // Use first PK column for row_id (backward compat)
        let row_id = row_data
            .get(&table_config.primary_key[0])
            .and_then(|v| v.as_i64());

        Ok(CdcEvent {
            seq,
            timestamp: ts_val,
            operation,
            table: table_config.table_name.clone(),
            row_id,
            old_data: None, // Timestamp polling doesn't provide old data
            new_data: Some(serde_json::Value::Object(row_data)),
        })
    }

    /// Execute a query on the source connection via spawn_blocking.
    async fn exec_query(&self, sql: String) -> Result<Vec<RecordBatch>, SidecarError> {
        let conn = self.conn.clone();
        tokio::task::spawn_blocking(move || {
            let mut conn = conn.blocking_lock();
            conn.query(&sql)
        })
        .await
        .map_err(|e| SidecarError::Join(e.to_string()))?
    }

    /// Persist the current watermark state to the store.
    ///
    /// `seq` is passed explicitly so callers can pass the post-advance value
    /// (which may not yet be written to `self.global_seq`).
    fn persist_watermarks(
        &self,
        watermarks: &HashMap<String, Watermark>,
        seq: i64,
    ) -> Result<(), SidecarError> {
        for (table, wm) in watermarks {
            self.watermark_store.save(table, wm)?;
        }
        self.watermark_store.save_global_seq(seq)?;
        Ok(())
    }
}

/// Convert a watermark PK value (stored as serde_json stringified) to a SQL literal.
/// Numeric values are emitted bare; strings are single-quoted with escaping.
fn pk_to_sql_literal(pk: &str) -> String {
    // serde_json::Value::to_string() produces:
    //   integers: "42"
    //   strings:  "\"abc\"" (JSON-quoted)
    // We need proper SQL literals.
    if pk.parse::<i64>().is_ok() || pk.parse::<f64>().is_ok() {
        pk.to_string()
    } else {
        // Strip JSON quotes if present, then SQL-escape
        let unquoted = pk.trim_matches('"');
        format!("'{}'", unquoted.replace('\'', "''"))
    }
}

/// Extract a value from an Arrow array at a given index as a serde_json::Value.
fn array_value_to_json(array: &dyn Array, idx: usize) -> serde_json::Value {
    use arrow::array::*;

    if array.is_null(idx) {
        return serde_json::Value::Null;
    }

    if let Some(arr) = array.as_any().downcast_ref::<Int64Array>() {
        serde_json::Value::Number(arr.value(idx).into())
    } else if let Some(arr) = array.as_any().downcast_ref::<Int32Array>() {
        serde_json::Value::Number((arr.value(idx) as i64).into())
    } else if let Some(arr) = array.as_any().downcast_ref::<Int16Array>() {
        serde_json::Value::Number((arr.value(idx) as i64).into())
    } else if let Some(arr) = array.as_any().downcast_ref::<Float64Array>() {
        serde_json::json!(arr.value(idx))
    } else if let Some(arr) = array.as_any().downcast_ref::<Float32Array>() {
        serde_json::json!(arr.value(idx) as f64)
    } else if let Some(arr) = array.as_any().downcast_ref::<StringArray>() {
        serde_json::Value::String(arr.value(idx).to_string())
    } else if let Some(arr) = array.as_any().downcast_ref::<LargeStringArray>() {
        serde_json::Value::String(arr.value(idx).to_string())
    } else if let Some(arr) = array.as_any().downcast_ref::<BooleanArray>() {
        serde_json::Value::Bool(arr.value(idx))
    } else {
        warn!(data_type = ?array.data_type(), "unsupported Arrow type in array_value_to_json, returning null");
        serde_json::Value::Null
    }
}

impl<S: SourceConnector> rhei_core::CdcConsumer for TimestampCdcConsumer<S> {
    type Error = SidecarError;

    async fn poll(
        &self,
        _after_seq: Option<i64>,
        limit: u32,
    ) -> Result<Vec<CdcEvent>, Self::Error> {
        let mut all_events = Vec::new();
        let mut watermarks = self.watermarks.lock().await;

        // ---------------------------------------------------------------
        // Snapshot pre-poll state so we can roll back if persistence fails.
        // We must NOT leave in-memory state advanced past what the store has
        // durably committed, otherwise a failed poll will silently skip rows
        // on the next call.
        // ---------------------------------------------------------------
        let snapshot = PollSnapshot {
            watermarks: watermarks.clone(),
            global_seq: self.global_seq.load(Ordering::Relaxed),
        };

        // Accumulate the post-advance seq so we can pass it to persist_watermarks.
        // We track it separately rather than reading self.global_seq after the loop
        // to avoid a TOCTOU race (even though the Mutex guards the critical section).
        let mut next_seq = snapshot.global_seq;

        for table_config in &self.config.tables {
            let wm = watermarks
                .get(&table_config.table_name)
                .cloned()
                .unwrap_or_else(Watermark::new);

            let query = Self::build_poll_query(table_config, &wm, limit);
            debug!(table = %table_config.table_name, query = %query, "polling external source");

            let batches = self.exec_query(query).await?;

            let mut last_ts = wm.timestamp;
            let mut last_pk: Option<Vec<String>> = wm.last_pk.clone();
            let events_before = all_events.len();

            for batch in &batches {
                for row in 0..batch.num_rows() {
                    let seq = self.global_seq.fetch_add(1, Ordering::Relaxed);
                    next_seq = seq + 1;
                    let event = Self::row_to_event(batch, row, table_config, seq)?;

                    last_ts = event.timestamp;
                    if let Some(ref new_data) = event.new_data {
                        let pk_values = Self::extract_pk_values(
                            new_data.as_object().unwrap(),
                            &table_config.primary_key,
                        );
                        last_pk = Some(pk_values);
                    }

                    all_events.push(event);
                }
            }

            // Advance watermark only if this table produced new events
            let table_had_events = all_events.len() > events_before;
            if table_had_events {
                if let Some(pk_values) = last_pk {
                    watermarks
                        .entry(table_config.table_name.clone())
                        .or_insert_with(Watermark::new)
                        .advance(last_ts, pk_values);
                }
            }

            // Delete detection
            if let DeleteDetection::FullDiff { .. } = self.config.delete_detection {
                warn!(table = %table_config.table_name, "FullDiff delete detection is not yet implemented; no deletes will be detected");
            }
            if let DeleteDetection::SoftDelete { ref column } = self.config.delete_detection {
                let delete_query = Self::build_soft_delete_query(table_config, column, &wm, limit);
                let delete_batches = self.exec_query(delete_query).await?;

                for batch in &delete_batches {
                    let del_schema = batch.schema();
                    for row in 0..batch.num_rows() {
                        let seq = self.global_seq.fetch_add(1, Ordering::Relaxed);
                        next_seq = seq + 1;

                        let mut row_data = serde_json::Map::new();
                        for (col_idx, field) in del_schema.fields().iter().enumerate() {
                            let val = array_value_to_json(batch.column(col_idx), row);
                            row_data.insert(field.name().clone(), val);
                        }

                        // Use first PK column for row_id (backward compat)
                        let row_id = row_data
                            .get(&table_config.primary_key[0])
                            .and_then(|v| v.as_i64());

                        // Use the row's deleted_at timestamp (not the old watermark)
                        // so temporal validity boundaries are correct.
                        let delete_ts = row_data
                            .get(column)
                            .and_then(|v| v.as_i64())
                            .unwrap_or(wm.timestamp);

                        all_events.push(CdcEvent {
                            seq,
                            timestamp: delete_ts,
                            operation: CdcOperation::Delete,
                            table: table_config.table_name.clone(),
                            row_id,
                            old_data: Some(serde_json::Value::Object(row_data)),
                            new_data: None,
                        });
                    }
                }
            }
        }

        // Persist watermarks after the full poll cycle.
        // If persistence fails, restore in-memory state from the pre-poll
        // snapshot so the next poll re-fetches the same rows rather than
        // silently skipping them.
        if !all_events.is_empty() {
            if let Err(e) = self.persist_watermarks(&watermarks, next_seq) {
                // Roll back in-memory state to the pre-poll snapshot
                *watermarks = snapshot.watermarks;
                self.global_seq
                    .store(snapshot.global_seq, Ordering::Relaxed);
                return Err(e);
            }
        }

        debug!(events = all_events.len(), "polled external source");
        Ok(all_events)
    }

    async fn latest_seq(&self) -> Result<Option<i64>, Self::Error> {
        let seq = self.global_seq.load(Ordering::Relaxed);
        if seq <= 1 {
            Ok(None)
        } else {
            Ok(Some(seq - 1))
        }
    }

    async fn prune(&self, _up_to_seq: i64) -> Result<u64, Self::Error> {
        Ok(0)
    }
}

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

    fn test_table_config() -> TimestampTableConfig {
        TimestampTableConfig {
            table_name: "events".to_string(),
            created_at_column: "created_at".to_string(),
            updated_at_column: "updated_at".to_string(),
            primary_key: vec!["id".to_string()],
            columns: vec![],
        }
    }

    fn composite_pk_table_config() -> TimestampTableConfig {
        TimestampTableConfig {
            table_name: "order_items".to_string(),
            created_at_column: "created_at".to_string(),
            updated_at_column: "updated_at".to_string(),
            primary_key: vec!["order_id".to_string(), "item_id".to_string()],
            columns: vec![],
        }
    }

    // Use SQLiteConnection as the SourceConnector for tests
    #[cfg(feature = "sqlite")]
    type TestConsumer = TimestampCdcConsumer<connector_arrow::rusqlite::SQLiteConnection>;

    #[test]
    fn test_poll_query_generation() {
        let config = test_table_config();
        let wm = Watermark::new();

        let query = TestConsumer::build_poll_query(&config, &wm, 100);
        assert_eq!(
            query,
            "SELECT * FROM events WHERE updated_at > 0 ORDER BY updated_at ASC, id ASC LIMIT 100"
        );

        let mut wm2 = Watermark::new();
        wm2.advance(1000, vec!["42".to_string()]);
        let query2 = TestConsumer::build_poll_query(&config, &wm2, 50);
        assert_eq!(
            query2,
            "SELECT * FROM events WHERE updated_at > 1000 OR (updated_at = 1000 AND (id) > (42)) ORDER BY updated_at ASC, id ASC LIMIT 50"
        );
    }

    #[test]
    fn test_poll_query_composite_pk() {
        let config = composite_pk_table_config();
        let wm = Watermark::new();

        let query = TestConsumer::build_poll_query(&config, &wm, 100);
        assert_eq!(
            query,
            "SELECT * FROM order_items WHERE updated_at > 0 ORDER BY updated_at ASC, order_id ASC, item_id ASC LIMIT 100"
        );

        let mut wm2 = Watermark::new();
        wm2.advance(1000, vec!["10".to_string(), "3".to_string()]);
        let query2 = TestConsumer::build_poll_query(&config, &wm2, 50);
        assert_eq!(
            query2,
            "SELECT * FROM order_items WHERE updated_at > 1000 OR (updated_at = 1000 AND (order_id, item_id) > (10, 3)) ORDER BY updated_at ASC, order_id ASC, item_id ASC LIMIT 50"
        );
    }

    #[test]
    fn test_poll_query_composite_pk_string_values() {
        let config = composite_pk_table_config();

        let mut wm = Watermark::new();
        wm.advance(500, vec!["\"tenant_a\"".to_string(), "42".to_string()]);
        let query = TestConsumer::build_poll_query(&config, &wm, 10);
        assert!(query.contains("('tenant_a', 42)"));
    }

    #[test]
    fn test_poll_query_with_columns() {
        let mut config = test_table_config();
        config.columns = vec![
            "id".to_string(),
            "name".to_string(),
            "updated_at".to_string(),
        ];
        let wm = Watermark::new();

        let query = TestConsumer::build_poll_query(&config, &wm, 10);
        assert!(query.starts_with("SELECT id, name, updated_at FROM events"));
    }

    #[test]
    fn test_soft_delete_query() {
        let config = test_table_config();
        let wm = Watermark::new();

        let query = TestConsumer::build_soft_delete_query(&config, "deleted_at", &wm, 100);
        assert!(query.contains("deleted_at IS NOT NULL"));
        assert!(query.contains("deleted_at > 0"));
        assert!(query.contains("ORDER BY deleted_at ASC, id ASC"));
    }

    #[test]
    fn test_soft_delete_query_composite_pk() {
        let config = composite_pk_table_config();
        let wm = Watermark::new();

        let query = TestConsumer::build_soft_delete_query(&config, "deleted_at", &wm, 50);
        assert!(query.contains("ORDER BY deleted_at ASC, order_id ASC, item_id ASC"));
    }

    #[test]
    fn test_insert_vs_update_heuristic() {
        let schema = arrow::datatypes::Schema::new(vec![
            arrow::datatypes::Field::new("id", arrow::datatypes::DataType::Int64, false),
            arrow::datatypes::Field::new("created_at", arrow::datatypes::DataType::Int64, false),
            arrow::datatypes::Field::new("updated_at", arrow::datatypes::DataType::Int64, false),
        ]);
        let batch = RecordBatch::try_new(
            Arc::new(schema),
            vec![
                Arc::new(arrow::array::Int64Array::from(vec![1])),
                Arc::new(arrow::array::Int64Array::from(vec![1000])),
                Arc::new(arrow::array::Int64Array::from(vec![1000])),
            ],
        )
        .unwrap();

        let config = test_table_config();
        let event = TestConsumer::row_to_event(&batch, 0, &config, 1).unwrap();
        assert_eq!(event.operation, CdcOperation::Insert);

        let schema2 = arrow::datatypes::Schema::new(vec![
            arrow::datatypes::Field::new("id", arrow::datatypes::DataType::Int64, false),
            arrow::datatypes::Field::new("created_at", arrow::datatypes::DataType::Int64, false),
            arrow::datatypes::Field::new("updated_at", arrow::datatypes::DataType::Int64, false),
        ]);
        let batch2 = RecordBatch::try_new(
            Arc::new(schema2),
            vec![
                Arc::new(arrow::array::Int64Array::from(vec![1])),
                Arc::new(arrow::array::Int64Array::from(vec![1000])),
                Arc::new(arrow::array::Int64Array::from(vec![2000])),
            ],
        )
        .unwrap();

        let event2 = TestConsumer::row_to_event(&batch2, 0, &config, 2).unwrap();
        assert_eq!(event2.operation, CdcOperation::Update);
    }

    #[cfg(feature = "sqlite")]
    #[test]
    fn test_prune_is_noop() {
        let rt = tokio::runtime::Builder::new_current_thread()
            .enable_all()
            .build()
            .unwrap();
        let conn = connector_arrow::rusqlite::SQLiteConnection::new(
            connector_arrow::rusqlite::rusqlite::Connection::open_in_memory().unwrap(),
        );
        let config = TimestampCdcConfig {
            tables: vec![],
            poll_batch_size: 100,
            delete_detection: DeleteDetection::Disabled,
        };
        let consumer = TimestampCdcConsumer::new(conn, config);

        rt.block_on(async {
            use rhei_core::CdcConsumer;
            let pruned = consumer.prune(999).await.unwrap();
            assert_eq!(pruned, 0);
        });
    }

    #[cfg(feature = "sqlite")]
    #[tokio::test]
    async fn test_watermark_persistence_across_restarts() {
        use crate::watermark::RocksDbWatermarkStore;

        let dir = tempfile::tempdir().unwrap();
        let wm_path = dir.path().join("wm_restart_test");
        let wm_path_str = wm_path.to_str().unwrap().to_string();

        // Create an in-memory SQLite source with a test table
        let raw_conn = connector_arrow::rusqlite::rusqlite::Connection::open_in_memory().unwrap();
        raw_conn
            .execute_batch(
                "CREATE TABLE events (id INTEGER PRIMARY KEY, name TEXT, created_at INTEGER, updated_at INTEGER);
                 INSERT INTO events VALUES (1, 'a', 100, 100);
                 INSERT INTO events VALUES (2, 'b', 200, 200);",
            )
            .unwrap();
        let sqlite_conn = connector_arrow::rusqlite::SQLiteConnection::new(raw_conn);

        let table_config = test_table_config();
        let cdc_config = TimestampCdcConfig {
            tables: vec![table_config],
            poll_batch_size: 100,
            delete_detection: DeleteDetection::Disabled,
        };

        // First consumer — poll all events
        let store = Box::new(RocksDbWatermarkStore::open(&wm_path_str).unwrap());
        let consumer =
            TimestampCdcConsumer::with_watermark_store(sqlite_conn, cdc_config.clone(), store);

        let events = {
            use rhei_core::CdcConsumer;
            consumer.poll(None, 100).await.unwrap()
        };
        assert_eq!(events.len(), 2);
        drop(consumer);

        // Second consumer with same store path — should resume and find nothing new
        let raw_conn2 = connector_arrow::rusqlite::rusqlite::Connection::open_in_memory().unwrap();
        raw_conn2
            .execute_batch(
                "CREATE TABLE events (id INTEGER PRIMARY KEY, name TEXT, created_at INTEGER, updated_at INTEGER);
                 INSERT INTO events VALUES (1, 'a', 100, 100);
                 INSERT INTO events VALUES (2, 'b', 200, 200);",
            )
            .unwrap();
        let sqlite_conn2 = connector_arrow::rusqlite::SQLiteConnection::new(raw_conn2);

        let store2 = Box::new(RocksDbWatermarkStore::open(&wm_path_str).unwrap());
        let consumer2 =
            TimestampCdcConsumer::with_watermark_store(sqlite_conn2, cdc_config.clone(), store2);

        let events2 = {
            use rhei_core::CdcConsumer;
            consumer2.poll(None, 100).await.unwrap()
        };
        // No new events — watermark resumed from persisted state
        assert_eq!(events2.len(), 0);

        // Add a new row and verify it is picked up
        {
            let conn_guard = consumer2.conn.clone();
            tokio::task::spawn_blocking(move || {
                let mut conn = conn_guard.blocking_lock();
                conn.query("INSERT INTO events VALUES (3, 'c', 300, 300)")
                    .ok();
            })
            .await
            .unwrap();
        }

        let events3 = {
            use rhei_core::CdcConsumer;
            consumer2.poll(None, 100).await.unwrap()
        };
        assert_eq!(events3.len(), 1);
        assert_eq!(events3[0].table, "events");
    }

    // -----------------------------------------------------------------------
    // Test helpers: FailingWatermarkStore for injecting persistence failures
    // -----------------------------------------------------------------------

    /// A [`WatermarkStore`] that delegates loads to an inner store but always
    /// fails save operations. Used to simulate disk-full / I/O errors.
    struct FailingSaveStore {
        inner: Box<dyn WatermarkStore>,
    }

    impl FailingSaveStore {
        fn new(inner: Box<dyn WatermarkStore>) -> Self {
            Self { inner }
        }
    }

    impl WatermarkStore for FailingSaveStore {
        fn load(&self, table_name: &str) -> Result<Option<Watermark>, SidecarError> {
            self.inner.load(table_name)
        }
        fn save(&self, _table_name: &str, _wm: &Watermark) -> Result<(), SidecarError> {
            Err(SidecarError::WatermarkStore(
                "injected save failure".to_string(),
            ))
        }
        fn load_global_seq(&self) -> Result<i64, SidecarError> {
            self.inner.load_global_seq()
        }
        fn save_global_seq(&self, _seq: i64) -> Result<(), SidecarError> {
            Err(SidecarError::WatermarkStore(
                "injected save_global_seq failure".to_string(),
            ))
        }
    }

    /// A [`WatermarkStore`] whose loads always return an error. Used to test
    /// that `try_with_watermark_store` propagates startup errors.
    struct FailingLoadStore;

    impl WatermarkStore for FailingLoadStore {
        fn load(&self, _table_name: &str) -> Result<Option<Watermark>, SidecarError> {
            Err(SidecarError::WatermarkStore(
                "injected load failure (corrupt store)".to_string(),
            ))
        }
        fn save(&self, _table_name: &str, _wm: &Watermark) -> Result<(), SidecarError> {
            Ok(())
        }
        fn load_global_seq(&self) -> Result<i64, SidecarError> {
            Err(SidecarError::WatermarkStore(
                "injected load_global_seq failure (corrupt store)".to_string(),
            ))
        }
        fn save_global_seq(&self, _seq: i64) -> Result<(), SidecarError> {
            Ok(())
        }
    }

    // -----------------------------------------------------------------------
    // Issue 1 — Event loss when watermark persistence fails
    // -----------------------------------------------------------------------

    /// When `persist_watermarks` fails mid-poll, the in-memory cursor must be
    /// rolled back to its pre-poll state. The next `poll` call must re-fetch
    /// the same rows so no CDC events are permanently dropped.
    #[cfg(feature = "sqlite")]
    #[tokio::test]
    async fn test_poll_rollback_on_persist_failure() {
        use crate::watermark::NullWatermarkStore;
        use rhei_core::CdcConsumer;

        // Build a SQLite source with two rows
        let raw_conn = connector_arrow::rusqlite::rusqlite::Connection::open_in_memory().unwrap();
        raw_conn
            .execute_batch(
                "CREATE TABLE events (id INTEGER PRIMARY KEY, name TEXT, created_at INTEGER, updated_at INTEGER);
                 INSERT INTO events VALUES (1, 'a', 100, 100);
                 INSERT INTO events VALUES (2, 'b', 200, 200);",
            )
            .unwrap();
        let sqlite_conn = connector_arrow::rusqlite::SQLiteConnection::new(raw_conn);

        let table_config = test_table_config();
        let cdc_config = TimestampCdcConfig {
            tables: vec![table_config],
            poll_batch_size: 100,
            delete_detection: DeleteDetection::Disabled,
        };

        // Use a FailingSaveStore so every persist attempt returns an error
        let failing_store = Box::new(FailingSaveStore::new(Box::new(NullWatermarkStore)));
        let consumer =
            TimestampCdcConsumer::with_watermark_store(sqlite_conn, cdc_config, failing_store);

        // First poll: rows exist in source, but persistence will fail
        let first_result = consumer.poll(None, 100).await;
        assert!(
            first_result.is_err(),
            "poll should propagate the persistence error"
        );

        // Verify in-memory state was rolled back: watermarks still at zero,
        // global_seq still at 1
        let wm_guard = consumer.watermarks.lock().await;
        let events_wm = wm_guard
            .get("events")
            .cloned()
            .unwrap_or_else(Watermark::new);
        assert_eq!(
            events_wm.timestamp, 0,
            "watermark timestamp should be rolled back to pre-poll value"
        );
        assert!(
            events_wm.last_pk.is_none(),
            "watermark last_pk should be rolled back to pre-poll value"
        );
        drop(wm_guard);

        assert_eq!(
            consumer.global_seq.load(Ordering::Relaxed),
            1,
            "global_seq should be rolled back to pre-poll value"
        );
    }

    // -----------------------------------------------------------------------
    // Issue 2 — Startup silently resets on corruption
    // -----------------------------------------------------------------------

    /// `try_with_watermark_store` must return `Err` when the store fails to
    /// load the global sequence counter (e.g. corrupt RocksDB data), rather
    /// than silently resetting to seq=1 and replaying history from the start.
    #[cfg(feature = "sqlite")]
    #[test]
    fn test_try_with_watermark_store_fails_on_corrupt_load() {
        let raw_conn = connector_arrow::rusqlite::rusqlite::Connection::open_in_memory().unwrap();
        let sqlite_conn = connector_arrow::rusqlite::SQLiteConnection::new(raw_conn);

        let table_config = test_table_config();
        let cdc_config = TimestampCdcConfig {
            tables: vec![table_config],
            poll_batch_size: 100,
            delete_detection: DeleteDetection::Disabled,
        };

        let result = TimestampCdcConsumer::try_with_watermark_store(
            sqlite_conn,
            cdc_config,
            Box::new(FailingLoadStore),
        );

        assert!(
            result.is_err(),
            "try_with_watermark_store must return Err on corrupt watermark load"
        );
        let err_str = result.err().unwrap().to_string();
        assert!(
            err_str.contains("watermark store"),
            "error message should mention watermark store, got: {err_str}"
        );
    }

    /// `with_watermark_store` (permissive) must succeed even when the store
    /// returns errors, falling back to zero watermarks with a warning logged.
    #[cfg(feature = "sqlite")]
    #[test]
    fn test_with_watermark_store_falls_back_on_corrupt_load() {
        let raw_conn = connector_arrow::rusqlite::rusqlite::Connection::open_in_memory().unwrap();
        let sqlite_conn = connector_arrow::rusqlite::SQLiteConnection::new(raw_conn);

        let table_config = test_table_config();
        let cdc_config = TimestampCdcConfig {
            tables: vec![table_config],
            poll_batch_size: 100,
            delete_detection: DeleteDetection::Disabled,
        };

        // Should NOT panic or return Err — the permissive variant falls back
        let consumer = TimestampCdcConsumer::with_watermark_store(
            sqlite_conn,
            cdc_config,
            Box::new(FailingLoadStore),
        );

        // Watermark should be at default (zero) after fallback
        let rt = tokio::runtime::Builder::new_current_thread()
            .enable_all()
            .build()
            .unwrap();
        rt.block_on(async {
            let wm_guard = consumer.watermarks.lock().await;
            let events_wm = wm_guard
                .get("events")
                .cloned()
                .unwrap_or_else(Watermark::new);
            assert_eq!(
                events_wm.timestamp, 0,
                "permissive fallback should use zero watermark"
            );
        });
        assert_eq!(
            consumer.global_seq.load(Ordering::Relaxed),
            1,
            "permissive fallback should use seq=1"
        );
    }
}