risingwave_compute/memory/
manager.rs

1// Copyright 2025 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use std::sync::atomic::{AtomicU64, Ordering};
16use std::sync::{Arc, Mutex};
17use std::time::Duration;
18
19use risingwave_common::sequence::AtomicSequence;
20use risingwave_stream::executor::monitor::StreamingMetrics;
21
22use super::controller::LruWatermarkController;
23
24pub struct MemoryManagerConfig {
25    /// [`MemoryManager`] will try to control the jemalloc-reported memory usage
26    /// to be lower than this
27    pub target_memory: usize,
28
29    pub threshold_aggressive: f64,
30    pub threshold_graceful: f64,
31    pub threshold_stable: f64,
32
33    pub eviction_factor_stable: f64,
34    pub eviction_factor_graceful: f64,
35    pub eviction_factor_aggressive: f64,
36
37    pub metrics: Arc<StreamingMetrics>,
38}
39
40/// Compute node uses [`MemoryManager`] to limit the memory usage.
41pub struct MemoryManager {
42    /// All cached data before the watermark should be evicted.
43    watermark_sequence: Arc<AtomicSequence>,
44
45    metrics: Arc<StreamingMetrics>,
46
47    controller: Mutex<LruWatermarkController>,
48}
49
50impl MemoryManager {
51    // Arbitrarily set a minimal barrier interval in case it is too small,
52    // especially when it's 0.
53    const MIN_INTERVAL: Duration = Duration::from_millis(10);
54
55    pub fn new(config: MemoryManagerConfig) -> Arc<Self> {
56        let controller = Mutex::new(LruWatermarkController::new(&config));
57        tracing::info!("LRU watermark controller: {:?}", &controller);
58
59        Arc::new(Self {
60            watermark_sequence: Arc::new(0.into()),
61            metrics: config.metrics,
62            controller,
63        })
64    }
65
66    pub fn get_watermark_sequence(&self) -> Arc<AtomicU64> {
67        self.watermark_sequence.clone()
68    }
69
70    pub async fn run(self: Arc<Self>, interval: Duration) {
71        // Loop interval of running control policy
72        let interval = std::cmp::max(interval, Self::MIN_INTERVAL);
73        tracing::info!("start running MemoryManager with interval {interval:?}",);
74
75        // Keep same interval with the barrier interval
76        let mut tick_interval = tokio::time::interval(interval);
77
78        loop {
79            tick_interval.tick().await;
80
81            let new_watermark_sequence = self.controller.lock().unwrap().tick();
82
83            self.watermark_sequence
84                .store(new_watermark_sequence, Ordering::Relaxed);
85
86            self.metrics.lru_runtime_loop_count.inc();
87        }
88    }
89}