risingwave_stream/common/table/
state_table_cache.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
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use risingwave_common::array::Op;
use risingwave_common::row::{OwnedRow, Row, RowExt};
use risingwave_common::types::{DefaultOrdered, ScalarRefImpl};
use risingwave_common_estimate_size::EstimateSize;

use crate::common::state_cache::{StateCache, TopNStateCache};

/// The watermark cache key is just an `OwnedRow` wrapped in `DefaultOrdered`.
/// This is because we want to use the `DefaultOrdered` implementation of `Ord`.
/// The assumption is that the watermark column is the first column in the row.
/// So it should automatically be ordered by the watermark column.
/// We disregard the ordering of the remaining PK datums.
///
/// TODO(kwannoel):
/// We can also store bytes from encoded pk.
/// The benefit is we deserialization of PK when refiling the cache.
type WatermarkCacheKey = DefaultOrdered<OwnedRow>;

/// Cache Invariants
/// -----------------
/// - If cache is synced, it will ALWAYS contain TopN entries. NOTE: TopN here refers to the lowest
///   N entries, where the first is the minimum and so on.
/// - Cache will not contain NULL values in the watermark column, they are ignored in watermark
///   state cleaning. They will not be included in the cache's table row count either, since they
///   are treated as invisible.
///
/// Updates to Cache
/// -----------------
/// INSERT
///    A. Cache evicted. Update cached value.
///    B. Cache uninitialized. Initialize cache, insert into `TopNCache`.
///    C. Cache not empty. Insert into `TopNCache`.
///
/// DELETE
///    A. Matches lowest value pk. Remove lowest value. Mark cache as Evicted.
///       Later on Barrier we will refresh the cache with table scan.
///       Since on barrier we will clean up all values before watermark,
///       We have less rows to scan.
///    B. Does not match. Do nothing.
///
/// UPDATE
///    Nothing. Watermark is part of pk. Pk won't change.
///
/// BARRIER
///    State table commit. See below.
///
/// STATE TABLE COMMIT
///    A. Decide whether to do state cleaning:
///        if `watermark_to_be_cleaned` < smallest val
///           OR no value in cache + cache is synced:
///        No need issue delete range.
///        if `watermark_to_be_cleaned` => smallest val OR cache not synced:
///        Issue delete ranges.
///
///    B. Refreshing the cache:
///        On barrier, do table scan from `most_recently_cleaned_watermark` (inclusive) to +inf.
///        Take the Top N rows and insert into cache.
///        This has to be implemented in `state_table`.
///        We do not need to store any values, just the keys.
///
/// TODO(kwannoel):
/// Optimization: If cache is not full,
/// we can also do point delete for each cache entry.
/// Not sure if this is more optimal,
/// we have to measure this scenario to see if it is indeed better.
#[derive(EstimateSize, Clone)]
pub struct StateTableWatermarkCache {
    inner: TopNStateCache<WatermarkCacheKey, ()>,
}

impl StateTableWatermarkCache {
    pub fn new(size: usize) -> Self {
        Self {
            inner: TopNStateCache::new(size),
        }
    }

    /// NOTE(kwannoel): Unused. Requires row count from State Table.
    /// On first initialization, we can use `row_count` 0.
    /// But if state table is reconstructed after recovery, we need to obtain row count meta-data.
    #[allow(dead_code)]
    fn new_with_row_count(size: usize, row_count: usize) -> Self {
        Self {
            inner: TopNStateCache::with_table_row_count(size, row_count),
        }
    }

    /// Get the lowest key.
    fn first_key(&self) -> Option<&WatermarkCacheKey> {
        self.inner.first_key_value().map(|(k, _)| k)
    }

    // Get the watermark value from the top key.
    pub fn lowest_key(&self) -> Option<ScalarRefImpl<'_>> {
        self.first_key().and_then(|k| k.0.datum_at(0))
    }

    /// Insert a new value.
    pub fn insert(&mut self, key: &impl Row) {
        if !key.is_null_at(0) {
            self.inner.insert(DefaultOrdered(key.into_owned_row()), ());
        }
    }

    /// Delete a value
    /// If the watermark col is NULL, it will just be ignored.
    pub fn delete(&mut self, key: &impl Row) {
        if !key.is_null_at(0) {
            self.inner.delete(&DefaultOrdered(key.into_owned_row()));
        }
    }

    pub fn capacity(&self) -> usize {
        self.inner.capacity()
    }

    pub fn len(&self) -> usize {
        self.inner.len()
    }

    pub fn set_table_row_count(&mut self, table_row_count: usize) {
        self.inner.set_table_row_count(table_row_count)
    }

    #[cfg(test)]
    pub fn get_table_row_count(&self) -> &Option<usize> {
        self.inner.get_table_row_count()
    }
}

