From 9a43c04a19c8577466dd6e992991a2aaeec6e556 Mon Sep 17 00:00:00 2001 From: Arseny Sher Date: Mon, 1 Jan 2024 14:38:08 +0300 Subject: [PATCH 01/35] compute_ctl: kill postgres and sync-safekeeprs on exit. Otherwise they are left orphaned when compute_ctl is terminated with a signal. It was invisible most of the time because normally neon_local or k8s kills postgres directly and then compute_ctl finishes gracefully. However, in some tests compute_ctl gets stuck waiting for sync-safekeepers which intentionally never ends because safekeepers are offline, and we want to stop compute_ctl without leaving orphanes behind. This is a quite rough approach which doesn't wait for children termination. A better way would be to convert compute_ctl to async which would make waiting easy. --- Cargo.lock | 2 ++ compute_tools/Cargo.toml | 2 ++ compute_tools/src/bin/compute_ctl.rs | 32 +++++++++++++++++++++++++++- compute_tools/src/compute.rs | 8 +++++++ control_plane/src/endpoint.rs | 18 ++++++++++++---- 5 files changed, 57 insertions(+), 5 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index abd87dc0da..8e0ad7c8ee 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1161,6 +1161,7 @@ dependencies = [ "flate2", "futures", "hyper", + "nix 0.26.2", "notify", "num_cpus", "opentelemetry", @@ -1171,6 +1172,7 @@ dependencies = [ "rust-ini", "serde", "serde_json", + "signal-hook", "tar", "tokio", "tokio-postgres", diff --git a/compute_tools/Cargo.toml b/compute_tools/Cargo.toml index 142fa08495..759a117ee9 100644 --- a/compute_tools/Cargo.toml +++ b/compute_tools/Cargo.toml @@ -13,6 +13,7 @@ clap.workspace = true flate2.workspace = true futures.workspace = true hyper = { workspace = true, features = ["full"] } +nix.workspace = true notify.workspace = true num_cpus.workspace = true opentelemetry.workspace = true @@ -20,6 +21,7 @@ postgres.workspace = true regex.workspace = true serde.workspace = true serde_json.workspace = true +signal-hook.workspace = true tar.workspace = true reqwest = { workspace = true, features = ["json"] } tokio = { workspace = true, features = ["rt", "rt-multi-thread"] } diff --git a/compute_tools/src/bin/compute_ctl.rs b/compute_tools/src/bin/compute_ctl.rs index 436db59088..eb1d746f04 100644 --- a/compute_tools/src/bin/compute_ctl.rs +++ b/compute_tools/src/bin/compute_ctl.rs @@ -40,18 +40,22 @@ use std::collections::HashMap; use std::fs::File; use std::path::Path; use std::process::exit; +use std::sync::atomic::Ordering; use std::sync::{mpsc, Arc, Condvar, Mutex, RwLock}; use std::{thread, time::Duration}; use anyhow::{Context, Result}; use chrono::Utc; use clap::Arg; +use nix::sys::signal::{kill, Signal}; +use signal_hook::consts::{SIGQUIT, SIGTERM}; +use signal_hook::{consts::SIGINT, iterator::Signals}; use tracing::{error, info}; use url::Url; use compute_api::responses::ComputeStatus; -use compute_tools::compute::{ComputeNode, ComputeState, ParsedSpec}; +use compute_tools::compute::{ComputeNode, ComputeState, ParsedSpec, PG_PID, SYNC_SAFEKEEPERS_PID}; use compute_tools::configurator::launch_configurator; use compute_tools::extension_server::get_pg_version; use compute_tools::http::api::launch_http_server; @@ -67,6 +71,13 @@ const BUILD_TAG_DEFAULT: &str = "latest"; fn main() -> Result<()> { init_tracing_and_logging(DEFAULT_LOG_LEVEL)?; + let mut signals = Signals::new([SIGINT, SIGTERM, SIGQUIT])?; + thread::spawn(move || { + for sig in signals.forever() { + handle_exit_signal(sig); + } + }); + let build_tag = option_env!("BUILD_TAG") .unwrap_or(BUILD_TAG_DEFAULT) .to_string(); @@ -346,6 +357,7 @@ fn main() -> Result<()> { let ecode = pg .wait() .expect("failed to start waiting on Postgres process"); + PG_PID.store(0, Ordering::SeqCst); info!("Postgres exited with code {}, shutting down", ecode); exit_code = ecode.code() } @@ -519,6 +531,24 @@ fn cli() -> clap::Command { ) } +/// When compute_ctl is killed, send also termination signal to sync-safekeepers +/// to prevent leakage. TODO: it is better to convert compute_ctl to async and +/// wait for termination which would be easy then. +fn handle_exit_signal(sig: i32) { + info!("received {sig} termination signal"); + let ss_pid = SYNC_SAFEKEEPERS_PID.load(Ordering::SeqCst); + if ss_pid != 0 { + let ss_pid = nix::unistd::Pid::from_raw(ss_pid as i32); + kill(ss_pid, Signal::SIGTERM).ok(); + } + let pg_pid = PG_PID.load(Ordering::SeqCst); + if pg_pid != 0 { + let pg_pid = nix::unistd::Pid::from_raw(pg_pid as i32); + kill(pg_pid, Signal::SIGTERM).ok(); + } + exit(1); +} + #[test] fn verify_cli() { cli().debug_assert() diff --git a/compute_tools/src/compute.rs b/compute_tools/src/compute.rs index cd7be0520e..13701b7378 100644 --- a/compute_tools/src/compute.rs +++ b/compute_tools/src/compute.rs @@ -6,6 +6,8 @@ use std::os::unix::fs::PermissionsExt; use std::path::Path; use std::process::{Command, Stdio}; use std::str::FromStr; +use std::sync::atomic::AtomicU32; +use std::sync::atomic::Ordering; use std::sync::{Condvar, Mutex, RwLock}; use std::thread; use std::time::Instant; @@ -34,6 +36,9 @@ use crate::spec::*; use crate::sync_sk::{check_if_synced, ping_safekeeper}; use crate::{config, extension_server}; +pub static SYNC_SAFEKEEPERS_PID: AtomicU32 = AtomicU32::new(0); +pub static PG_PID: AtomicU32 = AtomicU32::new(0); + /// Compute node info shared across several `compute_ctl` threads. pub struct ComputeNode { // Url type maintains proper escaping @@ -501,6 +506,7 @@ impl ComputeNode { .stdout(Stdio::piped()) .spawn() .expect("postgres --sync-safekeepers failed to start"); + SYNC_SAFEKEEPERS_PID.store(sync_handle.id(), Ordering::SeqCst); // `postgres --sync-safekeepers` will print all log output to stderr and // final LSN to stdout. So we pipe only stdout, while stderr will be automatically @@ -508,6 +514,7 @@ impl ComputeNode { let sync_output = sync_handle .wait_with_output() .expect("postgres --sync-safekeepers failed"); + SYNC_SAFEKEEPERS_PID.store(0, Ordering::SeqCst); if !sync_output.status.success() { anyhow::bail!( @@ -662,6 +669,7 @@ impl ComputeNode { }) .spawn() .expect("cannot start postgres process"); + PG_PID.store(pg.id(), Ordering::SeqCst); wait_for_postgres(&mut pg, pgdata_path)?; diff --git a/control_plane/src/endpoint.rs b/control_plane/src/endpoint.rs index 55b66742ca..3d5dfd6311 100644 --- a/control_plane/src/endpoint.rs +++ b/control_plane/src/endpoint.rs @@ -46,6 +46,8 @@ use std::time::Duration; use anyhow::{anyhow, bail, Context, Result}; use compute_api::spec::RemoteExtSpec; +use nix::sys::signal::kill; +use nix::sys::signal::Signal; use serde::{Deserialize, Serialize}; use utils::id::{NodeId, TenantId, TimelineId}; @@ -439,11 +441,14 @@ impl Endpoint { Ok(()) } - fn wait_for_compute_ctl_to_exit(&self) -> Result<()> { + fn wait_for_compute_ctl_to_exit(&self, send_sigterm: bool) -> Result<()> { // TODO use background_process::stop_process instead let pidfile_path = self.endpoint_path().join("compute_ctl.pid"); let pid: u32 = std::fs::read_to_string(pidfile_path)?.parse()?; let pid = nix::unistd::Pid::from_raw(pid as i32); + if send_sigterm { + kill(pid, Signal::SIGTERM).ok(); + } crate::background_process::wait_until_stopped("compute_ctl", pid)?; Ok(()) } @@ -733,10 +738,15 @@ impl Endpoint { &None, )?; - // Also wait for the compute_ctl process to die. It might have some cleanup - // work to do after postgres stops, like syncing safekeepers, etc. + // Also wait for the compute_ctl process to die. It might have some + // cleanup work to do after postgres stops, like syncing safekeepers, + // etc. // - self.wait_for_compute_ctl_to_exit()?; + // If destroying, send it SIGTERM before waiting. Sometimes we do *not* + // want this cleanup: tests intentionally do stop when majority of + // safekeepers is down, so sync-safekeepers would hang otherwise. This + // could be a separate flag though. + self.wait_for_compute_ctl_to_exit(destroy)?; if destroy { println!( "Destroying postgres data directory '{}'", From 90ef48aab8d66e1fc3ad4a8cb187f13b8426bbb0 Mon Sep 17 00:00:00 2001 From: Arseny Sher Date: Mon, 1 Jan 2024 14:43:44 +0300 Subject: [PATCH 02/35] Fix safekeeper START_REPLICATION (term=n). It was giving WAL only up to commit_lsn instead of flush_lsn, so recovery of uncommitted WAL since cdb08f03 hanged. Add test for this. --- safekeeper/src/send_wal.rs | 11 +---- .../regress/test_wal_acceptor_async.py | 40 +++++++++++++++++++ 2 files changed, 42 insertions(+), 9 deletions(-) diff --git a/safekeeper/src/send_wal.rs b/safekeeper/src/send_wal.rs index 44f14f8c7e..70590a0f95 100644 --- a/safekeeper/src/send_wal.rs +++ b/safekeeper/src/send_wal.rs @@ -391,15 +391,8 @@ impl SafekeeperPostgresHandler { // application_name: give only committed WAL (used by pageserver) or all // existing WAL (up to flush_lsn, used by walproposer or peer recovery). // The second case is always driven by a consensus leader which term - // must generally be also supplied. However we're sloppy to do this in - // walproposer recovery which will be removed soon. So TODO is to make - // it not Option'al then. - // - // Fetching WAL without term in recovery creates a small risk of this - // WAL getting concurrently garbaged if another compute rises which - // collects majority and starts fixing log on this safekeeper itself. - // That's ok as (old) proposer will never be able to commit such WAL. - let end_watch = if self.is_walproposer_recovery() { + // must be supplied. + let end_watch = if term.is_some() { EndWatch::Flush(tli.get_term_flush_lsn_watch_rx()) } else { EndWatch::Commit(tli.get_commit_lsn_watch_rx()) diff --git a/test_runner/regress/test_wal_acceptor_async.py b/test_runner/regress/test_wal_acceptor_async.py index feab7e605b..77d67cd63a 100644 --- a/test_runner/regress/test_wal_acceptor_async.py +++ b/test_runner/regress/test_wal_acceptor_async.py @@ -475,6 +475,46 @@ def test_unavailability(neon_env_builder: NeonEnvBuilder): asyncio.run(run_unavailability(env, endpoint)) +async def run_recovery_uncommitted(env: NeonEnv): + (sk1, sk2, _) = env.safekeepers + + env.neon_cli.create_branch("test_recovery_uncommitted") + ep = env.endpoints.create_start("test_recovery_uncommitted") + ep.safe_psql("create table t(key int, value text)") + ep.safe_psql("insert into t select generate_series(1, 100), 'payload'") + + # insert with only one safekeeper up to create tail of flushed but not committed WAL + sk1.stop() + sk2.stop() + conn = await ep.connect_async() + # query should hang, so execute in separate task + bg_query = asyncio.create_task( + conn.execute("insert into t select generate_series(1, 2000), 'payload'") + ) + sleep_sec = 2 + await asyncio.sleep(sleep_sec) + # it must still be not finished + assert not bg_query.done() + # note: destoy will kill compute_ctl, preventing it waiting for hanging sync-safekeepers. + ep.stop_and_destroy() + + # Start one of sks to make quorum online plus compute and ensure they can + # sync. + sk2.start() + ep = env.endpoints.create_start( + "test_recovery_uncommitted", + ) + ep.safe_psql("insert into t select generate_series(1, 2000), 'payload'") + + +# Test pulling uncommitted WAL (up to flush_lsn) during recovery. +def test_recovery_uncommitted(neon_env_builder: NeonEnvBuilder): + neon_env_builder.num_safekeepers = 3 + env = neon_env_builder.init_start() + + asyncio.run(run_recovery_uncommitted(env)) + + @dataclass class RaceConditionTest: iteration: int From dbd36e40dcc60fc9ced780c4ca0161a9c85fdc06 Mon Sep 17 00:00:00 2001 From: Arseny Sher Date: Mon, 1 Jan 2024 22:33:27 +0300 Subject: [PATCH 03/35] Move failpoint support code to utils. To enable them in safekeeper as well. --- Cargo.lock | 1 + libs/pageserver_api/src/models.rs | 13 ----- libs/utils/Cargo.toml | 7 +++ .../utils}/src/failpoint_support.rs | 57 ++++++++++++++++++- libs/utils/src/lib.rs | 2 + pageserver/src/bin/pageserver.rs | 3 +- pageserver/src/http/routes.rs | 32 +---------- pageserver/src/lib.rs | 2 - pageserver/src/tenant.rs | 9 ++- pageserver/src/walingest.rs | 5 +- 10 files changed, 74 insertions(+), 57 deletions(-) rename {pageserver => libs/utils}/src/failpoint_support.rs (61%) diff --git a/Cargo.lock b/Cargo.lock index 8e0ad7c8ee..73cb83d3a7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5880,6 +5880,7 @@ dependencies = [ "chrono", "const_format", "criterion", + "fail", "futures", "heapless", "hex", diff --git a/libs/pageserver_api/src/models.rs b/libs/pageserver_api/src/models.rs index be41b610b8..dea925b468 100644 --- a/libs/pageserver_api/src/models.rs +++ b/libs/pageserver_api/src/models.rs @@ -557,19 +557,6 @@ pub enum DownloadRemoteLayersTaskState { ShutDown, } -pub type ConfigureFailpointsRequest = Vec; - -/// Information for configuring a single fail point -#[derive(Debug, Serialize, Deserialize)] -pub struct FailpointConfig { - /// Name of the fail point - pub name: String, - /// List of actions to take, using the format described in `fail::cfg` - /// - /// We also support `actions = "exit"` to cause the fail point to immediately exit. - pub actions: String, -} - #[derive(Debug, Serialize, Deserialize)] pub struct TimelineGcRequest { pub gc_horizon: Option, diff --git a/libs/utils/Cargo.toml b/libs/utils/Cargo.toml index af0414daa2..706b7a3187 100644 --- a/libs/utils/Cargo.toml +++ b/libs/utils/Cargo.toml @@ -4,6 +4,12 @@ version = "0.1.0" edition.workspace = true license.workspace = true +[features] +default = [] +# Enables test-only APIs, incuding failpoints. In particular, enables the `fail_point!` macro, +# which adds some runtime cost to run tests on outage conditions +testing = ["fail/failpoints"] + [dependencies] arc-swap.workspace = true sentry.workspace = true @@ -16,6 +22,7 @@ chrono.workspace = true heapless.workspace = true hex = { workspace = true, features = ["serde"] } hyper = { workspace = true, features = ["full"] } +fail.workspace = true futures = { workspace = true} jsonwebtoken.workspace = true nix.workspace = true diff --git a/pageserver/src/failpoint_support.rs b/libs/utils/src/failpoint_support.rs similarity index 61% rename from pageserver/src/failpoint_support.rs rename to libs/utils/src/failpoint_support.rs index 2190eba18a..5ec532e2a6 100644 --- a/pageserver/src/failpoint_support.rs +++ b/libs/utils/src/failpoint_support.rs @@ -1,3 +1,14 @@ +//! Failpoint support code shared between pageserver and safekeepers. + +use crate::http::{ + error::ApiError, + json::{json_request, json_response}, +}; +use hyper::{Body, Request, Response, StatusCode}; +use serde::{Deserialize, Serialize}; +use tokio_util::sync::CancellationToken; +use tracing::*; + /// use with fail::cfg("$name", "return(2000)") /// /// The effect is similar to a "sleep(2000)" action, i.e. we sleep for the @@ -25,7 +36,7 @@ pub use __failpoint_sleep_millis_async as sleep_millis_async; // Helper function used by the macro. (A function has nicer scoping so we // don't need to decorate everything with "::") #[doc(hidden)] -pub(crate) async fn failpoint_sleep_helper(name: &'static str, duration_str: String) { +pub async fn failpoint_sleep_helper(name: &'static str, duration_str: String) { let millis = duration_str.parse::().unwrap(); let d = std::time::Duration::from_millis(millis); @@ -71,7 +82,7 @@ pub fn init() -> fail::FailScenario<'static> { scenario } -pub(crate) fn apply_failpoint(name: &str, actions: &str) -> Result<(), String> { +pub fn apply_failpoint(name: &str, actions: &str) -> Result<(), String> { if actions == "exit" { fail::cfg_callback(name, exit_failpoint) } else { @@ -84,3 +95,45 @@ fn exit_failpoint() { tracing::info!("Exit requested by failpoint"); std::process::exit(1); } + +pub type ConfigureFailpointsRequest = Vec; + +/// Information for configuring a single fail point +#[derive(Debug, Serialize, Deserialize)] +pub struct FailpointConfig { + /// Name of the fail point + pub name: String, + /// List of actions to take, using the format described in `fail::cfg` + /// + /// We also support `actions = "exit"` to cause the fail point to immediately exit. + pub actions: String, +} + +/// Configure failpoints through http. +pub async fn failpoints_handler( + mut request: Request, + _cancel: CancellationToken, +) -> Result, ApiError> { + if !fail::has_failpoints() { + return Err(ApiError::BadRequest(anyhow::anyhow!( + "Cannot manage failpoints because storage was compiled without failpoints support" + ))); + } + + let failpoints: ConfigureFailpointsRequest = json_request(&mut request).await?; + for fp in failpoints { + info!("cfg failpoint: {} {}", fp.name, fp.actions); + + // We recognize one extra "action" that's not natively recognized + // by the failpoints crate: exit, to immediately kill the process + let cfg_result = apply_failpoint(&fp.name, &fp.actions); + + if let Err(err_msg) = cfg_result { + return Err(ApiError::BadRequest(anyhow::anyhow!( + "Failed to configure failpoints: {err_msg}" + ))); + } + } + + json_response(StatusCode::OK, ()) +} diff --git a/libs/utils/src/lib.rs b/libs/utils/src/lib.rs index bb6c848bf4..9e9b0adfe5 100644 --- a/libs/utils/src/lib.rs +++ b/libs/utils/src/lib.rs @@ -83,6 +83,8 @@ pub mod timeout; pub mod sync; +pub mod failpoint_support; + /// This is a shortcut to embed git sha into binaries and avoid copying the same build script to all packages /// /// we have several cases: diff --git a/pageserver/src/bin/pageserver.rs b/pageserver/src/bin/pageserver.rs index f65c4f4580..621ad050f4 100644 --- a/pageserver/src/bin/pageserver.rs +++ b/pageserver/src/bin/pageserver.rs @@ -31,6 +31,7 @@ use pageserver::{ virtual_file, }; use postgres_backend::AuthType; +use utils::failpoint_support; use utils::logging::TracingErrorLayerEnablement; use utils::signals::ShutdownSignals; use utils::{ @@ -126,7 +127,7 @@ fn main() -> anyhow::Result<()> { } // Initialize up failpoints support - let scenario = pageserver::failpoint_support::init(); + let scenario = failpoint_support::init(); // Basic initialization of things that don't change after startup virtual_file::init(conf.max_file_descriptors); diff --git a/pageserver/src/http/routes.rs b/pageserver/src/http/routes.rs index 11a3a2c872..157e6b4e3e 100644 --- a/pageserver/src/http/routes.rs +++ b/pageserver/src/http/routes.rs @@ -25,6 +25,7 @@ use tenant_size_model::{SizeResult, StorageModel}; use tokio_util::sync::CancellationToken; use tracing::*; use utils::auth::JwtAuth; +use utils::failpoint_support::failpoints_handler; use utils::http::endpoint::request_span; use utils::http::json::json_request_or_empty_body; use utils::http::request::{get_request_param, must_get_query_param, parse_query_param}; @@ -66,9 +67,6 @@ use utils::{ lsn::Lsn, }; -// Imports only used for testing APIs -use pageserver_api::models::ConfigureFailpointsRequest; - // For APIs that require an Active tenant, how long should we block waiting for that state? // This is not functionally necessary (clients will retry), but avoids generating a lot of // failed API calls while tenants are activating. @@ -1293,34 +1291,6 @@ async fn handle_tenant_break( json_response(StatusCode::OK, ()) } -async fn failpoints_handler( - mut request: Request, - _cancel: CancellationToken, -) -> Result, ApiError> { - if !fail::has_failpoints() { - return Err(ApiError::BadRequest(anyhow!( - "Cannot manage failpoints because pageserver was compiled without failpoints support" - ))); - } - - let failpoints: ConfigureFailpointsRequest = json_request(&mut request).await?; - for fp in failpoints { - info!("cfg failpoint: {} {}", fp.name, fp.actions); - - // We recognize one extra "action" that's not natively recognized - // by the failpoints crate: exit, to immediately kill the process - let cfg_result = crate::failpoint_support::apply_failpoint(&fp.name, &fp.actions); - - if let Err(err_msg) = cfg_result { - return Err(ApiError::BadRequest(anyhow!( - "Failed to configure failpoints: {err_msg}" - ))); - } - } - - json_response(StatusCode::OK, ()) -} - // Run GC immediately on given timeline. async fn timeline_gc_handler( mut request: Request, diff --git a/pageserver/src/lib.rs b/pageserver/src/lib.rs index 58adf6e8c4..c1ce0af47b 100644 --- a/pageserver/src/lib.rs +++ b/pageserver/src/lib.rs @@ -25,8 +25,6 @@ pub mod walingest; pub mod walrecord; pub mod walredo; -pub mod failpoint_support; - use crate::task_mgr::TaskKind; use camino::Utf8Path; use deletion_queue::DeletionQueue; diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index 2f2169d194..e50987c84b 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -33,6 +33,7 @@ use tracing::*; use utils::backoff; use utils::completion; use utils::crashsafe::path_with_suffix_extension; +use utils::failpoint_support; use utils::fs_ext; use utils::sync::gate::Gate; use utils::sync::gate::GateGuard; @@ -890,7 +891,7 @@ impl Tenant { ) -> anyhow::Result<()> { span::debug_assert_current_span_has_tenant_id(); - crate::failpoint_support::sleep_millis_async!("before-attaching-tenant"); + failpoint_support::sleep_millis_async!("before-attaching-tenant"); let preload = match preload { Some(p) => p, @@ -1002,7 +1003,7 @@ impl Tenant { // IndexPart is the source of truth. self.clean_up_timelines(&existent_timelines)?; - crate::failpoint_support::sleep_millis_async!("attach-before-activate"); + failpoint_support::sleep_millis_async!("attach-before-activate"); info!("Done"); @@ -2839,9 +2840,7 @@ impl Tenant { } }; - crate::failpoint_support::sleep_millis_async!( - "gc_iteration_internal_after_getting_gc_timelines" - ); + failpoint_support::sleep_millis_async!("gc_iteration_internal_after_getting_gc_timelines"); // If there is nothing to GC, we don't want any messages in the INFO log. if !gc_timelines.is_empty() { diff --git a/pageserver/src/walingest.rs b/pageserver/src/walingest.rs index 1d14214030..a6a8972970 100644 --- a/pageserver/src/walingest.rs +++ b/pageserver/src/walingest.rs @@ -29,6 +29,7 @@ use postgres_ffi::{fsm_logical_to_physical, page_is_new, page_set_lsn}; use anyhow::{bail, Context, Result}; use bytes::{Buf, Bytes, BytesMut}; use tracing::*; +use utils::failpoint_support; use crate::context::RequestContext; use crate::metrics::WAL_INGEST; @@ -344,9 +345,7 @@ impl<'a> WalIngest<'a> { // particular point in the WAL. For more fine-grained control, // we could peek into the message and only pause if it contains // a particular string, for example, but this is enough for now. - crate::failpoint_support::sleep_millis_async!( - "wal-ingest-logical-message-sleep" - ); + failpoint_support::sleep_millis_async!("wal-ingest-logical-message-sleep"); } else if let Some(path) = prefix.strip_prefix("neon-file:") { modification.put_file(path, message, ctx).await?; } From e79a19339c2a6b8bb089a64db5e346e8f19a75d2 Mon Sep 17 00:00:00 2001 From: Arseny Sher Date: Mon, 1 Jan 2024 23:32:24 +0300 Subject: [PATCH 04/35] Add failpoint support to safekeeper. Just a copy paste from pageserver. --- Cargo.lock | 1 + safekeeper/Cargo.toml | 7 ++++++ safekeeper/src/bin/safekeeper.rs | 17 ++++++++++++- safekeeper/src/http/routes.rs | 8 ++++++ test_runner/fixtures/neon_fixtures.py | 36 +++++++++++++++++++++++---- 5 files changed, 63 insertions(+), 6 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 73cb83d3a7..55e868a6d5 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4449,6 +4449,7 @@ dependencies = [ "clap", "const_format", "crc32c", + "fail", "fs2", "futures", "git-version", diff --git a/safekeeper/Cargo.toml b/safekeeper/Cargo.toml index cccb4ebd79..4015c27933 100644 --- a/safekeeper/Cargo.toml +++ b/safekeeper/Cargo.toml @@ -4,6 +4,12 @@ version = "0.1.0" edition.workspace = true license.workspace = true +[features] +default = [] +# Enables test-only APIs, incuding failpoints. In particular, enables the `fail_point!` macro, +# which adds some runtime cost to run tests on outage conditions +testing = ["fail/failpoints"] + [dependencies] async-stream.workspace = true anyhow.workspace = true @@ -16,6 +22,7 @@ chrono.workspace = true clap = { workspace = true, features = ["derive"] } const_format.workspace = true crc32c.workspace = true +fail.workspace = true fs2.workspace = true git-version.workspace = true hex.workspace = true diff --git a/safekeeper/src/bin/safekeeper.rs b/safekeeper/src/bin/safekeeper.rs index e59deb9fda..33047051df 100644 --- a/safekeeper/src/bin/safekeeper.rs +++ b/safekeeper/src/bin/safekeeper.rs @@ -54,6 +54,19 @@ const ID_FILE_NAME: &str = "safekeeper.id"; project_git_version!(GIT_VERSION); project_build_tag!(BUILD_TAG); +const FEATURES: &[&str] = &[ + #[cfg(feature = "testing")] + "testing", +]; + +fn version() -> String { + format!( + "{GIT_VERSION} failpoints: {}, features: {:?}", + fail::has_failpoints(), + FEATURES, + ) +} + const ABOUT: &str = r#" A fleet of safekeepers is responsible for reliably storing WAL received from compute, passing it through consensus (mitigating potential computes brain @@ -167,7 +180,9 @@ async fn main() -> anyhow::Result<()> { // getting 'argument cannot be used multiple times' error. This seems to be // impossible with pure Derive API, so convert struct to Command, modify it, // parse arguments, and then fill the struct back. - let cmd = ::command().args_override_self(true); + let cmd = ::command() + .args_override_self(true) + .version(version()); let mut matches = cmd.get_matches(); let mut args = ::from_arg_matches_mut(&mut matches)?; diff --git a/safekeeper/src/http/routes.rs b/safekeeper/src/http/routes.rs index c48b5330b3..25a3334e63 100644 --- a/safekeeper/src/http/routes.rs +++ b/safekeeper/src/http/routes.rs @@ -12,6 +12,8 @@ use storage_broker::proto::SafekeeperTimelineInfo; use storage_broker::proto::TenantTimelineId as ProtoTenantTimelineId; use tokio::fs::File; use tokio::io::AsyncReadExt; +use tokio_util::sync::CancellationToken; +use utils::failpoint_support::failpoints_handler; use std::io::Write as _; use tokio::sync::mpsc; @@ -444,6 +446,12 @@ pub fn make_router(conf: SafeKeeperConf) -> RouterBuilder .data(Arc::new(conf)) .data(auth) .get("/v1/status", |r| request_span(r, status_handler)) + .put("/v1/failpoints", |r| { + request_span(r, move |r| async { + let cancel = CancellationToken::new(); + failpoints_handler(r, cancel).await + }) + }) // Will be used in the future instead of implicit timeline creation .post("/v1/tenant/timeline", |r| { request_span(r, timeline_create_handler) diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index 597e311e02..9aa82d8854 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -890,8 +890,8 @@ class NeonEnv: """Get list of safekeeper endpoints suitable for safekeepers GUC""" return ",".join(f"localhost:{wa.port.pg}" for wa in self.safekeepers) - def get_pageserver_version(self) -> str: - bin_pageserver = str(self.neon_binpath / "pageserver") + def get_binary_version(self, binary_name: str) -> str: + bin_pageserver = str(self.neon_binpath / binary_name) res = subprocess.run( [bin_pageserver, "--version"], check=True, @@ -1656,7 +1656,7 @@ class NeonPageserver(PgProtocol): self.running = False self.service_port = port self.config_override = config_override - self.version = env.get_pageserver_version() + self.version = env.get_binary_version("pageserver") # After a test finishes, we will scrape the log to see if there are any # unexpected error messages. If your test expects an error, add it to @@ -2924,7 +2924,8 @@ class Safekeeper: return res def http_client(self, auth_token: Optional[str] = None) -> SafekeeperHttpClient: - return SafekeeperHttpClient(port=self.port.http, auth_token=auth_token) + is_testing_enabled = '"testing"' in self.env.get_binary_version("safekeeper") + return SafekeeperHttpClient(port=self.port.http, auth_token=auth_token, is_testing_enabled=is_testing_enabled) def data_dir(self) -> str: return os.path.join(self.env.repo_dir, "safekeepers", f"sk{self.id}") @@ -2975,10 +2976,11 @@ class SafekeeperMetrics: class SafekeeperHttpClient(requests.Session): HTTPError = requests.HTTPError - def __init__(self, port: int, auth_token: Optional[str] = None): + def __init__(self, port: int, auth_token: Optional[str] = None, is_testing_enabled = False): super().__init__() self.port = port self.auth_token = auth_token + self.is_testing_enabled = is_testing_enabled if auth_token is not None: self.headers["Authorization"] = f"Bearer {auth_token}" @@ -2986,6 +2988,30 @@ class SafekeeperHttpClient(requests.Session): def check_status(self): self.get(f"http://localhost:{self.port}/v1/status").raise_for_status() + def is_testing_enabled_or_skip(self): + if not self.is_testing_enabled: + pytest.skip("safekeeper was built without 'testing' feature") + + def configure_failpoints(self, config_strings: Tuple[str, str] | List[Tuple[str, str]]): + self.is_testing_enabled_or_skip() + + if isinstance(config_strings, tuple): + pairs = [config_strings] + else: + pairs = config_strings + + log.info(f"Requesting config failpoints: {repr(pairs)}") + + res = self.put( + f"http://localhost:{self.port}/v1/failpoints", + json=[{"name": name, "actions": actions} for name, actions in pairs], + ) + log.info(f"Got failpoints request response code {res.status_code}") + res.raise_for_status() + res_json = res.json() + assert res_json is None + return res_json + def debug_dump(self, params: Optional[Dict[str, str]] = None) -> Dict[str, Any]: params = params or {} res = self.get(f"http://localhost:{self.port}/v1/debug_dump", params=params) From aaaa39d9f52a46641c86314ddc9d15565275d9c2 Mon Sep 17 00:00:00 2001 From: Arseny Sher Date: Fri, 29 Dec 2023 23:09:36 +0300 Subject: [PATCH 05/35] Add large insertion and slow WAL sending to test_hot_standby. To exercise MAX_SEND_SIZE sending from safekeeper; we've had a bug with WAL records torn across several XLogData messages. Add failpoint to safekeeper to slow down sending. Also check for corrupted WAL complains in standby log. Make the test a bit simpler in passing, e.g. we don't need explicit commits as autocommit is enabled by default. https://neondb.slack.com/archives/C05L7D1JAUS/p1703774799114719 https://github.com/neondatabase/cloud/issues/9057 --- safekeeper/src/send_wal.rs | 6 ++ test_runner/fixtures/neon_fixtures.py | 17 +++-- test_runner/regress/test_hot_standby.py | 91 +++++++++++++++---------- 3 files changed, 73 insertions(+), 41 deletions(-) diff --git a/safekeeper/src/send_wal.rs b/safekeeper/src/send_wal.rs index 70590a0f95..bd1d306968 100644 --- a/safekeeper/src/send_wal.rs +++ b/safekeeper/src/send_wal.rs @@ -17,6 +17,7 @@ use postgres_ffi::{TimestampTz, MAX_SEND_SIZE}; use pq_proto::{BeMessage, WalSndKeepAlive, XLogDataBody}; use serde::{Deserialize, Serialize}; use tokio::io::{AsyncRead, AsyncWrite}; +use utils::failpoint_support; use utils::id::TenantTimelineId; use utils::lsn::AtomicLsn; use utils::pageserver_feedback::PageserverFeedback; @@ -559,6 +560,11 @@ impl WalSender<'_, IO> { })) .await?; + if let Some(appname) = &self.appname { + if appname == "replica" { + failpoint_support::sleep_millis_async!("sk-send-wal-replica-sleep"); + } + } trace!( "sent {} bytes of WAL {}-{}", send_size, diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index 9aa82d8854..5b1a8ba27d 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -347,7 +347,9 @@ class PgProtocol: """ return self.safe_psql_many([query], **kwargs)[0] - def safe_psql_many(self, queries: List[str], **kwargs: Any) -> List[List[Tuple[Any, ...]]]: + def safe_psql_many( + self, queries: List[str], log_query=True, **kwargs: Any + ) -> List[List[Tuple[Any, ...]]]: """ Execute queries against the node and return all rows. This method passes all extra params to connstr. @@ -356,7 +358,8 @@ class PgProtocol: with closing(self.connect(**kwargs)) as conn: with conn.cursor() as cur: for query in queries: - log.info(f"Executing query: {query}") + if log_query: + log.info(f"Executing query: {query}") cur.execute(query) if cur.description is None: @@ -365,11 +368,11 @@ class PgProtocol: result.append(cur.fetchall()) return result - def safe_psql_scalar(self, query) -> Any: + def safe_psql_scalar(self, query, log_query=True) -> Any: """ Execute query returning single row with single column. """ - return self.safe_psql(query)[0][0] + return self.safe_psql(query, log_query=log_query)[0][0] @dataclass @@ -2925,7 +2928,9 @@ class Safekeeper: def http_client(self, auth_token: Optional[str] = None) -> SafekeeperHttpClient: is_testing_enabled = '"testing"' in self.env.get_binary_version("safekeeper") - return SafekeeperHttpClient(port=self.port.http, auth_token=auth_token, is_testing_enabled=is_testing_enabled) + return SafekeeperHttpClient( + port=self.port.http, auth_token=auth_token, is_testing_enabled=is_testing_enabled + ) def data_dir(self) -> str: return os.path.join(self.env.repo_dir, "safekeepers", f"sk{self.id}") @@ -2976,7 +2981,7 @@ class SafekeeperMetrics: class SafekeeperHttpClient(requests.Session): HTTPError = requests.HTTPError - def __init__(self, port: int, auth_token: Optional[str] = None, is_testing_enabled = False): + def __init__(self, port: int, auth_token: Optional[str] = None, is_testing_enabled=False): super().__init__() self.port = port self.auth_token = auth_token diff --git a/test_runner/regress/test_hot_standby.py b/test_runner/regress/test_hot_standby.py index 031fd2857d..7822e29ed9 100644 --- a/test_runner/regress/test_hot_standby.py +++ b/test_runner/regress/test_hot_standby.py @@ -1,19 +1,59 @@ +import os +import re import time -from fixtures.neon_fixtures import NeonEnv +from fixtures.log_helper import log +from fixtures.neon_fixtures import Endpoint, NeonEnv + + +def wait_caughtup(primary: Endpoint, secondary: Endpoint): + primary_lsn = primary.safe_psql_scalar( + "SELECT pg_current_wal_insert_lsn()::text", log_query=False + ) + while True: + secondary_lsn = secondary.safe_psql_scalar( + "SELECT pg_last_wal_replay_lsn()", log_query=False + ) + caught_up = secondary_lsn >= primary_lsn + log.info(f"caughtup={caught_up}, primary_lsn={primary_lsn}, secondary_lsn={secondary_lsn}") + if caught_up: + return + time.sleep(1) + + +# Check for corrupted WAL messages which might otherwise go unnoticed if +# reconnection fixes this. +def scan_standby_log_for_errors(secondary): + log_path = secondary.endpoint_path() / "compute.log" + with log_path.open("r") as f: + markers = re.compile( + r"incorrect resource manager data|record with incorrect|invalid magic number|unexpected pageaddr" + ) + for line in f: + if markers.search(line): + log.info(f"bad error in standby log: {line}") + raise AssertionError() def test_hot_standby(neon_simple_env: NeonEnv): env = neon_simple_env + # We've had a bug caused by WAL records split across multiple XLogData + # messages resulting in corrupted WAL complains on standby. It reproduced + # only when sending from safekeeper is slow enough to grab full + # MAX_SEND_SIZE messages. So insert sleep through failpoints, but only in + # one conf to decrease test time. + slow_down_send = "[debug-pg16]" in os.environ.get("PYTEST_CURRENT_TEST", "") + if slow_down_send: + sk_http = env.safekeepers[0].http_client() + sk_http.configure_failpoints([("sk-send-wal-replica-sleep", "return(100)")]) + with env.endpoints.create_start( branch_name="main", endpoint_id="primary", ) as primary: time.sleep(1) with env.endpoints.new_replica_start(origin=primary, endpoint_id="secondary") as secondary: - primary_lsn = None - caught_up = False queries = [ "SHOW neon.timeline_id", "SHOW neon.tenant_id", @@ -26,23 +66,6 @@ def test_hot_standby(neon_simple_env: NeonEnv): with p_con.cursor() as p_cur: p_cur.execute("CREATE TABLE test AS SELECT generate_series(1, 100) AS i") - # Explicit commit to make sure other connections (and replicas) can - # see the changes of this commit. - p_con.commit() - - with p_con.cursor() as p_cur: - p_cur.execute("SELECT pg_current_wal_insert_lsn()::text") - res = p_cur.fetchone() - assert res is not None - (lsn,) = res - primary_lsn = lsn - - # Explicit commit to make sure other connections (and replicas) can - # see the changes of this commit. - # Note that this may generate more WAL if the transaction has changed - # things, but we don't care about that. - p_con.commit() - for query in queries: with p_con.cursor() as p_cur: p_cur.execute(query) @@ -51,30 +74,28 @@ def test_hot_standby(neon_simple_env: NeonEnv): response = res responses[query] = response + # insert more data to make safekeeper send MAX_SEND_SIZE messages + if slow_down_send: + primary.safe_psql("create table t(key int, value text)") + primary.safe_psql("insert into t select generate_series(1, 100000), 'payload'") + + wait_caughtup(primary, secondary) + with secondary.connect() as s_con: with s_con.cursor() as s_cur: s_cur.execute("SELECT 1 WHERE pg_is_in_recovery()") res = s_cur.fetchone() assert res is not None - while not caught_up: - with s_con.cursor() as secondary_cursor: - secondary_cursor.execute("SELECT pg_last_wal_replay_lsn()") - res = secondary_cursor.fetchone() - assert res is not None - (secondary_lsn,) = res - # There may be more changes on the primary after we got our LSN - # due to e.g. autovacuum, but that shouldn't impact the content - # of the tables, so we check whether we've replayed up to at - # least after the commit of the `test` table. - caught_up = secondary_lsn >= primary_lsn - - # Explicit commit to flush any transient transaction-level state. - s_con.commit() - for query in queries: with s_con.cursor() as secondary_cursor: secondary_cursor.execute(query) response = secondary_cursor.fetchone() assert response is not None assert response == responses[query] + + scan_standby_log_for_errors(secondary) + + # clean up + if slow_down_send: + sk_http.configure_failpoints(("sk-send-wal-replica-sleep", "off")) From e92c9f42c05a6c10b53269999c4555d9c8a8f6c9 Mon Sep 17 00:00:00 2001 From: Arseny Sher Date: Sat, 30 Dec 2023 00:31:19 +0300 Subject: [PATCH 06/35] Don't split WAL record across two XLogData's when sending from safekeepers. As protocol demands. Not following this makes standby complain about corrupted WAL in various ways. https://neondb.slack.com/archives/C05L7D1JAUS/p1703774799114719 closes https://github.com/neondatabase/cloud/issues/9057 --- safekeeper/src/send_wal.rs | 22 +++++++++++++++------- safekeeper/src/wal_storage.rs | 3 +++ 2 files changed, 18 insertions(+), 7 deletions(-) diff --git a/safekeeper/src/send_wal.rs b/safekeeper/src/send_wal.rs index bd1d306968..9a5657a40d 100644 --- a/safekeeper/src/send_wal.rs +++ b/safekeeper/src/send_wal.rs @@ -529,12 +529,19 @@ impl WalSender<'_, IO> { ); // try to send as much as available, capped by MAX_SEND_SIZE - let mut send_size = self - .end_pos - .checked_sub(self.start_pos) - .context("reading wal without waiting for it first")? - .0 as usize; - send_size = min(send_size, self.send_buf.len()); + let mut chunk_end_pos = self.start_pos + MAX_SEND_SIZE as u64; + // if we went behind available WAL, back off + if chunk_end_pos >= self.end_pos { + chunk_end_pos = self.end_pos; + } else { + // If sending not up to end pos, round down to page boundary to + // avoid breaking WAL record not at page boundary, as protocol + // demands. See walsender.c (XLogSendPhysical). + chunk_end_pos = chunk_end_pos + .checked_sub(chunk_end_pos.block_offset()) + .unwrap(); + } + let send_size = (chunk_end_pos.0 - self.start_pos.0) as usize; let send_buf = &mut self.send_buf[..send_size]; let send_size: usize; { @@ -545,7 +552,8 @@ impl WalSender<'_, IO> { } else { None }; - // read wal into buffer + // Read WAL into buffer. send_size can be additionally capped to + // segment boundary here. send_size = self.wal_reader.read(send_buf).await? }; let send_buf = &send_buf[..send_size]; diff --git a/safekeeper/src/wal_storage.rs b/safekeeper/src/wal_storage.rs index fa44b24258..e7538f805c 100644 --- a/safekeeper/src/wal_storage.rs +++ b/safekeeper/src/wal_storage.rs @@ -565,6 +565,9 @@ impl WalReader { }) } + /// Read WAL at current position into provided buf, returns number of bytes + /// read. It can be smaller than buf size only if segment boundary is + /// reached. pub async fn read(&mut self, buf: &mut [u8]) -> Result { // If this timeline is new, we may not have a full segment yet, so // we pad the first bytes of the timeline's first WAL segment with 0s From ea9fad419eceb7af4d340e308b25727c39eeb622 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Tue, 5 Dec 2023 15:04:26 -0800 Subject: [PATCH 07/35] Add exponential backoff to page_server->send --- pgxn/neon/pagestore_smgr.c | 27 ++++++++++++++++++++++++++- 1 file changed, 26 insertions(+), 1 deletion(-) diff --git a/pgxn/neon/pagestore_smgr.c b/pgxn/neon/pagestore_smgr.c index 8888cd89c6..6cf2762179 100644 --- a/pgxn/neon/pagestore_smgr.c +++ b/pgxn/neon/pagestore_smgr.c @@ -275,6 +275,26 @@ static inline void prefetch_set_unused(uint64 ring_index); static XLogRecPtr neon_get_request_lsn(bool *latest, NRelFileInfo rinfo, ForkNumber forknum, BlockNumber blkno); + +#define INITIAL_EXPONENTIAL_BACKOFF_DELAY 1000 +#define EXPONENTIAL_BACKOFF_EXPONENT 2 +#define MAX_EXPONENTIAL_BACKOFF_DELAY (1000*1000) + +static void +InitExponentialBackoff(long *delay) +{ + *delay = INITIAL_EXPONENTIAL_BACKOFF_DELAY; +} + +static void +PerformExponentialBackoff(long *delay) +{ + pg_usleep(*delay); + *delay *= EXPONENTIAL_BACKOFF_EXPONENT; + if(*delay >= MAX_EXPONENTIAL_BACKOFF_DELAY) + *delay = MAX_EXPONENTIAL_BACKOFF_DELAY; +} + static bool compact_prefetch_buffers(void) { @@ -662,6 +682,7 @@ prefetch_do_request(PrefetchRequest *slot, bool *force_latest, XLogRecPtr *force .forknum = slot->buftag.forkNum, .blkno = slot->buftag.blockNum, }; + long backoff_delay_us; if (force_lsn && force_latest) { @@ -704,7 +725,11 @@ prefetch_do_request(PrefetchRequest *slot, bool *force_latest, XLogRecPtr *force Assert(slot->response == NULL); Assert(slot->my_ring_index == MyPState->ring_unused); - while (!page_server->send((NeonRequest *) &request)); + InitExponentialBackoff(&backoff_delay_us); + while (!page_server->send((NeonRequest *) &request)) + { + PerformExponentialBackoff(&backoff_delay_us); + } /* update prefetch state */ MyPState->n_requests_inflight += 1; From 091a0cda9d2b309f06189b4d976bb91fd3de7dc1 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Fri, 15 Dec 2023 12:35:38 -0800 Subject: [PATCH 08/35] Switch to rate-limiting strategy --- pgxn/neon/libpagestore.c | 15 ++++++++++++++- pgxn/neon/pagestore_smgr.c | 27 +-------------------------- 2 files changed, 15 insertions(+), 27 deletions(-) diff --git a/pgxn/neon/libpagestore.c b/pgxn/neon/libpagestore.c index 3b038f906f..5db9e5e08e 100644 --- a/pgxn/neon/libpagestore.c +++ b/pgxn/neon/libpagestore.c @@ -133,6 +133,9 @@ pageserver_connect(int elevel) const char *values[3]; int n; + static TimestampTz last_connect_time = 0; + TimestampTz now; + Assert(!connected); if (CheckConnstringUpdated()) @@ -140,6 +143,17 @@ pageserver_connect(int elevel) ReloadConnstring(); } + now = GetCurrentTimestamp(); + if ((now - last_connect_time) < RECONNECT_INTERVAL_USEC) + { + pg_usleep(RECONNECT_INTERVAL_USEC); + last_connect_time = GetCurrentTimestamp(); + } + else + { + last_connect_time = now; + } + /* * Connect using the connection string we got from the * neon.pageserver_connstring GUC. If the NEON_AUTH_TOKEN environment @@ -333,7 +347,6 @@ pageserver_send(NeonRequest *request) { HandleMainLoopInterrupts(); n_reconnect_attempts += 1; - pg_usleep(RECONNECT_INTERVAL_USEC); } n_reconnect_attempts = 0; } diff --git a/pgxn/neon/pagestore_smgr.c b/pgxn/neon/pagestore_smgr.c index 6cf2762179..8888cd89c6 100644 --- a/pgxn/neon/pagestore_smgr.c +++ b/pgxn/neon/pagestore_smgr.c @@ -275,26 +275,6 @@ static inline void prefetch_set_unused(uint64 ring_index); static XLogRecPtr neon_get_request_lsn(bool *latest, NRelFileInfo rinfo, ForkNumber forknum, BlockNumber blkno); - -#define INITIAL_EXPONENTIAL_BACKOFF_DELAY 1000 -#define EXPONENTIAL_BACKOFF_EXPONENT 2 -#define MAX_EXPONENTIAL_BACKOFF_DELAY (1000*1000) - -static void -InitExponentialBackoff(long *delay) -{ - *delay = INITIAL_EXPONENTIAL_BACKOFF_DELAY; -} - -static void -PerformExponentialBackoff(long *delay) -{ - pg_usleep(*delay); - *delay *= EXPONENTIAL_BACKOFF_EXPONENT; - if(*delay >= MAX_EXPONENTIAL_BACKOFF_DELAY) - *delay = MAX_EXPONENTIAL_BACKOFF_DELAY; -} - static bool compact_prefetch_buffers(void) { @@ -682,7 +662,6 @@ prefetch_do_request(PrefetchRequest *slot, bool *force_latest, XLogRecPtr *force .forknum = slot->buftag.forkNum, .blkno = slot->buftag.blockNum, }; - long backoff_delay_us; if (force_lsn && force_latest) { @@ -725,11 +704,7 @@ prefetch_do_request(PrefetchRequest *slot, bool *force_latest, XLogRecPtr *force Assert(slot->response == NULL); Assert(slot->my_ring_index == MyPState->ring_unused); - InitExponentialBackoff(&backoff_delay_us); - while (!page_server->send((NeonRequest *) &request)) - { - PerformExponentialBackoff(&backoff_delay_us); - } + while (!page_server->send((NeonRequest *) &request)); /* update prefetch state */ MyPState->n_requests_inflight += 1; From 4e1d16f3112f6296f5e4616a0fb67a60adde9dbd Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Tue, 26 Dec 2023 14:14:37 -0800 Subject: [PATCH 09/35] Switch to exponential rate-limiting --- pgxn/neon/libpagestore.c | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/pgxn/neon/libpagestore.c b/pgxn/neon/libpagestore.c index 5db9e5e08e..574e68af66 100644 --- a/pgxn/neon/libpagestore.c +++ b/pgxn/neon/libpagestore.c @@ -35,7 +35,8 @@ #define PageStoreTrace DEBUG5 -#define RECONNECT_INTERVAL_USEC 1000000 +#define MAX_RECONNECT_INTERVAL_USEC 100 +#define MAX_RECONNECT_INTERVAL_USEC 1000000 bool connected = false; PGconn *pageserver_conn = NULL; @@ -134,7 +135,9 @@ pageserver_connect(int elevel) int n; static TimestampTz last_connect_time = 0; + static uint64_t delay_us = MIN_RECONNECT_INTERVAL_USEC; TimestampTz now; + uint64_t us_since_last_connect; Assert(!connected); @@ -144,13 +147,18 @@ pageserver_connect(int elevel) } now = GetCurrentTimestamp(); - if ((now - last_connect_time) < RECONNECT_INTERVAL_USEC) + us_since_last_connect = now - last_connect_time; + if (us_since_last_connect < delay_us) { - pg_usleep(RECONNECT_INTERVAL_USEC); + pg_usleep(delay_us - us_since_last_connect); + delay_us *= 2; + if (delay_us > MAX_RECONNECT_INTERVAL_USEC) + delay_us = MAX_RECONNECT_INTERVAL_USEC; last_connect_time = GetCurrentTimestamp(); } else { + delay_us = MIN_RECONNECT_INTERVAL_USEC; last_connect_time = now; } From ce13281d542be2d6fc50c7884fbe5f541cd5a64d Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Wed, 27 Dec 2023 08:52:46 -0800 Subject: [PATCH 10/35] MIN not MAX --- pgxn/neon/libpagestore.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pgxn/neon/libpagestore.c b/pgxn/neon/libpagestore.c index 574e68af66..e58c28d7d5 100644 --- a/pgxn/neon/libpagestore.c +++ b/pgxn/neon/libpagestore.c @@ -35,7 +35,7 @@ #define PageStoreTrace DEBUG5 -#define MAX_RECONNECT_INTERVAL_USEC 100 +#define MIN_RECONNECT_INTERVAL_USEC 100 #define MAX_RECONNECT_INTERVAL_USEC 1000000 bool connected = false; From 946c6a000686ce8d790deaa74fafa6d1ed72e372 Mon Sep 17 00:00:00 2001 From: Joonas Koivunen Date: Tue, 2 Jan 2024 17:22:16 +0200 Subject: [PATCH 11/35] scrubber: use adaptive config with retries, check subset of tenants (#6219) The tool still needs a lot of work. These are the easiest fix and feature: - use similar adaptive config with s3 as remote_storage, use retries - process only particular tenants Tenants need to be from the correct region, they are not deduplicated, but the feature is useful for re-checking small amount of tenants after a large run. --- Cargo.lock | 2 ++ s3_scrubber/Cargo.toml | 3 ++ s3_scrubber/src/lib.rs | 8 ++++- s3_scrubber/src/main.rs | 57 +++++++++++++++++--------------- s3_scrubber/src/scan_metadata.rs | 11 ++++-- 5 files changed, 52 insertions(+), 29 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 55e868a6d5..93efbadd79 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4405,12 +4405,14 @@ dependencies = [ "async-stream", "aws-config", "aws-sdk-s3", + "aws-smithy-async", "bincode", "bytes", "chrono", "clap", "crc32c", "either", + "futures", "futures-util", "hex", "histogram", diff --git a/s3_scrubber/Cargo.toml b/s3_scrubber/Cargo.toml index fdae378d55..4d136472e0 100644 --- a/s3_scrubber/Cargo.toml +++ b/s3_scrubber/Cargo.toml @@ -6,6 +6,7 @@ license.workspace = true [dependencies] aws-sdk-s3.workspace = true +aws-smithy-async.workspace = true either.workspace = true tokio-rustls.workspace = true anyhow.workspace = true @@ -39,3 +40,5 @@ tracing-subscriber.workspace = true clap.workspace = true tracing-appender = "0.2" histogram = "0.7" + +futures.workspace = true diff --git a/s3_scrubber/src/lib.rs b/s3_scrubber/src/lib.rs index 8fb1346c8e..d2842877d0 100644 --- a/s3_scrubber/src/lib.rs +++ b/s3_scrubber/src/lib.rs @@ -16,10 +16,12 @@ use aws_config::environment::EnvironmentVariableCredentialsProvider; use aws_config::imds::credentials::ImdsCredentialsProvider; use aws_config::meta::credentials::CredentialsProviderChain; use aws_config::profile::ProfileFileCredentialsProvider; +use aws_config::retry::RetryConfig; use aws_config::sso::SsoCredentialsProvider; use aws_config::BehaviorVersion; -use aws_sdk_s3::config::Region; +use aws_sdk_s3::config::{AsyncSleep, Region, SharedAsyncSleep}; use aws_sdk_s3::{Client, Config}; +use aws_smithy_async::rt::sleep::TokioSleep; use clap::ValueEnum; use pageserver::tenant::TENANTS_SEGMENT_NAME; @@ -283,9 +285,13 @@ pub fn init_s3_client(account_id: Option, bucket_region: Region) -> Clie ) }; + let sleep_impl: Arc = Arc::new(TokioSleep::new()); + let mut builder = Config::builder() .behavior_version(BehaviorVersion::v2023_11_09()) .region(bucket_region) + .retry_config(RetryConfig::adaptive().with_max_attempts(3)) + .sleep_impl(SharedAsyncSleep::from(sleep_impl)) .credentials_provider(credentials_provider); if let Ok(endpoint) = env::var("AWS_ENDPOINT_URL") { diff --git a/s3_scrubber/src/main.rs b/s3_scrubber/src/main.rs index ef020edc2a..957213856b 100644 --- a/s3_scrubber/src/main.rs +++ b/s3_scrubber/src/main.rs @@ -1,3 +1,4 @@ +use pageserver_api::shard::TenantShardId; use s3_scrubber::garbage::{find_garbage, purge_garbage, PurgeMode}; use s3_scrubber::scan_metadata::scan_metadata; use s3_scrubber::{init_logging, BucketConfig, ConsoleConfig, NodeKind, TraversingDepth}; @@ -34,6 +35,8 @@ enum Command { ScanMetadata { #[arg(short, long, default_value_t = false)] json: bool, + #[arg(long = "tenant-id", num_args = 0..)] + tenant_ids: Vec, }, } @@ -57,35 +60,37 @@ async fn main() -> anyhow::Result<()> { )); match cli.command { - Command::ScanMetadata { json } => match scan_metadata(bucket_config.clone()).await { - Err(e) => { - tracing::error!("Failed: {e}"); - Err(e) - } - Ok(summary) => { - if json { - println!("{}", serde_json::to_string(&summary).unwrap()) - } else { - println!("{}", summary.summary_string()); + Command::ScanMetadata { json, tenant_ids } => { + match scan_metadata(bucket_config.clone(), tenant_ids).await { + Err(e) => { + tracing::error!("Failed: {e}"); + Err(e) } - if summary.is_fatal() { - Err(anyhow::anyhow!("Fatal scrub errors detected")) - } else if summary.is_empty() { - // Strictly speaking an empty bucket is a valid bucket, but if someone ran the - // scrubber they were likely expecting to scan something, and if we see no timelines - // at all then it's likely due to some configuration issues like a bad prefix - Err(anyhow::anyhow!( - "No timelines found in bucket {} prefix {}", - bucket_config.bucket, - bucket_config - .prefix_in_bucket - .unwrap_or("".to_string()) - )) - } else { - Ok(()) + Ok(summary) => { + if json { + println!("{}", serde_json::to_string(&summary).unwrap()) + } else { + println!("{}", summary.summary_string()); + } + if summary.is_fatal() { + Err(anyhow::anyhow!("Fatal scrub errors detected")) + } else if summary.is_empty() { + // Strictly speaking an empty bucket is a valid bucket, but if someone ran the + // scrubber they were likely expecting to scan something, and if we see no timelines + // at all then it's likely due to some configuration issues like a bad prefix + Err(anyhow::anyhow!( + "No timelines found in bucket {} prefix {}", + bucket_config.bucket, + bucket_config + .prefix_in_bucket + .unwrap_or("".to_string()) + )) + } else { + Ok(()) + } } } - }, + } Command::FindGarbage { node_kind, depth, diff --git a/s3_scrubber/src/scan_metadata.rs b/s3_scrubber/src/scan_metadata.rs index bcc4d2e618..bfde8f0213 100644 --- a/s3_scrubber/src/scan_metadata.rs +++ b/s3_scrubber/src/scan_metadata.rs @@ -187,10 +187,17 @@ Timeline layer count: {6} } /// Scan the pageserver metadata in an S3 bucket, reporting errors and statistics. -pub async fn scan_metadata(bucket_config: BucketConfig) -> anyhow::Result { +pub async fn scan_metadata( + bucket_config: BucketConfig, + tenant_ids: Vec, +) -> anyhow::Result { let (s3_client, target) = init_remote(bucket_config, NodeKind::Pageserver)?; - let tenants = stream_tenants(&s3_client, &target); + let tenants = if tenant_ids.is_empty() { + futures::future::Either::Left(stream_tenants(&s3_client, &target)) + } else { + futures::future::Either::Right(futures::stream::iter(tenant_ids.into_iter().map(Ok))) + }; // How many tenants to process in parallel. We need to be mindful of pageservers // accessing the same per tenant prefixes, so use a lower setting than pageservers. From aa9f1d4b697eefb48e4ebff4aebb4881d2bb29c4 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Tue, 2 Jan 2024 17:57:29 +0100 Subject: [PATCH 12/35] pagebench get-page: default to latest=true, make configurable via flag (#6252) fixes https://github.com/neondatabase/neon/issues/6209 --- pageserver/client/src/page_service.rs | 9 +---- .../pagebench/src/cmd/getpage_latest_lsn.rs | 38 +++++++++++++------ 2 files changed, 28 insertions(+), 19 deletions(-) diff --git a/pageserver/client/src/page_service.rs b/pageserver/client/src/page_service.rs index fc0d2311f7..231461267a 100644 --- a/pageserver/client/src/page_service.rs +++ b/pageserver/client/src/page_service.rs @@ -115,15 +115,8 @@ impl PagestreamClient { pub async fn getpage( &mut self, - key: RelTagBlockNo, - lsn: Lsn, + req: PagestreamGetPageRequest, ) -> anyhow::Result { - let req = PagestreamGetPageRequest { - latest: false, - rel: key.rel_tag, - blkno: key.block_no, - lsn, - }; let req = PagestreamFeMessage::GetPage(req); let req: bytes::Bytes = req.serialize(); // let mut req = tokio_util::io::ReaderStream::new(&req); diff --git a/pageserver/pagebench/src/cmd/getpage_latest_lsn.rs b/pageserver/pagebench/src/cmd/getpage_latest_lsn.rs index 16d198ab0e..cb36a403f1 100644 --- a/pageserver/pagebench/src/cmd/getpage_latest_lsn.rs +++ b/pageserver/pagebench/src/cmd/getpage_latest_lsn.rs @@ -3,7 +3,7 @@ use futures::future::join_all; use pageserver::pgdatadir_mapping::key_to_rel_block; use pageserver::repository; use pageserver_api::key::is_rel_block_key; -use pageserver_client::page_service::RelTagBlockNo; +use pageserver_api::models::PagestreamGetPageRequest; use utils::id::TenantTimelineId; use utils::lsn::Lsn; @@ -39,6 +39,9 @@ pub(crate) struct Args { runtime: Option, #[clap(long)] per_target_rate_limit: Option, + /// Probability for sending `latest=true` in the request (uniform distribution). + #[clap(long, default_value = "1")] + req_latest_probability: f64, #[clap(long)] limit_to_first_n_targets: Option, targets: Option>, @@ -200,18 +203,26 @@ async fn main_impl( start_work_barrier.wait().await; loop { - let (range, key) = { + let (timeline, req) = { let mut rng = rand::thread_rng(); let r = &all_ranges[weights.sample(&mut rng)]; let key: i128 = rng.gen_range(r.start..r.end); let key = repository::Key::from_i128(key); let (rel_tag, block_no) = key_to_rel_block(key).expect("we filter non-rel-block keys out above"); - (r, RelTagBlockNo { rel_tag, block_no }) + ( + r.timeline, + PagestreamGetPageRequest { + latest: rng.gen_bool(args.req_latest_probability), + lsn: r.timeline_lsn, + rel: rel_tag, + blkno: block_no, + }, + ) }; - let sender = work_senders.get(&range.timeline).unwrap(); + let sender = work_senders.get(&timeline).unwrap(); // TODO: what if this blocks? - sender.send((key, range.timeline_lsn)).await.ok().unwrap(); + sender.send(req).await.ok().unwrap(); } }), Some(rps_limit) => Box::pin(async move { @@ -240,16 +251,21 @@ async fn main_impl( ); loop { ticker.tick().await; - let (range, key) = { + let req = { let mut rng = rand::thread_rng(); let r = &ranges[weights.sample(&mut rng)]; let key: i128 = rng.gen_range(r.start..r.end); let key = repository::Key::from_i128(key); let (rel_tag, block_no) = key_to_rel_block(key) .expect("we filter non-rel-block keys out above"); - (r, RelTagBlockNo { rel_tag, block_no }) + PagestreamGetPageRequest { + latest: rng.gen_bool(args.req_latest_probability), + lsn: r.timeline_lsn, + rel: rel_tag, + blkno: block_no, + } }; - sender.send((key, range.timeline_lsn)).await.ok().unwrap(); + sender.send(req).await.ok().unwrap(); } }) }; @@ -303,7 +319,7 @@ async fn client( args: &'static Args, timeline: TenantTimelineId, start_work_barrier: Arc, - mut work: tokio::sync::mpsc::Receiver<(RelTagBlockNo, Lsn)>, + mut work: tokio::sync::mpsc::Receiver, all_work_done_barrier: Arc, live_stats: Arc, ) { @@ -317,10 +333,10 @@ async fn client( .await .unwrap(); - while let Some((key, lsn)) = work.recv().await { + while let Some(req) = work.recv().await { let start = Instant::now(); client - .getpage(key, lsn) + .getpage(req) .await .with_context(|| format!("getpage for {timeline}")) .unwrap(); From ae3eaf99957433b2df51aa79fb7b63f6959156f9 Mon Sep 17 00:00:00 2001 From: Arseny Sher Date: Tue, 2 Jan 2024 20:27:53 +0300 Subject: [PATCH 13/35] Add [WP] prefix to all walproposer logging. - rename walpop_log to wp_log - create also wpg_log which is used in postgres-specific code - in passing format messages to start with lower case --- libs/walproposer/src/walproposer.rs | 2 +- pgxn/neon/walproposer.c | 240 ++++++++++++++-------------- pgxn/neon/walproposer.h | 16 +- pgxn/neon/walproposer_pg.c | 68 ++++---- 4 files changed, 169 insertions(+), 157 deletions(-) diff --git a/libs/walproposer/src/walproposer.rs b/libs/walproposer/src/walproposer.rs index 35c8f6904d..7251545792 100644 --- a/libs/walproposer/src/walproposer.rs +++ b/libs/walproposer/src/walproposer.rs @@ -425,7 +425,7 @@ mod tests { } fn log_internal(&self, _wp: &mut crate::bindings::WalProposer, level: Level, msg: &str) { - println!("walprop_log[{}] {}", level, msg); + println!("wp_log[{}] {}", level, msg); } fn after_election(&self, _wp: &mut crate::bindings::WalProposer) { diff --git a/pgxn/neon/walproposer.c b/pgxn/neon/walproposer.c index 7fb0cab9a0..2ea724f927 100644 --- a/pgxn/neon/walproposer.c +++ b/pgxn/neon/walproposer.c @@ -99,7 +99,7 @@ WalProposerCreate(WalProposerConfig *config, walproposer_api api) port = strchr(host, ':'); if (port == NULL) { - walprop_log(FATAL, "port is not specified"); + wp_log(FATAL, "port is not specified"); } *port++ = '\0'; sep = strchr(port, ','); @@ -107,7 +107,7 @@ WalProposerCreate(WalProposerConfig *config, walproposer_api api) *sep++ = '\0'; if (wp->n_safekeepers + 1 >= MAX_SAFEKEEPERS) { - walprop_log(FATAL, "Too many safekeepers"); + wp_log(FATAL, "too many safekeepers"); } wp->safekeeper[wp->n_safekeepers].host = host; wp->safekeeper[wp->n_safekeepers].port = port; @@ -123,7 +123,7 @@ WalProposerCreate(WalProposerConfig *config, walproposer_api api) "host=%s port=%s dbname=replication options='-c timeline_id=%s tenant_id=%s'", sk->host, sk->port, wp->config->neon_timeline, wp->config->neon_tenant); if (written > MAXCONNINFO || written < 0) - walprop_log(FATAL, "could not create connection string for safekeeper %s:%s", sk->host, sk->port); + wp_log(FATAL, "could not create connection string for safekeeper %s:%s", sk->host, sk->port); } initStringInfo(&wp->safekeeper[wp->n_safekeepers].outbuf); @@ -133,7 +133,7 @@ WalProposerCreate(WalProposerConfig *config, walproposer_api api) } if (wp->n_safekeepers < 1) { - walprop_log(FATAL, "Safekeepers addresses are not specified"); + wp_log(FATAL, "safekeepers addresses are not specified"); } wp->quorum = wp->n_safekeepers / 2 + 1; @@ -144,15 +144,15 @@ WalProposerCreate(WalProposerConfig *config, walproposer_api api) wp->api.strong_random(wp, &wp->greetRequest.proposerId, sizeof(wp->greetRequest.proposerId)); wp->greetRequest.systemId = wp->config->systemId; if (!wp->config->neon_timeline) - walprop_log(FATAL, "neon.timeline_id is not provided"); + wp_log(FATAL, "neon.timeline_id is not provided"); if (*wp->config->neon_timeline != '\0' && !HexDecodeString(wp->greetRequest.timeline_id, wp->config->neon_timeline, 16)) - walprop_log(FATAL, "Could not parse neon.timeline_id, %s", wp->config->neon_timeline); + wp_log(FATAL, "could not parse neon.timeline_id, %s", wp->config->neon_timeline); if (!wp->config->neon_tenant) - walprop_log(FATAL, "neon.tenant_id is not provided"); + wp_log(FATAL, "neon.tenant_id is not provided"); if (*wp->config->neon_tenant != '\0' && !HexDecodeString(wp->greetRequest.tenant_id, wp->config->neon_tenant, 16)) - walprop_log(FATAL, "Could not parse neon.tenant_id, %s", wp->config->neon_tenant); + wp_log(FATAL, "could not parse neon.tenant_id, %s", wp->config->neon_tenant); wp->greetRequest.timeline = wp->config->pgTimeline; wp->greetRequest.walSegSize = wp->config->wal_segment_size; @@ -274,8 +274,8 @@ WalProposerPoll(WalProposer *wp) if (TimestampDifferenceExceeds(sk->latestMsgReceivedAt, now, wp->config->safekeeper_connection_timeout)) { - walprop_log(WARNING, "terminating connection to safekeeper '%s:%s' in '%s' state: no messages received during the last %dms or connection attempt took longer than that", - sk->host, sk->port, FormatSafekeeperState(sk), wp->config->safekeeper_connection_timeout); + wp_log(WARNING, "terminating connection to safekeeper '%s:%s' in '%s' state: no messages received during the last %dms or connection attempt took longer than that", + sk->host, sk->port, FormatSafekeeperState(sk), wp->config->safekeeper_connection_timeout); ShutdownConnection(sk); } } @@ -356,8 +356,8 @@ ResetConnection(Safekeeper *sk) * * https://www.postgresql.org/docs/devel/libpq-connect.html#LIBPQ-PQCONNECTSTARTPARAMS */ - walprop_log(WARNING, "Immediate failure to connect with node '%s:%s':\n\terror: %s", - sk->host, sk->port, wp->api.conn_error_message(sk)); + wp_log(WARNING, "immediate failure to connect with node '%s:%s':\n\terror: %s", + sk->host, sk->port, wp->api.conn_error_message(sk)); /* * Even though the connection failed, we still need to clean up the @@ -380,7 +380,7 @@ ResetConnection(Safekeeper *sk) * (see libpqrcv_connect, defined in * src/backend/replication/libpqwalreceiver/libpqwalreceiver.c) */ - walprop_log(LOG, "connecting with node %s:%s", sk->host, sk->port); + wp_log(LOG, "connecting with node %s:%s", sk->host, sk->port); sk->state = SS_CONNECTING_WRITE; sk->latestMsgReceivedAt = wp->api.get_current_timestamp(wp); @@ -434,7 +434,7 @@ ReconnectSafekeepers(WalProposer *wp) static void AdvancePollState(Safekeeper *sk, uint32 events) { -#ifdef WALPROPOSER_LIB /* walprop_log needs wp in lib build */ +#ifdef WALPROPOSER_LIB /* wp_log needs wp in lib build */ WalProposer *wp = sk->wp; #endif @@ -452,8 +452,8 @@ AdvancePollState(Safekeeper *sk, uint32 events) * ResetConnection */ case SS_OFFLINE: - walprop_log(FATAL, "Unexpected safekeeper %s:%s state advancement: is offline", - sk->host, sk->port); + wp_log(FATAL, "unexpected safekeeper %s:%s state advancement: is offline", + sk->host, sk->port); break; /* actually unreachable, but prevents * -Wimplicit-fallthrough */ @@ -488,8 +488,8 @@ AdvancePollState(Safekeeper *sk, uint32 events) * requests. */ case SS_VOTING: - walprop_log(WARNING, "EOF from node %s:%s in %s state", sk->host, - sk->port, FormatSafekeeperState(sk)); + wp_log(WARNING, "EOF from node %s:%s in %s state", sk->host, + sk->port, FormatSafekeeperState(sk)); ResetConnection(sk); return; @@ -517,8 +517,8 @@ AdvancePollState(Safekeeper *sk, uint32 events) * Idle state for waiting votes from quorum. */ case SS_IDLE: - walprop_log(WARNING, "EOF from node %s:%s in %s state", sk->host, - sk->port, FormatSafekeeperState(sk)); + wp_log(WARNING, "EOF from node %s:%s in %s state", sk->host, + sk->port, FormatSafekeeperState(sk)); ResetConnection(sk); return; @@ -543,8 +543,8 @@ HandleConnectionEvent(Safekeeper *sk) switch (result) { case WP_CONN_POLLING_OK: - walprop_log(LOG, "connected with node %s:%s", sk->host, - sk->port); + wp_log(LOG, "connected with node %s:%s", sk->host, + sk->port); sk->latestMsgReceivedAt = wp->api.get_current_timestamp(wp); /* @@ -567,8 +567,8 @@ HandleConnectionEvent(Safekeeper *sk) break; case WP_CONN_POLLING_FAILED: - walprop_log(WARNING, "failed to connect to node '%s:%s': %s", - sk->host, sk->port, wp->api.conn_error_message(sk)); + wp_log(WARNING, "failed to connect to node '%s:%s': %s", + sk->host, sk->port, wp->api.conn_error_message(sk)); /* * If connecting failed, we don't want to restart the connection @@ -604,8 +604,8 @@ SendStartWALPush(Safekeeper *sk) if (!wp->api.conn_send_query(sk, "START_WAL_PUSH")) { - walprop_log(WARNING, "Failed to send 'START_WAL_PUSH' query to safekeeper %s:%s: %s", - sk->host, sk->port, wp->api.conn_error_message(sk)); + wp_log(WARNING, "failed to send 'START_WAL_PUSH' query to safekeeper %s:%s: %s", + sk->host, sk->port, wp->api.conn_error_message(sk)); ShutdownConnection(sk); return; } @@ -641,8 +641,8 @@ RecvStartWALPushResult(Safekeeper *sk) break; case WP_EXEC_FAILED: - walprop_log(WARNING, "Failed to send query to safekeeper %s:%s: %s", - sk->host, sk->port, wp->api.conn_error_message(sk)); + wp_log(WARNING, "failed to send query to safekeeper %s:%s: %s", + sk->host, sk->port, wp->api.conn_error_message(sk)); ShutdownConnection(sk); return; @@ -652,8 +652,8 @@ RecvStartWALPushResult(Safekeeper *sk) * wrong" */ case WP_EXEC_UNEXPECTED_SUCCESS: - walprop_log(WARNING, "Received bad response from safekeeper %s:%s query execution", - sk->host, sk->port); + wp_log(WARNING, "received bad response from safekeeper %s:%s query execution", + sk->host, sk->port); ShutdownConnection(sk); return; } @@ -688,7 +688,7 @@ RecvAcceptorGreeting(Safekeeper *sk) if (!AsyncReadMessage(sk, (AcceptorProposerMessage *) &sk->greetResponse)) return; - walprop_log(LOG, "received AcceptorGreeting from safekeeper %s:%s", sk->host, sk->port); + wp_log(LOG, "received AcceptorGreeting from safekeeper %s:%s", sk->host, sk->port); /* Protocol is all good, move to voting. */ sk->state = SS_VOTING; @@ -708,7 +708,7 @@ RecvAcceptorGreeting(Safekeeper *sk) if (wp->n_connected == wp->quorum) { wp->propTerm++; - walprop_log(LOG, "proposer connected to quorum (%d) safekeepers, propTerm=" INT64_FORMAT, wp->quorum, wp->propTerm); + wp_log(LOG, "proposer connected to quorum (%d) safekeepers, propTerm=" INT64_FORMAT, wp->quorum, wp->propTerm); wp->voteRequest = (VoteRequest) { @@ -721,9 +721,9 @@ RecvAcceptorGreeting(Safekeeper *sk) else if (sk->greetResponse.term > wp->propTerm) { /* Another compute with higher term is running. */ - walprop_log(FATAL, "WAL acceptor %s:%s with term " INT64_FORMAT " rejects our connection request with term " INT64_FORMAT "", - sk->host, sk->port, - sk->greetResponse.term, wp->propTerm); + wp_log(FATAL, "WAL acceptor %s:%s with term " INT64_FORMAT " rejects our connection request with term " INT64_FORMAT "", + sk->host, sk->port, + sk->greetResponse.term, wp->propTerm); } /* @@ -763,7 +763,7 @@ SendVoteRequest(Safekeeper *sk) WalProposer *wp = sk->wp; /* We have quorum for voting, send our vote request */ - walprop_log(LOG, "requesting vote from %s:%s for term " UINT64_FORMAT, sk->host, sk->port, wp->voteRequest.term); + wp_log(LOG, "requesting vote from %s:%s for term " UINT64_FORMAT, sk->host, sk->port, wp->voteRequest.term); /* On failure, logging & resetting is handled */ if (!BlockingWrite(sk, &wp->voteRequest, sizeof(wp->voteRequest), SS_WAIT_VERDICT)) return; @@ -780,12 +780,12 @@ RecvVoteResponse(Safekeeper *sk) if (!AsyncReadMessage(sk, (AcceptorProposerMessage *) &sk->voteResponse)) return; - walprop_log(LOG, - "got VoteResponse from acceptor %s:%s, voteGiven=" UINT64_FORMAT ", epoch=" UINT64_FORMAT ", flushLsn=%X/%X, truncateLsn=%X/%X, timelineStartLsn=%X/%X", - sk->host, sk->port, sk->voteResponse.voteGiven, GetHighestTerm(&sk->voteResponse.termHistory), - LSN_FORMAT_ARGS(sk->voteResponse.flushLsn), - LSN_FORMAT_ARGS(sk->voteResponse.truncateLsn), - LSN_FORMAT_ARGS(sk->voteResponse.timelineStartLsn)); + wp_log(LOG, + "got VoteResponse from acceptor %s:%s, voteGiven=" UINT64_FORMAT ", epoch=" UINT64_FORMAT ", flushLsn=%X/%X, truncateLsn=%X/%X, timelineStartLsn=%X/%X", + sk->host, sk->port, sk->voteResponse.voteGiven, GetHighestTerm(&sk->voteResponse.termHistory), + LSN_FORMAT_ARGS(sk->voteResponse.flushLsn), + LSN_FORMAT_ARGS(sk->voteResponse.truncateLsn), + LSN_FORMAT_ARGS(sk->voteResponse.timelineStartLsn)); /* * In case of acceptor rejecting our vote, bail out, but only if either it @@ -795,9 +795,9 @@ RecvVoteResponse(Safekeeper *sk) if ((!sk->voteResponse.voteGiven) && (sk->voteResponse.term > wp->propTerm || wp->n_votes < wp->quorum)) { - walprop_log(FATAL, "WAL acceptor %s:%s with term " INT64_FORMAT " rejects our connection request with term " INT64_FORMAT "", - sk->host, sk->port, - sk->voteResponse.term, wp->propTerm); + wp_log(FATAL, "WAL acceptor %s:%s with term " INT64_FORMAT " rejects our connection request with term " INT64_FORMAT "", + sk->host, sk->port, + sk->voteResponse.term, wp->propTerm); } Assert(sk->voteResponse.term == wp->propTerm); @@ -841,7 +841,7 @@ HandleElectedProposer(WalProposer *wp) */ if (!wp->api.recovery_download(wp, &wp->safekeeper[wp->donor])) { - walprop_log(FATAL, "failed to download WAL for logical replicaiton"); + wp_log(FATAL, "failed to download WAL for logical replicaiton"); } if (wp->truncateLsn == wp->propEpochStartLsn && wp->config->syncSafekeepers) @@ -948,10 +948,10 @@ DetermineEpochStartLsn(WalProposer *wp) if (wp->timelineStartLsn != InvalidXLogRecPtr && wp->timelineStartLsn != wp->safekeeper[i].voteResponse.timelineStartLsn) { - walprop_log(WARNING, - "inconsistent timelineStartLsn: current %X/%X, received %X/%X", - LSN_FORMAT_ARGS(wp->timelineStartLsn), - LSN_FORMAT_ARGS(wp->safekeeper[i].voteResponse.timelineStartLsn)); + wp_log(WARNING, + "inconsistent timelineStartLsn: current %X/%X, received %X/%X", + LSN_FORMAT_ARGS(wp->timelineStartLsn), + LSN_FORMAT_ARGS(wp->safekeeper[i].voteResponse.timelineStartLsn)); } wp->timelineStartLsn = wp->safekeeper[i].voteResponse.timelineStartLsn; } @@ -969,7 +969,7 @@ DetermineEpochStartLsn(WalProposer *wp) { wp->timelineStartLsn = wp->api.get_redo_start_lsn(wp); } - walprop_log(LOG, "bumped epochStartLsn to the first record %X/%X", LSN_FORMAT_ARGS(wp->propEpochStartLsn)); + wp_log(LOG, "bumped epochStartLsn to the first record %X/%X", LSN_FORMAT_ARGS(wp->propEpochStartLsn)); } /* @@ -996,12 +996,12 @@ DetermineEpochStartLsn(WalProposer *wp) wp->propTermHistory.entries[wp->propTermHistory.n_entries - 1].term = wp->propTerm; wp->propTermHistory.entries[wp->propTermHistory.n_entries - 1].lsn = wp->propEpochStartLsn; - walprop_log(LOG, "got votes from majority (%d) of nodes, term " UINT64_FORMAT ", epochStartLsn %X/%X, donor %s:%s, truncate_lsn %X/%X", - wp->quorum, - wp->propTerm, - LSN_FORMAT_ARGS(wp->propEpochStartLsn), - wp->safekeeper[wp->donor].host, wp->safekeeper[wp->donor].port, - LSN_FORMAT_ARGS(wp->truncateLsn)); + wp_log(LOG, "got votes from majority (%d) of nodes, term " UINT64_FORMAT ", epochStartLsn %X/%X, donor %s:%s, truncate_lsn %X/%X", + wp->quorum, + wp->propTerm, + LSN_FORMAT_ARGS(wp->propEpochStartLsn), + wp->safekeeper[wp->donor].host, wp->safekeeper[wp->donor].port, + LSN_FORMAT_ARGS(wp->truncateLsn)); /* * Ensure the basebackup we are running (at RedoStartLsn) matches LSN @@ -1034,10 +1034,10 @@ DetermineEpochStartLsn(WalProposer *wp) * scenario. */ disable_core_dump(); - walprop_log(PANIC, - "collected propEpochStartLsn %X/%X, but basebackup LSN %X/%X", - LSN_FORMAT_ARGS(wp->propEpochStartLsn), - LSN_FORMAT_ARGS(wp->api.get_redo_start_lsn(wp))); + wp_log(PANIC, + "collected propEpochStartLsn %X/%X, but basebackup LSN %X/%X", + LSN_FORMAT_ARGS(wp->propEpochStartLsn), + LSN_FORMAT_ARGS(wp->api.get_redo_start_lsn(wp))); } } walprop_shared->mineLastElectedTerm = wp->propTerm; @@ -1115,9 +1115,9 @@ SendProposerElected(Safekeeper *sk) */ sk->startStreamingAt = wp->truncateLsn; - walprop_log(WARNING, "empty safekeeper joined cluster as %s:%s, historyStart=%X/%X, sk->startStreamingAt=%X/%X", - sk->host, sk->port, LSN_FORMAT_ARGS(wp->propTermHistory.entries[0].lsn), - LSN_FORMAT_ARGS(sk->startStreamingAt)); + wp_log(WARNING, "empty safekeeper joined cluster as %s:%s, historyStart=%X/%X, sk->startStreamingAt=%X/%X", + sk->host, sk->port, LSN_FORMAT_ARGS(wp->propTermHistory.entries[0].lsn), + LSN_FORMAT_ARGS(sk->startStreamingAt)); } } else @@ -1150,9 +1150,9 @@ SendProposerElected(Safekeeper *sk) msg.timelineStartLsn = wp->timelineStartLsn; lastCommonTerm = i >= 0 ? wp->propTermHistory.entries[i].term : 0; - walprop_log(LOG, - "sending elected msg to node " UINT64_FORMAT " term=" UINT64_FORMAT ", startStreamingAt=%X/%X (lastCommonTerm=" UINT64_FORMAT "), termHistory.n_entries=%u to %s:%s, timelineStartLsn=%X/%X", - sk->greetResponse.nodeId, msg.term, LSN_FORMAT_ARGS(msg.startStreamingAt), lastCommonTerm, msg.termHistory->n_entries, sk->host, sk->port, LSN_FORMAT_ARGS(msg.timelineStartLsn)); + wp_log(LOG, + "sending elected msg to node " UINT64_FORMAT " term=" UINT64_FORMAT ", startStreamingAt=%X/%X (lastCommonTerm=" UINT64_FORMAT "), termHistory.n_entries=%u to %s:%s, timelineStartLsn=%X/%X", + sk->greetResponse.nodeId, msg.term, LSN_FORMAT_ARGS(msg.startStreamingAt), lastCommonTerm, msg.termHistory->n_entries, sk->host, sk->port, LSN_FORMAT_ARGS(msg.timelineStartLsn)); resetStringInfo(&sk->outbuf); pq_sendint64_le(&sk->outbuf, msg.tag); @@ -1261,8 +1261,8 @@ HandleActiveState(Safekeeper *sk, uint32 events) /* expected never to happen, c.f. walprop_pg_active_state_update_event_set */ if (events & WL_SOCKET_CLOSED) { - walprop_log(WARNING, "connection to %s:%s in active state failed, got WL_SOCKET_CLOSED on neon_walreader socket", - sk->host, sk->port); + wp_log(WARNING, "connection to %s:%s in active state failed, got WL_SOCKET_CLOSED on neon_walreader socket", + sk->host, sk->port); ShutdownConnection(sk); return; } @@ -1323,12 +1323,12 @@ SendAppendRequests(Safekeeper *sk) req = &sk->appendRequest; PrepareAppendRequest(sk->wp, &sk->appendRequest, sk->streamingAt, endLsn); - walprop_log(DEBUG5, "sending message len %ld beginLsn=%X/%X endLsn=%X/%X commitLsn=%X/%X truncateLsn=%X/%X to %s:%s", - req->endLsn - req->beginLsn, - LSN_FORMAT_ARGS(req->beginLsn), - LSN_FORMAT_ARGS(req->endLsn), - LSN_FORMAT_ARGS(req->commitLsn), - LSN_FORMAT_ARGS(wp->truncateLsn), sk->host, sk->port); + wp_log(DEBUG5, "sending message len %ld beginLsn=%X/%X endLsn=%X/%X commitLsn=%X/%X truncateLsn=%X/%X to %s:%s", + req->endLsn - req->beginLsn, + LSN_FORMAT_ARGS(req->beginLsn), + LSN_FORMAT_ARGS(req->endLsn), + LSN_FORMAT_ARGS(req->commitLsn), + LSN_FORMAT_ARGS(wp->truncateLsn), sk->host, sk->port); resetStringInfo(&sk->outbuf); @@ -1355,8 +1355,8 @@ SendAppendRequests(Safekeeper *sk) case NEON_WALREAD_WOULDBLOCK: return true; case NEON_WALREAD_ERROR: - walprop_log(WARNING, "WAL reading for node %s:%s failed: %s", - sk->host, sk->port, errmsg); + wp_log(WARNING, "WAL reading for node %s:%s failed: %s", + sk->host, sk->port, errmsg); ShutdownConnection(sk); return false; default: @@ -1388,9 +1388,9 @@ SendAppendRequests(Safekeeper *sk) return true; case PG_ASYNC_WRITE_FAIL: - walprop_log(WARNING, "failed to send to node %s:%s in %s state: %s", - sk->host, sk->port, FormatSafekeeperState(sk), - wp->api.conn_error_message(sk)); + wp_log(WARNING, "failed to send to node %s:%s in %s state: %s", + sk->host, sk->port, FormatSafekeeperState(sk), + wp->api.conn_error_message(sk)); ShutdownConnection(sk); return false; default: @@ -1429,11 +1429,11 @@ RecvAppendResponses(Safekeeper *sk) if (!AsyncReadMessage(sk, (AcceptorProposerMessage *) &sk->appendResponse)) break; - walprop_log(DEBUG2, "received message term=" INT64_FORMAT " flushLsn=%X/%X commitLsn=%X/%X from %s:%s", - sk->appendResponse.term, - LSN_FORMAT_ARGS(sk->appendResponse.flushLsn), - LSN_FORMAT_ARGS(sk->appendResponse.commitLsn), - sk->host, sk->port); + wp_log(DEBUG2, "received message term=" INT64_FORMAT " flushLsn=%X/%X commitLsn=%X/%X from %s:%s", + sk->appendResponse.term, + LSN_FORMAT_ARGS(sk->appendResponse.flushLsn), + LSN_FORMAT_ARGS(sk->appendResponse.commitLsn), + sk->host, sk->port); if (sk->appendResponse.term > wp->propTerm) { @@ -1443,9 +1443,9 @@ RecvAppendResponses(Safekeeper *sk) * core as this is kinda expected scenario. */ disable_core_dump(); - walprop_log(PANIC, "WAL acceptor %s:%s with term " INT64_FORMAT " rejected our request, our term " INT64_FORMAT "", - sk->host, sk->port, - sk->appendResponse.term, wp->propTerm); + wp_log(PANIC, "WAL acceptor %s:%s with term " INT64_FORMAT " rejected our request, our term " INT64_FORMAT "", + sk->host, sk->port, + sk->appendResponse.term, wp->propTerm); } readAnything = true; @@ -1489,32 +1489,32 @@ ParsePageserverFeedbackMessage(WalProposer *wp, StringInfo reply_message, Pagese pq_getmsgint(reply_message, sizeof(int32)); /* read value length */ rf->currentClusterSize = pq_getmsgint64(reply_message); - walprop_log(DEBUG2, "ParsePageserverFeedbackMessage: current_timeline_size %lu", - rf->currentClusterSize); + wp_log(DEBUG2, "ParsePageserverFeedbackMessage: current_timeline_size %lu", + rf->currentClusterSize); } else if ((strcmp(key, "ps_writelsn") == 0) || (strcmp(key, "last_received_lsn") == 0)) { pq_getmsgint(reply_message, sizeof(int32)); /* read value length */ rf->last_received_lsn = pq_getmsgint64(reply_message); - walprop_log(DEBUG2, "ParsePageserverFeedbackMessage: last_received_lsn %X/%X", - LSN_FORMAT_ARGS(rf->last_received_lsn)); + wp_log(DEBUG2, "ParsePageserverFeedbackMessage: last_received_lsn %X/%X", + LSN_FORMAT_ARGS(rf->last_received_lsn)); } else if ((strcmp(key, "ps_flushlsn") == 0) || (strcmp(key, "disk_consistent_lsn") == 0)) { pq_getmsgint(reply_message, sizeof(int32)); /* read value length */ rf->disk_consistent_lsn = pq_getmsgint64(reply_message); - walprop_log(DEBUG2, "ParsePageserverFeedbackMessage: disk_consistent_lsn %X/%X", - LSN_FORMAT_ARGS(rf->disk_consistent_lsn)); + wp_log(DEBUG2, "ParsePageserverFeedbackMessage: disk_consistent_lsn %X/%X", + LSN_FORMAT_ARGS(rf->disk_consistent_lsn)); } else if ((strcmp(key, "ps_applylsn") == 0) || (strcmp(key, "remote_consistent_lsn") == 0)) { pq_getmsgint(reply_message, sizeof(int32)); /* read value length */ rf->remote_consistent_lsn = pq_getmsgint64(reply_message); - walprop_log(DEBUG2, "ParsePageserverFeedbackMessage: remote_consistent_lsn %X/%X", - LSN_FORMAT_ARGS(rf->remote_consistent_lsn)); + wp_log(DEBUG2, "ParsePageserverFeedbackMessage: remote_consistent_lsn %X/%X", + LSN_FORMAT_ARGS(rf->remote_consistent_lsn)); } else if ((strcmp(key, "ps_replytime") == 0) || (strcmp(key, "replytime") == 0)) { @@ -1526,8 +1526,8 @@ ParsePageserverFeedbackMessage(WalProposer *wp, StringInfo reply_message, Pagese /* Copy because timestamptz_to_str returns a static buffer */ replyTimeStr = pstrdup(timestamptz_to_str(rf->replytime)); - walprop_log(DEBUG2, "ParsePageserverFeedbackMessage: replytime %lu reply_time: %s", - rf->replytime, replyTimeStr); + wp_log(DEBUG2, "ParsePageserverFeedbackMessage: replytime %lu reply_time: %s", + rf->replytime, replyTimeStr); pfree(replyTimeStr); } @@ -1541,7 +1541,7 @@ ParsePageserverFeedbackMessage(WalProposer *wp, StringInfo reply_message, Pagese * Skip unknown keys to support backward compatibile protocol * changes */ - walprop_log(LOG, "ParsePageserverFeedbackMessage: unknown key: %s len %d", key, len); + wp_log(LOG, "ParsePageserverFeedbackMessage: unknown key: %s len %d", key, len); pq_getmsgbytes(reply_message, len); }; } @@ -1606,7 +1606,7 @@ GetDonor(WalProposer *wp, XLogRecPtr *donor_lsn) if (wp->n_votes < wp->quorum) { - walprop_log(WARNING, "GetDonor called before elections are won"); + wp_log(WARNING, "GetDonor called before elections are won"); return NULL; } @@ -1734,9 +1734,9 @@ AsyncRead(Safekeeper *sk, char **buf, int *buf_size) return false; case PG_ASYNC_READ_FAIL: - walprop_log(WARNING, "Failed to read from node %s:%s in %s state: %s", sk->host, - sk->port, FormatSafekeeperState(sk), - wp->api.conn_error_message(sk)); + wp_log(WARNING, "failed to read from node %s:%s in %s state: %s", sk->host, + sk->port, FormatSafekeeperState(sk), + wp->api.conn_error_message(sk)); ShutdownConnection(sk); return false; } @@ -1774,8 +1774,8 @@ AsyncReadMessage(Safekeeper *sk, AcceptorProposerMessage *anymsg) tag = pq_getmsgint64_le(&s); if (tag != anymsg->tag) { - walprop_log(WARNING, "unexpected message tag %c from node %s:%s in state %s", (char) tag, sk->host, - sk->port, FormatSafekeeperState(sk)); + wp_log(WARNING, "unexpected message tag %c from node %s:%s in state %s", (char) tag, sk->host, + sk->port, FormatSafekeeperState(sk)); ResetConnection(sk); return false; } @@ -1851,9 +1851,9 @@ BlockingWrite(Safekeeper *sk, void *msg, size_t msg_size, SafekeeperState succes if (!wp->api.conn_blocking_write(sk, msg, msg_size)) { - walprop_log(WARNING, "Failed to send to node %s:%s in %s state: %s", - sk->host, sk->port, FormatSafekeeperState(sk), - wp->api.conn_error_message(sk)); + wp_log(WARNING, "failed to send to node %s:%s in %s state: %s", + sk->host, sk->port, FormatSafekeeperState(sk), + wp->api.conn_error_message(sk)); ShutdownConnection(sk); return false; } @@ -1904,9 +1904,9 @@ AsyncWrite(Safekeeper *sk, void *msg, size_t msg_size, SafekeeperState flush_sta wp->api.update_event_set(sk, WL_SOCKET_READABLE | WL_SOCKET_WRITEABLE); return false; case PG_ASYNC_WRITE_FAIL: - walprop_log(WARNING, "Failed to send to node %s:%s in %s state: %s", - sk->host, sk->port, FormatSafekeeperState(sk), - wp->api.conn_error_message(sk)); + wp_log(WARNING, "failed to send to node %s:%s in %s state: %s", + sk->host, sk->port, FormatSafekeeperState(sk), + wp->api.conn_error_message(sk)); ShutdownConnection(sk); return false; default: @@ -1943,9 +1943,9 @@ AsyncFlush(Safekeeper *sk) /* Nothing to do; try again when the socket's ready */ return false; case -1: - walprop_log(WARNING, "Failed to flush write to node %s:%s in %s state: %s", - sk->host, sk->port, FormatSafekeeperState(sk), - wp->api.conn_error_message(sk)); + wp_log(WARNING, "failed to flush write to node %s:%s in %s state: %s", + sk->host, sk->port, FormatSafekeeperState(sk), + wp->api.conn_error_message(sk)); ResetConnection(sk); return false; default: @@ -1974,11 +1974,11 @@ CompareLsn(const void *a, const void *b) * * The strings are intended to be used as a prefix to "state", e.g.: * - * walprop_log(LOG, "currently in %s state", FormatSafekeeperState(sk)); + * wp_log(LOG, "currently in %s state", FormatSafekeeperState(sk)); * * If this sort of phrasing doesn't fit the message, instead use something like: * - * walprop_log(LOG, "currently in state [%s]", FormatSafekeeperState(sk)); + * wp_log(LOG, "currently in state [%s]", FormatSafekeeperState(sk)); */ static char * FormatSafekeeperState(Safekeeper *sk) @@ -2059,8 +2059,8 @@ AssertEventsOkForState(uint32 events, Safekeeper *sk) * To give a descriptive message in the case of failure, we use elog * and then an assertion that's guaranteed to fail. */ - walprop_log(WARNING, "events %s mismatched for safekeeper %s:%s in state [%s]", - FormatEvents(wp, events), sk->host, sk->port, FormatSafekeeperState(sk)); + wp_log(WARNING, "events %s mismatched for safekeeper %s:%s in state [%s]", + FormatEvents(wp, events), sk->host, sk->port, FormatSafekeeperState(sk)); Assert(events_ok_for_state); } } @@ -2199,8 +2199,8 @@ FormatEvents(WalProposer *wp, uint32 events) if (events & (~all_flags)) { - walprop_log(WARNING, "Event formatting found unexpected component %d", - events & (~all_flags)); + wp_log(WARNING, "event formatting found unexpected component %d", + events & (~all_flags)); return_str[6] = '*'; return_str[7] = '\0'; } diff --git a/pgxn/neon/walproposer.h b/pgxn/neon/walproposer.h index 6d478076fe..688d8e6e52 100644 --- a/pgxn/neon/walproposer.h +++ b/pgxn/neon/walproposer.h @@ -707,11 +707,23 @@ extern Safekeeper *GetDonor(WalProposer *wp, XLogRecPtr *donor_lsn); #define WPEVENT 1337 /* special log level for walproposer internal * events */ +#define WP_LOG_PREFIX "[WP] " + +/* + * wp_log is used in pure wp code (walproposer.c), allowing API callback to + * catch logging. + */ #ifdef WALPROPOSER_LIB extern void WalProposerLibLog(WalProposer *wp, int elevel, char *fmt,...); -#define walprop_log(elevel, ...) WalProposerLibLog(wp, elevel, __VA_ARGS__) +#define wp_log(elevel, fmt, ...) WalProposerLibLog(wp, elevel, fmt, ## __VA_ARGS__) #else -#define walprop_log(elevel, ...) elog(elevel, __VA_ARGS__) +#define wp_log(elevel, fmt, ...) elog(elevel, WP_LOG_PREFIX fmt, ## __VA_ARGS__) #endif +/* + * And wpg_log is used all other (postgres specific) walproposer code, just + * adding prefix. + */ +#define wpg_log(elevel, fmt, ...) elog(elevel, WP_LOG_PREFIX fmt, ## __VA_ARGS__) + #endif /* __NEON_WALPROPOSER_H__ */ diff --git a/pgxn/neon/walproposer_pg.c b/pgxn/neon/walproposer_pg.c index 7773aabfab..a3edffa6cb 100644 --- a/pgxn/neon/walproposer_pg.c +++ b/pgxn/neon/walproposer_pg.c @@ -424,8 +424,8 @@ walprop_pg_start_streaming(WalProposer *wp, XLogRecPtr startpos) { StartReplicationCmd cmd; - elog(LOG, "WAL proposer starts streaming at %X/%X", - LSN_FORMAT_ARGS(startpos)); + wpg_log(LOG, "WAL proposer starts streaming at %X/%X", + LSN_FORMAT_ARGS(startpos)); cmd.slotname = WAL_PROPOSER_SLOT_NAME; cmd.timeline = wp->greetRequest.timeline; cmd.startpoint = startpos; @@ -549,7 +549,7 @@ walprop_pg_load_libpqwalreceiver(void) { load_file("libpqwalreceiver", false); if (WalReceiverFunctions == NULL) - elog(ERROR, "libpqwalreceiver didn't initialize correctly"); + wpg_log(ERROR, "libpqwalreceiver didn't initialize correctly"); } /* Helper function */ @@ -630,7 +630,7 @@ libpqwp_connect_start(char *conninfo) * PGconn structure" */ if (!pg_conn) - elog(FATAL, "failed to allocate new PGconn object"); + wpg_log(FATAL, "failed to allocate new PGconn object"); /* * And in theory this allocation can fail as well, but it's incredibly @@ -680,7 +680,7 @@ walprop_connect_poll(Safekeeper *sk) * unused. We'll expect it's never returned. */ case PGRES_POLLING_ACTIVE: - elog(FATAL, "Unexpected PGRES_POLLING_ACTIVE returned from PQconnectPoll"); + wpg_log(FATAL, "unexpected PGRES_POLLING_ACTIVE returned from PQconnectPoll"); /* * This return is never actually reached, but it's here to make @@ -745,7 +745,7 @@ libpqwp_get_query_result(WalProposerConn *conn) */ if (!result) { - elog(WARNING, "[libpqwalproposer] Unexpected successful end of command results"); + wpg_log(WARNING, "[libpqwalproposer] Unexpected successful end of command results"); return WP_EXEC_UNEXPECTED_SUCCESS; } @@ -793,7 +793,7 @@ libpqwp_get_query_result(WalProposerConn *conn) } if (unexpected_success) - elog(WARNING, "[libpqwalproposer] Unexpected successful %s", unexpected_success); + wpg_log(WARNING, "[libpqwalproposer] Unexpected successful %s", unexpected_success); return return_val; } @@ -872,7 +872,7 @@ libpqwp_async_read(WalProposerConn *conn, char **buf, int *amount) ExecStatusType status = PQresultStatus(PQgetResult(conn->pg_conn)); if (status != PGRES_FATAL_ERROR) - elog(FATAL, "unexpected result status %d after failed PQgetCopyData", status); + wpg_log(FATAL, "unexpected result status %d after failed PQgetCopyData", status); /* * If there was actually an error, it'll be properly reported @@ -937,7 +937,7 @@ walprop_async_write(Safekeeper *sk, void const *buf, size_t size) case -1: return PG_ASYNC_WRITE_FAIL; default: - elog(FATAL, "invalid return %d from PQputCopyData", result); + wpg_log(FATAL, "invalid return %d from PQputCopyData", result); } /* @@ -958,7 +958,7 @@ walprop_async_write(Safekeeper *sk, void const *buf, size_t size) case -1: return PG_ASYNC_WRITE_FAIL; default: - elog(FATAL, "invalid return %d from PQflush", result); + wpg_log(FATAL, "invalid return %d from PQflush", result); } } @@ -1247,8 +1247,8 @@ WalProposerRecovery(WalProposer *wp, Safekeeper *sk) if (max_slot_wal_keep_size_mb > 0 && download_range_mb >= max_slot_wal_keep_size_mb) { startpos = endpos - max_slot_wal_keep_size_mb * 1024 * 1024; - walprop_log(WARNING, "capped WAL download for logical replication to %X/%X as max_slot_wal_keep_size=%dMB", - LSN_FORMAT_ARGS(startpos), max_slot_wal_keep_size_mb); + wpg_log(WARNING, "capped WAL download for logical replication to %X/%X as max_slot_wal_keep_size=%dMB", + LSN_FORMAT_ARGS(startpos), max_slot_wal_keep_size_mb); } timeline = wp->greetRequest.timeline; @@ -1262,7 +1262,7 @@ WalProposerRecovery(WalProposer *wp, Safekeeper *sk) written = snprintf((char *) conninfo, MAXCONNINFO, "password=%s %s", neon_auth_token, sk->conninfo); if (written > MAXCONNINFO || written < 0) - elog(FATAL, "could not append password to the safekeeper connection string"); + wpg_log(FATAL, "could not append password to the safekeeper connection string"); } #if PG_MAJORVERSION_NUM < 16 @@ -1279,11 +1279,11 @@ WalProposerRecovery(WalProposer *wp, Safekeeper *sk) err))); return false; } - elog(LOG, - "start recovery for logical replication from %s:%s starting from %X/%08X till %X/%08X timeline " - "%d", - sk->host, sk->port, (uint32) (startpos >> 32), - (uint32) startpos, (uint32) (endpos >> 32), (uint32) endpos, timeline); + wpg_log(LOG, + "start recovery for logical replication from %s:%s starting from %X/%08X till %X/%08X timeline " + "%d", + sk->host, sk->port, (uint32) (startpos >> 32), + (uint32) startpos, (uint32) (endpos >> 32), (uint32) endpos, timeline); options.logical = false; options.startpoint = startpos; @@ -1481,11 +1481,11 @@ walprop_pg_wal_reader_allocate(Safekeeper *sk) { char log_prefix[64]; - snprintf(log_prefix, sizeof(log_prefix), "sk %s:%s nwr: ", sk->host, sk->port); + snprintf(log_prefix, sizeof(log_prefix), WP_LOG_PREFIX "sk %s:%s nwr: ", sk->host, sk->port); Assert(!sk->xlogreader); sk->xlogreader = NeonWALReaderAllocate(wal_segment_size, sk->wp->propEpochStartLsn, sk->wp, log_prefix); if (sk->xlogreader == NULL) - elog(FATAL, "Failed to allocate xlog reader"); + wpg_log(FATAL, "failed to allocate xlog reader"); } static NeonWALReadResult @@ -1549,7 +1549,7 @@ static void walprop_pg_init_event_set(WalProposer *wp) { if (waitEvents) - elog(FATAL, "double-initialization of event set"); + wpg_log(FATAL, "double-initialization of event set"); /* for each sk, we have socket plus potentially socket for neon walreader */ waitEvents = CreateWaitEventSet(TopMemoryContext, 2 + 2 * wp->n_safekeepers); @@ -1581,7 +1581,7 @@ add_nwr_event_set(Safekeeper *sk, uint32 events) Assert(sk->nwrEventPos == -1); sk->nwrEventPos = AddWaitEventToSet(waitEvents, events, NeonWALReaderSocket(sk->xlogreader), NULL, sk); sk->nwrConnEstablished = NeonWALReaderIsRemConnEstablished(sk->xlogreader); - elog(DEBUG5, "sk %s:%s: added nwr socket events %d", sk->host, sk->port, events); + wpg_log(DEBUG5, "sk %s:%s: added nwr socket events %d", sk->host, sk->port, events); } static void @@ -1680,8 +1680,8 @@ rm_safekeeper_event_set(Safekeeper *to_remove, bool is_sk) { WalProposer *wp = to_remove->wp; - elog(DEBUG5, "sk %s:%s: removing event, is_sk %d", - to_remove->host, to_remove->port, is_sk); + wpg_log(DEBUG5, "sk %s:%s: removing event, is_sk %d", + to_remove->host, to_remove->port, is_sk); /* * Shortpath for exiting if have nothing to do. We never call this @@ -1835,13 +1835,13 @@ GetLatestNeonFeedback(PageserverFeedback *rf, WalProposer *wp) rf->remote_consistent_lsn = wp->safekeeper[latest_safekeeper].appendResponse.rf.remote_consistent_lsn; rf->replytime = wp->safekeeper[latest_safekeeper].appendResponse.rf.replytime; - elog(DEBUG2, "GetLatestNeonFeedback: currentClusterSize %lu," - " last_received_lsn %X/%X, disk_consistent_lsn %X/%X, remote_consistent_lsn %X/%X, replytime %lu", - rf->currentClusterSize, - LSN_FORMAT_ARGS(rf->last_received_lsn), - LSN_FORMAT_ARGS(rf->disk_consistent_lsn), - LSN_FORMAT_ARGS(rf->remote_consistent_lsn), - rf->replytime); + wpg_log(DEBUG2, "GetLatestNeonFeedback: currentClusterSize %lu," + " last_received_lsn %X/%X, disk_consistent_lsn %X/%X, remote_consistent_lsn %X/%X, replytime %lu", + rf->currentClusterSize, + LSN_FORMAT_ARGS(rf->last_received_lsn), + LSN_FORMAT_ARGS(rf->disk_consistent_lsn), + LSN_FORMAT_ARGS(rf->remote_consistent_lsn), + rf->replytime); } /* @@ -1987,7 +1987,7 @@ GetLogRepRestartLSN(WalProposer *wp) { uint64 download_range_mb; - elog(LOG, "logical replication restart LSN %X/%X", LSN_FORMAT_ARGS(lrRestartLsn)); + wpg_log(LOG, "logical replication restart LSN %X/%X", LSN_FORMAT_ARGS(lrRestartLsn)); /* * If we need to download more than a max_slot_wal_keep_size, @@ -1999,8 +1999,8 @@ GetLogRepRestartLSN(WalProposer *wp) download_range_mb = (wp->propEpochStartLsn - lrRestartLsn) / MB; if (max_slot_wal_keep_size_mb > 0 && download_range_mb >= max_slot_wal_keep_size_mb) { - walprop_log(WARNING, "not downloading WAL for logical replication since %X/%X as max_slot_wal_keep_size=%dMB", - LSN_FORMAT_ARGS(lrRestartLsn), max_slot_wal_keep_size_mb); + wpg_log(WARNING, "not downloading WAL for logical replication since %X/%X as max_slot_wal_keep_size=%dMB", + LSN_FORMAT_ARGS(lrRestartLsn), max_slot_wal_keep_size_mb); return InvalidXLogRecPtr; } From f71110383c6647c0bf81f1c1f516a7c3decc8d66 Mon Sep 17 00:00:00 2001 From: Arseny Sher Date: Tue, 2 Jan 2024 10:10:41 +0300 Subject: [PATCH 14/35] Remove second check for max_slot_wal_keep_size download size. Already checked in GetLogRepRestartLSN, a rebase artifact. --- pgxn/neon/walproposer_pg.c | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/pgxn/neon/walproposer_pg.c b/pgxn/neon/walproposer_pg.c index a3edffa6cb..61a2a54809 100644 --- a/pgxn/neon/walproposer_pg.c +++ b/pgxn/neon/walproposer_pg.c @@ -1237,19 +1237,6 @@ WalProposerRecovery(WalProposer *wp, Safekeeper *sk) return true; /* recovery not needed */ endpos = wp->propEpochStartLsn; - /* - * If we need to download more than a max_slot_wal_keep_size, cap to it to - * avoid risk of exploding pg_wal. Logical replication won't work until - * recreated, but at least compute would start; this also follows - * max_slot_wal_keep_size semantics. - */ - download_range_mb = (endpos - startpos) / 1024 / 1024; - if (max_slot_wal_keep_size_mb > 0 && download_range_mb >= max_slot_wal_keep_size_mb) - { - startpos = endpos - max_slot_wal_keep_size_mb * 1024 * 1024; - wpg_log(WARNING, "capped WAL download for logical replication to %X/%X as max_slot_wal_keep_size=%dMB", - LSN_FORMAT_ARGS(startpos), max_slot_wal_keep_size_mb); - } timeline = wp->greetRequest.timeline; if (!neon_auth_token) From 42f41afcbd96be8dae785329495a7eed6cf55876 Mon Sep 17 00:00:00 2001 From: John Spray Date: Wed, 3 Jan 2024 10:36:53 +0000 Subject: [PATCH 15/35] tests: update pytest and boto3 dependencies (#6253) ## Problem The version of pytest we were using emits a number of DeprecationWarnings on latest python: these are fixed in latest release. boto3 and python-dateutil also have deprecation warnings, but unfortunately these aren't fixed upstream yet. ## Summary of changes - Update pytest - Update boto3 (this doesn't fix deprecation warnings, but by the time I figured that out I had already done the update, and it's good hygiene anyway) --- poetry.lock | 49 ++++++++++++++++++++++++++----------------------- pyproject.toml | 4 ++-- 2 files changed, 28 insertions(+), 25 deletions(-) diff --git a/poetry.lock b/poetry.lock index 76dfd6d37d..c597d811bd 100644 --- a/poetry.lock +++ b/poetry.lock @@ -339,19 +339,19 @@ uvloop = ["uvloop (>=0.15.2)"] [[package]] name = "boto3" -version = "1.26.16" +version = "1.34.11" description = "The AWS SDK for Python" optional = false -python-versions = ">= 3.7" +python-versions = ">= 3.8" files = [ - {file = "boto3-1.26.16-py3-none-any.whl", hash = "sha256:4f493a2aed71cee93e626de4f67ce58dd82c0473480a0fc45b131715cd8f4f30"}, - {file = "boto3-1.26.16.tar.gz", hash = "sha256:31c0adf71e4bd19a5428580bb229d7ea3b5795eecaa0847a85385df00c026116"}, + {file = "boto3-1.34.11-py3-none-any.whl", hash = "sha256:1af021e0c6e3040e8de66d403e963566476235bb70f9a8e3f6784813ac2d8026"}, + {file = "boto3-1.34.11.tar.gz", hash = "sha256:31c130a40ec0631059b77d7e87f67ad03ff1685a5b37638ac0c4687026a3259d"}, ] [package.dependencies] -botocore = ">=1.29.16,<1.30.0" +botocore = ">=1.34.11,<1.35.0" jmespath = ">=0.7.1,<2.0.0" -s3transfer = ">=0.6.0,<0.7.0" +s3transfer = ">=0.10.0,<0.11.0" [package.extras] crt = ["botocore[crt] (>=1.21.0,<2.0a0)"] @@ -702,22 +702,25 @@ xray = ["mypy-boto3-xray (>=1.26.0,<1.27.0)"] [[package]] name = "botocore" -version = "1.29.16" +version = "1.34.11" description = "Low-level, data-driven core of boto 3." optional = false -python-versions = ">= 3.7" +python-versions = ">= 3.8" files = [ - {file = "botocore-1.29.16-py3-none-any.whl", hash = "sha256:271b599e6cfe214405ed50d41cd967add1d5d469383dd81ff583bc818b47f59b"}, - {file = "botocore-1.29.16.tar.gz", hash = "sha256:8cfcc10f2f1751608c3cec694f2d6b5e16ebcd50d0a104f9914d5616227c62e9"}, + {file = "botocore-1.34.11-py3-none-any.whl", hash = "sha256:1ff1398b6ea670e1c01ac67a33af3da854f8e700d3528289c04f319c330d8250"}, + {file = "botocore-1.34.11.tar.gz", hash = "sha256:51905c3d623c60df5dc5794387de7caf886d350180a01a3dfa762e903edb45a9"}, ] [package.dependencies] jmespath = ">=0.7.1,<2.0.0" python-dateutil = ">=2.1,<3.0.0" -urllib3 = ">=1.25.4,<1.27" +urllib3 = [ + {version = ">=1.25.4,<1.27", markers = "python_version < \"3.10\""}, + {version = ">=1.25.4,<2.1", markers = "python_version >= \"3.10\""}, +] [package.extras] -crt = ["awscrt (==0.14.0)"] +crt = ["awscrt (==0.19.19)"] [[package]] name = "botocore-stubs" @@ -1889,13 +1892,13 @@ files = [ [[package]] name = "pytest" -version = "7.3.1" +version = "7.4.4" description = "pytest: simple powerful testing with Python" optional = false python-versions = ">=3.7" files = [ - {file = "pytest-7.3.1-py3-none-any.whl", hash = "sha256:3799fa815351fea3a5e96ac7e503a96fa51cc9942c3753cda7651b93c1cfa362"}, - {file = "pytest-7.3.1.tar.gz", hash = "sha256:434afafd78b1d78ed0addf160ad2b77a30d35d4bdf8af234fe621919d9ed15e3"}, + {file = "pytest-7.4.4-py3-none-any.whl", hash = "sha256:b090cdf5ed60bf4c45261be03239c2c1c22df034fbffe691abe93cd80cea01d8"}, + {file = "pytest-7.4.4.tar.gz", hash = "sha256:2cf0005922c6ace4a3e2ec8b4080eb0d9753fdc93107415332f50ce9e7994280"}, ] [package.dependencies] @@ -1907,7 +1910,7 @@ pluggy = ">=0.12,<2.0" tomli = {version = ">=1.0.0", markers = "python_version < \"3.11\""} [package.extras] -testing = ["argcomplete", "attrs (>=19.2.0)", "hypothesis (>=3.56)", "mock", "nose", "pygments (>=2.7.2)", "requests", "xmlschema"] +testing = ["argcomplete", "attrs (>=19.2.0)", "hypothesis (>=3.56)", "mock", "nose", "pygments (>=2.7.2)", "requests", "setuptools", "xmlschema"] [[package]] name = "pytest-asyncio" @@ -2230,20 +2233,20 @@ files = [ [[package]] name = "s3transfer" -version = "0.6.0" +version = "0.10.0" description = "An Amazon S3 Transfer Manager" optional = false -python-versions = ">= 3.7" +python-versions = ">= 3.8" files = [ - {file = "s3transfer-0.6.0-py3-none-any.whl", hash = "sha256:06176b74f3a15f61f1b4f25a1fc29a4429040b7647133a463da8fa5bd28d5ecd"}, - {file = "s3transfer-0.6.0.tar.gz", hash = "sha256:2ed07d3866f523cc561bf4a00fc5535827981b117dd7876f036b0c1aca42c947"}, + {file = "s3transfer-0.10.0-py3-none-any.whl", hash = "sha256:3cdb40f5cfa6966e812209d0994f2a4709b561c88e90cf00c2696d2df4e56b2e"}, + {file = "s3transfer-0.10.0.tar.gz", hash = "sha256:d0c8bbf672d5eebbe4e57945e23b972d963f07d82f661cabf678a5c88831595b"}, ] [package.dependencies] -botocore = ">=1.12.36,<2.0a.0" +botocore = ">=1.33.2,<2.0a.0" [package.extras] -crt = ["botocore[crt] (>=1.20.29,<2.0a.0)"] +crt = ["botocore[crt] (>=1.33.2,<2.0a.0)"] [[package]] name = "sarif-om" @@ -2740,4 +2743,4 @@ cffi = ["cffi (>=1.11)"] [metadata] lock-version = "2.0" python-versions = "^3.9" -content-hash = "c4e38082d246636903e15c02fbf8364c6afc1fd35d36a81c49f596ba68fc739b" +content-hash = "8de8b05a9b35a6f76da7d7e3652ddbb521f1eca53fce7b933f537080a9d6eada" diff --git a/pyproject.toml b/pyproject.toml index 401acaeba4..18c8ece4a7 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -6,7 +6,7 @@ authors = [] [tool.poetry.dependencies] python = "^3.9" -pytest = "^7.3.1" +pytest = "^7.4.4" psycopg2-binary = "^2.9.6" typing-extensions = "^4.6.1" PyJWT = {version = "^2.1.0", extras = ["crypto"]} @@ -17,7 +17,7 @@ aiopg = "^1.4.0" Jinja2 = "^3.0.2" types-requests = "^2.31.0.0" types-psycopg2 = "^2.9.21.10" -boto3 = "^1.26.16" +boto3 = "^1.34.11" boto3-stubs = {extras = ["s3"], version = "^1.26.16"} moto = {extras = ["server"], version = "^4.1.2"} backoff = "^2.2.1" From fb518aea0db046817987a463b1556ad950e97f09 Mon Sep 17 00:00:00 2001 From: Cuong Nguyen Date: Wed, 3 Jan 2024 05:41:58 -0500 Subject: [PATCH 16/35] Add batch ingestion mechanism to avoid high contention (#5886) ## Problem For context, this problem was observed in a research project where we try to make neon run in multiple regions and I was asked by @hlinnaka to make this PR. In our project, we use the pageserver in a non-conventional way such that we would send a larger number of requests to the pageserver than normal (imagine postgres without the buffer pool). I measured the time from the moment a WAL record left the safekeeper to when it reached the pageserver ([code](https://github.com/umd-dslam/sunstorm-neon/blob/e593db1f5ab2505eb176c9faaf2e9b9ba36cb2c4/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs#L282-L287)) and observed that when the number of get_page_at_lsn requests was high, the wal receiving time increased significantly (see the left side of the graphs below). Upon further investigation, I found that the delay was caused by this line https://github.com/neondatabase/neon/blob/d2ca4109191e92a9da340184e5bc71768853fe8e/pageserver/src/tenant/timeline.rs#L2348 The `get_layer_for_write` method is called for every value during WAL ingestion and it tries to acquire layers write lock every time, thus this results in high contention when read lock is acquired more frequently. ![Untitled](https://github.com/neondatabase/neon/assets/6244849/85460f4d-ead1-4532-bc64-736d0bfd7f16) ![Untitled2](https://github.com/neondatabase/neon/assets/6244849/84199ab7-5f0e-413b-a42b-f728f2225218) ## Summary of changes It is unnecessary to call `get_layer_for_write` repeatedly for all values in a WAL message since they would end up in the same memory layer anyway, so I created the batched versions of `InMemoryLayer::put_value`, `InMemoryLayer ::put_tombstone`, `Timeline::put_value`, and `Timeline::put_tombstone`, that acquire the locks once for a batch of values. Additionally, `DatadirModification` is changed to store multiple versions of uncommitted values, and `WalIngest::ingest_record()` can now ingest records without immediately committing them. With these new APIs, the new ingestion loop can be changed to commit for every `ingest_batch_size` records. The `ingest_batch_size` variable is exposed as a config. If it is set to 1 then we get the same behavior before this change. I found that setting this value to 100 seems to work the best, and you can see its effect on the right side of the above graphs. --------- Co-authored-by: John Spray --- pageserver/src/basebackup.rs | 15 +- pageserver/src/config.rs | 28 +- pageserver/src/import_datadir.rs | 9 +- pageserver/src/page_service.rs | 20 +- pageserver/src/pgdatadir_mapping.rs | 237 ++++++++++----- pageserver/src/tenant/config.rs | 2 + .../tenant/storage_layer/inmemory_layer.rs | 43 ++- pageserver/src/tenant/timeline.rs | 41 ++- pageserver/src/tenant/timeline/walreceiver.rs | 1 + .../walreceiver/connection_manager.rs | 3 + .../walreceiver/walreceiver_connection.rs | 36 ++- pageserver/src/walingest.rs | 272 ++++++++++-------- 12 files changed, 468 insertions(+), 239 deletions(-) diff --git a/pageserver/src/basebackup.rs b/pageserver/src/basebackup.rs index ed452eae7d..7e5ae892ad 100644 --- a/pageserver/src/basebackup.rs +++ b/pageserver/src/basebackup.rs @@ -23,6 +23,7 @@ use tracing::*; use tokio_tar::{Builder, EntryType, Header}; use crate::context::RequestContext; +use crate::pgdatadir_mapping::Version; use crate::tenant::Timeline; use pageserver_api::reltag::{RelTag, SlruKind}; @@ -174,7 +175,7 @@ where ] { for segno in self .timeline - .list_slru_segments(kind, self.lsn, self.ctx) + .list_slru_segments(kind, Version::Lsn(self.lsn), self.ctx) .await? { self.add_slru_segment(kind, segno).await?; @@ -192,7 +193,7 @@ where // Otherwise only include init forks of unlogged relations. let rels = self .timeline - .list_rels(spcnode, dbnode, self.lsn, self.ctx) + .list_rels(spcnode, dbnode, Version::Lsn(self.lsn), self.ctx) .await?; for &rel in rels.iter() { // Send init fork as main fork to provide well formed empty @@ -267,7 +268,7 @@ where async fn add_rel(&mut self, src: RelTag, dst: RelTag) -> anyhow::Result<()> { let nblocks = self .timeline - .get_rel_size(src, self.lsn, false, self.ctx) + .get_rel_size(src, Version::Lsn(self.lsn), false, self.ctx) .await?; // If the relation is empty, create an empty file @@ -288,7 +289,7 @@ where for blknum in startblk..endblk { let img = self .timeline - .get_rel_page_at_lsn(src, blknum, self.lsn, false, self.ctx) + .get_rel_page_at_lsn(src, blknum, Version::Lsn(self.lsn), false, self.ctx) .await?; segment_data.extend_from_slice(&img[..]); } @@ -310,7 +311,7 @@ where async fn add_slru_segment(&mut self, slru: SlruKind, segno: u32) -> anyhow::Result<()> { let nblocks = self .timeline - .get_slru_segment_size(slru, segno, self.lsn, self.ctx) + .get_slru_segment_size(slru, segno, Version::Lsn(self.lsn), self.ctx) .await?; let mut slru_buf: Vec = Vec::with_capacity(nblocks as usize * BLCKSZ as usize); @@ -352,7 +353,7 @@ where let relmap_img = if has_relmap_file { let img = self .timeline - .get_relmap_file(spcnode, dbnode, self.lsn, self.ctx) + .get_relmap_file(spcnode, dbnode, Version::Lsn(self.lsn), self.ctx) .await?; ensure!( @@ -399,7 +400,7 @@ where if !has_relmap_file && self .timeline - .list_rels(spcnode, dbnode, self.lsn, self.ctx) + .list_rels(spcnode, dbnode, Version::Lsn(self.lsn), self.ctx) .await? .is_empty() { diff --git a/pageserver/src/config.rs b/pageserver/src/config.rs index 8516f397ca..4560f5eca0 100644 --- a/pageserver/src/config.rs +++ b/pageserver/src/config.rs @@ -76,6 +76,8 @@ pub mod defaults { pub const DEFAULT_HEATMAP_UPLOAD_CONCURRENCY: usize = 8; + pub const DEFAULT_INGEST_BATCH_SIZE: u64 = 100; + /// /// Default built-in configuration file. /// @@ -88,6 +90,7 @@ pub mod defaults { #wait_lsn_timeout = '{DEFAULT_WAIT_LSN_TIMEOUT}' #wal_redo_timeout = '{DEFAULT_WAL_REDO_TIMEOUT}' +#page_cache_size = {DEFAULT_PAGE_CACHE_SIZE} #max_file_descriptors = {DEFAULT_MAX_FILE_DESCRIPTORS} # initial superuser role name to use when creating a new tenant @@ -108,6 +111,8 @@ pub mod defaults { #background_task_maximum_delay = '{DEFAULT_BACKGROUND_TASK_MAXIMUM_DELAY}' +#ingest_batch_size = {DEFAULT_INGEST_BATCH_SIZE} + [tenant_config] #checkpoint_distance = {DEFAULT_CHECKPOINT_DISTANCE} # in bytes #checkpoint_timeout = {DEFAULT_CHECKPOINT_TIMEOUT} @@ -233,6 +238,9 @@ pub struct PageServerConf { /// How many heatmap uploads may be done concurrency: lower values implicitly deprioritize /// heatmap uploads vs. other remote storage operations. pub heatmap_upload_concurrency: usize, + + /// Maximum number of WAL records to be ingested and committed at the same time + pub ingest_batch_size: u64, } /// We do not want to store this in a PageServerConf because the latter may be logged @@ -314,6 +322,8 @@ struct PageServerConfigBuilder { control_plane_emergency_mode: BuilderValue, heatmap_upload_concurrency: BuilderValue, + + ingest_batch_size: BuilderValue, } impl Default for PageServerConfigBuilder { @@ -386,6 +396,8 @@ impl Default for PageServerConfigBuilder { control_plane_emergency_mode: Set(false), heatmap_upload_concurrency: Set(DEFAULT_HEATMAP_UPLOAD_CONCURRENCY), + + ingest_batch_size: Set(DEFAULT_INGEST_BATCH_SIZE), } } } @@ -534,6 +546,10 @@ impl PageServerConfigBuilder { self.heatmap_upload_concurrency = BuilderValue::Set(value) } + pub fn ingest_batch_size(&mut self, ingest_batch_size: u64) { + self.ingest_batch_size = BuilderValue::Set(ingest_batch_size) + } + pub fn build(self) -> anyhow::Result { let concurrent_tenant_warmup = self .concurrent_tenant_warmup @@ -632,10 +648,12 @@ impl PageServerConfigBuilder { control_plane_emergency_mode: self .control_plane_emergency_mode .ok_or(anyhow!("missing control_plane_emergency_mode"))?, - heatmap_upload_concurrency: self .heatmap_upload_concurrency .ok_or(anyhow!("missing heatmap_upload_concurrency"))?, + ingest_batch_size: self + .ingest_batch_size + .ok_or(anyhow!("missing ingest_batch_size"))?, }) } } @@ -878,6 +896,7 @@ impl PageServerConf { "heatmap_upload_concurrency" => { builder.heatmap_upload_concurrency(parse_toml_u64(key, item)? as usize) }, + "ingest_batch_size" => builder.ingest_batch_size(parse_toml_u64(key, item)?), _ => bail!("unrecognized pageserver option '{key}'"), } } @@ -949,6 +968,7 @@ impl PageServerConf { control_plane_api_token: None, control_plane_emergency_mode: false, heatmap_upload_concurrency: defaults::DEFAULT_HEATMAP_UPLOAD_CONCURRENCY, + ingest_batch_size: defaults::DEFAULT_INGEST_BATCH_SIZE, } } } @@ -1177,7 +1197,8 @@ background_task_maximum_delay = '334 s' control_plane_api: None, control_plane_api_token: None, control_plane_emergency_mode: false, - heatmap_upload_concurrency: defaults::DEFAULT_HEATMAP_UPLOAD_CONCURRENCY + heatmap_upload_concurrency: defaults::DEFAULT_HEATMAP_UPLOAD_CONCURRENCY, + ingest_batch_size: defaults::DEFAULT_INGEST_BATCH_SIZE, }, "Correct defaults should be used when no config values are provided" ); @@ -1238,7 +1259,8 @@ background_task_maximum_delay = '334 s' control_plane_api: None, control_plane_api_token: None, control_plane_emergency_mode: false, - heatmap_upload_concurrency: defaults::DEFAULT_HEATMAP_UPLOAD_CONCURRENCY + heatmap_upload_concurrency: defaults::DEFAULT_HEATMAP_UPLOAD_CONCURRENCY, + ingest_batch_size: 100, }, "Should be able to parse all basic config values correctly" ); diff --git a/pageserver/src/import_datadir.rs b/pageserver/src/import_datadir.rs index d95d75449d..d66df36b3a 100644 --- a/pageserver/src/import_datadir.rs +++ b/pageserver/src/import_datadir.rs @@ -21,6 +21,7 @@ use tracing::*; use walkdir::WalkDir; use crate::context::RequestContext; +use crate::metrics::WAL_INGEST; use crate::pgdatadir_mapping::*; use crate::tenant::remote_timeline_client::INITDB_PATH; use crate::tenant::Timeline; @@ -312,13 +313,16 @@ async fn import_wal( waldecoder.feed_bytes(&buf); let mut nrecords = 0; - let mut modification = tline.begin_modification(endpoint); + let mut modification = tline.begin_modification(last_lsn); let mut decoded = DecodedWALRecord::default(); while last_lsn <= endpoint { if let Some((lsn, recdata)) = waldecoder.poll_decode()? { walingest .ingest_record(recdata, lsn, &mut modification, &mut decoded, ctx) .await?; + WAL_INGEST.records_committed.inc(); + + modification.commit(ctx).await?; last_lsn = lsn; nrecords += 1; @@ -448,13 +452,14 @@ pub async fn import_wal_from_tar( waldecoder.feed_bytes(&bytes[offset..]); - let mut modification = tline.begin_modification(end_lsn); + let mut modification = tline.begin_modification(last_lsn); let mut decoded = DecodedWALRecord::default(); while last_lsn <= end_lsn { if let Some((lsn, recdata)) = waldecoder.poll_decode()? { walingest .ingest_record(recdata, lsn, &mut modification, &mut decoded, ctx) .await?; + modification.commit(ctx).await?; last_lsn = lsn; debug!("imported record at {} (end {})", lsn, end_lsn); diff --git a/pageserver/src/page_service.rs b/pageserver/src/page_service.rs index d5ca7f7382..db07a600e5 100644 --- a/pageserver/src/page_service.rs +++ b/pageserver/src/page_service.rs @@ -53,7 +53,7 @@ use crate::context::{DownloadBehavior, RequestContext}; use crate::import_datadir::import_wal_from_tar; use crate::metrics; use crate::metrics::LIVE_CONNECTIONS_COUNT; -use crate::pgdatadir_mapping::rel_block_to_key; +use crate::pgdatadir_mapping::{rel_block_to_key, Version}; use crate::task_mgr; use crate::task_mgr::TaskKind; use crate::tenant::debug_assert_current_span_has_tenant_and_timeline_id; @@ -747,7 +747,7 @@ impl PageServerHandler { .await?; let exists = timeline - .get_rel_exists(req.rel, lsn, req.latest, ctx) + .get_rel_exists(req.rel, Version::Lsn(lsn), req.latest, ctx) .await?; Ok(PagestreamBeMessage::Exists(PagestreamExistsResponse { @@ -766,7 +766,9 @@ impl PageServerHandler { Self::wait_or_get_last_lsn(timeline, req.lsn, req.latest, &latest_gc_cutoff_lsn, ctx) .await?; - let n_blocks = timeline.get_rel_size(req.rel, lsn, req.latest, ctx).await?; + let n_blocks = timeline + .get_rel_size(req.rel, Version::Lsn(lsn), req.latest, ctx) + .await?; Ok(PagestreamBeMessage::Nblocks(PagestreamNblocksResponse { n_blocks, @@ -785,7 +787,13 @@ impl PageServerHandler { .await?; let total_blocks = timeline - .get_db_size(DEFAULTTABLESPACE_OID, req.dbnode, lsn, req.latest, ctx) + .get_db_size( + DEFAULTTABLESPACE_OID, + req.dbnode, + Version::Lsn(lsn), + req.latest, + ctx, + ) .await?; let db_size = total_blocks as i64 * BLCKSZ as i64; @@ -816,7 +824,7 @@ impl PageServerHandler { let key = rel_block_to_key(req.rel, req.blkno); let page = if timeline.get_shard_identity().is_key_local(&key) { timeline - .get_rel_page_at_lsn(req.rel, req.blkno, lsn, req.latest, ctx) + .get_rel_page_at_lsn(req.rel, req.blkno, Version::Lsn(lsn), req.latest, ctx) .await? } else { // The Tenant shard we looked up at connection start does not hold this particular @@ -853,7 +861,7 @@ impl PageServerHandler { // the GateGuard was already held over the whole connection. let _timeline_guard = timeline.gate.enter().map_err(|_| QueryError::Shutdown)?; timeline - .get_rel_page_at_lsn(req.rel, req.blkno, lsn, req.latest, ctx) + .get_rel_page_at_lsn(req.rel, req.blkno, Version::Lsn(lsn), req.latest, ctx) .await? }; diff --git a/pageserver/src/pgdatadir_mapping.rs b/pageserver/src/pgdatadir_mapping.rs index e9884a15f5..9fe75e5baf 100644 --- a/pageserver/src/pgdatadir_mapping.rs +++ b/pageserver/src/pgdatadir_mapping.rs @@ -11,7 +11,7 @@ use crate::context::RequestContext; use crate::keyspace::{KeySpace, KeySpaceAccum}; use crate::repository::*; use crate::walrecord::NeonWalRecord; -use anyhow::Context; +use anyhow::{ensure, Context}; use bytes::{Buf, Bytes}; use pageserver_api::key::is_rel_block_key; use pageserver_api::reltag::{RelTag, SlruKind}; @@ -147,6 +147,7 @@ impl Timeline { { DatadirModification { tline: self, + pending_lsns: Vec::new(), pending_updates: HashMap::new(), pending_deletions: Vec::new(), pending_nblocks: 0, @@ -163,7 +164,7 @@ impl Timeline { &self, tag: RelTag, blknum: BlockNumber, - lsn: Lsn, + version: Version<'_>, latest: bool, ctx: &RequestContext, ) -> Result { @@ -173,17 +174,20 @@ impl Timeline { )); } - let nblocks = self.get_rel_size(tag, lsn, latest, ctx).await?; + let nblocks = self.get_rel_size(tag, version, latest, ctx).await?; if blknum >= nblocks { debug!( "read beyond EOF at {} blk {} at {}, size is {}: returning all-zeros page", - tag, blknum, lsn, nblocks + tag, + blknum, + version.get_lsn(), + nblocks ); return Ok(ZERO_PAGE.clone()); } let key = rel_block_to_key(tag, blknum); - self.get(key, lsn, ctx).await + version.get(self, key, ctx).await } // Get size of a database in blocks @@ -191,16 +195,16 @@ impl Timeline { &self, spcnode: Oid, dbnode: Oid, - lsn: Lsn, + version: Version<'_>, latest: bool, ctx: &RequestContext, ) -> Result { let mut total_blocks = 0; - let rels = self.list_rels(spcnode, dbnode, lsn, ctx).await?; + let rels = self.list_rels(spcnode, dbnode, version, ctx).await?; for rel in rels { - let n_blocks = self.get_rel_size(rel, lsn, latest, ctx).await?; + let n_blocks = self.get_rel_size(rel, version, latest, ctx).await?; total_blocks += n_blocks as usize; } Ok(total_blocks) @@ -210,7 +214,7 @@ impl Timeline { pub async fn get_rel_size( &self, tag: RelTag, - lsn: Lsn, + version: Version<'_>, latest: bool, ctx: &RequestContext, ) -> Result { @@ -220,12 +224,12 @@ impl Timeline { )); } - if let Some(nblocks) = self.get_cached_rel_size(&tag, lsn) { + if let Some(nblocks) = self.get_cached_rel_size(&tag, version.get_lsn()) { return Ok(nblocks); } if (tag.forknum == FSM_FORKNUM || tag.forknum == VISIBILITYMAP_FORKNUM) - && !self.get_rel_exists(tag, lsn, latest, ctx).await? + && !self.get_rel_exists(tag, version, latest, ctx).await? { // FIXME: Postgres sometimes calls smgrcreate() to create // FSM, and smgrnblocks() on it immediately afterwards, @@ -235,7 +239,7 @@ impl Timeline { } let key = rel_size_to_key(tag); - let mut buf = self.get(key, lsn, ctx).await?; + let mut buf = version.get(self, key, ctx).await?; let nblocks = buf.get_u32_le(); if latest { @@ -246,7 +250,7 @@ impl Timeline { // latest=true, then it can not cause cache corruption, because with latest=true // pageserver choose max(request_lsn, last_written_lsn) and so cached value will be // associated with most recent value of LSN. - self.update_cached_rel_size(tag, lsn, nblocks); + self.update_cached_rel_size(tag, version.get_lsn(), nblocks); } Ok(nblocks) } @@ -255,7 +259,7 @@ impl Timeline { pub async fn get_rel_exists( &self, tag: RelTag, - lsn: Lsn, + version: Version<'_>, _latest: bool, ctx: &RequestContext, ) -> Result { @@ -266,12 +270,12 @@ impl Timeline { } // first try to lookup relation in cache - if let Some(_nblocks) = self.get_cached_rel_size(&tag, lsn) { + if let Some(_nblocks) = self.get_cached_rel_size(&tag, version.get_lsn()) { return Ok(true); } // fetch directory listing let key = rel_dir_to_key(tag.spcnode, tag.dbnode); - let buf = self.get(key, lsn, ctx).await?; + let buf = version.get(self, key, ctx).await?; match RelDirectory::des(&buf).context("deserialization failure") { Ok(dir) => { @@ -291,12 +295,12 @@ impl Timeline { &self, spcnode: Oid, dbnode: Oid, - lsn: Lsn, + version: Version<'_>, ctx: &RequestContext, ) -> Result, PageReconstructError> { // fetch directory listing let key = rel_dir_to_key(spcnode, dbnode); - let buf = self.get(key, lsn, ctx).await?; + let buf = version.get(self, key, ctx).await?; match RelDirectory::des(&buf).context("deserialization failure") { Ok(dir) => { @@ -332,11 +336,11 @@ impl Timeline { &self, kind: SlruKind, segno: u32, - lsn: Lsn, + version: Version<'_>, ctx: &RequestContext, ) -> Result { let key = slru_segment_size_to_key(kind, segno); - let mut buf = self.get(key, lsn, ctx).await?; + let mut buf = version.get(self, key, ctx).await?; Ok(buf.get_u32_le()) } @@ -345,12 +349,12 @@ impl Timeline { &self, kind: SlruKind, segno: u32, - lsn: Lsn, + version: Version<'_>, ctx: &RequestContext, ) -> Result { // fetch directory listing let key = slru_dir_to_key(kind); - let buf = self.get(key, lsn, ctx).await?; + let buf = version.get(self, key, ctx).await?; match SlruSegmentDirectory::des(&buf).context("deserialization failure") { Ok(dir) => { @@ -501,11 +505,11 @@ impl Timeline { mut f: impl FnMut(TimestampTz) -> ControlFlow, ) -> Result { for segno in self - .list_slru_segments(SlruKind::Clog, probe_lsn, ctx) + .list_slru_segments(SlruKind::Clog, Version::Lsn(probe_lsn), ctx) .await? { let nblocks = self - .get_slru_segment_size(SlruKind::Clog, segno, probe_lsn, ctx) + .get_slru_segment_size(SlruKind::Clog, segno, Version::Lsn(probe_lsn), ctx) .await?; for blknum in (0..nblocks).rev() { let clog_page = self @@ -531,13 +535,13 @@ impl Timeline { pub async fn list_slru_segments( &self, kind: SlruKind, - lsn: Lsn, + version: Version<'_>, ctx: &RequestContext, ) -> Result, PageReconstructError> { // fetch directory entry let key = slru_dir_to_key(kind); - let buf = self.get(key, lsn, ctx).await?; + let buf = version.get(self, key, ctx).await?; match SlruSegmentDirectory::des(&buf).context("deserialization failure") { Ok(dir) => Ok(dir.segments), Err(e) => Err(PageReconstructError::from(e)), @@ -548,12 +552,12 @@ impl Timeline { &self, spcnode: Oid, dbnode: Oid, - lsn: Lsn, + version: Version<'_>, ctx: &RequestContext, ) -> Result { let key = relmap_file_key(spcnode, dbnode); - let buf = self.get(key, lsn, ctx).await?; + let buf = version.get(self, key, ctx).await?; Ok(buf) } @@ -652,7 +656,10 @@ impl Timeline { let mut total_size: u64 = 0; for (spcnode, dbnode) in dbdir.dbdirs.keys() { - for rel in self.list_rels(*spcnode, *dbnode, lsn, ctx).await? { + for rel in self + .list_rels(*spcnode, *dbnode, Version::Lsn(lsn), ctx) + .await? + { if self.cancel.is_cancelled() { return Err(CalculateLogicalSizeError::Cancelled); } @@ -692,7 +699,7 @@ impl Timeline { result.add_key(rel_dir_to_key(spcnode, dbnode)); let mut rels: Vec = self - .list_rels(spcnode, dbnode, lsn, ctx) + .list_rels(spcnode, dbnode, Version::Lsn(lsn), ctx) .await? .into_iter() .collect(); @@ -799,18 +806,39 @@ pub struct DatadirModification<'a> { /// in the state in 'tline' yet. pub tline: &'a Timeline, - /// Lsn assigned by begin_modification - pub lsn: Lsn, + /// Current LSN of the modification + lsn: Lsn, // The modifications are not applied directly to the underlying key-value store. // The put-functions add the modifications here, and they are flushed to the // underlying key-value store by the 'finish' function. - pending_updates: HashMap, - pending_deletions: Vec>, + pending_lsns: Vec, + pending_updates: HashMap>, + pending_deletions: Vec<(Range, Lsn)>, pending_nblocks: i64, } impl<'a> DatadirModification<'a> { + /// Get the current lsn + pub(crate) fn get_lsn(&self) -> Lsn { + self.lsn + } + + /// Set the current lsn + pub(crate) fn set_lsn(&mut self, lsn: Lsn) -> anyhow::Result<()> { + ensure!( + lsn >= self.lsn, + "setting an older lsn {} than {} is not allowed", + lsn, + self.lsn + ); + if lsn > self.lsn { + self.pending_lsns.push(self.lsn); + self.lsn = lsn; + } + Ok(()) + } + /// Initialize a completely new repository. /// /// This inserts the directory metadata entries that are assumed to @@ -984,11 +1012,9 @@ impl<'a> DatadirModification<'a> { dbnode: Oid, ctx: &RequestContext, ) -> anyhow::Result<()> { - let req_lsn = self.tline.get_last_record_lsn(); - let total_blocks = self .tline - .get_db_size(spcnode, dbnode, req_lsn, true, ctx) + .get_db_size(spcnode, dbnode, Version::Modified(self), true, ctx) .await?; // Remove entry from dbdir @@ -1077,8 +1103,11 @@ impl<'a> DatadirModification<'a> { ctx: &RequestContext, ) -> anyhow::Result<()> { anyhow::ensure!(rel.relnode != 0, RelationError::InvalidRelnode); - let last_lsn = self.tline.get_last_record_lsn(); - if self.tline.get_rel_exists(rel, last_lsn, true, ctx).await? { + if self + .tline + .get_rel_exists(rel, Version::Modified(self), true, ctx) + .await? + { let size_key = rel_size_to_key(rel); // Fetch the old size first let old_size = self.get(size_key, ctx).await?.get_u32_le(); @@ -1323,17 +1352,23 @@ impl<'a> DatadirModification<'a> { let writer = self.tline.writer().await; // Flush relation and SLRU data blocks, keep metadata. - let mut retained_pending_updates = HashMap::new(); - for (key, value) in self.pending_updates.drain() { - if is_rel_block_key(&key) || is_slru_block_key(key) { - // This bails out on first error without modifying pending_updates. - // That's Ok, cf this function's doc comment. - writer.put(key, self.lsn, &value, ctx).await?; - } else { - retained_pending_updates.insert(key, value); + let mut retained_pending_updates = HashMap::<_, Vec<_>>::new(); + for (key, values) in self.pending_updates.drain() { + for (lsn, value) in values { + if is_rel_block_key(&key) || is_slru_block_key(key) { + // This bails out on first error without modifying pending_updates. + // That's Ok, cf this function's doc comment. + writer.put(key, lsn, &value, ctx).await?; + } else { + retained_pending_updates + .entry(key) + .or_default() + .push((lsn, value)); + } } } - self.pending_updates.extend(retained_pending_updates); + + self.pending_updates = retained_pending_updates; if pending_nblocks != 0 { writer.update_current_logical_size(pending_nblocks * i64::from(BLCKSZ)); @@ -1350,18 +1385,28 @@ impl<'a> DatadirModification<'a> { /// pub async fn commit(&mut self, ctx: &RequestContext) -> anyhow::Result<()> { let writer = self.tline.writer().await; - let lsn = self.lsn; + let pending_nblocks = self.pending_nblocks; self.pending_nblocks = 0; - for (key, value) in self.pending_updates.drain() { - writer.put(key, lsn, &value, ctx).await?; - } - for key_range in self.pending_deletions.drain(..) { - writer.delete(key_range, lsn).await?; + if !self.pending_updates.is_empty() { + writer.put_batch(&self.pending_updates, ctx).await?; + self.pending_updates.clear(); } - writer.finish_write(lsn); + if !self.pending_deletions.is_empty() { + writer.delete_batch(&self.pending_deletions).await?; + self.pending_deletions.clear(); + } + + self.pending_lsns.push(self.lsn); + for pending_lsn in self.pending_lsns.drain(..) { + // Ideally, we should be able to call writer.finish_write() only once + // with the highest LSN. However, the last_record_lsn variable in the + // timeline keeps track of the latest LSN and the immediate previous LSN + // so we need to record every LSN to not leave a gap between them. + writer.finish_write(pending_lsn); + } if pending_nblocks != 0 { writer.update_current_logical_size(pending_nblocks * i64::from(BLCKSZ)); @@ -1370,44 +1415,86 @@ impl<'a> DatadirModification<'a> { Ok(()) } - pub(crate) fn is_empty(&self) -> bool { - self.pending_updates.is_empty() && self.pending_deletions.is_empty() + pub(crate) fn len(&self) -> usize { + self.pending_updates.len() + self.pending_deletions.len() } // Internal helper functions to batch the modifications async fn get(&self, key: Key, ctx: &RequestContext) -> Result { - // Have we already updated the same key? Read the pending updated + // Have we already updated the same key? Read the latest pending updated // version in that case. // // Note: we don't check pending_deletions. It is an error to request a // value that has been removed, deletion only avoids leaking storage. - if let Some(value) = self.pending_updates.get(&key) { - if let Value::Image(img) = value { - Ok(img.clone()) - } else { - // Currently, we never need to read back a WAL record that we - // inserted in the same "transaction". All the metadata updates - // work directly with Images, and we never need to read actual - // data pages. We could handle this if we had to, by calling - // the walredo manager, but let's keep it simple for now. - Err(PageReconstructError::from(anyhow::anyhow!( - "unexpected pending WAL record" - ))) + if let Some(values) = self.pending_updates.get(&key) { + if let Some((_, value)) = values.last() { + return if let Value::Image(img) = value { + Ok(img.clone()) + } else { + // Currently, we never need to read back a WAL record that we + // inserted in the same "transaction". All the metadata updates + // work directly with Images, and we never need to read actual + // data pages. We could handle this if we had to, by calling + // the walredo manager, but let's keep it simple for now. + Err(PageReconstructError::from(anyhow::anyhow!( + "unexpected pending WAL record" + ))) + }; } - } else { - let lsn = Lsn::max(self.tline.get_last_record_lsn(), self.lsn); - self.tline.get(key, lsn, ctx).await } + let lsn = Lsn::max(self.tline.get_last_record_lsn(), self.lsn); + self.tline.get(key, lsn, ctx).await } fn put(&mut self, key: Key, val: Value) { - self.pending_updates.insert(key, val); + let values = self.pending_updates.entry(key).or_default(); + // Replace the previous value if it exists at the same lsn + if let Some((last_lsn, last_value)) = values.last_mut() { + if *last_lsn == self.lsn { + *last_value = val; + return; + } + } + values.push((self.lsn, val)); } fn delete(&mut self, key_range: Range) { trace!("DELETE {}-{}", key_range.start, key_range.end); - self.pending_deletions.push(key_range); + self.pending_deletions.push((key_range, self.lsn)); + } +} + +/// This struct facilitates accessing either a committed key from the timeline at a +/// specific LSN, or the latest uncommitted key from a pending modification. +/// During WAL ingestion, the records from multiple LSNs may be batched in the same +/// modification before being flushed to the timeline. Hence, the routines in WalIngest +/// need to look up the keys in the modification first before looking them up in the +/// timeline to not miss the latest updates. +#[derive(Clone, Copy)] +pub enum Version<'a> { + Lsn(Lsn), + Modified(&'a DatadirModification<'a>), +} + +impl<'a> Version<'a> { + async fn get( + &self, + timeline: &Timeline, + key: Key, + ctx: &RequestContext, + ) -> Result { + match self { + Version::Lsn(lsn) => timeline.get(key, *lsn, ctx).await, + Version::Modified(modification) => modification.get(key, ctx).await, + } + } + + fn get_lsn(&self) -> Lsn { + match self { + Version::Lsn(lsn) => *lsn, + Version::Modified(modification) => modification.lsn, + } } } diff --git a/pageserver/src/tenant/config.rs b/pageserver/src/tenant/config.rs index 25d97f51ce..2d4cd350d7 100644 --- a/pageserver/src/tenant/config.rs +++ b/pageserver/src/tenant/config.rs @@ -46,6 +46,8 @@ pub mod defaults { pub const DEFAULT_WALRECEIVER_LAGGING_WAL_TIMEOUT: &str = "10 seconds"; pub const DEFAULT_MAX_WALRECEIVER_LSN_WAL_LAG: u64 = 10 * 1024 * 1024; pub const DEFAULT_EVICTIONS_LOW_RESIDENCE_DURATION_METRIC_THRESHOLD: &str = "24 hour"; + + pub const DEFAULT_INGEST_BATCH_SIZE: u64 = 100; } #[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] diff --git a/pageserver/src/tenant/storage_layer/inmemory_layer.rs b/pageserver/src/tenant/storage_layer/inmemory_layer.rs index 003cf0e92b..7c9103eea8 100644 --- a/pageserver/src/tenant/storage_layer/inmemory_layer.rs +++ b/pageserver/src/tenant/storage_layer/inmemory_layer.rs @@ -23,7 +23,7 @@ use utils::{bin_ser::BeSer, id::TimelineId, lsn::Lsn, vec_map::VecMap}; // while being able to use std::fmt::Write's methods use std::fmt::Write as _; use std::ops::Range; -use tokio::sync::RwLock; +use tokio::sync::{RwLock, RwLockWriteGuard}; use super::{DeltaLayerWriter, ResidentLayer}; @@ -246,16 +246,43 @@ impl InMemoryLayer { /// Common subroutine of the public put_wal_record() and put_page_image() functions. /// Adds the page version to the in-memory tree - pub async fn put_value( + pub(crate) async fn put_value( &self, key: Key, lsn: Lsn, val: &Value, ctx: &RequestContext, ) -> Result<()> { - trace!("put_value key {} at {}/{}", key, self.timeline_id, lsn); - let inner: &mut _ = &mut *self.inner.write().await; + let mut inner = self.inner.write().await; self.assert_writable(); + self.put_value_locked(&mut inner, key, lsn, val, ctx).await + } + + pub(crate) async fn put_values( + &self, + values: &HashMap>, + ctx: &RequestContext, + ) -> Result<()> { + let mut inner = self.inner.write().await; + self.assert_writable(); + for (key, vals) in values { + for (lsn, val) in vals { + self.put_value_locked(&mut inner, *key, *lsn, val, ctx) + .await?; + } + } + Ok(()) + } + + async fn put_value_locked( + &self, + locked_inner: &mut RwLockWriteGuard<'_, InMemoryLayerInner>, + key: Key, + lsn: Lsn, + val: &Value, + ctx: &RequestContext, + ) -> Result<()> { + trace!("put_value key {} at {}/{}", key, self.timeline_id, lsn); let off = { // Avoid doing allocations for "small" values. @@ -264,7 +291,7 @@ impl InMemoryLayer { let mut buf = smallvec::SmallVec::<[u8; 256]>::new(); buf.clear(); val.ser_into(&mut buf)?; - inner + locked_inner .file .write_blob( &buf, @@ -275,7 +302,7 @@ impl InMemoryLayer { .await? }; - let vec_map = inner.index.entry(key).or_default(); + let vec_map = locked_inner.index.entry(key).or_default(); let old = vec_map.append_or_update_last(lsn, off).unwrap().0; if old.is_some() { // We already had an entry for this LSN. That's odd.. @@ -285,13 +312,11 @@ impl InMemoryLayer { Ok(()) } - pub async fn put_tombstone(&self, _key_range: Range, _lsn: Lsn) -> Result<()> { + pub(crate) async fn put_tombstones(&self, _key_ranges: &[(Range, Lsn)]) -> Result<()> { // TODO: Currently, we just leak the storage for any deleted keys - Ok(()) } - /// Make the layer non-writeable. Only call once. /// Records the end_lsn for non-dropped layers. /// `end_lsn` is exclusive pub async fn freeze(&self, end_lsn: Lsn) { diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 1e84fa1848..15a5ca1727 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -1459,6 +1459,7 @@ impl Timeline { max_lsn_wal_lag, auth_token: crate::config::SAFEKEEPER_AUTH_TOKEN.get().cloned(), availability_zone: self.conf.availability_zone.clone(), + ingest_batch_size: self.conf.ingest_batch_size, }, broker_client, ctx, @@ -2471,9 +2472,27 @@ impl Timeline { Ok(()) } - async fn put_tombstone(&self, key_range: Range, lsn: Lsn) -> anyhow::Result<()> { - let layer = self.get_layer_for_write(lsn).await?; - layer.put_tombstone(key_range, lsn).await?; + async fn put_values( + &self, + values: &HashMap>, + ctx: &RequestContext, + ) -> anyhow::Result<()> { + // Pick the first LSN in the batch to get the layer to write to. + for lsns in values.values() { + if let Some((lsn, _)) = lsns.first() { + let layer = self.get_layer_for_write(*lsn).await?; + layer.put_values(values, ctx).await?; + break; + } + } + Ok(()) + } + + async fn put_tombstones(&self, tombstones: &[(Range, Lsn)]) -> anyhow::Result<()> { + if let Some((_, lsn)) = tombstones.first() { + let layer = self.get_layer_for_write(*lsn).await?; + layer.put_tombstones(tombstones).await?; + } Ok(()) } @@ -4529,8 +4548,16 @@ impl<'a> TimelineWriter<'a> { self.tl.put_value(key, lsn, value, ctx).await } - pub async fn delete(&self, key_range: Range, lsn: Lsn) -> anyhow::Result<()> { - self.tl.put_tombstone(key_range, lsn).await + pub(crate) async fn put_batch( + &self, + batch: &HashMap>, + ctx: &RequestContext, + ) -> anyhow::Result<()> { + self.tl.put_values(batch, ctx).await + } + + pub(crate) async fn delete_batch(&self, batch: &[(Range, Lsn)]) -> anyhow::Result<()> { + self.tl.put_tombstones(batch).await } /// Track the end of the latest digested WAL record. @@ -4541,11 +4568,11 @@ impl<'a> TimelineWriter<'a> { /// 'lsn' must be aligned. This wakes up any wait_lsn() callers waiting for /// the 'lsn' or anything older. The previous last record LSN is stored alongside /// the latest and can be read. - pub fn finish_write(&self, new_lsn: Lsn) { + pub(crate) fn finish_write(&self, new_lsn: Lsn) { self.tl.finish_write(new_lsn); } - pub fn update_current_logical_size(&self, delta: i64) { + pub(crate) fn update_current_logical_size(&self, delta: i64) { self.tl.update_current_logical_size(delta) } } diff --git a/pageserver/src/tenant/timeline/walreceiver.rs b/pageserver/src/tenant/timeline/walreceiver.rs index e32265afb5..2fab6722b8 100644 --- a/pageserver/src/tenant/timeline/walreceiver.rs +++ b/pageserver/src/tenant/timeline/walreceiver.rs @@ -58,6 +58,7 @@ pub struct WalReceiverConf { pub max_lsn_wal_lag: NonZeroU64, pub auth_token: Option>, pub availability_zone: Option, + pub ingest_batch_size: u64, } pub struct WalReceiver { diff --git a/pageserver/src/tenant/timeline/walreceiver/connection_manager.rs b/pageserver/src/tenant/timeline/walreceiver/connection_manager.rs index 5a5b3d7586..7fa5bb7689 100644 --- a/pageserver/src/tenant/timeline/walreceiver/connection_manager.rs +++ b/pageserver/src/tenant/timeline/walreceiver/connection_manager.rs @@ -411,6 +411,7 @@ impl ConnectionManagerState { let node_id = new_sk.safekeeper_id; let connect_timeout = self.conf.wal_connect_timeout; + let ingest_batch_size = self.conf.ingest_batch_size; let timeline = Arc::clone(&self.timeline); let ctx = ctx.detached_child( TaskKind::WalReceiverConnectionHandler, @@ -430,6 +431,7 @@ impl ConnectionManagerState { connect_timeout, ctx, node_id, + ingest_batch_size, ) .await; @@ -1345,6 +1347,7 @@ mod tests { max_lsn_wal_lag: NonZeroU64::new(1024 * 1024).unwrap(), auth_token: None, availability_zone: None, + ingest_batch_size: 1, }, wal_connection: None, wal_stream_candidates: HashMap::new(), diff --git a/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs b/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs index 61ab236322..e398d683e5 100644 --- a/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs +++ b/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs @@ -26,7 +26,7 @@ use tracing::{debug, error, info, trace, warn, Instrument}; use super::TaskStateUpdate; use crate::{ context::RequestContext, - metrics::{LIVE_CONNECTIONS_COUNT, WALRECEIVER_STARTED_CONNECTIONS}, + metrics::{LIVE_CONNECTIONS_COUNT, WALRECEIVER_STARTED_CONNECTIONS, WAL_INGEST}, task_mgr, task_mgr::TaskKind, task_mgr::WALRECEIVER_RUNTIME, @@ -106,6 +106,7 @@ impl From for WalReceiverError { /// Open a connection to the given safekeeper and receive WAL, sending back progress /// messages as we go. +#[allow(clippy::too_many_arguments)] pub(super) async fn handle_walreceiver_connection( timeline: Arc, wal_source_connconf: PgConnectionConfig, @@ -114,6 +115,7 @@ pub(super) async fn handle_walreceiver_connection( connect_timeout: Duration, ctx: RequestContext, node: NodeId, + ingest_batch_size: u64, ) -> Result<(), WalReceiverError> { debug_assert_current_span_has_tenant_and_timeline_id(); @@ -305,7 +307,9 @@ pub(super) async fn handle_walreceiver_connection( { let mut decoded = DecodedWALRecord::default(); - let mut modification = timeline.begin_modification(endlsn); + let mut modification = timeline.begin_modification(startlsn); + let mut uncommitted_records = 0; + let mut filtered_records = 0; while let Some((lsn, recdata)) = waldecoder.poll_decode()? { // It is important to deal with the aligned records as lsn in getPage@LSN is // aligned and can be several bytes bigger. Without this alignment we are @@ -314,14 +318,40 @@ pub(super) async fn handle_walreceiver_connection( return Err(WalReceiverError::Other(anyhow!("LSN not aligned"))); } - walingest + // Ingest the records without immediately committing them. + let ingested = walingest .ingest_record(recdata, lsn, &mut modification, &mut decoded, &ctx) .await .with_context(|| format!("could not ingest record at {lsn}"))?; + if !ingested { + tracing::debug!("ingest: filtered out record @ LSN {lsn}"); + WAL_INGEST.records_filtered.inc(); + filtered_records += 1; + } fail_point!("walreceiver-after-ingest"); last_rec_lsn = lsn; + + // Commit every ingest_batch_size records. Even if we filtered out + // all records, we still need to call commit to advance the LSN. + uncommitted_records += 1; + if uncommitted_records >= ingest_batch_size { + WAL_INGEST + .records_committed + .inc_by(uncommitted_records - filtered_records); + modification.commit(&ctx).await?; + uncommitted_records = 0; + filtered_records = 0; + } + } + + // Commit the remaining records. + if uncommitted_records > 0 { + WAL_INGEST + .records_committed + .inc_by(uncommitted_records - filtered_records); + modification.commit(&ctx).await?; } } diff --git a/pageserver/src/walingest.rs b/pageserver/src/walingest.rs index a6a8972970..8df0c81c7a 100644 --- a/pageserver/src/walingest.rs +++ b/pageserver/src/walingest.rs @@ -48,20 +48,18 @@ use postgres_ffi::TransactionId; use postgres_ffi::BLCKSZ; use utils::lsn::Lsn; -pub struct WalIngest<'a> { +pub struct WalIngest { shard: ShardIdentity, - timeline: &'a Timeline, - checkpoint: CheckPoint, checkpoint_modified: bool, } -impl<'a> WalIngest<'a> { +impl WalIngest { pub async fn new( - timeline: &'a Timeline, + timeline: &Timeline, startpoint: Lsn, - ctx: &'_ RequestContext, - ) -> anyhow::Result> { + ctx: &RequestContext, + ) -> anyhow::Result { // Fetch the latest checkpoint into memory, so that we can compare with it // quickly in `ingest_record` and update it when it changes. let checkpoint_bytes = timeline.get_checkpoint(startpoint, ctx).await?; @@ -70,7 +68,6 @@ impl<'a> WalIngest<'a> { Ok(WalIngest { shard: *timeline.get_shard_identity(), - timeline, checkpoint, checkpoint_modified: false, }) @@ -84,6 +81,8 @@ impl<'a> WalIngest<'a> { /// Helper function to parse a WAL record and call the Timeline's PUT functions for all the /// relations/pages that the record affects. /// + /// This function returns `true` if the record was ingested, and `false` if it was filtered out + /// pub async fn ingest_record( &mut self, recdata: Bytes, @@ -91,11 +90,13 @@ impl<'a> WalIngest<'a> { modification: &mut DatadirModification<'_>, decoded: &mut DecodedWALRecord, ctx: &RequestContext, - ) -> anyhow::Result<()> { + ) -> anyhow::Result { WAL_INGEST.records_received.inc(); + let pg_version = modification.tline.pg_version; + let prev_len = modification.len(); - modification.lsn = lsn; - decode_wal_record(recdata, decoded, self.timeline.pg_version)?; + modification.set_lsn(lsn)?; + decode_wal_record(recdata, decoded, pg_version)?; let mut buf = decoded.record.clone(); buf.advance(decoded.main_data_offset); @@ -132,9 +133,9 @@ impl<'a> WalIngest<'a> { } pg_constants::RM_DBASE_ID => { let info = decoded.xl_info & pg_constants::XLR_RMGR_INFO_MASK; - debug!(%info, pg_version=%self.timeline.pg_version, "handle RM_DBASE_ID"); + debug!(%info, %pg_version, "handle RM_DBASE_ID"); - if self.timeline.pg_version == 14 { + if pg_version == 14 { if info == postgres_ffi::v14::bindings::XLOG_DBASE_CREATE { let createdb = XlCreateDatabase::decode(&mut buf); debug!("XLOG_DBASE_CREATE v14"); @@ -150,7 +151,7 @@ impl<'a> WalIngest<'a> { .await?; } } - } else if self.timeline.pg_version == 15 { + } else if pg_version == 15 { if info == postgres_ffi::v15::bindings::XLOG_DBASE_CREATE_WAL_LOG { debug!("XLOG_DBASE_CREATE_WAL_LOG: noop"); } else if info == postgres_ffi::v15::bindings::XLOG_DBASE_CREATE_FILE_COPY { @@ -170,7 +171,7 @@ impl<'a> WalIngest<'a> { .await?; } } - } else if self.timeline.pg_version == 16 { + } else if pg_version == 16 { if info == postgres_ffi::v16::bindings::XLOG_DBASE_CREATE_WAL_LOG { debug!("XLOG_DBASE_CREATE_WAL_LOG: noop"); } else if info == postgres_ffi::v16::bindings::XLOG_DBASE_CREATE_FILE_COPY { @@ -399,19 +400,11 @@ impl<'a> WalIngest<'a> { self.checkpoint_modified = false; } - if modification.is_empty() { - tracing::debug!("ingest: filtered out record @ LSN {lsn}"); - WAL_INGEST.records_filtered.inc(); - modification.tline.finish_write(lsn); - } else { - WAL_INGEST.records_committed.inc(); - modification.commit(ctx).await?; - } + // Note that at this point this record is only cached in the modification + // until commit() is called to flush the data into the repository and update + // the latest LSN. - // Now that this record has been fully handled, including updating the - // checkpoint data, let the repository know that it is up-to-date to this LSN. - - Ok(()) + Ok(modification.len() > prev_len) } /// Do not store this block, but observe it for the purposes of updating our relation size state. @@ -458,7 +451,7 @@ impl<'a> WalIngest<'a> { && (decoded.xl_info == pg_constants::XLOG_FPI || decoded.xl_info == pg_constants::XLOG_FPI_FOR_HINT) // compression of WAL is not yet supported: fall back to storing the original WAL record - && !postgres_ffi::bkpimage_is_compressed(blk.bimg_info, self.timeline.pg_version)? + && !postgres_ffi::bkpimage_is_compressed(blk.bimg_info, modification.tline.pg_version)? // do not materialize null pages because them most likely be soon replaced with real data && blk.bimg_len != 0 { @@ -511,7 +504,7 @@ impl<'a> WalIngest<'a> { let mut old_heap_blkno: Option = None; let mut flags = pg_constants::VISIBILITYMAP_VALID_BITS; - match self.timeline.pg_version { + match modification.tline.pg_version { 14 => { if decoded.xl_rmid == pg_constants::RM_HEAP_ID { let info = decoded.xl_info & pg_constants::XLOG_HEAP_OPMASK; @@ -735,7 +728,7 @@ impl<'a> WalIngest<'a> { // replaying it would fail to find the previous image of the page, because // it doesn't exist. So check if the VM page(s) exist, and skip the WAL // record if it doesn't. - let vm_size = self.get_relsize(vm_rel, modification.lsn, ctx).await?; + let vm_size = get_relsize(modification, vm_rel, ctx).await?; if let Some(blknum) = new_vm_blk { if blknum >= vm_size { new_vm_blk = None; @@ -816,10 +809,11 @@ impl<'a> WalIngest<'a> { let mut new_heap_blkno: Option = None; let mut old_heap_blkno: Option = None; let mut flags = pg_constants::VISIBILITYMAP_VALID_BITS; + let pg_version = modification.tline.pg_version; assert_eq!(decoded.xl_rmid, pg_constants::RM_NEON_ID); - match self.timeline.pg_version { + match pg_version { 16 => { let info = decoded.xl_info & pg_constants::XLOG_HEAP_OPMASK; @@ -882,7 +876,7 @@ impl<'a> WalIngest<'a> { } _ => bail!( "Neon RMGR has no known compatibility with PostgreSQL version {}", - self.timeline.pg_version + pg_version ), } @@ -905,7 +899,7 @@ impl<'a> WalIngest<'a> { // replaying it would fail to find the previous image of the page, because // it doesn't exist. So check if the VM page(s) exist, and skip the WAL // record if it doesn't. - let vm_size = self.get_relsize(vm_rel, modification.lsn, ctx).await?; + let vm_size = get_relsize(modification, vm_rel, ctx).await?; if let Some(blknum) = new_vm_blk { if blknum >= vm_size { new_vm_blk = None; @@ -983,16 +977,14 @@ impl<'a> WalIngest<'a> { let src_db_id = rec.src_db_id; let src_tablespace_id = rec.src_tablespace_id; - // Creating a database is implemented by copying the template (aka. source) database. - // To copy all the relations, we need to ask for the state as of the same LSN, but we - // cannot pass 'lsn' to the Timeline.get_* functions, or they will block waiting for - // the last valid LSN to advance up to it. So we use the previous record's LSN in the - // get calls instead. - let req_lsn = modification.tline.get_last_record_lsn(); - let rels = modification .tline - .list_rels(src_tablespace_id, src_db_id, req_lsn, ctx) + .list_rels( + src_tablespace_id, + src_db_id, + Version::Modified(modification), + ctx, + ) .await?; debug!("ingest_xlog_dbase_create: {} rels", rels.len()); @@ -1000,7 +992,12 @@ impl<'a> WalIngest<'a> { // Copy relfilemap let filemap = modification .tline - .get_relmap_file(src_tablespace_id, src_db_id, req_lsn, ctx) + .get_relmap_file( + src_tablespace_id, + src_db_id, + Version::Modified(modification), + ctx, + ) .await?; modification .put_relmap_file(tablespace_id, db_id, filemap, ctx) @@ -1014,7 +1011,7 @@ impl<'a> WalIngest<'a> { let nblocks = modification .tline - .get_rel_size(src_rel, req_lsn, true, ctx) + .get_rel_size(src_rel, Version::Modified(modification), true, ctx) .await?; let dst_rel = RelTag { spcnode: tablespace_id, @@ -1032,7 +1029,13 @@ impl<'a> WalIngest<'a> { let content = modification .tline - .get_rel_page_at_lsn(src_rel, blknum, req_lsn, true, ctx) + .get_rel_page_at_lsn( + src_rel, + blknum, + Version::Modified(modification), + true, + ctx, + ) .await?; modification.put_rel_page_image(dst_rel, blknum, content)?; num_blocks_copied += 1; @@ -1103,7 +1106,7 @@ impl<'a> WalIngest<'a> { modification.put_rel_page_image(rel, fsm_physical_page_no, ZERO_PAGE.clone())?; fsm_physical_page_no += 1; } - let nblocks = self.get_relsize(rel, modification.lsn, ctx).await?; + let nblocks = get_relsize(modification, rel, ctx).await?; if nblocks > fsm_physical_page_no { // check if something to do: FSM is larger than truncate position self.put_rel_truncation(modification, rel, fsm_physical_page_no, ctx) @@ -1125,7 +1128,7 @@ impl<'a> WalIngest<'a> { modification.put_rel_page_image(rel, vm_page_no, ZERO_PAGE.clone())?; vm_page_no += 1; } - let nblocks = self.get_relsize(rel, modification.lsn, ctx).await?; + let nblocks = get_relsize(modification, rel, ctx).await?; if nblocks > vm_page_no { // check if something to do: VM is larger than truncate position self.put_rel_truncation(modification, rel, vm_page_no, ctx) @@ -1198,10 +1201,9 @@ impl<'a> WalIngest<'a> { dbnode: xnode.dbnode, relnode: xnode.relnode, }; - let last_lsn = self.timeline.get_last_record_lsn(); if modification .tline - .get_rel_exists(rel, last_lsn, true, ctx) + .get_rel_exists(rel, Version::Modified(modification), true, ctx) .await? { self.put_rel_drop(modification, rel, ctx).await?; @@ -1255,10 +1257,9 @@ impl<'a> WalIngest<'a> { // will block waiting for the last valid LSN to advance up to // it. So we use the previous record's LSN in the get calls // instead. - let req_lsn = modification.tline.get_last_record_lsn(); for segno in modification .tline - .list_slru_segments(SlruKind::Clog, req_lsn, ctx) + .list_slru_segments(SlruKind::Clog, Version::Modified(modification), ctx) .await? { let segpage = segno * pg_constants::SLRU_PAGES_PER_SEGMENT; @@ -1470,20 +1471,6 @@ impl<'a> WalIngest<'a> { Ok(()) } - async fn get_relsize( - &mut self, - rel: RelTag, - lsn: Lsn, - ctx: &RequestContext, - ) -> anyhow::Result { - let nblocks = if !self.timeline.get_rel_exists(rel, lsn, true, ctx).await? { - 0 - } else { - self.timeline.get_rel_size(rel, lsn, true, ctx).await? - }; - Ok(nblocks) - } - async fn handle_rel_extend( &mut self, modification: &mut DatadirModification<'_>, @@ -1495,7 +1482,6 @@ impl<'a> WalIngest<'a> { // Check if the relation exists. We implicitly create relations on first // record. // TODO: would be nice if to be more explicit about it - let last_lsn = modification.lsn; // Get current size and put rel creation if rel doesn't exist // @@ -1503,11 +1489,14 @@ impl<'a> WalIngest<'a> { // check the cache too. This is because eagerly checking the cache results in // less work overall and 10% better performance. It's more work on cache miss // but cache miss is rare. - let old_nblocks = if let Some(nblocks) = self.timeline.get_cached_rel_size(&rel, last_lsn) { + let old_nblocks = if let Some(nblocks) = modification + .tline + .get_cached_rel_size(&rel, modification.get_lsn()) + { nblocks - } else if !self - .timeline - .get_rel_exists(rel, last_lsn, true, ctx) + } else if !modification + .tline + .get_rel_exists(rel, Version::Modified(modification), true, ctx) .await? { // create it with 0 size initially, the logic below will extend it @@ -1517,7 +1506,10 @@ impl<'a> WalIngest<'a> { .context("Relation Error")?; 0 } else { - self.timeline.get_rel_size(rel, last_lsn, true, ctx).await? + modification + .tline + .get_rel_size(rel, Version::Modified(modification), true, ctx) + .await? }; if new_nblocks > old_nblocks { @@ -1570,10 +1562,9 @@ impl<'a> WalIngest<'a> { // Check if the relation exists. We implicitly create relations on first // record. // TODO: would be nice if to be more explicit about it - let last_lsn = self.timeline.get_last_record_lsn(); - let old_nblocks = if !self - .timeline - .get_slru_segment_exists(kind, segno, last_lsn, ctx) + let old_nblocks = if !modification + .tline + .get_slru_segment_exists(kind, segno, Version::Modified(modification), ctx) .await? { // create it with 0 size initially, the logic below will extend it @@ -1582,8 +1573,9 @@ impl<'a> WalIngest<'a> { .await?; 0 } else { - self.timeline - .get_slru_segment_size(kind, segno, last_lsn, ctx) + modification + .tline + .get_slru_segment_size(kind, segno, Version::Modified(modification), ctx) .await? }; @@ -1606,6 +1598,26 @@ impl<'a> WalIngest<'a> { } } +async fn get_relsize( + modification: &DatadirModification<'_>, + rel: RelTag, + ctx: &RequestContext, +) -> anyhow::Result { + let nblocks = if !modification + .tline + .get_rel_exists(rel, Version::Modified(modification), true, ctx) + .await? + { + 0 + } else { + modification + .tline + .get_rel_size(rel, Version::Modified(modification), true, ctx) + .await? + }; + Ok(nblocks) +} + #[allow(clippy::bool_assert_comparison)] #[cfg(test)] mod tests { @@ -1632,10 +1644,7 @@ mod tests { static ZERO_CHECKPOINT: Bytes = Bytes::from_static(&[0u8; SIZEOF_CHECKPOINT]); - async fn init_walingest_test<'a>( - tline: &'a Timeline, - ctx: &RequestContext, - ) -> Result> { + async fn init_walingest_test(tline: &Timeline, ctx: &RequestContext) -> Result { let mut m = tline.begin_modification(Lsn(0x10)); m.put_checkpoint(ZERO_CHECKPOINT.clone())?; m.put_relmap_file(0, 111, Bytes::from(""), ctx).await?; // dummy relmapper file @@ -1680,29 +1689,29 @@ mod tests { // The relation was created at LSN 2, not visible at LSN 1 yet. assert_eq!( tline - .get_rel_exists(TESTREL_A, Lsn(0x10), false, &ctx) + .get_rel_exists(TESTREL_A, Version::Lsn(Lsn(0x10)), false, &ctx) .await?, false ); assert!(tline - .get_rel_size(TESTREL_A, Lsn(0x10), false, &ctx) + .get_rel_size(TESTREL_A, Version::Lsn(Lsn(0x10)), false, &ctx) .await .is_err()); assert_eq!( tline - .get_rel_exists(TESTREL_A, Lsn(0x20), false, &ctx) + .get_rel_exists(TESTREL_A, Version::Lsn(Lsn(0x20)), false, &ctx) .await?, true ); assert_eq!( tline - .get_rel_size(TESTREL_A, Lsn(0x20), false, &ctx) + .get_rel_size(TESTREL_A, Version::Lsn(Lsn(0x20)), false, &ctx) .await?, 1 ); assert_eq!( tline - .get_rel_size(TESTREL_A, Lsn(0x50), false, &ctx) + .get_rel_size(TESTREL_A, Version::Lsn(Lsn(0x50)), false, &ctx) .await?, 3 ); @@ -1710,46 +1719,46 @@ mod tests { // Check page contents at each LSN assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x20), false, &ctx) + .get_rel_page_at_lsn(TESTREL_A, 0, Version::Lsn(Lsn(0x20)), false, &ctx) .await?, TEST_IMG("foo blk 0 at 2") ); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x30), false, &ctx) + .get_rel_page_at_lsn(TESTREL_A, 0, Version::Lsn(Lsn(0x30)), false, &ctx) .await?, TEST_IMG("foo blk 0 at 3") ); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x40), false, &ctx) + .get_rel_page_at_lsn(TESTREL_A, 0, Version::Lsn(Lsn(0x40)), false, &ctx) .await?, TEST_IMG("foo blk 0 at 3") ); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x40), false, &ctx) + .get_rel_page_at_lsn(TESTREL_A, 1, Version::Lsn(Lsn(0x40)), false, &ctx) .await?, TEST_IMG("foo blk 1 at 4") ); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x50), false, &ctx) + .get_rel_page_at_lsn(TESTREL_A, 0, Version::Lsn(Lsn(0x50)), false, &ctx) .await?, TEST_IMG("foo blk 0 at 3") ); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x50), false, &ctx) + .get_rel_page_at_lsn(TESTREL_A, 1, Version::Lsn(Lsn(0x50)), false, &ctx) .await?, TEST_IMG("foo blk 1 at 4") ); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 2, Lsn(0x50), false, &ctx) + .get_rel_page_at_lsn(TESTREL_A, 2, Version::Lsn(Lsn(0x50)), false, &ctx) .await?, TEST_IMG("foo blk 2 at 5") ); @@ -1765,19 +1774,19 @@ mod tests { // Check reported size and contents after truncation assert_eq!( tline - .get_rel_size(TESTREL_A, Lsn(0x60), false, &ctx) + .get_rel_size(TESTREL_A, Version::Lsn(Lsn(0x60)), false, &ctx) .await?, 2 ); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x60), false, &ctx) + .get_rel_page_at_lsn(TESTREL_A, 0, Version::Lsn(Lsn(0x60)), false, &ctx) .await?, TEST_IMG("foo blk 0 at 3") ); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x60), false, &ctx) + .get_rel_page_at_lsn(TESTREL_A, 1, Version::Lsn(Lsn(0x60)), false, &ctx) .await?, TEST_IMG("foo blk 1 at 4") ); @@ -1785,13 +1794,13 @@ mod tests { // should still see the truncated block with older LSN assert_eq!( tline - .get_rel_size(TESTREL_A, Lsn(0x50), false, &ctx) + .get_rel_size(TESTREL_A, Version::Lsn(Lsn(0x50)), false, &ctx) .await?, 3 ); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 2, Lsn(0x50), false, &ctx) + .get_rel_page_at_lsn(TESTREL_A, 2, Version::Lsn(Lsn(0x50)), false, &ctx) .await?, TEST_IMG("foo blk 2 at 5") ); @@ -1804,7 +1813,7 @@ mod tests { m.commit(&ctx).await?; assert_eq!( tline - .get_rel_size(TESTREL_A, Lsn(0x68), false, &ctx) + .get_rel_size(TESTREL_A, Version::Lsn(Lsn(0x68)), false, &ctx) .await?, 0 ); @@ -1817,19 +1826,19 @@ mod tests { m.commit(&ctx).await?; assert_eq!( tline - .get_rel_size(TESTREL_A, Lsn(0x70), false, &ctx) + .get_rel_size(TESTREL_A, Version::Lsn(Lsn(0x70)), false, &ctx) .await?, 2 ); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 0, Lsn(0x70), false, &ctx) + .get_rel_page_at_lsn(TESTREL_A, 0, Version::Lsn(Lsn(0x70)), false, &ctx) .await?, ZERO_PAGE ); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 1, Lsn(0x70), false, &ctx) + .get_rel_page_at_lsn(TESTREL_A, 1, Version::Lsn(Lsn(0x70)), false, &ctx) .await?, TEST_IMG("foo blk 1") ); @@ -1842,21 +1851,21 @@ mod tests { m.commit(&ctx).await?; assert_eq!( tline - .get_rel_size(TESTREL_A, Lsn(0x80), false, &ctx) + .get_rel_size(TESTREL_A, Version::Lsn(Lsn(0x80)), false, &ctx) .await?, 1501 ); for blk in 2..1500 { assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, blk, Lsn(0x80), false, &ctx) + .get_rel_page_at_lsn(TESTREL_A, blk, Version::Lsn(Lsn(0x80)), false, &ctx) .await?, ZERO_PAGE ); } assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, 1500, Lsn(0x80), false, &ctx) + .get_rel_page_at_lsn(TESTREL_A, 1500, Version::Lsn(Lsn(0x80)), false, &ctx) .await?, TEST_IMG("foo blk 1500") ); @@ -1883,13 +1892,13 @@ mod tests { // Check that rel exists and size is correct assert_eq!( tline - .get_rel_exists(TESTREL_A, Lsn(0x20), false, &ctx) + .get_rel_exists(TESTREL_A, Version::Lsn(Lsn(0x20)), false, &ctx) .await?, true ); assert_eq!( tline - .get_rel_size(TESTREL_A, Lsn(0x20), false, &ctx) + .get_rel_size(TESTREL_A, Version::Lsn(Lsn(0x20)), false, &ctx) .await?, 1 ); @@ -1902,7 +1911,7 @@ mod tests { // Check that rel is not visible anymore assert_eq!( tline - .get_rel_exists(TESTREL_A, Lsn(0x30), false, &ctx) + .get_rel_exists(TESTREL_A, Version::Lsn(Lsn(0x30)), false, &ctx) .await?, false ); @@ -1920,13 +1929,13 @@ mod tests { // Check that rel exists and size is correct assert_eq!( tline - .get_rel_exists(TESTREL_A, Lsn(0x40), false, &ctx) + .get_rel_exists(TESTREL_A, Version::Lsn(Lsn(0x40)), false, &ctx) .await?, true ); assert_eq!( tline - .get_rel_size(TESTREL_A, Lsn(0x40), false, &ctx) + .get_rel_size(TESTREL_A, Version::Lsn(Lsn(0x40)), false, &ctx) .await?, 1 ); @@ -1959,24 +1968,24 @@ mod tests { // The relation was created at LSN 20, not visible at LSN 1 yet. assert_eq!( tline - .get_rel_exists(TESTREL_A, Lsn(0x10), false, &ctx) + .get_rel_exists(TESTREL_A, Version::Lsn(Lsn(0x10)), false, &ctx) .await?, false ); assert!(tline - .get_rel_size(TESTREL_A, Lsn(0x10), false, &ctx) + .get_rel_size(TESTREL_A, Version::Lsn(Lsn(0x10)), false, &ctx) .await .is_err()); assert_eq!( tline - .get_rel_exists(TESTREL_A, Lsn(0x20), false, &ctx) + .get_rel_exists(TESTREL_A, Version::Lsn(Lsn(0x20)), false, &ctx) .await?, true ); assert_eq!( tline - .get_rel_size(TESTREL_A, Lsn(0x20), false, &ctx) + .get_rel_size(TESTREL_A, Version::Lsn(Lsn(0x20)), false, &ctx) .await?, relsize ); @@ -1987,7 +1996,7 @@ mod tests { let data = format!("foo blk {} at {}", blkno, lsn); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, blkno, lsn, false, &ctx) + .get_rel_page_at_lsn(TESTREL_A, blkno, Version::Lsn(lsn), false, &ctx) .await?, TEST_IMG(&data) ); @@ -2004,7 +2013,7 @@ mod tests { // Check reported size and contents after truncation assert_eq!( tline - .get_rel_size(TESTREL_A, Lsn(0x60), false, &ctx) + .get_rel_size(TESTREL_A, Version::Lsn(Lsn(0x60)), false, &ctx) .await?, 1 ); @@ -2014,7 +2023,7 @@ mod tests { let data = format!("foo blk {} at {}", blkno, lsn); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, blkno, Lsn(0x60), false, &ctx) + .get_rel_page_at_lsn(TESTREL_A, blkno, Version::Lsn(Lsn(0x60)), false, &ctx) .await?, TEST_IMG(&data) ); @@ -2023,7 +2032,7 @@ mod tests { // should still see all blocks with older LSN assert_eq!( tline - .get_rel_size(TESTREL_A, Lsn(0x50), false, &ctx) + .get_rel_size(TESTREL_A, Version::Lsn(Lsn(0x50)), false, &ctx) .await?, relsize ); @@ -2032,7 +2041,7 @@ mod tests { let data = format!("foo blk {} at {}", blkno, lsn); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, blkno, Lsn(0x50), false, &ctx) + .get_rel_page_at_lsn(TESTREL_A, blkno, Version::Lsn(Lsn(0x50)), false, &ctx) .await?, TEST_IMG(&data) ); @@ -2052,13 +2061,13 @@ mod tests { assert_eq!( tline - .get_rel_exists(TESTREL_A, Lsn(0x80), false, &ctx) + .get_rel_exists(TESTREL_A, Version::Lsn(Lsn(0x80)), false, &ctx) .await?, true ); assert_eq!( tline - .get_rel_size(TESTREL_A, Lsn(0x80), false, &ctx) + .get_rel_size(TESTREL_A, Version::Lsn(Lsn(0x80)), false, &ctx) .await?, relsize ); @@ -2068,7 +2077,7 @@ mod tests { let data = format!("foo blk {} at {}", blkno, lsn); assert_eq!( tline - .get_rel_page_at_lsn(TESTREL_A, blkno, Lsn(0x80), false, &ctx) + .get_rel_page_at_lsn(TESTREL_A, blkno, Version::Lsn(Lsn(0x80)), false, &ctx) .await?, TEST_IMG(&data) ); @@ -2101,7 +2110,9 @@ mod tests { assert_current_logical_size(&tline, Lsn(lsn)); assert_eq!( - tline.get_rel_size(TESTREL_A, Lsn(lsn), false, &ctx).await?, + tline + .get_rel_size(TESTREL_A, Version::Lsn(Lsn(lsn)), false, &ctx) + .await?, RELSEG_SIZE + 1 ); @@ -2113,7 +2124,9 @@ mod tests { .await?; m.commit(&ctx).await?; assert_eq!( - tline.get_rel_size(TESTREL_A, Lsn(lsn), false, &ctx).await?, + tline + .get_rel_size(TESTREL_A, Version::Lsn(Lsn(lsn)), false, &ctx) + .await?, RELSEG_SIZE ); assert_current_logical_size(&tline, Lsn(lsn)); @@ -2126,7 +2139,9 @@ mod tests { .await?; m.commit(&ctx).await?; assert_eq!( - tline.get_rel_size(TESTREL_A, Lsn(lsn), false, &ctx).await?, + tline + .get_rel_size(TESTREL_A, Version::Lsn(Lsn(lsn)), false, &ctx) + .await?, RELSEG_SIZE - 1 ); assert_current_logical_size(&tline, Lsn(lsn)); @@ -2142,7 +2157,9 @@ mod tests { .await?; m.commit(&ctx).await?; assert_eq!( - tline.get_rel_size(TESTREL_A, Lsn(lsn), false, &ctx).await?, + tline + .get_rel_size(TESTREL_A, Version::Lsn(Lsn(lsn)), false, &ctx) + .await?, size as BlockNumber ); @@ -2179,7 +2196,7 @@ mod tests { let wal_segment_path = format!("{path}/000000010000000000000001.zst"); let source_initdb_path = format!("{path}/{INITDB_PATH}"); let startpoint = Lsn::from_hex("14AEC08").unwrap(); - let endpoint = Lsn::from_hex("1FFFF98").unwrap(); + let _endpoint = Lsn::from_hex("1FFFF98").unwrap(); let harness = TenantHarness::create("test_ingest_real_wal").unwrap(); let (tenant, ctx) = harness.load().await; @@ -2221,7 +2238,7 @@ mod tests { let mut walingest = WalIngest::new(tline.as_ref(), startpoint, &ctx) .await .unwrap(); - let mut modification = tline.begin_modification(endpoint); + let mut modification = tline.begin_modification(startpoint); let mut decoded = DecodedWALRecord::default(); println!("decoding {} bytes", bytes.len() - xlogoff); @@ -2235,6 +2252,7 @@ mod tests { .await .unwrap(); } + modification.commit(&ctx).await.unwrap(); } let duration = started_at.elapsed(); From 673a86505594d816b0eea2560a797291db8ed4bd Mon Sep 17 00:00:00 2001 From: John Spray Date: Wed, 3 Jan 2024 11:50:58 +0000 Subject: [PATCH 17/35] tests: tolerate 304 when evicting layers (#6261) In tests that evict layers, explicit eviction can race with automatic eviction of the same layer and result in a 304 --- test_runner/fixtures/pageserver/http.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test_runner/fixtures/pageserver/http.py b/test_runner/fixtures/pageserver/http.py index add6c4288a..6dea0d923d 100644 --- a/test_runner/fixtures/pageserver/http.py +++ b/test_runner/fixtures/pageserver/http.py @@ -714,7 +714,7 @@ class PageserverHttpClient(requests.Session): ) self.verbose_error(res) - assert res.status_code == 200 + assert res.status_code in (200, 304) def evict_all_layers(self, tenant_id: TenantId, timeline_id: TimelineId): info = self.layer_map_info(tenant_id, timeline_id) From 17b256679bb84c60401c801a779ecef41a395e00 Mon Sep 17 00:00:00 2001 From: Alexander Bayandin Date: Wed, 3 Jan 2024 13:02:04 +0000 Subject: [PATCH 18/35] vm-image-spec: build pgbouncer from Neon's fork (#6249) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## Problem We need to add one more patch to pgbouncer (for https://github.com/neondatabase/neon/issues/5801). I've decided to cherry-pick all required patches to a pgbouncer fork (`neondatabase/pgbouncer`) and use it instead. See https://github.com/neondatabase/pgbouncer/releases/tag/pgbouncer_1_21_0-neon-1 ## Summary of changes - Revert the previous patch (for deallocate/discard all) — the fork already contains it. - Remove `libssl-dev` dependency — we build pgbouncer without `openssl` support. - Clone git tag and build pgbouncer from source code. --- vm-image-spec.yaml | 21 ++++++++++----------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/vm-image-spec.yaml b/vm-image-spec.yaml index 68be0b3617..704e3721d6 100644 --- a/vm-image-spec.yaml +++ b/vm-image-spec.yaml @@ -167,22 +167,21 @@ build: | && apt-get update \ && apt-get install -y \ build-essential \ - curl \ + git \ libevent-dev \ - libssl-dev \ - patchutils \ + libtool \ pkg-config - ENV PGBOUNCER_VERSION 1.21.0 - ENV PGBOUNCER_GITPATH 1_21_0 + # Note, we use pgbouncer from neondatabase/pgbouncer fork, which could contain extra commits. + # Use `dist_man_MANS=` to skip manpage generation (which requires python3/pandoc) + ENV PGBOUNCER_TAG pgbouncer_1_21_0-neon-1 RUN set -e \ - && curl -sfSL https://github.com/pgbouncer/pgbouncer/releases/download/pgbouncer_${PGBOUNCER_GITPATH}/pgbouncer-${PGBOUNCER_VERSION}.tar.gz -o pgbouncer-${PGBOUNCER_VERSION}.tar.gz \ - && tar xzvf pgbouncer-${PGBOUNCER_VERSION}.tar.gz \ - && cd pgbouncer-${PGBOUNCER_VERSION} \ - && curl https://github.com/pgbouncer/pgbouncer/commit/a7b3c0a5f4caa9dbe92743d04cf1e28c4c05806c.patch | filterdiff --include a/src/server.c | patch -p1 \ + && git clone --recurse-submodules --depth 1 --branch ${PGBOUNCER_TAG} https://github.com/neondatabase/pgbouncer.git pgbouncer \ + && cd pgbouncer \ + && ./autogen.sh \ && LDFLAGS=-static ./configure --prefix=/usr/local/pgbouncer --without-openssl \ - && make -j $(nproc) \ - && make install + && make -j $(nproc) dist_man_MANS= \ + && make install dist_man_MANS= merge: | # tweak nofile limits RUN set -e \ From 65b4e6e7d6a89c1217c4c08c8118e252d3e17809 Mon Sep 17 00:00:00 2001 From: Arseny Sher Date: Tue, 2 Jan 2024 11:24:52 +0300 Subject: [PATCH 19/35] Remove empty safekeeper init since truncateLsn. It has caveats such as creating half empty segment which can't be offloaded. Instead we'll pursue approach of pull_timeline, seeding new state from some peer. --- pgxn/neon/walproposer.c | 34 ++++-------------------- test_runner/regress/test_wal_acceptor.py | 2 +- 2 files changed, 6 insertions(+), 30 deletions(-) diff --git a/pgxn/neon/walproposer.c b/pgxn/neon/walproposer.c index 2ea724f927..1f7c473e7d 100644 --- a/pgxn/neon/walproposer.c +++ b/pgxn/neon/walproposer.c @@ -1091,34 +1091,10 @@ SendProposerElected(Safekeeper *sk) { /* safekeeper is empty or no common point, start from the beginning */ sk->startStreamingAt = wp->propTermHistory.entries[0].lsn; - - if (sk->startStreamingAt < wp->truncateLsn) - { - /* - * There's a gap between the WAL starting point and a truncateLsn, - * which can't appear in a normal working cluster. That gap means - * that all safekeepers reported that they have persisted WAL up - * to the truncateLsn before, but now current safekeeper tells - * otherwise. - * - * Also we have a special condition here, which is empty - * safekeeper with no history. In combination with a gap, that can - * happen when we introduce a new safekeeper to the cluster. This - * is a rare case, which is triggered manually for now, and should - * be treated with care. - */ - - /* - * truncateLsn will not change without ack from current - * safekeeper, and it's aligned to the WAL record, so we can - * safely start streaming from this point. - */ - sk->startStreamingAt = wp->truncateLsn; - - wp_log(WARNING, "empty safekeeper joined cluster as %s:%s, historyStart=%X/%X, sk->startStreamingAt=%X/%X", - sk->host, sk->port, LSN_FORMAT_ARGS(wp->propTermHistory.entries[0].lsn), - LSN_FORMAT_ARGS(sk->startStreamingAt)); - } + wp_log(LOG, "no common point with sk %s:%s, streaming since first term at %X/%X, timelineStartLsn=%X/%X, termHistory.n_entries=%u" , + sk->host, sk->port, LSN_FORMAT_ARGS(sk->startStreamingAt), LSN_FORMAT_ARGS(wp->timelineStartLsn), wp->propTermHistory.n_entries); + /* wp->timelineStartLsn == InvalidXLogRecPtr can be only when timeline is created manually (test_s3_wal_replay) */ + Assert(sk->startStreamingAt == wp->timelineStartLsn || wp->timelineStartLsn == InvalidXLogRecPtr); } else { @@ -1141,7 +1117,7 @@ SendProposerElected(Safekeeper *sk) } } - Assert(sk->startStreamingAt >= wp->truncateLsn && sk->startStreamingAt <= wp->availableLsn); + Assert(sk->startStreamingAt <= wp->availableLsn); msg.tag = 'e'; msg.term = wp->propTerm; diff --git a/test_runner/regress/test_wal_acceptor.py b/test_runner/regress/test_wal_acceptor.py index cf8df389c8..4dfc883f4c 100644 --- a/test_runner/regress/test_wal_acceptor.py +++ b/test_runner/regress/test_wal_acceptor.py @@ -566,7 +566,7 @@ def test_s3_wal_replay(neon_env_builder: NeonEnvBuilder): f"Pageserver last_record_lsn={pageserver_lsn}; flush_lsn={last_lsn}; lag before replay is {lag / 1024}kb" ) - endpoint.stop_and_destroy() + endpoint.stop() timeline_delete_wait_completed(ps_http, tenant_id, timeline_id) # Also delete and manually create timeline on safekeepers -- this tests From edc962f1d7ac66bfb8dfe8f63d253d9389fc7b5c Mon Sep 17 00:00:00 2001 From: John Spray Date: Wed, 3 Jan 2024 14:22:17 +0000 Subject: [PATCH 20/35] test_runner: test_issue_5878 log allow list (#6259) ## Problem https://neon-github-public-dev.s3.amazonaws.com/reports/pr-6254/7388706419/index.html#suites/5a4b8734277a9878cb429b80c314f470/e54c4f6f6ed22672 ## Summary of changes Permit the log message: because the test helper's detach function increments the generation number, a detach/attach cycle can cause the error if the test runner node is slow enough for the opportunistic deletion queue flush on detach not to complete by the time we call attach. --- test_runner/regress/test_layers_from_future.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/test_runner/regress/test_layers_from_future.py b/test_runner/regress/test_layers_from_future.py index 340188c1ae..999e077e45 100644 --- a/test_runner/regress/test_layers_from_future.py +++ b/test_runner/regress/test_layers_from_future.py @@ -38,6 +38,9 @@ def test_issue_5878(neon_env_builder: NeonEnvBuilder): neon_env_builder.enable_pageserver_remote_storage(RemoteStorageKind.LOCAL_FS) env = neon_env_builder.init_start() + env.pageserver.allowed_errors.extend( + [".*Dropped remote consistent LSN updates.*", ".*Dropping stale deletions.*"] + ) ps_http = env.pageserver.http_client() From 753d97bd77f6d2c87f7167133585554eb463f01b Mon Sep 17 00:00:00 2001 From: John Spray Date: Fri, 22 Dec 2023 10:12:34 +0000 Subject: [PATCH 21/35] pageserver: don't delete ancestor shard layers --- .../src/tenant/remote_timeline_client.rs | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/pageserver/src/tenant/remote_timeline_client.rs b/pageserver/src/tenant/remote_timeline_client.rs index 1b0cf39fbe..60b40d70a7 100644 --- a/pageserver/src/tenant/remote_timeline_client.rs +++ b/pageserver/src/tenant/remote_timeline_client.rs @@ -818,8 +818,25 @@ impl RemoteTimelineClient { fn schedule_deletion_of_unlinked0( self: &Arc, upload_queue: &mut UploadQueueInitialized, - with_metadata: Vec<(LayerFileName, LayerFileMetadata)>, + mut with_metadata: Vec<(LayerFileName, LayerFileMetadata)>, ) { + // Filter out any layers which were not created by this tenant shard. These are + // layers that originate from some ancestor shard after a split, and may still + // be referenced by other shards. We are free to delete them locally and remove + // them from our index (and would have already done so when we reach this point + // in the code), but we may not delete them remotely. + with_metadata.retain(|(name, meta)| { + let retain = meta.shard.shard_number == self.tenant_shard_id.shard_number + && meta.shard.shard_count == self.tenant_shard_id.shard_count; + if !retain { + tracing::debug!( + "Skipping deletion of ancestor-shard layer {name}, from shard {}", + meta.shard + ); + } + retain + }); + for (name, meta) in &with_metadata { info!( "scheduling deletion of layer {}{} (shard {})", From 6c79e12630b1330e838af8eb07346550a8bf7912 Mon Sep 17 00:00:00 2001 From: John Spray Date: Wed, 27 Dec 2023 15:57:17 +0000 Subject: [PATCH 22/35] pageserver: drop unwanted keys during compaction after split --- libs/pageserver_api/src/keyspace.rs | 3 +++ libs/pageserver_api/src/shard.rs | 15 +++++++++++++ pageserver/src/tenant/timeline.rs | 33 ++++++++++++++++++++++++----- 3 files changed, 46 insertions(+), 5 deletions(-) diff --git a/libs/pageserver_api/src/keyspace.rs b/libs/pageserver_api/src/keyspace.rs index 16651c322e..80183506d8 100644 --- a/libs/pageserver_api/src/keyspace.rs +++ b/libs/pageserver_api/src/keyspace.rs @@ -124,6 +124,9 @@ impl KeySpaceAccum { if range.start == accum.end { accum.end = range.end; } else { + // TODO: to efficiently support small sharding stripe sizes, we should avoid starting + // a new range here if the skipped region was all keys that don't belong on this shard. + // (https://github.com/neondatabase/neon/issues/6247) assert!(range.start > accum.end); self.ranges.push(accum.clone()); *accum = range; diff --git a/libs/pageserver_api/src/shard.rs b/libs/pageserver_api/src/shard.rs index 3e4936eec4..a186d93bce 100644 --- a/libs/pageserver_api/src/shard.rs +++ b/libs/pageserver_api/src/shard.rs @@ -422,6 +422,21 @@ impl ShardIdentity { } } + /// Return true if the key should be discarded if found in this shard's + /// data store, e.g. during compaction after a split + pub fn is_key_disposable(&self, key: &Key) -> bool { + if key_is_shard0(key) { + // Q: Why can't we dispose of shard0 content if we're not shard 0? + // A: because the WAL ingestion logic currently ingests some shard 0 + // content on all shards, even though it's only read on shard 0. If we + // dropped it, then subsequent WAL ingest to these keys would encounter + // an error. + false + } else { + !self.is_key_local(key) + } + } + pub fn shard_slug(&self) -> String { if self.count > ShardCount(0) { format!("-{:02x}{:02x}", self.number.0, self.count.0) diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 15a5ca1727..e8340a74b2 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -496,6 +496,11 @@ impl Timeline { return Err(PageReconstructError::Other(anyhow::anyhow!("Invalid LSN"))); } + // This check is debug-only because of the cost of hashing, and because it's a double-check: we + // already checked the key against the shard_identity when looking up the Timeline from + // page_service. + debug_assert!(!self.shard_identity.is_key_disposable(&key)); + // XXX: structured stats collection for layer eviction here. trace!( "get page request for {}@{} from task kind {:?}", @@ -2224,13 +2229,13 @@ impl Timeline { return Err(layer_traversal_error( if cfg!(test) { format!( - "could not find data for key {} at LSN {}, for request at LSN {}\n{}", - key, cont_lsn, request_lsn, std::backtrace::Backtrace::force_capture(), + "could not find data for key {} (shard {:?}) at LSN {}, for request at LSN {}\n{}", + key, self.shard_identity.get_shard_number(&key), cont_lsn, request_lsn, std::backtrace::Backtrace::force_capture(), ) } else { format!( - "could not find data for key {} at LSN {}, for request at LSN {}", - key, cont_lsn, request_lsn + "could not find data for key {} (shard {:?}) at LSN {}, for request at LSN {}", + key, self.shard_identity.get_shard_number(&key), cont_lsn, request_lsn ) }, traversal_path, @@ -3054,6 +3059,15 @@ impl Timeline { for range in &partition.ranges { let mut key = range.start; while key < range.end { + if self.shard_identity.is_key_disposable(&key) { + debug!( + "Dropping key {} during compaction (it belongs on shard {:?})", + key, + self.shard_identity.get_shard_number(&key) + ); + key = key.next(); + continue; + } let img = match self.get(key, lsn, ctx).await { Ok(img) => img, Err(err) => { @@ -3080,6 +3094,7 @@ impl Timeline { } } }; + image_layer_writer.put_image(key, &img).await?; key = key.next(); } @@ -3650,7 +3665,15 @@ impl Timeline { ))) }); - writer.as_mut().unwrap().put_value(key, lsn, value).await?; + if !self.shard_identity.is_key_disposable(&key) { + writer.as_mut().unwrap().put_value(key, lsn, value).await?; + } else { + debug!( + "Dropping key {} during compaction (it belongs on shard {:?})", + key, + self.shard_identity.get_shard_number(&key) + ); + } if !new_layers.is_empty() { fail_point!("after-timeline-compacted-first-L1"); From ef7c9c2ccc1a385f74455f45b54faa5b101065e6 Mon Sep 17 00:00:00 2001 From: John Spray Date: Fri, 22 Dec 2023 13:58:00 +0000 Subject: [PATCH 23/35] pageserver: fix active tenant lookup hitting secondaries with sharding If there is some secondary shard for a tenant on the same node as an attached shard, the secondary shard could trip up this code and cause page_service to incorrectly get an error instead of finding the attached shard. --- pageserver/src/tenant/mgr.rs | 36 ++++++++++++++++++++---------------- 1 file changed, 20 insertions(+), 16 deletions(-) diff --git a/pageserver/src/tenant/mgr.rs b/pageserver/src/tenant/mgr.rs index 62922e8c99..250de7247d 100644 --- a/pageserver/src/tenant/mgr.rs +++ b/pageserver/src/tenant/mgr.rs @@ -130,7 +130,7 @@ impl TenantsMap { /// A page service client sends a TenantId, and to look up the correct Tenant we must /// resolve this to a fully qualified TenantShardId. - fn resolve_shard( + fn resolve_attached_shard( &self, tenant_id: &TenantId, selector: ShardSelector, @@ -140,25 +140,27 @@ impl TenantsMap { TenantsMap::Initializing => None, TenantsMap::Open(m) | TenantsMap::ShuttingDown(m) => { for slot in m.range(TenantShardId::tenant_range(*tenant_id)) { + // Ignore all slots that don't contain an attached tenant + let tenant = match &slot.1 { + TenantSlot::Attached(t) => t, + _ => continue, + }; + match selector { ShardSelector::First => return Some(*slot.0), ShardSelector::Zero if slot.0.shard_number == ShardNumber(0) => { return Some(*slot.0) } ShardSelector::Page(key) => { - if let Some(tenant) = slot.1.get_attached() { - // 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)); - } + // 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)); + } - if Some(tenant.shard_identity.number) == want_shard { - return Some(*slot.0); - } - } else { - continue; + if Some(tenant.shard_identity.number) == want_shard { + return Some(*slot.0); } } _ => continue, @@ -1257,9 +1259,11 @@ pub(crate) async fn get_active_tenant_with_timeout( let locked = TENANTS.read().unwrap(); // Resolve TenantId to TenantShardId - let tenant_shard_id = locked.resolve_shard(&tenant_id, shard_selector).ok_or( - GetActiveTenantError::NotFound(GetTenantError::NotFound(tenant_id)), - )?; + let tenant_shard_id = locked + .resolve_attached_shard(&tenant_id, shard_selector) + .ok_or(GetActiveTenantError::NotFound(GetTenantError::NotFound( + tenant_id, + )))?; let peek_slot = tenant_map_peek_slot(&locked, &tenant_shard_id, TenantSlotPeekMode::Read) .map_err(GetTenantError::MapState)?; From 34ebfbdd6f509f4bd2eab807c2730f987ba5b0df Mon Sep 17 00:00:00 2001 From: John Spray Date: Fri, 29 Dec 2023 15:13:00 +0000 Subject: [PATCH 24/35] pageserver: fix handling getpage with multiple shards on one node Previously, we would wait for the LSN to be visible on whichever timeline we happened to load at the start of the connection, then proceed to look up the correct timeline for the key and do the read. If the timeline holding the key was behind the timeline we used for the LSN wait, then we might serve an apparently-successful read result that actually contains data from behind the requested lsn. --- pageserver/src/page_service.rs | 41 +++++++++++++++++----------------- 1 file changed, 21 insertions(+), 20 deletions(-) diff --git a/pageserver/src/page_service.rs b/pageserver/src/page_service.rs index db07a600e5..be9f478f25 100644 --- a/pageserver/src/page_service.rs +++ b/pageserver/src/page_service.rs @@ -802,7 +802,7 @@ impl PageServerHandler { })) } - async fn handle_get_page_at_lsn_request( + async fn do_handle_get_page_at_lsn_request( &self, timeline: &Timeline, req: &PagestreamGetPageRequest, @@ -812,20 +812,25 @@ impl PageServerHandler { let lsn = Self::wait_or_get_last_lsn(timeline, req.lsn, req.latest, &latest_gc_cutoff_lsn, ctx) .await?; - /* - // Add a 1s delay to some requests. The delay helps the requests to - // hit the race condition from github issue #1047 more easily. - use rand::Rng; - if rand::thread_rng().gen::() < 5 { - std::thread::sleep(std::time::Duration::from_millis(1000)); - } - */ + let page = timeline + .get_rel_page_at_lsn(req.rel, req.blkno, Version::Lsn(lsn), req.latest, ctx) + .await?; + Ok(PagestreamBeMessage::GetPage(PagestreamGetPageResponse { + page, + })) + } + + async fn handle_get_page_at_lsn_request( + &self, + timeline: &Timeline, + req: &PagestreamGetPageRequest, + ctx: &RequestContext, + ) -> anyhow::Result { let key = rel_block_to_key(req.rel, req.blkno); - let page = if timeline.get_shard_identity().is_key_local(&key) { - timeline - .get_rel_page_at_lsn(req.rel, req.blkno, Version::Lsn(lsn), req.latest, ctx) - .await? + if timeline.get_shard_identity().is_key_local(&key) { + self.do_handle_get_page_at_lsn_request(timeline, req, ctx) + .await } else { // The Tenant shard we looked up at connection start does not hold this particular // key: look for other shards in this tenant. This scenario occurs if a pageserver @@ -860,14 +865,10 @@ impl PageServerHandler { // Take a GateGuard for the duration of this request. If we were using our main Timeline object, // the GateGuard was already held over the whole connection. let _timeline_guard = timeline.gate.enter().map_err(|_| QueryError::Shutdown)?; - timeline - .get_rel_page_at_lsn(req.rel, req.blkno, Version::Lsn(lsn), req.latest, ctx) - .await? - }; - Ok(PagestreamBeMessage::GetPage(PagestreamGetPageResponse { - page, - })) + self.do_handle_get_page_at_lsn_request(&timeline, req, ctx) + .await + } } #[allow(clippy::too_many_arguments)] From 73a944205b5d76efd791a3218004423204790789 Mon Sep 17 00:00:00 2001 From: John Spray Date: Fri, 22 Dec 2023 11:52:09 +0000 Subject: [PATCH 25/35] pageserver: log details on shard routing error --- pageserver/src/page_service.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pageserver/src/page_service.rs b/pageserver/src/page_service.rs index be9f478f25..d478d375f8 100644 --- a/pageserver/src/page_service.rs +++ b/pageserver/src/page_service.rs @@ -857,6 +857,8 @@ impl PageServerHandler { // informed yet. // // https://github.com/neondatabase/neon/issues/6038 + tracing::warn!("Page request routed to wrong shard: my identity {:?}, should go to shard {}, key {}", + timeline.get_shard_identity(), timeline.get_shard_identity().get_shard_number(&key).0, key); return Err(anyhow::anyhow!("Request routed to wrong shard")); } Err(e) => return Err(e.into()), From a2e083ebe04fd0283c26dba8d7fd9e981623aad9 Mon Sep 17 00:00:00 2001 From: John Spray Date: Fri, 29 Dec 2023 13:43:56 +0000 Subject: [PATCH 26/35] pageserver: make walredo shard-aware This does not have a functional impact, but enables all the logging in this code to include the shard_id label. --- pageserver/benches/bench_walredo.rs | 5 ++- pageserver/src/tenant.rs | 6 +-- pageserver/src/walredo.rs | 64 ++++++++++++++++------------- 3 files changed, 40 insertions(+), 35 deletions(-) diff --git a/pageserver/benches/bench_walredo.rs b/pageserver/benches/bench_walredo.rs index ba41866935..4837626086 100644 --- a/pageserver/benches/bench_walredo.rs +++ b/pageserver/benches/bench_walredo.rs @@ -13,6 +13,7 @@ use bytes::{Buf, Bytes}; use pageserver::{ config::PageServerConf, repository::Key, walrecord::NeonWalRecord, walredo::PostgresRedoManager, }; +use pageserver_api::shard::TenantShardId; use utils::{id::TenantId, lsn::Lsn}; use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; @@ -26,9 +27,9 @@ fn redo_scenarios(c: &mut Criterion) { let conf = PageServerConf::dummy_conf(repo_dir.path().to_path_buf()); let conf = Box::leak(Box::new(conf)); - let tenant_id = TenantId::generate(); + let tenant_shard_id = TenantShardId::unsharded(TenantId::generate()); - let manager = PostgresRedoManager::new(conf, tenant_id); + let manager = PostgresRedoManager::new(conf, tenant_shard_id); let manager = Arc::new(manager); diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index e50987c84b..1660de8923 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -595,10 +595,9 @@ impl Tenant { mode: SpawnMode, ctx: &RequestContext, ) -> anyhow::Result> { - // TODO(sharding): make WalRedoManager shard-aware let wal_redo_manager = Arc::new(WalRedoManager::from(PostgresRedoManager::new( conf, - tenant_shard_id.tenant_id, + tenant_shard_id, ))); let TenantSharedResources { @@ -1145,10 +1144,9 @@ impl Tenant { tenant_shard_id: TenantShardId, reason: String, ) -> Arc { - // TODO(sharding): make WalRedoManager shard-aware let wal_redo_manager = Arc::new(WalRedoManager::from(PostgresRedoManager::new( conf, - tenant_shard_id.tenant_id, + tenant_shard_id, ))); Arc::new(Tenant::new( TenantState::Broken { diff --git a/pageserver/src/walredo.rs b/pageserver/src/walredo.rs index 94e95fd3b3..6918698f29 100644 --- a/pageserver/src/walredo.rs +++ b/pageserver/src/walredo.rs @@ -22,6 +22,7 @@ use anyhow::Context; use byteorder::{ByteOrder, LittleEndian}; use bytes::{BufMut, Bytes, BytesMut}; use nix::poll::*; +use pageserver_api::shard::TenantShardId; use serde::Serialize; use std::collections::VecDeque; use std::io; @@ -35,14 +36,11 @@ use std::sync::{Arc, Mutex, MutexGuard, RwLock}; use std::time::Duration; use std::time::Instant; use tracing::*; -use utils::{bin_ser::BeSer, id::TenantId, lsn::Lsn, nonblock::set_nonblock}; +use utils::{bin_ser::BeSer, lsn::Lsn, nonblock::set_nonblock}; #[cfg(feature = "testing")] use std::sync::atomic::{AtomicUsize, Ordering}; -#[cfg(feature = "testing")] -use pageserver_api::shard::TenantShardId; - use crate::config::PageServerConf; use crate::metrics::{ WalRedoKillCause, WAL_REDO_BYTES_HISTOGRAM, WAL_REDO_PROCESS_COUNTERS, @@ -92,7 +90,7 @@ struct ProcessOutput { /// records. /// pub struct PostgresRedoManager { - tenant_id: TenantId, + tenant_shard_id: TenantShardId, conf: &'static PageServerConf, last_redo_at: std::sync::Mutex>, redo_process: RwLock>>, @@ -186,10 +184,13 @@ impl PostgresRedoManager { /// /// Create a new PostgresRedoManager. /// - pub fn new(conf: &'static PageServerConf, tenant_id: TenantId) -> PostgresRedoManager { + pub fn new( + conf: &'static PageServerConf, + tenant_shard_id: TenantShardId, + ) -> PostgresRedoManager { // The actual process is launched lazily, on first request. PostgresRedoManager { - tenant_id, + tenant_shard_id, conf, last_redo_at: std::sync::Mutex::default(), redo_process: RwLock::new(None), @@ -244,8 +245,12 @@ impl PostgresRedoManager { let timer = WAL_REDO_PROCESS_LAUNCH_DURATION_HISTOGRAM.start_timer(); let proc = Arc::new( - WalRedoProcess::launch(self.conf, self.tenant_id, pg_version) - .context("launch walredo process")?, + WalRedoProcess::launch( + self.conf, + self.tenant_shard_id, + pg_version, + ) + .context("launch walredo process")?, ); timer.observe_duration(); *proc_guard = Some(Arc::clone(&proc)); @@ -638,7 +643,7 @@ impl CloseFileDescriptors for C { struct WalRedoProcess { #[allow(dead_code)] conf: &'static PageServerConf, - tenant_id: TenantId, + tenant_shard_id: TenantShardId, // Some() on construction, only becomes None on Drop. child: Option, stdout: Mutex, @@ -652,10 +657,10 @@ impl WalRedoProcess { // // Start postgres binary in special WAL redo mode. // - #[instrument(skip_all,fields(tenant_id=%tenant_id, pg_version=pg_version))] + #[instrument(skip_all,fields(tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(), pg_version=pg_version))] fn launch( conf: &'static PageServerConf, - tenant_id: TenantId, + tenant_shard_id: TenantShardId, pg_version: u32, ) -> anyhow::Result { let pg_bin_dir_path = conf.pg_bin_dir(pg_version).context("pg_bin_dir")?; // TODO these should be infallible. @@ -680,7 +685,7 @@ impl WalRedoProcess { // as close-on-exec by default, but that's not enough, since we use // libraries that directly call libc open without setting that flag. .close_fds() - .spawn_no_leak_child(tenant_id) + .spawn_no_leak_child(tenant_shard_id) .context("spawn process")?; WAL_REDO_PROCESS_COUNTERS.started.inc(); let mut child = scopeguard::guard(child, |child| { @@ -741,12 +746,12 @@ impl WalRedoProcess { error!(error=?e, "failed to read from walredo stderr"); } } - }.instrument(tracing::info_span!(parent: None, "wal-redo-postgres-stderr", pid = child.id(), tenant_id = %tenant_id, %pg_version)) + }.instrument(tracing::info_span!(parent: None, "wal-redo-postgres-stderr", pid = child.id(), tenant_id = %tenant_shard_id.tenant_id, shard_id = %tenant_shard_id.shard_slug(), %pg_version)) ); Ok(Self { conf, - tenant_id, + tenant_shard_id, child: Some(child), stdin: Mutex::new(ProcessInput { stdin, @@ -772,7 +777,7 @@ impl WalRedoProcess { // Apply given WAL records ('records') over an old page image. Returns // new page image. // - #[instrument(skip_all, fields(tenant_id=%self.tenant_id, pid=%self.id()))] + #[instrument(skip_all, fields(tenant_id=%self.tenant_shard_id.tenant_id, shard_id=%self.tenant_shard_id.shard_slug(), pid=%self.id()))] fn apply_wal_records( &self, tag: BufferTag, @@ -966,11 +971,7 @@ impl WalRedoProcess { // these files will be collected to an allure report let filename = format!("walredo-{millis}-{}-{seq}.walredo", writebuf.len()); - // TODO(sharding): update this call when WalRedoProcess gets a TenantShardId. - let path = self - .conf - .tenant_path(&TenantShardId::unsharded(self.tenant_id)) - .join(&filename); + let path = self.conf.tenant_path(&self.tenant_shard_id).join(&filename); let res = std::fs::OpenOptions::new() .write(true) @@ -1004,7 +1005,7 @@ impl Drop for WalRedoProcess { /// Wrapper type around `std::process::Child` which guarantees that the child /// will be killed and waited-for by this process before being dropped. struct NoLeakChild { - tenant_id: TenantId, + tenant_id: TenantShardId, child: Option, } @@ -1023,7 +1024,7 @@ impl DerefMut for NoLeakChild { } impl NoLeakChild { - fn spawn(tenant_id: TenantId, command: &mut Command) -> io::Result { + fn spawn(tenant_id: TenantShardId, command: &mut Command) -> io::Result { let child = command.spawn()?; Ok(NoLeakChild { tenant_id, @@ -1078,7 +1079,7 @@ impl Drop for NoLeakChild { Some(child) => child, None => return, }; - let tenant_id = self.tenant_id; + let tenant_shard_id = self.tenant_id; // Offload the kill+wait of the child process into the background. // If someone stops the runtime, we'll leak the child process. // We can ignore that case because we only stop the runtime on pageserver exit. @@ -1086,7 +1087,11 @@ impl Drop for NoLeakChild { tokio::task::spawn_blocking(move || { // Intentionally don't inherit the tracing context from whoever is dropping us. // This thread here is going to outlive of our dropper. - let span = tracing::info_span!("walredo", %tenant_id); + let span = tracing::info_span!( + "walredo", + tenant_id = %tenant_shard_id.tenant_id, + shard_id = %tenant_shard_id.shard_slug() + ); let _entered = span.enter(); Self::kill_and_wait_impl(child, WalRedoKillCause::NoLeakChildDrop); }) @@ -1096,11 +1101,11 @@ impl Drop for NoLeakChild { } trait NoLeakChildCommandExt { - fn spawn_no_leak_child(&mut self, tenant_id: TenantId) -> io::Result; + fn spawn_no_leak_child(&mut self, tenant_id: TenantShardId) -> io::Result; } impl NoLeakChildCommandExt for Command { - fn spawn_no_leak_child(&mut self, tenant_id: TenantId) -> io::Result { + fn spawn_no_leak_child(&mut self, tenant_id: TenantShardId) -> io::Result { NoLeakChild::spawn(tenant_id, self) } } @@ -1155,6 +1160,7 @@ mod tests { use crate::repository::Key; use crate::{config::PageServerConf, walrecord::NeonWalRecord}; use bytes::Bytes; + use pageserver_api::shard::TenantShardId; use std::str::FromStr; use utils::{id::TenantId, lsn::Lsn}; @@ -1264,9 +1270,9 @@ mod tests { let repo_dir = camino_tempfile::tempdir()?; let conf = PageServerConf::dummy_conf(repo_dir.path().to_path_buf()); let conf = Box::leak(Box::new(conf)); - let tenant_id = TenantId::generate(); + let tenant_shard_id = TenantShardId::unsharded(TenantId::generate()); - let manager = PostgresRedoManager::new(conf, tenant_id); + let manager = PostgresRedoManager::new(conf, tenant_shard_id); Ok(RedoHarness { _repo_dir: repo_dir, From c119af8ddd019a5eebf37d28ce54c4424a1faab2 Mon Sep 17 00:00:00 2001 From: John Spray Date: Wed, 27 Dec 2023 16:02:09 +0000 Subject: [PATCH 27/35] pageserver: run at least 2 background task threads Otherwise an assertion in CONCURRENT_BACKGROUND_TASKS will trip if you try to run the pageserver on a single core. --- pageserver/src/task_mgr.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pageserver/src/task_mgr.rs b/pageserver/src/task_mgr.rs index cb1b2b8011..eabb1d0022 100644 --- a/pageserver/src/task_mgr.rs +++ b/pageserver/src/task_mgr.rs @@ -147,7 +147,7 @@ pub(crate) static BACKGROUND_RUNTIME_WORKER_THREADS: Lazy = Lazy::new(|| // else, but that has not been needed in a long time. std::env::var("TOKIO_WORKER_THREADS") .map(|s| s.parse::().unwrap()) - .unwrap_or_else(|_e| usize::max(1, num_cpus::get())) + .unwrap_or_else(|_e| usize::max(2, num_cpus::get())) }); #[derive(Debug, Clone, Copy)] From 7662df6ca0c5c0977571cb1fcae3746e6af4fbae Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Wed, 3 Jan 2024 11:36:38 -0800 Subject: [PATCH 28/35] Fix minimum backoff to 1ms --- pgxn/neon/libpagestore.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pgxn/neon/libpagestore.c b/pgxn/neon/libpagestore.c index e58c28d7d5..3a7c0f1bb6 100644 --- a/pgxn/neon/libpagestore.c +++ b/pgxn/neon/libpagestore.c @@ -35,7 +35,7 @@ #define PageStoreTrace DEBUG5 -#define MIN_RECONNECT_INTERVAL_USEC 100 +#define MIN_RECONNECT_INTERVAL_USEC 1000 #define MAX_RECONNECT_INTERVAL_USEC 1000000 bool connected = false; From 18e92081586c7b10c97b2981e8b26dfb24fe5095 Mon Sep 17 00:00:00 2001 From: John Spray Date: Thu, 4 Jan 2024 10:40:03 +0000 Subject: [PATCH 29/35] pageserver: improved error handling for shard routing error, timeline not found (#6262) ## Problem - When a client requests a key that isn't found in any shard on the node (edge case that only happens if a compute's config is out of date), we should prompt them to reconnect (as this includes a backoff), since they will not be able to complete the request until they eventually get a correct pageserver connection string. - QueryError::Other is used excessively: this contains a type-ambiguous anyhow::Error and is logged very verbosely (including backtrace). ## Summary of changes - Introduce PageStreamError to replace use of anyhow::Error in request handlers for getpage, etc. - Introduce Reconnect and NotFound variants to QueryError - Map the "shard routing error" case to PageStreamError::Reconnect -> QueryError::Reconnect - Update type conversions for LSN timeouts and tenant/timeline not found errors to use PageStreamError::NotFound->QueryError::NotFound --- libs/postgres_backend/src/lib.rs | 23 +++- libs/utils/src/http/error.rs | 7 ++ pageserver/src/http/routes.rs | 1 + pageserver/src/page_service.rs | 177 ++++++++++++++++++++-------- pageserver/src/pgdatadir_mapping.rs | 38 +++--- pageserver/src/tenant.rs | 11 +- pageserver/src/tenant/timeline.rs | 99 ++++++++-------- 7 files changed, 237 insertions(+), 119 deletions(-) diff --git a/libs/postgres_backend/src/lib.rs b/libs/postgres_backend/src/lib.rs index 1dae008a4f..73d25619c3 100644 --- a/libs/postgres_backend/src/lib.rs +++ b/libs/postgres_backend/src/lib.rs @@ -35,6 +35,12 @@ pub enum QueryError { /// We were instructed to shutdown while processing the query #[error("Shutting down")] Shutdown, + /// Query handler indicated that client should reconnect + #[error("Server requested reconnect")] + Reconnect, + /// Query named an entity that was not found + #[error("Not found: {0}")] + NotFound(std::borrow::Cow<'static, str>), /// Authentication failure #[error("Unauthorized: {0}")] Unauthorized(std::borrow::Cow<'static, str>), @@ -54,9 +60,9 @@ impl From for QueryError { impl QueryError { pub fn pg_error_code(&self) -> &'static [u8; 5] { match self { - Self::Disconnected(_) | Self::SimulatedConnectionError => b"08006", // connection failure + Self::Disconnected(_) | Self::SimulatedConnectionError | Self::Reconnect => b"08006", // connection failure Self::Shutdown => SQLSTATE_ADMIN_SHUTDOWN, - Self::Unauthorized(_) => SQLSTATE_INTERNAL_ERROR, + Self::Unauthorized(_) | Self::NotFound(_) => SQLSTATE_INTERNAL_ERROR, Self::Other(_) => SQLSTATE_INTERNAL_ERROR, // internal error } } @@ -425,6 +431,11 @@ impl PostgresBackend { info!("Stopped due to shutdown"); Ok(()) } + Err(QueryError::Reconnect) => { + // Dropping out of this loop implicitly disconnects + info!("Stopped due to handler reconnect request"); + Ok(()) + } Err(QueryError::Disconnected(e)) => { info!("Disconnected ({e:#})"); // Disconnection is not an error: we just use it that way internally to drop @@ -974,7 +985,9 @@ impl<'a, IO: AsyncRead + AsyncWrite + Unpin> AsyncWrite for CopyDataWriter<'a, I pub fn short_error(e: &QueryError) -> String { match e { QueryError::Disconnected(connection_error) => connection_error.to_string(), + QueryError::Reconnect => "reconnect".to_string(), QueryError::Shutdown => "shutdown".to_string(), + QueryError::NotFound(_) => "not found".to_string(), QueryError::Unauthorized(_e) => "JWT authentication error".to_string(), QueryError::SimulatedConnectionError => "simulated connection error".to_string(), QueryError::Other(e) => format!("{e:#}"), @@ -996,9 +1009,15 @@ fn log_query_error(query: &str, e: &QueryError) { QueryError::SimulatedConnectionError => { error!("query handler for query '{query}' failed due to a simulated connection error") } + QueryError::Reconnect => { + info!("query handler for '{query}' requested client to reconnect") + } QueryError::Shutdown => { info!("query handler for '{query}' cancelled during tenant shutdown") } + QueryError::NotFound(reason) => { + info!("query handler for '{query}' entity not found: {reason}") + } QueryError::Unauthorized(e) => { warn!("query handler for '{query}' failed with authentication error: {e}"); } diff --git a/libs/utils/src/http/error.rs b/libs/utils/src/http/error.rs index ac68b04888..3e9281ac81 100644 --- a/libs/utils/src/http/error.rs +++ b/libs/utils/src/http/error.rs @@ -31,6 +31,9 @@ pub enum ApiError { #[error("Shutting down")] ShuttingDown, + #[error("Timeout")] + Timeout(Cow<'static, str>), + #[error(transparent)] InternalServerError(anyhow::Error), } @@ -67,6 +70,10 @@ impl ApiError { err.to_string(), StatusCode::SERVICE_UNAVAILABLE, ), + ApiError::Timeout(err) => HttpErrorBody::response_from_msg_and_status( + err.to_string(), + StatusCode::REQUEST_TIMEOUT, + ), ApiError::InternalServerError(err) => HttpErrorBody::response_from_msg_and_status( err.to_string(), StatusCode::INTERNAL_SERVER_ERROR, diff --git a/pageserver/src/http/routes.rs b/pageserver/src/http/routes.rs index 157e6b4e3e..8265627cb5 100644 --- a/pageserver/src/http/routes.rs +++ b/pageserver/src/http/routes.rs @@ -152,6 +152,7 @@ impl From for ApiError { PageReconstructError::AncestorStopping(_) => { ApiError::ResourceUnavailable(format!("{pre}").into()) } + PageReconstructError::AncestorLsnTimeout(e) => ApiError::Timeout(format!("{e}").into()), PageReconstructError::WalRedo(pre) => ApiError::InternalServerError(pre), } } diff --git a/pageserver/src/page_service.rs b/pageserver/src/page_service.rs index d478d375f8..291490d016 100644 --- a/pageserver/src/page_service.rs +++ b/pageserver/src/page_service.rs @@ -25,6 +25,7 @@ use postgres_backend::{self, is_expected_io_error, AuthType, PostgresBackend, Qu use pq_proto::framed::ConnectionError; use pq_proto::FeStartupPacket; use pq_proto::{BeMessage, FeMessage, RowDescriptor}; +use std::borrow::Cow; use std::io; use std::net::TcpListener; use std::pin::pin; @@ -61,6 +62,9 @@ use crate::tenant::mgr; use crate::tenant::mgr::get_active_tenant_with_timeout; use crate::tenant::mgr::GetActiveTenantError; use crate::tenant::mgr::ShardSelector; +use crate::tenant::timeline::WaitLsnError; +use crate::tenant::GetTimelineError; +use crate::tenant::PageReconstructError; use crate::tenant::Timeline; use crate::trace::Tracer; @@ -283,6 +287,64 @@ struct PageServerHandler { connection_ctx: RequestContext, } +#[derive(thiserror::Error, Debug)] +enum PageStreamError { + /// We encountered an error that should prompt the client to reconnect: + /// in practice this means we drop the connection without sending a response. + #[error("Reconnect required: {0}")] + Reconnect(Cow<'static, str>), + + /// We were instructed to shutdown while processing the query + #[error("Shutting down")] + Shutdown, + + /// Something went wrong reading a page: this likely indicates a pageserver bug + #[error("Read error: {0}")] + Read(PageReconstructError), + + /// Ran out of time waiting for an LSN + #[error("LSN timeout: {0}")] + LsnTimeout(WaitLsnError), + + /// The entity required to serve the request (tenant or timeline) is not found, + /// or is not found in a suitable state to serve a request. + #[error("Not found: {0}")] + NotFound(std::borrow::Cow<'static, str>), + + /// Request asked for something that doesn't make sense, like an invalid LSN + #[error("Bad request: {0}")] + BadRequest(std::borrow::Cow<'static, str>), +} + +impl From for PageStreamError { + fn from(value: PageReconstructError) -> Self { + match value { + PageReconstructError::Cancelled => Self::Shutdown, + e => Self::Read(e), + } + } +} + +impl From for PageStreamError { + fn from(value: GetActiveTimelineError) -> Self { + match value { + GetActiveTimelineError::Tenant(GetActiveTenantError::Cancelled) => Self::Shutdown, + GetActiveTimelineError::Tenant(e) => Self::NotFound(format!("{e}").into()), + GetActiveTimelineError::Timeline(e) => Self::NotFound(format!("{e}").into()), + } + } +} + +impl From for PageStreamError { + fn from(value: WaitLsnError) -> Self { + match value { + e @ WaitLsnError::Timeout(_) => Self::LsnTimeout(e), + WaitLsnError::Shutdown => Self::Shutdown, + WaitLsnError::BadState => Self::Reconnect("Timeline is not active".into()), + } + } +} + impl PageServerHandler { pub fn new( conf: &'static PageServerConf, @@ -428,7 +490,7 @@ impl PageServerHandler { // Check that the timeline exists let timeline = tenant .get_timeline(timeline_id, true) - .map_err(|e| anyhow::anyhow!(e))?; + .map_err(|e| QueryError::NotFound(format!("{e}").into()))?; // Avoid starting new requests if the timeline has already started shutting down, // and block timeline shutdown until this request is complete, or drops out due @@ -520,32 +582,44 @@ impl PageServerHandler { } }; - if let Err(e) = &response { - // Requests may fail as soon as we are Stopping, even if the Timeline's cancellation token wasn't fired yet, - // because wait_lsn etc will drop out - // is_stopping(): [`Timeline::flush_and_shutdown`] has entered - // is_canceled(): [`Timeline::shutdown`]` has entered - if timeline.cancel.is_cancelled() || timeline.is_stopping() { + match response { + Err(PageStreamError::Shutdown) => { // If we fail to fulfil a request during shutdown, which may be _because_ of // shutdown, then do not send the error to the client. Instead just drop the // connection. - span.in_scope(|| info!("dropped response during shutdown: {e:#}")); + span.in_scope(|| info!("dropping connection due to shutdown")); return Err(QueryError::Shutdown); } + Err(PageStreamError::Reconnect(reason)) => { + span.in_scope(|| info!("handler requested reconnect: {reason}")); + return Err(QueryError::Reconnect); + } + Err(e) if timeline.cancel.is_cancelled() || timeline.is_stopping() => { + // This branch accomodates code within request handlers that returns an anyhow::Error instead of a clean + // shutdown error, this may be buried inside a PageReconstructError::Other for example. + // + // Requests may fail as soon as we are Stopping, even if the Timeline's cancellation token wasn't fired yet, + // because wait_lsn etc will drop out + // is_stopping(): [`Timeline::flush_and_shutdown`] has entered + // is_canceled(): [`Timeline::shutdown`]` has entered + span.in_scope(|| info!("dropped error response during shutdown: {e:#}")); + return Err(QueryError::Shutdown); + } + r => { + let response_msg = r.unwrap_or_else(|e| { + // print the all details to the log with {:#}, but for the client the + // error message is enough. Do not log if shutting down, as the anyhow::Error + // here includes cancellation which is not an error. + span.in_scope(|| error!("error reading relation or page version: {:#}", e)); + PagestreamBeMessage::Error(PagestreamErrorResponse { + message: e.to_string(), + }) + }); + + pgb.write_message_noflush(&BeMessage::CopyData(&response_msg.serialize()))?; + self.flush_cancellable(pgb, &timeline.cancel).await?; + } } - - let response = response.unwrap_or_else(|e| { - // print the all details to the log with {:#}, but for the client the - // error message is enough. Do not log if shutting down, as the anyhow::Error - // here includes cancellation which is not an error. - span.in_scope(|| error!("error reading relation or page version: {:#}", e)); - PagestreamBeMessage::Error(PagestreamErrorResponse { - message: e.to_string(), - }) - }); - - pgb.write_message_noflush(&BeMessage::CopyData(&response.serialize()))?; - self.flush_cancellable(pgb, &timeline.cancel).await?; } Ok(()) } @@ -692,7 +766,7 @@ impl PageServerHandler { latest: bool, latest_gc_cutoff_lsn: &RcuReadGuard, ctx: &RequestContext, - ) -> anyhow::Result { + ) -> Result { if latest { // Latest page version was requested. If LSN is given, it is a hint // to the page server that there have been no modifications to the @@ -723,15 +797,19 @@ impl PageServerHandler { } } else { if lsn == Lsn(0) { - anyhow::bail!("invalid LSN(0) in request"); + return Err(PageStreamError::BadRequest( + "invalid LSN(0) in request".into(), + )); } timeline.wait_lsn(lsn, ctx).await?; } - anyhow::ensure!( - lsn >= **latest_gc_cutoff_lsn, - "tried to request a page version that was garbage collected. requested at {} gc cutoff {}", - lsn, **latest_gc_cutoff_lsn - ); + + if lsn < **latest_gc_cutoff_lsn { + return Err(PageStreamError::BadRequest(format!( + "tried to request a page version that was garbage collected. requested at {} gc cutoff {}", + lsn, **latest_gc_cutoff_lsn + ).into())); + } Ok(lsn) } @@ -740,7 +818,7 @@ impl PageServerHandler { timeline: &Timeline, req: &PagestreamExistsRequest, ctx: &RequestContext, - ) -> anyhow::Result { + ) -> Result { let latest_gc_cutoff_lsn = timeline.get_latest_gc_cutoff_lsn(); let lsn = Self::wait_or_get_last_lsn(timeline, req.lsn, req.latest, &latest_gc_cutoff_lsn, ctx) @@ -760,7 +838,7 @@ impl PageServerHandler { timeline: &Timeline, req: &PagestreamNblocksRequest, ctx: &RequestContext, - ) -> anyhow::Result { + ) -> Result { let latest_gc_cutoff_lsn = timeline.get_latest_gc_cutoff_lsn(); let lsn = Self::wait_or_get_last_lsn(timeline, req.lsn, req.latest, &latest_gc_cutoff_lsn, ctx) @@ -780,7 +858,7 @@ impl PageServerHandler { timeline: &Timeline, req: &PagestreamDbSizeRequest, ctx: &RequestContext, - ) -> anyhow::Result { + ) -> Result { let latest_gc_cutoff_lsn = timeline.get_latest_gc_cutoff_lsn(); let lsn = Self::wait_or_get_last_lsn(timeline, req.lsn, req.latest, &latest_gc_cutoff_lsn, ctx) @@ -807,7 +885,7 @@ impl PageServerHandler { timeline: &Timeline, req: &PagestreamGetPageRequest, ctx: &RequestContext, - ) -> anyhow::Result { + ) -> Result { let latest_gc_cutoff_lsn = timeline.get_latest_gc_cutoff_lsn(); let lsn = Self::wait_or_get_last_lsn(timeline, req.lsn, req.latest, &latest_gc_cutoff_lsn, ctx) @@ -826,7 +904,7 @@ impl PageServerHandler { timeline: &Timeline, req: &PagestreamGetPageRequest, ctx: &RequestContext, - ) -> anyhow::Result { + ) -> Result { let key = rel_block_to_key(req.rel, req.blkno); if timeline.get_shard_identity().is_key_local(&key) { self.do_handle_get_page_at_lsn_request(timeline, req, ctx) @@ -849,24 +927,26 @@ impl PageServerHandler { Err(GetActiveTimelineError::Tenant(GetActiveTenantError::NotFound(_))) => { // We already know this tenant exists in general, because we resolved it at // start of connection. Getting a NotFound here indicates that the shard containing - // the requested page is not present on this node. - - // TODO: this should be some kind of structured error that the client will understand, - // so that it can block until its config is updated: this error is expected in the case - // that the Tenant's shards' placements are being updated and the client hasn't been - // informed yet. - // - // https://github.com/neondatabase/neon/issues/6038 - tracing::warn!("Page request routed to wrong shard: my identity {:?}, should go to shard {}, key {}", + // the requested page is not present on this node: the client's knowledge of shard->pageserver + // mapping is out of date. + tracing::info!("Page request routed to wrong shard: my identity {:?}, should go to shard {}, key {}", timeline.get_shard_identity(), timeline.get_shard_identity().get_shard_number(&key).0, key); - return Err(anyhow::anyhow!("Request routed to wrong shard")); + // Closing the connection by returning ``::Reconnect` has the side effect of rate-limiting above message, via + // client's reconnect backoff, as well as hopefully prompting the client to load its updated configuration + // and talk to a different pageserver. + return Err(PageStreamError::Reconnect( + "getpage@lsn request routed to wrong shard".into(), + )); } Err(e) => return Err(e.into()), }; // Take a GateGuard for the duration of this request. If we were using our main Timeline object, // the GateGuard was already held over the whole connection. - let _timeline_guard = timeline.gate.enter().map_err(|_| QueryError::Shutdown)?; + let _timeline_guard = timeline + .gate + .enter() + .map_err(|_| PageStreamError::Shutdown)?; self.do_handle_get_page_at_lsn_request(&timeline, req, ctx) .await @@ -1011,9 +1091,7 @@ impl PageServerHandler { ) .await .map_err(GetActiveTimelineError::Tenant)?; - let timeline = tenant - .get_timeline(timeline_id, true) - .map_err(|e| GetActiveTimelineError::Timeline(anyhow::anyhow!(e)))?; + let timeline = tenant.get_timeline(timeline_id, true)?; Ok(timeline) } } @@ -1435,14 +1513,15 @@ enum GetActiveTimelineError { #[error(transparent)] Tenant(GetActiveTenantError), #[error(transparent)] - Timeline(anyhow::Error), + Timeline(#[from] GetTimelineError), } impl From for QueryError { fn from(e: GetActiveTimelineError) -> Self { match e { + GetActiveTimelineError::Tenant(GetActiveTenantError::Cancelled) => QueryError::Shutdown, GetActiveTimelineError::Tenant(e) => e.into(), - GetActiveTimelineError::Timeline(e) => QueryError::Other(e), + GetActiveTimelineError::Timeline(e) => QueryError::NotFound(format!("{e}").into()), } } } diff --git a/pageserver/src/pgdatadir_mapping.rs b/pageserver/src/pgdatadir_mapping.rs index 9fe75e5baf..f11a72f2ab 100644 --- a/pageserver/src/pgdatadir_mapping.rs +++ b/pageserver/src/pgdatadir_mapping.rs @@ -160,7 +160,7 @@ impl Timeline { //------------------------------------------------------------------------------ /// Look up given page version. - pub async fn get_rel_page_at_lsn( + pub(crate) async fn get_rel_page_at_lsn( &self, tag: RelTag, blknum: BlockNumber, @@ -191,7 +191,7 @@ impl Timeline { } // Get size of a database in blocks - pub async fn get_db_size( + pub(crate) async fn get_db_size( &self, spcnode: Oid, dbnode: Oid, @@ -211,7 +211,7 @@ impl Timeline { } /// Get size of a relation file - pub async fn get_rel_size( + pub(crate) async fn get_rel_size( &self, tag: RelTag, version: Version<'_>, @@ -256,7 +256,7 @@ impl Timeline { } /// Does relation exist? - pub async fn get_rel_exists( + pub(crate) async fn get_rel_exists( &self, tag: RelTag, version: Version<'_>, @@ -291,7 +291,7 @@ impl Timeline { /// # Cancel-Safety /// /// This method is cancellation-safe. - pub async fn list_rels( + pub(crate) async fn list_rels( &self, spcnode: Oid, dbnode: Oid, @@ -319,7 +319,7 @@ impl Timeline { } /// Look up given SLRU page version. - pub async fn get_slru_page_at_lsn( + pub(crate) async fn get_slru_page_at_lsn( &self, kind: SlruKind, segno: u32, @@ -332,7 +332,7 @@ impl Timeline { } /// Get size of an SLRU segment - pub async fn get_slru_segment_size( + pub(crate) async fn get_slru_segment_size( &self, kind: SlruKind, segno: u32, @@ -345,7 +345,7 @@ impl Timeline { } /// Get size of an SLRU segment - pub async fn get_slru_segment_exists( + pub(crate) async fn get_slru_segment_exists( &self, kind: SlruKind, segno: u32, @@ -372,7 +372,7 @@ impl Timeline { /// so it's not well defined which LSN you get if there were multiple commits /// "in flight" at that point in time. /// - pub async fn find_lsn_for_timestamp( + pub(crate) async fn find_lsn_for_timestamp( &self, search_timestamp: TimestampTz, cancel: &CancellationToken, @@ -452,7 +452,7 @@ impl Timeline { /// Additionally, sets 'found_smaller'/'found_Larger, if encounters any commits /// with a smaller/larger timestamp. /// - pub async fn is_latest_commit_timestamp_ge_than( + pub(crate) async fn is_latest_commit_timestamp_ge_than( &self, search_timestamp: TimestampTz, probe_lsn: Lsn, @@ -475,7 +475,7 @@ impl Timeline { /// Obtain the possible timestamp range for the given lsn. /// /// If the lsn has no timestamps, returns None. returns `(min, max, median)` if it has timestamps. - pub async fn get_timestamp_for_lsn( + pub(crate) async fn get_timestamp_for_lsn( &self, probe_lsn: Lsn, ctx: &RequestContext, @@ -532,7 +532,7 @@ impl Timeline { } /// Get a list of SLRU segments - pub async fn list_slru_segments( + pub(crate) async fn list_slru_segments( &self, kind: SlruKind, version: Version<'_>, @@ -548,7 +548,7 @@ impl Timeline { } } - pub async fn get_relmap_file( + pub(crate) async fn get_relmap_file( &self, spcnode: Oid, dbnode: Oid, @@ -561,7 +561,7 @@ impl Timeline { Ok(buf) } - pub async fn list_dbdirs( + pub(crate) async fn list_dbdirs( &self, lsn: Lsn, ctx: &RequestContext, @@ -575,7 +575,7 @@ impl Timeline { } } - pub async fn get_twophase_file( + pub(crate) async fn get_twophase_file( &self, xid: TransactionId, lsn: Lsn, @@ -586,7 +586,7 @@ impl Timeline { Ok(buf) } - pub async fn list_twophase_files( + pub(crate) async fn list_twophase_files( &self, lsn: Lsn, ctx: &RequestContext, @@ -600,7 +600,7 @@ impl Timeline { } } - pub async fn get_control_file( + pub(crate) async fn get_control_file( &self, lsn: Lsn, ctx: &RequestContext, @@ -608,7 +608,7 @@ impl Timeline { self.get(CONTROLFILE_KEY, lsn, ctx).await } - pub async fn get_checkpoint( + pub(crate) async fn get_checkpoint( &self, lsn: Lsn, ctx: &RequestContext, @@ -616,7 +616,7 @@ impl Timeline { self.get(CHECKPOINT_KEY, lsn, ctx).await } - pub async fn list_aux_files( + pub(crate) async fn list_aux_files( &self, lsn: Lsn, ctx: &RequestContext, diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index 1660de8923..7c609452e5 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -56,6 +56,7 @@ use self::timeline::uninit::TimelineUninitMark; use self::timeline::uninit::UninitializedTimeline; use self::timeline::EvictionTaskTenantState; use self::timeline::TimelineResources; +use self::timeline::WaitLsnError; use crate::config::PageServerConf; use crate::context::{DownloadBehavior, RequestContext}; use crate::deletion_queue::DeletionQueueClient; @@ -1758,7 +1759,15 @@ impl Tenant { // decoding the new WAL might need to look up previous pages, relation // sizes etc. and that would get confused if the previous page versions // are not in the repository yet. - ancestor_timeline.wait_lsn(*lsn, ctx).await?; + ancestor_timeline + .wait_lsn(*lsn, ctx) + .await + .map_err(|e| match e { + e @ (WaitLsnError::Timeout(_) | WaitLsnError::BadState) => { + CreateTimelineError::AncestorLsn(anyhow::anyhow!(e)) + } + WaitLsnError::Shutdown => CreateTimelineError::ShuttingDown, + })?; } self.branch_timeline( diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index e8340a74b2..24a92859b7 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -373,15 +373,20 @@ pub struct GcInfo { } /// An error happened in a get() operation. -#[derive(thiserror::Error)] -pub enum PageReconstructError { +#[derive(thiserror::Error, Debug)] +pub(crate) enum PageReconstructError { #[error(transparent)] Other(#[from] anyhow::Error), + #[error("Ancestor LSN wait error: {0}")] + AncestorLsnTimeout(#[from] WaitLsnError), + /// The operation was cancelled + #[error("Cancelled")] Cancelled, /// The ancestor of this is being stopped + #[error("ancestor timeline {0} is being stopped")] AncestorStopping(TimelineId), /// An error happened replaying WAL records @@ -402,32 +407,6 @@ enum FlushLayerError { Other(#[from] anyhow::Error), } -impl std::fmt::Debug for PageReconstructError { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> { - match self { - Self::Other(err) => err.fmt(f), - Self::Cancelled => write!(f, "cancelled"), - Self::AncestorStopping(timeline_id) => { - write!(f, "ancestor timeline {timeline_id} is being stopped") - } - Self::WalRedo(err) => err.fmt(f), - } - } -} - -impl std::fmt::Display for PageReconstructError { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> Result<(), std::fmt::Error> { - match self { - Self::Other(err) => err.fmt(f), - Self::Cancelled => write!(f, "cancelled"), - Self::AncestorStopping(timeline_id) => { - write!(f, "ancestor timeline {timeline_id} is being stopped") - } - Self::WalRedo(err) => err.fmt(f), - } - } -} - #[derive(Clone, Copy)] pub enum LogicalSizeCalculationCause { Initial, @@ -452,6 +431,21 @@ impl std::fmt::Debug for Timeline { } } +#[derive(thiserror::Error, Debug)] +pub(crate) enum WaitLsnError { + // Called on a timeline which is shutting down + #[error("Shutdown")] + Shutdown, + + // Called on an timeline not in active state or shutting down + #[error("Bad state (not active)")] + BadState, + + // Timeout expired while waiting for LSN to catch up with goal. + #[error("{0}")] + Timeout(String), +} + /// Public interface functions impl Timeline { /// Get the LSN where this branch was created @@ -486,7 +480,7 @@ impl Timeline { /// # Cancel-Safety /// /// This method is cancellation-safe. - pub async fn get( + pub(crate) async fn get( &self, key: Key, lsn: Lsn, @@ -634,24 +628,28 @@ impl Timeline { /// You should call this before any of the other get_* or list_* functions. Calling /// those functions with an LSN that has been processed yet is an error. /// - pub async fn wait_lsn( + pub(crate) async fn wait_lsn( &self, lsn: Lsn, _ctx: &RequestContext, /* Prepare for use by cancellation */ - ) -> anyhow::Result<()> { - anyhow::ensure!(self.is_active(), "Cannot wait for Lsn on inactive timeline"); + ) -> Result<(), WaitLsnError> { + if self.cancel.is_cancelled() { + return Err(WaitLsnError::Shutdown); + } else if !self.is_active() { + return Err(WaitLsnError::BadState); + } // This should never be called from the WAL receiver, because that could lead // to a deadlock. - anyhow::ensure!( + debug_assert!( task_mgr::current_task_kind() != Some(TaskKind::WalReceiverManager), "wait_lsn cannot be called in WAL receiver" ); - anyhow::ensure!( + debug_assert!( task_mgr::current_task_kind() != Some(TaskKind::WalReceiverConnectionHandler), "wait_lsn cannot be called in WAL receiver" ); - anyhow::ensure!( + debug_assert!( task_mgr::current_task_kind() != Some(TaskKind::WalReceiverConnectionPoller), "wait_lsn cannot be called in WAL receiver" ); @@ -665,18 +663,22 @@ impl Timeline { { Ok(()) => Ok(()), Err(e) => { - // don't count the time spent waiting for lock below, and also in walreceiver.status(), towards the wait_lsn_time_histo - drop(_timer); - let walreceiver_status = self.walreceiver_status(); - Err(anyhow::Error::new(e).context({ - format!( + use utils::seqwait::SeqWaitError::*; + match e { + Shutdown => Err(WaitLsnError::Shutdown), + Timeout => { + // don't count the time spent waiting for lock below, and also in walreceiver.status(), towards the wait_lsn_time_histo + drop(_timer); + let walreceiver_status = self.walreceiver_status(); + Err(WaitLsnError::Timeout(format!( "Timed out while waiting for WAL record at LSN {} to arrive, last_record_lsn {} disk consistent LSN={}, WalReceiver status: {}", lsn, self.get_last_record_lsn(), self.get_disk_consistent_lsn(), walreceiver_status, - ) - })) + ))) + } + } } } } @@ -2295,11 +2297,12 @@ impl Timeline { ancestor .wait_lsn(timeline.ancestor_lsn, ctx) .await - .with_context(|| { - format!( - "wait for lsn {} on ancestor timeline_id={}", - timeline.ancestor_lsn, ancestor.timeline_id - ) + .map_err(|e| match e { + e @ WaitLsnError::Timeout(_) => PageReconstructError::AncestorLsnTimeout(e), + WaitLsnError::Shutdown => PageReconstructError::Cancelled, + e @ WaitLsnError::BadState => { + PageReconstructError::Other(anyhow::anyhow!(e)) + } })?; timeline_owned = ancestor; @@ -4228,7 +4231,7 @@ impl Timeline { .context("Failed to reconstruct a page image:") { Ok(img) => img, - Err(e) => return Err(PageReconstructError::from(e)), + Err(e) => return Err(PageReconstructError::WalRedo(e)), }; if img.len() == page_cache::PAGE_SZ { From f3b5db1443f7c55fc5dcae0032970846f5c69b7e Mon Sep 17 00:00:00 2001 From: Arthur Petukhovsky Date: Thu, 4 Jan 2024 21:40:38 +0400 Subject: [PATCH 30/35] Add API for safekeeper timeline copy (#6091) Implement API for cloning a single timeline inside a safekeeper. Also add API for calculating a sha256 hash of WAL, which is used in tests. `/copy` API works by copying objects inside S3 for all but the last segments, and the last segments are copied on-disk. A special temporary directory is created for a timeline, because copy can take a lot of time, especially for large timelines. After all files segments have been prepared, this directory is mounted to the main tree and timeline is loaded to memory. Some caveats: - large timelines can take a lot of time to copy, because we need to copy many S3 segments - caller should wait for HTTP call to finish indefinetely and don't close the HTTP connection, because it will stop the process, which is not continued in the background - `until_lsn` must be a valid LSN, otherwise bad things can happen - API will return 200 if specified `timeline_id` already exists, even if it's not a copy - each safekeeper will try to copy S3 segments, so it's better to not call this API in-parallel on different safekeepers --- Cargo.lock | 1 + libs/remote_storage/src/azure_blob.rs | 6 + libs/remote_storage/src/lib.rs | 13 + libs/remote_storage/src/local_fs.rs | 14 ++ libs/remote_storage/src/s3_bucket.rs | 32 +++ libs/remote_storage/src/s3_bucket/metrics.rs | 8 +- libs/remote_storage/src/simulate_failures.rs | 7 + libs/safekeeper_api/src/models.rs | 6 + safekeeper/Cargo.toml | 1 + safekeeper/src/control_file.rs | 7 +- safekeeper/src/copy_timeline.rs | 250 +++++++++++++++++++ safekeeper/src/debug_dump.rs | 57 +++++ safekeeper/src/http/routes.rs | 65 ++++- safekeeper/src/lib.rs | 1 + safekeeper/src/pull_timeline.rs | 131 +++++++--- safekeeper/src/timeline.rs | 3 +- safekeeper/src/timelines_global_map.rs | 15 +- safekeeper/src/wal_backup.rs | 61 ++++- safekeeper/src/wal_storage.rs | 2 +- test_runner/fixtures/neon_fixtures.py | 22 ++ test_runner/regress/test_wal_acceptor.py | 80 ++++++ 21 files changed, 727 insertions(+), 55 deletions(-) create mode 100644 safekeeper/src/copy_timeline.rs diff --git a/Cargo.lock b/Cargo.lock index 93efbadd79..4dd195a895 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4475,6 +4475,7 @@ dependencies = [ "serde", "serde_json", "serde_with", + "sha2", "signal-hook", "storage_broker", "thiserror", diff --git a/libs/remote_storage/src/azure_blob.rs b/libs/remote_storage/src/azure_blob.rs index 7ea1103eb2..18cf5d97ba 100644 --- a/libs/remote_storage/src/azure_blob.rs +++ b/libs/remote_storage/src/azure_blob.rs @@ -322,6 +322,12 @@ impl RemoteStorage for AzureBlobStorage { } Ok(()) } + + async fn copy(&self, _from: &RemotePath, _to: &RemotePath) -> anyhow::Result<()> { + Err(anyhow::anyhow!( + "copy for azure blob storage is not implemented" + )) + } } pin_project_lite::pin_project! { diff --git a/libs/remote_storage/src/lib.rs b/libs/remote_storage/src/lib.rs index 3e408e3119..942d0016b0 100644 --- a/libs/remote_storage/src/lib.rs +++ b/libs/remote_storage/src/lib.rs @@ -207,6 +207,9 @@ pub trait RemoteStorage: Send + Sync + 'static { async fn delete(&self, path: &RemotePath) -> anyhow::Result<()>; async fn delete_objects<'a>(&self, paths: &'a [RemotePath]) -> anyhow::Result<()>; + + /// Copy a remote object inside a bucket from one path to another. + async fn copy(&self, from: &RemotePath, to: &RemotePath) -> anyhow::Result<()>; } pub type DownloadStream = Pin> + Unpin + Send + Sync>>; @@ -374,6 +377,15 @@ impl GenericRemoteStorage { Self::Unreliable(s) => s.delete_objects(paths).await, } } + + pub async fn copy_object(&self, from: &RemotePath, to: &RemotePath) -> anyhow::Result<()> { + match self { + Self::LocalFs(s) => s.copy(from, to).await, + Self::AwsS3(s) => s.copy(from, to).await, + Self::AzureBlob(s) => s.copy(from, to).await, + Self::Unreliable(s) => s.copy(from, to).await, + } + } } impl GenericRemoteStorage { @@ -660,6 +672,7 @@ impl ConcurrencyLimiter { RequestKind::Put => &self.write, RequestKind::List => &self.read, RequestKind::Delete => &self.write, + RequestKind::Copy => &self.write, } } diff --git a/libs/remote_storage/src/local_fs.rs b/libs/remote_storage/src/local_fs.rs index d1e7d325b9..bf8b6b5dde 100644 --- a/libs/remote_storage/src/local_fs.rs +++ b/libs/remote_storage/src/local_fs.rs @@ -409,6 +409,20 @@ impl RemoteStorage for LocalFs { } Ok(()) } + + async fn copy(&self, from: &RemotePath, to: &RemotePath) -> anyhow::Result<()> { + let from_path = from.with_base(&self.storage_root); + let to_path = to.with_base(&self.storage_root); + create_target_directory(&to_path).await?; + fs::copy(&from_path, &to_path).await.with_context(|| { + format!( + "Failed to copy file from '{from_path}' to '{to_path}'", + from_path = from_path, + to_path = to_path + ) + })?; + Ok(()) + } } fn storage_metadata_path(original_path: &Utf8Path) -> Utf8PathBuf { diff --git a/libs/remote_storage/src/s3_bucket.rs b/libs/remote_storage/src/s3_bucket.rs index 0f95458ad1..d7b41edaaf 100644 --- a/libs/remote_storage/src/s3_bucket.rs +++ b/libs/remote_storage/src/s3_bucket.rs @@ -493,6 +493,38 @@ impl RemoteStorage for S3Bucket { Ok(()) } + async fn copy(&self, from: &RemotePath, to: &RemotePath) -> anyhow::Result<()> { + let kind = RequestKind::Copy; + let _guard = self.permit(kind).await; + + let started_at = start_measuring_requests(kind); + + // we need to specify bucket_name as a prefix + let copy_source = format!( + "{}/{}", + self.bucket_name, + self.relative_path_to_s3_object(from) + ); + + let res = self + .client + .copy_object() + .bucket(self.bucket_name.clone()) + .key(self.relative_path_to_s3_object(to)) + .copy_source(copy_source) + .send() + .await; + + let started_at = ScopeGuard::into_inner(started_at); + metrics::BUCKET_METRICS + .req_seconds + .observe_elapsed(kind, &res, started_at); + + res?; + + Ok(()) + } + async fn download(&self, from: &RemotePath) -> Result { // if prefix is not none then download file `prefix/from` // if prefix is none then download file `from` diff --git a/libs/remote_storage/src/s3_bucket/metrics.rs b/libs/remote_storage/src/s3_bucket/metrics.rs index ea11edafa5..21dde14906 100644 --- a/libs/remote_storage/src/s3_bucket/metrics.rs +++ b/libs/remote_storage/src/s3_bucket/metrics.rs @@ -11,6 +11,7 @@ pub(crate) enum RequestKind { Put = 1, Delete = 2, List = 3, + Copy = 4, } use RequestKind::*; @@ -22,6 +23,7 @@ impl RequestKind { Put => "put_object", Delete => "delete_object", List => "list_objects", + Copy => "copy_object", } } const fn as_index(&self) -> usize { @@ -29,7 +31,7 @@ impl RequestKind { } } -pub(super) struct RequestTyped([C; 4]); +pub(super) struct RequestTyped([C; 5]); impl RequestTyped { pub(super) fn get(&self, kind: RequestKind) -> &C { @@ -38,8 +40,8 @@ impl RequestTyped { fn build_with(mut f: impl FnMut(RequestKind) -> C) -> Self { use RequestKind::*; - let mut it = [Get, Put, Delete, List].into_iter(); - let arr = std::array::from_fn::(|index| { + let mut it = [Get, Put, Delete, List, Copy].into_iter(); + let arr = std::array::from_fn::(|index| { let next = it.next().unwrap(); assert_eq!(index, next.as_index()); f(next) diff --git a/libs/remote_storage/src/simulate_failures.rs b/libs/remote_storage/src/simulate_failures.rs index 802b0db7f5..7f5adcea30 100644 --- a/libs/remote_storage/src/simulate_failures.rs +++ b/libs/remote_storage/src/simulate_failures.rs @@ -162,4 +162,11 @@ impl RemoteStorage for UnreliableWrapper { } Ok(()) } + + async fn copy(&self, from: &RemotePath, to: &RemotePath) -> anyhow::Result<()> { + // copy is equivalent to download + upload + self.attempt(RemoteOp::Download(from.clone()))?; + self.attempt(RemoteOp::Upload(to.clone()))?; + self.inner.copy_object(from, to).await + } } diff --git a/libs/safekeeper_api/src/models.rs b/libs/safekeeper_api/src/models.rs index 786712deb1..ce5a1e411e 100644 --- a/libs/safekeeper_api/src/models.rs +++ b/libs/safekeeper_api/src/models.rs @@ -51,3 +51,9 @@ pub struct SkTimelineInfo { #[serde(default)] pub http_connstr: Option, } + +#[derive(Debug, Clone, Deserialize, Serialize)] +pub struct TimelineCopyRequest { + pub target_timeline_id: TimelineId, + pub until_lsn: Lsn, +} diff --git a/safekeeper/Cargo.toml b/safekeeper/Cargo.toml index 4015c27933..364cad7892 100644 --- a/safekeeper/Cargo.toml +++ b/safekeeper/Cargo.toml @@ -54,6 +54,7 @@ postgres_ffi.workspace = true pq_proto.workspace = true remote_storage.workspace = true safekeeper_api.workspace = true +sha2.workspace = true sd-notify.workspace = true storage_broker.workspace = true tokio-stream.workspace = true diff --git a/safekeeper/src/control_file.rs b/safekeeper/src/control_file.rs index 7aadd67ac6..591bfea182 100644 --- a/safekeeper/src/control_file.rs +++ b/safekeeper/src/control_file.rs @@ -66,12 +66,10 @@ impl FileStorage { /// Create file storage for a new timeline, but don't persist it yet. pub fn create_new( - ttid: &TenantTimelineId, + timeline_dir: Utf8PathBuf, conf: &SafeKeeperConf, state: SafeKeeperState, ) -> Result { - let timeline_dir = conf.timeline_dir(ttid); - let store = FileStorage { timeline_dir, conf: conf.clone(), @@ -277,7 +275,8 @@ mod test { .await .expect("failed to create timeline dir"); let state = SafeKeeperState::empty(); - let storage = FileStorage::create_new(ttid, conf, state.clone())?; + let timeline_dir = conf.timeline_dir(ttid); + let storage = FileStorage::create_new(timeline_dir, conf, state.clone())?; Ok((storage, state)) } diff --git a/safekeeper/src/copy_timeline.rs b/safekeeper/src/copy_timeline.rs new file mode 100644 index 0000000000..ef88eb27e3 --- /dev/null +++ b/safekeeper/src/copy_timeline.rs @@ -0,0 +1,250 @@ +use std::sync::Arc; + +use anyhow::{bail, Result}; +use camino::Utf8PathBuf; + +use postgres_ffi::{MAX_SEND_SIZE, WAL_SEGMENT_SIZE}; +use tokio::{ + fs::OpenOptions, + io::{AsyncSeekExt, AsyncWriteExt}, +}; +use tracing::{info, warn}; +use utils::{id::TenantTimelineId, lsn::Lsn}; + +use crate::{ + control_file::{FileStorage, Storage}, + pull_timeline::{create_temp_timeline_dir, load_temp_timeline, validate_temp_timeline}, + safekeeper::SafeKeeperState, + timeline::{Timeline, TimelineError}, + wal_backup::copy_s3_segments, + wal_storage::{wal_file_paths, WalReader}, + GlobalTimelines, SafeKeeperConf, +}; + +// we don't want to have more than 10 segments on disk after copy, because they take space +const MAX_BACKUP_LAG: u64 = 10 * WAL_SEGMENT_SIZE as u64; + +pub struct Request { + pub source: Arc, + pub until_lsn: Lsn, + pub destination_ttid: TenantTimelineId, +} + +pub async fn handle_request(request: Request) -> Result<()> { + // TODO: request.until_lsn MUST be a valid LSN, and we cannot check it :( + // if LSN will point to the middle of a WAL record, timeline will be in "broken" state + + match GlobalTimelines::get(request.destination_ttid) { + // timeline already exists. would be good to check that this timeline is the copy + // of the source timeline, but it isn't obvious how to do that + Ok(_) => return Ok(()), + // timeline not found, we are going to create it + Err(TimelineError::NotFound(_)) => {} + // error, probably timeline was deleted + res => { + res?; + } + } + + let conf = &GlobalTimelines::get_global_config(); + let ttid = request.destination_ttid; + + let (_tmp_dir, tli_dir_path) = create_temp_timeline_dir(conf, ttid).await?; + + let (mem_state, state) = request.source.get_state().await; + let start_lsn = state.timeline_start_lsn; + if start_lsn == Lsn::INVALID { + bail!("timeline is not initialized"); + } + let backup_lsn = mem_state.backup_lsn; + + { + let commit_lsn = mem_state.commit_lsn; + let flush_lsn = request.source.get_flush_lsn().await; + + info!( + "collected info about source timeline: start_lsn={}, backup_lsn={}, commit_lsn={}, flush_lsn={}", + start_lsn, backup_lsn, commit_lsn, flush_lsn + ); + + assert!(backup_lsn >= start_lsn); + assert!(commit_lsn >= start_lsn); + assert!(flush_lsn >= start_lsn); + + if request.until_lsn > flush_lsn { + bail!("requested LSN is beyond the end of the timeline"); + } + if request.until_lsn < start_lsn { + bail!("requested LSN is before the start of the timeline"); + } + + if request.until_lsn > commit_lsn { + warn!("copy_timeline WAL is not fully committed"); + } + + if backup_lsn < request.until_lsn && request.until_lsn.0 - backup_lsn.0 > MAX_BACKUP_LAG { + // we have a lot of segments that are not backed up. we can try to wait here until + // segments will be backed up to remote storage, but it's not clear how long to wait + bail!("too many segments are not backed up"); + } + } + + let wal_seg_size = state.server.wal_seg_size as usize; + if wal_seg_size == 0 { + bail!("wal_seg_size is not set"); + } + + let first_segment = start_lsn.segment_number(wal_seg_size); + let last_segment = request.until_lsn.segment_number(wal_seg_size); + + let new_backup_lsn = { + // we can't have new backup_lsn greater than existing backup_lsn or start of the last segment + let max_backup_lsn = backup_lsn.min(Lsn(last_segment * wal_seg_size as u64)); + + if max_backup_lsn <= start_lsn { + // probably we are starting from the first segment, which was not backed up yet. + // note that start_lsn can be in the middle of the segment + start_lsn + } else { + // we have some segments backed up, so we will assume all WAL below max_backup_lsn is backed up + assert!(max_backup_lsn.segment_offset(wal_seg_size) == 0); + max_backup_lsn + } + }; + + // all previous segments will be copied inside S3 + let first_ondisk_segment = new_backup_lsn.segment_number(wal_seg_size); + assert!(first_ondisk_segment <= last_segment); + assert!(first_ondisk_segment >= first_segment); + + copy_s3_segments( + wal_seg_size, + &request.source.ttid, + &request.destination_ttid, + first_segment, + first_ondisk_segment, + ) + .await?; + + copy_disk_segments( + conf, + &state, + wal_seg_size, + &request.source.ttid, + new_backup_lsn, + request.until_lsn, + &tli_dir_path, + ) + .await?; + + let mut new_state = SafeKeeperState::new( + &request.destination_ttid, + state.server.clone(), + vec![], + request.until_lsn, + start_lsn, + ); + new_state.timeline_start_lsn = start_lsn; + new_state.peer_horizon_lsn = request.until_lsn; + new_state.backup_lsn = new_backup_lsn; + + let mut file_storage = FileStorage::create_new(tli_dir_path.clone(), conf, new_state.clone())?; + file_storage.persist(&new_state).await?; + + // now we have a ready timeline in a temp directory + validate_temp_timeline(conf, request.destination_ttid, &tli_dir_path).await?; + load_temp_timeline(conf, request.destination_ttid, &tli_dir_path).await?; + + Ok(()) +} + +async fn copy_disk_segments( + conf: &SafeKeeperConf, + persisted_state: &SafeKeeperState, + wal_seg_size: usize, + source_ttid: &TenantTimelineId, + start_lsn: Lsn, + end_lsn: Lsn, + tli_dir_path: &Utf8PathBuf, +) -> Result<()> { + let mut wal_reader = WalReader::new( + conf.workdir.clone(), + conf.timeline_dir(source_ttid), + persisted_state, + start_lsn, + true, + )?; + + let mut buf = [0u8; MAX_SEND_SIZE]; + + let first_segment = start_lsn.segment_number(wal_seg_size); + let last_segment = end_lsn.segment_number(wal_seg_size); + + for segment in first_segment..=last_segment { + let segment_start = segment * wal_seg_size as u64; + let segment_end = segment_start + wal_seg_size as u64; + + let copy_start = segment_start.max(start_lsn.0); + let copy_end = segment_end.min(end_lsn.0); + + let copy_start = copy_start - segment_start; + let copy_end = copy_end - segment_start; + + let wal_file_path = { + let (normal, partial) = wal_file_paths(tli_dir_path, segment, wal_seg_size)?; + + if segment == last_segment { + partial + } else { + normal + } + }; + + write_segment( + &mut buf, + &wal_file_path, + wal_seg_size as u64, + copy_start, + copy_end, + &mut wal_reader, + ) + .await?; + } + + Ok(()) +} + +async fn write_segment( + buf: &mut [u8], + file_path: &Utf8PathBuf, + wal_seg_size: u64, + from: u64, + to: u64, + reader: &mut WalReader, +) -> Result<()> { + assert!(from <= to); + assert!(to <= wal_seg_size); + + let mut file = OpenOptions::new() + .create(true) + .write(true) + .open(&file_path) + .await?; + + // maybe fill with zeros, as in wal_storage.rs? + file.set_len(wal_seg_size).await?; + file.seek(std::io::SeekFrom::Start(from)).await?; + + let mut bytes_left = to - from; + while bytes_left > 0 { + let len = bytes_left as usize; + let len = len.min(buf.len()); + let len = reader.read(&mut buf[..len]).await?; + file.write_all(&buf[..len]).await?; + bytes_left -= len as u64; + } + + file.flush().await?; + file.sync_all().await?; + Ok(()) +} diff --git a/safekeeper/src/debug_dump.rs b/safekeeper/src/debug_dump.rs index daf9255ecb..c9ff1afdea 100644 --- a/safekeeper/src/debug_dump.rs +++ b/safekeeper/src/debug_dump.rs @@ -7,13 +7,16 @@ use std::io::Read; use std::path::PathBuf; use std::sync::Arc; +use anyhow::bail; use anyhow::Result; use camino::Utf8Path; use chrono::{DateTime, Utc}; use postgres_ffi::XLogSegNo; +use postgres_ffi::MAX_SEND_SIZE; use serde::Deserialize; use serde::Serialize; +use sha2::{Digest, Sha256}; use utils::id::NodeId; use utils::id::TenantTimelineId; use utils::id::{TenantId, TimelineId}; @@ -25,6 +28,7 @@ use crate::safekeeper::TermHistory; use crate::SafeKeeperConf; use crate::send_wal::WalSenderState; +use crate::wal_storage::WalReader; use crate::GlobalTimelines; /// Various filters that influence the resulting JSON output. @@ -300,3 +304,56 @@ fn build_config(config: SafeKeeperConf) -> Config { wal_backup_enabled: config.wal_backup_enabled, } } + +#[derive(Debug, Clone, Deserialize, Serialize)] +pub struct TimelineDigestRequest { + pub from_lsn: Lsn, + pub until_lsn: Lsn, +} + +#[derive(Debug, Serialize, Deserialize)] +pub struct TimelineDigest { + pub sha256: String, +} + +pub async fn calculate_digest( + tli: &Arc, + request: TimelineDigestRequest, +) -> Result { + if request.from_lsn > request.until_lsn { + bail!("from_lsn is greater than until_lsn"); + } + + let conf = GlobalTimelines::get_global_config(); + let (_, persisted_state) = tli.get_state().await; + + if persisted_state.timeline_start_lsn > request.from_lsn { + bail!("requested LSN is before the start of the timeline"); + } + + let mut wal_reader = WalReader::new( + conf.workdir.clone(), + tli.timeline_dir.clone(), + &persisted_state, + request.from_lsn, + true, + )?; + + let mut hasher = Sha256::new(); + let mut buf = [0u8; MAX_SEND_SIZE]; + + let mut bytes_left = (request.until_lsn.0 - request.from_lsn.0) as usize; + while bytes_left > 0 { + let bytes_to_read = std::cmp::min(buf.len(), bytes_left); + let bytes_read = wal_reader.read(&mut buf[..bytes_to_read]).await?; + if bytes_read == 0 { + bail!("wal_reader.read returned 0 bytes"); + } + hasher.update(&buf[..bytes_read]); + bytes_left -= bytes_read; + } + + let digest = hasher.finalize(); + let digest = hex::encode(digest); + Ok(TimelineDigest { sha256: digest }) +} diff --git a/safekeeper/src/http/routes.rs b/safekeeper/src/http/routes.rs index 25a3334e63..5283ea19c1 100644 --- a/safekeeper/src/http/routes.rs +++ b/safekeeper/src/http/routes.rs @@ -2,7 +2,7 @@ use hyper::{Body, Request, Response, StatusCode, Uri}; use once_cell::sync::Lazy; use postgres_ffi::WAL_SEGMENT_SIZE; -use safekeeper_api::models::SkTimelineInfo; +use safekeeper_api::models::{SkTimelineInfo, TimelineCopyRequest}; use serde::{Deserialize, Serialize}; use std::collections::{HashMap, HashSet}; use std::fmt; @@ -14,19 +14,21 @@ use tokio::fs::File; use tokio::io::AsyncReadExt; use tokio_util::sync::CancellationToken; use utils::failpoint_support::failpoints_handler; +use utils::http::request::parse_query_param; use std::io::Write as _; use tokio::sync::mpsc; use tokio_stream::wrappers::ReceiverStream; -use tracing::info_span; +use tracing::{info_span, Instrument}; use utils::http::endpoint::{request_span, ChannelWriter}; +use crate::debug_dump::TimelineDigestRequest; use crate::receive_wal::WalReceiverState; use crate::safekeeper::Term; use crate::safekeeper::{ServerInfo, TermLsn}; use crate::send_wal::WalSenderState; use crate::timeline::PeerInfo; -use crate::{debug_dump, pull_timeline}; +use crate::{copy_timeline, debug_dump, pull_timeline}; use crate::timelines_global_map::TimelineDeleteForceResult; use crate::GlobalTimelines; @@ -204,6 +206,56 @@ async fn timeline_pull_handler(mut request: Request) -> Result) -> Result, ApiError> { + check_permission(&request, None)?; + + let request_data: TimelineCopyRequest = json_request(&mut request).await?; + let ttid = TenantTimelineId::new( + parse_request_param(&request, "tenant_id")?, + parse_request_param(&request, "source_timeline_id")?, + ); + + let source = GlobalTimelines::get(ttid)?; + + copy_timeline::handle_request(copy_timeline::Request{ + source, + until_lsn: request_data.until_lsn, + destination_ttid: TenantTimelineId::new(ttid.tenant_id, request_data.target_timeline_id), + }) + .instrument(info_span!("copy_timeline", from=%ttid, to=%request_data.target_timeline_id, until_lsn=%request_data.until_lsn)) + .await + .map_err(ApiError::InternalServerError)?; + + json_response(StatusCode::OK, ()) +} + +async fn timeline_digest_handler(request: Request) -> Result, ApiError> { + let ttid = TenantTimelineId::new( + parse_request_param(&request, "tenant_id")?, + parse_request_param(&request, "timeline_id")?, + ); + check_permission(&request, Some(ttid.tenant_id))?; + + let from_lsn: Option = parse_query_param(&request, "from_lsn")?; + let until_lsn: Option = parse_query_param(&request, "until_lsn")?; + + let request = TimelineDigestRequest { + from_lsn: from_lsn.ok_or(ApiError::BadRequest(anyhow::anyhow!( + "from_lsn is required" + )))?, + until_lsn: until_lsn.ok_or(ApiError::BadRequest(anyhow::anyhow!( + "until_lsn is required" + )))?, + }; + + let tli = GlobalTimelines::get(ttid).map_err(ApiError::from)?; + + let response = debug_dump::calculate_digest(&tli, request) + .await + .map_err(ApiError::InternalServerError)?; + json_response(StatusCode::OK, response) +} + /// Download a file from the timeline directory. // TODO: figure out a better way to copy files between safekeepers async fn timeline_files_handler(request: Request) -> Result, ApiError> { @@ -472,11 +524,18 @@ pub fn make_router(conf: SafeKeeperConf) -> RouterBuilder "/v1/tenant/:tenant_id/timeline/:timeline_id/file/:filename", |r| request_span(r, timeline_files_handler), ) + .post( + "/v1/tenant/:tenant_id/timeline/:source_timeline_id/copy", + |r| request_span(r, timeline_copy_handler), + ) // for tests .post("/v1/record_safekeeper_info/:tenant_id/:timeline_id", |r| { request_span(r, record_safekeeper_info) }) .get("/v1/debug_dump", |r| request_span(r, dump_debug_handler)) + .get("/v1/tenant/:tenant_id/timeline/:timeline_id/digest", |r| { + request_span(r, timeline_digest_handler) + }) } #[cfg(test)] diff --git a/safekeeper/src/lib.rs b/safekeeper/src/lib.rs index 3a086f1f54..fc5f99eb00 100644 --- a/safekeeper/src/lib.rs +++ b/safekeeper/src/lib.rs @@ -16,6 +16,7 @@ mod auth; pub mod broker; pub mod control_file; pub mod control_file_upgrade; +pub mod copy_timeline; pub mod debug_dump; pub mod handler; pub mod http; diff --git a/safekeeper/src/pull_timeline.rs b/safekeeper/src/pull_timeline.rs index ad3a18a536..93b51f32c0 100644 --- a/safekeeper/src/pull_timeline.rs +++ b/safekeeper/src/pull_timeline.rs @@ -1,16 +1,24 @@ +use std::sync::Arc; + +use camino::Utf8PathBuf; +use camino_tempfile::Utf8TempDir; use chrono::{DateTime, Utc}; use serde::{Deserialize, Serialize}; use anyhow::{bail, Context, Result}; use tokio::io::AsyncWriteExt; use tracing::info; -use utils::id::{TenantId, TenantTimelineId, TimelineId}; +use utils::{ + id::{TenantId, TenantTimelineId, TimelineId}, + lsn::Lsn, +}; use crate::{ control_file, debug_dump, http::routes::TimelineStatus, + timeline::{Timeline, TimelineError}, wal_storage::{self, Storage}, - GlobalTimelines, + GlobalTimelines, SafeKeeperConf, }; /// Info about timeline on safekeeper ready for reporting. @@ -91,7 +99,7 @@ pub async fn handle_request(request: Request) -> Result { async fn pull_timeline(status: TimelineStatus, host: String) -> Result { let ttid = TenantTimelineId::new(status.tenant_id, status.timeline_id); info!( - "Pulling timeline {} from safekeeper {}, commit_lsn={}, flush_lsn={}, term={}, epoch={}", + "pulling timeline {} from safekeeper {}, commit_lsn={}, flush_lsn={}, term={}, epoch={}", ttid, host, status.commit_lsn, @@ -121,14 +129,14 @@ async fn pull_timeline(status: TimelineStatus, host: String) -> Result if dump.timelines.len() != 1 { bail!( - "Expected to fetch single timeline, got {} timelines", + "expected to fetch single timeline, got {} timelines", dump.timelines.len() ); } let timeline = dump.timelines.into_iter().next().unwrap(); let disk_content = timeline.disk_content.ok_or(anyhow::anyhow!( - "Timeline {} doesn't have disk content", + "timeline {} doesn't have disk content", ttid ))?; @@ -155,29 +163,12 @@ async fn pull_timeline(status: TimelineStatus, host: String) -> Result filenames.insert(0, "safekeeper.control".to_string()); info!( - "Downloading {} files from safekeeper {}", + "downloading {} files from safekeeper {}", filenames.len(), host ); - // Creating temp directory for a new timeline. It needs to be - // located on the same filesystem as the rest of the timelines. - - // conf.workdir is usually /storage/safekeeper/data - // will try to transform it into /storage/safekeeper/tmp - let temp_base = conf - .workdir - .parent() - .ok_or(anyhow::anyhow!("workdir has no parent"))? - .join("tmp"); - - tokio::fs::create_dir_all(&temp_base).await?; - - let tli_dir = camino_tempfile::Builder::new() - .suffix("_temptli") - .prefix(&format!("{}_{}_", ttid.tenant_id, ttid.timeline_id)) - .tempdir_in(temp_base)?; - let tli_dir_path = tli_dir.path().to_path_buf(); + let (_tmp_dir, tli_dir_path) = create_temp_timeline_dir(conf, ttid).await?; // Note: some time happens between fetching list of files and fetching files themselves. // It's possible that some files will be removed from safekeeper and we will fail to fetch them. @@ -201,47 +192,105 @@ async fn pull_timeline(status: TimelineStatus, host: String) -> Result // TODO: fsync? // Let's create timeline from temp directory and verify that it's correct + let (commit_lsn, flush_lsn) = validate_temp_timeline(conf, ttid, &tli_dir_path).await?; + info!( + "finished downloading timeline {}, commit_lsn={}, flush_lsn={}", + ttid, commit_lsn, flush_lsn + ); + assert!(status.commit_lsn <= status.flush_lsn); - let control_path = tli_dir_path.join("safekeeper.control"); + // Finally, load the timeline. + let _tli = load_temp_timeline(conf, ttid, &tli_dir_path).await?; + + Ok(Response { + safekeeper_host: host, + }) +} + +/// Create temp directory for a new timeline. It needs to be located on the same +/// filesystem as the rest of the timelines. It will be automatically deleted when +/// Utf8TempDir goes out of scope. +pub async fn create_temp_timeline_dir( + conf: &SafeKeeperConf, + ttid: TenantTimelineId, +) -> Result<(Utf8TempDir, Utf8PathBuf)> { + // conf.workdir is usually /storage/safekeeper/data + // will try to transform it into /storage/safekeeper/tmp + let temp_base = conf + .workdir + .parent() + .ok_or(anyhow::anyhow!("workdir has no parent"))? + .join("tmp"); + + tokio::fs::create_dir_all(&temp_base).await?; + + let tli_dir = camino_tempfile::Builder::new() + .suffix("_temptli") + .prefix(&format!("{}_{}_", ttid.tenant_id, ttid.timeline_id)) + .tempdir_in(temp_base)?; + + let tli_dir_path = tli_dir.path().to_path_buf(); + + Ok((tli_dir, tli_dir_path)) +} + +/// Do basic validation of a temp timeline, before moving it to the global map. +pub async fn validate_temp_timeline( + conf: &SafeKeeperConf, + ttid: TenantTimelineId, + path: &Utf8PathBuf, +) -> Result<(Lsn, Lsn)> { + let control_path = path.join("safekeeper.control"); let control_store = control_file::FileStorage::load_control_file(control_path)?; if control_store.server.wal_seg_size == 0 { bail!("wal_seg_size is not set"); } - let wal_store = - wal_storage::PhysicalStorage::new(&ttid, tli_dir_path.clone(), conf, &control_store)?; + let wal_store = wal_storage::PhysicalStorage::new(&ttid, path.clone(), conf, &control_store)?; - let commit_lsn = status.commit_lsn; + let commit_lsn = control_store.commit_lsn; let flush_lsn = wal_store.flush_lsn(); - info!( - "Finished downloading timeline {}, commit_lsn={}, flush_lsn={}", - ttid, commit_lsn, flush_lsn - ); - assert!(status.commit_lsn <= status.flush_lsn); + Ok((commit_lsn, flush_lsn)) +} + +/// Move timeline from a temp directory to the main storage, and load it to the global map. +/// This operation is done under a lock to prevent bugs if several concurrent requests are +/// trying to load the same timeline. Note that it doesn't guard against creating the +/// timeline with the same ttid, but no one should be doing this anyway. +pub async fn load_temp_timeline( + conf: &SafeKeeperConf, + ttid: TenantTimelineId, + tmp_path: &Utf8PathBuf, +) -> Result> { + // Take a lock to prevent concurrent loadings + let load_lock = GlobalTimelines::loading_lock().await; + let guard = load_lock.lock().await; + + if !matches!(GlobalTimelines::get(ttid), Err(TimelineError::NotFound(_))) { + bail!("timeline already exists, cannot overwrite it") + } // Move timeline dir to the correct location let timeline_path = conf.timeline_dir(&ttid); info!( - "Moving timeline {} from {} to {}", - ttid, tli_dir_path, timeline_path + "moving timeline {} from {} to {}", + ttid, tmp_path, timeline_path ); tokio::fs::create_dir_all(conf.tenant_dir(&ttid.tenant_id)).await?; - tokio::fs::rename(tli_dir_path, &timeline_path).await?; + tokio::fs::rename(tmp_path, &timeline_path).await?; - let tli = GlobalTimelines::load_timeline(ttid) + let tli = GlobalTimelines::load_timeline(&guard, ttid) .await .context("Failed to load timeline after copy")?; info!( - "Loaded timeline {}, flush_lsn={}", + "loaded timeline {}, flush_lsn={}", ttid, tli.get_flush_lsn().await ); - Ok(Response { - safekeeper_host: host, - }) + Ok(tli) } diff --git a/safekeeper/src/timeline.rs b/safekeeper/src/timeline.rs index bdc9088138..2f284abe8c 100644 --- a/safekeeper/src/timeline.rs +++ b/safekeeper/src/timeline.rs @@ -141,7 +141,8 @@ impl SharedState { // We don't want to write anything to disk, because we may have existing timeline there. // These functions should not change anything on disk. - let control_store = control_file::FileStorage::create_new(ttid, conf, state)?; + let timeline_dir = conf.timeline_dir(ttid); + let control_store = control_file::FileStorage::create_new(timeline_dir, conf, state)?; let wal_store = wal_storage::PhysicalStorage::new(ttid, conf.timeline_dir(ttid), conf, &control_store)?; let sk = SafeKeeper::new(control_store, wal_store, conf.my_id)?; diff --git a/safekeeper/src/timelines_global_map.rs b/safekeeper/src/timelines_global_map.rs index cbb3342e40..92ac5ba66d 100644 --- a/safekeeper/src/timelines_global_map.rs +++ b/safekeeper/src/timelines_global_map.rs @@ -21,8 +21,12 @@ struct GlobalTimelinesState { timelines: HashMap>, wal_backup_launcher_tx: Option>, conf: Option, + load_lock: Arc>, } +// Used to prevent concurrent timeline loading. +pub struct TimelineLoadLock; + impl GlobalTimelinesState { /// Get configuration, which must be set once during init. fn get_conf(&self) -> &SafeKeeperConf { @@ -63,6 +67,7 @@ static TIMELINES_STATE: Lazy> = Lazy::new(|| { timelines: HashMap::new(), wal_backup_launcher_tx: None, conf: None, + load_lock: Arc::new(tokio::sync::Mutex::new(TimelineLoadLock)), }) }); @@ -174,8 +179,16 @@ impl GlobalTimelines { Ok(()) } + /// Take a lock for timeline loading. + pub async fn loading_lock() -> Arc> { + TIMELINES_STATE.lock().unwrap().load_lock.clone() + } + /// Load timeline from disk to the memory. - pub async fn load_timeline(ttid: TenantTimelineId) -> Result> { + pub async fn load_timeline<'a>( + _guard: &tokio::sync::MutexGuard<'a, TimelineLoadLock>, + ttid: TenantTimelineId, + ) -> Result> { let (conf, wal_backup_launcher_tx) = TIMELINES_STATE.lock().unwrap().get_dependencies(); match Timeline::load_timeline(&conf, ttid, wal_backup_launcher_tx) { diff --git a/safekeeper/src/wal_backup.rs b/safekeeper/src/wal_backup.rs index c99bbc7d61..e4499eaf50 100644 --- a/safekeeper/src/wal_backup.rs +++ b/safekeeper/src/wal_backup.rs @@ -7,7 +7,7 @@ use tokio::task::JoinHandle; use utils::id::NodeId; use std::cmp::min; -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::pin::Pin; use std::sync::Arc; use std::time::Duration; @@ -531,3 +531,62 @@ pub async fn read_object( Ok(Box::pin(reader)) } + +/// Copy segments from one timeline to another. Used in copy_timeline. +pub async fn copy_s3_segments( + wal_seg_size: usize, + src_ttid: &TenantTimelineId, + dst_ttid: &TenantTimelineId, + from_segment: XLogSegNo, + to_segment: XLogSegNo, +) -> Result<()> { + const SEGMENTS_PROGRESS_REPORT_INTERVAL: u64 = 1024; + + let storage = REMOTE_STORAGE + .get() + .expect("failed to get remote storage") + .as_ref() + .unwrap(); + + let relative_dst_path = + Utf8Path::new(&dst_ttid.tenant_id.to_string()).join(dst_ttid.timeline_id.to_string()); + + let remote_path = RemotePath::new(&relative_dst_path)?; + + let files = storage.list_files(Some(&remote_path)).await?; + let uploaded_segments = &files + .iter() + .filter_map(|file| file.object_name().map(ToOwned::to_owned)) + .collect::>(); + + debug!( + "these segments have already been uploaded: {:?}", + uploaded_segments + ); + + let relative_src_path = + Utf8Path::new(&src_ttid.tenant_id.to_string()).join(src_ttid.timeline_id.to_string()); + + for segno in from_segment..to_segment { + if segno % SEGMENTS_PROGRESS_REPORT_INTERVAL == 0 { + info!("copied all segments from {} until {}", from_segment, segno); + } + + let segment_name = XLogFileName(PG_TLI, segno, wal_seg_size); + if uploaded_segments.contains(&segment_name) { + continue; + } + debug!("copying segment {}", segment_name); + + let from = RemotePath::new(&relative_src_path.join(&segment_name))?; + let to = RemotePath::new(&relative_dst_path.join(&segment_name))?; + + storage.copy_object(&from, &to).await?; + } + + info!( + "finished copying segments from {} until {}", + from_segment, to_segment + ); + Ok(()) +} diff --git a/safekeeper/src/wal_storage.rs b/safekeeper/src/wal_storage.rs index e7538f805c..8d138c701f 100644 --- a/safekeeper/src/wal_storage.rs +++ b/safekeeper/src/wal_storage.rs @@ -728,7 +728,7 @@ async fn write_zeroes(file: &mut File, mut count: usize) -> Result<()> { } /// Helper returning full path to WAL segment file and its .partial brother. -fn wal_file_paths( +pub fn wal_file_paths( timeline_dir: &Utf8Path, segno: XLogSegNo, wal_seg_size: usize, diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index 5b1a8ba27d..f33e17a76a 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -3032,6 +3032,28 @@ class SafekeeperHttpClient(requests.Session): assert isinstance(res_json, dict) return res_json + def copy_timeline(self, tenant_id: TenantId, timeline_id: TimelineId, body: Dict[str, Any]): + res = self.post( + f"http://localhost:{self.port}/v1/tenant/{tenant_id}/timeline/{timeline_id}/copy", + json=body, + ) + res.raise_for_status() + + def timeline_digest( + self, tenant_id: TenantId, timeline_id: TimelineId, from_lsn: Lsn, until_lsn: Lsn + ) -> Dict[str, Any]: + res = self.get( + f"http://localhost:{self.port}/v1/tenant/{tenant_id}/timeline/{timeline_id}/digest", + params={ + "from_lsn": str(from_lsn), + "until_lsn": str(until_lsn), + }, + ) + res.raise_for_status() + res_json = res.json() + assert isinstance(res_json, dict) + return res_json + def timeline_create( self, tenant_id: TenantId, diff --git a/test_runner/regress/test_wal_acceptor.py b/test_runner/regress/test_wal_acceptor.py index 4dfc883f4c..b4ce633531 100644 --- a/test_runner/regress/test_wal_acceptor.py +++ b/test_runner/regress/test_wal_acceptor.py @@ -1838,3 +1838,83 @@ def test_idle_reconnections(neon_env_builder: NeonEnvBuilder): assert final_stats.get("START_REPLICATION", 0) >= 1 # walproposer should connect to each safekeeper at least once assert final_stats.get("START_WAL_PUSH", 0) >= 3 + + +@pytest.mark.parametrize("insert_rows", [0, 100, 100000, 500000]) +def test_timeline_copy(neon_env_builder: NeonEnvBuilder, insert_rows: int): + target_percents = [10, 50, 90, 100] + + neon_env_builder.num_safekeepers = 3 + # we need remote storage that supports copy_object S3 API + neon_env_builder.enable_safekeeper_remote_storage(RemoteStorageKind.MOCK_S3) + env = neon_env_builder.init_start() + + tenant_id = env.initial_tenant + timeline_id = env.initial_timeline + + endpoint = env.endpoints.create_start("main") + + lsns = [] + + def remember_lsn(): + lsn = Lsn(endpoint.safe_psql("SELECT pg_current_wal_flush_lsn()")[0][0]) + lsns.append(lsn) + return lsn + + # remember LSN right after timeline creation + lsn = remember_lsn() + log.info(f"LSN after timeline creation: {lsn}") + + endpoint.safe_psql("create table t(key int, value text)") + + timeline_status = env.safekeepers[0].http_client().timeline_status(tenant_id, timeline_id) + timeline_start_lsn = timeline_status.timeline_start_lsn + log.info(f"Timeline start LSN: {timeline_start_lsn}") + + current_percent = 0.0 + for new_percent in target_percents: + new_rows = insert_rows * (new_percent - current_percent) / 100 + current_percent = new_percent + + if new_rows == 0: + continue + + endpoint.safe_psql( + f"insert into t select generate_series(1, {new_rows}), repeat('payload!', 10)" + ) + + # remember LSN right after reaching new_percent + lsn = remember_lsn() + log.info(f"LSN after inserting {new_rows} rows: {lsn}") + + # TODO: would be also good to test cases where not all segments are uploaded to S3 + + for lsn in lsns: + new_timeline_id = TimelineId.generate() + log.info(f"Copying branch for LSN {lsn}, to timeline {new_timeline_id}") + + orig_digest = ( + env.safekeepers[0] + .http_client() + .timeline_digest(tenant_id, timeline_id, timeline_start_lsn, lsn) + ) + log.info(f"Original digest: {orig_digest}") + + for sk in env.safekeepers: + sk.http_client().copy_timeline( + tenant_id, + timeline_id, + { + "target_timeline_id": str(new_timeline_id), + "until_lsn": str(lsn), + }, + ) + + new_digest = sk.http_client().timeline_digest( + tenant_id, new_timeline_id, timeline_start_lsn, lsn + ) + log.info(f"Digest after timeline copy on safekeeper {sk.id}: {new_digest}") + + assert orig_digest == new_digest + + # TODO: test timelines can start after copy From d260426a14606eceb5c7da447ccb089214bd0902 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Fri, 5 Jan 2024 11:48:06 +0100 Subject: [PATCH 31/35] is_rel_block_key: exclude the relsize key (#6266) Before this PR, `is_rel_block_key` returns true for the blknum `0xffffffff`, which is a blknum that's actually never written by Postgres, but used by Neon Pageserver to store the relsize. Quoting @MMeent: > PostgreSQL can't extend the relation beyond size of 0xFFFFFFFF blocks, > so block number 0xFFFFFFFE is the last valid block number. This PR changes the definition of the function to exclude blknum 0xffffffff. My motivation for doing this change is to fix the `pagebench` getpage benchmark, which uses `is_rel_block_key` to filter the keyspace for valid pages to request from page_service. fixes https://github.com/neondatabase/neon/issues/6210 I checked other users of the function. The first one is `key_is_shard0`, which already had added an exemption for 0xffffffff. So, there's no functional change with this PR. The second one is `DatadirModification::flush`[^1]. With this PR, `.flush()` will skip the relsize key, whereas it didn't before. This means we will pile up all the relsize key-value pairs `(Key,u32)` in `DatadirModification::pending_updates` until `.commit()` is called. The only place I can think of where that would be a problem is if we import from a full basebackup, and don't `.commit()` regularly, like we currently don't do in `import_basebackup_from_tar`. It exposes us to input-controlled allocations. However, that was already the case for the other keys that are skipped, so, one can argue that this change is not making the situation much worse. [^1]: That type's `flush()` and `commit()` methods are terribly named, but, that's for another time --- libs/pageserver_api/src/key.rs | 2 +- libs/pageserver_api/src/shard.rs | 7 +------ 2 files changed, 2 insertions(+), 7 deletions(-) diff --git a/libs/pageserver_api/src/key.rs b/libs/pageserver_api/src/key.rs index d680a5600e..3e1bba2a06 100644 --- a/libs/pageserver_api/src/key.rs +++ b/libs/pageserver_api/src/key.rs @@ -142,7 +142,7 @@ impl Key { } pub fn is_rel_block_key(key: &Key) -> bool { - key.field1 == 0x00 && key.field4 != 0 + key.field1 == 0x00 && key.field4 != 0 && key.field6 != 0xffffffff } impl std::str::FromStr for Key { diff --git a/libs/pageserver_api/src/shard.rs b/libs/pageserver_api/src/shard.rs index a186d93bce..18ef2be523 100644 --- a/libs/pageserver_api/src/shard.rs +++ b/libs/pageserver_api/src/shard.rs @@ -530,12 +530,7 @@ fn key_is_shard0(key: &Key) -> bool { // relation pages are distributed to shards other than shard zero. Everything else gets // stored on shard 0. This guarantees that shard 0 can independently serve basebackup // requests, and any request other than those for particular blocks in relations. - // - // In this condition: - // - is_rel_block_key includes only relations, i.e. excludes SLRU data and - // all metadata. - // - field6 is set to -1 for relation size pages. - !(is_rel_block_key(key) && key.field6 != 0xffffffff) + !is_rel_block_key(key) } /// Provide the same result as the function in postgres `hashfn.h` with the same name From 3c560d27a826ba54181208ec56bf5fbaab74844d Mon Sep 17 00:00:00 2001 From: John Spray Date: Fri, 5 Jan 2024 12:29:20 +0000 Subject: [PATCH 32/35] pageserver: implement secondary-mode downloads (#6123) Follows on from #6050 , in which we upload heatmaps. Secondary locations will now poll those heatmaps and download layers mentioned in the heatmap. TODO: - [X] ~Unify/reconcile stats for behind-schedule execution with warn_when_period_overrun (https://github.com/neondatabase/neon/pull/6050#discussion_r1426560695)~ - [x] Give downloads their own concurrency config independent of uploads Deferred optimizations: - https://github.com/neondatabase/neon/issues/6199 - https://github.com/neondatabase/neon/issues/6200 Eviction will be the next PR: - #5342 --- control_plane/src/pageserver.rs | 7 + control_plane/src/tenant_migration.rs | 21 +- libs/utils/src/lib.rs | 2 + libs/utils/src/sync/gate.rs | 6 + libs/utils/src/yielding_loop.rs | 35 + pageserver/client/src/mgmt_api.rs | 14 +- pageserver/src/config.rs | 30 +- pageserver/src/http/routes.rs | 35 + pageserver/src/lib.rs | 4 + pageserver/src/metrics.rs | 12 + pageserver/src/task_mgr.rs | 3 + pageserver/src/tenant/delete.rs | 2 +- pageserver/src/tenant/mgr.rs | 253 ++++-- .../src/tenant/remote_timeline_client.rs | 9 +- pageserver/src/tenant/secondary.rs | 142 +++- pageserver/src/tenant/secondary/downloader.rs | 801 ++++++++++++++++++ .../src/tenant/secondary/heatmap_uploader.rs | 502 ++++------- pageserver/src/tenant/secondary/scheduler.rs | 361 ++++++++ pageserver/src/tenant/tasks.rs | 2 + test_runner/fixtures/pageserver/http.py | 4 + .../regress/test_pageserver_secondary.py | 149 +++- 21 files changed, 1975 insertions(+), 419 deletions(-) create mode 100644 libs/utils/src/yielding_loop.rs create mode 100644 pageserver/src/tenant/secondary/downloader.rs create mode 100644 pageserver/src/tenant/secondary/scheduler.rs diff --git a/control_plane/src/pageserver.rs b/control_plane/src/pageserver.rs index 7d490016bf..fb0d251722 100644 --- a/control_plane/src/pageserver.rs +++ b/control_plane/src/pageserver.rs @@ -485,6 +485,13 @@ impl PageServerNode { Ok(self.http_client.list_timelines(*tenant_id).await?) } + pub async fn tenant_secondary_download(&self, tenant_id: &TenantShardId) -> anyhow::Result<()> { + Ok(self + .http_client + .tenant_secondary_download(*tenant_id) + .await?) + } + pub async fn timeline_create( &self, tenant_id: TenantId, diff --git a/control_plane/src/tenant_migration.rs b/control_plane/src/tenant_migration.rs index 79df108896..23ea8f4060 100644 --- a/control_plane/src/tenant_migration.rs +++ b/control_plane/src/tenant_migration.rs @@ -11,6 +11,7 @@ use crate::{ use pageserver_api::models::{ LocationConfig, LocationConfigMode, LocationConfigSecondary, TenantConfig, }; +use pageserver_api::shard::TenantShardId; use std::collections::HashMap; use std::time::Duration; use utils::{ @@ -40,9 +41,9 @@ async fn await_lsn( loop { let latest = match get_lsns(tenant_id, pageserver).await { Ok(l) => l, - Err(e) => { + Err(_e) => { println!( - "🕑 Can't get LSNs on pageserver {} yet, waiting ({e})", + "🕑 Waiting for pageserver {} to activate...", pageserver.conf.id ); std::thread::sleep(Duration::from_millis(500)); @@ -89,7 +90,7 @@ pub async fn migrate_tenant( tenant_id: TenantId, dest_ps: PageServerNode, ) -> anyhow::Result<()> { - // Get a new generation + println!("🤔 Checking existing status..."); let attachment_service = AttachmentService::from_env(env); fn build_location_config( @@ -135,6 +136,20 @@ pub async fn migrate_tenant( baseline_lsns = Some(get_lsns(tenant_id, &origin_ps).await?); } + println!( + "🔁 Downloading latest layers to destination pageserver {}", + dest_ps.conf.id + ); + match dest_ps + .tenant_secondary_download(&TenantShardId::unsharded(tenant_id)) + .await + { + Ok(()) => {} + Err(_) => { + println!(" (skipping, destination wasn't in secondary mode)") + } + } + let gen = attachment_service .attach_hook(tenant_id, dest_ps.conf.id) .await?; diff --git a/libs/utils/src/lib.rs b/libs/utils/src/lib.rs index 9e9b0adfe5..890061dc59 100644 --- a/libs/utils/src/lib.rs +++ b/libs/utils/src/lib.rs @@ -85,6 +85,8 @@ pub mod sync; pub mod failpoint_support; +pub mod yielding_loop; + /// This is a shortcut to embed git sha into binaries and avoid copying the same build script to all packages /// /// we have several cases: diff --git a/libs/utils/src/sync/gate.rs b/libs/utils/src/sync/gate.rs index 31c76d2f74..abc3842da8 100644 --- a/libs/utils/src/sync/gate.rs +++ b/libs/utils/src/sync/gate.rs @@ -15,6 +15,12 @@ pub struct Gate { name: String, } +impl std::fmt::Debug for Gate { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "Gate<{}>", self.name) + } +} + /// RAII guard for a [`Gate`]: as long as this exists, calls to [`Gate::close`] will /// not complete. #[derive(Debug)] diff --git a/libs/utils/src/yielding_loop.rs b/libs/utils/src/yielding_loop.rs new file mode 100644 index 0000000000..963279eb4c --- /dev/null +++ b/libs/utils/src/yielding_loop.rs @@ -0,0 +1,35 @@ +use tokio_util::sync::CancellationToken; + +#[derive(thiserror::Error, Debug)] +pub enum YieldingLoopError { + #[error("Cancelled")] + Cancelled, +} + +/// Helper for long synchronous loops, e.g. over all tenants in the system. Periodically +/// yields to avoid blocking the executor, and after resuming checks the provided +/// cancellation token to drop out promptly on shutdown. +#[inline(always)] +pub async fn yielding_loop( + interval: usize, + cancel: &CancellationToken, + iter: I, + mut visitor: F, +) -> Result<(), YieldingLoopError> +where + I: Iterator, + F: FnMut(T), +{ + for (i, item) in iter.enumerate() { + visitor(item); + + if i + 1 % interval == 0 { + tokio::task::yield_now().await; + if cancel.is_cancelled() { + return Err(YieldingLoopError::Cancelled); + } + } + } + + Ok(()) +} diff --git a/pageserver/client/src/mgmt_api.rs b/pageserver/client/src/mgmt_api.rs index 87e4ed8efd..4c285293f7 100644 --- a/pageserver/client/src/mgmt_api.rs +++ b/pageserver/client/src/mgmt_api.rs @@ -1,4 +1,4 @@ -use pageserver_api::models::*; +use pageserver_api::{models::*, shard::TenantShardId}; use reqwest::{IntoUrl, Method}; use utils::{ http::error::HttpErrorBody, @@ -164,6 +164,18 @@ impl Client { Ok(()) } + pub async fn tenant_secondary_download(&self, tenant_id: TenantShardId) -> Result<()> { + let uri = format!( + "{}/v1/tenant/{}/secondary/download", + self.mgmt_api_endpoint, tenant_id + ); + self.request(Method::POST, &uri, ()) + .await? + .error_for_status() + .map(|_| ()) + .map_err(|e| Error::ApiError(format!("{}", e))) + } + pub async fn location_config( &self, tenant_id: TenantId, diff --git a/pageserver/src/config.rs b/pageserver/src/config.rs index 4560f5eca0..7c03dc1bdd 100644 --- a/pageserver/src/config.rs +++ b/pageserver/src/config.rs @@ -37,8 +37,8 @@ use crate::tenant::{ TENANTS_SEGMENT_NAME, TENANT_DELETED_MARKER_FILE_NAME, TIMELINES_SEGMENT_NAME, }; use crate::{ - IGNORED_TENANT_FILE_NAME, METADATA_FILE_NAME, TENANT_CONFIG_NAME, TENANT_LOCATION_CONFIG_NAME, - TIMELINE_DELETE_MARK_SUFFIX, TIMELINE_UNINIT_MARK_SUFFIX, + IGNORED_TENANT_FILE_NAME, METADATA_FILE_NAME, TENANT_CONFIG_NAME, TENANT_HEATMAP_BASENAME, + TENANT_LOCATION_CONFIG_NAME, TIMELINE_DELETE_MARK_SUFFIX, TIMELINE_UNINIT_MARK_SUFFIX, }; use self::defaults::DEFAULT_CONCURRENT_TENANT_WARMUP; @@ -75,6 +75,7 @@ pub mod defaults { pub const DEFAULT_BACKGROUND_TASK_MAXIMUM_DELAY: &str = "10s"; pub const DEFAULT_HEATMAP_UPLOAD_CONCURRENCY: usize = 8; + pub const DEFAULT_SECONDARY_DOWNLOAD_CONCURRENCY: usize = 1; pub const DEFAULT_INGEST_BATCH_SIZE: u64 = 100; @@ -130,6 +131,7 @@ pub mod defaults { #gc_feedback = false #heatmap_upload_concurrency = {DEFAULT_HEATMAP_UPLOAD_CONCURRENCY} +#secondary_download_concurrency = {DEFAULT_SECONDARY_DOWNLOAD_CONCURRENCY} [remote_storage] @@ -239,6 +241,10 @@ pub struct PageServerConf { /// heatmap uploads vs. other remote storage operations. pub heatmap_upload_concurrency: usize, + /// How many remote storage downloads may be done for secondary tenants concurrently. Implicitly + /// deprioritises secondary downloads vs. remote storage operations for attached tenants. + pub secondary_download_concurrency: usize, + /// Maximum number of WAL records to be ingested and committed at the same time pub ingest_batch_size: u64, } @@ -322,6 +328,7 @@ struct PageServerConfigBuilder { control_plane_emergency_mode: BuilderValue, heatmap_upload_concurrency: BuilderValue, + secondary_download_concurrency: BuilderValue, ingest_batch_size: BuilderValue, } @@ -396,6 +403,7 @@ impl Default for PageServerConfigBuilder { control_plane_emergency_mode: Set(false), heatmap_upload_concurrency: Set(DEFAULT_HEATMAP_UPLOAD_CONCURRENCY), + secondary_download_concurrency: Set(DEFAULT_SECONDARY_DOWNLOAD_CONCURRENCY), ingest_batch_size: Set(DEFAULT_INGEST_BATCH_SIZE), } @@ -546,6 +554,10 @@ impl PageServerConfigBuilder { self.heatmap_upload_concurrency = BuilderValue::Set(value) } + pub fn secondary_download_concurrency(&mut self, value: usize) { + self.secondary_download_concurrency = BuilderValue::Set(value) + } + pub fn ingest_batch_size(&mut self, ingest_batch_size: u64) { self.ingest_batch_size = BuilderValue::Set(ingest_batch_size) } @@ -651,6 +663,9 @@ impl PageServerConfigBuilder { heatmap_upload_concurrency: self .heatmap_upload_concurrency .ok_or(anyhow!("missing heatmap_upload_concurrency"))?, + secondary_download_concurrency: self + .secondary_download_concurrency + .ok_or(anyhow!("missing secondary_download_concurrency"))?, ingest_batch_size: self .ingest_batch_size .ok_or(anyhow!("missing ingest_batch_size"))?, @@ -711,6 +726,11 @@ impl PageServerConf { .join(TENANT_LOCATION_CONFIG_NAME) } + pub(crate) fn tenant_heatmap_path(&self, tenant_shard_id: &TenantShardId) -> Utf8PathBuf { + self.tenant_path(tenant_shard_id) + .join(TENANT_HEATMAP_BASENAME) + } + pub fn timelines_path(&self, tenant_shard_id: &TenantShardId) -> Utf8PathBuf { self.tenant_path(tenant_shard_id) .join(TIMELINES_SEGMENT_NAME) @@ -896,6 +916,9 @@ impl PageServerConf { "heatmap_upload_concurrency" => { builder.heatmap_upload_concurrency(parse_toml_u64(key, item)? as usize) }, + "secondary_download_concurrency" => { + builder.secondary_download_concurrency(parse_toml_u64(key, item)? as usize) + }, "ingest_batch_size" => builder.ingest_batch_size(parse_toml_u64(key, item)?), _ => bail!("unrecognized pageserver option '{key}'"), } @@ -968,6 +991,7 @@ impl PageServerConf { control_plane_api_token: None, control_plane_emergency_mode: false, heatmap_upload_concurrency: defaults::DEFAULT_HEATMAP_UPLOAD_CONCURRENCY, + secondary_download_concurrency: defaults::DEFAULT_SECONDARY_DOWNLOAD_CONCURRENCY, ingest_batch_size: defaults::DEFAULT_INGEST_BATCH_SIZE, } } @@ -1198,6 +1222,7 @@ background_task_maximum_delay = '334 s' control_plane_api_token: None, control_plane_emergency_mode: false, heatmap_upload_concurrency: defaults::DEFAULT_HEATMAP_UPLOAD_CONCURRENCY, + secondary_download_concurrency: defaults::DEFAULT_SECONDARY_DOWNLOAD_CONCURRENCY, ingest_batch_size: defaults::DEFAULT_INGEST_BATCH_SIZE, }, "Correct defaults should be used when no config values are provided" @@ -1260,6 +1285,7 @@ background_task_maximum_delay = '334 s' control_plane_api_token: None, control_plane_emergency_mode: false, heatmap_upload_concurrency: defaults::DEFAULT_HEATMAP_UPLOAD_CONCURRENCY, + secondary_download_concurrency: defaults::DEFAULT_SECONDARY_DOWNLOAD_CONCURRENCY, ingest_batch_size: 100, }, "Should be able to parse all basic config values correctly" diff --git a/pageserver/src/http/routes.rs b/pageserver/src/http/routes.rs index 8265627cb5..5c7747d353 100644 --- a/pageserver/src/http/routes.rs +++ b/pageserver/src/http/routes.rs @@ -1274,6 +1274,23 @@ async fn put_tenant_location_config_handler( // which is not a 400 but a 409. .map_err(ApiError::BadRequest)?; + if let Some(_flush_ms) = flush { + match state + .secondary_controller + .upload_tenant(tenant_shard_id) + .await + { + Ok(()) => { + tracing::info!("Uploaded heatmap during flush"); + } + Err(e) => { + tracing::warn!("Failed to flush heatmap: {e}"); + } + } + } else { + tracing::info!("No flush requested when configuring"); + } + json_response(StatusCode::OK, ()) } @@ -1611,6 +1628,21 @@ async fn secondary_upload_handler( json_response(StatusCode::OK, ()) } +async fn secondary_download_handler( + request: Request, + _cancel: CancellationToken, +) -> Result, ApiError> { + let state = get_state(&request); + let tenant_shard_id: TenantShardId = parse_request_param(&request, "tenant_shard_id")?; + state + .secondary_controller + .download_tenant(tenant_shard_id) + .await + .map_err(ApiError::InternalServerError)?; + + json_response(StatusCode::OK, ()) +} + async fn handler_404(_: Request) -> Result, ApiError> { json_response( StatusCode::NOT_FOUND, @@ -1879,6 +1911,9 @@ pub fn make_router( .put("/v1/deletion_queue/flush", |r| { api_handler(r, deletion_queue_flush) }) + .post("/v1/tenant/:tenant_shard_id/secondary/download", |r| { + api_handler(r, secondary_download_handler) + }) .put("/v1/tenant/:tenant_shard_id/break", |r| { testing_api_handler("set tenant state to broken", r, handle_tenant_break) }) diff --git a/pageserver/src/lib.rs b/pageserver/src/lib.rs index c1ce0af47b..26070e0cc1 100644 --- a/pageserver/src/lib.rs +++ b/pageserver/src/lib.rs @@ -117,6 +117,10 @@ pub const TENANT_CONFIG_NAME: &str = "config"; /// Full path: `tenants//config`. pub const TENANT_LOCATION_CONFIG_NAME: &str = "config-v1"; +/// Per-tenant copy of their remote heatmap, downloaded into the local +/// tenant path while in secondary mode. +pub const TENANT_HEATMAP_BASENAME: &str = "heatmap-v1.json"; + /// A suffix used for various temporary files. Any temporary files found in the /// data directory at pageserver startup can be automatically removed. pub const TEMP_FILE_SUFFIX: &str = "___temp"; diff --git a/pageserver/src/metrics.rs b/pageserver/src/metrics.rs index 4725903783..c86adcfa3d 100644 --- a/pageserver/src/metrics.rs +++ b/pageserver/src/metrics.rs @@ -1369,6 +1369,8 @@ pub(crate) struct SecondaryModeMetrics { pub(crate) upload_heatmap: IntCounter, pub(crate) upload_heatmap_errors: IntCounter, pub(crate) upload_heatmap_duration: Histogram, + pub(crate) download_heatmap: IntCounter, + pub(crate) download_layer: IntCounter, } pub(crate) static SECONDARY_MODE: Lazy = Lazy::new(|| SecondaryModeMetrics { upload_heatmap: register_int_counter!( @@ -1386,6 +1388,16 @@ pub(crate) static SECONDARY_MODE: Lazy = Lazy::new(|| Seco "Time to build and upload a heatmap, including any waiting inside the S3 client" ) .expect("failed to define a metric"), + download_heatmap: register_int_counter!( + "pageserver_secondary_download_heatmap", + "Number of downloads of heatmaps by secondary mode locations" + ) + .expect("failed to define a metric"), + download_layer: register_int_counter!( + "pageserver_secondary_download_layer", + "Number of downloads of layers by secondary mode locations" + ) + .expect("failed to define a metric"), }); #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] diff --git a/pageserver/src/task_mgr.rs b/pageserver/src/task_mgr.rs index eabb1d0022..5a06a97525 100644 --- a/pageserver/src/task_mgr.rs +++ b/pageserver/src/task_mgr.rs @@ -258,6 +258,9 @@ pub enum TaskKind { /// See [`crate::disk_usage_eviction_task`]. DiskUsageEviction, + /// See [`crate::tenant::secondary`]. + SecondaryDownloads, + /// See [`crate::tenant::secondary`]. SecondaryUploads, diff --git a/pageserver/src/tenant/delete.rs b/pageserver/src/tenant/delete.rs index b21bad51ba..2f606ed822 100644 --- a/pageserver/src/tenant/delete.rs +++ b/pageserver/src/tenant/delete.rs @@ -588,7 +588,7 @@ impl DeleteTenantFlow { } break; } - TenantsMapRemoveResult::Occupied(TenantSlot::Secondary) => { + TenantsMapRemoveResult::Occupied(TenantSlot::Secondary(_)) => { // This is unexpected: this secondary tenants should not have been created, and we // are not in a position to shut it down from here. tracing::warn!("Tenant transitioned to secondary mode while deleting!"); diff --git a/pageserver/src/tenant/mgr.rs b/pageserver/src/tenant/mgr.rs index 250de7247d..70b41b7b1f 100644 --- a/pageserver/src/tenant/mgr.rs +++ b/pageserver/src/tenant/mgr.rs @@ -44,6 +44,7 @@ use utils::generation::Generation; use utils::id::{TenantId, TimelineId}; use super::delete::DeleteTenantError; +use super::secondary::SecondaryTenant; use super::TenantSharedResources; /// For a tenant that appears in TenantsMap, it may either be @@ -57,7 +58,7 @@ use super::TenantSharedResources; /// having a properly acquired generation (Secondary doesn't need a generation) pub(crate) enum TenantSlot { Attached(Arc), - Secondary, + Secondary(Arc), /// In this state, other administrative operations acting on the TenantId should /// block, or return a retry indicator equivalent to HTTP 503. InProgress(utils::completion::Barrier), @@ -67,7 +68,7 @@ impl std::fmt::Debug for TenantSlot { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { match self { Self::Attached(tenant) => write!(f, "Attached({})", tenant.current_state()), - Self::Secondary => write!(f, "Secondary"), + Self::Secondary(_) => write!(f, "Secondary"), Self::InProgress(_) => write!(f, "InProgress"), } } @@ -78,7 +79,7 @@ impl TenantSlot { fn get_attached(&self) -> Option<&Arc> { match self { Self::Attached(t) => Some(t), - Self::Secondary => None, + Self::Secondary(_) => None, Self::InProgress(_) => None, } } @@ -466,12 +467,18 @@ pub async fn init_tenant_mgr( *gen } else { match &location_conf.mode { - LocationMode::Secondary(_) => { + LocationMode::Secondary(secondary_config) => { // We do not require the control plane's permission for secondary mode // tenants, because they do no remote writes and hence require no // generation number info!(tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(), "Loaded tenant in secondary mode"); - tenants.insert(tenant_shard_id, TenantSlot::Secondary); + tenants.insert( + tenant_shard_id, + TenantSlot::Secondary(SecondaryTenant::new( + tenant_shard_id, + secondary_config, + )), + ); } LocationMode::Attached(_) => { // TODO: augment re-attach API to enable the control plane to @@ -663,8 +670,14 @@ async fn shutdown_all_tenants0(tenants: &std::sync::RwLock) { total_attached += 1; } - TenantSlot::Secondary => { - shutdown_state.insert(tenant_shard_id, TenantSlot::Secondary); + TenantSlot::Secondary(state) => { + // We don't need to wait for this individually per-tenant: the + // downloader task will be waited on eventually, this cancel + // is just to encourage it to drop out if it is doing work + // for this tenant right now. + state.cancel.cancel(); + + shutdown_state.insert(tenant_shard_id, TenantSlot::Secondary(state)); } TenantSlot::InProgress(notify) => { // InProgress tenants are not visible in TenantsMap::ShuttingDown: we will @@ -847,12 +860,28 @@ impl TenantManager { Some(TenantSlot::InProgress(_)) => { Err(GetTenantError::NotActive(tenant_shard_id.tenant_id)) } - None | Some(TenantSlot::Secondary) => { + None | Some(TenantSlot::Secondary(_)) => { Err(GetTenantError::NotFound(tenant_shard_id.tenant_id)) } } } + pub(crate) fn get_secondary_tenant_shard( + &self, + tenant_shard_id: TenantShardId, + ) -> Option> { + let locked = self.tenants.read().unwrap(); + + let peek_slot = tenant_map_peek_slot(&locked, &tenant_shard_id, TenantSlotPeekMode::Read) + .ok() + .flatten(); + + match peek_slot { + Some(TenantSlot::Secondary(s)) => Some(s.clone()), + _ => None, + } + } + #[instrument(skip_all, fields(tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug()))] pub(crate) async fn upsert_location( &self, @@ -864,10 +893,15 @@ impl TenantManager { debug_assert_current_span_has_tenant_id(); info!("configuring tenant location to state {new_location_config:?}"); - // Special case fast-path for updates to Tenant: if our upsert is only updating configuration, + enum FastPathModified { + Attached(Arc), + Secondary(Arc), + } + + // Special case fast-path for updates to existing slots: if our upsert is only updating configuration, // then we do not need to set the slot to InProgress, we can just call into the // existng tenant. - let modify_tenant = { + let fast_path_taken = { let locked = self.tenants.read().unwrap(); let peek_slot = tenant_map_peek_slot(&locked, &tenant_shard_id, TenantSlotPeekMode::Write)?; @@ -881,12 +915,19 @@ impl TenantManager { new_location_config.clone(), )?); - Some(tenant.clone()) + Some(FastPathModified::Attached(tenant.clone())) } else { // Different generations, fall through to general case None } } + ( + LocationMode::Secondary(secondary_conf), + Some(TenantSlot::Secondary(secondary_tenant)), + ) => { + secondary_tenant.set_config(secondary_conf); + Some(FastPathModified::Secondary(secondary_tenant.clone())) + } _ => { // Not an Attached->Attached transition, fall through to general case None @@ -895,34 +936,51 @@ impl TenantManager { }; // Fast-path continued: having dropped out of the self.tenants lock, do the async - // phase of waiting for flush, before returning. - if let Some(tenant) = modify_tenant { - // Transition to AttachedStale means we may well hold a valid generation - // still, and have been requested to go stale as part of a migration. If - // the caller set `flush`, then flush to remote storage. - if let LocationMode::Attached(AttachedLocationConfig { - generation: _, - attach_mode: AttachmentMode::Stale, - }) = &new_location_config.mode - { - if let Some(flush_timeout) = flush { - match tokio::time::timeout(flush_timeout, tenant.flush_remote()).await { - Ok(Err(e)) => { - return Err(e); - } - Ok(Ok(_)) => return Ok(()), - Err(_) => { - tracing::warn!( + // phase of writing config and/or waiting for flush, before returning. + match fast_path_taken { + Some(FastPathModified::Attached(tenant)) => { + Tenant::persist_tenant_config(self.conf, &tenant_shard_id, &new_location_config) + .await + .map_err(SetNewTenantConfigError::Persist)?; + + // Transition to AttachedStale means we may well hold a valid generation + // still, and have been requested to go stale as part of a migration. If + // the caller set `flush`, then flush to remote storage. + if let LocationMode::Attached(AttachedLocationConfig { + generation: _, + attach_mode: AttachmentMode::Stale, + }) = &new_location_config.mode + { + if let Some(flush_timeout) = flush { + match tokio::time::timeout(flush_timeout, tenant.flush_remote()).await { + Ok(Err(e)) => { + return Err(e); + } + Ok(Ok(_)) => return Ok(()), + Err(_) => { + tracing::warn!( timeout_ms = flush_timeout.as_millis(), "Timed out waiting for flush to remote storage, proceeding anyway." ) + } } } } - } - return Ok(()); - } + return Ok(()); + } + Some(FastPathModified::Secondary(_secondary_tenant)) => { + Tenant::persist_tenant_config(self.conf, &tenant_shard_id, &new_location_config) + .await + .map_err(SetNewTenantConfigError::Persist)?; + + return Ok(()); + } + None => { + // Proceed with the general case procedure, where we will shutdown & remove any existing + // slot contents and replace with a fresh one + } + }; // General case for upserts to TenantsMap, excluding the case above: we will substitute an // InProgress value to the slot while we make whatever changes are required. The state for @@ -931,33 +989,47 @@ impl TenantManager { // not do significant I/O, and shutdowns should be prompt via cancellation tokens. let mut slot_guard = tenant_map_acquire_slot(&tenant_shard_id, TenantSlotAcquireMode::Any)?; - if let Some(TenantSlot::Attached(tenant)) = slot_guard.get_old_value() { - // The case where we keep a Tenant alive was covered above in the special case - // for Attached->Attached transitions in the same generation. By this point, - // if we see an attached tenant we know it will be discarded and should be - // shut down. - let (_guard, progress) = utils::completion::channel(); + match slot_guard.get_old_value() { + Some(TenantSlot::Attached(tenant)) => { + // The case where we keep a Tenant alive was covered above in the special case + // for Attached->Attached transitions in the same generation. By this point, + // if we see an attached tenant we know it will be discarded and should be + // shut down. + let (_guard, progress) = utils::completion::channel(); - match tenant.get_attach_mode() { - AttachmentMode::Single | AttachmentMode::Multi => { - // Before we leave our state as the presumed holder of the latest generation, - // flush any outstanding deletions to reduce the risk of leaking objects. - self.resources.deletion_queue_client.flush_advisory() - } - AttachmentMode::Stale => { - // If we're stale there's not point trying to flush deletions - } - }; + match tenant.get_attach_mode() { + AttachmentMode::Single | AttachmentMode::Multi => { + // Before we leave our state as the presumed holder of the latest generation, + // flush any outstanding deletions to reduce the risk of leaking objects. + self.resources.deletion_queue_client.flush_advisory() + } + AttachmentMode::Stale => { + // If we're stale there's not point trying to flush deletions + } + }; - info!("Shutting down attached tenant"); - match tenant.shutdown(progress, false).await { - Ok(()) => {} - Err(barrier) => { - info!("Shutdown already in progress, waiting for it to complete"); - barrier.wait().await; + info!("Shutting down attached tenant"); + match tenant.shutdown(progress, false).await { + Ok(()) => {} + Err(barrier) => { + info!("Shutdown already in progress, waiting for it to complete"); + barrier.wait().await; + } } + slot_guard.drop_old_value().expect("We just shut it down"); + } + Some(TenantSlot::Secondary(state)) => { + info!("Shutting down secondary tenant"); + state.shutdown().await; + } + Some(TenantSlot::InProgress(_)) => { + // This should never happen: acquire_slot should error out + // if the contents of a slot were InProgress. + anyhow::bail!("Acquired an InProgress slot, this is a bug.") + } + None => { + // Slot was vacant, nothing needs shutting down. } - slot_guard.drop_old_value().expect("We just shut it down"); } let tenant_path = self.conf.tenant_path(&tenant_shard_id); @@ -980,7 +1052,9 @@ impl TenantManager { .map_err(SetNewTenantConfigError::Persist)?; let new_slot = match &new_location_config.mode { - LocationMode::Secondary(_) => TenantSlot::Secondary, + LocationMode::Secondary(secondary_config) => { + TenantSlot::Secondary(SecondaryTenant::new(tenant_shard_id, secondary_config)) + } LocationMode::Attached(_attach_config) => { let shard_identity = new_location_config.shard; let tenant = tenant_spawn( @@ -1093,6 +1167,30 @@ impl TenantManager { .collect(), } } + // Do some synchronous work for all tenant slots in Secondary state. The provided + // callback should be small and fast, as it will be called inside the global + // TenantsMap lock. + pub(crate) fn foreach_secondary_tenants(&self, mut func: F) + where + // TODO: let the callback return a hint to drop out of the loop early + F: FnMut(&TenantShardId, &Arc), + { + let locked = self.tenants.read().unwrap(); + + let map = match &*locked { + TenantsMap::Initializing | TenantsMap::ShuttingDown(_) => return, + TenantsMap::Open(m) => m, + }; + + for (tenant_id, slot) in map { + if let TenantSlot::Secondary(state) = slot { + // Only expose secondary tenants that are not currently shutting down + if !state.cancel.is_cancelled() { + func(tenant_id, state) + } + } + } + } pub(crate) async fn delete_tenant( &self, @@ -1207,7 +1305,7 @@ pub(crate) fn get_tenant( Some(TenantSlot::InProgress(_)) => { Err(GetTenantError::NotActive(tenant_shard_id.tenant_id)) } - None | Some(TenantSlot::Secondary) => { + None | Some(TenantSlot::Secondary(_)) => { Err(GetTenantError::NotFound(tenant_shard_id.tenant_id)) } } @@ -1280,7 +1378,7 @@ pub(crate) async fn get_active_tenant_with_timeout( } } } - Some(TenantSlot::Secondary) => { + Some(TenantSlot::Secondary(_)) => { return Err(GetActiveTenantError::NotFound(GetTenantError::NotActive( tenant_id, ))) @@ -1544,7 +1642,7 @@ pub(crate) async fn list_tenants() -> Result, Ok(m.iter() .filter_map(|(id, tenant)| match tenant { TenantSlot::Attached(tenant) => Some((*id, tenant.current_state())), - TenantSlot::Secondary => None, + TenantSlot::Secondary(_) => None, TenantSlot::InProgress(_) => None, }) .collect()) @@ -1801,11 +1899,7 @@ impl SlotGuard { fn old_value_is_shutdown(&self) -> bool { match self.old_value.as_ref() { Some(TenantSlot::Attached(tenant)) => tenant.gate.close_complete(), - Some(TenantSlot::Secondary) => { - // TODO: when adding secondary mode tenants, this will check for shutdown - // in the same way that we do for `Tenant` above - true - } + Some(TenantSlot::Secondary(secondary_tenant)) => secondary_tenant.gate.close_complete(), Some(TenantSlot::InProgress(_)) => { // A SlotGuard cannot be constructed for a slot that was already InProgress unreachable!() @@ -2015,26 +2109,19 @@ where let mut slot_guard = tenant_map_acquire_slot_impl(&tenant_shard_id, tenants, TenantSlotAcquireMode::MustExist)?; - // The SlotGuard allows us to manipulate the Tenant object without fear of some - // concurrent API request doing something else for the same tenant ID. - let attached_tenant = match slot_guard.get_old_value() { - Some(TenantSlot::Attached(t)) => Some(t), - _ => None, - }; - // allow pageserver shutdown to await for our completion let (_guard, progress) = completion::channel(); - // If the tenant was attached, shut it down gracefully. For secondary - // locations this part is not necessary - match &attached_tenant { - Some(attached_tenant) => { + // The SlotGuard allows us to manipulate the Tenant object without fear of some + // concurrent API request doing something else for the same tenant ID. + let attached_tenant = match slot_guard.get_old_value() { + Some(TenantSlot::Attached(tenant)) => { // whenever we remove a tenant from memory, we don't want to flush and wait for upload let freeze_and_flush = false; // shutdown is sure to transition tenant to stopping, and wait for all tasks to complete, so // that we can continue safely to cleanup. - match attached_tenant.shutdown(progress, freeze_and_flush).await { + match tenant.shutdown(progress, freeze_and_flush).await { Ok(()) => {} Err(_other) => { // if pageserver shutdown or other detach/ignore is already ongoing, we don't want to @@ -2043,11 +2130,19 @@ where return Err(TenantStateError::IsStopping(tenant_shard_id.tenant_id)); } } + Some(tenant) } - None => { - // Nothing to wait on when not attached, proceed. + Some(TenantSlot::Secondary(secondary_state)) => { + tracing::info!("Shutting down in secondary mode"); + secondary_state.shutdown().await; + None } - } + Some(TenantSlot::InProgress(_)) => { + // Acquiring a slot guarantees its old value was not InProgress + unreachable!(); + } + None => None, + }; match tenant_cleanup .await diff --git a/pageserver/src/tenant/remote_timeline_client.rs b/pageserver/src/tenant/remote_timeline_client.rs index 60b40d70a7..2ea3ced008 100644 --- a/pageserver/src/tenant/remote_timeline_client.rs +++ b/pageserver/src/tenant/remote_timeline_client.rs @@ -229,6 +229,7 @@ use crate::{ tenant::upload_queue::{ UploadOp, UploadQueue, UploadQueueInitialized, UploadQueueStopped, UploadTask, }, + TENANT_HEATMAP_BASENAME, }; use utils::id::{TenantId, TimelineId}; @@ -1741,11 +1742,11 @@ pub fn remote_index_path( .expect("Failed to construct path") } -pub const HEATMAP_BASENAME: &str = "heatmap-v1.json"; - pub(crate) fn remote_heatmap_path(tenant_shard_id: &TenantShardId) -> RemotePath { - RemotePath::from_string(&format!("tenants/{tenant_shard_id}/{HEATMAP_BASENAME}")) - .expect("Failed to construct path") + RemotePath::from_string(&format!( + "tenants/{tenant_shard_id}/{TENANT_HEATMAP_BASENAME}" + )) + .expect("Failed to construct path") } /// Given the key of an index, parse out the generation part of the name diff --git a/pageserver/src/tenant/secondary.rs b/pageserver/src/tenant/secondary.rs index d25fe56b92..2331447266 100644 --- a/pageserver/src/tenant/secondary.rs +++ b/pageserver/src/tenant/secondary.rs @@ -1,24 +1,48 @@ +mod downloader; pub mod heatmap; mod heatmap_uploader; +mod scheduler; use std::sync::Arc; use crate::task_mgr::{self, TaskKind, BACKGROUND_RUNTIME}; -use self::heatmap_uploader::heatmap_uploader_task; +use self::{ + downloader::{downloader_task, SecondaryDetail}, + heatmap_uploader::heatmap_uploader_task, +}; -use super::mgr::TenantManager; +use super::{config::SecondaryLocationConfig, mgr::TenantManager}; use pageserver_api::shard::TenantShardId; use remote_storage::GenericRemoteStorage; use tokio_util::sync::CancellationToken; -use utils::completion::Barrier; +use utils::{completion::Barrier, sync::gate::Gate}; +enum DownloadCommand { + Download(TenantShardId), +} enum UploadCommand { Upload(TenantShardId), } +impl UploadCommand { + fn get_tenant_shard_id(&self) -> &TenantShardId { + match self { + Self::Upload(id) => id, + } + } +} + +impl DownloadCommand { + fn get_tenant_shard_id(&self) -> &TenantShardId { + match self { + Self::Download(id) => id, + } + } +} + struct CommandRequest { payload: T, response_tx: tokio::sync::oneshot::Sender, @@ -28,12 +52,73 @@ struct CommandResponse { result: anyhow::Result<()>, } +// Whereas [`Tenant`] represents an attached tenant, this type represents the work +// we do for secondary tenant locations: where we are not serving clients or +// ingesting WAL, but we are maintaining a warm cache of layer files. +// +// This type is all about the _download_ path for secondary mode. The upload path +// runs separately (see [`heatmap_uploader`]) while a regular attached `Tenant` exists. +// +// This structure coordinates TenantManager and SecondaryDownloader, +// so that the downloader can indicate which tenants it is currently +// operating on, and the manager can indicate when a particular +// secondary tenant should cancel any work in flight. +#[derive(Debug)] +pub(crate) struct SecondaryTenant { + /// Carrying a tenant shard ID simplifies callers such as the downloader + /// which need to organize many of these objects by ID. + tenant_shard_id: TenantShardId, + + /// Cancellation token indicates to SecondaryDownloader that it should stop doing + /// any work for this tenant at the next opportunity. + pub(crate) cancel: CancellationToken, + + pub(crate) gate: Gate, + + detail: std::sync::Mutex, +} + +impl SecondaryTenant { + pub(crate) fn new( + tenant_shard_id: TenantShardId, + config: &SecondaryLocationConfig, + ) -> Arc { + Arc::new(Self { + tenant_shard_id, + // todo: shall we make this a descendent of the + // main cancellation token, or is it sufficient that + // on shutdown we walk the tenants and fire their + // individual cancellations? + cancel: CancellationToken::new(), + gate: Gate::new(format!("SecondaryTenant {tenant_shard_id}")), + + detail: std::sync::Mutex::new(SecondaryDetail::new(config.clone())), + }) + } + + pub(crate) async fn shutdown(&self) { + self.cancel.cancel(); + + // Wait for any secondary downloader work to complete + self.gate.close().await; + } + + pub(crate) fn set_config(&self, config: &SecondaryLocationConfig) { + self.detail.lock().unwrap().config = config.clone(); + } + + fn get_tenant_shard_id(&self) -> &TenantShardId { + &self.tenant_shard_id + } +} + /// The SecondaryController is a pseudo-rpc client for administrative control of secondary mode downloads, /// and heatmap uploads. This is not a hot data path: it's primarily a hook for tests, /// where we want to immediately upload/download for a particular tenant. In normal operation /// uploads & downloads are autonomous and not driven by this interface. pub struct SecondaryController { upload_req_tx: tokio::sync::mpsc::Sender>, + download_req_tx: tokio::sync::mpsc::Sender>, } impl SecondaryController { @@ -63,6 +148,13 @@ impl SecondaryController { self.dispatch(&self.upload_req_tx, UploadCommand::Upload(tenant_shard_id)) .await } + pub async fn download_tenant(&self, tenant_shard_id: TenantShardId) -> anyhow::Result<()> { + self.dispatch( + &self.download_req_tx, + DownloadCommand::Download(tenant_shard_id), + ) + .await + } } pub fn spawn_tasks( @@ -71,9 +163,37 @@ pub fn spawn_tasks( background_jobs_can_start: Barrier, cancel: CancellationToken, ) -> SecondaryController { + let mgr_clone = tenant_manager.clone(); + let storage_clone = remote_storage.clone(); + let cancel_clone = cancel.clone(); + let bg_jobs_clone = background_jobs_can_start.clone(); + + let (download_req_tx, download_req_rx) = + tokio::sync::mpsc::channel::>(16); let (upload_req_tx, upload_req_rx) = tokio::sync::mpsc::channel::>(16); + task_mgr::spawn( + BACKGROUND_RUNTIME.handle(), + TaskKind::SecondaryDownloads, + None, + None, + "secondary tenant downloads", + false, + async move { + downloader_task( + mgr_clone, + storage_clone, + download_req_rx, + bg_jobs_clone, + cancel_clone, + ) + .await; + + Ok(()) + }, + ); + task_mgr::spawn( BACKGROUND_RUNTIME.handle(), TaskKind::SecondaryUploads, @@ -89,16 +209,26 @@ pub fn spawn_tasks( background_jobs_can_start, cancel, ) - .await + .await; + + Ok(()) }, ); - SecondaryController { upload_req_tx } + SecondaryController { + download_req_tx, + upload_req_tx, + } } /// For running with remote storage disabled: a SecondaryController that is connected to nothing. pub fn null_controller() -> SecondaryController { + let (download_req_tx, _download_req_rx) = + tokio::sync::mpsc::channel::>(16); let (upload_req_tx, _upload_req_rx) = tokio::sync::mpsc::channel::>(16); - SecondaryController { upload_req_tx } + SecondaryController { + upload_req_tx, + download_req_tx, + } } diff --git a/pageserver/src/tenant/secondary/downloader.rs b/pageserver/src/tenant/secondary/downloader.rs new file mode 100644 index 0000000000..6fdee08a4e --- /dev/null +++ b/pageserver/src/tenant/secondary/downloader.rs @@ -0,0 +1,801 @@ +use std::{ + collections::{HashMap, HashSet}, + pin::Pin, + str::FromStr, + sync::Arc, + time::{Duration, Instant, SystemTime}, +}; + +use crate::{ + config::PageServerConf, + metrics::SECONDARY_MODE, + tenant::{ + config::SecondaryLocationConfig, + debug_assert_current_span_has_tenant_and_timeline_id, + remote_timeline_client::{ + index::LayerFileMetadata, FAILED_DOWNLOAD_WARN_THRESHOLD, FAILED_REMOTE_OP_RETRIES, + }, + span::debug_assert_current_span_has_tenant_id, + storage_layer::LayerFileName, + tasks::{warn_when_period_overrun, BackgroundLoopKind}, + }, + virtual_file::{on_fatal_io_error, MaybeFatalIo, VirtualFile}, + METADATA_FILE_NAME, TEMP_FILE_SUFFIX, +}; + +use super::{ + heatmap::HeatMapLayer, + scheduler::{self, Completion, JobGenerator, SchedulingResult, TenantBackgroundJobs}, + SecondaryTenant, +}; + +use crate::tenant::{ + mgr::TenantManager, + remote_timeline_client::{download::download_layer_file, remote_heatmap_path}, +}; + +use chrono::format::{DelayedFormat, StrftimeItems}; +use futures::Future; +use pageserver_api::shard::TenantShardId; +use rand::Rng; +use remote_storage::{DownloadError, GenericRemoteStorage}; + +use tokio_util::sync::CancellationToken; +use tracing::{info_span, instrument, Instrument}; +use utils::{ + backoff, completion::Barrier, crashsafe::path_with_suffix_extension, fs_ext, id::TimelineId, +}; + +use super::{ + heatmap::{HeatMapTenant, HeatMapTimeline}, + CommandRequest, DownloadCommand, +}; + +/// For each tenant, how long must have passed since the last download_tenant call before +/// calling it again. This is approximately the time by which local data is allowed +/// to fall behind remote data. +/// +/// TODO: this should just be a default, and the actual period should be controlled +/// via the heatmap itself +/// `` +const DOWNLOAD_FRESHEN_INTERVAL: Duration = Duration::from_millis(60000); + +pub(super) async fn downloader_task( + tenant_manager: Arc, + remote_storage: GenericRemoteStorage, + command_queue: tokio::sync::mpsc::Receiver>, + background_jobs_can_start: Barrier, + cancel: CancellationToken, +) { + let concurrency = tenant_manager.get_conf().secondary_download_concurrency; + + let generator = SecondaryDownloader { + tenant_manager, + remote_storage, + }; + let mut scheduler = Scheduler::new(generator, concurrency); + + scheduler + .run(command_queue, background_jobs_can_start, cancel) + .instrument(info_span!("secondary_downloads")) + .await +} + +struct SecondaryDownloader { + tenant_manager: Arc, + remote_storage: GenericRemoteStorage, +} + +#[derive(Debug, Clone)] +pub(super) struct OnDiskState { + metadata: LayerFileMetadata, + access_time: SystemTime, +} + +impl OnDiskState { + fn new( + _conf: &'static PageServerConf, + _tenant_shard_id: &TenantShardId, + _imeline_id: &TimelineId, + _ame: LayerFileName, + metadata: LayerFileMetadata, + access_time: SystemTime, + ) -> Self { + Self { + metadata, + access_time, + } + } +} + +#[derive(Debug, Clone, Default)] +pub(super) struct SecondaryDetailTimeline { + pub(super) on_disk_layers: HashMap, + + /// We remember when layers were evicted, to prevent re-downloading them. + pub(super) evicted_at: HashMap, +} + +/// This state is written by the secondary downloader, it is opaque +/// to TenantManager +#[derive(Debug)] +pub(super) struct SecondaryDetail { + pub(super) config: SecondaryLocationConfig, + + last_download: Option, + next_download: Option, + pub(super) timelines: HashMap, +} + +/// Helper for logging SystemTime +fn strftime(t: &'_ SystemTime) -> DelayedFormat> { + let datetime: chrono::DateTime = (*t).into(); + datetime.format("%d/%m/%Y %T") +} + +impl SecondaryDetail { + pub(super) fn new(config: SecondaryLocationConfig) -> Self { + Self { + config, + last_download: None, + next_download: None, + timelines: HashMap::new(), + } + } +} + +struct PendingDownload { + secondary_state: Arc, + last_download: Option, + target_time: Option, + period: Option, +} + +impl scheduler::PendingJob for PendingDownload { + fn get_tenant_shard_id(&self) -> &TenantShardId { + self.secondary_state.get_tenant_shard_id() + } +} + +struct RunningDownload { + barrier: Barrier, +} + +impl scheduler::RunningJob for RunningDownload { + fn get_barrier(&self) -> Barrier { + self.barrier.clone() + } +} + +struct CompleteDownload { + secondary_state: Arc, + completed_at: Instant, +} + +impl scheduler::Completion for CompleteDownload { + fn get_tenant_shard_id(&self) -> &TenantShardId { + self.secondary_state.get_tenant_shard_id() + } +} + +type Scheduler = TenantBackgroundJobs< + SecondaryDownloader, + PendingDownload, + RunningDownload, + CompleteDownload, + DownloadCommand, +>; + +#[async_trait::async_trait] +impl JobGenerator + for SecondaryDownloader +{ + #[instrument(skip_all, fields(tenant_id=%completion.get_tenant_shard_id().tenant_id, shard_id=%completion.get_tenant_shard_id().shard_slug()))] + fn on_completion(&mut self, completion: CompleteDownload) { + let CompleteDownload { + secondary_state, + completed_at: _completed_at, + } = completion; + + tracing::debug!("Secondary tenant download completed"); + + // Update freshened_at even if there was an error: we don't want errored tenants to implicitly + // take priority to run again. + let mut detail = secondary_state.detail.lock().unwrap(); + detail.next_download = Some(Instant::now() + DOWNLOAD_FRESHEN_INTERVAL); + } + + async fn schedule(&mut self) -> SchedulingResult { + let mut result = SchedulingResult { + jobs: Vec::new(), + want_interval: None, + }; + + // Step 1: identify some tenants that we may work on + let mut tenants: Vec> = Vec::new(); + self.tenant_manager + .foreach_secondary_tenants(|_id, secondary_state| { + tenants.push(secondary_state.clone()); + }); + + // Step 2: filter out tenants which are not yet elegible to run + let now = Instant::now(); + result.jobs = tenants + .into_iter() + .filter_map(|secondary_tenant| { + let (last_download, next_download) = { + let mut detail = secondary_tenant.detail.lock().unwrap(); + + if !detail.config.warm { + // Downloads are disabled for this tenant + detail.next_download = None; + return None; + } + + if detail.next_download.is_none() { + // Initialize with a jitter: this spreads initial downloads on startup + // or mass-attach across our freshen interval. + let jittered_period = + rand::thread_rng().gen_range(Duration::ZERO..DOWNLOAD_FRESHEN_INTERVAL); + detail.next_download = Some(now.checked_add(jittered_period).expect( + "Using our constant, which is known to be small compared with clock range", + )); + } + (detail.last_download, detail.next_download.unwrap()) + }; + + if now < next_download { + Some(PendingDownload { + secondary_state: secondary_tenant, + last_download, + target_time: Some(next_download), + period: Some(DOWNLOAD_FRESHEN_INTERVAL), + }) + } else { + None + } + }) + .collect(); + + // Step 3: sort by target execution time to run most urgent first. + result.jobs.sort_by_key(|j| j.target_time); + + result + } + + fn on_command(&mut self, command: DownloadCommand) -> anyhow::Result { + let tenant_shard_id = command.get_tenant_shard_id(); + + let tenant = self + .tenant_manager + .get_secondary_tenant_shard(*tenant_shard_id); + let Some(tenant) = tenant else { + { + return Err(anyhow::anyhow!("Not found or not in Secondary mode")); + } + }; + + Ok(PendingDownload { + target_time: None, + period: None, + last_download: None, + secondary_state: tenant, + }) + } + + fn spawn( + &mut self, + job: PendingDownload, + ) -> ( + RunningDownload, + Pin + Send>>, + ) { + let PendingDownload { + secondary_state, + last_download, + target_time, + period, + } = job; + + let (completion, barrier) = utils::completion::channel(); + let remote_storage = self.remote_storage.clone(); + let conf = self.tenant_manager.get_conf(); + let tenant_shard_id = *secondary_state.get_tenant_shard_id(); + (RunningDownload { barrier }, Box::pin(async move { + let _completion = completion; + + match TenantDownloader::new(conf, &remote_storage, &secondary_state) + .download() + .await + { + Err(UpdateError::NoData) => { + tracing::info!("No heatmap found for tenant. This is fine if it is new."); + }, + Err(UpdateError::NoSpace) => { + tracing::warn!("Insufficient space while downloading. Will retry later."); + } + Err(UpdateError::Cancelled) => { + tracing::debug!("Shut down while downloading"); + }, + Err(UpdateError::Deserialize(e)) => { + tracing::error!("Corrupt content while downloading tenant: {e}"); + }, + Err(e @ (UpdateError::DownloadError(_) | UpdateError::Other(_))) => { + tracing::error!("Error while downloading tenant: {e}"); + }, + Ok(()) => {} + }; + + // Irrespective of the result, we will reschedule ourselves to run after our usual period. + + // If the job had a target execution time, we may check our final execution + // time against that for observability purposes. + if let (Some(target_time), Some(period)) = (target_time, period) { + // Only track execution lag if this isn't our first download: otherwise, it is expected + // that execution will have taken longer than our configured interval, for example + // when starting up a pageserver and + if last_download.is_some() { + // Elapsed time includes any scheduling lag as well as the execution of the job + let elapsed = Instant::now().duration_since(target_time); + + warn_when_period_overrun( + elapsed, + period, + BackgroundLoopKind::SecondaryDownload, + ); + } + } + + CompleteDownload { + secondary_state, + completed_at: Instant::now(), + } + }.instrument(info_span!(parent: None, "secondary_download", tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug())))) + } +} + +/// This type is a convenience to group together the various functions involved in +/// freshening a secondary tenant. +struct TenantDownloader<'a> { + conf: &'static PageServerConf, + remote_storage: &'a GenericRemoteStorage, + secondary_state: &'a SecondaryTenant, +} + +/// Errors that may be encountered while updating a tenant +#[derive(thiserror::Error, Debug)] +enum UpdateError { + #[error("No remote data found")] + NoData, + #[error("Insufficient local storage space")] + NoSpace, + #[error("Failed to download")] + DownloadError(DownloadError), + #[error(transparent)] + Deserialize(#[from] serde_json::Error), + #[error("Cancelled")] + Cancelled, + #[error(transparent)] + Other(#[from] anyhow::Error), +} + +impl From for UpdateError { + fn from(value: DownloadError) -> Self { + match &value { + DownloadError::Cancelled => Self::Cancelled, + DownloadError::NotFound => Self::NoData, + _ => Self::DownloadError(value), + } + } +} + +impl From for UpdateError { + fn from(value: std::io::Error) -> Self { + if let Some(nix::errno::Errno::ENOSPC) = value.raw_os_error().map(nix::errno::from_i32) { + UpdateError::NoSpace + } else { + // An I/O error from e.g. tokio::io::copy is most likely a remote storage issue + UpdateError::Other(anyhow::anyhow!(value)) + } + } +} + +impl<'a> TenantDownloader<'a> { + fn new( + conf: &'static PageServerConf, + remote_storage: &'a GenericRemoteStorage, + secondary_state: &'a SecondaryTenant, + ) -> Self { + Self { + conf, + remote_storage, + secondary_state, + } + } + + async fn download(&self) -> Result<(), UpdateError> { + debug_assert_current_span_has_tenant_id(); + + // For the duration of a download, we must hold the SecondaryTenant::gate, to ensure + // cover our access to local storage. + let Ok(_guard) = self.secondary_state.gate.enter() else { + // Shutting down + return Ok(()); + }; + + let tenant_shard_id = self.secondary_state.get_tenant_shard_id(); + // Download the tenant's heatmap + let heatmap_bytes = tokio::select!( + bytes = self.download_heatmap() => {bytes?}, + _ = self.secondary_state.cancel.cancelled() => return Ok(()) + ); + + let heatmap = serde_json::from_slice::(&heatmap_bytes)?; + + // Save the heatmap: this will be useful on restart, allowing us to reconstruct + // layer metadata without having to re-download it. + let heatmap_path = self.conf.tenant_heatmap_path(tenant_shard_id); + + let temp_path = path_with_suffix_extension(&heatmap_path, TEMP_FILE_SUFFIX); + let context_msg = format!("write tenant {tenant_shard_id} heatmap to {heatmap_path}"); + let heatmap_path_bg = heatmap_path.clone(); + tokio::task::spawn_blocking(move || { + tokio::runtime::Handle::current().block_on(async move { + VirtualFile::crashsafe_overwrite(&heatmap_path_bg, &temp_path, &heatmap_bytes).await + }) + }) + .await + .expect("Blocking task is never aborted") + .maybe_fatal_err(&context_msg)?; + + tracing::debug!("Wrote local heatmap to {}", heatmap_path); + + // Download the layers in the heatmap + for timeline in heatmap.timelines { + if self.secondary_state.cancel.is_cancelled() { + return Ok(()); + } + + let timeline_id = timeline.timeline_id; + self.download_timeline(timeline) + .instrument(tracing::info_span!( + "secondary_download_timeline", + tenant_id=%tenant_shard_id.tenant_id, + shard_id=%tenant_shard_id.shard_slug(), + %timeline_id + )) + .await?; + } + + Ok(()) + } + + async fn download_heatmap(&self) -> Result, UpdateError> { + debug_assert_current_span_has_tenant_id(); + let tenant_shard_id = self.secondary_state.get_tenant_shard_id(); + // TODO: make download conditional on ETag having changed since last download + // (https://github.com/neondatabase/neon/issues/6199) + tracing::debug!("Downloading heatmap for secondary tenant",); + + let heatmap_path = remote_heatmap_path(tenant_shard_id); + + let heatmap_bytes = backoff::retry( + || async { + let download = self + .remote_storage + .download(&heatmap_path) + .await + .map_err(UpdateError::from)?; + let mut heatmap_bytes = Vec::new(); + let mut body = tokio_util::io::StreamReader::new(download.download_stream); + let _size = tokio::io::copy(&mut body, &mut heatmap_bytes).await?; + Ok(heatmap_bytes) + }, + |e| matches!(e, UpdateError::NoData | UpdateError::Cancelled), + FAILED_DOWNLOAD_WARN_THRESHOLD, + FAILED_REMOTE_OP_RETRIES, + "download heatmap", + backoff::Cancel::new(self.secondary_state.cancel.clone(), || { + UpdateError::Cancelled + }), + ) + .await?; + + SECONDARY_MODE.download_heatmap.inc(); + + Ok(heatmap_bytes) + } + + async fn download_timeline(&self, timeline: HeatMapTimeline) -> Result<(), UpdateError> { + debug_assert_current_span_has_tenant_and_timeline_id(); + let tenant_shard_id = self.secondary_state.get_tenant_shard_id(); + let timeline_path = self + .conf + .timeline_path(tenant_shard_id, &timeline.timeline_id); + + // Accumulate updates to the state + let mut touched = Vec::new(); + + // Clone a view of what layers already exist on disk + let timeline_state = self + .secondary_state + .detail + .lock() + .unwrap() + .timelines + .get(&timeline.timeline_id) + .cloned(); + + let timeline_state = match timeline_state { + Some(t) => t, + None => { + // We have no existing state: need to scan local disk for layers first. + let timeline_state = + init_timeline_state(self.conf, tenant_shard_id, &timeline).await; + + // Re-acquire detail lock now that we're done with async load from local FS + self.secondary_state + .detail + .lock() + .unwrap() + .timelines + .insert(timeline.timeline_id, timeline_state.clone()); + timeline_state + } + }; + + let layers_in_heatmap = timeline + .layers + .iter() + .map(|l| &l.name) + .collect::>(); + let layers_on_disk = timeline_state + .on_disk_layers + .iter() + .map(|l| l.0) + .collect::>(); + + // Remove on-disk layers that are no longer present in heatmap + for layer in layers_on_disk.difference(&layers_in_heatmap) { + let local_path = timeline_path.join(layer.to_string()); + tracing::info!("Removing secondary local layer {layer} because it's absent in heatmap",); + tokio::fs::remove_file(&local_path) + .await + .or_else(fs_ext::ignore_not_found) + .maybe_fatal_err("Removing secondary layer")?; + } + + // Download heatmap layers that are not present on local disk, or update their + // access time if they are already present. + for layer in timeline.layers { + if self.secondary_state.cancel.is_cancelled() { + return Ok(()); + } + + // Existing on-disk layers: just update their access time. + if let Some(on_disk) = timeline_state.on_disk_layers.get(&layer.name) { + tracing::debug!("Layer {} is already on disk", layer.name); + if on_disk.metadata != LayerFileMetadata::from(&layer.metadata) + || on_disk.access_time != layer.access_time + { + // We already have this layer on disk. Update its access time. + tracing::debug!( + "Access time updated for layer {}: {} -> {}", + layer.name, + strftime(&on_disk.access_time), + strftime(&layer.access_time) + ); + touched.push(layer); + } + continue; + } else { + tracing::debug!("Layer {} not present on disk yet", layer.name); + } + + // Eviction: if we evicted a layer, then do not re-download it unless it was accessed more + // recently than it was evicted. + if let Some(evicted_at) = timeline_state.evicted_at.get(&layer.name) { + if &layer.access_time > evicted_at { + tracing::info!( + "Re-downloading evicted layer {}, accessed at {}, evicted at {}", + layer.name, + strftime(&layer.access_time), + strftime(evicted_at) + ); + } else { + tracing::trace!( + "Not re-downloading evicted layer {}, accessed at {}, evicted at {}", + layer.name, + strftime(&layer.access_time), + strftime(evicted_at) + ); + continue; + } + } + + // Note: no backoff::retry wrapper here because download_layer_file does its own retries internally + let downloaded_bytes = match download_layer_file( + self.conf, + self.remote_storage, + *tenant_shard_id, + timeline.timeline_id, + &layer.name, + &LayerFileMetadata::from(&layer.metadata), + &self.secondary_state.cancel, + ) + .await + { + Ok(bytes) => bytes, + Err(e) => { + if let DownloadError::NotFound = e { + // A heatmap might be out of date and refer to a layer that doesn't exist any more. + // This is harmless: continue to download the next layer. It is expected during compaction + // GC. + tracing::debug!( + "Skipped downloading missing layer {}, raced with compaction/gc?", + layer.name + ); + continue; + } else { + return Err(e.into()); + } + } + }; + + if downloaded_bytes != layer.metadata.file_size { + let local_path = timeline_path.join(layer.name.to_string()); + + tracing::warn!( + "Downloaded layer {} with unexpected size {} != {}. Removing download.", + layer.name, + downloaded_bytes, + layer.metadata.file_size + ); + + tokio::fs::remove_file(&local_path) + .await + .or_else(fs_ext::ignore_not_found)?; + } + + SECONDARY_MODE.download_layer.inc(); + touched.push(layer) + } + + // Write updates to state to record layers we just downloaded or touched. + { + let mut detail = self.secondary_state.detail.lock().unwrap(); + let timeline_detail = detail.timelines.entry(timeline.timeline_id).or_default(); + + tracing::info!("Wrote timeline_detail for {} touched layers", touched.len()); + + for t in touched { + use std::collections::hash_map::Entry; + match timeline_detail.on_disk_layers.entry(t.name.clone()) { + Entry::Occupied(mut v) => { + v.get_mut().access_time = t.access_time; + } + Entry::Vacant(e) => { + e.insert(OnDiskState::new( + self.conf, + tenant_shard_id, + &timeline.timeline_id, + t.name, + LayerFileMetadata::from(&t.metadata), + t.access_time, + )); + } + } + } + } + + Ok(()) + } +} + +/// Scan local storage and build up Layer objects based on the metadata in a HeatMapTimeline +async fn init_timeline_state( + conf: &'static PageServerConf, + tenant_shard_id: &TenantShardId, + heatmap: &HeatMapTimeline, +) -> SecondaryDetailTimeline { + let timeline_path = conf.timeline_path(tenant_shard_id, &heatmap.timeline_id); + let mut detail = SecondaryDetailTimeline::default(); + + let mut dir = match tokio::fs::read_dir(&timeline_path).await { + Ok(d) => d, + Err(e) => { + if e.kind() == std::io::ErrorKind::NotFound { + let context = format!("Creating timeline directory {timeline_path}"); + tracing::info!("{}", context); + tokio::fs::create_dir_all(&timeline_path) + .await + .fatal_err(&context); + + // No entries to report: drop out. + return detail; + } else { + on_fatal_io_error(&e, &format!("Reading timeline dir {timeline_path}")); + } + } + }; + + // As we iterate through layers found on disk, we will look up their metadata from this map. + // Layers not present in metadata will be discarded. + let heatmap_metadata: HashMap<&LayerFileName, &HeatMapLayer> = + heatmap.layers.iter().map(|l| (&l.name, l)).collect(); + + while let Some(dentry) = dir + .next_entry() + .await + .fatal_err(&format!("Listing {timeline_path}")) + { + let dentry_file_name = dentry.file_name(); + let file_name = dentry_file_name.to_string_lossy(); + let local_meta = dentry.metadata().await.fatal_err(&format!( + "Read metadata on {}", + dentry.path().to_string_lossy() + )); + + // Secondary mode doesn't use local metadata files, but they might have been left behind by an attached tenant. + if file_name == METADATA_FILE_NAME { + continue; + } + + match LayerFileName::from_str(&file_name) { + Ok(name) => { + let remote_meta = heatmap_metadata.get(&name); + match remote_meta { + Some(remote_meta) => { + // TODO: checksums for layers (https://github.com/neondatabase/neon/issues/2784) + if local_meta.len() != remote_meta.metadata.file_size { + // This should not happen, because we do crashsafe write-then-rename when downloading + // layers, and layers in remote storage are immutable. Remove the local file because + // we cannot trust it. + tracing::warn!( + "Removing local layer {name} with unexpected local size {} != {}", + local_meta.len(), + remote_meta.metadata.file_size + ); + } else { + // We expect the access time to be initialized immediately afterwards, when + // the latest heatmap is applied to the state. + detail.on_disk_layers.insert( + name.clone(), + OnDiskState::new( + conf, + tenant_shard_id, + &heatmap.timeline_id, + name, + LayerFileMetadata::from(&remote_meta.metadata), + remote_meta.access_time, + ), + ); + } + } + None => { + // FIXME: consider some optimization when transitioning from attached to secondary: maybe + // wait until we have seen a heatmap that is more recent than the most recent on-disk state? Otherwise + // we will end up deleting any layers which were created+uploaded more recently than the heatmap. + tracing::info!( + "Removing secondary local layer {} because it's absent in heatmap", + name + ); + tokio::fs::remove_file(&dentry.path()) + .await + .or_else(fs_ext::ignore_not_found) + .fatal_err(&format!( + "Removing layer {}", + dentry.path().to_string_lossy() + )); + } + } + } + Err(_) => { + // Ignore it. + tracing::warn!("Unexpected file in timeline directory: {file_name}"); + } + } + } + + detail +} diff --git a/pageserver/src/tenant/secondary/heatmap_uploader.rs b/pageserver/src/tenant/secondary/heatmap_uploader.rs index ece2b93ce1..ef01c33e8e 100644 --- a/pageserver/src/tenant/secondary/heatmap_uploader.rs +++ b/pageserver/src/tenant/secondary/heatmap_uploader.rs @@ -1,5 +1,6 @@ use std::{ collections::HashMap, + pin::Pin, sync::{Arc, Weak}, time::{Duration, Instant}, }; @@ -7,35 +8,86 @@ use std::{ use crate::{ metrics::SECONDARY_MODE, tenant::{ - config::AttachmentMode, mgr::TenantManager, remote_timeline_client::remote_heatmap_path, - secondary::CommandResponse, span::debug_assert_current_span_has_tenant_id, Tenant, + config::AttachmentMode, + mgr::TenantManager, + remote_timeline_client::remote_heatmap_path, + span::debug_assert_current_span_has_tenant_id, + tasks::{warn_when_period_overrun, BackgroundLoopKind}, + Tenant, }, }; +use futures::Future; use md5; use pageserver_api::shard::TenantShardId; +use rand::Rng; use remote_storage::GenericRemoteStorage; -use tokio::task::JoinSet; +use super::{ + scheduler::{self, JobGenerator, RunningJob, SchedulingResult, TenantBackgroundJobs}, + CommandRequest, +}; use tokio_util::sync::CancellationToken; -use tracing::instrument; -use utils::{backoff, completion::Barrier}; +use tracing::{info_span, instrument, Instrument}; +use utils::{backoff, completion::Barrier, yielding_loop::yielding_loop}; -use super::{heatmap::HeatMapTenant, CommandRequest, UploadCommand}; +use super::{heatmap::HeatMapTenant, UploadCommand}; -/// Period between heatmap uploader walking Tenants to look for work to do. -/// If any tenants have a heatmap upload period lower than this, it will be adjusted -/// downward to match. -const DEFAULT_SCHEDULING_INTERVAL: Duration = Duration::from_millis(60000); -const MIN_SCHEDULING_INTERVAL: Duration = Duration::from_millis(1000); +pub(super) async fn heatmap_uploader_task( + tenant_manager: Arc, + remote_storage: GenericRemoteStorage, + command_queue: tokio::sync::mpsc::Receiver>, + background_jobs_can_start: Barrier, + cancel: CancellationToken, +) { + let concurrency = tenant_manager.get_conf().heatmap_upload_concurrency; + + let generator = HeatmapUploader { + tenant_manager, + remote_storage, + cancel: cancel.clone(), + tenants: HashMap::new(), + }; + let mut scheduler = Scheduler::new(generator, concurrency); + + scheduler + .run(command_queue, background_jobs_can_start, cancel) + .instrument(info_span!("heatmap_uploader")) + .await +} + +/// This type is owned by a single task ([`heatmap_uploader_task`]) which runs an event +/// handling loop and mutates it as needed: there are no locks here, because that event loop +/// can hold &mut references to this type throughout. +struct HeatmapUploader { + tenant_manager: Arc, + remote_storage: GenericRemoteStorage, + cancel: CancellationToken, + + tenants: HashMap, +} struct WriteInProgress { barrier: Barrier, } +impl RunningJob for WriteInProgress { + fn get_barrier(&self) -> Barrier { + self.barrier.clone() + } +} + struct UploadPending { tenant: Arc, last_digest: Option, + target_time: Option, + period: Option, +} + +impl scheduler::PendingJob for UploadPending { + fn get_tenant_shard_id(&self) -> &TenantShardId { + self.tenant.get_tenant_shard_id() + } } struct WriteComplete { @@ -45,6 +97,12 @@ struct WriteComplete { next_upload: Option, } +impl scheduler::Completion for WriteComplete { + fn get_tenant_shard_id(&self) -> &TenantShardId { + &self.tenant_shard_id + } +} + /// The heatmap uploader keeps a little bit of per-tenant state, mainly to remember /// when we last did a write. We only populate this after doing at least one /// write for a tenant -- this avoids holding state for tenants that have @@ -68,267 +126,111 @@ struct UploaderTenantState { next_upload: Option, } -/// This type is owned by a single task ([`heatmap_uploader_task`]) which runs an event -/// handling loop and mutates it as needed: there are no locks here, because that event loop -/// can hold &mut references to this type throughout. -struct HeatmapUploader { - tenant_manager: Arc, - remote_storage: GenericRemoteStorage, - cancel: CancellationToken, +type Scheduler = TenantBackgroundJobs< + HeatmapUploader, + UploadPending, + WriteInProgress, + WriteComplete, + UploadCommand, +>; - tenants: HashMap, - - /// Tenants with work to do, for which tasks should be spawned as soon as concurrency - /// limits permit it. - tenants_pending: std::collections::VecDeque, - - /// Tenants for which a task in `tasks` has been spawned. - tenants_uploading: HashMap, - - tasks: JoinSet<()>, - - /// Channel for our child tasks to send results to: we use a channel for results rather than - /// just getting task results via JoinSet because we need the channel's recv() "sleep until something - /// is available" semantic, rather than JoinSet::join_next()'s "sleep until next thing is available _or_ I'm empty" - /// behavior. - task_result_tx: tokio::sync::mpsc::UnboundedSender, - task_result_rx: tokio::sync::mpsc::UnboundedReceiver, - - concurrent_uploads: usize, - - scheduling_interval: Duration, -} - -/// The uploader task runs a loop that periodically wakes up and schedules tasks for -/// tenants that require an upload, or handles any commands that have been sent into -/// `command_queue`. No I/O is done in this loop: that all happens in the tasks we -/// spawn. -/// -/// Scheduling iterations are somewhat infrequent. However, each one will enqueue -/// all tenants that require an upload, and in between scheduling iterations we will -/// continue to spawn new tasks for pending tenants, as our concurrency limit permits. -/// -/// While we take a CancellationToken here, it is subordinate to the CancellationTokens -/// of tenants: i.e. we expect all Tenants to have been shut down before we are shut down, otherwise -/// we might block waiting on a Tenant. -pub(super) async fn heatmap_uploader_task( - tenant_manager: Arc, - remote_storage: GenericRemoteStorage, - mut command_queue: tokio::sync::mpsc::Receiver>, - background_jobs_can_start: Barrier, - cancel: CancellationToken, -) -> anyhow::Result<()> { - let concurrent_uploads = tenant_manager.get_conf().heatmap_upload_concurrency; - - let (result_tx, result_rx) = tokio::sync::mpsc::unbounded_channel(); - - let mut uploader = HeatmapUploader { - tenant_manager, - remote_storage, - cancel: cancel.clone(), - tasks: JoinSet::new(), - tenants: HashMap::new(), - tenants_pending: std::collections::VecDeque::new(), - tenants_uploading: HashMap::new(), - task_result_tx: result_tx, - task_result_rx: result_rx, - concurrent_uploads, - scheduling_interval: DEFAULT_SCHEDULING_INTERVAL, - }; - - tracing::info!("Waiting for background_jobs_can start..."); - background_jobs_can_start.wait().await; - tracing::info!("background_jobs_can is ready, proceeding."); - - while !cancel.is_cancelled() { - // Look for new work: this is relatively expensive because we have to go acquire the lock on - // the tenant manager to retrieve tenants, and then iterate over them to figure out which ones - // require an upload. - uploader.schedule_iteration().await?; - - // Between scheduling iterations, we will: - // - Drain any complete tasks and spawn pending tasks - // - Handle incoming administrative commands - // - Check our cancellation token - let next_scheduling_iteration = Instant::now() - .checked_add(uploader.scheduling_interval) - .unwrap_or_else(|| { - tracing::warn!( - "Scheduling interval invalid ({}s), running immediately!", - uploader.scheduling_interval.as_secs_f64() - ); - Instant::now() - }); - loop { - tokio::select! { - _ = cancel.cancelled() => { - // We do not simply drop the JoinSet, in order to have an orderly shutdown without cancellation. - tracing::info!("Heatmap uploader joining tasks"); - while let Some(_r) = uploader.tasks.join_next().await {}; - tracing::info!("Heatmap uploader terminating"); - - break; - }, - _ = tokio::time::sleep(next_scheduling_iteration.duration_since(Instant::now())) => { - tracing::debug!("heatmap_uploader_task: woke for scheduling interval"); - break;}, - cmd = command_queue.recv() => { - tracing::debug!("heatmap_uploader_task: woke for command queue"); - let cmd = match cmd { - Some(c) =>c, - None => { - // SecondaryController was destroyed, and this has raced with - // our CancellationToken - tracing::info!("Heatmap uploader terminating"); - cancel.cancel(); - break; - } - }; - - let CommandRequest{ - response_tx, - payload - } = cmd; - uploader.handle_command(payload, response_tx); - }, - _ = uploader.process_next_completion() => { - if !cancel.is_cancelled() { - uploader.spawn_pending(); - } - } - } - } - } - - Ok(()) -} - -impl HeatmapUploader { - /// Periodic execution phase: inspect all attached tenants and schedule any work they require. - async fn schedule_iteration(&mut self) -> anyhow::Result<()> { +#[async_trait::async_trait] +impl JobGenerator + for HeatmapUploader +{ + async fn schedule(&mut self) -> SchedulingResult { // Cull any entries in self.tenants whose Arc is gone self.tenants .retain(|_k, v| v.tenant.upgrade().is_some() && v.next_upload.is_some()); - // The priority order of previously scheduled work may be invalidated by current state: drop - // all pending work (it will be re-scheduled if still needed) - self.tenants_pending.clear(); - - // Used a fixed 'now' through the following loop, for efficiency and fairness. let now = Instant::now(); - // While iterating over the potentially-long list of tenants, we will periodically yield - // to avoid blocking executor. - const YIELD_ITERATIONS: usize = 1000; + let mut result = SchedulingResult { + jobs: Vec::new(), + want_interval: None, + }; - // Iterate over tenants looking for work to do. let tenants = self.tenant_manager.get_attached_active_tenant_shards(); - for (i, tenant) in tenants.into_iter().enumerate() { - // Process is shutting down, drop out - if self.cancel.is_cancelled() { - return Ok(()); - } - // Skip tenants that already have a write in flight - if self - .tenants_uploading - .contains_key(tenant.get_tenant_shard_id()) - { - continue; - } + yielding_loop(1000, &self.cancel, tenants.into_iter(), |tenant| { + let period = match tenant.get_heatmap_period() { + None => { + // Heatmaps are disabled for this tenant + return; + } + Some(period) => { + // If any tenant has asked for uploads more frequent than our scheduling interval, + // reduce it to match so that we can keep up. This is mainly useful in testing, where + // we may set rather short intervals. + result.want_interval = match result.want_interval { + None => Some(period), + Some(existing) => Some(std::cmp::min(period, existing)), + }; - self.maybe_schedule_upload(&now, tenant); + period + } + }; - if i + 1 % YIELD_ITERATIONS == 0 { - tokio::task::yield_now().await; - } - } - - // Spawn tasks for as many of our pending tenants as we can. - self.spawn_pending(); - - Ok(()) - } - - /// - /// Cancellation: this method is cancel-safe. - async fn process_next_completion(&mut self) { - match self.task_result_rx.recv().await { - Some(r) => { - self.on_completion(r); - } - None => { - unreachable!("Result sender is stored on Self"); - } - } - } - - /// The 'maybe' refers to the tenant's state: whether it is configured - /// for heatmap uploads at all, and whether sufficient time has passed - /// since the last upload. - fn maybe_schedule_upload(&mut self, now: &Instant, tenant: Arc) { - match tenant.get_heatmap_period() { - None => { - // Heatmaps are disabled for this tenant + // Stale attachments do not upload anything: if we are in this state, there is probably some + // other attachment in mode Single or Multi running on another pageserver, and we don't + // want to thrash and overwrite their heatmap uploads. + if tenant.get_attach_mode() == AttachmentMode::Stale { return; } - Some(period) => { - // If any tenant has asked for uploads more frequent than our scheduling interval, - // reduce it to match so that we can keep up. This is mainly useful in testing, where - // we may set rather short intervals. - if period < self.scheduling_interval { - self.scheduling_interval = std::cmp::max(period, MIN_SCHEDULING_INTERVAL); - } + + // Create an entry in self.tenants if one doesn't already exist: this will later be updated + // with the completion time in on_completion. + let state = self + .tenants + .entry(*tenant.get_tenant_shard_id()) + .or_insert_with(|| { + let jittered_period = rand::thread_rng().gen_range(Duration::ZERO..period); + + UploaderTenantState { + tenant: Arc::downgrade(&tenant), + last_upload: None, + next_upload: Some(now.checked_add(jittered_period).unwrap_or(now)), + last_digest: None, + } + }); + + // Decline to do the upload if insufficient time has passed + if state.next_upload.map(|nu| nu > now).unwrap_or(false) { + return; } - } - // Stale attachments do not upload anything: if we are in this state, there is probably some - // other attachment in mode Single or Multi running on another pageserver, and we don't - // want to thrash and overwrite their heatmap uploads. - if tenant.get_attach_mode() == AttachmentMode::Stale { - return; - } - - // Create an entry in self.tenants if one doesn't already exist: this will later be updated - // with the completion time in on_completion. - let state = self - .tenants - .entry(*tenant.get_tenant_shard_id()) - .or_insert_with(|| UploaderTenantState { - tenant: Arc::downgrade(&tenant), - last_upload: None, - next_upload: Some(Instant::now()), - last_digest: None, + let last_digest = state.last_digest; + result.jobs.push(UploadPending { + tenant, + last_digest, + target_time: state.next_upload, + period: Some(period), }); + }) + .await + .ok(); - // Decline to do the upload if insufficient time has passed - if state.next_upload.map(|nu| &nu > now).unwrap_or(false) { - return; - } + result + } - let last_digest = state.last_digest; - self.tenants_pending.push_back(UploadPending { + fn spawn( + &mut self, + job: UploadPending, + ) -> ( + WriteInProgress, + Pin + Send>>, + ) { + let UploadPending { tenant, last_digest, - }) - } + target_time, + period, + } = job; - fn spawn_pending(&mut self) { - while !self.tenants_pending.is_empty() - && self.tenants_uploading.len() < self.concurrent_uploads - { - // unwrap: loop condition includes !is_empty() - let pending = self.tenants_pending.pop_front().unwrap(); - self.spawn_upload(pending.tenant, pending.last_digest); - } - } - - fn spawn_upload(&mut self, tenant: Arc, last_digest: Option) { let remote_storage = self.remote_storage.clone(); - let tenant_shard_id = *tenant.get_tenant_shard_id(); let (completion, barrier) = utils::completion::channel(); - let result_tx = self.task_result_tx.clone(); - self.tasks.spawn(async move { + let tenant_shard_id = *tenant.get_tenant_shard_id(); + (WriteInProgress { barrier }, Box::pin(async move { // Guard for the barrier in [`WriteInProgress`] let _completion = completion; @@ -362,22 +264,47 @@ impl HeatmapUploader { }; let now = Instant::now(); + + // If the job had a target execution time, we may check our final execution + // time against that for observability purposes. + if let (Some(target_time), Some(period)) = (target_time, period) { + // Elapsed time includes any scheduling lag as well as the execution of the job + let elapsed = now.duration_since(target_time); + + warn_when_period_overrun(elapsed, period, BackgroundLoopKind::HeatmapUpload); + } + let next_upload = tenant .get_heatmap_period() .and_then(|period| now.checked_add(period)); - result_tx - .send(WriteComplete { + WriteComplete { tenant_shard_id: *tenant.get_tenant_shard_id(), completed_at: now, digest, next_upload, - }) - .ok(); - }); + } + }.instrument(info_span!(parent: None, "heatmap_upload", tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug())))) + } - self.tenants_uploading - .insert(tenant_shard_id, WriteInProgress { barrier }); + fn on_command(&mut self, command: UploadCommand) -> anyhow::Result { + let tenant_shard_id = command.get_tenant_shard_id(); + + tracing::info!( + tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(), + "Starting heatmap write on command"); + let tenant = self + .tenant_manager + .get_attached_tenant_shard(*tenant_shard_id, true) + .map_err(|e| anyhow::anyhow!(e))?; + + Ok(UploadPending { + // Ignore our state for last digest: this forces an upload even if nothing has changed + last_digest: None, + tenant, + target_time: None, + period: None, + }) } #[instrument(skip_all, fields(tenant_id=%completion.tenant_shard_id.tenant_id, shard_id=%completion.tenant_shard_id.shard_slug()))] @@ -389,7 +316,6 @@ impl HeatmapUploader { digest, next_upload, } = completion; - self.tenants_uploading.remove(&tenant_shard_id); use std::collections::hash_map::Entry; match self.tenants.entry(tenant_shard_id) { Entry::Vacant(_) => { @@ -402,69 +328,6 @@ impl HeatmapUploader { } } } - - fn handle_command( - &mut self, - command: UploadCommand, - response_tx: tokio::sync::oneshot::Sender, - ) { - match command { - UploadCommand::Upload(tenant_shard_id) => { - // If an upload was ongoing for this tenant, let it finish first. - let barrier = if let Some(writing_state) = - self.tenants_uploading.get(&tenant_shard_id) - { - tracing::info!( - tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(), - "Waiting for heatmap write to complete"); - writing_state.barrier.clone() - } else { - // Spawn the upload then immediately wait for it. This will block processing of other commands and - // starting of other background work. - tracing::info!( - tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(), - "Starting heatmap write on command"); - let tenant = match self - .tenant_manager - .get_attached_tenant_shard(tenant_shard_id, true) - { - Ok(t) => t, - Err(e) => { - // Drop result of send: we don't care if caller dropped their receiver - drop(response_tx.send(CommandResponse { - result: Err(e.into()), - })); - return; - } - }; - self.spawn_upload(tenant, None); - let writing_state = self - .tenants_uploading - .get(&tenant_shard_id) - .expect("We just inserted this"); - tracing::info!( - tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(), - "Waiting for heatmap upload to complete"); - - writing_state.barrier.clone() - }; - - // This task does no I/O: it only listens for a barrier's completion and then - // sends to the command response channel. It is therefore safe to spawn this without - // any gates/task_mgr hooks. - tokio::task::spawn(async move { - barrier.wait().await; - - tracing::info!( - tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(), - "Heatmap upload complete"); - - // Drop result of send: we don't care if caller dropped their receiver - drop(response_tx.send(CommandResponse { result: Ok(()) })) - }); - } - } - } } enum UploadHeatmapOutcome { @@ -487,7 +350,6 @@ enum UploadHeatmapError { /// The inner upload operation. This will skip if `last_digest` is Some and matches the digest /// of the object we would have uploaded. -#[instrument(skip_all, fields(tenant_id = %tenant.get_tenant_shard_id().tenant_id, shard_id = %tenant.get_tenant_shard_id().shard_slug()))] async fn upload_tenant_heatmap( remote_storage: GenericRemoteStorage, tenant: &Arc, diff --git a/pageserver/src/tenant/secondary/scheduler.rs b/pageserver/src/tenant/secondary/scheduler.rs new file mode 100644 index 0000000000..cf01a100d9 --- /dev/null +++ b/pageserver/src/tenant/secondary/scheduler.rs @@ -0,0 +1,361 @@ +use async_trait; +use futures::Future; +use std::{ + collections::HashMap, + marker::PhantomData, + pin::Pin, + time::{Duration, Instant}, +}; + +use pageserver_api::shard::TenantShardId; +use tokio::task::JoinSet; +use tokio_util::sync::CancellationToken; +use utils::{completion::Barrier, yielding_loop::yielding_loop}; + +use super::{CommandRequest, CommandResponse}; + +/// Scheduling interval is the time between calls to JobGenerator::schedule. +/// When we schedule jobs, the job generator may provide a hint of its preferred +/// interval, which we will respect within these intervals. +const MAX_SCHEDULING_INTERVAL: Duration = Duration::from_secs(10); +const MIN_SCHEDULING_INTERVAL: Duration = Duration::from_secs(1); + +/// Scheduling helper for background work across many tenants. +/// +/// Systems that need to run background work across many tenants may use this type +/// to schedule jobs within a concurrency limit, along with their own [`JobGenerator`] +/// implementation to provide the work to execute. This is a simple scheduler that just +/// polls the generator for outstanding work, replacing its queue of pending work with +/// what the generator yields on each call: the job generator can change its mind about +/// the order of jobs between calls. The job generator is notified when jobs complete, +/// and additionally may expose a command hook to generate jobs on-demand (e.g. to implement +/// admin APIs). +/// +/// For an example see [`crate::tenant::secondary::heatmap_uploader`] +/// +/// G: A JobGenerator that this scheduler will poll to find pending jobs +/// PJ: 'Pending Job': type for job descriptors that are ready to run +/// RJ: 'Running Job' type' for jobs that have been spawned +/// C : 'Completion' type that spawned jobs will send when they finish +/// CMD: 'Command' type that the job generator will accept to create jobs on-demand +pub(super) struct TenantBackgroundJobs +where + G: JobGenerator, + C: Completion, + PJ: PendingJob, + RJ: RunningJob, +{ + generator: G, + + /// Ready to run. Will progress to `running` once concurrent limit is satisfied, or + /// be removed on next scheduling pass. + pending: std::collections::VecDeque, + + /// Tasks currently running in Self::tasks for these tenants. Check this map + /// before pushing more work into pending for the same tenant. + running: HashMap, + + tasks: JoinSet, + + concurrency: usize, + + /// How often we would like schedule_interval to be called. + pub(super) scheduling_interval: Duration, + + _phantom: PhantomData<(PJ, RJ, C, CMD)>, +} + +#[async_trait::async_trait] +pub(crate) trait JobGenerator +where + C: Completion, + PJ: PendingJob, + RJ: RunningJob, +{ + /// Called at each scheduling interval. Return a list of jobs to run, most urgent first. + /// + /// This function may be expensive (e.g. walk all tenants), but should not do any I/O. + /// Implementations should take care to yield the executor periodically if running + /// very long loops. + /// + /// Yielding a job here does _not_ guarantee that it will run: if the queue of pending + /// jobs is not drained by the next scheduling interval, pending jobs will be cleared + /// and re-generated. + async fn schedule(&mut self) -> SchedulingResult; + + /// Called when a pending job is ready to be run. + /// + /// The job generation provides a future, and a RJ (Running Job) descriptor that tracks it. + fn spawn(&mut self, pending_job: PJ) -> (RJ, Pin + Send>>); + + /// Called when a job previously spawned with spawn() transmits its completion + fn on_completion(&mut self, completion: C); + + /// Called when a command is received. A job will be spawned immediately if the return + /// value is Some, ignoring concurrency limits and the pending queue. + fn on_command(&mut self, cmd: CMD) -> anyhow::Result; +} + +/// [`JobGenerator`] returns this to provide pending jobs, and hints about scheduling +pub(super) struct SchedulingResult { + pub(super) jobs: Vec, + /// The job generator would like to be called again this soon + pub(super) want_interval: Option, +} + +/// See [`TenantBackgroundJobs`]. +pub(super) trait PendingJob { + fn get_tenant_shard_id(&self) -> &TenantShardId; +} + +/// See [`TenantBackgroundJobs`]. +pub(super) trait Completion: Send + 'static { + fn get_tenant_shard_id(&self) -> &TenantShardId; +} + +/// See [`TenantBackgroundJobs`]. +pub(super) trait RunningJob { + fn get_barrier(&self) -> Barrier; +} + +impl TenantBackgroundJobs +where + C: Completion, + PJ: PendingJob, + RJ: RunningJob, + G: JobGenerator, +{ + pub(super) fn new(generator: G, concurrency: usize) -> Self { + Self { + generator, + pending: std::collections::VecDeque::new(), + running: HashMap::new(), + tasks: JoinSet::new(), + concurrency, + scheduling_interval: MAX_SCHEDULING_INTERVAL, + _phantom: PhantomData, + } + } + + pub(super) async fn run( + &mut self, + mut command_queue: tokio::sync::mpsc::Receiver>, + background_jobs_can_start: Barrier, + cancel: CancellationToken, + ) { + tracing::info!("Waiting for background_jobs_can start..."); + background_jobs_can_start.wait().await; + tracing::info!("background_jobs_can is ready, proceeding."); + + while !cancel.is_cancelled() { + // Look for new work: this is relatively expensive because we have to go acquire the lock on + // the tenant manager to retrieve tenants, and then iterate over them to figure out which ones + // require an upload. + self.schedule_iteration(&cancel).await; + + if cancel.is_cancelled() { + return; + } + + // Schedule some work, if concurrency limit permits it + self.spawn_pending(); + + // Between scheduling iterations, we will: + // - Drain any complete tasks and spawn pending tasks + // - Handle incoming administrative commands + // - Check our cancellation token + let next_scheduling_iteration = Instant::now() + .checked_add(self.scheduling_interval) + .unwrap_or_else(|| { + tracing::warn!( + "Scheduling interval invalid ({}s)", + self.scheduling_interval.as_secs_f64() + ); + // unwrap(): this constant is small, cannot fail to add to time unless + // we are close to the end of the universe. + Instant::now().checked_add(MIN_SCHEDULING_INTERVAL).unwrap() + }); + loop { + tokio::select! { + _ = cancel.cancelled() => { + tracing::info!("joining tasks"); + // We do not simply drop the JoinSet, in order to have an orderly shutdown without cancellation. + // It is the callers responsibility to make sure that the tasks they scheduled + // respect an appropriate cancellation token, to shut down promptly. It is only + // safe to wait on joining these tasks because we can see the cancellation token + // has been set. + while let Some(_r) = self.tasks.join_next().await {} + tracing::info!("terminating on cancellation token."); + + break; + }, + _ = tokio::time::sleep(next_scheduling_iteration.duration_since(Instant::now())) => { + tracing::debug!("woke for scheduling interval"); + break;}, + cmd = command_queue.recv() => { + tracing::debug!("woke for command queue"); + let cmd = match cmd { + Some(c) =>c, + None => { + // SecondaryController was destroyed, and this has raced with + // our CancellationToken + tracing::info!("terminating on command queue destruction"); + cancel.cancel(); + break; + } + }; + + let CommandRequest{ + response_tx, + payload + } = cmd; + self.handle_command(payload, response_tx); + }, + _ = async { + let completion = self.process_next_completion().await; + match completion { + Some(c) => { + self.generator.on_completion(c); + if !cancel.is_cancelled() { + self.spawn_pending(); + } + }, + None => { + // Nothing is running, so just wait: expect that this future + // will be dropped when something in the outer select! fires. + cancel.cancelled().await; + } + } + + } => {} + } + } + } + } + + fn do_spawn(&mut self, job: PJ) { + let tenant_shard_id = *job.get_tenant_shard_id(); + let (in_progress, fut) = self.generator.spawn(job); + + self.tasks.spawn(fut); + + self.running.insert(tenant_shard_id, in_progress); + } + + /// For all pending tenants that are elegible for execution, spawn their task. + /// + /// Caller provides the spawn operation, we track the resulting execution. + fn spawn_pending(&mut self) { + while !self.pending.is_empty() && self.running.len() < self.concurrency { + // unwrap: loop condition includes !is_empty() + let pending = self.pending.pop_front().unwrap(); + self.do_spawn(pending); + } + } + + /// For administrative commands: skip the pending queue, ignore concurrency limits + fn spawn_now(&mut self, job: PJ) -> &RJ { + let tenant_shard_id = *job.get_tenant_shard_id(); + self.do_spawn(job); + self.running + .get(&tenant_shard_id) + .expect("We just inserted this") + } + + /// Wait until the next task completes, and handle its completion + /// + /// Cancellation: this method is cancel-safe. + async fn process_next_completion(&mut self) -> Option { + match self.tasks.join_next().await { + Some(r) => { + // We use a channel to drive completions, but also + // need to drain the JoinSet to avoid completed tasks + // accumulating. These calls are 1:1 because every task + // we spawn into this joinset submits is result to the channel. + let completion = r.expect("Panic in background task"); + + self.running.remove(completion.get_tenant_shard_id()); + Some(completion) + } + None => { + // Nothing is running, so we have nothing to wait for. We may drop out: the + // main even loop will call us again after the next time it has run something. + None + } + } + } + + /// Convert the command into a pending job, spawn it, and when the spawned + /// job completes, send the result down `response_tx`. + fn handle_command( + &mut self, + cmd: CMD, + response_tx: tokio::sync::oneshot::Sender, + ) { + let job = match self.generator.on_command(cmd) { + Ok(j) => j, + Err(e) => { + response_tx.send(CommandResponse { result: Err(e) }).ok(); + return; + } + }; + + let tenant_shard_id = job.get_tenant_shard_id(); + let barrier = if let Some(barrier) = self.get_running(tenant_shard_id) { + barrier + } else { + let running = self.spawn_now(job); + running.get_barrier().clone() + }; + + // This task does no I/O: it only listens for a barrier's completion and then + // sends to the command response channel. It is therefore safe to spawn this without + // any gates/task_mgr hooks. + tokio::task::spawn(async move { + barrier.wait().await; + + response_tx.send(CommandResponse { result: Ok(()) }).ok(); + }); + } + + fn get_running(&self, tenant_shard_id: &TenantShardId) -> Option { + self.running.get(tenant_shard_id).map(|r| r.get_barrier()) + } + + /// Periodic execution phase: inspect all attached tenants and schedule any work they require. + /// + /// The type in `tenants` should be a tenant-like structure, e.g. [`crate::tenant::Tenant`] or [`crate::tenant::secondary::SecondaryTenant`] + /// + /// This function resets the pending list: it is assumed that the caller may change their mind about + /// which tenants need work between calls to schedule_iteration. + async fn schedule_iteration(&mut self, cancel: &CancellationToken) { + let SchedulingResult { + jobs, + want_interval, + } = self.generator.schedule().await; + + // Adjust interval based on feedback from the job generator + if let Some(want_interval) = want_interval { + // Calculation uses second granularity: this scheduler is not intended for high frequency tasks + self.scheduling_interval = Duration::from_secs(std::cmp::min( + std::cmp::max(MIN_SCHEDULING_INTERVAL.as_secs(), want_interval.as_secs()), + MAX_SCHEDULING_INTERVAL.as_secs(), + )); + } + + // The priority order of previously scheduled work may be invalidated by current state: drop + // all pending work (it will be re-scheduled if still needed) + self.pending.clear(); + + // While iterating over the potentially-long list of tenants, we will periodically yield + // to avoid blocking executor. + yielding_loop(1000, cancel, jobs.into_iter(), |job| { + // Skip tenants that already have a write in flight + if !self.running.contains_key(job.get_tenant_shard_id()) { + self.pending.push_back(job); + } + }) + .await + .ok(); + } +} diff --git a/pageserver/src/tenant/tasks.rs b/pageserver/src/tenant/tasks.rs index 7ff1873eda..aa5894cc37 100644 --- a/pageserver/src/tenant/tasks.rs +++ b/pageserver/src/tenant/tasks.rs @@ -45,6 +45,8 @@ pub(crate) enum BackgroundLoopKind { ConsumptionMetricsCollectMetrics, ConsumptionMetricsSyntheticSizeWorker, InitialLogicalSizeCalculation, + HeatmapUpload, + SecondaryDownload, } impl BackgroundLoopKind { diff --git a/test_runner/fixtures/pageserver/http.py b/test_runner/fixtures/pageserver/http.py index 6dea0d923d..ccacc0a987 100644 --- a/test_runner/fixtures/pageserver/http.py +++ b/test_runner/fixtures/pageserver/http.py @@ -326,6 +326,10 @@ class PageserverHttpClient(requests.Session): res = self.post(f"http://localhost:{self.port}/v1/tenant/{tenant_id}/heatmap_upload") self.verbose_error(res) + def tenant_secondary_download(self, tenant_id: TenantId): + res = self.post(f"http://localhost:{self.port}/v1/tenant/{tenant_id}/secondary/download") + self.verbose_error(res) + def set_tenant_config(self, tenant_id: TenantId, config: dict[str, Any]): assert "tenant_id" not in config.keys() res = self.put( diff --git a/test_runner/regress/test_pageserver_secondary.py b/test_runner/regress/test_pageserver_secondary.py index 8ae4297983..a9eff99a0c 100644 --- a/test_runner/regress/test_pageserver_secondary.py +++ b/test_runner/regress/test_pageserver_secondary.py @@ -1,9 +1,11 @@ import random +from pathlib import Path from typing import Any, Dict, Optional import pytest from fixtures.log_helper import log -from fixtures.neon_fixtures import NeonEnvBuilder, NeonPageserver +from fixtures.neon_fixtures import NeonEnvBuilder, NeonPageserver, S3Scrubber +from fixtures.pageserver.utils import assert_prefix_empty, tenant_delete_wait_completed from fixtures.remote_storage import LocalFsStorage, RemoteStorageKind from fixtures.types import TenantId, TimelineId from fixtures.utils import wait_until @@ -251,6 +253,9 @@ def test_live_migration(neon_env_builder: NeonEnvBuilder): flush_ms=5000, ) + # Encourage the new location to download while still in secondary mode + pageserver_b.http_client().tenant_secondary_download(tenant_id) + migrated_generation = env.attachment_service.attach_hook_issue(tenant_id, pageserver_b.id) log.info(f"Acquired generation {migrated_generation} for destination pageserver") assert migrated_generation == initial_generation + 1 @@ -258,8 +263,6 @@ def test_live_migration(neon_env_builder: NeonEnvBuilder): # Writes and reads still work in AttachedStale. workload.validate(pageserver_a.id) - # TODO: call into secondary mode API hooks to do an upload/download sync - # Generate some more dirty writes: we expect the origin to ingest WAL in # in AttachedStale workload.churn_rows(64, pageserver_a.id, upload=False) @@ -369,3 +372,143 @@ def test_heatmap_uploads(neon_env_builder: NeonEnvBuilder): log.info(f"Read back heatmap: {heatmap_second}") assert heatmap_second != heatmap_first validate_heatmap(heatmap_second) + + +def list_layers(pageserver, tenant_id: TenantId, timeline_id: TimelineId) -> list[Path]: + """ + Inspect local storage on a pageserver to discover which layer files are present. + + :return: list of relative paths to layers, from the timeline root. + """ + timeline_path = pageserver.timeline_dir(tenant_id, timeline_id) + + def relative(p: Path) -> Path: + return p.relative_to(timeline_path) + + return sorted( + list( + map( + relative, + filter( + lambda path: path.name != "metadata" + and "ephemeral" not in path.name + and "temp" not in path.name, + timeline_path.glob("*"), + ), + ) + ) + ) + + +def test_secondary_downloads(neon_env_builder: NeonEnvBuilder): + """ + Test the overall data flow in secondary mode: + - Heatmap uploads from the attached location + - Heatmap & layer downloads from the secondary location + - Eviction of layers on the attached location results in deletion + on the secondary location as well. + """ + neon_env_builder.num_pageservers = 2 + neon_env_builder.enable_pageserver_remote_storage( + remote_storage_kind=RemoteStorageKind.MOCK_S3, + ) + env = neon_env_builder.init_start(initial_tenant_conf=TENANT_CONF) + assert env.attachment_service is not None + + tenant_id = env.initial_tenant + timeline_id = env.initial_timeline + + ps_attached = env.pageservers[0] + ps_secondary = env.pageservers[1] + + workload = Workload(env, tenant_id, timeline_id) + workload.init(env.pageservers[0].id) + workload.write_rows(256, ps_attached.id) + + # Configure a secondary location + log.info("Setting up secondary location...") + ps_secondary.tenant_location_configure( + tenant_id, + { + "mode": "Secondary", + "secondary_conf": {"warm": True}, + "tenant_conf": {}, + }, + ) + readback_conf = ps_secondary.read_tenant_location_conf(tenant_id) + log.info(f"Read back conf: {readback_conf}") + + # Explicit upload/download cycle + # ============================== + log.info("Synchronizing after initial write...") + ps_attached.http_client().tenant_heatmap_upload(tenant_id) + + ps_secondary.http_client().tenant_secondary_download(tenant_id) + + assert list_layers(ps_attached, tenant_id, timeline_id) == list_layers( + ps_secondary, tenant_id, timeline_id + ) + + # Make changes on attached pageserver, check secondary downloads them + # =================================================================== + log.info("Synchronizing after subsequent write...") + workload.churn_rows(128, ps_attached.id) + + ps_attached.http_client().tenant_heatmap_upload(tenant_id) + ps_secondary.http_client().tenant_secondary_download(tenant_id) + + assert list_layers(ps_attached, tenant_id, timeline_id) == list_layers( + ps_secondary, tenant_id, timeline_id + ) + + # FIXME: this sleep is needed to avoid on-demand promotion of the layers we evict, while + # walreceiver is still doing something. + import time + + time.sleep(5) + + # Do evictions on attached pageserver, check secondary follows along + # ================================================================== + log.info("Evicting a layer...") + layer_to_evict = list_layers(ps_attached, tenant_id, timeline_id)[0] + ps_attached.http_client().evict_layer(tenant_id, timeline_id, layer_name=layer_to_evict.name) + + log.info("Synchronizing after eviction...") + ps_attached.http_client().tenant_heatmap_upload(tenant_id) + ps_secondary.http_client().tenant_secondary_download(tenant_id) + + assert layer_to_evict not in list_layers(ps_attached, tenant_id, timeline_id) + assert list_layers(ps_attached, tenant_id, timeline_id) == list_layers( + ps_secondary, tenant_id, timeline_id + ) + + # Scrub the remote storage + # ======================== + # This confirms that the scrubber isn't upset by the presence of the heatmap + S3Scrubber(neon_env_builder.test_output_dir, neon_env_builder).scan_metadata() + + # Detach secondary and delete tenant + # =================================== + # This confirms that the heatmap gets cleaned up as well as other normal content. + log.info("Detaching secondary location...") + ps_secondary.tenant_location_configure( + tenant_id, + { + "mode": "Detached", + "secondary_conf": None, + "tenant_conf": {}, + }, + ) + + log.info("Deleting tenant...") + tenant_delete_wait_completed(ps_attached.http_client(), tenant_id, 10) + + assert_prefix_empty( + neon_env_builder, + prefix="/".join( + ( + "tenants", + str(tenant_id), + ) + ), + ) From 7de829e475471cac50cfbcb80e2792a7658fac9f Mon Sep 17 00:00:00 2001 From: Alexander Bayandin Date: Fri, 5 Jan 2024 15:35:07 +0000 Subject: [PATCH 33/35] test_runner: replace black with ruff format (#6268) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## Problem `black` is slow sometimes, we can replace it with `ruff format` (a new feature in 0.1.2 [0]), which produces pretty similar to black style [1]. On my local machine (MacBook M1 Pro 16GB): ``` # `black` on main $ hyperfine "BLACK_CACHE_DIR=/dev/null poetry run black ." Benchmark 1: BLACK_CACHE_DIR=/dev/null poetry run black . Time (mean ± σ): 3.131 s ± 0.090 s [User: 5.194 s, System: 0.859 s] Range (min … max): 3.047 s … 3.354 s 10 runs ``` ``` # `ruff format` on the current PR $ hyperfine "RUFF_NO_CACHE=true poetry run ruff format" Benchmark 1: RUFF_NO_CACHE=true poetry run ruff format Time (mean ± σ): 300.7 ms ± 50.2 ms [User: 259.5 ms, System: 76.1 ms] Range (min … max): 267.5 ms … 420.2 ms 10 runs ``` ## Summary of changes - Replace `black` with `ruff format` everywhere - [0] https://docs.astral.sh/ruff/formatter/ - [1] https://docs.astral.sh/ruff/formatter/#black-compatibility --- .github/workflows/build_and_test.yml | 8 +- docs/sourcetree.md | 8 +- poetry.lock | 127 +++--------------- pre-commit.py | 24 ++-- pyproject.toml | 16 +-- scripts/export_import_between_pageservers.py | 2 +- scripts/reformat | 4 +- test_runner/fixtures/neon_fixtures.py | 4 +- test_runner/fixtures/pageserver/http.py | 4 +- test_runner/performance/test_perf_olap.py | 7 +- .../performance/test_wal_backpressure.py | 3 +- .../regress/test_attach_tenant_config.py | 13 +- test_runner/regress/test_compatibility.py | 8 +- test_runner/regress/test_crafted_wal_end.py | 1 - test_runner/regress/test_layer_eviction.py | 4 +- test_runner/regress/test_pageserver_api.py | 3 +- test_runner/regress/test_tenant_detach.py | 14 +- test_runner/regress/test_tenant_relocation.py | 6 +- .../test_tenants_with_remote_storage.py | 8 +- 19 files changed, 79 insertions(+), 185 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 78deff6e85..880d6044f2 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -105,11 +105,11 @@ jobs: - name: Install Python deps run: ./scripts/pysync - - name: Run ruff to ensure code format - run: poetry run ruff . + - name: Run `ruff check` to ensure code format + run: poetry run ruff check . - - name: Run black to ensure code format - run: poetry run black --diff --check . + - name: Run `ruff format` to ensure code format + run: poetry run ruff format --check . - name: Run mypy to check types run: poetry run mypy . diff --git a/docs/sourcetree.md b/docs/sourcetree.md index 95bed83ae5..12fa80349e 100644 --- a/docs/sourcetree.md +++ b/docs/sourcetree.md @@ -129,13 +129,13 @@ Run `poetry shell` to activate the virtual environment. Alternatively, use `poetry run` to run a single command in the venv, e.g. `poetry run pytest`. ### Obligatory checks -We force code formatting via `black`, `ruff`, and type hints via `mypy`. +We force code formatting via `ruff`, and type hints via `mypy`. Run the following commands in the repository's root (next to `pyproject.toml`): ```bash -poetry run black . # All code is reformatted -poetry run ruff . # Python linter -poetry run mypy . # Ensure there are no typing errors +poetry run ruff format . # All code is reformatted +poetry run ruff check . # Python linter +poetry run mypy . # Ensure there are no typing errors ``` **WARNING**: do not run `mypy` from a directory other than the root of the repository. diff --git a/poetry.lock b/poetry.lock index c597d811bd..428698cb5a 100644 --- a/poetry.lock +++ b/poetry.lock @@ -1,4 +1,4 @@ -# This file is automatically @generated by Poetry 1.5.1 and should not be changed by hand. +# This file is automatically @generated by Poetry 1.7.1 and should not be changed by hand. [[package]] name = "aiohttp" @@ -288,55 +288,6 @@ files = [ {file = "backoff-2.2.1.tar.gz", hash = "sha256:03f829f5bb1923180821643f8753b0502c3b682293992485b0eef2807afa5cba"}, ] -[[package]] -name = "black" -version = "23.3.0" -description = "The uncompromising code formatter." -optional = false -python-versions = ">=3.7" -files = [ - {file = "black-23.3.0-cp310-cp310-macosx_10_16_arm64.whl", hash = "sha256:0945e13506be58bf7db93ee5853243eb368ace1c08a24c65ce108986eac65915"}, - {file = "black-23.3.0-cp310-cp310-macosx_10_16_universal2.whl", hash = "sha256:67de8d0c209eb5b330cce2469503de11bca4085880d62f1628bd9972cc3366b9"}, - {file = "black-23.3.0-cp310-cp310-macosx_10_16_x86_64.whl", hash = "sha256:7c3eb7cea23904399866c55826b31c1f55bbcd3890ce22ff70466b907b6775c2"}, - {file = "black-23.3.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:32daa9783106c28815d05b724238e30718f34155653d4d6e125dc7daec8e260c"}, - {file = "black-23.3.0-cp310-cp310-win_amd64.whl", hash = "sha256:35d1381d7a22cc5b2be2f72c7dfdae4072a3336060635718cc7e1ede24221d6c"}, - {file = "black-23.3.0-cp311-cp311-macosx_10_16_arm64.whl", hash = "sha256:a8a968125d0a6a404842fa1bf0b349a568634f856aa08ffaff40ae0dfa52e7c6"}, - {file = "black-23.3.0-cp311-cp311-macosx_10_16_universal2.whl", hash = "sha256:c7ab5790333c448903c4b721b59c0d80b11fe5e9803d8703e84dcb8da56fec1b"}, - {file = "black-23.3.0-cp311-cp311-macosx_10_16_x86_64.whl", hash = "sha256:a6f6886c9869d4daae2d1715ce34a19bbc4b95006d20ed785ca00fa03cba312d"}, - {file = "black-23.3.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6f3c333ea1dd6771b2d3777482429864f8e258899f6ff05826c3a4fcc5ce3f70"}, - {file = "black-23.3.0-cp311-cp311-win_amd64.whl", hash = "sha256:11c410f71b876f961d1de77b9699ad19f939094c3a677323f43d7a29855fe326"}, - {file = "black-23.3.0-cp37-cp37m-macosx_10_16_x86_64.whl", hash = "sha256:1d06691f1eb8de91cd1b322f21e3bfc9efe0c7ca1f0e1eb1db44ea367dff656b"}, - {file = "black-23.3.0-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:50cb33cac881766a5cd9913e10ff75b1e8eb71babf4c7104f2e9c52da1fb7de2"}, - {file = "black-23.3.0-cp37-cp37m-win_amd64.whl", hash = "sha256:e114420bf26b90d4b9daa597351337762b63039752bdf72bf361364c1aa05925"}, - {file = "black-23.3.0-cp38-cp38-macosx_10_16_arm64.whl", hash = "sha256:48f9d345675bb7fbc3dd85821b12487e1b9a75242028adad0333ce36ed2a6d27"}, - {file = "black-23.3.0-cp38-cp38-macosx_10_16_universal2.whl", hash = "sha256:714290490c18fb0126baa0fca0a54ee795f7502b44177e1ce7624ba1c00f2331"}, - {file = "black-23.3.0-cp38-cp38-macosx_10_16_x86_64.whl", hash = "sha256:064101748afa12ad2291c2b91c960be28b817c0c7eaa35bec09cc63aa56493c5"}, - {file = "black-23.3.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:562bd3a70495facf56814293149e51aa1be9931567474993c7942ff7d3533961"}, - {file = "black-23.3.0-cp38-cp38-win_amd64.whl", hash = "sha256:e198cf27888ad6f4ff331ca1c48ffc038848ea9f031a3b40ba36aced7e22f2c8"}, - {file = "black-23.3.0-cp39-cp39-macosx_10_16_arm64.whl", hash = "sha256:3238f2aacf827d18d26db07524e44741233ae09a584273aa059066d644ca7b30"}, - {file = "black-23.3.0-cp39-cp39-macosx_10_16_universal2.whl", hash = "sha256:f0bd2f4a58d6666500542b26354978218a9babcdc972722f4bf90779524515f3"}, - {file = "black-23.3.0-cp39-cp39-macosx_10_16_x86_64.whl", hash = "sha256:92c543f6854c28a3c7f39f4d9b7694f9a6eb9d3c5e2ece488c327b6e7ea9b266"}, - {file = "black-23.3.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3a150542a204124ed00683f0db1f5cf1c2aaaa9cc3495b7a3b5976fb136090ab"}, - {file = "black-23.3.0-cp39-cp39-win_amd64.whl", hash = "sha256:6b39abdfb402002b8a7d030ccc85cf5afff64ee90fa4c5aebc531e3ad0175ddb"}, - {file = "black-23.3.0-py3-none-any.whl", hash = "sha256:ec751418022185b0c1bb7d7736e6933d40bbb14c14a0abcf9123d1b159f98dd4"}, - {file = "black-23.3.0.tar.gz", hash = "sha256:1c7b8d606e728a41ea1ccbd7264677e494e87cf630e399262ced92d4a8dac940"}, -] - -[package.dependencies] -click = ">=8.0.0" -mypy-extensions = ">=0.4.3" -packaging = ">=22.0" -pathspec = ">=0.9.0" -platformdirs = ">=2" -tomli = {version = ">=1.1.0", markers = "python_version < \"3.11\""} -typing-extensions = {version = ">=3.10.0.0", markers = "python_version < \"3.10\""} - -[package.extras] -colorama = ["colorama (>=0.4.3)"] -d = ["aiohttp (>=3.7.4)"] -jupyter = ["ipython (>=7.8.0)", "tokenize-rt (>=3.2.0)"] -uvloop = ["uvloop (>=0.15.2)"] - [[package]] name = "boto3" version = "1.34.11" @@ -1627,17 +1578,6 @@ files = [ {file = "packaging-23.0.tar.gz", hash = "sha256:b6ad297f8907de0fa2fe1ccbd26fdaf387f5f47c7275fedf8cce89f99446cf97"}, ] -[[package]] -name = "pathspec" -version = "0.9.0" -description = "Utility library for gitignore style pattern matching of file paths." -optional = false -python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*,>=2.7" -files = [ - {file = "pathspec-0.9.0-py2.py3-none-any.whl", hash = "sha256:7d15c4ddb0b5c802d161efc417ec1a2558ea2653c2e8ad9c19098201dc1c993a"}, - {file = "pathspec-0.9.0.tar.gz", hash = "sha256:e564499435a2673d586f6b2130bb5b95f04a3ba06f81b8f895b651a3c76aabb1"}, -] - [[package]] name = "pbr" version = "5.9.0" @@ -1649,21 +1589,6 @@ files = [ {file = "pbr-5.9.0.tar.gz", hash = "sha256:e8dca2f4b43560edef58813969f52a56cef023146cbb8931626db80e6c1c4308"}, ] -[[package]] -name = "platformdirs" -version = "2.5.2" -description = "A small Python module for determining appropriate platform-specific dirs, e.g. a \"user data dir\"." -optional = false -python-versions = ">=3.7" -files = [ - {file = "platformdirs-2.5.2-py3-none-any.whl", hash = "sha256:027d8e83a2d7de06bbac4e5ef7e023c02b863d7ea5d079477e722bb41ab25788"}, - {file = "platformdirs-2.5.2.tar.gz", hash = "sha256:58c8abb07dcb441e6ee4b11d8df0ac856038f944ab98b7be6b27b2a3c7feef19"}, -] - -[package.extras] -docs = ["furo (>=2021.7.5b38)", "proselint (>=0.10.2)", "sphinx (>=4)", "sphinx-autodoc-typehints (>=1.12)"] -test = ["appdirs (==1.4.4)", "pytest (>=6)", "pytest-cov (>=2.7)", "pytest-mock (>=3.6)"] - [[package]] name = "pluggy" version = "1.0.0" @@ -2207,28 +2132,28 @@ pyasn1 = ">=0.1.3" [[package]] name = "ruff" -version = "0.0.269" -description = "An extremely fast Python linter, written in Rust." +version = "0.1.11" +description = "An extremely fast Python linter and code formatter, written in Rust." optional = false python-versions = ">=3.7" files = [ - {file = "ruff-0.0.269-py3-none-macosx_10_7_x86_64.whl", hash = "sha256:3569bcdee679045c09c0161fabc057599759c49219a08d9a4aad2cc3982ccba3"}, - {file = "ruff-0.0.269-py3-none-macosx_10_9_x86_64.macosx_11_0_arm64.macosx_10_9_universal2.whl", hash = "sha256:56347da63757a56cbce7d4b3d6044ca4f1941cd1bbff3714f7554360c3361f83"}, - {file = "ruff-0.0.269-py3-none-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6da8ee25ef2f0cc6cc8e6e20942c1d44d25a36dce35070d7184655bc14f63f63"}, - {file = "ruff-0.0.269-py3-none-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:bd81b8e681b9eaa6cf15484f3985bd8bd97c3d114e95bff3e8ea283bf8865062"}, - {file = "ruff-0.0.269-py3-none-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:1f19f59ca3c28742955241fb452f3346241ddbd34e72ac5cb3d84fadebcf6bc8"}, - {file = "ruff-0.0.269-py3-none-manylinux_2_17_ppc64.manylinux2014_ppc64.whl", hash = "sha256:f062059b8289a4fab7f6064601b811d447c2f9d3d432a17f689efe4d68988450"}, - {file = "ruff-0.0.269-py3-none-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:3f5dc7aac52c58e82510217e3c7efd80765c134c097c2815d59e40face0d1fe6"}, - {file = "ruff-0.0.269-py3-none-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e131b4dbe798c391090c6407641d6ab12c0fa1bb952379dde45e5000e208dabb"}, - {file = "ruff-0.0.269-py3-none-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a374434e588e06550df0f8dcb74777290f285678de991fda4e1063c367ab2eb2"}, - {file = "ruff-0.0.269-py3-none-musllinux_1_2_aarch64.whl", hash = "sha256:cec2f4b84a14b87f1b121488649eb5b4eaa06467a2387373f750da74bdcb5679"}, - {file = "ruff-0.0.269-py3-none-musllinux_1_2_armv7l.whl", hash = "sha256:374b161753a247904aec7a32d45e165302b76b6e83d22d099bf3ff7c232c888f"}, - {file = "ruff-0.0.269-py3-none-musllinux_1_2_i686.whl", hash = "sha256:9ca0a1ddb1d835b5f742db9711c6cf59f213a1ad0088cb1e924a005fd399e7d8"}, - {file = "ruff-0.0.269-py3-none-musllinux_1_2_x86_64.whl", hash = "sha256:5a20658f0b97d207c7841c13d528f36d666bf445b00b01139f28a8ccb80093bb"}, - {file = "ruff-0.0.269-py3-none-win32.whl", hash = "sha256:03ff42bc91ceca58e0f0f072cb3f9286a9208f609812753474e799a997cdad1a"}, - {file = "ruff-0.0.269-py3-none-win_amd64.whl", hash = "sha256:f3b59ccff57b21ef0967ea8021fd187ec14c528ec65507d8bcbe035912050776"}, - {file = "ruff-0.0.269-py3-none-win_arm64.whl", hash = "sha256:bbeb857b1e508a4487bdb02ca1e6d41dd8d5ac5335a5246e25de8a3dff38c1ff"}, - {file = "ruff-0.0.269.tar.gz", hash = "sha256:11ddcfbab32cf5c420ea9dd5531170ace5a3e59c16d9251c7bd2581f7b16f602"}, + {file = "ruff-0.1.11-py3-none-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:a7f772696b4cdc0a3b2e527fc3c7ccc41cdcb98f5c80fdd4f2b8c50eb1458196"}, + {file = "ruff-0.1.11-py3-none-macosx_10_12_x86_64.whl", hash = "sha256:934832f6ed9b34a7d5feea58972635c2039c7a3b434fe5ba2ce015064cb6e955"}, + {file = "ruff-0.1.11-py3-none-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ea0d3e950e394c4b332bcdd112aa566010a9f9c95814844a7468325290aabfd9"}, + {file = "ruff-0.1.11-py3-none-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:9bd4025b9c5b429a48280785a2b71d479798a69f5c2919e7d274c5f4b32c3607"}, + {file = "ruff-0.1.11-py3-none-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e1ad00662305dcb1e987f5ec214d31f7d6a062cae3e74c1cbccef15afd96611d"}, + {file = "ruff-0.1.11-py3-none-manylinux_2_17_ppc64.manylinux2014_ppc64.whl", hash = "sha256:4b077ce83f47dd6bea1991af08b140e8b8339f0ba8cb9b7a484c30ebab18a23f"}, + {file = "ruff-0.1.11-py3-none-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c4a88efecec23c37b11076fe676e15c6cdb1271a38f2b415e381e87fe4517f18"}, + {file = "ruff-0.1.11-py3-none-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:5b25093dad3b055667730a9b491129c42d45e11cdb7043b702e97125bcec48a1"}, + {file = "ruff-0.1.11-py3-none-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:231d8fb11b2cc7c0366a326a66dafc6ad449d7fcdbc268497ee47e1334f66f77"}, + {file = "ruff-0.1.11-py3-none-musllinux_1_2_aarch64.whl", hash = "sha256:09c415716884950080921dd6237767e52e227e397e2008e2bed410117679975b"}, + {file = "ruff-0.1.11-py3-none-musllinux_1_2_armv7l.whl", hash = "sha256:0f58948c6d212a6b8d41cd59e349751018797ce1727f961c2fa755ad6208ba45"}, + {file = "ruff-0.1.11-py3-none-musllinux_1_2_i686.whl", hash = "sha256:190a566c8f766c37074d99640cd9ca3da11d8deae2deae7c9505e68a4a30f740"}, + {file = "ruff-0.1.11-py3-none-musllinux_1_2_x86_64.whl", hash = "sha256:6464289bd67b2344d2a5d9158d5eb81025258f169e69a46b741b396ffb0cda95"}, + {file = "ruff-0.1.11-py3-none-win32.whl", hash = "sha256:9b8f397902f92bc2e70fb6bebfa2139008dc72ae5177e66c383fa5426cb0bf2c"}, + {file = "ruff-0.1.11-py3-none-win_amd64.whl", hash = "sha256:eb85ee287b11f901037a6683b2374bb0ec82928c5cbc984f575d0437979c521a"}, + {file = "ruff-0.1.11-py3-none-win_arm64.whl", hash = "sha256:97ce4d752f964ba559c7023a86e5f8e97f026d511e48013987623915431c7ea9"}, + {file = "ruff-0.1.11.tar.gz", hash = "sha256:f9d4d88cb6eeb4dfe20f9f0519bd2eaba8119bde87c3d5065c541dbae2b5a2cb"}, ] [[package]] @@ -2496,16 +2421,6 @@ files = [ {file = "wrapt-1.14.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:8ad85f7f4e20964db4daadcab70b47ab05c7c1cf2a7c1e51087bfaa83831854c"}, {file = "wrapt-1.14.1-cp310-cp310-win32.whl", hash = "sha256:a9a52172be0b5aae932bef82a79ec0a0ce87288c7d132946d645eba03f0ad8a8"}, {file = "wrapt-1.14.1-cp310-cp310-win_amd64.whl", hash = "sha256:6d323e1554b3d22cfc03cd3243b5bb815a51f5249fdcbb86fda4bf62bab9e164"}, - {file = "wrapt-1.14.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:ecee4132c6cd2ce5308e21672015ddfed1ff975ad0ac8d27168ea82e71413f55"}, - {file = "wrapt-1.14.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:2020f391008ef874c6d9e208b24f28e31bcb85ccff4f335f15a3251d222b92d9"}, - {file = "wrapt-1.14.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2feecf86e1f7a86517cab34ae6c2f081fd2d0dac860cb0c0ded96d799d20b335"}, - {file = "wrapt-1.14.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:240b1686f38ae665d1b15475966fe0472f78e71b1b4903c143a842659c8e4cb9"}, - {file = "wrapt-1.14.1-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a9008dad07d71f68487c91e96579c8567c98ca4c3881b9b113bc7b33e9fd78b8"}, - {file = "wrapt-1.14.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:6447e9f3ba72f8e2b985a1da758767698efa72723d5b59accefd716e9e8272bf"}, - {file = "wrapt-1.14.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:acae32e13a4153809db37405f5eba5bac5fbe2e2ba61ab227926a22901051c0a"}, - {file = "wrapt-1.14.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:49ef582b7a1152ae2766557f0550a9fcbf7bbd76f43fbdc94dd3bf07cc7168be"}, - {file = "wrapt-1.14.1-cp311-cp311-win32.whl", hash = "sha256:358fe87cc899c6bb0ddc185bf3dbfa4ba646f05b1b0b9b5a27c2cb92c2cea204"}, - {file = "wrapt-1.14.1-cp311-cp311-win_amd64.whl", hash = "sha256:26046cd03936ae745a502abf44dac702a5e6880b2b01c29aea8ddf3353b68224"}, {file = "wrapt-1.14.1-cp35-cp35m-manylinux1_i686.whl", hash = "sha256:43ca3bbbe97af00f49efb06e352eae40434ca9d915906f77def219b88e85d907"}, {file = "wrapt-1.14.1-cp35-cp35m-manylinux1_x86_64.whl", hash = "sha256:6b1a564e6cb69922c7fe3a678b9f9a3c54e72b469875aa8018f18b4d1dd1adf3"}, {file = "wrapt-1.14.1-cp35-cp35m-manylinux2010_i686.whl", hash = "sha256:00b6d4ea20a906c0ca56d84f93065b398ab74b927a7a3dbd470f6fc503f95dc3"}, @@ -2743,4 +2658,4 @@ cffi = ["cffi (>=1.11)"] [metadata] lock-version = "2.0" python-versions = "^3.9" -content-hash = "8de8b05a9b35a6f76da7d7e3652ddbb521f1eca53fce7b933f537080a9d6eada" +content-hash = "35c237fe6a9278b2dc65b06ed96bde5afb9e393d52c01b00c59acf1df3a8d482" diff --git a/pre-commit.py b/pre-commit.py index dc0b9ed588..c5ed63ac44 100755 --- a/pre-commit.py +++ b/pre-commit.py @@ -36,17 +36,17 @@ def rustfmt(fix_inplace: bool = False, no_color: bool = False) -> str: return cmd -def black(fix_inplace: bool) -> str: - cmd = "poetry run black" - if not fix_inplace: - cmd += " --diff --check" +def ruff_check(fix_inplace: bool) -> str: + cmd = "poetry run ruff check" + if fix_inplace: + cmd += " --fix" return cmd -def ruff(fix_inplace: bool) -> str: - cmd = "poetry run ruff" - if fix_inplace: - cmd += " --fix" +def ruff_format(fix_inplace: bool) -> str: + cmd = "poetry run ruff format" + if not fix_inplace: + cmd += " --diff --check" return cmd @@ -109,16 +109,16 @@ if __name__ == "__main__": no_color=args.no_color, ) check( - name="black", + name="ruff check", suffix=".py", - cmd=black(fix_inplace=args.fix_inplace), + cmd=ruff_check(fix_inplace=args.fix_inplace), changed_files=files, no_color=args.no_color, ) check( - name="ruff", + name="ruff format", suffix=".py", - cmd=ruff(fix_inplace=args.fix_inplace), + cmd=ruff_format(fix_inplace=args.fix_inplace), changed_files=files, no_color=args.no_color, ) diff --git a/pyproject.toml b/pyproject.toml index 18c8ece4a7..bb04123e05 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -40,22 +40,13 @@ pytest-split = "^0.8.1" zstandard = "^0.21.0" [tool.poetry.group.dev.dependencies] -black = "^23.3.0" mypy = "==1.3.0" -ruff = "^0.0.269" +ruff = "^0.1.11" [build-system] requires = ["poetry-core>=1.0.0"] build-backend = "poetry.core.masonry.api" -[tool.black] -line-length = 100 -extend-exclude = ''' -/( - vendor -)/ -''' - [tool.mypy] exclude = "^vendor/" check_untyped_defs = true @@ -82,7 +73,9 @@ ignore_missing_imports = true [tool.ruff] target-version = "py39" extend-exclude = ["vendor/"] -ignore = ["E501"] +ignore = [ + "E501", # Line too long, we don't want to be too strict about it +] select = [ "E", # pycodestyle "F", # Pyflakes @@ -90,3 +83,4 @@ select = [ "W", # pycodestyle "B", # bugbear ] +line-length = 100 # this setting is rather guidance, it won't fail if it can't make the shorter diff --git a/scripts/export_import_between_pageservers.py b/scripts/export_import_between_pageservers.py index ff584bd4b0..980f343047 100755 --- a/scripts/export_import_between_pageservers.py +++ b/scripts/export_import_between_pageservers.py @@ -63,7 +63,7 @@ def subprocess_capture(capture_dir: str, cmd: List[str], **kwargs: Any) -> str: If those files already exist, we will overwrite them. Returns basepath for files with captured output. """ - assert type(cmd) is list + assert isinstance(cmd, list) base = os.path.basename(cmd[0]) + "_{}".format(global_counter()) basepath = os.path.join(capture_dir, base) stdout_filename = basepath + ".stdout" diff --git a/scripts/reformat b/scripts/reformat index 8688044f66..3533c4dcb8 100755 --- a/scripts/reformat +++ b/scripts/reformat @@ -6,5 +6,5 @@ set -euox pipefail echo 'Reformatting Rust code' cargo fmt echo 'Reformatting Python code' -poetry run ruff --fix test_runner scripts -poetry run black test_runner scripts +poetry run ruff check --fix test_runner scripts +poetry run ruff format test_runner scripts diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index f33e17a76a..001d4e23a9 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -1101,8 +1101,8 @@ class AbstractNeonCli(abc.ABC): If `local_binpath` is true, then we are invoking a test utility """ - assert type(arguments) == list - assert type(self.COMMAND) == str + assert isinstance(arguments, list) + assert isinstance(self.COMMAND, str) if local_binpath: # Test utility diff --git a/test_runner/fixtures/pageserver/http.py b/test_runner/fixtures/pageserver/http.py index ccacc0a987..a779dcc436 100644 --- a/test_runner/fixtures/pageserver/http.py +++ b/test_runner/fixtures/pageserver/http.py @@ -365,9 +365,9 @@ class PageserverHttpClient(requests.Session): assert isinstance(res, dict) assert TenantId(res["id"]) == tenant_id size = res["size"] - assert type(size) == int + assert isinstance(size, int) inputs = res["inputs"] - assert type(inputs) is dict + assert isinstance(inputs, dict) return (size, inputs) def tenant_size_debug(self, tenant_id: TenantId) -> str: diff --git a/test_runner/performance/test_perf_olap.py b/test_runner/performance/test_perf_olap.py index 1e6e9a0174..8a9509ea44 100644 --- a/test_runner/performance/test_perf_olap.py +++ b/test_runner/performance/test_perf_olap.py @@ -42,9 +42,10 @@ def test_clickbench_create_pg_stat_statements(remote_compare: RemoteCompare): # Please do not alter the label for the query, as it is used to identify it. # Labels for ClickBench queries match the labels in ClickBench reports # on https://benchmark.clickhouse.com/ (the DB size may differ). +# +# Disable auto formatting for the list of queries so that it's easier to read +# fmt: off QUERIES: Tuple[LabelledQuery, ...] = ( - # Disable `black` formatting for the list of queries so that it's easier to read - # fmt: off ### ClickBench queries: LabelledQuery("Q0", r"SELECT COUNT(*) FROM hits;"), LabelledQuery("Q1", r"SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0;"), @@ -96,8 +97,8 @@ QUERIES: Tuple[LabelledQuery, ...] = ( # LabelledQuery("NQ0", r"..."), # LabelledQuery("NQ1", r"..."), # ... - # fmt: on ) +# fmt: on EXPLAIN_STRING: str = "EXPLAIN (ANALYZE, VERBOSE, BUFFERS, COSTS, SETTINGS, FORMAT JSON)" diff --git a/test_runner/performance/test_wal_backpressure.py b/test_runner/performance/test_wal_backpressure.py index 3cb4b667ff..7eb244d378 100644 --- a/test_runner/performance/test_wal_backpressure.py +++ b/test_runner/performance/test_wal_backpressure.py @@ -32,8 +32,7 @@ def pg_compare(request) -> PgCompare: else: assert ( len(x) == 2 - ), f"request param ({request.param}) should have a format of \ - `neon_{{safekeepers_enable_fsync}}`" + ), f"request param ({request.param}) should have a format of `neon_{{safekeepers_enable_fsync}}`" # `NeonCompare` interface neon_env_builder = request.getfixturevalue("neon_env_builder") diff --git a/test_runner/regress/test_attach_tenant_config.py b/test_runner/regress/test_attach_tenant_config.py index 32397bbcc1..ed389b1aa2 100644 --- a/test_runner/regress/test_attach_tenant_config.py +++ b/test_runner/regress/test_attach_tenant_config.py @@ -194,12 +194,13 @@ def test_fully_custom_config(positive_env: NeonEnv): assert set(our_tenant_config.effective_config.keys()) == set( fully_custom_config.keys() ), "ensure we cover all config options" - assert { - k: initial_tenant_config.effective_config[k] != our_tenant_config.effective_config[k] - for k in fully_custom_config.keys() - } == { - k: True for k in fully_custom_config.keys() - }, "ensure our custom config has different values than the default config for all config options, so we know we overrode everything" + assert ( + { + k: initial_tenant_config.effective_config[k] != our_tenant_config.effective_config[k] + for k in fully_custom_config.keys() + } + == {k: True for k in fully_custom_config.keys()} + ), "ensure our custom config has different values than the default config for all config options, so we know we overrode everything" ps_http.tenant_detach(tenant_id) env.pageserver.tenant_attach(tenant_id, config=fully_custom_config) diff --git a/test_runner/regress/test_compatibility.py b/test_runner/regress/test_compatibility.py index 5a9c2782e6..f9d6d0a934 100644 --- a/test_runner/regress/test_compatibility.py +++ b/test_runner/regress/test_compatibility.py @@ -186,9 +186,7 @@ def test_backward_compatibility( else: raise - assert ( - not breaking_changes_allowed - ), "Breaking changes are allowed by ALLOW_BACKWARD_COMPATIBILITY_BREAKAGE, but the test has passed without any breakage" + assert not breaking_changes_allowed, "Breaking changes are allowed by ALLOW_BACKWARD_COMPATIBILITY_BREAKAGE, but the test has passed without any breakage" @check_ondisk_data_compatibility_if_enabled @@ -247,9 +245,7 @@ def test_forward_compatibility( else: raise - assert ( - not breaking_changes_allowed - ), "Breaking changes are allowed by ALLOW_FORWARD_COMPATIBILITY_BREAKAGE, but the test has passed without any breakage" + assert not breaking_changes_allowed, "Breaking changes are allowed by ALLOW_FORWARD_COMPATIBILITY_BREAKAGE, but the test has passed without any breakage" def check_neon_works(env: NeonEnv, test_output_dir: Path, sql_dump_path: Path, repo_dir: Path): diff --git a/test_runner/regress/test_crafted_wal_end.py b/test_runner/regress/test_crafted_wal_end.py index 7ec901af34..01ecc2b95f 100644 --- a/test_runner/regress/test_crafted_wal_end.py +++ b/test_runner/regress/test_crafted_wal_end.py @@ -2,7 +2,6 @@ import pytest from fixtures.log_helper import log from fixtures.neon_fixtures import NeonEnvBuilder, WalCraft - # Restart nodes with WAL end having specially crafted shape, like last record # crossing segment boundary, to test decoding issues. diff --git a/test_runner/regress/test_layer_eviction.py b/test_runner/regress/test_layer_eviction.py index 2cd2406065..efba2033fb 100644 --- a/test_runner/regress/test_layer_eviction.py +++ b/test_runner/regress/test_layer_eviction.py @@ -102,9 +102,7 @@ def test_basic_eviction( ), f"Did not expect to find {local_layer} layer after evicting" empty_layers = list(filter(lambda path: path.name != "metadata", timeline_path.glob("*"))) - assert ( - not empty_layers - ), f"After evicting all layers, timeline {tenant_id}/{timeline_id} should have no layers locally, but got: {empty_layers}" + assert not empty_layers, f"After evicting all layers, timeline {tenant_id}/{timeline_id} should have no layers locally, but got: {empty_layers}" evicted_layer_map_info = client.layer_map_info(tenant_id=tenant_id, timeline_id=timeline_id) assert ( diff --git a/test_runner/regress/test_pageserver_api.py b/test_runner/regress/test_pageserver_api.py index 573d2139ce..e29db1e252 100644 --- a/test_runner/regress/test_pageserver_api.py +++ b/test_runner/regress/test_pageserver_api.py @@ -145,8 +145,7 @@ def expect_updated_msg_lsn( last_msg_lsn = Lsn(timeline_details["last_received_msg_lsn"]) assert ( prev_msg_lsn is None or prev_msg_lsn < last_msg_lsn - ), f"the last received message's LSN {last_msg_lsn} hasn't been updated \ - compared to the previous message's LSN {prev_msg_lsn}" + ), f"the last received message's LSN {last_msg_lsn} hasn't been updated compared to the previous message's LSN {prev_msg_lsn}" return last_msg_lsn diff --git a/test_runner/regress/test_tenant_detach.py b/test_runner/regress/test_tenant_detach.py index 0dcbb23ad4..c6dbc77885 100644 --- a/test_runner/regress/test_tenant_detach.py +++ b/test_runner/regress/test_tenant_detach.py @@ -391,8 +391,7 @@ def test_tenant_detach_ignored_tenant(neon_simple_env: NeonEnv): tenants_after_detach = [tenant["id"] for tenant in client.tenant_list()] assert ( tenant_id not in tenants_after_detach - ), f"Ignored and then detached tenant {tenant_id} \ - should not be present in pageserver's memory" + ), f"Ignored and then detached tenant {tenant_id} should not be present in pageserver's memory" # Creates a tenant, and detaches it with extra paremeter that forces ignored tenant detach. @@ -430,8 +429,7 @@ def test_tenant_detach_regular_tenant(neon_simple_env: NeonEnv): tenants_after_detach = [tenant["id"] for tenant in client.tenant_list()] assert ( tenant_id not in tenants_after_detach - ), f"Ignored and then detached tenant {tenant_id} \ - should not be present in pageserver's memory" + ), f"Ignored and then detached tenant {tenant_id} should not be present in pageserver's memory" def test_detach_while_attaching( @@ -817,9 +815,7 @@ def test_metrics_while_ignoring_broken_tenant_and_reloading( if found_broken: break time.sleep(0.5) - assert ( - found_broken - ), f"broken should still be in set, but it is not in the tenant state count: broken={broken}, broken_set={broken_set}" + assert found_broken, f"broken should still be in set, but it is not in the tenant state count: broken={broken}, broken_set={broken_set}" env.pageserver.tenant_load(env.initial_tenant) @@ -837,6 +833,4 @@ def test_metrics_while_ignoring_broken_tenant_and_reloading( break time.sleep(0.5) - assert ( - found_active - ), f"reloaded tenant should be active, and broken tenant set item removed: active={active}, broken_set={broken_set}" + assert found_active, f"reloaded tenant should be active, and broken tenant set item removed: active={active}, broken_set={broken_set}" diff --git a/test_runner/regress/test_tenant_relocation.py b/test_runner/regress/test_tenant_relocation.py index dcd7232b1b..1887bca23b 100644 --- a/test_runner/regress/test_tenant_relocation.py +++ b/test_runner/regress/test_tenant_relocation.py @@ -161,12 +161,10 @@ def switch_pg_to_new_pageserver( files_before_detach = os.listdir(timeline_to_detach_local_path) assert ( "metadata" in files_before_detach - ), f"Regular timeline {timeline_to_detach_local_path} should have the metadata file,\ - but got: {files_before_detach}" + ), f"Regular timeline {timeline_to_detach_local_path} should have the metadata file, but got: {files_before_detach}" assert ( len(files_before_detach) >= 2 - ), f"Regular timeline {timeline_to_detach_local_path} should have at least one layer file,\ - but got {files_before_detach}" + ), f"Regular timeline {timeline_to_detach_local_path} should have at least one layer file, but got {files_before_detach}" return timeline_to_detach_local_path diff --git a/test_runner/regress/test_tenants_with_remote_storage.py b/test_runner/regress/test_tenants_with_remote_storage.py index 07fb6dc5ca..6f05d7f7cb 100644 --- a/test_runner/regress/test_tenants_with_remote_storage.py +++ b/test_runner/regress/test_tenants_with_remote_storage.py @@ -201,8 +201,8 @@ def test_tenants_attached_after_download(neon_env_builder: NeonEnvBuilder): len(restored_timelines) == 1 ), f"Tenant {tenant_id} should have its timeline reattached after its layer is downloaded from the remote storage" restored_timeline = restored_timelines[0] - assert restored_timeline["timeline_id"] == str( - timeline_id + assert ( + restored_timeline["timeline_id"] == str(timeline_id) ), f"Tenant {tenant_id} should have its old timeline {timeline_id} restored from the remote storage" # Check that we had to retry the downloads @@ -280,8 +280,8 @@ def test_tenant_redownloads_truncated_file_on_startup( len(restored_timelines) == 1 ), f"Tenant {tenant_id} should have its timeline reattached after its layer is downloaded from the remote storage" retored_timeline = restored_timelines[0] - assert retored_timeline["timeline_id"] == str( - timeline_id + assert ( + retored_timeline["timeline_id"] == str(timeline_id) ), f"Tenant {tenant_id} should have its old timeline {timeline_id} restored from the remote storage" # Request non-incremental logical size. Calculating it needs the layer file that From a41c4122e39ab151f709262329576f9a9b5a2db7 Mon Sep 17 00:00:00 2001 From: Arseny Sher Date: Thu, 4 Jan 2024 01:21:33 +0300 Subject: [PATCH 34/35] Don't suspend compute if there is active LR subscriber. https://github.com/neondatabase/neon/issues/6258 --- compute_tools/src/monitor.rs | 25 ++++++++++++++++++++++++- 1 file changed, 24 insertions(+), 1 deletion(-) diff --git a/compute_tools/src/monitor.rs b/compute_tools/src/monitor.rs index f974d6023d..fd19b7e53f 100644 --- a/compute_tools/src/monitor.rs +++ b/compute_tools/src/monitor.rs @@ -3,7 +3,7 @@ use std::{thread, time::Duration}; use chrono::{DateTime, Utc}; use postgres::{Client, NoTls}; -use tracing::{debug, info}; +use tracing::{debug, info, warn}; use crate::compute::ComputeNode; @@ -84,6 +84,29 @@ fn watch_compute_activity(compute: &ComputeNode) { } } + // If there are existing (logical) walsenders, do not suspend. + // + // walproposer doesn't currently show up in pg_stat_replication, + // but protect if it will be + let ws_count_query = "select count(*) from pg_stat_replication where application_name != 'walproposer';"; + match cli.query_one(ws_count_query, &[]) { + Ok(r) => match r.try_get::<&str, i64>("count") { + Ok(num_ws) => { + if num_ws > 0 { + last_active = Some(Utc::now()); + } + } + Err(e) => { + warn!("failed to parse ws count: {:?}", e); + continue; + } + }, + Err(e) => { + warn!("failed to get list of walsenders: {:?}", e); + continue; + } + } + // Update the last activity in the shared state if we got a more recent one. let mut state = compute.state.lock().unwrap(); // NB: `Some()` is always greater than `None`. From d1c0232e2190d1d0ddc9320223012196a8055ade Mon Sep 17 00:00:00 2001 From: John Spray Date: Mon, 8 Jan 2024 03:53:15 +0000 Subject: [PATCH 35/35] pageserver: use `pub(crate)` in metrics.rs, and clean up unused items (#6275) ## Problem Noticed while making other changes that there were `pub` items that were unused. ## Summary of changes - Make everything `pub(crate)` in metrics.rs, apart from items used from `bin/` - Fix the timelines eviction metric: it was never being incremented - Remove an unused ephemeral_bytes counter. --- pageserver/src/metrics.rs | 84 +++++++++----------- pageserver/src/tenant/storage_layer/layer.rs | 1 + 2 files changed, 40 insertions(+), 45 deletions(-) diff --git a/pageserver/src/metrics.rs b/pageserver/src/metrics.rs index c86adcfa3d..6f4431c3cf 100644 --- a/pageserver/src/metrics.rs +++ b/pageserver/src/metrics.rs @@ -29,7 +29,7 @@ const CRITICAL_OP_BUCKETS: &[f64] = &[ // Metrics collected on operations on the storage repository. #[derive(Debug, EnumVariantNames, IntoStaticStr)] #[strum(serialize_all = "kebab_case")] -pub enum StorageTimeOperation { +pub(crate) enum StorageTimeOperation { #[strum(serialize = "layer flush")] LayerFlush, @@ -55,7 +55,7 @@ pub enum StorageTimeOperation { CreateTenant, } -pub static STORAGE_TIME_SUM_PER_TIMELINE: Lazy = Lazy::new(|| { +pub(crate) static STORAGE_TIME_SUM_PER_TIMELINE: Lazy = Lazy::new(|| { register_counter_vec!( "pageserver_storage_operations_seconds_sum", "Total time spent on storage operations with operation, tenant and timeline dimensions", @@ -64,7 +64,7 @@ pub static STORAGE_TIME_SUM_PER_TIMELINE: Lazy = Lazy::new(|| { .expect("failed to define a metric") }); -pub static STORAGE_TIME_COUNT_PER_TIMELINE: Lazy = Lazy::new(|| { +pub(crate) static STORAGE_TIME_COUNT_PER_TIMELINE: Lazy = Lazy::new(|| { register_int_counter_vec!( "pageserver_storage_operations_seconds_count", "Count of storage operations with operation, tenant and timeline dimensions", @@ -150,7 +150,7 @@ pub(crate) static MATERIALIZED_PAGE_CACHE_HIT: Lazy = Lazy::new(|| { .expect("failed to define a metric") }); -pub struct PageCacheMetricsForTaskKind { +pub(crate) struct PageCacheMetricsForTaskKind { pub read_accesses_materialized_page: IntCounter, pub read_accesses_immutable: IntCounter, @@ -159,7 +159,7 @@ pub struct PageCacheMetricsForTaskKind { pub read_hits_materialized_page_older_lsn: IntCounter, } -pub struct PageCacheMetrics { +pub(crate) struct PageCacheMetrics { map: EnumMap>, } @@ -181,7 +181,7 @@ static PAGE_CACHE_READ_ACCESSES: Lazy = Lazy::new(|| { .expect("failed to define a metric") }); -pub static PAGE_CACHE: Lazy = Lazy::new(|| PageCacheMetrics { +pub(crate) static PAGE_CACHE: Lazy = Lazy::new(|| PageCacheMetrics { map: EnumMap::from_array(std::array::from_fn(|task_kind| { let task_kind = ::from_usize(task_kind); let task_kind: &'static str = task_kind.into(); @@ -243,10 +243,9 @@ impl PageCacheMetrics { } } -pub struct PageCacheSizeMetrics { +pub(crate) struct PageCacheSizeMetrics { pub max_bytes: UIntGauge, - pub current_bytes_ephemeral: UIntGauge, pub current_bytes_immutable: UIntGauge, pub current_bytes_materialized_page: UIntGauge, } @@ -260,31 +259,26 @@ static PAGE_CACHE_SIZE_CURRENT_BYTES: Lazy = Lazy::new(|| { .expect("failed to define a metric") }); -pub static PAGE_CACHE_SIZE: Lazy = Lazy::new(|| PageCacheSizeMetrics { - max_bytes: { - register_uint_gauge!( - "pageserver_page_cache_size_max_bytes", - "Maximum size of the page cache in bytes" - ) - .expect("failed to define a metric") - }, - - current_bytes_ephemeral: { - PAGE_CACHE_SIZE_CURRENT_BYTES - .get_metric_with_label_values(&["ephemeral"]) - .unwrap() - }, - current_bytes_immutable: { - PAGE_CACHE_SIZE_CURRENT_BYTES - .get_metric_with_label_values(&["immutable"]) - .unwrap() - }, - current_bytes_materialized_page: { - PAGE_CACHE_SIZE_CURRENT_BYTES - .get_metric_with_label_values(&["materialized_page"]) - .unwrap() - }, -}); +pub(crate) static PAGE_CACHE_SIZE: Lazy = + Lazy::new(|| PageCacheSizeMetrics { + max_bytes: { + register_uint_gauge!( + "pageserver_page_cache_size_max_bytes", + "Maximum size of the page cache in bytes" + ) + .expect("failed to define a metric") + }, + current_bytes_immutable: { + PAGE_CACHE_SIZE_CURRENT_BYTES + .get_metric_with_label_values(&["immutable"]) + .unwrap() + }, + current_bytes_materialized_page: { + PAGE_CACHE_SIZE_CURRENT_BYTES + .get_metric_with_label_values(&["materialized_page"]) + .unwrap() + }, + }); pub(crate) mod page_cache_eviction_metrics { use std::num::NonZeroUsize; @@ -740,13 +734,13 @@ pub(crate) static TENANT: Lazy = Lazy::new(|| { /// Each `Timeline`'s [`EVICTIONS_WITH_LOW_RESIDENCE_DURATION`] metric. #[derive(Debug)] -pub struct EvictionsWithLowResidenceDuration { +pub(crate) struct EvictionsWithLowResidenceDuration { data_source: &'static str, threshold: Duration, counter: Option, } -pub struct EvictionsWithLowResidenceDurationBuilder { +pub(crate) struct EvictionsWithLowResidenceDurationBuilder { data_source: &'static str, threshold: Duration, } @@ -1009,7 +1003,7 @@ pub enum SmgrQueryType { } #[derive(Debug)] -pub struct SmgrQueryTimePerTimeline { +pub(crate) struct SmgrQueryTimePerTimeline { metrics: [GlobalAndPerTimelineHistogram; SmgrQueryType::COUNT], } @@ -1181,8 +1175,8 @@ static COMPUTE_STARTUP_BUCKETS: Lazy<[f64; 28]> = Lazy::new(|| { .map(|ms| (ms as f64) / 1000.0) }); -pub struct BasebackupQueryTime(HistogramVec); -pub static BASEBACKUP_QUERY_TIME: Lazy = Lazy::new(|| { +pub(crate) struct BasebackupQueryTime(HistogramVec); +pub(crate) static BASEBACKUP_QUERY_TIME: Lazy = Lazy::new(|| { BasebackupQueryTime({ register_histogram_vec!( "pageserver_basebackup_query_seconds", @@ -1202,7 +1196,7 @@ impl DurationResultObserver for BasebackupQueryTime { } } -pub static LIVE_CONNECTIONS_COUNT: Lazy = Lazy::new(|| { +pub(crate) static LIVE_CONNECTIONS_COUNT: Lazy = Lazy::new(|| { register_int_gauge_vec!( "pageserver_live_connections", "Number of live network connections", @@ -1667,7 +1661,7 @@ pub(crate) static WAL_REDO_PROCESS_COUNTERS: Lazy = Lazy::new(WalRedoProcessCounters::default); /// Similar to `prometheus::HistogramTimer` but does not record on drop. -pub struct StorageTimeMetricsTimer { +pub(crate) struct StorageTimeMetricsTimer { metrics: StorageTimeMetrics, start: Instant, } @@ -1692,7 +1686,7 @@ impl StorageTimeMetricsTimer { /// Timing facilities for an globally histogrammed metric, which is supported by per tenant and /// timeline total sum and count. #[derive(Clone, Debug)] -pub struct StorageTimeMetrics { +pub(crate) struct StorageTimeMetrics { /// Sum of f64 seconds, per operation, tenant_id and timeline_id timeline_sum: Counter, /// Number of oeprations, per operation, tenant_id and timeline_id @@ -1731,7 +1725,7 @@ impl StorageTimeMetrics { } #[derive(Debug)] -pub struct TimelineMetrics { +pub(crate) struct TimelineMetrics { tenant_id: String, shard_id: String, timeline_id: String, @@ -1939,7 +1933,7 @@ impl Drop for PerTimelineRemotePhysicalSizeGauge { } } -pub struct RemoteTimelineClientMetrics { +pub(crate) struct RemoteTimelineClientMetrics { tenant_id: String, timeline_id: String, remote_physical_size_gauge: Mutex>, @@ -2237,7 +2231,7 @@ impl Drop for RemoteTimelineClientMetrics { /// Wrapper future that measures the time spent by a remote storage operation, /// and records the time and success/failure as a prometheus metric. -pub trait MeasureRemoteOp: Sized { +pub(crate) trait MeasureRemoteOp: Sized { fn measure_remote_op( self, tenant_id: TenantId, @@ -2262,7 +2256,7 @@ pub trait MeasureRemoteOp: Sized { impl MeasureRemoteOp for T {} pin_project! { - pub struct MeasuredRemoteOp + pub(crate) struct MeasuredRemoteOp { #[pin] inner: F, diff --git a/pageserver/src/tenant/storage_layer/layer.rs b/pageserver/src/tenant/storage_layer/layer.rs index 8ae911b31e..f5adf9d639 100644 --- a/pageserver/src/tenant/storage_layer/layer.rs +++ b/pageserver/src/tenant/storage_layer/layer.rs @@ -1118,6 +1118,7 @@ impl LayerInner { tracing::info!("evicted layer after unknown residence period"); } } + timeline.metrics.evictions.inc(); timeline .metrics .resident_physical_size_sub(self.desc.file_size);