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

use std::sync::Arc;

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

use crate::{
    metrics::IndexerMetrics,
    store::{Connection, Store},
};

use super::{Handler, PrunerConfig};

/// The reader watermark task is responsible for updating the `reader_lo` and `pruner_timestamp`
/// values for a pipeline's row in the watermark table, based on the pruner configuration, and the
/// committer's progress.
///
/// `reader_lo` is the lowest checkpoint that readers are allowed to read from with a guarantee of
/// data availability for this pipeline, and `pruner_timestamp` is the timestamp at which this task
/// last updated that watermark. The timestamp is always fetched from the database (not from the
/// indexer or the reader), to avoid issues with drift between clocks.
///
/// If there is no pruner configuration, this task will immediately exit. Otherwise, the task exits
/// when the provided cancellation token is triggered.
pub(super) fn reader_watermark<H: Handler + 'static>(
    config: Option<PrunerConfig>,
    store: H::Store,
    metrics: Arc<IndexerMetrics>,
    cancel: CancellationToken,
) -> JoinHandle<()> {
    tokio::spawn(async move {
        let Some(config) = config else {
            info!(pipeline = H::NAME, "Skipping reader watermark task");
            return;
        };

        let mut poll = interval(config.interval());

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

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

                    let current = match conn.reader_watermark(H::NAME).await {
                        Ok(Some(current)) => current,

                        Ok(None) => {
                            warn!(pipeline = H::NAME, "No watermark for pipeline, skipping");
                            continue;
                        }

                        Err(e) => {
                            warn!(pipeline = H::NAME, "Failed to get current watermark: {e}");
                            continue;
                        }
                    };

                    // Calculate the new reader watermark based on the current high watermark.
                    let new_reader_lo = (current.checkpoint_hi_inclusive as u64 + 1)
                        .saturating_sub(config.retention);

                    if new_reader_lo <= current.reader_lo as u64 {
                        debug!(
                            pipeline = H::NAME,
                            current = current.reader_lo,
                            new = new_reader_lo,
                            "No change to reader watermark",
                        );
                        continue;
                    }

                    metrics
                        .watermark_reader_lo
                        .with_label_values(&[H::NAME])
                        .set(new_reader_lo as i64);

                    let Ok(updated) = conn.set_reader_watermark(H::NAME, new_reader_lo).await else {
                        warn!(pipeline = H::NAME, "Failed to update reader watermark");
                        continue;
                    };

                    if updated {
                        info!(pipeline = H::NAME, new_reader_lo, "Watermark");

                        metrics
                            .watermark_reader_lo_in_db
                            .with_label_values(&[H::NAME])
                            .set(new_reader_lo as i64);
                    }
                }
            }
        }

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

#[cfg(test)]
mod tests {
    use std::sync::{Arc, Mutex};
    use sui_pg_db::FieldCount;
    use sui_types::full_checkpoint_content::CheckpointData;
    use tokio::time::Duration;
    use tokio_util::sync::CancellationToken;

    use crate::{metrics::IndexerMetrics, pipeline::Processor, testing::mock_store::*};

    use super::*;

    // Fixed retention value used across all tests
    const TEST_RETENTION: u64 = 5;
    // Default timeout for test operations
    const TEST_TIMEOUT: Duration = Duration::from_secs(20);

    #[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::async_trait]
    impl Handler for DataPipeline {
        type Store = MockStore;