impl StateCache for StateTableWatermarkCache {
    type Filler<'a> = &'a mut TopNStateCache<WatermarkCacheKey, ()>;
    type Key = WatermarkCacheKey;
    type Value = ();

    fn is_synced(&self) -> bool {
        self.inner.is_synced()
    }

    fn begin_syncing(&mut self) -> Self::Filler<'_> {
        self.inner.begin_syncing()
    }

    fn insert(&mut self, key: Self::Key, value: Self::Value) -> Option<Self::Value> {
        self.inner.insert(key, value)
    }

    fn delete(&mut self, key: &Self::Key) -> Option<Self::Value> {
        self.inner.delete(key)
    }

    fn apply_batch(&mut self, batch: impl IntoIterator<Item = (Op, Self::Key, Self::Value)>) {
        self.inner.apply_batch(batch)
    }

    fn clear(&mut self) {
        self.inner.clear()
    }

    fn values(&self) -> impl Iterator<Item = &Self::Value> {
        self.inner.values()
    }

    fn first_key_value(&self) -> Option<(&Self::Key, &Self::Value)> {
        self.inner.first_key_value()
    }
}

#[cfg(test)]
mod tests {

    use risingwave_common::types::{Scalar, Timestamptz};

    use super::*;
    use crate::common::state_cache::StateCacheFiller;

    /// With capacity 3, test the following sequence of inserts:
    /// Insert
    /// [SYNC] an empty table first.
    /// [1000] should insert, cache is empty.
    /// [999] should insert, smaller than 1000, should be lowest value.
    /// [2000] should NOT insert.
    /// It is larger than 1000, and we don't know the state table size,
    /// for instance if state table just recovered, and does not have row count meta data.
    /// This means there could be a row like [1001] in the state table.
    /// [900, ...], should insert.
    /// [800], should insert, smaller than 900, should be lowest value.
    #[test]
    fn test_state_table_watermark_cache_inserts() {
        let v1 = [
            Some(Timestamptz::from_secs(1000).unwrap().to_scalar_value()),
            Some(Timestamptz::from_secs(1234).unwrap().to_scalar_value()),
        ];
        let v2 = [
            Some(Timestamptz::from_secs(999).unwrap().to_scalar_value()),
            Some(Timestamptz::from_secs(1234).unwrap().to_scalar_value()),
        ];
        let v3 = [
            Some(Timestamptz::from_secs(2000).unwrap().to_scalar_value()),
            Some(Timestamptz::from_secs(1234).unwrap().to_scalar_value()),
        ];
        let v4 = [
            Some(Timestamptz::from_secs(900).unwrap().to_scalar_value()),
            Some(Timestamptz::from_secs(1234).unwrap().to_scalar_value()),
        ];
        let v5 = [
            Some(Timestamptz::from_secs(800).unwrap().to_scalar_value()),
            Some(Timestamptz::from_secs(1234).unwrap().to_scalar_value()),
        ];
        let mut cache = StateTableWatermarkCache::new(3);
        assert_eq!(cache.capacity(), 3);
        let filler = cache.begin_syncing();
        filler.finish();

        // Test 1000
        cache.insert(&v1);
        assert_eq!(cache.len(), 1);
        assert_eq!(
            cache.lowest_key(),
            Some(v1[0].as_ref().unwrap().as_scalar_ref_impl())
        );

        // Test 999
        cache.insert(&v2);
        assert_eq!(cache.len(), 2);
        assert_eq!(
            cache.lowest_key(),
            Some(v2[0].as_ref().unwrap().as_scalar_ref_impl())
        );

        // Test 2000
        cache.insert(&v3);
        assert_eq!(cache.len(), 2);
        assert_eq!(
            cache.lowest_key(),
            Some(v2[0].as_ref().unwrap().as_scalar_ref_impl())
        );

        // Test 900
        cache.insert(&v4);
        assert_eq!(cache.len(), 3);
        assert_eq!(
            cache.lowest_key(),
            Some(v4[0].as_ref().unwrap().as_scalar_ref_impl())
        );

        // Test 800
        cache.insert(&v5);
        assert_eq!(cache.len(), 3);
        assert_eq!(
            cache.lowest_key(),
            Some(v5[0].as_ref().unwrap().as_scalar_ref_impl())
        );
    }

