risingwave_stream/executor/dedup/
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
// 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 std::hash::Hash;

use risingwave_common_estimate_size::EstimateSize;

use crate::cache::ManagedLruCache;
use crate::common::metrics::MetricsInfo;
use crate::executor::prelude::*;

/// [`DedupCache`] is used for key deduplication. Currently, the cache behaves like a set that only
/// accepts a key without a value. This could be refined in the future to support k-v pairs.
pub struct DedupCache<K: Hash + Eq + EstimateSize> {
    inner: ManagedLruCache<K, ()>,
}

impl<K: Hash + Eq + EstimateSize> DedupCache<K> {
    pub fn new(watermark_sequence: AtomicU64Ref, metrics_info: MetricsInfo) -> Self {
        let cache = ManagedLruCache::unbounded(watermark_sequence, metrics_info);
        Self { inner: cache }
    }

    /// Insert a `key` into the cache only if the `key` doesn't exist in the cache before. Return
    /// whether the `key` is successfully inserted.
    pub fn dedup_insert(&mut self, key: K) -> bool {
        self.inner.put(key, ()).is_none()
    }

    /// Insert a `key` into the cache without checking for duplication.
    pub fn insert(&mut self, key: K) {
        self.inner.push(key, ());
    }

    /// Check whether the given key is in the cache.
    pub fn contains(&self, key: &K) -> bool {
        self.inner.contains(key)
    }

    /// Evict the inner LRU cache according to the watermark epoch.
    pub fn evict(&mut self) {
        self.inner.evict()
    }

    /// Clear everything in the cache.
    pub fn clear(&mut self) {
        self.inner.clear()
    }
}

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

    use super::DedupCache;
    use crate::common::metrics::MetricsInfo;

    #[test]
    fn test_dedup_cache() {
        let mut cache = DedupCache::new(Arc::new(AtomicU64::new(10000)), MetricsInfo::for_test());

        cache.insert(10);
        assert!(cache.contains(&10));
        assert!(!cache.dedup_insert(10));

        assert!(cache.dedup_insert(20));
        assert!(cache.contains(&20));
        assert!(!cache.dedup_insert(20));

        cache.clear();
        assert!(!cache.contains(&10));
        assert!(!cache.contains(&20));
    }
}