sui_indexer_alt_framework/pipeline/concurrent/
committer.rs

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
// Copyright (c) Mysten Labs, Inc.
// SPDX-License-Identifier: Apache-2.0

use std::{sync::Arc, time::Duration};

use backoff::ExponentialBackoff;
use tokio::{sync::mpsc, task::JoinHandle};
use tokio_stream::wrappers::ReceiverStream;
use tokio_util::sync::CancellationToken;
use tracing::{debug, error, info, warn};

use crate::{
    metrics::{CheckpointLagMetricReporter, IndexerMetrics},
    pipeline::{Break, CommitterConfig, WatermarkPart},
    store::Store,
    task::TrySpawnStreamExt,
};

use super::{BatchedRows, Handler};

/// If the committer needs to retry a commit, it will wait this long initially.
const INITIAL_RETRY_INTERVAL: Duration = Duration::from_millis(100);

/// If the committer needs to retry a commit, it will wait at most this long between retries.
const MAX_RETRY_INTERVAL: Duration = Duration::from_secs(1);

/// The committer task is responsible for writing batches of rows to the database. It receives
/// batches on `rx` and writes them out to the `db` concurrently (`config.write_concurrency`
/// controls the degree of fan-out).
///
/// The writing of each batch will be repeatedly retried on an exponential back-off until it
/// succeeds. Once the write succeeds, the [WatermarkPart]s for that batch are sent on `tx` to the
/// watermark task, as long as `skip_watermark` is not true.
///
/// This task will shutdown via its `cancel`lation token, or if its receiver or sender channels are
/// closed.
pub(super) fn committer<H: Handler + 'static>(
    config: CommitterConfig,
    skip_watermark: bool,
    rx: mpsc::Receiver<BatchedRows<H>>,
    tx: mpsc::Sender<Vec<WatermarkPart>>,
    db: H::Store,
    metrics: Arc<IndexerMetrics>,
    cancel: CancellationToken,
) -> JoinHandle<()> {
    tokio::spawn(async move {
        info!(pipeline = H::NAME, "Starting committer");
        let checkpoint_lag_reporter = CheckpointLagMetricReporter::new_for_pipeline::<H>(
            &metrics.partially_committed_checkpoint_timestamp_lag,
            &metrics.latest_partially_committed_checkpoint_timestamp_lag_ms,
            &metrics.latest_partially_committed_checkpoint,
        );

        match ReceiverStream::new(rx)
            .try_for_each_spawned(
                config.write_concurrency,
                |BatchedRows { values, watermark }| {
                    let values = Arc::new(values);
                    let tx = tx.clone();
                    let db = db.clone();
                    let metrics = metrics.clone();
                    let cancel = cancel.clone();
                    let checkpoint_lag_reporter = checkpoint_lag_reporter.clone();

                    // Repeatedly try to get a connection to the DB and write the batch. Use an
                    // exponential backoff in case the failure is due to contention over the DB
                    // connection pool.
                    let backoff = ExponentialBackoff {
                        initial_interval: INITIAL_RETRY_INTERVAL,
                        current_interval: INITIAL_RETRY_INTERVAL,
                        max_interval: MAX_RETRY_INTERVAL,
                        max_elapsed_time: None,
                        ..Default::default()
                    };

                    let highest_checkpoint = watermark.iter().map(|w| w.checkpoint()).max();
                    let highest_checkpoint_timestamp =
                        watermark.iter().map(|w| w.timestamp_ms()).max();

                    use backoff::Error as BE;
                    let commit = move || {
                        let values = values.clone();
                        let db = db.clone();
                        let metrics = metrics.clone();
                        let checkpoint_lag_reporter = checkpoint_lag_reporter.clone();
                        async move {
                            if values.is_empty() {
                                return Ok(());
                            }

                            metrics
                                .total_committer_batches_attempted
                                .with_label_values(&[H::NAME])
                                .inc();

                            let guard = metrics
                                .committer_commit_latency
                                .with_label_values(&[H::NAME])
                                .start_timer();

                            let mut conn = db.connect().await.map_err(|e| {
                                warn!(
                                    pipeline = H::NAME,
                                    "Committed failed to get connection for DB"
                                );

                                metrics
                                    .total_committer_batches_failed
                                    .with_label_values(&[H::NAME])
                                    .inc();

                                BE::transient(Break::Err(e))
                            })?;

                            let affected = H::commit(values.as_slice(), &mut conn).await;
                            let elapsed = guard.stop_and_record();

                            match affected {
                                Ok(affected) => {
                                    debug!(
                                        pipeline = H::NAME,
                                        elapsed_ms = elapsed * 1000.0,
                                        affected,
                                        committed = values.len(),
                                        "Wrote batch",
                                    );

                                    checkpoint_lag_reporter.report_lag(
                                        // unwrap is safe because we would have returned if values is empty.
                                        highest_checkpoint.unwrap(),
                                        highest_checkpoint_timestamp.unwrap(),
                                    );

                                    metrics
                                        .total_committer_batches_succeeded
                                        .with_label_values(&[H::NAME])
                                        .inc();

                                    metrics
                                        .total_committer_rows_committed
                                        .with_label_values(&[H::NAME])
                                        .inc_by(values.len() as u64);

                                    metrics
                                        .total_committer_rows_affected
                                        .with_label_values(&[H::NAME])
                                        .inc_by(affected as u64);

                                    metrics
                                        .committer_tx_rows
                                        .with_label_values(&[H::NAME])
                                        .observe(affected as f64);

                                    Ok(())
                                }

                                Err(e) => {
                                    warn!(
                                        pipeline = H::NAME,
                                        elapsed_ms = elapsed * 1000.0,
                                        committed = values.len(),
                                        "Error writing batch: {e}",
                                    );

                                    metrics
                                        .total_committer_batches_failed
                                        .with_label_values(&[H::NAME])
                                        .inc();

                                    Err(BE::transient(Break::Err(e)))
                                }
                            }
                        }
                    };

                    async move {
                        tokio::select! {
                            _ = cancel.cancelled() => {
                                return Err(Break::Cancel);
                            }

                            // Double check that the commit actually went through, (this backoff should
                            // not produce any permanent errors, but if it does, we need to shutdown
                            // the pipeline).
                            commit = backoff::future::retry(backoff, commit) => {
                                let () = commit?;
                            }
                        };

                        if !skip_watermark && tx.send(watermark).await.is_err() {
                            info!(pipeline = H::NAME, "Watermark closed channel");
                            return Err(Break::Cancel);
                        }

                        Ok(())
                    }
                },
            )
            .await
        {
            Ok(()) => {
                info!(pipeline = H::NAME, "Batches done, stopping committer");
            }

            Err(Break::Cancel) => {
                info!(pipeline = H::NAME, "Shutdown received, stopping committer");
            }

            Err(Break::Err(e)) => {
                error!(pipeline = H::NAME, "Error from committer: {e}");
                cancel.cancel();
            }
        }
    })
}

