From b80704cd34baae1746a98c43db8dcb672e08dcf5 Mon Sep 17 00:00:00 2001 From: John Spray Date: Tue, 19 Mar 2024 10:30:33 +0000 Subject: [PATCH 01/53] tests: log hygiene checks for storage controller (#6710) ## Problem As with the pageserver, we should fail tests that emit unexpected log errors/warnings. ## Summary of changes - Refactor existing log checks to be reusable - Run log checks for attachment_service - Add allow lists as needed. --- .../attachment_service/src/service.rs | 25 ++++++++++++- test_runner/fixtures/neon_fixtures.py | 26 ++++++------- .../fixtures/pageserver/allowed_errors.py | 10 +++++ test_runner/fixtures/utils.py | 37 +++++++++++++++++++ test_runner/regress/test_branch_and_gc.py | 12 +++--- test_runner/regress/test_branch_behind.py | 9 +++-- test_runner/regress/test_compatibility.py | 4 ++ test_runner/regress/test_sharding.py | 14 +++++++ test_runner/regress/test_sharding_service.py | 10 +++++ test_runner/regress/test_tenants.py | 4 +- 10 files changed, 126 insertions(+), 25 deletions(-) diff --git a/control_plane/attachment_service/src/service.rs b/control_plane/attachment_service/src/service.rs index 29f87021b20b..addfd9c232c1 100644 --- a/control_plane/attachment_service/src/service.rs +++ b/control_plane/attachment_service/src/service.rs @@ -7,7 +7,9 @@ use std::{ time::{Duration, Instant}, }; -use crate::{id_lock_map::IdLockMap, persistence::AbortShardSplitStatus}; +use crate::{ + id_lock_map::IdLockMap, persistence::AbortShardSplitStatus, reconciler::ReconcileError, +}; use anyhow::Context; use control_plane::storage_controller::{ AttachHookRequest, AttachHookResponse, InspectRequest, InspectResponse, @@ -733,7 +735,19 @@ impl Service { tenant.waiter.advance(result.sequence); } Err(e) => { - tracing::warn!("Reconcile error: {}", e); + match e { + ReconcileError::Cancel => { + tracing::info!("Reconciler was cancelled"); + } + ReconcileError::Remote(mgmt_api::Error::Cancelled) => { + // This might be due to the reconciler getting cancelled, or it might + // be due to the `Node` being marked offline. + tracing::info!("Reconciler cancelled during pageserver API call"); + } + _ => { + tracing::warn!("Reconcile error: {}", e); + } + } // Ordering: populate last_error before advancing error_seq, // so that waiters will see the correct error after waiting. @@ -3631,6 +3645,13 @@ impl Service { observed_loc.conf = None; } + if new_nodes.len() == 1 { + // Special case for single-node cluster: there is no point trying to reschedule + // any tenant shards: avoid doing so, in order to avoid spewing warnings about + // failures to schedule them. + continue; + } + if tenant_state.intent.demote_attached(node_id) { tenant_state.sequence = tenant_state.sequence.next(); match tenant_state.schedule(scheduler) { diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index 56b23cef5960..3ecd3432246f 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -51,7 +51,7 @@ from fixtures.metrics import Metrics, MetricsGetter, parse_metrics from fixtures.pageserver.allowed_errors import ( DEFAULT_PAGESERVER_ALLOWED_ERRORS, - scan_pageserver_log_for_errors, + DEFAULT_STORAGE_CONTROLLER_ALLOWED_ERRORS, ) from fixtures.pageserver.http import PageserverHttpClient from fixtures.pageserver.types import IndexPartDump @@ -77,6 +77,7 @@ ATTACHMENT_NAME_REGEX, allure_add_grafana_links, allure_attach_from_dir, + assert_no_errors, get_self_dir, subprocess_capture, wait_until, @@ -944,6 +945,8 @@ def __exit__( for pageserver in self.env.pageservers: pageserver.assert_no_errors() + self.env.storage_controller.assert_no_errors() + try: self.overlay_cleanup_teardown() except Exception as e: @@ -1961,6 +1964,7 @@ def __init__(self, env: NeonEnv, auth_enabled: bool): self.env = env self.running = False self.auth_enabled = auth_enabled + self.allowed_errors: list[str] = DEFAULT_STORAGE_CONTROLLER_ALLOWED_ERRORS def start(self): assert not self.running @@ -1985,6 +1989,11 @@ def raise_api_exception(res: requests.Response): msg = "" raise StorageControllerApiException(msg, res.status_code) from e + def assert_no_errors(self): + assert_no_errors( + self.env.repo_dir / "storage_controller.log", "storage_controller", self.allowed_errors + ) + def pageserver_api(self) -> PageserverHttpClient: """ The storage controller implements a subset of the pageserver REST API, for mapping @@ -2357,18 +2366,9 @@ def workdir(self) -> Path: return self.env.repo_dir / f"pageserver_{self.id}" def assert_no_errors(self): - logfile = self.workdir / "pageserver.log" - if not logfile.exists(): - log.warning(f"Skipping log check: {logfile} does not exist") - return - - with logfile.open("r") as f: - errors = scan_pageserver_log_for_errors(f, self.allowed_errors) - - for _lineno, error in errors: - log.info(f"not allowed error: {error.strip()}") - - assert not errors + assert_no_errors( + self.workdir / "pageserver.log", f"pageserver_{self.id}", self.allowed_errors + ) def assert_no_metric_errors(self): """ diff --git a/test_runner/fixtures/pageserver/allowed_errors.py b/test_runner/fixtures/pageserver/allowed_errors.py index 839d4166c73a..ec0f81b380e5 100755 --- a/test_runner/fixtures/pageserver/allowed_errors.py +++ b/test_runner/fixtures/pageserver/allowed_errors.py @@ -89,6 +89,16 @@ def scan_pageserver_log_for_errors( ) +DEFAULT_STORAGE_CONTROLLER_ALLOWED_ERRORS = [ + # Many tests will take pageservers offline, resulting in log warnings on the controller + # failing to connect to them. + ".*Call to node.*management API.*failed.*receive body.*", + ".*Call to node.*management API.*failed.*ReceiveBody.*", + # Many tests will start up with a node offline + ".*startup_reconcile: Could not scan node.*", +] + + def _check_allowed_errors(input): allowed_errors: List[str] = list(DEFAULT_PAGESERVER_ALLOWED_ERRORS) diff --git a/test_runner/fixtures/utils.py b/test_runner/fixtures/utils.py index 7fc3bae3afd6..9365d65fc9e2 100644 --- a/test_runner/fixtures/utils.py +++ b/test_runner/fixtures/utils.py @@ -11,6 +11,7 @@ Any, Callable, Dict, + Iterable, List, Optional, Tuple, @@ -447,3 +448,39 @@ def humantime_to_ms(humantime: str) -> float: ) return round(total_ms, 3) + + +def scan_log_for_errors(input: Iterable[str], allowed_errors: List[str]) -> List[Tuple[int, str]]: + error_or_warn = re.compile(r"\s(ERROR|WARN)") + errors = [] + for lineno, line in enumerate(input, start=1): + if len(line) == 0: + continue + + if error_or_warn.search(line): + # Is this a torn log line? This happens when force-killing a process and restarting + # Example: "2023-10-25T09:38:31.752314Z WARN deletion executo2023-10-25T09:38:31.875947Z INFO version: git-env:0f9452f76e8ccdfc88291bccb3f53e3016f40192" + if re.match("\\d{4}-\\d{2}-\\d{2}T.+\\d{4}-\\d{2}-\\d{2}T.+INFO version.+", line): + continue + + # It's an ERROR or WARN. Is it in the allow-list? + for a in allowed_errors: + if re.match(a, line): + break + else: + errors.append((lineno, line)) + return errors + + +def assert_no_errors(log_file, service, allowed_errors): + if not log_file.exists(): + log.warning(f"Skipping {service} log check: {log_file} does not exist") + return + + with log_file.open("r") as f: + errors = scan_log_for_errors(f, allowed_errors) + + for _lineno, error in errors: + log.info(f"not allowed {service} error: {error.strip()}") + + assert not errors, f"Log errors on {service}: {errors[0]}" diff --git a/test_runner/regress/test_branch_and_gc.py b/test_runner/regress/test_branch_and_gc.py index bdc944f35218..ddd02238eaa1 100644 --- a/test_runner/regress/test_branch_and_gc.py +++ b/test_runner/regress/test_branch_and_gc.py @@ -120,12 +120,12 @@ def test_branch_creation_before_gc(neon_simple_env: NeonEnv): env = neon_simple_env pageserver_http_client = env.pageserver.http_client() - env.pageserver.allowed_errors.extend( - [ - ".*invalid branch start lsn: less than latest GC cutoff.*", - ".*invalid branch start lsn: less than planned GC cutoff.*", - ] - ) + error_regexes = [ + ".*invalid branch start lsn: less than latest GC cutoff.*", + ".*invalid branch start lsn: less than planned GC cutoff.*", + ] + env.pageserver.allowed_errors.extend(error_regexes) + env.storage_controller.allowed_errors.extend(error_regexes) # Disable background GC but set the `pitr_interval` to be small, so GC can delete something tenant, _ = env.neon_cli.create_tenant( diff --git a/test_runner/regress/test_branch_behind.py b/test_runner/regress/test_branch_behind.py index 46c74a26b897..b79cad979ff6 100644 --- a/test_runner/regress/test_branch_behind.py +++ b/test_runner/regress/test_branch_behind.py @@ -14,9 +14,12 @@ def test_branch_behind(neon_env_builder: NeonEnvBuilder): neon_env_builder.pageserver_config_override = "tenant_config={pitr_interval = '0 sec'}" env = neon_env_builder.init_start() - env.pageserver.allowed_errors.extend( - [".*invalid branch start lsn.*", ".*invalid start lsn .* for ancestor timeline.*"] - ) + error_regexes = [ + ".*invalid branch start lsn.*", + ".*invalid start lsn .* for ancestor timeline.*", + ] + env.pageserver.allowed_errors.extend(error_regexes) + env.storage_controller.allowed_errors.extend(error_regexes) # Branch at the point where only 100 rows were inserted branch_behind_timeline_id = env.neon_cli.create_branch("test_branch_behind") diff --git a/test_runner/regress/test_compatibility.py b/test_runner/regress/test_compatibility.py index 5f815d3e6c52..e0bb4c206281 100644 --- a/test_runner/regress/test_compatibility.py +++ b/test_runner/regress/test_compatibility.py @@ -238,6 +238,10 @@ def test_forward_compatibility( pg_distrib_dir=compatibility_postgres_distrib_dir, ) + # TODO: remove this workaround after release-5090 is no longer the most recent release. + # There was a bug in that code that generates a warning in the storage controller log. + env.storage_controller.allowed_errors.append(".*no tenant_shard_id specified.*") + # Use current neon_local even though we're using old binaries for # everything else: our test code is written for latest CLI args. env.neon_local_binpath = neon_local_binpath diff --git a/test_runner/regress/test_sharding.py b/test_runner/regress/test_sharding.py index 9e62933f7eb5..3470d2e6090f 100644 --- a/test_runner/regress/test_sharding.py +++ b/test_runner/regress/test_sharding.py @@ -725,6 +725,20 @@ def test_sharding_split_failures( tenant_id = env.initial_tenant timeline_id = env.initial_timeline + env.storage_controller.allowed_errors.extend( + [ + # All split failures log a warning when then enqueue the abort operation + ".*Enqueuing background abort.*", + # We exercise failure cases where abort itself will also fail (node offline) + ".*abort_tenant_shard_split.*", + ".*Failed to abort.*", + # Tolerate any error lots that mention a failpoint + ".*failpoint.*", + # Node offline cases will fail to send requests + ".*Reconcile error: receive body: error sending request for url.*", + ] + ) + for ps in env.pageservers: # When we do node failures and abandon a shard, it will de-facto have old generation and # thereby be unable to publish remote consistent LSN updates diff --git a/test_runner/regress/test_sharding_service.py b/test_runner/regress/test_sharding_service.py index 27ea425bb15f..a6b0f76c9613 100644 --- a/test_runner/regress/test_sharding_service.py +++ b/test_runner/regress/test_sharding_service.py @@ -177,6 +177,7 @@ def test_node_status_after_restart( assert len(nodes) == 2 env.pageservers[1].stop() + env.storage_controller.allowed_errors.extend([".*Could not scan node"]) env.storage_controller.stop() env.storage_controller.start() @@ -681,6 +682,9 @@ def test_sharding_service_auth(neon_env_builder: NeonEnvBuilder): tenant_id = TenantId.generate() body: Dict[str, Any] = {"new_tenant_id": str(tenant_id)} + env.storage_controller.allowed_errors.append(".*Unauthorized.*") + env.storage_controller.allowed_errors.append(".*Forbidden.*") + # No token with pytest.raises( StorageControllerApiException, @@ -843,6 +847,12 @@ def test_sharding_service_heartbeats( env = neon_env_builder.init_configs() env.start() + # Default log allow list permits connection errors, but this test will use error responses on + # the utilization endpoint. + env.storage_controller.allowed_errors.append( + ".*Call to node.*management API.*failed.*failpoint.*" + ) + # Initially we have two online pageservers nodes = env.storage_controller.node_list() assert len(nodes) == 2 diff --git a/test_runner/regress/test_tenants.py b/test_runner/regress/test_tenants.py index 1e13a2f20f95..f8701b65d7d9 100644 --- a/test_runner/regress/test_tenants.py +++ b/test_runner/regress/test_tenants.py @@ -36,7 +36,9 @@ def test_tenant_creation_fails(neon_simple_env: NeonEnv): ) [d for d in tenants_dir.iterdir()] - neon_simple_env.pageserver.allowed_errors.append(".*tenant-config-before-write.*") + error_regexes = [".*tenant-config-before-write.*"] + neon_simple_env.pageserver.allowed_errors.extend(error_regexes) + neon_simple_env.storage_controller.allowed_errors.extend(error_regexes) pageserver_http = neon_simple_env.pageserver.http_client() pageserver_http.configure_failpoints(("tenant-config-before-write", "return")) From a8384a074e658193d1c4005763153898358b9d18 Mon Sep 17 00:00:00 2001 From: Alex Chi Z Date: Tue, 19 Mar 2024 10:43:24 -0400 Subject: [PATCH 02/53] fixup(#7168): neon_local: use pageserver defaults for known but unspecified config overrides (#7166) e2e tests cannot run on macOS unless the file engine env var is supplied. ``` ./scripts/pytest test_runner/regress/test_neon_superuser.py -s ``` will fail with tokio-epoll-uring not supported. This is because we persist the file engine config by default. In this pull request, we only persist when someone specifies it, so that it can use the default platform-variant config in the page server. --------- Signed-off-by: Alex Chi Z --- control_plane/src/local_env.rs | 9 ++++----- control_plane/src/pageserver.rs | 12 ++++++++++-- 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/control_plane/src/local_env.rs b/control_plane/src/local_env.rs index c7f22cc8f809..bd3dbef4530c 100644 --- a/control_plane/src/local_env.rs +++ b/control_plane/src/local_env.rs @@ -127,8 +127,8 @@ pub struct PageServerConf { pub pg_auth_type: AuthType, pub http_auth_type: AuthType, - pub(crate) virtual_file_io_engine: String, - pub(crate) get_vectored_impl: String, + pub(crate) virtual_file_io_engine: Option, + pub(crate) get_vectored_impl: Option, } impl Default for PageServerConf { @@ -139,9 +139,8 @@ impl Default for PageServerConf { listen_http_addr: String::new(), pg_auth_type: AuthType::Trust, http_auth_type: AuthType::Trust, - // FIXME: use the ones exposed by pageserver crate - virtual_file_io_engine: "tokio-epoll-uring".to_owned(), - get_vectored_impl: "sequential".to_owned(), + virtual_file_io_engine: None, + get_vectored_impl: None, } } } diff --git a/control_plane/src/pageserver.rs b/control_plane/src/pageserver.rs index 2603515681c0..c5eabc46db81 100644 --- a/control_plane/src/pageserver.rs +++ b/control_plane/src/pageserver.rs @@ -101,8 +101,16 @@ impl PageServerNode { let pg_auth_type_param = format!("pg_auth_type='{}'", pg_auth_type); let listen_pg_addr_param = format!("listen_pg_addr='{}'", listen_pg_addr); - let virtual_file_io_engine = format!("virtual_file_io_engine='{virtual_file_io_engine}'"); - let get_vectored_impl = format!("get_vectored_impl='{get_vectored_impl}'"); + let virtual_file_io_engine = if let Some(virtual_file_io_engine) = virtual_file_io_engine { + format!("virtual_file_io_engine='{virtual_file_io_engine}'") + } else { + String::new() + }; + let get_vectored_impl = if let Some(get_vectored_impl) = get_vectored_impl { + format!("get_vectored_impl='{get_vectored_impl}'") + } else { + String::new() + }; let broker_endpoint_param = format!("broker_endpoint='{}'", self.env.broker.client_url()); From 64c6dfd3e44c4550604e6b97678140afa93f4409 Mon Sep 17 00:00:00 2001 From: Tristan Partin Date: Thu, 14 Mar 2024 14:35:34 -0500 Subject: [PATCH 03/53] Move functions for creating/extracting tarballs into utils Useful for other code paths which will handle zstd compression and decompression. --- Cargo.lock | 3 ++ libs/utils/Cargo.toml | 3 ++ libs/utils/src/lib.rs | 2 + libs/utils/src/zstd.rs | 78 ++++++++++++++++++++++++++++++++ pageserver/src/import_datadir.rs | 72 +---------------------------- pageserver/src/tenant.rs | 13 ++++-- 6 files changed, 97 insertions(+), 74 deletions(-) create mode 100644 libs/utils/src/zstd.rs diff --git a/Cargo.lock b/Cargo.lock index c4f925e3c762..70f427f97d62 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6468,6 +6468,7 @@ version = "0.1.0" dependencies = [ "anyhow", "arc-swap", + "async-compression", "async-trait", "bincode", "byteorder", @@ -6506,12 +6507,14 @@ dependencies = [ "thiserror", "tokio", "tokio-stream", + "tokio-tar", "tokio-util", "tracing", "tracing-error", "tracing-subscriber", "url", "uuid", + "walkdir", "workspace_hack", ] diff --git a/libs/utils/Cargo.toml b/libs/utils/Cargo.toml index 983e94d96383..c2d9d9d39677 100644 --- a/libs/utils/Cargo.toml +++ b/libs/utils/Cargo.toml @@ -13,6 +13,7 @@ testing = ["fail/failpoints"] [dependencies] arc-swap.workspace = true sentry.workspace = true +async-compression.workspace = true async-trait.workspace = true anyhow.workspace = true bincode.workspace = true @@ -36,6 +37,7 @@ serde_json.workspace = true signal-hook.workspace = true thiserror.workspace = true tokio.workspace = true +tokio-tar.workspace = true tokio-util.workspace = true tracing.workspace = true tracing-error.workspace = true @@ -46,6 +48,7 @@ strum.workspace = true strum_macros.workspace = true url.workspace = true uuid.workspace = true +walkdir.workspace = true pq_proto.workspace = true postgres_connection.workspace = true diff --git a/libs/utils/src/lib.rs b/libs/utils/src/lib.rs index 890061dc59b6..04ce0626c84a 100644 --- a/libs/utils/src/lib.rs +++ b/libs/utils/src/lib.rs @@ -87,6 +87,8 @@ pub mod failpoint_support; pub mod yielding_loop; +pub mod zstd; + /// 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/zstd.rs b/libs/utils/src/zstd.rs new file mode 100644 index 000000000000..be2dcc00f567 --- /dev/null +++ b/libs/utils/src/zstd.rs @@ -0,0 +1,78 @@ +use std::io::SeekFrom; + +use anyhow::{Context, Result}; +use async_compression::{ + tokio::{bufread::ZstdDecoder, write::ZstdEncoder}, + zstd::CParameter, + Level, +}; +use camino::Utf8Path; +use nix::NixPath; +use tokio::{ + fs::{File, OpenOptions}, + io::AsyncBufRead, + io::AsyncSeekExt, + io::AsyncWriteExt, +}; +use tokio_tar::{Archive, Builder, HeaderMode}; +use walkdir::WalkDir; + +/// Creates a Zstandard tarball. +pub async fn create_zst_tarball(path: &Utf8Path, tarball: &Utf8Path) -> Result<(File, u64)> { + let file = OpenOptions::new() + .create(true) + .truncate(true) + .read(true) + .write(true) + .open(&tarball) + .await + .with_context(|| format!("tempfile creation {tarball}"))?; + + let mut paths = Vec::new(); + for entry in WalkDir::new(path) { + let entry = entry?; + let metadata = entry.metadata().expect("error getting dir entry metadata"); + // Also allow directories so that we also get empty directories + if !(metadata.is_file() || metadata.is_dir()) { + continue; + } + let path = entry.into_path(); + paths.push(path); + } + // Do a sort to get a more consistent listing + paths.sort_unstable(); + let zstd = ZstdEncoder::with_quality_and_params( + file, + Level::Default, + &[CParameter::enable_long_distance_matching(true)], + ); + let mut builder = Builder::new(zstd); + // Use reproducible header mode + builder.mode(HeaderMode::Deterministic); + for p in paths { + let rel_path = p.strip_prefix(path)?; + if rel_path.is_empty() { + // The top directory should not be compressed, + // the tar crate doesn't like that + continue; + } + builder.append_path_with_name(&p, rel_path).await?; + } + let mut zstd = builder.into_inner().await?; + zstd.shutdown().await?; + let mut compressed = zstd.into_inner(); + let compressed_len = compressed.metadata().await?.len(); + compressed.seek(SeekFrom::Start(0)).await?; + Ok((compressed, compressed_len)) +} + +/// Creates a Zstandard tarball. +pub async fn extract_zst_tarball( + path: &Utf8Path, + tarball: impl AsyncBufRead + Unpin, +) -> Result<()> { + let decoder = Box::pin(ZstdDecoder::new(tarball)); + let mut archive = Archive::new(decoder); + archive.unpack(path).await?; + Ok(()) +} diff --git a/pageserver/src/import_datadir.rs b/pageserver/src/import_datadir.rs index d66df36b3a3e..343dec2ca173 100644 --- a/pageserver/src/import_datadir.rs +++ b/pageserver/src/import_datadir.rs @@ -2,28 +2,20 @@ //! Import data and WAL from a PostgreSQL data directory and WAL segments into //! a neon Timeline. //! -use std::io::SeekFrom; use std::path::{Path, PathBuf}; use anyhow::{bail, ensure, Context, Result}; -use async_compression::tokio::bufread::ZstdDecoder; -use async_compression::{tokio::write::ZstdEncoder, zstd::CParameter, Level}; use bytes::Bytes; use camino::Utf8Path; use futures::StreamExt; -use nix::NixPath; -use tokio::fs::{File, OpenOptions}; -use tokio::io::{AsyncBufRead, AsyncRead, AsyncReadExt, AsyncSeekExt, AsyncWriteExt}; +use tokio::io::{AsyncRead, AsyncReadExt}; use tokio_tar::Archive; -use tokio_tar::Builder; -use tokio_tar::HeaderMode; 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; use crate::walingest::WalIngest; use crate::walrecord::DecodedWALRecord; @@ -633,65 +625,3 @@ async fn read_all_bytes(reader: &mut (impl AsyncRead + Unpin)) -> Result reader.read_to_end(&mut buf).await?; Ok(Bytes::from(buf)) } - -pub async fn create_tar_zst(pgdata_path: &Utf8Path, tmp_path: &Utf8Path) -> Result<(File, u64)> { - let file = OpenOptions::new() - .create(true) - .truncate(true) - .read(true) - .write(true) - .open(&tmp_path) - .await - .with_context(|| format!("tempfile creation {tmp_path}"))?; - - let mut paths = Vec::new(); - for entry in WalkDir::new(pgdata_path) { - let entry = entry?; - let metadata = entry.metadata().expect("error getting dir entry metadata"); - // Also allow directories so that we also get empty directories - if !(metadata.is_file() || metadata.is_dir()) { - continue; - } - let path = entry.into_path(); - paths.push(path); - } - // Do a sort to get a more consistent listing - paths.sort_unstable(); - let zstd = ZstdEncoder::with_quality_and_params( - file, - Level::Default, - &[CParameter::enable_long_distance_matching(true)], - ); - let mut builder = Builder::new(zstd); - // Use reproducible header mode - builder.mode(HeaderMode::Deterministic); - for path in paths { - let rel_path = path.strip_prefix(pgdata_path)?; - if rel_path.is_empty() { - // The top directory should not be compressed, - // the tar crate doesn't like that - continue; - } - builder.append_path_with_name(&path, rel_path).await?; - } - let mut zstd = builder.into_inner().await?; - zstd.shutdown().await?; - let mut compressed = zstd.into_inner(); - let compressed_len = compressed.metadata().await?.len(); - const INITDB_TAR_ZST_WARN_LIMIT: u64 = 2 * 1024 * 1024; - if compressed_len > INITDB_TAR_ZST_WARN_LIMIT { - warn!("compressed {INITDB_PATH} size of {compressed_len} is above limit {INITDB_TAR_ZST_WARN_LIMIT}."); - } - compressed.seek(SeekFrom::Start(0)).await?; - Ok((compressed, compressed_len)) -} - -pub async fn extract_tar_zst( - pgdata_path: &Utf8Path, - tar_zst: impl AsyncBufRead + Unpin, -) -> Result<()> { - let tar = Box::pin(ZstdDecoder::new(tar_zst)); - let mut archive = Archive::new(tar); - archive.unpack(pgdata_path).await?; - Ok(()) -} diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index ddfb47369bf2..7a6ddd6a4edc 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -43,6 +43,8 @@ use utils::sync::gate::Gate; use utils::sync::gate::GateGuard; use utils::timeout::timeout_cancellable; use utils::timeout::TimeoutCancellableError; +use utils::zstd::create_zst_tarball; +use utils::zstd::extract_zst_tarball; use self::config::AttachedLocationConfig; use self::config::AttachmentMode; @@ -3042,8 +3044,13 @@ impl Tenant { } } - let (pgdata_zstd, tar_zst_size) = - import_datadir::create_tar_zst(pgdata_path, &temp_path).await?; + let (pgdata_zstd, tar_zst_size) = create_zst_tarball(pgdata_path, &temp_path).await?; + const INITDB_TAR_ZST_WARN_LIMIT: u64 = 2 * 1024 * 1024; + if tar_zst_size > INITDB_TAR_ZST_WARN_LIMIT { + warn!( + "compressed {temp_path} size of {tar_zst_size} is above limit {INITDB_TAR_ZST_WARN_LIMIT}." + ); + } pausable_failpoint!("before-initdb-upload"); @@ -3143,7 +3150,7 @@ impl Tenant { let buf_read = BufReader::with_capacity(remote_timeline_client::BUFFER_SIZE, initdb_tar_zst); - import_datadir::extract_tar_zst(&pgdata_path, buf_read) + extract_zst_tarball(&pgdata_path, buf_read) .await .context("extract initdb tar")?; } else { From a5d5c2a6a0c0e9da4ccbcd8e44dc97559eeee8c9 Mon Sep 17 00:00:00 2001 From: John Spray Date: Tue, 19 Mar 2024 16:08:20 +0000 Subject: [PATCH 04/53] storage controller: tech debt (#7165) This is a mixed bag of changes split out for separate review while working on other things, and batched together to reduce load on CI runners. Each commits stands alone for review purposes: - do_tenant_shard_split was a long function and had a synchronous validation phase at the start that could readily be pulled out into a separate function. This also avoids the special casing of ApiError::BadRequest when deciding whether an abort is needed on errors - Add a 'describe' API (GET on tenant ID) that will enable storcon-cli to see what's going on with a tenant - the 'locate' API wasn't really meant for use in the field. It's for tests: demote it to the /debug/ prefix - The `Single` placement policy was a redundant duplicate of Double(0), and Double was a bad name. Rename it Attached. (https://github.com/neondatabase/neon/issues/7107) - Some neon_local commands were added for debug/demos, which are now replaced by commands in storcon-cli (#7114 ). Even though that's not merged yet, we don't need the neon_local ones any more. Closes https://github.com/neondatabase/neon/issues/7107 ## Backward compat of Single/Double -> `Attached(n)` change A database migration is used to convert any existing values. --- .../2024-03-18-184429_rename_policy/down.sql | 3 + .../2024-03-18-184429_rename_policy/up.sql | 3 + control_plane/attachment_service/src/http.rs | 19 +- .../attachment_service/src/persistence.rs | 13 +- .../attachment_service/src/reconciler.rs | 2 +- .../attachment_service/src/service.rs | 184 +++++++++++++----- .../attachment_service/src/tenant_state.rs | 21 +- control_plane/src/bin/neon_local.rs | 97 +-------- control_plane/src/storage_controller.rs | 2 +- libs/pageserver_api/src/controller_api.rs | 45 ++++- test_runner/fixtures/neon_fixtures.py | 15 +- test_runner/fixtures/types.py | 3 + .../regress/test_pageserver_secondary.py | 2 +- test_runner/regress/test_sharding.py | 4 +- 14 files changed, 206 insertions(+), 207 deletions(-) create mode 100644 control_plane/attachment_service/migrations/2024-03-18-184429_rename_policy/down.sql create mode 100644 control_plane/attachment_service/migrations/2024-03-18-184429_rename_policy/up.sql diff --git a/control_plane/attachment_service/migrations/2024-03-18-184429_rename_policy/down.sql b/control_plane/attachment_service/migrations/2024-03-18-184429_rename_policy/down.sql new file mode 100644 index 000000000000..897c7e0d0145 --- /dev/null +++ b/control_plane/attachment_service/migrations/2024-03-18-184429_rename_policy/down.sql @@ -0,0 +1,3 @@ + +UPDATE tenant_shards set placement_policy='{"Double": 1}' where placement_policy='{"Attached": 1}'; +UPDATE tenant_shards set placement_policy='"Single"' where placement_policy='{"Attached": 0}'; \ No newline at end of file diff --git a/control_plane/attachment_service/migrations/2024-03-18-184429_rename_policy/up.sql b/control_plane/attachment_service/migrations/2024-03-18-184429_rename_policy/up.sql new file mode 100644 index 000000000000..c898ac9aee07 --- /dev/null +++ b/control_plane/attachment_service/migrations/2024-03-18-184429_rename_policy/up.sql @@ -0,0 +1,3 @@ + +UPDATE tenant_shards set placement_policy='{"Attached": 1}' where placement_policy='{"Double": 1}'; +UPDATE tenant_shards set placement_policy='{"Attached": 0}' where placement_policy='"Single"'; \ No newline at end of file diff --git a/control_plane/attachment_service/src/http.rs b/control_plane/attachment_service/src/http.rs index 45ee354822fd..076b3a2f70c5 100644 --- a/control_plane/attachment_service/src/http.rs +++ b/control_plane/attachment_service/src/http.rs @@ -353,6 +353,16 @@ async fn handle_tenant_locate( json_response(StatusCode::OK, service.tenant_locate(tenant_id)?) } +async fn handle_tenant_describe( + service: Arc, + req: Request, +) -> Result, ApiError> { + check_permissions(&req, Scope::Admin)?; + + let tenant_id: TenantId = parse_request_param(&req, "tenant_id")?; + json_response(StatusCode::OK, service.tenant_describe(tenant_id)?) +} + async fn handle_node_register(mut req: Request) -> Result, ApiError> { check_permissions(&req, Scope::Admin)?; @@ -559,6 +569,9 @@ pub fn make_router( request_span(r, handle_node_drop) }) .get("/debug/v1/tenant", |r| request_span(r, handle_tenants_dump)) + .get("/debug/v1/tenant/:tenant_id/locate", |r| { + tenant_service_handler(r, handle_tenant_locate) + }) .get("/debug/v1/scheduler", |r| { request_span(r, handle_scheduler_dump) }) @@ -568,9 +581,6 @@ pub fn make_router( .put("/debug/v1/failpoints", |r| { request_span(r, |r| failpoints_handler(r, CancellationToken::new())) }) - .get("/control/v1/tenant/:tenant_id/locate", |r| { - tenant_service_handler(r, handle_tenant_locate) - }) // Node operations .post("/control/v1/node", |r| { request_span(r, handle_node_register) @@ -586,6 +596,9 @@ pub fn make_router( .put("/control/v1/tenant/:tenant_id/shard_split", |r| { tenant_service_handler(r, handle_tenant_shard_split) }) + .get("/control/v1/tenant/:tenant_id", |r| { + tenant_service_handler(r, handle_tenant_describe) + }) // Tenant operations // The ^/v1/ endpoints act as a "Virtual Pageserver", enabling shard-naive clients to call into // this service to manage tenants that actually consist of many tenant shards, as if they are a single entity. diff --git a/control_plane/attachment_service/src/persistence.rs b/control_plane/attachment_service/src/persistence.rs index 3602cf8b1f21..209d8ff0752a 100644 --- a/control_plane/attachment_service/src/persistence.rs +++ b/control_plane/attachment_service/src/persistence.rs @@ -211,15 +211,10 @@ impl Persistence { let mut decoded = serde_json::from_slice::(&bytes) .map_err(|e| DatabaseError::Logical(format!("Deserialization error: {e}")))?; - for (tenant_id, tenant) in &mut decoded.tenants { - // Backward compat: an old attachments.json from before PR #6251, replace - // empty strings with proper defaults. - if tenant.tenant_id.is_empty() { - tenant.tenant_id = tenant_id.to_string(); - tenant.config = serde_json::to_string(&TenantConfig::default()) - .map_err(|e| DatabaseError::Logical(format!("Serialization error: {e}")))?; - tenant.placement_policy = serde_json::to_string(&PlacementPolicy::Single) - .map_err(|e| DatabaseError::Logical(format!("Serialization error: {e}")))?; + for shard in decoded.tenants.values_mut() { + if shard.placement_policy == "\"Single\"" { + // Backward compat for test data after PR https://github.com/neondatabase/neon/pull/7165 + shard.placement_policy = "{\"Attached\":0}".to_string(); } } diff --git a/control_plane/attachment_service/src/reconciler.rs b/control_plane/attachment_service/src/reconciler.rs index 3bf23275bd73..f00f35c74bc2 100644 --- a/control_plane/attachment_service/src/reconciler.rs +++ b/control_plane/attachment_service/src/reconciler.rs @@ -475,7 +475,7 @@ impl Reconciler { } } - // Downgrade the origin to secondary. If the tenant's policy is PlacementPolicy::Single, then + // Downgrade the origin to secondary. If the tenant's policy is PlacementPolicy::Attached(0), then // this location will be deleted in the general case reconciliation that runs after this. let origin_secondary_conf = build_location_config( &self.shard, diff --git a/control_plane/attachment_service/src/service.rs b/control_plane/attachment_service/src/service.rs index addfd9c232c1..e38007c7afdd 100644 --- a/control_plane/attachment_service/src/service.rs +++ b/control_plane/attachment_service/src/service.rs @@ -20,8 +20,9 @@ use hyper::StatusCode; use pageserver_api::{ controller_api::{ NodeAvailability, NodeRegisterRequest, NodeSchedulingPolicy, PlacementPolicy, - TenantCreateResponse, TenantCreateResponseShard, TenantLocateResponse, - TenantShardMigrateRequest, TenantShardMigrateResponse, UtilizationScore, + TenantCreateResponse, TenantCreateResponseShard, TenantDescribeResponse, + TenantDescribeResponseShard, TenantLocateResponse, TenantShardMigrateRequest, + TenantShardMigrateResponse, UtilizationScore, }, models::{SecondaryProgress, TenantConfigRequest}, }; @@ -202,6 +203,29 @@ enum TenantCreateOrUpdate { Update(Vec), } +struct ShardSplitParams { + old_shard_count: ShardCount, + new_shard_count: ShardCount, + new_stripe_size: Option, + targets: Vec, + policy: PlacementPolicy, + shard_ident: ShardIdentity, +} + +// When preparing for a shard split, we may either choose to proceed with the split, +// or find that the work is already done and return NoOp. +enum ShardSplitAction { + Split(ShardSplitParams), + NoOp(TenantShardSplitResponse), +} + +// A parent shard which will be split +struct ShardSplitTarget { + parent_id: TenantShardId, + node: Node, + child_ids: Vec, +} + /// When we tenant shard split operation fails, we may not be able to clean up immediately, because nodes /// might not be available. We therefore use a queue of abort operations processed in the background. struct TenantShardSplitAbort { @@ -1071,7 +1095,7 @@ impl Service { shard_stripe_size: 0, generation: Some(0), generation_pageserver: None, - placement_policy: serde_json::to_string(&PlacementPolicy::Single).unwrap(), + placement_policy: serde_json::to_string(&PlacementPolicy::Attached(0)).unwrap(), config: serde_json::to_string(&TenantConfig::default()).unwrap(), splitting: SplitState::default(), }; @@ -1098,7 +1122,7 @@ impl Service { TenantState::new( attach_req.tenant_shard_id, ShardIdentity::unsharded(), - PlacementPolicy::Single, + PlacementPolicy::Attached(0), ), ); tracing::info!("Inserted shard {} in memory", attach_req.tenant_shard_id); @@ -1127,7 +1151,7 @@ impl Service { self.persistence .update_tenant_shard( attach_req.tenant_shard_id, - PlacementPolicy::Single, + PlacementPolicy::Attached(0), conf, None, ) @@ -1152,7 +1176,7 @@ impl Service { if let Some(new_generation) = new_generation { tenant_state.generation = Some(new_generation); - tenant_state.policy = PlacementPolicy::Single; + tenant_state.policy = PlacementPolicy::Attached(0); } else { // This is a detach notification. We must update placement policy to avoid re-attaching // during background scheduling/reconciliation, or during storage controller restart. @@ -1505,11 +1529,11 @@ impl Service { &self, create_req: TenantCreateRequest, ) -> Result<(TenantCreateResponse, Vec), ApiError> { - // As a default, single is convenient for tests that don't choose a policy. let placement_policy = create_req .placement_policy .clone() - .unwrap_or(PlacementPolicy::Single); + // As a default, zero secondaries is convenient for tests that don't choose a policy. + .unwrap_or(PlacementPolicy::Attached(0)); // This service expects to handle sharding itself: it is an error to try and directly create // a particular shard here. @@ -1719,11 +1743,11 @@ impl Service { | LocationConfigMode::AttachedSingle | LocationConfigMode::AttachedStale => { if nodes.len() > 1 { - PlacementPolicy::Double(1) + PlacementPolicy::Attached(1) } else { // Convenience for dev/test: if we just have one pageserver, import - // tenants into Single mode so that scheduling will succeed. - PlacementPolicy::Single + // tenants into non-HA mode so that scheduling will succeed. + PlacementPolicy::Attached(0) } } }; @@ -2541,9 +2565,6 @@ impl Service { let locked = self.inner.read().unwrap(); tracing::info!("Locating shards for tenant {tenant_id}"); - // Take a snapshot of pageservers - let pageservers = locked.nodes.clone(); - let mut result = Vec::new(); let mut shard_params: Option = None; @@ -2557,7 +2578,8 @@ impl Service { "Cannot locate a tenant that is not attached" )))?; - let node = pageservers + let node = locked + .nodes .get(&node_id) .expect("Pageservers may not be deleted while referenced"); @@ -2605,6 +2627,47 @@ impl Service { }) } + pub(crate) fn tenant_describe( + &self, + tenant_id: TenantId, + ) -> Result { + let locked = self.inner.read().unwrap(); + + let mut shard_zero = None; + let mut shards = Vec::new(); + + for (tenant_shard_id, shard) in locked.tenants.range(TenantShardId::tenant_range(tenant_id)) + { + if tenant_shard_id.is_zero() { + shard_zero = Some(shard); + } + + let response_shard = TenantDescribeResponseShard { + tenant_shard_id: *tenant_shard_id, + node_attached: *shard.intent.get_attached(), + node_secondary: shard.intent.get_secondary().to_vec(), + last_error: shard.last_error.lock().unwrap().clone(), + is_reconciling: shard.reconciler.is_some(), + is_pending_compute_notification: shard.pending_compute_notification, + is_splitting: matches!(shard.splitting, SplitState::Splitting), + }; + shards.push(response_shard); + } + + let Some(shard_zero) = shard_zero else { + return Err(ApiError::NotFound( + anyhow::anyhow!("Tenant {tenant_id} not found").into(), + )); + }; + + Ok(TenantDescribeResponse { + shards, + stripe_size: shard_zero.shard.stripe_size, + policy: shard_zero.policy.clone(), + config: shard_zero.config.clone(), + }) + } + #[instrument(skip_all, fields(tenant_id=%op.tenant_id))] async fn abort_tenant_shard_split( &self, @@ -2828,7 +2891,7 @@ impl Service { generation, &child_shard, &config, - matches!(policy, PlacementPolicy::Double(n) if n > 0), + matches!(policy, PlacementPolicy::Attached(n) if n > 0), )), }, ); @@ -2875,17 +2938,23 @@ impl Service { let new_shard_count = ShardCount::new(split_req.new_shard_count); let new_stripe_size = split_req.new_stripe_size; - let r = self.do_tenant_shard_split(tenant_id, split_req).await; + // Validate the request and construct parameters. This phase is fallible, but does not require + // rollback on errors, as it does no I/O and mutates no state. + let shard_split_params = match self.prepare_tenant_shard_split(tenant_id, split_req)? { + ShardSplitAction::NoOp(resp) => return Ok(resp), + ShardSplitAction::Split(params) => params, + }; + + // Execute this split: this phase mutates state and does remote I/O on pageservers. If it fails, + // we must roll back. + let r = self + .do_tenant_shard_split(tenant_id, shard_split_params) + .await; match r { Ok(r) => Ok(r), - Err(ApiError::BadRequest(_)) => { - // A request validation error does not require rollback: we rejected it before we started making any changes: just - // return the error - r - } Err(e) => { - // General case error handling: split might be part-done, we must do work to abort it. + // Split might be part-done, we must do work to abort it. tracing::warn!("Enqueuing background abort of split on {tenant_id}"); self.abort_tx .send(TenantShardSplitAbort { @@ -2901,25 +2970,17 @@ impl Service { } } - pub(crate) async fn do_tenant_shard_split( + fn prepare_tenant_shard_split( &self, tenant_id: TenantId, split_req: TenantShardSplitRequest, - ) -> Result { - let mut policy = None; - let mut shard_ident = None; - - // A parent shard which will be split - struct SplitTarget { - parent_id: TenantShardId, - node: Node, - child_ids: Vec, - } - + ) -> Result { fail::fail_point!("shard-split-validation", |_| Err(ApiError::BadRequest( anyhow::anyhow!("failpoint") ))); + let mut policy = None; + let mut shard_ident = None; // Validate input, and calculate which shards we will create let (old_shard_count, targets) = { @@ -2995,7 +3056,7 @@ impl Service { // TODO: if any reconciliation is currently in progress for this shard, wait for it. - targets.push(SplitTarget { + targets.push(ShardSplitTarget { parent_id: *tenant_shard_id, node: node.clone(), child_ids: tenant_shard_id @@ -3005,9 +3066,9 @@ impl Service { if targets.is_empty() { if children_found.len() == split_req.new_shard_count as usize { - return Ok(TenantShardSplitResponse { + return Ok(ShardSplitAction::NoOp(TenantShardSplitResponse { new_shards: children_found, - }); + })); } else { // No shards found to split, and no existing children found: the // tenant doesn't exist at all. @@ -3038,12 +3099,36 @@ impl Service { }; let policy = policy.unwrap(); + Ok(ShardSplitAction::Split(ShardSplitParams { + old_shard_count, + new_shard_count: ShardCount::new(split_req.new_shard_count), + new_stripe_size: split_req.new_stripe_size, + targets, + policy, + shard_ident, + })) + } + + async fn do_tenant_shard_split( + &self, + tenant_id: TenantId, + params: ShardSplitParams, + ) -> Result { // FIXME: we have dropped self.inner lock, and not yet written anything to the database: another // request could occur here, deleting or mutating the tenant. begin_shard_split checks that the // parent shards exist as expected, but it would be neater to do the above pre-checks within the // same database transaction rather than pre-check in-memory and then maybe-fail the database write. // (https://github.com/neondatabase/neon/issues/6676) + let ShardSplitParams { + old_shard_count, + new_shard_count, + new_stripe_size, + targets, + policy, + shard_ident, + } = params; + // Before creating any new child shards in memory or on the pageservers, persist them: this // enables us to ensure that we will always be able to clean up if something goes wrong. This also // acts as the protection against two concurrent attempts to split: one of them will get a database @@ -3125,7 +3210,7 @@ impl Service { // N>1 shards into M shards -- initially we're usually splitting 1 shard into N). for target in &targets { - let SplitTarget { + let ShardSplitTarget { parent_id, node, child_ids, @@ -3135,8 +3220,8 @@ impl Service { .tenant_shard_split( *parent_id, TenantShardSplitRequest { - new_shard_count: split_req.new_shard_count, - new_stripe_size: split_req.new_stripe_size, + new_shard_count: new_shard_count.literal(), + new_stripe_size, }, ) .await @@ -3185,11 +3270,8 @@ impl Service { )); // Replace all the shards we just split with their children: this phase is infallible. - let (response, child_locations) = self.tenant_shard_split_commit_inmem( - tenant_id, - ShardCount::new(split_req.new_shard_count), - split_req.new_stripe_size, - ); + let (response, child_locations) = + self.tenant_shard_split_commit_inmem(tenant_id, new_shard_count, new_stripe_size); // Send compute notifications for all the new shards let mut failed_notifications = Vec::new(); @@ -3254,17 +3336,15 @@ impl Service { let old_attached = *shard.intent.get_attached(); match shard.policy { - PlacementPolicy::Single => { - shard.intent.clear_secondary(scheduler); - shard.intent.set_attached(scheduler, Some(migrate_req.node_id)); - } - PlacementPolicy::Double(_n) => { + PlacementPolicy::Attached(n) => { // If our new attached node was a secondary, it no longer should be. shard.intent.remove_secondary(scheduler, migrate_req.node_id); // If we were already attached to something, demote that to a secondary if let Some(old_attached) = old_attached { - shard.intent.push_secondary(scheduler, old_attached); + if n > 0 { + shard.intent.push_secondary(scheduler, old_attached); + } } shard.intent.set_attached(scheduler, Some(migrate_req.node_id)); diff --git a/control_plane/attachment_service/src/tenant_state.rs b/control_plane/attachment_service/src/tenant_state.rs index 39e557616d9c..9dd368bf411c 100644 --- a/control_plane/attachment_service/src/tenant_state.rs +++ b/control_plane/attachment_service/src/tenant_state.rs @@ -457,22 +457,7 @@ impl TenantState { // Add/remove nodes to fulfil policy use PlacementPolicy::*; match self.policy { - Single => { - // Should have exactly one attached, and zero secondaries - if !self.intent.secondary.is_empty() { - self.intent.clear_secondary(scheduler); - modified = true; - } - - let (modified_attached, _attached_node_id) = self.schedule_attached(scheduler)?; - modified |= modified_attached; - - if !self.intent.secondary.is_empty() { - self.intent.clear_secondary(scheduler); - modified = true; - } - } - Double(secondary_count) => { + Attached(secondary_count) => { let retain_secondaries = if self.intent.attached.is_none() && scheduler.node_preferred(&self.intent.secondary).is_some() { @@ -895,7 +880,7 @@ pub(crate) mod tests { let mut scheduler = Scheduler::new(nodes.values()); - let mut tenant_state = make_test_tenant_shard(PlacementPolicy::Double(1)); + let mut tenant_state = make_test_tenant_shard(PlacementPolicy::Attached(1)); tenant_state .schedule(&mut scheduler) .expect("we have enough nodes, scheduling should work"); @@ -943,7 +928,7 @@ pub(crate) mod tests { let nodes = make_test_nodes(3); let mut scheduler = Scheduler::new(nodes.values()); - let mut tenant_state = make_test_tenant_shard(PlacementPolicy::Double(1)); + let mut tenant_state = make_test_tenant_shard(PlacementPolicy::Attached(1)); tenant_state.observed.locations.insert( NodeId(3), diff --git a/control_plane/src/bin/neon_local.rs b/control_plane/src/bin/neon_local.rs index 6c722f36b4a5..401feae70697 100644 --- a/control_plane/src/bin/neon_local.rs +++ b/control_plane/src/bin/neon_local.rs @@ -437,7 +437,7 @@ async fn handle_tenant( let placement_policy = match create_match.get_one::("placement-policy") { Some(s) if !s.is_empty() => serde_json::from_str::(s)?, - _ => PlacementPolicy::Single, + _ => PlacementPolicy::Attached(0), }; let tenant_conf = PageServerNode::parse_config(tenant_conf)?; @@ -523,88 +523,6 @@ async fn handle_tenant( .with_context(|| format!("Tenant config failed for tenant with id {tenant_id}"))?; println!("tenant {tenant_id} successfully configured on the pageserver"); } - Some(("migrate", matches)) => { - let tenant_shard_id = get_tenant_shard_id(matches, env)?; - let new_pageserver = get_pageserver(env, matches)?; - let new_pageserver_id = new_pageserver.conf.id; - - let storage_controller = StorageController::from_env(env); - storage_controller - .tenant_migrate(tenant_shard_id, new_pageserver_id) - .await?; - - println!("tenant {tenant_shard_id} migrated to {}", new_pageserver_id); - } - Some(("status", matches)) => { - let tenant_id = get_tenant_id(matches, env)?; - - let mut shard_table = comfy_table::Table::new(); - shard_table.set_header(["Shard", "Pageserver", "Physical Size"]); - - let mut tenant_synthetic_size = None; - - let storage_controller = StorageController::from_env(env); - for shard in storage_controller.tenant_locate(tenant_id).await?.shards { - let pageserver = - PageServerNode::from_env(env, env.get_pageserver_conf(shard.node_id)?); - - let size = pageserver - .http_client - .tenant_details(shard.shard_id) - .await? - .tenant_info - .current_physical_size - .unwrap(); - - shard_table.add_row([ - format!("{}", shard.shard_id.shard_slug()), - format!("{}", shard.node_id.0), - format!("{} MiB", size / (1024 * 1024)), - ]); - - if shard.shard_id.is_zero() { - tenant_synthetic_size = - Some(pageserver.tenant_synthetic_size(shard.shard_id).await?); - } - } - - let Some(synthetic_size) = tenant_synthetic_size else { - bail!("Shard 0 not found") - }; - - let mut tenant_table = comfy_table::Table::new(); - tenant_table.add_row(["Tenant ID".to_string(), tenant_id.to_string()]); - tenant_table.add_row([ - "Synthetic size".to_string(), - format!("{} MiB", synthetic_size.size.unwrap_or(0) / (1024 * 1024)), - ]); - - println!("{tenant_table}"); - println!("{shard_table}"); - } - Some(("shard-split", matches)) => { - let tenant_id = get_tenant_id(matches, env)?; - let shard_count: u8 = matches.get_one::("shard-count").cloned().unwrap_or(0); - let shard_stripe_size: Option = matches - .get_one::>("shard-stripe-size") - .cloned() - .unwrap(); - - let storage_controller = StorageController::from_env(env); - let result = storage_controller - .tenant_split(tenant_id, shard_count, shard_stripe_size) - .await?; - println!( - "Split tenant {} into shards {}", - tenant_id, - result - .new_shards - .iter() - .map(|s| format!("{:?}", s)) - .collect::>() - .join(",") - ); - } Some((sub_name, _)) => bail!("Unexpected tenant subcommand '{}'", sub_name), None => bail!("no tenant subcommand provided"), @@ -1578,19 +1496,6 @@ fn cli() -> Command { .subcommand(Command::new("config") .arg(tenant_id_arg.clone()) .arg(Arg::new("config").short('c').num_args(1).action(ArgAction::Append).required(false))) - .subcommand(Command::new("migrate") - .about("Migrate a tenant from one pageserver to another") - .arg(tenant_id_arg.clone()) - .arg(pageserver_id_arg.clone())) - .subcommand(Command::new("status") - .about("Human readable summary of the tenant's shards and attachment locations") - .arg(tenant_id_arg.clone())) - .subcommand(Command::new("shard-split") - .about("Increase the number of shards in the tenant") - .arg(tenant_id_arg.clone()) - .arg(Arg::new("shard-count").value_parser(value_parser!(u8)).long("shard-count").action(ArgAction::Set).help("Number of shards in the new tenant (default 1)")) - .arg(Arg::new("shard-stripe-size").value_parser(value_parser!(u32)).long("shard-stripe-size").action(ArgAction::Set).help("Sharding stripe size in pages")) - ) ) .subcommand( Command::new("pageserver") diff --git a/control_plane/src/storage_controller.rs b/control_plane/src/storage_controller.rs index 18014adba4b8..e7697ecac80c 100644 --- a/control_plane/src/storage_controller.rs +++ b/control_plane/src/storage_controller.rs @@ -475,7 +475,7 @@ impl StorageController { pub async fn tenant_locate(&self, tenant_id: TenantId) -> anyhow::Result { self.dispatch::<(), _>( Method::GET, - format!("control/v1/tenant/{tenant_id}/locate"), + format!("debug/v1/tenant/{tenant_id}/locate"), None, ) .await diff --git a/libs/pageserver_api/src/controller_api.rs b/libs/pageserver_api/src/controller_api.rs index 6053e8b8ed4a..e33bd0f48690 100644 --- a/libs/pageserver_api/src/controller_api.rs +++ b/libs/pageserver_api/src/controller_api.rs @@ -6,7 +6,10 @@ use std::str::FromStr; use serde::{Deserialize, Serialize}; use utils::id::NodeId; -use crate::{models::ShardParameters, shard::TenantShardId}; +use crate::{ + models::{ShardParameters, TenantConfig}, + shard::{ShardStripeSize, TenantShardId}, +}; #[derive(Serialize, Deserialize)] pub struct TenantCreateResponseShard { @@ -57,6 +60,31 @@ pub struct TenantLocateResponse { pub shard_params: ShardParameters, } +#[derive(Serialize, Deserialize)] +pub struct TenantDescribeResponse { + pub shards: Vec, + pub stripe_size: ShardStripeSize, + pub policy: PlacementPolicy, + pub config: TenantConfig, +} + +#[derive(Serialize, Deserialize)] +pub struct TenantDescribeResponseShard { + pub tenant_shard_id: TenantShardId, + + pub node_attached: Option, + pub node_secondary: Vec, + + pub last_error: String, + + /// A task is currently running to reconcile this tenant's intent state with the state on pageservers + pub is_reconciling: bool, + /// This shard failed in sending a compute notification to the cloud control plane, and a retry is pending. + pub is_pending_compute_notification: bool, + /// A shard split is currently underway + pub is_splitting: bool, +} + /// Explicitly migrating a particular shard is a low level operation /// TODO: higher level "Reschedule tenant" operation where the request /// specifies some constraints, e.g. asking it to get off particular node(s) @@ -181,11 +209,8 @@ impl From for String { /// to create secondary locations. #[derive(Clone, Serialize, Deserialize, Debug, PartialEq, Eq)] pub enum PlacementPolicy { - /// Cheapest way to attach a tenant: just one pageserver, no secondary - Single, - /// Production-ready way to attach a tenant: one attached pageserver and - /// some number of secondaries. - Double(usize), + /// Normal live state: one attached pageserver and zero or more secondaries. + Attached(usize), /// Create one secondary mode locations. This is useful when onboarding /// a tenant, or for an idle tenant that we might want to bring online quickly. Secondary, @@ -207,14 +232,14 @@ mod test { /// Check stability of PlacementPolicy's serialization #[test] fn placement_policy_encoding() -> anyhow::Result<()> { - let v = PlacementPolicy::Double(1); + let v = PlacementPolicy::Attached(1); let encoded = serde_json::to_string(&v)?; - assert_eq!(encoded, "{\"Double\":1}"); + assert_eq!(encoded, "{\"Attached\":1}"); assert_eq!(serde_json::from_str::(&encoded)?, v); - let v = PlacementPolicy::Single; + let v = PlacementPolicy::Detached; let encoded = serde_json::to_string(&v)?; - assert_eq!(encoded, "\"Single\""); + assert_eq!(encoded, "\"Detached\""); assert_eq!(serde_json::from_str::(&encoded)?, v); Ok(()) } diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index 3ecd3432246f..1d30c4527830 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -1892,19 +1892,6 @@ def map_branch( return self.raw_cli(args, check_return_code=True) - def tenant_migrate( - self, tenant_shard_id: TenantShardId, new_pageserver: int, timeout_secs: Optional[int] - ): - args = [ - "tenant", - "migrate", - "--tenant-id", - str(tenant_shard_id), - "--id", - str(new_pageserver), - ] - return self.raw_cli(args, check_return_code=True, timeout=timeout_secs) - def start(self, check_return_code=True) -> "subprocess.CompletedProcess[str]": return self.raw_cli(["start"], check_return_code=check_return_code) @@ -2156,7 +2143,7 @@ def locate(self, tenant_id: TenantId) -> list[dict[str, Any]]: """ response = self.request( "GET", - f"{self.env.storage_controller_api}/control/v1/tenant/{tenant_id}/locate", + f"{self.env.storage_controller_api}/debug/v1/tenant/{tenant_id}/locate", headers=self.headers(TokenScope.ADMIN), ) body = response.json() diff --git a/test_runner/fixtures/types.py b/test_runner/fixtures/types.py index ea648e460d58..80c9b9ce9abf 100644 --- a/test_runner/fixtures/types.py +++ b/test_runner/fixtures/types.py @@ -158,6 +158,9 @@ def parse(cls: Type[TTenantShardId], input) -> TTenantShardId: def __str__(self): return f"{self.tenant_id}-{self.shard_number:02x}{self.shard_count:02x}" + def __repr__(self): + return self.__str__() + def _tuple(self) -> tuple[TenantId, int, int]: return (self.tenant_id, self.shard_number, self.shard_count) diff --git a/test_runner/regress/test_pageserver_secondary.py b/test_runner/regress/test_pageserver_secondary.py index 8ef75414a39b..e664547b690e 100644 --- a/test_runner/regress/test_pageserver_secondary.py +++ b/test_runner/regress/test_pageserver_secondary.py @@ -576,7 +576,7 @@ def test_slow_secondary_downloads(neon_env_builder: NeonEnvBuilder, via_controll timeline_id = TimelineId.generate() env.neon_cli.create_tenant( - tenant_id, timeline_id, conf=TENANT_CONF, placement_policy='{"Double":1}' + tenant_id, timeline_id, conf=TENANT_CONF, placement_policy='{"Attached":1}' ) attached_to_id = env.storage_controller.locate(tenant_id)[0]["node_id"] diff --git a/test_runner/regress/test_sharding.py b/test_runner/regress/test_sharding.py index 3470d2e6090f..cb58c640c38d 100644 --- a/test_runner/regress/test_sharding.py +++ b/test_runner/regress/test_sharding.py @@ -264,7 +264,7 @@ def shards_on_disk(shard_ids): destination = migrate_to_pageserver_ids.pop() log.info(f"Migrating shard {migrate_shard} from {ps_id} to {destination}") - env.neon_cli.tenant_migrate(migrate_shard, destination, timeout_secs=10) + env.storage_controller.tenant_shard_migrate(migrate_shard, destination) workload.validate() @@ -299,7 +299,7 @@ def shards_on_disk(shard_ids): locations = pageserver.http_client().tenant_list_locations() shards_exist.extend(TenantShardId.parse(s[0]) for s in locations["tenant_shards"]) - log.info("Shards after split: {shards_exist}") + log.info(f"Shards after split: {shards_exist}") assert len(shards_exist) == split_shard_count # Ensure post-split pageserver locations survive a restart (i.e. the child shards From 4ba3f3518eddd8e5eebca90c564857e0d285932d Mon Sep 17 00:00:00 2001 From: Vlad Lazar Date: Wed, 20 Mar 2024 10:24:59 +0000 Subject: [PATCH 05/53] test: fix on demand activation test flakyness (#7180) Warm-up (and the "tenant startup complete" metric update) happens in a background tokio task. The tenant map is eagerly updated (can happen before the task finishes). The test assumed that if the tenant map was updated, then the metric should reflect that. That's not the case, so we tweak the test to wait for the metric. Fixes https://github.com/neondatabase/neon/issues/7158 --- test_runner/regress/test_timeline_size.py | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/test_runner/regress/test_timeline_size.py b/test_runner/regress/test_timeline_size.py index 205ca18050a8..628c484fbdbd 100644 --- a/test_runner/regress/test_timeline_size.py +++ b/test_runner/regress/test_timeline_size.py @@ -20,6 +20,7 @@ VanillaPostgres, wait_for_last_flush_lsn, ) +from fixtures.pageserver.http import PageserverHttpClient from fixtures.pageserver.utils import ( assert_tenant_state, timeline_delete_wait_completed, @@ -684,6 +685,13 @@ def assert_physical_size_invariants(sizes: TimelinePhysicalSizeValues): # XXX would be nice to assert layer file physical storage utilization here as well, but we can only do that for LocalFS +def wait_for_tenant_startup_completions(client: PageserverHttpClient, count: int): + def condition(): + assert client.get_metric_value("pageserver_tenant_startup_complete_total") == count + + wait_until(5, 1.0, condition) + + def test_ondemand_activation(neon_env_builder: NeonEnvBuilder): """ Tenants warmuping up opportunistically will wait for one another's logical size calculations to complete @@ -767,10 +775,7 @@ def at_least_one_active(): # That one that we successfully accessed is now Active expect_activated += 1 assert pageserver_http.tenant_status(tenant_id=stuck_tenant_id)["state"]["slug"] == "Active" - assert ( - pageserver_http.get_metric_value("pageserver_tenant_startup_complete_total") - == expect_activated - 1 - ) + wait_for_tenant_startup_completions(pageserver_http, count=expect_activated - 1) # The ones we didn't touch are still in Attaching assert ( @@ -790,10 +795,7 @@ def at_least_one_active(): == n_tenants - expect_activated ) - assert ( - pageserver_http.get_metric_value("pageserver_tenant_startup_complete_total") - == expect_activated - 1 - ) + wait_for_tenant_startup_completions(pageserver_http, count=expect_activated - 1) # When we unblock logical size calculation, all tenants should proceed to active state via # the warmup route. @@ -813,7 +815,7 @@ def all_active(): assert ( pageserver_http.get_metric_value("pageserver_tenant_startup_scheduled_total") == n_tenants ) - assert pageserver_http.get_metric_value("pageserver_tenant_startup_complete_total") == n_tenants + wait_for_tenant_startup_completions(pageserver_http, count=n_tenants) # Check that tenant deletion/detach proactively wakes tenants: this is done separately to the main # body of the test because it will disrupt tenant counts From 6d996427b19ae20b0be30651838586307537b2b4 Mon Sep 17 00:00:00 2001 From: Conrad Ludgate Date: Wed, 20 Mar 2024 12:26:31 +0000 Subject: [PATCH 06/53] proxy: enable sha2 asm support (#7184) ## Problem faster sha2 hashing. ## Summary of changes enable asm feature for sha2. this feature will be default in sha2 0.11, so we might as well lean into it now. It provides a noticeable speed boost on macos aarch64. Haven't tested on x86 though --- Cargo.lock | 15 +++++++++++++-- proxy/Cargo.toml | 2 +- workspace_hack/Cargo.toml | 1 + 3 files changed, 15 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 70f427f97d62..cdbabf2f769d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5346,13 +5346,23 @@ checksum = "ae1a47186c03a32177042e55dbc5fd5aee900b8e0069a8d70fba96a9375cd012" [[package]] name = "sha2" -version = "0.10.6" +version = "0.10.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "82e6b795fe2e3b1e845bafcb27aa35405c4d47cdfc92af5fc8d3002f76cebdc0" +checksum = "793db75ad2bcafc3ffa7c68b215fee268f537982cd901d132f89c6343f3a3dc8" dependencies = [ "cfg-if", "cpufeatures", "digest", + "sha2-asm", +] + +[[package]] +name = "sha2-asm" +version = "0.6.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f27ba7066011e3fb30d808b51affff34f0a66d3a03a58edd787c6e420e40e44e" +dependencies = [ + "cc", ] [[package]] @@ -7032,6 +7042,7 @@ dependencies = [ "scopeguard", "serde", "serde_json", + "sha2", "smallvec", "subtle", "syn 1.0.109", diff --git a/proxy/Cargo.toml b/proxy/Cargo.toml index d8112c8bf0c8..b3a5bf873eda 100644 --- a/proxy/Cargo.toml +++ b/proxy/Cargo.toml @@ -59,7 +59,7 @@ rustls.workspace = true scopeguard.workspace = true serde.workspace = true serde_json.workspace = true -sha2.workspace = true +sha2 = { workspace = true, features = ["asm"] } smol_str.workspace = true smallvec.workspace = true socket2.workspace = true diff --git a/workspace_hack/Cargo.toml b/workspace_hack/Cargo.toml index 8593b752c209..152c452dd4ee 100644 --- a/workspace_hack/Cargo.toml +++ b/workspace_hack/Cargo.toml @@ -64,6 +64,7 @@ rustls = { version = "0.21", features = ["dangerous_configuration"] } scopeguard = { version = "1" } serde = { version = "1", features = ["alloc", "derive"] } serde_json = { version = "1", features = ["raw_value"] } +sha2 = { version = "0.10", features = ["asm"] } smallvec = { version = "1", default-features = false, features = ["write"] } subtle = { version = "2" } time = { version = "0.3", features = ["local-offset", "macros", "serde-well-known"] } From fb66a3dd857bec7f99bba9c5dd5ee80213761878 Mon Sep 17 00:00:00 2001 From: Joonas Koivunen Date: Wed, 20 Mar 2024 16:08:03 +0200 Subject: [PATCH 07/53] fix: ResidentLayer::load_keys should not create INFO level span (#7174) Since #6115 with more often used get_value_reconstruct_data and friends, we should not have needless INFO level span creation near hot paths. In our prod configuration, INFO spans are always created, but in practice, very rarely anything at INFO level is logged underneath. `ResidentLayer::load_keys` is only used during compaction so it is not that hot, but this aligns the access paths and their span usage. PR changes the span level to debug to align with others, and adds the layer name to the error which was missing. Split off from #7030. --- pageserver/src/tenant/storage_layer/layer.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pageserver/src/tenant/storage_layer/layer.rs b/pageserver/src/tenant/storage_layer/layer.rs index 0200ff8cf4ca..f37d7e64498b 100644 --- a/pageserver/src/tenant/storage_layer/layer.rs +++ b/pageserver/src/tenant/storage_layer/layer.rs @@ -1484,7 +1484,7 @@ impl ResidentLayer { } /// Loads all keys stored in the layer. Returns key, lsn and value size. - #[tracing::instrument(skip_all, fields(layer=%self))] + #[tracing::instrument(level = tracing::Level::DEBUG, skip_all, fields(layer=%self))] pub(crate) async fn load_keys<'a>( &'a self, ctx: &RequestContext, @@ -1504,9 +1504,9 @@ impl ResidentLayer { // while it's being held. delta_layer::DeltaLayerInner::load_keys(d, ctx) .await - .context("Layer index is corrupted") + .with_context(|| format!("Layer index is corrupted for {self}")) } - Image(_) => anyhow::bail!("cannot load_keys on a image layer"), + Image(_) => anyhow::bail!(format!("cannot load_keys on a image layer {self}")), } } From 3d16cda846f4e7b8e929c61db13093586dca93d1 Mon Sep 17 00:00:00 2001 From: Joonas Koivunen Date: Wed, 20 Mar 2024 18:03:09 +0200 Subject: [PATCH 08/53] refactor(layer): use detached init (#7152) The second part of work towards fixing `Layer::keep_resident` so that it does not need to repair the internal state. #7135 added a nicer API for initialization. This PR uses it to remove a few indentation levels and the loop construction. The next PR #7175 will use the refactorings done in this PR, and always initialize the internal state after a download. Cc: #5331 --- pageserver/src/tenant/storage_layer/layer.rs | 307 +++++++++--------- .../src/tenant/storage_layer/layer/tests.rs | 3 + 2 files changed, 155 insertions(+), 155 deletions(-) diff --git a/pageserver/src/tenant/storage_layer/layer.rs b/pageserver/src/tenant/storage_layer/layer.rs index f37d7e64498b..eed423c3e69e 100644 --- a/pageserver/src/tenant/storage_layer/layer.rs +++ b/pageserver/src/tenant/storage_layer/layer.rs @@ -702,181 +702,132 @@ impl LayerInner { allow_download: bool, ctx: Option<&RequestContext>, ) -> Result, DownloadError> { - let mut init_permit = None; + let (weak, permit) = { + let locked = self + .inner + .get_or_init_detached() + .await + .map(|mut guard| guard.get_and_upgrade().ok_or(guard)); - loop { - let download = move |permit| { - async move { - // disable any scheduled but not yet running eviction deletions for this - let next_version = 1 + self.version.fetch_add(1, Ordering::Relaxed); + match locked { + // this path could had been a RwLock::read + Ok(Ok((strong, upgraded))) if !upgraded => return Ok(strong), + Ok(Ok((strong, _))) => { + // when upgraded back, the Arc is still available, but + // previously a `evict_and_wait` was received. + self.wanted_evicted.store(false, Ordering::Relaxed); - // no need to make the evict_and_wait wait for the actual download to complete + // error out any `evict_and_wait` drop(self.status.send(Status::Downloaded)); + LAYER_IMPL_METRICS + .inc_eviction_cancelled(EvictionCancelled::UpgradedBackOnAccess); - let timeline = self - .timeline - .upgrade() - .ok_or_else(|| DownloadError::TimelineShutdown)?; - - // count cancellations, which currently remain largely unexpected - let init_cancelled = - scopeguard::guard((), |_| LAYER_IMPL_METRICS.inc_init_cancelled()); - - let can_ever_evict = timeline.remote_client.as_ref().is_some(); - - // check if we really need to be downloaded; could have been already downloaded by a - // cancelled previous attempt. - let needs_download = self - .needs_download() - .await - .map_err(DownloadError::PreStatFailed); - - let needs_download = match needs_download { - Ok(reason) => reason, - Err(e) => { - scopeguard::ScopeGuard::into_inner(init_cancelled); - return Err(e); - } - }; - - let (permit, downloaded) = if let Some(reason) = needs_download { - if let NeedsDownload::NotFile(ft) = reason { - return Err(DownloadError::NotFile(ft)); - } - - // only reset this after we've decided we really need to download. otherwise it'd - // be impossible to mark cancelled downloads for eviction, like one could imagine - // we would like to do for prefetching which was not needed. - self.wanted_evicted.store(false, Ordering::Release); - - if !can_ever_evict { - scopeguard::ScopeGuard::into_inner(init_cancelled); - return Err(DownloadError::NoRemoteStorage); - } - - if let Some(ctx) = ctx { - let res = self.check_expected_download(ctx); - if let Err(e) = res { - scopeguard::ScopeGuard::into_inner(init_cancelled); - return Err(e); - } - } + return Ok(strong); + } + Ok(Err(mut guard)) => { + // path to here: the evict_blocking is stuck on spawn_blocking queue. + // + // reset the contents, deactivating the eviction and causing a + // EvictionCancelled::LostToDownload or EvictionCancelled::VersionCheckFailed. + let (weak, permit) = guard.take_and_deinit(); + (Some(weak), permit) + } + Err(permit) => (None, permit), + } + }; - if !allow_download { - // this does look weird, but for LayerInner the "downloading" means also changing - // internal once related state ... - scopeguard::ScopeGuard::into_inner(init_cancelled); - return Err(DownloadError::DownloadRequired); - } + if let Some(weak) = weak { + // only drop the weak after dropping the heavier_once_cell guard + assert!( + matches!(weak, ResidentOrWantedEvicted::WantedEvicted(..)), + "unexpected {weak:?}, ResidentOrWantedEvicted::get_and_upgrade has a bug" + ); + } - tracing::info!(%reason, "downloading on-demand"); + async move { + // disable any scheduled but not yet running eviction deletions for this + let next_version = 1 + self.version.fetch_add(1, Ordering::Relaxed); - let permit = self.spawn_download_and_wait(timeline, permit).await; + // no need to make the evict_and_wait wait for the actual download to complete + drop(self.status.send(Status::Downloaded)); - let permit = match permit { - Ok(permit) => permit, - Err(e) => { - scopeguard::ScopeGuard::into_inner(init_cancelled); - return Err(e); - } - }; + let timeline = self + .timeline + .upgrade() + .ok_or_else(|| DownloadError::TimelineShutdown)?; - (permit, true) - } else { - // the file is present locally, probably by a previous but cancelled call to - // get_or_maybe_download. alternatively we might be running without remote storage. - LAYER_IMPL_METRICS.inc_init_needed_no_download(); + // count cancellations, which currently remain largely unexpected + let init_cancelled = scopeguard::guard((), |_| LAYER_IMPL_METRICS.inc_init_cancelled()); - (permit, false) - }; + // check if we really need to be downloaded; could have been already downloaded by a + // cancelled previous attempt. + let needs_download = self + .needs_download() + .await + .map_err(DownloadError::PreStatFailed); + let needs_download = match needs_download { + Ok(reason) => reason, + Err(e) => { scopeguard::ScopeGuard::into_inner(init_cancelled); + return Err(e); + } + }; - if downloaded { - let since_last_eviction = self - .last_evicted_at - .lock() - .unwrap() - .take() - .map(|ts| ts.elapsed()); - - if let Some(since_last_eviction) = since_last_eviction { - LAYER_IMPL_METRICS.record_redownloaded_after(since_last_eviction); - } - } - - let res = Arc::new(DownloadedLayer { - owner: Arc::downgrade(self), - kind: tokio::sync::OnceCell::default(), - version: next_version, - }); - - self.access_stats.record_residence_event( - LayerResidenceStatus::Resident, - LayerResidenceEventReason::ResidenceChange, - ); + let Some(reason) = needs_download else { + scopeguard::ScopeGuard::into_inner(init_cancelled); - let waiters = self.inner.initializer_count(); - if waiters > 0 { - tracing::info!( - waiters, - "completing the on-demand download for other tasks" - ); - } + // the file is present locally, probably by a previous but cancelled call to + // get_or_maybe_download. alternatively we might be running without remote storage. + LAYER_IMPL_METRICS.inc_init_needed_no_download(); - Ok((ResidentOrWantedEvicted::Resident(res), permit)) - } - .instrument(tracing::info_span!("get_or_maybe_download", layer=%self)) + let res = self.initialize_after_layer_is_on_disk(next_version, permit, false); + return Ok(res); }; - if let Some(init_permit) = init_permit.take() { - // use the already held initialization permit because it is impossible to hit the - // below paths anymore essentially limiting the max loop iterations to 2. - let (value, init_permit) = download(init_permit).await?; - let mut guard = self.inner.set(value, init_permit); - let (strong, _upgraded) = guard - .get_and_upgrade() - .expect("init creates strong reference, we held the init permit"); - return Ok(strong); + if let NeedsDownload::NotFile(ft) = reason { + scopeguard::ScopeGuard::into_inner(init_cancelled); + return Err(DownloadError::NotFile(ft)); } - let (weak, permit) = { - let mut locked = self.inner.get_or_init(download).await?; - - if let Some((strong, upgraded)) = locked.get_and_upgrade() { - if upgraded { - // when upgraded back, the Arc is still available, but - // previously a `evict_and_wait` was received. - self.wanted_evicted.store(false, Ordering::Relaxed); + // only reset this after we've decided we really need to download. otherwise it'd + // be impossible to mark cancelled downloads for eviction, like one could imagine + // we would like to do for prefetching which was not needed. + self.wanted_evicted.store(false, Ordering::Release); - // error out any `evict_and_wait` - drop(self.status.send(Status::Downloaded)); - LAYER_IMPL_METRICS - .inc_eviction_cancelled(EvictionCancelled::UpgradedBackOnAccess); - } + if timeline.remote_client.as_ref().is_none() { + scopeguard::ScopeGuard::into_inner(init_cancelled); + return Err(DownloadError::NoRemoteStorage); + } - return Ok(strong); - } else { - // path to here: the evict_blocking is stuck on spawn_blocking queue. - // - // reset the contents, deactivating the eviction and causing a - // EvictionCancelled::LostToDownload or EvictionCancelled::VersionCheckFailed. - locked.take_and_deinit() + if let Some(ctx) = ctx { + let res = self.check_expected_download(ctx); + if let Err(e) = res { + scopeguard::ScopeGuard::into_inner(init_cancelled); + return Err(e); } - }; + } - // unlock first, then drop the weak, but because upgrade failed, we - // know it cannot be a problem. + if !allow_download { + // this does look weird, but for LayerInner the "downloading" means also changing + // internal once related state ... + scopeguard::ScopeGuard::into_inner(init_cancelled); + return Err(DownloadError::DownloadRequired); + } - assert!( - matches!(weak, ResidentOrWantedEvicted::WantedEvicted(..)), - "unexpected {weak:?}, ResidentOrWantedEvicted::get_and_upgrade has a bug" - ); + tracing::info!(%reason, "downloading on-demand"); + + let permit = self.spawn_download_and_wait(timeline, permit).await; + + scopeguard::ScopeGuard::into_inner(init_cancelled); - init_permit = Some(permit); + let permit = permit?; - LAYER_IMPL_METRICS.inc_retried_get_or_maybe_download(); + let res = self.initialize_after_layer_is_on_disk(next_version, permit, true); + Ok(res) } + .instrument(tracing::info_span!("get_or_maybe_download", layer=%self)) + .await } /// Nag or fail per RequestContext policy @@ -1026,6 +977,59 @@ impl LayerInner { } } + /// Initializes the `Self::inner` to a "resident" state. + /// + /// Callers are assumed to ensure that the file is actually on disk with `Self::needs_download` + /// before calling this method. + /// + /// If this method is ever made async, it needs to be cancellation safe so that no state + /// changes are made before we can write to the OnceCell in non-cancellable fashion. + fn initialize_after_layer_is_on_disk( + self: &Arc, + next_version: usize, + permit: heavier_once_cell::InitPermit, + downloaded: bool, + ) -> Arc { + debug_assert_current_span_has_tenant_and_timeline_id(); + + if downloaded { + let since_last_eviction = self + .last_evicted_at + .lock() + .unwrap() + .take() + .map(|ts| ts.elapsed()); + if let Some(since_last_eviction) = since_last_eviction { + // FIXME: this will not always be recorded correctly until #6028 (the no + // download needed branch above) + LAYER_IMPL_METRICS.record_redownloaded_after(since_last_eviction); + } + } + + let res = Arc::new(DownloadedLayer { + owner: Arc::downgrade(self), + kind: tokio::sync::OnceCell::default(), + version: next_version, + }); + + // FIXME: this might now be double-accounted for !downloaded + self.access_stats.record_residence_event( + LayerResidenceStatus::Resident, + LayerResidenceEventReason::ResidenceChange, + ); + + let waiters = self.inner.initializer_count(); + if waiters > 0 { + tracing::info!(waiters, "completing the on-demand download for other tasks"); + } + + let value = ResidentOrWantedEvicted::Resident(res.clone()); + + self.inner.set(value, permit); + + res + } + async fn needs_download(&self) -> Result, std::io::Error> { match tokio::fs::metadata(&self.path).await { Ok(m) => Ok(self.is_file_present_and_good_size(&m).err()), @@ -1690,11 +1694,6 @@ impl LayerImplMetrics { self.rare_counters[RareEvent::RemoveOnDropFailed].inc(); } - /// Expected rare because requires a race with `evict_blocking` and `get_or_maybe_download`. - fn inc_retried_get_or_maybe_download(&self) { - self.rare_counters[RareEvent::RetriedGetOrMaybeDownload].inc(); - } - /// Expected rare because cancellations are unexpected, and failures are unexpected fn inc_download_failed_without_requester(&self) { self.rare_counters[RareEvent::DownloadFailedWithoutRequester].inc(); @@ -1779,7 +1778,6 @@ impl DeleteFailed { #[derive(enum_map::Enum)] enum RareEvent { RemoveOnDropFailed, - RetriedGetOrMaybeDownload, DownloadFailedWithoutRequester, UpgradedWantedEvicted, InitWithoutDownload, @@ -1793,7 +1791,6 @@ impl RareEvent { match self { RemoveOnDropFailed => "remove_on_drop_failed", - RetriedGetOrMaybeDownload => "retried_gomd", DownloadFailedWithoutRequester => "download_failed_without", UpgradedWantedEvicted => "raced_wanted_evicted", InitWithoutDownload => "init_needed_no_download", diff --git a/pageserver/src/tenant/storage_layer/layer/tests.rs b/pageserver/src/tenant/storage_layer/layer/tests.rs index b43534efd4dc..e7b2eb025a8b 100644 --- a/pageserver/src/tenant/storage_layer/layer/tests.rs +++ b/pageserver/src/tenant/storage_layer/layer/tests.rs @@ -254,6 +254,8 @@ async fn residency_check_while_evict_and_wait_on_clogged_spawn_blocking() { let h = TenantHarness::create("residency_check_while_evict_and_wait_on_clogged_spawn_blocking") .unwrap(); let (tenant, ctx) = h.load().await; + let span = h.span(); + let download_span = span.in_scope(|| tracing::info_span!("downloading", timeline_id = 1)); let timeline = tenant .create_test_timeline(TimelineId::generate(), Lsn(0x10), 14, &ctx) @@ -292,6 +294,7 @@ async fn residency_check_while_evict_and_wait_on_clogged_spawn_blocking() { // because no actual eviction happened, we get to just reinitialize the DownloadedLayer layer .keep_resident() + .instrument(download_span) .await .expect("keep_resident should had reinitialized without downloading") .expect("ResidentLayer"); From 2726b1934ebd9d12d976ce9e9a41783d9ab238a8 Mon Sep 17 00:00:00 2001 From: John Spray Date: Wed, 20 Mar 2024 18:07:45 +0000 Subject: [PATCH 09/53] pageserver: extra debug for test_secondary_downloads failures (#7183) - Enable debug logs for this test - Add some debug logging detail in downloader.rs - Add an info-level message in scheduler.rs that makes it obvious if a command is waiting for an existing task rather than spawning a new one. --- pageserver/src/tenant/secondary/downloader.rs | 13 ++++++++++++- pageserver/src/tenant/secondary/scheduler.rs | 1 + test_runner/regress/test_pageserver_secondary.py | 4 ++++ 3 files changed, 17 insertions(+), 1 deletion(-) diff --git a/pageserver/src/tenant/secondary/downloader.rs b/pageserver/src/tenant/secondary/downloader.rs index a5950961332d..82af7ed83b84 100644 --- a/pageserver/src/tenant/secondary/downloader.rs +++ b/pageserver/src/tenant/secondary/downloader.rs @@ -534,7 +534,11 @@ impl<'a> TenantDownloader<'a> { .await .maybe_fatal_err(&context_msg)?; - tracing::debug!("Wrote local heatmap to {}", heatmap_path); + tracing::debug!( + "Wrote local heatmap to {}, with {} timelines", + heatmap_path, + heatmap.timelines.len() + ); // Clean up any local layers that aren't in the heatmap. We do this first for all timelines, on the general // principle that deletions should be done before writes wherever possible, and so that we can use this @@ -547,6 +551,10 @@ impl<'a> TenantDownloader<'a> { // Download the layers in the heatmap for timeline in heatmap.timelines { if self.secondary_state.cancel.is_cancelled() { + tracing::debug!( + "Cancelled before downloading timeline {}", + timeline.timeline_id + ); return Ok(()); } @@ -764,10 +772,13 @@ impl<'a> TenantDownloader<'a> { } }; + tracing::debug!(timeline_id=%timeline.timeline_id, "Downloading layers, {} in heatmap", timeline.layers.len()); + // 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() { + tracing::debug!("Cancelled -- dropping out of layer loop"); return Ok(()); } diff --git a/pageserver/src/tenant/secondary/scheduler.rs b/pageserver/src/tenant/secondary/scheduler.rs index 58bdb54161f3..3bd7be782e19 100644 --- a/pageserver/src/tenant/secondary/scheduler.rs +++ b/pageserver/src/tenant/secondary/scheduler.rs @@ -300,6 +300,7 @@ where let tenant_shard_id = job.get_tenant_shard_id(); let barrier = if let Some(barrier) = self.get_running(tenant_shard_id) { + tracing::info!("Command already running, waiting for it"); barrier } else { let running = self.spawn_now(job); diff --git a/test_runner/regress/test_pageserver_secondary.py b/test_runner/regress/test_pageserver_secondary.py index e664547b690e..2e5713660750 100644 --- a/test_runner/regress/test_pageserver_secondary.py +++ b/test_runner/regress/test_pageserver_secondary.py @@ -432,6 +432,10 @@ def test_secondary_downloads(neon_env_builder: NeonEnvBuilder): - Eviction of layers on the attached location results in deletion on the secondary location as well. """ + + # For debug of https://github.com/neondatabase/neon/issues/6966 + neon_env_builder.rust_log_override = "DEBUG" + neon_env_builder.num_pageservers = 2 neon_env_builder.enable_pageserver_remote_storage( remote_storage_kind=RemoteStorageKind.MOCK_S3, From e961e0d3df1e7040221300fbb3d3e654257e4cad Mon Sep 17 00:00:00 2001 From: Joonas Koivunen Date: Wed, 20 Mar 2024 20:37:47 +0200 Subject: [PATCH 10/53] fix(Layer): always init after downloading in the spawned task (#7175) Before this PR, cancellation for `LayerInner::get_or_maybe_download` could occur so that we have downloaded the layer file in the filesystem, but because of the cancellation chance, we have not set the internal `LayerInner::inner` or initialized the state. With the detached init support introduced in #7135 and in place in #7152, we can now initialize the internal state after successfully downloading in the spawned task. The next PR will fix the remaining problems that this PR leaves: - `Layer::keep_resident` is still used because - `Layer::get_or_maybe_download` always cancels an eviction, even when canceled Split off from #7030. Stacked on top of #7152. Cc: #5331. --- pageserver/src/tenant/storage_layer/layer.rs | 330 +++++++++---------- 1 file changed, 161 insertions(+), 169 deletions(-) diff --git a/pageserver/src/tenant/storage_layer/layer.rs b/pageserver/src/tenant/storage_layer/layer.rs index eed423c3e69e..626fd69ef3df 100644 --- a/pageserver/src/tenant/storage_layer/layer.rs +++ b/pageserver/src/tenant/storage_layer/layer.rs @@ -702,6 +702,11 @@ impl LayerInner { allow_download: bool, ctx: Option<&RequestContext>, ) -> Result, DownloadError> { + // get_or_init_detached can: + // - be fast (mutex lock) OR uncontested semaphore permit acquire + // - be slow (wait for semaphore permit or closing) + let init_cancelled = scopeguard::guard((), |_| LAYER_IMPL_METRICS.inc_init_cancelled()); + let (weak, permit) = { let locked = self .inner @@ -736,6 +741,8 @@ impl LayerInner { } }; + scopeguard::ScopeGuard::into_inner(init_cancelled); + if let Some(weak) = weak { // only drop the weak after dropping the heavier_once_cell guard assert!( @@ -744,86 +751,57 @@ impl LayerInner { ); } - async move { - // disable any scheduled but not yet running eviction deletions for this - let next_version = 1 + self.version.fetch_add(1, Ordering::Relaxed); - - // no need to make the evict_and_wait wait for the actual download to complete - drop(self.status.send(Status::Downloaded)); + let timeline = self + .timeline + .upgrade() + .ok_or_else(|| DownloadError::TimelineShutdown)?; - let timeline = self - .timeline - .upgrade() - .ok_or_else(|| DownloadError::TimelineShutdown)?; + // count cancellations, which currently remain largely unexpected + let init_cancelled = scopeguard::guard((), |_| LAYER_IMPL_METRICS.inc_init_cancelled()); - // count cancellations, which currently remain largely unexpected - let init_cancelled = scopeguard::guard((), |_| LAYER_IMPL_METRICS.inc_init_cancelled()); - - // check if we really need to be downloaded; could have been already downloaded by a - // cancelled previous attempt. - let needs_download = self - .needs_download() - .await - .map_err(DownloadError::PreStatFailed); - - let needs_download = match needs_download { - Ok(reason) => reason, - Err(e) => { - scopeguard::ScopeGuard::into_inner(init_cancelled); - return Err(e); - } - }; + // check if we really need to be downloaded; could have been already downloaded by a + // cancelled previous attempt. + let needs_download = self + .needs_download() + .await + .map_err(DownloadError::PreStatFailed); - let Some(reason) = needs_download else { - scopeguard::ScopeGuard::into_inner(init_cancelled); + scopeguard::ScopeGuard::into_inner(init_cancelled); - // the file is present locally, probably by a previous but cancelled call to - // get_or_maybe_download. alternatively we might be running without remote storage. - LAYER_IMPL_METRICS.inc_init_needed_no_download(); + let needs_download = needs_download?; - let res = self.initialize_after_layer_is_on_disk(next_version, permit, false); - return Ok(res); - }; + let Some(reason) = needs_download else { + // the file is present locally, probably by a previous but cancelled call to + // get_or_maybe_download. alternatively we might be running without remote storage. + LAYER_IMPL_METRICS.inc_init_needed_no_download(); - if let NeedsDownload::NotFile(ft) = reason { - scopeguard::ScopeGuard::into_inner(init_cancelled); - return Err(DownloadError::NotFile(ft)); - } + return Ok(self.initialize_after_layer_is_on_disk(permit)); + }; - // only reset this after we've decided we really need to download. otherwise it'd - // be impossible to mark cancelled downloads for eviction, like one could imagine - // we would like to do for prefetching which was not needed. - self.wanted_evicted.store(false, Ordering::Release); + if let NeedsDownload::NotFile(ft) = reason { + return Err(DownloadError::NotFile(ft)); + } - if timeline.remote_client.as_ref().is_none() { - scopeguard::ScopeGuard::into_inner(init_cancelled); - return Err(DownloadError::NoRemoteStorage); - } + if timeline.remote_client.as_ref().is_none() { + return Err(DownloadError::NoRemoteStorage); + } - if let Some(ctx) = ctx { - let res = self.check_expected_download(ctx); - if let Err(e) = res { - scopeguard::ScopeGuard::into_inner(init_cancelled); - return Err(e); - } - } + if let Some(ctx) = ctx { + self.check_expected_download(ctx)?; + } - if !allow_download { - // this does look weird, but for LayerInner the "downloading" means also changing - // internal once related state ... - scopeguard::ScopeGuard::into_inner(init_cancelled); - return Err(DownloadError::DownloadRequired); - } + if !allow_download { + // this does look weird, but for LayerInner the "downloading" means also changing + // internal once related state ... + return Err(DownloadError::DownloadRequired); + } + async move { tracing::info!(%reason, "downloading on-demand"); - let permit = self.spawn_download_and_wait(timeline, permit).await; - + let init_cancelled = scopeguard::guard((), |_| LAYER_IMPL_METRICS.inc_init_cancelled()); + let res = self.download_init_and_wait(timeline, permit).await?; scopeguard::ScopeGuard::into_inner(init_cancelled); - - let permit = permit?; - - let res = self.initialize_after_layer_is_on_disk(next_version, permit, true); Ok(res) } .instrument(tracing::info_span!("get_or_maybe_download", layer=%self)) @@ -857,11 +835,11 @@ impl LayerInner { } /// Actual download, at most one is executed at the time. - async fn spawn_download_and_wait( + async fn download_init_and_wait( self: &Arc, timeline: Arc, permit: heavier_once_cell::InitPermit, - ) -> Result { + ) -> Result, DownloadError> { debug_assert_current_span_has_tenant_and_timeline_id(); let (tx, rx) = tokio::sync::oneshot::channel(); @@ -873,66 +851,24 @@ impl LayerInner { .enter() .map_err(|_| DownloadError::DownloadCancelled)?; - tokio::task::spawn(async move { - + tokio::task::spawn( + async move { let _guard = guard; - let client = timeline - .remote_client - .as_ref() - .expect("checked above with have_remote_client"); - - let result = client.download_layer_file( - &this.desc.filename(), - &this.metadata(), - &timeline.cancel - ) - .await; - - let result = match result { - Ok(size) => { - timeline.metrics.resident_physical_size_add(size); - Ok(()) - } - Err(e) => { - let consecutive_failures = - this.consecutive_failures.fetch_add(1, Ordering::Relaxed); - - let backoff = utils::backoff::exponential_backoff_duration_seconds( - consecutive_failures.min(u32::MAX as usize) as u32, - 1.5, - 60.0, - ); - - let backoff = std::time::Duration::from_secs_f64(backoff); - - tokio::select! { - _ = tokio::time::sleep(backoff) => {}, - _ = timeline.cancel.cancelled() => {}, - }; + drop(this.status.send(Status::Downloaded)); - Err(e) - } - }; + let res = this.download_and_init(timeline, permit).await; - if let Err(res) = tx.send((result, permit)) { + if let Err(res) = tx.send(res) { match res { - (Ok(()), _) => { - // our caller is cancellation safe so this is fine; if someone - // else requests the layer, they'll find it already downloaded. - // - // See counter [`LayerImplMetrics::inc_init_needed_no_download`] - // - // FIXME(#6028): however, could be that we should consider marking the - // layer for eviction? alas, cannot: because only DownloadedLayer will - // handle that. - }, - (Err(e), _) => { - // our caller is cancellation safe, but we might be racing with - // another attempt to initialize. before we have cancellation - // token support: these attempts should converge regardless of - // their completion order. - tracing::error!("layer file download failed, and additionally failed to communicate this to caller: {e:?}"); + Ok(_res) => { + tracing::debug!("layer initialized, but caller has been cancelled"); + LAYER_IMPL_METRICS.inc_init_completed_without_requester(); + } + Err(e) => { + tracing::info!( + "layer file download failed, and caller has been cancelled: {e:?}" + ); LAYER_IMPL_METRICS.inc_download_failed_without_requester(); } } @@ -942,41 +878,100 @@ impl LayerInner { ); match rx.await { - Ok((Ok(()), permit)) => { - if let Some(reason) = self - .needs_download() - .await - .map_err(DownloadError::PostStatFailed)? - { - // this is really a bug in needs_download or remote timeline client - panic!("post-condition failed: needs_download returned {reason:?}"); - } - - self.consecutive_failures.store(0, Ordering::Relaxed); - tracing::info!(size=%self.desc.file_size, "on-demand download successful"); - - Ok(permit) - } - Ok((Err(e), _permit)) => { + Ok(Ok(res)) => Ok(res), + Ok(Err(e)) => { // sleep already happened in the spawned task, if it was not cancelled - let consecutive_failures = self.consecutive_failures.load(Ordering::Relaxed); - match e.downcast_ref::() { // If the download failed due to its cancellation token, // propagate the cancellation error upstream. Some(remote_storage::DownloadError::Cancelled) => { Err(DownloadError::DownloadCancelled) } - _ => { - tracing::error!(consecutive_failures, "layer file download failed: {e:#}"); - Err(DownloadError::DownloadFailed) - } + _ => Err(DownloadError::DownloadFailed), } } Err(_gone) => Err(DownloadError::DownloadCancelled), } } + async fn download_and_init( + self: &Arc, + timeline: Arc, + permit: heavier_once_cell::InitPermit, + ) -> anyhow::Result> { + let client = timeline + .remote_client + .as_ref() + .expect("checked before download_init_and_wait"); + + let result = client + .download_layer_file(&self.desc.filename(), &self.metadata(), &timeline.cancel) + .await; + + match result { + Ok(size) => { + assert_eq!(size, self.desc.file_size); + + match self.needs_download().await { + Ok(Some(reason)) => { + // this is really a bug in needs_download or remote timeline client + panic!("post-condition failed: needs_download returned {reason:?}"); + } + Ok(None) => { + // as expected + } + Err(e) => { + panic!("post-condition failed: needs_download errored: {e:?}"); + } + } + + tracing::info!(size=%self.desc.file_size, "on-demand download successful"); + timeline + .metrics + .resident_physical_size_add(self.desc.file_size); + self.consecutive_failures.store(0, Ordering::Relaxed); + + let since_last_eviction = self + .last_evicted_at + .lock() + .unwrap() + .take() + .map(|ts| ts.elapsed()); + if let Some(since_last_eviction) = since_last_eviction { + LAYER_IMPL_METRICS.record_redownloaded_after(since_last_eviction); + } + + self.access_stats.record_residence_event( + LayerResidenceStatus::Resident, + LayerResidenceEventReason::ResidenceChange, + ); + + Ok(self.initialize_after_layer_is_on_disk(permit)) + } + Err(e) => { + let consecutive_failures = + 1 + self.consecutive_failures.fetch_add(1, Ordering::Relaxed); + + tracing::error!(consecutive_failures, "layer file download failed: {e:#}"); + + let backoff = utils::backoff::exponential_backoff_duration_seconds( + consecutive_failures.min(u32::MAX as usize) as u32, + 1.5, + 60.0, + ); + + let backoff = std::time::Duration::from_secs_f64(backoff); + + tokio::select! { + _ = tokio::time::sleep(backoff) => {}, + _ = timeline.cancel.cancelled() => {}, + }; + + Err(e) + } + } + } + /// Initializes the `Self::inner` to a "resident" state. /// /// Callers are assumed to ensure that the file is actually on disk with `Self::needs_download` @@ -986,25 +981,22 @@ impl LayerInner { /// changes are made before we can write to the OnceCell in non-cancellable fashion. fn initialize_after_layer_is_on_disk( self: &Arc, - next_version: usize, permit: heavier_once_cell::InitPermit, - downloaded: bool, ) -> Arc { debug_assert_current_span_has_tenant_and_timeline_id(); - if downloaded { - let since_last_eviction = self - .last_evicted_at - .lock() - .unwrap() - .take() - .map(|ts| ts.elapsed()); - if let Some(since_last_eviction) = since_last_eviction { - // FIXME: this will not always be recorded correctly until #6028 (the no - // download needed branch above) - LAYER_IMPL_METRICS.record_redownloaded_after(since_last_eviction); - } - } + // disable any scheduled but not yet running eviction deletions for this + let next_version = 1 + self.version.fetch_add(1, Ordering::Relaxed); + + // only reset this after we've decided we really need to download. otherwise it'd + // be impossible to mark cancelled downloads for eviction, like one could imagine + // we would like to do for prefetching which was not needed. + self.wanted_evicted.store(false, Ordering::Release); + + // re-send the notification we've already sent when we started to download, just so + // evict_and_wait does not need to wait for the download to complete. note that this is + // sent when initializing after finding the file on the disk. + drop(self.status.send(Status::Downloaded)); let res = Arc::new(DownloadedLayer { owner: Arc::downgrade(self), @@ -1012,15 +1004,9 @@ impl LayerInner { version: next_version, }); - // FIXME: this might now be double-accounted for !downloaded - self.access_stats.record_residence_event( - LayerResidenceStatus::Resident, - LayerResidenceEventReason::ResidenceChange, - ); - let waiters = self.inner.initializer_count(); if waiters > 0 { - tracing::info!(waiters, "completing the on-demand download for other tasks"); + tracing::info!(waiters, "completing layer init for other tasks"); } let value = ResidentOrWantedEvicted::Resident(res.clone()); @@ -1268,8 +1254,6 @@ pub(crate) enum DownloadError { DownloadCancelled, #[error("pre-condition: stat before download failed")] PreStatFailed(#[source] std::io::Error), - #[error("post-condition: stat after download failed")] - PostStatFailed(#[source] std::io::Error), } #[derive(Debug, PartialEq)] @@ -1694,6 +1678,12 @@ impl LayerImplMetrics { self.rare_counters[RareEvent::RemoveOnDropFailed].inc(); } + /// Expected rare just as cancellations are rare, but we could have cancellations separate from + /// the single caller which can start the download, so use this counter to separte them. + fn inc_init_completed_without_requester(&self) { + self.rare_counters[RareEvent::InitCompletedWithoutRequester].inc(); + } + /// Expected rare because cancellations are unexpected, and failures are unexpected fn inc_download_failed_without_requester(&self) { self.rare_counters[RareEvent::DownloadFailedWithoutRequester].inc(); @@ -1778,6 +1768,7 @@ impl DeleteFailed { #[derive(enum_map::Enum)] enum RareEvent { RemoveOnDropFailed, + InitCompletedWithoutRequester, DownloadFailedWithoutRequester, UpgradedWantedEvicted, InitWithoutDownload, @@ -1791,6 +1782,7 @@ impl RareEvent { match self { RemoveOnDropFailed => "remove_on_drop_failed", + InitCompletedWithoutRequester => "init_completed_without", DownloadFailedWithoutRequester => "download_failed_without", UpgradedWantedEvicted => "raced_wanted_evicted", InitWithoutDownload => "init_needed_no_download", From 34fa34d15c2a3fd13f3a475540991b5d9a63947a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Arpad=20M=C3=BCller?= Date: Wed, 20 Mar 2024 19:39:46 +0100 Subject: [PATCH 11/53] Dump layer map json in test_gc_feedback.py (#7179) The layer map json is an interesting file for that test, so dump it to make debugging easier. --- test_runner/fixtures/pageserver/http.py | 11 +++++++++++ test_runner/performance/test_gc_feedback.py | 7 +++++++ 2 files changed, 18 insertions(+) diff --git a/test_runner/fixtures/pageserver/http.py b/test_runner/fixtures/pageserver/http.py index 99ec894106a4..6aebfbc99cf7 100644 --- a/test_runner/fixtures/pageserver/http.py +++ b/test_runner/fixtures/pageserver/http.py @@ -626,6 +626,17 @@ def timeline_get_timestamp_of_lsn( res_json = res.json() return res_json + def timeline_layer_map_info( + self, tenant_id: Union[TenantId, TenantShardId], timeline_id: TimelineId + ): + log.info(f"Requesting layer map info of tenant {tenant_id}, timeline {timeline_id}") + res = self.get( + f"http://localhost:{self.port}/v1/tenant/{tenant_id}/timeline/{timeline_id}/layer", + ) + self.verbose_error(res) + res_json = res.json() + return res_json + def timeline_checkpoint( self, tenant_id: Union[TenantId, TenantShardId], diff --git a/test_runner/performance/test_gc_feedback.py b/test_runner/performance/test_gc_feedback.py index 48dd84fb0610..be56203b26b8 100644 --- a/test_runner/performance/test_gc_feedback.py +++ b/test_runner/performance/test_gc_feedback.py @@ -1,3 +1,5 @@ +import json + import pytest from fixtures.benchmark_fixture import MetricReport, NeonBenchmarker from fixtures.log_helper import log @@ -79,3 +81,8 @@ def test_gc_feedback(neon_env_builder: NeonEnvBuilder, zenbenchmark: NeonBenchma zenbenchmark.record( "physical/logical ratio", physical_size / logical_size, "", MetricReport.LOWER_IS_BETTER ) + + layer_map_path = env.repo_dir / "layer-map.json" + log.info(f"Writing layer map to {layer_map_path}") + with layer_map_path.open("w") as f: + f.write(json.dumps(client.timeline_layer_map_info(tenant_id, timeline_id))) From 5f0d9f2360e10bb9e3edc4978eda898be62f9fcb Mon Sep 17 00:00:00 2001 From: Alex Chi Z Date: Wed, 20 Mar 2024 14:40:48 -0400 Subject: [PATCH 12/53] fix: add safekeeper team to pgxn codeowners (#7170) `pgxn/` also contains WAL proposer code, so modifications to this directory should be able to be approved by the safekeeper team. Signed-off-by: Alex Chi Z --- CODEOWNERS | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/CODEOWNERS b/CODEOWNERS index 5b601f0566ac..9a23e8c95881 100644 --- a/CODEOWNERS +++ b/CODEOWNERS @@ -1,12 +1,13 @@ /compute_tools/ @neondatabase/control-plane @neondatabase/compute /control_plane/attachment_service @neondatabase/storage /libs/pageserver_api/ @neondatabase/storage -/libs/postgres_ffi/ @neondatabase/compute +/libs/postgres_ffi/ @neondatabase/compute @neondatabase/safekeepers /libs/remote_storage/ @neondatabase/storage /libs/safekeeper_api/ @neondatabase/safekeepers /libs/vm_monitor/ @neondatabase/autoscaling /pageserver/ @neondatabase/storage /pgxn/ @neondatabase/compute +/pgxn/neon/ @neondatabase/compute @neondatabase/safekeepers /proxy/ @neondatabase/proxy /safekeeper/ @neondatabase/safekeepers /vendor/ @neondatabase/compute From 55c4ef408b7e2305d1449c49d82d64ad095c949a Mon Sep 17 00:00:00 2001 From: Alex Chi Z Date: Wed, 20 Mar 2024 15:22:25 -0400 Subject: [PATCH 13/53] safekeeper: correctly handle signals (#7167) errno is not preserved in the signal handler. This pull request fixes it. Maybe related: https://github.com/neondatabase/neon/issues/6969, but does not fix the flaky test problem. Signed-off-by: Alex Chi Z --- pgxn/neon/walproposer_pg.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pgxn/neon/walproposer_pg.c b/pgxn/neon/walproposer_pg.c index c46fd9b3ec25..002bf4e2ce49 100644 --- a/pgxn/neon/walproposer_pg.c +++ b/pgxn/neon/walproposer_pg.c @@ -549,9 +549,10 @@ walprop_pg_init_standalone_sync_safekeepers(void) static void walprop_sigusr2(SIGNAL_ARGS) { + int save_errno = errno; got_SIGUSR2 = true; - SetLatch(MyLatch); + errno = save_errno; } static void From 041b653a1a31c369b349b2a7799af04379bb583b Mon Sep 17 00:00:00 2001 From: Tristan Partin Date: Mon, 18 Mar 2024 15:34:16 -0500 Subject: [PATCH 14/53] Add state diagram for compute Models a compute's lifetime. --- compute_tools/README.md | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/compute_tools/README.md b/compute_tools/README.md index 22a7de7cb761..8d84031efccb 100644 --- a/compute_tools/README.md +++ b/compute_tools/README.md @@ -32,6 +32,29 @@ compute_ctl -D /var/db/postgres/compute \ -b /usr/local/bin/postgres ``` +## State Diagram + +Computes can be in various states. Below is a diagram that details how a +compute moves between states. + +```mermaid +%% https://mermaid.js.org/syntax/stateDiagram.html +stateDiagram-v2 + [*] --> Empty : Compute spawned + Empty --> ConfigurationPending : Waiting for compute spec + ConfigurationPending --> Configuration : Received compute spec + Configuration --> Failed : Failed to configure the compute + Configuration --> Running : Compute has been configured + Empty --> Init : Compute spec is immediately available + Empty --> TerminationPending : Requested termination + Init --> Failed : Failed to start Postgres + Init --> Running : Started Postgres + Running --> TerminationPending : Requested termination + TerminationPending --> Terminated : Terminated compute + Failed --> [*] : Compute exited + Terminated --> [*] : Compute exited +``` + ## Tests Cargo formatter: From a95c41f463681eda15a89115f6f95aa20e55afa3 Mon Sep 17 00:00:00 2001 From: Joonas Koivunen Date: Thu, 21 Mar 2024 00:42:38 +0200 Subject: [PATCH 15/53] fix(heavier_once_cell): take_and_deinit should take ownership (#7185) Small fix to remove confusing `mut` bindings. Builds upon #7175, split off from #7030. Cc: #5331. --- libs/utils/src/sync/heavier_once_cell.rs | 4 ++-- pageserver/src/tenant/storage_layer/layer.rs | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/libs/utils/src/sync/heavier_once_cell.rs b/libs/utils/src/sync/heavier_once_cell.rs index a3aee45b583e..8eee1f72a6f2 100644 --- a/libs/utils/src/sync/heavier_once_cell.rs +++ b/libs/utils/src/sync/heavier_once_cell.rs @@ -245,7 +245,7 @@ impl<'a, T> Guard<'a, T> { /// /// The permit will be on a semaphore part of the new internal value, and any following /// [`OnceCell::get_or_init`] will wait on it to complete. - pub fn take_and_deinit(&mut self) -> (T, InitPermit) { + pub fn take_and_deinit(mut self) -> (T, InitPermit) { let mut swapped = Inner::default(); let sem = swapped.init_semaphore.clone(); // acquire and forget right away, moving the control over to InitPermit @@ -543,7 +543,7 @@ mod tests { target.set(42, permit); let (_answer, permit) = { - let mut guard = target + let guard = target .get_or_init(|permit| async { Ok::<_, Infallible>((11, permit)) }) .await .unwrap(); diff --git a/pageserver/src/tenant/storage_layer/layer.rs b/pageserver/src/tenant/storage_layer/layer.rs index 626fd69ef3df..c503d0d45445 100644 --- a/pageserver/src/tenant/storage_layer/layer.rs +++ b/pageserver/src/tenant/storage_layer/layer.rs @@ -729,7 +729,7 @@ impl LayerInner { return Ok(strong); } - Ok(Err(mut guard)) => { + Ok(Err(guard)) => { // path to here: the evict_blocking is stuck on spawn_blocking queue. // // reset the contents, deactivating the eviction and causing a @@ -1128,7 +1128,7 @@ impl LayerInner { let maybe_downloaded = self.inner.get(); let (_weak, permit) = match maybe_downloaded { - Some(mut guard) => { + Some(guard) => { if let ResidentOrWantedEvicted::WantedEvicted(_weak, version) = &*guard { if *version == only_version { guard.take_and_deinit() From 2206e14c261cb417a07e850a87fdf2b3cd9b07f8 Mon Sep 17 00:00:00 2001 From: Joonas Koivunen Date: Thu, 21 Mar 2024 03:19:08 +0200 Subject: [PATCH 16/53] fix(layer): remove the need to repair internal state (#7030) ## Problem The current implementation of struct Layer supports canceled read requests, but those will leave the internal state such that a following `Layer::keep_resident` call will need to repair the state. In pathological cases seen during generation numbers resetting in staging or with too many in-progress on-demand downloads, this repair activity will need to wait for the download to complete, which stalls disk usage-based eviction. Similar stalls have been observed in staging near disk-full situations, where downloads failed because the disk was full. Fixes #6028 or the "layer is present on filesystem but not evictable" problems by: 1. not canceling pending evictions by a canceled `LayerInner::get_or_maybe_download` 2. completing post-download initialization of the `LayerInner::inner` from the download task Not canceling evictions above case (1) and always initializing (2) lead to plain `LayerInner::inner` always having the up-to-date information, which leads to the old `Layer::keep_resident` never having to wait for downloads to complete. Finally, the `Layer::keep_resident` is replaced with `Layer::is_likely_resident`. These fix #7145. ## Summary of changes - add a new test showing that a canceled get_or_maybe_download should not cancel the eviction - switch to using a `watch` internally rather than a `broadcast` to avoid hanging eviction while a download is ongoing - doc changes for new semantics and cleanup - fix `Layer::keep_resident` to use just `self.0.inner.get()` as truth as `Layer::is_likely_resident` - remove `LayerInner::wanted_evicted` boolean as no longer needed Builds upon: #7185. Cc: #5331. --- pageserver/src/tenant/mgr.rs | 7 +- pageserver/src/tenant/storage_layer/layer.rs | 632 +++++++++++++----- .../tenant/storage_layer/layer/failpoints.rs | 119 ++++ .../src/tenant/storage_layer/layer/tests.rs | 591 +++++++++++++--- pageserver/src/tenant/timeline.rs | 13 +- .../src/tenant/timeline/eviction_task.rs | 23 +- .../src/tenant/timeline/layer_manager.rs | 28 +- 7 files changed, 1085 insertions(+), 328 deletions(-) create mode 100644 pageserver/src/tenant/storage_layer/layer/failpoints.rs diff --git a/pageserver/src/tenant/mgr.rs b/pageserver/src/tenant/mgr.rs index f456ca3006e1..7e0092d5b646 100644 --- a/pageserver/src/tenant/mgr.rs +++ b/pageserver/src/tenant/mgr.rs @@ -2,7 +2,6 @@ //! page server. use camino::{Utf8DirEntry, Utf8Path, Utf8PathBuf}; -use futures::stream::StreamExt; use itertools::Itertools; use pageserver_api::key::Key; use pageserver_api::models::ShardParameters; @@ -1662,9 +1661,9 @@ impl TenantManager { .layers .read() .await - .resident_layers() - .collect::>() - .await; + .likely_resident_layers() + .collect::>(); + for layer in timeline_layers { let relative_path = layer .local_path() diff --git a/pageserver/src/tenant/storage_layer/layer.rs b/pageserver/src/tenant/storage_layer/layer.rs index c503d0d45445..8ba37b5a8640 100644 --- a/pageserver/src/tenant/storage_layer/layer.rs +++ b/pageserver/src/tenant/storage_layer/layer.rs @@ -32,6 +32,9 @@ use utils::generation::Generation; #[cfg(test)] mod tests; +#[cfg(test)] +mod failpoints; + /// A Layer contains all data in a "rectangle" consisting of a range of keys and /// range of LSNs. /// @@ -46,7 +49,41 @@ mod tests; /// An image layer is a snapshot of all the data in a key-range, at a single /// LSN. /// -/// This type models the on-disk layers, which can be evicted and on-demand downloaded. +/// This type models the on-disk layers, which can be evicted and on-demand downloaded. As a +/// general goal, read accesses should always win eviction and eviction should not wait for +/// download. +/// +/// ### State transitions +/// +/// The internal state of `Layer` is composed of most importantly the on-filesystem state and the +/// [`ResidentOrWantedEvicted`] enum. On-filesystem state can be either present (fully downloaded, +/// right size) or deleted. +/// +/// Reads will always win requests to evict until `wait_for_turn_and_evict` has acquired the +/// `heavier_once_cell::InitPermit` and has started to `evict_blocking`. Before the +/// `heavier_once_cell::InitPermit` has been acquired, any read request +/// (`get_or_maybe_download`) can "re-initialize" using the existing downloaded file and thus +/// cancelling the eviction. +/// +/// ```text +/// +-----------------+ get_or_maybe_download +--------------------------------+ +/// | not initialized |--------------------------->| Resident(Arc) | +/// | ENOENT | /->| | +/// +-----------------+ | +--------------------------------+ +/// ^ | | ^ +/// | get_or_maybe_download | | | get_or_maybe_download, either: +/// evict_blocking | /-------------------------/ | | - upgrade weak to strong +/// | | | | - re-initialize without download +/// | | evict_and_wait | | +/// +-----------------+ v | +/// | not initialized | on_downloaded_layer_drop +--------------------------------------+ +/// | file is present |<---------------------------| WantedEvicted(Weak) | +/// +-----------------+ +--------------------------------------+ +/// ``` +/// +/// ### Unsupported +/// +/// - Evicting by the operator deleting files from the filesystem /// /// [`InMemoryLayer`]: super::inmemory_layer::InMemoryLayer #[derive(Clone)] @@ -211,8 +248,7 @@ impl Layer { /// /// Timeout is mandatory, because waiting for eviction is only needed for our tests; eviction /// will happen regardless the future returned by this method completing unless there is a - /// read access (currently including [`Layer::keep_resident`]) before eviction gets to - /// complete. + /// read access before eviction gets to complete. /// /// Technically cancellation safe, but cancelling might shift the viewpoint of what generation /// of download-evict cycle on retry. @@ -307,21 +343,28 @@ impl Layer { /// Assuming the layer is already downloaded, returns a guard which will prohibit eviction /// while the guard exists. /// - /// Returns None if the layer is currently evicted. - pub(crate) async fn keep_resident(&self) -> anyhow::Result> { - let downloaded = match self.0.get_or_maybe_download(false, None).await { - Ok(d) => d, - // technically there are a lot of possible errors, but in practice it should only be - // DownloadRequired which is tripped up. could work to improve this situation - // statically later. - Err(DownloadError::DownloadRequired) => return Ok(None), - Err(e) => return Err(e.into()), - }; + /// Returns None if the layer is currently evicted or becoming evicted. + #[cfg(test)] + pub(crate) async fn keep_resident(&self) -> Option { + let downloaded = self.0.inner.get().and_then(|rowe| rowe.get())?; - Ok(Some(ResidentLayer { + Some(ResidentLayer { downloaded, owner: self.clone(), - })) + }) + } + + /// Weak indicator of is the layer resident or not. Good enough for eviction, which can deal + /// with `EvictionError::NotFound`. + /// + /// Returns `true` if this layer might be resident, or `false`, if it most likely evicted or + /// will be unless a read happens soon. + pub(crate) fn is_likely_resident(&self) -> bool { + self.0 + .inner + .get() + .map(|rowe| rowe.is_likely_resident()) + .unwrap_or(false) } /// Downloads if necessary and creates a guard, which will keep this layer from being evicted. @@ -371,11 +414,11 @@ impl Layer { /// separatedly. #[cfg(any(feature = "testing", test))] pub(crate) fn wait_drop(&self) -> impl std::future::Future + 'static { - let mut rx = self.0.status.subscribe(); + let mut rx = self.0.status.as_ref().unwrap().subscribe(); async move { loop { - if let Err(tokio::sync::broadcast::error::RecvError::Closed) = rx.recv().await { + if rx.changed().await.is_err() { break; } } @@ -397,6 +440,32 @@ enum ResidentOrWantedEvicted { } impl ResidentOrWantedEvicted { + /// Non-mutating access to the a DownloadedLayer, if possible. + /// + /// This is not used on the read path (anything that calls + /// [`LayerInner::get_or_maybe_download`]) because it was decided that reads always win + /// evictions, and part of that winning is using [`ResidentOrWantedEvicted::get_and_upgrade`]. + #[cfg(test)] + fn get(&self) -> Option> { + match self { + ResidentOrWantedEvicted::Resident(strong) => Some(strong.clone()), + ResidentOrWantedEvicted::WantedEvicted(weak, _) => weak.upgrade(), + } + } + + /// Best-effort query for residency right now, not as strong guarantee as receiving a strong + /// reference from `ResidentOrWantedEvicted::get`. + fn is_likely_resident(&self) -> bool { + match self { + ResidentOrWantedEvicted::Resident(_) => true, + ResidentOrWantedEvicted::WantedEvicted(weak, _) => weak.strong_count() > 0, + } + } + + /// Upgrades any weak to strong if possible. + /// + /// Returns a strong reference if possible, along with a boolean telling if an upgrade + /// happened. fn get_and_upgrade(&mut self) -> Option<(Arc, bool)> { match self { ResidentOrWantedEvicted::Resident(strong) => Some((strong.clone(), false)), @@ -417,7 +486,7 @@ impl ResidentOrWantedEvicted { /// /// Returns `Some` if this was the first time eviction was requested. Care should be taken to /// drop the possibly last strong reference outside of the mutex of - /// heavier_once_cell::OnceCell. + /// [`heavier_once_cell::OnceCell`]. fn downgrade(&mut self) -> Option> { match self { ResidentOrWantedEvicted::Resident(strong) => { @@ -445,6 +514,9 @@ struct LayerInner { desc: PersistentLayerDesc, /// Timeline access is needed for remote timeline client and metrics. + /// + /// There should not be an access to timeline for any reason without entering the + /// [`Timeline::gate`] at the same time. timeline: Weak, /// Cached knowledge of [`Timeline::remote_client`] being `Some`. @@ -453,27 +525,38 @@ struct LayerInner { access_stats: LayerAccessStats, /// This custom OnceCell is backed by std mutex, but only held for short time periods. - /// Initialization and deinitialization are done while holding a permit. + /// + /// Filesystem changes (download, evict) are only done while holding a permit which the + /// `heavier_once_cell` provides. + /// + /// A number of fields in `Layer` are meant to only be updated when holding the InitPermit, but + /// possibly read while not holding it. inner: heavier_once_cell::OnceCell, /// Do we want to delete locally and remotely this when `LayerInner` is dropped wanted_deleted: AtomicBool, - /// Do we want to evict this layer as soon as possible? After being set to `true`, all accesses - /// will try to downgrade [`ResidentOrWantedEvicted`], which will eventually trigger - /// [`LayerInner::on_downloaded_layer_drop`]. - wanted_evicted: AtomicBool, - - /// Version is to make sure we will only evict a specific download of a file. + /// Version is to make sure we will only evict a specific initialization of the downloaded file. /// - /// Incremented for each download, stored in `DownloadedLayer::version` or + /// Incremented for each initialization, stored in `DownloadedLayer::version` or /// `ResidentOrWantedEvicted::WantedEvicted`. version: AtomicUsize, - /// Allow subscribing to when the layer actually gets evicted. - status: tokio::sync::broadcast::Sender, + /// Allow subscribing to when the layer actually gets evicted, a non-cancellable download + /// starts, or completes. + /// + /// Updates must only be posted while holding the InitPermit or the heavier_once_cell::Guard. + /// Holding the InitPermit is the only time we can do state transitions, but we also need to + /// cancel a pending eviction on upgrading a [`ResidentOrWantedEvicted::WantedEvicted`] back to + /// [`ResidentOrWantedEvicted::Resident`] on access. + /// + /// The sender is wrapped in an Option to facilitate moving it out on [`LayerInner::drop`]. + status: Option>, - /// Counter for exponential backoff with the download + /// Counter for exponential backoff with the download. + /// + /// This is atomic only for the purposes of having additional data only accessed while holding + /// the InitPermit. consecutive_failures: AtomicUsize, /// The generation of this Layer. @@ -491,7 +574,13 @@ struct LayerInner { /// a shard split since the layer was originally written. shard: ShardIndex, + /// When the Layer was last evicted but has not been downloaded since. + /// + /// This is used solely for updating metrics. See [`LayerImplMetrics::redownload_after`]. last_evicted_at: std::sync::Mutex>, + + #[cfg(test)] + failpoints: std::sync::Mutex>, } impl std::fmt::Display for LayerInner { @@ -508,16 +597,16 @@ impl AsLayerDesc for LayerInner { #[derive(Debug, Clone, Copy)] enum Status { + Resident, Evicted, - Downloaded, + Downloading, } impl Drop for LayerInner { fn drop(&mut self) { if !*self.wanted_deleted.get_mut() { - // should we try to evict if the last wish was for eviction? - // feels like there's some hazard of overcrowding near shutdown near by, but we don't - // run drops during shutdown (yet) + // should we try to evict if the last wish was for eviction? seems more like a hazard + // than a clear win. return; } @@ -528,9 +617,9 @@ impl Drop for LayerInner { let file_size = self.layer_desc().file_size; let timeline = self.timeline.clone(); let meta = self.metadata(); - let status = self.status.clone(); + let status = self.status.take(); - crate::task_mgr::BACKGROUND_RUNTIME.spawn_blocking(move || { + Self::spawn_blocking(move || { let _g = span.entered(); // carry this until we are finished for [`Layer::wait_drop`] support @@ -605,12 +694,16 @@ impl LayerInner { .timeline_path(&timeline.tenant_shard_id, &timeline.timeline_id) .join(desc.filename().to_string()); - let (inner, version) = if let Some(inner) = downloaded { + let (inner, version, init_status) = if let Some(inner) = downloaded { let version = inner.version; let resident = ResidentOrWantedEvicted::Resident(inner); - (heavier_once_cell::OnceCell::new(resident), version) + ( + heavier_once_cell::OnceCell::new(resident), + version, + Status::Resident, + ) } else { - (heavier_once_cell::OnceCell::default(), 0) + (heavier_once_cell::OnceCell::default(), 0, Status::Evicted) }; LayerInner { @@ -621,14 +714,15 @@ impl LayerInner { have_remote_client: timeline.remote_client.is_some(), access_stats, wanted_deleted: AtomicBool::new(false), - wanted_evicted: AtomicBool::new(false), inner, version: AtomicUsize::new(version), - status: tokio::sync::broadcast::channel(1).0, + status: Some(tokio::sync::watch::channel(init_status).0), consecutive_failures: AtomicUsize::new(0), generation, shard, last_evicted_at: std::sync::Mutex::default(), + #[cfg(test)] + failpoints: Default::default(), } } @@ -644,20 +738,34 @@ impl LayerInner { /// Cancellation safe, however dropping the future and calling this method again might result /// in a new attempt to evict OR join the previously started attempt. + #[tracing::instrument(level = tracing::Level::DEBUG, skip_all, ret, err(level = tracing::Level::DEBUG), fields(layer=%self))] pub(crate) async fn evict_and_wait(&self, timeout: Duration) -> Result<(), EvictionError> { - use tokio::sync::broadcast::error::RecvError; - assert!(self.have_remote_client); - let mut rx = self.status.subscribe(); + let mut rx = self.status.as_ref().unwrap().subscribe(); + + { + let current = rx.borrow_and_update(); + match &*current { + Status::Resident => { + // we might get lucky and evict this; continue + } + Status::Evicted | Status::Downloading => { + // it is already evicted + return Err(EvictionError::NotFound); + } + } + } let strong = { match self.inner.get() { - Some(mut either) => { - self.wanted_evicted.store(true, Ordering::Relaxed); - either.downgrade() + Some(mut either) => either.downgrade(), + None => { + // we already have a scheduled eviction, which just has not gotten to run yet. + // it might still race with a read access, but that could also get cancelled, + // so let's say this is not evictable. + return Err(EvictionError::NotFound); } - None => return Err(EvictionError::NotFound), } }; @@ -673,26 +781,26 @@ impl LayerInner { LAYER_IMPL_METRICS.inc_started_evictions(); } - match tokio::time::timeout(timeout, rx.recv()).await { - Ok(Ok(Status::Evicted)) => Ok(()), - Ok(Ok(Status::Downloaded)) => Err(EvictionError::Downloaded), - Ok(Err(RecvError::Closed)) => { - unreachable!("sender cannot be dropped while we are in &self method") - } - Ok(Err(RecvError::Lagged(_))) => { - // this is quite unlikely, but we are blocking a lot in the async context, so - // we might be missing this because we are stuck on a LIFO slot on a thread - // which is busy blocking for a 1TB database create_image_layers. - // - // use however late (compared to the initial expressing of wanted) as the - // "outcome" now - LAYER_IMPL_METRICS.inc_broadcast_lagged(); - match self.inner.get() { - Some(_) => Err(EvictionError::Downloaded), - None => Ok(()), - } - } - Err(_timeout) => Err(EvictionError::Timeout), + let changed = rx.changed(); + let changed = tokio::time::timeout(timeout, changed).await; + + let Ok(changed) = changed else { + return Err(EvictionError::Timeout); + }; + + let _: () = changed.expect("cannot be closed, because we are holding a strong reference"); + + let current = rx.borrow_and_update(); + + match &*current { + // the easiest case + Status::Evicted => Ok(()), + // it surely was evicted in between, but then there was a new access now; we can't know + // if it'll succeed so lets just call it evicted + Status::Downloading => Ok(()), + // either the download which was started after eviction completed already, or it was + // never evicted + Status::Resident => Err(EvictionError::Downloaded), } } @@ -702,38 +810,38 @@ impl LayerInner { allow_download: bool, ctx: Option<&RequestContext>, ) -> Result, DownloadError> { - // get_or_init_detached can: - // - be fast (mutex lock) OR uncontested semaphore permit acquire - // - be slow (wait for semaphore permit or closing) - let init_cancelled = scopeguard::guard((), |_| LAYER_IMPL_METRICS.inc_init_cancelled()); - let (weak, permit) = { + // get_or_init_detached can: + // - be fast (mutex lock) OR uncontested semaphore permit acquire + // - be slow (wait for semaphore permit or closing) + let init_cancelled = scopeguard::guard((), |_| LAYER_IMPL_METRICS.inc_init_cancelled()); + let locked = self .inner .get_or_init_detached() .await .map(|mut guard| guard.get_and_upgrade().ok_or(guard)); + scopeguard::ScopeGuard::into_inner(init_cancelled); + match locked { // this path could had been a RwLock::read Ok(Ok((strong, upgraded))) if !upgraded => return Ok(strong), Ok(Ok((strong, _))) => { // when upgraded back, the Arc is still available, but - // previously a `evict_and_wait` was received. - self.wanted_evicted.store(false, Ordering::Relaxed); - - // error out any `evict_and_wait` - drop(self.status.send(Status::Downloaded)); + // previously a `evict_and_wait` was received. this is the only place when we + // send out an update without holding the InitPermit. + // + // note that we also have dropped the Guard; this is fine, because we just made + // a state change and are holding a strong reference to be returned. + self.status.as_ref().unwrap().send_replace(Status::Resident); LAYER_IMPL_METRICS .inc_eviction_cancelled(EvictionCancelled::UpgradedBackOnAccess); return Ok(strong); } Ok(Err(guard)) => { - // path to here: the evict_blocking is stuck on spawn_blocking queue. - // - // reset the contents, deactivating the eviction and causing a - // EvictionCancelled::LostToDownload or EvictionCancelled::VersionCheckFailed. + // path to here: we won the eviction, the file should still be on the disk. let (weak, permit) = guard.take_and_deinit(); (Some(weak), permit) } @@ -741,8 +849,6 @@ impl LayerInner { } }; - scopeguard::ScopeGuard::into_inner(init_cancelled); - if let Some(weak) = weak { // only drop the weak after dropping the heavier_once_cell guard assert!( @@ -759,8 +865,11 @@ impl LayerInner { // count cancellations, which currently remain largely unexpected let init_cancelled = scopeguard::guard((), |_| LAYER_IMPL_METRICS.inc_init_cancelled()); - // check if we really need to be downloaded; could have been already downloaded by a - // cancelled previous attempt. + // check if we really need to be downloaded: this can happen if a read access won the + // semaphore before eviction. + // + // if we are cancelled while doing this `stat` the `self.inner` will be uninitialized. a + // pending eviction will try to evict even upon finding an uninitialized `self.inner`. let needs_download = self .needs_download() .await @@ -771,13 +880,20 @@ impl LayerInner { let needs_download = needs_download?; let Some(reason) = needs_download else { - // the file is present locally, probably by a previous but cancelled call to - // get_or_maybe_download. alternatively we might be running without remote storage. + // the file is present locally because eviction has not had a chance to run yet + + #[cfg(test)] + self.failpoint(failpoints::FailpointKind::AfterDeterminingLayerNeedsNoDownload) + .await?; + LAYER_IMPL_METRICS.inc_init_needed_no_download(); return Ok(self.initialize_after_layer_is_on_disk(permit)); }; + // we must download; getting cancelled before spawning the download is not an issue as + // any still running eviction would not find anything to evict. + if let NeedsDownload::NotFile(ft) = reason { return Err(DownloadError::NotFile(ft)); } @@ -791,8 +907,7 @@ impl LayerInner { } if !allow_download { - // this does look weird, but for LayerInner the "downloading" means also changing - // internal once related state ... + // this is only used from tests, but it is hard to test without the boolean return Err(DownloadError::DownloadRequired); } @@ -851,11 +966,22 @@ impl LayerInner { .enter() .map_err(|_| DownloadError::DownloadCancelled)?; - tokio::task::spawn( + Self::spawn( async move { let _guard = guard; - drop(this.status.send(Status::Downloaded)); + // now that we have commited to downloading, send out an update to: + // - unhang any pending eviction + // - break out of evict_and_wait + this.status + .as_ref() + .unwrap() + .send_replace(Status::Downloading); + + #[cfg(test)] + this.failpoint(failpoints::FailpointKind::WaitBeforeDownloading) + .await + .unwrap(); let res = this.download_and_init(timeline, permit).await; @@ -887,6 +1013,8 @@ impl LayerInner { Some(remote_storage::DownloadError::Cancelled) => { Err(DownloadError::DownloadCancelled) } + // FIXME: this is not embedding the error because historically it would had + // been output to compute, however that is no longer the case. _ => Err(DownloadError::DownloadFailed), } } @@ -985,18 +1113,9 @@ impl LayerInner { ) -> Arc { debug_assert_current_span_has_tenant_and_timeline_id(); - // disable any scheduled but not yet running eviction deletions for this + // disable any scheduled but not yet running eviction deletions for this initialization let next_version = 1 + self.version.fetch_add(1, Ordering::Relaxed); - - // only reset this after we've decided we really need to download. otherwise it'd - // be impossible to mark cancelled downloads for eviction, like one could imagine - // we would like to do for prefetching which was not needed. - self.wanted_evicted.store(false, Ordering::Release); - - // re-send the notification we've already sent when we started to download, just so - // evict_and_wait does not need to wait for the download to complete. note that this is - // sent when initializing after finding the file on the disk. - drop(self.status.send(Status::Downloaded)); + self.status.as_ref().unwrap().send_replace(Status::Resident); let res = Arc::new(DownloadedLayer { owner: Arc::downgrade(self), @@ -1049,9 +1168,11 @@ impl LayerInner { fn info(&self, reset: LayerAccessStatsReset) -> HistoricLayerInfo { let layer_file_name = self.desc.filename().file_name(); - // this is not accurate: we could have the file locally but there was a cancellation - // and now we are not in sync, or we are currently downloading it. - let remote = self.inner.get().is_none(); + let resident = self + .inner + .get() + .map(|rowe| rowe.is_likely_resident()) + .unwrap_or(false); let access_stats = self.access_stats.as_api_model(reset); @@ -1063,7 +1184,7 @@ impl LayerInner { layer_file_size: self.desc.file_size, lsn_start: lsn_range.start, lsn_end: lsn_range.end, - remote, + remote: !resident, access_stats, } } else { @@ -1073,94 +1194,195 @@ impl LayerInner { layer_file_name, layer_file_size: self.desc.file_size, lsn_start: lsn, - remote, + remote: !resident, access_stats, } } } /// `DownloadedLayer` is being dropped, so it calls this method. - fn on_downloaded_layer_drop(self: Arc, version: usize) { - let evict = self.wanted_evicted.load(Ordering::Acquire); + fn on_downloaded_layer_drop(self: Arc, only_version: usize) { let can_evict = self.have_remote_client; - if can_evict && evict { - let span = tracing::info_span!(parent: None, "layer_evict", tenant_id = %self.desc.tenant_shard_id.tenant_id, shard_id = %self.desc.tenant_shard_id.shard_slug(), timeline_id = %self.desc.timeline_id, layer=%self, %version); + // we cannot know without inspecting LayerInner::inner if we should evict or not, even + // though here it is very likely + let span = tracing::info_span!(parent: None, "layer_evict", tenant_id = %self.desc.tenant_shard_id.tenant_id, shard_id = %self.desc.tenant_shard_id.shard_slug(), timeline_id = %self.desc.timeline_id, layer=%self, version=%only_version); - // downgrade for queueing, in case there's a tear down already ongoing we should not - // hold it alive. - let this = Arc::downgrade(&self); - drop(self); + if !can_evict { + // it would be nice to assert this case out, but we are in drop + span.in_scope(|| { + tracing::error!("bug in struct Layer: ResidentOrWantedEvicted has been downgraded while we have no remote storage"); + }); + return; + } - // NOTE: this scope *must* never call `self.inner.get` because evict_and_wait might - // drop while the `self.inner` is being locked, leading to a deadlock. + // NOTE: this scope *must* never call `self.inner.get` because evict_and_wait might + // drop while the `self.inner` is being locked, leading to a deadlock. - crate::task_mgr::BACKGROUND_RUNTIME.spawn_blocking(move || { - let _g = span.entered(); + let start_evicting = async move { + #[cfg(test)] + self.failpoint(failpoints::FailpointKind::WaitBeforeStartingEvicting) + .await + .expect("failpoint should not have errored"); - // if LayerInner is already dropped here, do nothing because the delete on drop - // has already ran while we were in queue - let Some(this) = this.upgrade() else { - LAYER_IMPL_METRICS.inc_eviction_cancelled(EvictionCancelled::LayerGone); - return; - }; - match this.evict_blocking(version) { - Ok(()) => LAYER_IMPL_METRICS.inc_completed_evictions(), - Err(reason) => LAYER_IMPL_METRICS.inc_eviction_cancelled(reason), - } - }); - } - } + tracing::debug!("eviction started"); - fn evict_blocking(&self, only_version: usize) -> Result<(), EvictionCancelled> { - // deleted or detached timeline, don't do anything. - let Some(timeline) = self.timeline.upgrade() else { - return Err(EvictionCancelled::TimelineGone); + let res = self.wait_for_turn_and_evict(only_version).await; + // metrics: ignore the Ok branch, it is not done yet + if let Err(e) = res { + tracing::debug!(res=?Err::<(), _>(&e), "eviction completed"); + LAYER_IMPL_METRICS.inc_eviction_cancelled(e); + } }; + Self::spawn(start_evicting.instrument(span)); + } + + async fn wait_for_turn_and_evict( + self: Arc, + only_version: usize, + ) -> Result<(), EvictionCancelled> { + fn is_good_to_continue(status: &Status) -> Result<(), EvictionCancelled> { + use Status::*; + match status { + Resident => Ok(()), + Evicted => Err(EvictionCancelled::UnexpectedEvictedState), + Downloading => Err(EvictionCancelled::LostToDownload), + } + } + + let timeline = self + .timeline + .upgrade() + .ok_or(EvictionCancelled::TimelineGone)?; + + let mut rx = self + .status + .as_ref() + .expect("LayerInner cannot be dropped, holding strong ref") + .subscribe(); + + is_good_to_continue(&rx.borrow_and_update())?; + let Ok(_gate) = timeline.gate.enter() else { return Err(EvictionCancelled::TimelineGone); }; - // to avoid starting a new download while we evict, keep holding on to the - // permit. - let _permit = { - let maybe_downloaded = self.inner.get(); - - let (_weak, permit) = match maybe_downloaded { - Some(guard) => { - if let ResidentOrWantedEvicted::WantedEvicted(_weak, version) = &*guard { - if *version == only_version { - guard.take_and_deinit() - } else { - // this was not for us; maybe there's another eviction job - // TODO: does it make any sense to stall here? unique versions do not - // matter, we only want to make sure not to evict a resident, which we - // are not doing. + let permit = { + // we cannot just `std::fs::remove_file` because there might already be an + // get_or_maybe_download which will inspect filesystem and reinitialize. filesystem + // operations must be done while holding the heavier_once_cell::InitPermit + let mut wait = std::pin::pin!(self.inner.get_or_init_detached()); + + let waited = loop { + // we must race to the Downloading starting, otherwise we would have to wait until the + // completion of the download. waiting for download could be long and hinder our + // efforts to alert on "hanging" evictions. + tokio::select! { + res = &mut wait => break res, + _ = rx.changed() => { + is_good_to_continue(&rx.borrow_and_update())?; + // two possibilities for Status::Resident: + // - the layer was found locally from disk by a read + // - we missed a bunch of updates and now the layer is + // again downloaded -- assume we'll fail later on with + // version check or AlreadyReinitialized + } + } + }; + + // re-check now that we have the guard or permit; all updates should have happened + // while holding the permit. + is_good_to_continue(&rx.borrow_and_update())?; + + // the term deinitialize is used here, because we clearing out the Weak will eventually + // lead to deallocating the reference counted value, and the value we + // `Guard::take_and_deinit` is likely to be the last because the Weak is never cloned. + let (_weak, permit) = match waited { + Ok(guard) => { + match &*guard { + ResidentOrWantedEvicted::WantedEvicted(_weak, version) + if *version == only_version => + { + tracing::debug!(version, "deinitializing matching WantedEvicted"); + let (weak, permit) = guard.take_and_deinit(); + (Some(weak), permit) + } + ResidentOrWantedEvicted::WantedEvicted(_, version) => { + // if we were not doing the version check, we would need to try to + // upgrade the weak here to see if it really is dropped. version check + // is done instead assuming that it is cheaper. + tracing::debug!( + version, + only_version, + "version mismatch, not deinitializing" + ); return Err(EvictionCancelled::VersionCheckFailed); } - } else { - return Err(EvictionCancelled::AlreadyReinitialized); + ResidentOrWantedEvicted::Resident(_) => { + return Err(EvictionCancelled::AlreadyReinitialized); + } } } - None => { - // already deinitialized, perhaps get_or_maybe_download did this and is - // currently waiting to reinitialize it - return Err(EvictionCancelled::LostToDownload); + Err(permit) => { + tracing::debug!("continuing after cancelled get_or_maybe_download or eviction"); + (None, permit) } }; permit }; - // now accesses to inner.get_or_init wait on the semaphore or the `_permit` + let span = tracing::Span::current(); - self.access_stats.record_residence_event( - LayerResidenceStatus::Evicted, - LayerResidenceEventReason::ResidenceChange, - ); + let spawned_at = std::time::Instant::now(); - let res = match capture_mtime_and_remove(&self.path) { + // this is on purpose a detached spawn; we don't need to wait for it + // + // eviction completion reporting is the only thing hinging on this, and it can be just as + // well from a spawn_blocking thread. + // + // important to note that now that we've acquired the permit we have made sure the evicted + // file is either the exact `WantedEvicted` we wanted to evict, or uninitialized in case + // there are multiple evictions. The rest is not cancellable, and we've now commited to + // evicting. + // + // If spawn_blocking has a queue and maximum number of threads are in use, we could stall + // reads. We will need to add cancellation for that if necessary. + Self::spawn_blocking(move || { + let _span = span.entered(); + + let res = self.evict_blocking(&timeline, &permit); + + let waiters = self.inner.initializer_count(); + + if waiters > 0 { + LAYER_IMPL_METRICS.inc_evicted_with_waiters(); + } + + let completed_in = spawned_at.elapsed(); + LAYER_IMPL_METRICS.record_time_to_evict(completed_in); + + match res { + Ok(()) => LAYER_IMPL_METRICS.inc_completed_evictions(), + Err(e) => LAYER_IMPL_METRICS.inc_eviction_cancelled(e), + } + + tracing::debug!(?res, elapsed_ms=%completed_in.as_millis(), %waiters, "eviction completed"); + }); + + Ok(()) + } + + /// This is blocking only to do just one spawn_blocking hop compared to multiple via tokio::fs. + fn evict_blocking( + &self, + timeline: &Timeline, + _permit: &heavier_once_cell::InitPermit, + ) -> Result<(), EvictionCancelled> { + // now accesses to `self.inner.get_or_init*` wait on the semaphore or the `_permit` + + match capture_mtime_and_remove(&self.path) { Ok(local_layer_mtime) => { let duration = SystemTime::now().duration_since(local_layer_mtime); match duration { @@ -1184,33 +1406,60 @@ impl LayerInner { timeline .metrics .resident_physical_size_sub(self.desc.file_size); - - Ok(()) } Err(e) if e.kind() == std::io::ErrorKind::NotFound => { tracing::error!( layer_size = %self.desc.file_size, - "failed to evict layer from disk, it was already gone (metrics will be inaccurate)" + "failed to evict layer from disk, it was already gone" ); - Err(EvictionCancelled::FileNotFound) + return Err(EvictionCancelled::FileNotFound); } Err(e) => { + // FIXME: this should probably be an abort tracing::error!("failed to evict file from disk: {e:#}"); - Err(EvictionCancelled::RemoveFailed) + return Err(EvictionCancelled::RemoveFailed); } - }; + } + + self.access_stats.record_residence_event( + LayerResidenceStatus::Evicted, + LayerResidenceEventReason::ResidenceChange, + ); - // we are still holding the permit, so no new spawn_download_and_wait can happen - drop(self.status.send(Status::Evicted)); + self.status.as_ref().unwrap().send_replace(Status::Evicted); *self.last_evicted_at.lock().unwrap() = Some(std::time::Instant::now()); - res + Ok(()) } fn metadata(&self) -> LayerFileMetadata { LayerFileMetadata::new(self.desc.file_size, self.generation, self.shard) } + + /// Needed to use entered runtime in tests, but otherwise use BACKGROUND_RUNTIME. + /// + /// Synchronizing with spawned tasks is very complicated otherwise. + fn spawn(fut: F) + where + F: std::future::Future + Send + 'static, + { + #[cfg(test)] + tokio::task::spawn(fut); + #[cfg(not(test))] + crate::task_mgr::BACKGROUND_RUNTIME.spawn(fut); + } + + /// Needed to use entered runtime in tests, but otherwise use BACKGROUND_RUNTIME. + fn spawn_blocking(f: F) + where + F: FnOnce() + Send + 'static, + { + #[cfg(test)] + tokio::task::spawn_blocking(f); + #[cfg(not(test))] + crate::task_mgr::BACKGROUND_RUNTIME.spawn_blocking(f); + } } fn capture_mtime_and_remove(path: &Utf8Path) -> Result { @@ -1254,6 +1503,10 @@ pub(crate) enum DownloadError { DownloadCancelled, #[error("pre-condition: stat before download failed")] PreStatFailed(#[source] std::io::Error), + + #[cfg(test)] + #[error("failpoint: {0:?}")] + Failpoint(failpoints::FailpointKind), } #[derive(Debug, PartialEq)] @@ -1300,6 +1553,7 @@ impl Drop for DownloadedLayer { owner.on_downloaded_layer_drop(self.version); } else { // no need to do anything, we are shutting down + LAYER_IMPL_METRICS.inc_eviction_cancelled(EvictionCancelled::LayerGone); } } } @@ -1540,6 +1794,7 @@ pub(crate) struct LayerImplMetrics { rare_counters: enum_map::EnumMap, inits_cancelled: metrics::core::GenericCounter, redownload_after: metrics::Histogram, + time_to_evict: metrics::Histogram, } impl Default for LayerImplMetrics { @@ -1635,6 +1890,13 @@ impl Default for LayerImplMetrics { .unwrap() }; + let time_to_evict = metrics::register_histogram!( + "pageserver_layer_eviction_held_permit_seconds", + "Time eviction held the permit.", + vec![0.001, 0.010, 0.100, 0.500, 1.000, 5.000] + ) + .unwrap(); + Self { started_evictions, completed_evictions, @@ -1647,6 +1909,7 @@ impl Default for LayerImplMetrics { rare_counters, inits_cancelled, redownload_after, + time_to_evict, } } } @@ -1708,10 +1971,6 @@ impl LayerImplMetrics { self.rare_counters[RareEvent::PermanentLoadingFailure].inc(); } - fn inc_broadcast_lagged(&self) { - self.rare_counters[RareEvent::EvictAndWaitLagged].inc(); - } - fn inc_init_cancelled(&self) { self.inits_cancelled.inc() } @@ -1719,9 +1978,22 @@ impl LayerImplMetrics { fn record_redownloaded_after(&self, duration: std::time::Duration) { self.redownload_after.observe(duration.as_secs_f64()) } + + /// This would be bad if it ever happened, or mean extreme disk pressure. We should probably + /// instead cancel eviction if we would have read waiters. We cannot however separate reads + /// from other evictions, so this could have noise as well. + fn inc_evicted_with_waiters(&self) { + self.rare_counters[RareEvent::EvictedWithWaiters].inc(); + } + + /// Recorded at least initially as the permit is now acquired in async context before + /// spawn_blocking action. + fn record_time_to_evict(&self, duration: std::time::Duration) { + self.time_to_evict.observe(duration.as_secs_f64()) + } } -#[derive(enum_map::Enum)] +#[derive(Debug, Clone, Copy, enum_map::Enum)] enum EvictionCancelled { LayerGone, TimelineGone, @@ -1733,6 +2005,7 @@ enum EvictionCancelled { LostToDownload, /// After eviction, there was a new layer access which cancelled the eviction. UpgradedBackOnAccess, + UnexpectedEvictedState, } impl EvictionCancelled { @@ -1746,6 +2019,7 @@ impl EvictionCancelled { EvictionCancelled::AlreadyReinitialized => "already_reinitialized", EvictionCancelled::LostToDownload => "lost_to_download", EvictionCancelled::UpgradedBackOnAccess => "upgraded_back_on_access", + EvictionCancelled::UnexpectedEvictedState => "unexpected_evicted_state", } } } @@ -1773,7 +2047,7 @@ enum RareEvent { UpgradedWantedEvicted, InitWithoutDownload, PermanentLoadingFailure, - EvictAndWaitLagged, + EvictedWithWaiters, } impl RareEvent { @@ -1787,7 +2061,7 @@ impl RareEvent { UpgradedWantedEvicted => "raced_wanted_evicted", InitWithoutDownload => "init_needed_no_download", PermanentLoadingFailure => "permanent_loading_failure", - EvictAndWaitLagged => "broadcast_lagged", + EvictedWithWaiters => "evicted_with_waiters", } } } diff --git a/pageserver/src/tenant/storage_layer/layer/failpoints.rs b/pageserver/src/tenant/storage_layer/layer/failpoints.rs new file mode 100644 index 000000000000..6cedc41d98eb --- /dev/null +++ b/pageserver/src/tenant/storage_layer/layer/failpoints.rs @@ -0,0 +1,119 @@ +//! failpoints for unit tests, implying `#[cfg(test)]`. +//! +//! These are not accessible over http. + +use super::*; + +impl Layer { + /// Enable a failpoint from a unit test. + pub(super) fn enable_failpoint(&self, failpoint: Failpoint) { + self.0.failpoints.lock().unwrap().push(failpoint); + } +} + +impl LayerInner { + /// Query if this failpoint is enabled, as in, arrive at a failpoint. + /// + /// Calls to this method need to be `#[cfg(test)]` guarded. + pub(super) async fn failpoint(&self, kind: FailpointKind) -> Result<(), FailpointHit> { + let fut = { + let mut fps = self.failpoints.lock().unwrap(); + // find the *last* failpoint for cases in which we need to use multiple for the same + // thing (two blocked evictions) + let fp = fps.iter_mut().rfind(|x| x.kind() == kind); + + let Some(fp) = fp else { + return Ok(()); + }; + + fp.hit() + }; + + fut.await + } +} + +#[derive(Debug, PartialEq, Eq)] +pub(crate) enum FailpointKind { + /// Failpoint acts as an accurate cancelled by drop here; see the only site of use. + AfterDeterminingLayerNeedsNoDownload, + /// Failpoint for stalling eviction starting + WaitBeforeStartingEvicting, + /// Failpoint hit in the spawned task + WaitBeforeDownloading, +} + +pub(crate) enum Failpoint { + AfterDeterminingLayerNeedsNoDownload, + WaitBeforeStartingEvicting( + Option, + utils::completion::Barrier, + ), + WaitBeforeDownloading( + Option, + utils::completion::Barrier, + ), +} + +impl Failpoint { + fn kind(&self) -> FailpointKind { + match self { + Failpoint::AfterDeterminingLayerNeedsNoDownload => { + FailpointKind::AfterDeterminingLayerNeedsNoDownload + } + Failpoint::WaitBeforeStartingEvicting(..) => FailpointKind::WaitBeforeStartingEvicting, + Failpoint::WaitBeforeDownloading(..) => FailpointKind::WaitBeforeDownloading, + } + } + + fn hit(&mut self) -> impl std::future::Future> + 'static { + use futures::future::FutureExt; + + // use boxed futures to avoid Either hurdles + match self { + Failpoint::AfterDeterminingLayerNeedsNoDownload => { + let kind = self.kind(); + + async move { Err(FailpointHit(kind)) }.boxed() + } + Failpoint::WaitBeforeStartingEvicting(arrival, b) + | Failpoint::WaitBeforeDownloading(arrival, b) => { + // first one signals arrival + drop(arrival.take()); + + let b = b.clone(); + + async move { + tracing::trace!("waiting on a failpoint barrier"); + b.wait().await; + tracing::trace!("done waiting on a failpoint barrier"); + Ok(()) + } + .boxed() + } + } + } +} + +impl std::fmt::Display for FailpointKind { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + std::fmt::Debug::fmt(self, f) + } +} + +#[derive(Debug)] +pub(crate) struct FailpointHit(FailpointKind); + +impl std::fmt::Display for FailpointHit { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + std::fmt::Debug::fmt(self, f) + } +} + +impl std::error::Error for FailpointHit {} + +impl From for DownloadError { + fn from(value: FailpointHit) -> Self { + DownloadError::Failpoint(value.0) + } +} diff --git a/pageserver/src/tenant/storage_layer/layer/tests.rs b/pageserver/src/tenant/storage_layer/layer/tests.rs index e7b2eb025a8b..247ff123b5ce 100644 --- a/pageserver/src/tenant/storage_layer/layer/tests.rs +++ b/pageserver/src/tenant/storage_layer/layer/tests.rs @@ -1,14 +1,13 @@ -use futures::StreamExt; use pageserver_api::key::CONTROLFILE_KEY; use tokio::task::JoinSet; -use tracing::Instrument; use utils::{ completion::{self, Completion}, id::TimelineId, }; +use super::failpoints::{Failpoint, FailpointKind}; use super::*; -use crate::{context::DownloadBehavior, task_mgr::BACKGROUND_RUNTIME}; +use crate::context::DownloadBehavior; use crate::{task_mgr::TaskKind, tenant::harness::TenantHarness}; /// Used in tests to advance a future to wanted await point, and not futher. @@ -21,7 +20,7 @@ const FOREVER: std::time::Duration = std::time::Duration::from_secs(ADVANCE.as_s /// Demonstrate the API and resident -> evicted -> resident -> deleted transitions. #[tokio::test] async fn smoke_test() { - let handle = BACKGROUND_RUNTIME.handle(); + let handle = tokio::runtime::Handle::current(); let h = TenantHarness::create("smoke_test").unwrap(); let span = h.span(); @@ -38,7 +37,7 @@ async fn smoke_test() { let layer = { let mut layers = { let layers = timeline.layers.read().await; - layers.resident_layers().collect::>().await + layers.likely_resident_layers().collect::>() }; assert_eq!(layers.len(), 1); @@ -88,7 +87,7 @@ async fn smoke_test() { // // ZERO for timeout does not work reliably, so first take up all spawn_blocking slots to // artificially slow it down. - let helper = SpawnBlockingPoolHelper::consume_all_spawn_blocking_threads(handle).await; + let helper = SpawnBlockingPoolHelper::consume_all_spawn_blocking_threads(&handle).await; match layer .evict_and_wait(std::time::Duration::ZERO) @@ -99,7 +98,7 @@ async fn smoke_test() { // expected, but note that the eviction is "still ongoing" helper.release().await; // exhaust spawn_blocking pool to ensure it is now complete - SpawnBlockingPoolHelper::consume_and_release_all_of_spawn_blocking_threads(handle) + SpawnBlockingPoolHelper::consume_and_release_all_of_spawn_blocking_threads(&handle) .await; } other => unreachable!("{other:?}"), @@ -108,7 +107,7 @@ async fn smoke_test() { // only way to query if a layer is resident is to acquire a ResidentLayer instance. // Layer::keep_resident never downloads, but it might initialize if the layer file is found // downloaded locally. - let none = layer.keep_resident().await.unwrap(); + let none = layer.keep_resident().await; assert!( none.is_none(), "Expected none, because eviction removed the local file, found: {none:?}" @@ -167,6 +166,7 @@ async fn smoke_test() { rtc.wait_completion().await.unwrap(); assert_eq!(rtc.get_remote_physical_size(), 0); + assert_eq!(0, LAYER_IMPL_METRICS.inits_cancelled.get()) } /// This test demonstrates a previous hang when a eviction and deletion were requested at the same @@ -174,7 +174,7 @@ async fn smoke_test() { #[tokio::test(start_paused = true)] async fn evict_and_wait_on_wanted_deleted() { // this is the runtime on which Layer spawns the blocking tasks on - let handle = BACKGROUND_RUNTIME.handle(); + let handle = tokio::runtime::Handle::current(); let h = TenantHarness::create("evict_and_wait_on_wanted_deleted").unwrap(); utils::logging::replace_panic_hook_with_tracing_panic_hook().forget(); @@ -188,7 +188,7 @@ async fn evict_and_wait_on_wanted_deleted() { let layer = { let mut layers = { let layers = timeline.layers.read().await; - layers.resident_layers().collect::>().await + layers.likely_resident_layers().collect::>() }; assert_eq!(layers.len(), 1); @@ -213,11 +213,11 @@ async fn evict_and_wait_on_wanted_deleted() { drop(resident); // make sure the eviction task gets to run - SpawnBlockingPoolHelper::consume_and_release_all_of_spawn_blocking_threads(handle).await; + SpawnBlockingPoolHelper::consume_and_release_all_of_spawn_blocking_threads(&handle).await; let resident = layer.keep_resident().await; assert!( - matches!(resident, Ok(None)), + resident.is_none(), "keep_resident should not have re-initialized: {resident:?}" ); @@ -235,24 +235,408 @@ async fn evict_and_wait_on_wanted_deleted() { layers.finish_gc_timeline(&[layer]); } - SpawnBlockingPoolHelper::consume_and_release_all_of_spawn_blocking_threads(handle).await; + SpawnBlockingPoolHelper::consume_and_release_all_of_spawn_blocking_threads(&handle).await; assert_eq!(1, LAYER_IMPL_METRICS.started_deletes.get()); assert_eq!(1, LAYER_IMPL_METRICS.completed_deletes.get()); assert_eq!(1, LAYER_IMPL_METRICS.started_evictions.get()); assert_eq!(1, LAYER_IMPL_METRICS.completed_evictions.get()); + assert_eq!(0, LAYER_IMPL_METRICS.inits_cancelled.get()) } -/// This test shows that ensures we are able to read the layer while the layer eviction has been -/// started but not completed due to spawn_blocking pool being blocked. +/// This test ensures we are able to read the layer while the layer eviction has been +/// started but not completed. +#[test] +fn read_wins_pending_eviction() { + let rt = tokio::runtime::Builder::new_current_thread() + .max_blocking_threads(1) + .enable_all() + .start_paused(true) + .build() + .unwrap(); + + rt.block_on(async move { + // this is the runtime on which Layer spawns the blocking tasks on + let handle = tokio::runtime::Handle::current(); + let h = TenantHarness::create("read_wins_pending_eviction").unwrap(); + let (tenant, ctx) = h.load().await; + let span = h.span(); + let download_span = span.in_scope(|| tracing::info_span!("downloading", timeline_id = 1)); + + let timeline = tenant + .create_test_timeline(TimelineId::generate(), Lsn(0x10), 14, &ctx) + .await + .unwrap(); + + let layer = { + let mut layers = { + let layers = timeline.layers.read().await; + layers.likely_resident_layers().collect::>() + }; + + assert_eq!(layers.len(), 1); + + layers.swap_remove(0) + }; + + // setup done + + let resident = layer.keep_resident().await.unwrap(); + + let mut evict_and_wait = std::pin::pin!(layer.evict_and_wait(FOREVER)); + + // drive the future to await on the status channel + tokio::time::timeout(ADVANCE, &mut evict_and_wait) + .await + .expect_err("should had been a timeout since we are holding the layer resident"); + assert_eq!(1, LAYER_IMPL_METRICS.started_evictions.get()); + + let (completion, barrier) = utils::completion::channel(); + let (arrival, arrived_at_barrier) = utils::completion::channel(); + layer.enable_failpoint(Failpoint::WaitBeforeStartingEvicting( + Some(arrival), + barrier, + )); + + // now the eviction cannot proceed because the threads are consumed while completion exists + drop(resident); + arrived_at_barrier.wait().await; + assert!(!layer.is_likely_resident()); + + // because no actual eviction happened, we get to just reinitialize the DownloadedLayer + layer + .0 + .get_or_maybe_download(false, None) + .instrument(download_span) + .await + .expect("should had reinitialized without downloading"); + + assert!(layer.is_likely_resident()); + + // reinitialization notifies of new resident status, which should error out all evict_and_wait + let e = tokio::time::timeout(ADVANCE, &mut evict_and_wait) + .await + .expect("no timeout, because get_or_maybe_download re-initialized") + .expect_err("eviction should not have succeeded because re-initialized"); + + // works as intended: evictions lose to "downloads" + assert!(matches!(e, EvictionError::Downloaded), "{e:?}"); + assert_eq!(0, LAYER_IMPL_METRICS.completed_evictions.get()); + + // this is not wrong: the eviction is technically still "on the way" as it's still queued + // because of a failpoint + assert_eq!( + 0, + LAYER_IMPL_METRICS + .cancelled_evictions + .values() + .map(|ctr| ctr.get()) + .sum::() + ); + + drop(completion); + + tokio::time::sleep(ADVANCE).await; + SpawnBlockingPoolHelper::consume_and_release_all_of_spawn_blocking_threads0(&handle, 1) + .await; + + assert_eq!(0, LAYER_IMPL_METRICS.completed_evictions.get()); + + // now we finally can observe the original eviction failing + // it would had been possible to observe it earlier, but here it is guaranteed to have + // happened. + assert_eq!( + 1, + LAYER_IMPL_METRICS + .cancelled_evictions + .values() + .map(|ctr| ctr.get()) + .sum::() + ); + + assert_eq!( + 1, + LAYER_IMPL_METRICS.cancelled_evictions[EvictionCancelled::AlreadyReinitialized].get() + ); + + assert_eq!(0, LAYER_IMPL_METRICS.inits_cancelled.get()) + }); +} + +/// Use failpoint to delay an eviction starting to get a VersionCheckFailed. +#[test] +fn multiple_pending_evictions_in_order() { + let name = "multiple_pending_evictions_in_order"; + let in_order = true; + multiple_pending_evictions_scenario(name, in_order); +} + +/// Use failpoint to reorder later eviction before first to get a UnexpectedEvictedState. +#[test] +fn multiple_pending_evictions_out_of_order() { + let name = "multiple_pending_evictions_out_of_order"; + let in_order = false; + multiple_pending_evictions_scenario(name, in_order); +} + +fn multiple_pending_evictions_scenario(name: &'static str, in_order: bool) { + let rt = tokio::runtime::Builder::new_current_thread() + .max_blocking_threads(1) + .enable_all() + .start_paused(true) + .build() + .unwrap(); + + rt.block_on(async move { + // this is the runtime on which Layer spawns the blocking tasks on + let handle = tokio::runtime::Handle::current(); + let h = TenantHarness::create(name).unwrap(); + let (tenant, ctx) = h.load().await; + let span = h.span(); + let download_span = span.in_scope(|| tracing::info_span!("downloading", timeline_id = 1)); + + let timeline = tenant + .create_test_timeline(TimelineId::generate(), Lsn(0x10), 14, &ctx) + .await + .unwrap(); + + let layer = { + let mut layers = { + let layers = timeline.layers.read().await; + layers.likely_resident_layers().collect::>() + }; + + assert_eq!(layers.len(), 1); + + layers.swap_remove(0) + }; + + // setup done + + let resident = layer.keep_resident().await.unwrap(); + + let mut evict_and_wait = std::pin::pin!(layer.evict_and_wait(FOREVER)); + + // drive the future to await on the status channel + tokio::time::timeout(ADVANCE, &mut evict_and_wait) + .await + .expect_err("should had been a timeout since we are holding the layer resident"); + assert_eq!(1, LAYER_IMPL_METRICS.started_evictions.get()); + + let (completion1, barrier) = utils::completion::channel(); + let mut completion1 = Some(completion1); + let (arrival, arrived_at_barrier) = utils::completion::channel(); + layer.enable_failpoint(Failpoint::WaitBeforeStartingEvicting( + Some(arrival), + barrier, + )); + + // now the eviction cannot proceed because we are simulating arbitrary long delay for the + // eviction task start. + drop(resident); + assert!(!layer.is_likely_resident()); + + arrived_at_barrier.wait().await; + + // because no actual eviction happened, we get to just reinitialize the DownloadedLayer + layer + .0 + .get_or_maybe_download(false, None) + .instrument(download_span) + .await + .expect("should had reinitialized without downloading"); + + assert!(layer.is_likely_resident()); + + // reinitialization notifies of new resident status, which should error out all evict_and_wait + let e = tokio::time::timeout(ADVANCE, &mut evict_and_wait) + .await + .expect("no timeout, because get_or_maybe_download re-initialized") + .expect_err("eviction should not have succeeded because re-initialized"); + + // works as intended: evictions lose to "downloads" + assert!(matches!(e, EvictionError::Downloaded), "{e:?}"); + assert_eq!(0, LAYER_IMPL_METRICS.completed_evictions.get()); + + // this is not wrong: the eviction is technically still "on the way" as it's still queued + // because of a failpoint + assert_eq!( + 0, + LAYER_IMPL_METRICS + .cancelled_evictions + .values() + .map(|ctr| ctr.get()) + .sum::() + ); + + assert_eq!(0, LAYER_IMPL_METRICS.completed_evictions.get()); + + // configure another failpoint for the second eviction -- evictions are per initialization, + // so now that we've reinitialized the inner, we get to run two of them at the same time. + let (completion2, barrier) = utils::completion::channel(); + let (arrival, arrived_at_barrier) = utils::completion::channel(); + layer.enable_failpoint(Failpoint::WaitBeforeStartingEvicting( + Some(arrival), + barrier, + )); + + let mut second_eviction = std::pin::pin!(layer.evict_and_wait(FOREVER)); + + // advance to the wait on the queue + tokio::time::timeout(ADVANCE, &mut second_eviction) + .await + .expect_err("timeout because failpoint is blocking"); + + arrived_at_barrier.wait().await; + + assert_eq!(2, LAYER_IMPL_METRICS.started_evictions.get()); + + let mut release_earlier_eviction = |expected_reason| { + assert_eq!( + 0, + LAYER_IMPL_METRICS.cancelled_evictions[expected_reason].get(), + ); + + drop(completion1.take().unwrap()); + + let handle = &handle; + + async move { + tokio::time::sleep(ADVANCE).await; + SpawnBlockingPoolHelper::consume_and_release_all_of_spawn_blocking_threads0( + handle, 1, + ) + .await; + + assert_eq!( + 1, + LAYER_IMPL_METRICS.cancelled_evictions[expected_reason].get(), + ); + } + }; + + if in_order { + release_earlier_eviction(EvictionCancelled::VersionCheckFailed).await; + } + + // release the later eviction which is for the current version + drop(completion2); + tokio::time::sleep(ADVANCE).await; + SpawnBlockingPoolHelper::consume_and_release_all_of_spawn_blocking_threads0(&handle, 1) + .await; + + if !in_order { + release_earlier_eviction(EvictionCancelled::UnexpectedEvictedState).await; + } + + tokio::time::timeout(ADVANCE, &mut second_eviction) + .await + .expect("eviction goes through now that spawn_blocking is unclogged") + .expect("eviction should succeed, because version matches"); + + assert_eq!(1, LAYER_IMPL_METRICS.completed_evictions.get()); + + // ensure the cancelled are unchanged + assert_eq!( + 1, + LAYER_IMPL_METRICS + .cancelled_evictions + .values() + .map(|ctr| ctr.get()) + .sum::() + ); + + assert_eq!(0, LAYER_IMPL_METRICS.inits_cancelled.get()) + }); +} + +/// The test ensures with a failpoint that a pending eviction is not cancelled by what is currently +/// a `Layer::keep_resident` call. /// -/// Here `Layer::keep_resident` is used to "simulate" reads, because it cannot download. +/// This matters because cancelling the eviction would leave us in a state where the file is on +/// disk but the layer internal state says it has not been initialized. Futhermore, it allows us to +/// have non-repairing `Layer::is_likely_resident`. #[tokio::test(start_paused = true)] -async fn residency_check_while_evict_and_wait_on_clogged_spawn_blocking() { - // this is the runtime on which Layer spawns the blocking tasks on - let handle = BACKGROUND_RUNTIME.handle(); - let h = TenantHarness::create("residency_check_while_evict_and_wait_on_clogged_spawn_blocking") +async fn cancelled_get_or_maybe_download_does_not_cancel_eviction() { + let handle = tokio::runtime::Handle::current(); + let h = + TenantHarness::create("cancelled_get_or_maybe_download_does_not_cancel_eviction").unwrap(); + let (tenant, ctx) = h.load().await; + + let timeline = tenant + .create_test_timeline(TimelineId::generate(), Lsn(0x10), 14, &ctx) + .await .unwrap(); + + let layer = { + let mut layers = { + let layers = timeline.layers.read().await; + layers.likely_resident_layers().collect::>() + }; + + assert_eq!(layers.len(), 1); + + layers.swap_remove(0) + }; + + // this failpoint will simulate the `get_or_maybe_download` becoming cancelled (by returning an + // Err) at the right time as in "during" the `LayerInner::needs_download`. + layer.enable_failpoint(Failpoint::AfterDeterminingLayerNeedsNoDownload); + + let (completion, barrier) = utils::completion::channel(); + let (arrival, arrived_at_barrier) = utils::completion::channel(); + + layer.enable_failpoint(Failpoint::WaitBeforeStartingEvicting( + Some(arrival), + barrier, + )); + + tokio::time::timeout(ADVANCE, layer.evict_and_wait(FOREVER)) + .await + .expect_err("should had advanced to waiting on channel"); + + arrived_at_barrier.wait().await; + + // simulate a cancelled read which is cancelled before it gets to re-initialize + let e = layer + .0 + .get_or_maybe_download(false, None) + .await + .unwrap_err(); + assert!( + matches!( + e, + DownloadError::Failpoint(FailpointKind::AfterDeterminingLayerNeedsNoDownload) + ), + "{e:?}" + ); + + assert!( + layer.0.needs_download().await.unwrap().is_none(), + "file is still on disk" + ); + + // release the eviction task + drop(completion); + tokio::time::sleep(ADVANCE).await; + SpawnBlockingPoolHelper::consume_and_release_all_of_spawn_blocking_threads(&handle).await; + + // failpoint is still enabled, but it is not hit + let e = layer + .0 + .get_or_maybe_download(false, None) + .await + .unwrap_err(); + assert!(matches!(e, DownloadError::DownloadRequired), "{e:?}"); + + // failpoint is not counted as cancellation either + assert_eq!(0, LAYER_IMPL_METRICS.inits_cancelled.get()) +} + +#[tokio::test(start_paused = true)] +async fn evict_and_wait_does_not_wait_for_download() { + // let handle = tokio::runtime::Handle::current(); + let h = TenantHarness::create("evict_and_wait_does_not_wait_for_download").unwrap(); let (tenant, ctx) = h.load().await; let span = h.span(); let download_span = span.in_scope(|| tracing::info_span!("downloading", timeline_id = 1)); @@ -265,7 +649,7 @@ async fn residency_check_while_evict_and_wait_on_clogged_spawn_blocking() { let layer = { let mut layers = { let layers = timeline.layers.read().await; - layers.resident_layers().collect::>().await + layers.likely_resident_layers().collect::>() }; assert_eq!(layers.len(), 1); @@ -273,91 +657,76 @@ async fn residency_check_while_evict_and_wait_on_clogged_spawn_blocking() { layers.swap_remove(0) }; - // setup done - - let resident = layer.keep_resident().await.unwrap(); + // kind of forced setup: start an eviction but do not allow it progress until we are + // downloading + let (eviction_can_continue, barrier) = utils::completion::channel(); + let (arrival, eviction_arrived) = utils::completion::channel(); + layer.enable_failpoint(Failpoint::WaitBeforeStartingEvicting( + Some(arrival), + barrier, + )); let mut evict_and_wait = std::pin::pin!(layer.evict_and_wait(FOREVER)); - // drive the future to await on the status channel + // use this once-awaited other_evict to synchronize with the eviction + let other_evict = layer.evict_and_wait(FOREVER); + tokio::time::timeout(ADVANCE, &mut evict_and_wait) .await - .expect_err("should had been a timeout since we are holding the layer resident"); - assert_eq!(1, LAYER_IMPL_METRICS.started_evictions.get()); + .expect_err("should had advanced"); + eviction_arrived.wait().await; + drop(eviction_can_continue); + other_evict.await.unwrap(); - // clog up BACKGROUND_RUNTIME spawn_blocking - let helper = SpawnBlockingPoolHelper::consume_all_spawn_blocking_threads(handle).await; + // now the layer is evicted, and the "evict_and_wait" is waiting on the receiver + assert!(!layer.is_likely_resident()); - // now the eviction cannot proceed because the threads are consumed while completion exists - drop(resident); + // following new evict_and_wait will fail until we've completed the download + let e = layer.evict_and_wait(FOREVER).await.unwrap_err(); + assert!(matches!(e, EvictionError::NotFound), "{e:?}"); - // because no actual eviction happened, we get to just reinitialize the DownloadedLayer - layer - .keep_resident() - .instrument(download_span) - .await - .expect("keep_resident should had reinitialized without downloading") - .expect("ResidentLayer"); + let (download_can_continue, barrier) = utils::completion::channel(); + let (arrival, _download_arrived) = utils::completion::channel(); + layer.enable_failpoint(Failpoint::WaitBeforeDownloading(Some(arrival), barrier)); - // because the keep_resident check alters wanted evicted without sending a message, we will - // never get completed - let e = tokio::time::timeout(ADVANCE, &mut evict_and_wait) - .await - .expect("no timeout, because keep_resident re-initialized") - .expect_err("eviction should not have succeeded because re-initialized"); - - // works as intended: evictions lose to "downloads" - assert!(matches!(e, EvictionError::Downloaded), "{e:?}"); - assert_eq!(0, LAYER_IMPL_METRICS.completed_evictions.get()); - - // this is not wrong: the eviction is technically still "on the way" as it's still queued - // because spawn_blocking is clogged up - assert_eq!( - 0, - LAYER_IMPL_METRICS - .cancelled_evictions - .values() - .map(|ctr| ctr.get()) - .sum::() - ); + let mut download = std::pin::pin!(layer + .0 + .get_or_maybe_download(true, None) + .instrument(download_span)); - let mut second_eviction = std::pin::pin!(layer.evict_and_wait(FOREVER)); + assert!( + !layer.is_likely_resident(), + "during download layer is evicted" + ); - // advance to the wait on the queue - tokio::time::timeout(ADVANCE, &mut second_eviction) + tokio::time::timeout(ADVANCE, &mut download) .await - .expect_err("timeout because spawn_blocking is clogged"); + .expect_err("should had timed out because of failpoint"); - // in this case we don't leak started evictions, but I think there is still a chance of that - // happening, because we could have upgrades race multiple evictions while only one of them - // happens? - assert_eq!(2, LAYER_IMPL_METRICS.started_evictions.get()); + // now we finally get to continue, and because the latest state is downloading, we deduce that + // original eviction succeeded + evict_and_wait.await.unwrap(); - helper.release().await; + // however a new evict_and_wait will fail + let e = layer.evict_and_wait(FOREVER).await.unwrap_err(); + assert!(matches!(e, EvictionError::NotFound), "{e:?}"); - // the second_eviction gets to run here - // - // synchronize to be *strictly* after the second_eviction spawn_blocking run - SpawnBlockingPoolHelper::consume_and_release_all_of_spawn_blocking_threads(handle).await; + assert!(!layer.is_likely_resident()); - tokio::time::timeout(ADVANCE, &mut second_eviction) - .await - .expect("eviction goes through now that spawn_blocking is unclogged") - .expect("eviction should succeed, because version matches"); + drop(download_can_continue); + download.await.expect("download should had succeeded"); + assert!(layer.is_likely_resident()); - assert_eq!(1, LAYER_IMPL_METRICS.completed_evictions.get()); + // only now can we evict + layer.evict_and_wait(FOREVER).await.unwrap(); +} - // now we finally can observe the original spawn_blocking failing - // it would had been possible to observe it earlier, but here it is guaranteed to have - // happened. - assert_eq!( - 1, - LAYER_IMPL_METRICS - .cancelled_evictions - .values() - .map(|ctr| ctr.get()) - .sum::() - ); +#[test] +fn layer_size() { + assert_eq!(std::mem::size_of::(), 2040); + assert_eq!(std::mem::size_of::(), 104); + assert_eq!(std::mem::size_of::(), 2328); + // it also has the utf8 path } struct SpawnBlockingPoolHelper { @@ -374,31 +743,41 @@ impl SpawnBlockingPoolHelper { /// /// This should be no issue nowdays, because nextest runs each test in it's own process. async fn consume_all_spawn_blocking_threads(handle: &tokio::runtime::Handle) -> Self { - let (completion, barrier) = completion::channel(); - let (tx, mut rx) = tokio::sync::mpsc::channel(8); + let default_max_blocking_threads = 512; - let assumed_max_blocking_threads = 512; + Self::consume_all_spawn_blocking_threads0(handle, default_max_blocking_threads).await + } + + async fn consume_all_spawn_blocking_threads0( + handle: &tokio::runtime::Handle, + threads: usize, + ) -> Self { + assert_ne!(threads, 0); + + let (completion, barrier) = completion::channel(); + let (started, starts_completed) = completion::channel(); let mut blocking_tasks = JoinSet::new(); - for _ in 0..assumed_max_blocking_threads { + for _ in 0..threads { let barrier = barrier.clone(); - let tx = tx.clone(); + let started = started.clone(); blocking_tasks.spawn_blocking_on( move || { - tx.blocking_send(()).unwrap(); - drop(tx); + drop(started); tokio::runtime::Handle::current().block_on(barrier.wait()); }, handle, ); } + drop(started); + + starts_completed.wait().await; + drop(barrier); - for _ in 0..assumed_max_blocking_threads { - rx.recv().await.unwrap(); - } + tracing::trace!("consumed all threads"); SpawnBlockingPoolHelper { awaited_by_spawn_blocking_tasks: completion, @@ -418,13 +797,22 @@ impl SpawnBlockingPoolHelper { while let Some(res) = blocking_tasks.join_next().await { res.expect("none of the tasks should had panicked"); } + + tracing::trace!("released all threads"); } /// In the tests it is used as an easy way of making sure something scheduled on the target /// runtimes `spawn_blocking` has completed, because it must've been scheduled and completed /// before our tasks have a chance to schedule and complete. async fn consume_and_release_all_of_spawn_blocking_threads(handle: &tokio::runtime::Handle) { - Self::consume_all_spawn_blocking_threads(handle) + Self::consume_and_release_all_of_spawn_blocking_threads0(handle, 512).await + } + + async fn consume_and_release_all_of_spawn_blocking_threads0( + handle: &tokio::runtime::Handle, + threads: usize, + ) { + Self::consume_all_spawn_blocking_threads0(handle, threads) .await .release() .await @@ -438,7 +826,7 @@ fn spawn_blocking_pool_helper_actually_works() { // because the amount is not configurable for our helper, expect the same amount as // BACKGROUND_RUNTIME using the tokio defaults would have. let rt = tokio::runtime::Builder::new_current_thread() - .max_blocking_threads(512) + .max_blocking_threads(1) .enable_all() .build() .unwrap(); @@ -448,7 +836,8 @@ fn spawn_blocking_pool_helper_actually_works() { rt.block_on(async move { // this will not return until all threads are spun up and actually executing the code // waiting on `consumed` to be `SpawnBlockingPoolHelper::release`'d. - let consumed = SpawnBlockingPoolHelper::consume_all_spawn_blocking_threads(handle).await; + let consumed = + SpawnBlockingPoolHelper::consume_all_spawn_blocking_threads0(handle, 1).await; println!("consumed"); diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 2ab7301ccee3..0b8222bca7dd 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -13,7 +13,6 @@ use bytes::Bytes; use camino::Utf8Path; use enumset::EnumSet; use fail::fail_point; -use futures::stream::StreamExt; use once_cell::sync::Lazy; use pageserver_api::{ key::AUX_FILES_KEY, @@ -2442,7 +2441,7 @@ impl Timeline { let guard = self.layers.read().await; - let resident = guard.resident_layers().map(|layer| { + let resident = guard.likely_resident_layers().map(|layer| { let last_activity_ts = layer.access_stats().latest_activity_or_now(); HeatMapLayer::new( @@ -2452,7 +2451,7 @@ impl Timeline { ) }); - let layers = resident.collect().await; + let layers = resident.collect(); Some(HeatMapTimeline::new(self.timeline_id, layers)) } @@ -4302,7 +4301,7 @@ impl Timeline { let mut max_layer_size: Option = None; let resident_layers = guard - .resident_layers() + .likely_resident_layers() .map(|layer| { let file_size = layer.layer_desc().file_size; max_layer_size = max_layer_size.map_or(Some(file_size), |m| Some(m.max(file_size))); @@ -4315,8 +4314,7 @@ impl Timeline { relative_last_activity: finite_f32::FiniteF32::ZERO, } }) - .collect() - .await; + .collect(); DiskUsageEvictionInfo { max_layer_size, @@ -4713,7 +4711,6 @@ mod tests { .keep_resident() .await .expect("no download => no downloading errors") - .expect("should had been resident") .drop_eviction_guard(); let forever = std::time::Duration::from_secs(120); @@ -4724,7 +4721,7 @@ mod tests { let (first, second) = tokio::join!(first, second); let res = layer.keep_resident().await; - assert!(matches!(res, Ok(None)), "{res:?}"); + assert!(res.is_none(), "{res:?}"); match (first, second) { (Ok(()), Ok(())) => { diff --git a/pageserver/src/tenant/timeline/eviction_task.rs b/pageserver/src/tenant/timeline/eviction_task.rs index dd603135d25d..dd769d41216c 100644 --- a/pageserver/src/tenant/timeline/eviction_task.rs +++ b/pageserver/src/tenant/timeline/eviction_task.rs @@ -225,24 +225,18 @@ impl Timeline { { let guard = self.layers.read().await; let layers = guard.layer_map(); - for hist_layer in layers.iter_historic_layers() { - let hist_layer = guard.get_from_desc(&hist_layer); + for layer in layers.iter_historic_layers() { + let layer = guard.get_from_desc(&layer); // guard against eviction while we inspect it; it might be that eviction_task and // disk_usage_eviction_task both select the same layers to be evicted, and // seemingly free up double the space. both succeeding is of no consequence. - let guard = match hist_layer.keep_resident().await { - Ok(Some(l)) => l, - Ok(None) => continue, - Err(e) => { - // these should not happen, but we cannot make them statically impossible right - // now. - tracing::warn!(layer=%hist_layer, "failed to keep the layer resident: {e:#}"); - continue; - } - }; - let last_activity_ts = hist_layer.access_stats().latest_activity_or_now(); + if !layer.is_likely_resident() { + continue; + } + + let last_activity_ts = layer.access_stats().latest_activity_or_now(); let no_activity_for = match now.duration_since(last_activity_ts) { Ok(d) => d, @@ -265,9 +259,8 @@ impl Timeline { continue; } }; - let layer = guard.drop_eviction_guard(); + if no_activity_for > p.threshold { - // this could cause a lot of allocations in some cases js.spawn(async move { layer .evict_and_wait(std::time::Duration::from_secs(5)) diff --git a/pageserver/src/tenant/timeline/layer_manager.rs b/pageserver/src/tenant/timeline/layer_manager.rs index ebcdcfdb4d4a..d54dc1642c36 100644 --- a/pageserver/src/tenant/timeline/layer_manager.rs +++ b/pageserver/src/tenant/timeline/layer_manager.rs @@ -1,5 +1,4 @@ use anyhow::{bail, ensure, Context, Result}; -use futures::StreamExt; use pageserver_api::shard::TenantShardId; use std::{collections::HashMap, sync::Arc}; use tracing::trace; @@ -241,29 +240,16 @@ impl LayerManager { layer.delete_on_drop(); } - pub(crate) fn resident_layers(&self) -> impl futures::stream::Stream + '_ { + pub(crate) fn likely_resident_layers(&self) -> impl Iterator + '_ { // for small layer maps, we most likely have all resident, but for larger more are likely // to be evicted assuming lots of layers correlated with longer lifespan. - let layers = self - .layer_map() - .iter_historic_layers() - .map(|desc| self.get_from_desc(&desc)); - - let layers = futures::stream::iter(layers); - - layers.filter_map(|layer| async move { - // TODO(#6028): this query does not really need to see the ResidentLayer - match layer.keep_resident().await { - Ok(Some(layer)) => Some(layer.drop_eviction_guard()), - Ok(None) => None, - Err(e) => { - // these should not happen, but we cannot make them statically impossible right - // now. - tracing::warn!(%layer, "failed to keep the layer resident: {e:#}"); - None - } - } + self.layer_map().iter_historic_layers().filter_map(|desc| { + self.layer_fmgr + .0 + .get(&desc.key()) + .filter(|l| l.is_likely_resident()) + .cloned() }) } From 94138c1a28e998b6e0d70f3b72dc170b2af34ca6 Mon Sep 17 00:00:00 2001 From: Jure Bajic Date: Thu, 21 Mar 2024 10:17:24 +0100 Subject: [PATCH 17/53] Enforce LSN ordering of batch entries (#7071) ## Summary of changes Enforce LSN ordering of batch entries. Closes https://github.com/neondatabase/neon/issues/6707 --- libs/utils/src/vec_map.rs | 220 +++++++++++++++++++++++----- pageserver/src/pgdatadir_mapping.rs | 14 +- pageserver/src/tenant/timeline.rs | 10 +- 3 files changed, 193 insertions(+), 51 deletions(-) diff --git a/libs/utils/src/vec_map.rs b/libs/utils/src/vec_map.rs index 9953b447c8ee..18b2af14f130 100644 --- a/libs/utils/src/vec_map.rs +++ b/libs/utils/src/vec_map.rs @@ -1,27 +1,60 @@ use std::{alloc::Layout, cmp::Ordering, ops::RangeBounds}; +#[derive(Clone, Copy, Debug, PartialEq, Eq)] +pub enum VecMapOrdering { + Greater, + GreaterOrEqual, +} + /// Ordered map datastructure implemented in a Vec. /// Append only - can only add keys that are larger than the /// current max key. +/// Ordering can be adjusted using [`VecMapOrdering`] +/// during `VecMap` construction. #[derive(Clone, Debug)] -pub struct VecMap(Vec<(K, V)>); +pub struct VecMap { + data: Vec<(K, V)>, + ordering: VecMapOrdering, +} impl Default for VecMap { fn default() -> Self { - VecMap(Default::default()) + VecMap { + data: Default::default(), + ordering: VecMapOrdering::Greater, + } } } -#[derive(Debug)] -pub struct InvalidKey; +#[derive(thiserror::Error, Debug)] +pub enum VecMapError { + #[error("Key violates ordering constraint")] + InvalidKey, + #[error("Mismatched ordering constraints")] + ExtendOrderingError, +} impl VecMap { + pub fn new(ordering: VecMapOrdering) -> Self { + Self { + data: Vec::new(), + ordering, + } + } + + pub fn with_capacity(capacity: usize, ordering: VecMapOrdering) -> Self { + Self { + data: Vec::with_capacity(capacity), + ordering, + } + } + pub fn is_empty(&self) -> bool { - self.0.is_empty() + self.data.is_empty() } pub fn as_slice(&self) -> &[(K, V)] { - self.0.as_slice() + self.data.as_slice() } /// This function may panic if given a range where the lower bound is @@ -29,7 +62,7 @@ impl VecMap { pub fn slice_range>(&self, range: R) -> &[(K, V)] { use std::ops::Bound::*; - let binary_search = |k: &K| self.0.binary_search_by_key(&k, extract_key); + let binary_search = |k: &K| self.data.binary_search_by_key(&k, extract_key); let start_idx = match range.start_bound() { Unbounded => 0, @@ -41,7 +74,7 @@ impl VecMap { }; let end_idx = match range.end_bound() { - Unbounded => self.0.len(), + Unbounded => self.data.len(), Included(k) => match binary_search(k) { Ok(idx) => idx + 1, Err(idx) => idx, @@ -49,34 +82,30 @@ impl VecMap { Excluded(k) => binary_search(k).unwrap_or_else(std::convert::identity), }; - &self.0[start_idx..end_idx] + &self.data[start_idx..end_idx] } /// Add a key value pair to the map. - /// If `key` is less than or equal to the current maximum key - /// the pair will not be added and InvalidKey error will be returned. - pub fn append(&mut self, key: K, value: V) -> Result { - if let Some((last_key, _last_value)) = self.0.last() { - if &key <= last_key { - return Err(InvalidKey); - } - } + /// If `key` is not respective of the `self` ordering the + /// pair will not be added and `InvalidKey` error will be returned. + pub fn append(&mut self, key: K, value: V) -> Result { + self.validate_key_order(&key)?; let delta_size = self.instrument_vec_op(|vec| vec.push((key, value))); Ok(delta_size) } /// Update the maximum key value pair or add a new key value pair to the map. - /// If `key` is less than the current maximum key no updates or additions - /// will occur and InvalidKey error will be returned. + /// If `key` is not respective of the `self` ordering no updates or additions + /// will occur and `InvalidKey` error will be returned. pub fn append_or_update_last( &mut self, key: K, mut value: V, - ) -> Result<(Option, usize), InvalidKey> { - if let Some((last_key, last_value)) = self.0.last_mut() { + ) -> Result<(Option, usize), VecMapError> { + if let Some((last_key, last_value)) = self.data.last_mut() { match key.cmp(last_key) { - Ordering::Less => return Err(InvalidKey), + Ordering::Less => return Err(VecMapError::InvalidKey), Ordering::Equal => { std::mem::swap(last_value, &mut value); const DELTA_SIZE: usize = 0; @@ -100,40 +129,67 @@ impl VecMap { V: Clone, { let split_idx = self - .0 + .data .binary_search_by_key(&cutoff, extract_key) .unwrap_or_else(std::convert::identity); ( - VecMap(self.0[..split_idx].to_vec()), - VecMap(self.0[split_idx..].to_vec()), + VecMap { + data: self.data[..split_idx].to_vec(), + ordering: self.ordering, + }, + VecMap { + data: self.data[split_idx..].to_vec(), + ordering: self.ordering, + }, ) } /// Move items from `other` to the end of `self`, leaving `other` empty. - /// If any keys in `other` is less than or equal to any key in `self`, - /// `InvalidKey` error will be returned and no mutation will occur. - pub fn extend(&mut self, other: &mut Self) -> Result { - let self_last_opt = self.0.last().map(extract_key); - let other_first_opt = other.0.last().map(extract_key); - - if let (Some(self_last), Some(other_first)) = (self_last_opt, other_first_opt) { - if self_last >= other_first { - return Err(InvalidKey); - } + /// If the `other` ordering is different from `self` ordering + /// `ExtendOrderingError` error will be returned. + /// If any keys in `other` is not respective of the ordering defined in + /// `self`, `InvalidKey` error will be returned and no mutation will occur. + pub fn extend(&mut self, other: &mut Self) -> Result { + if self.ordering != other.ordering { + return Err(VecMapError::ExtendOrderingError); + } + + let other_first_opt = other.data.last().map(extract_key); + if let Some(other_first) = other_first_opt { + self.validate_key_order(other_first)?; } - let delta_size = self.instrument_vec_op(|vec| vec.append(&mut other.0)); + let delta_size = self.instrument_vec_op(|vec| vec.append(&mut other.data)); Ok(delta_size) } + /// Validate the current last key in `self` and key being + /// inserted against the order defined in `self`. + fn validate_key_order(&self, key: &K) -> Result<(), VecMapError> { + if let Some(last_key) = self.data.last().map(extract_key) { + match (&self.ordering, &key.cmp(last_key)) { + (VecMapOrdering::Greater, Ordering::Less | Ordering::Equal) => { + return Err(VecMapError::InvalidKey); + } + (VecMapOrdering::Greater, Ordering::Greater) => {} + (VecMapOrdering::GreaterOrEqual, Ordering::Less) => { + return Err(VecMapError::InvalidKey); + } + (VecMapOrdering::GreaterOrEqual, Ordering::Equal | Ordering::Greater) => {} + } + } + + Ok(()) + } + /// Instrument an operation on the underlying [`Vec`]. /// Will panic if the operation decreases capacity. /// Returns the increase in memory usage caused by the op. fn instrument_vec_op(&mut self, op: impl FnOnce(&mut Vec<(K, V)>)) -> usize { - let old_cap = self.0.capacity(); - op(&mut self.0); - let new_cap = self.0.capacity(); + let old_cap = self.data.capacity(); + op(&mut self.data); + let new_cap = self.data.capacity(); match old_cap.cmp(&new_cap) { Ordering::Less => { @@ -145,6 +201,36 @@ impl VecMap { Ordering::Greater => panic!("VecMap capacity shouldn't ever decrease"), } } + + /// Similar to `from_iter` defined in `FromIter` trait except + /// that it accepts an [`VecMapOrdering`] + pub fn from_iter>(iter: I, ordering: VecMapOrdering) -> Self { + let iter = iter.into_iter(); + let initial_capacity = { + match iter.size_hint() { + (lower_bound, None) => lower_bound, + (_, Some(upper_bound)) => upper_bound, + } + }; + + let mut vec_map = VecMap::with_capacity(initial_capacity, ordering); + for (key, value) in iter { + vec_map + .append(key, value) + .expect("The passed collection needs to be sorted!"); + } + + vec_map + } +} + +impl IntoIterator for VecMap { + type Item = (K, V); + type IntoIter = std::vec::IntoIter<(K, V)>; + + fn into_iter(self) -> Self::IntoIter { + self.data.into_iter() + } } fn extract_key(entry: &(K, V)) -> &K { @@ -155,7 +241,7 @@ fn extract_key(entry: &(K, V)) -> &K { mod tests { use std::{collections::BTreeMap, ops::Bound}; - use super::VecMap; + use super::{VecMap, VecMapOrdering}; #[test] fn unbounded_range() { @@ -310,5 +396,59 @@ mod tests { left.extend(&mut one_map).unwrap_err(); assert_eq!(left.as_slice(), &[(0, ()), (1, ())]); assert_eq!(one_map.as_slice(), &[(1, ())]); + + let mut map_greater_or_equal = VecMap::new(VecMapOrdering::GreaterOrEqual); + map_greater_or_equal.append(2, ()).unwrap(); + map_greater_or_equal.append(2, ()).unwrap(); + + left.extend(&mut map_greater_or_equal).unwrap_err(); + assert_eq!(left.as_slice(), &[(0, ()), (1, ())]); + assert_eq!(map_greater_or_equal.as_slice(), &[(2, ()), (2, ())]); + } + + #[test] + fn extend_with_ordering() { + let mut left = VecMap::new(VecMapOrdering::GreaterOrEqual); + left.append(0, ()).unwrap(); + assert_eq!(left.as_slice(), &[(0, ())]); + + let mut greater_right = VecMap::new(VecMapOrdering::Greater); + greater_right.append(0, ()).unwrap(); + left.extend(&mut greater_right).unwrap_err(); + assert_eq!(left.as_slice(), &[(0, ())]); + + let mut greater_or_equal_right = VecMap::new(VecMapOrdering::GreaterOrEqual); + greater_or_equal_right.append(2, ()).unwrap(); + greater_or_equal_right.append(2, ()).unwrap(); + left.extend(&mut greater_or_equal_right).unwrap(); + assert_eq!(left.as_slice(), &[(0, ()), (2, ()), (2, ())]); + } + + #[test] + fn vec_map_from_sorted() { + let vec = vec![(1, ()), (2, ()), (3, ()), (6, ())]; + let vec_map = VecMap::from_iter(vec, VecMapOrdering::Greater); + assert_eq!(vec_map.as_slice(), &[(1, ()), (2, ()), (3, ()), (6, ())]); + + let vec = vec![(1, ()), (2, ()), (3, ()), (3, ()), (6, ()), (6, ())]; + let vec_map = VecMap::from_iter(vec, VecMapOrdering::GreaterOrEqual); + assert_eq!( + vec_map.as_slice(), + &[(1, ()), (2, ()), (3, ()), (3, ()), (6, ()), (6, ())] + ); + } + + #[test] + #[should_panic] + fn vec_map_from_unsorted_greater() { + let vec = vec![(1, ()), (2, ()), (2, ()), (3, ()), (6, ())]; + let _ = VecMap::from_iter(vec, VecMapOrdering::Greater); + } + + #[test] + #[should_panic] + fn vec_map_from_unsorted_greater_or_equal() { + let vec = vec![(1, ()), (2, ()), (3, ()), (6, ()), (5, ())]; + let _ = VecMap::from_iter(vec, VecMapOrdering::GreaterOrEqual); } } diff --git a/pageserver/src/pgdatadir_mapping.rs b/pageserver/src/pgdatadir_mapping.rs index 727650a5a5ec..6f7d74bdee31 100644 --- a/pageserver/src/pgdatadir_mapping.rs +++ b/pageserver/src/pgdatadir_mapping.rs @@ -34,6 +34,7 @@ use strum::IntoEnumIterator; use tokio_util::sync::CancellationToken; use tracing::{debug, trace, warn}; use utils::bin_ser::DeserializeError; +use utils::vec_map::{VecMap, VecMapOrdering}; use utils::{bin_ser::BeSer, lsn::Lsn}; const MAX_AUX_FILE_DELTAS: usize = 1024; @@ -1546,12 +1547,13 @@ impl<'a> DatadirModification<'a> { if !self.pending_updates.is_empty() { // The put_batch call below expects expects the inputs to be sorted by Lsn, // so we do that first. - let lsn_ordered_batch: Vec<(Key, Lsn, Value)> = self - .pending_updates - .drain() - .map(|(key, vals)| vals.into_iter().map(move |(lsn, val)| (key, lsn, val))) - .kmerge_by(|lhs, rhs| lhs.1 .0 < rhs.1 .0) - .collect(); + let lsn_ordered_batch: VecMap = VecMap::from_iter( + self.pending_updates + .drain() + .map(|(key, vals)| vals.into_iter().map(move |(lsn, val)| (lsn, (key, val)))) + .kmerge_by(|lhs, rhs| lhs.0 < rhs.0), + VecMapOrdering::GreaterOrEqual, + ); writer.put_batch(lsn_ordered_batch, ctx).await?; } diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 0b8222bca7dd..7523130f2343 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -36,6 +36,7 @@ use tracing::*; use utils::{ bin_ser::BeSer, sync::gate::{Gate, GateGuard}, + vec_map::VecMap, }; use std::ops::{Deref, Range}; @@ -4616,16 +4617,15 @@ impl<'a> TimelineWriter<'a> { } } - /// Put a batch keys at the specified Lsns. + /// Put a batch of keys at the specified Lsns. /// - /// The batch should be sorted by Lsn such that it's safe - /// to roll the open layer mid batch. + /// The batch is sorted by Lsn (enforced by usage of [`utils::vec_map::VecMap`]. pub(crate) async fn put_batch( &mut self, - batch: Vec<(Key, Lsn, Value)>, + batch: VecMap, ctx: &RequestContext, ) -> anyhow::Result<()> { - for (key, lsn, val) in batch { + for (lsn, (key, val)) in batch { self.put(key, lsn, &val, ctx).await? } From 5ec6862bcf2437480964943a4bd1c5a059561693 Mon Sep 17 00:00:00 2001 From: Conrad Ludgate Date: Thu, 21 Mar 2024 10:58:41 +0000 Subject: [PATCH 18/53] proxy: async aware password validation (#7176) ## Problem spawn_blocking in #7171 was a hack ## Summary of changes https://github.com/neondatabase/rust-postgres/pull/29 --- Cargo.lock | 11 ++++++----- proxy/src/proxy/tests.rs | 11 ++++++----- proxy/src/proxy/tests/mitm.rs | 4 ++-- proxy/src/scram.rs | 2 +- proxy/src/scram/exchange.rs | 9 +-------- proxy/src/scram/secret.rs | 6 ++---- 6 files changed, 18 insertions(+), 25 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index cdbabf2f769d..96edba7ae509 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3901,7 +3901,7 @@ dependencies = [ [[package]] name = "postgres" version = "0.19.4" -source = "git+https://github.com/neondatabase/rust-postgres.git?branch=neon#988d0ddb4184c408fa7fc1bd0ecca7993c02978f" +source = "git+https://github.com/neondatabase/rust-postgres.git?branch=neon#20031d7a9ee1addeae6e0968e3899ae6bf01cee2" dependencies = [ "bytes", "fallible-iterator", @@ -3914,7 +3914,7 @@ dependencies = [ [[package]] name = "postgres-native-tls" version = "0.5.0" -source = "git+https://github.com/neondatabase/rust-postgres.git?branch=neon#988d0ddb4184c408fa7fc1bd0ecca7993c02978f" +source = "git+https://github.com/neondatabase/rust-postgres.git?branch=neon#20031d7a9ee1addeae6e0968e3899ae6bf01cee2" dependencies = [ "native-tls", "tokio", @@ -3925,7 +3925,7 @@ dependencies = [ [[package]] name = "postgres-protocol" version = "0.6.4" -source = "git+https://github.com/neondatabase/rust-postgres.git?branch=neon#988d0ddb4184c408fa7fc1bd0ecca7993c02978f" +source = "git+https://github.com/neondatabase/rust-postgres.git?branch=neon#20031d7a9ee1addeae6e0968e3899ae6bf01cee2" dependencies = [ "base64 0.20.0", "byteorder", @@ -3938,12 +3938,13 @@ dependencies = [ "rand 0.8.5", "sha2", "stringprep", + "tokio", ] [[package]] name = "postgres-types" version = "0.2.4" -source = "git+https://github.com/neondatabase/rust-postgres.git?branch=neon#988d0ddb4184c408fa7fc1bd0ecca7993c02978f" +source = "git+https://github.com/neondatabase/rust-postgres.git?branch=neon#20031d7a9ee1addeae6e0968e3899ae6bf01cee2" dependencies = [ "bytes", "fallible-iterator", @@ -5945,7 +5946,7 @@ dependencies = [ [[package]] name = "tokio-postgres" version = "0.7.7" -source = "git+https://github.com/neondatabase/rust-postgres.git?branch=neon#988d0ddb4184c408fa7fc1bd0ecca7993c02978f" +source = "git+https://github.com/neondatabase/rust-postgres.git?branch=neon#20031d7a9ee1addeae6e0968e3899ae6bf01cee2" dependencies = [ "async-trait", "byteorder", diff --git a/proxy/src/proxy/tests.rs b/proxy/src/proxy/tests.rs index 5d0340e85201..9c3be7361291 100644 --- a/proxy/src/proxy/tests.rs +++ b/proxy/src/proxy/tests.rs @@ -135,9 +135,10 @@ impl TestAuth for NoAuth {} struct Scram(scram::ServerSecret); impl Scram { - fn new(password: &str) -> anyhow::Result { - let secret = - scram::ServerSecret::build(password).context("failed to generate scram secret")?; + async fn new(password: &str) -> anyhow::Result { + let secret = scram::ServerSecret::build(password) + .await + .context("failed to generate scram secret")?; Ok(Scram(secret)) } @@ -284,7 +285,7 @@ async fn scram_auth_good(#[case] password: &str) -> anyhow::Result<()> { let proxy = tokio::spawn(dummy_proxy( client, Some(server_config), - Scram::new(password)?, + Scram::new(password).await?, )); let (_client, _conn) = tokio_postgres::Config::new() @@ -308,7 +309,7 @@ async fn scram_auth_disable_channel_binding() -> anyhow::Result<()> { let proxy = tokio::spawn(dummy_proxy( client, Some(server_config), - Scram::new("password")?, + Scram::new("password").await?, )); let (_client, _conn) = tokio_postgres::Config::new() diff --git a/proxy/src/proxy/tests/mitm.rs b/proxy/src/proxy/tests/mitm.rs index e0c2d836f460..3b760e5dab17 100644 --- a/proxy/src/proxy/tests/mitm.rs +++ b/proxy/src/proxy/tests/mitm.rs @@ -148,7 +148,7 @@ async fn scram_auth_disable_channel_binding() -> anyhow::Result<()> { let proxy = tokio::spawn(dummy_proxy( client, Some(server_config), - Scram::new("password")?, + Scram::new("password").await?, )); let _client_err = tokio_postgres::Config::new() @@ -231,7 +231,7 @@ async fn connect_failure( let proxy = tokio::spawn(dummy_proxy( client, Some(server_config), - Scram::new("password")?, + Scram::new("password").await?, )); let _client_err = tokio_postgres::Config::new() diff --git a/proxy/src/scram.rs b/proxy/src/scram.rs index df4b3ec8d70b..76541ae2f3d2 100644 --- a/proxy/src/scram.rs +++ b/proxy/src/scram.rs @@ -114,7 +114,7 @@ mod tests { } async fn run_round_trip_test(server_password: &str, client_password: &str) { - let scram_secret = ServerSecret::build(server_password).unwrap(); + let scram_secret = ServerSecret::build(server_password).await.unwrap(); let sasl_client = ScramSha256::new(client_password.as_bytes(), ChannelBinding::unsupported()); diff --git a/proxy/src/scram/exchange.rs b/proxy/src/scram/exchange.rs index 16575d5d9810..51c0ba4e094c 100644 --- a/proxy/src/scram/exchange.rs +++ b/proxy/src/scram/exchange.rs @@ -86,14 +86,7 @@ pub async fn exchange( .map_err(|e| std::io::Error::new(std::io::ErrorKind::Other, e))?; let sent = match init.transition(secret, &tls_server_end_point, client_first)? { Continue(sent, server_first) => { - // `client.update` might perform `pbkdf2(pw)`, best to spawn it in a blocking thread. - // TODO(conrad): take this code from tokio-postgres and make an async-aware pbkdf2 impl - client = tokio::task::spawn_blocking(move || { - client.update(server_first.as_bytes())?; - Ok::(client) - }) - .await - .expect("should not panic while performing password hash")?; + client.update(server_first.as_bytes()).await?; sent } Success(x, _) => match x {}, diff --git a/proxy/src/scram/secret.rs b/proxy/src/scram/secret.rs index fb3c45816e03..b46d8c3ab588 100644 --- a/proxy/src/scram/secret.rs +++ b/proxy/src/scram/secret.rs @@ -59,10 +59,8 @@ impl ServerSecret { /// Build a new server secret from the prerequisites. /// XXX: We only use this function in tests. #[cfg(test)] - pub fn build(password: &str) -> Option { - Self::parse(&postgres_protocol::password::scram_sha_256( - password.as_bytes(), - )) + pub async fn build(password: &str) -> Option { + Self::parse(&postgres_protocol::password::scram_sha_256(password.as_bytes()).await) } } From c75b58443069d74293d55a0ccb8f71a1b77f2770 Mon Sep 17 00:00:00 2001 From: Vlad Lazar Date: Thu, 21 Mar 2024 12:00:20 +0000 Subject: [PATCH 19/53] storage_controller: add metrics (#7178) ## Problem Storage controller had basically no metrics. ## Summary of changes 1. Migrate the existing metrics to use Conrad's [`measured`](https://docs.rs/measured/0.0.14/measured/) crate. 2. Add metrics for incoming http requests 3. Add metrics for outgoing http requests to the pageserver 4. Add metrics for outgoing pass through requests to the pageserver 5. Add metrics for database queries Note that the metrics response for the attachment service does not use chunked encoding like the rest of the metrics endpoints. Conrad has kindly extended the crate such that it can now be done. Let's leave it for a follow-up since the payload shouldn't be that big at this point. Fixes https://github.com/neondatabase/neon/issues/6875 --- Cargo.lock | 33 ++ Cargo.toml | 1 + control_plane/attachment_service/Cargo.toml | 4 + control_plane/attachment_service/src/http.rs | 264 +++++++++++++-- control_plane/attachment_service/src/lib.rs | 1 + .../attachment_service/src/metrics.rs | 300 ++++++++++++++++-- control_plane/attachment_service/src/node.rs | 14 +- .../src/pageserver_client.rs | 203 ++++++++++++ .../attachment_service/src/persistence.rs | 247 +++++++++----- .../attachment_service/src/reconciler.rs | 8 +- .../attachment_service/src/service.rs | 44 ++- .../attachment_service/src/tenant_state.rs | 44 +-- libs/utils/src/http/endpoint.rs | 3 +- pageserver/src/http/routes.rs | 2 + proxy/src/http/health_server.rs | 11 +- safekeeper/src/http/routes.rs | 3 +- test_runner/regress/test_sharding.py | 16 +- 17 files changed, 1002 insertions(+), 196 deletions(-) create mode 100644 control_plane/attachment_service/src/pageserver_client.rs diff --git a/Cargo.lock b/Cargo.lock index 96edba7ae509..dcef66c15d56 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -277,6 +277,7 @@ dependencies = [ "anyhow", "aws-config", "aws-sdk-secretsmanager", + "bytes", "camino", "clap", "control_plane", @@ -288,6 +289,8 @@ dependencies = [ "hex", "humantime", "hyper", + "lasso", + "measured", "metrics", "once_cell", "pageserver_api", @@ -295,6 +298,7 @@ dependencies = [ "postgres_connection", "r2d2", "reqwest", + "routerify", "serde", "serde_json", "thiserror", @@ -2880,6 +2884,35 @@ version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "490cc448043f947bae3cbee9c203358d62dbee0db12107a74be5c30ccfd09771" +[[package]] +name = "measured" +version = "0.0.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f246648d027839a34b420e27c7de1165ace96e19ef894985d0a6ff89a7840a9f" +dependencies = [ + "bytes", + "hashbrown 0.14.0", + "itoa", + "lasso", + "measured-derive", + "memchr", + "parking_lot 0.12.1", + "rustc-hash", + "ryu", +] + +[[package]] +name = "measured-derive" +version = "0.0.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "edaa5cc22d99d5d6d7d99c3b5b5f7e7f8034c22f1b5d62a1adecd2ed005d9b80" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 2.0.52", +] + [[package]] name = "memchr" version = "2.6.4" diff --git a/Cargo.toml b/Cargo.toml index 76f4ff041c09..0f3dbd498702 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -101,6 +101,7 @@ lasso = "0.7" leaky-bucket = "1.0.1" libc = "0.2" md5 = "0.7.0" +measured = { version = "0.0.13", features=["default", "lasso"] } memoffset = "0.8" native-tls = "0.2" nix = { version = "0.27", features = ["fs", "process", "socket", "signal", "poll"] } diff --git a/control_plane/attachment_service/Cargo.toml b/control_plane/attachment_service/Cargo.toml index f78f56c48039..34882659e359 100644 --- a/control_plane/attachment_service/Cargo.toml +++ b/control_plane/attachment_service/Cargo.toml @@ -17,6 +17,7 @@ testing = [] anyhow.workspace = true aws-config.workspace = true aws-sdk-secretsmanager.workspace = true +bytes.workspace = true camino.workspace = true clap.workspace = true fail.workspace = true @@ -25,17 +26,20 @@ git-version.workspace = true hex.workspace = true hyper.workspace = true humantime.workspace = true +lasso.workspace = true once_cell.workspace = true pageserver_api.workspace = true pageserver_client.workspace = true postgres_connection.workspace = true reqwest.workspace = true +routerify.workspace = true serde.workspace = true serde_json.workspace = true thiserror.workspace = true tokio.workspace = true tokio-util.workspace = true tracing.workspace = true +measured.workspace = true diesel = { version = "2.1.4", features = ["serde_json", "postgres", "r2d2"] } diesel_migrations = { version = "2.1.0" } diff --git a/control_plane/attachment_service/src/http.rs b/control_plane/attachment_service/src/http.rs index 076b3a2f70c5..036019cd384a 100644 --- a/control_plane/attachment_service/src/http.rs +++ b/control_plane/attachment_service/src/http.rs @@ -1,5 +1,11 @@ +use crate::metrics::{ + HttpRequestLatencyLabelGroup, HttpRequestStatusLabelGroup, PageserverRequestLabelGroup, + METRICS_REGISTRY, +}; use crate::reconciler::ReconcileError; use crate::service::{Service, STARTUP_RECONCILE_TIMEOUT}; +use futures::Future; +use hyper::header::CONTENT_TYPE; use hyper::{Body, Request, Response}; use hyper::{StatusCode, Uri}; use pageserver_api::models::{ @@ -34,6 +40,8 @@ use pageserver_api::upcall_api::{ReAttachRequest, ValidateRequest}; use control_plane::storage_controller::{AttachHookRequest, InspectRequest}; +use routerify::Middleware; + /// State available to HTTP request handlers #[derive(Clone)] pub struct HttpState { @@ -313,7 +321,7 @@ async fn handle_tenant_timeline_passthrough( tracing::info!("Proxying request for tenant {} ({})", tenant_id, path); // Find the node that holds shard zero - let (base_url, tenant_shard_id) = service.tenant_shard0_baseurl(tenant_id)?; + let (node, tenant_shard_id) = service.tenant_shard0_node(tenant_id)?; // Callers will always pass an unsharded tenant ID. Before proxying, we must // rewrite this to a shard-aware shard zero ID. @@ -322,12 +330,39 @@ async fn handle_tenant_timeline_passthrough( let tenant_shard_str = format!("{}", tenant_shard_id); let path = path.replace(&tenant_str, &tenant_shard_str); - let client = mgmt_api::Client::new(base_url, service.get_config().jwt_token.as_deref()); + let latency = &METRICS_REGISTRY + .metrics_group + .storage_controller_passthrough_request_latency; + + // This is a bit awkward. We remove the param from the request + // and join the words by '_' to get a label for the request. + let just_path = path.replace(&tenant_shard_str, ""); + let path_label = just_path + .split('/') + .filter(|token| !token.is_empty()) + .collect::>() + .join("_"); + let labels = PageserverRequestLabelGroup { + pageserver_id: &node.get_id().to_string(), + path: &path_label, + method: crate::metrics::Method::Get, + }; + + let _timer = latency.start_timer(labels.clone()); + + let client = mgmt_api::Client::new(node.base_url(), service.get_config().jwt_token.as_deref()); let resp = client.get_raw(path).await.map_err(|_e| // FIXME: give APiError a proper Unavailable variant. We return 503 here because // if we can't successfully send a request to the pageserver, we aren't available. ApiError::ShuttingDown)?; + if !resp.status().is_success() { + let error_counter = &METRICS_REGISTRY + .metrics_group + .storage_controller_passthrough_request_error; + error_counter.inc(labels); + } + // We have a reqest::Response, would like a http::Response let mut builder = hyper::Response::builder() .status(resp.status()) @@ -498,7 +533,11 @@ impl From for ApiError { /// Common wrapper for request handlers that call into Service and will operate on tenants: they must only /// be allowed to run if Service has finished its initial reconciliation. -async fn tenant_service_handler(request: Request, handler: H) -> R::Output +async fn tenant_service_handler( + request: Request, + handler: H, + request_name: RequestName, +) -> R::Output where R: std::future::Future, ApiError>> + Send + 'static, H: FnOnce(Arc, Request) -> R + Send + Sync + 'static, @@ -518,9 +557,10 @@ where )); } - request_span( + named_request_span( request, |request| async move { handler(service, request).await }, + request_name, ) .await } @@ -531,11 +571,98 @@ fn check_permissions(request: &Request, required_scope: Scope) -> Result<( }) } +#[derive(Clone, Debug)] +struct RequestMeta { + method: hyper::http::Method, + at: Instant, +} + +fn prologue_metrics_middleware( +) -> Middleware { + Middleware::pre(move |req| async move { + let meta = RequestMeta { + method: req.method().clone(), + at: Instant::now(), + }; + + req.set_context(meta); + + Ok(req) + }) +} + +fn epilogue_metrics_middleware( +) -> Middleware { + Middleware::post_with_info(move |resp, req_info| async move { + let request_name = match req_info.context::() { + Some(name) => name, + None => { + return Ok(resp); + } + }; + + if let Some(meta) = req_info.context::() { + let status = &crate::metrics::METRICS_REGISTRY + .metrics_group + .storage_controller_http_request_status; + let latency = &crate::metrics::METRICS_REGISTRY + .metrics_group + .storage_controller_http_request_latency; + + status.inc(HttpRequestStatusLabelGroup { + path: request_name.0, + method: meta.method.clone().into(), + status: crate::metrics::StatusCode(resp.status()), + }); + + latency.observe( + HttpRequestLatencyLabelGroup { + path: request_name.0, + method: meta.method.into(), + }, + meta.at.elapsed().as_secs_f64(), + ); + } + Ok(resp) + }) +} + +pub async fn measured_metrics_handler(_req: Request) -> Result, ApiError> { + pub const TEXT_FORMAT: &str = "text/plain; version=0.0.4"; + + let payload = crate::metrics::METRICS_REGISTRY.encode(); + let response = Response::builder() + .status(200) + .header(CONTENT_TYPE, TEXT_FORMAT) + .body(payload.into()) + .unwrap(); + + Ok(response) +} + +#[derive(Clone)] +struct RequestName(&'static str); + +async fn named_request_span( + request: Request, + handler: H, + name: RequestName, +) -> R::Output +where + R: Future, ApiError>> + Send + 'static, + H: FnOnce(Request) -> R + Send + Sync + 'static, +{ + request.set_context(name); + request_span(request, handler).await +} + pub fn make_router( service: Arc, auth: Option>, ) -> RouterBuilder { - let mut router = endpoint::make_router(); + let mut router = endpoint::make_router() + .middleware(prologue_metrics_middleware()) + .middleware(epilogue_metrics_middleware()); if auth.is_some() { router = router.middleware(auth_middleware(|request| { let state = get_state(request); @@ -544,99 +671,166 @@ pub fn make_router( } else { state.auth.as_deref() } - })) + })); } router .data(Arc::new(HttpState::new(service, auth))) + .get("/metrics", |r| { + named_request_span(r, measured_metrics_handler, RequestName("metrics")) + }) // Non-prefixed generic endpoints (status, metrics) - .get("/status", |r| request_span(r, handle_status)) - .get("/ready", |r| request_span(r, handle_ready)) + .get("/status", |r| { + named_request_span(r, handle_status, RequestName("status")) + }) + .get("/ready", |r| { + named_request_span(r, handle_ready, RequestName("ready")) + }) // Upcalls for the pageserver: point the pageserver's `control_plane_api` config to this prefix .post("/upcall/v1/re-attach", |r| { - request_span(r, handle_re_attach) + named_request_span(r, handle_re_attach, RequestName("upcall_v1_reattach")) + }) + .post("/upcall/v1/validate", |r| { + named_request_span(r, handle_validate, RequestName("upcall_v1_validate")) }) - .post("/upcall/v1/validate", |r| request_span(r, handle_validate)) // Test/dev/debug endpoints .post("/debug/v1/attach-hook", |r| { - request_span(r, handle_attach_hook) + named_request_span(r, handle_attach_hook, RequestName("debug_v1_attach_hook")) + }) + .post("/debug/v1/inspect", |r| { + named_request_span(r, handle_inspect, RequestName("debug_v1_inspect")) }) - .post("/debug/v1/inspect", |r| request_span(r, handle_inspect)) .post("/debug/v1/tenant/:tenant_id/drop", |r| { - request_span(r, handle_tenant_drop) + named_request_span(r, handle_tenant_drop, RequestName("debug_v1_tenant_drop")) }) .post("/debug/v1/node/:node_id/drop", |r| { - request_span(r, handle_node_drop) + named_request_span(r, handle_node_drop, RequestName("debug_v1_node_drop")) + }) + .get("/debug/v1/tenant", |r| { + named_request_span(r, handle_tenants_dump, RequestName("debug_v1_tenant")) }) - .get("/debug/v1/tenant", |r| request_span(r, handle_tenants_dump)) .get("/debug/v1/tenant/:tenant_id/locate", |r| { - tenant_service_handler(r, handle_tenant_locate) + tenant_service_handler( + r, + handle_tenant_locate, + RequestName("debug_v1_tenant_locate"), + ) }) .get("/debug/v1/scheduler", |r| { - request_span(r, handle_scheduler_dump) + named_request_span(r, handle_scheduler_dump, RequestName("debug_v1_scheduler")) }) .post("/debug/v1/consistency_check", |r| { - request_span(r, handle_consistency_check) + named_request_span( + r, + handle_consistency_check, + RequestName("debug_v1_consistency_check"), + ) }) .put("/debug/v1/failpoints", |r| { request_span(r, |r| failpoints_handler(r, CancellationToken::new())) }) // Node operations .post("/control/v1/node", |r| { - request_span(r, handle_node_register) + named_request_span(r, handle_node_register, RequestName("control_v1_node")) + }) + .get("/control/v1/node", |r| { + named_request_span(r, handle_node_list, RequestName("control_v1_node")) }) - .get("/control/v1/node", |r| request_span(r, handle_node_list)) .put("/control/v1/node/:node_id/config", |r| { - request_span(r, handle_node_configure) + named_request_span( + r, + handle_node_configure, + RequestName("control_v1_node_config"), + ) }) // Tenant Shard operations .put("/control/v1/tenant/:tenant_shard_id/migrate", |r| { - tenant_service_handler(r, handle_tenant_shard_migrate) + tenant_service_handler( + r, + handle_tenant_shard_migrate, + RequestName("control_v1_tenant_migrate"), + ) }) .put("/control/v1/tenant/:tenant_id/shard_split", |r| { - tenant_service_handler(r, handle_tenant_shard_split) + tenant_service_handler( + r, + handle_tenant_shard_split, + RequestName("control_v1_tenant_shard_split"), + ) }) .get("/control/v1/tenant/:tenant_id", |r| { - tenant_service_handler(r, handle_tenant_describe) + tenant_service_handler( + r, + handle_tenant_describe, + RequestName("control_v1_tenant_describe"), + ) }) // Tenant operations // The ^/v1/ endpoints act as a "Virtual Pageserver", enabling shard-naive clients to call into // this service to manage tenants that actually consist of many tenant shards, as if they are a single entity. .post("/v1/tenant", |r| { - tenant_service_handler(r, handle_tenant_create) + tenant_service_handler(r, handle_tenant_create, RequestName("v1_tenant")) }) .delete("/v1/tenant/:tenant_id", |r| { - tenant_service_handler(r, handle_tenant_delete) + tenant_service_handler(r, handle_tenant_delete, RequestName("v1_tenant")) }) .put("/v1/tenant/config", |r| { - tenant_service_handler(r, handle_tenant_config_set) + tenant_service_handler(r, handle_tenant_config_set, RequestName("v1_tenant_config")) }) .get("/v1/tenant/:tenant_id/config", |r| { - tenant_service_handler(r, handle_tenant_config_get) + tenant_service_handler(r, handle_tenant_config_get, RequestName("v1_tenant_config")) }) .put("/v1/tenant/:tenant_shard_id/location_config", |r| { - tenant_service_handler(r, handle_tenant_location_config) + tenant_service_handler( + r, + handle_tenant_location_config, + RequestName("v1_tenant_location_config"), + ) }) .put("/v1/tenant/:tenant_id/time_travel_remote_storage", |r| { - tenant_service_handler(r, handle_tenant_time_travel_remote_storage) + tenant_service_handler( + r, + handle_tenant_time_travel_remote_storage, + RequestName("v1_tenant_time_travel_remote_storage"), + ) }) .post("/v1/tenant/:tenant_id/secondary/download", |r| { - tenant_service_handler(r, handle_tenant_secondary_download) + tenant_service_handler( + r, + handle_tenant_secondary_download, + RequestName("v1_tenant_secondary_download"), + ) }) // Timeline operations .delete("/v1/tenant/:tenant_id/timeline/:timeline_id", |r| { - tenant_service_handler(r, handle_tenant_timeline_delete) + tenant_service_handler( + r, + handle_tenant_timeline_delete, + RequestName("v1_tenant_timeline"), + ) }) .post("/v1/tenant/:tenant_id/timeline", |r| { - tenant_service_handler(r, handle_tenant_timeline_create) + tenant_service_handler( + r, + handle_tenant_timeline_create, + RequestName("v1_tenant_timeline"), + ) }) // Tenant detail GET passthrough to shard zero .get("/v1/tenant/:tenant_id", |r| { - tenant_service_handler(r, handle_tenant_timeline_passthrough) + tenant_service_handler( + r, + handle_tenant_timeline_passthrough, + RequestName("v1_tenant_passthrough"), + ) }) // Timeline GET passthrough to shard zero. Note that the `*` in the URL is a wildcard: any future // timeline GET APIs will be implicitly included. .get("/v1/tenant/:tenant_id/timeline*", |r| { - tenant_service_handler(r, handle_tenant_timeline_passthrough) + tenant_service_handler( + r, + handle_tenant_timeline_passthrough, + RequestName("v1_tenant_timeline_passthrough"), + ) }) } diff --git a/control_plane/attachment_service/src/lib.rs b/control_plane/attachment_service/src/lib.rs index 4aff29f15b54..8bcd5c0ac47f 100644 --- a/control_plane/attachment_service/src/lib.rs +++ b/control_plane/attachment_service/src/lib.rs @@ -8,6 +8,7 @@ pub mod http; mod id_lock_map; pub mod metrics; mod node; +mod pageserver_client; pub mod persistence; mod reconciler; mod scheduler; diff --git a/control_plane/attachment_service/src/metrics.rs b/control_plane/attachment_service/src/metrics.rs index ffe093b9c887..ccf5e9b07c10 100644 --- a/control_plane/attachment_service/src/metrics.rs +++ b/control_plane/attachment_service/src/metrics.rs @@ -1,32 +1,284 @@ -use metrics::{register_int_counter, register_int_counter_vec, IntCounter, IntCounterVec}; +//! +//! This module provides metric definitions for the storage controller. +//! +//! All metrics are grouped in [`StorageControllerMetricGroup`]. [`StorageControllerMetrics`] holds +//! the mentioned metrics and their encoder. It's globally available via the [`METRICS_REGISTRY`] +//! constant. +//! +//! The rest of the code defines label group types and deals with converting outer types to labels. +//! +use bytes::Bytes; +use measured::{ + label::{LabelValue, StaticLabelSet}, + FixedCardinalityLabel, MetricGroup, +}; use once_cell::sync::Lazy; +use std::sync::Mutex; -pub(crate) struct ReconcilerMetrics { - pub(crate) spawned: IntCounter, - pub(crate) complete: IntCounterVec, +use crate::persistence::{DatabaseError, DatabaseOperation}; + +pub(crate) static METRICS_REGISTRY: Lazy = + Lazy::new(StorageControllerMetrics::default); + +pub fn preinitialize_metrics() { + Lazy::force(&METRICS_REGISTRY); } -impl ReconcilerMetrics { - // Labels used on [`Self::complete`] - pub(crate) const SUCCESS: &'static str = "ok"; - pub(crate) const ERROR: &'static str = "success"; - pub(crate) const CANCEL: &'static str = "cancel"; +pub(crate) struct StorageControllerMetrics { + pub(crate) metrics_group: StorageControllerMetricGroup, + encoder: Mutex, } -pub(crate) static RECONCILER: Lazy = Lazy::new(|| ReconcilerMetrics { - spawned: register_int_counter!( - "storage_controller_reconcile_spawn", - "Count of how many times we spawn a reconcile task", - ) - .expect("failed to define a metric"), - complete: register_int_counter_vec!( - "storage_controller_reconcile_complete", - "Reconciler tasks completed, broken down by success/failure/cancelled", - &["status"], - ) - .expect("failed to define a metric"), -}); +#[derive(measured::MetricGroup)] +pub(crate) struct StorageControllerMetricGroup { + /// Count of how many times we spawn a reconcile task + pub(crate) storage_controller_reconcile_spawn: measured::Counter, + /// Reconciler tasks completed, broken down by success/failure/cancelled + pub(crate) storage_controller_reconcile_complete: + measured::CounterVec, -pub fn preinitialize_metrics() { - Lazy::force(&RECONCILER); + /// HTTP request status counters for handled requests + pub(crate) storage_controller_http_request_status: + measured::CounterVec, + /// HTTP request handler latency across all status codes + pub(crate) storage_controller_http_request_latency: + measured::HistogramVec, + + /// Count of HTTP requests to the pageserver that resulted in an error, + /// broken down by the pageserver node id, request name and method + pub(crate) storage_controller_pageserver_request_error: + measured::CounterVec, + + /// Latency of HTTP requests to the pageserver, broken down by pageserver + /// node id, request name and method. This include both successful and unsuccessful + /// requests. + pub(crate) storage_controller_pageserver_request_latency: + measured::HistogramVec, + + /// Count of pass-through HTTP requests to the pageserver that resulted in an error, + /// broken down by the pageserver node id, request name and method + pub(crate) storage_controller_passthrough_request_error: + measured::CounterVec, + + /// Latency of pass-through HTTP requests to the pageserver, broken down by pageserver + /// node id, request name and method. This include both successful and unsuccessful + /// requests. + pub(crate) storage_controller_passthrough_request_latency: + measured::HistogramVec, + + /// Count of errors in database queries, broken down by error type and operation. + pub(crate) storage_controller_database_query_error: + measured::CounterVec, + + /// Latency of database queries, broken down by operation. + pub(crate) storage_controller_database_query_latency: + measured::HistogramVec, +} + +impl StorageControllerMetrics { + pub(crate) fn encode(&self) -> Bytes { + let mut encoder = self.encoder.lock().unwrap(); + self.metrics_group.collect_into(&mut *encoder); + encoder.finish() + } +} + +impl Default for StorageControllerMetrics { + fn default() -> Self { + Self { + metrics_group: StorageControllerMetricGroup::new(), + encoder: Mutex::new(measured::text::TextEncoder::new()), + } + } +} + +impl StorageControllerMetricGroup { + pub(crate) fn new() -> Self { + Self { + storage_controller_reconcile_spawn: measured::Counter::new(), + storage_controller_reconcile_complete: measured::CounterVec::new( + ReconcileCompleteLabelGroupSet { + status: StaticLabelSet::new(), + }, + ), + storage_controller_http_request_status: measured::CounterVec::new( + HttpRequestStatusLabelGroupSet { + path: lasso::ThreadedRodeo::new(), + method: StaticLabelSet::new(), + status: StaticLabelSet::new(), + }, + ), + storage_controller_http_request_latency: measured::HistogramVec::new( + measured::metric::histogram::Thresholds::exponential_buckets(0.1, 2.0), + ), + storage_controller_pageserver_request_error: measured::CounterVec::new( + PageserverRequestLabelGroupSet { + pageserver_id: lasso::ThreadedRodeo::new(), + path: lasso::ThreadedRodeo::new(), + method: StaticLabelSet::new(), + }, + ), + storage_controller_pageserver_request_latency: measured::HistogramVec::new( + measured::metric::histogram::Thresholds::exponential_buckets(0.1, 2.0), + ), + storage_controller_passthrough_request_error: measured::CounterVec::new( + PageserverRequestLabelGroupSet { + pageserver_id: lasso::ThreadedRodeo::new(), + path: lasso::ThreadedRodeo::new(), + method: StaticLabelSet::new(), + }, + ), + storage_controller_passthrough_request_latency: measured::HistogramVec::new( + measured::metric::histogram::Thresholds::exponential_buckets(0.1, 2.0), + ), + storage_controller_database_query_error: measured::CounterVec::new( + DatabaseQueryErrorLabelGroupSet { + operation: StaticLabelSet::new(), + error_type: StaticLabelSet::new(), + }, + ), + storage_controller_database_query_latency: measured::HistogramVec::new( + measured::metric::histogram::Thresholds::exponential_buckets(0.1, 2.0), + ), + } + } +} + +#[derive(measured::LabelGroup)] +#[label(set = ReconcileCompleteLabelGroupSet)] +pub(crate) struct ReconcileCompleteLabelGroup { + pub(crate) status: ReconcileOutcome, +} + +#[derive(measured::LabelGroup)] +#[label(set = HttpRequestStatusLabelGroupSet)] +pub(crate) struct HttpRequestStatusLabelGroup<'a> { + #[label(dynamic_with = lasso::ThreadedRodeo)] + pub(crate) path: &'a str, + pub(crate) method: Method, + pub(crate) status: StatusCode, +} + +#[derive(measured::LabelGroup)] +#[label(set = HttpRequestLatencyLabelGroupSet)] +pub(crate) struct HttpRequestLatencyLabelGroup<'a> { + #[label(dynamic_with = lasso::ThreadedRodeo)] + pub(crate) path: &'a str, + pub(crate) method: Method, +} + +impl Default for HttpRequestLatencyLabelGroupSet { + fn default() -> Self { + Self { + path: lasso::ThreadedRodeo::new(), + method: StaticLabelSet::new(), + } + } +} + +#[derive(measured::LabelGroup, Clone)] +#[label(set = PageserverRequestLabelGroupSet)] +pub(crate) struct PageserverRequestLabelGroup<'a> { + #[label(dynamic_with = lasso::ThreadedRodeo)] + pub(crate) pageserver_id: &'a str, + #[label(dynamic_with = lasso::ThreadedRodeo)] + pub(crate) path: &'a str, + pub(crate) method: Method, +} + +impl Default for PageserverRequestLabelGroupSet { + fn default() -> Self { + Self { + pageserver_id: lasso::ThreadedRodeo::new(), + path: lasso::ThreadedRodeo::new(), + method: StaticLabelSet::new(), + } + } +} + +#[derive(measured::LabelGroup)] +#[label(set = DatabaseQueryErrorLabelGroupSet)] +pub(crate) struct DatabaseQueryErrorLabelGroup { + pub(crate) error_type: DatabaseErrorLabel, + pub(crate) operation: DatabaseOperation, +} + +#[derive(measured::LabelGroup)] +#[label(set = DatabaseQueryLatencyLabelGroupSet)] +pub(crate) struct DatabaseQueryLatencyLabelGroup { + pub(crate) operation: DatabaseOperation, +} + +#[derive(FixedCardinalityLabel)] +pub(crate) enum ReconcileOutcome { + #[label(rename = "ok")] + Success, + Error, + Cancel, +} + +#[derive(FixedCardinalityLabel, Clone)] +pub(crate) enum Method { + Get, + Put, + Post, + Delete, + Other, +} + +impl From for Method { + fn from(value: hyper::Method) -> Self { + if value == hyper::Method::GET { + Method::Get + } else if value == hyper::Method::PUT { + Method::Put + } else if value == hyper::Method::POST { + Method::Post + } else if value == hyper::Method::DELETE { + Method::Delete + } else { + Method::Other + } + } +} + +pub(crate) struct StatusCode(pub(crate) hyper::http::StatusCode); + +impl LabelValue for StatusCode { + fn visit(&self, v: V) -> V::Output { + v.write_int(self.0.as_u16() as u64) + } +} + +impl FixedCardinalityLabel for StatusCode { + fn cardinality() -> usize { + (100..1000).len() + } + + fn encode(&self) -> usize { + self.0.as_u16() as usize + } + + fn decode(value: usize) -> Self { + Self(hyper::http::StatusCode::from_u16(u16::try_from(value).unwrap()).unwrap()) + } +} + +#[derive(FixedCardinalityLabel)] +pub(crate) enum DatabaseErrorLabel { + Query, + Connection, + ConnectionPool, + Logical, +} + +impl DatabaseError { + pub(crate) fn error_label(&self) -> DatabaseErrorLabel { + match self { + Self::Query(_) => DatabaseErrorLabel::Query, + Self::Connection(_) => DatabaseErrorLabel::Connection, + Self::ConnectionPool(_) => DatabaseErrorLabel::ConnectionPool, + Self::Logical(_) => DatabaseErrorLabel::Logical, + } + } } diff --git a/control_plane/attachment_service/src/node.rs b/control_plane/attachment_service/src/node.rs index 4167782715af..df40bff66f74 100644 --- a/control_plane/attachment_service/src/node.rs +++ b/control_plane/attachment_service/src/node.rs @@ -12,7 +12,9 @@ use serde::Serialize; use tokio_util::sync::CancellationToken; use utils::{backoff, id::NodeId}; -use crate::{persistence::NodePersistence, scheduler::MaySchedule}; +use crate::{ + pageserver_client::PageserverClient, persistence::NodePersistence, scheduler::MaySchedule, +}; /// Represents the in-memory description of a Node. /// @@ -202,7 +204,7 @@ impl Node { cancel: &CancellationToken, ) -> Option> where - O: FnMut(mgmt_api::Client) -> F, + O: FnMut(PageserverClient) -> F, F: std::future::Future>, { fn is_fatal(e: &mgmt_api::Error) -> bool { @@ -224,8 +226,12 @@ impl Node { .build() .expect("Failed to construct HTTP client"); - let client = - mgmt_api::Client::from_client(http_client, self.base_url(), jwt.as_deref()); + let client = PageserverClient::from_client( + self.get_id(), + http_client, + self.base_url(), + jwt.as_deref(), + ); let node_cancel_fut = self.cancel.cancelled(); diff --git a/control_plane/attachment_service/src/pageserver_client.rs b/control_plane/attachment_service/src/pageserver_client.rs new file mode 100644 index 000000000000..8237229d7bce --- /dev/null +++ b/control_plane/attachment_service/src/pageserver_client.rs @@ -0,0 +1,203 @@ +use pageserver_api::{ + models::{ + LocationConfig, LocationConfigListResponse, PageserverUtilization, SecondaryProgress, + TenantShardSplitRequest, TenantShardSplitResponse, TimelineCreateRequest, TimelineInfo, + }, + shard::TenantShardId, +}; +use pageserver_client::mgmt_api::{Client, Result}; +use reqwest::StatusCode; +use utils::id::{NodeId, TimelineId}; + +/// Thin wrapper around [`pageserver_client::mgmt_api::Client`]. It allows the storage +/// controller to collect metrics in a non-intrusive manner. +#[derive(Debug, Clone)] +pub(crate) struct PageserverClient { + inner: Client, + node_id_label: String, +} + +macro_rules! measured_request { + ($name:literal, $method:expr, $node_id: expr, $invoke:expr) => {{ + let labels = crate::metrics::PageserverRequestLabelGroup { + pageserver_id: $node_id, + path: $name, + method: $method, + }; + + let latency = &crate::metrics::METRICS_REGISTRY + .metrics_group + .storage_controller_pageserver_request_latency; + let _timer_guard = latency.start_timer(labels.clone()); + + let res = $invoke; + + if res.is_err() { + let error_counters = &crate::metrics::METRICS_REGISTRY + .metrics_group + .storage_controller_pageserver_request_error; + error_counters.inc(labels) + } + + res + }}; +} + +impl PageserverClient { + pub(crate) fn new(node_id: NodeId, mgmt_api_endpoint: String, jwt: Option<&str>) -> Self { + Self { + inner: Client::from_client(reqwest::Client::new(), mgmt_api_endpoint, jwt), + node_id_label: node_id.0.to_string(), + } + } + + pub(crate) fn from_client( + node_id: NodeId, + raw_client: reqwest::Client, + mgmt_api_endpoint: String, + jwt: Option<&str>, + ) -> Self { + Self { + inner: Client::from_client(raw_client, mgmt_api_endpoint, jwt), + node_id_label: node_id.0.to_string(), + } + } + + pub(crate) async fn tenant_delete(&self, tenant_shard_id: TenantShardId) -> Result { + measured_request!( + "tenant", + crate::metrics::Method::Delete, + &self.node_id_label, + self.inner.tenant_delete(tenant_shard_id).await + ) + } + + pub(crate) async fn tenant_time_travel_remote_storage( + &self, + tenant_shard_id: TenantShardId, + timestamp: &str, + done_if_after: &str, + ) -> Result<()> { + measured_request!( + "tenant_time_travel_remote_storage", + crate::metrics::Method::Put, + &self.node_id_label, + self.inner + .tenant_time_travel_remote_storage(tenant_shard_id, timestamp, done_if_after) + .await + ) + } + + pub(crate) async fn tenant_secondary_download( + &self, + tenant_id: TenantShardId, + wait: Option, + ) -> Result<(StatusCode, SecondaryProgress)> { + measured_request!( + "tenant_secondary_download", + crate::metrics::Method::Post, + &self.node_id_label, + self.inner.tenant_secondary_download(tenant_id, wait).await + ) + } + + pub(crate) async fn location_config( + &self, + tenant_shard_id: TenantShardId, + config: LocationConfig, + flush_ms: Option, + lazy: bool, + ) -> Result<()> { + measured_request!( + "location_config", + crate::metrics::Method::Put, + &self.node_id_label, + self.inner + .location_config(tenant_shard_id, config, flush_ms, lazy) + .await + ) + } + + pub(crate) async fn list_location_config(&self) -> Result { + measured_request!( + "location_configs", + crate::metrics::Method::Get, + &self.node_id_label, + self.inner.list_location_config().await + ) + } + + pub(crate) async fn get_location_config( + &self, + tenant_shard_id: TenantShardId, + ) -> Result> { + measured_request!( + "location_config", + crate::metrics::Method::Get, + &self.node_id_label, + self.inner.get_location_config(tenant_shard_id).await + ) + } + + pub(crate) async fn timeline_create( + &self, + tenant_shard_id: TenantShardId, + req: &TimelineCreateRequest, + ) -> Result { + measured_request!( + "timeline", + crate::metrics::Method::Post, + &self.node_id_label, + self.inner.timeline_create(tenant_shard_id, req).await + ) + } + + pub(crate) async fn timeline_delete( + &self, + tenant_shard_id: TenantShardId, + timeline_id: TimelineId, + ) -> Result { + measured_request!( + "timeline", + crate::metrics::Method::Delete, + &self.node_id_label, + self.inner + .timeline_delete(tenant_shard_id, timeline_id) + .await + ) + } + + pub(crate) async fn tenant_shard_split( + &self, + tenant_shard_id: TenantShardId, + req: TenantShardSplitRequest, + ) -> Result { + measured_request!( + "tenant_shard_split", + crate::metrics::Method::Put, + &self.node_id_label, + self.inner.tenant_shard_split(tenant_shard_id, req).await + ) + } + + pub(crate) async fn timeline_list( + &self, + tenant_shard_id: &TenantShardId, + ) -> Result> { + measured_request!( + "timelines", + crate::metrics::Method::Get, + &self.node_id_label, + self.inner.timeline_list(tenant_shard_id).await + ) + } + + pub(crate) async fn get_utilization(&self) -> Result { + measured_request!( + "utilization", + crate::metrics::Method::Get, + &self.node_id_label, + self.inner.get_utilization().await + ) + } +} diff --git a/control_plane/attachment_service/src/persistence.rs b/control_plane/attachment_service/src/persistence.rs index 209d8ff0752a..dafd52017b1e 100644 --- a/control_plane/attachment_service/src/persistence.rs +++ b/control_plane/attachment_service/src/persistence.rs @@ -19,6 +19,9 @@ use serde::{Deserialize, Serialize}; use utils::generation::Generation; use utils::id::{NodeId, TenantId}; +use crate::metrics::{ + DatabaseQueryErrorLabelGroup, DatabaseQueryLatencyLabelGroup, METRICS_REGISTRY, +}; use crate::node::Node; /// ## What do we store? @@ -75,6 +78,25 @@ pub(crate) enum DatabaseError { Logical(String), } +#[derive(measured::FixedCardinalityLabel, Clone)] +pub(crate) enum DatabaseOperation { + InsertNode, + UpdateNode, + DeleteNode, + ListNodes, + BeginShardSplit, + CompleteShardSplit, + AbortShardSplit, + Detach, + ReAttach, + IncrementGeneration, + ListTenantShards, + InsertTenantShards, + UpdateTenantShard, + DeleteTenant, + UpdateTenantConfig, +} + #[must_use] pub(crate) enum AbortShardSplitStatus { /// We aborted the split in the database by reverting to the parent shards @@ -115,6 +137,34 @@ impl Persistence { } } + /// Wraps `with_conn` in order to collect latency and error metrics + async fn with_measured_conn(&self, op: DatabaseOperation, func: F) -> DatabaseResult + where + F: Fn(&mut PgConnection) -> DatabaseResult + Send + 'static, + R: Send + 'static, + { + let latency = &METRICS_REGISTRY + .metrics_group + .storage_controller_database_query_latency; + let _timer = latency.start_timer(DatabaseQueryLatencyLabelGroup { + operation: op.clone(), + }); + + let res = self.with_conn(func).await; + + if let Err(err) = &res { + let error_counter = &METRICS_REGISTRY + .metrics_group + .storage_controller_database_query_error; + error_counter.inc(DatabaseQueryErrorLabelGroup { + error_type: err.error_label(), + operation: op, + }) + } + + res + } + /// Call the provided function in a tokio blocking thread, with a Diesel database connection. async fn with_conn(&self, func: F) -> DatabaseResult where @@ -130,21 +180,27 @@ impl Persistence { /// When a node is first registered, persist it before using it for anything pub(crate) async fn insert_node(&self, node: &Node) -> DatabaseResult<()> { let np = node.to_persistent(); - self.with_conn(move |conn| -> DatabaseResult<()> { - diesel::insert_into(crate::schema::nodes::table) - .values(&np) - .execute(conn)?; - Ok(()) - }) + self.with_measured_conn( + DatabaseOperation::InsertNode, + move |conn| -> DatabaseResult<()> { + diesel::insert_into(crate::schema::nodes::table) + .values(&np) + .execute(conn)?; + Ok(()) + }, + ) .await } /// At startup, populate the list of nodes which our shards may be placed on pub(crate) async fn list_nodes(&self) -> DatabaseResult> { let nodes: Vec = self - .with_conn(move |conn| -> DatabaseResult<_> { - Ok(crate::schema::nodes::table.load::(conn)?) - }) + .with_measured_conn( + DatabaseOperation::ListNodes, + move |conn| -> DatabaseResult<_> { + Ok(crate::schema::nodes::table.load::(conn)?) + }, + ) .await?; tracing::info!("list_nodes: loaded {} nodes", nodes.len()); @@ -159,7 +215,7 @@ impl Persistence { ) -> DatabaseResult<()> { use crate::schema::nodes::dsl::*; let updated = self - .with_conn(move |conn| { + .with_measured_conn(DatabaseOperation::UpdateNode, move |conn| { let updated = diesel::update(nodes) .filter(node_id.eq(input_node_id.0 as i64)) .set((scheduling_policy.eq(String::from(input_scheduling)),)) @@ -181,9 +237,12 @@ impl Persistence { /// be enriched at runtime with state discovered on pageservers. pub(crate) async fn list_tenant_shards(&self) -> DatabaseResult> { let loaded = self - .with_conn(move |conn| -> DatabaseResult<_> { - Ok(crate::schema::tenant_shards::table.load::(conn)?) - }) + .with_measured_conn( + DatabaseOperation::ListTenantShards, + move |conn| -> DatabaseResult<_> { + Ok(crate::schema::tenant_shards::table.load::(conn)?) + }, + ) .await?; if loaded.is_empty() { @@ -260,17 +319,20 @@ impl Persistence { shards: Vec, ) -> DatabaseResult<()> { use crate::schema::tenant_shards::dsl::*; - self.with_conn(move |conn| -> DatabaseResult<()> { - conn.transaction(|conn| -> QueryResult<()> { - for tenant in &shards { - diesel::insert_into(tenant_shards) - .values(tenant) - .execute(conn)?; - } + self.with_measured_conn( + DatabaseOperation::InsertTenantShards, + move |conn| -> DatabaseResult<()> { + conn.transaction(|conn| -> QueryResult<()> { + for tenant in &shards { + diesel::insert_into(tenant_shards) + .values(tenant) + .execute(conn)?; + } + Ok(()) + })?; Ok(()) - })?; - Ok(()) - }) + }, + ) .await } @@ -278,25 +340,31 @@ impl Persistence { /// the tenant from memory on this server. pub(crate) async fn delete_tenant(&self, del_tenant_id: TenantId) -> DatabaseResult<()> { use crate::schema::tenant_shards::dsl::*; - self.with_conn(move |conn| -> DatabaseResult<()> { - diesel::delete(tenant_shards) - .filter(tenant_id.eq(del_tenant_id.to_string())) - .execute(conn)?; + self.with_measured_conn( + DatabaseOperation::DeleteTenant, + move |conn| -> DatabaseResult<()> { + diesel::delete(tenant_shards) + .filter(tenant_id.eq(del_tenant_id.to_string())) + .execute(conn)?; - Ok(()) - }) + Ok(()) + }, + ) .await } pub(crate) async fn delete_node(&self, del_node_id: NodeId) -> DatabaseResult<()> { use crate::schema::nodes::dsl::*; - self.with_conn(move |conn| -> DatabaseResult<()> { - diesel::delete(nodes) - .filter(node_id.eq(del_node_id.0 as i64)) - .execute(conn)?; + self.with_measured_conn( + DatabaseOperation::DeleteNode, + move |conn| -> DatabaseResult<()> { + diesel::delete(nodes) + .filter(node_id.eq(del_node_id.0 as i64)) + .execute(conn)?; - Ok(()) - }) + Ok(()) + }, + ) .await } @@ -310,7 +378,7 @@ impl Persistence { ) -> DatabaseResult> { use crate::schema::tenant_shards::dsl::*; let updated = self - .with_conn(move |conn| { + .with_measured_conn(DatabaseOperation::ReAttach, move |conn| { let rows_updated = diesel::update(tenant_shards) .filter(generation_pageserver.eq(node_id.0 as i64)) .set(generation.eq(generation + 1)) @@ -360,7 +428,7 @@ impl Persistence { ) -> anyhow::Result { use crate::schema::tenant_shards::dsl::*; let updated = self - .with_conn(move |conn| { + .with_measured_conn(DatabaseOperation::IncrementGeneration, move |conn| { let updated = diesel::update(tenant_shards) .filter(tenant_id.eq(tenant_shard_id.tenant_id.to_string())) .filter(shard_number.eq(tenant_shard_id.shard_number.0 as i32)) @@ -404,7 +472,7 @@ impl Persistence { ) -> DatabaseResult<()> { use crate::schema::tenant_shards::dsl::*; - self.with_conn(move |conn| { + self.with_measured_conn(DatabaseOperation::UpdateTenantShard, move |conn| { let query = diesel::update(tenant_shards) .filter(tenant_id.eq(tenant_shard_id.tenant_id.to_string())) .filter(shard_number.eq(tenant_shard_id.shard_number.0 as i32)) @@ -445,7 +513,7 @@ impl Persistence { ) -> DatabaseResult<()> { use crate::schema::tenant_shards::dsl::*; - self.with_conn(move |conn| { + self.with_measured_conn(DatabaseOperation::UpdateTenantConfig, move |conn| { diesel::update(tenant_shards) .filter(tenant_id.eq(input_tenant_id.to_string())) .set((config.eq(serde_json::to_string(&input_config).unwrap()),)) @@ -460,7 +528,7 @@ impl Persistence { pub(crate) async fn detach(&self, tenant_shard_id: TenantShardId) -> anyhow::Result<()> { use crate::schema::tenant_shards::dsl::*; - self.with_conn(move |conn| { + self.with_measured_conn(DatabaseOperation::Detach, move |conn| { let updated = diesel::update(tenant_shards) .filter(tenant_id.eq(tenant_shard_id.tenant_id.to_string())) .filter(shard_number.eq(tenant_shard_id.shard_number.0 as i32)) @@ -490,7 +558,7 @@ impl Persistence { parent_to_children: Vec<(TenantShardId, Vec)>, ) -> DatabaseResult<()> { use crate::schema::tenant_shards::dsl::*; - self.with_conn(move |conn| -> DatabaseResult<()> { + self.with_measured_conn(DatabaseOperation::BeginShardSplit, move |conn| -> DatabaseResult<()> { conn.transaction(|conn| -> DatabaseResult<()> { // Mark parent shards as splitting @@ -554,26 +622,29 @@ impl Persistence { old_shard_count: ShardCount, ) -> DatabaseResult<()> { use crate::schema::tenant_shards::dsl::*; - self.with_conn(move |conn| -> DatabaseResult<()> { - conn.transaction(|conn| -> QueryResult<()> { - // Drop parent shards - diesel::delete(tenant_shards) - .filter(tenant_id.eq(split_tenant_id.to_string())) - .filter(shard_count.eq(old_shard_count.literal() as i32)) - .execute(conn)?; + self.with_measured_conn( + DatabaseOperation::CompleteShardSplit, + move |conn| -> DatabaseResult<()> { + conn.transaction(|conn| -> QueryResult<()> { + // Drop parent shards + diesel::delete(tenant_shards) + .filter(tenant_id.eq(split_tenant_id.to_string())) + .filter(shard_count.eq(old_shard_count.literal() as i32)) + .execute(conn)?; - // Clear sharding flag - let updated = diesel::update(tenant_shards) - .filter(tenant_id.eq(split_tenant_id.to_string())) - .set((splitting.eq(0),)) - .execute(conn)?; - debug_assert!(updated > 0); + // Clear sharding flag + let updated = diesel::update(tenant_shards) + .filter(tenant_id.eq(split_tenant_id.to_string())) + .set((splitting.eq(0),)) + .execute(conn)?; + debug_assert!(updated > 0); - Ok(()) - })?; + Ok(()) + })?; - Ok(()) - }) + Ok(()) + }, + ) .await } @@ -585,40 +656,44 @@ impl Persistence { new_shard_count: ShardCount, ) -> DatabaseResult { use crate::schema::tenant_shards::dsl::*; - self.with_conn(move |conn| -> DatabaseResult { - let aborted = conn.transaction(|conn| -> DatabaseResult { - // Clear the splitting state on parent shards - let updated = diesel::update(tenant_shards) - .filter(tenant_id.eq(split_tenant_id.to_string())) - .filter(shard_count.ne(new_shard_count.literal() as i32)) - .set((splitting.eq(0),)) - .execute(conn)?; + self.with_measured_conn( + DatabaseOperation::AbortShardSplit, + move |conn| -> DatabaseResult { + let aborted = + conn.transaction(|conn| -> DatabaseResult { + // Clear the splitting state on parent shards + let updated = diesel::update(tenant_shards) + .filter(tenant_id.eq(split_tenant_id.to_string())) + .filter(shard_count.ne(new_shard_count.literal() as i32)) + .set((splitting.eq(0),)) + .execute(conn)?; - // Parent shards are already gone: we cannot abort. - if updated == 0 { - return Ok(AbortShardSplitStatus::Complete); - } + // Parent shards are already gone: we cannot abort. + if updated == 0 { + return Ok(AbortShardSplitStatus::Complete); + } - // Sanity check: if parent shards were present, their cardinality should - // be less than the number of child shards. - if updated >= new_shard_count.count() as usize { - return Err(DatabaseError::Logical(format!( - "Unexpected parent shard count {updated} while aborting split to \ + // Sanity check: if parent shards were present, their cardinality should + // be less than the number of child shards. + if updated >= new_shard_count.count() as usize { + return Err(DatabaseError::Logical(format!( + "Unexpected parent shard count {updated} while aborting split to \ count {new_shard_count:?} on tenant {split_tenant_id}" - ))); - } + ))); + } - // Erase child shards - diesel::delete(tenant_shards) - .filter(tenant_id.eq(split_tenant_id.to_string())) - .filter(shard_count.eq(new_shard_count.literal() as i32)) - .execute(conn)?; + // Erase child shards + diesel::delete(tenant_shards) + .filter(tenant_id.eq(split_tenant_id.to_string())) + .filter(shard_count.eq(new_shard_count.literal() as i32)) + .execute(conn)?; - Ok(AbortShardSplitStatus::Aborted) - })?; + Ok(AbortShardSplitStatus::Aborted) + })?; - Ok(aborted) - }) + Ok(aborted) + }, + ) .await } } diff --git a/control_plane/attachment_service/src/reconciler.rs b/control_plane/attachment_service/src/reconciler.rs index f00f35c74bc2..32d2cb2643bc 100644 --- a/control_plane/attachment_service/src/reconciler.rs +++ b/control_plane/attachment_service/src/reconciler.rs @@ -1,3 +1,4 @@ +use crate::pageserver_client::PageserverClient; use crate::persistence::Persistence; use crate::service; use hyper::StatusCode; @@ -243,8 +244,11 @@ impl Reconciler { tenant_shard_id: TenantShardId, node: &Node, ) -> anyhow::Result> { - let client = - mgmt_api::Client::new(node.base_url(), self.service_config.jwt_token.as_deref()); + let client = PageserverClient::new( + node.get_id(), + node.base_url(), + self.service_config.jwt_token.as_deref(), + ); let timelines = client.timeline_list(&tenant_shard_id).await?; Ok(timelines diff --git a/control_plane/attachment_service/src/service.rs b/control_plane/attachment_service/src/service.rs index e38007c7afdd..98377cace672 100644 --- a/control_plane/attachment_service/src/service.rs +++ b/control_plane/attachment_service/src/service.rs @@ -27,6 +27,7 @@ use pageserver_api::{ models::{SecondaryProgress, TenantConfigRequest}, }; +use crate::pageserver_client::PageserverClient; use pageserver_api::{ models::{ self, LocationConfig, LocationConfigListResponse, LocationConfigMode, @@ -551,7 +552,11 @@ impl Service { break; } - let client = mgmt_api::Client::new(node.base_url(), self.config.jwt_token.as_deref()); + let client = PageserverClient::new( + node.get_id(), + node.base_url(), + self.config.jwt_token.as_deref(), + ); match client .location_config( tenant_shard_id, @@ -2096,8 +2101,11 @@ impl Service { }) .collect::>(); for tenant_shard_id in shard_ids { - let client = - mgmt_api::Client::new(node.base_url(), self.config.jwt_token.as_deref()); + let client = PageserverClient::new( + node.get_id(), + node.base_url(), + self.config.jwt_token.as_deref(), + ); tracing::info!("Doing time travel recovery for shard {tenant_shard_id}",); @@ -2149,7 +2157,11 @@ impl Service { // Issue concurrent requests to all shards' locations let mut futs = FuturesUnordered::new(); for (tenant_shard_id, node) in targets { - let client = mgmt_api::Client::new(node.base_url(), self.config.jwt_token.as_deref()); + let client = PageserverClient::new( + node.get_id(), + node.base_url(), + self.config.jwt_token.as_deref(), + ); futs.push(async move { let result = client .tenant_secondary_download(tenant_shard_id, wait) @@ -2242,7 +2254,11 @@ impl Service { // Phase 1: delete on the pageservers let mut any_pending = false; for (tenant_shard_id, node) in targets { - let client = mgmt_api::Client::new(node.base_url(), self.config.jwt_token.as_deref()); + let client = PageserverClient::new( + node.get_id(), + node.base_url(), + self.config.jwt_token.as_deref(), + ); // TODO: this, like many other places, requires proper retry handling for 503, timeout: those should not // surface immediately as an error to our caller. let status = client.tenant_delete(tenant_shard_id).await.map_err(|e| { @@ -2354,7 +2370,7 @@ impl Service { tenant_shard_id, create_req.new_timeline_id, ); - let client = mgmt_api::Client::new(node.base_url(), jwt.as_deref()); + let client = PageserverClient::new(node.get_id(), node.base_url(), jwt.as_deref()); client .timeline_create(tenant_shard_id, &create_req) @@ -2478,7 +2494,7 @@ impl Service { "Deleting timeline on shard {tenant_shard_id}/{timeline_id}, attached to node {node}", ); - let client = mgmt_api::Client::new(node.base_url(), jwt.as_deref()); + let client = PageserverClient::new(node.get_id(), node.base_url(), jwt.as_deref()); client .timeline_delete(tenant_shard_id, timeline_id) .await @@ -2519,11 +2535,11 @@ impl Service { } /// When you need to send an HTTP request to the pageserver that holds shard0 of a tenant, this - /// function looks it up and returns the url. If the tenant isn't found, returns Err(ApiError::NotFound) - pub(crate) fn tenant_shard0_baseurl( + /// function looks up and returns node. If the tenant isn't found, returns Err(ApiError::NotFound) + pub(crate) fn tenant_shard0_node( &self, tenant_id: TenantId, - ) -> Result<(String, TenantShardId), ApiError> { + ) -> Result<(Node, TenantShardId), ApiError> { let locked = self.inner.read().unwrap(); let Some((tenant_shard_id, shard)) = locked .tenants @@ -2555,7 +2571,7 @@ impl Service { ))); }; - Ok((node.base_url(), *tenant_shard_id)) + Ok((node.clone(), *tenant_shard_id)) } pub(crate) fn tenant_locate( @@ -3215,7 +3231,11 @@ impl Service { node, child_ids, } = target; - let client = mgmt_api::Client::new(node.base_url(), self.config.jwt_token.as_deref()); + let client = PageserverClient::new( + node.get_id(), + node.base_url(), + self.config.jwt_token.as_deref(), + ); let response = client .tenant_shard_split( *parent_id, diff --git a/control_plane/attachment_service/src/tenant_state.rs b/control_plane/attachment_service/src/tenant_state.rs index 9dd368bf411c..83c921dc58cf 100644 --- a/control_plane/attachment_service/src/tenant_state.rs +++ b/control_plane/attachment_service/src/tenant_state.rs @@ -4,7 +4,10 @@ use std::{ time::Duration, }; -use crate::{metrics, persistence::TenantShardPersistence}; +use crate::{ + metrics::{self, ReconcileCompleteLabelGroup, ReconcileOutcome}, + persistence::TenantShardPersistence, +}; use pageserver_api::controller_api::PlacementPolicy; use pageserver_api::{ models::{LocationConfig, LocationConfigMode, TenantConfig}, @@ -718,7 +721,10 @@ impl TenantState { let reconciler_span = tracing::info_span!(parent: None, "reconciler", seq=%reconcile_seq, tenant_id=%reconciler.tenant_shard_id.tenant_id, shard_id=%reconciler.tenant_shard_id.shard_slug()); - metrics::RECONCILER.spawned.inc(); + metrics::METRICS_REGISTRY + .metrics_group + .storage_controller_reconcile_spawn + .inc(); let result_tx = result_tx.clone(); let join_handle = tokio::task::spawn( async move { @@ -736,10 +742,12 @@ impl TenantState { // TODO: wrap all remote API operations in cancellation check // as well. if reconciler.cancel.is_cancelled() { - metrics::RECONCILER - .complete - .with_label_values(&[metrics::ReconcilerMetrics::CANCEL]) - .inc(); + metrics::METRICS_REGISTRY + .metrics_group + .storage_controller_reconcile_complete + .inc(ReconcileCompleteLabelGroup { + status: ReconcileOutcome::Cancel, + }); return; } @@ -754,18 +762,18 @@ impl TenantState { } // Update result counter - match &result { - Ok(_) => metrics::RECONCILER - .complete - .with_label_values(&[metrics::ReconcilerMetrics::SUCCESS]), - Err(ReconcileError::Cancel) => metrics::RECONCILER - .complete - .with_label_values(&[metrics::ReconcilerMetrics::CANCEL]), - Err(_) => metrics::RECONCILER - .complete - .with_label_values(&[metrics::ReconcilerMetrics::ERROR]), - } - .inc(); + let outcome_label = match &result { + Ok(_) => ReconcileOutcome::Success, + Err(ReconcileError::Cancel) => ReconcileOutcome::Cancel, + Err(_) => ReconcileOutcome::Error, + }; + + metrics::METRICS_REGISTRY + .metrics_group + .storage_controller_reconcile_complete + .inc(ReconcileCompleteLabelGroup { + status: outcome_label, + }); result_tx .send(ReconcileResult { diff --git a/libs/utils/src/http/endpoint.rs b/libs/utils/src/http/endpoint.rs index a60971abf0ad..f8a5f681315a 100644 --- a/libs/utils/src/http/endpoint.rs +++ b/libs/utils/src/http/endpoint.rs @@ -245,7 +245,7 @@ impl std::io::Write for ChannelWriter { } } -async fn prometheus_metrics_handler(_req: Request) -> Result, ApiError> { +pub async fn prometheus_metrics_handler(_req: Request) -> Result, ApiError> { SERVE_METRICS_COUNT.inc(); let started_at = std::time::Instant::now(); @@ -367,7 +367,6 @@ pub fn make_router() -> RouterBuilder { .middleware(Middleware::post_with_info( add_request_id_header_to_response, )) - .get("/metrics", |r| request_span(r, prometheus_metrics_handler)) .err_handler(route_error_handler) } diff --git a/pageserver/src/http/routes.rs b/pageserver/src/http/routes.rs index 229f3ae98f48..26f23fb8c2cb 100644 --- a/pageserver/src/http/routes.rs +++ b/pageserver/src/http/routes.rs @@ -36,6 +36,7 @@ use tokio_util::sync::CancellationToken; use tracing::*; use utils::auth::JwtAuth; use utils::failpoint_support::failpoints_handler; +use utils::http::endpoint::prometheus_metrics_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}; @@ -2266,6 +2267,7 @@ pub fn make_router( Ok(router .data(state) + .get("/metrics", |r| request_span(r, prometheus_metrics_handler)) .get("/v1/status", |r| api_handler(r, status_handler)) .put("/v1/failpoints", |r| { testing_api_handler("manage failpoints", r, failpoints_handler) diff --git a/proxy/src/http/health_server.rs b/proxy/src/http/health_server.rs index 6186ddde0d49..cbb17ebcb756 100644 --- a/proxy/src/http/health_server.rs +++ b/proxy/src/http/health_server.rs @@ -2,14 +2,21 @@ use anyhow::{anyhow, bail}; use hyper::{Body, Request, Response, StatusCode}; use std::{convert::Infallible, net::TcpListener}; use tracing::info; -use utils::http::{endpoint, error::ApiError, json::json_response, RouterBuilder, RouterService}; +use utils::http::{ + endpoint::{self, prometheus_metrics_handler, request_span}, + error::ApiError, + json::json_response, + RouterBuilder, RouterService, +}; async fn status_handler(_: Request) -> Result, ApiError> { json_response(StatusCode::OK, "") } fn make_router() -> RouterBuilder { - endpoint::make_router().get("/v1/status", status_handler) + endpoint::make_router() + .get("/metrics", |r| request_span(r, prometheus_metrics_handler)) + .get("/v1/status", status_handler) } pub async fn task_main(http_listener: TcpListener) -> anyhow::Result { diff --git a/safekeeper/src/http/routes.rs b/safekeeper/src/http/routes.rs index a0c0c7ca4c04..9ce26e6c5d04 100644 --- a/safekeeper/src/http/routes.rs +++ b/safekeeper/src/http/routes.rs @@ -20,7 +20,7 @@ use std::io::Write as _; use tokio::sync::mpsc; use tokio_stream::wrappers::ReceiverStream; use tracing::{info_span, Instrument}; -use utils::http::endpoint::{request_span, ChannelWriter}; +use utils::http::endpoint::{prometheus_metrics_handler, request_span, ChannelWriter}; use crate::debug_dump::TimelineDigestRequest; use crate::receive_wal::WalReceiverState; @@ -515,6 +515,7 @@ pub fn make_router(conf: SafeKeeperConf) -> RouterBuilder router .data(Arc::new(conf)) .data(auth) + .get("/metrics", |r| request_span(r, prometheus_metrics_handler)) .get("/v1/status", |r| request_span(r, status_handler)) .put("/v1/failpoints", |r| { request_span(r, move |r| async { diff --git a/test_runner/regress/test_sharding.py b/test_runner/regress/test_sharding.py index cb58c640c38d..57b2b2b0a178 100644 --- a/test_runner/regress/test_sharding.py +++ b/test_runner/regress/test_sharding.py @@ -278,18 +278,14 @@ def shards_on_disk(shard_ids): # Check that no cancelled or errored reconciliations occurred: this test does no # failure injection and should run clean. - assert ( - env.storage_controller.get_metric_value( - "storage_controller_reconcile_complete_total", filter={"status": "cancel"} - ) - is None + cancelled_reconciles = env.storage_controller.get_metric_value( + "storage_controller_reconcile_complete_total", filter={"status": "cancel"} ) - assert ( - env.storage_controller.get_metric_value( - "storage_controller_reconcile_complete_total", filter={"status": "error"} - ) - is None + errored_reconciles = env.storage_controller.get_metric_value( + "storage_controller_reconcile_complete_total", filter={"status": "error"} ) + assert cancelled_reconciles is not None and int(cancelled_reconciles) == 0 + assert errored_reconciles is not None and int(errored_reconciles) == 0 env.storage_controller.consistency_check() From 59cdee749edcfde5e57bc1eeea7df25b6a0af485 Mon Sep 17 00:00:00 2001 From: John Spray Date: Thu, 21 Mar 2024 12:06:57 +0000 Subject: [PATCH 20/53] storage controller: fixes to secondary location handling (#7169) Stacks on: - https://github.com/neondatabase/neon/pull/7165 Fixes while working on background optimization of scheduling after a split: - When a tenant has secondary locations, we weren't detaching the parent shards' secondary locations when doing a split - When a reconciler detaches a location, it was feeding back a locationconf with `Detached` mode in its `observed` object, whereas it should omit that location. This could cause the background reconcile task to keep kicking off no-op reconcilers forever (harmless but annoying). - During shard split, we were scheduling secondary locations for the child shards, but no reconcile was run for these until the next time the background reconcile task ran. Creating these ASAP is useful, because they'll be used shortly after a shard split as the destination locations for migrating the new shards to different nodes. --- .../attachment_service/src/reconciler.rs | 22 ++- .../attachment_service/src/service.rs | 70 ++++++++- pageserver/src/tenant/secondary/downloader.rs | 6 +- test_runner/fixtures/neon_fixtures.py | 12 ++ test_runner/regress/test_sharding.py | 139 +++++++++++++----- 5 files changed, 203 insertions(+), 46 deletions(-) diff --git a/control_plane/attachment_service/src/reconciler.rs b/control_plane/attachment_service/src/reconciler.rs index 32d2cb2643bc..a62357f9acc5 100644 --- a/control_plane/attachment_service/src/reconciler.rs +++ b/control_plane/attachment_service/src/reconciler.rs @@ -118,6 +118,15 @@ impl Reconciler { flush_ms: Option, lazy: bool, ) -> Result<(), ReconcileError> { + if !node.is_available() && config.mode == LocationConfigMode::Detached { + // Attempts to detach from offline nodes may be imitated without doing I/O: a node which is offline + // will get fully reconciled wrt the shard's intent state when it is reactivated, irrespective of + // what we put into `observed`, in [`crate::service::Service::node_activate_reconcile`] + tracing::info!("Node {node} is unavailable during detach: proceeding anyway, it will be detached on next activation"); + self.observed.locations.remove(&node.get_id()); + return Ok(()); + } + self.observed .locations .insert(node.get_id(), ObservedStateLocation { conf: None }); @@ -150,9 +159,16 @@ impl Reconciler { }; tracing::info!("location_config({node}) complete: {:?}", config); - self.observed - .locations - .insert(node.get_id(), ObservedStateLocation { conf: Some(config) }); + match config.mode { + LocationConfigMode::Detached => { + self.observed.locations.remove(&node.get_id()); + } + _ => { + self.observed + .locations + .insert(node.get_id(), ObservedStateLocation { conf: Some(config) }); + } + } Ok(()) } diff --git a/control_plane/attachment_service/src/service.rs b/control_plane/attachment_service/src/service.rs index 98377cace672..c886afaf1c6a 100644 --- a/control_plane/attachment_service/src/service.rs +++ b/control_plane/attachment_service/src/service.rs @@ -210,6 +210,7 @@ struct ShardSplitParams { new_stripe_size: Option, targets: Vec, policy: PlacementPolicy, + config: TenantConfig, shard_ident: ShardIdentity, } @@ -2741,7 +2742,7 @@ impl Service { let detach_locations: Vec<(Node, TenantShardId)> = { let mut detach_locations = Vec::new(); let mut locked = self.inner.write().unwrap(); - let (nodes, tenants, _scheduler) = locked.parts_mut(); + let (nodes, tenants, scheduler) = locked.parts_mut(); for (tenant_shard_id, shard) in tenants.range_mut(TenantShardId::tenant_range(op.tenant_id)) @@ -2774,6 +2775,13 @@ impl Service { tracing::info!("Restoring parent shard {tenant_shard_id}"); shard.splitting = SplitState::Idle; + if let Err(e) = shard.schedule(scheduler) { + // If this shard can't be scheduled now (perhaps due to offline nodes or + // capacity issues), that must not prevent us rolling back a split. In this + // case it should be eventually scheduled in the background. + tracing::warn!("Failed to schedule {tenant_shard_id} during shard abort: {e}") + } + self.maybe_reconcile_shard(shard, nodes); } @@ -2865,7 +2873,7 @@ impl Service { .map(|(shard_id, _)| *shard_id) .collect::>(); - let (_nodes, tenants, scheduler) = locked.parts_mut(); + let (nodes, tenants, scheduler) = locked.parts_mut(); for parent_id in parent_ids { let child_ids = parent_id.split(new_shard_count); @@ -2932,6 +2940,8 @@ impl Service { // find a secondary (e.g. because cluster is overloaded). tracing::warn!("Failed to schedule child shard {child}: {e}"); } + // In the background, attach secondary locations for the new shards + self.maybe_reconcile_shard(&mut child_state, nodes); tenants.insert(child, child_state); response.new_shards.push(child); @@ -2996,6 +3006,7 @@ impl Service { ))); let mut policy = None; + let mut config = None; let mut shard_ident = None; // Validate input, and calculate which shards we will create let (old_shard_count, targets) = @@ -3052,6 +3063,9 @@ impl Service { if shard_ident.is_none() { shard_ident = Some(shard.shard); } + if config.is_none() { + config = Some(shard.config.clone()); + } if tenant_shard_id.shard_count.count() == split_req.new_shard_count { tracing::info!( @@ -3070,8 +3084,6 @@ impl Service { .get(&node_id) .expect("Pageservers may not be deleted while referenced"); - // TODO: if any reconciliation is currently in progress for this shard, wait for it. - targets.push(ShardSplitTarget { parent_id: *tenant_shard_id, node: node.clone(), @@ -3114,6 +3126,7 @@ impl Service { shard_ident.unwrap() }; let policy = policy.unwrap(); + let config = config.unwrap(); Ok(ShardSplitAction::Split(ShardSplitParams { old_shard_count, @@ -3121,6 +3134,7 @@ impl Service { new_stripe_size: split_req.new_stripe_size, targets, policy, + config, shard_ident, })) } @@ -3140,11 +3154,49 @@ impl Service { old_shard_count, new_shard_count, new_stripe_size, - targets, + mut targets, policy, + config, shard_ident, } = params; + // Drop any secondary locations: pageservers do not support splitting these, and in any case the + // end-state for a split tenant will usually be to have secondary locations on different nodes. + // The reconciliation calls in this block also implicitly cancel+barrier wrt any ongoing reconciliation + // at the time of split. + let waiters = { + let mut locked = self.inner.write().unwrap(); + let mut waiters = Vec::new(); + let (nodes, tenants, scheduler) = locked.parts_mut(); + for target in &mut targets { + let Some(shard) = tenants.get_mut(&target.parent_id) else { + // Paranoia check: this shouldn't happen: we have the oplock for this tenant ID. + return Err(ApiError::InternalServerError(anyhow::anyhow!( + "Shard {} not found", + target.parent_id + ))); + }; + + if shard.intent.get_attached() != &Some(target.node.get_id()) { + // Paranoia check: this shouldn't happen: we have the oplock for this tenant ID. + return Err(ApiError::Conflict(format!( + "Shard {} unexpectedly rescheduled during split", + target.parent_id + ))); + } + + // Irrespective of PlacementPolicy, clear secondary locations from intent + shard.intent.clear_secondary(scheduler); + + // Run Reconciler to execute detach fo secondary locations. + if let Some(waiter) = self.maybe_reconcile_shard(shard, nodes) { + waiters.push(waiter); + } + } + waiters + }; + self.await_waiters(waiters, RECONCILE_TIMEOUT).await?; + // Before creating any new child shards in memory or on the pageservers, persist them: this // enables us to ensure that we will always be able to clean up if something goes wrong. This also // acts as the protection against two concurrent attempts to split: one of them will get a database @@ -3173,8 +3225,7 @@ impl Service { generation: None, generation_pageserver: Some(target.node.get_id().0 as i64), placement_policy: serde_json::to_string(&policy).unwrap(), - // TODO: get the config out of the map - config: serde_json::to_string(&TenantConfig::default()).unwrap(), + config: serde_json::to_string(&config).unwrap(), splitting: SplitState::Splitting, }); } @@ -3363,6 +3414,11 @@ impl Service { // If we were already attached to something, demote that to a secondary if let Some(old_attached) = old_attached { if n > 0 { + // Remove other secondaries to make room for the location we'll demote + while shard.intent.get_secondary().len() >= n { + shard.intent.pop_secondary(scheduler); + } + shard.intent.push_secondary(scheduler, old_attached); } } diff --git a/pageserver/src/tenant/secondary/downloader.rs b/pageserver/src/tenant/secondary/downloader.rs index 82af7ed83b84..40f19e3b0551 100644 --- a/pageserver/src/tenant/secondary/downloader.rs +++ b/pageserver/src/tenant/secondary/downloader.rs @@ -11,6 +11,7 @@ use crate::{ disk_usage_eviction_task::{ finite_f32, DiskUsageEvictionInfo, EvictionCandidate, EvictionLayer, EvictionSecondaryLayer, }, + is_temporary, metrics::SECONDARY_MODE, tenant::{ config::SecondaryLocationConfig, @@ -961,7 +962,10 @@ async fn init_timeline_state( // Secondary mode doesn't use local metadata files, but they might have been left behind by an attached tenant. warn!(path=?dentry.path(), "found legacy metadata file, these should have been removed in load_tenant_config"); continue; - } else if crate::is_temporary(&file_path) || is_temp_download_file(&file_path) { + } else if crate::is_temporary(&file_path) + || is_temp_download_file(&file_path) + || is_temporary(&file_path) + { // Temporary files are frequently left behind from restarting during downloads tracing::info!("Cleaning up temporary file {file_path}"); if let Err(e) = tokio::fs::remove_file(&file_path) diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index 1d30c4527830..f8994a8dccb3 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -2150,6 +2150,18 @@ def locate(self, tenant_id: TenantId) -> list[dict[str, Any]]: shards: list[dict[str, Any]] = body["shards"] return shards + def tenant_describe(self, tenant_id: TenantId): + """ + :return: list of {"shard_id": "", "node_id": int, "listen_pg_addr": str, "listen_pg_port": int, "listen_http_addr: str, "listen_http_port: int} + """ + response = self.request( + "GET", + f"{self.env.storage_controller_api}/control/v1/tenant/{tenant_id}", + headers=self.headers(TokenScope.ADMIN), + ) + response.raise_for_status() + return response.json() + def tenant_shard_split( self, tenant_id: TenantId, shard_count: int, shard_stripe_size: Optional[int] = None ) -> list[TenantShardId]: diff --git a/test_runner/regress/test_sharding.py b/test_runner/regress/test_sharding.py index 57b2b2b0a178..e6318aff6889 100644 --- a/test_runner/regress/test_sharding.py +++ b/test_runner/regress/test_sharding.py @@ -1,5 +1,6 @@ import os import time +from collections import defaultdict from typing import Dict, List, Optional, Union import pytest @@ -13,7 +14,7 @@ tenant_get_shards, ) from fixtures.remote_storage import s3_storage -from fixtures.types import Lsn, TenantShardId, TimelineId +from fixtures.types import Lsn, TenantId, TenantShardId, TimelineId from fixtures.utils import wait_until from fixtures.workload import Workload from pytest_httpserver import HTTPServer @@ -159,11 +160,20 @@ def test_sharding_split_smoke( neon_env_builder.preserve_database_files = True - env = neon_env_builder.init_start( - initial_tenant_shard_count=shard_count, initial_tenant_shard_stripe_size=stripe_size + non_default_tenant_config = {"gc_horizon": 77 * 1024 * 1024} + + env = neon_env_builder.init_configs(True) + neon_env_builder.start() + tenant_id = TenantId.generate() + timeline_id = TimelineId.generate() + env.neon_cli.create_tenant( + tenant_id, + timeline_id, + shard_count=shard_count, + shard_stripe_size=stripe_size, + placement_policy='{"Attached": 1}', + conf=non_default_tenant_config, ) - tenant_id = env.initial_tenant - timeline_id = env.initial_timeline workload = Workload(env, tenant_id, timeline_id, branch_name="main") workload.init() @@ -223,6 +233,14 @@ def shards_on_disk(shard_ids): # Before split, old shards exist assert shards_on_disk(old_shard_ids) + # Before split, we have done one reconcile for each shard + assert ( + env.storage_controller.get_metric_value( + "storage_controller_reconcile_complete_total", filter={"status": "ok"} + ) + == shard_count + ) + env.storage_controller.tenant_shard_split(tenant_id, shard_count=split_shard_count) post_split_pageserver_ids = [loc["node_id"] for loc in env.storage_controller.locate(tenant_id)] @@ -268,13 +286,20 @@ def shards_on_disk(shard_ids): workload.validate() - # Check that we didn't do any spurious reconciliations. - # Total number of reconciles should have been one per original shard, plus - # one for each shard that was migrated. + # Assert on how many reconciles happened during the process. This is something of an + # implementation detail, but it is useful to detect any bugs that might generate spurious + # extra reconcile iterations. + # + # We'll have: + # - shard_count reconciles for the original setup of the tenant + # - shard_count reconciles for detaching the original secondary locations during split + # - split_shard_count reconciles during shard splitting, for setting up secondaries. + # - shard_count reconciles for the migrations we did to move child shards away from their split location + expect_reconciles = shard_count * 2 + split_shard_count + shard_count reconcile_ok = env.storage_controller.get_metric_value( "storage_controller_reconcile_complete_total", filter={"status": "ok"} ) - assert reconcile_ok == shard_count + split_shard_count // 2 + assert reconcile_ok == expect_reconciles # Check that no cancelled or errored reconciliations occurred: this test does no # failure injection and should run clean. @@ -289,14 +314,34 @@ def shards_on_disk(shard_ids): env.storage_controller.consistency_check() - # Validate pageserver state - shards_exist: list[TenantShardId] = [] - for pageserver in env.pageservers: - locations = pageserver.http_client().tenant_list_locations() - shards_exist.extend(TenantShardId.parse(s[0]) for s in locations["tenant_shards"]) - - log.info(f"Shards after split: {shards_exist}") - assert len(shards_exist) == split_shard_count + def get_node_shard_counts(env: NeonEnv, tenant_ids): + total: defaultdict[int, int] = defaultdict(int) + attached: defaultdict[int, int] = defaultdict(int) + for tid in tenant_ids: + for shard in env.storage_controller.tenant_describe(tid)["shards"]: + log.info( + f"{shard['tenant_shard_id']}: attached={shard['node_attached']}, secondary={shard['node_secondary']} " + ) + for node in shard["node_secondary"]: + total[int(node)] += 1 + attached[int(shard["node_attached"])] += 1 + total[int(shard["node_attached"])] += 1 + + return total, attached + + def check_effective_tenant_config(): + # Expect our custom tenant configs to have survived the split + for shard in env.storage_controller.tenant_describe(tenant_id)["shards"]: + node = env.get_pageserver(int(shard["node_attached"])) + config = node.http_client().tenant_config(TenantShardId.parse(shard["tenant_shard_id"])) + for k, v in non_default_tenant_config.items(): + assert config.effective_config[k] == v + + # Validate pageserver state: expect every child shard to have an attached and secondary location + (total, attached) = get_node_shard_counts(env, tenant_ids=[tenant_id]) + assert sum(attached.values()) == split_shard_count + assert sum(total.values()) == split_shard_count * 2 + check_effective_tenant_config() # Ensure post-split pageserver locations survive a restart (i.e. the child shards # correctly wrote config to disk, and the storage controller responds correctly @@ -305,13 +350,11 @@ def shards_on_disk(shard_ids): pageserver.stop() pageserver.start() - shards_exist = [] - for pageserver in env.pageservers: - locations = pageserver.http_client().tenant_list_locations() - shards_exist.extend(TenantShardId.parse(s[0]) for s in locations["tenant_shards"]) - - log.info("Shards after restart: {shards_exist}") - assert len(shards_exist) == split_shard_count + # Validate pageserver state: expect every child shard to have an attached and secondary location + (total, attached) = get_node_shard_counts(env, tenant_ids=[tenant_id]) + assert sum(attached.values()) == split_shard_count + assert sum(total.values()) == split_shard_count * 2 + check_effective_tenant_config() workload.validate() @@ -717,9 +760,16 @@ def test_sharding_split_failures( initial_shard_count = 2 split_shard_count = 4 - env = neon_env_builder.init_start(initial_tenant_shard_count=initial_shard_count) - tenant_id = env.initial_tenant - timeline_id = env.initial_timeline + env = neon_env_builder.init_configs() + env.start() + + tenant_id = TenantId.generate() + timeline_id = TimelineId.generate() + + # Create a tenant with secondary locations enabled + env.neon_cli.create_tenant( + tenant_id, timeline_id, shard_count=initial_shard_count, placement_policy='{"Attached":1}' + ) env.storage_controller.allowed_errors.extend( [ @@ -732,6 +782,8 @@ def test_sharding_split_failures( ".*failpoint.*", # Node offline cases will fail to send requests ".*Reconcile error: receive body: error sending request for url.*", + # Node offline cases will fail inside reconciler when detaching secondaries + ".*Reconcile error on shard.*: receive body: error sending request for url.*", ] ) @@ -769,7 +821,8 @@ def test_sharding_split_failures( # will have succeeded: the net result should be to return to a clean state, including # detaching any child shards. def assert_rolled_back(exclude_ps_id=None) -> None: - count = 0 + secondary_count = 0 + attached_count = 0 for ps in env.pageservers: if exclude_ps_id is not None and ps.id == exclude_ps_id: continue @@ -777,13 +830,25 @@ def assert_rolled_back(exclude_ps_id=None) -> None: locations = ps.http_client().tenant_list_locations()["tenant_shards"] for loc in locations: tenant_shard_id = TenantShardId.parse(loc[0]) - log.info(f"Shard {tenant_shard_id} seen on node {ps.id}") + log.info(f"Shard {tenant_shard_id} seen on node {ps.id} in mode {loc[1]['mode']}") assert tenant_shard_id.shard_count == initial_shard_count - count += 1 - assert count == initial_shard_count + if loc[1]["mode"] == "Secondary": + secondary_count += 1 + else: + attached_count += 1 + + if exclude_ps_id is not None: + # For a node failure case, we expect there to be a secondary location + # scheduled on the offline node, so expect one fewer secondary in total + assert secondary_count == initial_shard_count - 1 + else: + assert secondary_count == initial_shard_count + + assert attached_count == initial_shard_count def assert_split_done(exclude_ps_id=None) -> None: - count = 0 + secondary_count = 0 + attached_count = 0 for ps in env.pageservers: if exclude_ps_id is not None and ps.id == exclude_ps_id: continue @@ -791,10 +856,14 @@ def assert_split_done(exclude_ps_id=None) -> None: locations = ps.http_client().tenant_list_locations()["tenant_shards"] for loc in locations: tenant_shard_id = TenantShardId.parse(loc[0]) - log.info(f"Shard {tenant_shard_id} seen on node {ps.id}") + log.info(f"Shard {tenant_shard_id} seen on node {ps.id} in mode {loc[1]['mode']}") assert tenant_shard_id.shard_count == split_shard_count - count += 1 - assert count == split_shard_count + if loc[1]["mode"] == "Secondary": + secondary_count += 1 + else: + attached_count += 1 + assert attached_count == split_shard_count + assert secondary_count == split_shard_count def finish_split(): # Having failed+rolled back, we should be able to split again From bb47d536fb6e79865d9876f7ed7a46fa57e988a2 Mon Sep 17 00:00:00 2001 From: John Spray Date: Thu, 21 Mar 2024 12:56:13 +0000 Subject: [PATCH 21/53] pageserver: quieten log on shutdown-while-attaching (#7177) ## Problem If a shutdown happens when a tenant is attaching, we were logging at ERROR severity and with a backtrace. Yuck. ## Summary of changes - Pass a flag into `make_broken` to enable quietening this non-scary case. --- pageserver/src/tenant.rs | 25 ++++++++++++++++++------- 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index 7a6ddd6a4edc..1c66f99ece9a 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -678,9 +678,20 @@ impl Tenant { } // Ideally we should use Tenant::set_broken_no_wait, but it is not supposed to be used when tenant is in loading state. + enum BrokenVerbosity { + Error, + Info + } let make_broken = - |t: &Tenant, err: anyhow::Error| { - error!("attach failed, setting tenant state to Broken: {err:?}"); + |t: &Tenant, err: anyhow::Error, verbosity: BrokenVerbosity| { + match verbosity { + BrokenVerbosity::Info => { + info!("attach cancelled, setting tenant state to Broken: {err}"); + }, + BrokenVerbosity::Error => { + error!("attach failed, setting tenant state to Broken: {err:?}"); + } + } t.state.send_modify(|state| { // The Stopping case is for when we have passed control on to DeleteTenantFlow: // if it errors, we will call make_broken when tenant is already in Stopping. @@ -744,7 +755,7 @@ impl Tenant { // Make the tenant broken so that set_stopping will not hang waiting for it to leave // the Attaching state. This is an over-reaction (nothing really broke, the tenant is // just shutting down), but ensures progress. - make_broken(&tenant_clone, anyhow::anyhow!("Shut down while Attaching")); + make_broken(&tenant_clone, anyhow::anyhow!("Shut down while Attaching"), BrokenVerbosity::Info); return Ok(()); }, ) @@ -766,7 +777,7 @@ impl Tenant { match res { Ok(p) => Some(p), Err(e) => { - make_broken(&tenant_clone, anyhow::anyhow!(e)); + make_broken(&tenant_clone, anyhow::anyhow!(e), BrokenVerbosity::Error); return Ok(()); } } @@ -790,7 +801,7 @@ impl Tenant { { Ok(should_resume_deletion) => should_resume_deletion, Err(err) => { - make_broken(&tenant_clone, anyhow::anyhow!(err)); + make_broken(&tenant_clone, anyhow::anyhow!(err), BrokenVerbosity::Error); return Ok(()); } } @@ -820,7 +831,7 @@ impl Tenant { .await; if let Err(e) = deleted { - make_broken(&tenant_clone, anyhow::anyhow!(e)); + make_broken(&tenant_clone, anyhow::anyhow!(e), BrokenVerbosity::Error); } return Ok(()); @@ -841,7 +852,7 @@ impl Tenant { tenant_clone.activate(broker_client, None, &ctx); } Err(e) => { - make_broken(&tenant_clone, anyhow::anyhow!(e)); + make_broken(&tenant_clone, anyhow::anyhow!(e), BrokenVerbosity::Error); } } From 06cb582d910f4949bcb3927ec40a7cba7a306ff3 Mon Sep 17 00:00:00 2001 From: John Spray Date: Thu, 21 Mar 2024 13:39:23 +0000 Subject: [PATCH 22/53] pageserver: extend /re-attach response to include tenant mode (#6941) This change improves the resilience of the system to unclean restarts. Previously, re-attach responses only included attached tenants - If the pageserver had local state for a secondary location, it would remain, but with no guarantee that it was still _meant_ to be there. After this change, the pageserver will only retain secondary locations if the /re-attach response indicates that they should still be there. - If the pageserver had local state for an attached location that was omitted from a re-attach response, it would be entirely detached. This is wasteful in a typical HA setup, where an offline node's tenants might have been re-attached elsewhere before it restarts, but the offline node's location should revert to a secondary location rather than being wiped. Including secondary tenants in the re-attach response enables the pageserver to avoid throwing away local state unnecessarily. In this PR: - The re-attach items are extended with a 'mode' field. - Storage controller populates 'mode' - Pageserver interprets it (default is attached if missing) to construct either a SecondaryTenant or a Tenant. - A new test exercises both cases. --- .../attachment_service/src/service.rs | 116 ++++----- libs/pageserver_api/src/upcall_api.rs | 18 +- pageserver/src/control_plane_client.rs | 11 +- pageserver/src/deletion_queue.rs | 7 +- pageserver/src/tenant.rs | 7 + pageserver/src/tenant/config.rs | 5 +- pageserver/src/tenant/mgr.rs | 233 +++++++++++------- test_runner/regress/test_sharding_service.py | 64 ++++- 8 files changed, 305 insertions(+), 156 deletions(-) diff --git a/control_plane/attachment_service/src/service.rs b/control_plane/attachment_service/src/service.rs index c886afaf1c6a..aa930014b280 100644 --- a/control_plane/attachment_service/src/service.rs +++ b/control_plane/attachment_service/src/service.rs @@ -1394,7 +1394,8 @@ impl Service { incremented_generations.len() ); - // Apply the updated generation to our in-memory state + // Apply the updated generation to our in-memory state, and + // gather discover secondary locations. let mut locked = self.inner.write().unwrap(); let (nodes, tenants, scheduler) = locked.parts_mut(); @@ -1402,62 +1403,65 @@ impl Service { tenants: Vec::new(), }; - for (tenant_shard_id, new_gen) in incremented_generations { - response.tenants.push(ReAttachResponseTenant { - id: tenant_shard_id, - gen: new_gen.into().unwrap(), - }); - // Apply the new generation number to our in-memory state - let shard_state = tenants.get_mut(&tenant_shard_id); - let Some(shard_state) = shard_state else { - // Not fatal. This edge case requires a re-attach to happen - // between inserting a new tenant shard in to the database, and updating our in-memory - // state to know about the shard, _and_ that the state inserted to the database referenced - // a pageserver. Should never happen, but handle it rather than panicking, since it should - // be harmless. - tracing::error!( - "Shard {} is in database for node {} but not in-memory state", - tenant_shard_id, - reattach_req.node_id - ); - continue; - }; + // TODO: cancel/restart any running reconciliation for this tenant, it might be trying + // to call location_conf API with an old generation. Wait for cancellation to complete + // before responding to this request. Requires well implemented CancellationToken logic + // all the way to where we call location_conf. Even then, there can still be a location_conf + // request in flight over the network: TODO handle that by making location_conf API refuse + // to go backward in generations. + + // Scan through all shards, applying updates for ones where we updated generation + // and identifying shards that intend to have a secondary location on this node. + for (tenant_shard_id, shard) in tenants { + if let Some(new_gen) = incremented_generations.get(tenant_shard_id) { + let new_gen = *new_gen; + response.tenants.push(ReAttachResponseTenant { + id: *tenant_shard_id, + gen: Some(new_gen.into().unwrap()), + // A tenant is only put into multi or stale modes in the middle of a [`Reconciler::live_migrate`] + // execution. If a pageserver is restarted during that process, then the reconcile pass will + // fail, and start from scratch, so it doesn't make sense for us to try and preserve + // the stale/multi states at this point. + mode: LocationConfigMode::AttachedSingle, + }); - // If [`Persistence::re_attach`] selected this shard, it must have alread - // had a generation set. - debug_assert!(shard_state.generation.is_some()); - let Some(old_gen) = shard_state.generation else { - // Should never happen: would only return incremented generation - // for a tenant that already had a non-null generation. - return Err(ApiError::InternalServerError(anyhow::anyhow!( - "Generation must be set while re-attaching" - ))); - }; - shard_state.generation = Some(std::cmp::max(old_gen, new_gen)); - if let Some(observed) = shard_state - .observed - .locations - .get_mut(&reattach_req.node_id) - { - if let Some(conf) = observed.conf.as_mut() { - conf.generation = new_gen.into(); - } - } else { - // This node has no observed state for the shard: perhaps it was offline - // when the pageserver restarted. Insert a None, so that the Reconciler - // will be prompted to learn the location's state before it makes changes. - shard_state - .observed - .locations - .insert(reattach_req.node_id, ObservedStateLocation { conf: None }); - } + shard.generation = std::cmp::max(shard.generation, Some(new_gen)); + if let Some(observed) = shard.observed.locations.get_mut(&reattach_req.node_id) { + // Why can we update `observed` even though we're not sure our response will be received + // by the pageserver? Because the pageserver will not proceed with startup until + // it has processed response: if it loses it, we'll see another request and increment + // generation again, avoiding any uncertainty about dirtiness of tenant's state. + if let Some(conf) = observed.conf.as_mut() { + conf.generation = new_gen.into(); + } + } else { + // This node has no observed state for the shard: perhaps it was offline + // when the pageserver restarted. Insert a None, so that the Reconciler + // will be prompted to learn the location's state before it makes changes. + shard + .observed + .locations + .insert(reattach_req.node_id, ObservedStateLocation { conf: None }); + } + } else if shard.intent.get_secondary().contains(&reattach_req.node_id) { + // Ordering: pageserver will not accept /location_config requests until it has + // finished processing the response from re-attach. So we can update our in-memory state + // now, and be confident that we are not stamping on the result of some later location config. + // TODO: however, we are not strictly ordered wrt ReconcileResults queue, + // so we might update observed state here, and then get over-written by some racing + // ReconcileResult. The impact is low however, since we have set state on pageserver something + // that matches intent, so worst case if we race then we end up doing a spurious reconcile. + + response.tenants.push(ReAttachResponseTenant { + id: *tenant_shard_id, + gen: None, + mode: LocationConfigMode::Secondary, + }); - // TODO: cancel/restart any running reconciliation for this tenant, it might be trying - // to call location_conf API with an old generation. Wait for cancellation to complete - // before responding to this request. Requires well implemented CancellationToken logic - // all the way to where we call location_conf. Even then, there can still be a location_conf - // request in flight over the network: TODO handle that by making location_conf API refuse - // to go backward in generations. + // We must not update observed, because we have no guarantee that our + // response will be received by the pageserver. This could leave it + // falsely dirty, but the resulting reconcile should be idempotent. + } } // We consider a node Active once we have composed a re-attach response, but we @@ -3446,7 +3450,7 @@ impl Service { if let Some(waiter) = waiter { waiter.wait_timeout(RECONCILE_TIMEOUT).await?; } else { - tracing::warn!("Migration is a no-op"); + tracing::info!("Migration is a no-op"); } Ok(TenantShardMigrateResponse {}) diff --git a/libs/pageserver_api/src/upcall_api.rs b/libs/pageserver_api/src/upcall_api.rs index 5472948091cb..2e88836bd071 100644 --- a/libs/pageserver_api/src/upcall_api.rs +++ b/libs/pageserver_api/src/upcall_api.rs @@ -6,7 +6,9 @@ use serde::{Deserialize, Serialize}; use utils::id::NodeId; -use crate::{controller_api::NodeRegisterRequest, shard::TenantShardId}; +use crate::{ + controller_api::NodeRegisterRequest, models::LocationConfigMode, shard::TenantShardId, +}; /// Upcall message sent by the pageserver to the configured `control_plane_api` on /// startup. @@ -20,12 +22,20 @@ pub struct ReAttachRequest { pub register: Option, } -#[derive(Serialize, Deserialize)] +fn default_mode() -> LocationConfigMode { + LocationConfigMode::AttachedSingle +} + +#[derive(Serialize, Deserialize, Debug)] pub struct ReAttachResponseTenant { pub id: TenantShardId, - pub gen: u32, -} + /// Mandatory if LocationConfigMode is None or set to an Attached* mode + pub gen: Option, + /// Default value only for backward compat: this field should be set + #[serde(default = "default_mode")] + pub mode: LocationConfigMode, +} #[derive(Serialize, Deserialize)] pub struct ReAttachResponse { pub tenants: Vec, diff --git a/pageserver/src/control_plane_client.rs b/pageserver/src/control_plane_client.rs index 1b3d76335dbc..42c800822b08 100644 --- a/pageserver/src/control_plane_client.rs +++ b/pageserver/src/control_plane_client.rs @@ -5,7 +5,8 @@ use pageserver_api::{ controller_api::NodeRegisterRequest, shard::TenantShardId, upcall_api::{ - ReAttachRequest, ReAttachResponse, ValidateRequest, ValidateRequestTenant, ValidateResponse, + ReAttachRequest, ReAttachResponse, ReAttachResponseTenant, ValidateRequest, + ValidateRequestTenant, ValidateResponse, }, }; use serde::{de::DeserializeOwned, Serialize}; @@ -37,7 +38,9 @@ pub trait ControlPlaneGenerationsApi { fn re_attach( &self, conf: &PageServerConf, - ) -> impl Future, RetryForeverError>> + Send; + ) -> impl Future< + Output = Result, RetryForeverError>, + > + Send; fn validate( &self, tenants: Vec<(TenantShardId, Generation)>, @@ -118,7 +121,7 @@ impl ControlPlaneGenerationsApi for ControlPlaneClient { async fn re_attach( &self, conf: &PageServerConf, - ) -> Result, RetryForeverError> { + ) -> Result, RetryForeverError> { let re_attach_path = self .base_url .join("re-attach") @@ -181,7 +184,7 @@ impl ControlPlaneGenerationsApi for ControlPlaneClient { Ok(response .tenants .into_iter() - .map(|t| (t.id, Generation::new(t.gen))) + .map(|rart| (rart.id, rart)) .collect::>()) } diff --git a/pageserver/src/deletion_queue.rs b/pageserver/src/deletion_queue.rs index b6aea8fae8db..e3c11cb29963 100644 --- a/pageserver/src/deletion_queue.rs +++ b/pageserver/src/deletion_queue.rs @@ -724,8 +724,8 @@ impl DeletionQueue { mod test { use camino::Utf8Path; use hex_literal::hex; - use pageserver_api::shard::ShardIndex; - use std::io::ErrorKind; + use pageserver_api::{shard::ShardIndex, upcall_api::ReAttachResponseTenant}; + use std::{io::ErrorKind, time::Duration}; use tracing::info; use remote_storage::{RemoteStorageConfig, RemoteStorageKind}; @@ -834,9 +834,10 @@ mod test { async fn re_attach( &self, _conf: &PageServerConf, - ) -> Result, RetryForeverError> { + ) -> Result, RetryForeverError> { unimplemented!() } + async fn validate( &self, tenants: Vec<(TenantShardId, Generation)>, diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index 1c66f99ece9a..fe48741a89d1 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -202,6 +202,13 @@ pub(super) struct AttachedTenantConf { } impl AttachedTenantConf { + fn new(tenant_conf: TenantConfOpt, location: AttachedLocationConfig) -> Self { + Self { + tenant_conf, + location, + } + } + fn try_from(location_conf: LocationConf) -> anyhow::Result { match &location_conf.mode { LocationMode::Attached(attach_conf) => Ok(Self { diff --git a/pageserver/src/tenant/config.rs b/pageserver/src/tenant/config.rs index 57fc444cdd8e..53a8c97e23f2 100644 --- a/pageserver/src/tenant/config.rs +++ b/pageserver/src/tenant/config.rs @@ -196,16 +196,17 @@ impl LocationConf { /// For use when attaching/re-attaching: update the generation stored in this /// structure. If we were in a secondary state, promote to attached (posession /// of a fresh generation implies this). - pub(crate) fn attach_in_generation(&mut self, generation: Generation) { + pub(crate) fn attach_in_generation(&mut self, mode: AttachmentMode, generation: Generation) { match &mut self.mode { LocationMode::Attached(attach_conf) => { attach_conf.generation = generation; + attach_conf.attach_mode = mode; } LocationMode::Secondary(_) => { // We are promoted to attached by the control plane's re-attach response self.mode = LocationMode::Attached(AttachedLocationConfig { generation, - attach_mode: AttachmentMode::Single, + attach_mode: mode, }) } } diff --git a/pageserver/src/tenant/mgr.rs b/pageserver/src/tenant/mgr.rs index 7e0092d5b646..97a505ded906 100644 --- a/pageserver/src/tenant/mgr.rs +++ b/pageserver/src/tenant/mgr.rs @@ -4,10 +4,11 @@ use camino::{Utf8DirEntry, Utf8Path, Utf8PathBuf}; use itertools::Itertools; use pageserver_api::key::Key; -use pageserver_api::models::ShardParameters; +use pageserver_api::models::{LocationConfigMode, ShardParameters}; use pageserver_api::shard::{ ShardCount, ShardIdentity, ShardNumber, ShardStripeSize, TenantShardId, }; +use pageserver_api::upcall_api::ReAttachResponseTenant; use rand::{distributions::Alphanumeric, Rng}; use std::borrow::Cow; use std::cmp::Ordering; @@ -124,6 +125,46 @@ pub(crate) enum ShardSelector { Page(Key), } +/// A convenience for use with the re_attach ControlPlaneClient function: rather +/// than the serializable struct, we build this enum that encapsulates +/// the invariant that attached tenants always have generations. +/// +/// This represents the subset of a LocationConfig that we receive during re-attach. +pub(crate) enum TenantStartupMode { + Attached((AttachmentMode, Generation)), + Secondary, +} + +impl TenantStartupMode { + /// Return the generation & mode that should be used when starting + /// this tenant. + /// + /// If this returns None, the re-attach struct is in an invalid state and + /// should be ignored in the response. + fn from_reattach_tenant(rart: ReAttachResponseTenant) -> Option { + match (rart.mode, rart.gen) { + (LocationConfigMode::Detached, _) => None, + (LocationConfigMode::Secondary, _) => Some(Self::Secondary), + (LocationConfigMode::AttachedMulti, Some(g)) => { + Some(Self::Attached((AttachmentMode::Multi, Generation::new(g)))) + } + (LocationConfigMode::AttachedSingle, Some(g)) => { + Some(Self::Attached((AttachmentMode::Single, Generation::new(g)))) + } + (LocationConfigMode::AttachedStale, Some(g)) => { + Some(Self::Attached((AttachmentMode::Stale, Generation::new(g)))) + } + _ => { + tracing::warn!( + "Received invalid re-attach state for tenant {}: {rart:?}", + rart.id + ); + None + } + } + } +} + impl TenantsMap { /// Convenience function for typical usage, where we want to get a `Tenant` object, for /// working with attached tenants. If the TenantId is in the map but in Secondary state, @@ -270,7 +311,7 @@ pub struct TenantManager { fn emergency_generations( tenant_confs: &HashMap>, -) -> HashMap { +) -> HashMap { tenant_confs .iter() .filter_map(|(tid, lc)| { @@ -278,12 +319,15 @@ fn emergency_generations( Ok(lc) => lc, Err(_) => return None, }; - let gen = match &lc.mode { - LocationMode::Attached(alc) => Some(alc.generation), - LocationMode::Secondary(_) => None, - }; - - gen.map(|g| (*tid, g)) + Some(( + *tid, + match &lc.mode { + LocationMode::Attached(alc) => { + TenantStartupMode::Attached((alc.attach_mode, alc.generation)) + } + LocationMode::Secondary(_) => TenantStartupMode::Secondary, + }, + )) }) .collect() } @@ -293,7 +337,7 @@ async fn init_load_generations( tenant_confs: &HashMap>, resources: &TenantSharedResources, cancel: &CancellationToken, -) -> anyhow::Result>> { +) -> anyhow::Result>> { let generations = if conf.control_plane_emergency_mode { error!( "Emergency mode! Tenants will be attached unsafely using their last known generation" @@ -303,7 +347,12 @@ async fn init_load_generations( info!("Calling control plane API to re-attach tenants"); // If we are configured to use the control plane API, then it is the source of truth for what tenants to load. match client.re_attach(conf).await { - Ok(tenants) => tenants, + Ok(tenants) => tenants + .into_iter() + .flat_map(|(id, rart)| { + TenantStartupMode::from_reattach_tenant(rart).map(|tsm| (id, tsm)) + }) + .collect(), Err(RetryForeverError::ShuttingDown) => { anyhow::bail!("Shut down while waiting for control plane re-attach response") } @@ -321,9 +370,17 @@ async fn init_load_generations( // Must only do this if remote storage is enabled, otherwise deletion queue // is not running and channel push will fail. if resources.remote_storage.is_some() { - resources - .deletion_queue_client - .recover(generations.clone())?; + let attached_tenants = generations + .iter() + .flat_map(|(id, start_mode)| { + match start_mode { + TenantStartupMode::Attached((_mode, generation)) => Some(generation), + TenantStartupMode::Secondary => None, + } + .map(|gen| (*id, *gen)) + }) + .collect(); + resources.deletion_queue_client.recover(attached_tenants)?; } Ok(Some(generations)) @@ -489,9 +546,8 @@ pub async fn init_tenant_mgr( // Scan local filesystem for attached tenants let tenant_configs = init_load_tenant_configs(conf).await?; - // Determine which tenants are to be attached - let tenant_generations = - init_load_generations(conf, &tenant_configs, &resources, &cancel).await?; + // Determine which tenants are to be secondary or attached, and in which generation + let tenant_modes = init_load_generations(conf, &tenant_configs, &resources, &cancel).await?; tracing::info!( "Attaching {} tenants at startup, warming up {} at a time", @@ -521,97 +577,102 @@ pub async fn init_tenant_mgr( } }; - let generation = if let Some(generations) = &tenant_generations { + // FIXME: if we were attached, and get demoted to secondary on re-attach, we + // don't have a place to get a config. + // (https://github.com/neondatabase/neon/issues/5377) + const DEFAULT_SECONDARY_CONF: SecondaryLocationConfig = + SecondaryLocationConfig { warm: true }; + + // Update the location config according to the re-attach response + if let Some(tenant_modes) = &tenant_modes { // We have a generation map: treat it as the authority for whether // this tenant is really attached. - if let Some(gen) = generations.get(&tenant_shard_id) { - if let LocationMode::Attached(attached) = &location_conf.mode { - if attached.generation > *gen { + match tenant_modes.get(&tenant_shard_id) { + None => { + info!(tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(), "Detaching tenant, control plane omitted it in re-attach response"); + if let Err(e) = safe_remove_tenant_dir_all(&tenant_dir_path).await { + error!(tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(), + "Failed to remove detached tenant directory '{tenant_dir_path}': {e:?}", + ); + } + + // We deleted local content: move on to next tenant, don't try and spawn this one. + continue; + } + Some(TenantStartupMode::Secondary) => { + if !matches!(location_conf.mode, LocationMode::Secondary(_)) { + location_conf.mode = LocationMode::Secondary(DEFAULT_SECONDARY_CONF); + } + } + Some(TenantStartupMode::Attached((attach_mode, generation))) => { + let old_gen_higher = match &location_conf.mode { + LocationMode::Attached(AttachedLocationConfig { + generation: old_generation, + attach_mode: _attach_mode, + }) => { + if old_generation > generation { + Some(old_generation) + } else { + None + } + } + _ => None, + }; + if let Some(old_generation) = old_gen_higher { tracing::error!(tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(), - "Control plane gave decreasing generation ({gen:?}) in re-attach response for tenant that was attached in generation {:?}, demoting to secondary", - attached.generation + "Control plane gave decreasing generation ({generation:?}) in re-attach response for tenant that was attached in generation {:?}, demoting to secondary", + old_generation ); // We cannot safely attach this tenant given a bogus generation number, but let's avoid throwing away // local disk content: demote to secondary rather than detaching. - tenants.insert( - tenant_shard_id, - TenantSlot::Secondary(SecondaryTenant::new( - tenant_shard_id, - location_conf.shard, - location_conf.tenant_conf.clone(), - &SecondaryLocationConfig { warm: false }, - )), - ); + location_conf.mode = LocationMode::Secondary(DEFAULT_SECONDARY_CONF); + } else { + location_conf.attach_in_generation(*attach_mode, *generation); } } - *gen - } else { - match &location_conf.mode { - 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(SecondaryTenant::new( - tenant_shard_id, - location_conf.shard, - location_conf.tenant_conf, - secondary_config, - )), - ); - } - LocationMode::Attached(_) => { - // TODO: augment re-attach API to enable the control plane to - // instruct us about secondary attachments. That way, instead of throwing - // away local state, we can gracefully fall back to secondary here, if the control - // plane tells us so. - // (https://github.com/neondatabase/neon/issues/5377) - info!(tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(), "Detaching tenant, control plane omitted it in re-attach response"); - if let Err(e) = safe_remove_tenant_dir_all(&tenant_dir_path).await { - error!(tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(), - "Failed to remove detached tenant directory '{tenant_dir_path}': {e:?}", - ); - } - } - }; - - continue; } } else { // Legacy mode: no generation information, any tenant present // on local disk may activate info!(tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(), "Starting tenant in legacy mode, no generation",); - Generation::none() }; // Presence of a generation number implies attachment: attach the tenant // if it wasn't already, and apply the generation number. - location_conf.attach_in_generation(generation); Tenant::persist_tenant_config(conf, &tenant_shard_id, &location_conf).await?; let shard_identity = location_conf.shard; - match tenant_spawn( - conf, - tenant_shard_id, - &tenant_dir_path, - resources.clone(), - AttachedTenantConf::try_from(location_conf)?, - shard_identity, - Some(init_order.clone()), - &TENANTS, - SpawnMode::Lazy, - &ctx, - ) { - Ok(tenant) => { - tenants.insert(tenant_shard_id, TenantSlot::Attached(tenant)); - } - Err(e) => { - error!(tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(), "Failed to start tenant: {e:#}"); + let slot = match location_conf.mode { + LocationMode::Attached(attached_conf) => { + match tenant_spawn( + conf, + tenant_shard_id, + &tenant_dir_path, + resources.clone(), + AttachedTenantConf::new(location_conf.tenant_conf, attached_conf), + shard_identity, + Some(init_order.clone()), + &TENANTS, + SpawnMode::Lazy, + &ctx, + ) { + Ok(tenant) => TenantSlot::Attached(tenant), + Err(e) => { + error!(tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(), "Failed to start tenant: {e:#}"); + continue; + } + } } - } + LocationMode::Secondary(secondary_conf) => TenantSlot::Secondary(SecondaryTenant::new( + tenant_shard_id, + shard_identity, + location_conf.tenant_conf, + &secondary_conf, + )), + }; + + tenants.insert(tenant_shard_id, slot); } info!("Processed {} local tenants at startup", tenants.len()); @@ -2142,7 +2203,7 @@ pub(crate) async fn load_tenant( let mut location_conf = Tenant::load_tenant_config(conf, &tenant_shard_id).map_err(TenantMapInsertError::Other)?; - location_conf.attach_in_generation(generation); + location_conf.attach_in_generation(AttachmentMode::Single, generation); Tenant::persist_tenant_config(conf, &tenant_shard_id, &location_conf).await?; diff --git a/test_runner/regress/test_sharding_service.py b/test_runner/regress/test_sharding_service.py index a6b0f76c9613..b7488cadd6db 100644 --- a/test_runner/regress/test_sharding_service.py +++ b/test_runner/regress/test_sharding_service.py @@ -23,7 +23,7 @@ ) from fixtures.pg_version import PgVersion from fixtures.remote_storage import RemoteStorageKind, s3_storage -from fixtures.types import TenantId, TimelineId +from fixtures.types import TenantId, TenantShardId, TimelineId from fixtures.utils import run_pg_bench_small, wait_until from mypy_boto3_s3.type_defs import ( ObjectTypeDef, @@ -948,3 +948,65 @@ def storage_controller_consistent(): env.storage_controller.consistency_check() wait_until(10, 1, storage_controller_consistent) + + +def test_sharding_service_re_attach(neon_env_builder: NeonEnvBuilder): + """ + Exercise the behavior of the /re-attach endpoint on pageserver startup when + pageservers have a mixture of attached and secondary locations + """ + + neon_env_builder.num_pageservers = 2 + env = neon_env_builder.init_configs() + env.start() + + # We'll have two tenants. + tenant_a = TenantId.generate() + env.neon_cli.create_tenant(tenant_a, placement_policy='{"Attached":1}') + tenant_b = TenantId.generate() + env.neon_cli.create_tenant(tenant_b, placement_policy='{"Attached":1}') + + # Each pageserver will have one attached and one secondary location + env.storage_controller.tenant_shard_migrate( + TenantShardId(tenant_a, 0, 0), env.pageservers[0].id + ) + env.storage_controller.tenant_shard_migrate( + TenantShardId(tenant_b, 0, 0), env.pageservers[1].id + ) + + # Hard-fail a pageserver + victim_ps = env.pageservers[1] + survivor_ps = env.pageservers[0] + victim_ps.stop(immediate=True) + + # Heatbeater will notice it's offline, and consequently attachments move to the other pageserver + def failed_over(): + locations = survivor_ps.http_client().tenant_list_locations()["tenant_shards"] + log.info(f"locations: {locations}") + assert len(locations) == 2 + assert all(loc[1]["mode"] == "AttachedSingle" for loc in locations) + + # We could pre-empty this by configuring the node to Offline, but it's preferable to test + # the realistic path we would take when a node restarts uncleanly. + # The delay here will be ~NEON_LOCAL_MAX_UNAVAILABLE_INTERVAL in neon_local + wait_until(30, 1, failed_over) + + reconciles_before_restart = env.storage_controller.get_metric_value( + "storage_controller_reconcile_complete_total", filter={"status": "ok"} + ) + + # Restart the failed pageserver + victim_ps.start() + + # We expect that the re-attach call correctly tipped off the pageserver that its locations + # are all secondaries now. + locations = victim_ps.http_client().tenant_list_locations()["tenant_shards"] + assert len(locations) == 2 + assert all(loc[1]["mode"] == "Secondary" for loc in locations) + + # We expect that this situation resulted from the re_attach call, and not any explicit + # Reconciler runs: assert that the reconciliation count has not gone up since we restarted. + reconciles_after_restart = env.storage_controller.get_metric_value( + "storage_controller_reconcile_complete_total", filter={"status": "ok"} + ) + assert reconciles_after_restart == reconciles_before_restart From d5304337cf2b15826f28e1de92e97d87ba620952 Mon Sep 17 00:00:00 2001 From: Conrad Ludgate Date: Thu, 21 Mar 2024 13:54:06 +0000 Subject: [PATCH 23/53] proxy: simplify password validation (#7188) ## Problem for HTTP/WS/password hack flows we imitate SCRAM to validate passwords. This code was unnecessarily complicated. ## Summary of changes Copy in the `pbkdf2` and 'derive keys' steps from the `postgres_protocol` crate in our `rust-postgres` fork. Derive the `client_key`, `server_key` and `stored_key` from the password directly. Use constant time equality to compare the `stored_key` and `server_key` with the ones we are sent from cplane. --- Cargo.lock | 1 + Cargo.toml | 1 + proxy/Cargo.toml | 1 + proxy/src/auth/flow.rs | 9 +--- proxy/src/sasl.rs | 4 ++ proxy/src/scram.rs | 15 ++---- proxy/src/scram/exchange.rs | 93 +++++++++++++++++++++++++------------ 7 files changed, 75 insertions(+), 49 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index dcef66c15d56..824cac13b345 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4259,6 +4259,7 @@ dependencies = [ "smallvec", "smol_str", "socket2 0.5.5", + "subtle", "sync_wrapper", "task-local-extensions", "thiserror", diff --git a/Cargo.toml b/Cargo.toml index 0f3dbd498702..44e6ec97443c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -149,6 +149,7 @@ smol_str = { version = "0.2.0", features = ["serde"] } socket2 = "0.5" strum = "0.24" strum_macros = "0.24" +"subtle" = "2.5.0" svg_fmt = "0.4.1" sync_wrapper = "0.1.2" tar = "0.4" diff --git a/proxy/Cargo.toml b/proxy/Cargo.toml index b3a5bf873eda..93a1fe85dbc4 100644 --- a/proxy/Cargo.toml +++ b/proxy/Cargo.toml @@ -63,6 +63,7 @@ sha2 = { workspace = true, features = ["asm"] } smol_str.workspace = true smallvec.workspace = true socket2.workspace = true +subtle.workspace = true sync_wrapper.workspace = true task-local-extensions.workspace = true thiserror.workspace = true diff --git a/proxy/src/auth/flow.rs b/proxy/src/auth/flow.rs index f26dcb7c9a56..45bbad8cb269 100644 --- a/proxy/src/auth/flow.rs +++ b/proxy/src/auth/flow.rs @@ -194,14 +194,7 @@ pub(crate) async fn validate_password_and_exchange( } // perform scram authentication as both client and server to validate the keys AuthSecret::Scram(scram_secret) => { - use postgres_protocol::authentication::sasl::{ChannelBinding, ScramSha256}; - let sasl_client = ScramSha256::new(password, ChannelBinding::unsupported()); - let outcome = crate::scram::exchange( - &scram_secret, - sasl_client, - crate::config::TlsServerEndPoint::Undefined, - ) - .await?; + let outcome = crate::scram::exchange(&scram_secret, password).await?; let client_key = match outcome { sasl::Outcome::Success(client_key) => client_key, diff --git a/proxy/src/sasl.rs b/proxy/src/sasl.rs index 1cf8b53e110a..0811416ca285 100644 --- a/proxy/src/sasl.rs +++ b/proxy/src/sasl.rs @@ -33,6 +33,9 @@ pub enum Error { #[error("Internal error: missing digest")] MissingBinding, + #[error("could not decode salt: {0}")] + Base64(#[from] base64::DecodeError), + #[error(transparent)] Io(#[from] io::Error), } @@ -55,6 +58,7 @@ impl ReportableError for Error { Error::ChannelBindingBadMethod(_) => crate::error::ErrorKind::User, Error::BadClientMessage(_) => crate::error::ErrorKind::User, Error::MissingBinding => crate::error::ErrorKind::Service, + Error::Base64(_) => crate::error::ErrorKind::ControlPlane, Error::Io(_) => crate::error::ErrorKind::ClientDisconnect, } } diff --git a/proxy/src/scram.rs b/proxy/src/scram.rs index 76541ae2f3d2..ed80675f8a40 100644 --- a/proxy/src/scram.rs +++ b/proxy/src/scram.rs @@ -56,8 +56,6 @@ fn sha256<'a>(parts: impl IntoIterator) -> [u8; 32] { #[cfg(test)] mod tests { - use postgres_protocol::authentication::sasl::{ChannelBinding, ScramSha256}; - use crate::sasl::{Mechanism, Step}; use super::{Exchange, ServerSecret}; @@ -115,16 +113,9 @@ mod tests { async fn run_round_trip_test(server_password: &str, client_password: &str) { let scram_secret = ServerSecret::build(server_password).await.unwrap(); - let sasl_client = - ScramSha256::new(client_password.as_bytes(), ChannelBinding::unsupported()); - - let outcome = super::exchange( - &scram_secret, - sasl_client, - crate::config::TlsServerEndPoint::Undefined, - ) - .await - .unwrap(); + let outcome = super::exchange(&scram_secret, client_password.as_bytes()) + .await + .unwrap(); match outcome { crate::sasl::Outcome::Success(_) => {} diff --git a/proxy/src/scram/exchange.rs b/proxy/src/scram/exchange.rs index 51c0ba4e094c..682cbe795fd0 100644 --- a/proxy/src/scram/exchange.rs +++ b/proxy/src/scram/exchange.rs @@ -2,7 +2,11 @@ use std::convert::Infallible; -use postgres_protocol::authentication::sasl::ScramSha256; +use hmac::{Hmac, Mac}; +use sha2::digest::FixedOutput; +use sha2::{Digest, Sha256}; +use subtle::{Choice, ConstantTimeEq}; +use tokio::task::yield_now; use super::messages::{ ClientFinalMessage, ClientFirstMessage, OwnedServerFirstMessage, SCRAM_RAW_NONCE_LEN, @@ -71,40 +75,71 @@ impl<'a> Exchange<'a> { } } -pub async fn exchange( - secret: &ServerSecret, - mut client: ScramSha256, - tls_server_end_point: config::TlsServerEndPoint, -) -> sasl::Result> { - use sasl::Step::*; +// copied from +async fn pbkdf2(str: &[u8], salt: &[u8], iterations: u32) -> [u8; 32] { + let hmac = Hmac::::new_from_slice(str).expect("HMAC is able to accept all key sizes"); + let mut prev = hmac + .clone() + .chain_update(salt) + .chain_update(1u32.to_be_bytes()) + .finalize() + .into_bytes(); - let init = SaslInitial { - nonce: rand::random, - }; + let mut hi = prev; - let client_first = std::str::from_utf8(client.message()) - .map_err(|e| std::io::Error::new(std::io::ErrorKind::Other, e))?; - let sent = match init.transition(secret, &tls_server_end_point, client_first)? { - Continue(sent, server_first) => { - client.update(server_first.as_bytes()).await?; - sent - } - Success(x, _) => match x {}, - Failure(msg) => return Ok(sasl::Outcome::Failure(msg)), - }; + for i in 1..iterations { + prev = hmac.clone().chain_update(prev).finalize().into_bytes(); - let client_final = std::str::from_utf8(client.message()) - .map_err(|e| std::io::Error::new(std::io::ErrorKind::Other, e))?; - let keys = match sent.transition(secret, &tls_server_end_point, client_final)? { - Success(keys, server_final) => { - client.finish(server_final.as_bytes())?; - keys + for (hi, prev) in hi.iter_mut().zip(prev) { + *hi ^= prev; + } + // yield every ~250us + // hopefully reduces tail latencies + if i % 1024 == 0 { + yield_now().await } - Continue(x, _) => match x {}, - Failure(msg) => return Ok(sasl::Outcome::Failure(msg)), + } + + hi.into() +} + +// copied from +async fn derive_keys(password: &[u8], salt: &[u8], iterations: u32) -> ([u8; 32], [u8; 32]) { + let salted_password = pbkdf2(password, salt, iterations).await; + + let make_key = |name| { + let key = Hmac::::new_from_slice(&salted_password) + .expect("HMAC is able to accept all key sizes") + .chain_update(name) + .finalize(); + + <[u8; 32]>::from(key.into_bytes()) }; - Ok(sasl::Outcome::Success(keys)) + (make_key(b"Client Key"), make_key(b"Server Key")) +} + +pub async fn exchange( + secret: &ServerSecret, + password: &[u8], +) -> sasl::Result> { + let salt = base64::decode(&secret.salt_base64)?; + let (client_key, server_key) = derive_keys(password, &salt, secret.iterations).await; + let stored_key: [u8; 32] = Sha256::default() + .chain_update(client_key) + .finalize_fixed() + .into(); + + // constant time to not leak partial key match + let valid = stored_key.ct_eq(&secret.stored_key.as_bytes()) + | server_key.ct_eq(&secret.server_key.as_bytes()) + | Choice::from(secret.doomed as u8); + + if valid.into() { + Ok(sasl::Outcome::Success(super::ScramKey::from(client_key))) + } else { + Ok(sasl::Outcome::Failure("password doesn't match")) + } } impl SaslInitial { From fb60278e0272d5981b40611ca2fb2b29c4404f5c Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Thu, 21 Mar 2024 15:24:56 +0100 Subject: [PATCH 24/53] walredo benchmark: throughput-oriented rewrite (#7190) See the updated `bench_walredo.rs` module comment. tl;dr: we measure avg latency of single redo operations issues against a single redo manager from N tokio tasks. part of https://github.com/neondatabase/neon/issues/6628 --- pageserver/benches/bench_walredo.rs | 345 ++++++++++++++-------------- 1 file changed, 172 insertions(+), 173 deletions(-) diff --git a/pageserver/benches/bench_walredo.rs b/pageserver/benches/bench_walredo.rs index 47c8bd75c63a..3efad546a6f0 100644 --- a/pageserver/benches/bench_walredo.rs +++ b/pageserver/benches/bench_walredo.rs @@ -1,160 +1,156 @@ -//! Simple benchmarking around walredo. +//! Quantify a single walredo manager's throughput under N concurrent callers. //! -//! Right now they hope to just set a baseline. Later we can try to expand into latency and -//! throughput after figuring out the coordinated omission problems below. +//! The benchmark implementation ([`bench_impl`]) is parametrized by +//! - `redo_work` => [`Request::short_request`] or [`Request::medium_request`] +//! - `n_redos` => number of times the benchmark shell execute the `redo_work` +//! - `nclients` => number of clients (more on this shortly). //! -//! There are two sets of inputs; `short` and `medium`. They were collected on postgres v14 by -//! logging what happens when a sequential scan is requested on a small table, then picking out two -//! suitable from logs. +//! The benchmark impl sets up a multi-threaded tokio runtime with default parameters. +//! It spawns `nclients` times [`client`] tokio tasks. +//! Each task executes the `redo_work` `n_redos/nclients` times. //! +//! We exercise the following combinations: +//! - `redo_work = short / medium`` +//! - `nclients = [1, 2, 4, 8, 16, 32, 64, 128]` //! -//! Reference data (git blame to see commit) on an i3en.3xlarge -// ```text -//! short/short/1 time: [39.175 µs 39.348 µs 39.536 µs] -//! short/short/2 time: [51.227 µs 51.487 µs 51.755 µs] -//! short/short/4 time: [76.048 µs 76.362 µs 76.674 µs] -//! short/short/8 time: [128.94 µs 129.82 µs 130.74 µs] -//! short/short/16 time: [227.84 µs 229.00 µs 230.28 µs] -//! short/short/32 time: [455.97 µs 457.81 µs 459.90 µs] -//! short/short/64 time: [902.46 µs 904.84 µs 907.32 µs] -//! short/short/128 time: [1.7416 ms 1.7487 ms 1.7561 ms] -//! `` - -use std::sync::Arc; +//! We let `criterion` determine the `n_redos` using `iter_custom`. +//! The idea is that for each `(redo_work, nclients)` combination, +//! criterion will run the `bench_impl` multiple times with different `n_redos`. +//! The `bench_impl` reports the aggregate wall clock time from the clients' perspective. +//! Criterion will divide that by `n_redos` to compute the "time per iteration". +//! In our case, "time per iteration" means "time per redo_work execution". +//! +//! NB: the way by which `iter_custom` determines the "number of iterations" +//! is called sampling. Apparently the idea here is to detect outliers. +//! We're not sure whether the current choice of sampling method makes sense. +//! See https://bheisler.github.io/criterion.rs/book/user_guide/command_line_output.html#collecting-samples +//! +//! # Reference Numbers +//! +//! 2024-03-20 on i3en.3xlarge +//! +//! ```text +//! short/1 time: [26.483 µs 26.614 µs 26.767 µs] +//! short/2 time: [32.223 µs 32.465 µs 32.767 µs] +//! short/4 time: [47.203 µs 47.583 µs 47.984 µs] +//! short/8 time: [89.135 µs 89.612 µs 90.139 µs] +//! short/16 time: [190.12 µs 191.52 µs 192.88 µs] +//! short/32 time: [380.96 µs 382.63 µs 384.20 µs] +//! short/64 time: [736.86 µs 741.07 µs 745.03 µs] +//! short/128 time: [1.4106 ms 1.4206 ms 1.4294 ms] +//! medium/1 time: [111.81 µs 112.25 µs 112.79 µs] +//! medium/2 time: [158.26 µs 159.13 µs 160.21 µs] +//! medium/4 time: [334.65 µs 337.14 µs 340.07 µs] +//! medium/8 time: [675.32 µs 679.91 µs 685.25 µs] +//! medium/16 time: [1.2929 ms 1.2996 ms 1.3067 ms] +//! medium/32 time: [2.4295 ms 2.4461 ms 2.4623 ms] +//! medium/64 time: [4.3973 ms 4.4458 ms 4.4875 ms] +//! medium/128 time: [7.5955 ms 7.7847 ms 7.9481 ms] +//! ``` use bytes::{Buf, Bytes}; -use pageserver::{ - config::PageServerConf, repository::Key, walrecord::NeonWalRecord, walredo::PostgresRedoManager, +use criterion::{BenchmarkId, Criterion}; +use pageserver::{config::PageServerConf, walrecord::NeonWalRecord, walredo::PostgresRedoManager}; +use pageserver_api::{key::Key, shard::TenantShardId}; +use std::{ + sync::Arc, + time::{Duration, Instant}, }; -use pageserver_api::shard::TenantShardId; -use tokio::task::JoinSet; +use tokio::{sync::Barrier, task::JoinSet}; use utils::{id::TenantId, lsn::Lsn}; -use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; +fn bench(c: &mut Criterion) { + { + let nclients = [1, 2, 4, 8, 16, 32, 64, 128]; + for nclients in nclients { + let mut group = c.benchmark_group("short"); + group.bench_with_input( + BenchmarkId::from_parameter(nclients), + &nclients, + |b, nclients| { + let redo_work = Arc::new(Request::short_input()); + b.iter_custom(|iters| bench_impl(Arc::clone(&redo_work), iters, *nclients)); + }, + ); + } + } -fn redo_scenarios(c: &mut Criterion) { - // logging should be enabled when adding more inputs, since walredo will only report malformed - // input to the stderr. - // utils::logging::init(utils::logging::LogFormat::Plain).unwrap(); + { + let nclients = [1, 2, 4, 8, 16, 32, 64, 128]; + for nclients in nclients { + let mut group = c.benchmark_group("medium"); + group.bench_with_input( + BenchmarkId::from_parameter(nclients), + &nclients, + |b, nclients| { + let redo_work = Arc::new(Request::medium_input()); + b.iter_custom(|iters| bench_impl(Arc::clone(&redo_work), iters, *nclients)); + }, + ); + } + } +} +criterion::criterion_group!(benches, bench); +criterion::criterion_main!(benches); +// Returns the sum of each client's wall-clock time spent executing their share of the n_redos. +fn bench_impl(redo_work: Arc, n_redos: u64, nclients: u64) -> Duration { let repo_dir = camino_tempfile::tempdir_in(env!("CARGO_TARGET_TMPDIR")).unwrap(); let conf = PageServerConf::dummy_conf(repo_dir.path().to_path_buf()); let conf = Box::leak(Box::new(conf)); let tenant_shard_id = TenantShardId::unsharded(TenantId::generate()); - let manager = PostgresRedoManager::new(conf, tenant_shard_id); - - let manager = Arc::new(manager); - - { - let rt = tokio::runtime::Builder::new_current_thread() - .enable_all() - .build() - .unwrap(); - tracing::info!("executing first"); - rt.block_on(short().execute(&manager)).unwrap(); - tracing::info!("first executed"); - } - - let thread_counts = [1, 2, 4, 8, 16, 32, 64, 128]; - - let mut group = c.benchmark_group("short"); - group.sampling_mode(criterion::SamplingMode::Flat); - - for thread_count in thread_counts { - group.bench_with_input( - BenchmarkId::new("short", thread_count), - &thread_count, - |b, thread_count| { - add_multithreaded_walredo_requesters(b, *thread_count, &manager, short); - }, - ); - } - drop(group); - - let mut group = c.benchmark_group("medium"); - group.sampling_mode(criterion::SamplingMode::Flat); - - for thread_count in thread_counts { - group.bench_with_input( - BenchmarkId::new("medium", thread_count), - &thread_count, - |b, thread_count| { - add_multithreaded_walredo_requesters(b, *thread_count, &manager, medium); - }, - ); - } - drop(group); -} - -/// Sets up a multi-threaded tokio runtime with default worker thread count, -/// then, spawn `requesters` tasks that repeatedly: -/// - get input from `input_factor()` -/// - call `manager.request_redo()` with their input -/// -/// This stress-tests the scalability of a single walredo manager at high tokio-level concurrency. -/// -/// Using tokio's default worker thread count means the results will differ on machines -/// with different core countrs. We don't care about that, the performance will always -/// be different on different hardware. To compare performance of different software versions, -/// use the same hardware. -fn add_multithreaded_walredo_requesters( - b: &mut criterion::Bencher, - nrequesters: usize, - manager: &Arc, - input_factory: fn() -> Request, -) { - assert_ne!(nrequesters, 0); - let rt = tokio::runtime::Builder::new_multi_thread() .enable_all() .build() .unwrap(); - let barrier = Arc::new(tokio::sync::Barrier::new(nrequesters + 1)); + let start = Arc::new(Barrier::new(nclients as usize)); - let mut requesters = JoinSet::new(); - for _ in 0..nrequesters { - let _entered = rt.enter(); - let manager = manager.clone(); - let barrier = barrier.clone(); - requesters.spawn(async move { - loop { - let input = input_factory(); - barrier.wait().await; - let page = input.execute(&manager).await.unwrap(); - assert_eq!(page.remaining(), 8192); - barrier.wait().await; - } - }); - } + let mut tasks = JoinSet::new(); - let do_one_iteration = || { - rt.block_on(async { - barrier.wait().await; - // wait for work to complete - barrier.wait().await; - }) - }; + let manager = PostgresRedoManager::new(conf, tenant_shard_id); + let manager = Arc::new(manager); - b.iter_batched( - || { - // warmup - do_one_iteration(); - }, - |()| { - // work loop - do_one_iteration(); - }, - criterion::BatchSize::PerIteration, - ); + for _ in 0..nclients { + rt.block_on(async { + tasks.spawn(client( + Arc::clone(&manager), + Arc::clone(&start), + Arc::clone(&redo_work), + // divide the amount of work equally among the clients + n_redos / nclients, + )) + }); + } - rt.block_on(requesters.shutdown()); + rt.block_on(async move { + let mut total_wallclock_time = std::time::Duration::from_millis(0); + while let Some(res) = tasks.join_next().await { + total_wallclock_time += res.unwrap(); + } + total_wallclock_time + }) } -criterion_group!(benches, redo_scenarios); -criterion_main!(benches); +async fn client( + mgr: Arc, + start: Arc, + redo_work: Arc, + n_redos: u64, +) -> Duration { + start.wait().await; + let start = Instant::now(); + for _ in 0..n_redos { + let page = redo_work.execute(&mgr).await.unwrap(); + assert_eq!(page.remaining(), 8192); + // The real pageserver will rarely if ever do 2 walredos in a row without + // yielding to the executor. + tokio::task::yield_now().await; + } + start.elapsed() +} macro_rules! lsn { ($input:expr) => {{ @@ -166,12 +162,46 @@ macro_rules! lsn { }}; } -/// Short payload, 1132 bytes. -// pg_records are copypasted from log, where they are put with Debug impl of Bytes, which uses \0 -// for null bytes. -#[allow(clippy::octal_escapes)] -fn short() -> Request { - Request { +/// Simple wrapper around `WalRedoManager::request_redo`. +/// +/// In benchmarks this is cloned around. +#[derive(Clone)] +struct Request { + key: Key, + lsn: Lsn, + base_img: Option<(Lsn, Bytes)>, + records: Vec<(Lsn, NeonWalRecord)>, + pg_version: u32, +} + +impl Request { + async fn execute(&self, manager: &PostgresRedoManager) -> anyhow::Result { + let Request { + key, + lsn, + base_img, + records, + pg_version, + } = self; + + // TODO: avoid these clones + manager + .request_redo(*key, *lsn, base_img.clone(), records.clone(), *pg_version) + .await + } + + fn pg_record(will_init: bool, bytes: &'static [u8]) -> NeonWalRecord { + let rec = Bytes::from_static(bytes); + NeonWalRecord::Postgres { will_init, rec } + } + + /// Short payload, 1132 bytes. + // pg_records are copypasted from log, where they are put with Debug impl of Bytes, which uses \0 + // for null bytes. + #[allow(clippy::octal_escapes)] + pub fn short_input() -> Request { + let pg_record = Self::pg_record; + Request { key: Key { field1: 0, field2: 1663, @@ -194,13 +224,14 @@ fn short() -> Request { ], pg_version: 14, } -} + } -/// Medium sized payload, serializes as 26393 bytes. -// see [`short`] -#[allow(clippy::octal_escapes)] -fn medium() -> Request { - Request { + /// Medium sized payload, serializes as 26393 bytes. + // see [`short`] + #[allow(clippy::octal_escapes)] + pub fn medium_input() -> Request { + let pg_record = Self::pg_record; + Request { key: Key { field1: 0, field2: 1663, @@ -442,37 +473,5 @@ fn medium() -> Request { ], pg_version: 14, } -} - -fn pg_record(will_init: bool, bytes: &'static [u8]) -> NeonWalRecord { - let rec = Bytes::from_static(bytes); - NeonWalRecord::Postgres { will_init, rec } -} - -/// Simple wrapper around `WalRedoManager::request_redo`. -/// -/// In benchmarks this is cloned around. -#[derive(Clone)] -struct Request { - key: Key, - lsn: Lsn, - base_img: Option<(Lsn, Bytes)>, - records: Vec<(Lsn, NeonWalRecord)>, - pg_version: u32, -} - -impl Request { - async fn execute(self, manager: &PostgresRedoManager) -> anyhow::Result { - let Request { - key, - lsn, - base_img, - records, - pg_version, - } = self; - - manager - .request_redo(key, lsn, base_img, records, pg_version) - .await } } From 3ee34a3f26c232b48c1f386675d9d809869c7ba6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Arpad=20M=C3=BCller?= Date: Fri, 22 Mar 2024 07:52:31 +0100 Subject: [PATCH 25/53] Update Rust to 1.77.0 (#7198) Release notes: https://blog.rust-lang.org/2024/03/21/Rust-1.77.0.html Thanks to #6886 the diff is reasonable, only for one new lint `clippy::suspicious_open_options`. I added `truncate()` calls to the places where it is obviously the right choice to me, and added allows everywhere else, leaving it for followups. I had to specify cargo install --locked because the build would fail otherwise. This was also recommended by upstream. --- Dockerfile.build-tools | 4 ++-- compute_tools/src/config.rs | 1 + libs/remote_storage/src/local_fs.rs | 1 + libs/utils/src/lock_file.rs | 1 + pageserver/src/tenant/delete.rs | 1 + rust-toolchain.toml | 2 +- safekeeper/src/copy_timeline.rs | 1 + safekeeper/src/wal_storage.rs | 1 + 8 files changed, 9 insertions(+), 3 deletions(-) diff --git a/Dockerfile.build-tools b/Dockerfile.build-tools index 3a452fec3232..1ed6f8747335 100644 --- a/Dockerfile.build-tools +++ b/Dockerfile.build-tools @@ -135,7 +135,7 @@ WORKDIR /home/nonroot # Rust # Please keep the version of llvm (installed above) in sync with rust llvm (`rustc --version --verbose | grep LLVM`) -ENV RUSTC_VERSION=1.76.0 +ENV RUSTC_VERSION=1.77.0 ENV RUSTUP_HOME="/home/nonroot/.rustup" ENV PATH="/home/nonroot/.cargo/bin:${PATH}" RUN curl -sSO https://static.rust-lang.org/rustup/dist/$(uname -m)-unknown-linux-gnu/rustup-init && whoami && \ @@ -149,7 +149,7 @@ RUN curl -sSO https://static.rust-lang.org/rustup/dist/$(uname -m)-unknown-linux cargo install --git https://github.com/paritytech/cachepot && \ cargo install rustfilt && \ cargo install cargo-hakari && \ - cargo install cargo-deny && \ + cargo install cargo-deny --locked && \ cargo install cargo-hack && \ cargo install cargo-nextest && \ rm -rf /home/nonroot/.cargo/registry && \ diff --git a/compute_tools/src/config.rs b/compute_tools/src/config.rs index 42b848021195..f1fd8637f5c2 100644 --- a/compute_tools/src/config.rs +++ b/compute_tools/src/config.rs @@ -17,6 +17,7 @@ pub fn line_in_file(path: &Path, line: &str) -> Result { .write(true) .create(true) .append(false) + .truncate(false) .open(path)?; let buf = io::BufReader::new(&file); let mut count: usize = 0; diff --git a/libs/remote_storage/src/local_fs.rs b/libs/remote_storage/src/local_fs.rs index 313d8226b124..8cad86373152 100644 --- a/libs/remote_storage/src/local_fs.rs +++ b/libs/remote_storage/src/local_fs.rs @@ -198,6 +198,7 @@ impl LocalFs { fs::OpenOptions::new() .write(true) .create(true) + .truncate(true) .open(&temp_file_path) .await .with_context(|| { diff --git a/libs/utils/src/lock_file.rs b/libs/utils/src/lock_file.rs index 987b9d9ad237..59c66ca7579e 100644 --- a/libs/utils/src/lock_file.rs +++ b/libs/utils/src/lock_file.rs @@ -63,6 +63,7 @@ impl UnwrittenLockFile { pub fn create_exclusive(lock_file_path: &Utf8Path) -> anyhow::Result { let lock_file = fs::OpenOptions::new() .create(true) // O_CREAT + .truncate(true) .write(true) .open(lock_file_path) .context("open lock file")?; diff --git a/pageserver/src/tenant/delete.rs b/pageserver/src/tenant/delete.rs index cab60c3111ff..7d37873a67e6 100644 --- a/pageserver/src/tenant/delete.rs +++ b/pageserver/src/tenant/delete.rs @@ -111,6 +111,7 @@ async fn create_local_delete_mark( let _ = std::fs::OpenOptions::new() .write(true) .create(true) + .truncate(true) .open(&marker_path) .with_context(|| format!("could not create delete marker file {marker_path:?}"))?; diff --git a/rust-toolchain.toml b/rust-toolchain.toml index b0949c32b103..50a5a4185b4a 100644 --- a/rust-toolchain.toml +++ b/rust-toolchain.toml @@ -1,5 +1,5 @@ [toolchain] -channel = "1.76.0" +channel = "1.77.0" profile = "default" # The default profile includes rustc, rust-std, cargo, rust-docs, rustfmt and clippy. # https://rust-lang.github.io/rustup/concepts/profiles.html diff --git a/safekeeper/src/copy_timeline.rs b/safekeeper/src/copy_timeline.rs index 5bc877adbdbb..3023d4e2cb19 100644 --- a/safekeeper/src/copy_timeline.rs +++ b/safekeeper/src/copy_timeline.rs @@ -225,6 +225,7 @@ async fn write_segment( assert!(from <= to); assert!(to <= wal_seg_size); + #[allow(clippy::suspicious_open_options)] let mut file = OpenOptions::new() .create(true) .write(true) diff --git a/safekeeper/src/wal_storage.rs b/safekeeper/src/wal_storage.rs index 8bbd95e9e85c..147f318b9fea 100644 --- a/safekeeper/src/wal_storage.rs +++ b/safekeeper/src/wal_storage.rs @@ -221,6 +221,7 @@ impl PhysicalStorage { // half initialized segment, first bake it under tmp filename and // then rename. let tmp_path = self.timeline_dir.join("waltmp"); + #[allow(clippy::suspicious_open_options)] let mut file = OpenOptions::new() .create(true) .write(true) From 6770ddba2e24b81429abc68576f78ff06816edb2 Mon Sep 17 00:00:00 2001 From: Anna Khanova <32508607+khanova@users.noreply.github.com> Date: Fri, 22 Mar 2024 09:38:04 +0100 Subject: [PATCH 26/53] proxy: connect redis with AWS IAM (#7189) ## Problem Support of IAM Roles for Service Accounts for authentication. ## Summary of changes * Obtain aws 15m-long credentials * Retrieve redis password from credentials * Update every 1h to keep connection for more than 12h * For now allow to have different endpoints for pubsub/stream redis. TODOs: * PubSub doesn't support credentials refresh, consider using stream instead. * We need an AWS role for proxy to be able to connect to both: S3 and elasticache. Credentials obtaining and connection refresh was tested on xenon preview. https://github.com/neondatabase/cloud/issues/10365 --- Cargo.lock | 117 ++++++--- Cargo.toml | 6 +- proxy/Cargo.toml | 5 + proxy/src/bin/proxy.rs | 108 ++++++++- proxy/src/cancellation.rs | 147 ++++++------ proxy/src/config.rs | 1 + proxy/src/metrics.rs | 3 + proxy/src/proxy.rs | 12 +- proxy/src/proxy/passthrough.rs | 6 +- proxy/src/redis.rs | 4 +- proxy/src/redis/cancellation_publisher.rs | 167 +++++++++++++ .../connection_with_credentials_provider.rs | 225 ++++++++++++++++++ proxy/src/redis/elasticache.rs | 110 +++++++++ proxy/src/redis/notifications.rs | 62 ++--- proxy/src/redis/publisher.rs | 80 ------- proxy/src/serverless.rs | 7 +- proxy/src/serverless/websocket.rs | 4 +- workspace_hack/Cargo.toml | 3 +- 18 files changed, 803 insertions(+), 264 deletions(-) create mode 100644 proxy/src/redis/cancellation_publisher.rs create mode 100644 proxy/src/redis/connection_with_credentials_provider.rs create mode 100644 proxy/src/redis/elasticache.rs delete mode 100644 proxy/src/redis/publisher.rs diff --git a/Cargo.lock b/Cargo.lock index 824cac13b345..dcf1c4992461 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -347,9 +347,9 @@ dependencies = [ [[package]] name = "aws-credential-types" -version = "1.1.4" +version = "1.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "33cc49dcdd31c8b6e79850a179af4c367669150c7ac0135f176c61bec81a70f7" +checksum = "fa8587ae17c8e967e4b05a62d495be2fb7701bec52a97f7acfe8a29f938384c8" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -359,9 +359,9 @@ dependencies = [ [[package]] name = "aws-runtime" -version = "1.1.4" +version = "1.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb031bff99877c26c28895766f7bb8484a05e24547e370768d6cc9db514662aa" +checksum = "b13dc54b4b49f8288532334bba8f87386a40571c47c37b1304979b556dc613c8" dependencies = [ "aws-credential-types", "aws-sigv4", @@ -381,6 +381,29 @@ dependencies = [ "uuid", ] +[[package]] +name = "aws-sdk-iam" +version = "1.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b8ae76026bfb1b80a6aed0bb400c1139cd9c0563e26bce1986cd021c6a968c7b" +dependencies = [ + "aws-credential-types", + "aws-runtime", + "aws-smithy-async", + "aws-smithy-http", + "aws-smithy-json", + "aws-smithy-query", + "aws-smithy-runtime", + "aws-smithy-runtime-api", + "aws-smithy-types", + "aws-smithy-xml", + "aws-types", + "http 0.2.9", + "once_cell", + "regex-lite", + "tracing", +] + [[package]] name = "aws-sdk-s3" version = "1.14.0" @@ -502,9 +525,9 @@ dependencies = [ [[package]] name = "aws-sigv4" -version = "1.1.4" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c371c6b0ac54d4605eb6f016624fb5c7c2925d315fdf600ac1bf21b19d5f1742" +checksum = "11d6f29688a4be9895c0ba8bef861ad0c0dac5c15e9618b9b7a6c233990fc263" dependencies = [ "aws-credential-types", "aws-smithy-eventstream", @@ -517,7 +540,7 @@ dependencies = [ "hex", "hmac", "http 0.2.9", - "http 1.0.0", + "http 1.1.0", "once_cell", "p256", "percent-encoding", @@ -531,9 +554,9 @@ dependencies = [ [[package]] name = "aws-smithy-async" -version = "1.1.4" +version = "1.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72ee2d09cce0ef3ae526679b522835d63e75fb427aca5413cd371e490d52dcc6" +checksum = "d26ea8fa03025b2face2b3038a63525a10891e3d8829901d502e5384a0d8cd46" dependencies = [ "futures-util", "pin-project-lite", @@ -574,9 +597,9 @@ dependencies = [ [[package]] name = "aws-smithy-http" -version = "0.60.4" +version = "0.60.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dab56aea3cd9e1101a0a999447fb346afb680ab1406cebc44b32346e25b4117d" +checksum = "3f10fa66956f01540051b0aa7ad54574640f748f9839e843442d99b970d3aff9" dependencies = [ "aws-smithy-eventstream", "aws-smithy-runtime-api", @@ -595,18 +618,18 @@ dependencies = [ [[package]] name = "aws-smithy-json" -version = "0.60.4" +version = "0.60.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd3898ca6518f9215f62678870064398f00031912390efd03f1f6ef56d83aa8e" +checksum = "4683df9469ef09468dad3473d129960119a0d3593617542b7d52086c8486f2d6" dependencies = [ "aws-smithy-types", ] [[package]] name = "aws-smithy-query" -version = "0.60.4" +version = "0.60.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bda4b1dfc9810e35fba8a620e900522cd1bd4f9578c446e82f49d1ce41d2e9f9" +checksum = "f2fbd61ceb3fe8a1cb7352e42689cec5335833cd9f94103a61e98f9bb61c64bb" dependencies = [ "aws-smithy-types", "urlencoding", @@ -614,9 +637,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime" -version = "1.1.4" +version = "1.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fafdab38f40ad7816e7da5dec279400dd505160780083759f01441af1bbb10ea" +checksum = "ec81002d883e5a7fd2bb063d6fb51c4999eb55d404f4fff3dd878bf4733b9f01" dependencies = [ "aws-smithy-async", "aws-smithy-http", @@ -639,14 +662,15 @@ dependencies = [ [[package]] name = "aws-smithy-runtime-api" -version = "1.1.4" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c18276dd28852f34b3bf501f4f3719781f4999a51c7bff1a5c6dc8c4529adc29" +checksum = "9acb931e0adaf5132de878f1398d83f8677f90ba70f01f65ff87f6d7244be1c5" dependencies = [ "aws-smithy-async", "aws-smithy-types", "bytes", "http 0.2.9", + "http 1.1.0", "pin-project-lite", "tokio", "tracing", @@ -655,9 +679,9 @@ dependencies = [ [[package]] name = "aws-smithy-types" -version = "1.1.4" +version = "1.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb3e134004170d3303718baa2a4eb4ca64ee0a1c0a7041dca31b38be0fb414f3" +checksum = "abe14dceea1e70101d38fbf2a99e6a34159477c0fb95e68e05c66bd7ae4c3729" dependencies = [ "base64-simd", "bytes", @@ -678,18 +702,18 @@ dependencies = [ [[package]] name = "aws-smithy-xml" -version = "0.60.4" +version = "0.60.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8604a11b25e9ecaf32f9aa56b9fe253c5e2f606a3477f0071e96d3155a5ed218" +checksum = "872c68cf019c0e4afc5de7753c4f7288ce4b71663212771bf5e4542eb9346ca9" dependencies = [ "xmlparser", ] [[package]] name = "aws-types" -version = "1.1.4" +version = "1.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "789bbe008e65636fe1b6dbbb374c40c8960d1232b96af5ff4aec349f9c4accf4" +checksum = "0dbf2f3da841a8930f159163175cf6a3d16ddde517c1b0fba7aa776822800f40" dependencies = [ "aws-credential-types", "aws-smithy-async", @@ -2396,9 +2420,9 @@ dependencies = [ [[package]] name = "http" -version = "1.0.0" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b32afd38673a8016f7c9ae69e5af41a58f81b1d31689040f2f1959594ce194ea" +checksum = "21b9ddb458710bc376481b842f5da65cdf31522de232c1ca8146abce2a358258" dependencies = [ "bytes", "fnv", @@ -2498,7 +2522,7 @@ dependencies = [ "hyper", "log", "rustls 0.21.9", - "rustls-native-certs", + "rustls-native-certs 0.6.2", "tokio", "tokio-rustls 0.24.0", ] @@ -4199,6 +4223,10 @@ version = "0.1.0" dependencies = [ "anyhow", "async-trait", + "aws-config", + "aws-sdk-iam", + "aws-sigv4", + "aws-types", "base64 0.13.1", "bstr", "bytes", @@ -4216,6 +4244,7 @@ dependencies = [ "hex", "hmac", "hostname", + "http 1.1.0", "humantime", "hyper", "hyper-tungstenite", @@ -4431,9 +4460,9 @@ dependencies = [ [[package]] name = "redis" -version = "0.24.0" +version = "0.25.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c580d9cbbe1d1b479e8d67cf9daf6a62c957e6846048408b80b43ac3f6af84cd" +checksum = "71d64e978fd98a0e6b105d066ba4889a7301fca65aeac850a877d8797343feeb" dependencies = [ "async-trait", "bytes", @@ -4442,15 +4471,15 @@ dependencies = [ "itoa", "percent-encoding", "pin-project-lite", - "rustls 0.21.9", - "rustls-native-certs", - "rustls-pemfile 1.0.2", - "rustls-webpki 0.101.7", + "rustls 0.22.2", + "rustls-native-certs 0.7.0", + "rustls-pemfile 2.1.1", + "rustls-pki-types", "ryu", "sha1_smol", - "socket2 0.4.9", + "socket2 0.5.5", "tokio", - "tokio-rustls 0.24.0", + "tokio-rustls 0.25.0", "tokio-util", "url", ] @@ -4879,6 +4908,19 @@ dependencies = [ "security-framework", ] +[[package]] +name = "rustls-native-certs" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f1fb85efa936c42c6d5fc28d2629bb51e4b2f4b8a5211e297d599cc5a093792" +dependencies = [ + "openssl-probe", + "rustls-pemfile 2.1.1", + "rustls-pki-types", + "schannel", + "security-framework", +] + [[package]] name = "rustls-pemfile" version = "1.0.2" @@ -6146,7 +6188,7 @@ dependencies = [ "percent-encoding", "pin-project", "prost", - "rustls-native-certs", + "rustls-native-certs 0.6.2", "rustls-pemfile 1.0.2", "tokio", "tokio-rustls 0.24.0", @@ -7031,7 +7073,6 @@ dependencies = [ "aws-sigv4", "aws-smithy-async", "aws-smithy-http", - "aws-smithy-runtime-api", "aws-smithy-types", "axum", "base64 0.21.1", diff --git a/Cargo.toml b/Cargo.toml index 44e6ec97443c..2741bd046bb6 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -53,9 +53,12 @@ async-trait = "0.1" aws-config = { version = "1.1.4", default-features = false, features=["rustls"] } aws-sdk-s3 = "1.14" aws-sdk-secretsmanager = { version = "1.14.0" } +aws-sdk-iam = "1.15.0" aws-smithy-async = { version = "1.1.4", default-features = false, features=["rt-tokio"] } aws-smithy-types = "1.1.4" aws-credential-types = "1.1.4" +aws-sigv4 = { version = "1.2.0", features = ["sign-http"] } +aws-types = "1.1.7" axum = { version = "0.6.20", features = ["ws"] } base64 = "0.13.0" bincode = "1.3" @@ -88,6 +91,7 @@ hex = "0.4" hex-literal = "0.4" hmac = "0.12.1" hostname = "0.3.1" +http = {version = "1.1.0", features = ["std"]} http-types = { version = "2", default-features = false } humantime = "2.1" humantime-serde = "1.1.1" @@ -121,7 +125,7 @@ procfs = "0.14" prometheus = {version = "0.13", default_features=false, features = ["process"]} # removes protobuf dependency prost = "0.11" rand = "0.8" -redis = { version = "0.24.0", features = ["tokio-rustls-comp", "keep-alive"] } +redis = { version = "0.25.2", features = ["tokio-rustls-comp", "keep-alive"] } regex = "1.10.2" reqwest = { version = "0.11", default-features = false, features = ["rustls-tls"] } reqwest-tracing = { version = "0.4.7", features = ["opentelemetry_0_20"] } diff --git a/proxy/Cargo.toml b/proxy/Cargo.toml index 93a1fe85dbc4..3566d8b7288e 100644 --- a/proxy/Cargo.toml +++ b/proxy/Cargo.toml @@ -11,6 +11,10 @@ testing = [] [dependencies] anyhow.workspace = true async-trait.workspace = true +aws-config.workspace = true +aws-sdk-iam.workspace = true +aws-sigv4.workspace = true +aws-types.workspace = true base64.workspace = true bstr.workspace = true bytes = { workspace = true, features = ["serde"] } @@ -27,6 +31,7 @@ hashlink.workspace = true hex.workspace = true hmac.workspace = true hostname.workspace = true +http.workspace = true humantime.workspace = true hyper-tungstenite.workspace = true hyper.workspace = true diff --git a/proxy/src/bin/proxy.rs b/proxy/src/bin/proxy.rs index b3d4fc041186..d38439c2a0d6 100644 --- a/proxy/src/bin/proxy.rs +++ b/proxy/src/bin/proxy.rs @@ -1,3 +1,10 @@ +use aws_config::environment::EnvironmentVariableCredentialsProvider; +use aws_config::imds::credentials::ImdsCredentialsProvider; +use aws_config::meta::credentials::CredentialsProviderChain; +use aws_config::meta::region::RegionProviderChain; +use aws_config::profile::ProfileFileCredentialsProvider; +use aws_config::provider_config::ProviderConfig; +use aws_config::web_identity_token::WebIdentityTokenCredentialsProvider; use futures::future::Either; use proxy::auth; use proxy::auth::backend::MaybeOwned; @@ -10,11 +17,14 @@ use proxy::config::ProjectInfoCacheOptions; use proxy::console; use proxy::context::parquet::ParquetUploadArgs; use proxy::http; +use proxy::metrics::NUM_CANCELLATION_REQUESTS_SOURCE_FROM_CLIENT; use proxy::rate_limiter::EndpointRateLimiter; use proxy::rate_limiter::RateBucketInfo; use proxy::rate_limiter::RateLimiterConfig; +use proxy::redis::cancellation_publisher::RedisPublisherClient; +use proxy::redis::connection_with_credentials_provider::ConnectionWithCredentialsProvider; +use proxy::redis::elasticache; use proxy::redis::notifications; -use proxy::redis::publisher::RedisPublisherClient; use proxy::serverless::GlobalConnPoolOptions; use proxy::usage_metrics; @@ -150,9 +160,24 @@ struct ProxyCliArgs { /// disable ip check for http requests. If it is too time consuming, it could be turned off. #[clap(long, default_value_t = false, value_parser = clap::builder::BoolishValueParser::new(), action = clap::ArgAction::Set)] disable_ip_check_for_http: bool, - /// redis url for notifications. + /// redis url for notifications (if empty, redis_host:port will be used for both notifications and streaming connections) #[clap(long)] redis_notifications: Option, + /// redis host for streaming connections (might be different from the notifications host) + #[clap(long)] + redis_host: Option, + /// redis port for streaming connections (might be different from the notifications host) + #[clap(long)] + redis_port: Option, + /// redis cluster name, used in aws elasticache + #[clap(long)] + redis_cluster_name: Option, + /// redis user_id, used in aws elasticache + #[clap(long)] + redis_user_id: Option, + /// aws region to retrieve credentials + #[clap(long, default_value_t = String::new())] + aws_region: String, /// cache for `project_info` (use `size=0` to disable) #[clap(long, default_value = config::ProjectInfoCacheOptions::CACHE_DEFAULT_OPTIONS)] project_info_cache: String, @@ -216,6 +241,61 @@ async fn main() -> anyhow::Result<()> { let config = build_config(&args)?; info!("Authentication backend: {}", config.auth_backend); + info!("Using region: {}", config.aws_region); + + let region_provider = RegionProviderChain::default_provider().or_else(&*config.aws_region); // Replace with your Redis region if needed + let provider_conf = + ProviderConfig::without_region().with_region(region_provider.region().await); + let aws_credentials_provider = { + // uses "AWS_ACCESS_KEY_ID", "AWS_SECRET_ACCESS_KEY" + CredentialsProviderChain::first_try("env", EnvironmentVariableCredentialsProvider::new()) + // uses "AWS_PROFILE" / `aws sso login --profile ` + .or_else( + "profile-sso", + ProfileFileCredentialsProvider::builder() + .configure(&provider_conf) + .build(), + ) + // uses "AWS_WEB_IDENTITY_TOKEN_FILE", "AWS_ROLE_ARN", "AWS_ROLE_SESSION_NAME" + // needed to access remote extensions bucket + .or_else( + "token", + WebIdentityTokenCredentialsProvider::builder() + .configure(&provider_conf) + .build(), + ) + // uses imds v2 + .or_else("imds", ImdsCredentialsProvider::builder().build()) + }; + let elasticache_credentials_provider = Arc::new(elasticache::CredentialsProvider::new( + elasticache::AWSIRSAConfig::new( + config.aws_region.clone(), + args.redis_cluster_name, + args.redis_user_id, + ), + aws_credentials_provider, + )); + let redis_notifications_client = + match (args.redis_notifications, (args.redis_host, args.redis_port)) { + (Some(url), _) => { + info!("Starting redis notifications listener ({url})"); + Some(ConnectionWithCredentialsProvider::new_with_static_credentials(url)) + } + (None, (Some(host), Some(port))) => Some( + ConnectionWithCredentialsProvider::new_with_credentials_provider( + host, + port, + elasticache_credentials_provider.clone(), + ), + ), + (None, (None, None)) => { + warn!("Redis is disabled"); + None + } + _ => { + bail!("redis-host and redis-port must be specified together"); + } + }; // Check that we can bind to address before further initialization let http_address: SocketAddr = args.http.parse()?; @@ -233,17 +313,22 @@ async fn main() -> anyhow::Result<()> { let endpoint_rate_limiter = Arc::new(EndpointRateLimiter::new(&config.endpoint_rps_limit)); let cancel_map = CancelMap::default(); - let redis_publisher = match &args.redis_notifications { - Some(url) => Some(Arc::new(Mutex::new(RedisPublisherClient::new( - url, + + // let redis_notifications_client = redis_notifications_client.map(|x| Box::leak(Box::new(x))); + let redis_publisher = match &redis_notifications_client { + Some(redis_publisher) => Some(Arc::new(Mutex::new(RedisPublisherClient::new( + redis_publisher.clone(), args.region.clone(), &config.redis_rps_limit, )?))), None => None, }; - let cancellation_handler = Arc::new(CancellationHandler::new( + let cancellation_handler = Arc::new(CancellationHandler::< + Option>>, + >::new( cancel_map.clone(), redis_publisher, + NUM_CANCELLATION_REQUESTS_SOURCE_FROM_CLIENT, )); // client facing tasks. these will exit on error or on cancellation @@ -290,17 +375,16 @@ async fn main() -> anyhow::Result<()> { if let auth::BackendType::Console(api, _) = &config.auth_backend { if let proxy::console::provider::ConsoleBackend::Console(api) = &**api { - let cache = api.caches.project_info.clone(); - if let Some(url) = args.redis_notifications { - info!("Starting redis notifications listener ({url})"); + if let Some(redis_notifications_client) = redis_notifications_client { + let cache = api.caches.project_info.clone(); maintenance_tasks.spawn(notifications::task_main( - url.to_owned(), + redis_notifications_client.clone(), cache.clone(), cancel_map.clone(), args.region.clone(), )); + maintenance_tasks.spawn(async move { cache.clone().gc_worker().await }); } - maintenance_tasks.spawn(async move { cache.clone().gc_worker().await }); } } @@ -445,8 +529,8 @@ fn build_config(args: &ProxyCliArgs) -> anyhow::Result<&'static ProxyConfig> { endpoint_rps_limit, redis_rps_limit, handshake_timeout: args.handshake_timeout, - // TODO: add this argument region: args.region.clone(), + aws_region: args.aws_region.clone(), })); Ok(config) diff --git a/proxy/src/cancellation.rs b/proxy/src/cancellation.rs index c9607909b34c..8054f33b6c93 100644 --- a/proxy/src/cancellation.rs +++ b/proxy/src/cancellation.rs @@ -1,4 +1,3 @@ -use async_trait::async_trait; use dashmap::DashMap; use pq_proto::CancelKeyData; use std::{net::SocketAddr, sync::Arc}; @@ -10,18 +9,26 @@ use tracing::info; use uuid::Uuid; use crate::{ - error::ReportableError, metrics::NUM_CANCELLATION_REQUESTS, - redis::publisher::RedisPublisherClient, + error::ReportableError, + metrics::NUM_CANCELLATION_REQUESTS, + redis::cancellation_publisher::{ + CancellationPublisher, CancellationPublisherMut, RedisPublisherClient, + }, }; pub type CancelMap = Arc>>; +pub type CancellationHandlerMain = CancellationHandler>>>; +pub type CancellationHandlerMainInternal = Option>>; /// Enables serving `CancelRequest`s. /// -/// If there is a `RedisPublisherClient` available, it will be used to publish the cancellation key to other proxy instances. -pub struct CancellationHandler { +/// If `CancellationPublisher` is available, cancel request will be used to publish the cancellation key to other proxy instances. +pub struct CancellationHandler

{ map: CancelMap, - redis_client: Option>>, + client: P, + /// This field used for the monitoring purposes. + /// Represents the source of the cancellation request. + from: &'static str, } #[derive(Debug, Error)] @@ -44,49 +51,9 @@ impl ReportableError for CancelError { } } -impl CancellationHandler { - pub fn new(map: CancelMap, redis_client: Option>>) -> Self { - Self { map, redis_client } - } - /// Cancel a running query for the corresponding connection. - pub async fn cancel_session( - &self, - key: CancelKeyData, - session_id: Uuid, - ) -> Result<(), CancelError> { - let from = "from_client"; - // NB: we should immediately release the lock after cloning the token. - let Some(cancel_closure) = self.map.get(&key).and_then(|x| x.clone()) else { - tracing::warn!("query cancellation key not found: {key}"); - if let Some(redis_client) = &self.redis_client { - NUM_CANCELLATION_REQUESTS - .with_label_values(&[from, "not_found"]) - .inc(); - info!("publishing cancellation key to Redis"); - match redis_client.lock().await.try_publish(key, session_id).await { - Ok(()) => { - info!("cancellation key successfuly published to Redis"); - } - Err(e) => { - tracing::error!("failed to publish a message: {e}"); - return Err(CancelError::IO(std::io::Error::new( - std::io::ErrorKind::Other, - e.to_string(), - ))); - } - } - } - return Ok(()); - }; - NUM_CANCELLATION_REQUESTS - .with_label_values(&[from, "found"]) - .inc(); - info!("cancelling query per user's request using key {key}"); - cancel_closure.try_cancel_query().await - } - +impl CancellationHandler

{ /// Run async action within an ephemeral session identified by [`CancelKeyData`]. - pub fn get_session(self: Arc) -> Session { + pub fn get_session(self: Arc) -> Session

{ // HACK: We'd rather get the real backend_pid but tokio_postgres doesn't // expose it and we don't want to do another roundtrip to query // for it. The client will be able to notice that this is not the @@ -112,9 +79,39 @@ impl CancellationHandler { cancellation_handler: self, } } + /// Try to cancel a running query for the corresponding connection. + /// If the cancellation key is not found, it will be published to Redis. + pub async fn cancel_session( + &self, + key: CancelKeyData, + session_id: Uuid, + ) -> Result<(), CancelError> { + // NB: we should immediately release the lock after cloning the token. + let Some(cancel_closure) = self.map.get(&key).and_then(|x| x.clone()) else { + tracing::warn!("query cancellation key not found: {key}"); + NUM_CANCELLATION_REQUESTS + .with_label_values(&[self.from, "not_found"]) + .inc(); + match self.client.try_publish(key, session_id).await { + Ok(()) => {} // do nothing + Err(e) => { + return Err(CancelError::IO(std::io::Error::new( + std::io::ErrorKind::Other, + e.to_string(), + ))); + } + } + return Ok(()); + }; + NUM_CANCELLATION_REQUESTS + .with_label_values(&[self.from, "found"]) + .inc(); + info!("cancelling query per user's request using key {key}"); + cancel_closure.try_cancel_query().await + } #[cfg(test)] - fn contains(&self, session: &Session) -> bool { + fn contains(&self, session: &Session

) -> bool { self.map.contains_key(&session.key) } @@ -124,31 +121,19 @@ impl CancellationHandler { } } -#[async_trait] -pub trait NotificationsCancellationHandler { - async fn cancel_session_no_publish(&self, key: CancelKeyData) -> Result<(), CancelError>; +impl CancellationHandler<()> { + pub fn new(map: CancelMap, from: &'static str) -> Self { + Self { + map, + client: (), + from, + } + } } -#[async_trait] -impl NotificationsCancellationHandler for CancellationHandler { - async fn cancel_session_no_publish(&self, key: CancelKeyData) -> Result<(), CancelError> { - let from = "from_redis"; - let cancel_closure = self.map.get(&key).and_then(|x| x.clone()); - match cancel_closure { - Some(cancel_closure) => { - NUM_CANCELLATION_REQUESTS - .with_label_values(&[from, "found"]) - .inc(); - cancel_closure.try_cancel_query().await - } - None => { - NUM_CANCELLATION_REQUESTS - .with_label_values(&[from, "not_found"]) - .inc(); - tracing::warn!("query cancellation key not found: {key}"); - Ok(()) - } - } +impl CancellationHandler>>> { + pub fn new(map: CancelMap, client: Option>>, from: &'static str) -> Self { + Self { map, client, from } } } @@ -178,14 +163,14 @@ impl CancelClosure { } /// Helper for registering query cancellation tokens. -pub struct Session { +pub struct Session

{ /// The user-facing key identifying this session. key: CancelKeyData, /// The [`CancelMap`] this session belongs to. - cancellation_handler: Arc, + cancellation_handler: Arc>, } -impl Session { +impl

Session

{ /// Store the cancel token for the given session. /// This enables query cancellation in `crate::proxy::prepare_client_connection`. pub fn enable_query_cancellation(&self, cancel_closure: CancelClosure) -> CancelKeyData { @@ -198,7 +183,7 @@ impl Session { } } -impl Drop for Session { +impl

Drop for Session

{ fn drop(&mut self) { self.cancellation_handler.map.remove(&self.key); info!("dropped query cancellation key {}", &self.key); @@ -207,14 +192,16 @@ impl Drop for Session { #[cfg(test)] mod tests { + use crate::metrics::NUM_CANCELLATION_REQUESTS_SOURCE_FROM_REDIS; + use super::*; #[tokio::test] async fn check_session_drop() -> anyhow::Result<()> { - let cancellation_handler = Arc::new(CancellationHandler { - map: CancelMap::default(), - redis_client: None, - }); + let cancellation_handler = Arc::new(CancellationHandler::<()>::new( + CancelMap::default(), + NUM_CANCELLATION_REQUESTS_SOURCE_FROM_REDIS, + )); let session = cancellation_handler.clone().get_session(); assert!(cancellation_handler.contains(&session)); diff --git a/proxy/src/config.rs b/proxy/src/config.rs index 437ec9f401ed..45f8d7614439 100644 --- a/proxy/src/config.rs +++ b/proxy/src/config.rs @@ -28,6 +28,7 @@ pub struct ProxyConfig { pub redis_rps_limit: Vec, pub region: String, pub handshake_timeout: Duration, + pub aws_region: String, } #[derive(Debug)] diff --git a/proxy/src/metrics.rs b/proxy/src/metrics.rs index 02ebcd6aaaf3..eed45e421b8f 100644 --- a/proxy/src/metrics.rs +++ b/proxy/src/metrics.rs @@ -161,6 +161,9 @@ pub static NUM_CANCELLATION_REQUESTS: Lazy = Lazy::new(|| { .unwrap() }); +pub const NUM_CANCELLATION_REQUESTS_SOURCE_FROM_CLIENT: &str = "from_client"; +pub const NUM_CANCELLATION_REQUESTS_SOURCE_FROM_REDIS: &str = "from_redis"; + pub enum Waiting { Cplane, Client, diff --git a/proxy/src/proxy.rs b/proxy/src/proxy.rs index ab5bf5d49498..843bfc08cfa3 100644 --- a/proxy/src/proxy.rs +++ b/proxy/src/proxy.rs @@ -10,7 +10,7 @@ pub mod wake_compute; use crate::{ auth, - cancellation::{self, CancellationHandler}, + cancellation::{self, CancellationHandlerMain, CancellationHandlerMainInternal}, compute, config::{ProxyConfig, TlsConfig}, context::RequestMonitoring, @@ -62,7 +62,7 @@ pub async fn task_main( listener: tokio::net::TcpListener, cancellation_token: CancellationToken, endpoint_rate_limiter: Arc, - cancellation_handler: Arc, + cancellation_handler: Arc, ) -> anyhow::Result<()> { scopeguard::defer! { info!("proxy has shut down"); @@ -233,12 +233,12 @@ impl ReportableError for ClientRequestError { pub async fn handle_client( config: &'static ProxyConfig, ctx: &mut RequestMonitoring, - cancellation_handler: Arc, + cancellation_handler: Arc, stream: S, mode: ClientMode, endpoint_rate_limiter: Arc, conn_gauge: IntCounterPairGuard, -) -> Result>, ClientRequestError> { +) -> Result>, ClientRequestError> { info!("handling interactive connection from client"); let proto = ctx.protocol; @@ -338,9 +338,9 @@ pub async fn handle_client( /// Finish client connection initialization: confirm auth success, send params, etc. #[tracing::instrument(skip_all)] -async fn prepare_client_connection( +async fn prepare_client_connection

( node: &compute::PostgresConnection, - session: &cancellation::Session, + session: &cancellation::Session

, stream: &mut PqStream, ) -> Result<(), std::io::Error> { // Register compute's query cancellation token and produce a new, unique one. diff --git a/proxy/src/proxy/passthrough.rs b/proxy/src/proxy/passthrough.rs index b2f682fd2feb..f6d43143915a 100644 --- a/proxy/src/proxy/passthrough.rs +++ b/proxy/src/proxy/passthrough.rs @@ -55,17 +55,17 @@ pub async fn proxy_pass( Ok(()) } -pub struct ProxyPassthrough { +pub struct ProxyPassthrough { pub client: Stream, pub compute: PostgresConnection, pub aux: MetricsAuxInfo, pub req: IntCounterPairGuard, pub conn: IntCounterPairGuard, - pub cancel: cancellation::Session, + pub cancel: cancellation::Session

, } -impl ProxyPassthrough { +impl ProxyPassthrough { pub async fn proxy_pass(self) -> anyhow::Result<()> { let res = proxy_pass(self.client, self.compute.stream, self.aux).await; self.compute.cancel_closure.try_cancel_query().await?; diff --git a/proxy/src/redis.rs b/proxy/src/redis.rs index 35d6db074ed2..a322f0368ce2 100644 --- a/proxy/src/redis.rs +++ b/proxy/src/redis.rs @@ -1,2 +1,4 @@ +pub mod cancellation_publisher; +pub mod connection_with_credentials_provider; +pub mod elasticache; pub mod notifications; -pub mod publisher; diff --git a/proxy/src/redis/cancellation_publisher.rs b/proxy/src/redis/cancellation_publisher.rs new file mode 100644 index 000000000000..d9efc3561bfc --- /dev/null +++ b/proxy/src/redis/cancellation_publisher.rs @@ -0,0 +1,167 @@ +use std::sync::Arc; + +use async_trait::async_trait; +use pq_proto::CancelKeyData; +use redis::AsyncCommands; +use tokio::sync::Mutex; +use uuid::Uuid; + +use crate::rate_limiter::{RateBucketInfo, RedisRateLimiter}; + +use super::{ + connection_with_credentials_provider::ConnectionWithCredentialsProvider, + notifications::{CancelSession, Notification, PROXY_CHANNEL_NAME}, +}; + +#[async_trait] +pub trait CancellationPublisherMut: Send + Sync + 'static { + async fn try_publish( + &mut self, + cancel_key_data: CancelKeyData, + session_id: Uuid, + ) -> anyhow::Result<()>; +} + +#[async_trait] +pub trait CancellationPublisher: Send + Sync + 'static { + async fn try_publish( + &self, + cancel_key_data: CancelKeyData, + session_id: Uuid, + ) -> anyhow::Result<()>; +} + +#[async_trait] +impl CancellationPublisherMut for () { + async fn try_publish( + &mut self, + _cancel_key_data: CancelKeyData, + _session_id: Uuid, + ) -> anyhow::Result<()> { + Ok(()) + } +} + +#[async_trait] +impl CancellationPublisher for P { + async fn try_publish( + &self, + _cancel_key_data: CancelKeyData, + _session_id: Uuid, + ) -> anyhow::Result<()> { + self.try_publish(_cancel_key_data, _session_id).await + } +} + +#[async_trait] +impl CancellationPublisher for Option

{ + async fn try_publish( + &self, + cancel_key_data: CancelKeyData, + session_id: Uuid, + ) -> anyhow::Result<()> { + if let Some(p) = self { + p.try_publish(cancel_key_data, session_id).await + } else { + Ok(()) + } + } +} + +#[async_trait] +impl CancellationPublisher for Arc> { + async fn try_publish( + &self, + cancel_key_data: CancelKeyData, + session_id: Uuid, + ) -> anyhow::Result<()> { + self.lock() + .await + .try_publish(cancel_key_data, session_id) + .await + } +} + +pub struct RedisPublisherClient { + client: ConnectionWithCredentialsProvider, + region_id: String, + limiter: RedisRateLimiter, +} + +impl RedisPublisherClient { + pub fn new( + client: ConnectionWithCredentialsProvider, + region_id: String, + info: &'static [RateBucketInfo], + ) -> anyhow::Result { + Ok(Self { + client, + region_id, + limiter: RedisRateLimiter::new(info), + }) + } + + async fn publish( + &mut self, + cancel_key_data: CancelKeyData, + session_id: Uuid, + ) -> anyhow::Result<()> { + let payload = serde_json::to_string(&Notification::Cancel(CancelSession { + region_id: Some(self.region_id.clone()), + cancel_key_data, + session_id, + }))?; + self.client.publish(PROXY_CHANNEL_NAME, payload).await?; + Ok(()) + } + pub async fn try_connect(&mut self) -> anyhow::Result<()> { + match self.client.connect().await { + Ok(()) => {} + Err(e) => { + tracing::error!("failed to connect to redis: {e}"); + return Err(e); + } + } + Ok(()) + } + async fn try_publish_internal( + &mut self, + cancel_key_data: CancelKeyData, + session_id: Uuid, + ) -> anyhow::Result<()> { + if !self.limiter.check() { + tracing::info!("Rate limit exceeded. Skipping cancellation message"); + return Err(anyhow::anyhow!("Rate limit exceeded")); + } + match self.publish(cancel_key_data, session_id).await { + Ok(()) => return Ok(()), + Err(e) => { + tracing::error!("failed to publish a message: {e}"); + } + } + tracing::info!("Publisher is disconnected. Reconnectiong..."); + self.try_connect().await?; + self.publish(cancel_key_data, session_id).await + } +} + +#[async_trait] +impl CancellationPublisherMut for RedisPublisherClient { + async fn try_publish( + &mut self, + cancel_key_data: CancelKeyData, + session_id: Uuid, + ) -> anyhow::Result<()> { + tracing::info!("publishing cancellation key to Redis"); + match self.try_publish_internal(cancel_key_data, session_id).await { + Ok(()) => { + tracing::info!("cancellation key successfuly published to Redis"); + Ok(()) + } + Err(e) => { + tracing::error!("failed to publish a message: {e}"); + Err(e) + } + } + } +} diff --git a/proxy/src/redis/connection_with_credentials_provider.rs b/proxy/src/redis/connection_with_credentials_provider.rs new file mode 100644 index 000000000000..d183abb53a2f --- /dev/null +++ b/proxy/src/redis/connection_with_credentials_provider.rs @@ -0,0 +1,225 @@ +use std::{sync::Arc, time::Duration}; + +use futures::FutureExt; +use redis::{ + aio::{ConnectionLike, MultiplexedConnection}, + ConnectionInfo, IntoConnectionInfo, RedisConnectionInfo, RedisResult, +}; +use tokio::task::JoinHandle; +use tracing::{error, info}; + +use super::elasticache::CredentialsProvider; + +enum Credentials { + Static(ConnectionInfo), + Dynamic(Arc, redis::ConnectionAddr), +} + +impl Clone for Credentials { + fn clone(&self) -> Self { + match self { + Credentials::Static(info) => Credentials::Static(info.clone()), + Credentials::Dynamic(provider, addr) => { + Credentials::Dynamic(Arc::clone(provider), addr.clone()) + } + } + } +} + +/// A wrapper around `redis::MultiplexedConnection` that automatically refreshes the token. +/// Provides PubSub connection without credentials refresh. +pub struct ConnectionWithCredentialsProvider { + credentials: Credentials, + con: Option, + refresh_token_task: Option>, + mutex: tokio::sync::Mutex<()>, +} + +impl Clone for ConnectionWithCredentialsProvider { + fn clone(&self) -> Self { + Self { + credentials: self.credentials.clone(), + con: None, + refresh_token_task: None, + mutex: tokio::sync::Mutex::new(()), + } + } +} + +impl ConnectionWithCredentialsProvider { + pub fn new_with_credentials_provider( + host: String, + port: u16, + credentials_provider: Arc, + ) -> Self { + Self { + credentials: Credentials::Dynamic( + credentials_provider, + redis::ConnectionAddr::TcpTls { + host, + port, + insecure: false, + tls_params: None, + }, + ), + con: None, + refresh_token_task: None, + mutex: tokio::sync::Mutex::new(()), + } + } + + pub fn new_with_static_credentials(params: T) -> Self { + Self { + credentials: Credentials::Static(params.into_connection_info().unwrap()), + con: None, + refresh_token_task: None, + mutex: tokio::sync::Mutex::new(()), + } + } + + pub async fn connect(&mut self) -> anyhow::Result<()> { + let _guard = self.mutex.lock().await; + if let Some(con) = self.con.as_mut() { + match redis::cmd("PING").query_async(con).await { + Ok(()) => { + return Ok(()); + } + Err(e) => { + error!("Error during PING: {e:?}"); + } + } + } else { + info!("Connection is not established"); + } + info!("Establishing a new connection..."); + self.con = None; + if let Some(f) = self.refresh_token_task.take() { + f.abort() + } + let con = self + .get_client() + .await? + .get_multiplexed_tokio_connection() + .await?; + if let Credentials::Dynamic(credentials_provider, _) = &self.credentials { + let credentials_provider = credentials_provider.clone(); + let con2 = con.clone(); + let f = tokio::spawn(async move { + let _ = Self::keep_connection(con2, credentials_provider).await; + }); + self.refresh_token_task = Some(f); + } + self.con = Some(con); + Ok(()) + } + + async fn get_connection_info(&self) -> anyhow::Result { + match &self.credentials { + Credentials::Static(info) => Ok(info.clone()), + Credentials::Dynamic(provider, addr) => { + let (username, password) = provider.provide_credentials().await?; + Ok(ConnectionInfo { + addr: addr.clone(), + redis: RedisConnectionInfo { + db: 0, + username: Some(username), + password: Some(password.clone()), + }, + }) + } + } + } + + async fn get_client(&self) -> anyhow::Result { + let client = redis::Client::open(self.get_connection_info().await?)?; + Ok(client) + } + + // PubSub does not support credentials refresh. + // Requires manual reconnection every 12h. + pub async fn get_async_pubsub(&self) -> anyhow::Result { + Ok(self.get_client().await?.get_async_pubsub().await?) + } + + // The connection lives for 12h. + // It can be prolonged with sending `AUTH` commands with the refreshed token. + // https://docs.aws.amazon.com/AmazonElastiCache/latest/red-ug/auth-iam.html#auth-iam-limits + async fn keep_connection( + mut con: MultiplexedConnection, + credentials_provider: Arc, + ) -> anyhow::Result<()> { + loop { + // The connection lives for 12h, for the sanity check we refresh it every hour. + tokio::time::sleep(Duration::from_secs(60 * 60)).await; + match Self::refresh_token(&mut con, credentials_provider.clone()).await { + Ok(()) => { + info!("Token refreshed"); + } + Err(e) => { + error!("Error during token refresh: {e:?}"); + } + } + } + } + async fn refresh_token( + con: &mut MultiplexedConnection, + credentials_provider: Arc, + ) -> anyhow::Result<()> { + let (user, password) = credentials_provider.provide_credentials().await?; + redis::cmd("AUTH") + .arg(user) + .arg(password) + .query_async(con) + .await?; + Ok(()) + } + /// Sends an already encoded (packed) command into the TCP socket and + /// reads the single response from it. + pub async fn send_packed_command(&mut self, cmd: &redis::Cmd) -> RedisResult { + // Clone connection to avoid having to lock the ArcSwap in write mode + let con = self.con.as_mut().ok_or(redis::RedisError::from(( + redis::ErrorKind::IoError, + "Connection not established", + )))?; + con.send_packed_command(cmd).await + } + + /// Sends multiple already encoded (packed) command into the TCP socket + /// and reads `count` responses from it. This is used to implement + /// pipelining. + pub async fn send_packed_commands( + &mut self, + cmd: &redis::Pipeline, + offset: usize, + count: usize, + ) -> RedisResult> { + // Clone shared connection future to avoid having to lock the ArcSwap in write mode + let con = self.con.as_mut().ok_or(redis::RedisError::from(( + redis::ErrorKind::IoError, + "Connection not established", + )))?; + con.send_packed_commands(cmd, offset, count).await + } +} + +impl ConnectionLike for ConnectionWithCredentialsProvider { + fn req_packed_command<'a>( + &'a mut self, + cmd: &'a redis::Cmd, + ) -> redis::RedisFuture<'a, redis::Value> { + (async move { self.send_packed_command(cmd).await }).boxed() + } + + fn req_packed_commands<'a>( + &'a mut self, + cmd: &'a redis::Pipeline, + offset: usize, + count: usize, + ) -> redis::RedisFuture<'a, Vec> { + (async move { self.send_packed_commands(cmd, offset, count).await }).boxed() + } + + fn get_db(&self) -> i64 { + 0 + } +} diff --git a/proxy/src/redis/elasticache.rs b/proxy/src/redis/elasticache.rs new file mode 100644 index 000000000000..eded8250afcd --- /dev/null +++ b/proxy/src/redis/elasticache.rs @@ -0,0 +1,110 @@ +use std::time::{Duration, SystemTime}; + +use aws_config::meta::credentials::CredentialsProviderChain; +use aws_sdk_iam::config::ProvideCredentials; +use aws_sigv4::http_request::{ + self, SignableBody, SignableRequest, SignatureLocation, SigningSettings, +}; +use tracing::info; + +#[derive(Debug)] +pub struct AWSIRSAConfig { + region: String, + service_name: String, + cluster_name: String, + user_id: String, + token_ttl: Duration, + action: String, +} + +impl AWSIRSAConfig { + pub fn new(region: String, cluster_name: Option, user_id: Option) -> Self { + AWSIRSAConfig { + region, + service_name: "elasticache".to_string(), + cluster_name: cluster_name.unwrap_or_default(), + user_id: user_id.unwrap_or_default(), + // "The IAM authentication token is valid for 15 minutes" + // https://docs.aws.amazon.com/memorydb/latest/devguide/auth-iam.html#auth-iam-limits + token_ttl: Duration::from_secs(15 * 60), + action: "connect".to_string(), + } + } +} + +/// Credentials provider for AWS elasticache authentication. +/// +/// Official documentation: +/// +/// +/// Useful resources: +/// +pub struct CredentialsProvider { + config: AWSIRSAConfig, + credentials_provider: CredentialsProviderChain, +} + +impl CredentialsProvider { + pub fn new(config: AWSIRSAConfig, credentials_provider: CredentialsProviderChain) -> Self { + CredentialsProvider { + config, + credentials_provider, + } + } + pub async fn provide_credentials(&self) -> anyhow::Result<(String, String)> { + let aws_credentials = self + .credentials_provider + .provide_credentials() + .await? + .into(); + info!("AWS credentials successfully obtained"); + info!("Connecting to Redis with configuration: {:?}", self.config); + let mut settings = SigningSettings::default(); + settings.signature_location = SignatureLocation::QueryParams; + settings.expires_in = Some(self.config.token_ttl); + let signing_params = aws_sigv4::sign::v4::SigningParams::builder() + .identity(&aws_credentials) + .region(&self.config.region) + .name(&self.config.service_name) + .time(SystemTime::now()) + .settings(settings) + .build()? + .into(); + let auth_params = [ + ("Action", &self.config.action), + ("User", &self.config.user_id), + ]; + let auth_params = url::form_urlencoded::Serializer::new(String::new()) + .extend_pairs(auth_params) + .finish(); + let auth_uri = http::Uri::builder() + .scheme("http") + .authority(self.config.cluster_name.as_bytes()) + .path_and_query(format!("/?{auth_params}")) + .build()?; + info!("{}", auth_uri); + + // Convert the HTTP request into a signable request + let signable_request = SignableRequest::new( + "GET", + auth_uri.to_string(), + std::iter::empty(), + SignableBody::Bytes(&[]), + )?; + + // Sign and then apply the signature to the request + let (si, _) = http_request::sign(signable_request, &signing_params)?.into_parts(); + let mut signable_request = http::Request::builder() + .method("GET") + .uri(auth_uri) + .body(())?; + si.apply_to_request_http1x(&mut signable_request); + Ok(( + self.config.user_id.clone(), + signable_request + .uri() + .to_string() + .replacen("http://", "", 1), + )) + } +} diff --git a/proxy/src/redis/notifications.rs b/proxy/src/redis/notifications.rs index 6ae848c0d2c7..8b7e3e3419a0 100644 --- a/proxy/src/redis/notifications.rs +++ b/proxy/src/redis/notifications.rs @@ -6,11 +6,12 @@ use redis::aio::PubSub; use serde::{Deserialize, Serialize}; use uuid::Uuid; +use super::connection_with_credentials_provider::ConnectionWithCredentialsProvider; use crate::{ cache::project_info::ProjectInfoCache, - cancellation::{CancelMap, CancellationHandler, NotificationsCancellationHandler}, + cancellation::{CancelMap, CancellationHandler}, intern::{ProjectIdInt, RoleNameInt}, - metrics::REDIS_BROKEN_MESSAGES, + metrics::{NUM_CANCELLATION_REQUESTS_SOURCE_FROM_REDIS, REDIS_BROKEN_MESSAGES}, }; const CPLANE_CHANNEL_NAME: &str = "neondb-proxy-ws-updates"; @@ -18,23 +19,13 @@ pub(crate) const PROXY_CHANNEL_NAME: &str = "neondb-proxy-to-proxy-updates"; const RECONNECT_TIMEOUT: std::time::Duration = std::time::Duration::from_secs(20); const INVALIDATION_LAG: std::time::Duration = std::time::Duration::from_secs(20); -struct RedisConsumerClient { - client: redis::Client, -} - -impl RedisConsumerClient { - pub fn new(url: &str) -> anyhow::Result { - let client = redis::Client::open(url)?; - Ok(Self { client }) - } - async fn try_connect(&self) -> anyhow::Result { - let mut conn = self.client.get_async_connection().await?.into_pubsub(); - tracing::info!("subscribing to a channel `{CPLANE_CHANNEL_NAME}`"); - conn.subscribe(CPLANE_CHANNEL_NAME).await?; - tracing::info!("subscribing to a channel `{PROXY_CHANNEL_NAME}`"); - conn.subscribe(PROXY_CHANNEL_NAME).await?; - Ok(conn) - } +async fn try_connect(client: &ConnectionWithCredentialsProvider) -> anyhow::Result { + let mut conn = client.get_async_pubsub().await?; + tracing::info!("subscribing to a channel `{CPLANE_CHANNEL_NAME}`"); + conn.subscribe(CPLANE_CHANNEL_NAME).await?; + tracing::info!("subscribing to a channel `{PROXY_CHANNEL_NAME}`"); + conn.subscribe(PROXY_CHANNEL_NAME).await?; + Ok(conn) } #[derive(Clone, Debug, Serialize, Deserialize, Eq, PartialEq)] @@ -80,21 +71,18 @@ where serde_json::from_str(&s).map_err(::custom) } -struct MessageHandler< - C: ProjectInfoCache + Send + Sync + 'static, - H: NotificationsCancellationHandler + Send + Sync + 'static, -> { +struct MessageHandler { cache: Arc, - cancellation_handler: Arc, + cancellation_handler: Arc>, region_id: String, } -impl< - C: ProjectInfoCache + Send + Sync + 'static, - H: NotificationsCancellationHandler + Send + Sync + 'static, - > MessageHandler -{ - pub fn new(cache: Arc, cancellation_handler: Arc, region_id: String) -> Self { +impl MessageHandler { + pub fn new( + cache: Arc, + cancellation_handler: Arc>, + region_id: String, + ) -> Self { Self { cache, cancellation_handler, @@ -139,7 +127,7 @@ impl< // This instance of cancellation_handler doesn't have a RedisPublisherClient so it can't publish the message. match self .cancellation_handler - .cancel_session_no_publish(cancel_session.cancel_key_data) + .cancel_session(cancel_session.cancel_key_data, uuid::Uuid::nil()) .await { Ok(()) => {} @@ -182,7 +170,7 @@ fn invalidate_cache(cache: Arc, msg: Notification) { /// Handle console's invalidation messages. #[tracing::instrument(name = "console_notifications", skip_all)] pub async fn task_main( - url: String, + redis: ConnectionWithCredentialsProvider, cache: Arc, cancel_map: CancelMap, region_id: String, @@ -193,13 +181,15 @@ where cache.enable_ttl(); let handler = MessageHandler::new( cache, - Arc::new(CancellationHandler::new(cancel_map, None)), + Arc::new(CancellationHandler::<()>::new( + cancel_map, + NUM_CANCELLATION_REQUESTS_SOURCE_FROM_REDIS, + )), region_id, ); loop { - let redis = RedisConsumerClient::new(&url)?; - let conn = match redis.try_connect().await { + let mut conn = match try_connect(&redis).await { Ok(conn) => { handler.disable_ttl(); conn @@ -212,7 +202,7 @@ where continue; } }; - let mut stream = conn.into_on_message(); + let mut stream = conn.on_message(); while let Some(msg) = stream.next().await { match handler.handle_message(msg).await { Ok(()) => {} diff --git a/proxy/src/redis/publisher.rs b/proxy/src/redis/publisher.rs deleted file mode 100644 index f85593afdda0..000000000000 --- a/proxy/src/redis/publisher.rs +++ /dev/null @@ -1,80 +0,0 @@ -use pq_proto::CancelKeyData; -use redis::AsyncCommands; -use uuid::Uuid; - -use crate::rate_limiter::{RateBucketInfo, RedisRateLimiter}; - -use super::notifications::{CancelSession, Notification, PROXY_CHANNEL_NAME}; - -pub struct RedisPublisherClient { - client: redis::Client, - publisher: Option, - region_id: String, - limiter: RedisRateLimiter, -} - -impl RedisPublisherClient { - pub fn new( - url: &str, - region_id: String, - info: &'static [RateBucketInfo], - ) -> anyhow::Result { - let client = redis::Client::open(url)?; - Ok(Self { - client, - publisher: None, - region_id, - limiter: RedisRateLimiter::new(info), - }) - } - pub async fn try_publish( - &mut self, - cancel_key_data: CancelKeyData, - session_id: Uuid, - ) -> anyhow::Result<()> { - if !self.limiter.check() { - tracing::info!("Rate limit exceeded. Skipping cancellation message"); - return Err(anyhow::anyhow!("Rate limit exceeded")); - } - match self.publish(cancel_key_data, session_id).await { - Ok(()) => return Ok(()), - Err(e) => { - tracing::error!("failed to publish a message: {e}"); - self.publisher = None; - } - } - tracing::info!("Publisher is disconnected. Reconnectiong..."); - self.try_connect().await?; - self.publish(cancel_key_data, session_id).await - } - - async fn publish( - &mut self, - cancel_key_data: CancelKeyData, - session_id: Uuid, - ) -> anyhow::Result<()> { - let conn = self - .publisher - .as_mut() - .ok_or_else(|| anyhow::anyhow!("not connected"))?; - let payload = serde_json::to_string(&Notification::Cancel(CancelSession { - region_id: Some(self.region_id.clone()), - cancel_key_data, - session_id, - }))?; - conn.publish(PROXY_CHANNEL_NAME, payload).await?; - Ok(()) - } - pub async fn try_connect(&mut self) -> anyhow::Result<()> { - match self.client.get_async_connection().await { - Ok(conn) => { - self.publisher = Some(conn); - } - Err(e) => { - tracing::error!("failed to connect to redis: {e}"); - return Err(e.into()); - } - } - Ok(()) - } -} diff --git a/proxy/src/serverless.rs b/proxy/src/serverless.rs index be9f90acde7b..a2010fd61325 100644 --- a/proxy/src/serverless.rs +++ b/proxy/src/serverless.rs @@ -21,11 +21,12 @@ pub use reqwest_retry::{policies::ExponentialBackoff, RetryTransientMiddleware}; use tokio_util::task::TaskTracker; use tracing::instrument::Instrumented; +use crate::cancellation::CancellationHandlerMain; +use crate::config::ProxyConfig; use crate::context::RequestMonitoring; use crate::protocol2::{ProxyProtocolAccept, WithClientIp, WithConnectionGuard}; use crate::rate_limiter::EndpointRateLimiter; use crate::serverless::backend::PoolingBackend; -use crate::{cancellation::CancellationHandler, config::ProxyConfig}; use hyper::{ server::conn::{AddrIncoming, AddrStream}, Body, Method, Request, Response, @@ -47,7 +48,7 @@ pub async fn task_main( ws_listener: TcpListener, cancellation_token: CancellationToken, endpoint_rate_limiter: Arc, - cancellation_handler: Arc, + cancellation_handler: Arc, ) -> anyhow::Result<()> { scopeguard::defer! { info!("websocket server has shut down"); @@ -237,7 +238,7 @@ async fn request_handler( config: &'static ProxyConfig, backend: Arc, ws_connections: TaskTracker, - cancellation_handler: Arc, + cancellation_handler: Arc, peer_addr: IpAddr, endpoint_rate_limiter: Arc, // used to cancel in-flight HTTP requests. not used to cancel websockets diff --git a/proxy/src/serverless/websocket.rs b/proxy/src/serverless/websocket.rs index a72ede6d0a1c..ada6c974f41d 100644 --- a/proxy/src/serverless/websocket.rs +++ b/proxy/src/serverless/websocket.rs @@ -1,5 +1,5 @@ use crate::{ - cancellation::CancellationHandler, + cancellation::CancellationHandlerMain, config::ProxyConfig, context::RequestMonitoring, error::{io_error, ReportableError}, @@ -134,7 +134,7 @@ pub async fn serve_websocket( config: &'static ProxyConfig, mut ctx: RequestMonitoring, websocket: HyperWebsocket, - cancellation_handler: Arc, + cancellation_handler: Arc, hostname: Option, endpoint_rate_limiter: Arc, ) -> anyhow::Result<()> { diff --git a/workspace_hack/Cargo.toml b/workspace_hack/Cargo.toml index 152c452dd4ee..7b8228a08296 100644 --- a/workspace_hack/Cargo.toml +++ b/workspace_hack/Cargo.toml @@ -19,8 +19,7 @@ aws-runtime = { version = "1", default-features = false, features = ["event-stre aws-sigv4 = { version = "1", features = ["http0-compat", "sign-eventstream", "sigv4a"] } aws-smithy-async = { version = "1", default-features = false, features = ["rt-tokio"] } aws-smithy-http = { version = "0.60", default-features = false, features = ["event-stream"] } -aws-smithy-runtime-api = { version = "1", features = ["client", "http-02x", "http-auth"] } -aws-smithy-types = { version = "1", default-features = false, features = ["byte-stream-poll-next", "http-body-0-4-x", "rt-tokio"] } +aws-smithy-types = { version = "1", default-features = false, features = ["byte-stream-poll-next", "http-body-0-4-x", "rt-tokio", "test-util"] } axum = { version = "0.6", features = ["ws"] } base64 = { version = "0.21", features = ["alloc"] } base64ct = { version = "1", default-features = false, features = ["std"] } From 62b318c928f365827039022e900bd6c80928792e Mon Sep 17 00:00:00 2001 From: John Spray Date: Fri, 22 Mar 2024 10:10:28 +0000 Subject: [PATCH 27/53] Fix ephemeral file warning on secondaries (#7201) A test was added which exercises secondary locations more, and there was a location in the secondary downloader that warned on ephemeral files. This was intended to be fixed in this faulty commit: https://github.com/neondatabase/neon/pull/7169/commits/8cea866adf15c3086dc16e5fa62f59d5604fdf1e --- pageserver/src/tenant/secondary/downloader.rs | 4 ++-- test_runner/regress/test_pageserver_metric_collection.py | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/pageserver/src/tenant/secondary/downloader.rs b/pageserver/src/tenant/secondary/downloader.rs index 40f19e3b0551..8782a9f04ef9 100644 --- a/pageserver/src/tenant/secondary/downloader.rs +++ b/pageserver/src/tenant/secondary/downloader.rs @@ -11,11 +11,11 @@ use crate::{ disk_usage_eviction_task::{ finite_f32, DiskUsageEvictionInfo, EvictionCandidate, EvictionLayer, EvictionSecondaryLayer, }, - is_temporary, metrics::SECONDARY_MODE, tenant::{ config::SecondaryLocationConfig, debug_assert_current_span_has_tenant_and_timeline_id, + ephemeral_file::is_ephemeral_file, remote_timeline_client::{ index::LayerFileMetadata, is_temp_download_file, FAILED_DOWNLOAD_WARN_THRESHOLD, FAILED_REMOTE_OP_RETRIES, @@ -964,7 +964,7 @@ async fn init_timeline_state( continue; } else if crate::is_temporary(&file_path) || is_temp_download_file(&file_path) - || is_temporary(&file_path) + || is_ephemeral_file(file_name) { // Temporary files are frequently left behind from restarting during downloads tracing::info!("Cleaning up temporary file {file_path}"); diff --git a/test_runner/regress/test_pageserver_metric_collection.py b/test_runner/regress/test_pageserver_metric_collection.py index 042961baa5f1..5799d1119014 100644 --- a/test_runner/regress/test_pageserver_metric_collection.py +++ b/test_runner/regress/test_pageserver_metric_collection.py @@ -70,6 +70,7 @@ def metrics_handler(request: Request) -> Response: # we have a fast rate of calculation, these can happen at shutdown ".*synthetic_size_worker:calculate_synthetic_size.*:gather_size_inputs.*: failed to calculate logical size at .*: cancelled.*", ".*synthetic_size_worker: failed to calculate synthetic size for tenant .*: failed to calculate some logical_sizes", + ".*metrics_collection: failed to upload to S3: Failed to upload data of length .* to storage path.*", ] ) From 77f3a30440aba4845da3a5203a2764fed4d96648 Mon Sep 17 00:00:00 2001 From: Conrad Ludgate Date: Fri, 22 Mar 2024 13:31:10 +0000 Subject: [PATCH 28/53] proxy: unit tests for auth_quirks (#7199) ## Problem I noticed code coverage for auth_quirks was pretty bare ## Summary of changes Adds 3 happy path unit tests for auth_quirks * scram * cleartext (websockets) * cleartext (password hack) --- Cargo.lock | 1 + Cargo.toml | 1 + proxy/Cargo.toml | 1 + proxy/src/auth/backend.rs | 225 +++++++++++++++++++++++++++++ proxy/src/compute.rs | 11 +- proxy/src/console.rs | 2 +- proxy/src/console/provider.rs | 5 +- proxy/src/console/provider/mock.rs | 2 - proxy/src/console/provider/neon.rs | 2 - proxy/src/scram/exchange.rs | 30 ++-- proxy/src/scram/key.rs | 16 +- proxy/src/scram/messages.rs | 22 +++ proxy/src/scram/secret.rs | 7 + 13 files changed, 286 insertions(+), 39 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index dcf1c4992461..6409c79ef95c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4237,6 +4237,7 @@ dependencies = [ "consumption_metrics", "dashmap", "env_logger", + "fallible-iterator", "futures", "git-version", "hashbrown 0.13.2", diff --git a/Cargo.toml b/Cargo.toml index 2741bd046bb6..4dda63ff584f 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -79,6 +79,7 @@ either = "1.8" enum-map = "2.4.2" enumset = "1.0.12" fail = "0.5.0" +fallible-iterator = "0.2" fs2 = "0.4.3" futures = "0.3" futures-core = "0.3" diff --git a/proxy/Cargo.toml b/proxy/Cargo.toml index 3566d8b7288e..57a2736d5b32 100644 --- a/proxy/Cargo.toml +++ b/proxy/Cargo.toml @@ -97,6 +97,7 @@ workspace_hack.workspace = true [dev-dependencies] camino-tempfile.workspace = true +fallible-iterator.workspace = true rcgen.workspace = true rstest.workspace = true tokio-postgres-rustls.workspace = true diff --git a/proxy/src/auth/backend.rs b/proxy/src/auth/backend.rs index bc307230dd17..04fe83d8ebc7 100644 --- a/proxy/src/auth/backend.rs +++ b/proxy/src/auth/backend.rs @@ -408,3 +408,228 @@ impl ComputeConnectBackend for BackendType<'_, ComputeCredentials, &()> { } } } + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use bytes::BytesMut; + use fallible_iterator::FallibleIterator; + use postgres_protocol::{ + authentication::sasl::{ChannelBinding, ScramSha256}, + message::{backend::Message as PgMessage, frontend}, + }; + use provider::AuthSecret; + use tokio::io::{AsyncRead, AsyncReadExt, AsyncWriteExt}; + + use crate::{ + auth::{ComputeUserInfoMaybeEndpoint, IpPattern}, + config::AuthenticationConfig, + console::{ + self, + provider::{self, CachedAllowedIps, CachedRoleSecret}, + CachedNodeInfo, + }, + context::RequestMonitoring, + proxy::NeonOptions, + scram::ServerSecret, + stream::{PqStream, Stream}, + }; + + use super::auth_quirks; + + struct Auth { + ips: Vec, + secret: AuthSecret, + } + + impl console::Api for Auth { + async fn get_role_secret( + &self, + _ctx: &mut RequestMonitoring, + _user_info: &super::ComputeUserInfo, + ) -> Result { + Ok(CachedRoleSecret::new_uncached(Some(self.secret.clone()))) + } + + async fn get_allowed_ips_and_secret( + &self, + _ctx: &mut RequestMonitoring, + _user_info: &super::ComputeUserInfo, + ) -> Result<(CachedAllowedIps, Option), console::errors::GetAuthInfoError> + { + Ok(( + CachedAllowedIps::new_uncached(Arc::new(self.ips.clone())), + Some(CachedRoleSecret::new_uncached(Some(self.secret.clone()))), + )) + } + + async fn wake_compute( + &self, + _ctx: &mut RequestMonitoring, + _user_info: &super::ComputeUserInfo, + ) -> Result { + unimplemented!() + } + } + + static CONFIG: &AuthenticationConfig = &AuthenticationConfig { + scram_protocol_timeout: std::time::Duration::from_secs(5), + }; + + async fn read_message(r: &mut (impl AsyncRead + Unpin), b: &mut BytesMut) -> PgMessage { + loop { + r.read_buf(&mut *b).await.unwrap(); + if let Some(m) = PgMessage::parse(&mut *b).unwrap() { + break m; + } + } + } + + #[tokio::test] + async fn auth_quirks_scram() { + let (mut client, server) = tokio::io::duplex(1024); + let mut stream = PqStream::new(Stream::from_raw(server)); + + let mut ctx = RequestMonitoring::test(); + let api = Auth { + ips: vec![], + secret: AuthSecret::Scram(ServerSecret::build("my-secret-password").await.unwrap()), + }; + + let user_info = ComputeUserInfoMaybeEndpoint { + user: "conrad".into(), + endpoint_id: Some("endpoint".into()), + options: NeonOptions::default(), + }; + + let handle = tokio::spawn(async move { + let mut scram = ScramSha256::new(b"my-secret-password", ChannelBinding::unsupported()); + + let mut read = BytesMut::new(); + + // server should offer scram + match read_message(&mut client, &mut read).await { + PgMessage::AuthenticationSasl(a) => { + let options: Vec<&str> = a.mechanisms().collect().unwrap(); + assert_eq!(options, ["SCRAM-SHA-256"]); + } + _ => panic!("wrong message"), + } + + // client sends client-first-message + let mut write = BytesMut::new(); + frontend::sasl_initial_response("SCRAM-SHA-256", scram.message(), &mut write).unwrap(); + client.write_all(&write).await.unwrap(); + + // server response with server-first-message + match read_message(&mut client, &mut read).await { + PgMessage::AuthenticationSaslContinue(a) => { + scram.update(a.data()).await.unwrap(); + } + _ => panic!("wrong message"), + } + + // client response with client-final-message + write.clear(); + frontend::sasl_response(scram.message(), &mut write).unwrap(); + client.write_all(&write).await.unwrap(); + + // server response with server-final-message + match read_message(&mut client, &mut read).await { + PgMessage::AuthenticationSaslFinal(a) => { + scram.finish(a.data()).unwrap(); + } + _ => panic!("wrong message"), + } + }); + + let _creds = auth_quirks(&mut ctx, &api, user_info, &mut stream, false, CONFIG) + .await + .unwrap(); + + handle.await.unwrap(); + } + + #[tokio::test] + async fn auth_quirks_cleartext() { + let (mut client, server) = tokio::io::duplex(1024); + let mut stream = PqStream::new(Stream::from_raw(server)); + + let mut ctx = RequestMonitoring::test(); + let api = Auth { + ips: vec![], + secret: AuthSecret::Scram(ServerSecret::build("my-secret-password").await.unwrap()), + }; + + let user_info = ComputeUserInfoMaybeEndpoint { + user: "conrad".into(), + endpoint_id: Some("endpoint".into()), + options: NeonOptions::default(), + }; + + let handle = tokio::spawn(async move { + let mut read = BytesMut::new(); + let mut write = BytesMut::new(); + + // server should offer cleartext + match read_message(&mut client, &mut read).await { + PgMessage::AuthenticationCleartextPassword => {} + _ => panic!("wrong message"), + } + + // client responds with password + write.clear(); + frontend::password_message(b"my-secret-password", &mut write).unwrap(); + client.write_all(&write).await.unwrap(); + }); + + let _creds = auth_quirks(&mut ctx, &api, user_info, &mut stream, true, CONFIG) + .await + .unwrap(); + + handle.await.unwrap(); + } + + #[tokio::test] + async fn auth_quirks_password_hack() { + let (mut client, server) = tokio::io::duplex(1024); + let mut stream = PqStream::new(Stream::from_raw(server)); + + let mut ctx = RequestMonitoring::test(); + let api = Auth { + ips: vec![], + secret: AuthSecret::Scram(ServerSecret::build("my-secret-password").await.unwrap()), + }; + + let user_info = ComputeUserInfoMaybeEndpoint { + user: "conrad".into(), + endpoint_id: None, + options: NeonOptions::default(), + }; + + let handle = tokio::spawn(async move { + let mut read = BytesMut::new(); + + // server should offer cleartext + match read_message(&mut client, &mut read).await { + PgMessage::AuthenticationCleartextPassword => {} + _ => panic!("wrong message"), + } + + // client responds with password + let mut write = BytesMut::new(); + frontend::password_message(b"endpoint=my-endpoint;my-secret-password", &mut write) + .unwrap(); + client.write_all(&write).await.unwrap(); + }); + + let creds = auth_quirks(&mut ctx, &api, user_info, &mut stream, true, CONFIG) + .await + .unwrap(); + + assert_eq!(creds.info.endpoint, "my-endpoint"); + + handle.await.unwrap(); + } +} diff --git a/proxy/src/compute.rs b/proxy/src/compute.rs index b61c1fb9ef92..65153babcb16 100644 --- a/proxy/src/compute.rs +++ b/proxy/src/compute.rs @@ -82,14 +82,13 @@ pub type ScramKeys = tokio_postgres::config::ScramKeys<32>; /// A config for establishing a connection to compute node. /// Eventually, `tokio_postgres` will be replaced with something better. /// Newtype allows us to implement methods on top of it. -#[derive(Clone)] -#[repr(transparent)] +#[derive(Clone, Default)] pub struct ConnCfg(Box); /// Creation and initialization routines. impl ConnCfg { pub fn new() -> Self { - Self(Default::default()) + Self::default() } /// Reuse password or auth keys from the other config. @@ -165,12 +164,6 @@ impl std::ops::DerefMut for ConnCfg { } } -impl Default for ConnCfg { - fn default() -> Self { - Self::new() - } -} - impl ConnCfg { /// Establish a raw TCP connection to the compute node. async fn connect_raw(&self, timeout: Duration) -> io::Result<(SocketAddr, TcpStream, &str)> { diff --git a/proxy/src/console.rs b/proxy/src/console.rs index fd3c46b946aa..ea95e8343710 100644 --- a/proxy/src/console.rs +++ b/proxy/src/console.rs @@ -6,7 +6,7 @@ pub mod messages; /// Wrappers for console APIs and their mocks. pub mod provider; -pub use provider::{errors, Api, AuthSecret, CachedNodeInfo, NodeInfo}; +pub(crate) use provider::{errors, Api, AuthSecret, CachedNodeInfo, NodeInfo}; /// Various cache-related types. pub mod caches { diff --git a/proxy/src/console/provider.rs b/proxy/src/console/provider.rs index 860960627352..69bfd6b045a1 100644 --- a/proxy/src/console/provider.rs +++ b/proxy/src/console/provider.rs @@ -14,7 +14,6 @@ use crate::{ context::RequestMonitoring, scram, EndpointCacheKey, ProjectId, }; -use async_trait::async_trait; use dashmap::DashMap; use std::{sync::Arc, time::Duration}; use tokio::sync::{OwnedSemaphorePermit, Semaphore}; @@ -326,8 +325,7 @@ pub type CachedAllowedIps = Cached<&'static ProjectInfoCacheImpl, Arc), } -#[async_trait] impl Api for ConsoleBackend { async fn get_role_secret( &self, diff --git a/proxy/src/console/provider/mock.rs b/proxy/src/console/provider/mock.rs index 0579ef6fc4ea..b759c8137360 100644 --- a/proxy/src/console/provider/mock.rs +++ b/proxy/src/console/provider/mock.rs @@ -8,7 +8,6 @@ use crate::console::provider::{CachedAllowedIps, CachedRoleSecret}; use crate::context::RequestMonitoring; use crate::{auth::backend::ComputeUserInfo, compute, error::io_error, scram, url::ApiUrl}; use crate::{auth::IpPattern, cache::Cached}; -use async_trait::async_trait; use futures::TryFutureExt; use std::{str::FromStr, sync::Arc}; use thiserror::Error; @@ -144,7 +143,6 @@ async fn get_execute_postgres_query( Ok(Some(entry)) } -#[async_trait] impl super::Api for Api { #[tracing::instrument(skip_all)] async fn get_role_secret( diff --git a/proxy/src/console/provider/neon.rs b/proxy/src/console/provider/neon.rs index b36663518d7c..89ebfa57f1b1 100644 --- a/proxy/src/console/provider/neon.rs +++ b/proxy/src/console/provider/neon.rs @@ -14,7 +14,6 @@ use crate::{ context::RequestMonitoring, metrics::{ALLOWED_IPS_BY_CACHE_OUTCOME, ALLOWED_IPS_NUMBER}, }; -use async_trait::async_trait; use futures::TryFutureExt; use std::sync::Arc; use tokio::time::Instant; @@ -168,7 +167,6 @@ impl Api { } } -#[async_trait] impl super::Api for Api { #[tracing::instrument(skip_all)] async fn get_role_secret( diff --git a/proxy/src/scram/exchange.rs b/proxy/src/scram/exchange.rs index 682cbe795fd0..89dd33e59f27 100644 --- a/proxy/src/scram/exchange.rs +++ b/proxy/src/scram/exchange.rs @@ -3,9 +3,7 @@ use std::convert::Infallible; use hmac::{Hmac, Mac}; -use sha2::digest::FixedOutput; -use sha2::{Digest, Sha256}; -use subtle::{Choice, ConstantTimeEq}; +use sha2::Sha256; use tokio::task::yield_now; use super::messages::{ @@ -13,6 +11,7 @@ use super::messages::{ }; use super::secret::ServerSecret; use super::signature::SignatureBuilder; +use super::ScramKey; use crate::config; use crate::sasl::{self, ChannelBinding, Error as SaslError}; @@ -104,7 +103,7 @@ async fn pbkdf2(str: &[u8], salt: &[u8], iterations: u32) -> [u8; 32] { } // copied from -async fn derive_keys(password: &[u8], salt: &[u8], iterations: u32) -> ([u8; 32], [u8; 32]) { +async fn derive_client_key(password: &[u8], salt: &[u8], iterations: u32) -> ScramKey { let salted_password = pbkdf2(password, salt, iterations).await; let make_key = |name| { @@ -116,7 +115,7 @@ async fn derive_keys(password: &[u8], salt: &[u8], iterations: u32) -> ([u8; 32] <[u8; 32]>::from(key.into_bytes()) }; - (make_key(b"Client Key"), make_key(b"Server Key")) + make_key(b"Client Key").into() } pub async fn exchange( @@ -124,21 +123,12 @@ pub async fn exchange( password: &[u8], ) -> sasl::Result> { let salt = base64::decode(&secret.salt_base64)?; - let (client_key, server_key) = derive_keys(password, &salt, secret.iterations).await; - let stored_key: [u8; 32] = Sha256::default() - .chain_update(client_key) - .finalize_fixed() - .into(); - - // constant time to not leak partial key match - let valid = stored_key.ct_eq(&secret.stored_key.as_bytes()) - | server_key.ct_eq(&secret.server_key.as_bytes()) - | Choice::from(secret.doomed as u8); - - if valid.into() { - Ok(sasl::Outcome::Success(super::ScramKey::from(client_key))) - } else { + let client_key = derive_client_key(password, &salt, secret.iterations).await; + + if secret.is_password_invalid(&client_key).into() { Ok(sasl::Outcome::Failure("password doesn't match")) + } else { + Ok(sasl::Outcome::Success(client_key)) } } @@ -220,7 +210,7 @@ impl SaslSentInner { .derive_client_key(&client_final_message.proof); // Auth fails either if keys don't match or it's pre-determined to fail. - if client_key.sha256() != secret.stored_key || secret.doomed { + if secret.is_password_invalid(&client_key).into() { return Ok(sasl::Step::Failure("password doesn't match")); } diff --git a/proxy/src/scram/key.rs b/proxy/src/scram/key.rs index 973126e729ec..32a3dbd203a7 100644 --- a/proxy/src/scram/key.rs +++ b/proxy/src/scram/key.rs @@ -1,17 +1,31 @@ //! Tools for client/server/stored key management. +use subtle::ConstantTimeEq; + /// Faithfully taken from PostgreSQL. pub const SCRAM_KEY_LEN: usize = 32; /// One of the keys derived from the user's password. /// We use the same structure for all keys, i.e. /// `ClientKey`, `StoredKey`, and `ServerKey`. -#[derive(Clone, Default, PartialEq, Eq, Debug)] +#[derive(Clone, Default, Eq, Debug)] #[repr(transparent)] pub struct ScramKey { bytes: [u8; SCRAM_KEY_LEN], } +impl PartialEq for ScramKey { + fn eq(&self, other: &Self) -> bool { + self.ct_eq(other).into() + } +} + +impl ConstantTimeEq for ScramKey { + fn ct_eq(&self, other: &Self) -> subtle::Choice { + self.bytes.ct_eq(&other.bytes) + } +} + impl ScramKey { pub fn sha256(&self) -> Self { super::sha256([self.as_ref()]).into() diff --git a/proxy/src/scram/messages.rs b/proxy/src/scram/messages.rs index b59baec508a7..f9372540ca47 100644 --- a/proxy/src/scram/messages.rs +++ b/proxy/src/scram/messages.rs @@ -206,6 +206,28 @@ mod tests { } } + #[test] + fn parse_client_first_message_with_invalid_gs2_authz() { + assert!(ClientFirstMessage::parse("n,authzid,n=user,r=nonce").is_none()) + } + + #[test] + fn parse_client_first_message_with_extra_params() { + let msg = ClientFirstMessage::parse("n,,n=user,r=nonce,a=foo,b=bar,c=baz").unwrap(); + assert_eq!(msg.bare, "n=user,r=nonce,a=foo,b=bar,c=baz"); + assert_eq!(msg.username, "user"); + assert_eq!(msg.nonce, "nonce"); + assert_eq!(msg.cbind_flag, ChannelBinding::NotSupportedClient); + } + + #[test] + fn parse_client_first_message_with_extra_params_invalid() { + // must be of the form `=<...>` + assert!(ClientFirstMessage::parse("n,,n=user,r=nonce,abc=foo").is_none()); + assert!(ClientFirstMessage::parse("n,,n=user,r=nonce,1=foo").is_none()); + assert!(ClientFirstMessage::parse("n,,n=user,r=nonce,a").is_none()); + } + #[test] fn parse_client_final_message() { let input = [ diff --git a/proxy/src/scram/secret.rs b/proxy/src/scram/secret.rs index b46d8c3ab588..f3414cb8ecc8 100644 --- a/proxy/src/scram/secret.rs +++ b/proxy/src/scram/secret.rs @@ -1,5 +1,7 @@ //! Tools for SCRAM server secret management. +use subtle::{Choice, ConstantTimeEq}; + use super::base64_decode_array; use super::key::ScramKey; @@ -40,6 +42,11 @@ impl ServerSecret { Some(secret) } + pub fn is_password_invalid(&self, client_key: &ScramKey) -> Choice { + // constant time to not leak partial key match + client_key.sha256().ct_ne(&self.stored_key) | Choice::from(self.doomed as u8) + } + /// To avoid revealing information to an attacker, we use a /// mocked server secret even if the user doesn't exist. /// See `auth-scram.c : mock_scram_secret` for details. From 2668a1dfabf703520b46726d73b4e924f9c9a5cd Mon Sep 17 00:00:00 2001 From: Alexander Bayandin Date: Fri, 22 Mar 2024 14:42:10 +0000 Subject: [PATCH 29/53] CI: deploy release version to a preprod region (#6811) ## Problem We want to deploy releases to a preprod region first to perform required checks ## Summary of changes - Deploy `release-XXX` / `release-proxy-YYY` docker tags to a preprod region --- .github/workflows/build_and_test.yml | 21 ++++++++++++++++++--- 1 file changed, 18 insertions(+), 3 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 2bcda7cc8e8f..d27713f08346 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -1121,10 +1121,16 @@ jobs: run: | if [[ "$GITHUB_REF_NAME" == "main" ]]; then gh workflow --repo neondatabase/aws run deploy-dev.yml --ref main -f branch=main -f dockerTag=${{needs.tag.outputs.build-tag}} -f deployPreprodRegion=false - - # TODO: move deployPreprodRegion to release (`"$GITHUB_REF_NAME" == "release"` block), once Staging support different compute tag prefixes for different regions - gh workflow --repo neondatabase/aws run deploy-dev.yml --ref main -f branch=main -f dockerTag=${{needs.tag.outputs.build-tag}} -f deployPreprodRegion=true elif [[ "$GITHUB_REF_NAME" == "release" ]]; then + gh workflow --repo neondatabase/aws run deploy-dev.yml --ref main \ + -f deployPgSniRouter=false \ + -f deployProxy=false \ + -f deployStorage=true \ + -f deployStorageBroker=true \ + -f branch=main \ + -f dockerTag=${{needs.tag.outputs.build-tag}} \ + -f deployPreprodRegion=true + gh workflow --repo neondatabase/aws run deploy-prod.yml --ref main \ -f deployPgSniRouter=false \ -f deployProxy=false \ @@ -1133,6 +1139,15 @@ jobs: -f branch=main \ -f dockerTag=${{needs.tag.outputs.build-tag}} elif [[ "$GITHUB_REF_NAME" == "release-proxy" ]]; then + gh workflow --repo neondatabase/aws run deploy-dev.yml --ref main \ + -f deployPgSniRouter=true \ + -f deployProxy=true \ + -f deployStorage=false \ + -f deployStorageBroker=false \ + -f branch=main \ + -f dockerTag=${{needs.tag.outputs.build-tag}} \ + -f deployPreprodRegion=true + gh workflow --repo neondatabase/aws run deploy-proxy-prod.yml --ref main \ -f deployPgSniRouter=true \ -f deployProxy=true \ From 1787cf19e3f6fa67edbeb40faa9f0287e864db07 Mon Sep 17 00:00:00 2001 From: John Spray Date: Fri, 22 Mar 2024 14:52:14 +0000 Subject: [PATCH 30/53] pageserver: write consumption metrics to S3 (#7200) ## Problem The service that receives consumption metrics has lower availability than S3. Writing metrics to S3 improves their availability. Closes: https://github.com/neondatabase/cloud/issues/9824 ## Summary of changes - The same data as consumption metrics POST bodies is also compressed and written to an S3 object with a timestamp-formatted path. - Set `metric_collection_bucket` (same format as `remote_storage` config) to configure the location to write to --- pageserver/src/bin/pageserver.rs | 1 + pageserver/src/config.rs | 18 ++++++ pageserver/src/consumption_metrics.rs | 28 ++++++++- pageserver/src/consumption_metrics/upload.rs | 62 ++++++++++++++++++- .../test_pageserver_metric_collection.py | 28 ++++++++- 5 files changed, 131 insertions(+), 6 deletions(-) diff --git a/pageserver/src/bin/pageserver.rs b/pageserver/src/bin/pageserver.rs index 1fd7c775d557..ef616c0a390c 100644 --- a/pageserver/src/bin/pageserver.rs +++ b/pageserver/src/bin/pageserver.rs @@ -615,6 +615,7 @@ fn start_pageserver( pageserver::consumption_metrics::collect_metrics( metric_collection_endpoint, + &conf.metric_collection_bucket, conf.metric_collection_interval, conf.cached_metric_collection_interval, conf.synthetic_size_calculation_interval, diff --git a/pageserver/src/config.rs b/pageserver/src/config.rs index 8ad9ade4a968..a29719e36f33 100644 --- a/pageserver/src/config.rs +++ b/pageserver/src/config.rs @@ -234,6 +234,7 @@ pub struct PageServerConf { // How often to send unchanged cached metrics to the metrics endpoint. pub cached_metric_collection_interval: Duration, pub metric_collection_endpoint: Option, + pub metric_collection_bucket: Option, pub synthetic_size_calculation_interval: Duration, pub disk_usage_based_eviction: Option, @@ -373,6 +374,7 @@ struct PageServerConfigBuilder { cached_metric_collection_interval: BuilderValue, metric_collection_endpoint: BuilderValue>, synthetic_size_calculation_interval: BuilderValue, + metric_collection_bucket: BuilderValue>, disk_usage_based_eviction: BuilderValue>, @@ -455,6 +457,8 @@ impl PageServerConfigBuilder { .expect("cannot parse default synthetic size calculation interval")), metric_collection_endpoint: Set(DEFAULT_METRIC_COLLECTION_ENDPOINT), + metric_collection_bucket: Set(None), + disk_usage_based_eviction: Set(None), test_remote_failures: Set(0), @@ -586,6 +590,13 @@ impl PageServerConfigBuilder { self.metric_collection_endpoint = BuilderValue::Set(metric_collection_endpoint) } + pub fn metric_collection_bucket( + &mut self, + metric_collection_bucket: Option, + ) { + self.metric_collection_bucket = BuilderValue::Set(metric_collection_bucket) + } + pub fn synthetic_size_calculation_interval( &mut self, synthetic_size_calculation_interval: Duration, @@ -694,6 +705,7 @@ impl PageServerConfigBuilder { metric_collection_interval, cached_metric_collection_interval, metric_collection_endpoint, + metric_collection_bucket, synthetic_size_calculation_interval, disk_usage_based_eviction, test_remote_failures, @@ -942,6 +954,9 @@ impl PageServerConf { let endpoint = parse_toml_string(key, item)?.parse().context("failed to parse metric_collection_endpoint")?; builder.metric_collection_endpoint(Some(endpoint)); }, + "metric_collection_bucket" => { + builder.metric_collection_bucket(RemoteStorageConfig::from_toml(item)?) + } "synthetic_size_calculation_interval" => builder.synthetic_size_calculation_interval(parse_toml_duration(key, item)?), "test_remote_failures" => builder.test_remote_failures(parse_toml_u64(key, item)?), @@ -1057,6 +1072,7 @@ impl PageServerConf { metric_collection_interval: Duration::from_secs(60), cached_metric_collection_interval: Duration::from_secs(60 * 60), metric_collection_endpoint: defaults::DEFAULT_METRIC_COLLECTION_ENDPOINT, + metric_collection_bucket: None, synthetic_size_calculation_interval: Duration::from_secs(60), disk_usage_based_eviction: None, test_remote_failures: 0, @@ -1289,6 +1305,7 @@ background_task_maximum_delay = '334 s' defaults::DEFAULT_CACHED_METRIC_COLLECTION_INTERVAL )?, metric_collection_endpoint: defaults::DEFAULT_METRIC_COLLECTION_ENDPOINT, + metric_collection_bucket: None, synthetic_size_calculation_interval: humantime::parse_duration( defaults::DEFAULT_SYNTHETIC_SIZE_CALCULATION_INTERVAL )?, @@ -1363,6 +1380,7 @@ background_task_maximum_delay = '334 s' metric_collection_interval: Duration::from_secs(222), cached_metric_collection_interval: Duration::from_secs(22200), metric_collection_endpoint: Some(Url::parse("http://localhost:80/metrics")?), + metric_collection_bucket: None, synthetic_size_calculation_interval: Duration::from_secs(333), disk_usage_based_eviction: None, test_remote_failures: 0, diff --git a/pageserver/src/consumption_metrics.rs b/pageserver/src/consumption_metrics.rs index c7f9d596c69c..3429e3a0a673 100644 --- a/pageserver/src/consumption_metrics.rs +++ b/pageserver/src/consumption_metrics.rs @@ -7,6 +7,7 @@ use crate::tenant::{mgr, LogicalSizeCalculationCause, PageReconstructError, Tena use camino::Utf8PathBuf; use consumption_metrics::EventType; use pageserver_api::models::TenantState; +use remote_storage::{GenericRemoteStorage, RemoteStorageConfig}; use reqwest::Url; use std::collections::HashMap; use std::sync::Arc; @@ -41,6 +42,7 @@ type Cache = HashMap; #[allow(clippy::too_many_arguments)] pub async fn collect_metrics( metric_collection_endpoint: &Url, + metric_collection_bucket: &Option, metric_collection_interval: Duration, _cached_metric_collection_interval: Duration, synthetic_size_calculation_interval: Duration, @@ -94,6 +96,20 @@ pub async fn collect_metrics( .build() .expect("Failed to create http client with timeout"); + let bucket_client = if let Some(bucket_config) = metric_collection_bucket { + match GenericRemoteStorage::from_config(bucket_config) { + Ok(client) => Some(client), + Err(e) => { + // Non-fatal error: if we were given an invalid config, we will proceed + // with sending metrics over the network, but not to S3. + tracing::warn!("Invalid configuration for metric_collection_bucket: {e}"); + None + } + } + } else { + None + }; + let node_id = node_id.to_string(); loop { @@ -118,10 +134,18 @@ pub async fn collect_metrics( tracing::error!("failed to persist metrics to {path:?}: {e:#}"); } } + + if let Some(bucket_client) = &bucket_client { + let res = + upload::upload_metrics_bucket(bucket_client, &cancel, &node_id, &metrics).await; + if let Err(e) = res { + tracing::error!("failed to upload to S3: {e:#}"); + } + } }; let upload = async { - let res = upload::upload_metrics( + let res = upload::upload_metrics_http( &client, metric_collection_endpoint, &cancel, @@ -132,7 +156,7 @@ pub async fn collect_metrics( .await; if let Err(e) = res { // serialization error which should never happen - tracing::error!("failed to upload due to {e:#}"); + tracing::error!("failed to upload via HTTP due to {e:#}"); } }; diff --git a/pageserver/src/consumption_metrics/upload.rs b/pageserver/src/consumption_metrics/upload.rs index 6b840a31364b..4e8283c3e4dc 100644 --- a/pageserver/src/consumption_metrics/upload.rs +++ b/pageserver/src/consumption_metrics/upload.rs @@ -1,4 +1,9 @@ +use std::time::SystemTime; + +use chrono::{DateTime, Utc}; use consumption_metrics::{Event, EventChunk, IdempotencyKey, CHUNK_SIZE}; +use remote_storage::{GenericRemoteStorage, RemotePath}; +use tokio::io::AsyncWriteExt; use tokio_util::sync::CancellationToken; use tracing::Instrument; @@ -13,8 +18,9 @@ struct Ids { pub(super) timeline_id: Option, } +/// Serialize and write metrics to an HTTP endpoint #[tracing::instrument(skip_all, fields(metrics_total = %metrics.len()))] -pub(super) async fn upload_metrics( +pub(super) async fn upload_metrics_http( client: &reqwest::Client, metric_collection_endpoint: &reqwest::Url, cancel: &CancellationToken, @@ -74,6 +80,60 @@ pub(super) async fn upload_metrics( Ok(()) } +/// Serialize and write metrics to a remote storage object +#[tracing::instrument(skip_all, fields(metrics_total = %metrics.len()))] +pub(super) async fn upload_metrics_bucket( + client: &GenericRemoteStorage, + cancel: &CancellationToken, + node_id: &str, + metrics: &[RawMetric], +) -> anyhow::Result<()> { + if metrics.is_empty() { + // Skip uploads if we have no metrics, so that readers don't have to handle the edge case + // of an empty object. + return Ok(()); + } + + // Compose object path + let datetime: DateTime = SystemTime::now().into(); + let ts_prefix = datetime.format("year=%Y/month=%m/day=%d/%H:%M:%SZ"); + let path = RemotePath::from_string(&format!("{ts_prefix}_{node_id}.ndjson.gz"))?; + + // Set up a gzip writer into a buffer + let mut compressed_bytes: Vec = Vec::new(); + let compressed_writer = std::io::Cursor::new(&mut compressed_bytes); + let mut gzip_writer = async_compression::tokio::write::GzipEncoder::new(compressed_writer); + + // Serialize and write into compressed buffer + let started_at = std::time::Instant::now(); + for res in serialize_in_chunks(CHUNK_SIZE, metrics, node_id) { + let (_chunk, body) = res?; + gzip_writer.write_all(&body).await?; + } + gzip_writer.flush().await?; + gzip_writer.shutdown().await?; + let compressed_length = compressed_bytes.len(); + + // Write to remote storage + client + .upload_storage_object( + futures::stream::once(futures::future::ready(Ok(compressed_bytes.into()))), + compressed_length, + &path, + cancel, + ) + .await?; + let elapsed = started_at.elapsed(); + + tracing::info!( + compressed_length, + elapsed_ms = elapsed.as_millis(), + "write metrics bucket at {path}", + ); + + Ok(()) +} + // The return type is quite ugly, but we gain testability in isolation fn serialize_in_chunks<'a, F>( chunk_size: usize, diff --git a/test_runner/regress/test_pageserver_metric_collection.py b/test_runner/regress/test_pageserver_metric_collection.py index 5799d1119014..c34ef46d07e1 100644 --- a/test_runner/regress/test_pageserver_metric_collection.py +++ b/test_runner/regress/test_pageserver_metric_collection.py @@ -1,4 +1,6 @@ +import gzip import json +import os import time from dataclasses import dataclass from pathlib import Path @@ -10,7 +12,11 @@ NeonEnvBuilder, wait_for_last_flush_lsn, ) -from fixtures.remote_storage import RemoteStorageKind +from fixtures.remote_storage import ( + LocalFsStorage, + RemoteStorageKind, + remote_storage_to_toml_inline_table, +) from fixtures.types import TenantId, TimelineId from pytest_httpserver import HTTPServer from werkzeug.wrappers.request import Request @@ -40,6 +46,9 @@ def metrics_handler(request: Request) -> Response: uploads.put((events, is_last == "true")) return Response(status=200) + neon_env_builder.enable_pageserver_remote_storage(RemoteStorageKind.LOCAL_FS) + assert neon_env_builder.pageserver_remote_storage is not None + # Require collecting metrics frequently, since we change # the timeline and want something to be logged about it. # @@ -48,12 +57,11 @@ def metrics_handler(request: Request) -> Response: neon_env_builder.pageserver_config_override = f""" metric_collection_interval="1s" metric_collection_endpoint="{metric_collection_endpoint}" + metric_collection_bucket={remote_storage_to_toml_inline_table(neon_env_builder.pageserver_remote_storage)} cached_metric_collection_interval="0s" synthetic_size_calculation_interval="3s" """ - neon_env_builder.enable_pageserver_remote_storage(RemoteStorageKind.LOCAL_FS) - log.info(f"test_metric_collection endpoint is {metric_collection_endpoint}") # mock http server that returns OK for the metrics @@ -167,6 +175,20 @@ def get_num_remote_ops(file_kind: str, op_kind: str) -> int: httpserver.check() + # Check that at least one bucket output object is present, and that all + # can be decompressed and decoded. + bucket_dumps = {} + assert isinstance(env.pageserver_remote_storage, LocalFsStorage) + for dirpath, _dirs, files in os.walk(env.pageserver_remote_storage.root): + for file in files: + file_path = os.path.join(dirpath, file) + log.info(file_path) + if file.endswith(".gz"): + bucket_dumps[file_path] = json.load(gzip.open(file_path)) + + assert len(bucket_dumps) >= 1 + assert all("events" in data for data in bucket_dumps.values()) + def test_metric_collection_cleans_up_tempfile( httpserver: HTTPServer, From 35f4c04c9b3ec6f0850d3835a0364439b9907f3f Mon Sep 17 00:00:00 2001 From: Konstantin Knizhnik Date: Fri, 22 Mar 2024 19:14:31 +0200 Subject: [PATCH 31/53] Remove Get/SetZenithCurrentClusterSize from Postgres core (#7196) ## Problem See https://neondb.slack.com/archives/C04DGM6SMTM/p1711003752072899 ## Summary of changes Move keeping of cluster size to neon extension --------- Co-authored-by: Konstantin Knizhnik --- pgxn/neon/neon.c | 2 +- pgxn/neon/neon.h | 2 ++ pgxn/neon/pagestore_smgr.c | 4 ++-- pgxn/neon/walproposer.h | 1 + pgxn/neon/walproposer_pg.c | 15 ++++++++++++++- .../tests/walproposer_sim/walproposer_api.rs | 1 + vendor/postgres-v14 | 2 +- vendor/postgres-v15 | 2 +- vendor/postgres-v16 | 2 +- vendor/revisions.json | 6 +++--- 10 files changed, 27 insertions(+), 10 deletions(-) diff --git a/pgxn/neon/neon.c b/pgxn/neon/neon.c index 6ede78a57692..8d236144b5c5 100644 --- a/pgxn/neon/neon.c +++ b/pgxn/neon/neon.c @@ -312,7 +312,7 @@ pg_cluster_size(PG_FUNCTION_ARGS) { int64 size; - size = GetZenithCurrentClusterSize(); + size = GetNeonCurrentClusterSize(); if (size == 0) PG_RETURN_NULL(); diff --git a/pgxn/neon/neon.h b/pgxn/neon/neon.h index a0f8c974979e..5c653fc6c6cc 100644 --- a/pgxn/neon/neon.h +++ b/pgxn/neon/neon.h @@ -26,6 +26,8 @@ extern void pg_init_libpagestore(void); extern void pg_init_walproposer(void); extern uint64 BackpressureThrottlingTime(void); +extern void SetNeonCurrentClusterSize(uint64 size); +extern uint64 GetNeonCurrentClusterSize(void); extern void replication_feedback_get_lsns(XLogRecPtr *writeLsn, XLogRecPtr *flushLsn, XLogRecPtr *applyLsn); extern void PGDLLEXPORT WalProposerSync(int argc, char *argv[]); diff --git a/pgxn/neon/pagestore_smgr.c b/pgxn/neon/pagestore_smgr.c index 0256de2b9aa8..2d222e3c7c15 100644 --- a/pgxn/neon/pagestore_smgr.c +++ b/pgxn/neon/pagestore_smgr.c @@ -1831,7 +1831,7 @@ neon_extend(SMgrRelation reln, ForkNumber forkNum, BlockNumber blkno, reln->smgr_relpersistence == RELPERSISTENCE_PERMANENT && !IsAutoVacuumWorkerProcess()) { - uint64 current_size = GetZenithCurrentClusterSize(); + uint64 current_size = GetNeonCurrentClusterSize(); if (current_size >= ((uint64) max_cluster_size) * 1024 * 1024) ereport(ERROR, @@ -1912,7 +1912,7 @@ neon_zeroextend(SMgrRelation reln, ForkNumber forkNum, BlockNumber blocknum, reln->smgr_relpersistence == RELPERSISTENCE_PERMANENT && !IsAutoVacuumWorkerProcess()) { - uint64 current_size = GetZenithCurrentClusterSize(); + uint64 current_size = GetNeonCurrentClusterSize(); if (current_size >= ((uint64) max_cluster_size) * 1024 * 1024) ereport(ERROR, diff --git a/pgxn/neon/walproposer.h b/pgxn/neon/walproposer.h index 28585eb4e72f..69a557fdf2d8 100644 --- a/pgxn/neon/walproposer.h +++ b/pgxn/neon/walproposer.h @@ -287,6 +287,7 @@ typedef struct WalproposerShmemState slock_t mutex; term_t mineLastElectedTerm; pg_atomic_uint64 backpressureThrottlingTime; + pg_atomic_uint64 currentClusterSize; /* last feedback from each shard */ PageserverFeedback shard_ps_feedback[MAX_SHARDS]; diff --git a/pgxn/neon/walproposer_pg.c b/pgxn/neon/walproposer_pg.c index 002bf4e2ce49..7debb6325ea2 100644 --- a/pgxn/neon/walproposer_pg.c +++ b/pgxn/neon/walproposer_pg.c @@ -282,6 +282,7 @@ WalproposerShmemInit(void) memset(walprop_shared, 0, WalproposerShmemSize()); SpinLockInit(&walprop_shared->mutex); pg_atomic_init_u64(&walprop_shared->backpressureThrottlingTime, 0); + pg_atomic_init_u64(&walprop_shared->currentClusterSize, 0); } LWLockRelease(AddinShmemInitLock); @@ -1972,7 +1973,7 @@ walprop_pg_process_safekeeper_feedback(WalProposer *wp, Safekeeper *sk) /* Only one main shard sends non-zero currentClusterSize */ if (sk->appendResponse.ps_feedback.currentClusterSize > 0) - SetZenithCurrentClusterSize(sk->appendResponse.ps_feedback.currentClusterSize); + SetNeonCurrentClusterSize(sk->appendResponse.ps_feedback.currentClusterSize); if (min_feedback.disk_consistent_lsn != standby_apply_lsn) { @@ -2094,6 +2095,18 @@ GetLogRepRestartLSN(WalProposer *wp) return lrRestartLsn; } +void SetNeonCurrentClusterSize(uint64 size) +{ + pg_atomic_write_u64(&walprop_shared->currentClusterSize, size); +} + +uint64 GetNeonCurrentClusterSize(void) +{ + return pg_atomic_read_u64(&walprop_shared->currentClusterSize); +} +uint64 GetNeonCurrentClusterSize(void); + + static const walproposer_api walprop_pg = { .get_shmem_state = walprop_pg_get_shmem_state, .start_streaming = walprop_pg_start_streaming, diff --git a/safekeeper/tests/walproposer_sim/walproposer_api.rs b/safekeeper/tests/walproposer_sim/walproposer_api.rs index 42340ba1df15..c49495a4f3fa 100644 --- a/safekeeper/tests/walproposer_sim/walproposer_api.rs +++ b/safekeeper/tests/walproposer_sim/walproposer_api.rs @@ -244,6 +244,7 @@ impl SimulationApi { mutex: 0, mineLastElectedTerm: 0, backpressureThrottlingTime: pg_atomic_uint64 { value: 0 }, + currentClusterSize: pg_atomic_uint64 { value: 0 }, shard_ps_feedback: [empty_feedback; 128], num_shards: 0, min_ps_feedback: empty_feedback, diff --git a/vendor/postgres-v14 b/vendor/postgres-v14 index 3b09894ddb88..c5d920a7d9e9 160000 --- a/vendor/postgres-v14 +++ b/vendor/postgres-v14 @@ -1 +1 @@ -Subproject commit 3b09894ddb8825b50c963942059eab1a2a0b0a89 +Subproject commit c5d920a7d9e9cbeb62b6c46f292db08162763f68 diff --git a/vendor/postgres-v15 b/vendor/postgres-v15 index 80cef885add1..af9ab67bc80a 160000 --- a/vendor/postgres-v15 +++ b/vendor/postgres-v15 @@ -1 +1 @@ -Subproject commit 80cef885add1af6741aa31944c7d2c84d8f9098f +Subproject commit af9ab67bc80afd94e4eb11c34f50c0a29c37eb1b diff --git a/vendor/postgres-v16 b/vendor/postgres-v16 index 90078947229a..111e82c45d79 160000 --- a/vendor/postgres-v16 +++ b/vendor/postgres-v16 @@ -1 +1 @@ -Subproject commit 90078947229aa7f9ac5f7ed4527b2c7386d5332b +Subproject commit 111e82c45d79728fdd3a4816605378c3cc5cfe84 diff --git a/vendor/revisions.json b/vendor/revisions.json index ae524d70b1ef..18959f2ef2b6 100644 --- a/vendor/revisions.json +++ b/vendor/revisions.json @@ -1,5 +1,5 @@ { - "postgres-v16": "90078947229aa7f9ac5f7ed4527b2c7386d5332b", - "postgres-v15": "80cef885add1af6741aa31944c7d2c84d8f9098f", - "postgres-v14": "3b09894ddb8825b50c963942059eab1a2a0b0a89" + "postgres-v16": "111e82c45d79728fdd3a4816605378c3cc5cfe84", + "postgres-v15": "af9ab67bc80afd94e4eb11c34f50c0a29c37eb1b", + "postgres-v14": "c5d920a7d9e9cbeb62b6c46f292db08162763f68" } From 643683f41a6b25865d516201fddbe03fae537077 Mon Sep 17 00:00:00 2001 From: Alex Chi Z Date: Fri, 22 Mar 2024 21:01:51 -0400 Subject: [PATCH 32/53] fixup(#7204 / postgres): revert `IsPrimaryAlive` checks (#7209) Fix #7204. https://github.com/neondatabase/postgres/pull/400 https://github.com/neondatabase/postgres/pull/401 https://github.com/neondatabase/postgres/pull/402 These commits never go into prod. Detailed investigation will be posted in another issue. Reverting the commits so that things can keep running in prod. This pull request adds the test to start two replicas. It fails on the current main https://github.com/neondatabase/neon/pull/7210 but passes in this pull request. --------- Signed-off-by: Alex Chi Z --- test_runner/regress/test_hot_standby.py | 18 ++++++++++++++++++ test_runner/regress/test_replication_start.py | 2 ++ vendor/postgres-v14 | 2 +- vendor/postgres-v15 | 2 +- vendor/postgres-v16 | 2 +- vendor/revisions.json | 6 +++--- 6 files changed, 26 insertions(+), 6 deletions(-) diff --git a/test_runner/regress/test_hot_standby.py b/test_runner/regress/test_hot_standby.py index 0497e1965c99..ac3315b86f86 100644 --- a/test_runner/regress/test_hot_standby.py +++ b/test_runner/regress/test_hot_standby.py @@ -84,3 +84,21 @@ def test_hot_standby(neon_simple_env: NeonEnv): # clean up if slow_down_send: sk_http.configure_failpoints(("sk-send-wal-replica-sleep", "off")) + + +def test_2_replicas_start(neon_simple_env: NeonEnv): + env = neon_simple_env + + 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="secondary1" + ) as secondary1: + with env.endpoints.new_replica_start( + origin=primary, endpoint_id="secondary2" + ) as secondary2: + wait_replica_caughtup(primary, secondary1) + wait_replica_caughtup(primary, secondary2) diff --git a/test_runner/regress/test_replication_start.py b/test_runner/regress/test_replication_start.py index b4699c7be87d..236074599021 100644 --- a/test_runner/regress/test_replication_start.py +++ b/test_runner/regress/test_replication_start.py @@ -1,7 +1,9 @@ +import pytest from fixtures.log_helper import log from fixtures.neon_fixtures import NeonEnv, wait_replica_caughtup +@pytest.mark.xfail def test_replication_start(neon_simple_env: NeonEnv): env = neon_simple_env diff --git a/vendor/postgres-v14 b/vendor/postgres-v14 index c5d920a7d9e9..748643b4683e 160000 --- a/vendor/postgres-v14 +++ b/vendor/postgres-v14 @@ -1 +1 @@ -Subproject commit c5d920a7d9e9cbeb62b6c46f292db08162763f68 +Subproject commit 748643b4683e9fe3b105011a6ba8a687d032cd65 diff --git a/vendor/postgres-v15 b/vendor/postgres-v15 index af9ab67bc80a..e7651e79c0c2 160000 --- a/vendor/postgres-v15 +++ b/vendor/postgres-v15 @@ -1 +1 @@ -Subproject commit af9ab67bc80afd94e4eb11c34f50c0a29c37eb1b +Subproject commit e7651e79c0c27fbddc3c724f5b9553222c28e395 diff --git a/vendor/postgres-v16 b/vendor/postgres-v16 index 111e82c45d79..3946b2e2ea71 160000 --- a/vendor/postgres-v16 +++ b/vendor/postgres-v16 @@ -1 +1 @@ -Subproject commit 111e82c45d79728fdd3a4816605378c3cc5cfe84 +Subproject commit 3946b2e2ea71d07af092099cb5bcae76a69b90d6 diff --git a/vendor/revisions.json b/vendor/revisions.json index 18959f2ef2b6..3c1b8661377b 100644 --- a/vendor/revisions.json +++ b/vendor/revisions.json @@ -1,5 +1,5 @@ { - "postgres-v16": "111e82c45d79728fdd3a4816605378c3cc5cfe84", - "postgres-v15": "af9ab67bc80afd94e4eb11c34f50c0a29c37eb1b", - "postgres-v14": "c5d920a7d9e9cbeb62b6c46f292db08162763f68" + "postgres-v16": "3946b2e2ea71d07af092099cb5bcae76a69b90d6", + "postgres-v15": "e7651e79c0c27fbddc3c724f5b9553222c28e395", + "postgres-v14": "748643b4683e9fe3b105011a6ba8a687d032cd65" } From 72103d481d1b27d9ae18e14b83ab4c985c3d42cf Mon Sep 17 00:00:00 2001 From: Conrad Ludgate Date: Sat, 23 Mar 2024 06:36:58 +0000 Subject: [PATCH 33/53] proxy: fix stack overflow in cancel publisher (#7212) ## Problem stack overflow in blanket impl for `CancellationPublisher` ## Summary of changes Removes `async_trait` and fixes the impl order to make it non-recursive. --- proxy/src/cancellation.rs | 15 ++++++++++++++ proxy/src/redis/cancellation_publisher.rs | 24 +++++++++-------------- 2 files changed, 24 insertions(+), 15 deletions(-) diff --git a/proxy/src/cancellation.rs b/proxy/src/cancellation.rs index 8054f33b6c93..6151513614b9 100644 --- a/proxy/src/cancellation.rs +++ b/proxy/src/cancellation.rs @@ -211,4 +211,19 @@ mod tests { Ok(()) } + + #[tokio::test] + async fn cancel_session_noop_regression() { + let handler = CancellationHandler::<()>::new(Default::default(), "local"); + handler + .cancel_session( + CancelKeyData { + backend_pid: 0, + cancel_key: 0, + }, + Uuid::new_v4(), + ) + .await + .unwrap(); + } } diff --git a/proxy/src/redis/cancellation_publisher.rs b/proxy/src/redis/cancellation_publisher.rs index d9efc3561bfc..422789813ca0 100644 --- a/proxy/src/redis/cancellation_publisher.rs +++ b/proxy/src/redis/cancellation_publisher.rs @@ -1,6 +1,5 @@ use std::sync::Arc; -use async_trait::async_trait; use pq_proto::CancelKeyData; use redis::AsyncCommands; use tokio::sync::Mutex; @@ -13,8 +12,8 @@ use super::{ notifications::{CancelSession, Notification, PROXY_CHANNEL_NAME}, }; -#[async_trait] pub trait CancellationPublisherMut: Send + Sync + 'static { + #[allow(async_fn_in_trait)] async fn try_publish( &mut self, cancel_key_data: CancelKeyData, @@ -22,8 +21,8 @@ pub trait CancellationPublisherMut: Send + Sync + 'static { ) -> anyhow::Result<()>; } -#[async_trait] pub trait CancellationPublisher: Send + Sync + 'static { + #[allow(async_fn_in_trait)] async fn try_publish( &self, cancel_key_data: CancelKeyData, @@ -31,10 +30,9 @@ pub trait CancellationPublisher: Send + Sync + 'static { ) -> anyhow::Result<()>; } -#[async_trait] -impl CancellationPublisherMut for () { +impl CancellationPublisher for () { async fn try_publish( - &mut self, + &self, _cancel_key_data: CancelKeyData, _session_id: Uuid, ) -> anyhow::Result<()> { @@ -42,18 +40,16 @@ impl CancellationPublisherMut for () { } } -#[async_trait] -impl CancellationPublisher for P { +impl CancellationPublisherMut for P { async fn try_publish( - &self, - _cancel_key_data: CancelKeyData, - _session_id: Uuid, + &mut self, + cancel_key_data: CancelKeyData, + session_id: Uuid, ) -> anyhow::Result<()> { - self.try_publish(_cancel_key_data, _session_id).await +

::try_publish(self, cancel_key_data, session_id).await } } -#[async_trait] impl CancellationPublisher for Option

{ async fn try_publish( &self, @@ -68,7 +64,6 @@ impl CancellationPublisher for Option

{ } } -#[async_trait] impl CancellationPublisher for Arc> { async fn try_publish( &self, @@ -145,7 +140,6 @@ impl RedisPublisherClient { } } -#[async_trait] impl CancellationPublisherMut for RedisPublisherClient { async fn try_publish( &mut self, From 3220f830b7fbb785d6db8a93775f46314f10a99b Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sat, 23 Mar 2024 19:25:11 +0100 Subject: [PATCH 34/53] pageserver: use a single tokio runtime (#6555) Before this PR, each core had 3 executor threads from 3 different runtimes. With this PR, we just have one runtime, with one thread per core. Switching to a single tokio runtime should reduce that effective over-commit of CPU and in theory help with tail latencies -- iff all tokio tasks are well-behaved and yield to the runtime regularly. Are All Tasks Well-Behaved? Are We Ready? ----------------------------------------- Sadly there doesn't seem to be good out-of-the box tokio tooling to answer this question. We *believe* all tasks are well behaved in today's code base, as of the switch to `virtual_file_io_engine = "tokio-epoll-uring"` in production (https://github.com/neondatabase/aws/pull/1121). The only remaining executor-thread-blocking code is walredo and some filesystem namespace operations. Filesystem namespace operations work is being tracked in #6663 and not considered likely to actually block at this time. Regarding walredo, it currently does a blocking `poll` for read/write to the pipe file descriptors we use for IPC with the walredo process. There is an ongoing experiment to make walredo async (#6628), but it needs more time because there are surprisingly tricky trade-offs that are articulated in that PR's description (which itself is still WIP). What's relevant for *this* PR is that 1. walredo is always CPU-bound 2. production tail latencies for walredo request-response (`pageserver_wal_redo_seconds_bucket`) are - p90: with few exceptions, low hundreds of micro-seconds - p95: except on very packed pageservers, below 1ms - p99: all below 50ms, vast majority below 1ms - p99.9: almost all around 50ms, rarely at >= 70ms - [Dashboard Link](https://neonprod.grafana.net/d/edgggcrmki3uof/2024-03-walredo-latency?orgId=1&var-ds=ZNX49CDVz&var-pXX_by_instance=0.9&var-pXX_by_instance=0.99&var-pXX_by_instance=0.95&var-adhoc=instance%7C%21%3D%7Cpageserver-30.us-west-2.aws.neon.tech&var-per_instance_pXX_max_seconds=0.0005&from=1711049688777&to=1711136088777) The ones below 1ms are below our current threshold for when we start thinking about yielding to the executor. The tens of milliseconds stalls aren't great, but, not least because of the implicit overcommit of CPU by the three runtimes, we can't be sure whether these tens of milliseconds are inherently necessary to do the walredo work or whether we could be faster if there was less contention for CPU. On the first item (walredo being always CPU-bound work): it means that walredo processes will always compete with the executor threads. We could yield, using async walredo, but then we hit the trade-offs explained in that PR. tl;dr: the risk of stalling executor threads through blocking walredo seems low, and switching to one runtime cleans up one potential source for higher-than-necessary stall times (explained in the previous paragraphs). Code Changes ------------ - Remove the 3 different runtime definitions. - Add a new definition called `THE_RUNTIME`. - Use it in all places that previously used one of the 3 removed runtimes. - Remove the argument from `task_mgr`. - Fix failpoint usage where `pausable_failpoint!` should have been used. We encountered some actual failures because of this, e.g., hung `get_metric()` calls during test teardown that would client-timeout after 300s. As indicated by the comment above `THE_RUNTIME`, we could take this clean-up further. But before we create so much churn, let's first validate that there's no perf regression. Performance ----------- We will test this in staging using the various nightly benchmark runs. However, the worst-case impact of this change is likely compaction (=>image layer creation) competing with compute requests. Image layer creation work can't be easily generated & repeated quickly by pagebench. So, we'll simply watch getpage & basebackup tail latencies in staging. Additionally, I have done manual benchmarking using pagebench. Report: https://neondatabase.notion.site/2024-03-23-oneruntime-change-benchmarking-22a399c411e24399a73311115fb703ec?pvs=4 Tail latencies and throughput are marginally better (no regression = good). Except in a workload with 128 clients against one tenant. There, the p99.9 and p99.99 getpage latency is about 2x worse (at slightly lower throughput). A dip in throughput every 20s (compaction_period_ is clearly visible, and probably responsible for that worse tail latency. This has potential to improve with async walredo, and is an edge case workload anyway. Future Work ----------- 1. Once this change has shown satisfying results in production, change the codebase to use the ambient runtime instead of explicitly referencing `THE_RUNTIME`. 2. Have a mode where we run with a single-threaded runtime, so we uncover executor stalls more quickly. 3. Switch or write our own failpoints library that is async-native: https://github.com/neondatabase/neon/issues/7216 --- pageserver/src/bin/pageserver.rs | 82 +++++++++---------- pageserver/src/consumption_metrics.rs | 3 +- pageserver/src/control_plane_client.rs | 4 +- pageserver/src/disk_usage_eviction_task.rs | 3 +- pageserver/src/page_service.rs | 1 - pageserver/src/task_mgr.rs | 37 ++------- pageserver/src/tenant.rs | 2 +- pageserver/src/tenant/delete.rs | 1 - pageserver/src/tenant/mgr.rs | 4 - .../src/tenant/remote_timeline_client.rs | 11 --- pageserver/src/tenant/secondary.rs | 4 +- pageserver/src/tenant/storage_layer/layer.rs | 4 +- pageserver/src/tenant/tasks.rs | 6 +- pageserver/src/tenant/timeline.rs | 6 +- pageserver/src/tenant/timeline/delete.rs | 1 - .../src/tenant/timeline/eviction_task.rs | 3 +- pageserver/src/tenant/timeline/walreceiver.rs | 5 +- .../walreceiver/walreceiver_connection.rs | 18 ++-- test_runner/regress/test_backpressure.py | 2 +- test_runner/regress/test_timeline_size.py | 26 ++++-- 20 files changed, 92 insertions(+), 131 deletions(-) diff --git a/pageserver/src/bin/pageserver.rs b/pageserver/src/bin/pageserver.rs index ef616c0a390c..f4a231f217a4 100644 --- a/pageserver/src/bin/pageserver.rs +++ b/pageserver/src/bin/pageserver.rs @@ -15,9 +15,9 @@ use metrics::launch_timestamp::{set_launch_timestamp_metric, LaunchTimestamp}; use pageserver::control_plane_client::ControlPlaneClient; use pageserver::disk_usage_eviction_task::{self, launch_disk_usage_global_eviction_task}; use pageserver::metrics::{STARTUP_DURATION, STARTUP_IS_LOADING}; -use pageserver::task_mgr::WALRECEIVER_RUNTIME; use pageserver::tenant::{secondary, TenantSharedResources}; use remote_storage::GenericRemoteStorage; +use tokio::signal::unix::SignalKind; use tokio::time::Instant; use tracing::*; @@ -28,7 +28,7 @@ use pageserver::{ deletion_queue::DeletionQueue, http, page_cache, page_service, task_mgr, task_mgr::TaskKind, - task_mgr::{BACKGROUND_RUNTIME, COMPUTE_REQUEST_RUNTIME, MGMT_REQUEST_RUNTIME}, + task_mgr::THE_RUNTIME, tenant::mgr, virtual_file, }; @@ -323,7 +323,7 @@ fn start_pageserver( // Launch broker client // The storage_broker::connect call needs to happen inside a tokio runtime thread. - let broker_client = WALRECEIVER_RUNTIME + let broker_client = THE_RUNTIME .block_on(async { // Note: we do not attempt connecting here (but validate endpoints sanity). storage_broker::connect(conf.broker_endpoint.clone(), conf.broker_keepalive_interval) @@ -391,7 +391,7 @@ fn start_pageserver( conf, ); if let Some(deletion_workers) = deletion_workers { - deletion_workers.spawn_with(BACKGROUND_RUNTIME.handle()); + deletion_workers.spawn_with(THE_RUNTIME.handle()); } // Up to this point no significant I/O has been done: this should have been fast. Record @@ -423,7 +423,7 @@ fn start_pageserver( // Scan the local 'tenants/' directory and start loading the tenants let deletion_queue_client = deletion_queue.new_client(); - let tenant_manager = BACKGROUND_RUNTIME.block_on(mgr::init_tenant_mgr( + let tenant_manager = THE_RUNTIME.block_on(mgr::init_tenant_mgr( conf, TenantSharedResources { broker_client: broker_client.clone(), @@ -435,7 +435,7 @@ fn start_pageserver( ))?; let tenant_manager = Arc::new(tenant_manager); - BACKGROUND_RUNTIME.spawn({ + THE_RUNTIME.spawn({ let shutdown_pageserver = shutdown_pageserver.clone(); let drive_init = async move { // NOTE: unlike many futures in pageserver, this one is cancellation-safe @@ -545,7 +545,7 @@ fn start_pageserver( // Start up the service to handle HTTP mgmt API request. We created the // listener earlier already. { - let _rt_guard = MGMT_REQUEST_RUNTIME.enter(); + let _rt_guard = THE_RUNTIME.enter(); let router_state = Arc::new( http::routes::State::new( @@ -569,7 +569,6 @@ fn start_pageserver( .with_graceful_shutdown(task_mgr::shutdown_watcher()); task_mgr::spawn( - MGMT_REQUEST_RUNTIME.handle(), TaskKind::HttpEndpointListener, None, None, @@ -594,7 +593,6 @@ fn start_pageserver( let local_disk_storage = conf.workdir.join("last_consumption_metrics.json"); task_mgr::spawn( - crate::BACKGROUND_RUNTIME.handle(), TaskKind::MetricsCollection, None, None, @@ -643,7 +641,6 @@ fn start_pageserver( DownloadBehavior::Error, ); task_mgr::spawn( - COMPUTE_REQUEST_RUNTIME.handle(), TaskKind::LibpqEndpointListener, None, None, @@ -667,42 +664,37 @@ fn start_pageserver( let mut shutdown_pageserver = Some(shutdown_pageserver.drop_guard()); // All started up! Now just sit and wait for shutdown signal. + { - use signal_hook::consts::*; - let signal_handler = BACKGROUND_RUNTIME.spawn_blocking(move || { - let mut signals = - signal_hook::iterator::Signals::new([SIGINT, SIGTERM, SIGQUIT]).unwrap(); - return signals - .forever() - .next() - .expect("forever() never returns None unless explicitly closed"); - }); - let signal = BACKGROUND_RUNTIME - .block_on(signal_handler) - .expect("join error"); - match signal { - SIGQUIT => { - info!("Got signal {signal}. Terminating in immediate shutdown mode",); - std::process::exit(111); - } - SIGINT | SIGTERM => { - info!("Got signal {signal}. Terminating gracefully in fast shutdown mode",); - - // This cancels the `shutdown_pageserver` cancellation tree. - // Right now that tree doesn't reach very far, and `task_mgr` is used instead. - // The plan is to change that over time. - shutdown_pageserver.take(); - let bg_remote_storage = remote_storage.clone(); - let bg_deletion_queue = deletion_queue.clone(); - BACKGROUND_RUNTIME.block_on(pageserver::shutdown_pageserver( - &tenant_manager, - bg_remote_storage.map(|_| bg_deletion_queue), - 0, - )); - unreachable!() - } - _ => unreachable!(), - } + THE_RUNTIME.block_on(async move { + let mut sigint = tokio::signal::unix::signal(SignalKind::interrupt()).unwrap(); + let mut sigterm = tokio::signal::unix::signal(SignalKind::terminate()).unwrap(); + let mut sigquit = tokio::signal::unix::signal(SignalKind::quit()).unwrap(); + let signal = tokio::select! { + _ = sigquit.recv() => { + info!("Got signal SIGQUIT. Terminating in immediate shutdown mode",); + std::process::exit(111); + } + _ = sigint.recv() => { "SIGINT" }, + _ = sigterm.recv() => { "SIGTERM" }, + }; + + info!("Got signal {signal}. Terminating gracefully in fast shutdown mode",); + + // This cancels the `shutdown_pageserver` cancellation tree. + // Right now that tree doesn't reach very far, and `task_mgr` is used instead. + // The plan is to change that over time. + shutdown_pageserver.take(); + let bg_remote_storage = remote_storage.clone(); + let bg_deletion_queue = deletion_queue.clone(); + pageserver::shutdown_pageserver( + &tenant_manager, + bg_remote_storage.map(|_| bg_deletion_queue), + 0, + ) + .await; + unreachable!() + }) } } diff --git a/pageserver/src/consumption_metrics.rs b/pageserver/src/consumption_metrics.rs index 3429e3a0a673..c82be8c581de 100644 --- a/pageserver/src/consumption_metrics.rs +++ b/pageserver/src/consumption_metrics.rs @@ -1,7 +1,7 @@ //! Periodically collect consumption metrics for all active tenants //! and push them to a HTTP endpoint. use crate::context::{DownloadBehavior, RequestContext}; -use crate::task_mgr::{self, TaskKind, BACKGROUND_RUNTIME}; +use crate::task_mgr::{self, TaskKind}; use crate::tenant::tasks::BackgroundLoopKind; use crate::tenant::{mgr, LogicalSizeCalculationCause, PageReconstructError, Tenant}; use camino::Utf8PathBuf; @@ -61,7 +61,6 @@ pub async fn collect_metrics( let worker_ctx = ctx.detached_child(TaskKind::CalculateSyntheticSize, DownloadBehavior::Download); task_mgr::spawn( - BACKGROUND_RUNTIME.handle(), TaskKind::CalculateSyntheticSize, None, None, diff --git a/pageserver/src/control_plane_client.rs b/pageserver/src/control_plane_client.rs index 42c800822b08..55d80c29668a 100644 --- a/pageserver/src/control_plane_client.rs +++ b/pageserver/src/control_plane_client.rs @@ -173,8 +173,6 @@ impl ControlPlaneGenerationsApi for ControlPlaneClient { register, }; - fail::fail_point!("control-plane-client-re-attach"); - let response: ReAttachResponse = self.retry_http_forever(&re_attach_path, request).await?; tracing::info!( "Received re-attach response with {} tenants", @@ -210,7 +208,7 @@ impl ControlPlaneGenerationsApi for ControlPlaneClient { .collect(), }; - fail::fail_point!("control-plane-client-validate"); + crate::tenant::pausable_failpoint!("control-plane-client-validate"); let response: ValidateResponse = self.retry_http_forever(&re_attach_path, request).await?; diff --git a/pageserver/src/disk_usage_eviction_task.rs b/pageserver/src/disk_usage_eviction_task.rs index 92c1475aeff5..6b68acd1c7d4 100644 --- a/pageserver/src/disk_usage_eviction_task.rs +++ b/pageserver/src/disk_usage_eviction_task.rs @@ -59,7 +59,7 @@ use utils::{completion, id::TimelineId}; use crate::{ config::PageServerConf, metrics::disk_usage_based_eviction::METRICS, - task_mgr::{self, TaskKind, BACKGROUND_RUNTIME}, + task_mgr::{self, TaskKind}, tenant::{ self, mgr::TenantManager, @@ -202,7 +202,6 @@ pub fn launch_disk_usage_global_eviction_task( info!("launching disk usage based eviction task"); task_mgr::spawn( - BACKGROUND_RUNTIME.handle(), TaskKind::DiskUsageEviction, None, None, diff --git a/pageserver/src/page_service.rs b/pageserver/src/page_service.rs index f3ceb7d3e6e5..fa1a0f535b50 100644 --- a/pageserver/src/page_service.rs +++ b/pageserver/src/page_service.rs @@ -180,7 +180,6 @@ pub async fn libpq_listener_main( // only deal with a particular timeline, but we don't know which one // yet. task_mgr::spawn( - &tokio::runtime::Handle::current(), TaskKind::PageRequestHandler, None, None, diff --git a/pageserver/src/task_mgr.rs b/pageserver/src/task_mgr.rs index 69e163effaa3..2d97389982ec 100644 --- a/pageserver/src/task_mgr.rs +++ b/pageserver/src/task_mgr.rs @@ -98,42 +98,22 @@ use utils::id::TimelineId; // other operations, if the upload tasks e.g. get blocked on locks. It shouldn't // happen, but still. // -pub static COMPUTE_REQUEST_RUNTIME: Lazy = Lazy::new(|| { - tokio::runtime::Builder::new_multi_thread() - .thread_name("compute request worker") - .enable_all() - .build() - .expect("Failed to create compute request runtime") -}); - -pub static MGMT_REQUEST_RUNTIME: Lazy = Lazy::new(|| { - tokio::runtime::Builder::new_multi_thread() - .thread_name("mgmt request worker") - .enable_all() - .build() - .expect("Failed to create mgmt request runtime") -}); - -pub static WALRECEIVER_RUNTIME: Lazy = Lazy::new(|| { - tokio::runtime::Builder::new_multi_thread() - .thread_name("walreceiver worker") - .enable_all() - .build() - .expect("Failed to create walreceiver runtime") -}); -pub static BACKGROUND_RUNTIME: Lazy = Lazy::new(|| { +/// The single tokio runtime used by all pageserver code. +/// In the past, we had multiple runtimes, and in the future we should weed out +/// remaining references to this global field and rely on ambient runtime instead, +/// i.e., use `tokio::spawn` instead of `THE_RUNTIME.spawn()`, etc. +pub static THE_RUNTIME: Lazy = Lazy::new(|| { tokio::runtime::Builder::new_multi_thread() - .thread_name("background op worker") // if you change the number of worker threads please change the constant below .enable_all() .build() .expect("Failed to create background op runtime") }); -pub(crate) static BACKGROUND_RUNTIME_WORKER_THREADS: Lazy = Lazy::new(|| { +pub(crate) static THE_RUNTIME_WORKER_THREADS: Lazy = Lazy::new(|| { // force init and thus panics - let _ = BACKGROUND_RUNTIME.handle(); + let _ = THE_RUNTIME.handle(); // replicates tokio-1.28.1::loom::sys::num_cpus which is not available publicly // tokio would had already panicked for parsing errors or NotUnicode // @@ -325,7 +305,6 @@ struct PageServerTask { /// Note: if shutdown_process_on_error is set to true failure /// of the task will lead to shutdown of entire process pub fn spawn( - runtime: &tokio::runtime::Handle, kind: TaskKind, tenant_shard_id: Option, timeline_id: Option, @@ -354,7 +333,7 @@ where let task_name = name.to_string(); let task_cloned = Arc::clone(&task); - let join_handle = runtime.spawn(task_wrapper( + let join_handle = THE_RUNTIME.spawn(task_wrapper( task_name, task_id, task_cloned, diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index fe48741a89d1..7bd85b6fd565 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -144,6 +144,7 @@ macro_rules! pausable_failpoint { } }; } +pub(crate) use pausable_failpoint; pub mod blob_io; pub mod block_io; @@ -661,7 +662,6 @@ impl Tenant { let tenant_clone = Arc::clone(&tenant); let ctx = ctx.detached_child(TaskKind::Attach, DownloadBehavior::Warn); task_mgr::spawn( - &tokio::runtime::Handle::current(), TaskKind::Attach, Some(tenant_shard_id), None, diff --git a/pageserver/src/tenant/delete.rs b/pageserver/src/tenant/delete.rs index 7d37873a67e6..3866136dbdfb 100644 --- a/pageserver/src/tenant/delete.rs +++ b/pageserver/src/tenant/delete.rs @@ -482,7 +482,6 @@ impl DeleteTenantFlow { let tenant_shard_id = tenant.tenant_shard_id; task_mgr::spawn( - task_mgr::BACKGROUND_RUNTIME.handle(), TaskKind::TimelineDeletionWorker, Some(tenant_shard_id), None, diff --git a/pageserver/src/tenant/mgr.rs b/pageserver/src/tenant/mgr.rs index 97a505ded906..34ca43a173e6 100644 --- a/pageserver/src/tenant/mgr.rs +++ b/pageserver/src/tenant/mgr.rs @@ -1850,7 +1850,6 @@ impl TenantManager { let task_tenant_id = None; task_mgr::spawn( - task_mgr::BACKGROUND_RUNTIME.handle(), TaskKind::MgmtRequest, task_tenant_id, None, @@ -2816,15 +2815,12 @@ pub(crate) fn immediate_gc( // TODO: spawning is redundant now, need to hold the gate task_mgr::spawn( - &tokio::runtime::Handle::current(), TaskKind::GarbageCollector, Some(tenant_shard_id), Some(timeline_id), &format!("timeline_gc_handler garbage collection run for tenant {tenant_shard_id} timeline {timeline_id}"), false, async move { - fail::fail_point!("immediate_gc_task_pre"); - #[allow(unused_mut)] let mut result = tenant .gc_iteration(Some(timeline_id), gc_horizon, pitr, &cancel, &ctx) diff --git a/pageserver/src/tenant/remote_timeline_client.rs b/pageserver/src/tenant/remote_timeline_client.rs index 40be2ca8f318..c0a150eb0d90 100644 --- a/pageserver/src/tenant/remote_timeline_client.rs +++ b/pageserver/src/tenant/remote_timeline_client.rs @@ -223,7 +223,6 @@ use crate::{ config::PageServerConf, task_mgr, task_mgr::TaskKind, - task_mgr::BACKGROUND_RUNTIME, tenant::metadata::TimelineMetadata, tenant::upload_queue::{ UploadOp, UploadQueue, UploadQueueInitialized, UploadQueueStopped, UploadTask, @@ -307,8 +306,6 @@ pub enum PersistIndexPartWithDeletedFlagError { pub struct RemoteTimelineClient { conf: &'static PageServerConf, - runtime: tokio::runtime::Handle, - tenant_shard_id: TenantShardId, timeline_id: TimelineId, generation: Generation, @@ -341,12 +338,6 @@ impl RemoteTimelineClient { ) -> RemoteTimelineClient { RemoteTimelineClient { conf, - runtime: if cfg!(test) { - // remote_timeline_client.rs tests rely on current-thread runtime - tokio::runtime::Handle::current() - } else { - BACKGROUND_RUNTIME.handle().clone() - }, tenant_shard_id, timeline_id, generation, @@ -1281,7 +1272,6 @@ impl RemoteTimelineClient { let tenant_shard_id = self.tenant_shard_id; let timeline_id = self.timeline_id; task_mgr::spawn( - &self.runtime, TaskKind::RemoteUploadTask, Some(self.tenant_shard_id), Some(self.timeline_id), @@ -1876,7 +1866,6 @@ mod tests { fn build_client(&self, generation: Generation) -> Arc { Arc::new(RemoteTimelineClient { conf: self.harness.conf, - runtime: tokio::runtime::Handle::current(), tenant_shard_id: self.harness.tenant_shard_id, timeline_id: TIMELINE_ID, generation, diff --git a/pageserver/src/tenant/secondary.rs b/pageserver/src/tenant/secondary.rs index 19f36c722e7b..b0babb130834 100644 --- a/pageserver/src/tenant/secondary.rs +++ b/pageserver/src/tenant/secondary.rs @@ -8,7 +8,7 @@ use std::{sync::Arc, time::SystemTime}; use crate::{ config::PageServerConf, disk_usage_eviction_task::DiskUsageEvictionInfo, - task_mgr::{self, TaskKind, BACKGROUND_RUNTIME}, + task_mgr::{self, TaskKind}, virtual_file::MaybeFatalIo, }; @@ -317,7 +317,6 @@ pub fn spawn_tasks( tokio::sync::mpsc::channel::>(16); task_mgr::spawn( - BACKGROUND_RUNTIME.handle(), TaskKind::SecondaryDownloads, None, None, @@ -338,7 +337,6 @@ pub fn spawn_tasks( ); task_mgr::spawn( - BACKGROUND_RUNTIME.handle(), TaskKind::SecondaryUploads, None, None, diff --git a/pageserver/src/tenant/storage_layer/layer.rs b/pageserver/src/tenant/storage_layer/layer.rs index 8ba37b5a8640..e101a40da49b 100644 --- a/pageserver/src/tenant/storage_layer/layer.rs +++ b/pageserver/src/tenant/storage_layer/layer.rs @@ -1447,7 +1447,7 @@ impl LayerInner { #[cfg(test)] tokio::task::spawn(fut); #[cfg(not(test))] - crate::task_mgr::BACKGROUND_RUNTIME.spawn(fut); + crate::task_mgr::THE_RUNTIME.spawn(fut); } /// Needed to use entered runtime in tests, but otherwise use BACKGROUND_RUNTIME. @@ -1458,7 +1458,7 @@ impl LayerInner { #[cfg(test)] tokio::task::spawn_blocking(f); #[cfg(not(test))] - crate::task_mgr::BACKGROUND_RUNTIME.spawn_blocking(f); + crate::task_mgr::THE_RUNTIME.spawn_blocking(f); } } diff --git a/pageserver/src/tenant/tasks.rs b/pageserver/src/tenant/tasks.rs index e4f5f7513288..db32223a601e 100644 --- a/pageserver/src/tenant/tasks.rs +++ b/pageserver/src/tenant/tasks.rs @@ -8,7 +8,7 @@ use std::time::{Duration, Instant}; use crate::context::{DownloadBehavior, RequestContext}; use crate::metrics::TENANT_TASK_EVENTS; use crate::task_mgr; -use crate::task_mgr::{TaskKind, BACKGROUND_RUNTIME}; +use crate::task_mgr::TaskKind; use crate::tenant::throttle::Stats; use crate::tenant::timeline::CompactionError; use crate::tenant::{Tenant, TenantState}; @@ -18,7 +18,7 @@ use utils::{backoff, completion}; static CONCURRENT_BACKGROUND_TASKS: once_cell::sync::Lazy = once_cell::sync::Lazy::new(|| { - let total_threads = *task_mgr::BACKGROUND_RUNTIME_WORKER_THREADS; + let total_threads = *crate::task_mgr::THE_RUNTIME_WORKER_THREADS; let permits = usize::max( 1, // while a lot of the work is done on spawn_blocking, we still do @@ -85,7 +85,6 @@ pub fn start_background_loops( ) { let tenant_shard_id = tenant.tenant_shard_id; task_mgr::spawn( - BACKGROUND_RUNTIME.handle(), TaskKind::Compaction, Some(tenant_shard_id), None, @@ -109,7 +108,6 @@ pub fn start_background_loops( }, ); task_mgr::spawn( - BACKGROUND_RUNTIME.handle(), TaskKind::GarbageCollector, Some(tenant_shard_id), None, diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 7523130f2343..289dee75abe2 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -1723,7 +1723,6 @@ impl Timeline { initdb_optimization_count: 0, }; task_mgr::spawn( - task_mgr::BACKGROUND_RUNTIME.handle(), task_mgr::TaskKind::LayerFlushTask, Some(self.tenant_shard_id), Some(self.timeline_id), @@ -2086,7 +2085,6 @@ impl Timeline { DownloadBehavior::Download, ); task_mgr::spawn( - task_mgr::BACKGROUND_RUNTIME.handle(), task_mgr::TaskKind::InitialLogicalSizeCalculation, Some(self.tenant_shard_id), Some(self.timeline_id), @@ -2264,7 +2262,6 @@ impl Timeline { DownloadBehavior::Download, ); task_mgr::spawn( - task_mgr::BACKGROUND_RUNTIME.handle(), task_mgr::TaskKind::OndemandLogicalSizeCalculation, Some(self.tenant_shard_id), Some(self.timeline_id), @@ -3840,7 +3837,7 @@ impl Timeline { }; let timer = self.metrics.garbage_collect_histo.start_timer(); - fail_point!("before-timeline-gc"); + pausable_failpoint!("before-timeline-gc"); // Is the timeline being deleted? if self.is_stopping() { @@ -4151,7 +4148,6 @@ impl Timeline { let self_clone = Arc::clone(&self); let task_id = task_mgr::spawn( - task_mgr::BACKGROUND_RUNTIME.handle(), task_mgr::TaskKind::DownloadAllRemoteLayers, Some(self.tenant_shard_id), Some(self.timeline_id), diff --git a/pageserver/src/tenant/timeline/delete.rs b/pageserver/src/tenant/timeline/delete.rs index a0c9d99196bb..d2272fc75fb4 100644 --- a/pageserver/src/tenant/timeline/delete.rs +++ b/pageserver/src/tenant/timeline/delete.rs @@ -443,7 +443,6 @@ impl DeleteTimelineFlow { let timeline_id = timeline.timeline_id; task_mgr::spawn( - task_mgr::BACKGROUND_RUNTIME.handle(), TaskKind::TimelineDeletionWorker, Some(tenant_shard_id), Some(timeline_id), diff --git a/pageserver/src/tenant/timeline/eviction_task.rs b/pageserver/src/tenant/timeline/eviction_task.rs index dd769d41216c..f84a4b0dac02 100644 --- a/pageserver/src/tenant/timeline/eviction_task.rs +++ b/pageserver/src/tenant/timeline/eviction_task.rs @@ -28,7 +28,7 @@ use tracing::{debug, error, info, info_span, instrument, warn, Instrument}; use crate::{ context::{DownloadBehavior, RequestContext}, pgdatadir_mapping::CollectKeySpaceError, - task_mgr::{self, TaskKind, BACKGROUND_RUNTIME}, + task_mgr::{self, TaskKind}, tenant::{ tasks::BackgroundLoopKind, timeline::EvictionError, LogicalSizeCalculationCause, Tenant, }, @@ -56,7 +56,6 @@ impl Timeline { let self_clone = Arc::clone(self); let background_tasks_can_start = background_tasks_can_start.cloned(); task_mgr::spawn( - BACKGROUND_RUNTIME.handle(), TaskKind::Eviction, Some(self.tenant_shard_id), Some(self.timeline_id), diff --git a/pageserver/src/tenant/timeline/walreceiver.rs b/pageserver/src/tenant/timeline/walreceiver.rs index 2fab6722b8f5..3592dda8d72c 100644 --- a/pageserver/src/tenant/timeline/walreceiver.rs +++ b/pageserver/src/tenant/timeline/walreceiver.rs @@ -24,7 +24,7 @@ mod connection_manager; mod walreceiver_connection; use crate::context::{DownloadBehavior, RequestContext}; -use crate::task_mgr::{self, TaskKind, WALRECEIVER_RUNTIME}; +use crate::task_mgr::{self, TaskKind}; use crate::tenant::debug_assert_current_span_has_tenant_and_timeline_id; use crate::tenant::timeline::walreceiver::connection_manager::{ connection_manager_loop_step, ConnectionManagerState, @@ -82,7 +82,6 @@ impl WalReceiver { let loop_status = Arc::new(std::sync::RwLock::new(None)); let manager_status = Arc::clone(&loop_status); task_mgr::spawn( - WALRECEIVER_RUNTIME.handle(), TaskKind::WalReceiverManager, Some(timeline.tenant_shard_id), Some(timeline_id), @@ -181,7 +180,7 @@ impl TaskHandle { let (events_sender, events_receiver) = watch::channel(TaskStateUpdate::Started); let cancellation_clone = cancellation.clone(); - let join_handle = WALRECEIVER_RUNTIME.spawn(async move { + let join_handle = tokio::spawn(async move { events_sender.send(TaskStateUpdate::Started).ok(); task(events_sender, cancellation_clone).await // events_sender is dropped at some point during the .await above. diff --git a/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs b/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs index d9f780cfd180..cf87cc6ce029 100644 --- a/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs +++ b/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs @@ -11,7 +11,6 @@ use std::{ use anyhow::{anyhow, Context}; use bytes::BytesMut; use chrono::{NaiveDateTime, Utc}; -use fail::fail_point; use futures::StreamExt; use postgres::{error::SqlState, SimpleQueryMessage, SimpleQueryRow}; use postgres_ffi::WAL_SEGMENT_SIZE; @@ -27,9 +26,7 @@ use super::TaskStateUpdate; use crate::{ context::RequestContext, metrics::{LIVE_CONNECTIONS_COUNT, WALRECEIVER_STARTED_CONNECTIONS, WAL_INGEST}, - task_mgr, - task_mgr::TaskKind, - task_mgr::WALRECEIVER_RUNTIME, + task_mgr::{self, TaskKind}, tenant::{debug_assert_current_span_has_tenant_and_timeline_id, Timeline, WalReceiverInfo}, walingest::WalIngest, walrecord::DecodedWALRecord, @@ -163,7 +160,6 @@ pub(super) async fn handle_walreceiver_connection( ); let connection_cancellation = cancellation.clone(); task_mgr::spawn( - WALRECEIVER_RUNTIME.handle(), TaskKind::WalReceiverConnectionPoller, Some(timeline.tenant_shard_id), Some(timeline.timeline_id), @@ -329,7 +325,17 @@ pub(super) async fn handle_walreceiver_connection( filtered_records += 1; } - fail_point!("walreceiver-after-ingest"); + // don't simply use pausable_failpoint here because its spawn_blocking slows + // slows down the tests too much. + fail::fail_point!("walreceiver-after-ingest-blocking"); + if let Err(()) = (|| { + fail::fail_point!("walreceiver-after-ingest-pause-activate", |_| { + Err(()) + }); + Ok(()) + })() { + pausable_failpoint!("walreceiver-after-ingest-pause"); + } last_rec_lsn = lsn; diff --git a/test_runner/regress/test_backpressure.py b/test_runner/regress/test_backpressure.py index 819912dd0517..af17a2e89d3c 100644 --- a/test_runner/regress/test_backpressure.py +++ b/test_runner/regress/test_backpressure.py @@ -116,7 +116,7 @@ def test_backpressure_received_lsn_lag(neon_env_builder: NeonEnvBuilder): # Configure failpoint to slow down walreceiver ingest with closing(env.pageserver.connect()) as psconn: with psconn.cursor(cursor_factory=psycopg2.extras.DictCursor) as pscur: - pscur.execute("failpoints walreceiver-after-ingest=sleep(20)") + pscur.execute("failpoints walreceiver-after-ingest-blocking=sleep(20)") # FIXME # Wait for the check thread to start diff --git a/test_runner/regress/test_timeline_size.py b/test_runner/regress/test_timeline_size.py index 628c484fbdbd..efd257900d57 100644 --- a/test_runner/regress/test_timeline_size.py +++ b/test_runner/regress/test_timeline_size.py @@ -931,7 +931,7 @@ def test_timeline_logical_size_task_priority(neon_env_builder: NeonEnvBuilder): env.pageserver.stop() env.pageserver.start( extra_env_vars={ - "FAILPOINTS": "initial-size-calculation-permit-pause=pause;walreceiver-after-ingest=pause" + "FAILPOINTS": "initial-size-calculation-permit-pause=pause;walreceiver-after-ingest-pause-activate=return(1);walreceiver-after-ingest-pause=pause" } ) @@ -953,7 +953,11 @@ def assert_initial_logical_size_not_prioritised(): assert details["current_logical_size_is_accurate"] is True client.configure_failpoints( - [("initial-size-calculation-permit-pause", "off"), ("walreceiver-after-ingest", "off")] + [ + ("initial-size-calculation-permit-pause", "off"), + ("walreceiver-after-ingest-pause-activate", "off"), + ("walreceiver-after-ingest-pause", "off"), + ] ) @@ -983,7 +987,7 @@ def test_eager_attach_does_not_queue_up(neon_env_builder: NeonEnvBuilder): # pause at logical size calculation, also pause before walreceiver can give feedback so it will give priority to logical size calculation env.pageserver.start( extra_env_vars={ - "FAILPOINTS": "timeline-calculate-logical-size-pause=pause;walreceiver-after-ingest=pause" + "FAILPOINTS": "timeline-calculate-logical-size-pause=pause;walreceiver-after-ingest-pause-activate=return(1);walreceiver-after-ingest-pause=pause" } ) @@ -1029,7 +1033,11 @@ def eager_tenant_is_active(): other_is_attaching() client.configure_failpoints( - [("timeline-calculate-logical-size-pause", "off"), ("walreceiver-after-ingest", "off")] + [ + ("timeline-calculate-logical-size-pause", "off"), + ("walreceiver-after-ingest-pause-activate", "off"), + ("walreceiver-after-ingest-pause", "off"), + ] ) @@ -1059,7 +1067,7 @@ def test_lazy_attach_activation(neon_env_builder: NeonEnvBuilder, activation_met # pause at logical size calculation, also pause before walreceiver can give feedback so it will give priority to logical size calculation env.pageserver.start( extra_env_vars={ - "FAILPOINTS": "timeline-calculate-logical-size-pause=pause;walreceiver-after-ingest=pause" + "FAILPOINTS": "timeline-calculate-logical-size-pause=pause;walreceiver-after-ingest-pause-activate=return(1);walreceiver-after-ingest-pause=pause" } ) @@ -1111,3 +1119,11 @@ def lazy_tenant_is_active(): delete_lazy_activating(lazy_tenant, env.pageserver, expect_attaching=True) else: raise RuntimeError(activation_method) + + client.configure_failpoints( + [ + ("timeline-calculate-logical-size-pause", "off"), + ("walreceiver-after-ingest-pause-activate", "off"), + ("walreceiver-after-ingest-pause", "off"), + ] + ) From 3a4ebfb95dd19a499b574c812f6c8cd4adebb172 Mon Sep 17 00:00:00 2001 From: Vlad Lazar Date: Mon, 25 Mar 2024 09:38:12 +0000 Subject: [PATCH 35/53] test: fix `test_pageserver_recovery` flakyness (#7207) ## Problem We recently introduced log file validation for the storage controller. The heartbeater will WARN when it fails for a node, hence the test fails. Closes https://github.com/neondatabase/neon/issues/7159 ## Summary of changes * Warn only once for each set of heartbeat retries * Allow list heartbeat warns --- control_plane/attachment_service/src/heartbeater.rs | 2 +- test_runner/regress/test_recovery.py | 7 +++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/control_plane/attachment_service/src/heartbeater.rs b/control_plane/attachment_service/src/heartbeater.rs index e15de289204f..7669680eb645 100644 --- a/control_plane/attachment_service/src/heartbeater.rs +++ b/control_plane/attachment_service/src/heartbeater.rs @@ -139,7 +139,7 @@ impl HeartbeaterTask { .with_client_retries( |client| async move { client.get_utilization().await }, &jwt_token, - 2, + 3, 3, Duration::from_secs(1), &cancel, diff --git a/test_runner/regress/test_recovery.py b/test_runner/regress/test_recovery.py index 6aac1e1d84da..ab5c8be256ba 100644 --- a/test_runner/regress/test_recovery.py +++ b/test_runner/regress/test_recovery.py @@ -15,6 +15,13 @@ def test_pageserver_recovery(neon_env_builder: NeonEnvBuilder): env = neon_env_builder.init_start() env.pageserver.is_testing_enabled_or_skip() + # We expect the pageserver to exit, which will cause storage storage controller + # requests to fail and warn. + env.storage_controller.allowed_errors.append(".*management API still failed.*") + env.storage_controller.allowed_errors.append( + ".*Reconcile error.*error sending request for url.*" + ) + # Create a branch for us env.neon_cli.create_branch("test_pageserver_recovery", "main") From 0099dfa56b1b24519b4948fe8705006c79b484a7 Mon Sep 17 00:00:00 2001 From: John Spray Date: Mon, 25 Mar 2024 11:52:33 +0000 Subject: [PATCH 36/53] storage controller: tighten up secrets handling (#7105) - Remove code for using AWS secrets manager, as we're deploying with k8s->env vars instead - Load each secret independently, so that one can mix CLI args with environment variables, rather than requiring that all secrets are loaded with the same mechanism. - Add a 'strict mode', enabled by default, which will refuse to start if secrets are not loaded. This avoids the risk of accidentially disabling auth by omitting the public key, for example --- Cargo.lock | 24 --- Cargo.toml | 1 - control_plane/attachment_service/Cargo.toml | 1 - control_plane/attachment_service/src/main.rs | 180 ++++++++---------- control_plane/src/storage_controller.rs | 1 + .../fixtures/pageserver/allowed_errors.py | 2 + 6 files changed, 81 insertions(+), 128 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 6409c79ef95c..45b802c54fdd 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -276,7 +276,6 @@ version = "0.1.0" dependencies = [ "anyhow", "aws-config", - "aws-sdk-secretsmanager", "bytes", "camino", "clap", @@ -433,29 +432,6 @@ dependencies = [ "url", ] -[[package]] -name = "aws-sdk-secretsmanager" -version = "1.14.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a0b64e61e7d632d9df90a2e0f32630c68c24960cab1d27d848718180af883d3" -dependencies = [ - "aws-credential-types", - "aws-runtime", - "aws-smithy-async", - "aws-smithy-http", - "aws-smithy-json", - "aws-smithy-runtime", - "aws-smithy-runtime-api", - "aws-smithy-types", - "aws-types", - "bytes", - "fastrand 2.0.0", - "http 0.2.9", - "once_cell", - "regex-lite", - "tracing", -] - [[package]] name = "aws-sdk-sso" version = "1.12.0" diff --git a/Cargo.toml b/Cargo.toml index 4dda63ff584f..309ebbe1196c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -52,7 +52,6 @@ async-stream = "0.3" async-trait = "0.1" aws-config = { version = "1.1.4", default-features = false, features=["rustls"] } aws-sdk-s3 = "1.14" -aws-sdk-secretsmanager = { version = "1.14.0" } aws-sdk-iam = "1.15.0" aws-smithy-async = { version = "1.1.4", default-features = false, features=["rt-tokio"] } aws-smithy-types = "1.1.4" diff --git a/control_plane/attachment_service/Cargo.toml b/control_plane/attachment_service/Cargo.toml index 34882659e359..0201e0ed863d 100644 --- a/control_plane/attachment_service/Cargo.toml +++ b/control_plane/attachment_service/Cargo.toml @@ -16,7 +16,6 @@ testing = [] [dependencies] anyhow.workspace = true aws-config.workspace = true -aws-sdk-secretsmanager.workspace = true bytes.workspace = true camino.workspace = true clap.workspace = true diff --git a/control_plane/attachment_service/src/main.rs b/control_plane/attachment_service/src/main.rs index 0a925a63f69b..bd8d7f5c59bf 100644 --- a/control_plane/attachment_service/src/main.rs +++ b/control_plane/attachment_service/src/main.rs @@ -3,7 +3,6 @@ use attachment_service::http::make_router; use attachment_service::metrics::preinitialize_metrics; use attachment_service::persistence::Persistence; use attachment_service::service::{Config, Service, MAX_UNAVAILABLE_INTERVAL_DEFAULT}; -use aws_config::{BehaviorVersion, Region}; use camino::Utf8PathBuf; use clap::Parser; use diesel::Connection; @@ -55,11 +54,31 @@ struct Cli { #[arg(long)] database_url: Option, + /// Flag to enable dev mode, which permits running without auth + #[arg(long, default_value = "false")] + dev: bool, + /// Grace period before marking unresponsive pageserver offline #[arg(long)] max_unavailable_interval: Option, } +enum StrictMode { + /// In strict mode, we will require that all secrets are loaded, i.e. security features + /// may not be implicitly turned off by omitting secrets in the environment. + Strict, + /// In dev mode, secrets are optional, and omitting a particular secret will implicitly + /// disable the auth related to it (e.g. no pageserver jwt key -> send unauthenticated + /// requests, no public key -> don't authenticate incoming requests). + Dev, +} + +impl Default for StrictMode { + fn default() -> Self { + Self::Strict + } +} + /// Secrets may either be provided on the command line (for testing), or loaded from AWS SecretManager: this /// type encapsulates the logic to decide which and do the loading. struct Secrets { @@ -70,13 +89,6 @@ struct Secrets { } impl Secrets { - const DATABASE_URL_SECRET: &'static str = "rds-neon-storage-controller-url"; - const PAGESERVER_JWT_TOKEN_SECRET: &'static str = - "neon-storage-controller-pageserver-jwt-token"; - const CONTROL_PLANE_JWT_TOKEN_SECRET: &'static str = - "neon-storage-controller-control-plane-jwt-token"; - const PUBLIC_KEY_SECRET: &'static str = "neon-storage-controller-public-key"; - const DATABASE_URL_ENV: &'static str = "DATABASE_URL"; const PAGESERVER_JWT_TOKEN_ENV: &'static str = "PAGESERVER_JWT_TOKEN"; const CONTROL_PLANE_JWT_TOKEN_ENV: &'static str = "CONTROL_PLANE_JWT_TOKEN"; @@ -87,111 +99,41 @@ impl Secrets { /// - Environment variables if DATABASE_URL is set. /// - AWS Secrets Manager secrets async fn load(args: &Cli) -> anyhow::Result { - match &args.database_url { - Some(url) => Self::load_cli(url, args), - None => match std::env::var(Self::DATABASE_URL_ENV) { - Ok(database_url) => Self::load_env(database_url), - Err(_) => Self::load_aws_sm().await, - }, - } - } - - fn load_env(database_url: String) -> anyhow::Result { - let public_key = match std::env::var(Self::PUBLIC_KEY_ENV) { - Ok(public_key) => Some(JwtAuth::from_key(public_key).context("Loading public key")?), - Err(_) => None, - }; - Ok(Self { - database_url, - public_key, - jwt_token: std::env::var(Self::PAGESERVER_JWT_TOKEN_ENV).ok(), - control_plane_jwt_token: std::env::var(Self::CONTROL_PLANE_JWT_TOKEN_ENV).ok(), - }) - } - - async fn load_aws_sm() -> anyhow::Result { - let Ok(region) = std::env::var("AWS_REGION") else { - anyhow::bail!("AWS_REGION is not set, cannot load secrets automatically: either set this, or use CLI args to supply secrets"); - }; - let config = aws_config::defaults(BehaviorVersion::v2023_11_09()) - .region(Region::new(region.clone())) - .load() - .await; - - let asm = aws_sdk_secretsmanager::Client::new(&config); - - let Some(database_url) = asm - .get_secret_value() - .secret_id(Self::DATABASE_URL_SECRET) - .send() - .await? - .secret_string() - .map(str::to_string) + let Some(database_url) = + Self::load_secret(&args.database_url, Self::DATABASE_URL_ENV).await else { anyhow::bail!( - "Database URL secret not found at {region}/{}", - Self::DATABASE_URL_SECRET + "Database URL is not set (set `--database-url`, or `DATABASE_URL` environment)" ) }; - let jwt_token = asm - .get_secret_value() - .secret_id(Self::PAGESERVER_JWT_TOKEN_SECRET) - .send() - .await? - .secret_string() - .map(str::to_string); - if jwt_token.is_none() { - tracing::warn!("No pageserver JWT token set: this will only work if authentication is disabled on the pageserver"); - } - - let control_plane_jwt_token = asm - .get_secret_value() - .secret_id(Self::CONTROL_PLANE_JWT_TOKEN_SECRET) - .send() - .await? - .secret_string() - .map(str::to_string); - if jwt_token.is_none() { - tracing::warn!("No control plane JWT token set: this will only work if authentication is disabled on the pageserver"); - } - - let public_key = asm - .get_secret_value() - .secret_id(Self::PUBLIC_KEY_SECRET) - .send() - .await? - .secret_string() - .map(str::to_string); - let public_key = match public_key { - Some(key) => Some(JwtAuth::from_key(key)?), - None => { - tracing::warn!( - "No public key set: inccoming HTTP requests will not be authenticated" - ); - None - } + let public_key = match Self::load_secret(&args.public_key, Self::PUBLIC_KEY_ENV).await { + Some(v) => Some(JwtAuth::from_key(v).context("Loading public key")?), + None => None, }; - Ok(Self { + let this = Self { database_url, public_key, - jwt_token, - control_plane_jwt_token, - }) + jwt_token: Self::load_secret(&args.jwt_token, Self::PAGESERVER_JWT_TOKEN_ENV).await, + control_plane_jwt_token: Self::load_secret( + &args.control_plane_jwt_token, + Self::CONTROL_PLANE_JWT_TOKEN_ENV, + ) + .await, + }; + + Ok(this) } - fn load_cli(database_url: &str, args: &Cli) -> anyhow::Result { - let public_key = match &args.public_key { - None => None, - Some(key) => Some(JwtAuth::from_key(key.clone()).context("Loading public key")?), - }; - Ok(Self { - database_url: database_url.to_owned(), - public_key, - jwt_token: args.jwt_token.clone(), - control_plane_jwt_token: args.control_plane_jwt_token.clone(), - }) + async fn load_secret(cli: &Option, env_name: &str) -> Option { + if let Some(v) = cli { + Some(v.clone()) + } else if let Ok(v) = std::env::var(env_name) { + Some(v) + } else { + None + } } } @@ -247,8 +189,42 @@ async fn async_main() -> anyhow::Result<()> { args.listen ); + let strict_mode = if args.dev { + StrictMode::Dev + } else { + StrictMode::Strict + }; + let secrets = Secrets::load(&args).await?; + // Validate required secrets and arguments are provided in strict mode + match strict_mode { + StrictMode::Strict + if (secrets.public_key.is_none() + || secrets.jwt_token.is_none() + || secrets.control_plane_jwt_token.is_none()) => + { + // Production systems should always have secrets configured: if public_key was not set + // then we would implicitly disable auth. + anyhow::bail!( + "Insecure config! One or more secrets is not set. This is only permitted in `--dev` mode" + ); + } + StrictMode::Strict if args.compute_hook_url.is_none() => { + // Production systems should always have a compute hook set, to prevent falling + // back to trying to use neon_local. + anyhow::bail!( + "`--compute-hook-url` is not set: this is only permitted in `--dev` mode" + ); + } + StrictMode::Strict => { + tracing::info!("Starting in strict mode: configuration is OK.") + } + StrictMode::Dev => { + tracing::warn!("Starting in dev mode: this may be an insecure configuration.") + } + } + let config = Config { jwt_token: secrets.jwt_token, control_plane_jwt_token: secrets.control_plane_jwt_token, diff --git a/control_plane/src/storage_controller.rs b/control_plane/src/storage_controller.rs index e7697ecac80c..7f2b97339106 100644 --- a/control_plane/src/storage_controller.rs +++ b/control_plane/src/storage_controller.rs @@ -279,6 +279,7 @@ impl StorageController { &self.listen, "-p", self.path.as_ref(), + "--dev", "--database-url", &database_url, "--max-unavailable-interval", diff --git a/test_runner/fixtures/pageserver/allowed_errors.py b/test_runner/fixtures/pageserver/allowed_errors.py index ec0f81b380e5..d7f682dad335 100755 --- a/test_runner/fixtures/pageserver/allowed_errors.py +++ b/test_runner/fixtures/pageserver/allowed_errors.py @@ -96,6 +96,8 @@ def scan_pageserver_log_for_errors( ".*Call to node.*management API.*failed.*ReceiveBody.*", # Many tests will start up with a node offline ".*startup_reconcile: Could not scan node.*", + # Tests run in dev mode + ".*Starting in dev mode.*", ] From adb05262628a1d3259617a066eeb555d3075e4d2 Mon Sep 17 00:00:00 2001 From: John Spray Date: Mon, 25 Mar 2024 11:52:50 +0000 Subject: [PATCH 37/53] pageserver: track total ephemeral layer bytes (#7182) ## Problem Large quantities of ephemeral layer data can lead to excessive memory consumption (https://github.com/neondatabase/neon/issues/6939). We currently don't have a way to know how much ephemeral layer data is present on a pageserver. Before we can add new behaviors to proactively roll layers in response to too much ephemeral data, we must calculate that total. Related: https://github.com/neondatabase/neon/issues/6916 ## Summary of changes - Create GlobalResources and GlobalResourceUnits types, where timelines carry a GlobalResourceUnits in their TimelineWriterState. - Periodically update the size in GlobalResourceUnits: - During tick() - During layer roll - During put() if the latest value has drifted more than 10MB since our last update - Expose the value of the global ephemeral layer bytes counter as a prometheus metric. - Extend the lifetime of TimelineWriterState: - Instead of dropping it in TimelineWriter::drop, let it remain. - Drop TimelineWriterState in roll_layer: this drops our guard on the global byte count to reflect the fact that we're freezing the layer. - Ensure the validity of the later in the writer state by clearing the state in the same place we freeze layers, and asserting on the write-ability of the layer in `writer()` - Add a 'context' parameter to `get_open_layer_action` so that it can skip the prev_lsn==lsn check when called in tick() -- this is needed because now tick is called with a populated state, where prev_lsn==Some(lsn) is true for an idle timeline. - Extend layer rolling test to use this metric --- pageserver/src/metrics.rs | 8 ++ .../tenant/storage_layer/inmemory_layer.rs | 111 ++++++++++++++++++ pageserver/src/tenant/timeline.rs | 3 + test_runner/fixtures/pageserver/utils.py | 4 +- .../test_pageserver_small_inmemory_layers.py | 45 ++++++- 5 files changed, 162 insertions(+), 9 deletions(-) diff --git a/pageserver/src/metrics.rs b/pageserver/src/metrics.rs index 075bb76a1bba..6de284ee8b72 100644 --- a/pageserver/src/metrics.rs +++ b/pageserver/src/metrics.rs @@ -699,6 +699,14 @@ pub static STARTUP_IS_LOADING: Lazy = Lazy::new(|| { .expect("Failed to register pageserver_startup_is_loading") }); +pub(crate) static TIMELINE_EPHEMERAL_BYTES: Lazy = Lazy::new(|| { + register_uint_gauge!( + "pageserver_timeline_ephemeral_bytes", + "Total number of bytes in ephemeral layers, summed for all timelines. Approximate, lazily updated." + ) + .expect("Failed to register metric") +}); + /// Metrics related to the lifecycle of a [`crate::tenant::Tenant`] object: things /// like how long it took to load. /// diff --git a/pageserver/src/tenant/storage_layer/inmemory_layer.rs b/pageserver/src/tenant/storage_layer/inmemory_layer.rs index 5f1db21d493b..869d175d8df9 100644 --- a/pageserver/src/tenant/storage_layer/inmemory_layer.rs +++ b/pageserver/src/tenant/storage_layer/inmemory_layer.rs @@ -23,8 +23,12 @@ use tracing::*; use utils::{bin_ser::BeSer, id::TimelineId, lsn::Lsn, vec_map::VecMap}; // avoid binding to Write (conflicts with std::io::Write) // while being able to use std::fmt::Write's methods +use crate::metrics::TIMELINE_EPHEMERAL_BYTES; +use std::cmp::Ordering; use std::fmt::Write as _; use std::ops::Range; +use std::sync::atomic::Ordering as AtomicOrdering; +use std::sync::atomic::{AtomicU64, AtomicUsize}; use tokio::sync::{RwLock, RwLockWriteGuard}; use super::{ @@ -70,6 +74,8 @@ pub struct InMemoryLayerInner { /// Each serialized Value is preceded by a 'u32' length field. /// PerSeg::page_versions map stores offsets into this file. file: EphemeralFile, + + resource_units: GlobalResourceUnits, } impl std::fmt::Debug for InMemoryLayerInner { @@ -78,6 +84,101 @@ impl std::fmt::Debug for InMemoryLayerInner { } } +/// State shared by all in-memory (ephemeral) layers. Updated infrequently during background ticks in Timeline, +/// to minimize contention. +/// +/// This global state is used to implement behaviors that require a global view of the system, e.g. +/// rolling layers proactively to limit the total amount of dirty data. +struct GlobalResources { + // How many bytes are in all EphemeralFile objects + dirty_bytes: AtomicU64, + // How many layers are contributing to dirty_bytes + dirty_layers: AtomicUsize, +} + +// Per-timeline RAII struct for its contribution to [`GlobalResources`] +struct GlobalResourceUnits { + // How many dirty bytes have I added to the global dirty_bytes: this guard object is responsible + // for decrementing the global counter by this many bytes when dropped. + dirty_bytes: u64, +} + +impl GlobalResourceUnits { + // Hint for the layer append path to update us when the layer size differs from the last + // call to update_size by this much. If we don't reach this threshold, we'll still get + // updated when the Timeline "ticks" in the background. + const MAX_SIZE_DRIFT: u64 = 10 * 1024 * 1024; + + fn new() -> Self { + GLOBAL_RESOURCES + .dirty_layers + .fetch_add(1, AtomicOrdering::Relaxed); + Self { dirty_bytes: 0 } + } + + /// Do not call this frequently: all timelines will write to these same global atomics, + /// so this is a relatively expensive operation. Wait at least a few seconds between calls. + fn publish_size(&mut self, size: u64) { + let new_global_dirty_bytes = match size.cmp(&self.dirty_bytes) { + Ordering::Equal => { + return; + } + Ordering::Greater => { + let delta = size - self.dirty_bytes; + let old = GLOBAL_RESOURCES + .dirty_bytes + .fetch_add(delta, AtomicOrdering::Relaxed); + old + delta + } + Ordering::Less => { + let delta = self.dirty_bytes - size; + let old = GLOBAL_RESOURCES + .dirty_bytes + .fetch_sub(delta, AtomicOrdering::Relaxed); + old - delta + } + }; + + // This is a sloppy update: concurrent updates to the counter will race, and the exact + // value of the metric might not be the exact latest value of GLOBAL_RESOURCES::dirty_bytes. + // That's okay: as long as the metric contains some recent value, it doesn't have to always + // be literally the last update. + TIMELINE_EPHEMERAL_BYTES.set(new_global_dirty_bytes); + + self.dirty_bytes = size; + } + + // Call publish_size if the input size differs from last published size by more than + // the drift limit + fn maybe_publish_size(&mut self, size: u64) { + let publish = match size.cmp(&self.dirty_bytes) { + Ordering::Equal => false, + Ordering::Greater => size - self.dirty_bytes > Self::MAX_SIZE_DRIFT, + Ordering::Less => self.dirty_bytes - size > Self::MAX_SIZE_DRIFT, + }; + + if publish { + self.publish_size(size); + } + } +} + +impl Drop for GlobalResourceUnits { + fn drop(&mut self) { + GLOBAL_RESOURCES + .dirty_layers + .fetch_sub(1, AtomicOrdering::Relaxed); + + // Subtract our contribution to the global total dirty bytes + self.publish_size(0); + } +} + +static GLOBAL_RESOURCES: GlobalResources = GlobalResources { + dirty_bytes: AtomicU64::new(0), + dirty_layers: AtomicUsize::new(0), +}; + impl InMemoryLayer { pub(crate) fn get_timeline_id(&self) -> TimelineId { self.timeline_id @@ -328,6 +429,7 @@ impl InMemoryLayer { inner: RwLock::new(InMemoryLayerInner { index: HashMap::new(), file, + resource_units: GlobalResourceUnits::new(), }), }) } @@ -378,9 +480,18 @@ impl InMemoryLayer { warn!("Key {} at {} already exists", key, lsn); } + let size = locked_inner.file.len(); + locked_inner.resource_units.maybe_publish_size(size); + Ok(()) } + pub(crate) async fn tick(&self) { + let mut inner = self.inner.write().await; + let size = inner.file.len(); + inner.resource_units.publish_size(size); + } + pub(crate) async fn put_tombstones(&self, _key_ranges: &[(Range, Lsn)]) -> Result<()> { // TODO: Currently, we just leak the storage for any deleted keys Ok(()) diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 289dee75abe2..15ffa72aaac3 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -4465,6 +4465,9 @@ impl<'a> TimelineWriter<'a> { let action = self.get_open_layer_action(last_record_lsn, 0); if action == OpenLayerAction::Roll { self.roll_layer(last_record_lsn).await?; + } else if let Some(writer_state) = &mut *self.write_guard { + // Periodic update of statistics + writer_state.open_layer.tick().await; } Ok(()) diff --git a/test_runner/fixtures/pageserver/utils.py b/test_runner/fixtures/pageserver/utils.py index cf64c86821b0..693771dd3dc9 100644 --- a/test_runner/fixtures/pageserver/utils.py +++ b/test_runner/fixtures/pageserver/utils.py @@ -62,9 +62,7 @@ def wait_for_upload( ) time.sleep(1) raise Exception( - "timed out while waiting for remote_consistent_lsn to reach {}, was {}".format( - lsn, current_lsn - ) + f"timed out while waiting for {tenant}/{timeline} remote_consistent_lsn to reach {lsn}, was {current_lsn}" ) diff --git a/test_runner/regress/test_pageserver_small_inmemory_layers.py b/test_runner/regress/test_pageserver_small_inmemory_layers.py index 5d55020e3c67..714d1c12296c 100644 --- a/test_runner/regress/test_pageserver_small_inmemory_layers.py +++ b/test_runner/regress/test_pageserver_small_inmemory_layers.py @@ -1,5 +1,4 @@ import asyncio -import time from typing import Tuple import pytest @@ -10,7 +9,7 @@ tenant_get_shards, ) from fixtures.pageserver.http import PageserverHttpClient -from fixtures.pageserver.utils import wait_for_last_record_lsn +from fixtures.pageserver.utils import wait_for_last_record_lsn, wait_for_upload from fixtures.types import Lsn, TenantId, TimelineId from fixtures.utils import wait_until @@ -61,6 +60,15 @@ def wait_until_pageserver_is_caught_up( assert waited >= last_flush_lsn +def wait_until_pageserver_has_uploaded( + env: NeonEnv, last_flush_lsns: list[Tuple[TenantId, TimelineId, Lsn]] +): + for tenant, timeline, last_flush_lsn in last_flush_lsns: + shards = tenant_get_shards(env, tenant) + for tenant_shard_id, pageserver in shards: + wait_for_upload(pageserver.http_client(), tenant_shard_id, timeline, last_flush_lsn) + + def wait_for_wal_ingest_metric(pageserver_http: PageserverHttpClient) -> float: def query(): value = pageserver_http.get_metric_value("pageserver_wal_ingest_records_received_total") @@ -86,25 +94,50 @@ def test_pageserver_small_inmemory_layers( The workload creates a number of timelines and writes some data to each, but not enough to trigger flushes via the `checkpoint_distance` config. """ + + def get_dirty_bytes(): + v = ( + env.pageserver.http_client().get_metric_value("pageserver_timeline_ephemeral_bytes") + or 0 + ) + log.info(f"dirty_bytes: {v}") + return v + + def assert_dirty_bytes(v): + assert get_dirty_bytes() == v + env = neon_env_builder.init_configs() env.start() last_flush_lsns = asyncio.run(workload(env, TIMELINE_COUNT, ENTRIES_PER_TIMELINE)) wait_until_pageserver_is_caught_up(env, last_flush_lsns) + # We didn't write enough data to trigger a size-based checkpoint + assert get_dirty_bytes() > 0 + ps_http_client = env.pageserver.http_client() total_wal_ingested_before_restart = wait_for_wal_ingest_metric(ps_http_client) - log.info("Sleeping for checkpoint timeout ...") - time.sleep(CHECKPOINT_TIMEOUT_SECONDS + 5) + # Within ~ the checkpoint interval, all the ephemeral layers should be frozen and flushed, + # such that there are zero bytes of ephemeral layer left on the pageserver + log.info("Waiting for background checkpoints...") + wait_until(CHECKPOINT_TIMEOUT_SECONDS * 2, 1, lambda: assert_dirty_bytes(0)) # type: ignore + + # Zero ephemeral layer bytes does not imply that all the frozen layers were uploaded: they + # must be uploaded to remain visible to the pageserver after restart. + wait_until_pageserver_has_uploaded(env, last_flush_lsns) env.pageserver.restart(immediate=immediate_shutdown) wait_until_pageserver_is_caught_up(env, last_flush_lsns) + # Catching up with WAL ingest should have resulted in zero bytes of ephemeral layers, since + # we froze, flushed and uploaded everything before restarting. There can be no more WAL writes + # because we shut down compute endpoints before flushing. + assert get_dirty_bytes() == 0 + total_wal_ingested_after_restart = wait_for_wal_ingest_metric(ps_http_client) log.info(f"WAL ingested before restart: {total_wal_ingested_before_restart}") log.info(f"WAL ingested after restart: {total_wal_ingested_after_restart}") - leeway = total_wal_ingested_before_restart * 5 / 100 - assert total_wal_ingested_after_restart <= leeway + assert total_wal_ingested_after_restart == 0 From a6c1fdcaf657ad3de8cbdac514d44a9f1a0ecef8 Mon Sep 17 00:00:00 2001 From: Arseny Sher Date: Fri, 15 Mar 2024 18:04:05 +0300 Subject: [PATCH 38/53] Try to fix test_crafted_wal_end flakiness. Postgres can always write some more WAL, so previous checks that WAL doesn't change after something had been crafted were wrong; remove them. Add comments here and there. should fix https://github.com/neondatabase/neon/issues/4691 --- .../wal_craft/src/bin/wal_craft.rs | 6 +- libs/postgres_ffi/wal_craft/src/lib.rs | 126 ++++++++---------- .../wal_craft/src/xlog_utils_test.rs | 35 +++-- 3 files changed, 77 insertions(+), 90 deletions(-) diff --git a/libs/postgres_ffi/wal_craft/src/bin/wal_craft.rs b/libs/postgres_ffi/wal_craft/src/bin/wal_craft.rs index e87ca27e90e6..41afcea6c2fb 100644 --- a/libs/postgres_ffi/wal_craft/src/bin/wal_craft.rs +++ b/libs/postgres_ffi/wal_craft/src/bin/wal_craft.rs @@ -1,5 +1,6 @@ use anyhow::*; use clap::{value_parser, Arg, ArgMatches, Command}; +use postgres::Client; use std::{path::PathBuf, str::FromStr}; use wal_craft::*; @@ -8,8 +9,8 @@ fn main() -> Result<()> { .init(); let arg_matches = cli().get_matches(); - let wal_craft = |arg_matches: &ArgMatches, client| { - let (intermediate_lsns, end_of_wal_lsn) = match arg_matches + let wal_craft = |arg_matches: &ArgMatches, client: &mut Client| { + let intermediate_lsns = match arg_matches .get_one::("type") .map(|s| s.as_str()) .context("'type' is required")? @@ -25,6 +26,7 @@ fn main() -> Result<()> { LastWalRecordCrossingSegment::NAME => LastWalRecordCrossingSegment::craft(client)?, a => panic!("Unknown --type argument: {a}"), }; + let end_of_wal_lsn = client.pg_current_wal_insert_lsn()?; for lsn in intermediate_lsns { println!("intermediate_lsn = {lsn}"); } diff --git a/libs/postgres_ffi/wal_craft/src/lib.rs b/libs/postgres_ffi/wal_craft/src/lib.rs index 281a180e3ba3..23786e3b089c 100644 --- a/libs/postgres_ffi/wal_craft/src/lib.rs +++ b/libs/postgres_ffi/wal_craft/src/lib.rs @@ -5,7 +5,6 @@ use postgres::types::PgLsn; use postgres::Client; use postgres_ffi::{WAL_SEGMENT_SIZE, XLOG_BLCKSZ}; use postgres_ffi::{XLOG_SIZE_OF_XLOG_RECORD, XLOG_SIZE_OF_XLOG_SHORT_PHD}; -use std::cmp::Ordering; use std::path::{Path, PathBuf}; use std::process::Command; use std::time::{Duration, Instant}; @@ -232,59 +231,52 @@ pub fn ensure_server_config(client: &mut impl postgres::GenericClient) -> anyhow pub trait Crafter { const NAME: &'static str; - /// Generates WAL using the client `client`. Returns a pair of: - /// * A vector of some valid "interesting" intermediate LSNs which one may start reading from. - /// May include or exclude Lsn(0) and the end-of-wal. - /// * The expected end-of-wal LSN. - fn craft(client: &mut impl postgres::GenericClient) -> anyhow::Result<(Vec, PgLsn)>; + /// Generates WAL using the client `client`. Returns a vector of some valid + /// "interesting" intermediate LSNs which one may start reading from. + /// test_end_of_wal uses this to check various starting points. + /// + /// Note that postgres is generally keen about writing some WAL. While we + /// try to disable it (autovacuum, big wal_writer_delay, etc) it is always + /// possible, e.g. xl_running_xacts are dumped each 15s. So checks about + /// stable WAL end would be flaky unless postgres is shut down. For this + /// reason returning potential end of WAL here is pointless. Most of the + /// time this doesn't happen though, so it is reasonable to create needed + /// WAL structure and immediately kill postgres like test_end_of_wal does. + fn craft(client: &mut impl postgres::GenericClient) -> anyhow::Result>; } +/// Wraps some WAL craft function, providing current LSN to it before the +/// insertion and flushing WAL afterwards. Also pushes initial LSN to the +/// result. fn craft_internal( client: &mut C, - f: impl Fn(&mut C, PgLsn) -> anyhow::Result<(Vec, Option)>, -) -> anyhow::Result<(Vec, PgLsn)> { + f: impl Fn(&mut C, PgLsn) -> anyhow::Result>, +) -> anyhow::Result> { ensure_server_config(client)?; let initial_lsn = client.pg_current_wal_insert_lsn()?; info!("LSN initial = {}", initial_lsn); - let (mut intermediate_lsns, last_lsn) = f(client, initial_lsn)?; - let last_lsn = match last_lsn { - None => client.pg_current_wal_insert_lsn()?, - Some(last_lsn) => { - let insert_lsn = client.pg_current_wal_insert_lsn()?; - match last_lsn.cmp(&insert_lsn) { - Ordering::Less => bail!( - "Some records were inserted after the crafted WAL: {} vs {}", - last_lsn, - insert_lsn - ), - Ordering::Equal => last_lsn, - Ordering::Greater => bail!("Reported LSN is greater than insert_lsn"), - } - } - }; + let mut intermediate_lsns = f(client, initial_lsn)?; if !intermediate_lsns.starts_with(&[initial_lsn]) { intermediate_lsns.insert(0, initial_lsn); } // Some records may be not flushed, e.g. non-transactional logical messages. + // + // Note: this is broken if pg_current_wal_insert_lsn is at page boundary + // because pg_current_wal_insert_lsn skips page headers. client.execute("select neon_xlogflush(pg_current_wal_insert_lsn())", &[])?; - match last_lsn.cmp(&client.pg_current_wal_flush_lsn()?) { - Ordering::Less => bail!("Some records were flushed after the crafted WAL"), - Ordering::Equal => {} - Ordering::Greater => bail!("Reported LSN is greater than flush_lsn"), - } - Ok((intermediate_lsns, last_lsn)) + Ok(intermediate_lsns) } pub struct Simple; impl Crafter for Simple { const NAME: &'static str = "simple"; - fn craft(client: &mut impl postgres::GenericClient) -> anyhow::Result<(Vec, PgLsn)> { + fn craft(client: &mut impl postgres::GenericClient) -> anyhow::Result> { craft_internal(client, |client, _| { client.execute("CREATE table t(x int)", &[])?; - Ok((Vec::new(), None)) + Ok(Vec::new()) }) } } @@ -292,29 +284,36 @@ impl Crafter for Simple { pub struct LastWalRecordXlogSwitch; impl Crafter for LastWalRecordXlogSwitch { const NAME: &'static str = "last_wal_record_xlog_switch"; - fn craft(client: &mut impl postgres::GenericClient) -> anyhow::Result<(Vec, PgLsn)> { - // Do not use generate_internal because here we end up with flush_lsn exactly on + fn craft(client: &mut impl postgres::GenericClient) -> anyhow::Result> { + // Do not use craft_internal because here we end up with flush_lsn exactly on // the segment boundary and insert_lsn after the initial page header, which is unusual. ensure_server_config(client)?; client.execute("CREATE table t(x int)", &[])?; let before_xlog_switch = client.pg_current_wal_insert_lsn()?; - let after_xlog_switch: PgLsn = client.query_one("SELECT pg_switch_wal()", &[])?.get(0); - let next_segment = PgLsn::from(0x0200_0000); + // pg_switch_wal returns end of last record of the switched segment, + // i.e. end of SWITCH itself. + let xlog_switch_record_end: PgLsn = client.query_one("SELECT pg_switch_wal()", &[])?.get(0); + let before_xlog_switch_u64 = u64::from(before_xlog_switch); + let next_segment = PgLsn::from( + before_xlog_switch_u64 - (before_xlog_switch_u64 % WAL_SEGMENT_SIZE as u64) + + WAL_SEGMENT_SIZE as u64, + ); ensure!( - after_xlog_switch <= next_segment, - "XLOG_SWITCH message ended after the expected segment boundary: {} > {}", - after_xlog_switch, + xlog_switch_record_end <= next_segment, + "XLOG_SWITCH record ended after the expected segment boundary: {} > {}", + xlog_switch_record_end, next_segment ); - Ok((vec![before_xlog_switch, after_xlog_switch], next_segment)) + Ok(vec![before_xlog_switch, xlog_switch_record_end]) } } pub struct LastWalRecordXlogSwitchEndsOnPageBoundary; +/// Craft xlog SWITCH record ending at page boundary. impl Crafter for LastWalRecordXlogSwitchEndsOnPageBoundary { const NAME: &'static str = "last_wal_record_xlog_switch_ends_on_page_boundary"; - fn craft(client: &mut impl postgres::GenericClient) -> anyhow::Result<(Vec, PgLsn)> { + fn craft(client: &mut impl postgres::GenericClient) -> anyhow::Result> { // Do not use generate_internal because here we end up with flush_lsn exactly on // the segment boundary and insert_lsn after the initial page header, which is unusual. ensure_server_config(client)?; @@ -361,28 +360,29 @@ impl Crafter for LastWalRecordXlogSwitchEndsOnPageBoundary { // Emit the XLOG_SWITCH let before_xlog_switch = client.pg_current_wal_insert_lsn()?; - let after_xlog_switch: PgLsn = client.query_one("SELECT pg_switch_wal()", &[])?.get(0); + let xlog_switch_record_end: PgLsn = client.query_one("SELECT pg_switch_wal()", &[])?.get(0); let next_segment = PgLsn::from(0x0200_0000); ensure!( - after_xlog_switch < next_segment, - "XLOG_SWITCH message ended on or after the expected segment boundary: {} > {}", - after_xlog_switch, + xlog_switch_record_end < next_segment, + "XLOG_SWITCH record ended on or after the expected segment boundary: {} > {}", + xlog_switch_record_end, next_segment ); ensure!( - u64::from(after_xlog_switch) as usize % XLOG_BLCKSZ == XLOG_SIZE_OF_XLOG_SHORT_PHD, + u64::from(xlog_switch_record_end) as usize % XLOG_BLCKSZ == XLOG_SIZE_OF_XLOG_SHORT_PHD, "XLOG_SWITCH message ended not on page boundary: {}, offset = {}", - after_xlog_switch, - u64::from(after_xlog_switch) as usize % XLOG_BLCKSZ + xlog_switch_record_end, + u64::from(xlog_switch_record_end) as usize % XLOG_BLCKSZ ); - Ok((vec![before_xlog_switch, after_xlog_switch], next_segment)) + Ok(vec![before_xlog_switch, xlog_switch_record_end]) } } -fn craft_single_logical_message( +/// Write ~16MB logical message; it should cross WAL segment. +fn craft_seg_size_logical_message( client: &mut impl postgres::GenericClient, transactional: bool, -) -> anyhow::Result<(Vec, PgLsn)> { +) -> anyhow::Result> { craft_internal(client, |client, initial_lsn| { ensure!( initial_lsn < PgLsn::from(0x0200_0000 - 1024 * 1024), @@ -405,34 +405,24 @@ fn craft_single_logical_message( "Logical message crossed two segments" ); - if transactional { - // Transactional logical messages are part of a transaction, so the one above is - // followed by a small COMMIT record. - - let after_message_lsn = client.pg_current_wal_insert_lsn()?; - ensure!( - message_lsn < after_message_lsn, - "No record found after the emitted message" - ); - Ok((vec![message_lsn], Some(after_message_lsn))) - } else { - Ok((Vec::new(), Some(message_lsn))) - } + Ok(vec![message_lsn]) }) } pub struct WalRecordCrossingSegmentFollowedBySmallOne; impl Crafter for WalRecordCrossingSegmentFollowedBySmallOne { const NAME: &'static str = "wal_record_crossing_segment_followed_by_small_one"; - fn craft(client: &mut impl postgres::GenericClient) -> anyhow::Result<(Vec, PgLsn)> { - craft_single_logical_message(client, true) + fn craft(client: &mut impl postgres::GenericClient) -> anyhow::Result> { + // Transactional message crossing WAL segment will be followed by small + // commit record. + craft_seg_size_logical_message(client, true) } } pub struct LastWalRecordCrossingSegment; impl Crafter for LastWalRecordCrossingSegment { const NAME: &'static str = "last_wal_record_crossing_segment"; - fn craft(client: &mut impl postgres::GenericClient) -> anyhow::Result<(Vec, PgLsn)> { - craft_single_logical_message(client, false) + fn craft(client: &mut impl postgres::GenericClient) -> anyhow::Result> { + craft_seg_size_logical_message(client, false) } } diff --git a/libs/postgres_ffi/wal_craft/src/xlog_utils_test.rs b/libs/postgres_ffi/wal_craft/src/xlog_utils_test.rs index 6ff4c563b226..496458b2e42d 100644 --- a/libs/postgres_ffi/wal_craft/src/xlog_utils_test.rs +++ b/libs/postgres_ffi/wal_craft/src/xlog_utils_test.rs @@ -11,13 +11,15 @@ use utils::const_assert; use utils::lsn::Lsn; fn init_logging() { - let _ = env_logger::Builder::from_env(env_logger::Env::default().default_filter_or( - format!("crate=info,postgres_ffi::{PG_MAJORVERSION}::xlog_utils=trace"), - )) + let _ = env_logger::Builder::from_env(env_logger::Env::default().default_filter_or(format!( + "crate=info,postgres_ffi::{PG_MAJORVERSION}::xlog_utils=trace" + ))) .is_test(true) .try_init(); } +/// Test that find_end_of_wal returns the same results as pg_dump on various +/// WALs created by Crafter. fn test_end_of_wal(test_name: &str) { use crate::*; @@ -38,13 +40,13 @@ fn test_end_of_wal(test_name: &str) { } cfg.initdb().unwrap(); let srv = cfg.start_server().unwrap(); - let (intermediate_lsns, expected_end_of_wal_partial) = - C::craft(&mut srv.connect_with_timeout().unwrap()).unwrap(); + let intermediate_lsns = C::craft(&mut srv.connect_with_timeout().unwrap()).unwrap(); let intermediate_lsns: Vec = intermediate_lsns .iter() .map(|&lsn| u64::from(lsn).into()) .collect(); - let expected_end_of_wal: Lsn = u64::from(expected_end_of_wal_partial).into(); + // Kill postgres. Note that it might have inserted to WAL something after + // 'craft' did its job. srv.kill(); // Check find_end_of_wal on the initial WAL @@ -56,7 +58,7 @@ fn test_end_of_wal(test_name: &str) { .filter(|fname| IsXLogFileName(fname)) .max() .unwrap(); - check_pg_waldump_end_of_wal(&cfg, &last_segment, expected_end_of_wal); + let expected_end_of_wal = find_pg_waldump_end_of_wal(&cfg, &last_segment); for start_lsn in intermediate_lsns .iter() .chain(std::iter::once(&expected_end_of_wal)) @@ -91,11 +93,7 @@ fn test_end_of_wal(test_name: &str) { } } -fn check_pg_waldump_end_of_wal( - cfg: &crate::Conf, - last_segment: &str, - expected_end_of_wal: Lsn, -) { +fn find_pg_waldump_end_of_wal(cfg: &crate::Conf, last_segment: &str) -> Lsn { // Get the actual end of WAL by pg_waldump let waldump_output = cfg .pg_waldump("000000010000000000000001", last_segment) @@ -113,11 +111,8 @@ fn check_pg_waldump_end_of_wal( } }; let waldump_wal_end = Lsn::from_str(caps.get(1).unwrap().as_str()).unwrap(); - info!( - "waldump erred on {}, expected wal end at {}", - waldump_wal_end, expected_end_of_wal - ); - assert_eq!(waldump_wal_end, expected_end_of_wal); + info!("waldump erred on {}", waldump_wal_end); + waldump_wal_end } fn check_end_of_wal( @@ -210,9 +205,9 @@ pub fn test_update_next_xid() { #[test] pub fn test_encode_logical_message() { let expected = [ - 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 21, 0, 0, 170, 34, 166, 227, 255, - 38, 0, 0, 0, 0, 0, 0, 0, 0, 7, 0, 0, 0, 0, 0, 0, 0, 7, 0, 0, 0, 0, 0, 0, 0, 112, 114, - 101, 102, 105, 120, 0, 109, 101, 115, 115, 97, 103, 101, + 64, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 21, 0, 0, 170, 34, 166, 227, 255, 38, + 0, 0, 0, 0, 0, 0, 0, 0, 7, 0, 0, 0, 0, 0, 0, 0, 7, 0, 0, 0, 0, 0, 0, 0, 112, 114, 101, 102, + 105, 120, 0, 109, 101, 115, 115, 97, 103, 101, ]; let actual = encode_logical_message("prefix", "message"); assert_eq!(expected, actual[..]); From 271314230848a4f978aa6fa589bae8583b3e2740 Mon Sep 17 00:00:00 2001 From: John Spray Date: Mon, 25 Mar 2024 14:35:24 +0000 Subject: [PATCH 39/53] tests: stabilize compat tests (#7227) This test had two flaky failure modes: - pageserver log error for timeline not found: this resulted from changes for DR when timeline destroy/create was added, but endpoint was left running during that operation. - storage controller log error because the test was running for long enough that a background reconcile happened at almost the exact moment of test teardown, and our test fixtures tear down the pageservers before the controller. Closes: #7224 --- test_runner/fixtures/neon_fixtures.py | 6 +++++- test_runner/regress/test_compatibility.py | 9 ++++++++- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index f8994a8dccb3..f22ce10c2024 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -1155,13 +1155,17 @@ def stop(self, immediate=False, ps_assert_metric_no_errors=False): After this method returns, there should be no child processes running. """ self.endpoints.stop_all() + + # Stop storage controller before pageservers: we don't want it to spuriously + # detect a pageserver "failure" during test teardown + self.storage_controller.stop(immediate=immediate) + for sk in self.safekeepers: sk.stop(immediate=immediate) for pageserver in self.pageservers: if ps_assert_metric_no_errors: pageserver.assert_no_metric_errors() pageserver.stop(immediate=immediate) - self.storage_controller.stop(immediate=immediate) self.broker.stop(immediate=immediate) @property diff --git a/test_runner/regress/test_compatibility.py b/test_runner/regress/test_compatibility.py index e0bb4c206281..5406acc00517 100644 --- a/test_runner/regress/test_compatibility.py +++ b/test_runner/regress/test_compatibility.py @@ -267,9 +267,10 @@ def test_forward_compatibility( def check_neon_works(env: NeonEnv, test_output_dir: Path, sql_dump_path: Path, repo_dir: Path): ep = env.endpoints.create_start("main") + connstr = ep.connstr() + pg_bin = PgBin(test_output_dir, env.pg_distrib_dir, env.pg_version) - connstr = ep.connstr() pg_bin.run_capture( ["pg_dumpall", f"--dbname={connstr}", f"--file={test_output_dir / 'dump.sql'}"] ) @@ -286,6 +287,9 @@ def check_neon_works(env: NeonEnv, test_output_dir: Path, sql_dump_path: Path, r timeline_id = env.initial_timeline pg_version = env.pg_version + # Stop endpoint while we recreate timeline + ep.stop() + try: pageserver_http.timeline_preserve_initdb_archive(tenant_id, timeline_id) except PageserverApiException as e: @@ -310,6 +314,9 @@ def check_neon_works(env: NeonEnv, test_output_dir: Path, sql_dump_path: Path, r existing_initdb_timeline_id=timeline_id, ) + # Timeline exists again: restart the endpoint + ep.start() + pg_bin.run_capture( ["pg_dumpall", f"--dbname={connstr}", f"--file={test_output_dir / 'dump-from-wal.sql'}"] ) From d837ce0686046837f558d0202716c22937d6213b Mon Sep 17 00:00:00 2001 From: George Ma <164313692+availhang@users.noreply.github.com> Date: Mon, 25 Mar 2024 23:43:02 +0800 Subject: [PATCH 40/53] chore: remove repetitive words (#7206) Signed-off-by: availhang --- control_plane/src/background_process.rs | 2 +- control_plane/src/endpoint.rs | 2 +- libs/tenant_size_model/tests/tests.rs | 2 +- libs/vm_monitor/src/runner.rs | 2 +- pageserver/src/metrics.rs | 2 +- pageserver/src/virtual_file.rs | 2 +- test_runner/regress/test_auth.py | 2 +- test_runner/regress/test_remote_storage.py | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/control_plane/src/background_process.rs b/control_plane/src/background_process.rs index 0e59b28230bd..2fced7d77869 100644 --- a/control_plane/src/background_process.rs +++ b/control_plane/src/background_process.rs @@ -294,7 +294,7 @@ where // is in state 'taken' but the thread that would unlock it is // not there. // 2. A rust object that represented some external resource in the - // parent now got implicitly copied by the the fork, even though + // parent now got implicitly copied by the fork, even though // the object's type is not `Copy`. The parent program may use // non-copyability as way to enforce unique ownership of an // external resource in the typesystem. The fork breaks that diff --git a/control_plane/src/endpoint.rs b/control_plane/src/endpoint.rs index 5206222961ce..03f7db99fbcb 100644 --- a/control_plane/src/endpoint.rs +++ b/control_plane/src/endpoint.rs @@ -12,7 +12,7 @@ //! //! The endpoint is managed by the `compute_ctl` binary. When an endpoint is //! started, we launch `compute_ctl` It synchronizes the safekeepers, downloads -//! the basebackup from the pageserver to initialize the the data directory, and +//! the basebackup from the pageserver to initialize the data directory, and //! finally launches the PostgreSQL process. It watches the PostgreSQL process //! until it exits. //! diff --git a/libs/tenant_size_model/tests/tests.rs b/libs/tenant_size_model/tests/tests.rs index 7660d41c56e7..0ffea0f2cdc6 100644 --- a/libs/tenant_size_model/tests/tests.rs +++ b/libs/tenant_size_model/tests/tests.rs @@ -247,7 +247,7 @@ fn scenario_4() { // // This is in total 5000 + 1000 + 5000 + 1000 = 12000 // - // (If we used the the method from the previous scenario, and + // (If we used the method from the previous scenario, and // kept only snapshot at the branch point, we'd need to keep // all the WAL between 10000-18000 on the main branch, so // the total size would be 5000 + 1000 + 8000 = 14000. The diff --git a/libs/vm_monitor/src/runner.rs b/libs/vm_monitor/src/runner.rs index ba37966476c5..ca02637ecf9d 100644 --- a/libs/vm_monitor/src/runner.rs +++ b/libs/vm_monitor/src/runner.rs @@ -69,7 +69,7 @@ pub struct Config { /// should be removed once we have a better solution there. sys_buffer_bytes: u64, - /// Minimum fraction of total system memory reserved *before* the the cgroup threshold; in + /// Minimum fraction of total system memory reserved *before* the cgroup threshold; in /// other words, providing a ceiling for the highest value of the threshold by enforcing that /// there's at least `cgroup_min_overhead_fraction` of the total memory remaining beyond the /// threshold. diff --git a/pageserver/src/metrics.rs b/pageserver/src/metrics.rs index 6de284ee8b72..cc661194e99c 100644 --- a/pageserver/src/metrics.rs +++ b/pageserver/src/metrics.rs @@ -435,7 +435,7 @@ pub(crate) static RESIDENT_PHYSICAL_SIZE_GLOBAL: Lazy = Lazy::new(|| static REMOTE_PHYSICAL_SIZE: Lazy = Lazy::new(|| { register_uint_gauge_vec!( "pageserver_remote_physical_size", - "The size of the layer files present in the remote storage that are listed in the the remote index_part.json.", + "The size of the layer files present in the remote storage that are listed in the remote index_part.json.", // Corollary: If any files are missing from the index part, they won't be included here. &["tenant_id", "shard_id", "timeline_id"] ) diff --git a/pageserver/src/virtual_file.rs b/pageserver/src/virtual_file.rs index dee36d8afd6c..0cf6a0019b2f 100644 --- a/pageserver/src/virtual_file.rs +++ b/pageserver/src/virtual_file.rs @@ -782,7 +782,7 @@ where } } // NB: don't use `buf.is_empty()` here; it is from the - // `impl Deref for Slice { Target = [u8] }`; the the &[u8] + // `impl Deref for Slice { Target = [u8] }`; the &[u8] // returned by it only covers the initialized portion of `buf`. // Whereas we're interested in ensuring that we filled the entire // buffer that the user passed in. diff --git a/test_runner/regress/test_auth.py b/test_runner/regress/test_auth.py index ea88b5d8e96c..bb622c0d5903 100644 --- a/test_runner/regress/test_auth.py +++ b/test_runner/regress/test_auth.py @@ -105,7 +105,7 @@ def test_pageserver_multiple_keys(neon_env_builder: NeonEnvBuilder): # The neon_local tool generates one key pair at a hardcoded path by default. # As a preparation for our test, move the public key of the key pair into a # directory at the same location as the hardcoded path by: - # 1. moving the the file at `configured_pub_key_path` to a temporary location + # 1. moving the file at `configured_pub_key_path` to a temporary location # 2. creating a new directory at `configured_pub_key_path` # 3. moving the file from the temporary location into the newly created directory configured_pub_key_path = Path(env.repo_dir) / "auth_public_key.pem" diff --git a/test_runner/regress/test_remote_storage.py b/test_runner/regress/test_remote_storage.py index 05f769b0e3e8..986d6c4dbf1c 100644 --- a/test_runner/regress/test_remote_storage.py +++ b/test_runner/regress/test_remote_storage.py @@ -838,7 +838,7 @@ def test_compaction_waits_for_upload( # upload_stuck_layers and the original initdb L0 client.timeline_checkpoint(tenant_id, timeline_id) - # as uploads are paused, the the upload_stuck_layers should still be with us + # as uploads are paused, the upload_stuck_layers should still be with us for name in upload_stuck_layers: path = env.pageserver.timeline_dir(tenant_id, timeline_id) / name assert path.exists(), "uploads are stuck still over compaction" From f72415e1fd952274f132a47baaddbf0a4ac912de Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 25 Mar 2024 18:42:18 +0100 Subject: [PATCH 41/53] refactor(remote_timeline_client): infallible stop() and shutdown() (#7234) preliminary refactoring for https://github.com/neondatabase/neon/pull/7233 part of #7062 --- pageserver/src/tenant.rs | 2 +- .../src/tenant/remote_timeline_client.rs | 77 ++++++++----------- pageserver/src/tenant/timeline.rs | 15 +--- pageserver/src/tenant/timeline/delete.rs | 18 +---- pageserver/src/tenant/upload_queue.rs | 14 +++- 5 files changed, 51 insertions(+), 75 deletions(-) diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index 7bd85b6fd565..b923e473cef1 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -2141,7 +2141,7 @@ impl Tenant { // Shut down the timeline's remote client: this means that the indices we write // for child shards will not be invalidated by the parent shard deleting layers. - tl_client.shutdown().await?; + tl_client.shutdown().await; // Download methods can still be used after shutdown, as they don't flow through the remote client's // queue. In principal the RemoteTimelineClient could provide this without downloading it, but this diff --git a/pageserver/src/tenant/remote_timeline_client.rs b/pageserver/src/tenant/remote_timeline_client.rs index c0a150eb0d90..b4b3243d11a8 100644 --- a/pageserver/src/tenant/remote_timeline_client.rs +++ b/pageserver/src/tenant/remote_timeline_client.rs @@ -217,7 +217,7 @@ use crate::task_mgr::shutdown_token; use crate::tenant::debug_assert_current_span_has_tenant_and_timeline_id; use crate::tenant::remote_timeline_client::download::download_retry; use crate::tenant::storage_layer::AsLayerDesc; -use crate::tenant::upload_queue::Delete; +use crate::tenant::upload_queue::{Delete, UploadQueueStoppedDeletable}; use crate::tenant::TIMELINES_SEGMENT_NAME; use crate::{ config::PageServerConf, @@ -265,15 +265,6 @@ pub enum MaybeDeletedIndexPart { Deleted(IndexPart), } -/// Errors that can arise when calling [`RemoteTimelineClient::stop`]. -#[derive(Debug, thiserror::Error)] -pub enum StopError { - /// Returned if the upload queue was never initialized. - /// See [`RemoteTimelineClient::init_upload_queue`] and [`RemoteTimelineClient::init_upload_queue_for_empty_remote`]. - #[error("queue is not initialized")] - QueueUninitialized, -} - #[derive(Debug, thiserror::Error)] pub enum PersistIndexPartWithDeletedFlagError { #[error("another task is already setting the deleted_flag, started at {0:?}")] @@ -390,15 +381,10 @@ impl RemoteTimelineClient { "bug: it is responsibility of the caller to provide index part from MaybeDeletedIndexPart::Deleted" ))?; - { - let mut upload_queue = self.upload_queue.lock().unwrap(); - upload_queue.initialize_with_current_remote_index_part(index_part)?; - self.update_remote_physical_size_gauge(Some(index_part)); - } - // also locks upload queue, without dropping the guard above it will be a deadlock - self.stop().expect("initialized line above"); - let mut upload_queue = self.upload_queue.lock().unwrap(); + upload_queue.initialize_with_current_remote_index_part(index_part)?; + self.update_remote_physical_size_gauge(Some(index_part)); + self.stop_impl(&mut upload_queue); upload_queue .stopped_mut() @@ -412,7 +398,8 @@ impl RemoteTimelineClient { match &mut *self.upload_queue.lock().unwrap() { UploadQueue::Uninitialized => None, UploadQueue::Initialized(q) => q.get_last_remote_consistent_lsn_projected(), - UploadQueue::Stopped(q) => q + UploadQueue::Stopped(UploadQueueStopped::Uninitialized) => None, + UploadQueue::Stopped(UploadQueueStopped::Deletable(q)) => q .upload_queue_for_deletion .get_last_remote_consistent_lsn_projected(), } @@ -422,7 +409,8 @@ impl RemoteTimelineClient { match &mut *self.upload_queue.lock().unwrap() { UploadQueue::Uninitialized => None, UploadQueue::Initialized(q) => Some(q.get_last_remote_consistent_lsn_visible()), - UploadQueue::Stopped(q) => Some( + UploadQueue::Stopped(UploadQueueStopped::Uninitialized) => None, + UploadQueue::Stopped(UploadQueueStopped::Deletable(q)) => Some( q.upload_queue_for_deletion .get_last_remote_consistent_lsn_visible(), ), @@ -889,7 +877,7 @@ impl RemoteTimelineClient { /// Wait for all previously scheduled operations to complete, and then stop. /// /// Not cancellation safe - pub(crate) async fn shutdown(self: &Arc) -> Result<(), StopError> { + pub(crate) async fn shutdown(self: &Arc) { // On cancellation the queue is left in ackward state of refusing new operations but // proper stop is yet to be called. On cancel the original or some later task must call // `stop` or `shutdown`. @@ -900,8 +888,12 @@ impl RemoteTimelineClient { let fut = { let mut guard = self.upload_queue.lock().unwrap(); let upload_queue = match &mut *guard { - UploadQueue::Stopped(_) => return Ok(()), - UploadQueue::Uninitialized => return Err(StopError::QueueUninitialized), + UploadQueue::Stopped(_) => return, + UploadQueue::Uninitialized => { + // transition into Stopped state + self.stop_impl(&mut guard); + return; + } UploadQueue::Initialized(ref mut init) => init, }; @@ -933,7 +925,7 @@ impl RemoteTimelineClient { } } - self.stop() + self.stop(); } /// Set the deleted_at field in the remote index file. @@ -1314,12 +1306,7 @@ impl RemoteTimelineClient { // upload finishes or times out soon enough. if cancel.is_cancelled() { info!("upload task cancelled by shutdown request"); - match self.stop() { - Ok(()) => {} - Err(StopError::QueueUninitialized) => { - unreachable!("we never launch an upload task if the queue is uninitialized, and once it is initialized, we never go back") - } - } + self.stop(); return; } @@ -1574,17 +1561,23 @@ impl RemoteTimelineClient { /// In-progress operations will still be running after this function returns. /// Use `task_mgr::shutdown_tasks(None, Some(self.tenant_id), Some(timeline_id))` /// to wait for them to complete, after calling this function. - pub(crate) fn stop(&self) -> Result<(), StopError> { + pub(crate) fn stop(&self) { // Whichever *task* for this RemoteTimelineClient grabs the mutex first will transition the queue // into stopped state, thereby dropping all off the queued *ops* which haven't become *tasks* yet. // The other *tasks* will come here and observe an already shut down queue and hence simply wrap up their business. let mut guard = self.upload_queue.lock().unwrap(); - match &mut *guard { - UploadQueue::Uninitialized => Err(StopError::QueueUninitialized), + self.stop_impl(&mut guard); + } + + fn stop_impl(&self, guard: &mut std::sync::MutexGuard) { + match &mut **guard { + UploadQueue::Uninitialized => { + info!("UploadQueue is in state Uninitialized, nothing to do"); + **guard = UploadQueue::Stopped(UploadQueueStopped::Uninitialized); + } UploadQueue::Stopped(_) => { // nothing to do info!("another concurrent task already shut down the queue"); - Ok(()) } UploadQueue::Initialized(initialized) => { info!("shutting down upload queue"); @@ -1617,11 +1610,13 @@ impl RemoteTimelineClient { }; let upload_queue = std::mem::replace( - &mut *guard, - UploadQueue::Stopped(UploadQueueStopped { - upload_queue_for_deletion, - deleted_at: SetDeletedFlagProgress::NotRunning, - }), + &mut **guard, + UploadQueue::Stopped(UploadQueueStopped::Deletable( + UploadQueueStoppedDeletable { + upload_queue_for_deletion, + deleted_at: SetDeletedFlagProgress::NotRunning, + }, + )), ); if let UploadQueue::Initialized(qi) = upload_queue { qi @@ -1650,10 +1645,6 @@ impl RemoteTimelineClient { // which is exactly what we want to happen. drop(op); } - - // We're done. - drop(guard); - Ok(()) } } } diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 15ffa72aaac3..6c6bb4b78875 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -54,6 +54,7 @@ use std::{ ops::ControlFlow, }; +use crate::deletion_queue::DeletionQueueClient; use crate::tenant::timeline::logical_size::CurrentLogicalSize; use crate::tenant::{ layer_map::{LayerMap, SearchResult}, @@ -64,7 +65,6 @@ use crate::{ disk_usage_eviction_task::DiskUsageEvictionInfo, pgdatadir_mapping::CollectKeySpaceError, }; -use crate::{deletion_queue::DeletionQueueClient, tenant::remote_timeline_client::StopError}; use crate::{ disk_usage_eviction_task::finite_f32, tenant::storage_layer::{ @@ -1241,11 +1241,7 @@ impl Timeline { // what is problematic is the shutting down of RemoteTimelineClient, because // obviously it does not make sense to stop while we wait for it, but what // about corner cases like s3 suddenly hanging up? - if let Err(e) = client.shutdown().await { - // Non-fatal. Shutdown is infallible. Failures to flush just mean that - // we have some extra WAL replay to do next time the timeline starts. - warn!("failed to flush to remote storage: {e:#}"); - } + client.shutdown().await; } } Err(e) => { @@ -1282,12 +1278,7 @@ impl Timeline { // Shut down remote timeline client: this gracefully moves its metadata into its Stopping state in // case our caller wants to use that for a deletion if let Some(remote_client) = self.remote_client.as_ref() { - match remote_client.stop() { - Ok(()) => {} - Err(StopError::QueueUninitialized) => { - // Shutting down during initialization is legal - } - } + remote_client.stop(); } tracing::debug!("Waiting for tasks..."); diff --git a/pageserver/src/tenant/timeline/delete.rs b/pageserver/src/tenant/timeline/delete.rs index d2272fc75fb4..e9afbfd8ba05 100644 --- a/pageserver/src/tenant/timeline/delete.rs +++ b/pageserver/src/tenant/timeline/delete.rs @@ -16,9 +16,7 @@ use crate::{ tenant::{ debug_assert_current_span_has_tenant_and_timeline_id, metadata::TimelineMetadata, - remote_timeline_client::{ - self, PersistIndexPartWithDeletedFlagError, RemoteTimelineClient, - }, + remote_timeline_client::{PersistIndexPartWithDeletedFlagError, RemoteTimelineClient}, CreateTimelineCause, DeleteTimelineError, Tenant, }, }; @@ -50,19 +48,7 @@ async fn stop_tasks(timeline: &Timeline) -> Result<(), DeleteTimelineError> { // Prevent new uploads from starting. if let Some(remote_client) = timeline.remote_client.as_ref() { - let res = remote_client.stop(); - match res { - Ok(()) => {} - Err(e) => match e { - remote_timeline_client::StopError::QueueUninitialized => { - // This case shouldn't happen currently because the - // load and attach code bails out if _any_ of the timeline fails to fetch its IndexPart. - // That is, before we declare the Tenant as Active. - // But we only allow calls to delete_timeline on Active tenants. - return Err(DeleteTimelineError::Other(anyhow::anyhow!("upload queue is uninitialized, likely the timeline was in Broken state prior to this call because it failed to fetch IndexPart during load or attach, check the logs"))); - } - }, - } + remote_client.stop(); } // Stop & wait for the remaining timeline tasks, including upload tasks. diff --git a/pageserver/src/tenant/upload_queue.rs b/pageserver/src/tenant/upload_queue.rs index a5516bb9a998..0bf4d1e5991f 100644 --- a/pageserver/src/tenant/upload_queue.rs +++ b/pageserver/src/tenant/upload_queue.rs @@ -121,11 +121,16 @@ pub(super) enum SetDeletedFlagProgress { Successful(NaiveDateTime), } -pub(super) struct UploadQueueStopped { +pub(super) struct UploadQueueStoppedDeletable { pub(super) upload_queue_for_deletion: UploadQueueInitialized, pub(super) deleted_at: SetDeletedFlagProgress, } +pub(super) enum UploadQueueStopped { + Deletable(UploadQueueStoppedDeletable), + Uninitialized, +} + #[derive(thiserror::Error, Debug)] pub(crate) enum NotInitialized { #[error("queue is in state Uninitialized")] @@ -249,12 +254,15 @@ impl UploadQueue { } } - pub(crate) fn stopped_mut(&mut self) -> anyhow::Result<&mut UploadQueueStopped> { + pub(crate) fn stopped_mut(&mut self) -> anyhow::Result<&mut UploadQueueStoppedDeletable> { match self { UploadQueue::Initialized(_) | UploadQueue::Uninitialized => { anyhow::bail!("queue is in state {}", self.as_str()) } - UploadQueue::Stopped(stopped) => Ok(stopped), + UploadQueue::Stopped(UploadQueueStopped::Uninitialized) => { + anyhow::bail!("queue is in state Stopped(Uninitialized)") + } + UploadQueue::Stopped(UploadQueueStopped::Deletable(deletable)) => Ok(deletable), } } } From 6313f1fa7a36a91a83158a381bd850f0147cb772 Mon Sep 17 00:00:00 2001 From: John Spray Date: Tue, 26 Mar 2024 09:56:47 +0000 Subject: [PATCH 42/53] tests: tolerate transient unavailability in test_sharding_split_failures (#7223) ## Problem While most forms of split rollback don't interrupt clients, there are a couple of cases that do -- this interruption is brief, driven by the time it takes the controller to kick off Reconcilers during the async abort of the split, so it's operationally fine, but can trip up a test. - #7148 ## Summary of changes - Relax test check to require that the tenant is eventually available after split failure, rather than immediately. In the vast majority of cases this will pass on the first iteration. --- test_runner/regress/test_sharding.py | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/test_runner/regress/test_sharding.py b/test_runner/regress/test_sharding.py index e6318aff6889..9aebf16c68f8 100644 --- a/test_runner/regress/test_sharding.py +++ b/test_runner/regress/test_sharding.py @@ -874,11 +874,17 @@ def finish_split(): workload.validate() if failure.expect_available(): - # Even though the split failed partway through, this should not have interrupted - # clients. Disable waiting for pageservers in the workload helper, because our - # failpoints may prevent API access. - # This only applies for failure modes that leave pageserver page_service API available. - workload.churn_rows(10, upload=False, ingest=False) + # Even though the split failed partway through, this should not leave the tenant in + # an unavailable state. + # - Disable waiting for pageservers in the workload helper, because our + # failpoints may prevent API access. This only applies for failure modes that + # leave pageserver page_service API available. + # - This is a wait_until because clients may see transient errors in some split error cases, + # e.g. while waiting for a storage controller to re-attach a parent shard if we failed + # inside the pageserver and the storage controller responds by detaching children and attaching + # parents concurrently (https://github.com/neondatabase/neon/issues/7148) + wait_until(10, 1, lambda: workload.churn_rows(10, upload=False, ingest=False)) # type: ignore + workload.validate() if failure.fails_forward(env): From 5dee58f492a80a76e1b01b32d4f02a42fba7cd47 Mon Sep 17 00:00:00 2001 From: John Spray Date: Tue, 26 Mar 2024 10:59:16 +0000 Subject: [PATCH 43/53] tests: wait for uploads in test_secondary_downloads (#7220) ## Problem - https://github.com/neondatabase/neon/issues/6966 This test occasionally failed with some layers unexpectedly not present on the secondary pageserver. The issue in that failure is the attached pageserver uploading heatmaps that refer to not-yet-uploaded layers. ## Summary of changes After uploading heatmap, drain upload queue on attached pageserver, to guarantee that all the layers referenced in the haetmap are uploaded. --- test_runner/regress/test_pageserver_secondary.py | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/test_runner/regress/test_pageserver_secondary.py b/test_runner/regress/test_pageserver_secondary.py index 2e5713660750..25510c50e6bd 100644 --- a/test_runner/regress/test_pageserver_secondary.py +++ b/test_runner/regress/test_pageserver_secondary.py @@ -11,6 +11,7 @@ assert_prefix_empty, poll_for_remote_storage_iterations, tenant_delete_wait_completed, + wait_for_upload_queue_empty, ) from fixtures.remote_storage import LocalFsStorage, RemoteStorageKind, S3Storage from fixtures.types import TenantId, TimelineId @@ -472,6 +473,10 @@ def test_secondary_downloads(neon_env_builder: NeonEnvBuilder): log.info("Synchronizing after initial write...") ps_attached.http_client().tenant_heatmap_upload(tenant_id) + # Ensure that everything which appears in the heatmap is also present in S3: heatmap writers + # are allowed to upload heatmaps that reference layers which are only enqueued for upload + wait_for_upload_queue_empty(ps_attached.http_client(), tenant_id, timeline_id) + ps_secondary.http_client().tenant_secondary_download(tenant_id) assert list_layers(ps_attached, tenant_id, timeline_id) == list_layers( @@ -484,6 +489,11 @@ def test_secondary_downloads(neon_env_builder: NeonEnvBuilder): workload.churn_rows(128, ps_attached.id) ps_attached.http_client().tenant_heatmap_upload(tenant_id) + + # Ensure that everything which appears in the heatmap is also present in S3: heatmap writers + # are allowed to upload heatmaps that reference layers which are only enqueued for upload + wait_for_upload_queue_empty(ps_attached.http_client(), tenant_id, timeline_id) + ps_secondary.http_client().tenant_secondary_download(tenant_id) assert list_layers(ps_attached, tenant_id, timeline_id) == list_layers( From 6c18109734e77bba7b93862d7f5ca54879bf5052 Mon Sep 17 00:00:00 2001 From: Anna Khanova <32508607+khanova@users.noreply.github.com> Date: Tue, 26 Mar 2024 12:27:48 +0100 Subject: [PATCH 44/53] proxy: reuse sess_id as request_id for the cplane requests (#7245) ## Problem https://github.com/neondatabase/cloud/issues/11599 ## Summary of changes Reuse the same sess_id for requests within the one session. TODO: get rid of `session_id` in query params. --- proxy/src/console/provider/neon.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/proxy/src/console/provider/neon.rs b/proxy/src/console/provider/neon.rs index 89ebfa57f1b1..289b0c08f78a 100644 --- a/proxy/src/console/provider/neon.rs +++ b/proxy/src/console/provider/neon.rs @@ -55,7 +55,7 @@ impl Api { ctx: &mut RequestMonitoring, user_info: &ComputeUserInfo, ) -> Result { - let request_id = uuid::Uuid::new_v4().to_string(); + let request_id = ctx.session_id.to_string(); let application_name = ctx.console_application_name(); async { let request = self @@ -112,7 +112,7 @@ impl Api { ctx: &mut RequestMonitoring, user_info: &ComputeUserInfo, ) -> Result { - let request_id = uuid::Uuid::new_v4().to_string(); + let request_id = ctx.session_id.to_string(); let application_name = ctx.console_application_name(); async { let mut request_builder = self From ad072de4209193fd21314cf7f03f14df4fa55eb1 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Tue, 26 Mar 2024 15:24:18 +0100 Subject: [PATCH 45/53] Revert "pageserver: use a single tokio runtime (#6555)" (#7246) --- pageserver/src/bin/pageserver.rs | 82 ++++++++++--------- pageserver/src/consumption_metrics.rs | 3 +- pageserver/src/control_plane_client.rs | 4 +- pageserver/src/disk_usage_eviction_task.rs | 3 +- pageserver/src/page_service.rs | 1 + pageserver/src/task_mgr.rs | 37 +++++++-- pageserver/src/tenant.rs | 2 +- pageserver/src/tenant/delete.rs | 1 + pageserver/src/tenant/mgr.rs | 4 + .../src/tenant/remote_timeline_client.rs | 11 +++ pageserver/src/tenant/secondary.rs | 4 +- pageserver/src/tenant/storage_layer/layer.rs | 4 +- pageserver/src/tenant/tasks.rs | 6 +- pageserver/src/tenant/timeline.rs | 6 +- pageserver/src/tenant/timeline/delete.rs | 1 + .../src/tenant/timeline/eviction_task.rs | 3 +- pageserver/src/tenant/timeline/walreceiver.rs | 5 +- .../walreceiver/walreceiver_connection.rs | 18 ++-- test_runner/regress/test_backpressure.py | 2 +- test_runner/regress/test_timeline_size.py | 26 ++---- 20 files changed, 131 insertions(+), 92 deletions(-) diff --git a/pageserver/src/bin/pageserver.rs b/pageserver/src/bin/pageserver.rs index f4a231f217a4..ef616c0a390c 100644 --- a/pageserver/src/bin/pageserver.rs +++ b/pageserver/src/bin/pageserver.rs @@ -15,9 +15,9 @@ use metrics::launch_timestamp::{set_launch_timestamp_metric, LaunchTimestamp}; use pageserver::control_plane_client::ControlPlaneClient; use pageserver::disk_usage_eviction_task::{self, launch_disk_usage_global_eviction_task}; use pageserver::metrics::{STARTUP_DURATION, STARTUP_IS_LOADING}; +use pageserver::task_mgr::WALRECEIVER_RUNTIME; use pageserver::tenant::{secondary, TenantSharedResources}; use remote_storage::GenericRemoteStorage; -use tokio::signal::unix::SignalKind; use tokio::time::Instant; use tracing::*; @@ -28,7 +28,7 @@ use pageserver::{ deletion_queue::DeletionQueue, http, page_cache, page_service, task_mgr, task_mgr::TaskKind, - task_mgr::THE_RUNTIME, + task_mgr::{BACKGROUND_RUNTIME, COMPUTE_REQUEST_RUNTIME, MGMT_REQUEST_RUNTIME}, tenant::mgr, virtual_file, }; @@ -323,7 +323,7 @@ fn start_pageserver( // Launch broker client // The storage_broker::connect call needs to happen inside a tokio runtime thread. - let broker_client = THE_RUNTIME + let broker_client = WALRECEIVER_RUNTIME .block_on(async { // Note: we do not attempt connecting here (but validate endpoints sanity). storage_broker::connect(conf.broker_endpoint.clone(), conf.broker_keepalive_interval) @@ -391,7 +391,7 @@ fn start_pageserver( conf, ); if let Some(deletion_workers) = deletion_workers { - deletion_workers.spawn_with(THE_RUNTIME.handle()); + deletion_workers.spawn_with(BACKGROUND_RUNTIME.handle()); } // Up to this point no significant I/O has been done: this should have been fast. Record @@ -423,7 +423,7 @@ fn start_pageserver( // Scan the local 'tenants/' directory and start loading the tenants let deletion_queue_client = deletion_queue.new_client(); - let tenant_manager = THE_RUNTIME.block_on(mgr::init_tenant_mgr( + let tenant_manager = BACKGROUND_RUNTIME.block_on(mgr::init_tenant_mgr( conf, TenantSharedResources { broker_client: broker_client.clone(), @@ -435,7 +435,7 @@ fn start_pageserver( ))?; let tenant_manager = Arc::new(tenant_manager); - THE_RUNTIME.spawn({ + BACKGROUND_RUNTIME.spawn({ let shutdown_pageserver = shutdown_pageserver.clone(); let drive_init = async move { // NOTE: unlike many futures in pageserver, this one is cancellation-safe @@ -545,7 +545,7 @@ fn start_pageserver( // Start up the service to handle HTTP mgmt API request. We created the // listener earlier already. { - let _rt_guard = THE_RUNTIME.enter(); + let _rt_guard = MGMT_REQUEST_RUNTIME.enter(); let router_state = Arc::new( http::routes::State::new( @@ -569,6 +569,7 @@ fn start_pageserver( .with_graceful_shutdown(task_mgr::shutdown_watcher()); task_mgr::spawn( + MGMT_REQUEST_RUNTIME.handle(), TaskKind::HttpEndpointListener, None, None, @@ -593,6 +594,7 @@ fn start_pageserver( let local_disk_storage = conf.workdir.join("last_consumption_metrics.json"); task_mgr::spawn( + crate::BACKGROUND_RUNTIME.handle(), TaskKind::MetricsCollection, None, None, @@ -641,6 +643,7 @@ fn start_pageserver( DownloadBehavior::Error, ); task_mgr::spawn( + COMPUTE_REQUEST_RUNTIME.handle(), TaskKind::LibpqEndpointListener, None, None, @@ -664,37 +667,42 @@ fn start_pageserver( let mut shutdown_pageserver = Some(shutdown_pageserver.drop_guard()); // All started up! Now just sit and wait for shutdown signal. - { - THE_RUNTIME.block_on(async move { - let mut sigint = tokio::signal::unix::signal(SignalKind::interrupt()).unwrap(); - let mut sigterm = tokio::signal::unix::signal(SignalKind::terminate()).unwrap(); - let mut sigquit = tokio::signal::unix::signal(SignalKind::quit()).unwrap(); - let signal = tokio::select! { - _ = sigquit.recv() => { - info!("Got signal SIGQUIT. Terminating in immediate shutdown mode",); - std::process::exit(111); - } - _ = sigint.recv() => { "SIGINT" }, - _ = sigterm.recv() => { "SIGTERM" }, - }; - - info!("Got signal {signal}. Terminating gracefully in fast shutdown mode",); - - // This cancels the `shutdown_pageserver` cancellation tree. - // Right now that tree doesn't reach very far, and `task_mgr` is used instead. - // The plan is to change that over time. - shutdown_pageserver.take(); - let bg_remote_storage = remote_storage.clone(); - let bg_deletion_queue = deletion_queue.clone(); - pageserver::shutdown_pageserver( - &tenant_manager, - bg_remote_storage.map(|_| bg_deletion_queue), - 0, - ) - .await; - unreachable!() - }) + use signal_hook::consts::*; + let signal_handler = BACKGROUND_RUNTIME.spawn_blocking(move || { + let mut signals = + signal_hook::iterator::Signals::new([SIGINT, SIGTERM, SIGQUIT]).unwrap(); + return signals + .forever() + .next() + .expect("forever() never returns None unless explicitly closed"); + }); + let signal = BACKGROUND_RUNTIME + .block_on(signal_handler) + .expect("join error"); + match signal { + SIGQUIT => { + info!("Got signal {signal}. Terminating in immediate shutdown mode",); + std::process::exit(111); + } + SIGINT | SIGTERM => { + info!("Got signal {signal}. Terminating gracefully in fast shutdown mode",); + + // This cancels the `shutdown_pageserver` cancellation tree. + // Right now that tree doesn't reach very far, and `task_mgr` is used instead. + // The plan is to change that over time. + shutdown_pageserver.take(); + let bg_remote_storage = remote_storage.clone(); + let bg_deletion_queue = deletion_queue.clone(); + BACKGROUND_RUNTIME.block_on(pageserver::shutdown_pageserver( + &tenant_manager, + bg_remote_storage.map(|_| bg_deletion_queue), + 0, + )); + unreachable!() + } + _ => unreachable!(), + } } } diff --git a/pageserver/src/consumption_metrics.rs b/pageserver/src/consumption_metrics.rs index c82be8c581de..3429e3a0a673 100644 --- a/pageserver/src/consumption_metrics.rs +++ b/pageserver/src/consumption_metrics.rs @@ -1,7 +1,7 @@ //! Periodically collect consumption metrics for all active tenants //! and push them to a HTTP endpoint. use crate::context::{DownloadBehavior, RequestContext}; -use crate::task_mgr::{self, TaskKind}; +use crate::task_mgr::{self, TaskKind, BACKGROUND_RUNTIME}; use crate::tenant::tasks::BackgroundLoopKind; use crate::tenant::{mgr, LogicalSizeCalculationCause, PageReconstructError, Tenant}; use camino::Utf8PathBuf; @@ -61,6 +61,7 @@ pub async fn collect_metrics( let worker_ctx = ctx.detached_child(TaskKind::CalculateSyntheticSize, DownloadBehavior::Download); task_mgr::spawn( + BACKGROUND_RUNTIME.handle(), TaskKind::CalculateSyntheticSize, None, None, diff --git a/pageserver/src/control_plane_client.rs b/pageserver/src/control_plane_client.rs index 55d80c29668a..42c800822b08 100644 --- a/pageserver/src/control_plane_client.rs +++ b/pageserver/src/control_plane_client.rs @@ -173,6 +173,8 @@ impl ControlPlaneGenerationsApi for ControlPlaneClient { register, }; + fail::fail_point!("control-plane-client-re-attach"); + let response: ReAttachResponse = self.retry_http_forever(&re_attach_path, request).await?; tracing::info!( "Received re-attach response with {} tenants", @@ -208,7 +210,7 @@ impl ControlPlaneGenerationsApi for ControlPlaneClient { .collect(), }; - crate::tenant::pausable_failpoint!("control-plane-client-validate"); + fail::fail_point!("control-plane-client-validate"); let response: ValidateResponse = self.retry_http_forever(&re_attach_path, request).await?; diff --git a/pageserver/src/disk_usage_eviction_task.rs b/pageserver/src/disk_usage_eviction_task.rs index 6b68acd1c7d4..92c1475aeff5 100644 --- a/pageserver/src/disk_usage_eviction_task.rs +++ b/pageserver/src/disk_usage_eviction_task.rs @@ -59,7 +59,7 @@ use utils::{completion, id::TimelineId}; use crate::{ config::PageServerConf, metrics::disk_usage_based_eviction::METRICS, - task_mgr::{self, TaskKind}, + task_mgr::{self, TaskKind, BACKGROUND_RUNTIME}, tenant::{ self, mgr::TenantManager, @@ -202,6 +202,7 @@ pub fn launch_disk_usage_global_eviction_task( info!("launching disk usage based eviction task"); task_mgr::spawn( + BACKGROUND_RUNTIME.handle(), TaskKind::DiskUsageEviction, None, None, diff --git a/pageserver/src/page_service.rs b/pageserver/src/page_service.rs index fa1a0f535b50..f3ceb7d3e6e5 100644 --- a/pageserver/src/page_service.rs +++ b/pageserver/src/page_service.rs @@ -180,6 +180,7 @@ pub async fn libpq_listener_main( // only deal with a particular timeline, but we don't know which one // yet. task_mgr::spawn( + &tokio::runtime::Handle::current(), TaskKind::PageRequestHandler, None, None, diff --git a/pageserver/src/task_mgr.rs b/pageserver/src/task_mgr.rs index 2d97389982ec..69e163effaa3 100644 --- a/pageserver/src/task_mgr.rs +++ b/pageserver/src/task_mgr.rs @@ -98,22 +98,42 @@ use utils::id::TimelineId; // other operations, if the upload tasks e.g. get blocked on locks. It shouldn't // happen, but still. // +pub static COMPUTE_REQUEST_RUNTIME: Lazy = Lazy::new(|| { + tokio::runtime::Builder::new_multi_thread() + .thread_name("compute request worker") + .enable_all() + .build() + .expect("Failed to create compute request runtime") +}); + +pub static MGMT_REQUEST_RUNTIME: Lazy = Lazy::new(|| { + tokio::runtime::Builder::new_multi_thread() + .thread_name("mgmt request worker") + .enable_all() + .build() + .expect("Failed to create mgmt request runtime") +}); + +pub static WALRECEIVER_RUNTIME: Lazy = Lazy::new(|| { + tokio::runtime::Builder::new_multi_thread() + .thread_name("walreceiver worker") + .enable_all() + .build() + .expect("Failed to create walreceiver runtime") +}); -/// The single tokio runtime used by all pageserver code. -/// In the past, we had multiple runtimes, and in the future we should weed out -/// remaining references to this global field and rely on ambient runtime instead, -/// i.e., use `tokio::spawn` instead of `THE_RUNTIME.spawn()`, etc. -pub static THE_RUNTIME: Lazy = Lazy::new(|| { +pub static BACKGROUND_RUNTIME: Lazy = Lazy::new(|| { tokio::runtime::Builder::new_multi_thread() + .thread_name("background op worker") // if you change the number of worker threads please change the constant below .enable_all() .build() .expect("Failed to create background op runtime") }); -pub(crate) static THE_RUNTIME_WORKER_THREADS: Lazy = Lazy::new(|| { +pub(crate) static BACKGROUND_RUNTIME_WORKER_THREADS: Lazy = Lazy::new(|| { // force init and thus panics - let _ = THE_RUNTIME.handle(); + let _ = BACKGROUND_RUNTIME.handle(); // replicates tokio-1.28.1::loom::sys::num_cpus which is not available publicly // tokio would had already panicked for parsing errors or NotUnicode // @@ -305,6 +325,7 @@ struct PageServerTask { /// Note: if shutdown_process_on_error is set to true failure /// of the task will lead to shutdown of entire process pub fn spawn( + runtime: &tokio::runtime::Handle, kind: TaskKind, tenant_shard_id: Option, timeline_id: Option, @@ -333,7 +354,7 @@ where let task_name = name.to_string(); let task_cloned = Arc::clone(&task); - let join_handle = THE_RUNTIME.spawn(task_wrapper( + let join_handle = runtime.spawn(task_wrapper( task_name, task_id, task_cloned, diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index b923e473cef1..dcf9b1a605ba 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -144,7 +144,6 @@ macro_rules! pausable_failpoint { } }; } -pub(crate) use pausable_failpoint; pub mod blob_io; pub mod block_io; @@ -662,6 +661,7 @@ impl Tenant { let tenant_clone = Arc::clone(&tenant); let ctx = ctx.detached_child(TaskKind::Attach, DownloadBehavior::Warn); task_mgr::spawn( + &tokio::runtime::Handle::current(), TaskKind::Attach, Some(tenant_shard_id), None, diff --git a/pageserver/src/tenant/delete.rs b/pageserver/src/tenant/delete.rs index 3866136dbdfb..7d37873a67e6 100644 --- a/pageserver/src/tenant/delete.rs +++ b/pageserver/src/tenant/delete.rs @@ -482,6 +482,7 @@ impl DeleteTenantFlow { let tenant_shard_id = tenant.tenant_shard_id; task_mgr::spawn( + task_mgr::BACKGROUND_RUNTIME.handle(), TaskKind::TimelineDeletionWorker, Some(tenant_shard_id), None, diff --git a/pageserver/src/tenant/mgr.rs b/pageserver/src/tenant/mgr.rs index 34ca43a173e6..97a505ded906 100644 --- a/pageserver/src/tenant/mgr.rs +++ b/pageserver/src/tenant/mgr.rs @@ -1850,6 +1850,7 @@ impl TenantManager { let task_tenant_id = None; task_mgr::spawn( + task_mgr::BACKGROUND_RUNTIME.handle(), TaskKind::MgmtRequest, task_tenant_id, None, @@ -2815,12 +2816,15 @@ pub(crate) fn immediate_gc( // TODO: spawning is redundant now, need to hold the gate task_mgr::spawn( + &tokio::runtime::Handle::current(), TaskKind::GarbageCollector, Some(tenant_shard_id), Some(timeline_id), &format!("timeline_gc_handler garbage collection run for tenant {tenant_shard_id} timeline {timeline_id}"), false, async move { + fail::fail_point!("immediate_gc_task_pre"); + #[allow(unused_mut)] let mut result = tenant .gc_iteration(Some(timeline_id), gc_horizon, pitr, &cancel, &ctx) diff --git a/pageserver/src/tenant/remote_timeline_client.rs b/pageserver/src/tenant/remote_timeline_client.rs index b4b3243d11a8..cbd942d706bd 100644 --- a/pageserver/src/tenant/remote_timeline_client.rs +++ b/pageserver/src/tenant/remote_timeline_client.rs @@ -223,6 +223,7 @@ use crate::{ config::PageServerConf, task_mgr, task_mgr::TaskKind, + task_mgr::BACKGROUND_RUNTIME, tenant::metadata::TimelineMetadata, tenant::upload_queue::{ UploadOp, UploadQueue, UploadQueueInitialized, UploadQueueStopped, UploadTask, @@ -297,6 +298,8 @@ pub enum PersistIndexPartWithDeletedFlagError { pub struct RemoteTimelineClient { conf: &'static PageServerConf, + runtime: tokio::runtime::Handle, + tenant_shard_id: TenantShardId, timeline_id: TimelineId, generation: Generation, @@ -329,6 +332,12 @@ impl RemoteTimelineClient { ) -> RemoteTimelineClient { RemoteTimelineClient { conf, + runtime: if cfg!(test) { + // remote_timeline_client.rs tests rely on current-thread runtime + tokio::runtime::Handle::current() + } else { + BACKGROUND_RUNTIME.handle().clone() + }, tenant_shard_id, timeline_id, generation, @@ -1264,6 +1273,7 @@ impl RemoteTimelineClient { let tenant_shard_id = self.tenant_shard_id; let timeline_id = self.timeline_id; task_mgr::spawn( + &self.runtime, TaskKind::RemoteUploadTask, Some(self.tenant_shard_id), Some(self.timeline_id), @@ -1857,6 +1867,7 @@ mod tests { fn build_client(&self, generation: Generation) -> Arc { Arc::new(RemoteTimelineClient { conf: self.harness.conf, + runtime: tokio::runtime::Handle::current(), tenant_shard_id: self.harness.tenant_shard_id, timeline_id: TIMELINE_ID, generation, diff --git a/pageserver/src/tenant/secondary.rs b/pageserver/src/tenant/secondary.rs index b0babb130834..19f36c722e7b 100644 --- a/pageserver/src/tenant/secondary.rs +++ b/pageserver/src/tenant/secondary.rs @@ -8,7 +8,7 @@ use std::{sync::Arc, time::SystemTime}; use crate::{ config::PageServerConf, disk_usage_eviction_task::DiskUsageEvictionInfo, - task_mgr::{self, TaskKind}, + task_mgr::{self, TaskKind, BACKGROUND_RUNTIME}, virtual_file::MaybeFatalIo, }; @@ -317,6 +317,7 @@ pub fn spawn_tasks( tokio::sync::mpsc::channel::>(16); task_mgr::spawn( + BACKGROUND_RUNTIME.handle(), TaskKind::SecondaryDownloads, None, None, @@ -337,6 +338,7 @@ pub fn spawn_tasks( ); task_mgr::spawn( + BACKGROUND_RUNTIME.handle(), TaskKind::SecondaryUploads, None, None, diff --git a/pageserver/src/tenant/storage_layer/layer.rs b/pageserver/src/tenant/storage_layer/layer.rs index e101a40da49b..8ba37b5a8640 100644 --- a/pageserver/src/tenant/storage_layer/layer.rs +++ b/pageserver/src/tenant/storage_layer/layer.rs @@ -1447,7 +1447,7 @@ impl LayerInner { #[cfg(test)] tokio::task::spawn(fut); #[cfg(not(test))] - crate::task_mgr::THE_RUNTIME.spawn(fut); + crate::task_mgr::BACKGROUND_RUNTIME.spawn(fut); } /// Needed to use entered runtime in tests, but otherwise use BACKGROUND_RUNTIME. @@ -1458,7 +1458,7 @@ impl LayerInner { #[cfg(test)] tokio::task::spawn_blocking(f); #[cfg(not(test))] - crate::task_mgr::THE_RUNTIME.spawn_blocking(f); + crate::task_mgr::BACKGROUND_RUNTIME.spawn_blocking(f); } } diff --git a/pageserver/src/tenant/tasks.rs b/pageserver/src/tenant/tasks.rs index db32223a601e..e4f5f7513288 100644 --- a/pageserver/src/tenant/tasks.rs +++ b/pageserver/src/tenant/tasks.rs @@ -8,7 +8,7 @@ use std::time::{Duration, Instant}; use crate::context::{DownloadBehavior, RequestContext}; use crate::metrics::TENANT_TASK_EVENTS; use crate::task_mgr; -use crate::task_mgr::TaskKind; +use crate::task_mgr::{TaskKind, BACKGROUND_RUNTIME}; use crate::tenant::throttle::Stats; use crate::tenant::timeline::CompactionError; use crate::tenant::{Tenant, TenantState}; @@ -18,7 +18,7 @@ use utils::{backoff, completion}; static CONCURRENT_BACKGROUND_TASKS: once_cell::sync::Lazy = once_cell::sync::Lazy::new(|| { - let total_threads = *crate::task_mgr::THE_RUNTIME_WORKER_THREADS; + let total_threads = *task_mgr::BACKGROUND_RUNTIME_WORKER_THREADS; let permits = usize::max( 1, // while a lot of the work is done on spawn_blocking, we still do @@ -85,6 +85,7 @@ pub fn start_background_loops( ) { let tenant_shard_id = tenant.tenant_shard_id; task_mgr::spawn( + BACKGROUND_RUNTIME.handle(), TaskKind::Compaction, Some(tenant_shard_id), None, @@ -108,6 +109,7 @@ pub fn start_background_loops( }, ); task_mgr::spawn( + BACKGROUND_RUNTIME.handle(), TaskKind::GarbageCollector, Some(tenant_shard_id), None, diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 6c6bb4b78875..0b8cdac1cc9c 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -1714,6 +1714,7 @@ impl Timeline { initdb_optimization_count: 0, }; task_mgr::spawn( + task_mgr::BACKGROUND_RUNTIME.handle(), task_mgr::TaskKind::LayerFlushTask, Some(self.tenant_shard_id), Some(self.timeline_id), @@ -2076,6 +2077,7 @@ impl Timeline { DownloadBehavior::Download, ); task_mgr::spawn( + task_mgr::BACKGROUND_RUNTIME.handle(), task_mgr::TaskKind::InitialLogicalSizeCalculation, Some(self.tenant_shard_id), Some(self.timeline_id), @@ -2253,6 +2255,7 @@ impl Timeline { DownloadBehavior::Download, ); task_mgr::spawn( + task_mgr::BACKGROUND_RUNTIME.handle(), task_mgr::TaskKind::OndemandLogicalSizeCalculation, Some(self.tenant_shard_id), Some(self.timeline_id), @@ -3828,7 +3831,7 @@ impl Timeline { }; let timer = self.metrics.garbage_collect_histo.start_timer(); - pausable_failpoint!("before-timeline-gc"); + fail_point!("before-timeline-gc"); // Is the timeline being deleted? if self.is_stopping() { @@ -4139,6 +4142,7 @@ impl Timeline { let self_clone = Arc::clone(&self); let task_id = task_mgr::spawn( + task_mgr::BACKGROUND_RUNTIME.handle(), task_mgr::TaskKind::DownloadAllRemoteLayers, Some(self.tenant_shard_id), Some(self.timeline_id), diff --git a/pageserver/src/tenant/timeline/delete.rs b/pageserver/src/tenant/timeline/delete.rs index e9afbfd8ba05..ab0a88c76471 100644 --- a/pageserver/src/tenant/timeline/delete.rs +++ b/pageserver/src/tenant/timeline/delete.rs @@ -429,6 +429,7 @@ impl DeleteTimelineFlow { let timeline_id = timeline.timeline_id; task_mgr::spawn( + task_mgr::BACKGROUND_RUNTIME.handle(), TaskKind::TimelineDeletionWorker, Some(tenant_shard_id), Some(timeline_id), diff --git a/pageserver/src/tenant/timeline/eviction_task.rs b/pageserver/src/tenant/timeline/eviction_task.rs index f84a4b0dac02..dd769d41216c 100644 --- a/pageserver/src/tenant/timeline/eviction_task.rs +++ b/pageserver/src/tenant/timeline/eviction_task.rs @@ -28,7 +28,7 @@ use tracing::{debug, error, info, info_span, instrument, warn, Instrument}; use crate::{ context::{DownloadBehavior, RequestContext}, pgdatadir_mapping::CollectKeySpaceError, - task_mgr::{self, TaskKind}, + task_mgr::{self, TaskKind, BACKGROUND_RUNTIME}, tenant::{ tasks::BackgroundLoopKind, timeline::EvictionError, LogicalSizeCalculationCause, Tenant, }, @@ -56,6 +56,7 @@ impl Timeline { let self_clone = Arc::clone(self); let background_tasks_can_start = background_tasks_can_start.cloned(); task_mgr::spawn( + BACKGROUND_RUNTIME.handle(), TaskKind::Eviction, Some(self.tenant_shard_id), Some(self.timeline_id), diff --git a/pageserver/src/tenant/timeline/walreceiver.rs b/pageserver/src/tenant/timeline/walreceiver.rs index 3592dda8d72c..2fab6722b8f5 100644 --- a/pageserver/src/tenant/timeline/walreceiver.rs +++ b/pageserver/src/tenant/timeline/walreceiver.rs @@ -24,7 +24,7 @@ mod connection_manager; mod walreceiver_connection; use crate::context::{DownloadBehavior, RequestContext}; -use crate::task_mgr::{self, TaskKind}; +use crate::task_mgr::{self, TaskKind, WALRECEIVER_RUNTIME}; use crate::tenant::debug_assert_current_span_has_tenant_and_timeline_id; use crate::tenant::timeline::walreceiver::connection_manager::{ connection_manager_loop_step, ConnectionManagerState, @@ -82,6 +82,7 @@ impl WalReceiver { let loop_status = Arc::new(std::sync::RwLock::new(None)); let manager_status = Arc::clone(&loop_status); task_mgr::spawn( + WALRECEIVER_RUNTIME.handle(), TaskKind::WalReceiverManager, Some(timeline.tenant_shard_id), Some(timeline_id), @@ -180,7 +181,7 @@ impl TaskHandle { let (events_sender, events_receiver) = watch::channel(TaskStateUpdate::Started); let cancellation_clone = cancellation.clone(); - let join_handle = tokio::spawn(async move { + let join_handle = WALRECEIVER_RUNTIME.spawn(async move { events_sender.send(TaskStateUpdate::Started).ok(); task(events_sender, cancellation_clone).await // events_sender is dropped at some point during the .await above. diff --git a/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs b/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs index cf87cc6ce029..d9f780cfd180 100644 --- a/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs +++ b/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs @@ -11,6 +11,7 @@ use std::{ use anyhow::{anyhow, Context}; use bytes::BytesMut; use chrono::{NaiveDateTime, Utc}; +use fail::fail_point; use futures::StreamExt; use postgres::{error::SqlState, SimpleQueryMessage, SimpleQueryRow}; use postgres_ffi::WAL_SEGMENT_SIZE; @@ -26,7 +27,9 @@ use super::TaskStateUpdate; use crate::{ context::RequestContext, metrics::{LIVE_CONNECTIONS_COUNT, WALRECEIVER_STARTED_CONNECTIONS, WAL_INGEST}, - task_mgr::{self, TaskKind}, + task_mgr, + task_mgr::TaskKind, + task_mgr::WALRECEIVER_RUNTIME, tenant::{debug_assert_current_span_has_tenant_and_timeline_id, Timeline, WalReceiverInfo}, walingest::WalIngest, walrecord::DecodedWALRecord, @@ -160,6 +163,7 @@ pub(super) async fn handle_walreceiver_connection( ); let connection_cancellation = cancellation.clone(); task_mgr::spawn( + WALRECEIVER_RUNTIME.handle(), TaskKind::WalReceiverConnectionPoller, Some(timeline.tenant_shard_id), Some(timeline.timeline_id), @@ -325,17 +329,7 @@ pub(super) async fn handle_walreceiver_connection( filtered_records += 1; } - // don't simply use pausable_failpoint here because its spawn_blocking slows - // slows down the tests too much. - fail::fail_point!("walreceiver-after-ingest-blocking"); - if let Err(()) = (|| { - fail::fail_point!("walreceiver-after-ingest-pause-activate", |_| { - Err(()) - }); - Ok(()) - })() { - pausable_failpoint!("walreceiver-after-ingest-pause"); - } + fail_point!("walreceiver-after-ingest"); last_rec_lsn = lsn; diff --git a/test_runner/regress/test_backpressure.py b/test_runner/regress/test_backpressure.py index af17a2e89d3c..819912dd0517 100644 --- a/test_runner/regress/test_backpressure.py +++ b/test_runner/regress/test_backpressure.py @@ -116,7 +116,7 @@ def test_backpressure_received_lsn_lag(neon_env_builder: NeonEnvBuilder): # Configure failpoint to slow down walreceiver ingest with closing(env.pageserver.connect()) as psconn: with psconn.cursor(cursor_factory=psycopg2.extras.DictCursor) as pscur: - pscur.execute("failpoints walreceiver-after-ingest-blocking=sleep(20)") + pscur.execute("failpoints walreceiver-after-ingest=sleep(20)") # FIXME # Wait for the check thread to start diff --git a/test_runner/regress/test_timeline_size.py b/test_runner/regress/test_timeline_size.py index efd257900d57..628c484fbdbd 100644 --- a/test_runner/regress/test_timeline_size.py +++ b/test_runner/regress/test_timeline_size.py @@ -931,7 +931,7 @@ def test_timeline_logical_size_task_priority(neon_env_builder: NeonEnvBuilder): env.pageserver.stop() env.pageserver.start( extra_env_vars={ - "FAILPOINTS": "initial-size-calculation-permit-pause=pause;walreceiver-after-ingest-pause-activate=return(1);walreceiver-after-ingest-pause=pause" + "FAILPOINTS": "initial-size-calculation-permit-pause=pause;walreceiver-after-ingest=pause" } ) @@ -953,11 +953,7 @@ def assert_initial_logical_size_not_prioritised(): assert details["current_logical_size_is_accurate"] is True client.configure_failpoints( - [ - ("initial-size-calculation-permit-pause", "off"), - ("walreceiver-after-ingest-pause-activate", "off"), - ("walreceiver-after-ingest-pause", "off"), - ] + [("initial-size-calculation-permit-pause", "off"), ("walreceiver-after-ingest", "off")] ) @@ -987,7 +983,7 @@ def test_eager_attach_does_not_queue_up(neon_env_builder: NeonEnvBuilder): # pause at logical size calculation, also pause before walreceiver can give feedback so it will give priority to logical size calculation env.pageserver.start( extra_env_vars={ - "FAILPOINTS": "timeline-calculate-logical-size-pause=pause;walreceiver-after-ingest-pause-activate=return(1);walreceiver-after-ingest-pause=pause" + "FAILPOINTS": "timeline-calculate-logical-size-pause=pause;walreceiver-after-ingest=pause" } ) @@ -1033,11 +1029,7 @@ def eager_tenant_is_active(): other_is_attaching() client.configure_failpoints( - [ - ("timeline-calculate-logical-size-pause", "off"), - ("walreceiver-after-ingest-pause-activate", "off"), - ("walreceiver-after-ingest-pause", "off"), - ] + [("timeline-calculate-logical-size-pause", "off"), ("walreceiver-after-ingest", "off")] ) @@ -1067,7 +1059,7 @@ def test_lazy_attach_activation(neon_env_builder: NeonEnvBuilder, activation_met # pause at logical size calculation, also pause before walreceiver can give feedback so it will give priority to logical size calculation env.pageserver.start( extra_env_vars={ - "FAILPOINTS": "timeline-calculate-logical-size-pause=pause;walreceiver-after-ingest-pause-activate=return(1);walreceiver-after-ingest-pause=pause" + "FAILPOINTS": "timeline-calculate-logical-size-pause=pause;walreceiver-after-ingest=pause" } ) @@ -1119,11 +1111,3 @@ def lazy_tenant_is_active(): delete_lazy_activating(lazy_tenant, env.pageserver, expect_attaching=True) else: raise RuntimeError(activation_method) - - client.configure_failpoints( - [ - ("timeline-calculate-logical-size-pause", "off"), - ("walreceiver-after-ingest-pause-activate", "off"), - ("walreceiver-after-ingest-pause", "off"), - ] - ) From de03742ca33ac5881b7639b7cc863c80e0830c53 Mon Sep 17 00:00:00 2001 From: Vlad Lazar Date: Tue, 26 Mar 2024 14:35:36 +0000 Subject: [PATCH 46/53] pageserver: drop layer map lock in Timeline::get (#7217) ## Problem We currently hold the layer map read lock while doing IO on the read path. This is not required for correctness. ## Summary of changes Drop the layer map lock after figuring out which layer we wish to read from. Why is this correct: * `Layer` models the lifecycle of an on disk layer. In the event the layer is removed from local disk, it will be on demand downloaded * `InMemoryLayer` holds the `EphemeralFile` which wraps the on disk file. As long as the `InMemoryLayer` is in scope, it's safe to read from it. Related https://github.com/neondatabase/neon/issues/6833 --- pageserver/src/tenant/timeline.rs | 20 ++++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 0b8cdac1cc9c..8b6e93d500aa 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -2587,6 +2587,10 @@ impl Timeline { // Get all the data needed to reconstruct the page version from this layer. // But if we have an older cached page image, no need to go past that. let lsn_floor = max(cached_lsn + 1, start_lsn); + + let open_layer = open_layer.clone(); + drop(guard); + result = match open_layer .get_value_reconstruct_data( key, @@ -2604,10 +2608,7 @@ impl Timeline { traversal_path.push(( result, cont_lsn, - Box::new({ - let open_layer = Arc::clone(open_layer); - move || open_layer.traversal_id() - }), + Box::new(move || open_layer.traversal_id()), )); continue 'outer; } @@ -2617,6 +2618,10 @@ impl Timeline { if cont_lsn > start_lsn { //info!("CHECKING for {} at {} on frozen layer {}", key, cont_lsn, frozen_layer.filename().display()); let lsn_floor = max(cached_lsn + 1, start_lsn); + + let frozen_layer = frozen_layer.clone(); + drop(guard); + result = match frozen_layer .get_value_reconstruct_data( key, @@ -2634,10 +2639,7 @@ impl Timeline { traversal_path.push(( result, cont_lsn, - Box::new({ - let frozen_layer = Arc::clone(frozen_layer); - move || frozen_layer.traversal_id() - }), + Box::new(move || frozen_layer.traversal_id()), )); continue 'outer; } @@ -2645,6 +2647,8 @@ impl Timeline { if let Some(SearchResult { lsn_floor, layer }) = layers.search(key, cont_lsn) { let layer = guard.get_from_desc(&layer); + drop(guard); + // Get all the data needed to reconstruct the page version from this layer. // But if we have an older cached page image, no need to go past that. let lsn_floor = max(cached_lsn + 1, lsn_floor); From 3426619a7949dd46652ef89e7b81cbac15a4fbe1 Mon Sep 17 00:00:00 2001 From: Alexander Bayandin Date: Tue, 26 Mar 2024 15:10:15 +0000 Subject: [PATCH 47/53] test_runner/performance: skip test_bulk_insert (#7238) ## Problem `test_bulk_insert` becomes too slow, and it fails constantly: https://github.com/neondatabase/neon/issues/7124 ## Summary of changes - Skip `test_bulk_insert` until it's fixed --- test_runner/performance/test_bulk_insert.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/test_runner/performance/test_bulk_insert.py b/test_runner/performance/test_bulk_insert.py index 9e3f6022375d..1df3f2f5f19d 100644 --- a/test_runner/performance/test_bulk_insert.py +++ b/test_runner/performance/test_bulk_insert.py @@ -1,5 +1,6 @@ from contextlib import closing +import pytest from fixtures.benchmark_fixture import MetricReport from fixtures.compare_fixtures import NeonCompare, PgCompare from fixtures.pageserver.utils import wait_tenant_status_404 @@ -17,6 +18,7 @@ # 3. Disk space used # 4. Peak memory usage # +@pytest.mark.skip("See https://github.com/neondatabase/neon/issues/7124") def test_bulk_insert(neon_with_baseline: PgCompare): env = neon_with_baseline From 8dfe3a070cd04dd2310ed07e1f38f4257dd43cd8 Mon Sep 17 00:00:00 2001 From: John Spray Date: Tue, 26 Mar 2024 15:20:05 +0000 Subject: [PATCH 48/53] pageserver: return 429 on timeline creation in progress (#7225) ## Problem Currently, we return 409 (Conflict) in two cases: - Temporary: Timeline creation cannot proceed because another timeline with the same ID is being created - Permanent: Timeline creation cannot proceed because another timeline exists with different parameters but the same ID. Callers which time out a request and retry should be able to distinguish these cases. Closes: #7208 ## Summary of changes - Expose `AlreadyCreating` errors as 429 instead of 409 --- pageserver/src/http/openapi_spec.yml | 10 ++++++++-- pageserver/src/http/routes.rs | 11 +++++++---- test_runner/regress/test_tenants.py | 3 +++ 3 files changed, 18 insertions(+), 6 deletions(-) diff --git a/pageserver/src/http/openapi_spec.yml b/pageserver/src/http/openapi_spec.yml index 07712298451e..bb477f89c5bf 100644 --- a/pageserver/src/http/openapi_spec.yml +++ b/pageserver/src/http/openapi_spec.yml @@ -1038,7 +1038,7 @@ paths: format: hex responses: "201": - description: TimelineInfo + description: Timeline was created, or already existed with matching parameters content: application/json: schema: @@ -1068,11 +1068,17 @@ paths: schema: $ref: "#/components/schemas/Error" "409": - description: Timeline already exists, creation skipped + description: Timeline already exists, with different parameters. Creation cannot proceed. content: application/json: schema: $ref: "#/components/schemas/ConflictError" + "429": + description: A creation request was sent for the same Timeline Id while a creation was already in progress. Back off and retry. + content: + application/json: + schema: + $ref: "#/components/schemas/Error" "500": description: Generic operation error content: diff --git a/pageserver/src/http/routes.rs b/pageserver/src/http/routes.rs index 26f23fb8c2cb..3cc92216ed2a 100644 --- a/pageserver/src/http/routes.rs +++ b/pageserver/src/http/routes.rs @@ -535,10 +535,13 @@ async fn timeline_create_handler( HttpErrorBody::from_msg("Tenant shutting down".to_string()), ) } - Err( - e @ tenant::CreateTimelineError::Conflict - | e @ tenant::CreateTimelineError::AlreadyCreating, - ) => json_response(StatusCode::CONFLICT, HttpErrorBody::from_msg(e.to_string())), + Err(e @ tenant::CreateTimelineError::Conflict) => { + json_response(StatusCode::CONFLICT, HttpErrorBody::from_msg(e.to_string())) + } + Err(e @ tenant::CreateTimelineError::AlreadyCreating) => json_response( + StatusCode::TOO_MANY_REQUESTS, + HttpErrorBody::from_msg(e.to_string()), + ), Err(tenant::CreateTimelineError::AncestorLsn(err)) => json_response( StatusCode::NOT_ACCEPTABLE, HttpErrorBody::from_msg(format!("{err:#}")), diff --git a/test_runner/regress/test_tenants.py b/test_runner/regress/test_tenants.py index f8701b65d7d9..2832304dcce9 100644 --- a/test_runner/regress/test_tenants.py +++ b/test_runner/regress/test_tenants.py @@ -389,6 +389,9 @@ def create_bg(delay_ms): if e.status_code == 409: log.info(f"delay_ms={delay_ms} 409") pass + elif e.status_code == 429: + log.info(f"delay_ms={delay_ms} 429") + pass elif e.status_code == 400: if "is less than existing" in e.message: # We send creation requests very close together in time: it is expected that these From 47d2b3a4830f6d5ecb84086e785ec0f913390176 Mon Sep 17 00:00:00 2001 From: John Spray Date: Tue, 26 Mar 2024 15:45:32 +0000 Subject: [PATCH 49/53] pageserver: limit total ephemeral layer bytes (#7218) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## Problem Follows: https://github.com/neondatabase/neon/pull/7182 - Sufficient concurrent writes could OOM a pageserver from the size of indices on all the InMemoryLayer instances. - Enforcement of checkpoint_period only happened if there were some writes. Closes: https://github.com/neondatabase/neon/issues/6916 ## Summary of changes - Add `ephemeral_bytes_per_memory_kb` config property. This controls the ratio of ephemeral layer capacity to memory capacity. The weird unit is to enable making the ratio less than 1:1 (set this property to 1024 to use 1MB of ephemeral layers for every 1MB of RAM, set it smaller to get a fraction). - Implement background layer rolling checks in Timeline::compaction_iteration -- this ensures we apply layer rolling policy in the absence of writes. - During background checks, if the total ephemeral layer size has exceeded the limit, then roll layers whose size is greater than the mean size of all ephemeral layers. - Remove the tick() path from walreceiver: it isn't needed any more now that we do equivalent checks from compaction_iteration. - Add tests for the above. --------- Co-authored-by: Arpad Müller --- Cargo.lock | 1 + pageserver/Cargo.toml | 1 + pageserver/src/config.rs | 25 ++ pageserver/src/tenant/mgr.rs | 14 + pageserver/src/tenant/storage_layer.rs | 2 +- .../tenant/storage_layer/inmemory_layer.rs | 40 ++- pageserver/src/tenant/timeline.rs | 218 ++++++++------ .../walreceiver/walreceiver_connection.rs | 11 - .../regress/test_pageserver_layer_rolling.py | 275 ++++++++++++++++++ .../test_pageserver_small_inmemory_layers.py | 143 --------- 10 files changed, 483 insertions(+), 247 deletions(-) create mode 100644 test_runner/regress/test_pageserver_layer_rolling.py delete mode 100644 test_runner/regress/test_pageserver_small_inmemory_layers.py diff --git a/Cargo.lock b/Cargo.lock index 45b802c54fdd..c1c245fa9cb0 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3581,6 +3581,7 @@ dependencies = [ "strum_macros", "svg_fmt", "sync_wrapper", + "sysinfo", "tenant_size_model", "thiserror", "tokio", diff --git a/pageserver/Cargo.toml b/pageserver/Cargo.toml index f3042945910d..7a11610a9150 100644 --- a/pageserver/Cargo.toml +++ b/pageserver/Cargo.toml @@ -59,6 +59,7 @@ signal-hook.workspace = true smallvec = { workspace = true, features = ["write"] } svg_fmt.workspace = true sync_wrapper.workspace = true +sysinfo.workspace = true tokio-tar.workspace = true thiserror.workspace = true tokio = { workspace = true, features = ["process", "sync", "fs", "rt", "io-util", "time"] } diff --git a/pageserver/src/config.rs b/pageserver/src/config.rs index a29719e36f33..1837da34ce4b 100644 --- a/pageserver/src/config.rs +++ b/pageserver/src/config.rs @@ -95,6 +95,8 @@ pub mod defaults { pub const DEFAULT_VALIDATE_VECTORED_GET: bool = true; + pub const DEFAULT_EPHEMERAL_BYTES_PER_MEMORY_KB: usize = 0; + /// /// Default built-in configuration file. /// @@ -156,6 +158,8 @@ pub mod defaults { #heatmap_upload_concurrency = {DEFAULT_HEATMAP_UPLOAD_CONCURRENCY} #secondary_download_concurrency = {DEFAULT_SECONDARY_DOWNLOAD_CONCURRENCY} +#ephemeral_bytes_per_memory_kb = {DEFAULT_EPHEMERAL_BYTES_PER_MEMORY_KB} + [remote_storage] "# @@ -279,6 +283,13 @@ pub struct PageServerConf { pub max_vectored_read_bytes: MaxVectoredReadBytes, pub validate_vectored_get: bool, + + /// How many bytes of ephemeral layer content will we allow per kilobyte of RAM. When this + /// is exceeded, we start proactively closing ephemeral layers to limit the total amount + /// of ephemeral data. + /// + /// Setting this to zero disables limits on total ephemeral layer size. + pub ephemeral_bytes_per_memory_kb: usize, } /// We do not want to store this in a PageServerConf because the latter may be logged @@ -400,6 +411,8 @@ struct PageServerConfigBuilder { max_vectored_read_bytes: BuilderValue, validate_vectored_get: BuilderValue, + + ephemeral_bytes_per_memory_kb: BuilderValue, } impl PageServerConfigBuilder { @@ -486,6 +499,7 @@ impl PageServerConfigBuilder { NonZeroUsize::new(DEFAULT_MAX_VECTORED_READ_BYTES).unwrap(), )), validate_vectored_get: Set(DEFAULT_VALIDATE_VECTORED_GET), + ephemeral_bytes_per_memory_kb: Set(DEFAULT_EPHEMERAL_BYTES_PER_MEMORY_KB), } } } @@ -665,6 +679,10 @@ impl PageServerConfigBuilder { self.validate_vectored_get = BuilderValue::Set(value); } + pub fn get_ephemeral_bytes_per_memory_kb(&mut self, value: usize) { + self.ephemeral_bytes_per_memory_kb = BuilderValue::Set(value); + } + pub fn build(self) -> anyhow::Result { let default = Self::default_values(); @@ -720,6 +738,7 @@ impl PageServerConfigBuilder { get_vectored_impl, max_vectored_read_bytes, validate_vectored_get, + ephemeral_bytes_per_memory_kb, } CUSTOM LOGIC { @@ -1010,6 +1029,9 @@ impl PageServerConf { "validate_vectored_get" => { builder.get_validate_vectored_get(parse_toml_bool("validate_vectored_get", item)?) } + "ephemeral_bytes_per_memory_kb" => { + builder.get_ephemeral_bytes_per_memory_kb(parse_toml_u64("ephemeral_bytes_per_memory_kb", item)? as usize) + } _ => bail!("unrecognized pageserver option '{key}'"), } } @@ -1091,6 +1113,7 @@ impl PageServerConf { .expect("Invalid default constant"), ), validate_vectored_get: defaults::DEFAULT_VALIDATE_VECTORED_GET, + ephemeral_bytes_per_memory_kb: defaults::DEFAULT_EPHEMERAL_BYTES_PER_MEMORY_KB, } } } @@ -1328,6 +1351,7 @@ background_task_maximum_delay = '334 s' .expect("Invalid default constant") ), validate_vectored_get: defaults::DEFAULT_VALIDATE_VECTORED_GET, + ephemeral_bytes_per_memory_kb: defaults::DEFAULT_EPHEMERAL_BYTES_PER_MEMORY_KB }, "Correct defaults should be used when no config values are provided" ); @@ -1399,6 +1423,7 @@ background_task_maximum_delay = '334 s' .expect("Invalid default constant") ), validate_vectored_get: defaults::DEFAULT_VALIDATE_VECTORED_GET, + ephemeral_bytes_per_memory_kb: defaults::DEFAULT_EPHEMERAL_BYTES_PER_MEMORY_KB }, "Should be able to parse all basic config values correctly" ); diff --git a/pageserver/src/tenant/mgr.rs b/pageserver/src/tenant/mgr.rs index 97a505ded906..8e3eae7cf6d7 100644 --- a/pageserver/src/tenant/mgr.rs +++ b/pageserver/src/tenant/mgr.rs @@ -16,6 +16,7 @@ use std::collections::{BTreeMap, HashMap}; use std::ops::Deref; use std::sync::Arc; use std::time::{Duration, Instant}; +use sysinfo::SystemExt; use tokio::fs; use utils::timeout::{timeout_cancellable, TimeoutCancellableError}; @@ -43,6 +44,7 @@ use crate::tenant::config::{ }; use crate::tenant::delete::DeleteTenantFlow; use crate::tenant::span::debug_assert_current_span_has_tenant_id; +use crate::tenant::storage_layer::inmemory_layer; use crate::tenant::{AttachedTenantConf, SpawnMode, Tenant, TenantState}; use crate::{InitializationOrder, IGNORED_TENANT_FILE_NAME, METADATA_FILE_NAME, TEMP_FILE_SUFFIX}; @@ -543,6 +545,18 @@ pub async fn init_tenant_mgr( let ctx = RequestContext::todo_child(TaskKind::Startup, DownloadBehavior::Warn); + // Initialize dynamic limits that depend on system resources + let system_memory = + sysinfo::System::new_with_specifics(sysinfo::RefreshKind::new().with_memory()) + .total_memory(); + let max_ephemeral_layer_bytes = + conf.ephemeral_bytes_per_memory_kb as u64 * (system_memory / 1024); + tracing::info!("Initialized ephemeral layer size limit to {max_ephemeral_layer_bytes}, for {system_memory} bytes of memory"); + inmemory_layer::GLOBAL_RESOURCES.max_dirty_bytes.store( + max_ephemeral_layer_bytes, + std::sync::atomic::Ordering::Relaxed, + ); + // Scan local filesystem for attached tenants let tenant_configs = init_load_tenant_configs(conf).await?; diff --git a/pageserver/src/tenant/storage_layer.rs b/pageserver/src/tenant/storage_layer.rs index 5c3bab986888..f44a92a2d790 100644 --- a/pageserver/src/tenant/storage_layer.rs +++ b/pageserver/src/tenant/storage_layer.rs @@ -3,7 +3,7 @@ pub mod delta_layer; mod filename; pub mod image_layer; -mod inmemory_layer; +pub(crate) mod inmemory_layer; pub(crate) mod layer; mod layer_desc; diff --git a/pageserver/src/tenant/storage_layer/inmemory_layer.rs b/pageserver/src/tenant/storage_layer/inmemory_layer.rs index 869d175d8df9..628f12065f1d 100644 --- a/pageserver/src/tenant/storage_layer/inmemory_layer.rs +++ b/pageserver/src/tenant/storage_layer/inmemory_layer.rs @@ -89,7 +89,10 @@ impl std::fmt::Debug for InMemoryLayerInner { /// /// This global state is used to implement behaviors that require a global view of the system, e.g. /// rolling layers proactively to limit the total amount of dirty data. -struct GlobalResources { +pub(crate) struct GlobalResources { + // Limit on how high dirty_bytes may grow before we start freezing layers to reduce it. + // Zero means unlimited. + pub(crate) max_dirty_bytes: AtomicU64, // How many bytes are in all EphemeralFile objects dirty_bytes: AtomicU64, // How many layers are contributing to dirty_bytes @@ -118,11 +121,12 @@ impl GlobalResourceUnits { /// Do not call this frequently: all timelines will write to these same global atomics, /// so this is a relatively expensive operation. Wait at least a few seconds between calls. - fn publish_size(&mut self, size: u64) { + /// + /// Returns the effective layer size limit that should be applied, if any, to keep + /// the total number of dirty bytes below the configured maximum. + fn publish_size(&mut self, size: u64) -> Option { let new_global_dirty_bytes = match size.cmp(&self.dirty_bytes) { - Ordering::Equal => { - return; - } + Ordering::Equal => GLOBAL_RESOURCES.dirty_bytes.load(AtomicOrdering::Relaxed), Ordering::Greater => { let delta = size - self.dirty_bytes; let old = GLOBAL_RESOURCES @@ -146,6 +150,21 @@ impl GlobalResourceUnits { TIMELINE_EPHEMERAL_BYTES.set(new_global_dirty_bytes); self.dirty_bytes = size; + + let max_dirty_bytes = GLOBAL_RESOURCES + .max_dirty_bytes + .load(AtomicOrdering::Relaxed); + if max_dirty_bytes > 0 && new_global_dirty_bytes > max_dirty_bytes { + // Set the layer file limit to the average layer size: this implies that all above-average + // sized layers will be elegible for freezing. They will be frozen in the order they + // next enter publish_size. + Some( + new_global_dirty_bytes + / GLOBAL_RESOURCES.dirty_layers.load(AtomicOrdering::Relaxed) as u64, + ) + } else { + None + } } // Call publish_size if the input size differs from last published size by more than @@ -174,7 +193,8 @@ impl Drop for GlobalResourceUnits { } } -static GLOBAL_RESOURCES: GlobalResources = GlobalResources { +pub(crate) static GLOBAL_RESOURCES: GlobalResources = GlobalResources { + max_dirty_bytes: AtomicU64::new(0), dirty_bytes: AtomicU64::new(0), dirty_layers: AtomicUsize::new(0), }; @@ -194,6 +214,10 @@ impl InMemoryLayer { } } + pub(crate) fn try_len(&self) -> Option { + self.inner.try_read().map(|i| i.file.len()).ok() + } + pub(crate) fn assert_writable(&self) { assert!(self.end_lsn.get().is_none()); } @@ -486,10 +510,10 @@ impl InMemoryLayer { Ok(()) } - pub(crate) async fn tick(&self) { + pub(crate) async fn tick(&self) -> Option { let mut inner = self.inner.write().await; let size = inner.file.len(); - inner.resource_units.publish_size(size); + inner.resource_units.publish_size(size) } pub(crate) async fn put_tombstones(&self, _key_ranges: &[(Range, Lsn)]) -> Result<()> { diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 8b6e93d500aa..38292b6d78e8 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -19,7 +19,7 @@ use pageserver_api::{ keyspace::KeySpaceAccum, models::{ CompactionAlgorithm, DownloadRemoteLayersTaskInfo, DownloadRemoteLayersTaskSpawnRequest, - EvictionPolicy, LayerMapInfo, TimelineState, + EvictionPolicy, InMemoryLayerInfo, LayerMapInfo, TimelineState, }, reltag::BlockNumber, shard::{ShardIdentity, TenantShardId}, @@ -1142,6 +1142,79 @@ impl Timeline { self.flush_frozen_layers_and_wait().await } + /// If there is no writer, and conditions for rolling the latest layer are met, then freeze it. + /// + /// This is for use in background housekeeping, to provide guarantees of layers closing eventually + /// even if there are no ongoing writes to drive that. + async fn maybe_freeze_ephemeral_layer(&self) { + let Ok(_write_guard) = self.write_lock.try_lock() else { + // If the write lock is held, there is an active wal receiver: rolling open layers + // is their responsibility while they hold this lock. + return; + }; + + let Ok(layers_guard) = self.layers.try_read() else { + // Don't block if the layer lock is busy + return; + }; + + let Some(open_layer) = &layers_guard.layer_map().open_layer else { + // No open layer, no work to do. + return; + }; + + let Some(current_size) = open_layer.try_len() else { + // Unexpected: since we hold the write guard, nobody else should be writing to this layer, so + // read lock to get size should always succeed. + tracing::warn!("Lock conflict while reading size of open layer"); + return; + }; + + let current_lsn = self.get_last_record_lsn(); + + let checkpoint_distance_override = open_layer.tick().await; + + if let Some(size_override) = checkpoint_distance_override { + if current_size > size_override { + // This is not harmful, but it only happens in relatively rare cases where + // time-based checkpoints are not happening fast enough to keep the amount of + // ephemeral data within configured limits. It's a sign of stress on the system. + tracing::info!("Early-rolling open layer at size {current_size} (limit {size_override}) due to dirty data pressure"); + } + } + + let checkpoint_distance = + checkpoint_distance_override.unwrap_or(self.get_checkpoint_distance()); + + if self.should_roll( + current_size, + current_size, + checkpoint_distance, + self.get_last_record_lsn(), + self.last_freeze_at.load(), + *self.last_freeze_ts.read().unwrap(), + ) { + match open_layer.info() { + InMemoryLayerInfo::Frozen { lsn_start, lsn_end } => { + // We may reach this point if the layer was already frozen by not yet flushed: flushing + // happens asynchronously in the background. + tracing::debug!( + "Not freezing open layer, it's already frozen ({lsn_start}..{lsn_end})" + ); + } + InMemoryLayerInfo::Open { .. } => { + // Upgrade to a write lock and freeze the layer + drop(layers_guard); + let mut layers_guard = self.layers.write().await; + layers_guard + .try_freeze_in_memory_layer(current_lsn, &self.last_freeze_at) + .await; + } + } + self.flush_frozen_layers(); + } + } + /// Outermost timeline compaction operation; downloads needed layers. pub(crate) async fn compact( self: &Arc, @@ -1164,6 +1237,11 @@ impl Timeline { (guard, permit) }; + // Prior to compaction, check if an open ephemeral layer should be closed: this provides + // background enforcement of checkpoint interval if there is no active WAL receiver, to avoid keeping + // an ephemeral layer open forever when idle. + self.maybe_freeze_ephemeral_layer().await; + // this wait probably never needs any "long time spent" logging, because we already nag if // compaction task goes over it's period (20s) which is quite often in production. let (_guard, _permit) = tokio::select! { @@ -1434,6 +1512,53 @@ impl Timeline { Err(EvictionError::Timeout) => Ok(Some(false)), } } + + fn should_roll( + &self, + layer_size: u64, + projected_layer_size: u64, + checkpoint_distance: u64, + projected_lsn: Lsn, + last_freeze_at: Lsn, + last_freeze_ts: Instant, + ) -> bool { + let distance = projected_lsn.widening_sub(last_freeze_at); + + // Rolling the open layer can be triggered by: + // 1. The distance from the last LSN we rolled at. This bounds the amount of WAL that + // the safekeepers need to store. For sharded tenants, we multiply by shard count to + // account for how writes are distributed across shards: we expect each node to consume + // 1/count of the LSN on average. + // 2. The size of the currently open layer. + // 3. The time since the last roll. It helps safekeepers to regard pageserver as caught + // up and suspend activity. + if distance >= checkpoint_distance as i128 * self.shard_identity.count.count() as i128 { + info!( + "Will roll layer at {} with layer size {} due to LSN distance ({})", + projected_lsn, layer_size, distance + ); + + true + } else if projected_layer_size >= checkpoint_distance { + info!( + "Will roll layer at {} with layer size {} due to layer size ({})", + projected_lsn, layer_size, projected_layer_size + ); + + true + } else if distance > 0 && last_freeze_ts.elapsed() >= self.get_checkpoint_timeout() { + info!( + "Will roll layer at {} with layer size {} due to time since last flush ({:?})", + projected_lsn, + layer_size, + last_freeze_ts.elapsed() + ); + + true + } else { + false + } + } } /// Number of times we will compute partition within a checkpoint distance. @@ -4455,52 +4580,6 @@ impl<'a> TimelineWriter<'a> { res } - /// "Tick" the timeline writer: it will roll the open layer if required - /// and do nothing else. - pub(crate) async fn tick(&mut self) -> anyhow::Result<()> { - self.open_layer_if_present().await?; - - let last_record_lsn = self.get_last_record_lsn(); - let action = self.get_open_layer_action(last_record_lsn, 0); - if action == OpenLayerAction::Roll { - self.roll_layer(last_record_lsn).await?; - } else if let Some(writer_state) = &mut *self.write_guard { - // Periodic update of statistics - writer_state.open_layer.tick().await; - } - - Ok(()) - } - - /// Populate the timeline writer state only if an in-memory layer - /// is already open. - async fn open_layer_if_present(&mut self) -> anyhow::Result<()> { - assert!(self.write_guard.is_none()); - - let open_layer = { - let guard = self.layers.read().await; - let layers = guard.layer_map(); - match layers.open_layer { - Some(ref open_layer) => open_layer.clone(), - None => { - return Ok(()); - } - } - }; - - let initial_size = open_layer.size().await?; - let last_freeze_at = self.last_freeze_at.load(); - let last_freeze_ts = *self.last_freeze_ts.read().unwrap(); - self.write_guard.replace(TimelineWriterState::new( - open_layer, - initial_size, - last_freeze_at, - last_freeze_ts, - )); - - Ok(()) - } - async fn handle_open_layer_action( &mut self, at: Lsn, @@ -4572,43 +4651,14 @@ impl<'a> TimelineWriter<'a> { return OpenLayerAction::None; } - let distance = lsn.widening_sub(state.cached_last_freeze_at); - let proposed_open_layer_size = state.current_size + new_value_size; - - // Rolling the open layer can be triggered by: - // 1. The distance from the last LSN we rolled at. This bounds the amount of WAL that - // the safekeepers need to store. For sharded tenants, we multiply by shard count to - // account for how writes are distributed across shards: we expect each node to consume - // 1/count of the LSN on average. - // 2. The size of the currently open layer. - // 3. The time since the last roll. It helps safekeepers to regard pageserver as caught - // up and suspend activity. - if distance - >= self.get_checkpoint_distance() as i128 * self.shard_identity.count.count() as i128 - { - info!( - "Will roll layer at {} with layer size {} due to LSN distance ({})", - lsn, state.current_size, distance - ); - - OpenLayerAction::Roll - } else if proposed_open_layer_size >= self.get_checkpoint_distance() { - info!( - "Will roll layer at {} with layer size {} due to layer size ({})", - lsn, state.current_size, proposed_open_layer_size - ); - - OpenLayerAction::Roll - } else if distance > 0 - && state.cached_last_freeze_ts.elapsed() >= self.get_checkpoint_timeout() - { - info!( - "Will roll layer at {} with layer size {} due to time since last flush ({:?})", - lsn, - state.current_size, - state.cached_last_freeze_ts.elapsed() - ); - + if self.tl.should_roll( + state.current_size, + state.current_size + new_value_size, + self.get_checkpoint_distance(), + lsn, + state.cached_last_freeze_at, + state.cached_last_freeze_ts, + ) { OpenLayerAction::Roll } else { OpenLayerAction::None diff --git a/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs b/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs index d9f780cfd180..00a9dbd760df 100644 --- a/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs +++ b/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs @@ -389,17 +389,6 @@ pub(super) async fn handle_walreceiver_connection( } } - { - // This is a hack. It piggybacks on the keepalive messages sent by the - // safekeeper in order to enforce `checkpoint_timeout` on the currently - // open layer. This hack doesn't provide a bound on the total size of - // in-memory layers on a pageserver. See https://github.com/neondatabase/neon/issues/6916. - let mut writer = timeline.writer().await; - if let Err(err) = writer.tick().await { - warn!("Timeline writer tick failed: {err}"); - } - } - if let Some(last_lsn) = status_update { let timeline_remote_consistent_lsn = timeline .get_remote_consistent_lsn_visible() diff --git a/test_runner/regress/test_pageserver_layer_rolling.py b/test_runner/regress/test_pageserver_layer_rolling.py new file mode 100644 index 000000000000..c7e1e88468a1 --- /dev/null +++ b/test_runner/regress/test_pageserver_layer_rolling.py @@ -0,0 +1,275 @@ +import asyncio +import os +from typing import Tuple + +import psutil +import pytest +from fixtures.log_helper import log +from fixtures.neon_fixtures import ( + NeonEnv, + NeonEnvBuilder, + tenant_get_shards, +) +from fixtures.pageserver.http import PageserverHttpClient +from fixtures.pageserver.utils import wait_for_last_record_lsn, wait_for_upload +from fixtures.types import Lsn, TenantId, TimelineId +from fixtures.utils import wait_until + +TIMELINE_COUNT = 10 +ENTRIES_PER_TIMELINE = 10_000 +CHECKPOINT_TIMEOUT_SECONDS = 60 + + +async def run_worker(env: NeonEnv, tenant_conf, entries: int) -> Tuple[TenantId, TimelineId, Lsn]: + tenant, timeline = env.neon_cli.create_tenant(conf=tenant_conf) + with env.endpoints.create_start("main", tenant_id=tenant) as ep: + conn = await ep.connect_async() + try: + await conn.execute("CREATE TABLE IF NOT EXISTS t(key serial primary key, value text)") + await conn.execute( + f"INSERT INTO t SELECT i, CONCAT('payload_', i) FROM generate_series(0,{entries}) as i" + ) + finally: + await conn.close(timeout=10) + + last_flush_lsn = Lsn(ep.safe_psql("SELECT pg_current_wal_flush_lsn()")[0][0]) + return tenant, timeline, last_flush_lsn + + +async def workload( + env: NeonEnv, tenant_conf, timelines: int, entries: int +) -> list[Tuple[TenantId, TimelineId, Lsn]]: + workers = [asyncio.create_task(run_worker(env, tenant_conf, entries)) for _ in range(timelines)] + return await asyncio.gather(*workers) + + +def wait_until_pageserver_is_caught_up( + env: NeonEnv, last_flush_lsns: list[Tuple[TenantId, TimelineId, Lsn]] +): + for tenant, timeline, last_flush_lsn in last_flush_lsns: + shards = tenant_get_shards(env, tenant) + for tenant_shard_id, pageserver in shards: + waited = wait_for_last_record_lsn( + pageserver.http_client(), tenant_shard_id, timeline, last_flush_lsn + ) + assert waited >= last_flush_lsn + + +def wait_until_pageserver_has_uploaded( + env: NeonEnv, last_flush_lsns: list[Tuple[TenantId, TimelineId, Lsn]] +): + for tenant, timeline, last_flush_lsn in last_flush_lsns: + shards = tenant_get_shards(env, tenant) + for tenant_shard_id, pageserver in shards: + wait_for_upload(pageserver.http_client(), tenant_shard_id, timeline, last_flush_lsn) + + +def wait_for_wal_ingest_metric(pageserver_http: PageserverHttpClient) -> float: + def query(): + value = pageserver_http.get_metric_value("pageserver_wal_ingest_records_received_total") + assert value is not None + return value + + # The metric gets initialised on the first update. + # Retry a few times, but return 0 if it's stable. + try: + return float(wait_until(3, 0.5, query)) + except Exception: + return 0 + + +def get_dirty_bytes(env): + v = env.pageserver.http_client().get_metric_value("pageserver_timeline_ephemeral_bytes") or 0 + log.info(f"dirty_bytes: {v}") + return v + + +def assert_dirty_bytes(env, v): + assert get_dirty_bytes(env) == v + + +def assert_dirty_bytes_nonzero(env): + assert get_dirty_bytes(env) > 0 + + +@pytest.mark.parametrize("immediate_shutdown", [True, False]) +def test_pageserver_small_inmemory_layers( + neon_env_builder: NeonEnvBuilder, immediate_shutdown: bool +): + """ + Test that open layers get flushed after the `checkpoint_timeout` config + and do not require WAL reingest upon restart. + + The workload creates a number of timelines and writes some data to each, + but not enough to trigger flushes via the `checkpoint_distance` config. + """ + tenant_conf = { + # Large `checkpoint_distance` effectively disables size + # based checkpointing. + "checkpoint_distance": f"{2 * 1024 ** 3}", + "checkpoint_timeout": f"{CHECKPOINT_TIMEOUT_SECONDS}s", + "compaction_period": "1s", + } + + env = neon_env_builder.init_configs() + env.start() + + last_flush_lsns = asyncio.run(workload(env, tenant_conf, TIMELINE_COUNT, ENTRIES_PER_TIMELINE)) + wait_until_pageserver_is_caught_up(env, last_flush_lsns) + + # We didn't write enough data to trigger a size-based checkpoint: we should see dirty data. + wait_until(10, 1, lambda: assert_dirty_bytes_nonzero(env)) # type: ignore + + ps_http_client = env.pageserver.http_client() + total_wal_ingested_before_restart = wait_for_wal_ingest_metric(ps_http_client) + + # Within ~ the checkpoint interval, all the ephemeral layers should be frozen and flushed, + # such that there are zero bytes of ephemeral layer left on the pageserver + log.info("Waiting for background checkpoints...") + wait_until(CHECKPOINT_TIMEOUT_SECONDS * 2, 1, lambda: assert_dirty_bytes(env, 0)) # type: ignore + + # Zero ephemeral layer bytes does not imply that all the frozen layers were uploaded: they + # must be uploaded to remain visible to the pageserver after restart. + wait_until_pageserver_has_uploaded(env, last_flush_lsns) + + env.pageserver.restart(immediate=immediate_shutdown) + wait_until_pageserver_is_caught_up(env, last_flush_lsns) + + # Catching up with WAL ingest should have resulted in zero bytes of ephemeral layers, since + # we froze, flushed and uploaded everything before restarting. There can be no more WAL writes + # because we shut down compute endpoints before flushing. + assert get_dirty_bytes(env) == 0 + + total_wal_ingested_after_restart = wait_for_wal_ingest_metric(ps_http_client) + + log.info(f"WAL ingested before restart: {total_wal_ingested_before_restart}") + log.info(f"WAL ingested after restart: {total_wal_ingested_after_restart}") + + assert total_wal_ingested_after_restart == 0 + + +def test_idle_checkpoints(neon_env_builder: NeonEnvBuilder): + """ + Test that `checkpoint_timeout` is enforced even if there is no safekeeper input. + """ + tenant_conf = { + # Large `checkpoint_distance` effectively disables size + # based checkpointing. + "checkpoint_distance": f"{2 * 1024 ** 3}", + "checkpoint_timeout": f"{CHECKPOINT_TIMEOUT_SECONDS}s", + "compaction_period": "1s", + } + + env = neon_env_builder.init_configs() + env.start() + + last_flush_lsns = asyncio.run(workload(env, tenant_conf, TIMELINE_COUNT, ENTRIES_PER_TIMELINE)) + wait_until_pageserver_is_caught_up(env, last_flush_lsns) + + # We didn't write enough data to trigger a size-based checkpoint: we should see dirty data. + wait_until(10, 1, lambda: assert_dirty_bytes_nonzero(env)) # type: ignore + + # Stop the safekeepers, so that we cannot have any more WAL receiver connections + for sk in env.safekeepers: + sk.stop() + + # We should have got here fast enough that we didn't hit the background interval yet, + # and the teardown of SK connections shouldn't prompt any layer freezing. + assert get_dirty_bytes(env) > 0 + + # Within ~ the checkpoint interval, all the ephemeral layers should be frozen and flushed, + # such that there are zero bytes of ephemeral layer left on the pageserver + log.info("Waiting for background checkpoints...") + wait_until(CHECKPOINT_TIMEOUT_SECONDS * 2, 1, lambda: assert_dirty_bytes(env, 0)) # type: ignore + + +@pytest.mark.skipif( + # We have to use at least ~100MB of data to hit the lowest limit we can configure, which is + # prohibitively slow in debug mode + os.getenv("BUILD_TYPE") == "debug", + reason="Avoid running bulkier ingest tests in debug mode", +) +def test_total_size_limit(neon_env_builder: NeonEnvBuilder): + """ + Test that checkpoints are done based on total ephemeral layer size, even if no one timeline is + individually exceeding checkpoint thresholds. + """ + + system_memory = psutil.virtual_memory().total + + # The smallest total size limit we can configure is 1/1024th of the system memory (e.g. 128MB on + # a system with 128GB of RAM). We will then write enough data to violate this limit. + max_dirty_data = 128 * 1024 * 1024 + ephemeral_bytes_per_memory_kb = (max_dirty_data * 1024) // system_memory + assert ephemeral_bytes_per_memory_kb > 0 + + neon_env_builder.pageserver_config_override = f""" + ephemeral_bytes_per_memory_kb={ephemeral_bytes_per_memory_kb} + """ + + compaction_period_s = 10 + + tenant_conf = { + # Large space + time thresholds: effectively disable these limits + "checkpoint_distance": f"{1024 ** 4}", + "checkpoint_timeout": "3600s", + "compaction_period": f"{compaction_period_s}s", + } + + env = neon_env_builder.init_configs() + env.start() + + timeline_count = 10 + + # This is about 2MiB of data per timeline + entries_per_timeline = 100_000 + + last_flush_lsns = asyncio.run(workload(env, tenant_conf, timeline_count, entries_per_timeline)) + wait_until_pageserver_is_caught_up(env, last_flush_lsns) + + total_bytes_ingested = 0 + for tenant, timeline, last_flush_lsn in last_flush_lsns: + http_client = env.pageserver.http_client() + initdb_lsn = Lsn(http_client.timeline_detail(tenant, timeline)["initdb_lsn"]) + total_bytes_ingested += last_flush_lsn - initdb_lsn + + log.info(f"Ingested {total_bytes_ingested} bytes since initdb (vs max dirty {max_dirty_data})") + assert total_bytes_ingested > max_dirty_data + + # Expected end state: the total physical size of all the tenants is in excess of the max dirty + # data, but the total amount of dirty data is less than the limit: this demonstrates that we + # have exceeded the threshold but then rolled layers in response + def get_total_historic_layers(): + total_ephemeral_layers = 0 + total_historic_bytes = 0 + for tenant, timeline, _last_flush_lsn in last_flush_lsns: + http_client = env.pageserver.http_client() + initdb_lsn = Lsn(http_client.timeline_detail(tenant, timeline)["initdb_lsn"]) + layer_map = http_client.layer_map_info(tenant, timeline) + total_historic_bytes += sum( + layer.layer_file_size + for layer in layer_map.historic_layers + if layer.layer_file_size is not None and Lsn(layer.lsn_start) > initdb_lsn + ) + total_ephemeral_layers += len(layer_map.in_memory_layers) + + log.info( + f"Total historic layer bytes: {total_historic_bytes} ({total_ephemeral_layers} ephemeral layers)" + ) + + return total_historic_bytes + + def assert_bytes_rolled(): + assert total_bytes_ingested - get_total_historic_layers() <= max_dirty_data + + # Wait until enough layers have rolled that the amount of dirty data is under the threshold. + # We do this indirectly via layer maps, rather than the dirty bytes metric, to avoid false-passing + # if that metric isn't updated quickly enough to reflect the dirty bytes exceeding the limit. + wait_until(compaction_period_s * 2, 1, assert_bytes_rolled) + + # The end state should also have the reported metric under the limit + def assert_dirty_data_limited(): + dirty_bytes = get_dirty_bytes(env) + assert dirty_bytes < max_dirty_data + + wait_until(compaction_period_s * 2, 1, lambda: assert_dirty_data_limited()) # type: ignore diff --git a/test_runner/regress/test_pageserver_small_inmemory_layers.py b/test_runner/regress/test_pageserver_small_inmemory_layers.py deleted file mode 100644 index 714d1c12296c..000000000000 --- a/test_runner/regress/test_pageserver_small_inmemory_layers.py +++ /dev/null @@ -1,143 +0,0 @@ -import asyncio -from typing import Tuple - -import pytest -from fixtures.log_helper import log -from fixtures.neon_fixtures import ( - NeonEnv, - NeonEnvBuilder, - tenant_get_shards, -) -from fixtures.pageserver.http import PageserverHttpClient -from fixtures.pageserver.utils import wait_for_last_record_lsn, wait_for_upload -from fixtures.types import Lsn, TenantId, TimelineId -from fixtures.utils import wait_until - -TIMELINE_COUNT = 10 -ENTRIES_PER_TIMELINE = 10_000 -CHECKPOINT_TIMEOUT_SECONDS = 60 - -TENANT_CONF = { - # Large `checkpoint_distance` effectively disables size - # based checkpointing. - "checkpoint_distance": f"{2 * 1024 ** 3}", - "checkpoint_timeout": f"{CHECKPOINT_TIMEOUT_SECONDS}s", -} - - -async def run_worker(env: NeonEnv, entries: int) -> Tuple[TenantId, TimelineId, Lsn]: - tenant, timeline = env.neon_cli.create_tenant(conf=TENANT_CONF) - with env.endpoints.create_start("main", tenant_id=tenant) as ep: - conn = await ep.connect_async() - try: - await conn.execute("CREATE TABLE IF NOT EXISTS t(key serial primary key, value text)") - await conn.execute( - f"INSERT INTO t SELECT i, CONCAT('payload_', i) FROM generate_series(0,{entries}) as i" - ) - finally: - await conn.close(timeout=10) - - last_flush_lsn = Lsn(ep.safe_psql("SELECT pg_current_wal_flush_lsn()")[0][0]) - return tenant, timeline, last_flush_lsn - - -async def workload( - env: NeonEnv, timelines: int, entries: int -) -> list[Tuple[TenantId, TimelineId, Lsn]]: - workers = [asyncio.create_task(run_worker(env, entries)) for _ in range(timelines)] - return await asyncio.gather(*workers) - - -def wait_until_pageserver_is_caught_up( - env: NeonEnv, last_flush_lsns: list[Tuple[TenantId, TimelineId, Lsn]] -): - for tenant, timeline, last_flush_lsn in last_flush_lsns: - shards = tenant_get_shards(env, tenant) - for tenant_shard_id, pageserver in shards: - waited = wait_for_last_record_lsn( - pageserver.http_client(), tenant_shard_id, timeline, last_flush_lsn - ) - assert waited >= last_flush_lsn - - -def wait_until_pageserver_has_uploaded( - env: NeonEnv, last_flush_lsns: list[Tuple[TenantId, TimelineId, Lsn]] -): - for tenant, timeline, last_flush_lsn in last_flush_lsns: - shards = tenant_get_shards(env, tenant) - for tenant_shard_id, pageserver in shards: - wait_for_upload(pageserver.http_client(), tenant_shard_id, timeline, last_flush_lsn) - - -def wait_for_wal_ingest_metric(pageserver_http: PageserverHttpClient) -> float: - def query(): - value = pageserver_http.get_metric_value("pageserver_wal_ingest_records_received_total") - assert value is not None - return value - - # The metric gets initialised on the first update. - # Retry a few times, but return 0 if it's stable. - try: - return float(wait_until(3, 0.5, query)) - except Exception: - return 0 - - -@pytest.mark.parametrize("immediate_shutdown", [True, False]) -def test_pageserver_small_inmemory_layers( - neon_env_builder: NeonEnvBuilder, immediate_shutdown: bool -): - """ - Test that open layers get flushed after the `checkpoint_timeout` config - and do not require WAL reingest upon restart. - - The workload creates a number of timelines and writes some data to each, - but not enough to trigger flushes via the `checkpoint_distance` config. - """ - - def get_dirty_bytes(): - v = ( - env.pageserver.http_client().get_metric_value("pageserver_timeline_ephemeral_bytes") - or 0 - ) - log.info(f"dirty_bytes: {v}") - return v - - def assert_dirty_bytes(v): - assert get_dirty_bytes() == v - - env = neon_env_builder.init_configs() - env.start() - - last_flush_lsns = asyncio.run(workload(env, TIMELINE_COUNT, ENTRIES_PER_TIMELINE)) - wait_until_pageserver_is_caught_up(env, last_flush_lsns) - - # We didn't write enough data to trigger a size-based checkpoint - assert get_dirty_bytes() > 0 - - ps_http_client = env.pageserver.http_client() - total_wal_ingested_before_restart = wait_for_wal_ingest_metric(ps_http_client) - - # Within ~ the checkpoint interval, all the ephemeral layers should be frozen and flushed, - # such that there are zero bytes of ephemeral layer left on the pageserver - log.info("Waiting for background checkpoints...") - wait_until(CHECKPOINT_TIMEOUT_SECONDS * 2, 1, lambda: assert_dirty_bytes(0)) # type: ignore - - # Zero ephemeral layer bytes does not imply that all the frozen layers were uploaded: they - # must be uploaded to remain visible to the pageserver after restart. - wait_until_pageserver_has_uploaded(env, last_flush_lsns) - - env.pageserver.restart(immediate=immediate_shutdown) - wait_until_pageserver_is_caught_up(env, last_flush_lsns) - - # Catching up with WAL ingest should have resulted in zero bytes of ephemeral layers, since - # we froze, flushed and uploaded everything before restarting. There can be no more WAL writes - # because we shut down compute endpoints before flushing. - assert get_dirty_bytes() == 0 - - total_wal_ingested_after_restart = wait_for_wal_ingest_metric(ps_http_client) - - log.info(f"WAL ingested before restart: {total_wal_ingested_before_restart}") - log.info(f"WAL ingested after restart: {total_wal_ingested_after_restart}") - - assert total_wal_ingested_after_restart == 0 From b3bb1d1cad76f1a6cddf4c94d240705f8d58c427 Mon Sep 17 00:00:00 2001 From: John Spray Date: Tue, 26 Mar 2024 16:57:35 +0000 Subject: [PATCH 50/53] storage controller: make direct tenant creation more robust (#7247) ## Problem - Creations were not idempotent (unique key violation) - Creations waited for reconciliation, which control plane blocks while an operation is in flight ## Summary of changes - Handle unique key constraint violation as an OK situation: if we're creating the same tenant ID and shard count, it's reasonable to assume this is a duplicate creation. - Make the wait for reconcile during creation tolerate failures: this is similar to location_conf, where the cloud control plane blocks our notification calls until it is done with calling into our API (in future this constraint is expected to relax as the cloud control plane learns to run multiple operations concurrently for a tenant) --- .../attachment_service/src/service.rs | 31 +++++++++++++++---- test_runner/fixtures/neon_fixtures.py | 3 ++ test_runner/regress/test_sharding_service.py | 5 +++ 3 files changed, 33 insertions(+), 6 deletions(-) diff --git a/control_plane/attachment_service/src/service.rs b/control_plane/attachment_service/src/service.rs index aa930014b280..925910253b9c 100644 --- a/control_plane/attachment_service/src/service.rs +++ b/control_plane/attachment_service/src/service.rs @@ -1523,6 +1523,8 @@ impl Service { &self, create_req: TenantCreateRequest, ) -> Result { + let tenant_id = create_req.new_tenant_id.tenant_id; + // Exclude any concurrent attempts to create/access the same tenant ID let _tenant_lock = self .tenant_op_locks @@ -1531,7 +1533,12 @@ impl Service { let (response, waiters) = self.do_tenant_create(create_req).await?; - self.await_waiters(waiters, SHORT_RECONCILE_TIMEOUT).await?; + if let Err(e) = self.await_waiters(waiters, SHORT_RECONCILE_TIMEOUT).await { + // Avoid deadlock: reconcile may fail while notifying compute, if the cloud control plane refuses to + // accept compute notifications while it is in the process of creating. Reconciliation will + // be retried in the background. + tracing::warn!(%tenant_id, "Reconcile not done yet while creating tenant ({e})"); + } Ok(response) } @@ -1610,13 +1617,25 @@ impl Service { splitting: SplitState::default(), }) .collect(); - self.persistence + + match self + .persistence .insert_tenant_shards(persist_tenant_shards) .await - .map_err(|e| { - // TODO: distinguish primary key constraint (idempotent, OK), from other errors - ApiError::InternalServerError(anyhow::anyhow!(e)) - })?; + { + Ok(_) => {} + Err(DatabaseError::Query(diesel::result::Error::DatabaseError( + DatabaseErrorKind::UniqueViolation, + _, + ))) => { + // Unique key violation: this is probably a retry. Because the shard count is part of the unique key, + // if we see a unique key violation it means that the creation request's shard count matches the previous + // creation's shard count. + tracing::info!("Tenant shards already present in database, proceeding with idempotent creation..."); + } + // Any other database error is unexpected and a bug. + Err(e) => return Err(ApiError::InternalServerError(anyhow::anyhow!(e))), + }; let (waiters, response_shards) = { let mut locked = self.inner.write().unwrap(); diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index f22ce10c2024..3d60f9bef58f 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -2126,6 +2126,8 @@ def tenant_create( shard_params = {"count": shard_count} if shard_stripe_size is not None: shard_params["stripe_size"] = shard_stripe_size + else: + shard_params["stripe_size"] = 32768 body["shard_parameters"] = shard_params @@ -2139,6 +2141,7 @@ def tenant_create( json=body, headers=self.headers(TokenScope.PAGE_SERVER_API), ) + response.raise_for_status() log.info(f"tenant_create success: {response.json()}") def locate(self, tenant_id: TenantId) -> list[dict[str, Any]]: diff --git a/test_runner/regress/test_sharding_service.py b/test_runner/regress/test_sharding_service.py index b7488cadd6db..fc6c13766766 100644 --- a/test_runner/regress/test_sharding_service.py +++ b/test_runner/regress/test_sharding_service.py @@ -89,6 +89,11 @@ def test_sharding_service_smoke( for tid in tenant_ids: env.neon_cli.create_tenant(tid, shard_count=shards_per_tenant) + # Repeating a creation should be idempotent (we are just testing it doesn't return an error) + env.storage_controller.tenant_create( + tenant_id=next(iter(tenant_ids)), shard_count=shards_per_tenant + ) + for node_id, count in get_node_shard_counts(env, tenant_ids).items(): # we used a multiple of pagservers for the total shard count, # so expect equal number on all pageservers From 6814bb4b59809b7d08657fd57a05b6f6dbf7a409 Mon Sep 17 00:00:00 2001 From: John Spray Date: Tue, 26 Mar 2024 17:44:18 +0000 Subject: [PATCH 51/53] tests: add a log allow list to stabilize benchmarks (#7251) ## Problem https://github.com/neondatabase/neon/pull/7227 destabilized various tests in the performance suite, with log errors during shutdown. It's because we switched shutdown order to stop the storage controller before the pageservers. ## Summary of changes - Tolerate "connection failed" errors from pageservers trying to validation their deletion queue. --- test_runner/fixtures/pageserver/allowed_errors.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/test_runner/fixtures/pageserver/allowed_errors.py b/test_runner/fixtures/pageserver/allowed_errors.py index d7f682dad335..8b895dcd9299 100755 --- a/test_runner/fixtures/pageserver/allowed_errors.py +++ b/test_runner/fixtures/pageserver/allowed_errors.py @@ -86,6 +86,9 @@ def scan_pageserver_log_for_errors( # This is especially pronounced in tests that set small checkpoint # distances. ".*Flushed oversized open layer with size.*", + # During teardown, we stop the storage controller before the pageservers, so pageservers + # can experience connection errors doing background deletion queue work. + ".*WARN deletion backend: calling control plane generation validation API failed.*Connection refused.*", ) From b3b7ce457cdb5d0f6aa9d01cb3aaedf16c6d51c3 Mon Sep 17 00:00:00 2001 From: John Spray Date: Tue, 26 Mar 2024 18:29:08 +0000 Subject: [PATCH 52/53] pageserver: remove bare mgr::get_tenant, mgr::list_tenants (#7237) ## Problem This is a refactor. This PR was a precursor to a much smaller change https://github.com/neondatabase/neon/commit/e5bd602dc11dc0fbc483ce0abaf486b3407f42dc, where as I was writing it I found that we were not far from getting rid of the last non-deprecated code paths that use `mgr::` scoped functions to get at the TenantManager state. We're almost done cleaning this up as per https://github.com/neondatabase/neon/issues/5796. The only significant remaining mgr:: item is `get_active_tenant_with_timeout`, which is page_service's path for fetching tenants. ## Summary of changes - Remove the bool argument to get_attached_tenant_shard: this was almost always false from API use cases, and in cases when it was true, it was readily replacable with an explicit check of the returned tenant's status. - Rather than letting the timeline eviction task query any tenant it likes via `mgr::`, pass an `Arc` into the task. This is still an ugly circular reference, but should eventually go away: either when we switch to exclusively using disk usage eviction, or when we change metadata storage to avoid the need to imitate layer accesses. - Convert all the mgr::get_tenant call sites to use TenantManager::get_attached_tenant_shard - Move list_tenants into TenantManager. --- pageserver/src/bin/pageserver.rs | 58 +++--- pageserver/src/consumption_metrics.rs | 42 +++-- pageserver/src/consumption_metrics/metrics.rs | 7 +- pageserver/src/disk_usage_eviction_task.rs | 13 +- pageserver/src/http/routes.rs | 110 +++++++----- pageserver/src/page_service.rs | 1 + pageserver/src/tenant.rs | 18 +- pageserver/src/tenant/mgr.rs | 168 ++++-------------- .../src/tenant/secondary/heatmap_uploader.rs | 6 +- pageserver/src/tenant/timeline.rs | 3 +- .../src/tenant/timeline/eviction_task.rs | 37 ++-- pageserver/src/tenant/timeline/uninit.rs | 3 +- .../regress/test_pageserver_secondary.py | 2 + 13 files changed, 223 insertions(+), 245 deletions(-) diff --git a/pageserver/src/bin/pageserver.rs b/pageserver/src/bin/pageserver.rs index ef616c0a390c..c80230d4d719 100644 --- a/pageserver/src/bin/pageserver.rs +++ b/pageserver/src/bin/pageserver.rs @@ -600,33 +600,37 @@ fn start_pageserver( None, "consumption metrics collection", true, - async move { - // first wait until background jobs are cleared to launch. - // - // this is because we only process active tenants and timelines, and the - // Timeline::get_current_logical_size will spawn the logical size calculation, - // which will not be rate-limited. - let cancel = task_mgr::shutdown_token(); - - tokio::select! { - _ = cancel.cancelled() => { return Ok(()); }, - _ = background_jobs_barrier.wait() => {} - }; - - pageserver::consumption_metrics::collect_metrics( - metric_collection_endpoint, - &conf.metric_collection_bucket, - conf.metric_collection_interval, - conf.cached_metric_collection_interval, - conf.synthetic_size_calculation_interval, - conf.id, - local_disk_storage, - cancel, - metrics_ctx, - ) - .instrument(info_span!("metrics_collection")) - .await?; - Ok(()) + { + let tenant_manager = tenant_manager.clone(); + async move { + // first wait until background jobs are cleared to launch. + // + // this is because we only process active tenants and timelines, and the + // Timeline::get_current_logical_size will spawn the logical size calculation, + // which will not be rate-limited. + let cancel = task_mgr::shutdown_token(); + + tokio::select! { + _ = cancel.cancelled() => { return Ok(()); }, + _ = background_jobs_barrier.wait() => {} + }; + + pageserver::consumption_metrics::collect_metrics( + tenant_manager, + metric_collection_endpoint, + &conf.metric_collection_bucket, + conf.metric_collection_interval, + conf.cached_metric_collection_interval, + conf.synthetic_size_calculation_interval, + conf.id, + local_disk_storage, + cancel, + metrics_ctx, + ) + .instrument(info_span!("metrics_collection")) + .await?; + Ok(()) + } }, ); } diff --git a/pageserver/src/consumption_metrics.rs b/pageserver/src/consumption_metrics.rs index 3429e3a0a673..f5540e896f68 100644 --- a/pageserver/src/consumption_metrics.rs +++ b/pageserver/src/consumption_metrics.rs @@ -3,7 +3,9 @@ use crate::context::{DownloadBehavior, RequestContext}; use crate::task_mgr::{self, TaskKind, BACKGROUND_RUNTIME}; use crate::tenant::tasks::BackgroundLoopKind; -use crate::tenant::{mgr, LogicalSizeCalculationCause, PageReconstructError, Tenant}; +use crate::tenant::{ + mgr::TenantManager, LogicalSizeCalculationCause, PageReconstructError, Tenant, +}; use camino::Utf8PathBuf; use consumption_metrics::EventType; use pageserver_api::models::TenantState; @@ -41,6 +43,7 @@ type Cache = HashMap; /// Main thread that serves metrics collection #[allow(clippy::too_many_arguments)] pub async fn collect_metrics( + tenant_manager: Arc, metric_collection_endpoint: &Url, metric_collection_bucket: &Option, metric_collection_interval: Duration, @@ -67,15 +70,19 @@ pub async fn collect_metrics( None, "synthetic size calculation", false, - async move { - calculate_synthetic_size_worker( - synthetic_size_calculation_interval, - &cancel, - &worker_ctx, - ) - .instrument(info_span!("synthetic_size_worker")) - .await?; - Ok(()) + { + let tenant_manager = tenant_manager.clone(); + async move { + calculate_synthetic_size_worker( + tenant_manager, + synthetic_size_calculation_interval, + &cancel, + &worker_ctx, + ) + .instrument(info_span!("synthetic_size_worker")) + .await?; + Ok(()) + } }, ); @@ -116,7 +123,7 @@ pub async fn collect_metrics( let started_at = Instant::now(); // these are point in time, with variable "now" - let metrics = metrics::collect_all_metrics(&cached_metrics, &ctx).await; + let metrics = metrics::collect_all_metrics(&tenant_manager, &cached_metrics, &ctx).await; let metrics = Arc::new(metrics); @@ -271,6 +278,7 @@ async fn reschedule( /// Caclculate synthetic size for each active tenant async fn calculate_synthetic_size_worker( + tenant_manager: Arc, synthetic_size_calculation_interval: Duration, cancel: &CancellationToken, ctx: &RequestContext, @@ -283,7 +291,7 @@ async fn calculate_synthetic_size_worker( loop { let started_at = Instant::now(); - let tenants = match mgr::list_tenants().await { + let tenants = match tenant_manager.list_tenants() { Ok(tenants) => tenants, Err(e) => { warn!("cannot get tenant list: {e:#}"); @@ -302,10 +310,14 @@ async fn calculate_synthetic_size_worker( continue; } - let Ok(tenant) = mgr::get_tenant(tenant_shard_id, true) else { + let Ok(tenant) = tenant_manager.get_attached_tenant_shard(tenant_shard_id) else { continue; }; + if !tenant.is_active() { + continue; + } + // there is never any reason to exit calculate_synthetic_size_worker following any // return value -- we don't need to care about shutdown because no tenant is found when // pageserver is shut down. @@ -343,9 +355,7 @@ async fn calculate_and_log(tenant: &Tenant, cancel: &CancellationToken, ctx: &Re }; // this error can be returned if timeline is shutting down, but it does not - // mean the synthetic size worker should terminate. we do not need any checks - // in this function because `mgr::get_tenant` will error out after shutdown has - // progressed to shutting down tenants. + // mean the synthetic size worker should terminate. let shutting_down = matches!( e.downcast_ref::(), Some(PageReconstructError::Cancelled | PageReconstructError::AncestorStopping(_)) diff --git a/pageserver/src/consumption_metrics/metrics.rs b/pageserver/src/consumption_metrics/metrics.rs index 26b299a71d49..6740c1360bad 100644 --- a/pageserver/src/consumption_metrics/metrics.rs +++ b/pageserver/src/consumption_metrics/metrics.rs @@ -1,3 +1,4 @@ +use crate::tenant::mgr::TenantManager; use crate::{context::RequestContext, tenant::timeline::logical_size::CurrentLogicalSize}; use chrono::{DateTime, Utc}; use consumption_metrics::EventType; @@ -181,6 +182,7 @@ impl MetricsKey { } pub(super) async fn collect_all_metrics( + tenant_manager: &Arc, cached_metrics: &Cache, ctx: &RequestContext, ) -> Vec { @@ -188,7 +190,7 @@ pub(super) async fn collect_all_metrics( let started_at = std::time::Instant::now(); - let tenants = match crate::tenant::mgr::list_tenants().await { + let tenants = match tenant_manager.list_tenants() { Ok(tenants) => tenants, Err(err) => { tracing::error!("failed to list tenants: {:?}", err); @@ -200,7 +202,8 @@ pub(super) async fn collect_all_metrics( if state != TenantState::Active || !id.is_zero() { None } else { - crate::tenant::mgr::get_tenant(id, true) + tenant_manager + .get_attached_tenant_shard(id) .ok() .map(|tenant| (id.tenant_id, tenant)) } diff --git a/pageserver/src/disk_usage_eviction_task.rs b/pageserver/src/disk_usage_eviction_task.rs index 92c1475aeff5..6248424cee49 100644 --- a/pageserver/src/disk_usage_eviction_task.rs +++ b/pageserver/src/disk_usage_eviction_task.rs @@ -61,7 +61,6 @@ use crate::{ metrics::disk_usage_based_eviction::METRICS, task_mgr::{self, TaskKind, BACKGROUND_RUNTIME}, tenant::{ - self, mgr::TenantManager, remote_timeline_client::LayerFileMetadata, secondary::SecondaryTenant, @@ -814,8 +813,8 @@ async fn collect_eviction_candidates( const LOG_DURATION_THRESHOLD: std::time::Duration = std::time::Duration::from_secs(10); // get a snapshot of the list of tenants - let tenants = tenant::mgr::list_tenants() - .await + let tenants = tenant_manager + .list_tenants() .context("get list of tenants")?; // TODO: avoid listing every layer in every tenant: this loop can block the executor, @@ -827,8 +826,12 @@ async fn collect_eviction_candidates( if cancel.is_cancelled() { return Ok(EvictionCandidates::Cancelled); } - let tenant = match tenant::mgr::get_tenant(tenant_id, true) { - Ok(tenant) => tenant, + let tenant = match tenant_manager.get_attached_tenant_shard(tenant_id) { + Ok(tenant) if tenant.is_active() => tenant, + Ok(_) => { + debug!(tenant_id=%tenant_id.tenant_id, shard_id=%tenant_id.shard_slug(), "Tenant shard is not active"); + continue; + } Err(e) => { // this can happen if tenant has lifecycle transition after we fetched it debug!("failed to get tenant: {e:#}"); diff --git a/pageserver/src/http/routes.rs b/pageserver/src/http/routes.rs index 3cc92216ed2a..759a1b25ee17 100644 --- a/pageserver/src/http/routes.rs +++ b/pageserver/src/http/routes.rs @@ -49,8 +49,8 @@ use crate::task_mgr::TaskKind; use crate::tenant::config::{LocationConf, TenantConfOpt}; use crate::tenant::mgr::GetActiveTenantError; use crate::tenant::mgr::{ - GetTenantError, SetNewTenantConfigError, TenantManager, TenantMapError, TenantMapInsertError, - TenantSlotError, TenantSlotUpsertError, TenantStateError, + GetTenantError, TenantManager, TenantMapError, TenantMapInsertError, TenantSlotError, + TenantSlotUpsertError, TenantStateError, }; use crate::tenant::mgr::{TenantSlot, UpsertLocationError}; use crate::tenant::remote_timeline_client; @@ -249,16 +249,11 @@ impl From for ApiError { fn from(tse: GetTenantError) -> ApiError { match tse { GetTenantError::NotFound(tid) => ApiError::NotFound(anyhow!("tenant {}", tid).into()), - GetTenantError::Broken(reason) => { - ApiError::InternalServerError(anyhow!("tenant is broken: {}", reason)) - } GetTenantError::NotActive(_) => { // Why is this not `ApiError::NotFound`? // Because we must be careful to never return 404 for a tenant if it does // in fact exist locally. If we did, the caller could draw the conclusion // that it can attach the tenant to another PS and we'd be in split-brain. - // - // (We can produce this variant only in `mgr::get_tenant(..., active=true)` calls). ApiError::ResourceUnavailable("Tenant not yet active".into()) } GetTenantError::MapState(e) => ApiError::ResourceUnavailable(format!("{e}").into()), @@ -269,6 +264,9 @@ impl From for ApiError { impl From for ApiError { fn from(e: GetActiveTenantError) -> ApiError { match e { + GetActiveTenantError::Broken(reason) => { + ApiError::InternalServerError(anyhow!("tenant is broken: {}", reason)) + } GetActiveTenantError::WillNotBecomeActive(_) => ApiError::Conflict(format!("{}", e)), GetActiveTenantError::Cancelled => ApiError::ShuttingDown, GetActiveTenantError::NotFound(gte) => gte.into(), @@ -279,19 +277,6 @@ impl From for ApiError { } } -impl From for ApiError { - fn from(e: SetNewTenantConfigError) -> ApiError { - match e { - SetNewTenantConfigError::GetTenant(tid) => { - ApiError::NotFound(anyhow!("tenant {}", tid).into()) - } - e @ (SetNewTenantConfigError::Persist(_) | SetNewTenantConfigError::Other(_)) => { - ApiError::InternalServerError(anyhow::Error::new(e)) - } - } - } -} - impl From for ApiError { fn from(value: crate::tenant::DeleteTimelineError) -> Self { use crate::tenant::DeleteTimelineError::*; @@ -495,7 +480,7 @@ async fn timeline_create_handler( async { let tenant = state .tenant_manager - .get_attached_tenant_shard(tenant_shard_id, false)?; + .get_attached_tenant_shard(tenant_shard_id)?; tenant.wait_to_become_active(ACTIVE_TENANT_TIMEOUT).await?; @@ -584,7 +569,7 @@ async fn timeline_list_handler( let response_data = async { let tenant = state .tenant_manager - .get_attached_tenant_shard(tenant_shard_id, false)?; + .get_attached_tenant_shard(tenant_shard_id)?; tenant.wait_to_become_active(ACTIVE_TENANT_TIMEOUT).await?; @@ -622,6 +607,7 @@ async fn timeline_preserve_initdb_handler( let tenant_shard_id: TenantShardId = parse_request_param(&request, "tenant_shard_id")?; let timeline_id: TimelineId = parse_request_param(&request, "timeline_id")?; check_permission(&request, Some(tenant_shard_id.tenant_id))?; + let state = get_state(&request); // Part of the process for disaster recovery from safekeeper-stored WAL: // If we don't recover into a new timeline but want to keep the timeline ID, @@ -629,7 +615,9 @@ async fn timeline_preserve_initdb_handler( // location where timeline recreation cand find it. async { - let tenant = mgr::get_tenant(tenant_shard_id, false)?; + let tenant = state + .tenant_manager + .get_attached_tenant_shard(tenant_shard_id)?; let timeline = tenant .get_timeline(timeline_id, false) @@ -671,7 +659,7 @@ async fn timeline_detail_handler( let timeline_info = async { let tenant = state .tenant_manager - .get_attached_tenant_shard(tenant_shard_id, false)?; + .get_attached_tenant_shard(tenant_shard_id)?; tenant.wait_to_become_active(ACTIVE_TENANT_TIMEOUT).await?; @@ -858,7 +846,7 @@ async fn timeline_delete_handler( let tenant = state .tenant_manager - .get_attached_tenant_shard(tenant_shard_id, false) + .get_attached_tenant_shard(tenant_shard_id) .map_err(|e| { match e { // GetTenantError has a built-in conversion to ApiError, but in this context we don't @@ -976,10 +964,11 @@ async fn tenant_list_handler( _cancel: CancellationToken, ) -> Result, ApiError> { check_permission(&request, None)?; + let state = get_state(&request); - let response_data = mgr::list_tenants() - .instrument(info_span!("tenant_list")) - .await + let response_data = state + .tenant_manager + .list_tenants() .map_err(|_| { ApiError::ResourceUnavailable("Tenant map is initializing or shutting down".into()) })? @@ -1002,9 +991,12 @@ async fn tenant_status( ) -> Result, ApiError> { let tenant_shard_id: TenantShardId = parse_request_param(&request, "tenant_shard_id")?; check_permission(&request, Some(tenant_shard_id.tenant_id))?; + let state = get_state(&request); let tenant_info = async { - let tenant = mgr::get_tenant(tenant_shard_id, false)?; + let tenant = state + .tenant_manager + .get_attached_tenant_shard(tenant_shard_id)?; // Calculate total physical size of all timelines let mut current_physical_size = 0; @@ -1077,9 +1069,7 @@ async fn tenant_size_handler( let inputs_only: Option = parse_query_param(&request, "inputs_only")?; let retention_period: Option = parse_query_param(&request, "retention_period")?; let headers = request.headers(); - - let ctx = RequestContext::new(TaskKind::MgmtRequest, DownloadBehavior::Download); - let tenant = mgr::get_tenant(tenant_shard_id, true)?; + let state = get_state(&request); if !tenant_shard_id.is_zero() { return Err(ApiError::BadRequest(anyhow!( @@ -1087,6 +1077,12 @@ async fn tenant_size_handler( ))); } + let ctx = RequestContext::new(TaskKind::MgmtRequest, DownloadBehavior::Download); + let tenant = state + .tenant_manager + .get_attached_tenant_shard(tenant_shard_id)?; + tenant.wait_to_become_active(ACTIVE_TENANT_TIMEOUT).await?; + // this can be long operation let inputs = tenant .gather_size_inputs( @@ -1155,10 +1151,15 @@ async fn tenant_shard_split_handler( let state = get_state(&request); let ctx = RequestContext::new(TaskKind::MgmtRequest, DownloadBehavior::Warn); + let tenant = state + .tenant_manager + .get_attached_tenant_shard(tenant_shard_id)?; + tenant.wait_to_become_active(ACTIVE_TENANT_TIMEOUT).await?; + let new_shards = state .tenant_manager .shard_split( - tenant_shard_id, + tenant, ShardCount::new(req.new_shard_count), req.new_stripe_size, &ctx, @@ -1376,8 +1377,11 @@ async fn get_tenant_config_handler( ) -> Result, ApiError> { let tenant_shard_id: TenantShardId = parse_request_param(&request, "tenant_shard_id")?; check_permission(&request, Some(tenant_shard_id.tenant_id))?; + let state = get_state(&request); - let tenant = mgr::get_tenant(tenant_shard_id, false)?; + let tenant = state + .tenant_manager + .get_attached_tenant_shard(tenant_shard_id)?; let response = HashMap::from([ ( @@ -1405,15 +1409,31 @@ async fn update_tenant_config_handler( let tenant_id = request_data.tenant_id; check_permission(&request, Some(tenant_id))?; - let tenant_conf = + let new_tenant_conf = TenantConfOpt::try_from(&request_data.config).map_err(ApiError::BadRequest)?; let state = get_state(&request); - state + + let tenant_shard_id = TenantShardId::unsharded(tenant_id); + + let tenant = state .tenant_manager - .set_new_tenant_config(tenant_conf, tenant_id) - .instrument(info_span!("tenant_config", %tenant_id)) - .await?; + .get_attached_tenant_shard(tenant_shard_id)?; + tenant.wait_to_become_active(ACTIVE_TENANT_TIMEOUT).await?; + + // This is a legacy API that only operates on attached tenants: the preferred + // API to use is the location_config/ endpoint, which lets the caller provide + // the full LocationConf. + let location_conf = LocationConf::attached_single( + new_tenant_conf.clone(), + tenant.get_generation(), + &ShardParameters::default(), + ); + + crate::tenant::Tenant::persist_tenant_config(state.conf, &tenant_shard_id, &location_conf) + .await + .map_err(ApiError::InternalServerError)?; + tenant.set_new_tenant_config(new_tenant_conf); json_response(StatusCode::OK, ()) } @@ -1637,10 +1657,12 @@ async fn handle_tenant_break( ) -> Result, ApiError> { let tenant_shard_id: TenantShardId = parse_request_param(&r, "tenant_shard_id")?; - let tenant = crate::tenant::mgr::get_tenant(tenant_shard_id, true) - .map_err(|_| ApiError::Conflict(String::from("no active tenant found")))?; - - tenant.set_broken("broken from test".to_owned()).await; + let state = get_state(&r); + state + .tenant_manager + .get_attached_tenant_shard(tenant_shard_id)? + .set_broken("broken from test".to_owned()) + .await; json_response(StatusCode::OK, ()) } @@ -1884,7 +1906,7 @@ async fn active_timeline_of_active_tenant( tenant_shard_id: TenantShardId, timeline_id: TimelineId, ) -> Result, ApiError> { - let tenant = tenant_manager.get_attached_tenant_shard(tenant_shard_id, false)?; + let tenant = tenant_manager.get_attached_tenant_shard(tenant_shard_id)?; tenant.wait_to_become_active(ACTIVE_TENANT_TIMEOUT).await?; diff --git a/pageserver/src/page_service.rs b/pageserver/src/page_service.rs index f3ceb7d3e6e5..3d622f18712c 100644 --- a/pageserver/src/page_service.rs +++ b/pageserver/src/page_service.rs @@ -760,6 +760,7 @@ impl PageServerHandler { let mut copyin_reader = pin!(StreamReader::new(self.copyin_stream(pgb, &tenant.cancel))); timeline .import_basebackup_from_tar( + tenant.clone(), &mut copyin_reader, base_lsn, self.broker_client.clone(), diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index dcf9b1a605ba..792d9e548d21 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -1411,7 +1411,7 @@ impl Tenant { /// the same timeline ID already exists, returns CreateTimelineError::AlreadyExists. #[allow(clippy::too_many_arguments)] pub(crate) async fn create_timeline( - &self, + self: &Arc, new_timeline_id: TimelineId, ancestor_timeline_id: Option, mut ancestor_start_lsn: Option, @@ -1559,7 +1559,7 @@ impl Tenant { })?; } - loaded_timeline.activate(broker_client, None, ctx); + loaded_timeline.activate(self.clone(), broker_client, None, ctx); Ok(loaded_timeline) } @@ -1731,7 +1731,12 @@ impl Tenant { let mut activated_timelines = 0; for timeline in timelines_to_activate { - timeline.activate(broker_client.clone(), background_jobs_can_start, ctx); + timeline.activate( + self.clone(), + broker_client.clone(), + background_jobs_can_start, + ctx, + ); activated_timelines += 1; } @@ -2063,7 +2068,12 @@ impl Tenant { TenantState::Active { .. } => { return Ok(()); } - TenantState::Broken { .. } | TenantState::Stopping { .. } => { + TenantState::Broken { reason, .. } => { + // This is fatal, and reported distinctly from the general case of "will never be active" because + // it's logically a 500 to external API users (broken is always a bug). + return Err(GetActiveTenantError::Broken(reason)); + } + TenantState::Stopping { .. } => { // There's no chance the tenant can transition back into ::Active return Err(GetActiveTenantError::WillNotBecomeActive(current_state)); } diff --git a/pageserver/src/tenant/mgr.rs b/pageserver/src/tenant/mgr.rs index 8e3eae7cf6d7..f01fb9791c19 100644 --- a/pageserver/src/tenant/mgr.rs +++ b/pageserver/src/tenant/mgr.rs @@ -4,7 +4,7 @@ use camino::{Utf8DirEntry, Utf8Path, Utf8PathBuf}; use itertools::Itertools; use pageserver_api::key::Key; -use pageserver_api::models::{LocationConfigMode, ShardParameters}; +use pageserver_api::models::LocationConfigMode; use pageserver_api::shard::{ ShardCount, ShardIdentity, ShardNumber, ShardStripeSize, TenantShardId, }; @@ -40,7 +40,6 @@ use crate::metrics::{TENANT, TENANT_MANAGER as METRICS}; use crate::task_mgr::{self, TaskKind}; use crate::tenant::config::{ AttachedLocationConfig, AttachmentMode, LocationConf, LocationMode, SecondaryLocationConfig, - TenantConfOpt, }; use crate::tenant::delete::DeleteTenantFlow; use crate::tenant::span::debug_assert_current_span_has_tenant_id; @@ -889,16 +888,6 @@ async fn shutdown_all_tenants0(tenants: &std::sync::RwLock) { // caller will log how long we took } -#[derive(Debug, thiserror::Error)] -pub(crate) enum SetNewTenantConfigError { - #[error(transparent)] - GetTenant(#[from] GetTenantError), - #[error(transparent)] - Persist(anyhow::Error), - #[error(transparent)] - Other(anyhow::Error), -} - #[derive(thiserror::Error, Debug)] pub(crate) enum UpsertLocationError { #[error("Bad config request: {0}")] @@ -924,32 +913,21 @@ impl TenantManager { self.conf } - /// Gets the attached tenant from the in-memory data, erroring if it's absent, in secondary mode, or is not fitting to the query. - /// `active_only = true` allows to query only tenants that are ready for operations, erroring on other kinds of tenants. + /// Gets the attached tenant from the in-memory data, erroring if it's absent, in secondary mode, or currently + /// undergoing a state change (i.e. slot is InProgress). + /// + /// The return Tenant is not guaranteed to be active: check its status after obtaing it, or + /// use [`Tenant::wait_to_become_active`] before using it if you will do I/O on it. pub(crate) fn get_attached_tenant_shard( &self, tenant_shard_id: TenantShardId, - active_only: bool, ) -> Result, GetTenantError> { let locked = self.tenants.read().unwrap(); let peek_slot = tenant_map_peek_slot(&locked, &tenant_shard_id, TenantSlotPeekMode::Read)?; match peek_slot { - Some(TenantSlot::Attached(tenant)) => match tenant.current_state() { - TenantState::Broken { - reason, - backtrace: _, - } if active_only => Err(GetTenantError::Broken(reason)), - TenantState::Active => Ok(Arc::clone(tenant)), - _ => { - if active_only { - Err(GetTenantError::NotActive(tenant_shard_id)) - } else { - Ok(Arc::clone(tenant)) - } - } - }, + Some(TenantSlot::Attached(tenant)) => Ok(Arc::clone(tenant)), Some(TenantSlot::InProgress(_)) => Err(GetTenantError::NotActive(tenant_shard_id)), None | Some(TenantSlot::Secondary(_)) => { Err(GetTenantError::NotFound(tenant_shard_id.tenant_id)) @@ -1442,7 +1420,8 @@ impl TenantManager { .wait_to_become_active(activation_timeout) .await .map_err(|e| match e { - GetActiveTenantError::WillNotBecomeActive(_) => { + GetActiveTenantError::WillNotBecomeActive(_) + | GetActiveTenantError::Broken(_) => { DeleteTenantError::InvalidState(tenant.current_state()) } GetActiveTenantError::Cancelled => DeleteTenantError::Cancelled, @@ -1469,29 +1448,30 @@ impl TenantManager { result } - #[instrument(skip_all, fields(tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(), new_shard_count=%new_shard_count.literal()))] + #[instrument(skip_all, fields(tenant_id=%tenant.get_tenant_shard_id().tenant_id, shard_id=%tenant.get_tenant_shard_id().shard_slug(), new_shard_count=%new_shard_count.literal()))] pub(crate) async fn shard_split( &self, - tenant_shard_id: TenantShardId, + tenant: Arc, new_shard_count: ShardCount, new_stripe_size: Option, ctx: &RequestContext, ) -> anyhow::Result> { + let tenant_shard_id = *tenant.get_tenant_shard_id(); let r = self - .do_shard_split(tenant_shard_id, new_shard_count, new_stripe_size, ctx) + .do_shard_split(tenant, new_shard_count, new_stripe_size, ctx) .await; if r.is_err() { // Shard splitting might have left the original shard in a partially shut down state (it // stops the shard's remote timeline client). Reset it to ensure we leave things in // a working state. if self.get(tenant_shard_id).is_some() { - tracing::warn!("Resetting {tenant_shard_id} after shard split failure"); + tracing::warn!("Resetting after shard split failure"); if let Err(e) = self.reset_tenant(tenant_shard_id, false, ctx).await { // Log this error because our return value will still be the original error, not this one. This is // a severe error: if this happens, we might be leaving behind a tenant that is not fully functional // (e.g. has uploads disabled). We can't do anything else: if reset fails then shutting the tenant down or // setting it broken probably won't help either. - tracing::error!("Failed to reset {tenant_shard_id}: {e}"); + tracing::error!("Failed to reset: {e}"); } } } @@ -1501,12 +1481,12 @@ impl TenantManager { pub(crate) async fn do_shard_split( &self, - tenant_shard_id: TenantShardId, + tenant: Arc, new_shard_count: ShardCount, new_stripe_size: Option, ctx: &RequestContext, ) -> anyhow::Result> { - let tenant = get_tenant(tenant_shard_id, true)?; + let tenant_shard_id = *tenant.get_tenant_shard_id(); // Validate the incoming request if new_shard_count.count() <= tenant_shard_id.shard_count.count() { @@ -1552,7 +1532,6 @@ impl TenantManager { // If [`Tenant::split_prepare`] fails, we must reload the tenant, because it might // have been left in a partially-shut-down state. tracing::warn!("Failed to prepare for split: {e}, reloading Tenant before returning"); - self.reset_tenant(tenant_shard_id, false, ctx).await?; return Err(e); } @@ -1950,38 +1929,23 @@ impl TenantManager { removal_result } - pub(crate) async fn set_new_tenant_config( + pub(crate) fn list_tenants( &self, - new_tenant_conf: TenantConfOpt, - tenant_id: TenantId, - ) -> Result<(), SetNewTenantConfigError> { - // Legacy API: does not support sharding - let tenant_shard_id = TenantShardId::unsharded(tenant_id); - - info!("configuring tenant {tenant_id}"); - let tenant = get_tenant(tenant_shard_id, true)?; - - if !tenant.tenant_shard_id().shard_count.is_unsharded() { - // Note that we use ShardParameters::default below. - return Err(SetNewTenantConfigError::Other(anyhow::anyhow!( - "This API may only be used on single-sharded tenants, use the /location_config API for sharded tenants" - ))); - } - - // This is a legacy API that only operates on attached tenants: the preferred - // API to use is the location_config/ endpoint, which lets the caller provide - // the full LocationConf. - let location_conf = LocationConf::attached_single( - new_tenant_conf.clone(), - tenant.generation, - &ShardParameters::default(), - ); - - Tenant::persist_tenant_config(self.conf, &tenant_shard_id, &location_conf) - .await - .map_err(SetNewTenantConfigError::Persist)?; - tenant.set_new_tenant_config(new_tenant_conf); - Ok(()) + ) -> Result, TenantMapListError> { + let tenants = TENANTS.read().unwrap(); + let m = match &*tenants { + TenantsMap::Initializing => return Err(TenantMapListError::Initializing), + TenantsMap::Open(m) | TenantsMap::ShuttingDown(m) => m, + }; + Ok(m.iter() + .filter_map(|(id, tenant)| match tenant { + TenantSlot::Attached(tenant) => { + Some((*id, tenant.current_state(), tenant.generation())) + } + TenantSlot::Secondary(_) => None, + TenantSlot::InProgress(_) => None, + }) + .collect()) } } @@ -1994,51 +1958,12 @@ pub(crate) enum GetTenantError { #[error("Tenant {0} is not active")] NotActive(TenantShardId), - /// Broken is logically a subset of NotActive, but a distinct error is useful as - /// NotActive is usually a retryable state for API purposes, whereas Broken - /// is a stuck error state - #[error("Tenant is broken: {0}")] - Broken(String), // Initializing or shutting down: cannot authoritatively say whether we have this tenant #[error("Tenant map is not available: {0}")] MapState(#[from] TenantMapError), } -/// Gets the tenant from the in-memory data, erroring if it's absent or is not fitting to the query. -/// `active_only = true` allows to query only tenants that are ready for operations, erroring on other kinds of tenants. -/// -/// This method is cancel-safe. -pub(crate) fn get_tenant( - tenant_shard_id: TenantShardId, - active_only: bool, -) -> Result, GetTenantError> { - let locked = TENANTS.read().unwrap(); - - let peek_slot = tenant_map_peek_slot(&locked, &tenant_shard_id, TenantSlotPeekMode::Read)?; - - match peek_slot { - Some(TenantSlot::Attached(tenant)) => match tenant.current_state() { - TenantState::Broken { - reason, - backtrace: _, - } if active_only => Err(GetTenantError::Broken(reason)), - TenantState::Active => Ok(Arc::clone(tenant)), - _ => { - if active_only { - Err(GetTenantError::NotActive(tenant_shard_id)) - } else { - Ok(Arc::clone(tenant)) - } - } - }, - Some(TenantSlot::InProgress(_)) => Err(GetTenantError::NotActive(tenant_shard_id)), - None | Some(TenantSlot::Secondary(_)) => { - Err(GetTenantError::NotFound(tenant_shard_id.tenant_id)) - } - } -} - #[derive(thiserror::Error, Debug)] pub(crate) enum GetActiveTenantError { /// We may time out either while TenantSlot is InProgress, or while the Tenant @@ -2062,6 +1987,12 @@ pub(crate) enum GetActiveTenantError { /// Tenant exists, but is in a state that cannot become active (e.g. Stopping, Broken) #[error("will not become active. Current state: {0}")] WillNotBecomeActive(TenantState), + + /// Broken is logically a subset of WillNotBecomeActive, but a distinct error is useful as + /// WillNotBecomeActive is a permitted error under some circumstances, whereas broken should + /// never happen. + #[error("Tenant is broken: {0}")] + Broken(String), } /// Get a [`Tenant`] in its active state. If the tenant_id is currently in [`TenantSlot::InProgress`] @@ -2281,27 +2212,6 @@ pub(crate) enum TenantMapListError { Initializing, } -/// -/// Get list of tenants, for the mgmt API -/// -pub(crate) async fn list_tenants( -) -> Result, TenantMapListError> { - let tenants = TENANTS.read().unwrap(); - let m = match &*tenants { - TenantsMap::Initializing => return Err(TenantMapListError::Initializing), - TenantsMap::Open(m) | TenantsMap::ShuttingDown(m) => m, - }; - Ok(m.iter() - .filter_map(|(id, tenant)| match tenant { - TenantSlot::Attached(tenant) => { - Some((*id, tenant.current_state(), tenant.generation())) - } - TenantSlot::Secondary(_) => None, - TenantSlot::InProgress(_) => None, - }) - .collect()) -} - #[derive(Debug, thiserror::Error)] pub(crate) enum TenantMapInsertError { #[error(transparent)] diff --git a/pageserver/src/tenant/secondary/heatmap_uploader.rs b/pageserver/src/tenant/secondary/heatmap_uploader.rs index a8b05f4c0ea1..39d088ffc3af 100644 --- a/pageserver/src/tenant/secondary/heatmap_uploader.rs +++ b/pageserver/src/tenant/secondary/heatmap_uploader.rs @@ -9,6 +9,7 @@ use crate::{ metrics::SECONDARY_MODE, tenant::{ config::AttachmentMode, + mgr::GetTenantError, mgr::TenantManager, remote_timeline_client::remote_heatmap_path, span::debug_assert_current_span_has_tenant_id, @@ -292,8 +293,11 @@ impl JobGenerator "Starting heatmap write on command"); let tenant = self .tenant_manager - .get_attached_tenant_shard(*tenant_shard_id, true) + .get_attached_tenant_shard(*tenant_shard_id) .map_err(|e| anyhow::anyhow!(e))?; + if !tenant.is_active() { + return Err(GetTenantError::NotActive(*tenant_shard_id).into()); + } Ok(UploadPending { // Ignore our state for last digest: this forces an upload even if nothing has changed diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 38292b6d78e8..bc3fc1df1fd1 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -1274,6 +1274,7 @@ impl Timeline { pub(crate) fn activate( self: &Arc, + parent: Arc, broker_client: BrokerClientChannel, background_jobs_can_start: Option<&completion::Barrier>, ctx: &RequestContext, @@ -1284,7 +1285,7 @@ impl Timeline { } self.launch_wal_receiver(ctx, broker_client); self.set_state(TimelineState::Active); - self.launch_eviction_task(background_jobs_can_start); + self.launch_eviction_task(parent, background_jobs_can_start); } /// Graceful shutdown, may do a lot of I/O as we flush any open layers to disk and then diff --git a/pageserver/src/tenant/timeline/eviction_task.rs b/pageserver/src/tenant/timeline/eviction_task.rs index dd769d41216c..ebcd70bd3986 100644 --- a/pageserver/src/tenant/timeline/eviction_task.rs +++ b/pageserver/src/tenant/timeline/eviction_task.rs @@ -51,6 +51,7 @@ pub struct EvictionTaskTenantState { impl Timeline { pub(super) fn launch_eviction_task( self: &Arc, + parent: Arc, background_tasks_can_start: Option<&completion::Barrier>, ) { let self_clone = Arc::clone(self); @@ -72,14 +73,14 @@ impl Timeline { _ = completion::Barrier::maybe_wait(background_tasks_can_start) => {} }; - self_clone.eviction_task(cancel).await; + self_clone.eviction_task(parent, cancel).await; Ok(()) }, ); } #[instrument(skip_all, fields(tenant_id = %self.tenant_shard_id.tenant_id, shard_id = %self.tenant_shard_id.shard_slug(), timeline_id = %self.timeline_id))] - async fn eviction_task(self: Arc, cancel: CancellationToken) { + async fn eviction_task(self: Arc, tenant: Arc, cancel: CancellationToken) { use crate::tenant::tasks::random_init_delay; // acquire the gate guard only once within a useful span @@ -103,7 +104,7 @@ impl Timeline { loop { let policy = self.get_eviction_policy(); let cf = self - .eviction_iteration(&policy, &cancel, &guard, &ctx) + .eviction_iteration(&tenant, &policy, &cancel, &guard, &ctx) .await; match cf { @@ -123,6 +124,7 @@ impl Timeline { #[instrument(skip_all, fields(policy_kind = policy.discriminant_str()))] async fn eviction_iteration( self: &Arc, + tenant: &Tenant, policy: &EvictionPolicy, cancel: &CancellationToken, gate: &GateGuard, @@ -137,7 +139,7 @@ impl Timeline { } EvictionPolicy::LayerAccessThreshold(p) => { match self - .eviction_iteration_threshold(p, cancel, gate, ctx) + .eviction_iteration_threshold(tenant, p, cancel, gate, ctx) .await { ControlFlow::Break(()) => return ControlFlow::Break(()), @@ -146,7 +148,11 @@ impl Timeline { (p.period, p.threshold) } EvictionPolicy::OnlyImitiate(p) => { - if self.imitiate_only(p, cancel, gate, ctx).await.is_break() { + if self + .imitiate_only(tenant, p, cancel, gate, ctx) + .await + .is_break() + { return ControlFlow::Break(()); } (p.period, p.threshold) @@ -175,6 +181,7 @@ impl Timeline { async fn eviction_iteration_threshold( self: &Arc, + tenant: &Tenant, p: &EvictionPolicyLayerAccessThreshold, cancel: &CancellationToken, gate: &GateGuard, @@ -193,7 +200,10 @@ impl Timeline { _ = self.cancel.cancelled() => return ControlFlow::Break(()), }; - match self.imitate_layer_accesses(p, cancel, gate, ctx).await { + match self + .imitate_layer_accesses(tenant, p, cancel, gate, ctx) + .await + { ControlFlow::Break(()) => return ControlFlow::Break(()), ControlFlow::Continue(()) => (), } @@ -315,6 +325,7 @@ impl Timeline { /// disk usage based eviction task. async fn imitiate_only( self: &Arc, + tenant: &Tenant, p: &EvictionPolicyLayerAccessThreshold, cancel: &CancellationToken, gate: &GateGuard, @@ -331,7 +342,8 @@ impl Timeline { _ = self.cancel.cancelled() => return ControlFlow::Break(()), }; - self.imitate_layer_accesses(p, cancel, gate, ctx).await + self.imitate_layer_accesses(tenant, p, cancel, gate, ctx) + .await } /// If we evict layers but keep cached values derived from those layers, then @@ -361,6 +373,7 @@ impl Timeline { #[instrument(skip_all)] async fn imitate_layer_accesses( &self, + tenant: &Tenant, p: &EvictionPolicyLayerAccessThreshold, cancel: &CancellationToken, gate: &GateGuard, @@ -396,17 +409,11 @@ impl Timeline { // Make one of the tenant's timelines draw the short straw and run the calculation. // The others wait until the calculation is done so that they take into account the // imitated accesses that the winner made. - let tenant = match crate::tenant::mgr::get_tenant(self.tenant_shard_id, true) { - Ok(t) => t, - Err(_) => { - return ControlFlow::Break(()); - } - }; let mut state = tenant.eviction_task_tenant_state.lock().await; match state.last_layer_access_imitation { Some(ts) if ts.elapsed() < inter_imitate_period => { /* no need to run */ } _ => { - self.imitate_synthetic_size_calculation_worker(&tenant, cancel, ctx) + self.imitate_synthetic_size_calculation_worker(tenant, cancel, ctx) .await; state.last_layer_access_imitation = Some(tokio::time::Instant::now()); } @@ -480,7 +487,7 @@ impl Timeline { #[instrument(skip_all)] async fn imitate_synthetic_size_calculation_worker( &self, - tenant: &Arc, + tenant: &Tenant, cancel: &CancellationToken, ctx: &RequestContext, ) { diff --git a/pageserver/src/tenant/timeline/uninit.rs b/pageserver/src/tenant/timeline/uninit.rs index e1034a9fe2f6..2b60e670ea9e 100644 --- a/pageserver/src/tenant/timeline/uninit.rs +++ b/pageserver/src/tenant/timeline/uninit.rs @@ -86,6 +86,7 @@ impl<'t> UninitializedTimeline<'t> { /// Prepares timeline data by loading it from the basebackup archive. pub(crate) async fn import_basebackup_from_tar( self, + tenant: Arc, copyin_read: &mut (impl tokio::io::AsyncRead + Send + Sync + Unpin), base_lsn: Lsn, broker_client: storage_broker::BrokerClientChannel, @@ -114,7 +115,7 @@ impl<'t> UninitializedTimeline<'t> { // All the data has been imported. Insert the Timeline into the tenant's timelines map let tl = self.finish_creation()?; - tl.activate(broker_client, None, ctx); + tl.activate(tenant, broker_client, None, ctx); Ok(tl) } diff --git a/test_runner/regress/test_pageserver_secondary.py b/test_runner/regress/test_pageserver_secondary.py index 25510c50e6bd..ca6f77c75f6c 100644 --- a/test_runner/regress/test_pageserver_secondary.py +++ b/test_runner/regress/test_pageserver_secondary.py @@ -90,6 +90,8 @@ def test_location_conf_churn(neon_env_builder: NeonEnvBuilder, seed: int): # this shutdown case is logged at WARN severity by the time it bubbles up to logical size calculation code # WARN ...: initial size calculation failed: downloading failed, possibly for shutdown ".*downloading failed, possibly for shutdown", + # {tenant_id=... timeline_id=...}:handle_pagerequests:handle_get_page_at_lsn_request{rel=1664/0/1260 blkno=0 req_lsn=0/149F0D8}: error reading relation or page version: Not found: will not become active. Current state: Stopping\n' + ".*page_service.*will not become active.*", ] ) From 12512f31736a5c5b3d3973c5c5cfd43dd58acb3d Mon Sep 17 00:00:00 2001 From: Conrad Ludgate Date: Tue, 26 Mar 2024 19:31:19 +0000 Subject: [PATCH 53/53] add authentication rate limiting (#6865) ## Problem https://github.com/neondatabase/cloud/issues/9642 ## Summary of changes 1. Make `EndpointRateLimiter` generic, renamed as `BucketRateLimiter` 2. Add support for claiming multiple tokens at once 3. Add `AuthRateLimiter` alias. 4. Check `(Endpoint, IP)` pair during authentication, weighted by how many hashes proxy would be doing. TODO: handle ipv6 subnets. will do this in a separate PR. --- libs/metrics/src/hll.rs | 2 +- proxy/src/auth/backend.rs | 90 ++++++++++++++++---- proxy/src/bin/proxy.rs | 13 ++- proxy/src/cache/common.rs | 10 +++ proxy/src/cache/project_info.rs | 30 ++----- proxy/src/config.rs | 8 +- proxy/src/metrics.rs | 22 ++++- proxy/src/proxy.rs | 2 +- proxy/src/proxy/tests.rs | 10 +-- proxy/src/rate_limiter.rs | 2 +- proxy/src/rate_limiter/limiter.rs | 132 +++++++++++++++++++++--------- proxy/src/scram/secret.rs | 12 +-- proxy/src/serverless/backend.rs | 7 +- 13 files changed, 241 insertions(+), 99 deletions(-) diff --git a/libs/metrics/src/hll.rs b/libs/metrics/src/hll.rs index 46a623b0e243..dfb4461ce952 100644 --- a/libs/metrics/src/hll.rs +++ b/libs/metrics/src/hll.rs @@ -40,7 +40,7 @@ macro_rules! register_hll { }}; ($N:literal, $NAME:expr, $HELP:expr $(,)?) => {{ - $crate::register_hll!($N, $crate::opts!($NAME, $HELP), $LABELS_NAMES) + $crate::register_hll!($N, $crate::opts!($NAME, $HELP)) }}; } diff --git a/proxy/src/auth/backend.rs b/proxy/src/auth/backend.rs index 04fe83d8ebc7..e421798067b9 100644 --- a/proxy/src/auth/backend.rs +++ b/proxy/src/auth/backend.rs @@ -12,6 +12,8 @@ use crate::console::errors::GetAuthInfoError; use crate::console::provider::{CachedRoleSecret, ConsoleBackend}; use crate::console::{AuthSecret, NodeInfo}; use crate::context::RequestMonitoring; +use crate::intern::EndpointIdInt; +use crate::metrics::{AUTH_RATE_LIMIT_HITS, ENDPOINTS_AUTH_RATE_LIMITED}; use crate::proxy::connect_compute::ComputeConnectBackend; use crate::proxy::NeonOptions; use crate::stream::Stream; @@ -28,7 +30,7 @@ use crate::{ use crate::{scram, EndpointCacheKey, EndpointId, RoleName}; use std::sync::Arc; use tokio::io::{AsyncRead, AsyncWrite}; -use tracing::info; +use tracing::{info, warn}; /// Alternative to [`std::borrow::Cow`] but doesn't need `T: ToOwned` as we don't need that functionality pub enum MaybeOwned<'a, T> { @@ -174,6 +176,52 @@ impl TryFrom for ComputeUserInfo { } } +impl AuthenticationConfig { + pub fn check_rate_limit( + &self, + + ctx: &mut RequestMonitoring, + secret: AuthSecret, + endpoint: &EndpointId, + is_cleartext: bool, + ) -> auth::Result { + // we have validated the endpoint exists, so let's intern it. + let endpoint_int = EndpointIdInt::from(endpoint); + + // only count the full hash count if password hack or websocket flow. + // in other words, if proxy needs to run the hashing + let password_weight = if is_cleartext { + match &secret { + #[cfg(any(test, feature = "testing"))] + AuthSecret::Md5(_) => 1, + AuthSecret::Scram(s) => s.iterations + 1, + } + } else { + // validating scram takes just 1 hmac_sha_256 operation. + 1 + }; + + let limit_not_exceeded = self + .rate_limiter + .check((endpoint_int, ctx.peer_addr), password_weight); + + if !limit_not_exceeded { + warn!( + enabled = self.rate_limiter_enabled, + "rate limiting authentication" + ); + AUTH_RATE_LIMIT_HITS.inc(); + ENDPOINTS_AUTH_RATE_LIMITED.measure(endpoint); + + if self.rate_limiter_enabled { + return Err(auth::AuthError::too_many_connections()); + } + } + + Ok(secret) + } +} + /// True to its name, this function encapsulates our current auth trade-offs. /// Here, we choose the appropriate auth flow based on circumstances. /// @@ -214,14 +262,24 @@ async fn auth_quirks( Some(secret) => secret, None => api.get_role_secret(ctx, &info).await?, }; + let (cached_entry, secret) = cached_secret.take_value(); + + let secret = match secret { + Some(secret) => config.check_rate_limit( + ctx, + secret, + &info.endpoint, + unauthenticated_password.is_some() || allow_cleartext, + )?, + None => { + // If we don't have an authentication secret, we mock one to + // prevent malicious probing (possible due to missing protocol steps). + // This mocked secret will never lead to successful authentication. + info!("authentication info not found, mocking it"); + AuthSecret::Scram(scram::ServerSecret::mock(rand::random())) + } + }; - let secret = cached_secret.value.clone().unwrap_or_else(|| { - // If we don't have an authentication secret, we mock one to - // prevent malicious probing (possible due to missing protocol steps). - // This mocked secret will never lead to successful authentication. - info!("authentication info not found, mocking it"); - AuthSecret::Scram(scram::ServerSecret::mock(&info.user, rand::random())) - }); match authenticate_with_secret( ctx, secret, @@ -237,7 +295,7 @@ async fn auth_quirks( Err(e) => { if e.is_auth_failed() { // The password could have been changed, so we invalidate the cache. - cached_secret.invalidate(); + cached_entry.invalidate(); } Err(e) } @@ -415,6 +473,7 @@ mod tests { use bytes::BytesMut; use fallible_iterator::FallibleIterator; + use once_cell::sync::Lazy; use postgres_protocol::{ authentication::sasl::{ChannelBinding, ScramSha256}, message::{backend::Message as PgMessage, frontend}, @@ -432,6 +491,7 @@ mod tests { }, context::RequestMonitoring, proxy::NeonOptions, + rate_limiter::{AuthRateLimiter, RateBucketInfo}, scram::ServerSecret, stream::{PqStream, Stream}, }; @@ -473,9 +533,11 @@ mod tests { } } - static CONFIG: &AuthenticationConfig = &AuthenticationConfig { + static CONFIG: Lazy = Lazy::new(|| AuthenticationConfig { scram_protocol_timeout: std::time::Duration::from_secs(5), - }; + rate_limiter_enabled: true, + rate_limiter: AuthRateLimiter::new(&RateBucketInfo::DEFAULT_AUTH_SET), + }); async fn read_message(r: &mut (impl AsyncRead + Unpin), b: &mut BytesMut) -> PgMessage { loop { @@ -544,7 +606,7 @@ mod tests { } }); - let _creds = auth_quirks(&mut ctx, &api, user_info, &mut stream, false, CONFIG) + let _creds = auth_quirks(&mut ctx, &api, user_info, &mut stream, false, &CONFIG) .await .unwrap(); @@ -584,7 +646,7 @@ mod tests { client.write_all(&write).await.unwrap(); }); - let _creds = auth_quirks(&mut ctx, &api, user_info, &mut stream, true, CONFIG) + let _creds = auth_quirks(&mut ctx, &api, user_info, &mut stream, true, &CONFIG) .await .unwrap(); @@ -624,7 +686,7 @@ mod tests { client.write_all(&write).await.unwrap(); }); - let creds = auth_quirks(&mut ctx, &api, user_info, &mut stream, true, CONFIG) + let creds = auth_quirks(&mut ctx, &api, user_info, &mut stream, true, &CONFIG) .await .unwrap(); diff --git a/proxy/src/bin/proxy.rs b/proxy/src/bin/proxy.rs index d38439c2a0d6..88b847f5f106 100644 --- a/proxy/src/bin/proxy.rs +++ b/proxy/src/bin/proxy.rs @@ -18,6 +18,7 @@ use proxy::console; use proxy::context::parquet::ParquetUploadArgs; use proxy::http; use proxy::metrics::NUM_CANCELLATION_REQUESTS_SOURCE_FROM_CLIENT; +use proxy::rate_limiter::AuthRateLimiter; use proxy::rate_limiter::EndpointRateLimiter; use proxy::rate_limiter::RateBucketInfo; use proxy::rate_limiter::RateLimiterConfig; @@ -141,10 +142,16 @@ struct ProxyCliArgs { /// /// Provided in the form '@'. /// Can be given multiple times for different bucket sizes. - #[clap(long, default_values_t = RateBucketInfo::DEFAULT_SET)] + #[clap(long, default_values_t = RateBucketInfo::DEFAULT_ENDPOINT_SET)] endpoint_rps_limit: Vec, + /// Whether the auth rate limiter actually takes effect (for testing) + #[clap(long, default_value_t = false, value_parser = clap::builder::BoolishValueParser::new(), action = clap::ArgAction::Set)] + auth_rate_limit_enabled: bool, + /// Authentication rate limiter max number of hashes per second. + #[clap(long, default_values_t = RateBucketInfo::DEFAULT_AUTH_SET)] + auth_rate_limit: Vec, /// Redis rate limiter max number of requests per second. - #[clap(long, default_values_t = RateBucketInfo::DEFAULT_SET)] + #[clap(long, default_values_t = RateBucketInfo::DEFAULT_ENDPOINT_SET)] redis_rps_limit: Vec, /// Initial limit for dynamic rate limiter. Makes sense only if `rate_limit_algorithm` is *not* `None`. #[clap(long, default_value_t = 100)] @@ -510,6 +517,8 @@ fn build_config(args: &ProxyCliArgs) -> anyhow::Result<&'static ProxyConfig> { }; let authentication_config = AuthenticationConfig { scram_protocol_timeout: args.scram_protocol_timeout, + rate_limiter_enabled: args.auth_rate_limit_enabled, + rate_limiter: AuthRateLimiter::new(args.auth_rate_limit.clone()), }; let mut endpoint_rps_limit = args.endpoint_rps_limit.clone(); diff --git a/proxy/src/cache/common.rs b/proxy/src/cache/common.rs index 2af6a70e9072..bc1c37512bce 100644 --- a/proxy/src/cache/common.rs +++ b/proxy/src/cache/common.rs @@ -43,6 +43,16 @@ impl Cached { Self { token: None, value } } + pub fn take_value(self) -> (Cached, V) { + ( + Cached { + token: self.token, + value: (), + }, + self.value, + ) + } + /// Drop this entry from a cache if it's still there. pub fn invalidate(self) -> V { if let Some((cache, info)) = &self.token { diff --git a/proxy/src/cache/project_info.rs b/proxy/src/cache/project_info.rs index 6e3eb8c1b028..5a3660520bf1 100644 --- a/proxy/src/cache/project_info.rs +++ b/proxy/src/cache/project_info.rs @@ -373,10 +373,7 @@ mod tests { let endpoint_id = "endpoint".into(); let user1: RoleName = "user1".into(); let user2: RoleName = "user2".into(); - let secret1 = Some(AuthSecret::Scram(ServerSecret::mock( - user1.as_str(), - [1; 32], - ))); + let secret1 = Some(AuthSecret::Scram(ServerSecret::mock([1; 32]))); let secret2 = None; let allowed_ips = Arc::new(vec![ "127.0.0.1".parse().unwrap(), @@ -395,10 +392,7 @@ mod tests { // Shouldn't add more than 2 roles. let user3: RoleName = "user3".into(); - let secret3 = Some(AuthSecret::Scram(ServerSecret::mock( - user3.as_str(), - [3; 32], - ))); + let secret3 = Some(AuthSecret::Scram(ServerSecret::mock([3; 32]))); cache.insert_role_secret(&project_id, &endpoint_id, &user3, secret3.clone()); assert!(cache.get_role_secret(&endpoint_id, &user3).is_none()); @@ -431,14 +425,8 @@ mod tests { let endpoint_id = "endpoint".into(); let user1: RoleName = "user1".into(); let user2: RoleName = "user2".into(); - let secret1 = Some(AuthSecret::Scram(ServerSecret::mock( - user1.as_str(), - [1; 32], - ))); - let secret2 = Some(AuthSecret::Scram(ServerSecret::mock( - user2.as_str(), - [2; 32], - ))); + let secret1 = Some(AuthSecret::Scram(ServerSecret::mock([1; 32]))); + let secret2 = Some(AuthSecret::Scram(ServerSecret::mock([2; 32]))); let allowed_ips = Arc::new(vec![ "127.0.0.1".parse().unwrap(), "127.0.0.2".parse().unwrap(), @@ -486,14 +474,8 @@ mod tests { let endpoint_id = "endpoint".into(); let user1: RoleName = "user1".into(); let user2: RoleName = "user2".into(); - let secret1 = Some(AuthSecret::Scram(ServerSecret::mock( - user1.as_str(), - [1; 32], - ))); - let secret2 = Some(AuthSecret::Scram(ServerSecret::mock( - user2.as_str(), - [2; 32], - ))); + let secret1 = Some(AuthSecret::Scram(ServerSecret::mock([1; 32]))); + let secret2 = Some(AuthSecret::Scram(ServerSecret::mock([2; 32]))); let allowed_ips = Arc::new(vec![ "127.0.0.1".parse().unwrap(), "127.0.0.2".parse().unwrap(), diff --git a/proxy/src/config.rs b/proxy/src/config.rs index 45f8d7614439..361c3ef519c4 100644 --- a/proxy/src/config.rs +++ b/proxy/src/config.rs @@ -1,4 +1,8 @@ -use crate::{auth, rate_limiter::RateBucketInfo, serverless::GlobalConnPoolOptions}; +use crate::{ + auth, + rate_limiter::{AuthRateLimiter, RateBucketInfo}, + serverless::GlobalConnPoolOptions, +}; use anyhow::{bail, ensure, Context, Ok}; use itertools::Itertools; use rustls::{ @@ -50,6 +54,8 @@ pub struct HttpConfig { pub struct AuthenticationConfig { pub scram_protocol_timeout: tokio::time::Duration, + pub rate_limiter_enabled: bool, + pub rate_limiter: AuthRateLimiter, } impl TlsConfig { diff --git a/proxy/src/metrics.rs b/proxy/src/metrics.rs index eed45e421b8f..4172dc19daf9 100644 --- a/proxy/src/metrics.rs +++ b/proxy/src/metrics.rs @@ -4,7 +4,10 @@ use ::metrics::{ register_int_gauge_vec, Histogram, HistogramVec, HyperLogLogVec, IntCounterPairVec, IntCounterVec, IntGauge, IntGaugeVec, }; -use metrics::{register_int_counter, register_int_counter_pair, IntCounter, IntCounterPair}; +use metrics::{ + register_hll, register_int_counter, register_int_counter_pair, HyperLogLog, IntCounter, + IntCounterPair, +}; use once_cell::sync::Lazy; use tokio::time::{self, Instant}; @@ -358,3 +361,20 @@ pub static TLS_HANDSHAKE_FAILURES: Lazy = Lazy::new(|| { ) .unwrap() }); + +pub static ENDPOINTS_AUTH_RATE_LIMITED: Lazy> = Lazy::new(|| { + register_hll!( + 32, + "proxy_endpoints_auth_rate_limits", + "Number of endpoints affected by authentication rate limits", + ) + .unwrap() +}); + +pub static AUTH_RATE_LIMIT_HITS: Lazy = Lazy::new(|| { + register_int_counter!( + "proxy_requests_auth_rate_limits_total", + "Number of connection requests affected by authentication rate limits", + ) + .unwrap() +}); diff --git a/proxy/src/proxy.rs b/proxy/src/proxy.rs index 843bfc08cfa3..6051c0a81242 100644 --- a/proxy/src/proxy.rs +++ b/proxy/src/proxy.rs @@ -280,7 +280,7 @@ pub async fn handle_client( // check rate limit if let Some(ep) = user_info.get_endpoint() { - if !endpoint_rate_limiter.check(ep) { + if !endpoint_rate_limiter.check(ep, 1) { return stream .throw_error(auth::AuthError::too_many_connections()) .await?; diff --git a/proxy/src/proxy/tests.rs b/proxy/src/proxy/tests.rs index 9c3be7361291..a4051447c1b0 100644 --- a/proxy/src/proxy/tests.rs +++ b/proxy/src/proxy/tests.rs @@ -142,8 +142,8 @@ impl Scram { Ok(Scram(secret)) } - fn mock(user: &str) -> Self { - Scram(scram::ServerSecret::mock(user, rand::random())) + fn mock() -> Self { + Scram(scram::ServerSecret::mock(rand::random())) } } @@ -330,11 +330,7 @@ async fn scram_auth_mock() -> anyhow::Result<()> { let (client_config, server_config) = generate_tls_config("generic-project-name.localhost", "localhost")?; - let proxy = tokio::spawn(dummy_proxy( - client, - Some(server_config), - Scram::mock("user"), - )); + let proxy = tokio::spawn(dummy_proxy(client, Some(server_config), Scram::mock())); use rand::{distributions::Alphanumeric, Rng}; let password: String = rand::thread_rng() diff --git a/proxy/src/rate_limiter.rs b/proxy/src/rate_limiter.rs index f0da4ead230c..13dffffca01e 100644 --- a/proxy/src/rate_limiter.rs +++ b/proxy/src/rate_limiter.rs @@ -4,4 +4,4 @@ mod limiter; pub use aimd::Aimd; pub use limit_algorithm::{AimdConfig, Fixed, RateLimitAlgorithm, RateLimiterConfig}; pub use limiter::Limiter; -pub use limiter::{EndpointRateLimiter, RateBucketInfo, RedisRateLimiter}; +pub use limiter::{AuthRateLimiter, EndpointRateLimiter, RateBucketInfo, RedisRateLimiter}; diff --git a/proxy/src/rate_limiter/limiter.rs b/proxy/src/rate_limiter/limiter.rs index 3181060e2f93..f590896dd9f4 100644 --- a/proxy/src/rate_limiter/limiter.rs +++ b/proxy/src/rate_limiter/limiter.rs @@ -1,6 +1,8 @@ use std::{ + borrow::Cow, collections::hash_map::RandomState, - hash::BuildHasher, + hash::{BuildHasher, Hash}, + net::IpAddr, sync::{ atomic::{AtomicUsize, Ordering}, Arc, Mutex, @@ -15,7 +17,7 @@ use tokio::sync::{Mutex as AsyncMutex, Semaphore, SemaphorePermit}; use tokio::time::{timeout, Duration, Instant}; use tracing::info; -use crate::EndpointId; +use crate::{intern::EndpointIdInt, EndpointId}; use super::{ limit_algorithm::{LimitAlgorithm, Sample}, @@ -49,11 +51,11 @@ impl RedisRateLimiter { .data .iter_mut() .zip(self.info) - .all(|(bucket, info)| bucket.should_allow_request(info, now)); + .all(|(bucket, info)| bucket.should_allow_request(info, now, 1)); if should_allow_request { // only increment the bucket counts if the request will actually be accepted - self.data.iter_mut().for_each(RateBucket::inc); + self.data.iter_mut().for_each(|b| b.inc(1)); } should_allow_request @@ -71,9 +73,14 @@ impl RedisRateLimiter { // saw SNI, before doing TLS handshake. User-side error messages in that case // does not look very nice (`SSL SYSCALL error: Undefined error: 0`), so for now // I went with a more expensive way that yields user-friendlier error messages. -pub struct EndpointRateLimiter { - map: DashMap, Hasher>, - info: &'static [RateBucketInfo], +pub type EndpointRateLimiter = BucketRateLimiter; + +// This can't be just per IP because that would limit some PaaS that share IP addresses +pub type AuthRateLimiter = BucketRateLimiter<(EndpointIdInt, IpAddr), StdRng, RandomState>; + +pub struct BucketRateLimiter { + map: DashMap, Hasher>, + info: Cow<'static, [RateBucketInfo]>, access_count: AtomicUsize, rand: Mutex, } @@ -85,9 +92,9 @@ struct RateBucket { } impl RateBucket { - fn should_allow_request(&mut self, info: &RateBucketInfo, now: Instant) -> bool { + fn should_allow_request(&mut self, info: &RateBucketInfo, now: Instant, n: u32) -> bool { if now - self.start < info.interval { - self.count < info.max_rpi + self.count + n <= info.max_rpi } else { // bucket expired, reset self.count = 0; @@ -97,8 +104,8 @@ impl RateBucket { } } - fn inc(&mut self) { - self.count += 1; + fn inc(&mut self, n: u32) { + self.count += n; } } @@ -111,7 +118,7 @@ pub struct RateBucketInfo { impl std::fmt::Display for RateBucketInfo { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - let rps = self.max_rpi * 1000 / self.interval.as_millis() as u32; + let rps = (self.max_rpi as u64) * 1000 / self.interval.as_millis() as u64; write!(f, "{rps}@{}", humantime::format_duration(self.interval)) } } @@ -136,12 +143,25 @@ impl std::str::FromStr for RateBucketInfo { } impl RateBucketInfo { - pub const DEFAULT_SET: [Self; 3] = [ + pub const DEFAULT_ENDPOINT_SET: [Self; 3] = [ Self::new(300, Duration::from_secs(1)), Self::new(200, Duration::from_secs(60)), Self::new(100, Duration::from_secs(600)), ]; + /// All of these are per endpoint-ip pair. + /// Context: 4096 rounds of pbkdf2 take about 1ms of cpu time to execute (1 milli-cpu-second or 1mcpus). + /// + /// First bucket: 300mcpus total per endpoint-ip pair + /// * 1228800 requests per second with 1 hash rounds. (endpoint rate limiter will catch this first) + /// * 300 requests per second with 4096 hash rounds. + /// * 2 requests per second with 600000 hash rounds. + pub const DEFAULT_AUTH_SET: [Self; 3] = [ + Self::new(300 * 4096, Duration::from_secs(1)), + Self::new(200 * 4096, Duration::from_secs(60)), + Self::new(100 * 4096, Duration::from_secs(600)), + ]; + pub fn validate(info: &mut [Self]) -> anyhow::Result<()> { info.sort_unstable_by_key(|info| info.interval); let invalid = info @@ -150,7 +170,7 @@ impl RateBucketInfo { .find(|(a, b)| a.max_rpi > b.max_rpi); if let Some((a, b)) = invalid { bail!( - "invalid endpoint RPS limits. {b} allows fewer requests per bucket than {a} ({} vs {})", + "invalid bucket RPS limits. {b} allows fewer requests per bucket than {a} ({} vs {})", b.max_rpi, a.max_rpi, ); @@ -162,19 +182,24 @@ impl RateBucketInfo { pub const fn new(max_rps: u32, interval: Duration) -> Self { Self { interval, - max_rpi: max_rps * interval.as_millis() as u32 / 1000, + max_rpi: ((max_rps as u64) * (interval.as_millis() as u64) / 1000) as u32, } } } -impl EndpointRateLimiter { - pub fn new(info: &'static [RateBucketInfo]) -> Self { +impl BucketRateLimiter { + pub fn new(info: impl Into>) -> Self { Self::new_with_rand_and_hasher(info, StdRng::from_entropy(), RandomState::new()) } } -impl EndpointRateLimiter { - fn new_with_rand_and_hasher(info: &'static [RateBucketInfo], rand: R, hasher: S) -> Self { +impl BucketRateLimiter { + fn new_with_rand_and_hasher( + info: impl Into>, + rand: R, + hasher: S, + ) -> Self { + let info = info.into(); info!(buckets = ?info, "endpoint rate limiter"); Self { info, @@ -185,7 +210,7 @@ impl EndpointRateLimiter { } /// Check that number of connections to the endpoint is below `max_rps` rps. - pub fn check(&self, endpoint: EndpointId) -> bool { + pub fn check(&self, key: K, n: u32) -> bool { // do a partial GC every 2k requests. This cleans up ~ 1/64th of the map. // worst case memory usage is about: // = 2 * 2048 * 64 * (48B + 72B) @@ -195,7 +220,7 @@ impl EndpointRateLimiter { } let now = Instant::now(); - let mut entry = self.map.entry(endpoint).or_insert_with(|| { + let mut entry = self.map.entry(key).or_insert_with(|| { vec![ RateBucket { start: now, @@ -207,12 +232,12 @@ impl EndpointRateLimiter { let should_allow_request = entry .iter_mut() - .zip(self.info) - .all(|(bucket, info)| bucket.should_allow_request(info, now)); + .zip(&*self.info) + .all(|(bucket, info)| bucket.should_allow_request(info, now, n)); if should_allow_request { // only increment the bucket counts if the request will actually be accepted - entry.iter_mut().for_each(RateBucket::inc); + entry.iter_mut().for_each(|b| b.inc(n)); } should_allow_request @@ -223,7 +248,7 @@ impl EndpointRateLimiter { /// But that way deletion does not aquire mutex on each entry access. pub fn do_gc(&self) { info!( - "cleaning up endpoint rate limiter, current size = {}", + "cleaning up bucket rate limiter, current size = {}", self.map.len() ); let n = self.map.shards().len(); @@ -534,7 +559,7 @@ mod tests { use rustc_hash::FxHasher; use tokio::time; - use super::{EndpointRateLimiter, Limiter, Outcome}; + use super::{BucketRateLimiter, EndpointRateLimiter, Limiter, Outcome}; use crate::{ rate_limiter::{RateBucketInfo, RateLimitAlgorithm}, EndpointId, @@ -672,12 +697,12 @@ mod tests { #[test] fn default_rate_buckets() { - let mut defaults = RateBucketInfo::DEFAULT_SET; + let mut defaults = RateBucketInfo::DEFAULT_ENDPOINT_SET; RateBucketInfo::validate(&mut defaults[..]).unwrap(); } #[test] - #[should_panic = "invalid endpoint RPS limits. 10@10s allows fewer requests per bucket than 300@1s (100 vs 300)"] + #[should_panic = "invalid bucket RPS limits. 10@10s allows fewer requests per bucket than 300@1s (100 vs 300)"] fn rate_buckets_validate() { let mut rates: Vec = ["300@1s", "10@10s"] .into_iter() @@ -693,42 +718,42 @@ mod tests { .map(|s| s.parse().unwrap()) .collect(); RateBucketInfo::validate(&mut rates).unwrap(); - let limiter = EndpointRateLimiter::new(Vec::leak(rates)); + let limiter = EndpointRateLimiter::new(rates); let endpoint = EndpointId::from("ep-my-endpoint-1234"); time::pause(); for _ in 0..100 { - assert!(limiter.check(endpoint.clone())); + assert!(limiter.check(endpoint.clone(), 1)); } // more connections fail - assert!(!limiter.check(endpoint.clone())); + assert!(!limiter.check(endpoint.clone(), 1)); // fail even after 500ms as it's in the same bucket time::advance(time::Duration::from_millis(500)).await; - assert!(!limiter.check(endpoint.clone())); + assert!(!limiter.check(endpoint.clone(), 1)); // after a full 1s, 100 requests are allowed again time::advance(time::Duration::from_millis(500)).await; for _ in 1..6 { - for _ in 0..100 { - assert!(limiter.check(endpoint.clone())); + for _ in 0..50 { + assert!(limiter.check(endpoint.clone(), 2)); } time::advance(time::Duration::from_millis(1000)).await; } // more connections after 600 will exceed the 20rps@30s limit - assert!(!limiter.check(endpoint.clone())); + assert!(!limiter.check(endpoint.clone(), 1)); // will still fail before the 30 second limit time::advance(time::Duration::from_millis(30_000 - 6_000 - 1)).await; - assert!(!limiter.check(endpoint.clone())); + assert!(!limiter.check(endpoint.clone(), 1)); // after the full 30 seconds, 100 requests are allowed again time::advance(time::Duration::from_millis(1)).await; for _ in 0..100 { - assert!(limiter.check(endpoint.clone())); + assert!(limiter.check(endpoint.clone(), 1)); } } @@ -738,14 +763,41 @@ mod tests { let rand = rand::rngs::StdRng::from_seed([1; 32]); let hasher = BuildHasherDefault::::default(); - let limiter = EndpointRateLimiter::new_with_rand_and_hasher( - &RateBucketInfo::DEFAULT_SET, + let limiter = BucketRateLimiter::new_with_rand_and_hasher( + &RateBucketInfo::DEFAULT_ENDPOINT_SET, rand, hasher, ); for i in 0..1_000_000 { - limiter.check(format!("{i}").into()); + limiter.check(i, 1); } assert!(limiter.map.len() < 150_000); } + + #[test] + fn test_default_auth_set() { + // these values used to exceed u32::MAX + assert_eq!( + RateBucketInfo::DEFAULT_AUTH_SET, + [ + RateBucketInfo { + interval: Duration::from_secs(1), + max_rpi: 300 * 4096, + }, + RateBucketInfo { + interval: Duration::from_secs(60), + max_rpi: 200 * 4096 * 60, + }, + RateBucketInfo { + interval: Duration::from_secs(600), + max_rpi: 100 * 4096 * 600, + } + ] + ); + + for x in RateBucketInfo::DEFAULT_AUTH_SET { + let y = x.to_string().parse().unwrap(); + assert_eq!(x, y); + } + } } diff --git a/proxy/src/scram/secret.rs b/proxy/src/scram/secret.rs index f3414cb8ecc8..44c4f9e44aec 100644 --- a/proxy/src/scram/secret.rs +++ b/proxy/src/scram/secret.rs @@ -50,13 +50,13 @@ impl ServerSecret { /// To avoid revealing information to an attacker, we use a /// mocked server secret even if the user doesn't exist. /// See `auth-scram.c : mock_scram_secret` for details. - pub fn mock(user: &str, nonce: [u8; 32]) -> Self { - // Refer to `auth-scram.c : scram_mock_salt`. - let mocked_salt = super::sha256([user.as_bytes(), &nonce]); - + pub fn mock(nonce: [u8; 32]) -> Self { Self { - iterations: 4096, - salt_base64: base64::encode(mocked_salt), + // this doesn't reveal much information as we're going to use + // iteration count 1 for our generated passwords going forward. + // PG16 users can set iteration count=1 already today. + iterations: 1, + salt_base64: base64::encode(nonce), stored_key: ScramKey::default(), server_key: ScramKey::default(), doomed: true, diff --git a/proxy/src/serverless/backend.rs b/proxy/src/serverless/backend.rs index 72b55c45f013..f10779d7ba0d 100644 --- a/proxy/src/serverless/backend.rs +++ b/proxy/src/serverless/backend.rs @@ -42,7 +42,12 @@ impl PoolingBackend { }; let secret = match cached_secret.value.clone() { - Some(secret) => secret, + Some(secret) => self.config.authentication_config.check_rate_limit( + ctx, + secret, + &user_info.endpoint, + true, + )?, None => { // If we don't have an authentication secret, for the http flow we can just return an error. info!("authentication info not found");