risingwave_frontend/monitor/
stats.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
// 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 core::mem;
use std::collections::HashMap;
use std::sync::{Arc, LazyLock};
use std::time::Duration;

use prometheus::core::{AtomicI64, AtomicU64, GenericCounter, GenericGauge};
use prometheus::{
    exponential_buckets, histogram_opts, register_histogram_vec_with_registry,
    register_histogram_with_registry, register_int_counter_with_registry,
    register_int_gauge_with_registry, Histogram, HistogramVec, IntGauge, Registry,
};
use risingwave_common::metrics::TrAdderGauge;
use risingwave_common::monitor::GLOBAL_METRICS_REGISTRY;
use tokio::task::JoinHandle;

use crate::session::SessionMapRef;

#[derive(Clone)]
pub struct FrontendMetrics {
    pub query_counter_local_execution: GenericCounter<AtomicU64>,
    pub latency_local_execution: Histogram,
    pub active_sessions: IntGauge,
    pub batch_total_mem: TrAdderGauge,
}

pub static GLOBAL_FRONTEND_METRICS: LazyLock<FrontendMetrics> =
    LazyLock::new(|| FrontendMetrics::new(&GLOBAL_METRICS_REGISTRY));

impl FrontendMetrics {
    fn new(registry: &Registry) -> Self {
        let query_counter_local_execution = register_int_counter_with_registry!(
            "frontend_query_counter_local_execution",
            "Total query number of local execution mode",
            registry
        )
        .unwrap();

        let opts = histogram_opts!(
            "frontend_latency_local_execution",
            "latency of local execution mode",
            exponential_buckets(0.01, 2.0, 23).unwrap()
        );
        let latency_local_execution = register_histogram_with_registry!(opts, registry).unwrap();

        let active_sessions = register_int_gauge_with_registry!(
            "frontend_active_sessions",
            "Total number of active sessions in frontend",
            registry
        )
        .unwrap();

        let batch_total_mem = TrAdderGauge::new(
            "frontend_batch_total_mem",
            "All memory usage of batch executors in bytes",
        )
        .unwrap();

        registry
            .register(Box::new(batch_total_mem.clone()))
            .unwrap();

        Self {
            query_counter_local_execution,
            latency_local_execution,
            active_sessions,
            batch_total_mem,
        }
    }

    /// Create a new `FrontendMetrics` instance used in tests or other places.
    pub fn for_test() -> Self {
        GLOBAL_FRONTEND_METRICS.clone()
    }
}

pub static GLOBAL_CURSOR_METRICS: LazyLock<CursorMetrics> =
    LazyLock::new(|| CursorMetrics::new(&GLOBAL_METRICS_REGISTRY));

#[derive(Clone)]
pub struct CursorMetrics {
    pub subscription_cursor_error_count: GenericCounter<AtomicU64>,
    pub subscription_cursor_query_duration: HistogramVec,
    pub subscription_cursor_declare_duration: HistogramVec,
    pub subscription_cursor_fetch_duration: HistogramVec,
    subsription_cursor_nums: GenericGauge<AtomicI64>,
    invalid_subsription_cursor_nums: GenericGauge<AtomicI64>,
    subscription_cursor_last_fetch_duration: HistogramVec,
    _cursor_metrics_collector: Option<Arc<CursorMetricsCollector>>,
}

impl CursorMetrics {
    pub fn new(registry: &Registry) -> Self {
        let subscription_cursor_error_count = register_int_counter_with_registry!(
            "subscription_cursor_error_count",
            "The subscription error num of cursor",
            registry
        )
        .unwrap();
        let opts = histogram_opts!(
            "subscription_cursor_query_duration",
            "The amount of time a query exists inside the cursor",
            exponential_buckets(1.0, 5.0, 11).unwrap(),
        );
        let subscription_cursor_query_duration =
            register_histogram_vec_with_registry!(opts, &["subscription_name"], registry).unwrap();

        let opts = histogram_opts!(
            "subscription_cursor_declare_duration",
            "Subscription cursor duration of declare",
            exponential_buckets(1.0, 5.0, 11).unwrap(),
        );
        let subscription_cursor_declare_duration =
            register_histogram_vec_with_registry!(opts, &["subscription_name"], registry).unwrap();

        let opts = histogram_opts!(
            "subscription_cursor_fetch_duration",
            "Subscription cursor duration of fetch",
            exponential_buckets(1.0, 5.0, 11).unwrap(),
        );
        let subscription_cursor_fetch_duration =
            register_histogram_vec_with_registry!(opts, &["subscription_name"], registry).unwrap();

        let subsription_cursor_nums = register_int_gauge_with_registry!(
            "subsription_cursor_nums",
            "The number of subscription cursor",
            registry
        )
        .unwrap();
        let invalid_subsription_cursor_nums = register_int_gauge_with_registry!(
            "invalid_subsription_cursor_nums",
            "The number of invalid subscription cursor",
            registry
        )
        .unwrap();

        let opts = histogram_opts!(
            "subscription_cursor_last_fetch_duration",
            "Since the last fetch, the time up to now",
            exponential_buckets(1.0, 5.0, 11).unwrap(),
        );
        let subscription_cursor_last_fetch_duration =
            register_histogram_vec_with_registry!(opts, &["subscription_name"], registry).unwrap();
        Self {
            _cursor_metrics_collector: None,
            subscription_cursor_error_count,
            subscription_cursor_query_duration,
            subscription_cursor_declare_duration,
            subscription_cursor_fetch_duration,
            subsription_cursor_nums,
            invalid_subsription_cursor_nums,
            subscription_cursor_last_fetch_duration,
        }
    }