    #[test]
    fn test_state_table_watermark_cache_delete_non_existent_value() {
        let mut cache = StateTableWatermarkCache::new(3);
        assert_eq!(cache.capacity(), 3);
        let filler = cache.begin_syncing();
        filler.finish();
        let v1 = [
            Some(Timestamptz::from_secs(1000).unwrap().to_scalar_value()),
            Some(Timestamptz::from_secs(1234).unwrap().to_scalar_value()),
        ];
        cache.delete(&v1);
    }

    /// With capacity 3, test the following sequence of inserts:
    /// Insert
    /// [1000, ...], should insert, cache is empty.
    /// [999, ...], should insert, smaller than 1000, should be lowest value.
    /// [2000, ...], should insert, although larger than largest val (1000), cache rows still match
    /// state table rows. [3000, ...], should be ignored
    /// [900, ...], should evict 2000
    #[test]
    fn test_state_table_watermark_cache_with_row_count_inserts() {
        let mut cache = StateTableWatermarkCache::new_with_row_count(3, 0);
        assert_eq!(cache.capacity(), 3);
        let filler = cache.begin_syncing();
        filler.finish();
        assert!(cache.first_key_value().is_none());
        assert!(cache.lowest_key().is_none());

        let v1 = [
            Some(Timestamptz::from_secs(1000).unwrap().to_scalar_value()),
            Some(Timestamptz::from_secs(1000).unwrap().to_scalar_value()),
        ];
        cache.insert(&v1);
        let lowest = cache.lowest_key().unwrap();
        assert_eq!(lowest, v1[0].clone().unwrap().as_scalar_ref_impl());

        let v2 = [
            Some(Timestamptz::from_secs(999).unwrap().to_scalar_value()),
            Some(Timestamptz::from_secs(1234).unwrap().to_scalar_value()),
        ];
        cache.insert(&v2);
        assert_eq!(cache.len(), 2);
        let lowest = cache.lowest_key().unwrap();
        assert_eq!(lowest, v2[0].clone().unwrap().as_scalar_ref_impl());

        let v3 = [
            Some(Timestamptz::from_secs(2000).unwrap().to_scalar_value()),
            Some(Timestamptz::from_secs(1234).unwrap().to_scalar_value()),
        ];
        cache.insert(&v3);
        assert_eq!(cache.len(), 3);

        let v4 = [
            Some(Timestamptz::from_secs(3000).unwrap().to_scalar_value()),
            Some(Timestamptz::from_secs(1234).unwrap().to_scalar_value()),
        ];
        cache.insert(&v4);
        assert_eq!(cache.len(), 3);

        let v5 = [
            Some(Timestamptz::from_secs(900).unwrap().to_scalar_value()),
            Some(Timestamptz::from_secs(1234).unwrap().to_scalar_value()),
        ];
        cache.insert(&v5);
        assert_eq!(cache.len(), 3);
        let lowest = cache.lowest_key().unwrap();
        assert_eq!(lowest, v5[0].clone().unwrap().as_scalar_ref_impl());
    }

