From f62047ae97cc5185fbb025118d6cc4f18662d944 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Wed, 12 Feb 2025 17:12:21 +0100 Subject: [PATCH] pageserver: add separate semaphore for L0 compaction (#10780) ## Problem L0 compaction frequently gets starved out by other background tasks and image/GC compaction. L0 compaction must be responsive to keep read amplification under control. Touches #10694. Resolves #10689. ## Summary of changes Use a separate semaphore for the L0-only compaction pass. * Add a `CONCURRENT_L0_COMPACTION_TASKS` semaphore and `BackgroundLoopKind::L0Compaction`. * Add a setting `compaction_l0_semaphore` (default off via `compaction_l0_first`). * Use the L0 semaphore when doing an `OnlyL0Compaction` pass. * Use the background semaphore when doing a regular compaction pass (which includes an initial L0 pass). * While waiting for the background semaphore, yield for L0 compaction if triggered. * Add `CompactFlags::NoYield` to disable L0 yielding, and set it for the HTTP API route. * Remove the old `use_compaction_semaphore` setting and compaction-scoped semaphore. * Remove the warning when waiting for a semaphore; it's noisy and we have metrics. --- control_plane/src/pageserver.rs | 5 ++ libs/pageserver_api/src/config.rs | 7 +- libs/pageserver_api/src/models.rs | 8 +++ pageserver/src/config.rs | 6 -- pageserver/src/http/routes.rs | 1 + pageserver/src/tenant.rs | 1 + pageserver/src/tenant/config.rs | 13 ++++ pageserver/src/tenant/tasks.rs | 57 +++++++--------- pageserver/src/tenant/timeline.rs | 66 ++++++++++++++----- pageserver/src/tenant/timeline/compaction.rs | 5 +- .../src/tenant/timeline/eviction_task.rs | 7 +- .../regress/test_attach_tenant_config.py | 1 + 12 files changed, 110 insertions(+), 67 deletions(-) diff --git a/control_plane/src/pageserver.rs b/control_plane/src/pageserver.rs index e15b30236e..28d130d9e0 100644 --- a/control_plane/src/pageserver.rs +++ b/control_plane/src/pageserver.rs @@ -362,6 +362,11 @@ impl PageServerNode { .map(|x| x.parse::()) .transpose() .context("Failed to parse 'compaction_l0_first' as a bool")?, + compaction_l0_semaphore: settings + .remove("compaction_l0_semaphore") + .map(|x| x.parse::()) + .transpose() + .context("Failed to parse 'compaction_l0_semaphore' as a bool")?, l0_flush_delay_threshold: settings .remove("l0_flush_delay_threshold") .map(|x| x.parse::()) diff --git a/libs/pageserver_api/src/config.rs b/libs/pageserver_api/src/config.rs index 9bc1b6d359..79f068a47b 100644 --- a/libs/pageserver_api/src/config.rs +++ b/libs/pageserver_api/src/config.rs @@ -94,7 +94,6 @@ pub struct ConfigToml { pub ondemand_download_behavior_treat_error_as_warn: bool, #[serde(with = "humantime_serde")] pub background_task_maximum_delay: Duration, - pub use_compaction_semaphore: bool, pub control_plane_api: Option, pub control_plane_api_token: Option, pub control_plane_emergency_mode: bool, @@ -266,6 +265,9 @@ pub struct TenantConfigToml { pub compaction_algorithm: crate::models::CompactionAlgorithmSettings, /// If true, compact down L0 across all tenant timelines before doing regular compaction. pub compaction_l0_first: bool, + /// If true, use a separate semaphore (i.e. concurrency limit) for the L0 compaction pass. Only + /// has an effect if `compaction_l0_first` is `true`. + pub compaction_l0_semaphore: bool, /// Level0 delta layer threshold at which to delay layer flushes for compaction backpressure, /// such that they take 2x as long, and start waiting for layer flushes during ephemeral layer /// rolls. This helps compaction keep up with WAL ingestion, and avoids read amplification @@ -474,7 +476,6 @@ impl Default for ConfigToml { DEFAULT_BACKGROUND_TASK_MAXIMUM_DELAY, ) .unwrap()), - use_compaction_semaphore: false, control_plane_api: (None), control_plane_api_token: (None), @@ -548,6 +549,7 @@ pub mod tenant_conf_defaults { // calculation to avoid loading all keys into the memory). So with this config, we can get a maximum peak compaction usage of 18GB. pub const DEFAULT_COMPACTION_UPPER_LIMIT: usize = 50; pub const DEFAULT_COMPACTION_L0_FIRST: bool = false; + pub const DEFAULT_COMPACTION_L0_SEMAPHORE: bool = true; pub const DEFAULT_COMPACTION_ALGORITHM: crate::models::CompactionAlgorithm = crate::models::CompactionAlgorithm::Legacy; @@ -598,6 +600,7 @@ impl Default for TenantConfigToml { kind: DEFAULT_COMPACTION_ALGORITHM, }, compaction_l0_first: DEFAULT_COMPACTION_L0_FIRST, + compaction_l0_semaphore: DEFAULT_COMPACTION_L0_SEMAPHORE, l0_flush_delay_threshold: None, l0_flush_stall_threshold: None, l0_flush_wait_upload: DEFAULT_L0_FLUSH_WAIT_UPLOAD, diff --git a/libs/pageserver_api/src/models.rs b/libs/pageserver_api/src/models.rs index 335ac4cec5..6dbfbec345 100644 --- a/libs/pageserver_api/src/models.rs +++ b/libs/pageserver_api/src/models.rs @@ -466,6 +466,8 @@ pub struct TenantConfigPatch { #[serde(skip_serializing_if = "FieldPatch::is_noop")] pub compaction_l0_first: FieldPatch, #[serde(skip_serializing_if = "FieldPatch::is_noop")] + pub compaction_l0_semaphore: FieldPatch, + #[serde(skip_serializing_if = "FieldPatch::is_noop")] pub l0_flush_delay_threshold: FieldPatch, #[serde(skip_serializing_if = "FieldPatch::is_noop")] pub l0_flush_stall_threshold: FieldPatch, @@ -532,6 +534,7 @@ pub struct TenantConfig { // defer parsing compaction_algorithm, like eviction_policy pub compaction_algorithm: Option, pub compaction_l0_first: Option, + pub compaction_l0_semaphore: Option, pub l0_flush_delay_threshold: Option, pub l0_flush_stall_threshold: Option, pub l0_flush_wait_upload: Option, @@ -571,6 +574,7 @@ impl TenantConfig { mut compaction_upper_limit, mut compaction_algorithm, mut compaction_l0_first, + mut compaction_l0_semaphore, mut l0_flush_delay_threshold, mut l0_flush_stall_threshold, mut l0_flush_wait_upload, @@ -611,6 +615,9 @@ impl TenantConfig { .apply(&mut compaction_upper_limit); patch.compaction_algorithm.apply(&mut compaction_algorithm); patch.compaction_l0_first.apply(&mut compaction_l0_first); + patch + .compaction_l0_semaphore + .apply(&mut compaction_l0_semaphore); patch .l0_flush_delay_threshold .apply(&mut l0_flush_delay_threshold); @@ -675,6 +682,7 @@ impl TenantConfig { compaction_upper_limit, compaction_algorithm, compaction_l0_first, + compaction_l0_semaphore, l0_flush_delay_threshold, l0_flush_stall_threshold, l0_flush_wait_upload, diff --git a/pageserver/src/config.rs b/pageserver/src/config.rs index 3dd519de75..c5368f6806 100644 --- a/pageserver/src/config.rs +++ b/pageserver/src/config.rs @@ -140,10 +140,6 @@ pub struct PageServerConf { /// not terrible. pub background_task_maximum_delay: Duration, - /// If true, use a separate semaphore for compaction tasks instead of the common background task - /// semaphore. Defaults to false. - pub use_compaction_semaphore: bool, - pub control_plane_api: Option, /// JWT token for use with the control plane API. @@ -340,7 +336,6 @@ impl PageServerConf { test_remote_failures, ondemand_download_behavior_treat_error_as_warn, background_task_maximum_delay, - use_compaction_semaphore, control_plane_api, control_plane_api_token, control_plane_emergency_mode, @@ -395,7 +390,6 @@ impl PageServerConf { test_remote_failures, ondemand_download_behavior_treat_error_as_warn, background_task_maximum_delay, - use_compaction_semaphore, control_plane_api, control_plane_emergency_mode, heatmap_upload_concurrency, diff --git a/pageserver/src/http/routes.rs b/pageserver/src/http/routes.rs index 1d5edaa571..bd196621c1 100644 --- a/pageserver/src/http/routes.rs +++ b/pageserver/src/http/routes.rs @@ -2151,6 +2151,7 @@ async fn timeline_compact_handler( let state = get_state(&request); let mut flags = EnumSet::empty(); + flags |= CompactFlags::NoYield; // run compaction to completion if Some(true) == parse_query_param::<_, bool>(&request, "force_l0_compaction")? { flags |= CompactFlags::ForceL0Compaction; diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index 8520ae62e8..4c65991e45 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -5542,6 +5542,7 @@ pub(crate) mod harness { compaction_upper_limit: Some(tenant_conf.compaction_upper_limit), compaction_algorithm: Some(tenant_conf.compaction_algorithm), compaction_l0_first: Some(tenant_conf.compaction_l0_first), + compaction_l0_semaphore: Some(tenant_conf.compaction_l0_semaphore), l0_flush_delay_threshold: tenant_conf.l0_flush_delay_threshold, l0_flush_stall_threshold: tenant_conf.l0_flush_stall_threshold, l0_flush_wait_upload: Some(tenant_conf.l0_flush_wait_upload), diff --git a/pageserver/src/tenant/config.rs b/pageserver/src/tenant/config.rs index cff33afffd..7fdfd736ad 100644 --- a/pageserver/src/tenant/config.rs +++ b/pageserver/src/tenant/config.rs @@ -289,6 +289,10 @@ pub struct TenantConfOpt { #[serde(default)] pub compaction_l0_first: Option, + #[serde(skip_serializing_if = "Option::is_none")] + #[serde(default)] + pub compaction_l0_semaphore: Option, + #[serde(skip_serializing_if = "Option::is_none")] #[serde(default)] pub l0_flush_delay_threshold: Option, @@ -423,6 +427,9 @@ impl TenantConfOpt { compaction_l0_first: self .compaction_l0_first .unwrap_or(global_conf.compaction_l0_first), + compaction_l0_semaphore: self + .compaction_l0_semaphore + .unwrap_or(global_conf.compaction_l0_semaphore), l0_flush_delay_threshold: self .l0_flush_delay_threshold .or(global_conf.l0_flush_delay_threshold), @@ -501,6 +508,7 @@ impl TenantConfOpt { mut compaction_upper_limit, mut compaction_algorithm, mut compaction_l0_first, + mut compaction_l0_semaphore, mut l0_flush_delay_threshold, mut l0_flush_stall_threshold, mut l0_flush_wait_upload, @@ -547,6 +555,9 @@ impl TenantConfOpt { .apply(&mut compaction_upper_limit); patch.compaction_algorithm.apply(&mut compaction_algorithm); patch.compaction_l0_first.apply(&mut compaction_l0_first); + patch + .compaction_l0_semaphore + .apply(&mut compaction_l0_semaphore); patch .l0_flush_delay_threshold .apply(&mut l0_flush_delay_threshold); @@ -629,6 +640,7 @@ impl TenantConfOpt { compaction_upper_limit, compaction_algorithm, compaction_l0_first, + compaction_l0_semaphore, l0_flush_delay_threshold, l0_flush_stall_threshold, l0_flush_wait_upload, @@ -692,6 +704,7 @@ impl From for models::TenantConfig { compaction_threshold: value.compaction_threshold, compaction_upper_limit: value.compaction_upper_limit, compaction_l0_first: value.compaction_l0_first, + compaction_l0_semaphore: value.compaction_l0_semaphore, l0_flush_delay_threshold: value.l0_flush_delay_threshold, l0_flush_stall_threshold: value.l0_flush_stall_threshold, l0_flush_wait_upload: value.l0_flush_wait_upload, diff --git a/pageserver/src/tenant/tasks.rs b/pageserver/src/tenant/tasks.rs index 1fa01e4229..029444e973 100644 --- a/pageserver/src/tenant/tasks.rs +++ b/pageserver/src/tenant/tasks.rs @@ -4,7 +4,7 @@ use std::cmp::max; use std::future::Future; use std::ops::{ControlFlow, RangeInclusive}; use std::pin::pin; -use std::sync::{Arc, Mutex}; +use std::sync::Arc; use std::time::{Duration, Instant}; use once_cell::sync::Lazy; @@ -15,7 +15,7 @@ use tokio_util::sync::CancellationToken; use tracing::*; use crate::context::{DownloadBehavior, RequestContext}; -use crate::metrics::{BackgroundLoopSemaphoreMetricsRecorder, TENANT_TASK_EVENTS}; +use crate::metrics::{self, BackgroundLoopSemaphoreMetricsRecorder, TENANT_TASK_EVENTS}; use crate::task_mgr::{self, TaskKind, BACKGROUND_RUNTIME, TOKIO_WORKER_THREADS}; use crate::tenant::throttle::Stats; use crate::tenant::timeline::compaction::CompactionOutcome; @@ -25,7 +25,6 @@ use pageserver_api::config::tenant_conf_defaults::DEFAULT_COMPACTION_PERIOD; use utils::backoff::exponential_backoff_duration; use utils::completion::Barrier; use utils::pausable_failpoint; -use utils::rate_limit::RateLimit; /// Semaphore limiting concurrent background tasks (across all tenants). /// @@ -38,17 +37,17 @@ static CONCURRENT_BACKGROUND_TASKS: Lazy = Lazy::new(|| { Semaphore::new(permits) }); -/// Semaphore limiting concurrent compaction tasks (across all tenants). This is disabled by -/// default, see `use_compaction_semaphore`. -/// -/// We use 3/4 Tokio threads, to avoid blocking all threads in case we do any CPU-heavy work. +/// Semaphore limiting concurrent L0 compaction tasks (across all tenants). This is only used if +/// both `compaction_l0_semaphore` and `compaction_l0_first` are enabled. /// /// This is a separate semaphore from background tasks, because L0 compaction needs to be responsive -/// to avoid high read amp during heavy write workloads. +/// to avoid high read amp during heavy write workloads. Regular image/GC compaction is less +/// important (e.g. due to page images in delta layers) and can wait for other background tasks. /// -/// TODO: split image compaction and L0 compaction, and move image compaction to background tasks. -/// Only L0 compaction needs to be responsive, and it shouldn't block on image compaction. -static CONCURRENT_COMPACTION_TASKS: Lazy = Lazy::new(|| { +/// We use 3/4 Tokio threads, to avoid blocking all threads in case we do any CPU-heavy work. Note +/// that this runs on the same Tokio runtime as `CONCURRENT_BACKGROUND_TASKS`, and shares the same +/// thread pool. +static CONCURRENT_L0_COMPACTION_TASKS: Lazy = Lazy::new(|| { let total_threads = TOKIO_WORKER_THREADS.get(); let permits = max(1, (total_threads * 3).checked_div(4).unwrap_or(0)); assert_ne!(permits, 0, "we will not be adding in permits later"); @@ -59,7 +58,7 @@ static CONCURRENT_COMPACTION_TASKS: Lazy = Lazy::new(|| { /// Background jobs. /// /// NB: not all of these acquire a CONCURRENT_BACKGROUND_TASKS semaphore permit, only the ones that -/// do any significant IO. +/// do any significant IO or CPU work. #[derive( Debug, PartialEq, @@ -72,6 +71,9 @@ static CONCURRENT_COMPACTION_TASKS: Lazy = Lazy::new(|| { )] #[strum(serialize_all = "snake_case")] pub(crate) enum BackgroundLoopKind { + /// L0Compaction runs as a separate pass within the Compaction loop, not a separate loop. It is + /// used to request the `CONCURRENT_L0_COMPACTION_TASKS` semaphore and associated metrics. + L0Compaction, Compaction, Gc, Eviction, @@ -91,37 +93,22 @@ pub struct BackgroundLoopSemaphorePermit<'a> { /// Acquires a semaphore permit, to limit concurrent background jobs. pub(crate) async fn acquire_concurrency_permit( loop_kind: BackgroundLoopKind, - use_compaction_semaphore: bool, _ctx: &RequestContext, ) -> BackgroundLoopSemaphorePermit<'static> { - // TODO: use a lower threshold and remove the pacer once we resolve some blockage. - const WARN_THRESHOLD: Duration = Duration::from_secs(600); - static WARN_PACER: Lazy> = - Lazy::new(|| Mutex::new(RateLimit::new(Duration::from_secs(10)))); - - let mut recorder = crate::metrics::BACKGROUND_LOOP_SEMAPHORE.record(loop_kind); + let mut recorder = metrics::BACKGROUND_LOOP_SEMAPHORE.record(loop_kind); if loop_kind == BackgroundLoopKind::InitialLogicalSizeCalculation { pausable_failpoint!("initial-size-calculation-permit-pause"); } // TODO: assert that we run on BACKGROUND_RUNTIME; requires tokio_unstable Handle::id(); - let permit = if loop_kind == BackgroundLoopKind::Compaction && use_compaction_semaphore { - CONCURRENT_COMPACTION_TASKS.acquire().await - } else { - assert!(!use_compaction_semaphore); - CONCURRENT_BACKGROUND_TASKS.acquire().await - } - .expect("should never close"); + let semaphore = match loop_kind { + BackgroundLoopKind::L0Compaction => &CONCURRENT_L0_COMPACTION_TASKS, + _ => &CONCURRENT_BACKGROUND_TASKS, + }; + let permit = semaphore.acquire().await.expect("should never close"); - let waited = recorder.acquired(); - if waited >= WARN_THRESHOLD { - let waited = waited.as_secs_f64(); - WARN_PACER - .lock() - .unwrap() - .call(|| warn!("{loop_kind} task waited {waited:.3}s for semaphore permit")); - } + recorder.acquired(); BackgroundLoopSemaphorePermit { _permit: permit, @@ -589,7 +576,7 @@ pub(crate) fn warn_when_period_overrun( ?task, "task iteration took longer than the configured period" ); - crate::metrics::BACKGROUND_LOOP_PERIOD_OVERRUN_COUNT + metrics::BACKGROUND_LOOP_PERIOD_OVERRUN_COUNT .with_label_values(&[task.into(), &format!("{}", period.as_secs())]) .inc(); } diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index afa8efa453..33ca75de17 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -879,6 +879,9 @@ pub(crate) enum CompactFlags { OnlyL0Compaction, EnhancedGcBottomMostCompaction, DryRun, + /// Disables compaction yielding e.g. due to high L0 count. This is set e.g. when requesting + /// compaction via HTTP API. + NoYield, } #[serde_with::serde_as] @@ -1787,34 +1790,46 @@ impl Timeline { .await } - /// Outermost timeline compaction operation; downloads needed layers. Returns whether we have pending - /// compaction tasks. + /// Outermost timeline compaction operation; downloads needed layers. + /// + /// NB: the cancellation token is usually from a background task, but can also come from a + /// request task. pub(crate) async fn compact_with_options( self: &Arc, cancel: &CancellationToken, options: CompactOptions, ctx: &RequestContext, ) -> Result { - // most likely the cancellation token is from background task, but in tests it could be the - // request task as well. + // Acquire the compaction lock and task semaphore. + // + // L0-only compaction uses a separate semaphore (if enabled) to make sure it isn't starved + // out by other background tasks (including image compaction). We request this via + // `BackgroundLoopKind::L0Compaction`. + // + // If this is a regular compaction pass, and L0-only compaction is enabled in the config, + // then we should yield for immediate L0 compaction if necessary while we're waiting for the + // background task semaphore. There's no point yielding otherwise, since we'd just end up + // right back here. + let is_l0_only = options.flags.contains(CompactFlags::OnlyL0Compaction); + let semaphore_kind = match is_l0_only && self.get_compaction_l0_semaphore() { + true => BackgroundLoopKind::L0Compaction, + false => BackgroundLoopKind::Compaction, + }; + let yield_for_l0 = !is_l0_only + && self.get_compaction_l0_first() + && !options.flags.contains(CompactFlags::NoYield); - let prepare = async move { + let acquire = async move { let guard = self.compaction_lock.lock().await; - - let permit = super::tasks::acquire_concurrency_permit( - BackgroundLoopKind::Compaction, - self.conf.use_compaction_semaphore, - ctx, - ) - .await; - + let permit = super::tasks::acquire_concurrency_permit(semaphore_kind, ctx).await; (guard, permit) }; - // this wait probably never needs any "long time spent" logging, because we already nag if - // compaction task goes over it's period (20s) which is quite often in production. let (_guard, _permit) = tokio::select! { - tuple = prepare => { tuple }, + (guard, permit) = acquire => (guard, permit), + _ = self.l0_compaction_trigger.notified(), if yield_for_l0 => { + return Ok(CompactionOutcome::YieldForL0); + } _ = self.cancel.cancelled() => return Ok(CompactionOutcome::Skipped), _ = cancel.cancelled() => return Ok(CompactionOutcome::Skipped), }; @@ -2326,6 +2341,20 @@ impl Timeline { .unwrap_or(self.conf.default_tenant_conf.compaction_upper_limit) } + pub fn get_compaction_l0_first(&self) -> bool { + let tenant_conf = self.tenant_conf.load().tenant_conf.clone(); + tenant_conf + .compaction_l0_first + .unwrap_or(self.conf.default_tenant_conf.compaction_l0_first) + } + + pub fn get_compaction_l0_semaphore(&self) -> bool { + let tenant_conf = self.tenant_conf.load().tenant_conf.clone(); + tenant_conf + .compaction_l0_semaphore + .unwrap_or(self.conf.default_tenant_conf.compaction_l0_semaphore) + } + fn get_l0_flush_delay_threshold(&self) -> Option { // Disable L0 flushes by default. This and compaction needs further tuning. const DEFAULT_L0_FLUSH_DELAY_FACTOR: usize = 0; // TODO: default to e.g. 3 @@ -3143,7 +3172,6 @@ impl Timeline { async move { let wait_for_permit = super::tasks::acquire_concurrency_permit( BackgroundLoopKind::InitialLogicalSizeCalculation, - false, background_ctx, ); @@ -4188,6 +4216,7 @@ impl Timeline { ImageLayerCreationMode::Initial, ctx, LastImageLayerCreationStatus::Initial, + false, // don't yield for L0, we're flushing L0 ) .await?; debug_assert!( @@ -4760,6 +4789,7 @@ impl Timeline { mode: ImageLayerCreationMode, ctx: &RequestContext, last_status: LastImageLayerCreationStatus, + yield_for_l0: bool, ) -> Result<(Vec, LastImageLayerCreationStatus), CreateImageLayersError> { let timer = self.metrics.create_images_time_histo.start_timer(); @@ -4956,7 +4986,7 @@ impl Timeline { if let ImageLayerCreationMode::Try = mode { // We have at least made some progress - if batch_image_writer.pending_layer_num() >= 1 { + if yield_for_l0 && batch_image_writer.pending_layer_num() >= 1 { // The `Try` mode is currently only used on the compaction path. We want to avoid // image layer generation taking too long time and blocking L0 compaction. So in this // mode, we also inspect the current number of L0 layers and skip image layer generation diff --git a/pageserver/src/tenant/timeline/compaction.rs b/pageserver/src/tenant/timeline/compaction.rs index aea92d34e0..5b915c50d3 100644 --- a/pageserver/src/tenant/timeline/compaction.rs +++ b/pageserver/src/tenant/timeline/compaction.rs @@ -726,7 +726,9 @@ impl Timeline { } // Yield if we have pending L0 compaction. The scheduler will do another pass. - if l0_outcome == CompactionOutcome::Pending || l0_outcome == CompactionOutcome::YieldForL0 { + if (l0_outcome == CompactionOutcome::Pending || l0_outcome == CompactionOutcome::YieldForL0) + && !options.flags.contains(CompactFlags::NoYield) + { info!("image/ancestor compaction yielding for L0 compaction"); return Ok(CompactionOutcome::YieldForL0); } @@ -774,6 +776,7 @@ impl Timeline { .load() .as_ref() .clone(), + !options.flags.contains(CompactFlags::NoYield), ) .await .inspect_err(|err| { diff --git a/pageserver/src/tenant/timeline/eviction_task.rs b/pageserver/src/tenant/timeline/eviction_task.rs index 42e5f1496d..77c33349e0 100644 --- a/pageserver/src/tenant/timeline/eviction_task.rs +++ b/pageserver/src/tenant/timeline/eviction_task.rs @@ -332,11 +332,8 @@ impl Timeline { cancel: &CancellationToken, ctx: &RequestContext, ) -> ControlFlow<(), BackgroundLoopSemaphorePermit<'static>> { - let acquire_permit = crate::tenant::tasks::acquire_concurrency_permit( - BackgroundLoopKind::Eviction, - false, - ctx, - ); + let acquire_permit = + crate::tenant::tasks::acquire_concurrency_permit(BackgroundLoopKind::Eviction, ctx); tokio::select! { permit = acquire_permit => ControlFlow::Continue(permit), diff --git a/test_runner/regress/test_attach_tenant_config.py b/test_runner/regress/test_attach_tenant_config.py index 34d56c5cb1..07600dd911 100644 --- a/test_runner/regress/test_attach_tenant_config.py +++ b/test_runner/regress/test_attach_tenant_config.py @@ -141,6 +141,7 @@ def test_fully_custom_config(positive_env: NeonEnv): "compaction_threshold": 13, "compaction_upper_limit": 100, "compaction_l0_first": False, + "compaction_l0_semaphore": False, "l0_flush_delay_threshold": 25, "l0_flush_stall_threshold": 42, "l0_flush_wait_upload": False,