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

use std::{
    cmp::Ordering,
    collections::{btree_map::Entry, BTreeMap},
    sync::Arc,
};

use tokio::{
    sync::mpsc,
    task::JoinHandle,
    time::{interval, MissedTickBehavior},
};
use tokio_util::sync::CancellationToken;
use tracing::{debug, error, info, warn};

use crate::{
    metrics::{CheckpointLagMetricReporter, IndexerMetrics},
    pipeline::{logging::WatermarkLogger, CommitterConfig, WatermarkPart, WARN_PENDING_WATERMARKS},
    store::{CommitterWatermark, Connection, Store},
};

use super::Handler;

/// The watermark task is responsible for keeping track of a pipeline's out-of-order commits and
/// updating its row in the `watermarks` table when a continuous run of checkpoints have landed
/// since the last watermark update.
///
/// It receives watermark "parts" that detail the proportion of each checkpoint's data that has
/// been written out by the committer and periodically (on a configurable interval) checks if the
/// watermark for the pipeline can be pushed forward. The watermark can be pushed forward if there
/// is one or more complete (all data for that checkpoint written out) watermarks spanning
/// contiguously from the current high watermark into the future.
///
/// If it detects that more than [WARN_PENDING_WATERMARKS] watermarks have built up, it will issue
/// a warning, as this could be the indication of a memory leak, and the caller probably intended
/// to run the indexer with watermarking disabled (e.g. if they are running a backfill).
///
/// The task regularly traces its progress, outputting at a higher log level every
/// [LOUD_WATERMARK_UPDATE_INTERVAL]-many checkpoints.
///
/// The task will shutdown if the `cancel` token is signalled, or if the `rx` channel closes and
/// the watermark cannot be progressed. If `skip_watermark` is set, the task will shutdown
/// immediately.
pub(super) fn commit_watermark<H: Handler + 'static>(
    initial_watermark: Option<CommitterWatermark>,
    config: CommitterConfig,
    skip_watermark: bool,
    mut rx: mpsc::Receiver<Vec<WatermarkPart>>,
    store: H::Store,
    metrics: Arc<IndexerMetrics>,
    cancel: CancellationToken,
) -> JoinHandle<()> {
    tokio::spawn(async move {
        if skip_watermark {
            info!(pipeline = H::NAME, "Skipping commit watermark task");
            return;
        }

        let mut poll = interval(config.watermark_interval());
        poll.set_missed_tick_behavior(MissedTickBehavior::Delay);

        // To correctly update the watermark, the task tracks the watermark it last tried to write
        // and the watermark parts for any checkpoints that have been written since then
        // ("pre-committed"). After each batch is written, the task will try to progress the
        // watermark as much as possible without going over any holes in the sequence of
        // checkpoints (entirely missing watermarks, or incomplete watermarks).
        let mut precommitted: BTreeMap<u64, WatermarkPart> = BTreeMap::new();
        let (mut watermark, mut next_checkpoint) = if let Some(watermark) = initial_watermark {
            let next = watermark.checkpoint_hi_inclusive + 1;
            (watermark, next)
        } else {
            (CommitterWatermark::default(), 0)
        };

        // The watermark task will periodically output a log message at a higher log level to
        // demonstrate that the pipeline is making progress.
        let mut logger = WatermarkLogger::new("concurrent_committer", &watermark);

        let checkpoint_lag_reporter = CheckpointLagMetricReporter::new_for_pipeline::<H>(
            &metrics.watermarked_checkpoint_timestamp_lag,
            &metrics.latest_watermarked_checkpoint_timestamp_lag_ms,
            &metrics.watermark_checkpoint_in_db,
        );

        info!(pipeline = H::NAME, ?watermark, "Starting commit watermark");

        loop {
            tokio::select! {
                _ = cancel.cancelled() => {
                    info!(pipeline = H::NAME, "Shutdown received");
                    break;
                }

                _ = poll.tick() => {
                    if precommitted.len() > WARN_PENDING_WATERMARKS {
                        warn!(
                            pipeline = H::NAME,
                            pending = precommitted.len(),
                            "Pipeline has a large number of pending commit watermarks",
                        );
                    }

                    let Ok(mut conn) = store.connect().await else {
                        warn!(pipeline = H::NAME, "Commit watermark task failed to get connection for DB");
                        continue;
                    };

                    // Check if the pipeline's watermark needs to be updated
                    let guard = metrics
                        .watermark_gather_latency
                        .with_label_values(&[H::NAME])
                        .start_timer();

                    let mut watermark_needs_update = false;
                    while let Some(pending) = precommitted.first_entry() {
                        let part = pending.get();

                        // Some rows from the next watermark have not landed yet.
                        if !part.is_complete() {
                            break;
                        }

                        match next_checkpoint.cmp(&part.watermark.checkpoint_hi_inclusive) {
                            // Next pending checkpoint is from the future.
                            Ordering::Less => break,

                            // This is the next checkpoint -- include it.
                            Ordering::Equal => {
                                watermark = pending.remove().watermark;
                                watermark_needs_update = true;
                                next_checkpoint += 1;
                            }

                            // Next pending checkpoint is in the past. Out of order watermarks can
                            // be encountered when a pipeline is starting up, because ingestion
                            // must start at the lowest checkpoint across all pipelines, or because
                            // of a backfill, where the initial checkpoint has been overridden.
                            Ordering::Greater => {
                                // Track how many we see to make sure it doesn't grow without
                                // bound.
                                metrics
                                    .total_watermarks_out_of_order
                                    .with_label_values(&[H::NAME])
                                    .inc();

                                pending.remove();
                            }
                        }
                    }

                    let elapsed = guard.stop_and_record();

                    metrics
                        .watermark_epoch
                        .with_label_values(&[H::NAME])
                        .set(watermark.epoch_hi_inclusive as i64);

                    metrics
                        .watermark_checkpoint
                        .with_label_values(&[H::NAME])
                        .set(watermark.checkpoint_hi_inclusive as i64);

                    metrics
                        .watermark_transaction
                        .with_label_values(&[H::NAME])
                        .set(watermark.tx_hi as i64);

                    metrics
                        .watermark_timestamp_ms
                        .with_label_values(&[H::NAME])
                        .set(watermark.timestamp_ms_hi_inclusive as i64);

                    debug!(
                        pipeline = H::NAME,
                        elapsed_ms = elapsed * 1000.0,
                        watermark = watermark.checkpoint_hi_inclusive,
                        timestamp = %watermark.timestamp(),
                        pending = precommitted.len(),
                        "Gathered watermarks",
                    );

                    if watermark_needs_update {
                        let guard = metrics
                            .watermark_commit_latency
                            .with_label_values(&[H::NAME])
                            .start_timer();

                        // TODO: If initial_watermark is empty, when we update watermark
                        // for the first time, we should also update the low watermark.
                        match conn.set_committer_watermark(
                            H::NAME,
                            watermark,
                        ).await {
                            // If there's an issue updating the watermark, log it but keep going,
                            // it's OK for the watermark to lag from a correctness perspective.
                            Err(e) => {
                                let elapsed = guard.stop_and_record();
                                error!(
                                    pipeline = H::NAME,
                                    elapsed_ms = elapsed * 1000.0,
                                    ?watermark,
                                    "Error updating commit watermark: {e}",
                                );
                            }

                            Ok(true) => {
                                let elapsed = guard.stop_and_record();

                                logger.log::<H>(&watermark, elapsed);

                                checkpoint_lag_reporter.report_lag(
                                    watermark.checkpoint_hi_inclusive,
                                    watermark.timestamp_ms_hi_inclusive
                                );

                                metrics
                                    .watermark_epoch_in_db
                                    .with_label_values(&[H::NAME])
                                    .set(watermark.epoch_hi_inclusive as i64);

                                metrics
                                    .watermark_transaction_in_db
                                    .with_label_values(&[H::NAME])
                                    .set(watermark.tx_hi as i64);

                                metrics
                                    .watermark_timestamp_in_db_ms
                                    .with_label_values(&[H::NAME])
                                    .set(watermark.timestamp_ms_hi_inclusive as i64);
                            }
                            Ok(false) => {}
                        }
                    }

                    if rx.is_closed() && rx.is_empty() {
                        info!(pipeline = H::NAME, "Committer closed channel");
                        break;
                    }
                }

                Some(parts) = rx.recv() => {
                    for part in parts {
                        match precommitted.entry(part.checkpoint()) {
                            Entry::Vacant(entry) => {
                                entry.insert(part);
                            }

                            Entry::Occupied(mut entry) => {
                                entry.get_mut().add(part);
                            }
                        }
                    }
                }
            }
        }

        info!(
            pipeline = H::NAME,
            ?watermark,
            "Stopping committer watermark task"
        );
    })
}