    /// With capacity 3, seed the following sequence of inserts:
    /// Insert:
    /// [1000, ...]
    /// [999, ...]
    /// [2000, ...]
    /// [3000, ...]
    /// [900, ...]
    ///
    /// In the cache there should be:
    /// [900, 999, 1000]
    /// Then run one DELETE.
    /// [999].
    ///
    /// The cache should be:
    /// [900, 1000].
    /// Lowest val: 900.
    ///
    /// Then run one INSERT.
    /// [1001].
    /// This should be ignored. It is larger than the largest val in the cache (1000).
    /// And cache no longer matches state table rows.
    ///
    /// Then run another INSERT.
    /// [950].
    /// This should be accepted. It is smaller than the largest val in the cache (1000).
    ///
    /// Then run DELETE.
    /// [900].
    /// Lowest val: 1000.
    ///
    /// Then run DELETE.
    /// [1000].
    /// Lowest val: None.
    ///
    /// Then run INSERT.
    /// Cache should be out of sync, should reject the insert.
    /// Cache len = 0.
    #[test]
    fn test_state_table_watermark_cache_with_row_count_deletes() {
        // Initial INSERT
        let mut cache = StateTableWatermarkCache::new_with_row_count(3, 0);
        assert_eq!(cache.capacity(), 3);
        let filler = cache.begin_syncing();
        filler.finish();
        let v1 = [
            Some(Timestamptz::from_secs(1000).unwrap().to_scalar_value()),
            Some(Timestamptz::from_secs(1234).unwrap().to_scalar_value()),
        ];
        let v2 = [
            Some(Timestamptz::from_secs(999).unwrap().to_scalar_value()),
            Some(Timestamptz::from_secs(1234).unwrap().to_scalar_value()),
        ];
        let v3 = [
            Some(Timestamptz::from_secs(2000).unwrap().to_scalar_value()),
            Some(Timestamptz::from_secs(1234).unwrap().to_scalar_value()),
        ];
        let v4 = [
            Some(Timestamptz::from_secs(3000).unwrap().to_scalar_value()),
            Some(Timestamptz::from_secs(1234).unwrap().to_scalar_value()),
        ];
        let v5 = [
            Some(Timestamptz::from_secs(900).unwrap().to_scalar_value()),
            Some(Timestamptz::from_secs(1234).unwrap().to_scalar_value()),
        ];
        cache.insert(&v1);
        cache.insert(&v2);
        cache.insert(&v3);
        cache.insert(&v4);
        cache.insert(&v5);

        // First Delete
        cache.delete(&v2);
        assert_eq!(cache.len(), 2);
        let lowest = cache.lowest_key().unwrap();
        assert_eq!(lowest, v5[0].clone().unwrap().as_scalar_ref_impl());

        // Insert 1001
        let v6 = [
            Some(Timestamptz::from_secs(1001).unwrap().to_scalar_value()),
            Some(Timestamptz::from_secs(1234).unwrap().to_scalar_value()),
        ];
        cache.insert(&v6);
        assert_eq!(cache.len(), 2);

        // Insert 950
        let v7 = [
            Some(Timestamptz::from_secs(950).unwrap().to_scalar_value()),
            Some(Timestamptz::from_secs(1234).unwrap().to_scalar_value()),
        ];
        cache.insert(&v7);
        assert_eq!(cache.len(), 3);

        // Delete 950
        cache.delete(&v7);
        assert_eq!(cache.len(), 2);

        // DELETE
        cache.delete(&v5);
        assert_eq!(cache.len(), 1);
        assert_eq!(
            cache.lowest_key().unwrap(),
            v1[0].clone().unwrap().as_scalar_ref_impl()
        );

        // DELETE
        cache.delete(&v1);
        assert_eq!(cache.len(), 0);
        assert!(!cache.is_synced());

        // INSERT after Out of sync
        cache.insert(&v1);
        assert_eq!(cache.len(), 0);
    }

    #[test]
    fn test_watermark_cache_syncing() {
        let v1 = [
            Some(Timestamptz::from_secs(1000).unwrap().to_scalar_value()),
            Some(1000i64.into()),
        ];
        let v2 = [
            Some(Timestamptz::from_secs(3000).unwrap().to_scalar_value()),
            Some(1000i64.into()),
        ];
        let v3 = [
            Some(Timestamptz::from_secs(2000).unwrap().to_scalar_value()),
            Some(1000i64.into()),
        ];
        let mut cache = StateTableWatermarkCache::new(3);
        let mut filler = cache.begin_syncing();
        filler.insert_unchecked(DefaultOrdered(v1.to_owned_row()), ());
        filler.insert_unchecked(DefaultOrdered(v2.to_owned_row()), ());
        filler.insert_unchecked(DefaultOrdered(v3.to_owned_row()), ());
        filler.finish();
        assert_eq!(cache.len(), 3);
        assert_eq!(
            cache.lowest_key().unwrap(),
            v1[0].clone().unwrap().as_scalar_ref_impl()
        );
    }
}