    pub fn for_test() -> Self {
        GLOBAL_CURSOR_METRICS.clone()
    }

    pub fn start_with_session_map(&mut self, session_map: SessionMapRef) {
        self._cursor_metrics_collector = Some(Arc::new(CursorMetricsCollector::new(
            session_map,
            self.subsription_cursor_nums.clone(),
            self.invalid_subsription_cursor_nums.clone(),
            self.subscription_cursor_last_fetch_duration.clone(),
        )));
    }

    pub fn init(session_map: SessionMapRef) -> Self {
        let mut cursor_metrics = GLOBAL_CURSOR_METRICS.clone();
        cursor_metrics.start_with_session_map(session_map);
        cursor_metrics
    }
}

pub struct PeriodicCursorMetrics {
    pub subsription_cursor_nums: i64,
    pub invalid_subsription_cursor_nums: i64,
    pub subscription_cursor_last_fetch_duration: HashMap<String, f64>,
}

struct CursorMetricsCollector {
    _join_handle: JoinHandle<()>,
    shutdown_tx: Option<tokio::sync::oneshot::Sender<()>>,
}
impl CursorMetricsCollector {
    fn new(
        session_map: SessionMapRef,
        subsription_cursor_nums: GenericGauge<AtomicI64>,
        invalid_subsription_cursor_nums: GenericGauge<AtomicI64>,
        subscription_cursor_last_fetch_duration: HistogramVec,
    ) -> Self {
        const COLLECT_INTERVAL_SECONDS: u64 = 60;

        let (shutdown_tx, mut shutdown_rx) = tokio::sync::oneshot::channel();
        let join_handle = tokio::spawn(async move {
            let mut monitor_interval =
                tokio::time::interval(Duration::from_secs(COLLECT_INTERVAL_SECONDS));
            monitor_interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay);
            loop {
                tokio::select! {
                    // Wait for interval
                    _ = monitor_interval.tick() => {},
                    // Shutdown monitor
                    _ = &mut shutdown_rx => {
                        tracing::info!("Fragment info monitor is stopped");
                        return;
                    }
                }

                let session_vec = { session_map.read().values().cloned().collect::<Vec<_>>() };
                let mut subsription_cursor_nums_value = 0;
                let mut invalid_subsription_cursor_nums_value = 0;
                for session in &session_vec {
                    let periodic_cursor_metrics = session
                        .get_cursor_manager()
                        .get_periodic_cursor_metrics()
                        .await;
                    subsription_cursor_nums_value +=
                        periodic_cursor_metrics.subsription_cursor_nums;
                    invalid_subsription_cursor_nums_value +=
                        periodic_cursor_metrics.invalid_subsription_cursor_nums;
                    for (subscription_name, duration) in
                        &periodic_cursor_metrics.subscription_cursor_last_fetch_duration
                    {
                        subscription_cursor_last_fetch_duration
                            .with_label_values(&[subscription_name])
                            .observe(*duration);
                    }
                }
                subsription_cursor_nums.set(subsription_cursor_nums_value);
                invalid_subsription_cursor_nums.set(invalid_subsription_cursor_nums_value);
            }
        });
        Self {
            _join_handle: join_handle,
            shutdown_tx: Some(shutdown_tx),
        }
    }
}
impl Drop for CursorMetricsCollector {
    fn drop(&mut self) {
        if let Some(shutdown_tx) = mem::take(&mut self.shutdown_tx) {
            shutdown_tx.send(()).ok();
        }
    }
}