#[cfg(test)]
mod tests {
    use std::sync::Arc;

    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::{CommitterConfig, Processor, WatermarkPart},
        store::CommitterWatermark,
        testing::mock_store::*,
        FieldCount,
    };

    use super::*;

    #[derive(Clone, FieldCount)]
    pub struct StoredData;

    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> {
            Ok(0)
        }
    }

    struct TestSetup {
        store: MockStore,
        watermark_tx: mpsc::Sender<Vec<WatermarkPart>>,
        commit_watermark_handle: JoinHandle<()>,
        cancel: CancellationToken,
    }

    fn setup_test<H: Handler<Store = MockStore> + 'static>(
        config: CommitterConfig,
        initial_watermark: Option<CommitterWatermark>,
        store: MockStore,
    ) -> TestSetup {
        let (watermark_tx, watermark_rx) = mpsc::channel(100);
        let metrics = IndexerMetrics::new(&Default::default());
        let cancel = CancellationToken::new();

        let store_clone = store.clone();
        let cancel_clone = cancel.clone();

        let commit_watermark_handle = commit_watermark::<H>(
            initial_watermark,
            config,
            false,
            watermark_rx,
            store_clone,
            metrics,
            cancel_clone,
        );

        TestSetup {
            store,
            watermark_tx,
            commit_watermark_handle,
            cancel,
        }
    }

    fn create_watermark_part_for_checkpoint(checkpoint: u64) -> WatermarkPart {
        WatermarkPart {
            watermark: CommitterWatermark {
                checkpoint_hi_inclusive: checkpoint,
                ..Default::default()
            },
            batch_rows: 1,
            total_rows: 1,
        }
    }

    #[tokio::test]
    async fn test_basic_watermark_progression() {
        let config = CommitterConfig::default();
        let initial_watermark = Some(CommitterWatermark {
            checkpoint_hi_inclusive: 0,
            ..Default::default()
        });
        let setup = setup_test::<DataPipeline>(config, initial_watermark, MockStore::default());

        // Send watermark parts in order
        for cp in 1..4 {
            let part = create_watermark_part_for_checkpoint(cp);
            setup.watermark_tx.send(vec![part]).await.unwrap();
        }

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

        // Verify watermark progression
        let watermark = setup.store.get_watermark();
        assert_eq!(watermark.checkpoint_hi_inclusive, 3);

        // Clean up
        setup.cancel.cancel();
        let _ = setup.commit_watermark_handle.await;
    }

    #[tokio::test]
    async fn test_out_of_order_watermarks() {
        let config = CommitterConfig::default();
        let initial_watermark = Some(CommitterWatermark {
            checkpoint_hi_inclusive: 0,
            ..Default::default()
        });
        let setup = setup_test::<DataPipeline>(config, initial_watermark, MockStore::default());

        // Send watermark parts out of order
        let parts = vec![
            create_watermark_part_for_checkpoint(4),
            create_watermark_part_for_checkpoint(2),
            create_watermark_part_for_checkpoint(1),
        ];
        setup.watermark_tx.send(parts).await.unwrap();

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

        // Verify watermark hasn't progressed past 2
        let watermark = setup.store.get_watermark();
        assert_eq!(watermark.checkpoint_hi_inclusive, 2);

        // Send checkpoint 3 to fill the gap
        setup
            .watermark_tx
            .send(vec![create_watermark_part_for_checkpoint(3)])
            .await
            .unwrap();

        // Wait for the next polling and processing
        tokio::time::sleep(tokio::time::Duration::from_secs(1)).await;

        // Verify watermark has progressed to 4
        let watermark = setup.store.get_watermark();
        assert_eq!(watermark.checkpoint_hi_inclusive, 4);

        // Clean up
        setup.cancel.cancel();
        let _ = setup.commit_watermark_handle.await;
    }

    #[tokio::test]
    async fn test_watermark_with_connection_failure() {
        let config = CommitterConfig {
            watermark_interval_ms: 1_000, // Long polling interval to test connection retry
            ..Default::default()
        };
        let initial_watermark = Some(CommitterWatermark {
            checkpoint_hi_inclusive: 0,
            ..Default::default()
        });
        let store = MockStore::default().with_connection_failures(1);
        let setup = setup_test::<DataPipeline>(config, initial_watermark, store);

        // Send watermark part
        let part = create_watermark_part_for_checkpoint(1);
        setup.watermark_tx.send(vec![part]).await.unwrap();

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

        // Verify watermark hasn't progressed
        let watermark = setup.store.get_watermark();
        assert_eq!(watermark.checkpoint_hi_inclusive, 0);

        // Wait for next polling and processing
        tokio::time::sleep(tokio::time::Duration::from_millis(1_200)).await;

        // Verify watermark has progressed
        let watermark = setup.store.get_watermark();
        assert_eq!(watermark.checkpoint_hi_inclusive, 1);

        // Clean up
        setup.cancel.cancel();
        let _ = setup.commit_watermark_handle.await;
    }

    #[tokio::test]
    async fn test_incomplete_watermark() {
        let config = CommitterConfig {
            watermark_interval_ms: 1_000, // Long polling interval to test adding complete part
            ..Default::default()
        };
        let initial_watermark = Some(CommitterWatermark {
            checkpoint_hi_inclusive: 0,
            ..Default::default()
        });
        let setup = setup_test::<DataPipeline>(config, initial_watermark, MockStore::default());

        // Send the first incomplete watermark part
        let part = WatermarkPart {
            watermark: CommitterWatermark {
                checkpoint_hi_inclusive: 1,
                ..Default::default()
            },
            batch_rows: 1,
            total_rows: 3,
        };
        setup.watermark_tx.send(vec![part.clone()]).await.unwrap();

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

        // Verify watermark hasn't progressed
        let watermark = setup.store.get_watermark();
        assert_eq!(watermark.checkpoint_hi_inclusive, 0);

        // Send the other two parts to complete the watermark
        setup
            .watermark_tx
            .send(vec![part.clone(), part.clone()])
            .await
            .unwrap();

        // Wait for next polling and processing
        tokio::time::sleep(tokio::time::Duration::from_millis(1_200)).await;

        // Verify watermark has progressed
        let watermark = setup.store.get_watermark();
        assert_eq!(watermark.checkpoint_hi_inclusive, 1);

        // Clean up
        setup.cancel.cancel();
        let _ = setup.commit_watermark_handle.await;
    }

    #[tokio::test]
    async fn test_no_initial_watermark() {
        let config = CommitterConfig::default();
        let initial_watermark = None;
        let setup = setup_test::<DataPipeline>(config, initial_watermark, MockStore::default());

        // Send the checkpoint 1 watermark
        setup
            .watermark_tx
            .send(vec![create_watermark_part_for_checkpoint(1)])
            .await
            .unwrap();

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

        // Verify watermark hasn't progressed
        let watermark = setup.store.get_watermark();
        assert_eq!(watermark.checkpoint_hi_inclusive, 0);

        // Send the checkpoint 0 watermark to fill the gap.
        setup
            .watermark_tx
            .send(vec![create_watermark_part_for_checkpoint(0)])
            .await
            .unwrap();

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

        // Verify watermark has progressed
        let watermark = setup.store.get_watermark();
        assert_eq!(watermark.checkpoint_hi_inclusive, 1);

        // Clean up
        setup.cancel.cancel();
        let _ = setup.commit_watermark_handle.await;
    }
}