#[cfg(test)]
mod tests {
    use std::{
        collections::HashMap,
        sync::{
            atomic::{AtomicUsize, Ordering},
            Arc, Mutex,
        },
    };

    use anyhow::ensure;
    use async_trait::async_trait;
    use sui_types::full_checkpoint_content::CheckpointData;
    use tokio::sync::mpsc;
    use tokio_util::sync::CancellationToken;

    use crate::{
        metrics::IndexerMetrics,
        pipeline::{
            concurrent::{BatchedRows, Handler},
            Processor, WatermarkPart,
        },
        store::CommitterWatermark,
        testing::mock_store::*,
        FieldCount,
    };

    use super::*;

    #[derive(Clone, FieldCount, Default)]
    pub struct StoredData {
        pub cp_sequence_number: u64,
        pub tx_sequence_numbers: Vec<u64>,
        /// Tracks remaining commit failures for testing retry logic.
        /// The committer spawns concurrent tasks that call H::commit,
        /// so this needs to be thread-safe (hence Arc<AtomicUsize>).
        pub commit_failure_remaining: Arc<AtomicUsize>,
        pub commit_delay_ms: u64,
    }

    pub struct DataPipeline;

    impl Processor for DataPipeline {
        const NAME: &'static str = "data";

        type Value = StoredData;

        fn process(&self, _checkpoint: &Arc<CheckpointData>) -> anyhow::Result<Vec<Self::Value>> {
            Ok(vec![])
        }
    }

    #[async_trait]
    impl Handler for DataPipeline {
        type Store = MockStore;

