mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-16 18:02:56 +00:00
pageserver: only throttle pagestream requests & bring back throttling deduction for smgr latency metrics (#9962)
## Problem In the batching PR - https://github.com/neondatabase/neon/pull/9870 I stopped deducting the time-spent-in-throttle fro latency metrics, i.e., - smgr latency metrics (`SmgrOpTimer`) - basebackup latency (+scan latency, which I think is part of basebackup). The reason for stopping the deduction was that with the introduction of batching, the trick with tracking time-spent-in-throttle inside RequestContext and swap-replacing it from the `impl Drop for SmgrOpTimer` no longer worked with >1 requests in a batch. However, deducting time-spent-in-throttle is desirable because our internal latency SLO definition does not account for throttling. ## Summary of changes - Redefine throttling to be a page_service pagestream request throttle instead of a throttle for repository `Key` reads through `Timeline::get` / `Timeline::get_vectored`. - This means reads done by `basebackup` are no longer subject to any throttle. - The throttle applies after batching, before handling of the request. - Drive-by fix: make throttle sensitive to cancellation. - Rename metric label `kind` from `timeline_get` to `pagestream` to reflect the new scope of throttling. To avoid config format breakage, we leave the config field named `timeline_get_throttle` and ignore the `task_kinds` field. This will be cleaned up in a future PR. ## Trade-Offs Ideally, we would apply the throttle before reading a request off the connection, so that we queue the minimal amount of work inside the process. However, that's not possible because we need to do shard routing. The redefinition of the throttle to limit pagestream request rate instead of repository `Key` rate comes with several downsides: - We're no longer able to use the throttle mechanism for other other tasks, e.g. image layer creation. However, in practice, we never used that capability anyways. - We no longer throttle basebackup.
This commit is contained in:
committed by
Ivan Efremov
parent
907e4aa3c4
commit
63cb8ce975
@@ -501,7 +501,9 @@ pub struct EvictionPolicyLayerAccessThreshold {
|
||||
|
||||
#[derive(Debug, Serialize, Deserialize, Clone, PartialEq, Eq)]
|
||||
pub struct ThrottleConfig {
|
||||
pub task_kinds: Vec<String>, // TaskKind
|
||||
/// See [`ThrottleConfigTaskKinds`] for why we do the serde `rename`.
|
||||
#[serde(rename = "task_kinds")]
|
||||
pub enabled: ThrottleConfigTaskKinds,
|
||||
pub initial: u32,
|
||||
#[serde(with = "humantime_serde")]
|
||||
pub refill_interval: Duration,
|
||||
@@ -509,10 +511,38 @@ pub struct ThrottleConfig {
|
||||
pub max: u32,
|
||||
}
|
||||
|
||||
/// Before <https://github.com/neondatabase/neon/pull/9962>
|
||||
/// the throttle was a per `Timeline::get`/`Timeline::get_vectored` call.
|
||||
/// The `task_kinds` field controlled which Pageserver "Task Kind"s
|
||||
/// were subject to the throttle.
|
||||
///
|
||||
/// After that PR, the throttle is applied at pagestream request level
|
||||
/// and the `task_kinds` field does not apply since the only task kind
|
||||
/// that us subject to the throttle is that of the page service.
|
||||
///
|
||||
/// However, we don't want to make a breaking config change right now
|
||||
/// because it means we have to migrate all the tenant configs.
|
||||
/// This will be done in a future PR.
|
||||
///
|
||||
/// In the meantime, we use emptiness / non-emptsiness of the `task_kinds`
|
||||
/// field to determine if the throttle is enabled or not.
|
||||
#[derive(Serialize, Deserialize, Debug, Clone, PartialEq, Eq)]
|
||||
#[serde(transparent)]
|
||||
pub struct ThrottleConfigTaskKinds(Vec<String>);
|
||||
|
||||
impl ThrottleConfigTaskKinds {
|
||||
pub fn disabled() -> Self {
|
||||
Self(vec![])
|
||||
}
|
||||
pub fn is_enabled(&self) -> bool {
|
||||
!self.0.is_empty()
|
||||
}
|
||||
}
|
||||
|
||||
impl ThrottleConfig {
|
||||
pub fn disabled() -> Self {
|
||||
Self {
|
||||
task_kinds: vec![], // effectively disables the throttle
|
||||
enabled: ThrottleConfigTaskKinds::disabled(),
|
||||
// other values don't matter with emtpy `task_kinds`.
|
||||
initial: 0,
|
||||
refill_interval: Duration::from_millis(1),
|
||||
@@ -526,6 +556,30 @@ impl ThrottleConfig {
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod throttle_config_tests {
|
||||
use super::*;
|
||||
|
||||
#[test]
|
||||
fn test_disabled_is_disabled() {
|
||||
let config = ThrottleConfig::disabled();
|
||||
assert!(!config.enabled.is_enabled());
|
||||
}
|
||||
#[test]
|
||||
fn test_enabled_backwards_compat() {
|
||||
let input = serde_json::json!({
|
||||
"task_kinds": ["PageRequestHandler"],
|
||||
"initial": 40000,
|
||||
"refill_interval": "50ms",
|
||||
"refill_amount": 1000,
|
||||
"max": 40000,
|
||||
"fair": true
|
||||
});
|
||||
let config: ThrottleConfig = serde_json::from_value(input).unwrap();
|
||||
assert!(config.enabled.is_enabled());
|
||||
}
|
||||
}
|
||||
|
||||
/// A flattened analog of a `pagesever::tenant::LocationMode`, which
|
||||
/// lists out all possible states (and the virtual "Detached" state)
|
||||
/// in a flat form rather than using rust-style enums.
|
||||
|
||||
@@ -217,31 +217,16 @@ impl<'a> ScanLatencyOngoingRecording<'a> {
|
||||
ScanLatencyOngoingRecording { parent, start }
|
||||
}
|
||||
|
||||
pub(crate) fn observe(self, throttled: Option<Duration>) {
|
||||
pub(crate) fn observe(self) {
|
||||
let elapsed = self.start.elapsed();
|
||||
let ex_throttled = if let Some(throttled) = throttled {
|
||||
elapsed.checked_sub(throttled)
|
||||
} else {
|
||||
Some(elapsed)
|
||||
};
|
||||
if let Some(ex_throttled) = ex_throttled {
|
||||
self.parent.observe(ex_throttled.as_secs_f64());
|
||||
} else {
|
||||
use utils::rate_limit::RateLimit;
|
||||
static LOGGED: Lazy<Mutex<RateLimit>> =
|
||||
Lazy::new(|| Mutex::new(RateLimit::new(Duration::from_secs(10))));
|
||||
let mut rate_limit = LOGGED.lock().unwrap();
|
||||
rate_limit.call(|| {
|
||||
warn!("error deducting time spent throttled; this message is logged at a global rate limit");
|
||||
});
|
||||
}
|
||||
self.parent.observe(elapsed.as_secs_f64());
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) static GET_VECTORED_LATENCY: Lazy<GetVectoredLatency> = Lazy::new(|| {
|
||||
let inner = register_histogram_vec!(
|
||||
"pageserver_get_vectored_seconds",
|
||||
"Time spent in get_vectored, excluding time spent in timeline_get_throttle.",
|
||||
"Time spent in get_vectored.",
|
||||
&["task_kind"],
|
||||
CRITICAL_OP_BUCKETS.into(),
|
||||
)
|
||||
@@ -264,7 +249,7 @@ pub(crate) static GET_VECTORED_LATENCY: Lazy<GetVectoredLatency> = Lazy::new(||
|
||||
pub(crate) static SCAN_LATENCY: Lazy<ScanLatency> = Lazy::new(|| {
|
||||
let inner = register_histogram_vec!(
|
||||
"pageserver_scan_seconds",
|
||||
"Time spent in scan, excluding time spent in timeline_get_throttle.",
|
||||
"Time spent in scan.",
|
||||
&["task_kind"],
|
||||
CRITICAL_OP_BUCKETS.into(),
|
||||
)
|
||||
@@ -1227,11 +1212,44 @@ pub(crate) struct SmgrOpTimer {
|
||||
per_timeline_latency_histo: Option<Histogram>,
|
||||
|
||||
start: Instant,
|
||||
throttled: Duration,
|
||||
op: SmgrQueryType,
|
||||
}
|
||||
|
||||
impl SmgrOpTimer {
|
||||
pub(crate) fn deduct_throttle(&mut self, throttle: &Option<Duration>) {
|
||||
let Some(throttle) = throttle else {
|
||||
return;
|
||||
};
|
||||
self.throttled += *throttle;
|
||||
}
|
||||
}
|
||||
|
||||
impl Drop for SmgrOpTimer {
|
||||
fn drop(&mut self) {
|
||||
let elapsed = self.start.elapsed().as_secs_f64();
|
||||
let elapsed = self.start.elapsed();
|
||||
|
||||
let elapsed = match elapsed.checked_sub(self.throttled) {
|
||||
Some(elapsed) => elapsed,
|
||||
None => {
|
||||
use utils::rate_limit::RateLimit;
|
||||
static LOGGED: Lazy<Mutex<enum_map::EnumMap<SmgrQueryType, RateLimit>>> =
|
||||
Lazy::new(|| {
|
||||
Mutex::new(enum_map::EnumMap::from_array(std::array::from_fn(|_| {
|
||||
RateLimit::new(Duration::from_secs(10))
|
||||
})))
|
||||
});
|
||||
let mut guard = LOGGED.lock().unwrap();
|
||||
let rate_limit = &mut guard[self.op];
|
||||
rate_limit.call(|| {
|
||||
warn!(op=?self.op, ?elapsed, ?self.throttled, "implementation error: time spent throttled exceeds total request wall clock time");
|
||||
});
|
||||
elapsed // un-throttled time, more info than just saturating to 0
|
||||
}
|
||||
};
|
||||
|
||||
let elapsed = elapsed.as_secs_f64();
|
||||
|
||||
self.global_latency_histo.observe(elapsed);
|
||||
if let Some(per_timeline_getpage_histo) = &self.per_timeline_latency_histo {
|
||||
per_timeline_getpage_histo.observe(elapsed);
|
||||
@@ -1491,6 +1509,8 @@ impl SmgrQueryTimePerTimeline {
|
||||
global_latency_histo: self.global_latency[op as usize].clone(),
|
||||
per_timeline_latency_histo,
|
||||
start: started_at,
|
||||
op,
|
||||
throttled: Duration::ZERO,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -3299,7 +3319,7 @@ pub(crate) mod tenant_throttling {
|
||||
use once_cell::sync::Lazy;
|
||||
use utils::shard::TenantShardId;
|
||||
|
||||
use crate::tenant::{self, throttle::Metric};
|
||||
use crate::tenant::{self};
|
||||
|
||||
struct GlobalAndPerTenantIntCounter {
|
||||
global: IntCounter,
|
||||
@@ -3318,7 +3338,7 @@ pub(crate) mod tenant_throttling {
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) struct TimelineGet {
|
||||
pub(crate) struct Metrics<const KIND: usize> {
|
||||
count_accounted_start: GlobalAndPerTenantIntCounter,
|
||||
count_accounted_finish: GlobalAndPerTenantIntCounter,
|
||||
wait_time: GlobalAndPerTenantIntCounter,
|
||||
@@ -3391,40 +3411,41 @@ pub(crate) mod tenant_throttling {
|
||||
.unwrap()
|
||||
});
|
||||
|
||||
const KIND: &str = "timeline_get";
|
||||
const KINDS: &[&str] = &["pagestream"];
|
||||
pub type Pagestream = Metrics<0>;
|
||||
|
||||
impl TimelineGet {
|
||||
impl<const KIND: usize> Metrics<KIND> {
|
||||
pub(crate) fn new(tenant_shard_id: &TenantShardId) -> Self {
|
||||
let per_tenant_label_values = &[
|
||||
KIND,
|
||||
KINDS[KIND],
|
||||
&tenant_shard_id.tenant_id.to_string(),
|
||||
&tenant_shard_id.shard_slug().to_string(),
|
||||
];
|
||||
TimelineGet {
|
||||
Metrics {
|
||||
count_accounted_start: {
|
||||
GlobalAndPerTenantIntCounter {
|
||||
global: COUNT_ACCOUNTED_START.with_label_values(&[KIND]),
|
||||
global: COUNT_ACCOUNTED_START.with_label_values(&[KINDS[KIND]]),
|
||||
per_tenant: COUNT_ACCOUNTED_START_PER_TENANT
|
||||
.with_label_values(per_tenant_label_values),
|
||||
}
|
||||
},
|
||||
count_accounted_finish: {
|
||||
GlobalAndPerTenantIntCounter {
|
||||
global: COUNT_ACCOUNTED_FINISH.with_label_values(&[KIND]),
|
||||
global: COUNT_ACCOUNTED_FINISH.with_label_values(&[KINDS[KIND]]),
|
||||
per_tenant: COUNT_ACCOUNTED_FINISH_PER_TENANT
|
||||
.with_label_values(per_tenant_label_values),
|
||||
}
|
||||
},
|
||||
wait_time: {
|
||||
GlobalAndPerTenantIntCounter {
|
||||
global: WAIT_USECS.with_label_values(&[KIND]),
|
||||
global: WAIT_USECS.with_label_values(&[KINDS[KIND]]),
|
||||
per_tenant: WAIT_USECS_PER_TENANT
|
||||
.with_label_values(per_tenant_label_values),
|
||||
}
|
||||
},
|
||||
count_throttled: {
|
||||
GlobalAndPerTenantIntCounter {
|
||||
global: WAIT_COUNT.with_label_values(&[KIND]),
|
||||
global: WAIT_COUNT.with_label_values(&[KINDS[KIND]]),
|
||||
per_tenant: WAIT_COUNT_PER_TENANT
|
||||
.with_label_values(per_tenant_label_values),
|
||||
}
|
||||
@@ -3447,15 +3468,17 @@ pub(crate) mod tenant_throttling {
|
||||
&WAIT_USECS_PER_TENANT,
|
||||
&WAIT_COUNT_PER_TENANT,
|
||||
] {
|
||||
let _ = m.remove_label_values(&[
|
||||
KIND,
|
||||
&tenant_shard_id.tenant_id.to_string(),
|
||||
&tenant_shard_id.shard_slug().to_string(),
|
||||
]);
|
||||
for kind in KINDS {
|
||||
let _ = m.remove_label_values(&[
|
||||
kind,
|
||||
&tenant_shard_id.tenant_id.to_string(),
|
||||
&tenant_shard_id.shard_slug().to_string(),
|
||||
]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Metric for TimelineGet {
|
||||
impl<const KIND: usize> tenant::throttle::Metric for Metrics<KIND> {
|
||||
#[inline(always)]
|
||||
fn accounting_start(&self) {
|
||||
self.count_accounted_start.inc();
|
||||
|
||||
@@ -574,6 +574,41 @@ enum BatchedFeMessage {
|
||||
},
|
||||
}
|
||||
|
||||
impl BatchedFeMessage {
|
||||
async fn throttle(&mut self, cancel: &CancellationToken) -> Result<(), QueryError> {
|
||||
let (shard, tokens, timers) = match self {
|
||||
BatchedFeMessage::Exists { shard, timer, .. }
|
||||
| BatchedFeMessage::Nblocks { shard, timer, .. }
|
||||
| BatchedFeMessage::DbSize { shard, timer, .. }
|
||||
| BatchedFeMessage::GetSlruSegment { shard, timer, .. } => {
|
||||
(
|
||||
shard,
|
||||
// 1 token is probably under-estimating because these
|
||||
// request handlers typically do several Timeline::get calls.
|
||||
1,
|
||||
itertools::Either::Left(std::iter::once(timer)),
|
||||
)
|
||||
}
|
||||
BatchedFeMessage::GetPage { shard, pages, .. } => (
|
||||
shard,
|
||||
pages.len(),
|
||||
itertools::Either::Right(pages.iter_mut().map(|(_, _, timer)| timer)),
|
||||
),
|
||||
BatchedFeMessage::RespondError { .. } => return Ok(()),
|
||||
};
|
||||
let throttled = tokio::select! {
|
||||
throttled = shard.pagestream_throttle.throttle(tokens) => { throttled }
|
||||
_ = cancel.cancelled() => {
|
||||
return Err(QueryError::Shutdown);
|
||||
}
|
||||
};
|
||||
for timer in timers {
|
||||
timer.deduct_throttle(&throttled);
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
impl PageServerHandler {
|
||||
pub fn new(
|
||||
tenant_manager: Arc<TenantManager>,
|
||||
@@ -1157,13 +1192,18 @@ impl PageServerHandler {
|
||||
Ok(msg) => msg,
|
||||
Err(e) => break e,
|
||||
};
|
||||
let msg = match msg {
|
||||
let mut msg = match msg {
|
||||
Some(msg) => msg,
|
||||
None => {
|
||||
debug!("pagestream subprotocol end observed");
|
||||
return ((pgb_reader, timeline_handles), Ok(()));
|
||||
}
|
||||
};
|
||||
|
||||
if let Err(cancelled) = msg.throttle(&self.cancel).await {
|
||||
break cancelled;
|
||||
}
|
||||
|
||||
let err = self
|
||||
.pagesteam_handle_batched_message(pgb_writer, msg, &cancel, ctx)
|
||||
.await;
|
||||
@@ -1321,12 +1361,13 @@ impl PageServerHandler {
|
||||
return Ok(());
|
||||
}
|
||||
};
|
||||
let batch = match batch {
|
||||
let mut batch = match batch {
|
||||
Ok(batch) => batch,
|
||||
Err(e) => {
|
||||
return Err(e);
|
||||
}
|
||||
};
|
||||
batch.throttle(&self.cancel).await?;
|
||||
self.pagesteam_handle_batched_message(pgb_writer, batch, &cancel, &ctx)
|
||||
.await?;
|
||||
}
|
||||
|
||||
@@ -357,8 +357,8 @@ pub struct Tenant {
|
||||
|
||||
/// Throttle applied at the top of [`Timeline::get`].
|
||||
/// All [`Tenant::timelines`] of a given [`Tenant`] instance share the same [`throttle::Throttle`] instance.
|
||||
pub(crate) timeline_get_throttle:
|
||||
Arc<throttle::Throttle<crate::metrics::tenant_throttling::TimelineGet>>,
|
||||
pub(crate) pagestream_throttle:
|
||||
Arc<throttle::Throttle<crate::metrics::tenant_throttling::Pagestream>>,
|
||||
|
||||
/// An ongoing timeline detach concurrency limiter.
|
||||
///
|
||||
@@ -1678,7 +1678,7 @@ impl Tenant {
|
||||
remote_metadata,
|
||||
TimelineResources {
|
||||
remote_client,
|
||||
timeline_get_throttle: self.timeline_get_throttle.clone(),
|
||||
pagestream_throttle: self.pagestream_throttle.clone(),
|
||||
l0_flush_global_state: self.l0_flush_global_state.clone(),
|
||||
},
|
||||
LoadTimelineCause::Attach,
|
||||
@@ -3835,7 +3835,7 @@ impl Tenant {
|
||||
}
|
||||
}
|
||||
|
||||
fn get_timeline_get_throttle_config(
|
||||
fn get_pagestream_throttle_config(
|
||||
psconf: &'static PageServerConf,
|
||||
overrides: &TenantConfOpt,
|
||||
) -> throttle::Config {
|
||||
@@ -3846,8 +3846,8 @@ impl Tenant {
|
||||
}
|
||||
|
||||
pub(crate) fn tenant_conf_updated(&self, new_conf: &TenantConfOpt) {
|
||||
let conf = Self::get_timeline_get_throttle_config(self.conf, new_conf);
|
||||
self.timeline_get_throttle.reconfigure(conf)
|
||||
let conf = Self::get_pagestream_throttle_config(self.conf, new_conf);
|
||||
self.pagestream_throttle.reconfigure(conf)
|
||||
}
|
||||
|
||||
/// Helper function to create a new Timeline struct.
|
||||
@@ -4009,9 +4009,9 @@ impl Tenant {
|
||||
attach_wal_lag_cooldown: Arc::new(std::sync::OnceLock::new()),
|
||||
cancel: CancellationToken::default(),
|
||||
gate: Gate::default(),
|
||||
timeline_get_throttle: Arc::new(throttle::Throttle::new(
|
||||
Tenant::get_timeline_get_throttle_config(conf, &attached_conf.tenant_conf),
|
||||
crate::metrics::tenant_throttling::TimelineGet::new(&tenant_shard_id),
|
||||
pagestream_throttle: Arc::new(throttle::Throttle::new(
|
||||
Tenant::get_pagestream_throttle_config(conf, &attached_conf.tenant_conf),
|
||||
crate::metrics::tenant_throttling::Metrics::new(&tenant_shard_id),
|
||||
)),
|
||||
tenant_conf: Arc::new(ArcSwap::from_pointee(attached_conf)),
|
||||
ongoing_timeline_detach: std::sync::Mutex::default(),
|
||||
@@ -4909,7 +4909,7 @@ impl Tenant {
|
||||
fn build_timeline_resources(&self, timeline_id: TimelineId) -> TimelineResources {
|
||||
TimelineResources {
|
||||
remote_client: self.build_timeline_remote_client(timeline_id),
|
||||
timeline_get_throttle: self.timeline_get_throttle.clone(),
|
||||
pagestream_throttle: self.pagestream_throttle.clone(),
|
||||
l0_flush_global_state: self.l0_flush_global_state.clone(),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -471,14 +471,14 @@ async fn ingest_housekeeping_loop(tenant: Arc<Tenant>, cancel: CancellationToken
|
||||
|
||||
// TODO: rename the background loop kind to something more generic, like, tenant housekeeping.
|
||||
// Or just spawn another background loop for this throttle, it's not like it's super costly.
|
||||
info_span!(parent: None, "timeline_get_throttle", tenant_id=%tenant.tenant_shard_id, shard_id=%tenant.tenant_shard_id.shard_slug()).in_scope(|| {
|
||||
info_span!(parent: None, "pagestream_throttle", tenant_id=%tenant.tenant_shard_id, shard_id=%tenant.tenant_shard_id.shard_slug()).in_scope(|| {
|
||||
let now = Instant::now();
|
||||
let prev = std::mem::replace(&mut last_throttle_flag_reset_at, now);
|
||||
let Stats { count_accounted_start, count_accounted_finish, count_throttled, sum_throttled_usecs} = tenant.timeline_get_throttle.reset_stats();
|
||||
let Stats { count_accounted_start, count_accounted_finish, count_throttled, sum_throttled_usecs} = tenant.pagestream_throttle.reset_stats();
|
||||
if count_throttled == 0 {
|
||||
return;
|
||||
}
|
||||
let allowed_rps = tenant.timeline_get_throttle.steady_rps();
|
||||
let allowed_rps = tenant.pagestream_throttle.steady_rps();
|
||||
let delta = now - prev;
|
||||
info!(
|
||||
n_seconds=%format_args!("{:.3}", delta.as_secs_f64()),
|
||||
|
||||
@@ -1,5 +1,4 @@
|
||||
use std::{
|
||||
str::FromStr,
|
||||
sync::{
|
||||
atomic::{AtomicU64, Ordering},
|
||||
Arc,
|
||||
@@ -8,12 +7,8 @@ use std::{
|
||||
};
|
||||
|
||||
use arc_swap::ArcSwap;
|
||||
use enumset::EnumSet;
|
||||
use tracing::error;
|
||||
use utils::leaky_bucket::{LeakyBucketConfig, RateLimiter};
|
||||
|
||||
use crate::{context::RequestContext, task_mgr::TaskKind};
|
||||
|
||||
/// Throttle for `async` functions.
|
||||
///
|
||||
/// Runtime reconfigurable.
|
||||
@@ -35,7 +30,7 @@ pub struct Throttle<M: Metric> {
|
||||
}
|
||||
|
||||
pub struct Inner {
|
||||
task_kinds: EnumSet<TaskKind>,
|
||||
enabled: bool,
|
||||
rate_limiter: Arc<RateLimiter>,
|
||||
}
|
||||
|
||||
@@ -79,26 +74,12 @@ where
|
||||
}
|
||||
fn new_inner(config: Config) -> Inner {
|
||||
let Config {
|
||||
task_kinds,
|
||||
enabled,
|
||||
initial,
|
||||
refill_interval,
|
||||
refill_amount,
|
||||
max,
|
||||
} = config;
|
||||
let task_kinds: EnumSet<TaskKind> = task_kinds
|
||||
.iter()
|
||||
.filter_map(|s| match TaskKind::from_str(s) {
|
||||
Ok(v) => Some(v),
|
||||
Err(e) => {
|
||||
// TODO: avoid this failure mode
|
||||
error!(
|
||||
"cannot parse task kind, ignoring for rate limiting {}",
|
||||
utils::error::report_compact_sources(&e)
|
||||
);
|
||||
None
|
||||
}
|
||||
})
|
||||
.collect();
|
||||
|
||||
// steady rate, we expect `refill_amount` requests per `refill_interval`.
|
||||
// dividing gives us the rps.
|
||||
@@ -112,7 +93,7 @@ where
|
||||
let rate_limiter = RateLimiter::with_initial_tokens(config, f64::from(initial_tokens));
|
||||
|
||||
Inner {
|
||||
task_kinds,
|
||||
enabled: enabled.is_enabled(),
|
||||
rate_limiter: Arc::new(rate_limiter),
|
||||
}
|
||||
}
|
||||
@@ -141,11 +122,13 @@ where
|
||||
self.inner.load().rate_limiter.steady_rps()
|
||||
}
|
||||
|
||||
pub async fn throttle(&self, ctx: &RequestContext, key_count: usize) -> Option<Duration> {
|
||||
pub async fn throttle(&self, key_count: usize) -> Option<Duration> {
|
||||
let inner = self.inner.load_full(); // clones the `Inner` Arc
|
||||
if !inner.task_kinds.contains(ctx.task_kind()) {
|
||||
|
||||
if !inner.enabled {
|
||||
return None;
|
||||
};
|
||||
}
|
||||
|
||||
let start = std::time::Instant::now();
|
||||
|
||||
self.metric.accounting_start();
|
||||
|
||||
@@ -208,8 +208,8 @@ fn drop_wlock<T>(rlock: tokio::sync::RwLockWriteGuard<'_, T>) {
|
||||
/// The outward-facing resources required to build a Timeline
|
||||
pub struct TimelineResources {
|
||||
pub remote_client: RemoteTimelineClient,
|
||||
pub timeline_get_throttle:
|
||||
Arc<crate::tenant::throttle::Throttle<crate::metrics::tenant_throttling::TimelineGet>>,
|
||||
pub pagestream_throttle:
|
||||
Arc<crate::tenant::throttle::Throttle<crate::metrics::tenant_throttling::Pagestream>>,
|
||||
pub l0_flush_global_state: l0_flush::L0FlushGlobalState,
|
||||
}
|
||||
|
||||
@@ -411,9 +411,9 @@ pub struct Timeline {
|
||||
/// Timeline deletion will acquire both compaction and gc locks in whatever order.
|
||||
gc_lock: tokio::sync::Mutex<()>,
|
||||
|
||||
/// Cloned from [`super::Tenant::timeline_get_throttle`] on construction.
|
||||
timeline_get_throttle:
|
||||
Arc<crate::tenant::throttle::Throttle<crate::metrics::tenant_throttling::TimelineGet>>,
|
||||
/// Cloned from [`super::Tenant::pagestream_throttle`] on construction.
|
||||
pub(crate) pagestream_throttle:
|
||||
Arc<crate::tenant::throttle::Throttle<crate::metrics::tenant_throttling::Pagestream>>,
|
||||
|
||||
/// Size estimator for aux file v2
|
||||
pub(crate) aux_file_size_estimator: AuxFileSizeEstimator,
|
||||
@@ -949,7 +949,7 @@ impl Timeline {
|
||||
/// If a remote layer file is needed, it is downloaded as part of this
|
||||
/// call.
|
||||
///
|
||||
/// This method enforces [`Self::timeline_get_throttle`] internally.
|
||||
/// This method enforces [`Self::pagestream_throttle`] internally.
|
||||
///
|
||||
/// NOTE: It is considered an error to 'get' a key that doesn't exist. The
|
||||
/// abstraction above this needs to store suitable metadata to track what
|
||||
@@ -977,8 +977,6 @@ impl Timeline {
|
||||
// page_service.
|
||||
debug_assert!(!self.shard_identity.is_key_disposable(&key));
|
||||
|
||||
self.timeline_get_throttle.throttle(ctx, 1).await;
|
||||
|
||||
let keyspace = KeySpace {
|
||||
ranges: vec![key..key.next()],
|
||||
};
|
||||
@@ -1058,14 +1056,6 @@ impl Timeline {
|
||||
.for_task_kind(ctx.task_kind())
|
||||
.map(|metric| (metric, Instant::now()));
|
||||
|
||||
// start counting after throttle so that throttle time
|
||||
// is always less than observation time and we don't
|
||||
// underflow when computing `ex_throttled` below.
|
||||
let throttled = self
|
||||
.timeline_get_throttle
|
||||
.throttle(ctx, key_count as usize)
|
||||
.await;
|
||||
|
||||
let res = self
|
||||
.get_vectored_impl(
|
||||
keyspace.clone(),
|
||||
@@ -1077,23 +1067,7 @@ impl Timeline {
|
||||
|
||||
if let Some((metric, start)) = start {
|
||||
let elapsed = start.elapsed();
|
||||
let ex_throttled = if let Some(throttled) = throttled {
|
||||
elapsed.checked_sub(throttled)
|
||||
} else {
|
||||
Some(elapsed)
|
||||
};
|
||||
|
||||
if let Some(ex_throttled) = ex_throttled {
|
||||
metric.observe(ex_throttled.as_secs_f64());
|
||||
} else {
|
||||
use utils::rate_limit::RateLimit;
|
||||
static LOGGED: Lazy<Mutex<RateLimit>> =
|
||||
Lazy::new(|| Mutex::new(RateLimit::new(Duration::from_secs(10))));
|
||||
let mut rate_limit = LOGGED.lock().unwrap();
|
||||
rate_limit.call(|| {
|
||||
warn!("error deducting time spent throttled; this message is logged at a global rate limit");
|
||||
});
|
||||
}
|
||||
metric.observe(elapsed.as_secs_f64());
|
||||
}
|
||||
|
||||
res
|
||||
@@ -1138,16 +1112,6 @@ impl Timeline {
|
||||
.for_task_kind(ctx.task_kind())
|
||||
.map(ScanLatencyOngoingRecording::start_recording);
|
||||
|
||||
// start counting after throttle so that throttle time
|
||||
// is always less than observation time and we don't
|
||||
// underflow when computing the `ex_throttled` value in
|
||||
// `recording.observe(throttled)` below.
|
||||
let throttled = self
|
||||
.timeline_get_throttle
|
||||
// assume scan = 1 quota for now until we find a better way to process this
|
||||
.throttle(ctx, 1)
|
||||
.await;
|
||||
|
||||
let vectored_res = self
|
||||
.get_vectored_impl(
|
||||
keyspace.clone(),
|
||||
@@ -1158,7 +1122,7 @@ impl Timeline {
|
||||
.await;
|
||||
|
||||
if let Some(recording) = start {
|
||||
recording.observe(throttled);
|
||||
recording.observe();
|
||||
}
|
||||
|
||||
vectored_res
|
||||
@@ -2374,7 +2338,7 @@ impl Timeline {
|
||||
|
||||
standby_horizon: AtomicLsn::new(0),
|
||||
|
||||
timeline_get_throttle: resources.timeline_get_throttle,
|
||||
pagestream_throttle: resources.pagestream_throttle,
|
||||
|
||||
aux_file_size_estimator: AuxFileSizeEstimator::new(aux_file_metrics),
|
||||
|
||||
|
||||
@@ -298,7 +298,7 @@ impl DeleteTimelineFlow {
|
||||
None, // Ancestor is not needed for deletion.
|
||||
TimelineResources {
|
||||
remote_client,
|
||||
timeline_get_throttle: tenant.timeline_get_throttle.clone(),
|
||||
pagestream_throttle: tenant.pagestream_throttle.clone(),
|
||||
l0_flush_global_state: tenant.l0_flush_global_state.clone(),
|
||||
},
|
||||
// Important. We dont pass ancestor above because it can be missing.
|
||||
|
||||
@@ -33,7 +33,9 @@ def test_pageserver_getpage_throttle(neon_env_builder: NeonEnvBuilder, pg_bin: P
|
||||
conf={
|
||||
"compaction_period": f"{compaction_period}s",
|
||||
"timeline_get_throttle": {
|
||||
"task_kinds": ["PageRequestHandler"],
|
||||
"task_kinds": [
|
||||
"PageRequestHandler"
|
||||
], # any non-empty array will do here https://github.com/neondatabase/neon/pull/9962
|
||||
"initial": 0,
|
||||
"refill_interval": "100ms",
|
||||
"refill_amount": int(rate_limit_rps / 10),
|
||||
@@ -116,7 +118,6 @@ def test_pageserver_getpage_throttle(neon_env_builder: NeonEnvBuilder, pg_bin: P
|
||||
timeout=compaction_period,
|
||||
)
|
||||
|
||||
log.info("the smgr metric includes throttle time")
|
||||
smgr_query_seconds_post = ps_http.get_metric_value(smgr_metric_name, smgr_metrics_query)
|
||||
assert smgr_query_seconds_post is not None
|
||||
throttled_usecs_post = ps_http.get_metric_value(throttle_metric_name, throttle_metrics_query)
|
||||
@@ -125,13 +126,14 @@ def test_pageserver_getpage_throttle(neon_env_builder: NeonEnvBuilder, pg_bin: P
|
||||
actual_throttled_usecs = throttled_usecs_post - throttled_usecs_pre
|
||||
actual_throttled_secs = actual_throttled_usecs / 1_000_000
|
||||
|
||||
log.info("validate that the metric doesn't include throttle wait time")
|
||||
assert (
|
||||
pytest.approx(duration_secs, 0.1) == actual_smgr_query_seconds
|
||||
), "smgr metrics include throttle wait time"
|
||||
smgr_ex_throttle = actual_smgr_query_seconds - actual_throttled_secs
|
||||
assert smgr_ex_throttle > 0
|
||||
duration_secs >= 10 * actual_smgr_query_seconds
|
||||
), "smgr metrics should not include throttle wait time"
|
||||
|
||||
log.info("validate that the throttling wait time metrics is correct")
|
||||
assert (
|
||||
duration_secs > 10 * smgr_ex_throttle
|
||||
pytest.approx(actual_throttled_secs + actual_smgr_query_seconds, 0.1) == duration_secs
|
||||
), "most of the time in this test is spent throttled because the rate-limit's contribution to latency dominates"
|
||||
|
||||
|
||||
|
||||
Reference in New Issue
Block a user