        async fn commit<'a>(
            _values: &[Self::Value],
            _conn: &mut MockConnection<'a>,
        ) -> anyhow::Result<usize> {
            Ok(0)
        }
    }

    struct TestSetup {
        store: MockStore,
        handle: JoinHandle<()>,
        cancel: CancellationToken,
    }

    async fn setup_test(
        watermark: MockWatermark,
        interval_ms: u64,
        connection_failure_attempts: usize,
        set_reader_watermark_failure_attempts: usize,
    ) -> TestSetup {
        let store = MockStore {
            watermarks: Arc::new(Mutex::new(watermark)),
            set_reader_watermark_failure_attempts: Arc::new(Mutex::new(
                set_reader_watermark_failure_attempts,
            )),
            connection_failure: Arc::new(Mutex::new(ConnectionFailure {
                connection_failure_attempts,
                ..Default::default()
            })),
            ..Default::default()
        };

        let config = PrunerConfig {
            interval_ms,
            delay_ms: 100,
            retention: TEST_RETENTION,
            max_chunk_size: 100,
            prune_concurrency: 1,
        };

        let metrics = IndexerMetrics::new(&Default::default());
        let cancel = CancellationToken::new();

        let store_clone = store.clone();
        let cancel_clone = cancel.clone();
        let handle =
            reader_watermark::<DataPipeline>(Some(config), store_clone, metrics, cancel_clone);

        TestSetup {
            store,
            handle,
            cancel,
        }
    }

    #[tokio::test]
    async fn test_reader_watermark_updates() {
        let watermark = MockWatermark {
            epoch_hi_inclusive: 0,
            checkpoint_hi_inclusive: 10, // Current high watermark
            tx_hi: 100,
            timestamp_ms_hi_inclusive: 1000,
            reader_lo: 0, // Initial reader_lo
            pruner_timestamp: 0,
            pruner_hi: 0,
        };
        let polling_interval_ms = 100;
        let connection_failure_attempts = 0;
        let set_reader_watermark_failure_attempts = 0;
        let setup = setup_test(
            watermark,
            polling_interval_ms,
            connection_failure_attempts,
            set_reader_watermark_failure_attempts,
        )
        .await;

        // Wait for a few intervals to allow the task to update the watermark
        tokio::time::sleep(Duration::from_millis(200)).await;

        // new reader_lo = checkpoint_hi_inclusive (10) - retention (5) + 1 = 6
        {
            let watermarks = setup.store.watermarks.lock().unwrap();
            assert_eq!(watermarks.reader_lo, 6);
        }

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

    #[tokio::test]
    async fn test_reader_watermark_does_not_update_smaller_reader_lo() {
        let watermark = MockWatermark {
            epoch_hi_inclusive: 0,
            checkpoint_hi_inclusive: 10, // Current high watermark
            tx_hi: 100,
            timestamp_ms_hi_inclusive: 1000,
            reader_lo: 7, // Initial reader_lo
            pruner_timestamp: 0,
            pruner_hi: 0,
        };
        let polling_interval_ms = 100;
        let connection_failure_attempts = 0;
        let set_reader_watermark_failure_attempts = 0;
        let setup = setup_test(
            watermark,
            polling_interval_ms,
            connection_failure_attempts,
            set_reader_watermark_failure_attempts,
        )
        .await;

        // Wait for a few intervals to allow the task to update the watermark
        tokio::time::sleep(Duration::from_millis(200)).await;

        // new reader_lo = checkpoint_hi_inclusive (10) - retention (5) + 1 = 6,
        // which is smaller than current reader_lo (7). Therefore, the reader_lo was not updated.
        {
            let watermarks = setup.store.watermarks.lock().unwrap();
            assert_eq!(
                watermarks.reader_lo, 7,
                "Reader watermark should not be updated when new value is smaller"
            );
        }

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

    #[tokio::test]
    async fn test_reader_watermark_retry_update_after_connection_failure() {
        let watermark = MockWatermark {
            epoch_hi_inclusive: 0,
            checkpoint_hi_inclusive: 10, // Current high watermark
            tx_hi: 100,
            timestamp_ms_hi_inclusive: 1000,
            reader_lo: 0, // Initial reader_lo
            pruner_timestamp: 0,
            pruner_hi: 0,
        };
        let polling_interval_ms = 1_000; // Long interval for testing retry
        let connection_failure_attempts = 1;
        let set_reader_watermark_failure_attempts = 0;
        let setup = setup_test(
            watermark,
            polling_interval_ms,
            connection_failure_attempts,
            set_reader_watermark_failure_attempts,
        )
        .await;

        // Wait for first connection attempt (which should fail)
        setup
            .store
            .wait_for_connection_attempts(1, TEST_TIMEOUT)
            .await;

        // Verify state before retry succeeds
        let watermark = setup.store.get_watermark();
        assert_eq!(
            watermark.reader_lo, 0,
            "Reader watermark should not be updated due to DB connection failure"
        );

        // Wait for second connection attempt (which should succeed)
        setup
            .store
            .wait_for_connection_attempts(2, TEST_TIMEOUT)
            .await;

        // Wait a bit more for the watermark update to complete
        tokio::time::sleep(Duration::from_millis(100)).await;

        // Verify state after retry succeeds
        let watermark = setup.store.get_watermark();
        assert_eq!(
            watermark.reader_lo, 6,
            "Reader watermark should be updated after retry succeeds"
        );

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

    #[tokio::test]
    async fn test_reader_watermark_retry_update_after_set_watermark_failure() {
        let watermark = MockWatermark {
            epoch_hi_inclusive: 0,
            checkpoint_hi_inclusive: 10, // Current high watermark
            tx_hi: 100,
            timestamp_ms_hi_inclusive: 1000,
            reader_lo: 0, // Initial reader_lo
            pruner_timestamp: 0,
            pruner_hi: 0,
        };
        let polling_interval_ms = 1_000; // Long interval for testing retry
        let connection_failure_attempts = 0;
        let set_reader_watermark_failure_attempts = 1;
        let setup = setup_test(
            watermark,
            polling_interval_ms,
            connection_failure_attempts,
            set_reader_watermark_failure_attempts,
        )
        .await;

        // Wait for first failed attempt
        tokio::time::sleep(Duration::from_millis(200)).await;

        // Verify state before retry succeeds
        {
            let watermarks = setup.store.watermarks.lock().unwrap();
            assert_eq!(
                watermarks.reader_lo, 0,
                "Reader watermark should not be updated due to set_reader_watermark failure"
            );
        }

        // Wait for next polling for second attempt
        tokio::time::sleep(Duration::from_millis(1200)).await;

        // Verify state after retry succeeds
        {
            let watermarks = setup.store.watermarks.lock().unwrap();
            assert_eq!(watermarks.reader_lo, 6);
        }

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