        async fn commit<'a>(
            values: &[StoredData],
            conn: &mut MockConnection<'a>,
        ) -> anyhow::Result<usize> {
            for value in values {
                // If there's a delay, sleep for that duration
                if value.commit_delay_ms > 0 {
                    tokio::time::sleep(Duration::from_millis(value.commit_delay_ms)).await;
                }

                // If there are remaining failures, fail the commit and decrement the counter
                {
                    let remaining = value
                        .commit_failure_remaining
                        .fetch_sub(1, Ordering::Relaxed);
                    ensure!(
                        remaining == 0,
                        "Commit failed, remaining failures: {}",
                        remaining - 1
                    );
                }

                // Lock, insert, and immediately drop the lock
                {
                    let mut data = conn.0.data.lock().unwrap();
                    data.insert(value.cp_sequence_number, value.tx_sequence_numbers.clone());
                }
            }
            Ok(values.len())
        }
    }

    struct TestSetup {
        store: MockStore,
        batch_tx: mpsc::Sender<BatchedRows<DataPipeline>>,
        watermark_rx: mpsc::Receiver<Vec<WatermarkPart>>,
        committer_handle: JoinHandle<()>,
    }

    /// Creates and spawns a committer task with the provided mock store, along with
    /// all necessary channels and configuration. The committer runs in the background
    /// and can be interacted with through the returned channels.
    ///
    /// # Arguments
    /// * `store` - The mock store to use for testing
    /// * `skip_watermark` - Whether to skip sending watermarks to the watermark channel
    async fn setup_test(store: MockStore, skip_watermark: bool) -> TestSetup {
        let config = CommitterConfig::default();
        let metrics = IndexerMetrics::new(&Default::default());
        let cancel = CancellationToken::new();

        let (batch_tx, batch_rx) = mpsc::channel::<BatchedRows<DataPipeline>>(10);
        let (watermark_tx, watermark_rx) = mpsc::channel(10);

        let store_clone = store.clone();
        let committer_handle = tokio::spawn(async move {
            let _ = committer(
                config,
                skip_watermark,
                batch_rx,
                watermark_tx,
                store_clone,
                metrics,
                cancel,
            )
            .await;
        });

        TestSetup {
            store,
            batch_tx,
            watermark_rx,
            committer_handle,
        }
    }

    #[tokio::test]
    async fn test_concurrent_batch_processing() {
        let mut setup = setup_test(MockStore::default(), false).await;

        // Send batches
        let batch1 = BatchedRows {
            values: vec![
                StoredData {
                    cp_sequence_number: 1,
                    tx_sequence_numbers: vec![1, 2, 3],
                    ..Default::default()
                },
                StoredData {
                    cp_sequence_number: 2,
                    tx_sequence_numbers: vec![4, 5, 6],
                    ..Default::default()
                },
            ],
            watermark: vec![
                WatermarkPart {
                    watermark: CommitterWatermark {
                        epoch_hi_inclusive: 0,
                        checkpoint_hi_inclusive: 1,
                        tx_hi: 3,
                        timestamp_ms_hi_inclusive: 1000,
                    },
                    batch_rows: 1,
                    total_rows: 1, // Total rows from checkpoint 1
                },
                WatermarkPart {
                    watermark: CommitterWatermark {
                        epoch_hi_inclusive: 0,
                        checkpoint_hi_inclusive: 2,
                        tx_hi: 6,
                        timestamp_ms_hi_inclusive: 2000,
                    },
                    batch_rows: 1,
                    total_rows: 1, // Total rows from checkpoint 2
                },
            ],
        };

        let batch2 = BatchedRows {
            values: vec![StoredData {
                cp_sequence_number: 3,
                tx_sequence_numbers: vec![7, 8, 9],
                ..Default::default()
            }],
            watermark: vec![WatermarkPart {
                watermark: CommitterWatermark {
                    epoch_hi_inclusive: 0,
                    checkpoint_hi_inclusive: 3,
                    tx_hi: 9,
                    timestamp_ms_hi_inclusive: 3000,
                },
                batch_rows: 1,
                total_rows: 1, // Total rows from checkpoint 3
            }],
        };

        setup.batch_tx.send(batch1).await.unwrap();
        setup.batch_tx.send(batch2).await.unwrap();

        // Verify watermarks. Blocking until committer has processed.
        let watermark1 = setup.watermark_rx.recv().await.unwrap();
        let watermark2 = setup.watermark_rx.recv().await.unwrap();
        assert_eq!(watermark1.len(), 2);
        assert_eq!(watermark2.len(), 1);

        // Verify data was committed
        {
            let data: std::sync::MutexGuard<'_, HashMap<u64, Vec<u64>>> =
                setup.store.data.lock().unwrap();
            assert_eq!(data.len(), 3);
            assert_eq!(data.get(&1).unwrap(), &vec![1, 2, 3]);
            assert_eq!(data.get(&2).unwrap(), &vec![4, 5, 6]);
            assert_eq!(data.get(&3).unwrap(), &vec![7, 8, 9]);
        }

        // Clean up
        drop(setup.batch_tx);
        let _ = setup.committer_handle.await;
    }

    #[tokio::test]
    async fn test_commit_with_retries_for_commit_failure() {
        let mut setup = setup_test(MockStore::default(), false).await;

        // Create a batch with a single item that will fail once before succeeding
        let batch = BatchedRows {
            values: vec![StoredData {
                cp_sequence_number: 1,
                tx_sequence_numbers: vec![1, 2, 3],
                commit_failure_remaining: Arc::new(AtomicUsize::new(1)),
                commit_delay_ms: 1_000, // Long commit delay for testing state between retry
            }],
            watermark: vec![WatermarkPart {
                watermark: CommitterWatermark {
                    epoch_hi_inclusive: 0,
                    checkpoint_hi_inclusive: 1,
                    tx_hi: 3,
                    timestamp_ms_hi_inclusive: 1000,
                },
                batch_rows: 1,
                total_rows: 1,
            }],
        };

        // Send the batch
        setup.batch_tx.send(batch).await.unwrap();

        // Wait for the first attempt to fail and before the retry succeeds
        tokio::time::sleep(Duration::from_millis(1_500)).await;

        // Verify state before retry succeeds
        {
            let data = setup.store.data.lock().unwrap();
            assert!(
                data.is_empty(),
                "Data should not be committed before retry succeeds"
            );
        }
        assert!(
            setup.watermark_rx.try_recv().is_err(),
            "No watermark should be received before retry succeeds"
        );

        // Wait for the retry to succeed
        tokio::time::sleep(Duration::from_millis(1_500)).await;

        // Verify state after retry succeeds
        {
            let data = setup.store.data.lock().unwrap();
            assert_eq!(data.get(&1).unwrap(), &vec![1, 2, 3]);
        }
        let watermark = setup.watermark_rx.recv().await.unwrap();
        assert_eq!(watermark.len(), 1);

        // Clean up
        drop(setup.batch_tx);
        let _ = setup.committer_handle.await;
    }

    #[tokio::test]
    async fn test_commit_with_retries_for_connection_failure() {
        // Create a batch with a single item
        let store = MockStore {
            connection_failure: Arc::new(Mutex::new(ConnectionFailure {
                connection_failure_attempts: 1,
                connection_delay_ms: 1_000, // Long connection delay for testing state between retry
                ..Default::default()
            })),
            ..Default::default()
        };
        let mut setup = setup_test(store, false).await;

        let batch = BatchedRows {
            values: vec![StoredData {
                cp_sequence_number: 1,
                tx_sequence_numbers: vec![1, 2, 3],
                ..Default::default()
            }],
            watermark: vec![WatermarkPart {
                watermark: CommitterWatermark {
                    epoch_hi_inclusive: 0,
                    checkpoint_hi_inclusive: 1,
                    tx_hi: 3,
                    timestamp_ms_hi_inclusive: 1000,
                },
                batch_rows: 1,
                total_rows: 1,
            }],
        };

        // Send the batch
        setup.batch_tx.send(batch).await.unwrap();

        // Wait for the first attempt to fail and before the retry succeeds
        tokio::time::sleep(Duration::from_millis(1_500)).await;

        // Verify state before retry succeeds
        {
            let data = setup.store.data.lock().unwrap();
            assert!(
                data.is_empty(),
                "Data should not be committed before retry succeeds"
            );
        }
        assert!(
            setup.watermark_rx.try_recv().is_err(),
            "No watermark should be received before retry succeeds"
        );

        // Wait for the retry to succeed
        tokio::time::sleep(Duration::from_millis(1_500)).await;

        // Verify state after retry succeeds
        {
            let data = setup.store.data.lock().unwrap();
            assert_eq!(data.get(&1).unwrap(), &vec![1, 2, 3]);
        }
        let watermark = setup.watermark_rx.recv().await.unwrap();
        assert_eq!(watermark.len(), 1);

        // Clean up
        drop(setup.batch_tx);
        let _ = setup.committer_handle.await;
    }

    #[tokio::test]
    async fn test_empty_batch_handling() {
        let mut setup = setup_test(MockStore::default(), false).await;

        let empty_batch = BatchedRows {
            values: vec![], // Empty values
            watermark: vec![WatermarkPart {
                watermark: CommitterWatermark {
                    epoch_hi_inclusive: 0,
                    checkpoint_hi_inclusive: 1,
                    tx_hi: 0,
                    timestamp_ms_hi_inclusive: 1000,
                },
                batch_rows: 0,
                total_rows: 0,
            }],
        };

        // Send the empty batch
        setup.batch_tx.send(empty_batch).await.unwrap();

        // Verify watermark is still sent (empty batches should still produce watermarks)
        let watermark = setup.watermark_rx.recv().await.unwrap();
        assert_eq!(watermark.len(), 1);
        assert_eq!(watermark[0].batch_rows, 0);
        assert_eq!(watermark[0].total_rows, 0);

        // Verify no data was committed (since batch was empty)
        {
            let data = setup.store.data.lock().unwrap();
            assert!(
                data.is_empty(),
                "No data should be committed for empty batch"
            );
        }

        // Clean up
        drop(setup.batch_tx);
        let _ = setup.committer_handle.await;
    }

    #[tokio::test]
    async fn test_skip_watermark_mode() {
        let mut setup = setup_test(MockStore::default(), true).await;

        let batch = BatchedRows {
            values: vec![StoredData {
                cp_sequence_number: 1,
                tx_sequence_numbers: vec![1, 2, 3],
                ..Default::default()
            }],
            watermark: vec![WatermarkPart {
                watermark: CommitterWatermark {
                    epoch_hi_inclusive: 0,
                    checkpoint_hi_inclusive: 1,
                    tx_hi: 3,
                    timestamp_ms_hi_inclusive: 1000,
                },
                batch_rows: 1,
                total_rows: 1,
            }],
        };

        // Send the batch
        setup.batch_tx.send(batch).await.unwrap();

        // Wait for processing
        tokio::time::sleep(Duration::from_millis(200)).await;

        // Verify data was committed
        {
            let data = setup.store.data.lock().unwrap();
            assert_eq!(data.get(&1).unwrap(), &vec![1, 2, 3]);
        }

        // Verify no watermark was sent (skip_watermark mode)
        assert!(
            setup.watermark_rx.try_recv().is_err(),
            "No watermark should be sent in skip_watermark mode"
        );

        // Clean up
        drop(setup.batch_tx);
        let _ = setup.committer_handle.await;
    }

    #[tokio::test]
    async fn test_watermark_channel_closed() {
        let setup = setup_test(MockStore::default(), false).await;

        let batch = BatchedRows {
            values: vec![StoredData {
                cp_sequence_number: 1,
                tx_sequence_numbers: vec![1, 2, 3],
                ..Default::default()
            }],
            watermark: vec![WatermarkPart {
                watermark: CommitterWatermark {
                    epoch_hi_inclusive: 0,
                    checkpoint_hi_inclusive: 1,
                    tx_hi: 3,
                    timestamp_ms_hi_inclusive: 1000,
                },
                batch_rows: 1,
                total_rows: 1,
            }],
        };

        // Send the batch
        setup.batch_tx.send(batch).await.unwrap();

        // Wait for processing.
        tokio::time::sleep(Duration::from_millis(100)).await;

        // Close the watermark channel by dropping the receiver
        drop(setup.watermark_rx);

        // Wait a bit more for the committer to handle the channel closure
        tokio::time::sleep(Duration::from_millis(200)).await;

        // Verify data was still committed despite watermark channel closure
        {
            let data = setup.store.data.lock().unwrap();
            assert_eq!(data.get(&1).unwrap(), &vec![1, 2, 3]);
        }

        // Close the batch channel to allow the committer to terminate
        drop(setup.batch_tx);

        // Verify the committer task has terminated due to watermark channel closure
        // The task should exit gracefully when it can't send watermarks (returns Break::Cancel)
        let result = setup.committer_handle.await;
        assert!(
            result.is_ok(),
            "Committer should terminate gracefully when watermark channel is closed"
        );
    }
}