diff --git a/components/health_controller/src/lib.rs b/components/health_controller/src/lib.rs new file mode 100644 index 00000000000..75427cd8e7c --- /dev/null +++ b/components/health_controller/src/lib.rs @@ -0,0 +1,457 @@ +// Copyright 2024 TiKV Project Authors. Licensed under Apache-2.0. + +//! This module contains utilities to manage and retrieve the health status of +//! TiKV instance in a unified way. +//! +//! ## [`HealthController`] +//! +//! [`HealthController`] is the core of the module. It's a unified place where +//! the server's health status is managed and collected, including the [gRPC +//! `HealthService`](grpcio_health::HealthService). It provides interfaces to +//! retrieve the collected information, and actively setting whether +//! the gRPC `HealthService` should report a `Serving` or `NotServing` status. +//! +//! ## Reporters +//! +//! [`HealthController`] doesn't provide ways to update most of the states +//! directly. Instead, each module in TiKV tha need to report its health status +//! need to create a corresponding reporter. +//! +//! The reason why the reporters is split out from the `HealthController` is: +//! +//! * Reporters can have different designs to fit the special use patterns of +//! different modules. +//! * `HealthController` internally contains states that are shared in different +//! modules and threads. If some module need to store internal states to +//! calculate the health status, they can be put in the reporter instead of +//! the `HealthController`, which makes it possible to avoid unnecessary +//! synchronization like mutexes. +//! * To avoid the `HealthController` itself contains too many different APIs +//! that are specific to different modules, increasing the complexity and +//! possibility to misuse of `HealthController`. + +#![feature(div_duration)] + +pub mod reporters; +pub mod slow_score; +pub mod trend; +pub mod types; + +use std::{ + collections::HashSet, + ops::Deref, + sync::{ + atomic::{AtomicU64, AtomicUsize, Ordering}, + Arc, + }, +}; + +use grpcio_health::HealthService; +use kvproto::pdpb::SlowTrend as SlowTrendPb; +use parking_lot::{Mutex, RwLock}; +pub use types::{LatencyInspector, RaftstoreDuration}; + +struct ServingStatus { + is_serving: bool, + unhealthy_modules: HashSet<&'static str>, +} + +impl ServingStatus { + fn to_serving_status_pb(&self) -> grpcio_health::ServingStatus { + match (self.is_serving, self.unhealthy_modules.is_empty()) { + (true, true) => grpcio_health::ServingStatus::Serving, + (true, false) => grpcio_health::ServingStatus::ServiceUnknown, + (false, _) => grpcio_health::ServingStatus::NotServing, + } + } +} + +struct HealthControllerInner { + // Internally stores a `f64` type. + raftstore_slow_score: AtomicU64, + raftstore_slow_trend: RollingRetriever, + + /// gRPC's builtin `HealthService`. + /// + /// **Note**: DO NOT update its state directly. Only change its state while + /// holding the mutex of `current_serving_status`, and keep consistent + /// with value of `current_serving_status`, unless `health_service` is + /// already shutdown. + /// + /// TiKV uses gRPC's builtin `HealthService` to provide information about + /// whether the TiKV server is normally running. To keep its behavior + /// consistent with earlier versions without the `HealthController`, + /// it's used in such pattern: + /// + /// * Only an empty service name is used, representing the status of the + /// whole server. + /// * When `current_serving_status.is_serving` is set to false (by calling + /// [`set_is_serving(false)`](HealthController::set_is_serving)), the + /// serving status is set to `NotServing`. + /// * If `current_serving_status.is_serving` is true, but + /// `current_serving_status.unhealthy_modules` is not empty, the serving + /// status is set to `ServiceUnknown`. + /// * Otherwise, the TiKV instance is regarded operational and the serving + /// status is set to `Serving`. + health_service: HealthService, + current_serving_status: Mutex, +} + +impl HealthControllerInner { + fn new() -> Self { + let health_service = HealthService::default(); + health_service.set_serving_status("", grpcio_health::ServingStatus::NotServing); + Self { + raftstore_slow_score: AtomicU64::new(f64::to_bits(1.0)), + raftstore_slow_trend: RollingRetriever::new(), + + health_service, + current_serving_status: Mutex::new(ServingStatus { + is_serving: false, + unhealthy_modules: HashSet::default(), + }), + } + } + + /// Marks a module (identified by name) to be unhealthy. Adding an unhealthy + /// will make the serving status of the TiKV server, reported via the + /// gRPC `HealthService`, to become `ServiceUnknown`. + /// + /// This is not an public API. This method is expected to be called only + /// from reporters. + fn add_unhealthy_module(&self, module_name: &'static str) { + let mut status = self.current_serving_status.lock(); + if !status.unhealthy_modules.insert(module_name) { + // Nothing changed. + return; + } + if status.unhealthy_modules.len() == 1 && status.is_serving { + debug_assert_eq!( + status.to_serving_status_pb(), + grpcio_health::ServingStatus::ServiceUnknown + ); + self.health_service + .set_serving_status("", grpcio_health::ServingStatus::ServiceUnknown); + } + } + + /// Removes a module (identified by name) that was marked unhealthy before. + /// When the unhealthy modules are cleared, the serving status reported + /// via the gRPC `HealthService` will change from `ServiceUnknown` to + /// `Serving`. + /// + /// This is not an public API. This method is expected to be called only + /// from reporters. + fn remove_unhealthy_module(&self, module_name: &'static str) { + let mut status = self.current_serving_status.lock(); + if !status.unhealthy_modules.remove(module_name) { + // Nothing changed. + return; + } + if status.unhealthy_modules.is_empty() && status.is_serving { + debug_assert_eq!( + status.to_serving_status_pb(), + grpcio_health::ServingStatus::Serving + ); + self.health_service + .set_serving_status("", grpcio_health::ServingStatus::Serving); + } + } + + /// Sets whether the TiKV server is serving. This is currently used to pause + /// the server, which has implementation in code but not commonly used. + /// + /// The effect of setting not serving overrides the effect of + /// [`add_on_healthy_module`](Self::add_unhealthy_module). + fn set_is_serving(&self, is_serving: bool) { + let mut status = self.current_serving_status.lock(); + if is_serving == status.is_serving { + // Nothing to do. + return; + } + status.is_serving = is_serving; + self.health_service + .set_serving_status("", status.to_serving_status_pb()); + } + + /// Gets the current serving status that is being reported by + /// `health_service`, if it's not shutdown. + fn get_serving_status(&self) -> grpcio_health::ServingStatus { + let status = self.current_serving_status.lock(); + status.to_serving_status_pb() + } + + fn update_raftstore_slow_score(&self, value: f64) { + self.raftstore_slow_score + .store(value.to_bits(), Ordering::Release); + } + + fn get_raftstore_slow_score(&self) -> f64 { + f64::from_bits(self.raftstore_slow_score.load(Ordering::Acquire)) + } + + fn update_raftstore_slow_trend(&self, slow_trend_pb: SlowTrendPb) { + self.raftstore_slow_trend.put(slow_trend_pb); + } + + fn get_raftstore_slow_trend(&self) -> SlowTrendPb { + self.raftstore_slow_trend.get_cloned() + } + + fn shutdown(&self) { + self.health_service.shutdown(); + } +} + +#[derive(Clone)] +pub struct HealthController { + inner: Arc, +} + +impl HealthController { + pub fn new() -> Self { + Self { + inner: Arc::new(HealthControllerInner::new()), + } + } + + pub fn get_raftstore_slow_score(&self) -> f64 { + self.inner.get_raftstore_slow_score() + } + + pub fn get_raftstore_slow_trend(&self) -> SlowTrendPb { + self.inner.get_raftstore_slow_trend() + } + + /// Get the gRPC `HealthService`. + /// + /// Only use this when it's necessary to startup the gRPC server or for test + /// purpose. Do not change the `HealthService`'s state manually. + /// + /// If it's necessary to update `HealthService`'s state, consider using + /// [`set_is_serving`](Self::set_is_serving) or use a reporter to add an + /// unhealthy module. An example: + /// [`RaftstoreReporter::set_is_healthy`](reporters::RaftstoreReporter::set_is_healthy). + pub fn get_grpc_health_service(&self) -> HealthService { + self.inner.health_service.clone() + } + + pub fn get_serving_status(&self) -> grpcio_health::ServingStatus { + self.inner.get_serving_status() + } + + /// Set whether the TiKV server is serving. This controls the state reported + /// by the gRPC `HealthService`. + pub fn set_is_serving(&self, is_serving: bool) { + self.inner.set_is_serving(is_serving); + } + + pub fn shutdown(&self) { + self.inner.shutdown(); + } +} + +// Make clippy happy. +impl Default for HealthControllerInner { + fn default() -> Self { + Self::new() + } +} + +impl Default for HealthController { + fn default() -> Self { + Self::new() + } +} + +/// An alternative util to simple RwLock. It allows writing not blocking +/// reading, at the expense of linearizability between reads and writes. +/// +/// This is suitable for use cases where atomic storing and loading is expected, +/// but atomic variables is not applicable due to the inner type larger than 8 +/// bytes. When writing is in progress, readings will get the previous value. +/// Writes will block each other, and fast and frequent writes may also block or +/// be blocked by slow reads. +struct RollingRetriever { + content: [RwLock; 2], + current_index: AtomicUsize, + write_mutex: Mutex<()>, +} + +impl RollingRetriever { + pub fn new() -> Self { + Self { + content: [RwLock::new(T::default()), RwLock::new(T::default())], + current_index: AtomicUsize::new(0), + write_mutex: Mutex::new(()), + } + } +} + +impl RollingRetriever { + #[inline] + pub fn put(&self, new_value: T) { + self.put_with(|| new_value) + } + + fn put_with(&self, f: impl FnOnce() -> T) { + let _write_guard = self.write_mutex.lock(); + // Update the item that is not the currently active one + let index = self.current_index.load(Ordering::Acquire) ^ 1; + + let mut data_guard = self.content[index].write(); + *data_guard = f(); + + drop(data_guard); + self.current_index.store(index, Ordering::Release); + } + + pub fn read(&self, f: impl FnOnce(&T) -> R) -> R { + let index = self.current_index.load(Ordering::Acquire); + let guard = self.content[index].read(); + f(guard.deref()) + } +} + +impl RollingRetriever { + pub fn get_cloned(&self) -> T { + self.read(|r| r.clone()) + } +} + +#[cfg(test)] +mod tests { + use std::{ + sync::mpsc::{sync_channel, RecvTimeoutError}, + time::Duration, + }; + + use super::*; + + #[test] + fn test_health_controller_update_service_status() { + let h = HealthController::new(); + + // Initial value of slow score + assert_eq!(h.get_raftstore_slow_score(), 1.0); + + assert_eq!( + h.get_serving_status(), + grpcio_health::ServingStatus::NotServing + ); + + h.set_is_serving(true); + assert_eq!( + h.get_serving_status(), + grpcio_health::ServingStatus::Serving + ); + + h.inner.add_unhealthy_module("A"); + assert_eq!( + h.get_serving_status(), + grpcio_health::ServingStatus::ServiceUnknown + ); + h.inner.add_unhealthy_module("B"); + assert_eq!( + h.get_serving_status(), + grpcio_health::ServingStatus::ServiceUnknown + ); + + h.inner.remove_unhealthy_module("A"); + assert_eq!( + h.get_serving_status(), + grpcio_health::ServingStatus::ServiceUnknown + ); + h.inner.remove_unhealthy_module("B"); + assert_eq!( + h.get_serving_status(), + grpcio_health::ServingStatus::Serving + ); + + h.set_is_serving(false); + assert_eq!( + h.get_serving_status(), + grpcio_health::ServingStatus::NotServing + ); + h.inner.add_unhealthy_module("A"); + assert_eq!( + h.get_serving_status(), + grpcio_health::ServingStatus::NotServing + ); + + h.set_is_serving(true); + assert_eq!( + h.get_serving_status(), + grpcio_health::ServingStatus::ServiceUnknown + ); + + h.inner.remove_unhealthy_module("A"); + assert_eq!( + h.get_serving_status(), + grpcio_health::ServingStatus::Serving + ); + } + + #[test] + fn test_rolling_retriever() { + let r = Arc::new(RollingRetriever::::new()); + assert_eq!(r.get_cloned(), 0); + + for i in 1..=10 { + r.put(i); + assert_eq!(r.get_cloned(), i); + } + + // Writing doesn't block reading. + let r1 = r.clone(); + let (write_continue_tx, rx) = sync_channel(0); + let write_handle = std::thread::spawn(move || { + r1.put_with(move || { + rx.recv().unwrap(); + 11 + }) + }); + for _ in 1..10 { + std::thread::sleep(Duration::from_millis(5)); + assert_eq!(r.get_cloned(), 10) + } + write_continue_tx.send(()).unwrap(); + write_handle.join().unwrap(); + assert_eq!(r.get_cloned(), 11); + + // Writing block each other. + let r1 = r.clone(); + let (write1_tx, rx1) = sync_channel(0); + let write1_handle = std::thread::spawn(move || { + r1.put_with(move || { + // Receive once for notifying lock acquired. + rx1.recv().unwrap(); + // Receive again to be notified ready to continue. + rx1.recv().unwrap(); + 12 + }) + }); + write1_tx.send(()).unwrap(); + let r1 = r.clone(); + let (write2_tx, rx2) = sync_channel(0); + let write2_handle = std::thread::spawn(move || { + r1.put_with(move || { + write2_tx.send(()).unwrap(); + 13 + }) + }); + // Write 2 cannot continue as blocked by write 1. + assert_eq!( + rx2.recv_timeout(Duration::from_millis(50)).unwrap_err(), + RecvTimeoutError::Timeout + ); + // Continue write1 + write1_tx.send(()).unwrap(); + write1_handle.join().unwrap(); + assert_eq!(r.get_cloned(), 12); + // Continue write2 + rx2.recv().unwrap(); + write2_handle.join().unwrap(); + assert_eq!(r.get_cloned(), 13); + } +} diff --git a/components/health_controller/src/reporters.rs b/components/health_controller/src/reporters.rs new file mode 100644 index 00000000000..56624c37d64 --- /dev/null +++ b/components/health_controller/src/reporters.rs @@ -0,0 +1,332 @@ +// Copyright 2024 TiKV Project Authors. Licensed under Apache-2.0. + +use std::{ + sync::Arc, + time::{Duration, Instant}, +}; + +use kvproto::pdpb; +use pdpb::SlowTrend as SlowTrendPb; +use prometheus::IntGauge; + +use crate::{ + slow_score::{SlowScore, SlowScoreTickResult}, + trend::{RequestPerSecRecorder, Trend}, + types::InspectFactor, + HealthController, HealthControllerInner, RaftstoreDuration, +}; + +/// The parameters for building a [`RaftstoreReporter`]. +/// +/// For slow trend related parameters (unsensitive_cause, unsensitive_result, +/// cause_*, result_*), please refer to : [`SlowTrendStatistics::new`] and +/// [`Trend`]. +pub struct RaftstoreReporterConfig { + /// The interval to tick the [`RaftstoreReporter`]. + /// + /// The `RaftstoreReporter` doesn't tick by itself, the caller (the PD + /// worker) is expected to tick it. But the interval is necessary in + /// some internal calculations. + pub inspect_interval: Duration, + pub inspect_kvdb_interval: Duration, + + pub unsensitive_cause: f64, + pub unsensitive_result: f64, + pub net_io_factor: f64, + + // Metrics about slow trend. + pub cause_spike_filter_value_gauge: IntGauge, + pub cause_spike_filter_count_gauge: IntGauge, + pub cause_l1_gap_gauges: IntGauge, + pub cause_l2_gap_gauges: IntGauge, + pub result_spike_filter_value_gauge: IntGauge, + pub result_spike_filter_count_gauge: IntGauge, + pub result_l1_gap_gauges: IntGauge, + pub result_l2_gap_gauges: IntGauge, +} + +/// A unified slow score that combines multiple slow scores. +/// +/// It calculates the final slow score of a store by picking the maximum +/// score among multiple factors. Each factor represents a different aspect of +/// the store's performance. Typically, we have two factors: Raft Disk I/O and +/// KvDB Disk I/O. If there are more factors in the future, we can add them +/// here. +#[derive(Default)] +pub struct UnifiedSlowScore { + factors: Vec, +} + +impl UnifiedSlowScore { + pub fn new(cfg: &RaftstoreReporterConfig) -> Self { + let mut unified_slow_score = UnifiedSlowScore::default(); + // The first factor is for Raft Disk I/O. + unified_slow_score + .factors + .push(SlowScore::new(cfg.inspect_interval)); + // The second factor is for KvDB Disk I/O. + unified_slow_score + .factors + .push(SlowScore::new_with_extra_config( + cfg.inspect_kvdb_interval, + 0.6, + )); + unified_slow_score + } + + #[inline] + pub fn record( + &mut self, + id: u64, + factor: InspectFactor, + duration: &RaftstoreDuration, + not_busy: bool, + ) { + self.factors[factor as usize].record(id, duration.delays_on_disk_io(false), not_busy); + } + + #[inline] + pub fn get(&self, factor: InspectFactor) -> &SlowScore { + &self.factors[factor as usize] + } + + #[inline] + pub fn get_mut(&mut self, factor: InspectFactor) -> &mut SlowScore { + &mut self.factors[factor as usize] + } + + // Returns the maximum score of all factors. + pub fn get_score(&self) -> f64 { + self.factors + .iter() + .map(|factor| factor.get()) + .fold(1.0, f64::max) + } + + pub fn last_tick_finished(&self) -> bool { + self.factors.iter().all(SlowScore::last_tick_finished) + } +} + +pub struct RaftstoreReporter { + health_controller_inner: Arc, + slow_score: UnifiedSlowScore, + slow_trend: SlowTrendStatistics, + is_healthy: bool, +} + +impl RaftstoreReporter { + const MODULE_NAME: &'static str = "raftstore"; + + pub fn new(health_controller: &HealthController, cfg: RaftstoreReporterConfig) -> Self { + Self { + health_controller_inner: health_controller.inner.clone(), + slow_score: UnifiedSlowScore::new(&cfg), + slow_trend: SlowTrendStatistics::new(cfg), + is_healthy: true, + } + } + + pub fn get_slow_score(&self) -> f64 { + self.slow_score.get_score() + } + + pub fn get_slow_trend(&self) -> &SlowTrendStatistics { + &self.slow_trend + } + + pub fn record_raftstore_duration( + &mut self, + id: u64, + factor: InspectFactor, + duration: RaftstoreDuration, + store_not_busy: bool, + ) { + // Fine-tuned, `SlowScore` only takes the I/O jitters on the disk into account. + self.slow_score + .record(id, factor, &duration, store_not_busy); + self.slow_trend.record(duration); + + // Publish slow score to health controller + self.health_controller_inner + .update_raftstore_slow_score(self.slow_score.get_score()); + } + + fn is_healthy(&self) -> bool { + self.is_healthy + } + + fn set_is_healthy(&mut self, is_healthy: bool) { + if is_healthy == self.is_healthy { + return; + } + + self.is_healthy = is_healthy; + if is_healthy { + self.health_controller_inner + .remove_unhealthy_module(Self::MODULE_NAME); + } else { + self.health_controller_inner + .add_unhealthy_module(Self::MODULE_NAME); + } + } + + pub fn tick(&mut self, store_maybe_busy: bool, factor: InspectFactor) -> SlowScoreTickResult { + // Record a fairly great value when timeout + self.slow_trend.slow_cause.record(500_000, Instant::now()); + + // healthy: The health status of the current store. + // all_ticks_finished: The last tick of all factors is finished. + // factor_tick_finished: The last tick of the current factor is finished. + let (healthy, all_ticks_finished, factor_tick_finished) = ( + self.is_healthy(), + self.slow_score.last_tick_finished(), + self.slow_score.get(factor).last_tick_finished(), + ); + // The health status is recovered to serving as long as any tick + // does not timeout. + if !healthy && all_ticks_finished { + self.set_is_healthy(true); + } + if !all_ticks_finished { + // If the last tick is not finished, it means that the current store might + // be busy on handling requests or delayed on I/O operations. And only when + // the current store is not busy, it should record the last_tick as a timeout. + if !store_maybe_busy && !factor_tick_finished { + self.slow_score.get_mut(factor).record_timeout(); + } + } + + let slow_score_tick_result = self.slow_score.get_mut(factor).tick(); + if slow_score_tick_result.updated_score.is_some() && !slow_score_tick_result.has_new_record + { + self.set_is_healthy(false); + } + + // Publish the slow score to health controller + if slow_score_tick_result.updated_score.is_some() { + self.health_controller_inner + .update_raftstore_slow_score(self.slow_score.get_score()); + } + + slow_score_tick_result + } + + pub fn update_slow_trend( + &mut self, + observed_request_count: u64, + now: Instant, + ) -> (Option, SlowTrendPb) { + let requests_per_sec = self + .slow_trend + .slow_result_recorder + .record_and_get_current_rps(observed_request_count, now); + + let slow_trend_cause_rate = self.slow_trend.slow_cause.increasing_rate(); + let mut slow_trend_pb = SlowTrendPb::default(); + slow_trend_pb.set_cause_rate(slow_trend_cause_rate); + slow_trend_pb.set_cause_value(self.slow_trend.slow_cause.l0_avg()); + if let Some(requests_per_sec) = requests_per_sec { + self.slow_trend + .slow_result + .record(requests_per_sec as u64, Instant::now()); + slow_trend_pb.set_result_value(self.slow_trend.slow_result.l0_avg()); + let slow_trend_result_rate = self.slow_trend.slow_result.increasing_rate(); + slow_trend_pb.set_result_rate(slow_trend_result_rate); + } + + // Publish the result to health controller. + self.health_controller_inner + .update_raftstore_slow_trend(slow_trend_pb.clone()); + + (requests_per_sec, slow_trend_pb) + } +} + +pub struct SlowTrendStatistics { + net_io_factor: f64, + /// Detector to detect NetIo&DiskIo jitters. + pub slow_cause: Trend, + /// Reactor as an assistant detector to detect the QPS jitters. + pub slow_result: Trend, + pub slow_result_recorder: RequestPerSecRecorder, +} + +impl SlowTrendStatistics { + #[inline] + pub fn new(config: RaftstoreReporterConfig) -> Self { + Self { + slow_cause: Trend::new( + // Disable SpikeFilter for now + Duration::from_secs(0), + config.cause_spike_filter_value_gauge, + config.cause_spike_filter_count_gauge, + Duration::from_secs(180), + Duration::from_secs(30), + Duration::from_secs(120), + Duration::from_secs(600), + 1, + tikv_util::time::duration_to_us(Duration::from_micros(500)), + config.cause_l1_gap_gauges, + config.cause_l2_gap_gauges, + config.unsensitive_cause, + ), + slow_result: Trend::new( + // Disable SpikeFilter for now + Duration::from_secs(0), + config.result_spike_filter_value_gauge, + config.result_spike_filter_count_gauge, + Duration::from_secs(120), + Duration::from_secs(15), + Duration::from_secs(60), + Duration::from_secs(300), + 1, + 2000, + config.result_l1_gap_gauges, + config.result_l2_gap_gauges, + config.unsensitive_result, + ), + slow_result_recorder: RequestPerSecRecorder::new(), + net_io_factor: config.net_io_factor, /* FIXME: add extra parameter in + * Config to control it. */ + } + } + + #[inline] + pub fn record(&mut self, duration: RaftstoreDuration) { + // TODO: It's more appropriate to divide the factor into `Disk IO factor` and + // `Net IO factor`. + // Currently, when `network ratio == 1`, it summarizes all factors by `sum` + // simplily, approved valid to common cases when there exists IO jitters on + // Network or Disk. + let latency = || -> u64 { + if self.net_io_factor as u64 >= 1 { + return tikv_util::time::duration_to_us(duration.sum()); + } + let disk_io_latency = + tikv_util::time::duration_to_us(duration.delays_on_disk_io(true)) as f64; + let network_io_latency = + tikv_util::time::duration_to_us(duration.delays_on_net_io()) as f64; + (disk_io_latency + network_io_latency * self.net_io_factor) as u64 + }(); + self.slow_cause.record(latency, Instant::now()); + } +} + +/// A reporter that can set states directly, for testing purposes. +pub struct TestReporter { + health_controller_inner: Arc, +} + +impl TestReporter { + pub fn new(health_controller: &HealthController) -> Self { + Self { + health_controller_inner: health_controller.inner.clone(), + } + } + + pub fn set_raftstore_slow_score(&self, slow_score: f64) { + self.health_controller_inner + .update_raftstore_slow_score(slow_score); + } +} diff --git a/components/health_controller/src/slow_score.rs b/components/health_controller/src/slow_score.rs new file mode 100644 index 00000000000..846e3f98517 --- /dev/null +++ b/components/health_controller/src/slow_score.rs @@ -0,0 +1,287 @@ +// Copyright 2024 TiKV Project Authors. Licensed under Apache-2.0. + +use std::{ + cmp, + time::{Duration, Instant}, +}; + +use ordered_float::OrderedFloat; + +/// Interval for updating the slow score. +const UPDATE_INTERVALS: Duration = Duration::from_secs(10); +/// Recovery intervals for the slow score. +/// If the score has reached 100 and there is no timeout inspecting requests +/// during this interval, the score will go back to 1 after 5min. +const RECOVERY_INTERVALS: Duration = Duration::from_secs(60 * 5); +// Slow score is a value that represents the speed of a store and ranges in [1, +// 100]. It is maintained in the AIMD way. +// If there are some inspecting requests timeout during a round, by default the +// score will be increased at most 1x when above 10% inspecting requests +// timeout. If there is not any timeout inspecting requests, the score will go +// back to 1 in at least 5min. +pub struct SlowScore { + value: OrderedFloat, + last_record_time: Instant, + last_update_time: Instant, + + timeout_requests: usize, + total_requests: usize, + + inspect_interval: Duration, + // The maximal tolerated timeout ratio. + ratio_thresh: OrderedFloat, + // Minimal time that the score could be decreased from 100 to 1. + min_ttr: Duration, + + // After how many ticks the value need to be updated. + round_ticks: u64, + // Identify every ticks. + last_tick_id: u64, + // If the last tick does not finished, it would be recorded as a timeout. + last_tick_finished: bool, +} + +impl SlowScore { + pub fn new(inspect_interval: Duration) -> SlowScore { + SlowScore { + value: OrderedFloat(1.0), + + timeout_requests: 0, + total_requests: 0, + + inspect_interval, + ratio_thresh: OrderedFloat(0.1), + min_ttr: RECOVERY_INTERVALS, + last_record_time: Instant::now(), + last_update_time: Instant::now(), + round_ticks: 30, + last_tick_id: 0, + last_tick_finished: true, + } + } + + // Only for kvdb. + pub fn new_with_extra_config(inspect_interval: Duration, timeout_ratio: f64) -> SlowScore { + SlowScore { + value: OrderedFloat(1.0), + + timeout_requests: 0, + total_requests: 0, + + inspect_interval, + ratio_thresh: OrderedFloat(timeout_ratio), + min_ttr: RECOVERY_INTERVALS, + last_record_time: Instant::now(), + last_update_time: Instant::now(), + // The minimal round ticks is 1 for kvdb. + round_ticks: cmp::max( + UPDATE_INTERVALS.div_duration_f64(inspect_interval) as u64, + 1_u64, + ), + last_tick_id: 0, + last_tick_finished: true, + } + } + + pub fn record(&mut self, id: u64, duration: Duration, not_busy: bool) { + self.last_record_time = Instant::now(); + if id != self.last_tick_id { + return; + } + self.last_tick_finished = true; + self.total_requests += 1; + if not_busy && duration >= self.inspect_interval { + self.timeout_requests += 1; + } + } + + pub fn record_timeout(&mut self) { + self.last_tick_finished = true; + self.total_requests += 1; + self.timeout_requests += 1; + } + + pub fn update(&mut self) -> f64 { + let elapsed = self.last_update_time.elapsed(); + self.update_impl(elapsed).into() + } + + pub fn get(&self) -> f64 { + self.value.into() + } + + // Update the score in a AIMD way. + fn update_impl(&mut self, elapsed: Duration) -> OrderedFloat { + if self.timeout_requests == 0 { + let desc = 100.0 * (elapsed.as_millis() as f64 / self.min_ttr.as_millis() as f64); + if OrderedFloat(desc) > self.value - OrderedFloat(1.0) { + self.value = 1.0.into(); + } else { + self.value -= desc; + } + } else { + let timeout_ratio = self.timeout_requests as f64 / self.total_requests as f64; + let near_thresh = + cmp::min(OrderedFloat(timeout_ratio), self.ratio_thresh) / self.ratio_thresh; + let value = self.value * (OrderedFloat(1.0) + near_thresh); + self.value = cmp::min(OrderedFloat(100.0), value); + } + + self.total_requests = 0; + self.timeout_requests = 0; + self.last_update_time = Instant::now(); + self.value + } + + pub fn should_force_report_slow_store(&self) -> bool { + self.value >= OrderedFloat(100.0) && (self.last_tick_id % self.round_ticks == 0) + } + + pub fn get_inspect_interval(&self) -> Duration { + self.inspect_interval + } + + pub fn last_tick_finished(&self) -> bool { + self.last_tick_finished + } + + pub fn tick(&mut self) -> SlowScoreTickResult { + let should_force_report_slow_store = self.should_force_report_slow_store(); + + let id = self.last_tick_id + 1; + self.last_tick_id += 1; + self.last_tick_finished = false; + + let (updated_score, has_new_record) = if self.last_tick_id % self.round_ticks == 0 { + // `last_update_time` is refreshed every round. If no update happens in a whole + // round, we set the status to unknown. + let has_new_record = self.last_record_time >= self.last_update_time; + + let slow_score = self.update(); + (Some(slow_score), has_new_record) + } else { + (None, false) + }; + + SlowScoreTickResult { + tick_id: id, + updated_score, + has_new_record, + should_force_report_slow_store, + } + } +} + +pub struct SlowScoreTickResult { + pub tick_id: u64, + // None if skipped in this tick + pub updated_score: Option, + pub has_new_record: bool, + pub should_force_report_slow_store: bool, +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_slow_score() { + let mut slow_score = SlowScore::new(Duration::from_millis(500)); + slow_score.timeout_requests = 5; + slow_score.total_requests = 100; + assert_eq!( + OrderedFloat(1.5), + slow_score.update_impl(Duration::from_secs(10)) + ); + + slow_score.timeout_requests = 10; + slow_score.total_requests = 100; + assert_eq!( + OrderedFloat(3.0), + slow_score.update_impl(Duration::from_secs(10)) + ); + + slow_score.timeout_requests = 20; + slow_score.total_requests = 100; + assert_eq!( + OrderedFloat(6.0), + slow_score.update_impl(Duration::from_secs(10)) + ); + + slow_score.timeout_requests = 100; + slow_score.total_requests = 100; + assert_eq!( + OrderedFloat(12.0), + slow_score.update_impl(Duration::from_secs(10)) + ); + + slow_score.timeout_requests = 11; + slow_score.total_requests = 100; + assert_eq!( + OrderedFloat(24.0), + slow_score.update_impl(Duration::from_secs(10)) + ); + + slow_score.timeout_requests = 0; + slow_score.total_requests = 100; + assert_eq!( + OrderedFloat(19.0), + slow_score.update_impl(Duration::from_secs(15)) + ); + + slow_score.timeout_requests = 0; + slow_score.total_requests = 100; + assert_eq!( + OrderedFloat(1.0), + slow_score.update_impl(Duration::from_secs(57)) + ); + } + + #[test] + fn test_slow_score_extra() { + let mut slow_score = SlowScore::new_with_extra_config(Duration::from_millis(1000), 0.6); + slow_score.timeout_requests = 1; + slow_score.total_requests = 10; + let score = slow_score.update_impl(Duration::from_secs(10)); + assert!(score > OrderedFloat(1.16)); + assert!(score < OrderedFloat(1.17)); + + slow_score.timeout_requests = 2; + slow_score.total_requests = 10; + let score = slow_score.update_impl(Duration::from_secs(10)); + assert!(score > OrderedFloat(1.5)); + assert!(score < OrderedFloat(1.6)); + + slow_score.timeout_requests = 0; + slow_score.total_requests = 100; + assert_eq!( + OrderedFloat(1.0), + slow_score.update_impl(Duration::from_secs(57)) + ); + + slow_score.timeout_requests = 3; + slow_score.total_requests = 10; + assert_eq!( + OrderedFloat(1.5), + slow_score.update_impl(Duration::from_secs(10)) + ); + + slow_score.timeout_requests = 6; + slow_score.total_requests = 10; + assert_eq!( + OrderedFloat(3.0), + slow_score.update_impl(Duration::from_secs(10)) + ); + + slow_score.timeout_requests = 10; + slow_score.total_requests = 10; + assert_eq!( + OrderedFloat(6.0), + slow_score.update_impl(Duration::from_secs(10)) + ); + + // Test too large inspect interval. + let slow_score = SlowScore::new_with_extra_config(Duration::from_secs(11), 0.1); + assert_eq!(slow_score.round_ticks, 1); + } +} diff --git a/components/health_controller/src/types.rs b/components/health_controller/src/types.rs new file mode 100644 index 00000000000..7342273e972 --- /dev/null +++ b/components/health_controller/src/types.rs @@ -0,0 +1,123 @@ +// Copyright 2024 TiKV Project Authors. Licensed under Apache-2.0. + +use std::{fmt::Debug, u64}; + +/// Represent the duration of all stages of raftstore recorded by one +/// inspecting. +#[derive(Default, Debug)] +pub struct RaftstoreDuration { + pub store_wait_duration: Option, + pub store_process_duration: Option, + pub store_write_duration: Option, + pub store_commit_duration: Option, + pub apply_wait_duration: Option, + pub apply_process_duration: Option, +} + +impl RaftstoreDuration { + #[inline] + pub fn sum(&self) -> std::time::Duration { + self.delays_on_disk_io(true) + self.delays_on_net_io() + } + + #[inline] + /// Returns the delayed duration on Disk I/O. + pub fn delays_on_disk_io(&self, include_wait_duration: bool) -> std::time::Duration { + let duration = self.store_process_duration.unwrap_or_default() + + self.store_write_duration.unwrap_or_default() + + self.apply_process_duration.unwrap_or_default(); + if include_wait_duration { + duration + + self.store_wait_duration.unwrap_or_default() + + self.apply_wait_duration.unwrap_or_default() + } else { + duration + } + } + + #[inline] + /// Returns the delayed duration on Network I/O. + /// + /// Normally, it can be reflected by the duraiton on + /// `store_commit_duraiton`. + pub fn delays_on_net_io(&self) -> std::time::Duration { + // The `store_commit_duration` serves as an indicator for latency + // during the duration of transferring Raft logs to peers and appending + // logs. In most scenarios, instances of latency fluctuations in the + // network are reflected by this duration. Hence, it is selected as a + // representative of network latency. + self.store_commit_duration.unwrap_or_default() + } +} + +#[derive(Debug, Clone, Copy, PartialEq)] +pub enum InspectFactor { + RaftDisk = 0, + KvDisk, + // TODO: Add more factors, like network io. +} + +impl InspectFactor { + pub fn as_str(&self) -> &str { + match *self { + InspectFactor::RaftDisk => "raft", + InspectFactor::KvDisk => "kvdb", + } + } +} + +/// Used to inspect the latency of all stages of raftstore. +pub struct LatencyInspector { + id: u64, + duration: RaftstoreDuration, + cb: Box, +} + +impl Debug for LatencyInspector { + fn fmt(&self, fmt: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!( + fmt, + "LatencyInspector: id {} duration: {:?}", + self.id, self.duration + ) + } +} + +impl LatencyInspector { + pub fn new(id: u64, cb: Box) -> Self { + Self { + id, + cb, + duration: RaftstoreDuration::default(), + } + } + + pub fn record_store_wait(&mut self, duration: std::time::Duration) { + self.duration.store_wait_duration = Some(duration); + } + + pub fn record_store_process(&mut self, duration: std::time::Duration) { + self.duration.store_process_duration = Some(duration); + } + + pub fn record_store_write(&mut self, duration: std::time::Duration) { + self.duration.store_write_duration = Some(duration); + } + + pub fn record_store_commit(&mut self, duration: std::time::Duration) { + self.duration.store_commit_duration = Some(duration); + } + + pub fn record_apply_wait(&mut self, duration: std::time::Duration) { + self.duration.apply_wait_duration = Some(duration); + } + + pub fn record_apply_process(&mut self, duration: std::time::Duration) { + self.duration.apply_process_duration = Some(duration); + } + + /// Call the callback. + pub fn finish(self) { + (self.cb)(self.id, self.duration); + } +} diff --git a/components/raftstore-v2/src/worker/pd/mod.rs b/components/raftstore-v2/src/worker/pd/mod.rs index 77915dd0378..540e29dba52 100644 --- a/components/raftstore-v2/src/worker/pd/mod.rs +++ b/components/raftstore-v2/src/worker/pd/mod.rs @@ -9,6 +9,10 @@ use causal_ts::CausalTsProviderImpl; use collections::HashMap; use concurrency_manager::ConcurrencyManager; use engine_traits::{KvEngine, RaftEngine, TabletRegistry}; +<<<<<<< HEAD +======= +use health_controller::types::{InspectFactor, LatencyInspector, RaftstoreDuration}; +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) use kvproto::{metapb, pdpb}; use pd_client::{BucketStat, PdClient}; use raftstore::store::{ @@ -257,6 +261,7 @@ where store_heartbeat_interval / NUM_COLLECT_STORE_INFOS_PER_HEARTBEAT, cfg.value().report_min_resolved_ts_interval.0, cfg.value().inspect_interval.0, + std::time::Duration::default(), PdReporter::new(pd_scheduler, logger.clone()), ); stats_monitor.start( @@ -436,7 +441,7 @@ impl StoreStatsReporter for PdReporter { } } - fn update_latency_stats(&self, timer_tick: u64) { + fn update_latency_stats(&self, timer_tick: u64, _factor: InspectFactor) { // Tick slowness statistics. { if let Err(e) = self.scheduler.schedule(Task::TickSlownessStats) { diff --git a/components/raftstore/src/store/config.rs b/components/raftstore/src/store/config.rs index 7c13446c185..b39e7e94eca 100644 --- a/components/raftstore/src/store/config.rs +++ b/components/raftstore/src/store/config.rs @@ -341,19 +341,38 @@ pub struct Config { #[deprecated = "The configuration has been removed. The time to clean stale peer safely can be decided based on RocksDB snapshot sequence number."] pub clean_stale_peer_delay: ReadableDuration, - // Interval to inspect the latency of raftstore for slow store detection. + #[online_config(hidden)] + // Interval to inspect the latency of flushing raft logs for slow store detection. pub inspect_interval: ReadableDuration, + // Interval to inspect the latency of flushes on kvdb for slow store detection. + // If the kvdb uses the same mount path with raftdb, the default value will be + // optimized to `0` to avoid duplicated inspection. + #[doc(hidden)] + #[online_config(hidden)] + pub inspect_kvdb_interval: ReadableDuration, /// Threshold of CPU utilization to inspect for slow store detection. #[doc(hidden)] + #[online_config(hidden)] pub inspect_cpu_util_thd: f64, + #[doc(hidden)] + #[online_config(hidden)] // The unsensitive(increase it to reduce sensitiveness) of the cause-trend detection pub slow_trend_unsensitive_cause: f64, + #[doc(hidden)] + #[online_config(hidden)] // The unsensitive(increase it to reduce sensitiveness) of the result-trend detection pub slow_trend_unsensitive_result: f64, +<<<<<<< HEAD // Interval to report min resolved ts, if it is zero, it means disabled. pub report_min_resolved_ts_interval: ReadableDuration, +======= + #[doc(hidden)] + #[online_config(hidden)] + // The sensitiveness of slowness on network-io. + pub slow_trend_network_io_factor: f64, +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) /// Interval to check whether to reactivate in-memory pessimistic lock after /// being disabled before transferring leader. @@ -513,6 +532,7 @@ impl Default for Config { region_split_size: ReadableSize(0), clean_stale_peer_delay: ReadableDuration::minutes(0), inspect_interval: ReadableDuration::millis(100), + inspect_kvdb_interval: ReadableDuration::secs(2), // The default value of `inspect_cpu_util_thd` is 0.4, which means // when the cpu utilization is greater than 40%, the store might be // regarded as a slow node if there exists delayed inspected messages. @@ -645,6 +665,29 @@ impl Config { } } + /// Optimize the interval of different inspectors according to the + /// configuration. + pub fn optimize_inspector(&mut self, separated_raft_mount_path: bool) { + // If the kvdb uses the same mount path with raftdb, the health status + // of kvdb will be inspected by raftstore automatically. So it's not necessary + // to inspect kvdb. + if !separated_raft_mount_path { + self.inspect_kvdb_interval = ReadableDuration::ZERO; + } else { + // If the inspect_kvdb_interval is less than inspect_interval, it should + // use `inspect_interval` * 10 as an empirical inspect interval for KvDB Disk + // I/O. + let inspect_kvdb_interval = if self.inspect_kvdb_interval < self.inspect_interval + && self.inspect_kvdb_interval != ReadableDuration::ZERO + { + self.inspect_interval * 10 + } else { + self.inspect_kvdb_interval + }; + self.inspect_kvdb_interval = inspect_kvdb_interval; + } + } + pub fn validate( &mut self, region_split_size: ReadableSize, @@ -1561,5 +1604,35 @@ mod tests { cfg.raft_log_gc_count_limit(), split_size * 3 / 4 / ReadableSize::kb(1) ); +<<<<<<< HEAD +======= + + cfg = Config::new(); + cfg.optimize_for(false); + cfg.raft_write_wait_duration = ReadableDuration::micros(1001); + cfg.validate(split_size, true, split_size / 20, false) + .unwrap_err(); + + cfg = Config::new(); + cfg.optimize_inspector(false); + assert_eq!(cfg.inspect_kvdb_interval, ReadableDuration::ZERO); + + cfg = Config::new(); + cfg.inspect_kvdb_interval = ReadableDuration::secs(1); + cfg.optimize_inspector(false); + assert_eq!(cfg.inspect_kvdb_interval, ReadableDuration::ZERO); + cfg.optimize_inspector(true); + assert_eq!(cfg.inspect_kvdb_interval, ReadableDuration::ZERO); + + cfg.inspect_kvdb_interval = ReadableDuration::secs(1); + cfg.optimize_inspector(true); + assert_eq!(cfg.inspect_kvdb_interval, ReadableDuration::secs(1)); + + cfg = Config::new(); + cfg.inspect_kvdb_interval = ReadableDuration::millis(1); + cfg.inspect_interval = ReadableDuration::millis(100); + cfg.optimize_inspector(true); + assert_eq!(cfg.inspect_kvdb_interval, ReadableDuration::secs(1)); +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) } } diff --git a/components/raftstore/src/store/fsm/store.rs b/components/raftstore/src/store/fsm/store.rs index ad21cc64fec..9797e486ee9 100644 --- a/components/raftstore/src/store/fsm/store.rs +++ b/components/raftstore/src/store/fsm/store.rs @@ -33,7 +33,14 @@ use engine_traits::{ use fail::fail_point; use file_system::{IoType, WithIoType}; use futures::{compat::Future01CompatExt, FutureExt}; +<<<<<<< HEAD use grpcio_health::HealthService; +======= +use health_controller::{ + types::{InspectFactor, LatencyInspector}, + HealthController, +}; +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) use itertools::Itertools; use keys::{self, data_end_key, data_key, enc_end_key, enc_start_key}; use kvproto::{ @@ -102,9 +109,16 @@ use crate::{ worker::{ AutoSplitController, CleanupRunner, CleanupSstRunner, CleanupSstTask, CleanupTask, CompactRunner, CompactTask, ConsistencyCheckRunner, ConsistencyCheckTask, +<<<<<<< HEAD GcSnapshotRunner, GcSnapshotTask, PdRunner, RaftlogGcRunner, RaftlogGcTask, ReadDelegate, RefreshConfigRunner, RefreshConfigTask, RegionRunner, RegionTask, SplitCheckTask, +======= + DiskCheckRunner, DiskCheckTask, GcSnapshotRunner, GcSnapshotTask, PdRunner, + RaftlogGcRunner, RaftlogGcTask, ReadDelegate, RefreshConfigRunner, RefreshConfigTask, + RegionRunner, RegionTask, SnapGenRunner, SnapGenTask, SplitCheckTask, + SNAP_GENERATOR_MAX_POOL_SIZE, +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) }, Callback, CasualMessage, CompactThreshold, GlobalReplicationState, InspectedRaftMessage, MergeResultKind, PdTask, PeerMsg, PeerTick, RaftCommand, SignificantMsg, SnapManager, @@ -553,7 +567,12 @@ where pub cleanup_scheduler: Scheduler, pub raftlog_gc_scheduler: Scheduler, pub raftlog_fetch_scheduler: Scheduler>, +<<<<<<< HEAD pub region_scheduler: Scheduler>, +======= + pub region_scheduler: Scheduler, + pub disk_check_scheduler: Scheduler, +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) pub apply_router: ApplyRouter, pub router: RaftRouter, pub importer: Arc, @@ -862,11 +881,43 @@ impl<'a, EK: KvEngine + 'static, ER: RaftEngine + 'static, T: Transport> #[cfg(any(test, feature = "testexport"))] StoreMsg::Validate(f) => f(&self.ctx.cfg), StoreMsg::LatencyInspect { + factor, send_time, mut inspector, } => { +<<<<<<< HEAD inspector.record_store_wait(send_time.saturating_elapsed()); self.ctx.pending_latency_inspect.push(inspector); +======= + match factor { + InspectFactor::RaftDisk => { + inspector.record_store_wait(send_time.saturating_elapsed()); + inspector.record_store_commit( + self.ctx + .raft_metrics + .health_stats + .avg(InspectIoType::Network), + ); + // Reset the health_stats and wait it to be refreshed in the next tick. + self.ctx.raft_metrics.health_stats.reset(); + self.ctx.pending_latency_inspect.push(inspector); + } + InspectFactor::KvDisk => { + // Send LatencyInspector to disk_check_scheduler to inspect latency. + if let Err(e) = self + .ctx + .disk_check_scheduler + .schedule(DiskCheckTask::InspectLatency { inspector }) + { + warn!( + "Failed to schedule disk check task"; + "error" => ?e, + "store_id" => self.fsm.store.id + ); + } + } + } +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) } StoreMsg::UnsafeRecoveryReport(report) => self.store_heartbeat_pd(Some(report)), StoreMsg::UnsafeRecoveryCreatePeer { syncer, create } => { @@ -1221,7 +1272,13 @@ pub struct RaftPollerBuilder { cleanup_scheduler: Scheduler, raftlog_gc_scheduler: Scheduler, raftlog_fetch_scheduler: Scheduler>, +<<<<<<< HEAD pub region_scheduler: Scheduler>, +======= + pub snap_gen_scheduler: Scheduler>, + disk_check_scheduler: Scheduler, + pub region_scheduler: Scheduler, +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) apply_router: ApplyRouter, pub router: RaftRouter, pub importer: Arc, @@ -1456,6 +1513,7 @@ where store: self.store.clone(), pd_scheduler: self.pd_scheduler.clone(), consistency_check_scheduler: self.consistency_check_scheduler.clone(), + disk_check_scheduler: self.disk_check_scheduler.clone(), split_check_scheduler: self.split_check_scheduler.clone(), region_scheduler: self.region_scheduler.clone(), apply_router: self.apply_router.clone(), @@ -1533,6 +1591,11 @@ where cleanup_scheduler: self.cleanup_scheduler.clone(), raftlog_gc_scheduler: self.raftlog_gc_scheduler.clone(), raftlog_fetch_scheduler: self.raftlog_fetch_scheduler.clone(), +<<<<<<< HEAD +======= + snap_gen_scheduler: self.snap_gen_scheduler.clone(), + disk_check_scheduler: self.disk_check_scheduler.clone(), +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) region_scheduler: self.region_scheduler.clone(), apply_router: self.apply_router.clone(), router: self.router.clone(), @@ -1622,6 +1685,7 @@ impl RaftBatchSystem { collector_reg_handle: CollectorRegHandle, health_service: Option, causal_ts_provider: Option>, // used for rawkv apiv2 + mut disk_check_runner: DiskCheckRunner, grpc_service_mgr: GrpcServiceManager, safe_point: Arc, ) -> Result<()> { @@ -1710,6 +1774,12 @@ impl RaftBatchSystem { let consistency_check_scheduler = workers .background_worker .start("consistency-check", consistency_check_runner); + // The scheduler dedicated to health checking the KvEngine disk when it's using + // a separate disk from RaftEngine. + disk_check_runner.bind_background_worker(workers.background_worker.clone()); + let disk_check_scheduler = workers + .background_worker + .start("disk-check-worker", disk_check_runner); self.store_writers.spawn( meta.get_id(), @@ -1728,6 +1798,11 @@ impl RaftBatchSystem { router: self.router.clone(), split_check_scheduler, region_scheduler, +<<<<<<< HEAD +======= + snap_gen_scheduler, + disk_check_scheduler, +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) pd_scheduler: workers.pd_worker.scheduler(), consistency_check_scheduler, cleanup_scheduler, @@ -1874,7 +1949,7 @@ impl RaftBatchSystem { causal_ts_provider, grpc_service_mgr, ); - assert!(workers.pd_worker.start_with_timer(pd_runner)); + assert!(workers.pd_worker.start(pd_runner)); if let Err(e) = sys_util::thread::set_priority(sys_util::HIGH_PRI) { warn!("set thread priority for raftstore failed"; "error" => ?e); diff --git a/components/raftstore/src/store/metrics.rs b/components/raftstore/src/store/metrics.rs index 7973d9ad042..399e2cb9401 100644 --- a/components/raftstore/src/store/metrics.rs +++ b/components/raftstore/src/store/metrics.rs @@ -829,8 +829,11 @@ lazy_static! { exponential_buckets(0.00001, 2.0, 26).unwrap() ).unwrap(); - pub static ref STORE_SLOW_SCORE_GAUGE: Gauge = - register_gauge!("tikv_raftstore_slow_score", "Slow score of the store.").unwrap(); + pub static ref STORE_SLOW_SCORE_GAUGE: IntGaugeVec = register_int_gauge_vec!( + "tikv_raftstore_slow_score", + "Slow score of the store.", + &["type"] + ).unwrap(); pub static ref STORE_SLOW_TREND_GAUGE: Gauge = register_gauge!("tikv_raftstore_slow_trend", "Slow trend changing rate.").unwrap(); diff --git a/components/raftstore/src/store/mod.rs b/components/raftstore/src/store/mod.rs index cccab6f72b0..a6b3c05a4b4 100644 --- a/components/raftstore/src/store/mod.rs +++ b/components/raftstore/src/store/mod.rs @@ -88,6 +88,7 @@ pub use self::{ worker::{ metrics as worker_metrics, need_compact, AutoSplitController, BatchComponent, Bucket, BucketRange, BucketStatsInfo, CachedReadDelegate, CheckLeaderRunner, CheckLeaderTask, +<<<<<<< HEAD CompactThreshold, FlowStatistics, FlowStatsReporter, KeyEntry, LocalReadContext, LocalReader, LocalReaderCore, PdStatsMonitor, PdTask, ReadDelegate, ReadExecutor, ReadExecutorProvider, ReadProgress, ReadStats, RefreshConfigTask, RegionTask, @@ -96,5 +97,16 @@ pub use self::{ BIG_REGION_CPU_OVERLOAD_THRESHOLD_RATIO, DEFAULT_BIG_REGION_BYTE_THRESHOLD, DEFAULT_BIG_REGION_QPS_THRESHOLD, DEFAULT_BYTE_THRESHOLD, DEFAULT_QPS_THRESHOLD, NUM_COLLECT_STORE_INFOS_PER_HEARTBEAT, REGION_CPU_OVERLOAD_THRESHOLD_RATIO, +======= + CompactThreshold, DiskCheckRunner, FlowStatistics, FlowStatsReporter, + FullCompactController, KeyEntry, LocalReadContext, LocalReader, LocalReaderCore, + PdStatsMonitor, PdTask, ReadDelegate, ReadExecutor, ReadExecutorProvider, ReadProgress, + ReadStats, RefreshConfigTask, RegionTask, SnapGenTask, SplitCheckRunner, SplitCheckTask, + SplitConfig, SplitConfigManager, SplitInfo, StoreMetaDelegate, StoreStatsReporter, + TrackVer, WriteStats, WriterContoller, BIG_REGION_CPU_OVERLOAD_THRESHOLD_RATIO, + DEFAULT_BIG_REGION_BYTE_THRESHOLD, DEFAULT_BIG_REGION_QPS_THRESHOLD, + DEFAULT_BYTE_THRESHOLD, DEFAULT_QPS_THRESHOLD, NUM_COLLECT_STORE_INFOS_PER_HEARTBEAT, + REGION_CPU_OVERLOAD_THRESHOLD_RATIO, +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) }, }; diff --git a/components/raftstore/src/store/msg.rs b/components/raftstore/src/store/msg.rs index 1d01caa1c76..4f0f6ba7ec3 100644 --- a/components/raftstore/src/store/msg.rs +++ b/components/raftstore/src/store/msg.rs @@ -8,6 +8,10 @@ use std::{borrow::Cow, fmt}; use collections::HashSet; use engine_traits::{CompactedEvent, KvEngine, Snapshot}; use futures::channel::mpsc::UnboundedSender; +<<<<<<< HEAD +======= +use health_controller::types::{InspectFactor, LatencyInspector}; +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) use kvproto::{ brpb::CheckAdminResponse, kvrpcpb::{DiskFullOpt, ExtraOp as TxnExtraOp}, @@ -882,6 +886,7 @@ where /// Inspect the latency of raftstore. LatencyInspect { + factor: InspectFactor, send_time: Instant, inspector: LatencyInspector, }, diff --git a/components/raftstore/src/store/worker/disk_check.rs b/components/raftstore/src/store/worker/disk_check.rs new file mode 100644 index 00000000000..44c66892041 --- /dev/null +++ b/components/raftstore/src/store/worker/disk_check.rs @@ -0,0 +1,178 @@ +// Copyright 2024 TiKV Project Authors. Licensed under Apache-2.0. + +use std::{ + fmt::{self, Display, Formatter}, + io::Write, + path::PathBuf, + time::Duration, +}; + +use crossbeam::channel::{bounded, Receiver, Sender}; +use health_controller::types::LatencyInspector; +use tikv_util::{ + time::Instant, + warn, + worker::{Runnable, Worker}, +}; + +#[derive(Debug)] +pub enum Task { + InspectLatency { inspector: LatencyInspector }, +} + +impl Display for Task { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + match *self { + Task::InspectLatency { .. } => write!(f, "InspectLatency"), + } + } +} + +#[derive(Clone)] +/// A simple inspector to measure the latency of disk IO. +/// +/// This is used to measure the latency of disk IO, which is used to determine +/// the health status of the TiKV server. +/// The inspector writes a file to the disk and measures the time it takes to +/// complete the write operation. +pub struct Runner { + target: PathBuf, + notifier: Sender, + receiver: Receiver, + bg_worker: Option, +} + +impl Runner { + /// The filename to write to the disk to measure the latency. + const DISK_IO_LATENCY_INSPECT_FILENAME: &'static str = ".disk_latency_inspector.tmp"; + /// The content to write to the file to measure the latency. + const DISK_IO_LATENCY_INSPECT_FLUSH_STR: &'static [u8] = b"inspect disk io latency"; + + #[inline] + fn build(target: PathBuf) -> Self { + // The disk check mechanism only cares about the latency of the most + // recent request; older requests become stale and irrelevant. To avoid + // unnecessary accumulation of multiple requests, we set a small + // `capacity` for the disk check worker. + let (notifier, receiver) = bounded(3); + Self { + target, + notifier, + receiver, + bg_worker: None, + } + } + + #[inline] + pub fn new(inspect_dir: PathBuf) -> Self { + Self::build(inspect_dir.join(Self::DISK_IO_LATENCY_INSPECT_FILENAME)) + } + + #[inline] + /// Only for test. + /// Generate a dummy Runner. + pub fn dummy() -> Self { + Self::build(PathBuf::from("./").join(Self::DISK_IO_LATENCY_INSPECT_FILENAME)) + } + + #[inline] + pub fn bind_background_worker(&mut self, bg_worker: Worker) { + self.bg_worker = Some(bg_worker); + } + + fn inspect(&self) -> Option { + let mut file = std::fs::OpenOptions::new() + .create(true) + .write(true) + .truncate(true) + .open(&self.target) + .ok()?; + + let start = Instant::now(); + // Ignore the error + file.write_all(Self::DISK_IO_LATENCY_INSPECT_FLUSH_STR) + .ok()?; + file.sync_all().ok()?; + Some(start.saturating_elapsed()) + } + + fn execute(&self) { + if let Ok(task) = self.receiver.try_recv() { + match task { + Task::InspectLatency { mut inspector } => { + if let Some(latency) = self.inspect() { + inspector.record_apply_process(latency); + inspector.finish(); + } else { + warn!("failed to inspect disk io latency"); + } + } + } + } + } +} + +impl Runnable for Runner { + type Task = Task; + + fn run(&mut self, task: Task) { + // Send the task to the limited capacity channel. + if let Err(e) = self.notifier.try_send(task) { + warn!("failed to send task to disk check bg_worker: {:?}", e); + } else { + let runner = self.clone(); + if let Some(bg_worker) = self.bg_worker.as_ref() { + bg_worker.spawn_async_task(async move { + runner.execute(); + }); + } + } + } +} + +#[cfg(test)] +mod tests { + use tikv_util::worker::Builder; + + use super::*; + + #[test] + fn test_disk_check_runner() { + let background_worker = Builder::new("disk-check-worker") + .pending_capacity(256) + .create(); + let (tx, rx) = std::sync::mpsc::sync_channel(1); + let mut runner = Runner::dummy(); + runner.bind_background_worker(background_worker); + // Validate the disk check runner. + { + let tx_1 = tx.clone(); + let inspector = LatencyInspector::new( + 1, + Box::new(move |_, duration| { + let dur = duration.sum(); + tx_1.send(dur).unwrap(); + }), + ); + runner.run(Task::InspectLatency { inspector }); + let latency = rx.recv().unwrap(); + assert!(latency > Duration::from_secs(0)); + } + // Invalid bg_worker and out of capacity + { + runner.bg_worker = None; + for i in 2..=10 { + let tx_2 = tx.clone(); + let inspector = LatencyInspector::new( + i as u64, + Box::new(move |_, duration| { + let dur = duration.sum(); + tx_2.send(dur).unwrap(); + }), + ); + runner.run(Task::InspectLatency { inspector }); + rx.recv_timeout(Duration::from_secs(1)).unwrap_err(); + } + } + } +} diff --git a/components/raftstore/src/store/worker/mod.rs b/components/raftstore/src/store/worker/mod.rs index c6783238520..865326e1c4a 100644 --- a/components/raftstore/src/store/worker/mod.rs +++ b/components/raftstore/src/store/worker/mod.rs @@ -6,6 +6,7 @@ mod cleanup_snapshot; mod cleanup_sst; mod compact; mod consistency_check; +mod disk_check; pub mod metrics; mod pd; mod raftlog_gc; @@ -25,6 +26,7 @@ pub use self::{ cleanup_sst::{Runner as CleanupSstRunner, Task as CleanupSstTask}, compact::{need_compact, CompactThreshold, Runner as CompactRunner, Task as CompactTask}, consistency_check::{Runner as ConsistencyCheckRunner, Task as ConsistencyCheckTask}, + disk_check::{Runner as DiskCheckRunner, Task as DiskCheckTask}, pd::{ new_change_peer_v2_request, FlowStatistics, FlowStatsReporter, HeartbeatTask, Runner as PdRunner, StatsMonitor as PdStatsMonitor, StoreStatsReporter, Task as PdTask, diff --git a/components/raftstore/src/store/worker/pd.rs b/components/raftstore/src/store/worker/pd.rs index 62ccc0418cb..4c647f291e4 100644 --- a/components/raftstore/src/store/worker/pd.rs +++ b/components/raftstore/src/store/worker/pd.rs @@ -20,7 +20,15 @@ use concurrency_manager::ConcurrencyManager; use engine_traits::{KvEngine, RaftEngine}; use fail::fail_point; use futures::{compat::Future01CompatExt, FutureExt}; +<<<<<<< HEAD use grpcio_health::{HealthService, ServingStatus}; +======= +use health_controller::{ + reporters::{RaftstoreReporter, RaftstoreReporterConfig}, + types::{InspectFactor, LatencyInspector, RaftstoreDuration}, + HealthController, +}; +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) use kvproto::{ kvrpcpb::DiskFullOpt, metapb, pdpb, @@ -48,7 +56,7 @@ use tikv_util::{ topn::TopN, trend::{RequestPerSecRecorder, Trend}, warn, - worker::{Runnable, RunnableWithTimer, ScheduleError, Scheduler}, + worker::{Runnable, ScheduleError, Scheduler}, }; use txn_types::TimeStamp; use yatp::Remote; @@ -199,6 +207,7 @@ where }, UpdateSlowScore { id: u64, + factor: InspectFactor, duration: RaftstoreDuration, }, RegionCpuRecords(Arc), @@ -208,6 +217,9 @@ where }, ReportBuckets(BucketStat), ControlGrpcServer(pdpb::ControlGrpcEvent), + InspectLatency { + factor: InspectFactor, + }, } pub struct StoreStat { @@ -445,8 +457,16 @@ where Task::QueryRegionLeader { region_id } => { write!(f, "query the leader of region {}", region_id) } - Task::UpdateSlowScore { id, ref duration } => { - write!(f, "compute slow score: id {}, duration {:?}", id, duration) + Task::UpdateSlowScore { + id, + factor, + ref duration, + } => { + write!( + f, + "compute slow score: id {}, factor: {:?}, duration {:?}", + id, factor, duration + ) } Task::RegionCpuRecords(ref cpu_records) => { write!(f, "get region cpu records: {:?}", cpu_records) @@ -467,6 +487,9 @@ where Task::ControlGrpcServer(ref event) => { write!(f, "control grpc server: {:?}", event) } + Task::InspectLatency { factor } => { + write!(f, "inspect raftstore latency: {:?}", factor) + } } } } @@ -525,7 +548,7 @@ pub trait StoreStatsReporter: Send + Clone + Sync + 'static + Collector { ); fn report_min_resolved_ts(&self, store_id: u64, min_resolved_ts: u64); fn auto_split(&self, split_infos: Vec); - fn update_latency_stats(&self, timer_tick: u64); + fn update_latency_stats(&self, timer_tick: u64, factor: InspectFactor); } impl StoreStatsReporter for WrappedScheduler @@ -575,9 +598,16 @@ where } } - fn update_latency_stats(&self, timer_tick: u64) { - debug!("update latency statistics not implemented for raftstore-v1"; + fn update_latency_stats(&self, timer_tick: u64, factor: InspectFactor) { + debug!("update latency statistics for raftstore-v1"; "tick" => timer_tick); + let task = Task::InspectLatency { factor }; + if let Err(e) = self.0.schedule(task) { + error!( + "failed to send inspect raftstore latency task to pd worker"; + "err" => ?e, + ); + } } } @@ -595,6 +625,7 @@ where collect_tick_interval: Duration, report_min_resolved_ts_interval: Duration, inspect_latency_interval: Duration, + inspect_kvdb_latency_interval: Duration, } impl StatsMonitor @@ -603,8 +634,13 @@ where { pub fn new( interval: Duration, +<<<<<<< HEAD report_min_resolved_ts_interval: Duration, inspect_latency_interval: Duration, +======= + inspect_latency_interval: Duration, + inspect_kvdb_latency_interval: Duration, +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) reporter: T, ) -> Self { StatsMonitor { @@ -625,6 +661,7 @@ where cmp::min(default_collect_tick_interval(), interval), ), inspect_latency_interval, + inspect_kvdb_latency_interval, } } @@ -656,12 +693,21 @@ where let load_base_split_check_interval = self .load_base_split_check_interval .div_duration_f64(tick_interval) as u64; +<<<<<<< HEAD let report_min_resolved_ts_interval = self .report_min_resolved_ts_interval .div_duration_f64(tick_interval) as u64; let update_latency_stats_interval = self .inspect_latency_interval .div_duration_f64(tick_interval) as u64; +======= + let update_raftdisk_latency_stats_interval = + self.inspect_latency_interval + .div_duration_f64(tick_interval) as u64; + let update_kvdisk_latency_stats_interval = + self.inspect_kvdb_latency_interval + .div_duration_f64(tick_interval) as u64; +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) let (timer_tx, timer_rx) = mpsc::channel(); self.timer = Some(timer_tx); @@ -722,6 +768,7 @@ where &mut region_cpu_records_collector, ); } +<<<<<<< HEAD if is_enable_tick(timer_cnt, report_min_resolved_ts_interval) { reporter.report_min_resolved_ts( store_id, @@ -730,6 +777,13 @@ where } if is_enable_tick(timer_cnt, update_latency_stats_interval) { reporter.update_latency_stats(timer_cnt); +======= + if is_enable_tick(timer_cnt, update_raftdisk_latency_stats_interval) { + reporter.update_latency_stats(timer_cnt, InspectFactor::RaftDisk); + } + if is_enable_tick(timer_cnt, update_kvdisk_latency_stats_interval) { + reporter.update_latency_stats(timer_cnt, InspectFactor::KvDisk); +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) } timer_cnt += 1; } @@ -1027,6 +1081,7 @@ where interval, cfg.report_min_resolved_ts_interval.0, cfg.inspect_interval.0, + cfg.inspect_kvdb_interval.0, WrappedScheduler(scheduler.clone()), ); if let Err(e) = stats_monitor.start( @@ -1038,6 +1093,37 @@ where error!("failed to start stats collector, error = {:?}", e); } +<<<<<<< HEAD +======= + let health_reporter_config = RaftstoreReporterConfig { + inspect_interval: cfg.inspect_interval.0, + inspect_kvdb_interval: cfg.inspect_kvdb_interval.0, + + unsensitive_cause: cfg.slow_trend_unsensitive_cause, + unsensitive_result: cfg.slow_trend_unsensitive_result, + net_io_factor: cfg.slow_trend_network_io_factor, + + cause_spike_filter_value_gauge: STORE_SLOW_TREND_MISC_GAUGE_VEC + .with_label_values(&["spike_filter_value"]), + cause_spike_filter_count_gauge: STORE_SLOW_TREND_MISC_GAUGE_VEC + .with_label_values(&["spike_filter_count"]), + cause_l1_gap_gauges: STORE_SLOW_TREND_MARGIN_ERROR_WINDOW_GAP_GAUGE_VEC + .with_label_values(&["L1"]), + cause_l2_gap_gauges: STORE_SLOW_TREND_MARGIN_ERROR_WINDOW_GAP_GAUGE_VEC + .with_label_values(&["L2"]), + result_spike_filter_value_gauge: STORE_SLOW_TREND_RESULT_MISC_GAUGE_VEC + .with_label_values(&["spike_filter_value"]), + result_spike_filter_count_gauge: STORE_SLOW_TREND_RESULT_MISC_GAUGE_VEC + .with_label_values(&["spike_filter_count"]), + result_l1_gap_gauges: STORE_SLOW_TREND_RESULT_MARGIN_ERROR_WINDOW_GAP_GAUGE_VEC + .with_label_values(&["L1"]), + result_l2_gap_gauges: STORE_SLOW_TREND_RESULT_MARGIN_ERROR_WINDOW_GAP_GAUGE_VEC + .with_label_values(&["L2"]), + }; + + let health_reporter = RaftstoreReporter::new(&health_controller, health_reporter_config); + +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) Runner { store_id, pd_client, @@ -2052,6 +2138,89 @@ where } } } + + fn handle_inspect_latency(&mut self, factor: InspectFactor) { + let slow_score_tick_result = self + .health_reporter + .tick(self.store_stat.maybe_busy(), factor); + if let Some(score) = slow_score_tick_result.updated_score { + STORE_SLOW_SCORE_GAUGE + .with_label_values(&[factor.as_str()]) + .set(score as i64); + } + let id = slow_score_tick_result.tick_id; + let scheduler = self.scheduler.clone(); + let inspector = { + match factor { + InspectFactor::RaftDisk => { + // If the last slow_score already reached abnormal state and was delayed for + // reporting by `store-heartbeat` to PD, we should report it here manually as + // a FAKE `store-heartbeat`. + if slow_score_tick_result.should_force_report_slow_store + && self.is_store_heartbeat_delayed() + { + self.handle_fake_store_heartbeat(); + } + LatencyInspector::new( + id, + Box::new(move |id, duration| { + STORE_INSPECT_DURATION_HISTOGRAM + .with_label_values(&["store_wait"]) + .observe(tikv_util::time::duration_to_sec( + duration.store_wait_duration.unwrap_or_default(), + )); + STORE_INSPECT_DURATION_HISTOGRAM + .with_label_values(&["store_commit"]) + .observe(tikv_util::time::duration_to_sec( + duration.store_commit_duration.unwrap_or_default(), + )); + + STORE_INSPECT_DURATION_HISTOGRAM + .with_label_values(&["all"]) + .observe(tikv_util::time::duration_to_sec(duration.sum())); + if let Err(e) = scheduler.schedule(Task::UpdateSlowScore { + id, + factor, + duration, + }) { + warn!("schedule pd task failed"; "err" => ?e); + } + }), + ) + } + InspectFactor::KvDisk => LatencyInspector::new( + id, + Box::new(move |id, duration| { + STORE_INSPECT_DURATION_HISTOGRAM + .with_label_values(&["apply_wait"]) + .observe(tikv_util::time::duration_to_sec( + duration.apply_wait_duration.unwrap_or_default(), + )); + STORE_INSPECT_DURATION_HISTOGRAM + .with_label_values(&["apply_process"]) + .observe(tikv_util::time::duration_to_sec( + duration.apply_process_duration.unwrap_or_default(), + )); + if let Err(e) = scheduler.schedule(Task::UpdateSlowScore { + id, + factor, + duration, + }) { + warn!("schedule pd task failed"; "err" => ?e); + } + }), + ), + } + }; + let msg = StoreMsg::LatencyInspect { + factor, + send_time: TiInstant::now(), + inspector, + }; + if let Err(e) = self.router.send_control(msg) { + warn!("pd worker send latency inspecter failed"; "err" => ?e); + } + } } fn calculate_region_cpu_records( @@ -2295,11 +2464,23 @@ where txn_ext, } => self.handle_update_max_timestamp(region_id, initial_status, txn_ext), Task::QueryRegionLeader { region_id } => self.handle_query_region_leader(region_id), +<<<<<<< HEAD Task::UpdateSlowScore { id, duration } => { // Fine-tuned, `SlowScore` only takes the I/O jitters on the disk into account. self.slow_score.record( id, duration.delays_on_disk_io(false), +======= + Task::UpdateSlowScore { + id, + factor, + duration, + } => { + self.health_reporter.record_raftstore_duration( + id, + factor, + duration, +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) !self.store_stat.maybe_busy(), ); } @@ -2314,6 +2495,9 @@ where Task::ControlGrpcServer(event) => { self.handle_control_grpc_server(event); } + Task::InspectLatency { factor } => { + self.handle_inspect_latency(factor); + } }; } @@ -2322,6 +2506,7 @@ where } } +<<<<<<< HEAD impl RunnableWithTimer for Runner where EK: KvEngine, @@ -2409,6 +2594,8 @@ where } } +======= +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) fn new_change_peer_request(change_type: ConfChangeType, peer: metapb::Peer) -> AdminRequest { let mut req = AdminRequest::default(); req.set_cmd_type(AdminCmdType::ChangePeer); @@ -2700,6 +2887,7 @@ mod tests { Duration::from_secs(interval), Duration::from_secs(0), Duration::from_secs(interval), + Duration::default(), WrappedScheduler(scheduler), ); let store_meta = Arc::new(Mutex::new(StoreMeta::new(0))); @@ -3006,6 +3194,7 @@ mod tests { Duration::from_secs(interval), Duration::from_secs(0), Duration::from_secs(interval), + Duration::default(), WrappedScheduler(pd_worker.scheduler()), ); let store_meta = Arc::new(Mutex::new(StoreMeta::new(0))); diff --git a/components/server/src/server.rs b/components/server/src/server.rs index 7a1108e54c2..fc77e2465dc 100644 --- a/components/server/src/server.rs +++ b/components/server/src/server.rs @@ -64,8 +64,8 @@ use raftstore::{ }, memory::MEMTRACE_ROOT as MEMTRACE_RAFTSTORE, snapshot_backup::PrepareDiskSnapObserver, - AutoSplitController, CheckLeaderRunner, LocalReader, SnapManager, SnapManagerBuilder, - SplitCheckRunner, SplitConfigManager, StoreMetaDelegate, + AutoSplitController, CheckLeaderRunner, DiskCheckRunner, LocalReader, SnapManager, + SnapManagerBuilder, SplitCheckRunner, SplitConfigManager, StoreMetaDelegate, }, RaftRouterCompactedEventSender, }; @@ -772,6 +772,13 @@ where let server_config = Arc::new(VersionTrack::new(self.core.config.server.clone())); self.core.config.raft_store.optimize_for(false); + self.core + .config + .raft_store + .optimize_inspector(path_in_diff_mount_point( + engines.engines.raft.get_engine_path().to_string().as_str(), + engines.engines.kv.path(), + )); self.core .config .raft_store @@ -975,6 +982,7 @@ where .registry .register_consistency_check_observer(100, observer); +<<<<<<< HEAD node.start( engines.engines.clone(), server.transport(), @@ -992,6 +1000,29 @@ where safe_point.clone(), ) .unwrap_or_else(|e| fatal!("failed to start node: {}", e)); +======= + let disk_check_runner = DiskCheckRunner::new(self.core.store_path.clone()); + + raft_server + .start( + engines.engines.clone(), + server.transport(), + snap_mgr, + pd_worker, + engines.store_meta.clone(), + self.coprocessor_host.clone().unwrap(), + importer.clone(), + split_check_scheduler, + auto_split_controller, + self.concurrency_manager.clone(), + collector_reg_handle, + self.causal_ts_provider.clone(), + disk_check_runner, + self.grpc_service_mgr.clone(), + safe_point.clone(), + ) + .unwrap_or_else(|e| fatal!("failed to start raft_server: {}", e)); +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) // Start auto gc. Must after `Node::start` because `node_id` is initialized // there. diff --git a/components/test_raftstore/src/node.rs b/components/test_raftstore/src/node.rs index f429f27ff8b..6b743a62e1b 100644 --- a/components/test_raftstore/src/node.rs +++ b/components/test_raftstore/src/node.rs @@ -334,6 +334,7 @@ impl Simulator for NodeCluster { cm, CollectorRegHandle::new_for_test(), None, + DiskCheckRunner::dummy(), GrpcServiceManager::dummy(), Arc::new(AtomicU64::new(0)), )?; diff --git a/components/test_raftstore/src/server.rs b/components/test_raftstore/src/server.rs index f5c64fa86e9..a6673ebb66e 100644 --- a/components/test_raftstore/src/server.rs +++ b/components/test_raftstore/src/server.rs @@ -37,8 +37,9 @@ use raftstore::{ store::{ fsm::{store::StoreMeta, ApplyRouter, RaftBatchSystem, RaftRouter}, msg::RaftCmdExtraOpts, - AutoSplitController, Callback, CheckLeaderRunner, LocalReader, RegionSnapshot, SnapManager, - SnapManagerBuilder, SplitCheckRunner, SplitConfigManager, StoreMetaDelegate, + AutoSplitController, Callback, CheckLeaderRunner, DiskCheckRunner, LocalReader, + RegionSnapshot, SnapManager, SnapManagerBuilder, SplitCheckRunner, SplitConfigManager, + StoreMetaDelegate, }, Result, }; @@ -617,6 +618,7 @@ impl ServerCluster { concurrency_manager.clone(), collector_reg_handle, causal_ts_provider, + DiskCheckRunner::dummy(), GrpcServiceManager::dummy(), Arc::new(AtomicU64::new(0)), )?; diff --git a/metrics/grafana/tikv_details.dashboard.py b/metrics/grafana/tikv_details.dashboard.py new file mode 100644 index 00000000000..95b915d6173 --- /dev/null +++ b/metrics/grafana/tikv_details.dashboard.py @@ -0,0 +1,10069 @@ +import os +import sys + +sys.path.append(os.path.dirname(__file__)) + +from common import ( + DATASOURCE, + DATASOURCE_INPUT, + Layout, + expr_avg, + expr_count_rate, + expr_histogram_avg, + expr_histogram_quantile, + expr_max, + expr_max_rate, + expr_min, + expr_operator, + expr_simple, + expr_sum, + expr_sum_aggr_over_time, + expr_sum_delta, + expr_sum_increase, + expr_sum_rate, + expr_topk, + graph_legend, + graph_panel, + graph_panel_histogram_quantiles, + heatmap_panel, + heatmap_panel_graph_panel_histogram_quantile_pairs, + series_override, + stat_panel, + target, + template, + yaxes, + yaxis, +) +from grafanalib import formatunits as UNITS +from grafanalib.core import ( + GRAPH_TOOLTIP_MODE_SHARED_CROSSHAIR, + HIDE_VARIABLE, + NULL_AS_NULL, + SHOW, + Dashboard, + GraphThreshold, + RowPanel, + StatValueMappingItem, + StatValueMappings, + Templating, +) + +#### Metrics Definition Start #### + + +def Templates() -> Templating: + return Templating( + list=[ + template( + name="k8s_cluster", + type="query", + query="label_values(tikv_engine_block_cache_size_bytes, k8s_cluster)", + data_source=DATASOURCE, + hide=HIDE_VARIABLE, + ), + template( + name="tidb_cluster", + type="query", + query='label_values(tikv_engine_block_cache_size_bytes{k8s_cluster ="$k8s_cluster"}, tidb_cluster)', + data_source=DATASOURCE, + hide=HIDE_VARIABLE, + ), + template( + name="db", + type="query", + query='label_values(tikv_engine_block_cache_size_bytes{k8s_cluster="$k8s_cluster", tidb_cluster="$tidb_cluster"}, db)', + data_source=DATASOURCE, + hide=SHOW, + multi=True, + include_all=True, + ), + template( + name="command", + type="query", + query='query_result(tikv_storage_command_total{k8s_cluster="$k8s_cluster", tidb_cluster="$tidb_cluster"} != 0)', + data_source=DATASOURCE, + hide=SHOW, + regex='/\\btype="([^"]+)"/', + multi=True, + include_all=True, + ), + template( + name="instance", + type="query", + query='label_values(tikv_engine_size_bytes{k8s_cluster ="$k8s_cluster", tidb_cluster="$tidb_cluster"}, instance)', + data_source=DATASOURCE, + hide=SHOW, + multi=True, + include_all=True, + all_value=".*", + ), + template( + name="titan_db", + type="query", + query='label_values(tikv_engine_titandb_num_live_blob_file{k8s_cluster="$k8s_cluster", tidb_cluster="$tidb_cluster"}, db)', + data_source=DATASOURCE, + hide=HIDE_VARIABLE, + multi=True, + include_all=True, + ), + template( + name="additional_groupby", + type="custom", + query="none,instance", + data_source=DATASOURCE, + hide=SHOW, + ), + ] + ) + + +def Duration() -> RowPanel: + layout = Layout(title="Duration") + layout.row( + [ + graph_panel( + title="Write Pipeline Duration", + description="Write Pipeline Composition", + yaxes=yaxes(left_format=UNITS.SECONDS), + lines=False, + stack=True, + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, "tikv_raftstore_append_log_duration_seconds" + ), + legend_format="Write Raft Log .99", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_raftstore_request_wait_time_duration_secs", + ), + legend_format="Propose Wait .99", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.99, "tikv_raftstore_apply_wait_time_duration_secs" + ), + legend_format="Apply Wait .99", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.99, "tikv_raftstore_commit_log_duration_seconds" + ), + legend_format="Replicate Raft Log .99", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.99, "tikv_raftstore_apply_log_duration_seconds" + ), + legend_format="Apply Duration .99", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Cop Read Duration", + description="Read Duration Composition", + yaxes=yaxes(left_format=UNITS.SECONDS), + lines=False, + stack=True, + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_storage_engine_async_request_duration_seconds", + ['type="snapshot"'], + ), + legend_format="Get Snapshot .99", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_coprocessor_request_wait_seconds", + ['type="all"'], + ), + legend_format="Cop Wait .99", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.95, + "tikv_coprocessor_request_handle_seconds", + ), + legend_format="Cop Handle .99", + additional_groupby=True, + ), + ], + ), + ] + ) + return layout.row_panel + + +def Cluster() -> RowPanel: + layout = Layout(title="Cluster") + layout.row( + [ + graph_panel( + title="Store size", + description="The storage size per TiKV instance", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + fill=1, + stack=True, + legend=graph_legend(max=False), + targets=[ + target( + expr=expr_sum( + "tikv_store_size_bytes", + label_selectors=['type = "used"'], + ), + ), + ], + ), + graph_panel( + title="Available size", + description="The available capacity size of each TiKV instance", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + fill=1, + stack=True, + legend=graph_legend(max=False), + targets=[ + target( + expr=expr_sum( + "tikv_store_size_bytes", + label_selectors=['type="available"'], + ), + ), + ], + ), + graph_panel( + title="Capacity size", + description="The capacity size per TiKV instance", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + fill=1, + stack=True, + legend=graph_legend(max=False), + targets=[ + target( + expr=expr_sum( + "tikv_store_size_bytes", + label_selectors=['type="capacity"'], + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="CPU", + description="The CPU usage of each TiKV instance", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "process_cpu_seconds_total", + label_selectors=['job=~".*tikv"'], + ), + ), + target( + expr=expr_simple( + "tikv_server_cpu_cores_quota", + label_selectors=['job=~".*tikv"'], + ), + legend_format=r"quota-{{instance}}", + ), + ], + ), + graph_panel( + title="Memory", + description="The memory usage per TiKV instance", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum( + "process_resident_memory_bytes", + label_selectors=['job=~".*tikv"'], + ), + ), + target( + expr=expr_simple( + "tikv_server_memory_quota_bytes", + label_selectors=['job=~".*tikv"'], + ), + legend_format=r"quota-{{instance}}", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="IO utilization", + description="The I/O utilization per TiKV instance", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "node_disk_io_time_seconds_total", + ), + legend_format=r"{{instance}}-{{device}}", + ), + ], + ), + graph_panel( + title="MBps", + description="The total bytes of read and write in each TiKV instance", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_flow_bytes", + label_selectors=['type="wal_file_bytes"'], + ), + legend_format=r"{{instance}}-write", + ), + target( + expr=expr_sum_rate( + "tikv_engine_flow_bytes", + label_selectors=['type=~"bytes_read|iter_bytes_read"'], + ), + legend_format=r"{{instance}}-read", + ), + target( + expr=expr_sum_rate( + "tikv_in_memory_engine_flow", + label_selectors=['type=~"bytes_read|iter_bytes_read"'], + ), + legend_format=r"{{instance}}-in-memory-engine-read", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="QPS", + description="The number of leaders on each TiKV instance", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_grpc_msg_duration_seconds_count", + label_selectors=['type!="kv_gc"'], + by_labels=["instance", "type"], + ), + legend_format=r"{{instance}}-{{type}}", + ), + ], + ), + graph_panel( + title="Errps", + description="The total number of the gRPC message failures", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_grpc_msg_fail_total", + label_selectors=['type!="kv_gc"'], + ), + legend_format=r"{{instance}}-grpc-msg-fail", + ), + target( + expr=expr_sum_delta( + "tikv_pd_heartbeat_message_total", + label_selectors=['type="noop"'], + ).extra(extra_expr="< 1"), + legend_format=r"{{instance}}-pd-heartbeat", + ), + target( + expr=expr_sum_rate( + "tikv_critical_error_total", + by_labels=["instance", "type"], + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Leader", + description="The number of leaders on each TiKV instance", + targets=[ + target( + expr=expr_sum( + "tikv_raftstore_region_count", + label_selectors=['type="leader"'], + ), + ), + ], + ), + graph_panel( + title="Region", + description="The number of Regions and Buckets on each TiKV instance", + targets=[ + target( + expr=expr_sum( + "tikv_raftstore_region_count", + label_selectors=['type="region"'], + ), + ), + target( + expr=expr_sum( + "tikv_raftstore_region_count", + label_selectors=['type="buckets"'], + ), + legend_format=r"{{instance}}-buckets", + hide=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Uptime", + description="TiKV uptime since the last restart", + yaxes=yaxes(left_format=UNITS.SECONDS, log_base=2), + targets=[ + target( + expr=expr_operator( + "time()", + "-", + expr_simple( + "process_start_time_seconds", + label_selectors=['job=~".*tikv"'], + ), + ), + legend_format=r"{{instance}}", + ), + ], + ) + ] + ) + return layout.row_panel + + +def Errors() -> RowPanel: + layout = Layout(title="Errors") + layout.row( + [ + graph_panel( + title="Critical error", + targets=[ + target( + expr=expr_sum_rate( + "tikv_critical_error_total", + by_labels=["instance", "type"], + ), + ), + ], + thresholds=[GraphThreshold(value=0.0)], + ) + ] + ) + layout.row( + [ + graph_panel( + title="Server is busy", + description=""" +Indicates occurrences of events that make the TiKV instance unavailable +temporarily, such as Write Stall, Channel Full, Scheduler Busy, and Coprocessor +Full""", + targets=[ + target( + expr=expr_sum_rate( + "tikv_scheduler_too_busy_total", + ), + legend_format=r"scheduler-{{instance}}", + ), + target( + expr=expr_sum_rate( + "tikv_channel_full_total", + by_labels=["instance", "type"], + ), + legend_format=r"channelfull-{{instance}}-{{type}}", + ), + target( + expr=expr_sum_rate( + "tikv_coprocessor_request_error", + label_selectors=['type="full"'], + ), + legend_format=r"coprocessor-{{instance}}", + ), + target( + expr=expr_avg( + "tikv_engine_write_stall", + label_selectors=[ + 'type="write_stall_percentile99"', + 'db=~"$db"', + ], + by_labels=["instance", "db"], + ), + legend_format=r"stall-{{instance}}-{{db}}", + ), + target( + expr=expr_sum_rate( + "tikv_raftstore_store_write_msg_block_wait_duration_seconds_count", + ), + legend_format=r"store-write-channelfull-{{instance}}", + ), + target( + expr=expr_sum( + "tikv_raftstore_process_busy", + by_labels=["instance", "type"], + ), + legend_format=r"{{instance}}-{{type}}", + ), + ], + ), + graph_panel( + title="Server report failures", + description="The total number of reporting failure messages", + targets=[ + target( + expr=expr_sum_rate( + "tikv_server_report_failure_msg_total", + by_labels=["type", "instance", "store_id"], + ), + legend_format=r"{{instance}}-{{type}}-to-{{store_id}}", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Raftstore error", + description="The number of different raftstore errors on each TiKV instance", + targets=[ + target( + expr=expr_sum_rate( + "tikv_storage_engine_async_request_total", + label_selectors=['status!~"success|all"'], + by_labels=["instance", "status"], + ), + ), + ], + ), + graph_panel( + title="Scheduler error", + description="The number of scheduler errors per type on each TiKV instance", + targets=[ + target( + expr=expr_sum_rate( + "tikv_scheduler_stage_total", + label_selectors=['stage=~"snapshot_err|prepare_write_err"'], + by_labels=["instance", "stage"], + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Coprocessor error", + description="The number of different coprocessor errors on each TiKV instance", + targets=[ + target( + expr=expr_sum_rate( + "tikv_coprocessor_request_error", + by_labels=["instance", "reason"], + ), + ), + ], + ), + graph_panel( + title="gRPC message error", + description="The number of gRPC message errors per type on each TiKV instance", + targets=[ + target( + expr=expr_sum_rate( + "tikv_grpc_msg_fail_total", + by_labels=["instance", "type"], + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Leader drop", + description="The count of dropped leaders per TiKV instance", + targets=[ + target( + expr=expr_sum_delta( + "tikv_raftstore_region_count", + label_selectors=['type="leader"'], + ), + ), + ], + ), + graph_panel( + title="Leader missing", + description="The count of missing leaders per TiKV instance", + targets=[ + target( + expr=expr_sum( + "tikv_raftstore_leader_missing", + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Damaged files", + description="RocksDB damaged SST files", + targets=[ + target( + expr=expr_simple("tikv_rocksdb_damaged_files"), + legend_format=r"{{instance}}-existed", + ), + target( + expr=expr_simple("tikv_rocksdb_damaged_files_deleted"), + legend_format=r"{{instance}}-deleted", + ), + ], + ), + graph_panel( + title="Log Replication Rejected", + description="The count of Log Replication Reject caused by follower memory insufficient", + targets=[ + target( + expr=expr_sum_rate( + "tikv_server_raft_append_rejects", + ), + ), + ], + ), + ] + ) + return layout.row_panel + + +def Server() -> RowPanel: + layout = Layout(title="Server") + layout.row( + [ + graph_panel( + title="CF size", + description="The size of each column family", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum( + "tikv_engine_size_bytes", + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Channel full", + description="The total number of channel full errors on each TiKV instance", + targets=[ + target( + expr=expr_sum_rate( + "tikv_channel_full_total", by_labels=["instance", "type"] + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Active written leaders", + description="The number of leaders being written on each TiKV instance", + targets=[ + target( + expr=expr_sum_rate( + "tikv_region_written_keys_count", + ), + ), + ], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Approximate region size", + metric="tikv_raftstore_region_size_bucket", + yaxis=yaxis(format=UNITS.BYTES_IEC), + ), + graph_panel_histogram_quantiles( + title="Approximate region size", + description="The approximate Region size", + metric="tikv_raftstore_region_size", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + hide_count=True, + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Region written bytes", + metric="tikv_region_written_bytes_bucket", + yaxis=yaxis(format=UNITS.BYTES_IEC), + ), + graph_panel( + title="Region average written bytes", + description="The average rate of writing bytes to Regions per TiKV instance", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_histogram_avg("tikv_region_written_bytes"), + legend_format="{{instance}}", + ), + ], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Region written keys", + metric="tikv_region_written_keys_bucket", + ), + graph_panel( + title="Region average written keys", + description="The average rate of written keys to Regions per TiKV instance", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_histogram_avg("tikv_region_written_keys"), + legend_format="{{instance}}", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel_histogram_quantiles( + title="Clear overlap region duration", + description="Bucketed histogram of clear overlap region duration.", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_raftstore_clear_overlap_region_duration_seconds", + by_labels=["type"], + hide_count=True, + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Written key size", + description="The key size for every put of apply worker", + metric="tikv_raftstore_apply_key_size_bucket", + yaxis=yaxis(format=UNITS.BYTES_IEC), + ), + heatmap_panel( + title="Written value size for every put of apply worker", + metric="tikv_raftstore_apply_value_size_bucket", + yaxis=yaxis(format=UNITS.BYTES_IEC), + ), + ] + ) + layout.row( + [ + graph_panel( + title="Hibernate Peers", + description="The number of peers in hibernated state", + targets=[ + target( + expr=expr_sum( + "tikv_raftstore_hibernated_peer_state", + by_labels=["instance", "state"], + ), + ), + ], + ), + graph_panel( + title="Memory trace", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_simple( + "tikv_server_mem_trace_sum", + label_selectors=['name=~"raftstore-.*"'], + ), + legend_format="{{instance}}-{{name}}", + ), + target( + expr=expr_simple( + "raft_engine_memory_usage", + ), + legend_format="{{instance}}-raft-engine", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Raft Entry Cache Evicts", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_raft_entries_evict_bytes", + ), + ), + ], + ), + graph_panel( + title="Resolve address duration", + yaxes=yaxes(left_format=UNITS.SECONDS), + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_server_address_resolve_duration_secs", + by_labels=["instance"], + ), + legend_format="{{instance}}", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="99% Thread Pool Schedule Wait Duration", + yaxes=yaxes(left_format=UNITS.SECONDS, log_base=2), + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_yatp_pool_schedule_wait_duration", + by_labels=["name", "priority"], + ), + legend_format="{{name}}-{{priority}}", + additional_groupby=True, + ), + ], + thresholds=[GraphThreshold(value=1.0)], + ), + graph_panel( + title="Average Thread Pool Schedule Wait Duration", + description="The average rate of written keys to Regions per TiKV instance", + yaxes=yaxes(left_format=UNITS.SECONDS, log_base=2), + targets=[ + target( + expr=expr_histogram_avg( + "tikv_yatp_pool_schedule_wait_duration", + by_labels=["name", "priority"], + ), + legend_format="{{name}}-{{priority}}", + additional_groupby=True, + ), + ], + thresholds=[GraphThreshold(value=1.0)], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Disk IO time per second", + yaxes=yaxes(left_format=UNITS.NANO_SECONDS), + lines=False, + stack=True, + targets=[ + target( + expr=expr_sum_rate( + "tikv_storage_rocksdb_perf", + label_selectors=['metric="block_read_time"'], + by_labels=["req"], + ), + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_coprocessor_rocksdb_perf", + label_selectors=['metric="block_read_time"'], + by_labels=["req"], + ), + legend_format="copr-{{req}}", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Disk IO bytes per second", + yaxes=yaxes(left_format=UNITS.NANO_SECONDS), + lines=False, + stack=True, + targets=[ + target( + expr=expr_sum_rate( + "tikv_storage_rocksdb_perf", + label_selectors=['metric="block_read_byte"'], + by_labels=["req"], + ), + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_coprocessor_rocksdb_perf", + label_selectors=['metric="block_read_byte"'], + by_labels=["req"], + ), + legend_format="copr-{{req}}", + additional_groupby=True, + ), + ], + ), + ] + ) + return layout.row_panel + + +def gRPC() -> RowPanel: + layout = Layout(title="gRPC") + layout.row( + [ + graph_panel( + title="gRPC message count", + description="The count of different kinds of gRPC message", + yaxes=yaxes(left_format=UNITS.REQUESTS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_grpc_msg_duration_seconds_count", + label_selectors=['type!="kv_gc"'], + by_labels=["type"], + ), + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_grpc_msg_duration_seconds_count", + label_selectors=['type!="kv_gc"'], + by_labels=["type", "priority"], + ), + hide=True, + additional_groupby=True, + ), + ], + ), + graph_panel( + title="gRPC message failed", + description="The count of different kinds of gRPC message which is failed", + yaxes=yaxes(left_format=UNITS.REQUESTS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_grpc_msg_fail_total", + label_selectors=['type!="kv_gc"'], + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title=r"99% gRPC message duration", + description=r"The 99% percentile of execution time of gRPC message", + yaxes=yaxes(left_format=UNITS.SECONDS, log_base=2), + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_grpc_msg_duration_seconds", + label_selectors=['type!="kv_gc"'], + by_labels=["type"], + ), + legend_format="{{type}}", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_grpc_msg_duration_seconds", + label_selectors=['type!="kv_gc"'], + by_labels=["type", "priority"], + ), + legend_format="{{type}}-{{priority}}", + hide=True, + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Average gRPC message duration", + description="The average execution time of gRPC message", + yaxes=yaxes(left_format=UNITS.SECONDS, log_base=2), + targets=[ + target( + expr=expr_histogram_avg( + "tikv_grpc_msg_duration_seconds", + by_labels=["type"], + ), + legend_format="{{type}}", + additional_groupby=True, + ), + target( + expr=expr_histogram_avg( + "tikv_grpc_msg_duration_seconds", + by_labels=["type", "priority"], + ), + legend_format="{{type}}-{{priority}}", + hide=True, + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="gRPC batch size", + description=r"The 99% percentile of execution time of gRPC message", + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_server_grpc_req_batch_size", + ), + legend_format=r"99% request", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_server_grpc_resp_batch_size", + ), + legend_format=r"99% response", + additional_groupby=True, + ), + target( + expr=expr_histogram_avg( + "tikv_server_grpc_req_batch_size", + by_labels=[], # override default by instance. + ), + legend_format="avg request", + additional_groupby=True, + ), + target( + expr=expr_histogram_avg( + "tikv_server_grpc_resp_batch_size", + by_labels=[], # override default by instance. + ), + legend_format="avg response", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_server_request_batch_size", + ), + legend_format=r"99% kv get batch", + additional_groupby=True, + ), + target( + expr=expr_histogram_avg( + "tikv_server_request_batch_size", + by_labels=[], # override default by instance. + ), + legend_format="avg kv batch", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="raft message batch size", + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_server_raft_message_batch_size", + ), + legend_format=r"99%", + additional_groupby=True, + ), + target( + expr=expr_histogram_avg( + "tikv_server_raft_message_batch_size", + by_labels=[], # override default by instance. + ), + legend_format="avg", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="gRPC request sources QPS", + description="The QPS of different sources of gRPC request", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_grpc_request_source_counter_vec", + by_labels=["source"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="gRPC request sources duration", + description="The duration of different sources of gRPC request", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS), + lines=False, + stack=True, + targets=[ + target( + expr=expr_sum_rate( + "tikv_grpc_request_source_duration_vec", + by_labels=["source"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="gRPC resource group QPS", + description="The QPS of different resource groups of gRPC request", + targets=[ + target( + expr=expr_sum_rate( + "tikv_grpc_resource_group_total", by_labels=["name"] + ), + additional_groupby=True, + ), + ], + ), + ] + ) + return layout.row_panel + + +def ThreadCPU() -> RowPanel: + layout = Layout(title="Thread CPU") + layout.row( + [ + graph_panel( + title="Raft store CPU", + description="The CPU utilization of raftstore thread", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"(raftstore|rs)_.*"'], + ), + ), + ], + ), + graph_panel( + title="Async apply CPU", + description="The CPU utilization of async apply", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"apply_[0-9]+"'], + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Store writer CPU", + description="The CPU utilization of store writer thread", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"store_write.*"'], + ), + ), + ], + thresholds=[GraphThreshold(value=0.8)], + ), + graph_panel( + title="gRPC poll CPU", + description="The CPU utilization of gRPC", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"grpc.*"'], + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Scheduler worker CPU", + description="The CPU utilization of scheduler worker", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"sched_.*"'], + ), + ), + ], + thresholds=[GraphThreshold(value=3.6)], + ), + graph_panel( + title="Unified read pool CPU", + description="The CPU utilization of the unified read pool", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"unified_read_po.*"'], + ), + ), + ], + thresholds=[GraphThreshold(value=7.2)], + ), + ] + ) + layout.row( + [ + graph_panel( + title="RocksDB CPU", + description="The CPU utilization of RocksDB", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"rocksdb.*"'], + ), + ), + ], + ), + graph_panel( + title="GC worker CPU", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"gc_worker.*"'], + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Region worker CPU", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"region_worker.*"'], + ), + ), + ], + ), + graph_panel( + title="Snap generator CPU", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"snap_generator.*"'], + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Background worker CPU", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"background.*"'], + ), + ), + ], + ), + graph_panel( + title="Raftlog fetch worker CPU", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"raftlog_fetch.*"'], + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Import CPU", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"sst_.*"'], + ), + ), + ], + ), + graph_panel( + title="Backup CPU", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=[ + 'name=~"(backup-worker|bkwkr|backup_endpoint).*"' + ], + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="CDC worker CPU", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"cdcwkr.*"'], + ), + legend_format="{{instance}}-worker", + ), + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"tso"'], + ), + legend_format="{{instance}}-tso", + ), + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"cdc_.*"'], + ), + legend_format="{{instance}}-endpoint", + ), + ], + ), + graph_panel( + title="TSO worker CPU", + description="The CPU utilization of raftstore thread", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"tso_worker"'], + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Storage read pool CPU", + description="The CPU utilization of storage read pool", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"store_read_norm.*"'], + ), + legend_format="{{instance}}-normal", + ), + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"store_read_high.*"'], + ), + legend_format="{{instance}}-high", + ), + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"store_read_low.*"'], + ), + legend_format="{{instance}}-low", + ), + ], + thresholds=[GraphThreshold(value=3.6)], + ), + graph_panel( + title="Coprocessor read pool CPU", + description="The CPU utilization of coprocessor read pool", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"cop_normal.*"'], + ), + legend_format="{{instance}}-normal", + ), + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"cop_high.*"'], + ), + legend_format="{{instance}}-high", + ), + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"cop_low.*"'], + ), + legend_format="{{instance}}-low", + ), + ], + thresholds=[GraphThreshold(value=7.2)], + ), + ] + ) + layout.row( + [ + graph_panel( + title="IME CPU", + description="The CPU utilization of IME threads", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"ime.*"'], + by_labels=["instance"], + ), + legend_format="{{instance}}", + ), + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"ime.*"'], + by_labels=["instance", "name"], + ), + legend_format="{{instance}}-{{name}}", + hide=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Busy Threads (>80%)", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_topk( + 20, + "%s" + % expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name!~"rocksdb.*"'], + by_labels=["instance", "name"], + ).extra(extra_expr="> 0.8"), + ), + legend_format="{{name}}-{{instance}}", + ), + ], + ), + ] + ) + return layout.row_panel + + +def TTL() -> RowPanel: + layout = Layout(title="TTL") + layout.row( + [ + graph_panel( + title="TTL expire count", + targets=[ + target( + expr=expr_sum_rate( + "tikv_ttl_expire_kv_count_total", + ), + ), + ], + ), + graph_panel( + title="TTL expire size", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_ttl_expire_kv_size_total", + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="TTL check progress", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_operator( + expr_sum_rate( + "tikv_ttl_checker_processed_regions", + ), + "/", + expr_sum_rate( + "tikv_raftstore_region_count", + label_selectors=['type="region"'], + ), + ), + legend_format="{{instance}}", + ), + ], + ), + graph_panel( + title="TTL checker actions", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_ttl_checker_actions", by_labels=["type"] + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel_histogram_quantiles( + title="TTL checker compact duration", + description="The time consumed when executing GC tasks", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_ttl_checker_compact_duration", + ), + stat_panel( + title="TTL checker poll interval", + format=UNITS.MILLI_SECONDS, + targets=[ + target( + expr=expr_max( + "tikv_ttl_checker_poll_interval", + label_selectors=['type="tikv_gc_run_interval"'], + by_labels=[], # override default by instance. + ), + ), + ], + ), + ] + ) + return layout.row_panel + + +def PD() -> RowPanel: + layout = Layout(title="PD") + layout.row( + [ + graph_panel( + title="PD requests", + description="The count of requests that TiKV sends to PD", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_pd_request_duration_seconds_count", + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="PD request duration (average)", + description="The time consumed by requests that TiKV sends to PD", + yaxes=yaxes(left_format=UNITS.SECONDS), + targets=[ + target( + expr=expr_histogram_avg( + "tikv_pd_request_duration_seconds", + by_labels=["type"], + ), + legend_format="{{type}}", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="PD heartbeats", + description="The total number of PD heartbeat messages", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_pd_heartbeat_message_total", + by_labels=["type"], + ), + additional_groupby=True, + ), + target( + expr=expr_sum( + "tikv_pd_pending_heartbeat_total", + ), + legend_format="{{instance}}-pending", + ), + ], + ), + graph_panel( + title="PD validate peers", + description="The total number of peers validated by the PD worker", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_pd_validate_peer_total", + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="PD reconnection", + description="The count of reconnection between TiKV and PD", + yaxes=yaxes(left_format=UNITS.OPS_PER_MIN), + targets=[ + target( + expr=expr_sum_delta( + "tikv_pd_reconnect_total", + range_selector="$__rate_interval", + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="PD forward status", + description="The forward status of PD client", + targets=[ + target( + expr=expr_simple( + "tikv_pd_request_forwarded", + ), + legend_format="{{instance}}-{{host}}", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Pending TSO Requests", + description="The number of TSO requests waiting in the queue.", + yaxes=yaxes(left_format=UNITS.OPS_PER_MIN), + targets=[ + target( + expr=expr_sum( + "tikv_pd_pending_tso_request_total", + ), + additional_groupby=True, + ), + ], + ), + ] + ) + return layout.row_panel + + +def IOBreakdown() -> RowPanel: + layout = Layout(title="IO Breakdown") + layout.row( + [ + graph_panel( + title="Write IO bytes", + description="The throughput of disk write per IO type", + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_io_bytes", + label_selectors=['op="write"'], + by_labels=["type"], + ), + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_io_bytes", + label_selectors=['op="write"'], + by_labels=[], # override default by instance. + ), + legend_format="total", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Read IO bytes", + description="The throughput of disk read per IO type", + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_io_bytes", + label_selectors=['op="read"'], + by_labels=["type"], + ), + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_io_bytes", + label_selectors=['op="read"'], + by_labels=[], # override default by instance. + ), + legend_format="total", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="IO threshold", + description="The threshold of disk IOs per priority", + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC), + targets=[ + target( + expr=expr_avg( + "tikv_rate_limiter_max_bytes_per_sec", + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Rate Limiter Request Wait Duration", + description="IO rate limiter request wait duration.", + yaxes=yaxes(left_format=UNITS.SECONDS), + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_rate_limiter_request_wait_duration_seconds", + by_labels=["type"], + ), + legend_format=r"{{type}}-99%", + additional_groupby=True, + ), + target( + expr=expr_histogram_avg( + "tikv_rate_limiter_request_wait_duration_seconds", + by_labels=[], # override default by instance. + ), + legend_format="avg", + additional_groupby=True, + ), + ], + ), + ] + ) + return layout.row_panel + + +def RaftWaterfall() -> RowPanel: + layout = Layout(title="Raft Waterfall") + layout.row( + [ + graph_panel_histogram_quantiles( + title="Storage async write duration", + description="The time consumed by processing asynchronous write requests", + yaxes=yaxes(left_format=UNITS.SECONDS, right_format=UNITS.NONE_FORMAT), + metric="tikv_storage_engine_async_request_duration_seconds", + label_selectors=['type="write"'], + ), + ] + ) + layout.row( + [ + graph_panel_histogram_quantiles( + title="Store duration", + description="The store time duration of each request", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_raftstore_store_duration_secs", + ), + graph_panel_histogram_quantiles( + title="Apply duration", + description="The apply time duration of each request", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_raftstore_apply_duration_secs", + ), + ] + ) + layout.row( + [ + graph_panel_histogram_quantiles( + title="Store propose wait duration", + description="Time from request scheduling to when it is handled by Raftstore", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_raftstore_request_wait_time_duration_secs", + ), + graph_panel_histogram_quantiles( + title="Store batch wait duration", + description="Time from request scheduling to when a batch of requests is formed and prepared to be proposed to Raft", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_raftstore_store_wf_batch_wait_duration_seconds", + ), + ] + ) + layout.row( + [ + graph_panel_histogram_quantiles( + title="Store send to write queue duration", + description="Time from request scheduling to just before it is sent to the store writer thread", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_raftstore_store_wf_send_to_queue_duration_seconds", + ), + graph_panel_histogram_quantiles( + title="Store send proposal duration", + description="Time from request scheduling to just before it is sent to followers", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_raftstore_store_wf_send_proposal_duration_seconds", + ), + ] + ) + layout.row( + [ + graph_panel_histogram_quantiles( + title="Store write kv db end duration", + description="Time from request scheduling to when the batch's snapshot state is written to KV DB", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_raftstore_store_wf_write_kvdb_end_duration_seconds", + ), + graph_panel_histogram_quantiles( + title="Store before write duration", + description="Time from request scheduling to just before it is written to Raft Engine", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_raftstore_store_wf_before_write_duration_seconds", + ), + ] + ) + layout.row( + [ + graph_panel_histogram_quantiles( + title="Store write end duration", + description="Time from request scheduling to when it is written to Raft Engine", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_raftstore_store_wf_write_end_duration_seconds", + ), + graph_panel_histogram_quantiles( + title="Store persist duration", + description="Time from request scheduling to when its associated ready is persisted on the leader", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_raftstore_store_wf_persist_duration_seconds", + ), + ] + ) + layout.row( + [ + graph_panel_histogram_quantiles( + title="Store commit but not persist duration", + description="Time from request scheduling to when it is committed; at the time of commit, it has not been persisted on the leader", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_raftstore_store_wf_commit_not_persist_log_duration_seconds", + ), + graph_panel_histogram_quantiles( + title="Store commit and persist duration", + description="Time from request scheduling to when it is committed; at the time of commit, it has been persisted on the leader", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_raftstore_store_wf_commit_log_duration_seconds", + ), + ] + ) + return layout.row_panel + + +def RaftIO() -> RowPanel: + layout = Layout(title="Raft IO") + layout.row( + heatmap_panel_graph_panel_histogram_quantile_pairs( + heatmap_title="Process ready duration", + heatmap_description="The time taken by Raftstore to complete processing a poll round, which includes a batch of region peers", + graph_title="99% Process ready duration per server", + graph_description="The time taken by Raftstore to complete processing a poll round, which includes a batch of region peers", + graph_by_labels=["instance"], + graph_hides=["count", "avg"], + yaxis_format=UNITS.SECONDS, + metric="tikv_raftstore_raft_process_duration_secs", + label_selectors=['type="ready"'], + ) + ) + layout.row( + heatmap_panel_graph_panel_histogram_quantile_pairs( + heatmap_title="Store write loop duration", + heatmap_description="The time duration of store write loop when store-io-pool-size is not zero.", + graph_title="99% Store write loop duration per server", + graph_description="The time duration of store write loop on each TiKV instance when store-io-pool-size is not zero.", + graph_by_labels=["instance"], + graph_hides=["count", "avg"], + yaxis_format=UNITS.SECONDS, + metric="tikv_raftstore_store_write_loop_duration_seconds", + ) + ) + layout.row( + heatmap_panel_graph_panel_histogram_quantile_pairs( + heatmap_title="Append log duration", + heatmap_description="The time consumed when Raft appends log", + graph_title="99% Append log duration per server", + graph_description="The time consumed when Raft appends log on each TiKV instance", + graph_by_labels=["instance"], + graph_hides=["count", "avg"], + yaxis_format=UNITS.SECONDS, + metric="tikv_raftstore_append_log_duration_seconds", + ) + ) + layout.row( + heatmap_panel_graph_panel_histogram_quantile_pairs( + heatmap_title="Commit log duration", + heatmap_description="The time consumed when Raft commits log", + graph_title="99% Commit log duration per server", + graph_description="The time consumed when Raft commits log on each TiKV instance", + graph_by_labels=["instance"], + graph_hides=["count", "avg"], + yaxis_format=UNITS.SECONDS, + metric="tikv_raftstore_commit_log_duration_seconds", + ) + ) + layout.row( + heatmap_panel_graph_panel_histogram_quantile_pairs( + heatmap_title="Apply log duration", + heatmap_description="The time consumed when Raft applies log", + graph_title="99% Apply log duration per server", + graph_description="The time consumed for Raft to apply logs per TiKV instance", + graph_by_labels=["instance"], + graph_hides=["count", "avg"], + yaxis_format=UNITS.SECONDS, + metric="tikv_raftstore_apply_log_duration_seconds", + ) + ) + layout.row( + heatmap_panel_graph_panel_histogram_quantile_pairs( + heatmap_title="Raft Client Wait Connection Ready Duration", + heatmap_description="The time consumed for Raft Client wait connection ready", + graph_title="99% Raft Client Wait Connection Ready Duration", + graph_description="The time consumed for Raft Client wait connection ready per TiKV instance", + yaxis_format=UNITS.SECONDS, + metric="tikv_server_raft_client_wait_ready_duration", + graph_by_labels=["to"], + ) + ) + layout.row( + [ + graph_panel( + title="Store io task reschedule", + description="The throughput of disk write per IO type", + targets=[ + target( + expr=expr_sum( + "tikv_raftstore_io_reschedule_region_total", + ), + legend_format="rechedule-{{instance}}", + ), + target( + expr=expr_sum( + "tikv_raftstore_io_reschedule_pending_tasks_total", + ), + legend_format="pending-task-{{instance}}", + ), + ], + ), + graph_panel( + title="99% Write task block duration per server", + description="The time consumed when store write task block on each TiKV instance", + yaxes=yaxes(left_format=UNITS.SECONDS), + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_raftstore_store_write_msg_block_wait_duration_seconds", + by_labels=["instance"], + ), + legend_format="{{instance}}", + ), + ], + ), + ] + ) + return layout.row_panel + + +def RaftPropose() -> RowPanel: + layout = Layout(title="Raft Propose") + layout.row( + [ + graph_panel( + title="Raft proposals per ready", + description="The proposal count of a Regions in a tick", + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_raftstore_apply_proposal", + by_labels=["instance"], + ), + legend_format="{{instance}}", + ), + ], + ), + graph_panel( + title="Raft read/write proposals", + description="The number of proposals per type", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_raftstore_proposal_total", + label_selectors=['type=~"local_read|normal|read_index"'], + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Raft read proposals per server", + description="The number of read proposals which are made by each TiKV instance", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_raftstore_proposal_total", + label_selectors=['type=~"local_read|read_index"'], + ), + ), + ], + ), + graph_panel( + title="Raft write proposals per server", + description="The number of write proposals which are made by each TiKV instance", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_raftstore_proposal_total", + label_selectors=['type=~"normal"'], + ), + ), + ], + ), + ] + ) + layout.row( + heatmap_panel_graph_panel_histogram_quantile_pairs( + heatmap_title="Propose wait duration", + heatmap_description="The wait time of each proposal", + graph_title="99% Propose wait duration per server", + graph_description="The wait time of each proposal in each TiKV instance", + graph_by_labels=["instance"], + graph_hides=["count", "avg"], + yaxis_format=UNITS.SECONDS, + metric="tikv_raftstore_request_wait_time_duration_secs", + ) + ) + layout.row( + heatmap_panel_graph_panel_histogram_quantile_pairs( + heatmap_title="Store write wait duration", + heatmap_description="The wait time of each store write task", + graph_title="99% Store write wait duration per server", + graph_description="The wait time of each store write task in each TiKV instance", + graph_by_labels=["instance"], + graph_hides=["count", "avg"], + yaxis_format=UNITS.SECONDS, + metric="tikv_raftstore_store_write_task_wait_duration_secs", + ) + ) + layout.row( + heatmap_panel_graph_panel_histogram_quantile_pairs( + heatmap_title="Apply wait duration", + heatmap_description="The wait time of each apply task", + graph_title="99% Apply wait duration per server", + graph_description="The wait time of each apply task in each TiKV instance", + graph_by_labels=["instance"], + graph_hides=["count", "avg"], + yaxis_format=UNITS.SECONDS, + metric="tikv_raftstore_apply_wait_time_duration_secs", + ) + ) + layout.row( + [ + heatmap_panel( + title="Store write handle msg duration", + description="The handle duration of each store write task msg", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_raftstore_store_write_handle_msg_duration_secs_bucket", + ), + heatmap_panel( + title="Store write trigger size", + description="The distribution of write trigger size", + yaxis=yaxis(format=UNITS.BYTES_IEC), + metric="tikv_raftstore_store_write_trigger_wb_bytes_bucket", + ), + ] + ) + layout.row( + [ + graph_panel( + title="Raft propose speed", + description="The rate at which peers propose logs", + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_raftstore_propose_log_size_sum", + ), + ), + ], + ), + graph_panel( + title="Perf Context duration", + description="The rate at which peers propose logs", + yaxes=yaxes(left_format=UNITS.SECONDS), + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_raftstore_store_perf_context_time_duration_secs", + by_labels=["type"], + ), + legend_format="store-{{type}}", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_raftstore_apply_perf_context_time_duration_secs", + by_labels=["type"], + ), + legend_format="apply-{{type}}", + additional_groupby=True, + ), + ], + ), + ] + ) + return layout.row_panel + + +def RaftProcess() -> RowPanel: + layout = Layout(title="Raft Process") + layout.row( + [ + graph_panel( + title="Ready handled", + description="The count of different ready type of Raft", + targets=[ + target( + expr=expr_sum_rate( + "tikv_raftstore_raft_ready_handled_total", + by_labels=["type"], + ), + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_raftstore_raft_process_duration_secs_count", + label_selectors=['type="ready"'], + by_labels=[], # overwrite default by instance. + ), + legend_format="count", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Max duration of raft store events", + description="The max time consumed by raftstore events", + yaxes=yaxes(left_format=UNITS.SECONDS), + targets=[ + target( + expr=expr_histogram_quantile( + 0.999999, + "tikv_raftstore_event_duration", + by_labels=["type"], + ), + legend_format="{{type}}", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.999999, + "tikv_broadcast_normal_duration_seconds", + ), + legend_format="broadcast_normal", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Replica read lock checking duration", + description="Replica read lock checking duration", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_replica_read_lock_check_duration_seconds_bucket", + ), + graph_panel( + title="Fsm reschedule ops", + description="The number of fsm reschedule ops", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_batch_system_fsm_reschedule_total", + by_labels=["type"], + ), + ), + ], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Store fsm schedule wait duration", + description="Duration of store fsm waiting to be polled", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_batch_system_fsm_schedule_wait_seconds_bucket", + label_selectors=['type="store"'], + ), + heatmap_panel( + title="Apply fsm schedule wait duration", + description="Duration of apply fsm waiting to be polled.e", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_batch_system_fsm_schedule_wait_seconds_bucket", + label_selectors=['type="apply"'], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Store fsm poll duration", + description="Total time for an store FSM to finish processing all messages, potentially over multiple polling rounds.", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_batch_system_fsm_poll_seconds_bucket", + label_selectors=['type="store"'], + ), + heatmap_panel( + title="Apply fsm poll duration", + description="Total time for an apply FSM to finish processing all messages, potentially over multiple polling rounds", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_batch_system_fsm_poll_seconds_bucket", + label_selectors=['type="apply"'], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Store fsm poll round", + description="Number of polling rounds for an store FSM to finish processing all messages", + metric="tikv_batch_system_fsm_poll_rounds_bucket", + label_selectors=['type="store"'], + ), + heatmap_panel( + title="Apply fsm poll round", + description="Number of polling rounds for an apply FSM to finish processing all messages", + metric="tikv_batch_system_fsm_poll_rounds_bucket", + label_selectors=['type="apply"'], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Store fsm count per poll", + description="Number of store fsm polled in one poll", + metric="tikv_batch_system_fsm_count_per_poll_bucket", + label_selectors=['type="store"'], + ), + heatmap_panel( + title="Apply fsm count per poll", + description="Number of apply fsm polled in one poll", + metric="tikv_batch_system_fsm_count_per_poll_bucket", + label_selectors=['type="apply"'], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Peer msg length distribution", + description="The length of peer msgs for each round handling", + metric="tikv_raftstore_peer_msg_len_bucket", + ), + heatmap_panel( + title="Apply msg length distribution", + description="The length of apply msgs for each round handling", + metric="tikv_raftstore_apply_msg_len_bucket", + ), + ] + ) + layout.row( + [ + graph_panel( + title="Enable apply unpersisted log region count", + description="The number of regions that enable apply unpersisted raft log", + yaxes=yaxes(left_format=UNITS.SHORT), + targets=[ + target( + expr=expr_simple( + "tikv_raft_enable_unpersisted_apply_regions", + ), + legend_format="{{instance}}", + ), + ], + ), + graph_panel( + title="Apply ahead of persistence raft log count", + description="The number of raft logs between apply and persisted index", + yaxes=yaxes(left_format=UNITS.SHORT), + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_raft_apply_ahead_of_persist", + by_labels=["instance"], + ), + legend_format="{{instance}}-99%", + ), + target( + expr=expr_histogram_quantile( + 1, + "tikv_raft_apply_ahead_of_persist", + by_labels=["instance"], + ), + legend_format="{{instance}}-max", + ), + ], + ), + ] + ) + return layout.row_panel + + +def RaftMessage() -> RowPanel: + layout = Layout(title="Raft Message") + layout.row( + [ + graph_panel( + title="Sent messages per server", + description="The number of Raft messages sent by each TiKV instance", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_raftstore_raft_sent_message_total", + ), + ), + ], + ), + graph_panel( + title="Flush messages per server", + description="The number of Raft messages flushed by each TiKV instance", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_server_raft_message_flush_total", + by_labels=["instance", "reason"], + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Receive messages per server", + description="The number of Raft messages received by each TiKV instance", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_server_raft_message_recv_total", + ), + ), + ], + ), + graph_panel( + title="Messages", + description="The number of different types of Raft messages that are sent", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_raftstore_raft_sent_message_total", + label_selectors=['status="accept"'], + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Vote", + description="The total number of vote messages that are sent in Raft", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_raftstore_raft_sent_message_total", + label_selectors=['type="vote"'], + ), + ), + ], + ), + graph_panel( + title="Raft dropped messages", + description="The number of dropped Raft messages per type", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_raftstore_raft_dropped_message_total", + by_labels=["type"], + ), + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_raftstore_raft_sent_message_total", + label_selectors=['status="drop"'], + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + heatmap_panel_graph_panel_histogram_quantile_pairs( + heatmap_title="Raft Message Send Wait duration", + heatmap_description="The time consumed waiting to send Raft Messages", + graph_title="99% Raft Message Send Wait Duration", + graph_description="The time consumed waiting to send Raft Messages per TiKV instance", + graph_by_labels=["instance"], + yaxis_format=UNITS.SECONDS, + metric="tikv_server_raft_message_duration_seconds", + label_selectors=['type="send_wait"'], + ) + ) + layout.row( + heatmap_panel_graph_panel_histogram_quantile_pairs( + heatmap_title="Raft Message Receive Delay duration", + heatmap_description="The time consumed to transmit Raft Messages over the network, reported by the receiver", + graph_title="99% Raft Message Receive Delay Duration", + graph_description="The time consumed to transmit Raft Messages over the network per TiKV instance, reported by the receiver", + graph_by_labels=["instance"], + yaxis_format=UNITS.SECONDS, + metric="tikv_server_raft_message_duration_seconds", + label_selectors=['type="receive_delay"'], + ) + ) + return layout.row_panel + + +def RaftAdmin() -> RowPanel: + layout = Layout(title="Raft Admin") + layout.row( + [ + graph_panel( + title="Admin proposals", + description="The number of admin proposals", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_raftstore_proposal_total", + label_selectors=['type=~"conf_change|transfer_leader"'], + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Admin apply", + description="The number of the processed apply command", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_raftstore_admin_cmd_total", + label_selectors=['type!="compact"'], + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Check split", + description="The number of raftstore split checks", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_raftstore_check_split_total", + label_selectors=['type!="ignore"'], + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="99.99% Check split duration", + description="The time consumed when running split check in .9999", + yaxes=yaxes(left_format=UNITS.SECONDS), + targets=[ + target( + expr=expr_histogram_quantile( + 0.9999, + "tikv_raftstore_check_split_duration_seconds", + by_labels=["instance"], + ), + legend_format="{{instance}}", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Load base split event", + yaxes=yaxes(left_format=UNITS.OPS_PER_MIN), + targets=[ + target( + expr=expr_sum_delta( + "tikv_load_base_split_event", + range_selector="1m", + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Load base split duration", + yaxes=yaxes(left_format=UNITS.SECONDS), + targets=[ + target( + expr=expr_histogram_quantile( + 0.80, + "tikv_load_base_split_duration_seconds", + by_labels=["instance"], + ), + legend_format="80%-{{instance}}", + ), + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_load_base_split_duration_seconds", + by_labels=["instance"], + ), + legend_format="99%-{{instance}}", + ), + target( + expr=expr_histogram_avg( + "tikv_load_base_split_duration_seconds", + by_labels=["instance"], + ), + legend_format="avg-{{instance}}", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Peer in Flashback State", + targets=[ + target( + expr=expr_sum( + "tikv_raftstore_peer_in_flashback_state", + ), + ), + ], + ), + ] + ) + return layout.row_panel + + +def RaftLog() -> RowPanel: + layout = Layout(title="Raft Log") + layout.row( + [ + graph_panel( + title="Raft log GC write duration", + yaxes=yaxes(left_format=UNITS.SECONDS, log_base=10), + targets=[ + target( + expr=expr_histogram_quantile( + 0.9999, + "tikv_raftstore_raft_log_gc_write_duration_secs", + by_labels=["instance"], + ), + legend_format="99.99%-{{instance}}", + ), + target( + expr=expr_histogram_avg( + "tikv_raftstore_raft_log_gc_write_duration_secs", + by_labels=["instance"], + ), + legend_format="avg-{{instance}}", + ), + ], + ), + graph_panel( + title="Raft log GC kv sync duration", + yaxes=yaxes(left_format=UNITS.SECONDS, log_base=10), + targets=[ + target( + expr=expr_histogram_quantile( + 0.9999, + "tikv_raftstore_raft_log_kv_sync_duration_secs", + by_labels=["instance"], + ), + legend_format="99.99%-{{instance}}", + ), + target( + expr=expr_histogram_avg( + "tikv_raftstore_raft_log_kv_sync_duration_secs", + by_labels=["instance"], + ), + legend_format="avg-{{instance}}", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Raft log GC write operations", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_raftstore_raft_log_gc_write_duration_secs_count", + ), + ), + ], + ), + graph_panel( + title="Raft log GC seek operations ", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_raftstore_raft_log_gc_seek_operations_count", + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Raft log lag", + targets=[ + target( + expr=expr_sum_rate( + "tikv_raftstore_log_lag_sum", + ), + ), + ], + ), + graph_panel( + title="Raft log gc skipped", + targets=[ + target( + expr=expr_sum_rate( + "tikv_raftstore_raft_log_gc_skipped", + by_labels=["instance", "reason"], + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Raft log GC failed", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_raftstore_raft_log_gc_failed", + ), + ), + ], + ), + graph_panel( + title="Raft log fetch ", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_raftstore_entry_fetches", + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Raft log async fetch task duration", + yaxes=yaxes(left_format=UNITS.SECONDS, log_base=10), + targets=[ + target( + expr=expr_histogram_quantile( + 0.9999, + "tikv_raftstore_entry_fetches_task_duration_seconds", + ), + legend_format="99.99%", + additional_groupby=True, + ), + target( + expr=expr_histogram_avg( + "tikv_raftstore_entry_fetches_task_duration_seconds", + by_labels=["instance"], + ), + legend_format="avg-{{instance}}", + ), + target( + expr=expr_sum( + "tikv_worker_pending_task_total", + label_selectors=['name=~"raftlog-fetch-worker"'], + ), + legend_format="pending-task", + additional_groupby=True, + ), + ], + series_overrides=[ + series_override( + alias="/pending-task/", + yaxis=2, + transform_negative_y=True, + ), + ], + ), + ] + ) + return layout.row_panel + + +def LocalReader() -> RowPanel: + layout = Layout(title="Local Reader") + layout.row( + [ + graph_panel( + title="Local reader requests", + targets=[ + target( + expr=expr_sum_rate( + "tikv_raftstore_local_read_reject_total", + by_labels=["instance", "reason"], + ), + legend_format="{{instance}}-reject-by-{{reason}}", + ), + target( + expr=expr_sum_rate( + "tikv_raftstore_local_read_executed_requests", + ), + legend_format="{{instance}}-total", + ), + target( + expr=expr_sum_rate( + "tikv_raftstore_local_read_executed_stale_read_requests", + ), + legend_format="{{instance}}-stale-read", + ), + ], + series_overrides=[ + series_override( + alias="/.*-total/", + yaxis=2, + ), + ], + ), + ] + ) + return layout.row_panel + + +def UnifiedReadPool() -> RowPanel: + return YatpPool( + title="Unified Read Pool", + pool_name_prefix="unified-read", + running_task_metric="tikv_unified_read_pool_running_tasks", + running_task_metric_label="priority", + ) + + +def YatpPool( + title: str, + pool_name_prefix: str, + running_task_metric: str, + running_task_metric_label: str, +) -> RowPanel: + layout = Layout(title) + layout.row( + [ + graph_panel( + title="Time used by level", + description="The time used by each level in the yatp thread pool per second. Level 0 refers to small queries.", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS), + targets=[ + target( + expr=expr_sum_rate( + "tikv_multilevel_level_elapsed", + label_selectors=[f'name=~"{pool_name_prefix}.*"'], + by_labels=["level"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Level 0 chance", + description="The chance that level 0 (small) tasks are scheduled in the yatp thread pool.", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_simple( + "tikv_multilevel_level0_chance", + label_selectors=[f'name=~"{pool_name_prefix}.*"'], + ), + legend_format="{{instance}}", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Running tasks", + description="The number of concurrently running tasks in the yatp thread pool.", + targets=[ + target( + expr=expr_sum_aggr_over_time( + running_task_metric, + "avg", + "1m", + by_labels=[running_task_metric_label], + ), + additional_groupby=True, + ), + ], + ), + heatmap_panel( + title="Wait Duration", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_yatp_pool_schedule_wait_duration_bucket", + label_selectors=[f'name=~"{pool_name_prefix}.*"'], + ), + ] + ) + layout.row( + [ + graph_panel_histogram_quantiles( + title="Duration of One Time Slice", + description="Task execution time during one schedule.", + yaxes=yaxes(left_format=UNITS.SECONDS, log_base=2), + metric="tikv_yatp_task_poll_duration", + label_selectors=[f'name=~"{pool_name_prefix}.*"'], + hide_count=True, + ), + graph_panel_histogram_quantiles( + title="Task Execute Duration", + description="Task total execution duration.", + yaxes=yaxes(left_format=UNITS.SECONDS, log_base=2), + metric="tikv_yatp_task_exec_duration", + label_selectors=[f'name=~"{pool_name_prefix}.*"'], + hide_count=True, + ), + ] + ) + layout.row( + [ + graph_panel_histogram_quantiles( + title="Task Schedule Times", + description="Task schedule number of times.", + yaxes=yaxes(left_format=UNITS.NONE_FORMAT, log_base=2), + metric="tikv_yatp_task_execute_times", + label_selectors=[f'name=~"{pool_name_prefix}.*"'], + hide_count=True, + ), + ] + ) + return layout.row_panel + + +def Storage() -> RowPanel: + layout = Layout(title="Storage") + layout.row( + [ + graph_panel( + title="Storage command total", + description="The total count of different kinds of commands received", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC, log_base=10), + targets=[ + target( + expr=expr_sum_rate( + "tikv_storage_command_total", + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Storage async request error", + description="The total number of engine asynchronous request errors", + targets=[ + target( + expr=expr_sum_rate( + "tikv_storage_engine_async_request_total", + label_selectors=['status!~"all|success"'], + by_labels=["status"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + heatmap_panel_graph_panel_histogram_quantile_pairs( + heatmap_title="Storage async write duration", + heatmap_description="The time consumed by processing asynchronous write requests", + graph_title="Storage async write duration", + graph_description="The storage async write duration", + yaxis_format=UNITS.SECONDS, + metric="tikv_storage_engine_async_request_duration_seconds", + label_selectors=['type="write"'], + ), + ) + layout.row( + heatmap_panel_graph_panel_histogram_quantile_pairs( + heatmap_title="Storage async snapshot duration", + heatmap_description="The time consumed by processing asynchronous snapshot requests", + graph_title="Storage async snapshot duration", + graph_description="The storage async snapshot duration", + yaxis_format=UNITS.SECONDS, + metric="tikv_storage_engine_async_request_duration_seconds", + label_selectors=['type="snapshot"'], + ), + ) + layout.row( + heatmap_panel_graph_panel_histogram_quantile_pairs( + heatmap_title="Storage async snapshot duration (pure local read)", + heatmap_description="The storage async snapshot duration without the involving of raftstore", + graph_title="Storage async snapshot duration (pure local read)", + graph_description="The storage async snapshot duration without the involving of raftstore", + yaxis_format=UNITS.SECONDS, + metric="tikv_storage_engine_async_request_duration_seconds", + label_selectors=['type="snapshot_local_read"'], + ), + ) + layout.row( + heatmap_panel_graph_panel_histogram_quantile_pairs( + heatmap_title="Read index propose wait duration", + heatmap_description="Read index propose wait duration associated with async snapshot", + graph_title="Read index propose wait duration", + graph_description="Read index propose wait duration associated with async snapshot", + yaxis_format=UNITS.SECONDS, + metric="tikv_storage_engine_async_request_duration_seconds", + label_selectors=['type="snapshot_read_index_propose_wait"'], + ), + ) + layout.row( + heatmap_panel_graph_panel_histogram_quantile_pairs( + heatmap_title="Read index confirm duration", + heatmap_description="Read index confirm duration associated with async snapshot", + graph_title="Read index confirm duration", + graph_description="Read index confirm duration associated with async snapshot", + yaxis_format=UNITS.SECONDS, + metric="tikv_storage_engine_async_request_duration_seconds", + label_selectors=['type="snapshot_read_index_confirm"'], + ), + ) + layout.row( + [ + graph_panel( + title="Process Stat Cpu Usage", + description="CPU usage measured over a 30 second window", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum( + "tikv_storage_process_stat_cpu_usage", + ), + additional_groupby=True, + ), + ], + ), + graph_panel_histogram_quantiles( + title="Full compaction duration seconds", + description="", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_storage_full_compact_duration_seconds", + hide_count=True, + ), + ] + ) + layout.row( + [ + graph_panel_histogram_quantiles( + title="Full compaction pause duration", + description="", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_storage_full_compact_pause_duration_seconds", + hide_count=True, + ), + graph_panel_histogram_quantiles( + title="Full compaction per-increment duration", + description="", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_storage_full_compact_increment_duration_seconds", + hide_count=True, + ), + ] + ) + return layout.row_panel + + +def FlowControl() -> RowPanel: + layout = Layout(title="Flow Control") + layout.row( + [ + graph_panel( + title="Scheduler flow", + description="", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum( + "tikv_scheduler_write_flow", + ), + legend_format="write-{{instance}}", + ), + target( + expr=expr_sum( + "tikv_scheduler_throttle_flow", + ).extra(" != 0"), + legend_format="throttle-{{instance}}", + ), + ], + ), + graph_panel( + title="Scheduler discard ratio", + description="", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum( + "tikv_scheduler_discard_ratio", + by_labels=["type"], + ).extra(" / 10000000"), + ), + ], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Throttle duration", + metric="tikv_scheduler_throttle_duration_seconds_bucket", + yaxis=yaxis(format=UNITS.SECONDS), + ), + graph_panel( + title="Scheduler throttled CF", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_simple( + "tikv_scheduler_throttle_cf", + ).extra(" != 0"), + legend_format="{{instance}}-{{cf}}", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Flow controller actions", + description="", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_scheduler_throttle_action_total", + by_labels=["type", "cf"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Flush/L0 flow", + description="", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum( + "tikv_scheduler_l0_flow", + by_labels=["instance", "cf"], + ), + legend_format="{{cf}}_l0_flow-{{instance}}", + ), + target( + expr=expr_sum( + "tikv_scheduler_flush_flow", + by_labels=["instance", "cf"], + ), + legend_format="{{cf}}_flush_flow-{{instance}}", + ), + target( + expr=expr_sum( + "tikv_scheduler_l0_flow", + ), + legend_format="total_l0_flow-{{instance}}", + ), + target( + expr=expr_sum( + "tikv_scheduler_flush_flow", + ), + legend_format="total_flush_flow-{{instance}}", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Flow controller factors", + description="", + targets=[ + target( + expr=expr_max( + "tikv_scheduler_l0", + ), + legend_format="l0-{{instance}}", + ), + target( + expr=expr_max( + "tikv_scheduler_memtable", + ), + legend_format="memtable-{{instance}}", + ), + target( + expr=expr_max( + "tikv_scheduler_l0_avg", + ), + legend_format="avg_l0-{{instance}}", + ), + ], + ), + graph_panel( + title="Compaction pending bytes", + description="", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum( + "tikv_engine_pending_compaction_bytes", + label_selectors=['db="kv"'], + by_labels=["cf"], + ), + additional_groupby=True, + ), + target( + expr=expr_sum( + "tikv_scheduler_pending_compaction_bytes", + by_labels=["cf"], + ).extra(" / 10000000"), + legend_format="pending-bytes", + hide=True, + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Txn command throttled duration", + description="Throttle time for txn storage commands in 1 minute.", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS), + targets=[ + target( + expr=expr_sum_rate( + "tikv_txn_command_throttle_time_total", + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Non-txn command throttled duration", + description="Throttle time for non-txn related processing like analyze or dag in 1 minute.", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS), + targets=[ + target( + expr=expr_sum_rate( + "tikv_non_txn_command_throttle_time_total", + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + return layout.row_panel + + +def SchedulerCommands() -> RowPanel: + layout = Layout(title="Scheduler", repeat="command") + layout.row( + [ + graph_panel( + title="Scheduler stage total", + description="The total number of commands on each stage in commit command", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_scheduler_too_busy_total", + label_selectors=['type="$command"'], + ), + legend_format="busy-{{instance}}", + ), + target( + expr=expr_sum_rate( + "tikv_scheduler_stage_total", + label_selectors=['type="$command"'], + by_labels=["stage"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel_histogram_quantiles( + title="Scheduler command duration", + description="The time consumed when executing commit command", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_scheduler_command_duration_seconds", + label_selectors=['type="$command"'], + hide_count=True, + ), + graph_panel_histogram_quantiles( + title="Scheduler latch wait duration", + description="The time which is caused by latch wait in commit command", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_scheduler_latch_wait_duration_seconds", + label_selectors=['type="$command"'], + hide_count=True, + ), + ] + ) + layout.row( + [ + graph_panel_histogram_quantiles( + title="Scheduler keys read", + description="The count of keys read by a commit command", + yaxes=yaxes(left_format=UNITS.NONE_FORMAT), + metric="tikv_scheduler_kv_command_key_read", + label_selectors=['type="$command"'], + hide_count=True, + ), + graph_panel_histogram_quantiles( + title="Scheduler keys written", + description="The count of keys written by a commit command", + yaxes=yaxes(left_format=UNITS.NONE_FORMAT), + metric="tikv_scheduler_kv_command_key_write", + label_selectors=['type="$command"'], + hide_count=True, + ), + ] + ) + layout.row( + [ + graph_panel( + title="Scheduler scan details", + description="The keys scan details of each CF when executing commit command", + yaxes=yaxes(left_format=UNITS.NONE_FORMAT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_scheduler_kv_scan_details", + label_selectors=['req="$command"'], + by_labels=["tag"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Scheduler scan details [lock]", + description="The keys scan details of lock CF when executing commit command", + yaxes=yaxes(left_format=UNITS.NONE_FORMAT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_scheduler_kv_scan_details", + label_selectors=['req="$command", cf="lock"'], + by_labels=["tag"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Scheduler scan details [write]", + description="The keys scan details of write CF when executing commit command", + yaxes=yaxes(left_format=UNITS.NONE_FORMAT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_scheduler_kv_scan_details", + label_selectors=['req="$command", cf="write"'], + by_labels=["tag"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Scheduler scan details [default]", + description="The keys scan details of default CF when executing commit command", + yaxes=yaxes(left_format=UNITS.NONE_FORMAT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_scheduler_kv_scan_details", + label_selectors=['req="$command", cf="default"'], + by_labels=["tag"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel_histogram_quantiles( + title="Scheduler command read duration", + description="The time consumed on reading when executing commit command", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_scheduler_processing_read_duration_seconds", + label_selectors=['type="$command"'], + hide_count=True, + ), + heatmap_panel( + title="Check memory locks duration", + description="The time consumed on checking memory locks", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_storage_check_mem_lock_duration_seconds_bucket", + label_selectors=['type="$command"'], + ), + ] + ) + return layout.row_panel + + +def Scheduler() -> RowPanel: + layout = Layout(title="Scheduler") + layout.row( + [ + graph_panel( + title="Scheduler stage total", + description="The total number of commands on each stage", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_scheduler_too_busy_total", + by_labels=["stage"], + ), + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_scheduler_stage_total", + by_labels=["stage"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Scheduler priority commands", + description="The count of different priority commands", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_scheduler_commands_pri_total", + by_labels=["priority"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Scheduler pending commands", + description="The count of pending commands per TiKV instance", + yaxes=yaxes(left_format=UNITS.NONE_FORMAT), + targets=[ + target( + expr=expr_sum( + "tikv_scheduler_contex_total", + ), + ), + ], + ), + graph_panel( + title="Scheduler running commands", + description="The count of running commands per TiKV instance", + yaxes=yaxes(left_format=UNITS.NONE_FORMAT), + targets=[ + target( + expr=expr_sum( + "tikv_scheduler_running_commands", + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Scheduler writing bytes", + description="The total writing bytes of commands on each stage", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum( + "tikv_scheduler_writing_bytes", + ), + ), + ], + ), + graph_panel( + title="Scheduler memory quota", + description="The number of bytes used by scheduler", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum( + "tikv_scheduler_memory_quota_size", + label_selectors=['type="in_use"'], + ), + ), + target( + expr=expr_sum( + "tikv_scheduler_memory_quota_size", + label_selectors=['type="capacity"'], + ), + hide=True, + ), + ], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Txn Scheduler Pool Wait Duration", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_yatp_pool_schedule_wait_duration_bucket", + label_selectors=['name=~"sched-worker.*"'], + ), + ] + ) + return layout.row_panel + + +def SchedulerWorkerPool() -> RowPanel: + return YatpPool( + title="Scheduler Worker Pool", + pool_name_prefix="sched-worker", + running_task_metric="tikv_scheduler_running_commands", + running_task_metric_label="instance", + ) + + +def GC() -> RowPanel: + layout = Layout(title="GC") + layout.row( + [ + graph_panel( + title="GC tasks", + description="The count of GC tasks processed by gc_worker", + targets=[ + target( + expr=expr_sum_rate( + "tikv_gcworker_gc_tasks_vec", + by_labels=["task"], + ), + legend_format="total-{{task}}", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_storage_gc_skipped_counter", + by_labels=["task"], + ), + legend_format="skipped-{{task}}", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_gcworker_gc_task_fail_vec", + by_labels=["task"], + ), + legend_format="failed-{{task}}", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_gc_worker_too_busy", + by_labels=[], + ), + legend_format="gcworker-too-busy", + additional_groupby=True, + ), + ], + ), + graph_panel_histogram_quantiles( + title="GC tasks duration", + description="The time consumed when executing GC tasks", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_gcworker_gc_task_duration_vec", + label_selectors=['type="$command"'], + hide_count=True, + ), + ] + ) + layout.row( + [ + graph_panel( + title="TiDB GC seconds", + description="The GC duration", + yaxes=yaxes(left_format=UNITS.SECONDS), + targets=[ + target( + expr=expr_histogram_quantile( + 1, "tidb_tikvclient_gc_seconds", by_labels=["instance"] + ).skip_default_instance_selector(), + legend_format="{{instance}}", + ), + ], + ), + graph_panel( + title="TiDB GC worker actions", + description="The count of TiDB GC worker actions", + targets=[ + target( + expr=expr_sum_rate( + "tidb_tikvclient_gc_worker_actions_total", + by_labels=["type"], + ).skip_default_instance_selector(), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="ResolveLocks Progress", + description="Progress of ResolveLocks, the first phase of GC", + targets=[ + target( + expr=expr_max( + "tidb_tikvclient_range_task_stats", + label_selectors=['type=~"resolve-locks.*"'], + by_labels=["result"], + ).skip_default_instance_selector(), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="TiKV Auto GC Progress", + description="Progress of TiKV's GC", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_operator( + expr_sum( + "tikv_gcworker_autogc_processed_regions", + label_selectors=['type="scan"'], + ), + "/", + expr_sum( + "tikv_raftstore_region_count", + label_selectors=['type="region"'], + ), + ), + legend_format="{{instance}}", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="GC speed", + description="keys / second", + targets=[ + target( + expr=expr_sum_rate( + "tikv_storage_mvcc_gc_delete_versions_sum", + by_labels=["key_mode"], + ), + legend_format="{{key_mode}}_keys/s", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="TiKV Auto GC SafePoint", + description="SafePoint used for TiKV's Auto GC", + yaxes=yaxes(left_format=UNITS.DATE_TIME_ISO), + targets=[ + target( + expr=expr_max( + "tikv_gcworker_autogc_safe_point", + ) + .extra("/ (2^18)") + .skip_default_instance_selector(), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.half_row( + [ + stat_panel( + title="GC lifetime", + description="The lifetime of TiDB GC", + format=UNITS.SECONDS, + targets=[ + target( + expr=expr_max( + "tidb_tikvclient_gc_config", + label_selectors=['type="tikv_gc_life_time"'], + by_labels=[], + ).skip_default_instance_selector(), + ), + ], + ), + stat_panel( + title="GC interval", + description="The interval of TiDB GC", + format=UNITS.SECONDS, + targets=[ + target( + expr=expr_max( + "tidb_tikvclient_gc_config", + label_selectors=['type="tikv_gc_run_interval"'], + by_labels=[], + ).skip_default_instance_selector(), + ), + ], + ), + ] + ) + layout.half_row( + [ + graph_panel( + title="GC in Compaction Filter", + description="Keys handled in GC compaction filter", + targets=[ + target( + expr=expr_sum_rate( + "tikv_gc_compaction_filtered", + by_labels=["key_mode"], + ), + legend_format="{{key_mode}}_filtered", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_gc_compaction_filter_skip", + by_labels=["key_mode"], + ), + legend_format="{{key_mode}}_skipped", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_gc_compaction_mvcc_rollback", + by_labels=["key_mode"], + ), + legend_format="{{key_mode}}_mvcc-rollback/mvcc-lock", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_gc_compaction_filter_orphan_versions", + by_labels=["key_mode"], + ), + legend_format="{{key_mode}}_orphan-versions", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_gc_compaction_filter_perform", + by_labels=["key_mode"], + ), + legend_format="{{key_mode}}_performed-times", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_gc_compaction_failure", + by_labels=["key_mode", "type"], + ), + legend_format="{{key_mode}}_failure-{{type}}", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_gc_compaction_filter_mvcc_deletion_met", + by_labels=["key_mode"], + ), + legend_format="{{key_mode}}_mvcc-deletion-met", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_gc_compaction_filter_mvcc_deletion_handled", + by_labels=["key_mode"], + ), + legend_format="{{key_mode}}_mvcc-deletion-handled", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_gc_compaction_filter_mvcc_deletion_wasted", + by_labels=["key_mode"], + ), + legend_format="{{key_mode}}_mvcc-deletion-wasted", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="GC scan write details", + description="GC scan write details", + targets=[ + target( + expr=expr_sum_rate( + "tikv_gcworker_gc_keys", + label_selectors=['cf="write"'], + by_labels=["key_mode", "tag"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="GC scan default details", + description="GC scan default details", + targets=[ + target( + expr=expr_sum_rate( + "tikv_gcworker_gc_keys", + label_selectors=['cf="default"'], + by_labels=["key_mode", "tag"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + return layout.row_panel + + +def Snapshot() -> RowPanel: + layout = Layout(title="Snapshot") + layout.row( + [ + graph_panel( + title="Rate snapshot message", + description="The rate of Raft snapshot messages sent", + yaxes=yaxes(left_format=UNITS.OPS_PER_MIN), + targets=[ + target( + expr=expr_sum_delta( + "tikv_raftstore_raft_sent_message_total", + range_selector="1m", + label_selectors=['type="snapshot"'], + ), + ), + ], + ), + graph_panel( + title="Snapshot state count", + description="The number of snapshots in different states", + targets=[ + target( + expr=expr_sum( + "tikv_raftstore_snapshot_traffic_total", + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="99% Snapshot generation/apply wait duration", + description="The time snapshot generation/apply tasks spent waiting to be executed.", + yaxes=yaxes(left_format=UNITS.SECONDS), + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_raftstore_snapshot_generation_wait_duration_seconds", + by_labels=["instance"], + ), + legend_format="{{instance}}-generate", + ), + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_raftstore_snapshot_apply_wait_duration_seconds", + by_labels=["instance"], + ), + legend_format="{{instance}}-apply", + ), + ], + ), + graph_panel( + title="99% Handle snapshot duration", + description="The time consumed when handling snapshots", + yaxes=yaxes(left_format=UNITS.SECONDS), + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_server_send_snapshot_duration_seconds", + ), + legend_format="send", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_raftstore_snapshot_duration_seconds", + label_selectors=['type="apply"'], + ), + legend_format="apply", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_raftstore_snapshot_duration_seconds", + label_selectors=['type="generate"'], + ), + legend_format="generate", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="99.99% Snapshot size", + description="The snapshot size (P99.99).9999", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_histogram_quantile( + 0.9999, + "tikv_snapshot_size", + ), + legend_format="size", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="99.99% Snapshot KV count", + description="The number of KV within a snapshot in .9999", + targets=[ + target( + expr=expr_histogram_quantile( + 0.9999, + "tikv_snapshot_kv_count", + ), + legend_format="count", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Snapshot Actions", + description="Action stats for snapshot generating and applying", + yaxes=yaxes(left_format=UNITS.OPS_PER_MIN), + targets=[ + target( + expr=expr_sum_delta( + "tikv_raftstore_snapshot_total", + range_selector="1m", + by_labels=["type", "status"], + ), + additional_groupby=True, + ), + target( + expr=expr_sum_delta( + "tikv_raftstore_clean_region_count", + range_selector="1m", + by_labels=["type", "status"], + ), + legend_format="clean-region-by-{{type}}", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Snapshot transport speed", + description="The speed of sending or receiving snapshot", + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_snapshot_limit_transport_bytes", + by_labels=["instance", "type"], + ), + ), + target( + expr=expr_sum_rate( + "tikv_snapshot_limit_generate_bytes", + ), + legend_format="{{instance}}-generate", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Snapshot pending applies", + description="The number of snapshots waiting to be applied", + yaxes=yaxes(left_format=UNITS.SHORT), + targets=[ + target( + expr=expr_simple( + "tikv_raftstore_snapshot_pending_applies", + ), + legend_format="{{instance}}", + ), + ], + ), + ] + ) + return layout.row_panel + + +def Task() -> RowPanel: + layout = Layout(title="Task") + layout.row( + [ + graph_panel( + title="Worker handled tasks", + description="The number of tasks handled by worker", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_worker_handled_task_total", + by_labels=["name"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Worker pending tasks", + description="Current pending and running tasks of worker", + targets=[ + target( + expr=expr_sum( + "tikv_worker_pending_task_total", + by_labels=["name"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="FuturePool handled tasks", + description="The number of tasks handled by future_pool", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_futurepool_handled_task_total", + by_labels=["name"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="FuturePool pending tasks", + description="Current pending and running tasks of future_pool", + targets=[ + target( + expr=expr_sum_aggr_over_time( + "tikv_futurepool_pending_task_total", + "avg", + range_selector="1m", + by_labels=["name"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + return layout.row_panel + + +def CoprocessorOverview() -> RowPanel: + layout = Layout(title="Coprocessor Overview") + layout.row( + heatmap_panel_graph_panel_histogram_quantile_pairs( + heatmap_title="Request duration", + heatmap_description="The time consumed to handle coprocessor read requests", + graph_title="Request duration", + graph_description="The time consumed to handle coprocessor read requests", + yaxis_format=UNITS.SECONDS, + metric="tikv_coprocessor_request_duration_seconds", + graph_by_labels=["req"], + ), + ) + layout.row( + [ + graph_panel( + title="Total Requests", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_coprocessor_request_duration_seconds_count", + by_labels=["req"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Total Request Errors", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_coprocessor_request_error", + by_labels=["reason"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="KV Cursor Operations", + targets=[ + target( + expr=expr_sum_rate( + "tikv_coprocessor_scan_keys_sum", + by_labels=["req"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel_histogram_quantiles( + title="KV Cursor Operations", + description="", + metric="tikv_coprocessor_scan_keys", + yaxes=yaxes(left_format=UNITS.SHORT), + by_labels=["req"], + hide_count=True, + ), + ] + ) + layout.row( + [ + graph_panel( + title="Total RocksDB Perf Statistics", + targets=[ + target( + expr=expr_sum_rate( + "tikv_coprocessor_rocksdb_perf", + label_selectors=['metric="internal_delete_skipped_count"'], + by_labels=["req"], + ), + legend_format="delete_skipped-{{req}}", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Total Response Size", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_coprocessor_response_bytes", + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Memory Quota", + description="Total bytes of memory used by coprocessor requests", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum( + "tikv_coprocessor_memory_quota", + by_labels=["instance", "type"], + ), + ), + ], + ) + ] + ) + return layout.row_panel + + +def CoprocessorDetail() -> RowPanel: + layout = Layout(title="Coprocessor Detail") + layout.row( + [ + graph_panel_histogram_quantiles( + title="Handle duration", + description="The time consumed when handling coprocessor requests", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_coprocessor_request_handle_seconds", + by_labels=["req"], + hide_avg=True, + hide_count=True, + ), + graph_panel_histogram_quantiles( + title="Handle duration by store", + description="The time consumed to handle coprocessor requests per TiKV instance", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_coprocessor_request_handle_seconds", + by_labels=["req", "instance"], + hide_avg=True, + hide_count=True, + additional_groupby=False, + ), + ] + ) + layout.row( + [ + graph_panel_histogram_quantiles( + title="Wait duration", + description="The time consumed when coprocessor requests are wait for being handled", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_coprocessor_request_wait_seconds", + label_selectors=['type="all"'], + by_labels=["req"], + hide_avg=True, + hide_count=True, + ), + graph_panel_histogram_quantiles( + title="Wait duration by store", + description="The time consumed when coprocessor requests are wait for being handled in each TiKV instance", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_coprocessor_request_wait_seconds", + label_selectors=['type="all"'], + by_labels=["req", "instance"], + hide_avg=True, + hide_count=True, + additional_groupby=False, + ), + ] + ) + layout.row( + [ + graph_panel( + title="Total DAG Requests", + targets=[ + target( + expr=expr_sum_rate( + "tikv_coprocessor_dag_request_count", + by_labels=["vec_type"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Total DAG Executors", + description="The total number of DAG executors", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_coprocessor_executor_count", + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Total Ops Details (Table Scan)", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_coprocessor_scan_details", + label_selectors=[ + 'req=~"select|select_by_in_memory_engine"' + ], + by_labels=["tag"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Total Ops Details (Index Scan)", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_coprocessor_scan_details", + label_selectors=['req=~"index|index_by_in_memory_engine"'], + by_labels=["tag"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Total Ops Details by CF (Table Scan)", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_coprocessor_scan_details", + label_selectors=[ + 'req=~"select|select_by_in_memory_engine"' + ], + by_labels=["cf", "tag"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Total Ops Details by CF (Index Scan)", + yaxes=yaxes(left_format=UNITS.OPS_PER_MIN), + targets=[ + target( + expr=expr_sum_rate( + "tikv_coprocessor_scan_details", + label_selectors=['req=~"index|index_by_in_memory_engine"'], + by_labels=["cf", "tag"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + heatmap_panel_graph_panel_histogram_quantile_pairs( + heatmap_title="Memory lock checking duration", + heatmap_description="The time consumed on checking memory locks for coprocessor requests", + graph_title="Memory lock checking duration", + graph_description="The time consumed on checking memory locks for coprocessor requests", + yaxis_format=UNITS.SECONDS, + metric="tikv_coprocessor_mem_lock_check_duration_seconds", + ), + ) + return layout.row_panel + + +def InMemoryEngine() -> RowPanel: + layout = Layout(title="In Memory Engine") + layout.row( + [ + graph_panel( + title="OPS", + description="Operation per second for cf", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_in_memory_engine_kv_operations", + by_labels=["instance", "type"], + ), + legend_format="{{type}}-{{instance}}", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Read MBps", + description="The total bytes of read in RocksDB and in-memory engine(the same with panel Cluster/MBps for read)", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_flow_bytes", + label_selectors=['type=~"bytes_read|iter_bytes_read"'], + ), + legend_format=r"rocksdb-{{instance}}", + ), + target( + expr=expr_sum_rate( + "tikv_in_memory_engine_flow", + label_selectors=['type=~"bytes_read|iter_bytes_read"'], + ), + legend_format=r"in-memory-engine-{{instance}}", + ), + ], + ), + graph_panel_histogram_quantiles( + title="Coprocessor Handle duration", + description="The time consumed when handling coprocessor requests", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_coprocessor_request_handle_seconds", + by_labels=["req"], + hide_avg=True, + hide_count=True, + ), + ] + ) + layout.row( + [ + graph_panel( + title="Region Cache Hit", + description="Count of region cache hit", + targets=[ + target( + expr=expr_sum_rate( + "tikv_snapshot_type_count", + label_selectors=['type="in_memory_engine"'], + by_labels=["instance"], + ), + legend_format="count-{{instance}}", + ), + ], + ), + graph_panel( + title="Region Cache Hit Rate", + description="Region cache hit rate", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_operator( + expr_sum_rate( + "tikv_snapshot_type_count", + label_selectors=['type="in_memory_engine"'], + by_labels=["instance"], + ), + "/", + expr_sum_rate( + "tikv_snapshot_type_count", + by_labels=["instance"], + ), + ), + legend_format="rate-{{instance}}", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Region Cache Miss Reason", + description="Reasons for region cache miss", + targets=[ + target( + expr=expr_sum_rate( + "tikv_in_memory_engine_snapshot_acquire_failed_reason_count", + by_labels=["instance", "type"], + ), + legend_format="{{type}}-{{instance}}", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Memory Usage", + description="The memory usage of the in-memory engine", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_avg( + "tikv_in_memory_engine_memory_usage_bytes", + by_labels=["instance"], + ), + ), + ], + ), + graph_panel( + title="Region Count", + description="The count of different types of region", + targets=[ + target( + expr=expr_avg( + "tikv_in_memory_engine_cache_count", + by_labels=["instance", "type"], + ), + legend_format="{{instance}}--{{type}}", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="GC Filter", + description="Rang cache engine garbage collection information", + targets=[ + target( + expr=expr_sum_rate( + "tikv_in_memory_engine_gc_filtered", + by_labels=["type"], + ), + legend_format="{{type}}", + additional_groupby=True, + ), + ], + ), + heatmap_panel( + title="Region GC Duration", + description="The handle duration of region gc", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_in_memory_engine_gc_duration_secs_bucket", + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Region Load Duration", + description="The handle duration of region load", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_in_memory_engine_load_duration_secs_bucket", + ), + graph_panel( + title="Region Load Count", + description="The count of region loading per seconds", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_delta( + "tikv_in_memory_engine_load_duration_secs_count", + by_labels=["instance"], + ), + legend_format="{{instance}}", + ), + ], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Region Eviction Duration", + description="The handle duration of region eviction", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_in_memory_engine_eviction_duration_secs_bucket", + ), + graph_panel( + title="Region Eviction Count", + description="The count of region eviction per seconds", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_delta( + "tikv_in_memory_engine_eviction_duration_secs_count", + by_labels=["type"], + ), + legend_format="{{type}}", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + heatmap_panel_graph_panel_histogram_quantile_pairs( + heatmap_title="Write duration", + heatmap_description="The time consumed of write in region cache engine", + graph_title="99% In-memory engine write duration per server", + graph_description="The time consumed of write in region cache engine per TiKV instance", + graph_by_labels=["instance"], + graph_hides=["count", "avg"], + yaxis_format=UNITS.SECONDS, + metric="tikv_in_memory_engine_write_duration_seconds", + ) + ) + layout.row( + heatmap_panel_graph_panel_histogram_quantile_pairs( + heatmap_title="Prepare for write duration", + heatmap_description="The time consumed of prepare for write in the in-memory engine", + graph_title="99% In-memory engine prepare for write duration per server", + graph_description="The time consumed of prepare for write in the in-memory engine per TiKV instance", + graph_by_labels=["instance"], + graph_hides=["count", "avg"], + yaxis_format=UNITS.SECONDS, + metric="tikv_in_memory_engine_prepare_for_write_duration_seconds", + ) + ) + layout.row( + [ + graph_panel( + title="Iterator operations", + description="The count of different type of iteration operations", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_in_memory_engine_locate", + label_selectors=[ + 'type="number_db_seek"', + ], + by_labels=[], # override default by instance. + ), + legend_format="seek", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_in_memory_engine_locate", + label_selectors=[ + 'type="number_db_seek_found"', + ], + by_labels=[], # override default by instance. + ), + legend_format="seek_found", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_in_memory_engine_locate", + label_selectors=[ + 'type="number_db_next"', + ], + by_labels=[], # override default by instance. + ), + legend_format="next", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_in_memory_engine_locate", + label_selectors=[ + 'type="number_db_next_found"', + ], + by_labels=[], # override default by instance. + ), + legend_format="next_found", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_in_memory_engine_locate", + label_selectors=[ + 'type="number_db_prev"', + ], + by_labels=[], # override default by instance. + ), + legend_format="prev", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_in_memory_engine_locate", + label_selectors=[ + 'type="number_db_prev_found"', + ], + by_labels=[], # override default by instance. + ), + legend_format="prev_found", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Seek duration", + description="The time consumed when executing seek operation", + yaxes=yaxes(left_format=UNITS.SECONDS, log_base=2), + targets=[ + target( + expr=expr_histogram_quantile( + 1, + "tikv_in_memory_engine_seek_duration", + ), + legend_format="max", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_in_memory_engine_seek_duration", + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.95, + "tikv_in_memory_engine_seek_duration", + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_histogram_avg( + "tikv_in_memory_engine_seek_duration", + by_labels=["type"], + ), + legend_format="avg", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Oldest Auto GC SafePoint", + description="Unlike the auto gc safe point used for TiKV, the safe point for in-memory engine is per region and this is the oldest one", + yaxes=yaxes(left_format=UNITS.DATE_TIME_ISO), + targets=[ + target( + expr=expr_max( + "tikv_in_memory_engine_oldest_safe_point", + ) + .extra("/ (2^18)") + .skip_default_instance_selector(), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Newest Auto GC SafePoint", + description="Unlike the auto gc safe point used for TiKV, the safe point for in-memory engine is per region and this is the newest one", + yaxes=yaxes(left_format=UNITS.DATE_TIME_ISO), + targets=[ + target( + expr=expr_max( + "tikv_in_memory_engine_newest_safe_point", + ) + .extra("/ (2^18)") + .skip_default_instance_selector(), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Auto GC SafePoint Gap", + description="The gap between newest auto gc safe point and oldest auto gc safe point of regions cached in the in-memroy engine", + yaxes=yaxes(left_format=UNITS.MILLI_SECONDS), + targets=[ + target( + expr=expr_operator( + expr_sum( + "tikv_in_memory_engine_newest_safe_point", + ) + .extra("/ (2^18)") + .skip_default_instance_selector(), + "-", + expr_sum( + "tikv_in_memory_engine_oldest_safe_point", + ) + .extra("/ (2^18)") + .skip_default_instance_selector(), + ), + additional_groupby=True, + legend_format="{{instance}}", + ), + ], + ), + graph_panel( + title="Auto GC SafePoint Gap With TiKV", + description="The gap between tikv auto gc safe point and in-memory engine oldest auto gc safe point", + yaxes=yaxes(left_format=UNITS.MILLI_SECONDS), + targets=[ + target( + expr=expr_max( + "tikv_safe_point_gap_with_in_memory_engine", + ) + .extra("/ (2^18)") + .skip_default_instance_selector(), + additional_groupby=True, + ), + ], + ), + ] + ) + return layout.row_panel + + +def Threads() -> RowPanel: + layout = Layout(title="Threads") + layout.row( + [ + graph_panel( + title="Threads state", + targets=[ + target( + expr=expr_sum( + "tikv_threads_state", + by_labels=["instance", "state"], + ), + ), + target( + expr=expr_sum( + "tikv_threads_state", + by_labels=["instance"], + ), + legend_format="{{instance}}-total", + ), + ], + ), + graph_panel( + title="Threads IO", + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC), + targets=[ + target( + expr=expr_topk( + 20, + "%s" + % expr_sum_rate( + "tikv_threads_io_bytes_total", + by_labels=["name", "io"], + ).extra("> 1024"), + ), + legend_format="{{name}}", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Thread Voluntary Context Switches", + targets=[ + target( + expr=expr_topk( + 20, + "%s" + % expr_max_rate( + "tikv_thread_voluntary_context_switches", + by_labels=["name"], + ).extra("> 100"), + ), + legend_format="{{name}}", + ), + ], + ), + graph_panel( + title="Thread Nonvoluntary Context Switches", + targets=[ + target( + expr=expr_topk( + 20, + "%s" + % expr_max_rate( + "tikv_thread_nonvoluntary_context_switches", + by_labels=["name"], + ).extra("> 100"), + ), + legend_format="{{name}}", + ), + ], + ), + ] + ) + return layout.row_panel + + +def RocksDB() -> RowPanel: + layout = Layout(title="RocksDB", repeat="db") + layout.row( + [ + graph_panel( + title="Get operations", + description="The count of get operations", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_memtable_efficiency", + label_selectors=[ + 'db="$db"', + 'type="memtable_hit"', + ], + by_labels=[], # override default by instance. + ), + legend_format="memtable", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type=~"block_cache_data_hit|block_cache_filter_hit"', + ], + by_labels=[], # override default by instance. + ), + legend_format="block_cache", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_get_served", + label_selectors=[ + 'db="$db"', + 'type="get_hit_l0"', + ], + by_labels=[], # override default by instance. + ), + legend_format="l0", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_get_served", + label_selectors=[ + 'db="$db"', + 'type="get_hit_l1"', + ], + by_labels=[], # override default by instance. + ), + legend_format="l1", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_get_served", + label_selectors=[ + 'db="$db"', + 'type="get_hit_l2_and_up"', + ], + by_labels=[], # override default by instance. + ), + legend_format="l2_and_up", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Get duration", + description="The time consumed when executing get operations", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS, log_base=2), + targets=[ + target( + expr=expr_max( + "tikv_engine_get_micro_seconds", + label_selectors=[ + 'db="$db"', + 'type="get_max"', + ], + by_labels=[], # override default by instance. + ), + legend_format="max", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_get_micro_seconds", + label_selectors=[ + 'db="$db"', + 'type="get_percentile99"', + ], + by_labels=[], # override default by instance. + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_get_micro_seconds", + label_selectors=[ + 'db="$db"', + 'type="get_percentile95"', + ], + by_labels=[], # override default by instance. + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_get_micro_seconds", + label_selectors=[ + 'db="$db"', + 'type="get_average"', + ], + by_labels=[], # override default by instance. + ), + legend_format="avg", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Seek operations", + description="The count of seek operations", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_locate", + label_selectors=[ + 'db="$db"', + 'type="number_db_seek"', + ], + by_labels=[], # override default by instance. + ), + legend_format="seek", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_locate", + label_selectors=[ + 'db="$db"', + 'type="number_db_seek_found"', + ], + by_labels=[], # override default by instance. + ), + legend_format="seek_found", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_locate", + label_selectors=[ + 'db="$db"', + 'type="number_db_next"', + ], + by_labels=[], # override default by instance. + ), + legend_format="next", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_locate", + label_selectors=[ + 'db="$db"', + 'type="number_db_next_found"', + ], + by_labels=[], # override default by instance. + ), + legend_format="next_found", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_locate", + label_selectors=[ + 'db="$db"', + 'type="number_db_prev"', + ], + by_labels=[], # override default by instance. + ), + legend_format="prev", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_locate", + label_selectors=[ + 'db="$db"', + 'type="number_db_prev_found"', + ], + by_labels=[], # override default by instance. + ), + legend_format="prev_found", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Seek duration", + description="The time consumed when executing seek operation", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS, log_base=2), + targets=[ + target( + expr=expr_max( + "tikv_engine_seek_micro_seconds", + label_selectors=[ + 'db="$db"', + 'type="seek_max"', + ], + by_labels=[], # override default by instance. + ), + legend_format="max", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_seek_micro_seconds", + label_selectors=[ + 'db="$db"', + 'type="seek_percentile99"', + ], + by_labels=[], # override default by instance. + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_seek_micro_seconds", + label_selectors=[ + 'db="$db"', + 'type="seek_percentile95"', + ], + by_labels=[], # override default by instance. + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_seek_micro_seconds", + label_selectors=[ + 'db="$db"', + 'type="seek_average"', + ], + by_labels=[], # override default by instance. + ), + legend_format="avg", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Write operations", + description="The count of write operations", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_write_served", + label_selectors=[ + 'db="$db"', + 'type=~"write_done_by_self|write_done_by_other"', + ], + by_labels=[], # override default by instance. + ), + legend_format="done", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_write_served", + label_selectors=[ + 'db="$db"', + 'type="write_timeout"', + ], + by_labels=[], # override default by instance. + ), + legend_format="timeout", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_write_served", + label_selectors=[ + 'db="$db"', + 'type="write_with_wal"', + ], + by_labels=[], # override default by instance. + ), + legend_format="with_wal", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Write duration", + description="The time consumed when executing write operation", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS, log_base=2), + targets=[ + target( + expr=expr_max( + "tikv_engine_write_micro_seconds", + label_selectors=[ + 'db="$db"', + 'type="write_max"', + ], + by_labels=[], # override default by instance. + ), + legend_format="max", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_write_micro_seconds", + label_selectors=[ + 'db="$db"', + 'type="write_percentile99"', + ], + by_labels=[], # override default by instance. + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_write_micro_seconds", + label_selectors=[ + 'db="$db"', + 'type="write_percentile95"', + ], + by_labels=[], # override default by instance. + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_write_micro_seconds", + label_selectors=[ + 'db="$db"', + 'type="write_average"', + ], + by_labels=[], # override default by instance. + ), + legend_format="avg", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="WAL sync operations", + description="The count of WAL sync operations", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_wal_file_synced", + label_selectors=[ + 'db="$db"', + ], + by_labels=[], # override default by instance. + ), + legend_format="sync", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Write WAL duration", + description="The time consumed when executing write wal operation", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS, log_base=2), + targets=[ + target( + expr=expr_max( + "tikv_engine_write_wal_time_micro_seconds", + label_selectors=[ + 'db="$db"', + 'type="write_wal_micros_max"', + ], + by_labels=[], # override default by instance. + ), + legend_format="max", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_write_wal_time_micro_seconds", + label_selectors=[ + 'db="$db"', + 'type="write_wal_micros_percentile99"', + ], + by_labels=[], # override default by instance. + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_write_wal_time_micro_seconds", + label_selectors=[ + 'db="$db"', + 'type="write_wal_micros_percentile95"', + ], + by_labels=[], # override default by instance. + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_write_wal_time_micro_seconds", + label_selectors=[ + 'db="$db"', + 'type="write_wal_micros_average"', + ], + by_labels=[], # override default by instance. + ), + legend_format="avg", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Compaction operations", + description="The count of compaction and flush operations", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_event_total", + label_selectors=[ + 'db="$db"', + ], + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="WAL sync duration", + description="The time consumed when executing WAL sync operation", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS, log_base=10), + targets=[ + target( + expr=expr_max( + "tikv_engine_wal_file_sync_micro_seconds", + label_selectors=[ + 'db="$db"', + 'type="wal_file_sync_max"', + ], + by_labels=[], # override default by instance. + ), + legend_format="max", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_wal_file_sync_micro_seconds", + label_selectors=[ + 'db="$db"', + 'type="wal_file_sync_percentile99"', + ], + by_labels=[], # override default by instance. + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_wal_file_sync_micro_seconds", + label_selectors=[ + 'db="$db"', + 'type="wal_file_sync_percentile95"', + ], + by_labels=[], # override default by instance. + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_wal_file_sync_micro_seconds", + label_selectors=[ + 'db="$db"', + 'type="wal_file_sync_average"', + ], + by_labels=[], # override default by instance. + ), + legend_format="avg", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Compaction guard actions", + description="Compaction guard actions", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_raftstore_compaction_guard_action_total", + label_selectors=[ + 'cf=~"default|write"', + ], + by_labels=["cf", " type"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Compaction duration", + description="The time consumed when executing the compaction and flush operations", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS, log_base=2), + targets=[ + target( + expr=expr_max( + "tikv_engine_compaction_time", + label_selectors=[ + 'db="$db"', + 'type="compaction_time_max"', + ], + by_labels=[], # override default by instance. + ), + legend_format="max", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_compaction_time", + label_selectors=[ + 'db="$db"', + 'type="compaction_time_percentile99"', + ], + by_labels=[], # override default by instance. + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_compaction_time", + label_selectors=[ + 'db="$db"', + 'type="compaction_time_percentile95"', + ], + by_labels=[], # override default by instance. + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_compaction_time", + label_selectors=[ + 'db="$db"', + 'type="compaction_time_average"', + ], + by_labels=[], # override default by instance. + ), + legend_format="avg", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Compaction Job Size(files)", + description="How many sst files are compacted in a compaction job", + yaxes=yaxes(left_format=UNITS.SHORT, log_base=2), + targets=[ + target( + expr=expr_max( + "tikv_engine_num_files_in_single_compaction", + label_selectors=[ + 'db="$db"', + 'type="num_files_in_single_compaction_max"', + ], + by_labels=[], # override default by instance. + ), + legend_format="max", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_num_files_in_single_compaction", + label_selectors=[ + 'db="$db"', + 'type="num_files_in_single_compaction_percentile99"', + ], + by_labels=[], # override default by instance. + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_num_files_in_single_compaction", + label_selectors=[ + 'db="$db"', + 'type="num_files_in_single_compaction_percentile95"', + ], + by_labels=[], # override default by instance. + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_num_files_in_single_compaction", + label_selectors=[ + 'db="$db"', + 'type="num_files_in_single_compaction_average"', + ], + by_labels=[], # override default by instance. + ), + legend_format="avg", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="SST read duration", + description="The time consumed when reading SST files", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS, log_base=2), + targets=[ + target( + expr=expr_max( + "tikv_engine_sst_read_micros", + label_selectors=[ + 'db="$db"', + 'type="sst_read_micros_max"', + ], + by_labels=[], # override default by instance. + ), + legend_format="max", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_sst_read_micros", + label_selectors=[ + 'db="$db"', + 'type="sst_read_micros_percentile99"', + ], + by_labels=[], # override default by instance. + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_sst_read_micros", + label_selectors=[ + 'db="$db"', + 'type="sst_read_micros_percentile95"', + ], + by_labels=[], # override default by instance. + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_sst_read_micros", + label_selectors=[ + 'db="$db"', + 'type="sst_read_micros_average"', + ], + by_labels=[], # override default by instance. + ), + legend_format="avg", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Compaction reason", + description=None, + yaxes=yaxes(left_format=UNITS.SHORT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_compaction_reason", + label_selectors=[ + 'db="$db"', + ], + by_labels=["cf", "reason"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Block cache size", + description="The block cache size. Broken down by column family if shared block cache is disabled.", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_topk( + 20, + "%s" + % expr_avg( + "tikv_engine_block_cache_size_bytes", + label_selectors=[ + 'db="$db"', + ], + by_labels=["cf", "instance"], + ), + ), + legend_format="{{instance}}-{{cf}}", + ), + ], + ), + graph_panel( + title="Memtable hit", + description="The hit rate of memtable", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_operator( + expr_sum_rate( + "tikv_engine_memtable_efficiency", + label_selectors=[ + 'db="$db"', + 'type="memtable_hit"', + ], + by_labels=[], # override default by instance. + ), + "/", + expr_operator( + expr_sum_rate( + "tikv_engine_memtable_efficiency", + label_selectors=[ + 'db="$db"', + 'type="memtable_hit"', + ], + by_labels=[], # override default by instance. + ), + "+", + expr_sum_rate( + "tikv_engine_memtable_efficiency", + label_selectors=[ + 'db="$db"', + 'type="memtable_miss"', + ], + by_labels=[], # override default by instance. + ), + ), + ), + legend_format="hit", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Block cache flow", + description="The flow of different kinds of block cache operations", + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC, log_base=10), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_flow_bytes", + label_selectors=[ + 'db="$db"', + 'type="block_cache_byte_read"', + ], + by_labels=[], # override default by instance. + ), + legend_format="total_read", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_flow_bytes", + label_selectors=[ + 'db="$db"', + 'type="block_cache_byte_write"', + ], + by_labels=[], # override default by instance. + ), + legend_format="total_written", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_data_bytes_insert"', + ], + by_labels=[], # override default by instance. + ), + legend_format="data_insert", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_filter_bytes_insert"', + ], + by_labels=[], # override default by instance. + ), + legend_format="filter_insert", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_filter_bytes_evict"', + ], + by_labels=[], # override default by instance. + ), + legend_format="filter_evict", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_index_bytes_insert"', + ], + by_labels=[], # override default by instance. + ), + legend_format="index_insert", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_index_bytes_evict"', + ], + by_labels=[], # override default by instance. + ), + legend_format="index_evict", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Block cache hit", + description="The hit rate of block cache", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_operator( + expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_hit"', + ], + by_labels=[], # override default by instance. + ), + "/", + expr_operator( + expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_hit"', + ], + by_labels=[], # override default by instance. + ), + "+", + expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_miss"', + ], + by_labels=[], # override default by instance. + ), + ), + ), + legend_format="all", + additional_groupby=True, + ), + target( + expr=expr_operator( + expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_data_hit"', + ], + by_labels=[], # override default by instance. + ), + "/", + expr_operator( + expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_data_hit"', + ], + by_labels=[], # override default by instance. + ), + "+", + expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_data_miss"', + ], + by_labels=[], # override default by instance. + ), + ), + ), + legend_format="data", + additional_groupby=True, + ), + target( + expr=expr_operator( + expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_filter_hit"', + ], + by_labels=[], # override default by instance. + ), + "/", + expr_operator( + expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_filter_hit"', + ], + by_labels=[], # override default by instance. + ), + "+", + expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_filter_miss"', + ], + by_labels=[], # override default by instance. + ), + ), + ), + legend_format="filter", + additional_groupby=True, + ), + target( + expr=expr_operator( + expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_index_hit"', + ], + by_labels=[], # override default by instance. + ), + "/", + expr_operator( + expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_index_hit"', + ], + by_labels=[], # override default by instance. + ), + "+", + expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_index_miss"', + ], + by_labels=[], # override default by instance. + ), + ), + ), + legend_format="index", + additional_groupby=True, + ), + target( + expr=expr_operator( + expr_operator( + expr_sum_rate( + "tikv_engine_bloom_efficiency", + label_selectors=[ + 'db="$db"', + 'type="last_level_seek_filtered"', + ], + by_labels=[], # override default by instance. + ), + "+", + expr_sum_rate( + "tikv_engine_bloom_efficiency", + label_selectors=[ + 'db="$db"', + 'type="non_last_level_seek_filtered"', + ], + by_labels=[], # override default by instance. + ), + ), + "/", + expr_operator( + expr_operator( + expr_sum_rate( + "tikv_engine_bloom_efficiency", + label_selectors=[ + 'db="$db"', + 'type="last_level_seek_filtered"', + ], + by_labels=[], # override default by instance. + ), + "+", + expr_sum_rate( + "tikv_engine_bloom_efficiency", + label_selectors=[ + 'db="$db"', + 'type="non_last_level_seek_filtered"', + ], + by_labels=[], # override default by instance. + ), + ), + "+", + expr_operator( + expr_sum_rate( + "tikv_engine_bloom_efficiency", + label_selectors=[ + 'db="$db"', + 'type="last_level_seek_filter_match"', + ], + by_labels=[], # override default by instance. + ), + "+", + expr_sum_rate( + "tikv_engine_bloom_efficiency", + label_selectors=[ + 'db="$db"', + 'type="non_last_level_seek_filter_match"', + ], + by_labels=[], # override default by instance. + ), + ), + ), + ), + legend_format="bloom prefix", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Keys flow", + description="The flow of different kinds of operations on keys", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_flow_bytes", + label_selectors=[ + 'db="$db"', + 'type="keys_read"', + ], + by_labels=[], # override default by instance. + ), + legend_format="read", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_flow_bytes", + label_selectors=[ + 'db="$db"', + 'type="keys_written"', + ], + by_labels=[], # override default by instance. + ), + legend_format="written", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_compaction_num_corrupt_keys", + label_selectors=[ + 'db="$db"', + ], + by_labels=[], # override default by instance. + ), + legend_format="corrupt", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Block cache operations", + description="The count of different kinds of block cache operations", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_add"', + ], + by_labels=[], # override default by instance. + ), + legend_format="total_add", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_data_add"', + ], + by_labels=[], # override default by instance. + ), + legend_format="data_add", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_filter_add"', + ], + by_labels=[], # override default by instance. + ), + legend_format="filter_add", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_index_add"', + ], + by_labels=[], # override default by instance. + ), + legend_format="index_add", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_cache_efficiency", + label_selectors=[ + 'db="$db"', + 'type="block_cache_add_failures"', + ], + by_labels=[], # override default by instance. + ), + legend_format="add_failures", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Read flow", + description="The flow rate of read operations per type", + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_flow_bytes", + label_selectors=[ + 'db="$db"', + 'type="bytes_read"', + ], + by_labels=[], # override default by instance. + ), + legend_format="get", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_flow_bytes", + label_selectors=[ + 'db="$db"', + 'type="iter_bytes_read"', + ], + by_labels=[], # override default by instance. + ), + legend_format="scan", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Total keys", + description="The count of keys in each column family", + yaxes=yaxes(left_format=UNITS.SHORT), + targets=[ + target( + expr=expr_sum( + "tikv_engine_estimate_num_keys", + label_selectors=[ + 'db="$db"', + ], + by_labels=["cf"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Write flow", + description="The flow of different kinds of write operations", + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_flow_bytes", + label_selectors=[ + 'db="$db"', + 'type="wal_file_bytes"', + ], + by_labels=[], # override default by instance. + ), + legend_format="wal", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_flow_bytes", + label_selectors=[ + 'db="$db"', + 'type="bytes_written"', + ], + by_labels=[], # override default by instance. + ), + legend_format="write", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Bytes / Read", + description="The bytes per read", + yaxes=yaxes(left_format=UNITS.BYTES_IEC, log_base=10), + targets=[ + target( + expr=expr_max( + "tikv_engine_bytes_per_read", + label_selectors=[ + 'db="$db"', + 'type="bytes_per_read_max"', + ], + by_labels=[], # override default by instance. + ), + legend_format="max", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_bytes_per_read", + label_selectors=[ + 'db="$db"', + 'type="bytes_per_read_percentile99"', + ], + by_labels=[], # override default by instance. + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_bytes_per_read", + label_selectors=[ + 'db="$db"', + 'type="bytes_per_read_percentile95"', + ], + by_labels=[], # override default by instance. + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_bytes_per_read", + label_selectors=[ + 'db="$db"', + 'type="bytes_per_read_average"', + ], + by_labels=[], # override default by instance. + ), + legend_format="avg", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Compaction flow", + description="The flow rate of compaction operations per type", + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_compaction_flow_bytes", + label_selectors=[ + 'db="$db"', + 'type="bytes_read"', + ], + by_labels=[], # override default by instance. + ), + legend_format="read", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_compaction_flow_bytes", + label_selectors=[ + 'db="$db"', + 'type="bytes_written"', + ], + by_labels=[], # override default by instance. + ), + legend_format="written", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_flow_bytes", + label_selectors=[ + 'db="$db"', + 'type="flush_write_bytes"', + ], + by_labels=[], # override default by instance. + ), + legend_format="flushed", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Bytes / Write", + description="The bytes per write", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_max( + "tikv_engine_bytes_per_write", + label_selectors=['db="$db"', 'type="bytes_per_write_max"'], + by_labels=[], # override default by instance. + ), + legend_format="max", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_bytes_per_write", + label_selectors=[ + 'db="$db"', + 'type="bytes_per_write_percentile99"', + ], + by_labels=[], # override default by instance. + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_bytes_per_write", + label_selectors=[ + 'db="$db"', + 'type="bytes_per_write_percentile95"', + ], + by_labels=[], # override default by instance. + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_bytes_per_write", + label_selectors=[ + 'db="$db"', + 'type="bytes_per_write_average"', + ], + by_labels=[], # override default by instance. + ), + legend_format="avg", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Read amplification", + description="The read amplification per TiKV instance", + yaxes=yaxes(left_format=UNITS.SHORT), + targets=[ + target( + expr=expr_operator( + expr_sum_rate( + "tikv_engine_read_amp_flow_bytes", + label_selectors=[ + 'db="$db"', + 'type="read_amp_total_read_bytes"', + ], + ), + "/", + expr_sum_rate( + "tikv_engine_read_amp_flow_bytes", + label_selectors=[ + 'db="$db"', + 'type="read_amp_estimate_useful_bytes"', + ], + ), + ), + legend_format="{{instance}}", + ), + ], + ), + graph_panel( + title="Compaction pending bytes", + description="The pending bytes to be compacted", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum( + "tikv_engine_pending_compaction_bytes", + label_selectors=['db="$db"'], + by_labels=["cf"], + ), + legend_format="{{cf}}", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Number of snapshots", + description="The number of snapshot of each TiKV instance", + yaxes=yaxes(left_format=UNITS.SHORT), + targets=[ + target( + expr=expr_simple( + "tikv_engine_num_snapshots", + label_selectors=['db="$db"'], + ), + legend_format="{{instance}}", + ), + ], + ), + graph_panel( + title="Compression ratio", + description="The compression ratio of each level", + yaxes=yaxes(left_format=UNITS.SHORT), + targets=[ + target( + expr=expr_avg( + "tikv_engine_compression_ratio", + label_selectors=['db="$db"'], + by_labels=["cf", "level"], + ), + legend_format="{{cf}}-L{{level}}", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Number files at each level", + description="The number of SST files for different column families in each level", + yaxes=yaxes(left_format=UNITS.SHORT), + targets=[ + target( + expr=expr_avg( + "tikv_engine_num_files_at_level", + label_selectors=['db="$db"'], + by_labels=["cf", "level"], + ), + legend_format="{{cf}}-L{{level}}", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Oldest snapshots duration", + description="The time that the oldest unreleased snapshot survivals", + yaxes=yaxes(left_format=UNITS.SECONDS), + targets=[ + target( + expr=expr_simple( + "tikv_engine_oldest_snapshot_duration", + label_selectors=['db="$db"'], + ), + legend_format="{{instance}}", + ), + ], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Ingestion picked level", + description="The level that the external file ingests into", + yaxis=yaxis(format=UNITS.SHORT), + metric="tikv_engine_ingestion_picked_level_bucket", + label_selectors=['db="$db"'], + ), + graph_panel_histogram_quantiles( + title="Ingest SST duration seconds", + description="Bucketed histogram of ingest external SST files duration.", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_storage_ingest_external_file_duration_secs", + by_labels=["cf", "type"], + hide_count=True, + ), + ] + ) + layout.row( + [ + graph_panel( + title="Write Stall Reason", + description=None, + yaxes=yaxes(left_format=UNITS.SHORT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_write_stall_reason", + label_selectors=['db="$db"'], + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Write stall duration", + description="The time which is caused by write stall", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS), + targets=[ + target( + expr=expr_max( + "tikv_engine_write_stall", + label_selectors=['db="$db"', 'type="write_stall_max"'], + by_labels=[], # override default by instance. + ), + legend_format="max", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_write_stall", + label_selectors=[ + 'db="$db"', + 'type="write_stall_percentile99"', + ], + by_labels=[], # override default by instance. + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_write_stall", + label_selectors=[ + 'db="$db"', + 'type="write_stall_percentile95"', + ], + by_labels=[], # override default by instance. + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_write_stall", + label_selectors=['db="$db"', 'type="write_stall_average"'], + by_labels=[], # override default by instance. + ), + legend_format="avg", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Stall conditions changed of each CF", + description="Stall conditions changed of each column family", + yaxes=yaxes(left_format=UNITS.SHORT), + targets=[ + target( + expr=expr_simple( + "tikv_engine_stall_conditions_changed", + label_selectors=['db="$db"'], + ), + legend_format="{{instance}}-{{cf}}-{{type}}", + ), + ], + ), + graph_panel( + title="Memtable size", + description="The memtable size of each column family", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_avg( + "tikv_engine_memory_bytes", + label_selectors=['db="$db"', 'type="mem-tables-all"'], + by_labels=["cf"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + return layout.row_panel + + +def RaftEngine() -> RowPanel: + layout = Layout(title="Raft Engine") + layout.row( + [ + graph_panel( + title="Operation", + description="The count of operations per second", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "raft_engine_write_apply_duration_seconds_count", + by_labels=[], # override default by instance. + ), + legend_format="write", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "raft_engine_read_entry_duration_seconds_count", + by_labels=[], # override default by instance. + ), + legend_format="read_entry", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "raft_engine_read_message_duration_seconds_count", + by_labels=[], # override default by instance. + ), + legend_format="read_message", + additional_groupby=True, + ), + ], + ), + graph_panel_histogram_quantiles( + title="Write Duration", + description="The time used in write operation", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="raft_engine_write_duration_seconds", + hide_count=True, + ), + ] + ) + layout.row( + [ + graph_panel( + title="Flow", + description="The I/O flow rate", + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC), + targets=[ + target( + expr=expr_sum_rate( + "raft_engine_write_size_sum", + by_labels=[], # override default by instance. + ), + legend_format="write", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "raft_engine_background_rewrite_bytes_sum", + by_labels=["type"], + ), + legend_format="rewrite-{{type}}", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Write Duration Breakdown (99%)", + description="99% duration breakdown of write operation", + yaxes=yaxes(left_format=UNITS.SECONDS), + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, "raft_engine_write_preprocess_duration_seconds" + ), + legend_format="wait", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.99, "raft_engine_write_leader_duration_seconds" + ), + legend_format="wal", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.99, "raft_engine_write_apply_duration_seconds" + ), + legend_format="apply", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel_histogram_quantiles( + title="Bytes / Written", + description="The bytes per write", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + metric="raft_engine_write_size", + hide_count=True, + ), + graph_panel( + title="WAL Duration Breakdown (999%)", + description="999% duration breakdown of WAL write operation", + yaxes=yaxes(left_format=UNITS.SECONDS), + targets=[ + target( + expr=expr_histogram_quantile( + 0.999, "raft_engine_write_leader_duration_seconds" + ), + legend_format="total", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.999, "raft_engine_sync_log_duration_seconds" + ), + legend_format="sync", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.999, "raft_engine_allocate_log_duration_seconds" + ), + legend_format="allocate", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.999, "raft_engine_rotate_log_duration_seconds" + ), + legend_format="rotate", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="File Count", + description="The average number of files", + yaxes=yaxes(left_format=UNITS.SHORT), + targets=[ + target( + expr=expr_avg( + "raft_engine_log_file_count", + by_labels=["type"], + ), + additional_groupby=True, + ), + target( + expr=expr_avg( + "raft_engine_swap_file_count", + by_labels=[], # override default by instance. + ), + legend_format="swap", + additional_groupby=True, + ), + target( + expr=expr_avg( + "raft_engine_recycled_file_count", + by_labels=["type"], + ), + legend_format="{{type}}-recycle", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Other Durations (99%)", + description="The 99% duration of operations other than write", + yaxes=yaxes(left_format=UNITS.SECONDS, log_base=2), + targets=[ + target( + expr=expr_histogram_quantile( + 0.999, "raft_engine_read_entry_duration_seconds" + ), + legend_format="read_entry", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.999, "raft_engine_read_message_duration_seconds" + ), + legend_format="read_message", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.999, "raft_engine_purge_duration_seconds" + ), + legend_format="purge", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Entry Count", + description="The average number of log entries", + yaxes=yaxes(left_format=UNITS.SHORT), + targets=[ + target( + expr=expr_avg( + "raft_engine_log_entry_count", + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel_histogram_quantiles( + title="Write Compression Ratio", + description="The compression ratio per write", + yaxes=yaxes(left_format=UNITS.NONE_FORMAT), + metric="raft_engine_write_compression_ratio", + ), + ] + ) + return layout.row_panel + + +def Titan() -> RowPanel: + layout = Layout(title="Titan", repeat="titan_db") + layout.row( + [ + graph_panel( + title="Blob file count", + targets=[ + target( + expr=expr_sum( + "tikv_engine_titandb_num_live_blob_file", + label_selectors=['db="$titan_db"'], + by_labels=[], # override default by instance. + ), + legend_format="live blob file num", + additional_groupby=True, + ), + target( + expr=expr_sum( + "tikv_engine_titandb_num_obsolete_blob_file", + label_selectors=['db="$titan_db"'], + by_labels=[], # override default by instance. + ), + legend_format="obsolete blob file num", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Blob file size", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum( + "tikv_engine_titandb_live_blob_file_size", + label_selectors=['db="$titan_db"'], + by_labels=[], # override default by instance. + ), + legend_format="live blob file size", + additional_groupby=True, + ), + target( + expr=expr_sum( + "tikv_engine_titandb_obsolete_blob_file_size", + label_selectors=['db="$titan_db"'], + by_labels=[], # override default by instance. + ), + legend_format="obsolete blob file size", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Blob cache size", + description="The blob cache size.", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_topk( + 20, + "%s" + % expr_avg( + "tikv_engine_blob_cache_size_bytes", + label_selectors=['db="$titan_db"'], + by_labels=["cf", "instance"], + ), + ), + legend_format="{{instance}}-{{cf}}", + ), + ], + ), + graph_panel( + title="Blob cache hit", + description="The hit rate of block cache", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_operator( + expr_sum_rate( + "tikv_engine_blob_cache_efficiency", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_cache_hit"', + ], + by_labels=[], # override default by instance. + ), + "/", + expr_operator( + expr_sum_rate( + "tikv_engine_blob_cache_efficiency", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_cache_hit"', + ], + by_labels=[], # override default by instance. + ), + "+", + expr_sum_rate( + "tikv_engine_blob_cache_efficiency", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_cache_miss"', + ], + by_labels=[], # override default by instance. + ), + ), + ), + legend_format="all", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Iter touched blob file count", + targets=[ + target( + expr=expr_avg( + "tikv_engine_blob_iter_touch_blob_file_count", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_iter_touch_blob_file_count_average"', + ], + by_labels=[], # override default by instance. + ), + legend_format="avg", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_iter_touch_blob_file_count", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_iter_touch_blob_file_count_percentile95"', + ], + by_labels=[], # override default by instance. + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_iter_touch_blob_file_count", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_iter_touch_blob_file_count_percentile99"', + ], + by_labels=[], # override default by instance. + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_max( + "tikv_engine_blob_iter_touch_blob_file_count", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_iter_touch_blob_file_count_max"', + ], + by_labels=[], # override default by instance. + ), + legend_format="max", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Blob key size", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_avg( + "tikv_engine_blob_key_size", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_key_size_average"', + ], + by_labels=[], # override default by instance. + ), + legend_format="avg", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_key_size", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_key_size_percentile95"', + ], + by_labels=[], # override default by instance. + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_key_size", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_key_size_percentile99"', + ], + by_labels=[], # override default by instance. + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_max( + "tikv_engine_blob_key_size", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_key_size_max"', + ], + by_labels=[], # override default by instance. + ), + legend_format="max", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Blob value size", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_avg( + "tikv_engine_blob_value_size", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_value_size_average"', + ], + by_labels=[], # override default by instance. + ), + legend_format="avg", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_value_size", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_value_size_percentile95"', + ], + by_labels=[], # override default by instance. + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_value_size", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_value_size_percentile99"', + ], + by_labels=[], # override default by instance. + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_max( + "tikv_engine_blob_value_size", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_value_size_max"', + ], + by_labels=[], # override default by instance. + ), + legend_format="max", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Blob get operations", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_blob_locate", + label_selectors=[ + 'db="$titan_db"', + 'type="number_blob_get"', + ], + by_labels=[], # override default by instance. + ), + legend_format="get", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Blob get duration", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS), + targets=[ + target( + expr=expr_avg( + "tikv_engine_blob_get_micros_seconds", + label_selectors=['db="$titan_db"', 'type=~".*_average"'], + by_labels=["type"], + ), + legend_format="avg-{{type}}", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_get_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type=~".*_percentile95"', + ], + by_labels=["type"], + ), + legend_format="95%-{{type}}", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_get_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type=~".*_percentile99"', + ], + by_labels=["type"], + ), + legend_format="99%-{{type}}", + additional_groupby=True, + ), + target( + expr=expr_max( + "tikv_engine_blob_get_micros_seconds", + label_selectors=['db="$titan_db"', 'type=~".*_max"'], + by_labels=["type"], + ), + legend_format="max-{{type}}", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Blob file discardable ratio distribution", + targets=[ + target( + expr=expr_sum( + "tikv_engine_titandb_blob_file_discardable_ratio", + label_selectors=['db="$titan_db"'], + by_labels=["ratio"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Blob iter operations", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_blob_locate", + label_selectors=[ + 'db="$titan_db"', + 'type="number_blob_seek"', + ], + by_labels=[], # override default by instance. + ), + legend_format="seek", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_blob_locate", + label_selectors=[ + 'db="$titan_db"', + 'type="number_blob_prev"', + ], + by_labels=[], # override default by instance. + ), + legend_format="prev", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_engine_blob_locate", + label_selectors=[ + 'db="$titan_db"', + 'type="number_blob_next"', + ], + by_labels=[], # override default by instance. + ), + legend_format="next", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Blob seek duration", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS), + targets=[ + target( + expr=expr_avg( + "tikv_engine_blob_seek_micros_seconds", + label_selectors=['db="$titan_db"', 'type=~".*_average"'], + by_labels=[], # override default by instance. + ), + legend_format="avg", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_seek_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type=~".*_percentile95"', + ], + by_labels=[], # override default by instance. + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_seek_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type=~".*_percentile99"', + ], + by_labels=[], # override default by instance. + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_max( + "tikv_engine_blob_seek_micros_seconds", + label_selectors=['db="$titan_db"', 'type=~".*_max"'], + by_labels=[], # override default by instance. + ), + legend_format="max", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Blob next duration", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS), + targets=[ + target( + expr=expr_avg( + "tikv_engine_blob_next_micros_seconds", + label_selectors=['db="$titan_db"', 'type=~".*_average"'], + by_labels=[], # override default by instance. + ), + legend_format="avg", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_next_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type=~".*_percentile95"', + ], + by_labels=[], # override default by instance. + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_next_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type=~".*_percentile99"', + ], + by_labels=[], # override default by instance. + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_max( + "tikv_engine_blob_next_micros_seconds", + label_selectors=['db="$titan_db"', 'type=~".*_max"'], + by_labels=[], # override default by instance. + ), + legend_format="max", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Blob prev duration", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS), + targets=[ + target( + expr=expr_avg( + "tikv_engine_blob_prev_micros_seconds", + label_selectors=['db="$titan_db"', 'type=~".*_average"'], + by_labels=["type"], + ), + legend_format="avg-{{type}}", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_prev_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type=~".*_percentile95"', + ], + by_labels=["type"], + ), + legend_format="95%-{{type}}", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_prev_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type=~".*_percentile99"', + ], + by_labels=["type"], + ), + legend_format="99%-{{type}}", + additional_groupby=True, + ), + target( + expr=expr_max( + "tikv_engine_blob_prev_micros_seconds", + label_selectors=['db="$titan_db"', 'type=~".*_max"'], + by_labels=["type"], + ), + legend_format="max-{{type}}", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Blob keys flow", + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_blob_flow_bytes", + label_selectors=['db="$titan_db"', 'type=~"keys.*"'], + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Blob file read duration", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS), + targets=[ + target( + expr=expr_avg( + "tikv_engine_blob_file_read_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_file_read_micros_average"', + ], + by_labels=["type"], + ), + legend_format="avg", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_file_read_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_file_read_micros_percentile99"', + ], + by_labels=["type"], + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_file_read_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_file_read_micros_percentile95"', + ], + by_labels=["type"], + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_max( + "tikv_engine_blob_file_read_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_file_read_micros_max"', + ], + by_labels=["type"], + ), + legend_format="max", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Blob bytes flow", + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_blob_flow_bytes", + label_selectors=['db="$titan_db"', 'type=~"bytes.*"'], + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Blob file write duration", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS), + targets=[ + target( + expr=expr_avg( + "tikv_engine_blob_file_write_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_file_write_micros_average"', + ], + by_labels=["type"], + ), + legend_format="avg", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_file_write_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_file_write_micros_percentile99"', + ], + by_labels=["type"], + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_file_write_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_file_write_micros_percentile95"', + ], + by_labels=["type"], + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_max( + "tikv_engine_blob_file_write_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_file_write_micros_max"', + ], + by_labels=["type"], + ), + legend_format="max", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Blob file sync operations", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_blob_file_synced", + label_selectors=['db="$titan_db"'], + by_labels=[], # override default by instance. + ), + legend_format="sync", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Blob GC action", + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_blob_gc_action_count", + label_selectors=['db="$titan_db"'], + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Blob file sync duration", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS), + targets=[ + target( + expr=expr_avg( + "tikv_engine_blob_file_sync_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_file_sync_micros_average"', + ], + by_labels=["type"], + ), + legend_format="avg", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_file_sync_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_file_sync_micros_percentile95"', + ], + by_labels=["type"], + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_file_sync_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_file_sync_micros_percentile99"', + ], + by_labels=["type"], + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_max( + "tikv_engine_blob_file_sync_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_file_sync_micros_max"', + ], + by_labels=["type"], + ), + legend_format="max", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Blob GC duration", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS), + targets=[ + target( + expr=expr_avg( + "tikv_engine_blob_gc_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_gc_micros_average"', + ], + by_labels=["type"], + ), + legend_format="avg", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_gc_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_gc_micros_percentile95"', + ], + by_labels=["type"], + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_gc_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_gc_micros_percentile99"', + ], + by_labels=["type"], + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_max( + "tikv_engine_blob_gc_micros_seconds", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_gc_micros_max"', + ], + by_labels=["type"], + ), + legend_format="max", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Blob GC keys flow", + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_blob_gc_flow_bytes", + label_selectors=['db="$titan_db"', 'type=~"keys.*"'], + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Blob GC input file size", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_avg( + "tikv_engine_blob_gc_input_file", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_gc_input_file_average"', + ], + by_labels=[], # override default by instance. + ), + legend_format="avg", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_gc_input_file", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_gc_input_file_percentile95"', + ], + by_labels=[], # override default by instance. + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_gc_input_file", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_gc_input_file_percentile99"', + ], + by_labels=[], # override default by instance. + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_max( + "tikv_engine_blob_gc_input_file", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_gc_input_file_max"', + ], + by_labels=[], # override default by instance. + ), + legend_format="max", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Blob GC bytes flow", + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_blob_gc_flow_bytes", + label_selectors=['db="$titan_db"', 'type=~"bytes.*"'], + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Blob GC output file size", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_avg( + "tikv_engine_blob_gc_output_file", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_gc_output_file_average"', + ], + by_labels=[], # override default by instance. + ), + legend_format="avg", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_gc_output_file", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_gc_output_file_percentile95"', + ], + by_labels=[], # override default by instance. + ), + legend_format="95%", + additional_groupby=True, + ), + target( + expr=expr_avg( + "tikv_engine_blob_gc_output_file", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_gc_output_file_percentile99"', + ], + by_labels=[], # override default by instance. + ), + legend_format="99%", + additional_groupby=True, + ), + target( + expr=expr_max( + "tikv_engine_blob_gc_output_file", + label_selectors=[ + 'db="$titan_db"', + 'type="blob_gc_output_file_max"', + ], + by_labels=[], # override default by instance. + ), + legend_format="max", + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Blob GC file count", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_engine_blob_gc_file_count", + label_selectors=['db="$titan_db"'], + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + return layout.row_panel + + +def PessimisticLocking() -> RowPanel: + layout = Layout(title="Pessimistic Locking") + layout.row( + [ + graph_panel( + title="Lock Manager Thread CPU", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"waiter_manager.*"'], + by_labels=["instance", "name"], + ), + ), + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"deadlock_detect.*"'], + by_labels=["instance", "name"], + ), + ), + ], + ), + graph_panel( + title="Lock Manager Handled tasks", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_lock_manager_task_counter", + by_labels=["type"], + ), + additional_groupby=True, + ) + ], + ), + ] + ) + layout.row( + [ + graph_panel_histogram_quantiles( + title="Waiter lifetime duration", + description="", + yaxes=yaxes(left_format=UNITS.SECONDS, log_base=2), + metric="tikv_lock_manager_waiter_lifetime_duration", + hide_count=True, + ), + graph_panel( + title="Lock Waiting Queue", + yaxes=yaxes(left_format=UNITS.SHORT), + targets=[ + target( + expr=expr_sum_aggr_over_time( + "tikv_lock_manager_wait_table_status", + "max", + "30s", + by_labels=["type"], + ), + additional_groupby=True, + ), + target( + expr=expr_sum_aggr_over_time( + "tikv_lock_wait_queue_entries_gauge_vec", + "max", + "30s", + by_labels=["type"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel_histogram_quantiles( + title="Deadlock detect duration", + description="", + yaxes=yaxes(left_format=UNITS.SECONDS, log_base=2), + metric="tikv_lock_manager_detect_duration", + hide_count=True, + ), + graph_panel( + title="Detect error", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_lock_manager_error_counter", by_labels=["type"] + ), + additional_groupby=True, + ) + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Deadlock detector leader", + targets=[ + target( + expr=expr_sum_aggr_over_time( + "tikv_lock_manager_detector_leader_heartbeat", + "max", + "30s", + ), + ) + ], + ), + graph_panel( + title="Total pessimistic locks memory size", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_simple("tikv_pessimistic_lock_memory_size"), + legend_format="{{instance}}", + ) + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="In-memory pessimistic locking result", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_in_memory_pessimistic_locking", by_labels=["result"] + ), + additional_groupby=True, + ) + ], + ), + graph_panel( + title="Pessimistic lock activities", + description="The number of active keys and waiters.", + targets=[ + target( + expr=expr_sum( + "tikv_lock_wait_queue_entries_gauge_vec", by_labels=["type"] + ), + additional_groupby=True, + ) + ], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Lengths of lock wait queues when transaction enqueues", + description="The length includes the entering transaction itself", + yaxis=yaxis(format=UNITS.SHORT), + metric="tikv_lock_wait_queue_length_bucket", + ), + graph_panel_histogram_quantiles( + title="In-memory scan lock read duration", + description="The duration scan in-memory pessimistic locks with read lock", + yaxes=yaxes(left_format=UNITS.SECONDS, log_base=2), + metric="tikv_storage_mvcc_scan_lock_read_duration_seconds", + by_labels=["type"], + hide_count=True, + hide_avg=True, + ), + ] + ) + return layout.row_panel + + +def PointInTimeRestore() -> RowPanel: + layout = Layout(title="Point In Time Restore") + layout.row( + [ + graph_panel( + title="CPU Usage", + description=None, + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=[ + 'name=~"sst_.*"', + ], + ), + ), + ], + ), + graph_panel( + title="P99 RPC Duration", + description=None, + yaxes=yaxes(left_format=UNITS.SECONDS, log_base=1), + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_import_rpc_duration", + label_selectors=[ + 'request="apply"', + ], + ), + legend_format="total-99", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_import_apply_duration", + label_selectors=[ + 'type=~"queue|exec_download"', + ], + by_labels=["le", "type"], + ), + legend_format="(DL){{type}}-99", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_import_engine_request", + by_labels=["le", "type"], + ), + legend_format="(AP){{type}}-99", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Import RPC Ops", + description="", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_import_rpc_duration_count", + label_selectors=[ + 'request="apply"', + ], + by_labels=["instance", "request"], + ), + ), + target( + expr=expr_sum_rate( + "tikv_import_rpc_duration_count", + label_selectors=[ + 'request!="switch_mode"', + ], + by_labels=["request"], + ), + legend_format="total-{{request}}", + ), + ], + ), + graph_panel( + title="Import RPC Count", + targets=[ + target( + expr=expr_simple( + "tikv_import_rpc_count", + label_selectors=[ + 'type="apply"', + ], + ), + legend_format="{{type}}-{{instance}}", + ), + ], + ), + graph_panel( + title="Cache Events", + description=None, + yaxes=yaxes(left_format=UNITS.COUNTS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_import_apply_cache_event", + label_selectors=[], + by_labels=["type", "instance"], + ), + ), + ], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Overall RPC Duration", + description=None, + yaxis=yaxis(format=UNITS.SECONDS, log_base=1), + metric="tikv_import_rpc_duration_bucket", + label_selectors=[ + 'request="apply"', + ], + ), + heatmap_panel( + title="Read File into Memory Duration", + description=None, + yaxis=yaxis(format=UNITS.SECONDS, log_base=1), + metric="tikv_import_apply_duration_bucket", + label_selectors=[ + 'type="exec_download"', + ], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Queuing Time", + description=None, + yaxis=yaxis(format=UNITS.SECONDS, log_base=1), + metric="tikv_import_engine_request_bucket", + label_selectors=[ + 'type="queuing"', + ], + ), + graph_panel( + title="Apply Request Throughput", + description=None, + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_import_apply_bytes_sum", + ), + ), + ], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Downloaded File Size", + description=None, + yaxis=yaxis(format=UNITS.BYTES_IEC), + metric="tikv_import_download_bytes_bucket", + ), + heatmap_panel( + title="Apply Batch Size", + description=None, + yaxis=yaxis(format=UNITS.BYTES_IEC), + metric="tikv_import_apply_bytes_bucket", + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Blocked by Concurrency Time", + description=None, + yaxis=yaxis(format=UNITS.SECONDS, log_base=1), + metric="tikv_import_engine_request_bucket", + label_selectors=[ + 'type="get_permit"', + ], + ), + graph_panel( + title="Apply Request Speed", + description=None, + yaxes=yaxes( + left_format=UNITS.OPS_PER_SEC, + log_base=1, + ), + targets=[ + target( + expr=expr_sum_rate( + "tikv_import_applier_event", + label_selectors=[ + 'type="begin_req"', + ], + by_labels=["instance", "type"], + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Cached File in Memory", + description=None, + yaxes=yaxes(left_format=UNITS.BYTES_IEC, log_base=1), + targets=[ + target( + expr=expr_sum("tikv_import_apply_cached_bytes"), + ), + ], + ), + graph_panel( + title="Engine Requests Unfinished", + description=None, + yaxes=yaxes( + left_format=UNITS.SHORT, + log_base=1, + ), + targets=[ + target( + expr=expr_sum_rate( + "tikv_import_applier_event", + label_selectors=[ + 'type!="begin_req"', + ], + by_labels=["instance", "type"], + ), + ), + ], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Apply Time", + description=None, + yaxis=yaxis(format=UNITS.SECONDS, log_base=1), + metric="tikv_import_engine_request_bucket", + label_selectors=[ + 'type="apply"', + ], + ), + graph_panel( + title="Raft Store Memory Usage", + description="", + yaxes=yaxes(left_format=UNITS.BYTES_IEC, log_base=1), + targets=[ + target( + expr=expr_sum( + "tikv_server_mem_trace_sum", + label_selectors=[ + 'name=~"raftstore-.*"', + ], + ), + ), + ], + ), + ] + ) + return layout.row_panel + + +def ResolvedTS() -> RowPanel: + layout = Layout(title="Resolved TS") + layout.row( + [ + graph_panel( + title="Resolved TS Worker CPU", + description="The CPU utilization of resolved ts worker", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=[ + 'name=~"resolved_ts.*"', + ], + ), + ) + ], + ), + graph_panel( + title="Advance ts Worker CPU", + description="The CPU utilization of advance ts worker", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=[ + 'name=~"advance_ts.*"', + ], + ), + ) + ], + ), + graph_panel( + title="Scan lock Worker CPU", + description="The CPU utilization of scan lock worker", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=[ + 'name=~"inc_scan.*"', + ], + ), + ) + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Max gap of resolved-ts", + description="The gap between resolved ts (the maximum candidate of safe-ts) and current time.", + yaxes=yaxes(left_format=UNITS.MILLI_SECONDS), + targets=[ + target( + expr=expr_sum( + "tikv_resolved_ts_min_resolved_ts_gap_millis", + ), + ) + ], + ), + graph_panel( + title="Min Resolved TS Region", + description="The region that has minimal resolved ts", + targets=[ + target( + expr=expr_sum( + "tikv_resolved_ts_min_resolved_ts_region", + ), + ) + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Max gap of follower safe-ts", + description="The gap between now() and the minimal (non-zero) safe ts for followers", + yaxes=yaxes(left_format=UNITS.MILLI_SECONDS), + targets=[ + target( + expr=expr_sum( + "tikv_resolved_ts_min_follower_safe_ts_gap_millis", + ), + ) + ], + ), + graph_panel( + title="Min Safe TS Follower Region", + description="The region id of the follower that has minimal safe ts", + targets=[ + target( + expr=expr_sum( + "tikv_resolved_ts_min_follower_safe_ts_region", + ), + ) + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Max gap of resolved-ts in region leaders", + description="The gap between resolved ts of leaders and current time", + yaxes=yaxes(left_format=UNITS.MILLI_SECONDS), + targets=[ + target( + expr=expr_sum( + "tikv_resolved_ts_min_leader_resolved_ts_gap_millis", + ), + ) + ], + ), + graph_panel( + title="Min Leader Resolved TS Region", + description="The region that its leader has minimal resolved ts.", + targets=[ + target( + expr=expr_sum( + "tikv_resolved_ts_min_leader_resolved_ts_region", + ), + ) + ], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Check leader duration", + description="The time consumed when handle a check leader request", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_resolved_ts_check_leader_duration_seconds_bucket", + ), + graph_panel( + title="99% CheckLeader request region count", + description="Bucketed histogram of region count in a check leader request", + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_check_leader_request_item_count", + by_labels=["instance"], + ), + legend_format="{{instance}}", + ) + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="99% CheckLeader request size", + description="Bucketed histogram of the check leader request size", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_check_leader_request_size_bytes", + by_labels=["instance"], + ), + legend_format="{{instance}}", + ), + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_check_leader_request_item_count", + by_labels=["instance"], + ), + legend_format="{{instance}}-check-num", + ), + ], + ), + graph_panel( + title="Fail advance ts count", + description="The count of fail to advance resolved-ts", + targets=[ + target( + expr=expr_sum_delta( + "tikv_resolved_ts_fail_advance_count", + by_labels=["instance", "reason"], + ), + ), + target( + expr=expr_sum_delta( + "tikv_raftstore_check_stale_peer", + by_labels=["instance"], + ), + legend_format="{{instance}}-stale-peer", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Lock heap size", + description="Total bytes in memory of resolved-ts observe regions's lock heap", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_avg( + "tikv_resolved_ts_lock_heap_bytes", + ), + ) + ], + ), + heatmap_panel( + title="Initial scan backoff duration", + description="The backoff duration before starting initial scan", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_resolved_ts_initial_scan_backoff_duration_seconds_bucket", + ), + ] + ) + layout.row( + [ + graph_panel( + title="Observe region status", + description="The status of resolved-ts observe regions", + targets=[ + target( + expr=expr_sum( + "tikv_resolved_ts_region_resolve_status", + by_labels=["type"], + ), + additional_groupby=True, + ) + ], + ), + graph_panel( + title="Pending command size", + description="Total bytes of pending commands in the channel", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_avg( + "tikv_resolved_ts_channel_penging_cmd_bytes_total", + ), + ) + ], + ), + ] + ) + return layout.row_panel + + +def Memory() -> RowPanel: + layout = Layout(title="Memory") + layout.row( + [ + graph_panel( + title="Allocator Stats", + description=None, + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum( + "tikv_allocator_stats", by_labels=["instance", "type"] + ) + ) + ], + ), + graph_panel( + title="Send Allocated(+) / Release Received(-) Bytes Rate", + description=None, + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC), + targets=[ + target( + expr=expr_operator( + expr_sum_rate( + "tikv_allocator_thread_allocation", + label_selectors=['type="alloc"'], + by_labels=["thread_name"], + ), + "-", + expr_sum_rate( + "tikv_allocator_thread_allocation", + label_selectors=['type="dealloc"'], + by_labels=["thread_name"], + ), + ), + legend_format="{{thread_name}}", + additional_groupby=True, + ) + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Allocated Bytes Rate per Thread", + description=None, + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_allocator_thread_allocation", + label_selectors=['type="alloc"'], + by_labels=["thread_name"], + ), + additional_groupby=True, + ) + ], + ), + graph_panel( + title="Released Bytes Rate per Thread", + description=None, + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_allocator_thread_allocation", + label_selectors=['type="dealloc"'], + by_labels=["thread_name"], + ), + additional_groupby=True, + ) + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Mapped Allocation per Thread", + description=None, + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum( + "tikv_allocator_thread_stats", + label_selectors=['type="mapped"'], + by_labels=["thread_name"], + ), + additional_groupby=True, + ) + ], + ), + graph_panel( + title="Arena Count", + description=None, + targets=[ + target( + expr=expr_sum( + "tikv_allocator_arena_count", + by_labels=["instance"], + ), + ) + ], + ), + ] + ) + return layout.row_panel + + +def BackupImport() -> RowPanel: + layout = Layout(title="Backup & Import") + layout.row( + [ + graph_panel( + title="Backup CPU Utilization", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=[ + 'name=~"b.*k.*w.*k.*"', + ], + ), + legend_format="backup-{{instance}}", + ), + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=[ + 'name=~"backup_io"', + ], + ), + legend_format="backup-io-{{instance}}", + ), + target( + expr=expr_simple( + "tikv_backup_softlimit", + ), + legend_format="backup-auto-throttle-{{instance}}", + ), + ], + ), + graph_panel( + title="Backup Thread Count", + targets=[ + target( + expr=expr_sum( + "tikv_backup_thread_pool_size", + ), + ), + ], + ), + graph_panel( + title="Backup Errors", + description="", + targets=[ + target( + expr=expr_sum_delta( + "tikv_backup_error_counter", + by_labels=["instance", "error"], + ), + ), + ], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Backup Write CF SST Size", + yaxis=yaxis(format=UNITS.BYTES_IEC), + metric="tikv_backup_range_size_bytes_bucket", + label_selectors=['cf="write"'], + ), + heatmap_panel( + title="Backup Default CF SST Size", + yaxis=yaxis(format=UNITS.BYTES_IEC), + metric="tikv_backup_range_size_bytes_bucket", + label_selectors=['cf="default"'], + ), + graph_panel( + title="Backup SST Generation Throughput", + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_backup_range_size_bytes_sum", + by_labels=[], # override default by instance. + ), + legend_format="total", + ), + target( + expr=expr_sum_rate( + "tikv_backup_range_size_bytes_sum", + by_labels=["instance", "cf"], + ), + ), + ], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Backup Scan SST Duration", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_backup_range_duration_seconds_bucket", + label_selectors=['type="snapshot"'], + ), + heatmap_panel( + title="Backup Scan SST Duration", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_backup_range_duration_seconds_bucket", + label_selectors=['type="scan"'], + ), + heatmap_panel( + title="Backup Save SST Duration", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_backup_range_duration_seconds_bucket", + label_selectors=['type=~"save.*"'], + ), + graph_panel( + title="Backup SST Duration", + yaxes=yaxes(left_format=UNITS.SECONDS), + targets=[ + target( + expr=expr_histogram_quantile( + 0.999, + "tikv_backup_range_duration_seconds", + by_labels=["type"], + ), + legend_format="{{type}}-99.9%", + additional_groupby=True, + ), + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_backup_range_duration_seconds", + by_labels=["type"], + ), + legend_format="{{type}}-99%", + additional_groupby=True, + ), + target( + expr=expr_operator( + expr_sum( + "tikv_backup_range_duration_seconds_sum", + by_labels=["type"], + ), + "/", + expr_sum( + "tikv_backup_range_duration_seconds_count", + by_labels=["type"], + ), + ), + legend_format="{{type}}-avg", + additional_groupby=True, + ), + ], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="External Storage Create Duration", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_external_storage_create_seconds_bucket", + ), + graph_panel_histogram_quantiles( + title="External Storage Create Duration", + description="", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_external_storage_create_seconds", + hide_avg=True, + hide_count=True, + ), + ] + ) + layout.row( + [ + graph_panel_histogram_quantiles( + title="Checksum Request Duration", + description="", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_coprocessor_request_duration_seconds", + label_selectors=['req=~"analyze.*|checksum.*"'], + by_labels=["req"], + hide_avg=True, + hide_count=True, + ), + graph_panel( + title="IO Utilization", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "node_disk_io_time_seconds_total", + by_labels=["instance", "device"], + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Import CPU Utilization", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"sst_.*"'], + by_labels=["instance"], + ), + legend_format="import-{{instance}}", + ), + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"sst_.*"'], + by_labels=["instance", "tid"], + ).extra("> 0"), + legend_format="import-{{instance}}-{{tid}}", + hide=True, + ), + target( + expr=expr_count_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"sst_.*"'], + ), + legend_format="import-count-{{instance}}", + hide=True, + ), + ], + ), + graph_panel( + title="Import Thread Count", + targets=[ + target( + expr=expr_count_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=['name=~"sst_.*"'], + by_labels=["instance"], + ), + legend_format="{{instance}}", + ), + ], + ), + graph_panel( + title="Import Errors", + targets=[ + target( + expr=expr_sum_delta( + "tikv_import_error_counter", + by_labels=["type", "error", "instance"], + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel_histogram_quantiles( + title="Import RPC Duration", + description="", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_import_rpc_duration", + by_labels=["request"], + hide_count=True, + ), + graph_panel( + title="Import RPC Ops", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_import_rpc_duration_count", + label_selectors=['request!="switch_mode"'], + by_labels=["request"], + ), + additional_groupby=True, + ), + ], + ), + graph_panel( + title="Import RPC Count", + targets=[ + target( + expr=expr_simple( + "tikv_import_rpc_count", + ), + legend_format="{{type}}-{{instance}}", + ), + ], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Import Write/Download RPC Duration", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_import_rpc_duration_bucket", + label_selectors=['request=~"download|write"'], + ), + heatmap_panel( + title="Import Wait Duration", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_import_download_duration_bucket", + label_selectors=['type="queue"'], + ), + heatmap_panel( + title="Import Read SST Duration", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_import_download_duration_bucket", + label_selectors=['type="read"'], + ), + heatmap_panel( + title="Import Rewrite SST Duration", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_import_download_duration_bucket", + label_selectors=['type="rewrite"'], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Import Ingest RPC Duration", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_import_rpc_duration_bucket", + label_selectors=['request=~"ingest"'], + ), + heatmap_panel( + title="Import Ingest SST Duration", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_import_ingest_duration_bucket", + label_selectors=['type=~"ingest"'], + ), + heatmap_panel( + title="Import Ingest SST Bytes", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_import_ingest_byte_bucket", + ), + graph_panel( + title="Import Download SST Throughput", + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_import_download_bytes_sum", + ), + ), + target( + expr=expr_sum_rate( + "tikv_import_download_bytes_sum", + by_labels=[], + ), + legend_format="total", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Import Local Write keys", + targets=[ + target( + expr=expr_sum_delta( + "tikv_import_local_write_keys", + by_labels=["type", "instance"], + ), + ), + ], + ), + graph_panel( + title="Import Local Write bytes", + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_import_local_write_bytes", + by_labels=["type", "instance"], + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="TTL Expired", + targets=[ + target( + expr=expr_sum( + "tikv_backup_raw_expired_count", + ), + ), + target( + expr=expr_sum( + "tikv_backup_raw_expired_count", + by_labels=[], + ), + legend_format="sum", + ), + ], + ), + graph_panel( + title="cloud request", + description="", + yaxes=yaxes(left_format=UNITS.SHORT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_cloud_request_duration_seconds_count", + by_labels=["cloud", "req"], + ), + ), + ], + ), + ] + ) + return layout.row_panel + + +def Encryption() -> RowPanel: + layout = Layout(title="Encryption") + layout.row( + [ + graph_panel( + title="Encryption data keys", + description="Total number of encryption data keys in use", + targets=[ + target( + expr=expr_sum( + "tikv_encryption_data_key_storage_total", + ), + legend_format="{{instance}}", + ), + ], + ), + graph_panel( + title="Encrypted files", + description="Number of files being encrypted", + targets=[ + target( + expr=expr_sum( + "tikv_encryption_file_num", + ), + legend_format="{{instance}}", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Encryption initialized", + description="Flag to indicate if encryption is initialized", + targets=[ + target( + expr=expr_simple( + "tikv_encryption_is_initialized", + ), + legend_format="{{instance}}", + ), + ], + ), + graph_panel( + title="Encryption meta files size", + description="Total size of encryption meta files", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_simple( + "tikv_encryption_meta_file_size_bytes", + ), + legend_format="{{name}}-{{instance}}", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Encrypt/decrypt data nanos", + description="", + targets=[ + target( + expr=expr_sum_rate( + "tikv_coprocessor_rocksdb_perf", + label_selectors=[ + 'metric="encrypt_data_nanos"', + ], + by_labels=["req"], + ), + legend_format="encrypt-{{req}}", + additional_groupby=True, + ), + target( + expr=expr_sum_rate( + "tikv_coprocessor_rocksdb_perf", + label_selectors=[ + 'metric="decrypt_data_nanos"', + ], + by_labels=["req"], + ), + legend_format="decrypt-{{req}}", + additional_groupby=True, + ), + ], + ), + graph_panel_histogram_quantiles( + title="Read/write encryption meta duration", + description="Writing or reading file duration (second)", + yaxes=yaxes(left_format=UNITS.SECONDS), + metric="tikv_encryption_write_read_file_duration_seconds", + hide_count=True, + ), + ] + ) + return layout.row_panel + + +def BackupLog() -> RowPanel: + layout = Layout(title="Backup Log") + layout.row( + [ + stat_panel( + title="Endpoint Status", + targets=[ + target( + expr=expr_simple("tikv_log_backup_enabled"), + legend_format="{{ instance }}", + ), + ], + mappings=[ + StatValueMappings( + StatValueMappingItem("Disabled", "0", "red"), + StatValueMappingItem("Enabled", "1", "green"), + ), + ], + ), + stat_panel( + title="Task Status", + targets=[ + target( + expr=expr_min("tikv_log_backup_task_status"), + ), + ], + mappings=[ + StatValueMappings( + StatValueMappingItem("Running", "0", "green"), + StatValueMappingItem("Paused", "1", "yellow"), + StatValueMappingItem("Error", "2", "red"), + ), + ], + ), + stat_panel( + title="Advancer Owner", + text_mode="name", + targets=[ + target( + expr="tidb_log_backup_advancer_owner > 0", + legend_format="{{ instance }}", + ), + ], + ), + stat_panel( + title="Average Flush Size", + description="The average flush size of last 30mins.", + format=UNITS.BYTES_IEC, + targets=[ + target( + expr=expr_operator( + expr_sum_increase( + "tikv_log_backup_flush_file_size_sum", + range_selector="30m", + ), + "/", + expr_sum_increase( + "tikv_log_backup_flush_duration_sec_count", + label_selectors=['stage=~"save_files"'], + range_selector="30m", + ), + ), + legend_format="{{ instance }}", + ), + ], + ), + ] + ) + layout.row( + [ + stat_panel( + title="Flushed Files (Last 30m) Per Host", + description="The current total flushed file number of this run.", + decimals=0, + targets=[ + target( + expr=expr_sum_delta( + "tikv_log_backup_flush_file_size_count", + range_selector="30m", + ).extra("> 0"), + ), + ], + ), + stat_panel( + title="Flush Times (Last 30m)", + description="This is the summary of the file count has been flushed, summered by the data each TiKV has flushed since last boot.\n**NOTE: The size may get reduced if some of TiKVs reboot.**", + decimals=0, + targets=[ + target( + expr=expr_sum_delta( + "tikv_log_backup_flush_duration_sec_count", + range_selector="30m", + label_selectors=['stage=~"save_files"'], + ), + ), + ], + ), + stat_panel( + title="Total Flushed Size (Last 30m)", + description="This is the summary of the size has been flushed, summered by the data each TiKV has flushed since last boot.\n**NOTE: The size may get reduced if some of TiKVs reboot.**", + format=UNITS.BYTES_IEC, + targets=[ + target( + expr=expr_sum_delta( + "tikv_log_backup_flush_file_size_sum", + range_selector="30m", + ), + ), + ], + ), + stat_panel( + title="Flush Files (Last 30m)", + description="This is the summary of the file count has been flushed, summered by the data each TiKV has flushed since last boot.\n**NOTE: The size may get reduced if some of TiKVs reboot.**", + decimals=0, + targets=[ + target( + expr=expr_sum_delta( + "tikv_log_backup_flush_file_size_count", + range_selector="30m", + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="CPU Usage", + description="The CPU utilization of log backup threads. \n**(Note this is the average usage for a period of time, some peak of CPU usage may be lost.)**", + yaxes=yaxes(left_format=UNITS.PERCENT_UNIT), + targets=[ + target( + expr=expr_sum_rate( + "tikv_thread_cpu_seconds_total", + label_selectors=[ + 'name=~"backup_stream|log-backup-scan(-[0-9]+)?"' + ], + ), + ) + ], + ), + graph_panel( + title="Handle Event Rate", + description="", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_log_backup_handle_kv_batch_sum", + ), + ) + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Initial Scan Generate Event Throughput", + description="The data rate of initial scanning emitting events.", + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_log_backup_incremental_scan_bytes_sum", + ), + ) + ], + ), + graph_panel( + title="Abnormal Checkpoint TS Lag", + description=None, + yaxes=yaxes(left_format=UNITS.MILLI_SECONDS), + targets=[ + target( + expr=expr_operator( + "time() * 1000", + "-", + expr_max( + "tidb_log_backup_last_checkpoint", by_labels=["task"] + ).extra("/ 262144 > 0"), + ), + legend_format="{{ task }}", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Memory Of Events", + description="The estimated memory usage by the streaming backup module.", + yaxes=yaxes(left_format=UNITS.BYTES_IEC), + targets=[ + target( + expr=expr_sum("tikv_log_backup_heap_memory"), + ) + ], + ), + graph_panel( + title="Observed Region Count", + description="", + targets=[ + target( + expr=expr_sum("tikv_log_backup_observed_region"), + ), + target( + expr=expr_sum( + "tikv_log_backup_observed_region", + ), + legend_format="{{instance}}-total", + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Errors", + description="The errors met when backing up.\n**They are retryable, don't worry.**", + yaxes=yaxes(left_format=UNITS.OPS_PER_MIN), + targets=[ + target( + expr=expr_sum_delta( + "tikv_log_backup_errors", + range_selector="1m", + by_labels=["type", "instance"], + ), + ), + ], + ), + graph_panel( + title="Fatal Errors", + description="The errors met when backing up.", + yaxes=yaxes(left_format=UNITS.OPS_PER_MIN), + targets=[ + target( + expr=expr_sum_delta( + "tikv_log_backup_fatal_errors", + range_selector="1m", + by_labels=["type", "instance"], + ), + ), + ], + ), + graph_panel( + title="Checkpoint TS of Tasks", + description=None, + yaxes=yaxes(left_format=UNITS.DATE_TIME_ISO_TODAY), + null_point_mode=NULL_AS_NULL, + targets=[ + target( + expr=expr_max( + "tidb_log_backup_last_checkpoint", by_labels=["task"] + ).extra("/ 262144 > 0"), + ), + target(expr="time() * 1000", legend_format="Current Time"), + ], + series_overrides=[ + series_override( + alias="Current Time", + fill=0, + dashes=True, + ), + ], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Flush Duration", + description="The duration of flushing a batch of file.", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_log_backup_flush_duration_sec_bucket", + label_selectors=['stage=~"save_files"'], + ), + heatmap_panel( + title="Initial scanning duration", + description="The duration of scanning the initial data from local DB and transform them into apply events.", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_log_backup_initial_scan_duration_sec_bucket", + ), + heatmap_panel( + title="Convert Raft Event duration", + description="The duration of converting a raft request into a apply event.", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_log_backup_event_handle_duration_sec_bucket", + label_selectors=['stage=~"to_stream_event"'], + ), + heatmap_panel( + title="Wait for Lock Duration", + description="The duration of waiting the mutex of the controller.", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_log_backup_event_handle_duration_sec_bucket", + label_selectors=['stage=~"get_router_lock"'], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Command Batch Size", + description="The number of KV-modify of each raft command observed.", + yaxis=yaxis(format=UNITS.SHORT), + metric="tikv_log_backup_handle_kv_batch_bucket", + ), + heatmap_panel( + title="Save to Temp File Duration", + description="The total cost of saving an event into temporary file.", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_log_backup_event_handle_duration_sec_bucket", + label_selectors=['stage=~"save_to_temp_file"'], + ), + heatmap_panel( + title="Write to Temp File Duration", + description="The total cost of writing a event into temporary file.\nComparing to the ***Save*** duration, it doesn't contain the time cost of routing the task by range / task.", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_log_backup_on_event_duration_seconds_bucket", + label_selectors=['stage="write_to_tempfile"'], + ), + heatmap_panel( + title="System Write Call Duration", + description="The duration of collecting metadata and call the UNIX system call *write* for each event.", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tikv_log_backup_on_event_duration_seconds_bucket", + label_selectors=['stage="syscall_write"'], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Internal Message Type", + description="The internal message type count.", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC, log_base=2), + targets=[ + target( + expr=expr_sum_rate( + "tikv_log_backup_internal_actor_acting_duration_sec_count", + by_labels=["message"], + ), + ) + ], + ), + graph_panel( + title="Internal Message Handling Duration (P99)", + description="The internal handling message duration.", + yaxes=yaxes(left_format=UNITS.SECONDS), + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_log_backup_internal_actor_acting_duration_sec", + by_labels=["message"], + ), + legend_format="{{message}}", + ) + ], + ), + graph_panel( + title="Internal Message Handling Duration (P90)", + description="The internal handling message duration.", + yaxes=yaxes(left_format=UNITS.SECONDS), + targets=[ + target( + expr=expr_histogram_quantile( + 0.9, + "tikv_log_backup_internal_actor_acting_duration_sec", + by_labels=["message"], + ), + legend_format="{{message}}", + ) + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Initial Scan RocksDB Throughput", + description="The internal read throughput of RocksDB during initial scanning. This panel can roughly present the read through to the hard disk of initial scanning.", + yaxes=yaxes(left_format=UNITS.BYTES_SEC_IEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_log_backup_initial_scan_operations", + label_selectors=['op=~"read_bytes"'], + by_labels=["cf"], + ), + ) + ], + ), + graph_panel( + title="Initial Scan RocksDB Operation", + description="Misc statistics of RocksDB during initial scanning.", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_log_backup_initial_scan_operations", + label_selectors=['op!~"read_bytes"'], + by_labels=["cf", "op"], + ).extra("> 0"), + ) + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Initial Scanning Trigger Reason", + description="The reason of triggering initial scanning.", + targets=[ + target( + expr=expr_sum_rate( + "tikv_log_backup_initial_scan_reason", + by_labels=["reason"], + ), + ) + ], + ), + graph_panel( + title="Initial Scanning Task Status", + description="The task status of initial scanning.", + targets=[ + target( + expr=expr_sum( + "tikv_log_backup_pending_initial_scan", + by_labels=["stage"], + ), + ) + ], + ), + graph_panel( + title="Region Checkpoint Key Putting", + description="", + yaxes=yaxes(left_format=UNITS.COUNTS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_log_backup_metadata_key_operation", + by_labels=["type"], + ), + ) + ], + ), + ] + ) + layout.row( + [ + heatmap_panel( + title="Request Checkpoint Batch Size", + metric="tidb_log_backup_advancer_batch_size_bucket", + label_selectors=['type="checkpoint"'], + ), + heatmap_panel( + title="Tick Duration", + yaxis=yaxis(format=UNITS.SECONDS), + metric="tidb_log_backup_advancer_tick_duration_sec_bucket", + label_selectors=['step="tick"'], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Region Checkpoint Failure Reason", + description="The reason of advancer failed to be advanced.", + targets=[ + target( + expr=expr_sum_rate( + "tidb_log_backup_region_request_failure", + label_selectors=['reason!="retryable-scan-region"'], + by_labels=["reason"], + ), + ), + ], + ), + graph_panel( + title="Request Result", + description="The result of getting region checkpoints.", + targets=[ + target( + expr=expr_sum_rate( + "tidb_log_backup_region_request", + by_labels=["result"], + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Tick Duration (P99)", + description="The internal handling message duration.", + yaxes=yaxes(left_format=UNITS.SECONDS), + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, + "tidb_log_backup_advancer_tick_duration_sec", + by_labels=["step"], + ), + legend_format="{{ step }}", + ) + ], + ), + graph_panel( + title="Tick Duration (P90)", + description="The internal handling message duration.", + yaxes=yaxes(left_format=UNITS.SECONDS), + targets=[ + target( + expr=expr_histogram_quantile( + 0.9, + "tidb_log_backup_advancer_tick_duration_sec", + by_labels=["step"], + ), + legend_format="{{ step }}", + ) + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Get Region Operation Count", + description="The frequent of getting region level checkpoint.", + targets=[ + target( + expr=expr_sum_rate( + "tidb_log_backup_advancer_tick_duration_sec_count", + label_selectors=['step="get-regions-in-range"'], + by_labels=["step", "instance"], + ), + ) + ], + ), + graph_panel( + title="Try Advance Trigger Time", + description="The variant of checkpoint group.", + targets=[ + target( + expr=expr_sum_rate( + "tidb_log_backup_advancer_tick_duration_sec_count", + label_selectors=['step="try-advance"'], + by_labels=["step", "instance"], + ), + ) + ], + ), + ] + ) + return layout.row_panel + + +def SlowTrendStatistics() -> RowPanel: + layout = Layout(title="Slow Trend Statistics") + layout.row( + [ + graph_panel( + title="Inspected duration per server", + description="The duration that recorded by inspecting messages.", + yaxes=yaxes(left_format=UNITS.SECONDS), + targets=[ + target( + expr=expr_histogram_quantile( + 0.99, + "tikv_raftstore_inspect_duration_seconds", + by_labels=["instance", "type"], + ), + legend_format="{{instance}}-{{type}}", + ), + ], + ), + graph_panel( + title="Store Slow Score", + description="The slow score of stores", + targets=[ + target( + expr=expr_sum( + "tikv_raftstore_slow_score", + by_labels=["instance", "type"], + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="Slow Trend", + description="The changing trend of the slowness on I/O operations. 'value > 0' means the related store might have a slow trend.", + targets=[ + target( + expr=expr_sum( + "tikv_raftstore_slow_trend", + ), + ), + ], + ), + graph_panel( + title="QPS Changing Trend", + description="The changing trend of QPS on each store. 'value < 0' means the QPS has a dropping trend.", + targets=[ + target( + expr=expr_sum( + "tikv_raftstore_slow_trend_result", + ), + ), + ], + ), + ] + ) + layout.row( + [ + graph_panel( + title="AVG Sampling Latency", + description="The sampling latency of recent queries. A larger value indicates that the store is more likely to be the slowest store.", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS), + targets=[ + target( + expr=expr_sum( + "tikv_raftstore_slow_trend_l0", + ), + ), + ], + ), + graph_panel( + title="QPS of each store", + description="The QPS of each store.", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum( + "tikv_raftstore_slow_trend_result_value", + ), + ), + ], + ), + ] + ) + return layout.row_panel + + +def StatusServer() -> RowPanel: + layout = Layout(title="Status Server") + layout.row( + [ + graph_panel_histogram_quantiles( + title="Status API Request Duration", + description="The 99 quantile durtion of status server API requests", + metric="tikv_status_server_request_duration_seconds", + yaxes=yaxes(left_format=UNITS.SECONDS), + by_labels=["path"], + hide_p9999=True, + hide_count=True, + hide_avg=True, + ), + graph_panel( + title="Status API Request (op/s)", + yaxes=yaxes(left_format=UNITS.OPS_PER_SEC), + targets=[ + target( + expr=expr_sum_rate( + "tikv_status_server_request_duration_seconds_count", + by_labels=["path"], + ), + additional_groupby=True, + ), + ], + ), + ] + ) + return layout.row_panel + + +def ResourceControl() -> RowPanel: + layout = Layout(title="Resource Control") + layout.row( + [ + graph_panel( + title="Background Task Total Wait Duration", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS), + targets=[ + target( + expr=expr_sum_rate( + "tikv_resource_control_background_task_wait_duration", + by_labels=["instance", "resource_group"], + ), + ), + ], + ), + graph_panel( + title="Priority Quota Limit", + description="The memory usage of the resource control module.", + yaxes=yaxes(left_format=UNITS.MICRO_SECONDS), + targets=[ + target( + expr=expr_sum( + "tikv_resource_control_priority_quota_limit", + by_labels=["instance", "priority"], + ), + ), + ], + ), + ] + ) + return layout.row_panel + + +#### Metrics Definition End #### + + +dashboard = Dashboard( + title="Test-Cluster-TiKV-Details", + uid="RDVQiEzZz", + timezone="browser", + refresh="1m", + inputs=[DATASOURCE_INPUT], + editable=True, + templating=Templates(), + panels=[ + # Overview + Duration(), + Cluster(), + Errors(), + Server(), + # Entrance of Write and Read + gRPC(), + Storage(), + LocalReader(), + # CPU and IO + ThreadCPU(), + IOBreakdown(), + # Raftstore + RaftWaterfall(), + RaftIO(), + RaftPropose(), + RaftProcess(), + RaftMessage(), + RaftAdmin(), + RaftLog(), + # Engine + RaftEngine(), + RocksDB(), + Titan(), + InMemoryEngine(), + # Scheduler and Read Pools + FlowControl(), + Scheduler(), + SchedulerWorkerPool(), + SchedulerCommands(), + CoprocessorOverview(), + CoprocessorDetail(), + UnifiedReadPool(), + # Transaction + GC(), + PessimisticLocking(), + # Background Tasks + Task(), + PD(), + SlowTrendStatistics(), + Snapshot(), + # Tools + ResolvedTS(), + PointInTimeRestore(), + BackupImport(), + BackupLog(), + # Advanced Debugging for CPU and Memory + Threads(), + Memory(), + # Infrequently Used + ResourceControl(), + StatusServer(), + Encryption(), + TTL(), + ], + # Set 14 or larger to support shared crosshair or shared tooltip. + # See https://github.com/grafana/grafana/blob/v10.2.2/public/app/features/dashboard/state/DashboardMigrator.ts#L443-L445 + schemaVersion=14, + graphTooltip=GRAPH_TOOLTIP_MODE_SHARED_CROSSHAIR, +).auto_panel_ids() diff --git a/metrics/grafana/tikv_details.json b/metrics/grafana/tikv_details.json index b729113a03a..f858b20f465 100644 --- a/metrics/grafana/tikv_details.json +++ b/metrics/grafana/tikv_details.json @@ -32310,6 +32310,2989 @@ }, "yaxes": [ { +<<<<<<< HEAD +======= + "decimals": null, + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Current pending and running tasks of worker", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 0 + }, + "height": null, + "hideTimeOverride": false, + "id": 398, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "sum((\n tikv_worker_pending_task_total\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (name, $additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{name}} {{$additional_groupby}}", + "metric": "", + "query": "sum((\n tikv_worker_pending_task_total\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (name, $additional_groupby) ", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "Worker pending tasks", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of tasks handled by future_pool", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 7 + }, + "height": null, + "hideTimeOverride": false, + "id": 399, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "sum(rate(\n tikv_futurepool_handled_task_total\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (name, $additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{name}} {{$additional_groupby}}", + "metric": "", + "query": "sum(rate(\n tikv_futurepool_handled_task_total\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (name, $additional_groupby) ", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "FuturePool handled tasks", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Current pending and running tasks of future_pool", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 7 + }, + "height": null, + "hideTimeOverride": false, + "id": 400, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "sum(avg_over_time(\n tikv_futurepool_pending_task_total\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [1m]\n)) by (name, $additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{name}} {{$additional_groupby}}", + "metric": "", + "query": "sum(avg_over_time(\n tikv_futurepool_pending_task_total\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [1m]\n)) by (name, $additional_groupby) ", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "FuturePool pending tasks", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + } + ], + "repeat": null, + "repeatDirection": null, + "span": null, + "targets": [], + "timeFrom": null, + "timeShift": null, + "title": "Task", + "transformations": [], + "transparent": false, + "type": "row" + }, + { + "cacheTimeout": null, + "collapsed": true, + "datasource": null, + "description": null, + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "gridPos": { + "h": 7, + "w": 24, + "x": 0, + "y": 0 + }, + "height": null, + "hideTimeOverride": false, + "id": 401, + "interval": null, + "links": [], + "maxDataPoints": 100, + "maxPerRow": null, + "minSpan": null, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The count of requests that TiKV sends to PD", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 0 + }, + "height": null, + "hideTimeOverride": false, + "id": 402, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "sum(rate(\n tikv_pd_request_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (type, $additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}} {{$additional_groupby}}", + "metric": "", + "query": "sum(rate(\n tikv_pd_request_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (type, $additional_groupby) ", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "PD requests", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The time consumed by requests that TiKV sends to PD", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 0 + }, + "height": null, + "hideTimeOverride": false, + "id": 403, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "(sum(rate(\n tikv_pd_request_duration_seconds_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (type, $additional_groupby) / sum(rate(\n tikv_pd_request_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (type, $additional_groupby) )", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}} {{$additional_groupby}}", + "metric": "", + "query": "(sum(rate(\n tikv_pd_request_duration_seconds_sum\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (type, $additional_groupby) / sum(rate(\n tikv_pd_request_duration_seconds_count\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (type, $additional_groupby) )", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "PD request duration (average)", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The total number of PD heartbeat messages", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 7 + }, + "height": null, + "hideTimeOverride": false, + "id": 404, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "sum(rate(\n tikv_pd_heartbeat_message_total\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (type, $additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}} {{$additional_groupby}}", + "metric": "", + "query": "sum(rate(\n tikv_pd_heartbeat_message_total\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (type, $additional_groupby) ", + "refId": "", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "sum((\n tikv_pd_pending_heartbeat_total\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (instance) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-pending", + "metric": "", + "query": "sum((\n tikv_pd_pending_heartbeat_total\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (instance) ", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "PD heartbeats", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The total number of peers validated by the PD worker", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 7 + }, + "height": null, + "hideTimeOverride": false, + "id": 405, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "sum(rate(\n tikv_pd_validate_peer_total\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (type, $additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}} {{$additional_groupby}}", + "metric": "", + "query": "sum(rate(\n tikv_pd_validate_peer_total\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (type, $additional_groupby) ", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "PD validate peers", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The count of reconnection between TiKV and PD", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 14 + }, + "height": null, + "hideTimeOverride": false, + "id": 406, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "sum(delta(\n tikv_pd_reconnect_total\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (type, $additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}} {{$additional_groupby}}", + "metric": "", + "query": "sum(delta(\n tikv_pd_reconnect_total\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (type, $additional_groupby) ", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "PD reconnection", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "opm", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The forward status of PD client", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 14 + }, + "height": null, + "hideTimeOverride": false, + "id": 407, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "((\n tikv_pd_request_forwarded\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{host}}", + "metric": "", + "query": "((\n tikv_pd_request_forwarded\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) ", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "PD forward status", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of TSO requests waiting in the queue.", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 24, + "x": 0, + "y": 21 + }, + "height": null, + "hideTimeOverride": false, + "id": 408, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "sum((\n tikv_pd_pending_tso_request_total\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (instance, $additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}} {{$additional_groupby}}", + "metric": "", + "query": "sum((\n tikv_pd_pending_tso_request_total\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (instance, $additional_groupby) ", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "Pending TSO Requests", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "opm", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + } + ], + "repeat": null, + "repeatDirection": null, + "span": null, + "targets": [], + "timeFrom": null, + "timeShift": null, + "title": "PD", + "transformations": [], + "transparent": false, + "type": "row" + }, + { + "cacheTimeout": null, + "collapsed": true, + "datasource": null, + "description": null, + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "gridPos": { + "h": 7, + "w": 24, + "x": 0, + "y": 0 + }, + "height": null, + "hideTimeOverride": false, + "id": 409, + "interval": null, + "links": [], + "maxDataPoints": 100, + "maxPerRow": null, + "minSpan": null, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The duration that recorded by inspecting messages.", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 0 + }, + "height": null, + "hideTimeOverride": false, + "id": 410, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "histogram_quantile(0.99,(\n sum(rate(\n tikv_raftstore_inspect_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (instance, type, le) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}", + "metric": "", + "query": "histogram_quantile(0.99,(\n sum(rate(\n tikv_raftstore_inspect_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (instance, type, le) \n \n \n)) ", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "Inspected duration per server", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The slow score of stores", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 0 + }, + "height": null, + "hideTimeOverride": false, + "id": 411, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "sum((\n tikv_raftstore_slow_score\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (instance, type) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}", + "metric": "", + "query": "sum((\n tikv_raftstore_slow_score\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (instance, type) ", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "Store Slow Score", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The changing trend of the slowness on I/O operations. 'value > 0' means the related store might have a slow trend.", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 7 + }, + "height": null, + "hideTimeOverride": false, + "id": 412, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "sum((\n tikv_raftstore_slow_trend\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (instance) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "metric": "", + "query": "sum((\n tikv_raftstore_slow_trend\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (instance) ", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "Slow Trend", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The changing trend of QPS on each store. 'value < 0' means the QPS has a dropping trend.", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 7 + }, + "height": null, + "hideTimeOverride": false, + "id": 413, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "sum((\n tikv_raftstore_slow_trend_result\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (instance) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "metric": "", + "query": "sum((\n tikv_raftstore_slow_trend_result\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (instance) ", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "QPS Changing Trend", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The sampling latency of recent queries. A larger value indicates that the store is more likely to be the slowest store.", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 14 + }, + "height": null, + "hideTimeOverride": false, + "id": 414, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "sum((\n tikv_raftstore_slow_trend_l0\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (instance) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "metric": "", + "query": "sum((\n tikv_raftstore_slow_trend_l0\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (instance) ", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "AVG Sampling Latency", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "\u00b5s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The QPS of each store.", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 14 + }, + "height": null, + "hideTimeOverride": false, + "id": 415, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "sum((\n tikv_raftstore_slow_trend_result_value\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (instance) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "metric": "", + "query": "sum((\n tikv_raftstore_slow_trend_result_value\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (instance) ", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "QPS of each store", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + } + ], + "repeat": null, + "repeatDirection": null, + "span": null, + "targets": [], + "timeFrom": null, + "timeShift": null, + "title": "Slow Trend Statistics", + "transformations": [], + "transparent": false, + "type": "row" + }, + { + "cacheTimeout": null, + "collapsed": true, + "datasource": null, + "description": null, + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "gridPos": { + "h": 7, + "w": 24, + "x": 0, + "y": 0 + }, + "height": null, + "hideTimeOverride": false, + "id": 416, + "interval": null, + "links": [], + "maxDataPoints": 100, + "maxPerRow": null, + "minSpan": null, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The rate of Raft snapshot messages sent", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 0 + }, + "height": null, + "hideTimeOverride": false, + "id": 417, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "sum(delta(\n tikv_raftstore_raft_sent_message_total\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\",type=\"snapshot\"}\n [1m]\n)) by (instance) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "metric": "", + "query": "sum(delta(\n tikv_raftstore_raft_sent_message_total\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\",type=\"snapshot\"}\n [1m]\n)) by (instance) ", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "Rate snapshot message", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "opm", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of snapshots in different states", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 0 + }, + "height": null, + "hideTimeOverride": false, + "id": 418, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "sum((\n tikv_raftstore_snapshot_traffic_total\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (type, $additional_groupby) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{type}} {{$additional_groupby}}", + "metric": "", + "query": "sum((\n tikv_raftstore_snapshot_traffic_total\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n \n)) by (type, $additional_groupby) ", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "Snapshot state count", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The time snapshot generation/apply tasks spent waiting to be executed.", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 7 + }, + "height": null, + "hideTimeOverride": false, + "id": 419, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "histogram_quantile(0.99,(\n sum(rate(\n tikv_raftstore_snapshot_generation_wait_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (instance, le) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-generate", + "metric": "", + "query": "histogram_quantile(0.99,(\n sum(rate(\n tikv_raftstore_snapshot_generation_wait_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (instance, le) \n \n \n)) ", + "refId": "", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "histogram_quantile(0.99,(\n sum(rate(\n tikv_raftstore_snapshot_apply_wait_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (instance, le) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-apply", + "metric": "", + "query": "histogram_quantile(0.99,(\n sum(rate(\n tikv_raftstore_snapshot_apply_wait_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (instance, le) \n \n \n)) ", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "99% Snapshot generation/apply wait duration", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The time consumed when handling snapshots", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 7 + }, + "height": null, + "hideTimeOverride": false, + "id": 420, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "histogram_quantile(0.99,(\n sum(rate(\n tikv_server_send_snapshot_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "send {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.99,(\n sum(rate(\n tikv_server_send_snapshot_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "histogram_quantile(0.99,(\n sum(rate(\n tikv_raftstore_snapshot_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\",type=\"apply\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "apply {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.99,(\n sum(rate(\n tikv_raftstore_snapshot_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\",type=\"apply\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "", + "step": 10, + "target": "" + }, + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "histogram_quantile(0.99,(\n sum(rate(\n tikv_raftstore_snapshot_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\",type=\"generate\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "generate {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.99,(\n sum(rate(\n tikv_raftstore_snapshot_duration_seconds_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\",type=\"generate\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "99% Handle snapshot duration", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "decimals": null, + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The snapshot size (P99.99).9999", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": { + "thresholds": { + "mode": "absolute", + "steps": [] + } + } + }, + "fill": 1, + "fillGradient": 1, + "grid": { + "threshold1": null, + "threshold1Color": "rgba(216, 200, 27, 0.27)", + "threshold2": null, + "threshold2Color": "rgba(234, 112, 112, 0.22)" + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 14 + }, + "height": null, + "hideTimeOverride": false, + "id": 421, + "interval": null, + "isNew": true, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxDataPoints": null, + "maxPerRow": null, + "minSpan": null, + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true, + "dataLinks": [] + }, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": null, + "seriesOverrides": [], + "span": null, + "stack": false, + "steppedLine": false, + "targets": [ + { + "datasource": "${DS_TEST-CLUSTER}", + "expr": "histogram_quantile(0.9999,(\n sum(rate(\n tikv_snapshot_size_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "format": "time_series", + "hide": false, + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "size {{$additional_groupby}}", + "metric": "", + "query": "histogram_quantile(0.9999,(\n sum(rate(\n tikv_snapshot_size_bucket\n {k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",instance=~\"$instance\"}\n [$__rate_interval]\n)) by (le, $additional_groupby) \n \n \n)) ", + "refId": "", + "step": 10, + "target": "" + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "99.99% Snapshot size", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "transparent": false, + "type": "graph", + "xaxis": { + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": null, +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) "format": "bytes", "label": null, "logBase": 1, diff --git a/metrics/grafana/tikv_details.json.sha256 b/metrics/grafana/tikv_details.json.sha256 new file mode 100644 index 00000000000..0c7f268dfa0 --- /dev/null +++ b/metrics/grafana/tikv_details.json.sha256 @@ -0,0 +1 @@ +e93faab944914bbca21c74daee0223604dd57ba37115c7239d8b707468d5d8db ./metrics/grafana/tikv_details.json diff --git a/src/server/node.rs b/src/server/node.rs index 228f679ed14..4e7ca0fcfe6 100644 --- a/src/server/node.rs +++ b/src/server/node.rs @@ -20,8 +20,8 @@ use raftstore::{ store::{ self, fsm::{store::StoreMeta, ApplyRouter, RaftBatchSystem, RaftRouter}, - initial_region, AutoSplitController, Config as StoreConfig, GlobalReplicationState, PdTask, - RefreshConfigTask, SnapManager, SplitCheckTask, Transport, + initial_region, AutoSplitController, Config as StoreConfig, DiskCheckRunner, + GlobalReplicationState, PdTask, RefreshConfigTask, SnapManager, SplitCheckTask, Transport, }, }; use resource_metering::CollectorRegHandle; @@ -173,6 +173,7 @@ where concurrency_manager: ConcurrencyManager, collector_reg_handle: CollectorRegHandle, causal_ts_provider: Option>, // used for rawkv apiv2 + disk_check_runner: DiskCheckRunner, grpc_service_mgr: GrpcServiceManager, safe_point: Arc, ) -> Result<()> @@ -212,6 +213,7 @@ where concurrency_manager, collector_reg_handle, causal_ts_provider, + disk_check_runner, grpc_service_mgr, safe_point, )?; @@ -461,6 +463,7 @@ where concurrency_manager: ConcurrencyManager, collector_reg_handle: CollectorRegHandle, causal_ts_provider: Option>, // used for rawkv apiv2 + disk_check_runner: DiskCheckRunner, grpc_service_mgr: GrpcServiceManager, safe_point: Arc, ) -> Result<()> @@ -496,6 +499,7 @@ where collector_reg_handle, self.health_service.clone(), causal_ts_provider, + disk_check_runner, grpc_service_mgr, safe_point, )?; diff --git a/tests/integrations/config/dynamic/raftstore.rs b/tests/integrations/config/dynamic/raftstore.rs index 4d6551ea27c..0489103018f 100644 --- a/tests/integrations/config/dynamic/raftstore.rs +++ b/tests/integrations/config/dynamic/raftstore.rs @@ -15,7 +15,7 @@ use raftstore::{ store::{ config::{Config, RaftstoreConfigManager}, fsm::{StoreMeta, *}, - AutoSplitController, SnapManager, StoreMsg, Transport, + AutoSplitController, DiskCheckRunner, SnapManager, StoreMsg, Transport, }, Result, }; @@ -113,6 +113,7 @@ fn start_raftstore( CollectorRegHandle::new_for_test(), None, None, + DiskCheckRunner::dummy(), GrpcServiceManager::dummy(), Arc::new(AtomicU64::new(0)), ) diff --git a/tests/integrations/raftstore/test_bootstrap.rs b/tests/integrations/raftstore/test_bootstrap.rs index bca389b26e6..69fdb446b75 100644 --- a/tests/integrations/raftstore/test_bootstrap.rs +++ b/tests/integrations/raftstore/test_bootstrap.rs @@ -13,7 +13,10 @@ use engine_traits::{ use kvproto::{kvrpcpb::ApiVersion, metapb, raft_serverpb::RegionLocalState}; use raftstore::{ coprocessor::CoprocessorHost, - store::{bootstrap_store, fsm, fsm::store::StoreMeta, AutoSplitController, SnapManager}, + store::{ + bootstrap_store, fsm, fsm::store::StoreMeta, AutoSplitController, DiskCheckRunner, + SnapManager, + }, }; use raftstore_v2::router::PeerMsg; use resource_metering::CollectorRegHandle; @@ -121,6 +124,7 @@ fn test_node_bootstrap_with_prepared_data() { ConcurrencyManager::new(1.into()), CollectorRegHandle::new_for_test(), None, + DiskCheckRunner::dummy(), GrpcServiceManager::dummy(), Arc::new(AtomicU64::new(0)), ) diff --git a/tests/integrations/raftstore/test_status_command.rs b/tests/integrations/raftstore/test_status_command.rs index 8565d936d9f..323093269c6 100644 --- a/tests/integrations/raftstore/test_status_command.rs +++ b/tests/integrations/raftstore/test_status_command.rs @@ -1,10 +1,15 @@ // Copyright 2016 TiKV Project Authors. Licensed under Apache-2.0. +<<<<<<< HEAD use raftstore::store::{msg::StoreMsg as StoreMsgV1, util::LatencyInspector}; +======= +use health_controller::types::{InspectFactor, LatencyInspector}; +use raftstore::store::msg::StoreMsg as StoreMsgV1; +>>>>>>> 43e63b5614 (raftstore: calculate the slow score by considering individual disk performance factors (#17801)) use raftstore_v2::router::StoreMsg as StoreMsgV2; use test_raftstore::Simulator as S1; use test_raftstore_v2::Simulator as S2; -use tikv_util::{time::Instant, HandyRwLock}; +use tikv_util::{config::ReadableDuration, time::Instant, HandyRwLock}; #[test] fn test_region_detail() { @@ -32,6 +37,7 @@ fn test_region_detail() { fn test_latency_inspect() { let mut cluster_v1 = test_raftstore::new_node_cluster(0, 1); cluster_v1.cfg.raft_store.store_io_pool_size = 2; + cluster_v1.cfg.raft_store.inspect_kvdb_interval = ReadableDuration::millis(500); cluster_v1.run(); let mut cluster_v2 = test_raftstore_v2::new_node_cluster(0, 1); cluster_v2.run(); @@ -42,19 +48,24 @@ fn test_latency_inspect() { { // Test send LatencyInspect to V1. let (tx, rx) = std::sync::mpsc::sync_channel(10); - let inspector = LatencyInspector::new( - 1, - Box::new(move |_, duration| { - let dur = duration.sum(); - tx.send(dur).unwrap(); - }), - ); - let msg = StoreMsgV1::LatencyInspect { - send_time: Instant::now(), - inspector, - }; - router_v1.send_control(msg).unwrap(); - rx.recv_timeout(std::time::Duration::from_secs(2)).unwrap(); + // Inspect different factors. + for factor in [InspectFactor::RaftDisk, InspectFactor::KvDisk].iter() { + let cloned_tx = tx.clone(); + let inspector = LatencyInspector::new( + 1, + Box::new(move |_, duration| { + let dur = duration.sum(); + cloned_tx.send(dur).unwrap(); + }), + ); + let msg = StoreMsgV1::LatencyInspect { + factor: *factor, + send_time: Instant::now(), + inspector, + }; + router_v1.send_control(msg).unwrap(); + rx.recv_timeout(std::time::Duration::from_secs(2)).unwrap(); + } } { // Test send LatencyInspect to V2. @@ -82,17 +93,22 @@ fn test_sync_latency_inspect() { cluster.run(); let router = cluster.sim.wl().get_router(1).unwrap(); let (tx, rx) = std::sync::mpsc::sync_channel(10); - let inspector = LatencyInspector::new( - 1, - Box::new(move |_, duration| { - let dur = duration.sum(); - tx.send(dur).unwrap(); - }), - ); - let msg = StoreMsgV1::LatencyInspect { - send_time: Instant::now(), - inspector, - }; - router.send_control(msg).unwrap(); - rx.recv_timeout(std::time::Duration::from_secs(2)).unwrap(); + // Inspect different factors. + for factor in [InspectFactor::RaftDisk, InspectFactor::KvDisk].iter() { + let cloned_tx = tx.clone(); + let inspector = LatencyInspector::new( + 1, + Box::new(move |_, duration| { + let dur = duration.sum(); + cloned_tx.send(dur).unwrap(); + }), + ); + let msg = StoreMsgV1::LatencyInspect { + factor: *factor, + send_time: Instant::now(), + inspector, + }; + router.send_control(msg).unwrap(); + rx.recv_timeout(std::time::Duration::from_secs(2)).unwrap(); + } } diff --git a/tests/integrations/server/kv_service.rs b/tests/integrations/server/kv_service.rs index 24b6a87bfa5..164fafdf964 100644 --- a/tests/integrations/server/kv_service.rs +++ b/tests/integrations/server/kv_service.rs @@ -29,7 +29,7 @@ use pd_client::PdClient; use raft::eraftpb; use raftstore::{ coprocessor::CoprocessorHost, - store::{fsm::store::StoreMeta, AutoSplitController, SnapManager}, + store::{fsm::store::StoreMeta, AutoSplitController, DiskCheckRunner, SnapManager}, }; use resource_metering::CollectorRegHandle; use service::service_manager::GrpcServiceManager; @@ -1410,6 +1410,7 @@ fn test_double_run_node() { ConcurrencyManager::new(1.into()), CollectorRegHandle::new_for_test(), None, + DiskCheckRunner::dummy(), GrpcServiceManager::dummy(), Arc::new(AtomicU64::new(0)), )