From 605fb04f8912d5939d72c9c7b17b8c543f8fc078 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov <34828390+DimasKovas@users.noreply.github.com> Date: Thu, 26 Jun 2025 17:26:24 +0400 Subject: [PATCH 01/20] pageserver: use bounded sender for basebackup cache (#12342) ## Problem Basebackup cache now uses unbounded channel for prepare requests. In theory it can grow large if the cache is hung and does not process the requests. - Part of https://github.com/neondatabase/cloud/issues/29353 ## Summary of changes - Replace an unbounded channel with a bounded one, the size is configurable. - Add `pageserver_basebackup_cache_prepare_queue_size` to observe the size of the queue. - Refactor a bit to move all metrics logic to `basebackup_cache.rs` --- libs/pageserver_api/src/config.rs | 4 ++ pageserver/src/basebackup_cache.rs | 109 ++++++++++++++++++++++++----- pageserver/src/bin/pageserver.rs | 13 ++-- pageserver/src/metrics.rs | 8 +++ pageserver/src/page_service.rs | 15 +--- pageserver/src/tenant.rs | 20 +++--- pageserver/src/tenant/mgr.rs | 16 +++-- pageserver/src/tenant/timeline.rs | 30 ++++---- 8 files changed, 142 insertions(+), 73 deletions(-) diff --git a/libs/pageserver_api/src/config.rs b/libs/pageserver_api/src/config.rs index 7926e839cf..0cfa1c8485 100644 --- a/libs/pageserver_api/src/config.rs +++ b/libs/pageserver_api/src/config.rs @@ -371,6 +371,9 @@ pub struct BasebackupCacheConfig { // TODO(diko): support max_entry_size_bytes. // pub max_entry_size_bytes: u64, pub max_size_entries: usize, + /// Size of the channel used to send prepare requests to the basebackup cache worker. + /// If exceeded, new prepare requests will be dropped. + pub prepare_channel_size: usize, } impl Default for BasebackupCacheConfig { @@ -380,6 +383,7 @@ impl Default for BasebackupCacheConfig { max_total_size_bytes: 1024 * 1024 * 1024, // 1 GiB // max_entry_size_bytes: 16 * 1024 * 1024, // 16 MiB max_size_entries: 1000, + prepare_channel_size: 100, } } } diff --git a/pageserver/src/basebackup_cache.rs b/pageserver/src/basebackup_cache.rs index 69438dae7f..4966fee2d7 100644 --- a/pageserver/src/basebackup_cache.rs +++ b/pageserver/src/basebackup_cache.rs @@ -6,7 +6,7 @@ use metrics::core::{AtomicU64, GenericCounter}; use pageserver_api::{config::BasebackupCacheConfig, models::TenantState}; use tokio::{ io::{AsyncWriteExt, BufWriter}, - sync::mpsc::{UnboundedReceiver, UnboundedSender}, + sync::mpsc::{Receiver, Sender, error::TrySendError}, }; use tokio_util::sync::CancellationToken; use utils::{ @@ -19,8 +19,8 @@ use crate::{ basebackup::send_basebackup_tarball, context::{DownloadBehavior, RequestContext}, metrics::{ - BASEBACKUP_CACHE_ENTRIES, BASEBACKUP_CACHE_PREPARE, BASEBACKUP_CACHE_READ, - BASEBACKUP_CACHE_SIZE, + BASEBACKUP_CACHE_ENTRIES, BASEBACKUP_CACHE_PREPARE, BASEBACKUP_CACHE_PREPARE_QUEUE_SIZE, + BASEBACKUP_CACHE_READ, BASEBACKUP_CACHE_SIZE, }, task_mgr::TaskKind, tenant::{ @@ -35,8 +35,8 @@ pub struct BasebackupPrepareRequest { pub lsn: Lsn, } -pub type BasebackupPrepareSender = UnboundedSender; -pub type BasebackupPrepareReceiver = UnboundedReceiver; +pub type BasebackupPrepareSender = Sender; +pub type BasebackupPrepareReceiver = Receiver; #[derive(Clone)] struct CacheEntry { @@ -60,40 +60,65 @@ struct CacheEntry { /// and ~1 RPS for get requests. pub struct BasebackupCache { data_dir: Utf8PathBuf, + config: Option, entries: std::sync::Mutex>, + prepare_sender: BasebackupPrepareSender, + read_hit_count: GenericCounter, read_miss_count: GenericCounter, read_err_count: GenericCounter, + + prepare_skip_count: GenericCounter, } impl BasebackupCache { - /// Creates a BasebackupCache and spawns the background task. - /// The initialization of the cache is performed in the background and does not - /// block the caller. The cache will return `None` for any get requests until - /// initialization is complete. - pub fn spawn( - runtime_handle: &tokio::runtime::Handle, + /// Create a new BasebackupCache instance. + /// Also returns a BasebackupPrepareReceiver which is needed to start + /// the background task. + /// The cache is initialized from the data_dir in the background task. + /// The cache will return `None` for any get requests until the initialization is complete. + /// The background task is spawned separately using [`Self::spawn_background_task`] + /// to avoid a circular dependency between the cache and the tenant manager. + pub fn new( data_dir: Utf8PathBuf, config: Option, - prepare_receiver: BasebackupPrepareReceiver, - tenant_manager: Arc, - cancel: CancellationToken, - ) -> Arc { + ) -> (Arc, BasebackupPrepareReceiver) { + let chan_size = config.as_ref().map(|c| c.max_size_entries).unwrap_or(1); + + let (prepare_sender, prepare_receiver) = tokio::sync::mpsc::channel(chan_size); + let cache = Arc::new(BasebackupCache { data_dir, - + config, entries: std::sync::Mutex::new(HashMap::new()), + prepare_sender, read_hit_count: BASEBACKUP_CACHE_READ.with_label_values(&["hit"]), read_miss_count: BASEBACKUP_CACHE_READ.with_label_values(&["miss"]), read_err_count: BASEBACKUP_CACHE_READ.with_label_values(&["error"]), + + prepare_skip_count: BASEBACKUP_CACHE_PREPARE.with_label_values(&["skip"]), }); - if let Some(config) = config { + (cache, prepare_receiver) + } + + /// Spawns the background task. + /// The background task initializes the cache from the disk, + /// processes prepare requests, and cleans up outdated cache entries. + /// Noop if the cache is disabled (config is None). + pub fn spawn_background_task( + self: Arc, + runtime_handle: &tokio::runtime::Handle, + prepare_receiver: BasebackupPrepareReceiver, + tenant_manager: Arc, + cancel: CancellationToken, + ) { + if let Some(config) = self.config.clone() { let background = BackgroundTask { - c: cache.clone(), + c: self, config, tenant_manager, @@ -108,8 +133,45 @@ impl BasebackupCache { }; runtime_handle.spawn(background.run(prepare_receiver)); } + } - cache + /// Send a basebackup prepare request to the background task. + /// The basebackup will be prepared asynchronously, it does not block the caller. + /// The request will be skipped if any cache limits are exceeded. + pub fn send_prepare(&self, tenant_shard_id: TenantShardId, timeline_id: TimelineId, lsn: Lsn) { + let req = BasebackupPrepareRequest { + tenant_shard_id, + timeline_id, + lsn, + }; + + BASEBACKUP_CACHE_PREPARE_QUEUE_SIZE.inc(); + let res = self.prepare_sender.try_send(req); + + if let Err(e) = res { + BASEBACKUP_CACHE_PREPARE_QUEUE_SIZE.dec(); + self.prepare_skip_count.inc(); + match e { + TrySendError::Full(_) => { + // Basebackup prepares are pretty rare, normally we should not hit this. + tracing::info!( + tenant_id = %tenant_shard_id.tenant_id, + %timeline_id, + %lsn, + "Basebackup prepare channel is full, skipping the request" + ); + } + TrySendError::Closed(_) => { + // Normal during shutdown, not critical. + tracing::info!( + tenant_id = %tenant_shard_id.tenant_id, + %timeline_id, + %lsn, + "Basebackup prepare channel is closed, skipping the request" + ); + } + } + } } /// Gets a basebackup entry from the cache. @@ -122,6 +184,10 @@ impl BasebackupCache { timeline_id: TimelineId, lsn: Lsn, ) -> Option { + if !self.is_enabled() { + return None; + } + // Fast path. Check if the entry exists using the in-memory state. let tti = TenantTimelineId::new(tenant_id, timeline_id); if self.entries.lock().unwrap().get(&tti).map(|e| e.lsn) != Some(lsn) { @@ -149,6 +215,10 @@ impl BasebackupCache { } } + pub fn is_enabled(&self) -> bool { + self.config.is_some() + } + // Private methods. fn entry_filename(tenant_id: TenantId, timeline_id: TimelineId, lsn: Lsn) -> String { @@ -366,6 +436,7 @@ impl BackgroundTask { loop { tokio::select! { Some(req) = prepare_receiver.recv() => { + BASEBACKUP_CACHE_PREPARE_QUEUE_SIZE.dec(); if let Err(err) = self.prepare_basebackup( req.tenant_shard_id, req.timeline_id, diff --git a/pageserver/src/bin/pageserver.rs b/pageserver/src/bin/pageserver.rs index d137d651eb..327384fd82 100644 --- a/pageserver/src/bin/pageserver.rs +++ b/pageserver/src/bin/pageserver.rs @@ -569,8 +569,10 @@ fn start_pageserver( pageserver::l0_flush::L0FlushGlobalState::new(conf.l0_flush.clone()); // Scan the local 'tenants/' directory and start loading the tenants - let (basebackup_prepare_sender, basebackup_prepare_receiver) = - tokio::sync::mpsc::unbounded_channel(); + let (basebackup_cache, basebackup_prepare_receiver) = BasebackupCache::new( + conf.basebackup_cache_dir(), + conf.basebackup_cache_config.clone(), + ); let deletion_queue_client = deletion_queue.new_client(); let background_purges = mgr::BackgroundPurges::default(); @@ -582,7 +584,7 @@ fn start_pageserver( remote_storage: remote_storage.clone(), deletion_queue_client, l0_flush_global_state, - basebackup_prepare_sender, + basebackup_cache: Arc::clone(&basebackup_cache), feature_resolver: feature_resolver.clone(), }, shutdown_pageserver.clone(), @@ -590,10 +592,8 @@ fn start_pageserver( let tenant_manager = Arc::new(tenant_manager); BACKGROUND_RUNTIME.block_on(mgr::init_tenant_mgr(tenant_manager.clone(), order))?; - let basebackup_cache = BasebackupCache::spawn( + basebackup_cache.spawn_background_task( BACKGROUND_RUNTIME.handle(), - conf.basebackup_cache_dir(), - conf.basebackup_cache_config.clone(), basebackup_prepare_receiver, Arc::clone(&tenant_manager), shutdown_pageserver.child_token(), @@ -806,7 +806,6 @@ fn start_pageserver( } else { None }, - basebackup_cache, ); // Spawn a Pageserver gRPC server task. It will spawn separate tasks for diff --git a/pageserver/src/metrics.rs b/pageserver/src/metrics.rs index 7929b094b4..21faceef49 100644 --- a/pageserver/src/metrics.rs +++ b/pageserver/src/metrics.rs @@ -4439,6 +4439,14 @@ pub(crate) static BASEBACKUP_CACHE_SIZE: Lazy = Lazy::new(|| { .expect("failed to define a metric") }); +pub(crate) static BASEBACKUP_CACHE_PREPARE_QUEUE_SIZE: Lazy = Lazy::new(|| { + register_uint_gauge!( + "pageserver_basebackup_cache_prepare_queue_size", + "Number of requests in the basebackup prepare channel" + ) + .expect("failed to define a metric") +}); + static PAGESERVER_CONFIG_IGNORED_ITEMS: Lazy = Lazy::new(|| { register_uint_gauge_vec!( "pageserver_config_ignored_items", diff --git a/pageserver/src/page_service.rs b/pageserver/src/page_service.rs index dd02947e5c..0287a2bdb5 100644 --- a/pageserver/src/page_service.rs +++ b/pageserver/src/page_service.rs @@ -62,7 +62,6 @@ use utils::{failpoint_support, span_record}; use crate::auth::check_permission; use crate::basebackup::{self, BasebackupError}; -use crate::basebackup_cache::BasebackupCache; use crate::config::PageServerConf; use crate::context::{ DownloadBehavior, PerfInstrumentFutureExt, RequestContext, RequestContextBuilder, @@ -137,7 +136,6 @@ pub fn spawn( perf_trace_dispatch: Option, tcp_listener: tokio::net::TcpListener, tls_config: Option>, - basebackup_cache: Arc, ) -> Listener { let cancel = CancellationToken::new(); let libpq_ctx = RequestContext::todo_child( @@ -159,7 +157,6 @@ pub fn spawn( conf.pg_auth_type, tls_config, conf.page_service_pipelining.clone(), - basebackup_cache, libpq_ctx, cancel.clone(), ) @@ -218,7 +215,6 @@ pub async fn libpq_listener_main( auth_type: AuthType, tls_config: Option>, pipelining_config: PageServicePipeliningConfig, - basebackup_cache: Arc, listener_ctx: RequestContext, listener_cancel: CancellationToken, ) -> Connections { @@ -262,7 +258,6 @@ pub async fn libpq_listener_main( auth_type, tls_config.clone(), pipelining_config.clone(), - Arc::clone(&basebackup_cache), connection_ctx, connections_cancel.child_token(), gate_guard, @@ -305,7 +300,6 @@ async fn page_service_conn_main( auth_type: AuthType, tls_config: Option>, pipelining_config: PageServicePipeliningConfig, - basebackup_cache: Arc, connection_ctx: RequestContext, cancel: CancellationToken, gate_guard: GateGuard, @@ -371,7 +365,6 @@ async fn page_service_conn_main( pipelining_config, conf.get_vectored_concurrent_io, perf_span_fields, - basebackup_cache, connection_ctx, cancel.clone(), gate_guard, @@ -425,8 +418,6 @@ struct PageServerHandler { pipelining_config: PageServicePipeliningConfig, get_vectored_concurrent_io: GetVectoredConcurrentIo, - basebackup_cache: Arc, - gate_guard: GateGuard, } @@ -912,7 +903,6 @@ impl PageServerHandler { pipelining_config: PageServicePipeliningConfig, get_vectored_concurrent_io: GetVectoredConcurrentIo, perf_span_fields: ConnectionPerfSpanFields, - basebackup_cache: Arc, connection_ctx: RequestContext, cancel: CancellationToken, gate_guard: GateGuard, @@ -926,7 +916,6 @@ impl PageServerHandler { cancel, pipelining_config, get_vectored_concurrent_io, - basebackup_cache, gate_guard, } } @@ -2626,9 +2615,7 @@ impl PageServerHandler { && lsn.is_some() && prev_lsn.is_none() { - self.basebackup_cache - .get(tenant_id, timeline_id, lsn.unwrap()) - .await + timeline.get_cached_basebackup(lsn.unwrap()).await } else { None } diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index c71655ce17..2613528143 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -80,7 +80,7 @@ use self::timeline::uninit::{TimelineCreateGuard, TimelineExclusionError, Uninit use self::timeline::{ EvictionTaskTenantState, GcCutoffs, TimelineDeleteProgress, TimelineResources, WaitLsnError, }; -use crate::basebackup_cache::BasebackupPrepareSender; +use crate::basebackup_cache::BasebackupCache; use crate::config::PageServerConf; use crate::context; use crate::context::RequestContextBuilder; @@ -162,7 +162,7 @@ pub struct TenantSharedResources { pub remote_storage: GenericRemoteStorage, pub deletion_queue_client: DeletionQueueClient, pub l0_flush_global_state: L0FlushGlobalState, - pub basebackup_prepare_sender: BasebackupPrepareSender, + pub basebackup_cache: Arc, pub feature_resolver: FeatureResolver, } @@ -331,7 +331,7 @@ pub struct TenantShard { deletion_queue_client: DeletionQueueClient, /// A channel to send async requests to prepare a basebackup for the basebackup cache. - basebackup_prepare_sender: BasebackupPrepareSender, + basebackup_cache: Arc, /// Cached logical sizes updated updated on each [`TenantShard::gather_size_inputs`]. cached_logical_sizes: tokio::sync::Mutex>, @@ -1363,7 +1363,7 @@ impl TenantShard { remote_storage, deletion_queue_client, l0_flush_global_state, - basebackup_prepare_sender, + basebackup_cache, feature_resolver, } = resources; @@ -1380,7 +1380,7 @@ impl TenantShard { remote_storage.clone(), deletion_queue_client, l0_flush_global_state, - basebackup_prepare_sender, + basebackup_cache, feature_resolver, )); @@ -4380,7 +4380,7 @@ impl TenantShard { remote_storage: GenericRemoteStorage, deletion_queue_client: DeletionQueueClient, l0_flush_global_state: L0FlushGlobalState, - basebackup_prepare_sender: BasebackupPrepareSender, + basebackup_cache: Arc, feature_resolver: FeatureResolver, ) -> TenantShard { assert!(!attached_conf.location.generation.is_none()); @@ -4485,7 +4485,7 @@ impl TenantShard { ongoing_timeline_detach: std::sync::Mutex::default(), gc_block: Default::default(), l0_flush_global_state, - basebackup_prepare_sender, + basebackup_cache, feature_resolver, } } @@ -5414,7 +5414,7 @@ impl TenantShard { pagestream_throttle_metrics: self.pagestream_throttle_metrics.clone(), l0_compaction_trigger: self.l0_compaction_trigger.clone(), l0_flush_global_state: self.l0_flush_global_state.clone(), - basebackup_prepare_sender: self.basebackup_prepare_sender.clone(), + basebackup_cache: self.basebackup_cache.clone(), feature_resolver: self.feature_resolver.clone(), } } @@ -6000,7 +6000,7 @@ pub(crate) mod harness { ) -> anyhow::Result> { let walredo_mgr = Arc::new(WalRedoManager::from(TestRedoManager)); - let (basebackup_requst_sender, _) = tokio::sync::mpsc::unbounded_channel(); + let (basebackup_cache, _) = BasebackupCache::new(Utf8PathBuf::new(), None); let tenant = Arc::new(TenantShard::new( TenantState::Attaching, @@ -6018,7 +6018,7 @@ pub(crate) mod harness { self.deletion_queue.new_client(), // TODO: ideally we should run all unit tests with both configs L0FlushGlobalState::new(L0FlushConfig::default()), - basebackup_requst_sender, + basebackup_cache, FeatureResolver::new_disabled(), )); diff --git a/pageserver/src/tenant/mgr.rs b/pageserver/src/tenant/mgr.rs index 76937dd959..0a494e7923 100644 --- a/pageserver/src/tenant/mgr.rs +++ b/pageserver/src/tenant/mgr.rs @@ -2891,14 +2891,18 @@ mod tests { use std::collections::BTreeMap; use std::sync::Arc; + use camino::Utf8PathBuf; use storage_broker::BrokerClientChannel; use tracing::Instrument; use super::super::harness::TenantHarness; use super::TenantsMap; - use crate::tenant::{ - TenantSharedResources, - mgr::{BackgroundPurges, TenantManager, TenantSlot}, + use crate::{ + basebackup_cache::BasebackupCache, + tenant::{ + TenantSharedResources, + mgr::{BackgroundPurges, TenantManager, TenantSlot}, + }, }; #[tokio::test(start_paused = true)] @@ -2924,9 +2928,7 @@ mod tests { // Invoke remove_tenant_from_memory with a cleanup hook that blocks until we manually // permit it to proceed: that will stick the tenant in InProgress - let (basebackup_prepare_sender, _) = tokio::sync::mpsc::unbounded_channel::< - crate::basebackup_cache::BasebackupPrepareRequest, - >(); + let (basebackup_cache, _) = BasebackupCache::new(Utf8PathBuf::new(), None); let tenant_manager = TenantManager { tenants: std::sync::RwLock::new(TenantsMap::Open(tenants)), @@ -2940,7 +2942,7 @@ mod tests { l0_flush_global_state: crate::l0_flush::L0FlushGlobalState::new( h.conf.l0_flush.clone(), ), - basebackup_prepare_sender, + basebackup_cache, feature_resolver: crate::feature_resolver::FeatureResolver::new_disabled(), }, cancel: tokio_util::sync::CancellationToken::new(), diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 4ca005bfd4..bec2f0ed52 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -95,12 +95,12 @@ use super::storage_layer::{LayerFringe, LayerVisibilityHint, ReadableLayer}; use super::tasks::log_compaction_error; use super::upload_queue::NotInitialized; use super::{ - AttachedTenantConf, BasebackupPrepareSender, GcError, HeatMapTimeline, MaybeOffloaded, + AttachedTenantConf, GcError, HeatMapTimeline, MaybeOffloaded, debug_assert_current_span_has_tenant_and_timeline_id, }; use crate::PERF_TRACE_TARGET; use crate::aux_file::AuxFileSizeEstimator; -use crate::basebackup_cache::BasebackupPrepareRequest; +use crate::basebackup_cache::BasebackupCache; use crate::config::PageServerConf; use crate::context::{ DownloadBehavior, PerfInstrumentFutureExt, RequestContext, RequestContextBuilder, @@ -201,7 +201,7 @@ pub struct TimelineResources { pub pagestream_throttle_metrics: Arc, pub l0_compaction_trigger: Arc, pub l0_flush_global_state: l0_flush::L0FlushGlobalState, - pub basebackup_prepare_sender: BasebackupPrepareSender, + pub basebackup_cache: Arc, pub feature_resolver: FeatureResolver, } @@ -448,7 +448,7 @@ pub struct Timeline { wait_lsn_log_slow: tokio::sync::Semaphore, /// A channel to send async requests to prepare a basebackup for the basebackup cache. - basebackup_prepare_sender: BasebackupPrepareSender, + basebackup_cache: Arc, feature_resolver: FeatureResolver, } @@ -2500,6 +2500,13 @@ impl Timeline { .unwrap_or(self.conf.default_tenant_conf.basebackup_cache_enabled) } + /// Try to get a basebackup from the on-disk cache. + pub(crate) async fn get_cached_basebackup(&self, lsn: Lsn) -> Option { + self.basebackup_cache + .get(self.tenant_shard_id.tenant_id, self.timeline_id, lsn) + .await + } + /// Prepare basebackup for the given LSN and store it in the basebackup cache. /// The method is asynchronous and returns immediately. /// The actual basebackup preparation is performed in the background @@ -2521,17 +2528,8 @@ impl Timeline { return; } - let res = self - .basebackup_prepare_sender - .send(BasebackupPrepareRequest { - tenant_shard_id: self.tenant_shard_id, - timeline_id: self.timeline_id, - lsn, - }); - if let Err(e) = res { - // May happen during shutdown, it's not critical. - info!("Failed to send shutdown checkpoint: {e:#}"); - } + self.basebackup_cache + .send_prepare(self.tenant_shard_id, self.timeline_id, lsn); } } @@ -3088,7 +3086,7 @@ impl Timeline { wait_lsn_log_slow: tokio::sync::Semaphore::new(1), - basebackup_prepare_sender: resources.basebackup_prepare_sender, + basebackup_cache: resources.basebackup_cache, feature_resolver: resources.feature_resolver, }; From 33c0d5e2f46f04dc3384095f2b162f15ed9c46b0 Mon Sep 17 00:00:00 2001 From: "Alex Chi Z." <4198311+skyzh@users.noreply.github.com> Date: Thu, 26 Jun 2025 11:49:08 -0400 Subject: [PATCH 02/20] fix(pageserver): make posthog config parsing more robust (#12356) ## Problem In our infra config, we have to split server_api_key and other fields in two files: the former one in the sops file, and the latter one in the normal config. It creates the situation that we might misconfigure some regions that it only has part of the fields available, causing storcon/pageserver refuse to start. ## Summary of changes Allow PostHog config to have part of the fields available. Parse it later. Signed-off-by: Alex Chi Z --- Cargo.lock | 1 + libs/pageserver_api/Cargo.toml | 1 + libs/pageserver_api/src/config.rs | 48 +++++++++++++++++-- pageserver/src/config.rs | 17 +++++++ pageserver/src/feature_resolver.rs | 30 +++++++----- storage_controller/src/main.rs | 12 +++-- .../src/service/feature_flag.rs | 16 ++----- 7 files changed, 95 insertions(+), 30 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 1fee728d9c..7098711bb4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4408,6 +4408,7 @@ dependencies = [ "postgres_backend", "postgres_ffi_types", "postgres_versioninfo", + "posthog_client_lite", "rand 0.8.5", "remote_storage", "reqwest", diff --git a/libs/pageserver_api/Cargo.toml b/libs/pageserver_api/Cargo.toml index a34e065788..6dc17b670b 100644 --- a/libs/pageserver_api/Cargo.toml +++ b/libs/pageserver_api/Cargo.toml @@ -19,6 +19,7 @@ byteorder.workspace = true utils.workspace = true postgres_ffi_types.workspace = true postgres_versioninfo.workspace = true +posthog_client_lite.workspace = true enum-map.workspace = true strum.workspace = true strum_macros.workspace = true diff --git a/libs/pageserver_api/src/config.rs b/libs/pageserver_api/src/config.rs index 0cfa1c8485..6489fbe9a1 100644 --- a/libs/pageserver_api/src/config.rs +++ b/libs/pageserver_api/src/config.rs @@ -4,6 +4,7 @@ use camino::Utf8PathBuf; mod tests; use const_format::formatcp; +use posthog_client_lite::PostHogClientConfig; pub const DEFAULT_PG_LISTEN_PORT: u16 = 64000; pub const DEFAULT_PG_LISTEN_ADDR: &str = formatcp!("127.0.0.1:{DEFAULT_PG_LISTEN_PORT}"); pub const DEFAULT_HTTP_LISTEN_PORT: u16 = 9898; @@ -68,15 +69,25 @@ impl Display for NodeMetadata { #[derive(Debug, Clone, PartialEq, Eq, serde::Serialize, serde::Deserialize)] pub struct PostHogConfig { /// PostHog project ID - pub project_id: String, + #[serde(default)] + #[serde(skip_serializing_if = "Option::is_none")] + pub project_id: Option, /// Server-side (private) API key - pub server_api_key: String, + #[serde(default)] + #[serde(skip_serializing_if = "Option::is_none")] + pub server_api_key: Option, /// Client-side (public) API key - pub client_api_key: String, + #[serde(default)] + #[serde(skip_serializing_if = "Option::is_none")] + pub client_api_key: Option, /// Private API URL - pub private_api_url: String, + #[serde(default)] + #[serde(skip_serializing_if = "Option::is_none")] + pub private_api_url: Option, /// Public API URL - pub public_api_url: String, + #[serde(default)] + #[serde(skip_serializing_if = "Option::is_none")] + pub public_api_url: Option, /// Refresh interval for the feature flag spec. /// The storcon will push the feature flag spec to the pageserver. If the pageserver does not receive /// the spec for `refresh_interval`, it will fetch the spec from the PostHog API. @@ -86,6 +97,33 @@ pub struct PostHogConfig { pub refresh_interval: Option, } +impl PostHogConfig { + pub fn try_into_posthog_config(self) -> Result { + let Some(project_id) = self.project_id else { + return Err("project_id is required"); + }; + let Some(server_api_key) = self.server_api_key else { + return Err("server_api_key is required"); + }; + let Some(client_api_key) = self.client_api_key else { + return Err("client_api_key is required"); + }; + let Some(private_api_url) = self.private_api_url else { + return Err("private_api_url is required"); + }; + let Some(public_api_url) = self.public_api_url else { + return Err("public_api_url is required"); + }; + Ok(PostHogClientConfig { + project_id, + server_api_key, + client_api_key, + private_api_url, + public_api_url, + }) + } +} + /// `pageserver.toml` /// /// We use serde derive with `#[serde(default)]` to generate a deserializer diff --git a/pageserver/src/config.rs b/pageserver/src/config.rs index 5b51a9617b..12e2cd99d9 100644 --- a/pageserver/src/config.rs +++ b/pageserver/src/config.rs @@ -781,4 +781,21 @@ mod tests { PageServerConf::parse_and_validate(NodeId(0), config_toml, &workdir) .expect("parse_and_validate"); } + + #[test] + fn test_config_posthog_incomplete_config_is_valid() { + let input = r#" + control_plane_api = "http://localhost:6666" + + [posthog_config] + server_api_key = "phs_AAA" + private_api_url = "https://us.posthog.com" + public_api_url = "https://us.i.posthog.com" + "#; + let config_toml = toml_edit::de::from_str::(input) + .expect("posthogconfig is valid"); + let workdir = Utf8PathBuf::from("/nonexistent"); + PageServerConf::parse_and_validate(NodeId(0), config_toml, &workdir) + .expect("parse_and_validate"); + } } diff --git a/pageserver/src/feature_resolver.rs b/pageserver/src/feature_resolver.rs index 92a9ef2880..3080b0db34 100644 --- a/pageserver/src/feature_resolver.rs +++ b/pageserver/src/feature_resolver.rs @@ -3,7 +3,7 @@ use std::{collections::HashMap, sync::Arc, time::Duration}; use arc_swap::ArcSwap; use pageserver_api::config::NodeMetadata; use posthog_client_lite::{ - CaptureEvent, FeatureResolverBackgroundLoop, PostHogClientConfig, PostHogEvaluationError, + CaptureEvent, FeatureResolverBackgroundLoop, PostHogEvaluationError, PostHogFlagFilterPropertyValue, }; use remote_storage::RemoteStorageKind; @@ -45,16 +45,24 @@ impl FeatureResolver { ) -> anyhow::Result { // DO NOT block in this function: make it return as fast as possible to avoid startup delays. if let Some(posthog_config) = &conf.posthog_config { - let inner = FeatureResolverBackgroundLoop::new( - PostHogClientConfig { - server_api_key: posthog_config.server_api_key.clone(), - client_api_key: posthog_config.client_api_key.clone(), - project_id: posthog_config.project_id.clone(), - private_api_url: posthog_config.private_api_url.clone(), - public_api_url: posthog_config.public_api_url.clone(), - }, - shutdown_pageserver, - ); + let posthog_client_config = match posthog_config.clone().try_into_posthog_config() { + Ok(config) => config, + Err(e) => { + tracing::warn!( + "invalid posthog config, skipping posthog integration: {}", + e + ); + return Ok(FeatureResolver { + inner: None, + internal_properties: None, + force_overrides_for_testing: Arc::new(ArcSwap::new(Arc::new( + HashMap::new(), + ))), + }); + } + }; + let inner = + FeatureResolverBackgroundLoop::new(posthog_client_config, shutdown_pageserver); let inner = Arc::new(inner); // The properties shared by all tenants on this pageserver. diff --git a/storage_controller/src/main.rs b/storage_controller/src/main.rs index 296a98e620..d1c2858d6f 100644 --- a/storage_controller/src/main.rs +++ b/storage_controller/src/main.rs @@ -560,9 +560,15 @@ async fn async_main() -> anyhow::Result<()> { let cancel_bg = cancel.clone(); let task = tokio::task::spawn( async move { - let feature_flag_service = FeatureFlagService::new(service, posthog_config); - let feature_flag_service = Arc::new(feature_flag_service); - feature_flag_service.run(cancel_bg).await + match FeatureFlagService::new(service, posthog_config) { + Ok(feature_flag_service) => { + let feature_flag_service = Arc::new(feature_flag_service); + feature_flag_service.run(cancel_bg).await + } + Err(e) => { + tracing::warn!("Failed to create feature flag service: {}", e); + } + }; } .instrument(tracing::info_span!("feature_flag_service")), ); diff --git a/storage_controller/src/service/feature_flag.rs b/storage_controller/src/service/feature_flag.rs index 645eb75237..f44bf046b9 100644 --- a/storage_controller/src/service/feature_flag.rs +++ b/storage_controller/src/service/feature_flag.rs @@ -3,7 +3,7 @@ use std::{sync::Arc, time::Duration}; use futures::StreamExt; use pageserver_api::config::PostHogConfig; use pageserver_client::mgmt_api; -use posthog_client_lite::{PostHogClient, PostHogClientConfig}; +use posthog_client_lite::PostHogClient; use reqwest::StatusCode; use tokio::time::MissedTickBehavior; use tokio_util::sync::CancellationToken; @@ -20,20 +20,14 @@ pub struct FeatureFlagService { const DEFAULT_POSTHOG_REFRESH_INTERVAL: Duration = Duration::from_secs(30); impl FeatureFlagService { - pub fn new(service: Arc, config: PostHogConfig) -> Self { - let client = PostHogClient::new(PostHogClientConfig { - project_id: config.project_id.clone(), - server_api_key: config.server_api_key.clone(), - client_api_key: config.client_api_key.clone(), - private_api_url: config.private_api_url.clone(), - public_api_url: config.public_api_url.clone(), - }); - Self { + pub fn new(service: Arc, config: PostHogConfig) -> Result { + let client = PostHogClient::new(config.clone().try_into_posthog_config()?); + Ok(Self { service, config, client, http_client: reqwest::Client::new(), - } + }) } async fn refresh(self: Arc, cancel: CancellationToken) -> Result<(), anyhow::Error> { From a2d2108e6af466bb3d8ec11ab4c019726e07ff5c Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Thu, 26 Jun 2025 17:52:15 +0200 Subject: [PATCH 03/20] pageserver: use base backup cache with gRPC (#12352) ## Problem gRPC base backups do not use the base backup cache. Touches https://github.com/neondatabase/neon/issues/11728. ## Summary of changes Integrate gRPC base backups with the base backup cache. Also fixes a bug where the base backup cache did not differentiate between primary/replica base backups (at least I think that's a bug?). --- pageserver/src/page_service.rs | 67 ++++++++++++++++++------------- pageserver/src/tenant/timeline.rs | 24 +++++++++++ 2 files changed, 63 insertions(+), 28 deletions(-) diff --git a/pageserver/src/page_service.rs b/pageserver/src/page_service.rs index 0287a2bdb5..c04f6e2b47 100644 --- a/pageserver/src/page_service.rs +++ b/pageserver/src/page_service.rs @@ -12,7 +12,7 @@ use std::task::{Context, Poll}; use std::time::{Duration, Instant, SystemTime}; use std::{io, str}; -use anyhow::{Context as _, anyhow, bail}; +use anyhow::{Context as _, bail}; use bytes::{Buf as _, BufMut as _, BytesMut}; use futures::future::BoxFuture; use futures::{FutureExt, Stream}; @@ -2608,18 +2608,9 @@ impl PageServerHandler { } else { let mut writer = BufWriter::new(pgb.copyout_writer()); - let cached = { - // Basebackup is cached only for this combination of parameters. - if timeline.is_basebackup_cache_enabled() - && gzip - && lsn.is_some() - && prev_lsn.is_none() - { - timeline.get_cached_basebackup(lsn.unwrap()).await - } else { - None - } - }; + let cached = timeline + .get_cached_basebackup_if_enabled(lsn, prev_lsn, full_backup, replica, gzip) + .await; if let Some(mut cached) = cached { from_cache = true; @@ -3555,21 +3546,41 @@ impl proto::PageService for GrpcPageServiceHandler { page_api::BaseBackupCompression::Gzip => Some(async_compression::Level::Fastest), }; - let result = basebackup::send_basebackup_tarball( - &mut simplex_write, - &timeline, - req.lsn, - None, - req.full, - req.replica, - gzip_level, - &ctx, - ) - .instrument(span) // propagate request span - .await; - simplex_write.shutdown().await.map_err(|err| { - BasebackupError::Server(anyhow!("simplex shutdown failed: {err}")) - })?; + // Check for a cached basebackup. + let cached = timeline + .get_cached_basebackup_if_enabled( + req.lsn, + None, + req.full, + req.replica, + gzip_level.is_some(), + ) + .await; + + let result = if let Some(mut cached) = cached { + // If we have a cached basebackup, send it. + tokio::io::copy(&mut cached, &mut simplex_write) + .await + .map(|_| ()) + .map_err(|err| BasebackupError::Client(err, "cached,copy")) + } else { + basebackup::send_basebackup_tarball( + &mut simplex_write, + &timeline, + req.lsn, + None, + req.full, + req.replica, + gzip_level, + &ctx, + ) + .instrument(span) // propagate request span + .await + }; + simplex_write + .shutdown() + .await + .map_err(|err| BasebackupError::Client(err, "simplex_write"))?; result }); diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index bec2f0ed52..04a4bb84a3 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -2507,6 +2507,30 @@ impl Timeline { .await } + /// Convenience method to attempt fetching a basebackup for the timeline if enabled and safe for + /// the given request parameters. + /// + /// TODO: consider moving this onto GrpcPageServiceHandler once the libpq handler is gone. + pub async fn get_cached_basebackup_if_enabled( + &self, + lsn: Option, + prev_lsn: Option, + full: bool, + replica: bool, + gzip: bool, + ) -> Option { + if !self.is_basebackup_cache_enabled() || !self.basebackup_cache.is_enabled() { + return None; + } + // We have to know which LSN to fetch the basebackup for. + let lsn = lsn?; + // We only cache gzipped, non-full basebackups for primary computes with automatic prev_lsn. + if prev_lsn.is_some() || full || replica || !gzip { + return None; + } + self.get_cached_basebackup(lsn).await + } + /// Prepare basebackup for the given LSN and store it in the basebackup cache. /// The method is asynchronous and returns immediately. /// The actual basebackup preparation is performed in the background From 232f2447d4717be1331a0ffa82cd172f0b65b166 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Arpad=20M=C3=BCller?= Date: Thu, 26 Jun 2025 18:29:03 +0200 Subject: [PATCH 04/20] Support pull_timeline of timelines without writes (#12028) Make the safekeeper `pull_timeline` endpoint support timelines that haven't had any writes yet. In the storcon managed sk timelines world, if a safekeeper goes down temporarily, the storcon will schedule a `pull_timeline` call. There is no guarantee however that by when the safekeeper is online again, there have been writes to the timeline yet. The `snapshot` endpoint gives an error if the timeline hasn't had writes, so we avoid calling it if `timeline_start_lsn` indicates a freshly created timeline. Fixes #11422 Part of #11670 --- safekeeper/src/pull_timeline.rs | 78 ++++++++++++------- safekeeper/src/wal_storage.rs | 46 +++++------ .../regress/test_storage_controller.py | 16 +++- 3 files changed, 80 insertions(+), 60 deletions(-) diff --git a/safekeeper/src/pull_timeline.rs b/safekeeper/src/pull_timeline.rs index 14aef1ee5e..1c9e5bade5 100644 --- a/safekeeper/src/pull_timeline.rs +++ b/safekeeper/src/pull_timeline.rs @@ -1,5 +1,6 @@ use std::cmp::min; use std::io::{self, ErrorKind}; +use std::ops::RangeInclusive; use std::sync::Arc; use anyhow::{Context, Result, anyhow, bail}; @@ -34,7 +35,7 @@ use crate::control_file::CONTROL_FILE_NAME; use crate::state::{EvictionState, TimelinePersistentState}; use crate::timeline::{Timeline, TimelineError, WalResidentTimeline}; use crate::timelines_global_map::{create_temp_timeline_dir, validate_temp_timeline}; -use crate::wal_storage::open_wal_file; +use crate::wal_storage::{open_wal_file, wal_file_paths}; use crate::{GlobalTimelines, debug_dump, wal_backup}; /// Stream tar archive of timeline to tx. @@ -95,8 +96,8 @@ pub async fn stream_snapshot( /// State needed while streaming the snapshot. pub struct SnapshotContext { - pub from_segno: XLogSegNo, // including - pub upto_segno: XLogSegNo, // including + /// The interval of segment numbers. If None, the timeline hasn't had writes yet, so only send the control file + pub from_to_segno: Option>, pub term: Term, pub last_log_term: Term, pub flush_lsn: Lsn, @@ -174,23 +175,35 @@ pub async fn stream_snapshot_resident_guts( .await?; pausable_failpoint!("sk-snapshot-after-list-pausable"); - let tli_dir = tli.get_timeline_dir(); - info!( - "sending {} segments [{:#X}-{:#X}], term={}, last_log_term={}, flush_lsn={}", - bctx.upto_segno - bctx.from_segno + 1, - bctx.from_segno, - bctx.upto_segno, - bctx.term, - bctx.last_log_term, - bctx.flush_lsn, - ); - for segno in bctx.from_segno..=bctx.upto_segno { - let (mut sf, is_partial) = open_wal_file(&tli_dir, segno, bctx.wal_seg_size).await?; - let mut wal_file_name = XLogFileName(PG_TLI, segno, bctx.wal_seg_size); - if is_partial { - wal_file_name.push_str(".partial"); + if let Some(from_to_segno) = &bctx.from_to_segno { + let tli_dir = tli.get_timeline_dir(); + info!( + "sending {} segments [{:#X}-{:#X}], term={}, last_log_term={}, flush_lsn={}", + from_to_segno.end() - from_to_segno.start() + 1, + from_to_segno.start(), + from_to_segno.end(), + bctx.term, + bctx.last_log_term, + bctx.flush_lsn, + ); + for segno in from_to_segno.clone() { + let Some((mut sf, is_partial)) = + open_wal_file(&tli_dir, segno, bctx.wal_seg_size).await? + else { + // File is not found + let (wal_file_path, _wal_file_partial_path) = + wal_file_paths(&tli_dir, segno, bctx.wal_seg_size); + tracing::warn!("couldn't find WAL segment file {wal_file_path}"); + bail!("couldn't find WAL segment file {wal_file_path}") + }; + let mut wal_file_name = XLogFileName(PG_TLI, segno, bctx.wal_seg_size); + if is_partial { + wal_file_name.push_str(".partial"); + } + ar.append_file(&wal_file_name, &mut sf).await?; } - ar.append_file(&wal_file_name, &mut sf).await?; + } else { + info!("Not including any segments into the snapshot"); } // Do the term check before ar.finish to make archive corrupted in case of @@ -338,19 +351,26 @@ impl WalResidentTimeline { // removed further than `backup_lsn`. Since we're holding shared_state // lock and setting `wal_removal_on_hold` later, it guarantees that WAL // won't be removed until we're done. + let timeline_state = shared_state.sk.state(); let from_lsn = min( - shared_state.sk.state().remote_consistent_lsn, - shared_state.sk.state().backup_lsn, + timeline_state.remote_consistent_lsn, + timeline_state.backup_lsn, + ); + let flush_lsn = shared_state.sk.flush_lsn(); + let (send_segments, msg) = if from_lsn == Lsn::INVALID { + (false, "snapshot is called on uninitialized timeline") + } else { + (true, "timeline is initialized") + }; + tracing::info!( + remote_consistent_lsn=%timeline_state.remote_consistent_lsn, + backup_lsn=%timeline_state.backup_lsn, + %flush_lsn, + "{msg}" ); - if from_lsn == Lsn::INVALID { - // this is possible if snapshot is called before handling first - // elected message - bail!("snapshot is called on uninitialized timeline"); - } let from_segno = from_lsn.segment_number(wal_seg_size); let term = shared_state.sk.state().acceptor_state.term; let last_log_term = shared_state.sk.last_log_term(); - let flush_lsn = shared_state.sk.flush_lsn(); let upto_segno = flush_lsn.segment_number(wal_seg_size); // have some limit on max number of segments as a sanity check const MAX_ALLOWED_SEGS: u64 = 1000; @@ -376,9 +396,9 @@ impl WalResidentTimeline { drop(shared_state); let tli_copy = self.wal_residence_guard().await?; + let from_to_segno = send_segments.then_some(from_segno..=upto_segno); let bctx = SnapshotContext { - from_segno, - upto_segno, + from_to_segno, term, last_log_term, flush_lsn, diff --git a/safekeeper/src/wal_storage.rs b/safekeeper/src/wal_storage.rs index da00df2dd7..33310706be 100644 --- a/safekeeper/src/wal_storage.rs +++ b/safekeeper/src/wal_storage.rs @@ -9,7 +9,7 @@ use std::cmp::{max, min}; use std::future::Future; -use std::io::{self, SeekFrom}; +use std::io::{ErrorKind, SeekFrom}; use std::pin::Pin; use anyhow::{Context, Result, bail}; @@ -794,26 +794,13 @@ impl WalReader { // Try to open local file, if we may have WAL locally if self.pos >= self.local_start_lsn { - let res = open_wal_file(&self.timeline_dir, segno, self.wal_seg_size).await; - match res { - Ok((mut file, _)) => { - file.seek(SeekFrom::Start(xlogoff as u64)).await?; - return Ok(Box::pin(file)); - } - Err(e) => { - let is_not_found = e.chain().any(|e| { - if let Some(e) = e.downcast_ref::() { - e.kind() == io::ErrorKind::NotFound - } else { - false - } - }); - if !is_not_found { - return Err(e); - } - // NotFound is expected, fall through to remote read - } - }; + let res = open_wal_file(&self.timeline_dir, segno, self.wal_seg_size).await?; + if let Some((mut file, _)) = res { + file.seek(SeekFrom::Start(xlogoff as u64)).await?; + return Ok(Box::pin(file)); + } else { + // NotFound is expected, fall through to remote read + } } // Try to open remote file, if remote reads are enabled @@ -832,26 +819,31 @@ pub(crate) async fn open_wal_file( timeline_dir: &Utf8Path, segno: XLogSegNo, wal_seg_size: usize, -) -> Result<(tokio::fs::File, bool)> { +) -> Result> { let (wal_file_path, wal_file_partial_path) = wal_file_paths(timeline_dir, segno, wal_seg_size); // First try to open the .partial file. let mut partial_path = wal_file_path.to_owned(); partial_path.set_extension("partial"); if let Ok(opened_file) = tokio::fs::File::open(&wal_file_partial_path).await { - return Ok((opened_file, true)); + return Ok(Some((opened_file, true))); } // If that failed, try it without the .partial extension. - let pf = tokio::fs::File::open(&wal_file_path) - .await + let pf_res = tokio::fs::File::open(&wal_file_path).await; + if let Err(e) = &pf_res { + if e.kind() == ErrorKind::NotFound { + return Ok(None); + } + } + let pf = pf_res .with_context(|| format!("failed to open WAL file {wal_file_path:#}")) .map_err(|e| { - warn!("{}", e); + warn!("{e}"); e })?; - Ok((pf, false)) + Ok(Some((pf, false))) } /// Helper returning full path to WAL segment file and its .partial brother. diff --git a/test_runner/regress/test_storage_controller.py b/test_runner/regress/test_storage_controller.py index 70772766d7..290ebe456b 100644 --- a/test_runner/regress/test_storage_controller.py +++ b/test_runner/regress/test_storage_controller.py @@ -4168,13 +4168,20 @@ class DeletionSubject(Enum): TENANT = "tenant" +class EmptyTimeline(Enum): + EMPTY = "empty" + NONEMPTY = "nonempty" + + @run_only_on_default_postgres("PG version is not interesting here") @pytest.mark.parametrize("restart_storcon", [RestartStorcon.RESTART, RestartStorcon.ONLINE]) @pytest.mark.parametrize("deletetion_subject", [DeletionSubject.TENANT, DeletionSubject.TIMELINE]) +@pytest.mark.parametrize("empty_timeline", [EmptyTimeline.EMPTY, EmptyTimeline.NONEMPTY]) def test_storcon_create_delete_sk_down( neon_env_builder: NeonEnvBuilder, restart_storcon: RestartStorcon, deletetion_subject: DeletionSubject, + empty_timeline: EmptyTimeline, ): """ Test that the storcon can create and delete tenants and timelines with a safekeeper being down. @@ -4226,10 +4233,11 @@ def test_storcon_create_delete_sk_down( ep.start(safekeeper_generation=1, safekeepers=[1, 2, 3]) ep.safe_psql("CREATE TABLE IF NOT EXISTS t(key int, value text)") - with env.endpoints.create("child_of_main", tenant_id=tenant_id) as ep: - # endpoint should start. - ep.start(safekeeper_generation=1, safekeepers=[1, 2, 3]) - ep.safe_psql("CREATE TABLE IF NOT EXISTS t(key int, value text)") + if empty_timeline == EmptyTimeline.NONEMPTY: + with env.endpoints.create("child_of_main", tenant_id=tenant_id) as ep: + # endpoint should start. + ep.start(safekeeper_generation=1, safekeepers=[1, 2, 3]) + ep.safe_psql("CREATE TABLE IF NOT EXISTS t(key int, value text)") env.storage_controller.assert_log_contains("writing pending op for sk id 1") env.safekeepers[0].start() From 72b3c9cd11e909cbd7ad507da97dda38cde015b0 Mon Sep 17 00:00:00 2001 From: Vlad Lazar Date: Thu, 26 Jun 2025 17:35:34 +0100 Subject: [PATCH 05/20] pageserver: fix wal receiver hang on remote client shutdown (#12348) ## Problem Druing shard splits we shut down the remote client early and allow the parent shard to keep ingesting data. While ingesting data, the wal receiver task may wait for the current flush to complete in order to apply backpressure. Notifications are delivered via `Timeline::layer_flush_done_tx`. When the remote client was being shut down the flush loop exited whithout delivering a notification. This left `Timeline::wait_flush_completion` hanging indefinitely which blocked the shutdown of the wal receiver task, and, hence, the shard split. ## Summary of Changes Deliver a final notification when the flush loop is shutting down without the timeline cancel cancellation token having fired. I tried writing a test for this, but got stuck in failpoint hell and decided it's not worth it. `test_sharding_autosplit`, which reproduces this reliably in CI, passed with the proposed fix in https://github.com/neondatabase/neon/pull/12304. Closes https://github.com/neondatabase/neon/issues/12060 --- pageserver/src/tenant/timeline.rs | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 04a4bb84a3..7261ce783d 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -4680,6 +4680,16 @@ impl Timeline { mut layer_flush_start_rx: tokio::sync::watch::Receiver<(u64, Lsn)>, ctx: &RequestContext, ) { + // Always notify waiters about the flush loop exiting since the loop might stop + // when the timeline hasn't been cancelled. + let scopeguard_rx = layer_flush_start_rx.clone(); + scopeguard::defer! { + let (flush_counter, _) = *scopeguard_rx.borrow(); + let _ = self + .layer_flush_done_tx + .send_replace((flush_counter, Err(FlushLayerError::Cancelled))); + } + // Subscribe to L0 delta layer updates, for compaction backpressure. let mut watch_l0 = match self .layers @@ -4709,9 +4719,6 @@ impl Timeline { let result = loop { if self.cancel.is_cancelled() { info!("dropping out of flush loop for timeline shutdown"); - // Note: we do not bother transmitting into [`layer_flush_done_tx`], because - // anyone waiting on that will respect self.cancel as well: they will stop - // waiting at the same time we as drop out of this loop. return; } From 10afac87e761c223f2f6c62a28b7d1717f45c544 Mon Sep 17 00:00:00 2001 From: Heikki Linnakangas Date: Thu, 26 Jun 2025 19:45:34 +0300 Subject: [PATCH 06/20] impr(ci): Remove unnecessary 'make postgres-headers' build step (#12354) The 'make postgres' step includes installation of the headers, no need to do that separately. --- .github/workflows/build-macos.yml | 5 ----- Makefile | 6 +++++- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/.github/workflows/build-macos.yml b/.github/workflows/build-macos.yml index 160c3d05bc..7fd2626332 100644 --- a/.github/workflows/build-macos.yml +++ b/.github/workflows/build-macos.yml @@ -94,11 +94,6 @@ jobs: run: | make "neon-pg-ext-${{ matrix.postgres-version }}" -j$(sysctl -n hw.ncpu) - - name: Get postgres headers ${{ matrix.postgres-version }} - if: steps.cache_pg.outputs.cache-hit != 'true' - run: | - make postgres-headers-${{ matrix.postgres-version }} -j$(sysctl -n hw.ncpu) - - name: Upload "pg_install/${{ matrix.postgres-version }}" artifact uses: actions/upload-artifact@ea165f8d65b6e75b540449e92b4886f43607fa02 # v4.6.2 with: diff --git a/Makefile b/Makefile index d39b9b68c8..a43411518d 100644 --- a/Makefile +++ b/Makefile @@ -147,7 +147,11 @@ postgres-configure-v15: $(BUILD_DIR)/v15/config.status .PHONY: postgres-configure-v14 postgres-configure-v14: $(BUILD_DIR)/v14/config.status -# Install the PostgreSQL header files into $(POSTGRES_INSTALL_DIR)//include +# Install just the PostgreSQL header files into $(POSTGRES_INSTALL_DIR)//include +# +# This is implicitly included in the 'postgres-%' rule, but this can be handy if you +# want to just install the headers without building PostgreSQL, e.g. for building +# extensions. .PHONY: postgres-headers-% postgres-headers-%: postgres-configure-% +@echo "Installing PostgreSQL $* headers" From 6fa1562b571538278c586b5a68d0b9a88a85cf57 Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov <34828390+DimasKovas@users.noreply.github.com> Date: Fri, 27 Jun 2025 13:18:18 +0400 Subject: [PATCH 07/20] pageserver: increase default max_size_entries limit for basebackup cache (#12343) ## Problem Some pageservers hit `max_size_entries` limit in staging with only ~25 MiB storage used by basebackup cache. The limit is too strict. It should be safe to relax it. - Part of https://github.com/neondatabase/cloud/issues/29353 ## Summary of changes - Increase the default `max_size_entries` from 1000 to 10000 --- libs/pageserver_api/src/config.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/pageserver_api/src/config.rs b/libs/pageserver_api/src/config.rs index 6489fbe9a1..00d6b61399 100644 --- a/libs/pageserver_api/src/config.rs +++ b/libs/pageserver_api/src/config.rs @@ -420,7 +420,7 @@ impl Default for BasebackupCacheConfig { cleanup_period: Duration::from_secs(60), max_total_size_bytes: 1024 * 1024 * 1024, // 1 GiB // max_entry_size_bytes: 16 * 1024 * 1024, // 16 MiB - max_size_entries: 1000, + max_size_entries: 10000, prepare_channel_size: 100, } } From abc1efd5a63a186cc81ba30321fd34c50f5c42f4 Mon Sep 17 00:00:00 2001 From: Conrad Ludgate Date: Fri, 27 Jun 2025 11:36:27 +0100 Subject: [PATCH 08/20] [proxy] fix connect_to_compute retry handling (#12351) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit # Problem In #12335 I moved the `authenticate` method outside of the `connect_to_compute` loop. This triggered [e2e tests to become flaky](https://github.com/neondatabase/cloud/pull/30533). This highlighted an edge case we forgot to consider with that change. When we connect to compute, the compute IP might be cached. This cache hit might however be stale. Because we can't validate the IP is associated with a specific compute-id☨, we will succeed the connect_to_compute operation and fail when it comes to password authentication☨☨. Before the change, we were invalidating the cache and triggering wake_compute if the authentication failed. Additionally, I noticed some faulty logic I introduced 1 year ago https://github.com/neondatabase/neon/pull/8141/files#diff-5491e3afe62d8c5c77178149c665603b29d88d3ec2e47fc1b3bb119a0a970afaL145-R147 ☨ We can when we roll out TLS, as the certificate common name includes the compute-id. ☨☨ Technically password authentication could pass for the wrong compute, but I think this would only happen in the very very rare event that the IP got reused **and** the compute's endpoint happened to be a branch/replica. # Solution 1. Fix the broken logic 2. Simplify cache invalidation (I don't know why it was so convoluted) 3. Add a loop around connect_to_compute + authenticate to re-introduce the wake_compute invalidation we accidentally removed. I went with this approach to try and avoid interfering with https://github.com/neondatabase/neon/compare/main...cloneable/proxy-pglb-connect-compute-split. The changes made in commit 3 will move into `handle_client_request` I suspect, --- proxy/src/cache/timed_lru.rs | 63 ++++++++------------------ proxy/src/compute/mod.rs | 4 +- proxy/src/console_redirect_proxy.rs | 2 +- proxy/src/proxy/connect_compute.rs | 2 +- proxy/src/proxy/mod.rs | 69 +++++++++++++++++++++-------- proxy/src/proxy/retry.rs | 10 ++++- proxy/src/proxy/tests/mod.rs | 59 +++++++++++++++++++++--- 7 files changed, 137 insertions(+), 72 deletions(-) diff --git a/proxy/src/cache/timed_lru.rs b/proxy/src/cache/timed_lru.rs index 7cfe5100ea..183e1ea449 100644 --- a/proxy/src/cache/timed_lru.rs +++ b/proxy/src/cache/timed_lru.rs @@ -30,7 +30,7 @@ use super::{Cache, timed_lru}; /// /// * There's an API for immediate invalidation (removal) of a cache entry; /// It's useful in case we know for sure that the entry is no longer correct. -/// See [`timed_lru::LookupInfo`] & [`timed_lru::Cached`] for more information. +/// See [`timed_lru::Cached`] for more information. /// /// * Expired entries are kept in the cache, until they are evicted by the LRU policy, /// or by a successful lookup (i.e. the entry hasn't expired yet). @@ -54,7 +54,7 @@ pub(crate) struct TimedLru { impl Cache for TimedLru { type Key = K; type Value = V; - type LookupInfo = LookupInfo; + type LookupInfo = Key; fn invalidate(&self, info: &Self::LookupInfo) { self.invalidate_raw(info); @@ -87,30 +87,24 @@ impl TimedLru { /// Drop an entry from the cache if it's outdated. #[tracing::instrument(level = "debug", fields(cache = self.name), skip_all)] - fn invalidate_raw(&self, info: &LookupInfo) { - let now = Instant::now(); - + fn invalidate_raw(&self, key: &K) { // Do costly things before taking the lock. let mut cache = self.cache.lock(); - let raw_entry = match cache.raw_entry_mut().from_key(&info.key) { + let entry = match cache.raw_entry_mut().from_key(key) { RawEntryMut::Vacant(_) => return, - RawEntryMut::Occupied(x) => x, + RawEntryMut::Occupied(x) => x.remove(), }; - - // Remove the entry if it was created prior to lookup timestamp. - let entry = raw_entry.get(); - let (created_at, expires_at) = (entry.created_at, entry.expires_at); - let should_remove = created_at <= info.created_at || expires_at <= now; - - if should_remove { - raw_entry.remove(); - } - drop(cache); // drop lock before logging + + let Entry { + created_at, + expires_at, + .. + } = entry; + debug!( - created_at = format_args!("{created_at:?}"), - expires_at = format_args!("{expires_at:?}"), - entry_removed = should_remove, + ?created_at, + ?expires_at, "processed a cache entry invalidation event" ); } @@ -211,10 +205,10 @@ impl TimedLru { } pub(crate) fn insert_unit(&self, key: K, value: V) -> (Option, Cached<&Self, ()>) { - let (created_at, old) = self.insert_raw(key.clone(), value); + let (_, old) = self.insert_raw(key.clone(), value); let cached = Cached { - token: Some((self, LookupInfo { created_at, key })), + token: Some((self, key)), value: (), }; @@ -229,28 +223,9 @@ impl TimedLru { K: Borrow + Clone, Q: Hash + Eq + ?Sized, { - self.get_raw(key, |key, entry| { - let info = LookupInfo { - created_at: entry.created_at, - key: key.clone(), - }; - - Cached { - token: Some((self, info)), - value: entry.value.clone(), - } + self.get_raw(key, |key, entry| Cached { + token: Some((self, key.clone())), + value: entry.value.clone(), }) } } - -/// Lookup information for key invalidation. -pub(crate) struct LookupInfo { - /// Time of creation of a cache [`Entry`]. - /// We use this during invalidation lookups to prevent eviction of a newer - /// entry sharing the same key (it might've been inserted by a different - /// task after we got the entry we're trying to invalidate now). - created_at: Instant, - - /// Search by this key. - key: K, -} diff --git a/proxy/src/compute/mod.rs b/proxy/src/compute/mod.rs index 7fb88e6a45..0a19090ce0 100644 --- a/proxy/src/compute/mod.rs +++ b/proxy/src/compute/mod.rs @@ -236,7 +236,7 @@ impl AuthInfo { &self, ctx: &RequestContext, compute: &mut ComputeConnection, - user_info: ComputeUserInfo, + user_info: &ComputeUserInfo, ) -> Result { // client config with stubbed connect info. // TODO(conrad): should we rewrite this to bypass tokio-postgres2 entirely, @@ -272,7 +272,7 @@ impl AuthInfo { secret_key, }, compute.hostname.to_string(), - user_info, + user_info.clone(), ); Ok(PostgresSettings { diff --git a/proxy/src/console_redirect_proxy.rs b/proxy/src/console_redirect_proxy.rs index 112465a89b..d5903286a0 100644 --- a/proxy/src/console_redirect_proxy.rs +++ b/proxy/src/console_redirect_proxy.rs @@ -226,7 +226,7 @@ pub(crate) async fn handle_client( .await?; let pg_settings = auth_info - .authenticate(ctx, &mut node, user_info) + .authenticate(ctx, &mut node, &user_info) .or_else(|e| async { Err(stream.throw_error(e, Some(ctx)).await) }) .await?; diff --git a/proxy/src/proxy/connect_compute.rs b/proxy/src/proxy/connect_compute.rs index aa675a439e..9f642f52ab 100644 --- a/proxy/src/proxy/connect_compute.rs +++ b/proxy/src/proxy/connect_compute.rs @@ -112,7 +112,7 @@ where let node_info = if !node_info.cached() || !err.should_retry_wake_compute() { // If we just recieved this from cplane and didn't get it from cache, we shouldn't retry. // Do not need to retrieve a new node_info, just return the old one. - if should_retry(&err, num_retries, compute.retry) { + if !should_retry(&err, num_retries, compute.retry) { Metrics::get().proxy.retries_metric.observe( RetriesMetricGroup { outcome: ConnectOutcome::Failed, diff --git a/proxy/src/proxy/mod.rs b/proxy/src/proxy/mod.rs index 6b84e47982..d9c0585efb 100644 --- a/proxy/src/proxy/mod.rs +++ b/proxy/src/proxy/mod.rs @@ -18,9 +18,11 @@ use tokio::io::{AsyncRead, AsyncWrite}; use tokio_util::sync::CancellationToken; use tracing::{Instrument, debug, error, info, warn}; +use crate::cache::Cache; use crate::cancellation::{self, CancellationHandler}; use crate::config::{ProxyConfig, ProxyProtocolV2, TlsConfig}; use crate::context::RequestContext; +use crate::control_plane::client::ControlPlaneClient; use crate::error::{ReportableError, UserFacingError}; use crate::metrics::{Metrics, NumClientConnectionsGuard}; pub use crate::pglb::copy_bidirectional::{ErrorSource, copy_bidirectional_client_compute}; @@ -29,6 +31,7 @@ use crate::pglb::passthrough::ProxyPassthrough; use crate::pqproto::{BeMessage, CancelKeyData, StartupMessageParams}; use crate::protocol2::{ConnectHeader, ConnectionInfo, ConnectionInfoExtra, read_proxy_protocol}; use crate::proxy::connect_compute::{TcpMechanism, connect_to_compute}; +use crate::proxy::retry::ShouldRetryWakeCompute; use crate::rate_limiter::EndpointRateLimiter; use crate::stream::{PqStream, Stream}; use crate::types::EndpointCacheKey; @@ -349,26 +352,56 @@ pub(crate) async fn handle_client( let mut auth_info = compute::AuthInfo::with_auth_keys(creds.keys); auth_info.set_startup_params(¶ms, params_compat); - let res = connect_to_compute( - ctx, - &TcpMechanism { - locks: &config.connect_compute_locks, - }, - &auth::Backend::ControlPlane(cplane, creds.info.clone()), - config.wake_compute_retry_config, - &config.connect_to_compute, - ) - .await; - - let mut node = match res { - Ok(node) => node, - Err(e) => Err(stream.throw_error(e, Some(ctx)).await)?, + let mut node; + let mut attempt = 0; + let connect = TcpMechanism { + locks: &config.connect_compute_locks, }; + let backend = auth::Backend::ControlPlane(cplane, creds.info); - let pg_settings = auth_info.authenticate(ctx, &mut node, creds.info).await; - let pg_settings = match pg_settings { - Ok(pg_settings) => pg_settings, - Err(e) => Err(stream.throw_error(e, Some(ctx)).await)?, + // NOTE: This is messy, but should hopefully be detangled with PGLB. + // We wanted to separate the concerns of **connect** to compute (a PGLB operation), + // from **authenticate** to compute (a NeonKeeper operation). + // + // This unfortunately removed retry handling for one error case where + // the compute was cached, and we connected, but the compute cache was actually stale + // and is associated with the wrong endpoint. We detect this when the **authentication** fails. + // As such, we retry once here if the `authenticate` function fails and the error is valid to retry. + let pg_settings = loop { + attempt += 1; + + let res = connect_to_compute( + ctx, + &connect, + &backend, + config.wake_compute_retry_config, + &config.connect_to_compute, + ) + .await; + + match res { + Ok(n) => node = n, + Err(e) => return Err(stream.throw_error(e, Some(ctx)).await)?, + } + + let auth::Backend::ControlPlane(cplane, user_info) = &backend else { + unreachable!("ensured above"); + }; + + let res = auth_info.authenticate(ctx, &mut node, user_info).await; + match res { + Ok(pg_settings) => break pg_settings, + Err(e) if attempt < 2 && e.should_retry_wake_compute() => { + tracing::warn!(error = ?e, "retrying wake compute"); + + #[allow(irrefutable_let_patterns)] + if let ControlPlaneClient::ProxyV1(cplane_proxy_v1) = &**cplane { + let key = user_info.endpoint_cache_key(); + cplane_proxy_v1.caches.node_info.invalidate(&key); + } + } + Err(e) => Err(stream.throw_error(e, Some(ctx)).await)?, + } }; let session = cancellation_handler.get_key(); diff --git a/proxy/src/proxy/retry.rs b/proxy/src/proxy/retry.rs index e9eca95724..b06c3be72c 100644 --- a/proxy/src/proxy/retry.rs +++ b/proxy/src/proxy/retry.rs @@ -3,7 +3,7 @@ use std::io; use tokio::time; -use crate::compute; +use crate::compute::{self, PostgresError}; use crate::config::RetryConfig; pub(crate) trait CouldRetry { @@ -115,6 +115,14 @@ impl ShouldRetryWakeCompute for compute::ConnectionError { } } +impl ShouldRetryWakeCompute for PostgresError { + fn should_retry_wake_compute(&self) -> bool { + match self { + PostgresError::Postgres(error) => error.should_retry_wake_compute(), + } + } +} + pub(crate) fn retry_after(num_retries: u32, config: RetryConfig) -> time::Duration { config .base_delay diff --git a/proxy/src/proxy/tests/mod.rs b/proxy/src/proxy/tests/mod.rs index 29a269208a..4f27496019 100644 --- a/proxy/src/proxy/tests/mod.rs +++ b/proxy/src/proxy/tests/mod.rs @@ -374,6 +374,7 @@ fn connect_compute_total_wait() { #[derive(Clone, Copy, Debug)] enum ConnectAction { Wake, + WakeCold, WakeFail, WakeRetry, Connect, @@ -504,6 +505,9 @@ impl TestControlPlaneClient for TestConnectMechanism { *counter += 1; match action { ConnectAction::Wake => Ok(helper_create_cached_node_info(self.cache)), + ConnectAction::WakeCold => Ok(CachedNodeInfo::new_uncached( + helper_create_uncached_node_info(), + )), ConnectAction::WakeFail => { let err = control_plane::errors::ControlPlaneError::Message(Box::new( ControlPlaneErrorMessage { @@ -551,8 +555,8 @@ impl TestControlPlaneClient for TestConnectMechanism { } } -fn helper_create_cached_node_info(cache: &'static NodeInfoCache) -> CachedNodeInfo { - let node = NodeInfo { +fn helper_create_uncached_node_info() -> NodeInfo { + NodeInfo { conn_info: compute::ConnectInfo { host: "test".into(), port: 5432, @@ -566,7 +570,11 @@ fn helper_create_cached_node_info(cache: &'static NodeInfoCache) -> CachedNodeIn compute_id: "compute".into(), cold_start_info: crate::control_plane::messages::ColdStartInfo::Warm, }, - }; + } +} + +fn helper_create_cached_node_info(cache: &'static NodeInfoCache) -> CachedNodeInfo { + let node = helper_create_uncached_node_info(); let (_, node2) = cache.insert_unit("key".into(), Ok(node.clone())); node2.map(|()| node) } @@ -742,7 +750,7 @@ async fn fail_no_wake_skips_cache_invalidation() { let ctx = RequestContext::test(); let mech = TestConnectMechanism::new(vec![ ConnectAction::Wake, - ConnectAction::FailNoWake, + ConnectAction::RetryNoWake, ConnectAction::Connect, ]); let user = helper_create_connect_info(&mech); @@ -788,7 +796,7 @@ async fn retry_no_wake_skips_invalidation() { let ctx = RequestContext::test(); // Wake → RetryNoWake (retryable + NOT wakeable) - let mechanism = TestConnectMechanism::new(vec![Wake, RetryNoWake]); + let mechanism = TestConnectMechanism::new(vec![Wake, RetryNoWake, Fail]); let user_info = helper_create_connect_info(&mechanism); let cfg = config(); @@ -802,3 +810,44 @@ async fn retry_no_wake_skips_invalidation() { "invalidating stalled compute node info cache entry" )); } + +#[tokio::test] +#[traced_test] +async fn retry_no_wake_error_fast() { + let _ = env_logger::try_init(); + use ConnectAction::*; + + let ctx = RequestContext::test(); + // Wake → FailNoWake (not retryable + NOT wakeable) + let mechanism = TestConnectMechanism::new(vec![Wake, FailNoWake]); + let user_info = helper_create_connect_info(&mechanism); + let cfg = config(); + + connect_to_compute(&ctx, &mechanism, &user_info, cfg.retry, &cfg) + .await + .unwrap_err(); + mechanism.verify(); + + // Because FailNoWake has wakeable=false, we must NOT see invalidate_cache + assert!(!logs_contain( + "invalidating stalled compute node info cache entry" + )); +} + +#[tokio::test] +#[traced_test] +async fn retry_cold_wake_skips_invalidation() { + let _ = env_logger::try_init(); + use ConnectAction::*; + + let ctx = RequestContext::test(); + // WakeCold → FailNoWake (not retryable + NOT wakeable) + let mechanism = TestConnectMechanism::new(vec![WakeCold, Retry, Connect]); + let user_info = helper_create_connect_info(&mechanism); + let cfg = config(); + + connect_to_compute(&ctx, &mechanism, &user_info, cfg.retry, &cfg) + .await + .unwrap(); + mechanism.verify(); +} From ebc12a388c74911d2a67805e00af9348ce0b5e1d Mon Sep 17 00:00:00 2001 From: Mikhail Date: Fri, 27 Jun 2025 12:06:27 +0100 Subject: [PATCH 09/20] fix: endpoint_storage_addr as String (#12359) It's not a SocketAddr as we use k8s DNS https://github.com/neondatabase/cloud/issues/19011 --- compute_tools/src/compute.rs | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/compute_tools/src/compute.rs b/compute_tools/src/compute.rs index 70b2d28bf2..cf558ee01a 100644 --- a/compute_tools/src/compute.rs +++ b/compute_tools/src/compute.rs @@ -20,7 +20,6 @@ use postgres::NoTls; use postgres::error::SqlState; use remote_storage::{DownloadError, RemotePath}; use std::collections::{HashMap, HashSet}; -use std::net::SocketAddr; use std::os::unix::fs::{PermissionsExt, symlink}; use std::path::Path; use std::process::{Command, Stdio}; @@ -218,7 +217,8 @@ pub struct ParsedSpec { pub pageserver_connstr: String, pub safekeeper_connstrings: Vec, pub storage_auth_token: Option, - pub endpoint_storage_addr: Option, + /// k8s dns name and port + pub endpoint_storage_addr: Option, pub endpoint_storage_token: Option, } @@ -313,13 +313,10 @@ impl TryFrom for ParsedSpec { .or(Err("invalid timeline id"))? }; - let endpoint_storage_addr: Option = spec + let endpoint_storage_addr: Option = spec .endpoint_storage_addr .clone() - .or_else(|| spec.cluster.settings.find("neon.endpoint_storage_addr")) - .unwrap_or_default() - .parse() - .ok(); + .or_else(|| spec.cluster.settings.find("neon.endpoint_storage_addr")); let endpoint_storage_token = spec .endpoint_storage_token .clone() From ebb6e26a64b368f865bed420de274d23d46e0caa Mon Sep 17 00:00:00 2001 From: Vlad Lazar Date: Fri, 27 Jun 2025 13:46:18 +0100 Subject: [PATCH 10/20] pageserver: handle multiple attached children in shard resolution (#12336) ## Problem When resolving a shard during a split we might have multiple attached shards with the old shard count (i.e. not all of them are marked in progress and ignored). Hence, we can compute the desired shard number based on the old shard count and misroute the request. ## Summary of Changes Recompute the desired shard every time the shard count changes during the iteration --- pageserver/src/tenant/mgr.rs | 23 ++++++++++++++++------- 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/pageserver/src/tenant/mgr.rs b/pageserver/src/tenant/mgr.rs index 0a494e7923..248d92622e 100644 --- a/pageserver/src/tenant/mgr.rs +++ b/pageserver/src/tenant/mgr.rs @@ -2200,7 +2200,7 @@ impl TenantManager { selector: ShardSelector, ) -> ShardResolveResult { let tenants = self.tenants.read().unwrap(); - let mut want_shard = None; + let mut want_shard: Option = None; let mut any_in_progress = None; match &*tenants { @@ -2225,14 +2225,23 @@ impl TenantManager { return ShardResolveResult::Found(tenant.clone()); } ShardSelector::Page(key) => { - // First slot we see for this tenant, calculate the expected shard number - // for the key: we will use this for checking if this and subsequent - // slots contain the key, rather than recalculating the hash each time. - if want_shard.is_none() { - want_shard = Some(tenant.shard_identity.get_shard_number(&key)); + // Each time we find an attached slot with a different shard count, + // recompute the expected shard number: during shard splits we might + // have multiple shards with the old shard count. + if want_shard.is_none() + || want_shard.unwrap().shard_count != tenant.shard_identity.count + { + want_shard = Some(ShardIndex { + shard_number: tenant.shard_identity.get_shard_number(&key), + shard_count: tenant.shard_identity.count, + }); } - if Some(tenant.shard_identity.number) == want_shard { + if Some(ShardIndex { + shard_number: tenant.shard_identity.number, + shard_count: tenant.shard_identity.count, + }) == want_shard + { return ShardResolveResult::Found(tenant.clone()); } } From cc1664ef93171774b5b1d127ab9f5a978cf62f99 Mon Sep 17 00:00:00 2001 From: Vlad Lazar Date: Fri, 27 Jun 2025 14:13:11 +0100 Subject: [PATCH 11/20] pageserver: allow flush task cancelled error in sharding autosplit test (#12374) ## Problem Test is failing due to compaction shutdown noise (see https://github.com/neondatabase/neon/issues/12162). ## Summary of changes Allow list the noise. --- test_runner/performance/test_sharding_autosplit.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/test_runner/performance/test_sharding_autosplit.py b/test_runner/performance/test_sharding_autosplit.py index 04bebae92f..0bb210db23 100644 --- a/test_runner/performance/test_sharding_autosplit.py +++ b/test_runner/performance/test_sharding_autosplit.py @@ -62,7 +62,8 @@ def test_sharding_autosplit(neon_env_builder: NeonEnvBuilder, pg_bin: PgBin): ps.allowed_errors.extend( [ # We shut down pageservers while they might have some compaction work going on - ".*Compaction failed.*shutting down.*" + ".*Compaction failed.*shutting down.*", + ".*flush task cancelled.*", ] ) From 6f4198c78a93958ee02301b26ada27c9e91be494 Mon Sep 17 00:00:00 2001 From: Peter Bendel Date: Fri, 27 Jun 2025 15:49:26 +0200 Subject: [PATCH 12/20] treat strategy flag test_maintenance as boolean data type (#12373) ## Problem In large oltp test run https://github.com/neondatabase/neon/actions/runs/15905488707/job/44859116742 we see that the `Benchmark database maintenance` step is skipped in all 3 strategy variants, however it should be executed in two. This is due to treating the `test_maintenance` boolean type in the strategy in the condition of the `Benchmark database maintenance` step ## Summary of changes Use a boolean condition instead of a string comparison ## Test run from this pull request branch https://github.com/neondatabase/neon/actions/runs/15923605412 --- .github/workflows/large_oltp_benchmark.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/large_oltp_benchmark.yml b/.github/workflows/large_oltp_benchmark.yml index 050b9047c7..4f6858dcae 100644 --- a/.github/workflows/large_oltp_benchmark.yml +++ b/.github/workflows/large_oltp_benchmark.yml @@ -153,7 +153,7 @@ jobs: PERF_TEST_RESULT_CONNSTR: "${{ secrets.PERF_TEST_RESULT_CONNSTR }}" - name: Benchmark database maintenance - if: ${{ matrix.test_maintenance == 'true' }} + if: ${{ matrix.test_maintenance }} uses: ./.github/actions/run-python-test-set with: build_type: ${{ env.BUILD_TYPE }} From 37e181af8ae27963c02f6312e23362c395a84404 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Arpad=20M=C3=BCller?= Date: Fri, 27 Jun 2025 15:51:59 +0200 Subject: [PATCH 13/20] Update rust to 1.88.0 (#12364) We keep the practice of keeping the compiler up to date, pointing to the latest release. This is done by many other projects in the Rust ecosystem as well. [Announcement blog post](https://blog.rust-lang.org/2025/06/26/Rust-1.88.0/) Prior update was in https://github.com/neondatabase/neon/pull/11938 --- build-tools.Dockerfile | 4 ++-- pageserver/src/bin/test_helper_slow_client_reads.rs | 2 +- pageserver/src/tenant.rs | 4 ++-- rust-toolchain.toml | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/build-tools.Dockerfile b/build-tools.Dockerfile index f97f04968e..b70ced7886 100644 --- a/build-tools.Dockerfile +++ b/build-tools.Dockerfile @@ -179,7 +179,7 @@ RUN curl -sL "https://github.com/peak/s5cmd/releases/download/v${S5CMD_VERSION}/ && mv s5cmd /usr/local/bin/s5cmd # LLVM -ENV LLVM_VERSION=19 +ENV LLVM_VERSION=20 RUN curl -fsSL 'https://apt.llvm.org/llvm-snapshot.gpg.key' | apt-key add - \ && echo "deb http://apt.llvm.org/${DEBIAN_VERSION}/ llvm-toolchain-${DEBIAN_VERSION}-${LLVM_VERSION} main" > /etc/apt/sources.list.d/llvm.stable.list \ && apt update \ @@ -292,7 +292,7 @@ WORKDIR /home/nonroot # Rust # Please keep the version of llvm (installed above) in sync with rust llvm (`rustc --version --verbose | grep LLVM`) -ENV RUSTC_VERSION=1.87.0 +ENV RUSTC_VERSION=1.88.0 ENV RUSTUP_HOME="/home/nonroot/.rustup" ENV PATH="/home/nonroot/.cargo/bin:${PATH}" ARG RUSTFILT_VERSION=0.2.1 diff --git a/pageserver/src/bin/test_helper_slow_client_reads.rs b/pageserver/src/bin/test_helper_slow_client_reads.rs index be8e081945..8b641abee7 100644 --- a/pageserver/src/bin/test_helper_slow_client_reads.rs +++ b/pageserver/src/bin/test_helper_slow_client_reads.rs @@ -37,7 +37,7 @@ async fn main() -> anyhow::Result<()> { not_modified_since: Lsn(23), }, batch_key: 42, - message: format!("message {}", msg), + message: format!("message {msg}"), })); let Ok(res) = tokio::time::timeout(Duration::from_secs(10), fut).await else { eprintln!("pipe seems full"); diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index 2613528143..2e9dbdc539 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -11429,11 +11429,11 @@ mod tests { if left != right { eprintln!("---LEFT---"); for left in left.iter() { - eprintln!("{}", left); + eprintln!("{left}"); } eprintln!("---RIGHT---"); for right in right.iter() { - eprintln!("{}", right); + eprintln!("{right}"); } assert_eq!(left, right); } diff --git a/rust-toolchain.toml b/rust-toolchain.toml index c48def3483..d20b46e755 100644 --- a/rust-toolchain.toml +++ b/rust-toolchain.toml @@ -1,5 +1,5 @@ [toolchain] -channel = "1.87.0" +channel = "1.88.0" profile = "default" # The default profile includes rustc, rust-std, cargo, rust-docs, rustfmt and clippy. # https://rust-lang.github.io/rustup/concepts/profiles.html From 5a82182c48ffc7d7c50ac44e33d8d0e7353dc65a Mon Sep 17 00:00:00 2001 From: Heikki Linnakangas Date: Fri, 27 Jun 2025 17:49:52 +0300 Subject: [PATCH 14/20] impr(ci): Refactor postgres Makefile targets to a separate makefile (#12363) Mainly for general readability. Some notable changes: - Postgres can be built without the rest of the repository, and in particular without any of the Rust bits. Some CI scripts took advantage of that, so let's make that more explicit by separating those parts. Also add an explicit comment about that in the new postgres.mk file. - Add a new PG_INSTALL_CACHED variable. If it's set, `make all` and other top-Makefile targets skip checking if Postgres is up-to-date. This is also to be used in CI scripts that build and cache Postgres as separate steps. (It is currently only used in the macos walproposer-lib rule, but stay tuned for more.) - Introduce a POSTGRES_VERSIONS variable that lists all supported PostgreSQL versions. Refactor a few Makefile rules to use that. --- .dockerignore | 1 + .github/workflows/build-macos.yml | 8 +- Dockerfile | 1 + Makefile | 133 +++++++----------------------- postgres.mk | 121 +++++++++++++++++++++++++++ 5 files changed, 158 insertions(+), 106 deletions(-) create mode 100644 postgres.mk diff --git a/.dockerignore b/.dockerignore index 4bf1492ea3..4d9433764e 100644 --- a/.dockerignore +++ b/.dockerignore @@ -4,6 +4,7 @@ !Cargo.lock !Cargo.toml !Makefile +!postgres.mk !rust-toolchain.toml !scripts/ninstall.sh !docker-compose/run-tests.sh diff --git a/.github/workflows/build-macos.yml b/.github/workflows/build-macos.yml index 7fd2626332..7b2c9c2ce3 100644 --- a/.github/workflows/build-macos.yml +++ b/.github/workflows/build-macos.yml @@ -135,6 +135,12 @@ jobs: name: pg_install--v17 path: pg_install/v17 + # `actions/download-artifact` doesn't preserve permissions: + # https://github.com/actions/download-artifact?tab=readme-ov-file#permission-loss + - name: Make pg_install/v*/bin/* executable + run: | + chmod +x pg_install/v*/bin/* + - name: Cache walproposer-lib id: cache_walproposer_lib uses: actions/cache@5a3ec84eff668545956fd18022155c47e93e2684 # v4.2.3 @@ -162,7 +168,7 @@ jobs: - name: Build walproposer-lib (only for v17) if: steps.cache_walproposer_lib.outputs.cache-hit != 'true' run: - make walproposer-lib -j$(sysctl -n hw.ncpu) + make walproposer-lib -j$(sysctl -n hw.ncpu) PG_INSTALL_CACHED=1 - name: Upload "build/walproposer-lib" artifact uses: actions/upload-artifact@ea165f8d65b6e75b540449e92b4886f43607fa02 # v4.6.2 diff --git a/Dockerfile b/Dockerfile index 69657067de..d518370ab8 100644 --- a/Dockerfile +++ b/Dockerfile @@ -40,6 +40,7 @@ COPY --chown=nonroot vendor/postgres-v16 vendor/postgres-v16 COPY --chown=nonroot vendor/postgres-v17 vendor/postgres-v17 COPY --chown=nonroot pgxn pgxn COPY --chown=nonroot Makefile Makefile +COPY --chown=nonroot postgres.mk postgres.mk COPY --chown=nonroot scripts/ninstall.sh scripts/ninstall.sh ENV BUILD_TYPE=release diff --git a/Makefile b/Makefile index a43411518d..7f8f436a2e 100644 --- a/Makefile +++ b/Makefile @@ -4,11 +4,14 @@ ROOT_PROJECT_DIR := $(dir $(abspath $(lastword $(MAKEFILE_LIST)))) # managers. POSTGRES_INSTALL_DIR ?= $(ROOT_PROJECT_DIR)/pg_install/ +# Supported PostgreSQL versions +POSTGRES_VERSIONS = v17 v16 v15 v14 + # CARGO_BUILD_FLAGS: Extra flags to pass to `cargo build`. `--locked` # and `--features testing` are popular examples. # -# CARGO_PROFILE: You can also set to override the cargo profile to -# use. By default, it is derived from BUILD_TYPE. +# CARGO_PROFILE: Set to override the cargo profile to use. By default, +# it is derived from BUILD_TYPE. # All intermediate build artifacts are stored here. BUILD_DIR := build @@ -95,95 +98,24 @@ CACHEDIR_TAG_CONTENTS := "Signature: 8a477f597d28d172789f06886806bc55" # Top level Makefile to build Neon and PostgreSQL # .PHONY: all -all: neon postgres neon-pg-ext +all: neon postgres-install neon-pg-ext ### Neon Rust bits # # The 'postgres_ffi' depends on the Postgres headers. .PHONY: neon -neon: postgres-headers walproposer-lib cargo-target-dir +neon: postgres-headers-install walproposer-lib cargo-target-dir +@echo "Compiling Neon" $(CARGO_CMD_PREFIX) cargo build $(CARGO_BUILD_FLAGS) $(CARGO_PROFILE) + .PHONY: cargo-target-dir cargo-target-dir: # https://github.com/rust-lang/cargo/issues/14281 mkdir -p target test -e target/CACHEDIR.TAG || echo "$(CACHEDIR_TAG_CONTENTS)" > target/CACHEDIR.TAG -### PostgreSQL parts -# Some rules are duplicated for Postgres v14 and 15. We may want to refactor -# to avoid the duplication in the future, but it's tolerable for now. -# -$(BUILD_DIR)/%/config.status: - mkdir -p $(BUILD_DIR) - test -e $(BUILD_DIR)/CACHEDIR.TAG || echo "$(CACHEDIR_TAG_CONTENTS)" > $(BUILD_DIR)/CACHEDIR.TAG - - +@echo "Configuring Postgres $* build" - @test -s $(ROOT_PROJECT_DIR)/vendor/postgres-$*/configure || { \ - echo "\nPostgres submodule not found in $(ROOT_PROJECT_DIR)/vendor/postgres-$*/, execute "; \ - echo "'git submodule update --init --recursive --depth 2 --progress .' in project root.\n"; \ - exit 1; } - mkdir -p $(BUILD_DIR)/$* - - VERSION=$*; \ - EXTRA_VERSION=$$(cd $(ROOT_PROJECT_DIR)/vendor/postgres-$$VERSION && git rev-parse HEAD); \ - (cd $(BUILD_DIR)/$$VERSION && \ - env PATH="$(EXTRA_PATH_OVERRIDES):$$PATH" $(ROOT_PROJECT_DIR)/vendor/postgres-$$VERSION/configure \ - CFLAGS='$(PG_CFLAGS)' LDFLAGS='$(PG_LDFLAGS)' \ - $(PG_CONFIGURE_OPTS) --with-extra-version=" ($$EXTRA_VERSION)" \ - --prefix=$(abspath $(POSTGRES_INSTALL_DIR))/$$VERSION > configure.log) - -# nicer alias to run 'configure' -# Note: I've been unable to use templates for this part of our configuration. -# I'm not sure why it wouldn't work, but this is the only place (apart from -# the "build-all-versions" entry points) where direct mention of PostgreSQL -# versions is used. -.PHONY: postgres-configure-v17 -postgres-configure-v17: $(BUILD_DIR)/v17/config.status -.PHONY: postgres-configure-v16 -postgres-configure-v16: $(BUILD_DIR)/v16/config.status -.PHONY: postgres-configure-v15 -postgres-configure-v15: $(BUILD_DIR)/v15/config.status -.PHONY: postgres-configure-v14 -postgres-configure-v14: $(BUILD_DIR)/v14/config.status - -# Install just the PostgreSQL header files into $(POSTGRES_INSTALL_DIR)//include -# -# This is implicitly included in the 'postgres-%' rule, but this can be handy if you -# want to just install the headers without building PostgreSQL, e.g. for building -# extensions. -.PHONY: postgres-headers-% -postgres-headers-%: postgres-configure-% - +@echo "Installing PostgreSQL $* headers" - $(MAKE) -C $(BUILD_DIR)/$*/src/include MAKELEVEL=0 install - -# Compile and install PostgreSQL -.PHONY: postgres-% -postgres-%: postgres-configure-% \ - postgres-headers-% # to prevent `make install` conflicts with neon's `postgres-headers` - +@echo "Compiling PostgreSQL $*" - $(MAKE) -C $(BUILD_DIR)/$* MAKELEVEL=0 install - +@echo "Compiling pg_prewarm $*" - $(MAKE) -C $(BUILD_DIR)/$*/contrib/pg_prewarm install - +@echo "Compiling pg_buffercache $*" - $(MAKE) -C $(BUILD_DIR)/$*/contrib/pg_buffercache install - +@echo "Compiling pg_visibility $*" - $(MAKE) -C $(BUILD_DIR)/$*/contrib/pg_visibility install - +@echo "Compiling pageinspect $*" - $(MAKE) -C $(BUILD_DIR)/$*/contrib/pageinspect install - +@echo "Compiling pg_trgm $*" - $(MAKE) -C $(BUILD_DIR)/$*/contrib/pg_trgm install - +@echo "Compiling amcheck $*" - $(MAKE) -C $(BUILD_DIR)/$*/contrib/amcheck install - +@echo "Compiling test_decoding $*" - $(MAKE) -C $(BUILD_DIR)/$*/contrib/test_decoding install - -.PHONY: postgres-check-% -postgres-check-%: postgres-% - $(MAKE) -C $(BUILD_DIR)/$* MAKELEVEL=0 check - .PHONY: neon-pg-ext-% -neon-pg-ext-%: postgres-% +neon-pg-ext-%: postgres-install-% +@echo "Compiling neon-specific Postgres extensions for $*" mkdir -p $(BUILD_DIR)/pgxn-$* $(MAKE) PG_CONFIG=$(POSTGRES_INSTALL_DIR)/$*/bin/pg_config COPT='$(COPT)' \ @@ -222,39 +154,14 @@ ifeq ($(UNAME_S),Linux) pg_crc32c.o endif +# Shorthand to call neon-pg-ext-% target for all Postgres versions .PHONY: neon-pg-ext -neon-pg-ext: \ - neon-pg-ext-v14 \ - neon-pg-ext-v15 \ - neon-pg-ext-v16 \ - neon-pg-ext-v17 - -# shorthand to build all Postgres versions -.PHONY: postgres -postgres: \ - postgres-v14 \ - postgres-v15 \ - postgres-v16 \ - postgres-v17 - -.PHONY: postgres-headers -postgres-headers: \ - postgres-headers-v14 \ - postgres-headers-v15 \ - postgres-headers-v16 \ - postgres-headers-v17 - -.PHONY: postgres-check -postgres-check: \ - postgres-check-v14 \ - postgres-check-v15 \ - postgres-check-v16 \ - postgres-check-v17 +neon-pg-ext: $(foreach pg_version,$(POSTGRES_VERSIONS),neon-pg-ext-$(pg_version)) # This removes everything .PHONY: distclean distclean: - $(RM) -r $(POSTGRES_INSTALL_DIR) + $(RM) -r $(POSTGRES_INSTALL_DIR) $(BUILD_DIR) $(CARGO_CMD_PREFIX) cargo clean .PHONY: fmt @@ -302,3 +209,19 @@ neon-pgindent: postgres-v17-pg-bsd-indent neon-pg-ext-v17 .PHONY: setup-pre-commit-hook setup-pre-commit-hook: ln -s -f $(ROOT_PROJECT_DIR)/pre-commit.py .git/hooks/pre-commit + +# Targets for building PostgreSQL are defined in postgres.mk. +# +# But if the caller has indicated that PostgreSQL is already +# installed, by setting the PG_INSTALL_CACHED variable, skip it. +ifdef PG_INSTALL_CACHED +postgres-install: skip-install +$(foreach pg_version,$(POSTGRES_VERSIONS),postgres-install-$(pg_version)): skip-install +postgres-headers-install: + +@echo "Skipping installation of PostgreSQL headers because PG_INSTALL_CACHED is set" +skip-install: + +@echo "Skipping PostgreSQL installation because PG_INSTALL_CACHED is set" + +else +include postgres.mk +endif diff --git a/postgres.mk b/postgres.mk new file mode 100644 index 0000000000..eff882d1ff --- /dev/null +++ b/postgres.mk @@ -0,0 +1,121 @@ +# Sub-makefile for compiling PostgreSQL as part of Neon. This is +# included from the main Makefile, and is not meant to be called +# directly. +# +# CI workflows and Dockerfiles can take advantage of the following +# properties for caching: +# +# - Compiling the targets in this file only builds the PostgreSQL sources +# under the vendor/ subdirectory, nothing else from the repository. +# - All outputs go to POSTGRES_INSTALL_DIR (by default 'pg_install', +# see parent Makefile) +# - intermediate build artifacts go to BUILD_DIR +# +# +# Variables passed from the parent Makefile that control what gets +# installed and where: +# - POSTGRES_VERSIONS +# - POSTGRES_INSTALL_DIR +# - BUILD_DIR +# +# Variables passed from the parent Makefile that affect the build +# process and the resulting binaries: +# - PG_CONFIGURE_OPTS +# - PG_CFLAGS +# - PG_LDFLAGS +# - EXTRA_PATH_OVERRIDES + +### +### Main targets +### +### These are called from the main Makefile, and can also be called +### directly from command line + +# Compile and install a specific PostgreSQL version +postgres-install-%: postgres-configure-% \ + postgres-headers-install-% # to prevent `make install` conflicts with neon's `postgres-headers` + +# Install the PostgreSQL header files into $(POSTGRES_INSTALL_DIR)//include +# +# This is implicitly part of the 'postgres-install-%' target, but this can be handy +# if you want to install just the headers without building PostgreSQL, e.g. for building +# extensions. +postgres-headers-install-%: postgres-configure-% + +@echo "Installing PostgreSQL $* headers" + $(MAKE) -C $(BUILD_DIR)/$*/src/include MAKELEVEL=0 install + +# Run Postgres regression tests +postgres-check-%: postgres-install-% + $(MAKE) -C $(BUILD_DIR)/$* MAKELEVEL=0 check + +### +### Shorthands for the main targets, for convenience +### + +# Same as the above main targets, but for all supported PostgreSQL versions +# For example, 'make postgres-install' is equivalent to +# 'make postgres-install-v14 postgres-install-v15 postgres-install-v16 postgres-install-v17' +all_version_targets=postgres-install postgres-headers-install postgres-check +.PHONY: $(all_version_targets) +$(all_version_targets): postgres-%: $(foreach pg_version,$(POSTGRES_VERSIONS),postgres-%-$(pg_version)) + +.PHONY: postgres +postgres: postgres-install + +.PHONY: postgres-headers +postgres-headers: postgres-headers-install + +# 'postgres-v17' is an alias for 'postgres-install-v17' etc. +$(foreach pg_version,$(POSTGRES_VERSIONS),postgres-$(pg_version)): postgres-%: postgres-install-% + +### +### Intermediate targets +### +### These are not intended to be called directly, but are dependencies for the +### main targets. + +# Run 'configure' +$(BUILD_DIR)/%/config.status: + mkdir -p $(BUILD_DIR) + test -e $(BUILD_DIR)/CACHEDIR.TAG || echo "$(CACHEDIR_TAG_CONTENTS)" > $(BUILD_DIR)/CACHEDIR.TAG + + +@echo "Configuring Postgres $* build" + @test -s $(ROOT_PROJECT_DIR)/vendor/postgres-$*/configure || { \ + echo "\nPostgres submodule not found in $(ROOT_PROJECT_DIR)/vendor/postgres-$*/, execute "; \ + echo "'git submodule update --init --recursive --depth 2 --progress .' in project root.\n"; \ + exit 1; } + mkdir -p $(BUILD_DIR)/$* + + VERSION=$*; \ + EXTRA_VERSION=$$(cd $(ROOT_PROJECT_DIR)/vendor/postgres-$$VERSION && git rev-parse HEAD); \ + (cd $(BUILD_DIR)/$$VERSION && \ + env PATH="$(EXTRA_PATH_OVERRIDES):$$PATH" $(ROOT_PROJECT_DIR)/vendor/postgres-$$VERSION/configure \ + CFLAGS='$(PG_CFLAGS)' LDFLAGS='$(PG_LDFLAGS)' \ + $(PG_CONFIGURE_OPTS) --with-extra-version=" ($$EXTRA_VERSION)" \ + --prefix=$(abspath $(POSTGRES_INSTALL_DIR))/$$VERSION > configure.log) + +# nicer alias to run 'configure'. +# +# This tries to accomplish this rule: +# +# postgres-configure-%: $(BUILD_DIR)/%/config.status +# +# XXX: I'm not sure why the above rule doesn't work directly. But this accomplishses +# the same thing +$(foreach pg_version,$(POSTGRES_VERSIONS),postgres-configure-$(pg_version)): postgres-configure-%: FORCE $(BUILD_DIR)/%/config.status + +# Compile and install PostgreSQL (and a few contrib modules used in tests) +postgres-install-%: postgres-configure-% \ + postgres-headers-install-% # to prevent `make install` conflicts with neon's `postgres-headers-install` + +@echo "Compiling PostgreSQL $*" + $(MAKE) -C $(BUILD_DIR)/$* MAKELEVEL=0 install + $(MAKE) -C $(BUILD_DIR)/$*/contrib/pg_prewarm install + $(MAKE) -C $(BUILD_DIR)/$*/contrib/pg_buffercache install + $(MAKE) -C $(BUILD_DIR)/$*/contrib/pg_visibility install + $(MAKE) -C $(BUILD_DIR)/$*/contrib/pageinspect install + $(MAKE) -C $(BUILD_DIR)/$*/contrib/pg_trgm install + $(MAKE) -C $(BUILD_DIR)/$*/contrib/amcheck install + $(MAKE) -C $(BUILD_DIR)/$*/contrib/test_decoding install + +.PHONY: FORCE +FORCE: From 4c7956fa56e8b39e56a83342c72c774481dba295 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Arpad=20M=C3=BCller?= Date: Fri, 27 Jun 2025 17:14:55 +0200 Subject: [PATCH 15/20] Fix hang deleting offloaded timelines (#12366) We don't have cancellation support for timeline deletions. In other words, timeline deletion might still go on in an older generation while we are attaching it in a newer generation already, because the cancellation simply hasn't reached the deletion code. This has caused us to hit a situation with offloaded timelines in which the timeline was in an unrecoverable state: always returning an accepted response, but never a 404 like it should be. The detailed description can be found in [here](https://github.com/neondatabase/cloud/issues/30406#issuecomment-3008667859) (private repo link). TLDR: 1. we ask to delete timeline on old pageserver/generation, starts process in background 2. the storcon migrates the tenant to a different pageserver. - during attach, the pageserver still finds an index part, so it adds it to `offloaded_timelines` 4. the timeline deletion finishes, removing the index part in S3 5. there is a retry of the timeline deletion endpoint, sent to the new pageserver location. it is bound to fail however: - as the index part is gone, we print `Timeline already deleted in remote storage`. - the problem is that we then return an accepted response code, and not a 404. - this confuses the code calling us. it thinks the timeline is not deleted, so keeps retrying. - this state never gets recovered from until a reset/detach, because of the `offloaded_timelines` entry staying there. This is where this PR fixes things: if no index part can be found, we can safely assume that the timeline is gone in S3 (it's the last thing to be deleted), so we can remove it from `offloaded_timelines` and trigger a reupload of the manifest. Subsequent retries will pick that up. Why not improve the cancellation support? It is a more disruptive code change, that might have its own risks. So we don't do it for now. Fixes https://github.com/neondatabase/cloud/issues/30406 --- pageserver/src/tenant/timeline/delete.rs | 11 +- test_runner/regress/test_timeline_archive.py | 128 +++++++++++++++++++ 2 files changed, 138 insertions(+), 1 deletion(-) diff --git a/pageserver/src/tenant/timeline/delete.rs b/pageserver/src/tenant/timeline/delete.rs index 51bdd59f4f..f7dc44be90 100644 --- a/pageserver/src/tenant/timeline/delete.rs +++ b/pageserver/src/tenant/timeline/delete.rs @@ -241,8 +241,17 @@ impl DeleteTimelineFlow { { Ok(r) => r, Err(DownloadError::NotFound) => { - // Deletion is already complete + // Deletion is already complete. + // As we came here, we will need to remove the timeline from the tenant though. tracing::info!("Timeline already deleted in remote storage"); + if let TimelineOrOffloaded::Offloaded(_) = &timeline { + // We only supoprt this for offloaded timelines, as we don't know which state non-offloaded timelines are in. + tracing::info!( + "Timeline with gone index part is offloaded timeline. Removing from tenant." + ); + remove_maybe_offloaded_timeline_from_tenant(tenant, &timeline, &guard) + .await?; + } return Ok(()); } Err(e) => { diff --git a/test_runner/regress/test_timeline_archive.py b/test_runner/regress/test_timeline_archive.py index 8d46ef8306..41286a2adc 100644 --- a/test_runner/regress/test_timeline_archive.py +++ b/test_runner/regress/test_timeline_archive.py @@ -896,6 +896,134 @@ def test_timeline_retain_lsn( assert sum == pre_branch_sum +def test_timeline_offload_delete_race(neon_env_builder: NeonEnvBuilder): + """ + Regression test for https://github.com/neondatabase/cloud/issues/30406 + """ + remote_storage_kind = s3_storage() + neon_env_builder.enable_pageserver_remote_storage(remote_storage_kind) + neon_env_builder.num_pageservers = 2 + + env = neon_env_builder.init_start() + + # Turn off gc and compaction loops: we want to issue them manually for better reliability + tenant_id, root_timeline_id = env.create_tenant( + conf={ + "gc_period": "0s", + "compaction_period": "0s", + "checkpoint_distance": f"{1024**2}", + } + ) + + origin_ps = env.get_tenant_pageserver(tenant_id) + assert origin_ps + origin_ps.allowed_errors.extend( + [ + ".*Timed out waiting for deletion queue flush.*", + ".*Timed out waiting for flush to remote storage.*", + ] + ) + origin_ps_http = origin_ps.http_client() + + # We are not sharding this tenant + tenant_shard_id = TenantShardId(tenant_id, 0, 0) + + # Create a branch and archive it + child_timeline_id = env.create_branch("test_archived_branch_persisted", tenant_id) + + with env.endpoints.create_start( + "test_archived_branch_persisted", tenant_id=tenant_id + ) as endpoint: + endpoint.safe_psql_many( + [ + "CREATE TABLE foo(key serial primary key, t text default 'data_content')", + "INSERT INTO foo SELECT FROM generate_series(1,512)", + ] + ) + last_flush_lsn_upload(env, endpoint, tenant_id, child_timeline_id) + + assert_prefix_not_empty( + neon_env_builder.pageserver_remote_storage, + prefix=f"tenants/{str(tenant_id)}/", + ) + assert_prefix_not_empty( + neon_env_builder.pageserver_remote_storage, + prefix=f"tenants/{str(tenant_id)}/tenant-manifest", + ) + + origin_ps_http.timeline_archival_config( + tenant_id, + child_timeline_id, + state=TimelineArchivalState.ARCHIVED, + ) + + def timeline_offloaded_api(timeline_id: TimelineId) -> bool: + return any( + timeline["timeline_id"] == str(timeline_id) + for timeline in origin_ps_http.timeline_and_offloaded_list( + tenant_id=tenant_id + ).offloaded + ) + + def child_offloaded(): + origin_ps_http.timeline_offload(tenant_id=tenant_id, timeline_id=child_timeline_id) + assert timeline_offloaded_api(child_timeline_id) + + wait_until(child_offloaded) + + # Delete the timeline from the origin pageserver, holding up the deletion queue so that it doesn't finish + failpoint_deletion_queue = "deletion-queue-before-execute-pause" + origin_ps_http.configure_failpoints((failpoint_deletion_queue, "pause")) + origin_ps_http.timeline_delete(tenant_id, child_timeline_id) + + dest_ps = [ps for ps in env.pageservers if ps.id != origin_ps.id][0] + assert dest_ps + log.info(f"Migrating {tenant_id} {origin_ps.id}->{dest_ps.id}") + env.storage_controller.tenant_shard_migrate(tenant_shard_id, dest_ps_id=dest_ps.id) + + log.info("unstuck the DELETE") + origin_ps_http.configure_failpoints((failpoint_deletion_queue, "off")) + + def child_prefix_empty(): + assert_prefix_empty( + neon_env_builder.pageserver_remote_storage, + prefix=f"tenants/{str(tenant_id)}/{str(child_timeline_id)}/", + ) + + wait_until(child_prefix_empty) + + dest_ps_http = dest_ps.http_client() + + # We can't use timeline_delete_wait_completed here as timeline status will return 404, but we want to return 404 from the deletion endpoint + def timeline_is_missing(): + data = None + try: + data = dest_ps_http.timeline_delete(tenant_id, child_timeline_id) + log.info(f"timeline delete {data}") + except PageserverApiException as e: + log.debug(e) + if e.status_code == 404: + return + + raise RuntimeError(f"Timeline exists {data}") + + wait_until(timeline_is_missing) + # (dest_ps_http, tenant_id, child_timeline_id) + + # + # Now ensure that scrubber doesn't have anything to clean up. + # + + # Sleep some amount larger than min_age_secs + time.sleep(3) + + # Ensure that min_age_secs has a deletion impeding effect + gc_summary = env.storage_scrubber.pageserver_physical_gc(min_age_secs=1, mode="full") + assert gc_summary["remote_storage_errors"] == 0 + assert gc_summary["indices_deleted"] == 0 + assert gc_summary["tenant_manifests_deleted"] == 0 + + def test_timeline_offload_generations(neon_env_builder: NeonEnvBuilder): """ Test for scrubber deleting old generations of manifests From 0ee15002fc3ef3927b3c2c7acf33f5c410dbad3d Mon Sep 17 00:00:00 2001 From: Folke Behrens Date: Fri, 27 Jun 2025 17:20:23 +0200 Subject: [PATCH 16/20] proxy: Move client connection accept and handshake to pglb (#12380) * This must be a no-op. * Move proxy::task_main to pglb::task_main. * Move client accept, TLS and handshake to pglb. * Keep auth and wake in proxy. --- proxy/src/binary/pg_sni_router.rs | 3 +- proxy/src/binary/proxy.rs | 2 +- proxy/src/console_redirect_proxy.rs | 5 +- proxy/src/pglb/handshake.rs | 2 +- proxy/src/pglb/mod.rs | 329 ++++++++++++++++++++++++++ proxy/src/proxy/mod.rs | 346 +++------------------------- proxy/src/proxy/tests/mitm.rs | 3 + proxy/src/proxy/tests/mod.rs | 20 +- proxy/src/serverless/websocket.rs | 5 +- 9 files changed, 388 insertions(+), 327 deletions(-) diff --git a/proxy/src/binary/pg_sni_router.rs b/proxy/src/binary/pg_sni_router.rs index 070c73cdcf..b877aaddef 100644 --- a/proxy/src/binary/pg_sni_router.rs +++ b/proxy/src/binary/pg_sni_router.rs @@ -26,9 +26,10 @@ use utils::sentry_init::init_sentry; use crate::context::RequestContext; use crate::metrics::{Metrics, ThreadPoolMetrics}; +use crate::pglb::TlsRequired; use crate::pqproto::FeStartupPacket; use crate::protocol2::ConnectionInfo; -use crate::proxy::{ErrorSource, TlsRequired, copy_bidirectional_client_compute}; +use crate::proxy::{ErrorSource, copy_bidirectional_client_compute}; use crate::stream::{PqStream, Stream}; use crate::util::run_until_cancelled; diff --git a/proxy/src/binary/proxy.rs b/proxy/src/binary/proxy.rs index 9ead05d492..2133f33a4d 100644 --- a/proxy/src/binary/proxy.rs +++ b/proxy/src/binary/proxy.rs @@ -392,7 +392,7 @@ pub async fn run() -> anyhow::Result<()> { match auth_backend { Either::Left(auth_backend) => { if let Some(proxy_listener) = proxy_listener { - client_tasks.spawn(crate::proxy::task_main( + client_tasks.spawn(crate::pglb::task_main( config, auth_backend, proxy_listener, diff --git a/proxy/src/console_redirect_proxy.rs b/proxy/src/console_redirect_proxy.rs index d5903286a0..041a56e032 100644 --- a/proxy/src/console_redirect_proxy.rs +++ b/proxy/src/console_redirect_proxy.rs @@ -11,11 +11,12 @@ use crate::config::{ProxyConfig, ProxyProtocolV2}; use crate::context::RequestContext; use crate::error::ReportableError; use crate::metrics::{Metrics, NumClientConnectionsGuard}; +use crate::pglb::ClientRequestError; use crate::pglb::handshake::{HandshakeData, handshake}; use crate::pglb::passthrough::ProxyPassthrough; use crate::protocol2::{ConnectHeader, ConnectionInfo, read_proxy_protocol}; use crate::proxy::connect_compute::{TcpMechanism, connect_to_compute}; -use crate::proxy::{ClientRequestError, ErrorSource, prepare_client_connection}; +use crate::proxy::{ErrorSource, finish_client_init}; use crate::util::run_until_cancelled; pub async fn task_main( @@ -232,7 +233,7 @@ pub(crate) async fn handle_client( let session = cancellation_handler.get_key(); - prepare_client_connection(&pg_settings, *session.key(), &mut stream); + finish_client_init(&pg_settings, *session.key(), &mut stream); let stream = stream.flush_and_into_inner().await?; let session_id = ctx.session_id(); diff --git a/proxy/src/pglb/handshake.rs b/proxy/src/pglb/handshake.rs index 6970ab8714..25a2d01b4a 100644 --- a/proxy/src/pglb/handshake.rs +++ b/proxy/src/pglb/handshake.rs @@ -8,10 +8,10 @@ use crate::config::TlsConfig; use crate::context::RequestContext; use crate::error::ReportableError; use crate::metrics::Metrics; +use crate::pglb::TlsRequired; use crate::pqproto::{ BeMessage, CancelKeyData, FeStartupPacket, ProtocolVersion, StartupMessageParams, }; -use crate::proxy::TlsRequired; use crate::stream::{PqStream, Stream, StreamUpgradeError}; use crate::tls::PG_ALPN_PROTOCOL; diff --git a/proxy/src/pglb/mod.rs b/proxy/src/pglb/mod.rs index cb82524cf6..c4cab155c5 100644 --- a/proxy/src/pglb/mod.rs +++ b/proxy/src/pglb/mod.rs @@ -2,3 +2,332 @@ pub mod copy_bidirectional; pub mod handshake; pub mod inprocess; pub mod passthrough; + +use std::sync::Arc; + +use futures::FutureExt; +use smol_str::ToSmolStr; +use thiserror::Error; +use tokio::io::{AsyncRead, AsyncWrite}; +use tokio_util::sync::CancellationToken; +use tracing::{Instrument, debug, error, info, warn}; + +use crate::auth; +use crate::cancellation::{self, CancellationHandler}; +use crate::config::{ProxyConfig, ProxyProtocolV2, TlsConfig}; +use crate::context::RequestContext; +use crate::error::{ReportableError, UserFacingError}; +use crate::metrics::{Metrics, NumClientConnectionsGuard}; +pub use crate::pglb::copy_bidirectional::ErrorSource; +use crate::pglb::handshake::{HandshakeData, HandshakeError, handshake}; +use crate::pglb::passthrough::ProxyPassthrough; +use crate::protocol2::{ConnectHeader, ConnectionInfo, ConnectionInfoExtra, read_proxy_protocol}; +use crate::proxy::handle_client; +use crate::rate_limiter::EndpointRateLimiter; +use crate::stream::Stream; +use crate::util::run_until_cancelled; + +pub const ERR_INSECURE_CONNECTION: &str = "connection is insecure (try using `sslmode=require`)"; + +#[derive(Error, Debug)] +#[error("{ERR_INSECURE_CONNECTION}")] +pub struct TlsRequired; + +impl ReportableError for TlsRequired { + fn get_error_kind(&self) -> crate::error::ErrorKind { + crate::error::ErrorKind::User + } +} + +impl UserFacingError for TlsRequired {} + +pub async fn task_main( + config: &'static ProxyConfig, + auth_backend: &'static auth::Backend<'static, ()>, + listener: tokio::net::TcpListener, + cancellation_token: CancellationToken, + cancellation_handler: Arc, + endpoint_rate_limiter: Arc, +) -> anyhow::Result<()> { + scopeguard::defer! { + info!("proxy has shut down"); + } + + // When set for the server socket, the keepalive setting + // will be inherited by all accepted client sockets. + socket2::SockRef::from(&listener).set_keepalive(true)?; + + let connections = tokio_util::task::task_tracker::TaskTracker::new(); + let cancellations = tokio_util::task::task_tracker::TaskTracker::new(); + + while let Some(accept_result) = + run_until_cancelled(listener.accept(), &cancellation_token).await + { + let (socket, peer_addr) = accept_result?; + + let conn_gauge = Metrics::get() + .proxy + .client_connections + .guard(crate::metrics::Protocol::Tcp); + + let session_id = uuid::Uuid::new_v4(); + let cancellation_handler = Arc::clone(&cancellation_handler); + let cancellations = cancellations.clone(); + + debug!(protocol = "tcp", %session_id, "accepted new TCP connection"); + let endpoint_rate_limiter2 = endpoint_rate_limiter.clone(); + + connections.spawn(async move { + let (socket, conn_info) = match config.proxy_protocol_v2 { + ProxyProtocolV2::Required => { + match read_proxy_protocol(socket).await { + Err(e) => { + warn!("per-client task finished with an error: {e:#}"); + return; + } + // our load balancers will not send any more data. let's just exit immediately + Ok((_socket, ConnectHeader::Local)) => { + debug!("healthcheck received"); + return; + } + Ok((socket, ConnectHeader::Proxy(info))) => (socket, info), + } + } + // ignore the header - it cannot be confused for a postgres or http connection so will + // error later. + ProxyProtocolV2::Rejected => ( + socket, + ConnectionInfo { + addr: peer_addr, + extra: None, + }, + ), + }; + + match socket.set_nodelay(true) { + Ok(()) => {} + Err(e) => { + error!( + "per-client task finished with an error: failed to set socket option: {e:#}" + ); + return; + } + } + + let ctx = RequestContext::new(session_id, conn_info, crate::metrics::Protocol::Tcp); + + let res = handle_connection( + config, + auth_backend, + &ctx, + cancellation_handler, + socket, + ClientMode::Tcp, + endpoint_rate_limiter2, + conn_gauge, + cancellations, + ) + .instrument(ctx.span()) + .boxed() + .await; + + match res { + Err(e) => { + ctx.set_error_kind(e.get_error_kind()); + warn!(parent: &ctx.span(), "per-client task finished with an error: {e:#}"); + } + Ok(None) => { + ctx.set_success(); + } + Ok(Some(p)) => { + ctx.set_success(); + let _disconnect = ctx.log_connect(); + match p.proxy_pass().await { + Ok(()) => {} + Err(ErrorSource::Client(e)) => { + warn!( + ?session_id, + "per-client task finished with an IO error from the client: {e:#}" + ); + } + Err(ErrorSource::Compute(e)) => { + error!( + ?session_id, + "per-client task finished with an IO error from the compute: {e:#}" + ); + } + } + } + } + }); + } + + connections.close(); + cancellations.close(); + drop(listener); + + // Drain connections + connections.wait().await; + cancellations.wait().await; + + Ok(()) +} + +pub(crate) enum ClientMode { + Tcp, + Websockets { hostname: Option }, +} + +/// Abstracts the logic of handling TCP vs WS clients +impl ClientMode { + pub fn allow_cleartext(&self) -> bool { + match self { + ClientMode::Tcp => false, + ClientMode::Websockets { .. } => true, + } + } + + pub fn hostname<'a, S>(&'a self, s: &'a Stream) -> Option<&'a str> { + match self { + ClientMode::Tcp => s.sni_hostname(), + ClientMode::Websockets { hostname } => hostname.as_deref(), + } + } + + pub fn handshake_tls<'a>(&self, tls: Option<&'a TlsConfig>) -> Option<&'a TlsConfig> { + match self { + ClientMode::Tcp => tls, + // TLS is None here if using websockets, because the connection is already encrypted. + ClientMode::Websockets { .. } => None, + } + } +} + +#[derive(Debug, Error)] +// almost all errors should be reported to the user, but there's a few cases where we cannot +// 1. Cancellation: we are not allowed to tell the client any cancellation statuses for security reasons +// 2. Handshake: handshake reports errors if it can, otherwise if the handshake fails due to protocol violation, +// we cannot be sure the client even understands our error message +// 3. PrepareClient: The client disconnected, so we can't tell them anyway... +pub(crate) enum ClientRequestError { + #[error("{0}")] + Cancellation(#[from] cancellation::CancelError), + #[error("{0}")] + Handshake(#[from] HandshakeError), + #[error("{0}")] + HandshakeTimeout(#[from] tokio::time::error::Elapsed), + #[error("{0}")] + PrepareClient(#[from] std::io::Error), + #[error("{0}")] + ReportedError(#[from] crate::stream::ReportedError), +} + +impl ReportableError for ClientRequestError { + fn get_error_kind(&self) -> crate::error::ErrorKind { + match self { + ClientRequestError::Cancellation(e) => e.get_error_kind(), + ClientRequestError::Handshake(e) => e.get_error_kind(), + ClientRequestError::HandshakeTimeout(_) => crate::error::ErrorKind::RateLimit, + ClientRequestError::ReportedError(e) => e.get_error_kind(), + ClientRequestError::PrepareClient(_) => crate::error::ErrorKind::ClientDisconnect, + } + } +} + +#[allow(clippy::too_many_arguments)] +pub(crate) async fn handle_connection( + config: &'static ProxyConfig, + auth_backend: &'static auth::Backend<'static, ()>, + ctx: &RequestContext, + cancellation_handler: Arc, + client: S, + mode: ClientMode, + endpoint_rate_limiter: Arc, + conn_gauge: NumClientConnectionsGuard<'static>, + cancellations: tokio_util::task::task_tracker::TaskTracker, +) -> Result>, ClientRequestError> { + debug!( + protocol = %ctx.protocol(), + "handling interactive connection from client" + ); + + let metrics = &Metrics::get().proxy; + let proto = ctx.protocol(); + let request_gauge = metrics.connection_requests.guard(proto); + + let tls = config.tls_config.load(); + let tls = tls.as_deref(); + + let record_handshake_error = !ctx.has_private_peer_addr(); + let pause = ctx.latency_timer_pause(crate::metrics::Waiting::Client); + let do_handshake = handshake(ctx, client, mode.handshake_tls(tls), record_handshake_error); + + let (mut client, params) = match tokio::time::timeout(config.handshake_timeout, do_handshake) + .await?? + { + HandshakeData::Startup(client, params) => (client, params), + HandshakeData::Cancel(cancel_key_data) => { + // spawn a task to cancel the session, but don't wait for it + cancellations.spawn({ + let cancellation_handler_clone = Arc::clone(&cancellation_handler); + let ctx = ctx.clone(); + let cancel_span = tracing::span!(parent: None, tracing::Level::INFO, "cancel_session", session_id = ?ctx.session_id()); + cancel_span.follows_from(tracing::Span::current()); + async move { + cancellation_handler_clone + .cancel_session( + cancel_key_data, + ctx, + config.authentication_config.ip_allowlist_check_enabled, + config.authentication_config.is_vpc_acccess_proxy, + auth_backend.get_api(), + ) + .await + .inspect_err(|e | debug!(error = ?e, "cancel_session failed")).ok(); + }.instrument(cancel_span) + }); + + return Ok(None); + } + }; + drop(pause); + + ctx.set_db_options(params.clone()); + + let common_names = tls.map(|tls| &tls.common_names); + + let (node, cancel_on_shutdown) = handle_client( + config, + auth_backend, + ctx, + cancellation_handler, + &mut client, + &mode, + endpoint_rate_limiter, + common_names, + ¶ms, + ) + .await?; + + let client = client.flush_and_into_inner().await?; + + let private_link_id = match ctx.extra() { + Some(ConnectionInfoExtra::Aws { vpce_id }) => Some(vpce_id.clone()), + Some(ConnectionInfoExtra::Azure { link_id }) => Some(link_id.to_smolstr()), + None => None, + }; + + Ok(Some(ProxyPassthrough { + client, + compute: node.stream, + + aux: node.aux, + private_link_id, + + _cancel_on_shutdown: cancel_on_shutdown, + + _req: request_gauge, + _conn: conn_gauge, + _db_conn: node.guage, + })) +} diff --git a/proxy/src/proxy/mod.rs b/proxy/src/proxy/mod.rs index d9c0585efb..08c81afa04 100644 --- a/proxy/src/proxy/mod.rs +++ b/proxy/src/proxy/mod.rs @@ -5,326 +5,64 @@ pub(crate) mod connect_compute; pub(crate) mod retry; pub(crate) mod wake_compute; +use std::collections::HashSet; +use std::convert::Infallible; use std::sync::Arc; -use futures::FutureExt; use itertools::Itertools; use once_cell::sync::OnceCell; use regex::Regex; use serde::{Deserialize, Serialize}; -use smol_str::{SmolStr, ToSmolStr, format_smolstr}; -use thiserror::Error; +use smol_str::{SmolStr, format_smolstr}; use tokio::io::{AsyncRead, AsyncWrite}; -use tokio_util::sync::CancellationToken; -use tracing::{Instrument, debug, error, info, warn}; +use tokio::sync::oneshot; +use tracing::Instrument; use crate::cache::Cache; -use crate::cancellation::{self, CancellationHandler}; -use crate::config::{ProxyConfig, ProxyProtocolV2, TlsConfig}; +use crate::cancellation::CancellationHandler; +use crate::compute::ComputeConnection; +use crate::config::ProxyConfig; use crate::context::RequestContext; use crate::control_plane::client::ControlPlaneClient; -use crate::error::{ReportableError, UserFacingError}; -use crate::metrics::{Metrics, NumClientConnectionsGuard}; pub use crate::pglb::copy_bidirectional::{ErrorSource, copy_bidirectional_client_compute}; -use crate::pglb::handshake::{HandshakeData, HandshakeError, handshake}; -use crate::pglb::passthrough::ProxyPassthrough; +use crate::pglb::{ClientMode, ClientRequestError}; use crate::pqproto::{BeMessage, CancelKeyData, StartupMessageParams}; -use crate::protocol2::{ConnectHeader, ConnectionInfo, ConnectionInfoExtra, read_proxy_protocol}; use crate::proxy::connect_compute::{TcpMechanism, connect_to_compute}; use crate::proxy::retry::ShouldRetryWakeCompute; use crate::rate_limiter::EndpointRateLimiter; use crate::stream::{PqStream, Stream}; use crate::types::EndpointCacheKey; -use crate::util::run_until_cancelled; use crate::{auth, compute}; -const ERR_INSECURE_CONNECTION: &str = "connection is insecure (try using `sslmode=require`)"; - -#[derive(Error, Debug)] -#[error("{ERR_INSECURE_CONNECTION}")] -pub struct TlsRequired; - -impl ReportableError for TlsRequired { - fn get_error_kind(&self) -> crate::error::ErrorKind { - crate::error::ErrorKind::User - } -} - -impl UserFacingError for TlsRequired {} - -pub async fn task_main( - config: &'static ProxyConfig, - auth_backend: &'static auth::Backend<'static, ()>, - listener: tokio::net::TcpListener, - cancellation_token: CancellationToken, - cancellation_handler: Arc, - endpoint_rate_limiter: Arc, -) -> anyhow::Result<()> { - scopeguard::defer! { - info!("proxy has shut down"); - } - - // When set for the server socket, the keepalive setting - // will be inherited by all accepted client sockets. - socket2::SockRef::from(&listener).set_keepalive(true)?; - - let connections = tokio_util::task::task_tracker::TaskTracker::new(); - let cancellations = tokio_util::task::task_tracker::TaskTracker::new(); - - while let Some(accept_result) = - run_until_cancelled(listener.accept(), &cancellation_token).await - { - let (socket, peer_addr) = accept_result?; - - let conn_gauge = Metrics::get() - .proxy - .client_connections - .guard(crate::metrics::Protocol::Tcp); - - let session_id = uuid::Uuid::new_v4(); - let cancellation_handler = Arc::clone(&cancellation_handler); - let cancellations = cancellations.clone(); - - debug!(protocol = "tcp", %session_id, "accepted new TCP connection"); - let endpoint_rate_limiter2 = endpoint_rate_limiter.clone(); - - connections.spawn(async move { - let (socket, conn_info) = match config.proxy_protocol_v2 { - ProxyProtocolV2::Required => { - match read_proxy_protocol(socket).await { - Err(e) => { - warn!("per-client task finished with an error: {e:#}"); - return; - } - // our load balancers will not send any more data. let's just exit immediately - Ok((_socket, ConnectHeader::Local)) => { - debug!("healthcheck received"); - return; - } - Ok((socket, ConnectHeader::Proxy(info))) => (socket, info), - } - } - // ignore the header - it cannot be confused for a postgres or http connection so will - // error later. - ProxyProtocolV2::Rejected => ( - socket, - ConnectionInfo { - addr: peer_addr, - extra: None, - }, - ), - }; - - match socket.set_nodelay(true) { - Ok(()) => {} - Err(e) => { - error!( - "per-client task finished with an error: failed to set socket option: {e:#}" - ); - return; - } - } - - let ctx = RequestContext::new(session_id, conn_info, crate::metrics::Protocol::Tcp); - - let res = handle_client( - config, - auth_backend, - &ctx, - cancellation_handler, - socket, - ClientMode::Tcp, - endpoint_rate_limiter2, - conn_gauge, - cancellations, - ) - .instrument(ctx.span()) - .boxed() - .await; - - match res { - Err(e) => { - ctx.set_error_kind(e.get_error_kind()); - warn!(parent: &ctx.span(), "per-client task finished with an error: {e:#}"); - } - Ok(None) => { - ctx.set_success(); - } - Ok(Some(p)) => { - ctx.set_success(); - let _disconnect = ctx.log_connect(); - match p.proxy_pass().await { - Ok(()) => {} - Err(ErrorSource::Client(e)) => { - warn!( - ?session_id, - "per-client task finished with an IO error from the client: {e:#}" - ); - } - Err(ErrorSource::Compute(e)) => { - error!( - ?session_id, - "per-client task finished with an IO error from the compute: {e:#}" - ); - } - } - } - } - }); - } - - connections.close(); - cancellations.close(); - drop(listener); - - // Drain connections - connections.wait().await; - cancellations.wait().await; - - Ok(()) -} - -pub(crate) enum ClientMode { - Tcp, - Websockets { hostname: Option }, -} - -/// Abstracts the logic of handling TCP vs WS clients -impl ClientMode { - pub(crate) fn allow_cleartext(&self) -> bool { - match self { - ClientMode::Tcp => false, - ClientMode::Websockets { .. } => true, - } - } - - fn hostname<'a, S>(&'a self, s: &'a Stream) -> Option<&'a str> { - match self { - ClientMode::Tcp => s.sni_hostname(), - ClientMode::Websockets { hostname } => hostname.as_deref(), - } - } - - fn handshake_tls<'a>(&self, tls: Option<&'a TlsConfig>) -> Option<&'a TlsConfig> { - match self { - ClientMode::Tcp => tls, - // TLS is None here if using websockets, because the connection is already encrypted. - ClientMode::Websockets { .. } => None, - } - } -} - -#[derive(Debug, Error)] -// almost all errors should be reported to the user, but there's a few cases where we cannot -// 1. Cancellation: we are not allowed to tell the client any cancellation statuses for security reasons -// 2. Handshake: handshake reports errors if it can, otherwise if the handshake fails due to protocol violation, -// we cannot be sure the client even understands our error message -// 3. PrepareClient: The client disconnected, so we can't tell them anyway... -pub(crate) enum ClientRequestError { - #[error("{0}")] - Cancellation(#[from] cancellation::CancelError), - #[error("{0}")] - Handshake(#[from] HandshakeError), - #[error("{0}")] - HandshakeTimeout(#[from] tokio::time::error::Elapsed), - #[error("{0}")] - PrepareClient(#[from] std::io::Error), - #[error("{0}")] - ReportedError(#[from] crate::stream::ReportedError), -} - -impl ReportableError for ClientRequestError { - fn get_error_kind(&self) -> crate::error::ErrorKind { - match self { - ClientRequestError::Cancellation(e) => e.get_error_kind(), - ClientRequestError::Handshake(e) => e.get_error_kind(), - ClientRequestError::HandshakeTimeout(_) => crate::error::ErrorKind::RateLimit, - ClientRequestError::ReportedError(e) => e.get_error_kind(), - ClientRequestError::PrepareClient(_) => crate::error::ErrorKind::ClientDisconnect, - } - } -} - #[allow(clippy::too_many_arguments)] pub(crate) async fn handle_client( config: &'static ProxyConfig, auth_backend: &'static auth::Backend<'static, ()>, ctx: &RequestContext, cancellation_handler: Arc, - stream: S, - mode: ClientMode, + client: &mut PqStream>, + mode: &ClientMode, endpoint_rate_limiter: Arc, - conn_gauge: NumClientConnectionsGuard<'static>, - cancellations: tokio_util::task::task_tracker::TaskTracker, -) -> Result>, ClientRequestError> { - debug!( - protocol = %ctx.protocol(), - "handling interactive connection from client" - ); - - let metrics = &Metrics::get().proxy; - let proto = ctx.protocol(); - let request_gauge = metrics.connection_requests.guard(proto); - - let tls = config.tls_config.load(); - let tls = tls.as_deref(); - - let record_handshake_error = !ctx.has_private_peer_addr(); - let pause = ctx.latency_timer_pause(crate::metrics::Waiting::Client); - let do_handshake = handshake(ctx, stream, mode.handshake_tls(tls), record_handshake_error); - - let (mut stream, params) = match tokio::time::timeout(config.handshake_timeout, do_handshake) - .await?? - { - HandshakeData::Startup(stream, params) => (stream, params), - HandshakeData::Cancel(cancel_key_data) => { - // spawn a task to cancel the session, but don't wait for it - cancellations.spawn({ - let cancellation_handler_clone = Arc::clone(&cancellation_handler); - let ctx = ctx.clone(); - let cancel_span = tracing::span!(parent: None, tracing::Level::INFO, "cancel_session", session_id = ?ctx.session_id()); - cancel_span.follows_from(tracing::Span::current()); - async move { - cancellation_handler_clone - .cancel_session( - cancel_key_data, - ctx, - config.authentication_config.ip_allowlist_check_enabled, - config.authentication_config.is_vpc_acccess_proxy, - auth_backend.get_api(), - ) - .await - .inspect_err(|e | debug!(error = ?e, "cancel_session failed")).ok(); - }.instrument(cancel_span) - }); - - return Ok(None); - } - }; - drop(pause); - - ctx.set_db_options(params.clone()); - - let hostname = mode.hostname(stream.get_ref()); - - let common_names = tls.map(|tls| &tls.common_names); - + common_names: Option<&HashSet>, + params: &StartupMessageParams, +) -> Result<(ComputeConnection, oneshot::Sender), ClientRequestError> { + let hostname = mode.hostname(client.get_ref()); // Extract credentials which we're going to use for auth. let result = auth_backend .as_ref() - .map(|()| auth::ComputeUserInfoMaybeEndpoint::parse(ctx, ¶ms, hostname, common_names)) + .map(|()| auth::ComputeUserInfoMaybeEndpoint::parse(ctx, params, hostname, common_names)) .transpose(); let user_info = match result { Ok(user_info) => user_info, - Err(e) => Err(stream.throw_error(e, Some(ctx)).await)?, + Err(e) => Err(client.throw_error(e, Some(ctx)).await)?, }; let user = user_info.get_user().to_owned(); let user_info = match user_info .authenticate( ctx, - &mut stream, + client, mode.allow_cleartext(), &config.authentication_config, endpoint_rate_limiter, @@ -337,7 +75,7 @@ pub(crate) async fn handle_client( let app = params.get("application_name"); let params_span = tracing::info_span!("", ?user, ?db, ?app); - return Err(stream + return Err(client .throw_error(e, Some(ctx)) .instrument(params_span) .await)?; @@ -350,7 +88,7 @@ pub(crate) async fn handle_client( }; let params_compat = creds.info.options.get(NeonOptions::PARAMS_COMPAT).is_some(); let mut auth_info = compute::AuthInfo::with_auth_keys(creds.keys); - auth_info.set_startup_params(¶ms, params_compat); + auth_info.set_startup_params(params, params_compat); let mut node; let mut attempt = 0; @@ -370,6 +108,7 @@ pub(crate) async fn handle_client( let pg_settings = loop { attempt += 1; + // TODO: callback to pglb let res = connect_to_compute( ctx, &connect, @@ -381,7 +120,7 @@ pub(crate) async fn handle_client( match res { Ok(n) => node = n, - Err(e) => return Err(stream.throw_error(e, Some(ctx)).await)?, + Err(e) => return Err(client.throw_error(e, Some(ctx)).await)?, } let auth::Backend::ControlPlane(cplane, user_info) = &backend else { @@ -400,17 +139,16 @@ pub(crate) async fn handle_client( cplane_proxy_v1.caches.node_info.invalidate(&key); } } - Err(e) => Err(stream.throw_error(e, Some(ctx)).await)?, + Err(e) => Err(client.throw_error(e, Some(ctx)).await)?, } }; let session = cancellation_handler.get_key(); - prepare_client_connection(&pg_settings, *session.key(), &mut stream); - let stream = stream.flush_and_into_inner().await?; + finish_client_init(&pg_settings, *session.key(), client); let session_id = ctx.session_id(); - let (cancel_on_shutdown, cancel) = tokio::sync::oneshot::channel(); + let (cancel_on_shutdown, cancel) = oneshot::channel(); tokio::spawn(async move { session .maintain_cancel_key( @@ -422,50 +160,32 @@ pub(crate) async fn handle_client( .await; }); - let private_link_id = match ctx.extra() { - Some(ConnectionInfoExtra::Aws { vpce_id }) => Some(vpce_id.clone()), - Some(ConnectionInfoExtra::Azure { link_id }) => Some(link_id.to_smolstr()), - None => None, - }; - - Ok(Some(ProxyPassthrough { - client: stream, - compute: node.stream, - - aux: node.aux, - private_link_id, - - _cancel_on_shutdown: cancel_on_shutdown, - - _req: request_gauge, - _conn: conn_gauge, - _db_conn: node.guage, - })) + Ok((node, cancel_on_shutdown)) } /// Finish client connection initialization: confirm auth success, send params, etc. -pub(crate) fn prepare_client_connection( +pub(crate) fn finish_client_init( settings: &compute::PostgresSettings, cancel_key_data: CancelKeyData, - stream: &mut PqStream, + client: &mut PqStream, ) { // Forward all deferred notices to the client. for notice in &settings.delayed_notice { - stream.write_raw(notice.as_bytes().len(), b'N', |buf| { + client.write_raw(notice.as_bytes().len(), b'N', |buf| { buf.extend_from_slice(notice.as_bytes()); }); } // Forward all postgres connection params to the client. for (name, value) in &settings.params { - stream.write_message(BeMessage::ParameterStatus { + client.write_message(BeMessage::ParameterStatus { name: name.as_bytes(), value: value.as_bytes(), }); } - stream.write_message(BeMessage::BackendKeyData(cancel_key_data)); - stream.write_message(BeMessage::ReadyForQuery); + client.write_message(BeMessage::BackendKeyData(cancel_key_data)); + client.write_message(BeMessage::ReadyForQuery); } #[derive(Debug, Clone, PartialEq, Eq, Default, Serialize, Deserialize)] @@ -475,7 +195,7 @@ impl NeonOptions { // proxy options: /// `PARAMS_COMPAT` allows opting in to forwarding all startup parameters from client to compute. - const PARAMS_COMPAT: &str = "proxy_params_compat"; + pub const PARAMS_COMPAT: &str = "proxy_params_compat"; // cplane options: diff --git a/proxy/src/proxy/tests/mitm.rs b/proxy/src/proxy/tests/mitm.rs index 67dd0ab522..b09d8edc4c 100644 --- a/proxy/src/proxy/tests/mitm.rs +++ b/proxy/src/proxy/tests/mitm.rs @@ -14,6 +14,9 @@ use tokio::io::{AsyncReadExt, AsyncWriteExt, DuplexStream}; use tokio_util::codec::{Decoder, Encoder}; use super::*; +use crate::config::TlsConfig; +use crate::context::RequestContext; +use crate::pglb::handshake::{HandshakeData, handshake}; enum Intercept { None, diff --git a/proxy/src/proxy/tests/mod.rs b/proxy/src/proxy/tests/mod.rs index 4f27496019..dd89b05426 100644 --- a/proxy/src/proxy/tests/mod.rs +++ b/proxy/src/proxy/tests/mod.rs @@ -3,6 +3,7 @@ mod mitm; +use std::sync::Arc; use std::time::Duration; use anyhow::{Context, bail}; @@ -10,26 +11,31 @@ use async_trait::async_trait; use http::StatusCode; use postgres_client::config::SslMode; use postgres_client::tls::{MakeTlsConnect, NoTls}; -use retry::{ShouldRetryWakeCompute, retry_after}; use rstest::rstest; use rustls::crypto::ring; use rustls::pki_types; -use tokio::io::DuplexStream; +use tokio::io::{AsyncRead, AsyncWrite, DuplexStream}; use tracing_test::traced_test; use super::retry::CouldRetry; -use super::*; use crate::auth::backend::{ComputeUserInfo, MaybeOwned}; -use crate::config::{ComputeConfig, RetryConfig}; +use crate::config::{ComputeConfig, RetryConfig, TlsConfig}; +use crate::context::RequestContext; use crate::control_plane::client::{ControlPlaneClient, TestControlPlaneClient}; use crate::control_plane::messages::{ControlPlaneErrorMessage, Details, MetricsAuxInfo, Status}; use crate::control_plane::{self, CachedNodeInfo, NodeInfo, NodeInfoCache}; -use crate::error::ErrorKind; -use crate::proxy::connect_compute::ConnectMechanism; +use crate::error::{ErrorKind, ReportableError}; +use crate::pglb::ERR_INSECURE_CONNECTION; +use crate::pglb::handshake::{HandshakeData, handshake}; +use crate::pqproto::BeMessage; +use crate::proxy::NeonOptions; +use crate::proxy::connect_compute::{ConnectMechanism, connect_to_compute}; +use crate::proxy::retry::{ShouldRetryWakeCompute, retry_after}; +use crate::stream::{PqStream, Stream}; use crate::tls::client_config::compute_client_config_with_certs; use crate::tls::server_config::CertResolver; use crate::types::{BranchId, EndpointId, ProjectId}; -use crate::{sasl, scram}; +use crate::{auth, compute, sasl, scram}; /// Generate a set of TLS certificates: CA + server. fn generate_certs( diff --git a/proxy/src/serverless/websocket.rs b/proxy/src/serverless/websocket.rs index 0d374e6df2..1960709fba 100644 --- a/proxy/src/serverless/websocket.rs +++ b/proxy/src/serverless/websocket.rs @@ -17,7 +17,8 @@ use crate::config::ProxyConfig; use crate::context::RequestContext; use crate::error::ReportableError; use crate::metrics::Metrics; -use crate::proxy::{ClientMode, ErrorSource, handle_client}; +use crate::pglb::{ClientMode, handle_connection}; +use crate::proxy::ErrorSource; use crate::rate_limiter::EndpointRateLimiter; pin_project! { @@ -142,7 +143,7 @@ pub(crate) async fn serve_websocket( .client_connections .guard(crate::metrics::Protocol::Ws); - let res = Box::pin(handle_client( + let res = Box::pin(handle_connection( config, auth_backend, &ctx, From e33e1094031578b384f448876af0048b14421b50 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Fri, 27 Jun 2025 17:26:00 +0200 Subject: [PATCH 17/20] fix(pageserver): buffered writer cancellation error handling (#12376) ## Problem The problem has been well described in already-commited PR #11853. tl;dr: BufferedWriter is sensitive to cancellation, which the previous approach was not. The write path was most affected (ingest & compaction), which was mostly fixed in #11853: it introduced `PutError` and mapped instances of `PutError` that were due to cancellation of underlying buffered writer into `CreateImageLayersError::Cancelled`. However, there is a long tail of remaining errors that weren't caught by #11853 that result in `CompactionError::Other`s, which we log with great noise. ## Solution The stack trace logging for CompactionError::Other added in #11853 allows us to chop away at that long tail using the following pattern: - look at the stack trace - from leaf up, identify the place where we incorrectly map from the distinguished variant X indicating cancellation to an `anyhow::Error` - follow that anyhow further up, ensuring it stays the same anyhow all the way up in the `CompactionError::Other` - since it stayed one anyhow chain all the way up, root_cause() will yield us X - so, in `log_compaction_error`, add an additional `downcast_ref` check for X This PR specifically adds checks for - the flush task cancelling (FlushTaskError, BlobWriterError) - opening of the layer writer (GateError) That should cover all the reports in issues - https://github.com/neondatabase/cloud/issues/29434 - https://github.com/neondatabase/neon/issues/12162 ## Refs - follow-up to #11853 - fixup of / fixes https://github.com/neondatabase/neon/issues/11762 - fixes https://github.com/neondatabase/neon/issues/12162 - refs https://github.com/neondatabase/cloud/issues/29434 --- libs/utils/src/sync/gate.rs | 8 ++++++++ pageserver/src/tenant/tasks.rs | 18 +++++++++++++++++- 2 files changed, 25 insertions(+), 1 deletion(-) diff --git a/libs/utils/src/sync/gate.rs b/libs/utils/src/sync/gate.rs index 93460785bf..862b2cff9e 100644 --- a/libs/utils/src/sync/gate.rs +++ b/libs/utils/src/sync/gate.rs @@ -86,6 +86,14 @@ pub enum GateError { GateClosed, } +impl GateError { + pub fn is_cancel(&self) -> bool { + match self { + GateError::GateClosed => true, + } + } +} + impl Default for Gate { fn default() -> Self { Self { diff --git a/pageserver/src/tenant/tasks.rs b/pageserver/src/tenant/tasks.rs index 4709a6d616..954dd38bb4 100644 --- a/pageserver/src/tenant/tasks.rs +++ b/pageserver/src/tenant/tasks.rs @@ -17,14 +17,17 @@ use tracing::*; use utils::backoff::exponential_backoff_duration; use utils::completion::Barrier; use utils::pausable_failpoint; +use utils::sync::gate::GateError; use crate::context::{DownloadBehavior, RequestContext}; use crate::metrics::{self, BackgroundLoopSemaphoreMetricsRecorder, TENANT_TASK_EVENTS}; use crate::task_mgr::{self, BACKGROUND_RUNTIME, TOKIO_WORKER_THREADS, TaskKind}; +use crate::tenant::blob_io::WriteBlobError; use crate::tenant::throttle::Stats; use crate::tenant::timeline::CompactionError; use crate::tenant::timeline::compaction::CompactionOutcome; use crate::tenant::{TenantShard, TenantState}; +use crate::virtual_file::owned_buffers_io::write::FlushTaskError; /// Semaphore limiting concurrent background tasks (across all tenants). /// @@ -313,7 +316,20 @@ pub(crate) fn log_compaction_error( let timeline = root_cause .downcast_ref::() .is_some_and(|e| e.is_stopping()); - let is_stopping = upload_queue || timeline; + let buffered_writer_flush_task_canelled = root_cause + .downcast_ref::() + .is_some_and(|e| e.is_cancel()); + let write_blob_cancelled = root_cause + .downcast_ref::() + .is_some_and(|e| e.is_cancel()); + let gate_closed = root_cause + .downcast_ref::() + .is_some_and(|e| e.is_cancel()); + let is_stopping = upload_queue + || timeline + || buffered_writer_flush_task_canelled + || write_blob_cancelled + || gate_closed; if is_stopping { Level::INFO From e50b914a8eefa35a79a64bdb7715c0c102f94381 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Fri, 27 Jun 2025 18:39:00 +0200 Subject: [PATCH 18/20] compute_tools: support gRPC base backups in `compute_ctl` (#12244) ## Problem `compute_ctl` should support gRPC base backups. Requires #12111. Requires #12243. Touches #11926. ## Summary of changes Support `grpc://` connstrings for `compute_ctl` base backups. --- Cargo.lock | 6 +- Cargo.toml | 2 +- build-tools.Dockerfile | 1 + compute/compute-node.Dockerfile | 15 ++- compute_tools/Cargo.toml | 2 + compute_tools/src/compute.rs | 105 +++++++++++++++++---- pageserver/page_api/Cargo.toml | 3 +- pageserver/page_api/src/client.rs | 37 +++----- pageserver/page_api/src/model.rs | 4 +- pageserver/pagebench/src/cmd/basebackup.rs | 5 +- workspace_hack/Cargo.toml | 4 +- 11 files changed, 128 insertions(+), 56 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 7098711bb4..71e78243a6 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1316,6 +1316,7 @@ dependencies = [ "opentelemetry", "opentelemetry_sdk", "p256 0.13.2", + "pageserver_page_api", "postgres", "postgres_initdb", "postgres_versioninfo", @@ -1335,6 +1336,7 @@ dependencies = [ "tokio-postgres", "tokio-stream", "tokio-util", + "tonic 0.13.1", "tower 0.5.2", "tower-http", "tower-otel", @@ -4475,12 +4477,13 @@ dependencies = [ "bytes", "futures", "pageserver_api", - "postgres_ffi", + "postgres_ffi_types", "prost 0.13.5", "strum", "strum_macros", "thiserror 1.0.69", "tokio", + "tokio-util", "tonic 0.13.1", "tonic-build", "utils", @@ -8679,7 +8682,6 @@ dependencies = [ "num-iter", "num-rational", "num-traits", - "once_cell", "p256 0.13.2", "parquet", "prettyplease", diff --git a/Cargo.toml b/Cargo.toml index 857bc5d5d9..aeb7976b6c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -199,7 +199,7 @@ tokio-postgres-rustls = "0.12.0" tokio-rustls = { version = "0.26.0", default-features = false, features = ["tls12", "ring"]} tokio-stream = "0.1" tokio-tar = "0.3" -tokio-util = { version = "0.7.10", features = ["io", "rt"] } +tokio-util = { version = "0.7.10", features = ["io", "io-util", "rt"] } toml = "0.8" toml_edit = "0.22" tonic = { version = "0.13.1", default-features = false, features = ["channel", "codegen", "gzip", "prost", "router", "server", "tls-ring", "tls-native-roots", "zstd"] } diff --git a/build-tools.Dockerfile b/build-tools.Dockerfile index b70ced7886..14a52bd736 100644 --- a/build-tools.Dockerfile +++ b/build-tools.Dockerfile @@ -165,6 +165,7 @@ RUN curl -fsSL \ && rm sql_exporter.tar.gz # protobuf-compiler (protoc) +# Keep the version the same as in compute/compute-node.Dockerfile ENV PROTOC_VERSION=25.1 RUN curl -fsSL "https://github.com/protocolbuffers/protobuf/releases/download/v${PROTOC_VERSION}/protoc-${PROTOC_VERSION}-linux-$(uname -m | sed 's/aarch64/aarch_64/g').zip" -o "protoc.zip" \ && unzip -q protoc.zip -d protoc \ diff --git a/compute/compute-node.Dockerfile b/compute/compute-node.Dockerfile index 35ece73030..bce2a28b8b 100644 --- a/compute/compute-node.Dockerfile +++ b/compute/compute-node.Dockerfile @@ -115,6 +115,9 @@ ARG EXTENSIONS=all FROM $BASE_IMAGE_SHA AS build-deps ARG DEBIAN_VERSION +# Keep in sync with build-tools.Dockerfile +ENV PROTOC_VERSION=25.1 + # Use strict mode for bash to catch errors early SHELL ["/bin/bash", "-euo", "pipefail", "-c"] @@ -149,8 +152,14 @@ RUN case $DEBIAN_VERSION in \ libclang-dev \ jsonnet \ $VERSION_INSTALLS \ - && apt clean && rm -rf /var/lib/apt/lists/* && \ - useradd -ms /bin/bash nonroot -b /home + && apt clean && rm -rf /var/lib/apt/lists/* \ + && useradd -ms /bin/bash nonroot -b /home \ + # Install protoc from binary release, since Debian's versions are too old. + && curl -fsSL "https://github.com/protocolbuffers/protobuf/releases/download/v${PROTOC_VERSION}/protoc-${PROTOC_VERSION}-linux-$(uname -m | sed 's/aarch64/aarch_64/g').zip" -o "protoc.zip" \ + && unzip -q protoc.zip -d protoc \ + && mv protoc/bin/protoc /usr/local/bin/protoc \ + && mv protoc/include/google /usr/local/include/google \ + && rm -rf protoc.zip protoc ######################################################################################### # @@ -1170,7 +1179,7 @@ COPY --from=pgrag-src /ext-src/ /ext-src/ # Install it using virtual environment, because Python 3.11 (the default version on Debian 12 (Bookworm)) complains otherwise WORKDIR /ext-src/onnxruntime-src RUN apt update && apt install --no-install-recommends --no-install-suggests -y \ - python3 python3-pip python3-venv protobuf-compiler && \ + python3 python3-pip python3-venv && \ apt clean && rm -rf /var/lib/apt/lists/* && \ python3 -m venv venv && \ . venv/bin/activate && \ diff --git a/compute_tools/Cargo.toml b/compute_tools/Cargo.toml index a5879c4b7c..0a071c1ad1 100644 --- a/compute_tools/Cargo.toml +++ b/compute_tools/Cargo.toml @@ -38,6 +38,7 @@ once_cell.workspace = true opentelemetry.workspace = true opentelemetry_sdk.workspace = true p256 = { version = "0.13", features = ["pem"] } +pageserver_page_api.workspace = true postgres.workspace = true regex.workspace = true reqwest = { workspace = true, features = ["json"] } @@ -53,6 +54,7 @@ tokio = { workspace = true, features = ["rt", "rt-multi-thread"] } tokio-postgres.workspace = true tokio-util.workspace = true tokio-stream.workspace = true +tonic.workspace = true tower-otel.workspace = true tracing.workspace = true tracing-opentelemetry.workspace = true diff --git a/compute_tools/src/compute.rs b/compute_tools/src/compute.rs index cf558ee01a..7566626d57 100644 --- a/compute_tools/src/compute.rs +++ b/compute_tools/src/compute.rs @@ -1,4 +1,4 @@ -use anyhow::{Context, Result}; +use anyhow::{Context, Result, anyhow}; use chrono::{DateTime, Utc}; use compute_api::privilege::Privilege; use compute_api::responses::{ @@ -15,6 +15,7 @@ use itertools::Itertools; use nix::sys::signal::{Signal, kill}; use nix::unistd::Pid; use once_cell::sync::Lazy; +use pageserver_page_api::{self as page_api, BaseBackupCompression}; use postgres; use postgres::NoTls; use postgres::error::SqlState; @@ -35,6 +36,7 @@ use utils::id::{TenantId, TimelineId}; use utils::lsn::Lsn; use utils::measured_stream::MeasuredReader; use utils::pid_file; +use utils::shard::{ShardCount, ShardIndex, ShardNumber}; use crate::configurator::launch_configurator; use crate::disk_quota::set_disk_quota; @@ -995,13 +997,87 @@ impl ComputeNode { Ok(()) } - // Get basebackup from the libpq connection to pageserver using `connstr` and - // unarchive it to `pgdata` directory overriding all its previous content. + /// Fetches a basebackup from the Pageserver using the compute state's Pageserver connstring and + /// unarchives it to `pgdata` directory, replacing any existing contents. #[instrument(skip_all, fields(%lsn))] fn try_get_basebackup(&self, compute_state: &ComputeState, lsn: Lsn) -> Result<()> { let spec = compute_state.pspec.as_ref().expect("spec must be set"); - let start_time = Instant::now(); + // Detect the protocol scheme. If the URL doesn't have a scheme, assume libpq. + let shard0_connstr = spec.pageserver_connstr.split(',').next().unwrap(); + let scheme = match Url::parse(shard0_connstr) { + Ok(url) => url.scheme().to_lowercase().to_string(), + Err(url::ParseError::RelativeUrlWithoutBase) => "postgresql".to_string(), + Err(err) => return Err(anyhow!("invalid connstring URL: {err}")), + }; + + let started = Instant::now(); + let (connected, size) = match scheme.as_str() { + "postgresql" | "postgres" => self.try_get_basebackup_libpq(spec, lsn)?, + "grpc" => self.try_get_basebackup_grpc(spec, lsn)?, + scheme => return Err(anyhow!("unknown URL scheme {scheme}")), + }; + + let mut state = self.state.lock().unwrap(); + state.metrics.pageserver_connect_micros = + connected.duration_since(started).as_micros() as u64; + state.metrics.basebackup_bytes = size as u64; + state.metrics.basebackup_ms = started.elapsed().as_millis() as u64; + + Ok(()) + } + + /// Fetches a basebackup via gRPC. The connstring must use grpc://. Returns the timestamp when + /// the connection was established, and the (compressed) size of the basebackup. + fn try_get_basebackup_grpc(&self, spec: &ParsedSpec, lsn: Lsn) -> Result<(Instant, usize)> { + let shard0_connstr = spec + .pageserver_connstr + .split(',') + .next() + .unwrap() + .to_string(); + let shard_index = match spec.pageserver_connstr.split(',').count() as u8 { + 0 | 1 => ShardIndex::unsharded(), + count => ShardIndex::new(ShardNumber(0), ShardCount(count)), + }; + + let (reader, connected) = tokio::runtime::Handle::current().block_on(async move { + let mut client = page_api::Client::new( + shard0_connstr, + spec.tenant_id, + spec.timeline_id, + shard_index, + spec.storage_auth_token.clone(), + None, // NB: base backups use payload compression + ) + .await?; + let connected = Instant::now(); + let reader = client + .get_base_backup(page_api::GetBaseBackupRequest { + lsn: (lsn != Lsn(0)).then_some(lsn), + compression: BaseBackupCompression::Gzip, + replica: spec.spec.mode != ComputeMode::Primary, + full: false, + }) + .await?; + anyhow::Ok((reader, connected)) + })?; + + let mut reader = MeasuredReader::new(tokio_util::io::SyncIoBridge::new(reader)); + + // Set `ignore_zeros` so that unpack() reads the entire stream and doesn't just stop at the + // end-of-archive marker. If the server errors, the tar::Builder drop handler will write an + // end-of-archive marker before the error is emitted, and we would not see the error. + let mut ar = tar::Archive::new(flate2::read::GzDecoder::new(&mut reader)); + ar.set_ignore_zeros(true); + ar.unpack(&self.params.pgdata)?; + + Ok((connected, reader.get_byte_count())) + } + + /// Fetches a basebackup via libpq. The connstring must use postgresql://. Returns the timestamp + /// when the connection was established, and the (compressed) size of the basebackup. + fn try_get_basebackup_libpq(&self, spec: &ParsedSpec, lsn: Lsn) -> Result<(Instant, usize)> { let shard0_connstr = spec.pageserver_connstr.split(',').next().unwrap(); let mut config = postgres::Config::from_str(shard0_connstr)?; @@ -1015,16 +1091,14 @@ impl ComputeNode { } config.application_name("compute_ctl"); - if let Some(spec) = &compute_state.pspec { - config.options(&format!( - "-c neon.compute_mode={}", - spec.spec.mode.to_type_str() - )); - } + config.options(&format!( + "-c neon.compute_mode={}", + spec.spec.mode.to_type_str() + )); // Connect to pageserver let mut client = config.connect(NoTls)?; - let pageserver_connect_micros = start_time.elapsed().as_micros() as u64; + let connected = Instant::now(); let basebackup_cmd = match lsn { Lsn(0) => { @@ -1061,16 +1135,13 @@ impl ComputeNode { // Set `ignore_zeros` so that unpack() reads all the Copy data and // doesn't stop at the end-of-archive marker. Otherwise, if the server // sends an Error after finishing the tarball, we will not notice it. + // The tar::Builder drop handler will write an end-of-archive marker + // before emitting the error, and we would not see it otherwise. let mut ar = tar::Archive::new(flate2::read::GzDecoder::new(&mut bufreader)); ar.set_ignore_zeros(true); ar.unpack(&self.params.pgdata)?; - // Report metrics - let mut state = self.state.lock().unwrap(); - state.metrics.pageserver_connect_micros = pageserver_connect_micros; - state.metrics.basebackup_bytes = measured_reader.get_byte_count() as u64; - state.metrics.basebackup_ms = start_time.elapsed().as_millis() as u64; - Ok(()) + Ok((connected, measured_reader.get_byte_count())) } // Gets the basebackup in a retry loop diff --git a/pageserver/page_api/Cargo.toml b/pageserver/page_api/Cargo.toml index c5283c2b09..42ee9b50e9 100644 --- a/pageserver/page_api/Cargo.toml +++ b/pageserver/page_api/Cargo.toml @@ -9,12 +9,13 @@ anyhow.workspace = true bytes.workspace = true futures.workspace = true pageserver_api.workspace = true -postgres_ffi.workspace = true +postgres_ffi_types.workspace = true prost.workspace = true strum.workspace = true strum_macros.workspace = true thiserror.workspace = true tokio.workspace = true +tokio-util.workspace = true tonic.workspace = true utils.workspace = true workspace_hack.workspace = true diff --git a/pageserver/page_api/src/client.rs b/pageserver/page_api/src/client.rs index 71d539ab91..4b456787d2 100644 --- a/pageserver/page_api/src/client.rs +++ b/pageserver/page_api/src/client.rs @@ -1,8 +1,7 @@ -use std::convert::TryInto; - -use bytes::Bytes; -use futures::TryStreamExt; -use futures::{Stream, StreamExt}; +use anyhow::Result; +use futures::{Stream, StreamExt as _, TryStreamExt as _}; +use tokio::io::AsyncRead; +use tokio_util::io::StreamReader; use tonic::metadata::AsciiMetadataValue; use tonic::metadata::errors::InvalidMetadataValue; use tonic::transport::Channel; @@ -12,8 +11,6 @@ use utils::id::TenantId; use utils::id::TimelineId; use utils::shard::ShardIndex; -use anyhow::Result; - use crate::model; use crate::proto; @@ -69,6 +66,7 @@ impl tonic::service::Interceptor for AuthInterceptor { Ok(req) } } + #[derive(Clone)] pub struct Client { client: proto::PageServiceClient< @@ -120,22 +118,15 @@ impl Client { pub async fn get_base_backup( &mut self, req: model::GetBaseBackupRequest, - ) -> Result> + 'static, tonic::Status> { - let proto_req = proto::GetBaseBackupRequest::from(req); - - let response_stream: Streaming = - self.client.get_base_backup(proto_req).await?.into_inner(); - - // TODO: Consider dechunking internally - let domain_stream = response_stream.map(|chunk_res| { - chunk_res.and_then(|proto_chunk| { - proto_chunk.try_into().map_err(|e| { - tonic::Status::internal(format!("Failed to convert response chunk: {e}")) - }) - }) - }); - - Ok(domain_stream) + ) -> Result, tonic::Status> { + let req = proto::GetBaseBackupRequest::from(req); + let chunks = self.client.get_base_backup(req).await?.into_inner(); + let reader = StreamReader::new( + chunks + .map_ok(|resp| resp.chunk) + .map_err(std::io::Error::other), + ); + Ok(reader) } /// Returns the total size of a database, as # of bytes. diff --git a/pageserver/page_api/src/model.rs b/pageserver/page_api/src/model.rs index 1ca89b4870..0493f79781 100644 --- a/pageserver/page_api/src/model.rs +++ b/pageserver/page_api/src/model.rs @@ -18,8 +18,8 @@ use std::fmt::Display; use bytes::Bytes; -use postgres_ffi::Oid; -// TODO: split out Lsn, RelTag, SlruKind, Oid and other basic types to a separate crate, to avoid +use postgres_ffi_types::Oid; +// TODO: split out Lsn, RelTag, SlruKind and other basic types to a separate crate, to avoid // pulling in all of their other crate dependencies when building the client. use utils::lsn::Lsn; diff --git a/pageserver/pagebench/src/cmd/basebackup.rs b/pageserver/pagebench/src/cmd/basebackup.rs index 4111d09f92..4b7a70504a 100644 --- a/pageserver/pagebench/src/cmd/basebackup.rs +++ b/pageserver/pagebench/src/cmd/basebackup.rs @@ -355,9 +355,6 @@ impl Client for GrpcClient { full: false, compression: self.compression, }; - let stream = self.inner.get_base_backup(req).await?; - Ok(Box::pin(StreamReader::new( - stream.map_err(std::io::Error::other), - ))) + Ok(Box::pin(self.inner.get_base_backup(req).await?)) } } diff --git a/workspace_hack/Cargo.toml b/workspace_hack/Cargo.toml index b74df50f86..e9a77ca2d6 100644 --- a/workspace_hack/Cargo.toml +++ b/workspace_hack/Cargo.toml @@ -68,7 +68,6 @@ num-integer = { version = "0.1", features = ["i128"] } num-iter = { version = "0.1", default-features = false, features = ["i128", "std"] } num-rational = { version = "0.4", default-features = false, features = ["num-bigint-std", "std"] } num-traits = { version = "0.2", features = ["i128", "libm"] } -once_cell = { version = "1" } p256 = { version = "0.13", features = ["jwk"] } parquet = { version = "53", default-features = false, features = ["zstd"] } prost = { version = "0.13", features = ["no-recursion-limit", "prost-derive"] } @@ -97,7 +96,7 @@ time = { version = "0.3", features = ["macros", "serde-well-known"] } tokio = { version = "1", features = ["full", "test-util"] } tokio-rustls = { version = "0.26", default-features = false, features = ["logging", "ring", "tls12"] } tokio-stream = { version = "0.1", features = ["net"] } -tokio-util = { version = "0.7", features = ["codec", "compat", "io", "rt"] } +tokio-util = { version = "0.7", features = ["codec", "compat", "io-util", "rt"] } toml_edit = { version = "0.22", features = ["serde"] } tower = { version = "0.5", default-features = false, features = ["balance", "buffer", "limit", "log"] } tracing = { version = "0.1", features = ["log"] } @@ -134,7 +133,6 @@ num-integer = { version = "0.1", features = ["i128"] } num-iter = { version = "0.1", default-features = false, features = ["i128", "std"] } num-rational = { version = "0.4", default-features = false, features = ["num-bigint-std", "std"] } num-traits = { version = "0.2", features = ["i128", "libm"] } -once_cell = { version = "1" } parquet = { version = "53", default-features = false, features = ["zstd"] } prettyplease = { version = "0.2", default-features = false, features = ["verbatim"] } proc-macro2 = { version = "1" } From 47553dbaf946de9cdab759c220ec56dffa5f82cd Mon Sep 17 00:00:00 2001 From: Dmitrii Kovalkov <34828390+DimasKovas@users.noreply.github.com> Date: Sat, 28 Jun 2025 16:59:29 +0400 Subject: [PATCH 19/20] neon_local: set timeline_safekeeper_count if we have less than 3 safekeepers (#12378) ## Problem - Closes: https://github.com/neondatabase/neon/issues/12298 ## Summary of changes - Set `timeline_safekeeper_count` in `neon_local` if we have less than 3 safekeepers - Remove `cfg!(feature = "testing")` code from `safekeepers_for_new_timeline` - Change `timeline_safekeeper_count` type to `usize` --- control_plane/src/local_env.rs | 2 +- control_plane/src/storage_controller.rs | 8 +++++++- storage_controller/src/main.rs | 4 ++-- storage_controller/src/service.rs | 2 +- storage_controller/src/service/safekeeper_service.rs | 9 +-------- 5 files changed, 12 insertions(+), 13 deletions(-) diff --git a/control_plane/src/local_env.rs b/control_plane/src/local_env.rs index 16cd2d8c08..d0611113e8 100644 --- a/control_plane/src/local_env.rs +++ b/control_plane/src/local_env.rs @@ -212,7 +212,7 @@ pub struct NeonStorageControllerConf { pub use_local_compute_notifications: bool, - pub timeline_safekeeper_count: Option, + pub timeline_safekeeper_count: Option, pub posthog_config: Option, diff --git a/control_plane/src/storage_controller.rs b/control_plane/src/storage_controller.rs index dea7ae2ccf..bb83a6319c 100644 --- a/control_plane/src/storage_controller.rs +++ b/control_plane/src/storage_controller.rs @@ -638,7 +638,13 @@ impl StorageController { args.push("--timelines-onto-safekeepers".to_string()); } - if let Some(sk_cnt) = self.config.timeline_safekeeper_count { + // neon_local is used in test environments where we often have less than 3 safekeepers. + if self.config.timeline_safekeeper_count.is_some() || self.env.safekeepers.len() < 3 { + let sk_cnt = self + .config + .timeline_safekeeper_count + .unwrap_or(self.env.safekeepers.len()); + args.push(format!("--timeline-safekeeper-count={sk_cnt}")); } diff --git a/storage_controller/src/main.rs b/storage_controller/src/main.rs index d1c2858d6f..752262b65e 100644 --- a/storage_controller/src/main.rs +++ b/storage_controller/src/main.rs @@ -216,8 +216,8 @@ struct Cli { /// Number of safekeepers to choose for a timeline when creating it. /// Safekeepers will be choosen from different availability zones. /// This option exists primarily for testing purposes. - #[arg(long, default_value = "3", value_parser = clap::value_parser!(i64).range(1..))] - timeline_safekeeper_count: i64, + #[arg(long, default_value = "3", value_parser = clap::builder::RangedU64ValueParser::::new().range(1..))] + timeline_safekeeper_count: usize, /// When set, actively checks and initiates heatmap downloads/uploads during reconciliation. /// This speed up migrations by avoiding the default wait for the heatmap download interval. diff --git a/storage_controller/src/service.rs b/storage_controller/src/service.rs index b4dfd01249..19bb0f8671 100644 --- a/storage_controller/src/service.rs +++ b/storage_controller/src/service.rs @@ -471,7 +471,7 @@ pub struct Config { /// Number of safekeepers to choose for a timeline when creating it. /// Safekeepers will be choosen from different availability zones. - pub timeline_safekeeper_count: i64, + pub timeline_safekeeper_count: usize, /// PostHog integration config pub posthog_config: Option, diff --git a/storage_controller/src/service/safekeeper_service.rs b/storage_controller/src/service/safekeeper_service.rs index fec81fb661..92d15f3fca 100644 --- a/storage_controller/src/service/safekeeper_service.rs +++ b/storage_controller/src/service/safekeeper_service.rs @@ -1,4 +1,3 @@ -use std::cmp::max; use std::collections::HashSet; use std::str::FromStr; use std::sync::Arc; @@ -654,13 +653,7 @@ impl Service { ) }); // Number of safekeepers in different AZs we are looking for - let mut wanted_count = self.config.timeline_safekeeper_count as usize; - // TODO(diko): remove this when `timeline_safekeeper_count` option is in the release - // branch and is specified in tests/neon_local config. - if cfg!(feature = "testing") && all_safekeepers.len() < wanted_count { - // In testing mode, we can have less safekeepers than the config says - wanted_count = max(all_safekeepers.len(), 1); - } + let wanted_count = self.config.timeline_safekeeper_count; let mut sks = Vec::new(); let mut azs = HashSet::new(); From 9bb4688c541fd66b3a342ac4f1cd0784bd807fbd Mon Sep 17 00:00:00 2001 From: Aleksandr Sarantsev <99037063+ephemeralsad@users.noreply.github.com> Date: Mon, 30 Jun 2025 09:41:05 +0400 Subject: [PATCH 20/20] storcon: Remove testing feature from kick_secondary_downloads (#12383) ## Problem Some of the design decisions in PR #12256 were influenced by the requirements of consistency tests. These decisions introduced intermediate logic that is no longer needed and should be cleaned up. ## Summary of Changes - Remove the `feature("testing")` flag related to `kick_secondary_download`. - Set the default value of `kick_secondary_download` back to false, reflecting the intended production behavior. Co-authored-by: Aleksandr Sarantsev --- storage_controller/src/main.rs | 8 ++------ storage_controller/src/service.rs | 4 +--- test_runner/fixtures/neon_fixtures.py | 2 +- 3 files changed, 4 insertions(+), 10 deletions(-) diff --git a/storage_controller/src/main.rs b/storage_controller/src/main.rs index 752262b65e..2a851dc25b 100644 --- a/storage_controller/src/main.rs +++ b/storage_controller/src/main.rs @@ -6,9 +6,7 @@ use std::time::Duration; use anyhow::{Context, anyhow}; use camino::Utf8PathBuf; -#[cfg(feature = "testing")] -use clap::ArgAction; -use clap::Parser; +use clap::{ArgAction, Parser}; use futures::future::OptionFuture; use http_utils::tls_certs::ReloadingCertificateResolver; use hyper0::Uri; @@ -222,8 +220,7 @@ struct Cli { /// When set, actively checks and initiates heatmap downloads/uploads during reconciliation. /// This speed up migrations by avoiding the default wait for the heatmap download interval. /// Primarily useful for testing to reduce test execution time. - #[cfg(feature = "testing")] - #[arg(long, default_value = "true", action=ArgAction::Set)] + #[arg(long, default_value = "false", action=ArgAction::Set)] kick_secondary_downloads: bool, } @@ -472,7 +469,6 @@ async fn async_main() -> anyhow::Result<()> { use_local_compute_notifications: args.use_local_compute_notifications, timeline_safekeeper_count: args.timeline_safekeeper_count, posthog_config: posthog_config.clone(), - #[cfg(feature = "testing")] kick_secondary_downloads: args.kick_secondary_downloads, }; diff --git a/storage_controller/src/service.rs b/storage_controller/src/service.rs index 19bb0f8671..75ce7bc37b 100644 --- a/storage_controller/src/service.rs +++ b/storage_controller/src/service.rs @@ -476,7 +476,7 @@ pub struct Config { /// PostHog integration config pub posthog_config: Option, - #[cfg(feature = "testing")] + /// When set, actively checks and initiates heatmap downloads/uploads. pub kick_secondary_downloads: bool, } @@ -8364,7 +8364,6 @@ impl Service { "Skipping migration of {tenant_shard_id} to {node} because secondary isn't ready: {progress:?}" ); - #[cfg(feature = "testing")] if progress.heatmap_mtime.is_none() { // No heatmap might mean the attached location has never uploaded one, or that // the secondary download hasn't happened yet. This is relatively unusual in the field, @@ -8389,7 +8388,6 @@ impl Service { /// happens on multi-minute timescales in the field, which is fine because optimisation is meant /// to be a lazy background thing. However, when testing, it is not practical to wait around, so /// we have this helper to move things along faster. - #[cfg(feature = "testing")] async fn kick_secondary_download(&self, tenant_shard_id: TenantShardId) { if !self.config.kick_secondary_downloads { // No-op if kick_secondary_downloads functionaliuty is not configured diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index 4eb85119ca..48c6597c7c 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -453,7 +453,7 @@ class NeonEnvBuilder: pageserver_get_vectored_concurrent_io: str | None = None, pageserver_tracing_config: PageserverTracingConfig | None = None, pageserver_import_config: PageserverImportConfig | None = None, - storcon_kick_secondary_downloads: bool | None = None, + storcon_kick_secondary_downloads: bool | None = True, ): self.repo_dir = repo_dir self.rust_log_override = rust_log_override