From d3c583efbe2a5f736ae43da4de84479ec4ee81b4 Mon Sep 17 00:00:00 2001 From: John Spray Date: Thu, 7 Mar 2024 14:06:48 +0000 Subject: [PATCH 01/43] Rename binary attachment_service -> storage_controller (#7042) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## Problem The storage controller binary still has its historic `attachment_service` name -- it will be painful to change this later because we can't atomically update this repo and the helm charts used to deploy. Companion helm chart change: https://github.com/neondatabase/helm-charts/pull/70 ## Summary of changes - Change the name of the binary to `storage_controller` - Skipping renaming things in the source right now: this is just to get rid of the legacy name in external interfaces. --------- Co-authored-by: Arpad MΓΌller --- Dockerfile | 4 ++-- control_plane/attachment_service/Cargo.toml | 4 ++++ control_plane/src/attachment_service.rs | 2 +- control_plane/src/local_env.rs | 2 +- 4 files changed, 8 insertions(+), 4 deletions(-) diff --git a/Dockerfile b/Dockerfile index 47954a671b9c..5f82df3e1811 100644 --- a/Dockerfile +++ b/Dockerfile @@ -53,7 +53,7 @@ RUN set -e \ --bin pagectl \ --bin safekeeper \ --bin storage_broker \ - --bin attachment_service \ + --bin storage_controller \ --bin proxy \ --bin neon_local \ --locked --release \ @@ -81,7 +81,7 @@ COPY --from=build --chown=neon:neon /home/nonroot/target/release/pageserver COPY --from=build --chown=neon:neon /home/nonroot/target/release/pagectl /usr/local/bin COPY --from=build --chown=neon:neon /home/nonroot/target/release/safekeeper /usr/local/bin COPY --from=build --chown=neon:neon /home/nonroot/target/release/storage_broker /usr/local/bin -COPY --from=build --chown=neon:neon /home/nonroot/target/release/attachment_service /usr/local/bin +COPY --from=build --chown=neon:neon /home/nonroot/target/release/storage_controller /usr/local/bin COPY --from=build --chown=neon:neon /home/nonroot/target/release/proxy /usr/local/bin COPY --from=build --chown=neon:neon /home/nonroot/target/release/neon_local /usr/local/bin diff --git a/control_plane/attachment_service/Cargo.toml b/control_plane/attachment_service/Cargo.toml index bfdfd4c77d38..a5fad7216ca1 100644 --- a/control_plane/attachment_service/Cargo.toml +++ b/control_plane/attachment_service/Cargo.toml @@ -4,6 +4,10 @@ version = "0.1.0" edition.workspace = true license.workspace = true +[[bin]] +name = "storage_controller" +path = "src/main.rs" + [features] default = [] # Enables test-only APIs and behaviors diff --git a/control_plane/src/attachment_service.rs b/control_plane/src/attachment_service.rs index 610d7386d945..5c975619855c 100644 --- a/control_plane/src/attachment_service.rs +++ b/control_plane/src/attachment_service.rs @@ -34,7 +34,7 @@ pub struct AttachmentService { client: reqwest::Client, } -const COMMAND: &str = "attachment_service"; +const COMMAND: &str = "storage_controller"; const ATTACHMENT_SERVICE_POSTGRES_VERSION: u32 = 16; diff --git a/control_plane/src/local_env.rs b/control_plane/src/local_env.rs index a5e1325cfe02..03270723a62c 100644 --- a/control_plane/src/local_env.rs +++ b/control_plane/src/local_env.rs @@ -232,7 +232,7 @@ impl LocalEnv { // run from the same location as neon_local. This means that for compatibility // tests that run old pageserver/safekeeper, they still run latest attachment service. let neon_local_bin_dir = env::current_exe().unwrap().parent().unwrap().to_owned(); - neon_local_bin_dir.join("attachment_service") + neon_local_bin_dir.join("storage_controller") } pub fn safekeeper_bin(&self) -> PathBuf { From 602a4da9a5cdfac7f04509950704da811f08b968 Mon Sep 17 00:00:00 2001 From: Joonas Koivunen Date: Thu, 7 Mar 2024 16:23:42 +0200 Subject: [PATCH 02/43] bench: run branch_creation_many at 500, seeded (#6959) We have a benchmark for creating a lot of branches, but it does random things, and the branch count is not what we is the largest maximum we aim to support. If this PR would stabilize the benchmark total duration it means that there are some structures which are very much slower than others. Then we should add a seed-outputting variant to help find and reproduce such cases. Additionally, record for the benchmark: - shutdown duration - startup metrics once done (on restart) - duration of first compaction completion via debug logging --- pageserver/src/tenant/tasks.rs | 7 +- .../performance/test_branch_creation.py | 110 ++++++++++++++++-- 2 files changed, 109 insertions(+), 8 deletions(-) diff --git a/pageserver/src/tenant/tasks.rs b/pageserver/src/tenant/tasks.rs index 57c3edcddd29..e4f5f7513288 100644 --- a/pageserver/src/tenant/tasks.rs +++ b/pageserver/src/tenant/tasks.rs @@ -101,6 +101,7 @@ pub fn start_background_loops( _ = completion::Barrier::maybe_wait(background_jobs_can_start) => {} }; compaction_loop(tenant, cancel) + // If you rename this span, change the RUST_LOG env variable in test_runner/performance/test_branch_creation.py .instrument(info_span!("compaction_loop", tenant_id = %tenant_shard_id.tenant_id, shard_id = %tenant_shard_id.shard_slug())) .await; Ok(()) @@ -198,7 +199,11 @@ async fn compaction_loop(tenant: Arc, cancel: CancellationToken) { } }; - warn_when_period_overrun(started_at.elapsed(), period, BackgroundLoopKind::Compaction); + let elapsed = started_at.elapsed(); + warn_when_period_overrun(elapsed, period, BackgroundLoopKind::Compaction); + + // the duration is recorded by performance tests by enabling debug in this function + tracing::debug!(elapsed_ms=elapsed.as_millis(), "compaction iteration complete"); // Perhaps we did no work and the walredo process has been idle for some time: // give it a chance to shut down to avoid leaving walredo process running indefinitely. diff --git a/test_runner/performance/test_branch_creation.py b/test_runner/performance/test_branch_creation.py index 6edcb8f1f2fa..9777bf67489c 100644 --- a/test_runner/performance/test_branch_creation.py +++ b/test_runner/performance/test_branch_creation.py @@ -1,4 +1,5 @@ import random +import re import statistics import threading import time @@ -7,11 +8,14 @@ from typing import List import pytest -from fixtures.benchmark_fixture import MetricReport +from fixtures.benchmark_fixture import MetricReport, NeonBenchmarker from fixtures.compare_fixtures import NeonCompare from fixtures.log_helper import log +from fixtures.neon_fixtures import NeonPageserver from fixtures.pageserver.utils import wait_for_last_record_lsn from fixtures.types import Lsn +from fixtures.utils import wait_until +from prometheus_client.samples import Sample def _record_branch_creation_durations(neon_compare: NeonCompare, durs: List[float]): @@ -89,11 +93,17 @@ def run_pgbench(branch: str): _record_branch_creation_durations(neon_compare, branch_creation_durations) -@pytest.mark.parametrize("n_branches", [1024]) -# Test measures the latency of branch creation when creating a lot of branches. -def test_branch_creation_many(neon_compare: NeonCompare, n_branches: int): +@pytest.mark.parametrize("n_branches", [500, 1024]) +@pytest.mark.parametrize("shape", ["one_ancestor", "random"]) +def test_branch_creation_many(neon_compare: NeonCompare, n_branches: int, shape: str): + """ + Test measures the latency of branch creation when creating a lot of branches. + """ env = neon_compare.env + # seed the prng so we will measure the same structure every time + rng = random.Random("2024-02-29") + env.neon_cli.create_branch("b0") endpoint = env.endpoints.create_start("b0") @@ -102,15 +112,101 @@ def test_branch_creation_many(neon_compare: NeonCompare, n_branches: int): branch_creation_durations = [] for i in range(n_branches): - # random a source branch - p = random.randint(0, i) + if shape == "random": + parent = f"b{rng.randint(0, i)}" + elif shape == "one_ancestor": + parent = "b0" + else: + raise RuntimeError(f"unimplemented shape: {shape}") + timer = timeit.default_timer() - env.neon_cli.create_branch("b{}".format(i + 1), "b{}".format(p)) + # each of these uploads to remote storage before completion + env.neon_cli.create_branch(f"b{i + 1}", parent) dur = timeit.default_timer() - timer branch_creation_durations.append(dur) _record_branch_creation_durations(neon_compare, branch_creation_durations) + endpoint.stop_and_destroy() + + with neon_compare.record_duration("shutdown"): + # this sleeps 100ms between polls + env.pageserver.stop() + + startup_line = "INFO version: git(-env)?:" + + # find the first line of the log file so we can find the next start later + _, first_start = wait_until(5, 1, lambda: env.pageserver.assert_log_contains(startup_line)) + + # start without gc so we can time compaction with less noise; use shorter + # period for compaction so it starts earlier + env.pageserver.start( + overrides=( + "--pageserver-config-override=tenant_config={ compaction_period = '3s', gc_period = '0s' }", + ), + # this does print more than we want, but the number should be comparable between runs + extra_env_vars={ + "RUST_LOG": f"[compaction_loop{{tenant_id={env.initial_tenant}}}]=debug,info" + }, + ) + + _, second_start = wait_until( + 5, 1, lambda: env.pageserver.assert_log_contains(startup_line, first_start) + ) + env.pageserver.quiesce_tenants() + + wait_and_record_startup_metrics(env.pageserver, neon_compare.zenbenchmark, "restart_after") + + # wait for compaction to complete, which most likely has already done so multiple times + msg, _ = wait_until( + 30, + 1, + lambda: env.pageserver.assert_log_contains( + f".*tenant_id={env.initial_tenant}.*: compaction iteration complete.*", second_start + ), + ) + needle = re.search(" elapsed_ms=([0-9]+)", msg) + assert needle is not None, "failed to find the elapsed time" + duration = int(needle.group(1)) / 1000.0 + neon_compare.zenbenchmark.record("compaction", duration, "s", MetricReport.LOWER_IS_BETTER) + + +def wait_and_record_startup_metrics( + pageserver: NeonPageserver, target: NeonBenchmarker, prefix: str +): + """ + Waits until all startup metrics have non-zero values on the pageserver, then records them on the target + """ + + client = pageserver.http_client() + + expected_labels = set( + [ + "background_jobs_can_start", + "complete", + "initial", + "initial_tenant_load", + "initial_tenant_load_remote", + ] + ) + + def metrics_are_filled() -> List[Sample]: + m = client.get_metrics() + samples = m.query_all("pageserver_startup_duration_seconds") + # we should not have duplicate labels + matching = [ + x for x in samples if x.labels.get("phase") in expected_labels and x.value > 0.0 + ] + assert len(matching) == len(expected_labels) + return matching + + samples = wait_until(10, 1, metrics_are_filled) + + for sample in samples: + phase = sample.labels["phase"] + name = f"{prefix}.{phase}" + target.record(name, sample.value, "s", MetricReport.LOWER_IS_BETTER) + # Test measures the branch creation time when branching from a timeline with a lot of relations. # From 871977f14c2ca93f736a82c07da93a3c142d0ab0 Mon Sep 17 00:00:00 2001 From: Vlad Lazar Date: Thu, 7 Mar 2024 16:02:20 +0000 Subject: [PATCH 03/43] pageserver: fix early bail out in vectored get (#7038) ## Problem When vectored get encountered a portion of the key range that could not be mapped to any layer in the current timeline it would incorrectly bail out of the current timeline. This is incorrect since we may have had layers queued for a visit in the fringe. ## Summary of changes * Add a repro unit test * Remove the early bail out path * Simplify range search return value --- pageserver/src/tenant.rs | 165 +++++++++++++++++++++++++++-- pageserver/src/tenant/layer_map.rs | 24 +++-- pageserver/src/tenant/timeline.rs | 9 +- 3 files changed, 176 insertions(+), 22 deletions(-) diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index b24c06c4daa3..2f23e535faf2 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -3679,7 +3679,10 @@ pub(crate) mod harness { } impl TenantHarness { - pub fn create(test_name: &'static str) -> anyhow::Result { + pub fn create_custom( + test_name: &'static str, + tenant_conf: TenantConf, + ) -> anyhow::Result { setup_logging(); let repo_dir = PageServerConf::test_repo_dir(test_name); @@ -3691,14 +3694,6 @@ pub(crate) mod harness { // OK in a test. let conf: &'static PageServerConf = Box::leak(Box::new(conf)); - // Disable automatic GC and compaction to make the unit tests more deterministic. - // The tests perform them manually if needed. - let tenant_conf = TenantConf { - gc_period: Duration::ZERO, - compaction_period: Duration::ZERO, - ..TenantConf::default() - }; - let tenant_id = TenantId::generate(); let tenant_shard_id = TenantShardId::unsharded(tenant_id); fs::create_dir_all(conf.tenant_path(&tenant_shard_id))?; @@ -3726,6 +3721,18 @@ pub(crate) mod harness { }) } + pub fn create(test_name: &'static str) -> anyhow::Result { + // Disable automatic GC and compaction to make the unit tests more deterministic. + // The tests perform them manually if needed. + let tenant_conf = TenantConf { + gc_period: Duration::ZERO, + compaction_period: Duration::ZERO, + ..TenantConf::default() + }; + + Self::create_custom(test_name, tenant_conf) + } + pub fn span(&self) -> tracing::Span { info_span!("TenantHarness", tenant_id=%self.tenant_shard_id.tenant_id, shard_id=%self.tenant_shard_id.shard_slug()) } @@ -3833,6 +3840,7 @@ mod tests { use crate::keyspace::KeySpaceAccum; use crate::repository::{Key, Value}; use crate::tenant::harness::*; + use crate::tenant::timeline::CompactFlags; use crate::DEFAULT_PG_VERSION; use bytes::BytesMut; use hex_literal::hex; @@ -4637,6 +4645,145 @@ mod tests { Ok(()) } + // Test that vectored get handles layer gaps correctly + // by advancing into the next ancestor timeline if required. + // + // The test generates timelines that look like the diagram below. + // We leave a gap in one of the L1 layers at `gap_at_key` (`/` in the diagram). + // The reconstruct data for that key lies in the ancestor timeline (`X` in the diagram). + // + // ``` + //-------------------------------+ + // ... | + // [ L1 ] | + // [ / L1 ] | Child Timeline + // ... | + // ------------------------------+ + // [ X L1 ] | Parent Timeline + // ------------------------------+ + // ``` + #[tokio::test] + async fn test_get_vectored_key_gap() -> anyhow::Result<()> { + let tenant_conf = TenantConf { + // Make compaction deterministic + gc_period: Duration::ZERO, + compaction_period: Duration::ZERO, + // Encourage creation of L1 layers + checkpoint_distance: 16 * 1024, + compaction_target_size: 8 * 1024, + ..TenantConf::default() + }; + + let harness = TenantHarness::create_custom("test_get_vectored_key_gap", tenant_conf)?; + let (tenant, ctx) = harness.load().await; + + let mut current_key = Key::from_hex("010000000033333333444444445500000000").unwrap(); + let gap_at_key = current_key.add(100); + let mut current_lsn = Lsn(0x10); + + const KEY_COUNT: usize = 10_000; + + let timeline_id = TimelineId::generate(); + let current_timeline = tenant + .create_test_timeline(timeline_id, current_lsn, DEFAULT_PG_VERSION, &ctx) + .await?; + + current_lsn += 0x100; + + let writer = current_timeline.writer().await; + writer + .put( + gap_at_key, + current_lsn, + &Value::Image(test_img(&format!("{} at {}", gap_at_key, current_lsn))), + &ctx, + ) + .await?; + writer.finish_write(current_lsn); + drop(writer); + + let mut latest_lsns = HashMap::new(); + latest_lsns.insert(gap_at_key, current_lsn); + + current_timeline.freeze_and_flush().await?; + + let child_timeline_id = TimelineId::generate(); + + tenant + .branch_timeline_test( + ¤t_timeline, + child_timeline_id, + Some(current_lsn), + &ctx, + ) + .await?; + let child_timeline = tenant + .get_timeline(child_timeline_id, true) + .expect("Should have the branched timeline"); + + for i in 0..KEY_COUNT { + if current_key == gap_at_key { + current_key = current_key.next(); + continue; + } + + current_lsn += 0x10; + + let writer = child_timeline.writer().await; + writer + .put( + current_key, + current_lsn, + &Value::Image(test_img(&format!("{} at {}", current_key, current_lsn))), + &ctx, + ) + .await?; + writer.finish_write(current_lsn); + drop(writer); + + latest_lsns.insert(current_key, current_lsn); + current_key = current_key.next(); + + // Flush every now and then to encourage layer file creation. + if i % 500 == 0 { + child_timeline.freeze_and_flush().await?; + } + } + + child_timeline.freeze_and_flush().await?; + let mut flags = EnumSet::new(); + flags.insert(CompactFlags::ForceRepartition); + child_timeline + .compact(&CancellationToken::new(), flags, &ctx) + .await?; + + let key_near_end = { + let mut tmp = current_key; + tmp.field6 -= 10; + tmp + }; + + let key_near_gap = { + let mut tmp = gap_at_key; + tmp.field6 -= 10; + tmp + }; + + let read = KeySpace { + ranges: vec![key_near_gap..gap_at_key.next(), key_near_end..current_key], + }; + let results = child_timeline + .get_vectored_impl(read.clone(), current_lsn, &ctx) + .await?; + + for (key, img_res) in results { + let expected = test_img(&format!("{} at {}", key, latest_lsns[&key])); + assert_eq!(img_res?, expected); + } + + Ok(()) + } + #[tokio::test] async fn test_random_updates() -> anyhow::Result<()> { let harness = TenantHarness::create("test_random_updates")?; diff --git a/pageserver/src/tenant/layer_map.rs b/pageserver/src/tenant/layer_map.rs index 5f4814cc6bf5..b8ed69052f5e 100644 --- a/pageserver/src/tenant/layer_map.rs +++ b/pageserver/src/tenant/layer_map.rs @@ -460,15 +460,22 @@ impl LayerMap { } } - pub fn range_search(&self, key_range: Range, end_lsn: Lsn) -> Option { - let version = self.historic.get().unwrap().get_version(end_lsn.0 - 1)?; + pub fn range_search(&self, key_range: Range, end_lsn: Lsn) -> RangeSearchResult { + let version = match self.historic.get().unwrap().get_version(end_lsn.0 - 1) { + Some(version) => version, + None => { + let mut result = RangeSearchResult::new(); + result.not_found.add_range(key_range); + return result; + } + }; let raw_range = key_range.start.to_i128()..key_range.end.to_i128(); let delta_changes = version.delta_coverage.range_overlaps(&raw_range); let image_changes = version.image_coverage.range_overlaps(&raw_range); let collector = RangeSearchCollector::new(key_range, end_lsn, delta_changes, image_changes); - Some(collector.collect()) + collector.collect() } /// Start a batch of updates, applied on drop @@ -995,8 +1002,13 @@ mod tests { let layer_map = LayerMap::default(); let range = Key::from_i128(100)..Key::from_i128(200); - let res = layer_map.range_search(range, Lsn(100)); - assert!(res.is_none()); + let res = layer_map.range_search(range.clone(), Lsn(100)); + assert_eq!( + res.not_found.to_keyspace(), + KeySpace { + ranges: vec![range] + } + ); } #[test] @@ -1033,7 +1045,7 @@ mod tests { for start in 0..60 { for end in (start + 1)..60 { let range = Key::from_i128(start)..Key::from_i128(end); - let result = layer_map.range_search(range.clone(), Lsn(100)).unwrap(); + let result = layer_map.range_search(range.clone(), Lsn(100)); let expected = brute_force_range_search(&layer_map, range, Lsn(100)); assert_range_search_result_eq(result, expected); diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 7ac7c15876c4..71a958206c5c 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -2784,7 +2784,7 @@ impl Timeline { let guard = timeline.layers.read().await; let layers = guard.layer_map(); - 'outer: loop { + loop { if cancel.is_cancelled() { return Err(GetVectoredError::Cancelled); } @@ -2810,12 +2810,7 @@ impl Timeline { } None => { for range in unmapped_keyspace.ranges.iter() { - let results = match layers.range_search(range.clone(), cont_lsn) { - Some(res) => res, - None => { - break 'outer; - } - }; + let results = layers.range_search(range.clone(), cont_lsn); results .found From d5a6a2a16d7e63d21ef00b3d582da57485f42d06 Mon Sep 17 00:00:00 2001 From: John Spray Date: Thu, 7 Mar 2024 17:10:03 +0000 Subject: [PATCH 04/43] storage controller: robustness improvements (#7027) ## Problem Closes: https://github.com/neondatabase/neon/issues/6847 Closes: https://github.com/neondatabase/neon/issues/7006 ## Summary of changes - Pageserver API calls are wrapped in timeout/retry logic: this prevents a reconciler getting hung on a pageserver API hang, and prevents reconcilers having to totally retry if one API call returns a retryable error (e.g. 503). - Add a cancellation token to `Node`, so that when we mark a node offline we will cancel any API calls in progress to that node, and avoid issuing any more API calls to that offline node. - If the dirty locations of a shard are all on offline nodes, then don't spawn a reconciler - In re-attach, if we have no observed state object for a tenant then construct one with conf: None (which means "unknown"). Then in Reconciler, implement a TODO for scanning such locations before running, so that we will avoid spuriously incrementing a generation in the case of a node that was offline while we started (this is the case that tripped up #7006) - Refactoring: make Node contents private (and thereby guarantee that updates to availability mode reliably update the cancellation token.) - Refactoring: don't pass the whole map of nodes into Reconciler (and thereby remove a bunch of .expect() calls) Some of this was discovered/tested with a new failure injection test that will come in a separate PR, once it is stable enough for CI. --- control_plane/attachment_service/src/node.rs | 218 ++++++++++- .../attachment_service/src/reconciler.rs | 352 +++++++++++------- .../attachment_service/src/scheduler.rs | 30 +- .../attachment_service/src/service.rs | 348 ++++++++--------- .../attachment_service/src/tenant_state.rs | 129 +++++-- pageserver/client/src/mgmt_api.rs | 20 +- pageserver/src/http/routes.rs | 27 ++ pageserver/src/tenant/mgr.rs | 10 + 8 files changed, 747 insertions(+), 387 deletions(-) diff --git a/control_plane/attachment_service/src/node.rs b/control_plane/attachment_service/src/node.rs index 1f9dcef03374..27b03608fa25 100644 --- a/control_plane/attachment_service/src/node.rs +++ b/control_plane/attachment_service/src/node.rs @@ -1,6 +1,16 @@ -use pageserver_api::controller_api::{NodeAvailability, NodeSchedulingPolicy}; +use std::{str::FromStr, time::Duration}; + +use hyper::StatusCode; +use pageserver_api::{ + controller_api::{ + NodeAvailability, NodeRegisterRequest, NodeSchedulingPolicy, TenantLocateResponseShard, + }, + shard::TenantShardId, +}; +use pageserver_client::mgmt_api; use serde::Serialize; -use utils::id::NodeId; +use tokio_util::sync::CancellationToken; +use utils::{backoff, id::NodeId}; use crate::persistence::NodePersistence; @@ -12,16 +22,29 @@ use crate::persistence::NodePersistence; /// implementation of serialization on this type is only for debug dumps. #[derive(Clone, Serialize)] pub(crate) struct Node { - pub(crate) id: NodeId, + id: NodeId, + + availability: NodeAvailability, + scheduling: NodeSchedulingPolicy, - pub(crate) availability: NodeAvailability, - pub(crate) scheduling: NodeSchedulingPolicy, + listen_http_addr: String, + listen_http_port: u16, - pub(crate) listen_http_addr: String, - pub(crate) listen_http_port: u16, + listen_pg_addr: String, + listen_pg_port: u16, - pub(crate) listen_pg_addr: String, - pub(crate) listen_pg_port: u16, + // This cancellation token means "stop any RPCs in flight to this node, and don't start + // any more". It is not related to process shutdown. + #[serde(skip)] + cancel: CancellationToken, +} + +/// When updating [`Node::availability`] we use this type to indicate to the caller +/// whether/how they changed it. +pub(crate) enum AvailabilityTransition { + ToActive, + ToOffline, + Unchanged, } impl Node { @@ -29,6 +52,71 @@ impl Node { format!("http://{}:{}", self.listen_http_addr, self.listen_http_port) } + pub(crate) fn get_id(&self) -> NodeId { + self.id + } + + pub(crate) fn set_scheduling(&mut self, scheduling: NodeSchedulingPolicy) { + self.scheduling = scheduling + } + + /// Does this registration request match `self`? This is used when deciding whether a registration + /// request should be allowed to update an existing record with the same node ID. + pub(crate) fn registration_match(&self, register_req: &NodeRegisterRequest) -> bool { + self.id == register_req.node_id + && self.listen_http_addr == register_req.listen_http_addr + && self.listen_http_port == register_req.listen_http_port + && self.listen_pg_addr == register_req.listen_pg_addr + && self.listen_pg_port == register_req.listen_pg_port + } + + /// For a shard located on this node, populate a response object + /// with this node's address information. + pub(crate) fn shard_location(&self, shard_id: TenantShardId) -> TenantLocateResponseShard { + TenantLocateResponseShard { + shard_id, + node_id: self.id, + listen_http_addr: self.listen_http_addr.clone(), + listen_http_port: self.listen_http_port, + listen_pg_addr: self.listen_pg_addr.clone(), + listen_pg_port: self.listen_pg_port, + } + } + + pub(crate) fn set_availability( + &mut self, + availability: NodeAvailability, + ) -> AvailabilityTransition { + use NodeAvailability::*; + let transition = match (self.availability, availability) { + (Offline, Active) => { + // Give the node a new cancellation token, effectively resetting it to un-cancelled. Any + // users of previously-cloned copies of the node will still see the old cancellation + // state. For example, Reconcilers in flight will have to complete and be spawned + // again to realize that the node has become available. + self.cancel = CancellationToken::new(); + AvailabilityTransition::ToActive + } + (Active, Offline) => { + // Fire the node's cancellation token to cancel any in-flight API requests to it + self.cancel.cancel(); + AvailabilityTransition::ToOffline + } + _ => AvailabilityTransition::Unchanged, + }; + self.availability = availability; + transition + } + + /// Whether we may send API requests to this node. + pub(crate) fn is_available(&self) -> bool { + // When we clone a node, [`Self::availability`] is a snapshot, but [`Self::cancel`] holds + // a reference to the original Node's cancellation status. Checking both of these results + // in a "pessimistic" check where we will consider a Node instance unavailable if it was unavailable + // when we cloned it, or if the original Node instance's cancellation token was fired. + matches!(self.availability, NodeAvailability::Active) && !self.cancel.is_cancelled() + } + /// Is this node elegible to have work scheduled onto it? pub(crate) fn may_schedule(&self) -> bool { match self.availability { @@ -44,6 +132,26 @@ impl Node { } } + pub(crate) fn new( + id: NodeId, + listen_http_addr: String, + listen_http_port: u16, + listen_pg_addr: String, + listen_pg_port: u16, + ) -> Self { + Self { + id, + listen_http_addr, + listen_http_port, + listen_pg_addr, + listen_pg_port, + scheduling: NodeSchedulingPolicy::Filling, + // TODO: we shouldn't really call this Active until we've heartbeated it. + availability: NodeAvailability::Active, + cancel: CancellationToken::new(), + } + } + pub(crate) fn to_persistent(&self) -> NodePersistence { NodePersistence { node_id: self.id.0 as i64, @@ -54,4 +162,96 @@ impl Node { listen_pg_port: self.listen_pg_port as i32, } } + + pub(crate) fn from_persistent(np: NodePersistence) -> Self { + Self { + id: NodeId(np.node_id as u64), + // At startup we consider a node offline until proven otherwise. + availability: NodeAvailability::Offline, + scheduling: NodeSchedulingPolicy::from_str(&np.scheduling_policy) + .expect("Bad scheduling policy in DB"), + listen_http_addr: np.listen_http_addr, + listen_http_port: np.listen_http_port as u16, + listen_pg_addr: np.listen_pg_addr, + listen_pg_port: np.listen_pg_port as u16, + cancel: CancellationToken::new(), + } + } + + /// Wrapper for issuing requests to pageserver management API: takes care of generic + /// retry/backoff for retryable HTTP status codes. + /// + /// This will return None to indicate cancellation. Cancellation may happen from + /// the cancellation token passed in, or from Self's cancellation token (i.e. node + /// going offline). + pub(crate) async fn with_client_retries( + &self, + mut op: O, + jwt: &Option, + warn_threshold: u32, + max_retries: u32, + timeout: Duration, + cancel: &CancellationToken, + ) -> Option> + where + O: FnMut(mgmt_api::Client) -> F, + F: std::future::Future>, + { + fn is_fatal(e: &mgmt_api::Error) -> bool { + use mgmt_api::Error::*; + match e { + ReceiveBody(_) | ReceiveErrorBody(_) => false, + ApiError(StatusCode::SERVICE_UNAVAILABLE, _) + | ApiError(StatusCode::GATEWAY_TIMEOUT, _) + | ApiError(StatusCode::REQUEST_TIMEOUT, _) => false, + ApiError(_, _) => true, + Cancelled => true, + } + } + + backoff::retry( + || { + let http_client = reqwest::ClientBuilder::new() + .timeout(timeout) + .build() + .expect("Failed to construct HTTP client"); + + let client = + mgmt_api::Client::from_client(http_client, self.base_url(), jwt.as_deref()); + + let node_cancel_fut = self.cancel.cancelled(); + + let op_fut = op(client); + + async { + tokio::select! { + r = op_fut=> {r}, + _ = node_cancel_fut => { + Err(mgmt_api::Error::Cancelled) + }} + } + }, + is_fatal, + warn_threshold, + max_retries, + &format!( + "Call to node {} ({}:{}) management API", + self.id, self.listen_http_addr, self.listen_http_port + ), + cancel, + ) + .await + } +} + +impl std::fmt::Display for Node { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{} ({})", self.id, self.listen_http_addr) + } +} + +impl std::fmt::Debug for Node { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{} ({})", self.id, self.listen_http_addr) + } } diff --git a/control_plane/attachment_service/src/reconciler.rs b/control_plane/attachment_service/src/reconciler.rs index 0fa6e8e2f8d5..603da9bf022c 100644 --- a/control_plane/attachment_service/src/reconciler.rs +++ b/control_plane/attachment_service/src/reconciler.rs @@ -1,6 +1,5 @@ use crate::persistence::Persistence; use crate::service; -use pageserver_api::controller_api::NodeAvailability; use pageserver_api::models::{ LocationConfig, LocationConfigMode, LocationConfigSecondary, TenantConfig, }; @@ -28,15 +27,16 @@ pub(super) struct Reconciler { pub(crate) shard: ShardIdentity, pub(crate) generation: Option, pub(crate) intent: TargetState, + + /// Nodes not referenced by [`Self::intent`], from which we should try + /// to detach this tenant shard. + pub(crate) detach: Vec, + pub(crate) config: TenantConfig, pub(crate) observed: ObservedState, pub(crate) service_config: service::Config, - /// A snapshot of the pageservers as they were when we were asked - /// to reconcile. - pub(crate) pageservers: Arc>, - /// A hook to notify the running postgres instances when we change the location /// of a tenant. Use this via [`Self::compute_notify`] to update our failure flag /// and guarantee eventual retries. @@ -67,29 +67,37 @@ pub(super) struct Reconciler { /// and the TargetState is just the instruction for a particular Reconciler run. #[derive(Debug)] pub(crate) struct TargetState { - pub(crate) attached: Option, - pub(crate) secondary: Vec, + pub(crate) attached: Option, + pub(crate) secondary: Vec, } impl TargetState { - pub(crate) fn from_intent(intent: &IntentState) -> Self { + pub(crate) fn from_intent(nodes: &HashMap, intent: &IntentState) -> Self { Self { - attached: *intent.get_attached(), - secondary: intent.get_secondary().clone(), - } - } - - fn all_pageservers(&self) -> Vec { - let mut result = self.secondary.clone(); - if let Some(node_id) = &self.attached { - result.push(*node_id); + attached: intent.get_attached().map(|n| { + nodes + .get(&n) + .expect("Intent attached referenced non-existent node") + .clone() + }), + secondary: intent + .get_secondary() + .iter() + .map(|n| { + nodes + .get(n) + .expect("Intent secondary referenced non-existent node") + .clone() + }) + .collect(), } - result } } #[derive(thiserror::Error, Debug)] pub(crate) enum ReconcileError { + #[error(transparent)] + Remote(#[from] mgmt_api::Error), #[error(transparent)] Notify(#[from] NotifyError), #[error("Cancelled")] @@ -101,45 +109,83 @@ pub(crate) enum ReconcileError { impl Reconciler { async fn location_config( &mut self, - node_id: NodeId, + node: &Node, config: LocationConfig, flush_ms: Option, lazy: bool, - ) -> anyhow::Result<()> { - let node = self - .pageservers - .get(&node_id) - .expect("Pageserver may not be removed while referenced"); - + ) -> Result<(), ReconcileError> { self.observed .locations - .insert(node.id, ObservedStateLocation { conf: None }); - - tracing::info!("location_config({}) calling: {:?}", node_id, config); - let client = - mgmt_api::Client::new(node.base_url(), self.service_config.jwt_token.as_deref()); - client - .location_config(self.tenant_shard_id, config.clone(), flush_ms, lazy) - .await?; - tracing::info!("location_config({}) complete: {:?}", node_id, config); + .insert(node.get_id(), ObservedStateLocation { conf: None }); + + // TODO: amend locations that use long-polling: they will hit this timeout. + let timeout = Duration::from_secs(25); + + tracing::info!("location_config({node}) calling: {:?}", config); + let tenant_shard_id = self.tenant_shard_id; + let config_ref = &config; + match node + .with_client_retries( + |client| async move { + let config = config_ref.clone(); + client + .location_config(tenant_shard_id, config.clone(), flush_ms, lazy) + .await + }, + &self.service_config.jwt_token, + 1, + 3, + timeout, + &self.cancel, + ) + .await + { + Some(Ok(_)) => {} + Some(Err(e)) => return Err(e.into()), + None => return Err(ReconcileError::Cancel), + }; + tracing::info!("location_config({node}) complete: {:?}", config); self.observed .locations - .insert(node.id, ObservedStateLocation { conf: Some(config) }); + .insert(node.get_id(), ObservedStateLocation { conf: Some(config) }); Ok(()) } + fn get_node(&self, node_id: &NodeId) -> Option<&Node> { + if let Some(node) = self.intent.attached.as_ref() { + if node.get_id() == *node_id { + return Some(node); + } + } + + if let Some(node) = self + .intent + .secondary + .iter() + .find(|n| n.get_id() == *node_id) + { + return Some(node); + } + + if let Some(node) = self.detach.iter().find(|n| n.get_id() == *node_id) { + return Some(node); + } + + None + } + async fn maybe_live_migrate(&mut self) -> Result<(), ReconcileError> { - let destination = if let Some(node_id) = self.intent.attached { - match self.observed.locations.get(&node_id) { + let destination = if let Some(node) = &self.intent.attached { + match self.observed.locations.get(&node.get_id()) { Some(conf) => { // We will do a live migration only if the intended destination is not // currently in an attached state. match &conf.conf { Some(conf) if conf.mode == LocationConfigMode::Secondary => { // Fall through to do a live migration - node_id + node } None | Some(_) => { // Attached or uncertain: don't do a live migration, proceed @@ -152,7 +198,7 @@ impl Reconciler { None => { // Our destination is not attached: maybe live migrate if some other // node is currently attached. Fall through. - node_id + node } } } else { @@ -165,15 +211,13 @@ impl Reconciler { for (node_id, state) in &self.observed.locations { if let Some(observed_conf) = &state.conf { if observed_conf.mode == LocationConfigMode::AttachedSingle { - let node = self - .pageservers - .get(node_id) - .expect("Nodes may not be removed while referenced"); // We will only attempt live migration if the origin is not offline: this // avoids trying to do it while reconciling after responding to an HA failover. - if !matches!(node.availability, NodeAvailability::Offline) { - origin = Some(*node_id); - break; + if let Some(node) = self.get_node(node_id) { + if node.is_available() { + origin = Some(node.clone()); + break; + } } } } @@ -186,7 +230,7 @@ impl Reconciler { // We have an origin and a destination: proceed to do the live migration tracing::info!("Live migrating {}->{}", origin, destination); - self.live_migrate(origin, destination).await?; + self.live_migrate(origin, destination.clone()).await?; Ok(()) } @@ -194,13 +238,8 @@ impl Reconciler { async fn get_lsns( &self, tenant_shard_id: TenantShardId, - node_id: &NodeId, + node: &Node, ) -> anyhow::Result> { - let node = self - .pageservers - .get(node_id) - .expect("Pageserver may not be removed while referenced"); - let client = mgmt_api::Client::new(node.base_url(), self.service_config.jwt_token.as_deref()); @@ -211,19 +250,27 @@ impl Reconciler { .collect()) } - async fn secondary_download(&self, tenant_shard_id: TenantShardId, node_id: &NodeId) { - let node = self - .pageservers - .get(node_id) - .expect("Pageserver may not be removed while referenced"); - - let client = - mgmt_api::Client::new(node.base_url(), self.service_config.jwt_token.as_deref()); - - match client.tenant_secondary_download(tenant_shard_id).await { - Ok(()) => {} - Err(_) => { - tracing::info!(" (skipping, destination wasn't in secondary mode)") + async fn secondary_download( + &self, + tenant_shard_id: TenantShardId, + node: &Node, + ) -> Result<(), ReconcileError> { + match node + .with_client_retries( + |client| async move { client.tenant_secondary_download(tenant_shard_id).await }, + &self.service_config.jwt_token, + 1, + 1, + Duration::from_secs(60), + &self.cancel, + ) + .await + { + None => Err(ReconcileError::Cancel), + Some(Ok(_)) => Ok(()), + Some(Err(e)) => { + tracing::info!(" (skipping destination download: {})", e); + Ok(()) } } } @@ -231,17 +278,14 @@ impl Reconciler { async fn await_lsn( &self, tenant_shard_id: TenantShardId, - pageserver_id: &NodeId, + node: &Node, baseline: HashMap, ) -> anyhow::Result<()> { loop { - let latest = match self.get_lsns(tenant_shard_id, pageserver_id).await { + let latest = match self.get_lsns(tenant_shard_id, node).await { Ok(l) => l, Err(e) => { - println!( - "πŸ•‘ Can't get LSNs on pageserver {} yet, waiting ({e})", - pageserver_id - ); + tracing::info!("πŸ•‘ Can't get LSNs on node {node} yet, waiting ({e})",); std::thread::sleep(Duration::from_millis(500)); continue; } @@ -251,7 +295,7 @@ impl Reconciler { for (timeline_id, baseline_lsn) in &baseline { match latest.get(timeline_id) { Some(latest_lsn) => { - println!("πŸ•‘ LSN origin {baseline_lsn} vs destination {latest_lsn}"); + tracing::info!("πŸ•‘ LSN origin {baseline_lsn} vs destination {latest_lsn}"); if latest_lsn < baseline_lsn { any_behind = true; } @@ -266,7 +310,7 @@ impl Reconciler { } if !any_behind { - println!("βœ… LSN caught up. Proceeding..."); + tracing::info!("βœ… LSN caught up. Proceeding..."); break; } else { std::thread::sleep(Duration::from_millis(500)); @@ -278,11 +322,11 @@ impl Reconciler { pub async fn live_migrate( &mut self, - origin_ps_id: NodeId, - dest_ps_id: NodeId, - ) -> anyhow::Result<()> { + origin_ps: Node, + dest_ps: Node, + ) -> Result<(), ReconcileError> { // `maybe_live_migrate` is responsibble for sanity of inputs - assert!(origin_ps_id != dest_ps_id); + assert!(origin_ps.get_id() != dest_ps.get_id()); fn build_location_config( shard: &ShardIdentity, @@ -302,10 +346,7 @@ impl Reconciler { } } - tracing::info!( - "πŸ” Switching origin pageserver {} to stale mode", - origin_ps_id - ); + tracing::info!("πŸ” Switching origin node {origin_ps} to stale mode",); // FIXME: it is incorrect to use self.generation here, we should use the generation // from the ObservedState of the origin pageserver (it might be older than self.generation) @@ -316,26 +357,18 @@ impl Reconciler { self.generation, None, ); - self.location_config( - origin_ps_id, - stale_conf, - Some(Duration::from_secs(10)), - false, - ) - .await?; + self.location_config(&origin_ps, stale_conf, Some(Duration::from_secs(10)), false) + .await?; - let baseline_lsns = Some(self.get_lsns(self.tenant_shard_id, &origin_ps_id).await?); + let baseline_lsns = Some(self.get_lsns(self.tenant_shard_id, &origin_ps).await?); // If we are migrating to a destination that has a secondary location, warm it up first - if let Some(destination_conf) = self.observed.locations.get(&dest_ps_id) { + if let Some(destination_conf) = self.observed.locations.get(&dest_ps.get_id()) { if let Some(destination_conf) = &destination_conf.conf { if destination_conf.mode == LocationConfigMode::Secondary { - tracing::info!( - "πŸ” Downloading latest layers to destination pageserver {}", - dest_ps_id, - ); - self.secondary_download(self.tenant_shard_id, &dest_ps_id) - .await; + tracing::info!("πŸ” Downloading latest layers to destination node {dest_ps}",); + self.secondary_download(self.tenant_shard_id, &dest_ps) + .await?; } } } @@ -343,7 +376,7 @@ impl Reconciler { // Increment generation before attaching to new pageserver self.generation = Some( self.persistence - .increment_generation(self.tenant_shard_id, dest_ps_id) + .increment_generation(self.tenant_shard_id, dest_ps.get_id()) .await?, ); @@ -355,23 +388,23 @@ impl Reconciler { None, ); - tracing::info!("πŸ” Attaching to pageserver {}", dest_ps_id); - self.location_config(dest_ps_id, dest_conf, None, false) + tracing::info!("πŸ” Attaching to pageserver {dest_ps}"); + self.location_config(&dest_ps, dest_conf, None, false) .await?; if let Some(baseline) = baseline_lsns { tracing::info!("πŸ•‘ Waiting for LSN to catch up..."); - self.await_lsn(self.tenant_shard_id, &dest_ps_id, baseline) + self.await_lsn(self.tenant_shard_id, &dest_ps, baseline) .await?; } - tracing::info!("πŸ” Notifying compute to use pageserver {}", dest_ps_id); + tracing::info!("πŸ” Notifying compute to use pageserver {dest_ps}"); // During a live migration it is unhelpful to proceed if we couldn't notify compute: if we detach // the origin without notifying compute, we will render the tenant unavailable. while let Err(e) = self.compute_notify().await { match e { - NotifyError::Fatal(_) => return Err(anyhow::anyhow!(e)), + NotifyError::Fatal(_) => return Err(ReconcileError::Notify(e)), _ => { tracing::warn!( "Live migration blocked by compute notification error, retrying: {e}" @@ -389,22 +422,19 @@ impl Reconciler { None, Some(LocationConfigSecondary { warm: true }), ); - self.location_config(origin_ps_id, origin_secondary_conf.clone(), None, false) + self.location_config(&origin_ps, origin_secondary_conf.clone(), None, false) .await?; // TODO: we should also be setting the ObservedState on earlier API calls, in case we fail // partway through. In fact, all location conf API calls should be in a wrapper that sets // the observed state to None, then runs, then sets it to what we wrote. self.observed.locations.insert( - origin_ps_id, + origin_ps.get_id(), ObservedStateLocation { conf: Some(origin_secondary_conf), }, ); - println!( - "πŸ” Switching to AttachedSingle mode on pageserver {}", - dest_ps_id - ); + tracing::info!("πŸ” Switching to AttachedSingle mode on node {dest_ps}",); let dest_final_conf = build_location_config( &self.shard, &self.config, @@ -412,16 +442,61 @@ impl Reconciler { self.generation, None, ); - self.location_config(dest_ps_id, dest_final_conf.clone(), None, false) + self.location_config(&dest_ps, dest_final_conf.clone(), None, false) .await?; self.observed.locations.insert( - dest_ps_id, + dest_ps.get_id(), ObservedStateLocation { conf: Some(dest_final_conf), }, ); - println!("βœ… Migration complete"); + tracing::info!("βœ… Migration complete"); + + Ok(()) + } + + async fn maybe_refresh_observed(&mut self) -> Result<(), ReconcileError> { + // If the attached node has uncertain state, read it from the pageserver before proceeding: this + // is important to avoid spurious generation increments. + // + // We don't need to do this for secondary/detach locations because it's harmless to just PUT their + // location conf, whereas for attached locations it can interrupt clients if we spuriously destroy/recreate + // the `Timeline` object in the pageserver. + + let Some(attached_node) = self.intent.attached.as_ref() else { + // Nothing to do + return Ok(()); + }; + + if matches!( + self.observed.locations.get(&attached_node.get_id()), + Some(ObservedStateLocation { conf: None }) + ) { + let tenant_shard_id = self.tenant_shard_id; + let observed_conf = match attached_node + .with_client_retries( + |client| async move { client.get_location_config(tenant_shard_id).await }, + &self.service_config.jwt_token, + 1, + 1, + Duration::from_secs(5), + &self.cancel, + ) + .await + { + Some(Ok(observed)) => observed, + Some(Err(e)) => return Err(e.into()), + None => return Err(ReconcileError::Cancel), + }; + tracing::info!("Scanned location configuration on {attached_node}: {observed_conf:?}"); + self.observed.locations.insert( + attached_node.get_id(), + ObservedStateLocation { + conf: observed_conf, + }, + ); + } Ok(()) } @@ -433,14 +508,14 @@ impl Reconciler { /// general case reconciliation where we walk through the intent by pageserver /// and call out to the pageserver to apply the desired state. pub(crate) async fn reconcile(&mut self) -> Result<(), ReconcileError> { - // TODO: if any of self.observed is None, call to remote pageservers - // to learn correct state. + // Prepare: if we have uncertain `observed` state for our would-be attachement location, then refresh it + self.maybe_refresh_observed().await?; // Special case: live migration self.maybe_live_migrate().await?; // If the attached pageserver is not attached, do so now. - if let Some(node_id) = self.intent.attached { + if let Some(node) = self.intent.attached.as_ref() { // If we are in an attached policy, then generation must have been set (null generations // are only present when a tenant is initially loaded with a secondary policy) debug_assert!(self.generation.is_some()); @@ -451,10 +526,10 @@ impl Reconciler { }; let mut wanted_conf = attached_location_conf(generation, &self.shard, &self.config); - match self.observed.locations.get(&node_id) { + match self.observed.locations.get(&node.get_id()) { Some(conf) if conf.conf.as_ref() == Some(&wanted_conf) => { // Nothing to do - tracing::info!(%node_id, "Observed configuration already correct.") + tracing::info!(node_id=%node.get_id(), "Observed configuration already correct.") } observed => { // In all cases other than a matching observed configuration, we will @@ -492,16 +567,21 @@ impl Reconciler { if increment_generation { let generation = self .persistence - .increment_generation(self.tenant_shard_id, node_id) + .increment_generation(self.tenant_shard_id, node.get_id()) .await?; self.generation = Some(generation); wanted_conf.generation = generation.into(); } - tracing::info!(%node_id, "Observed configuration requires update."); + tracing::info!(node_id=%node.get_id(), "Observed configuration requires update."); + + // Because `node` comes from a ref to &self, clone it before calling into a &mut self + // function: this could be avoided by refactoring the state mutated by location_config into + // a separate type to Self. + let node = node.clone(); + // Use lazy=true, because we may run many of Self concurrently, and do not want to // overload the pageserver with logical size calculations. - self.location_config(node_id, wanted_conf, None, true) - .await?; + self.location_config(&node, wanted_conf, None, true).await?; self.compute_notify().await?; } } @@ -510,33 +590,27 @@ impl Reconciler { // Configure secondary locations: if these were previously attached this // implicitly downgrades them from attached to secondary. let mut changes = Vec::new(); - for node_id in &self.intent.secondary { + for node in &self.intent.secondary { let wanted_conf = secondary_location_conf(&self.shard, &self.config); - match self.observed.locations.get(node_id) { + match self.observed.locations.get(&node.get_id()) { Some(conf) if conf.conf.as_ref() == Some(&wanted_conf) => { // Nothing to do - tracing::info!(%node_id, "Observed configuration already correct.") + tracing::info!(node_id=%node.get_id(), "Observed configuration already correct.") } _ => { // In all cases other than a matching observed configuration, we will // reconcile this location. - tracing::info!(%node_id, "Observed configuration requires update."); - changes.push((*node_id, wanted_conf)) + tracing::info!(node_id=%node.get_id(), "Observed configuration requires update."); + changes.push((node.clone(), wanted_conf)) } } } // Detach any extraneous pageservers that are no longer referenced // by our intent. - let all_pageservers = self.intent.all_pageservers(); - for node_id in self.observed.locations.keys() { - if all_pageservers.contains(node_id) { - // We are only detaching pageservers that aren't used at all. - continue; - } - + for node in &self.detach { changes.push(( - *node_id, + node.clone(), LocationConfig { mode: LocationConfigMode::Detached, generation: None, @@ -549,11 +623,11 @@ impl Reconciler { )); } - for (node_id, conf) in changes { + for (node, conf) in changes { if self.cancel.is_cancelled() { return Err(ReconcileError::Cancel); } - self.location_config(node_id, conf, None, false).await?; + self.location_config(&node, conf, None, false).await?; } Ok(()) @@ -562,12 +636,12 @@ impl Reconciler { pub(crate) async fn compute_notify(&mut self) -> Result<(), NotifyError> { // Whenever a particular Reconciler emits a notification, it is always notifying for the intended // destination. - if let Some(node_id) = self.intent.attached { + if let Some(node) = &self.intent.attached { let result = self .compute_hook .notify( self.tenant_shard_id, - node_id, + node.get_id(), self.shard.stripe_size, &self.cancel, ) @@ -576,7 +650,7 @@ impl Reconciler { // It is up to the caller whether they want to drop out on this error, but they don't have to: // in general we should avoid letting unavailability of the cloud control plane stop us from // making progress. - tracing::warn!("Failed to notify compute of attached pageserver {node_id}: {e}"); + tracing::warn!("Failed to notify compute of attached pageserver {node}: {e}"); // Set this flag so that in our ReconcileResult we will set the flag on the shard that it // needs to retry at some point. self.compute_notify_failure = true; diff --git a/control_plane/attachment_service/src/scheduler.rs b/control_plane/attachment_service/src/scheduler.rs index 87fce3df2577..26a2707e8d73 100644 --- a/control_plane/attachment_service/src/scheduler.rs +++ b/control_plane/attachment_service/src/scheduler.rs @@ -43,7 +43,7 @@ impl Scheduler { let mut scheduler_nodes = HashMap::new(); for node in nodes { scheduler_nodes.insert( - node.id, + node.get_id(), SchedulerNode { shard_count: 0, may_schedule: node.may_schedule(), @@ -68,7 +68,7 @@ impl Scheduler { let mut expect_nodes: HashMap = HashMap::new(); for node in nodes { expect_nodes.insert( - node.id, + node.get_id(), SchedulerNode { shard_count: 0, may_schedule: node.may_schedule(), @@ -156,7 +156,7 @@ impl Scheduler { pub(crate) fn node_upsert(&mut self, node: &Node) { use std::collections::hash_map::Entry::*; - match self.nodes.entry(node.id) { + match self.nodes.entry(node.get_id()) { Occupied(mut entry) => { entry.get_mut().may_schedule = node.may_schedule(); } @@ -255,7 +255,6 @@ impl Scheduler { pub(crate) mod test_utils { use crate::node::Node; - use pageserver_api::controller_api::{NodeAvailability, NodeSchedulingPolicy}; use std::collections::HashMap; use utils::id::NodeId; /// Test helper: synthesize the requested number of nodes, all in active state. @@ -264,18 +263,17 @@ pub(crate) mod test_utils { pub(crate) fn make_test_nodes(n: u64) -> HashMap { (1..n + 1) .map(|i| { - ( - NodeId(i), - Node { - id: NodeId(i), - availability: NodeAvailability::Active, - scheduling: NodeSchedulingPolicy::Active, - listen_http_addr: format!("httphost-{i}"), - listen_http_port: 80 + i as u16, - listen_pg_addr: format!("pghost-{i}"), - listen_pg_port: 5432 + i as u16, - }, - ) + (NodeId(i), { + let node = Node::new( + NodeId(i), + format!("httphost-{i}"), + 80 + i as u16, + format!("pghost-{i}"), + 5432 + i as u16, + ); + assert!(node.is_available()); + node + }) }) .collect() } diff --git a/control_plane/attachment_service/src/service.rs b/control_plane/attachment_service/src/service.rs index d162ab5c65b3..f41c4f89b96b 100644 --- a/control_plane/attachment_service/src/service.rs +++ b/control_plane/attachment_service/src/service.rs @@ -16,9 +16,9 @@ use futures::{stream::FuturesUnordered, StreamExt}; use hyper::StatusCode; use pageserver_api::{ controller_api::{ - NodeAvailability, NodeConfigureRequest, NodeRegisterRequest, NodeSchedulingPolicy, - TenantCreateResponse, TenantCreateResponseShard, TenantLocateResponse, - TenantLocateResponseShard, TenantShardMigrateRequest, TenantShardMigrateResponse, + NodeAvailability, NodeConfigureRequest, NodeRegisterRequest, TenantCreateResponse, + TenantCreateResponseShard, TenantLocateResponse, TenantShardMigrateRequest, + TenantShardMigrateResponse, }, models::TenantConfigRequest, }; @@ -39,7 +39,6 @@ use pageserver_client::mgmt_api; use tokio_util::sync::CancellationToken; use tracing::instrument; use utils::{ - backoff, completion::Barrier, generation::Generation, http::error::ApiError, @@ -50,7 +49,7 @@ use utils::{ use crate::{ compute_hook::{self, ComputeHook}, - node::Node, + node::{AvailabilityTransition, Node}, persistence::{split_state::SplitState, DatabaseError, Persistence, TenantShardPersistence}, reconciler::attached_location_conf, scheduler::Scheduler, @@ -201,7 +200,8 @@ impl Service { async fn startup_reconcile(self: &Arc) { // For all tenant shards, a vector of observed states on nodes (where None means // indeterminate, same as in [`ObservedStateLocation`]) - let mut observed = HashMap::new(); + let mut observed: HashMap)>> = + HashMap::new(); let mut nodes_online = HashSet::new(); @@ -236,7 +236,8 @@ impl Service { nodes_online.insert(node_id); for (tenant_shard_id, conf_opt) in tenant_shards { - observed.insert(tenant_shard_id, (node_id, conf_opt)); + let shard_observations = observed.entry(tenant_shard_id).or_default(); + shard_observations.push((node_id, conf_opt)); } } @@ -252,27 +253,28 @@ impl Service { let mut new_nodes = (**nodes).clone(); for (node_id, node) in new_nodes.iter_mut() { if nodes_online.contains(node_id) { - node.availability = NodeAvailability::Active; + node.set_availability(NodeAvailability::Active); scheduler.node_upsert(node); } } *nodes = Arc::new(new_nodes); - for (tenant_shard_id, (node_id, observed_loc)) in observed { - let Some(tenant_state) = tenants.get_mut(&tenant_shard_id) else { - cleanup.push((tenant_shard_id, node_id)); - continue; - }; - - tenant_state - .observed - .locations - .insert(node_id, ObservedStateLocation { conf: observed_loc }); + for (tenant_shard_id, shard_observations) in observed { + for (node_id, observed_loc) in shard_observations { + let Some(tenant_state) = tenants.get_mut(&tenant_shard_id) else { + cleanup.push((tenant_shard_id, node_id)); + continue; + }; + tenant_state + .observed + .locations + .insert(node_id, ObservedStateLocation { conf: observed_loc }); + } } // Populate each tenant's intent state for (tenant_shard_id, tenant_state) in tenants.iter_mut() { - tenant_state.intent_from_observed(); + tenant_state.intent_from_observed(scheduler); if let Err(e) = tenant_state.schedule(scheduler) { // Non-fatal error: we are unable to properly schedule the tenant, perhaps because // not enough pageservers are available. The tenant may well still be available @@ -359,40 +361,19 @@ impl Service { for node in nodes.values() { node_list_futs.push({ async move { - let http_client = reqwest::ClientBuilder::new() - .timeout(Duration::from_secs(5)) - .build() - .expect("Failed to construct HTTP client"); - let client = mgmt_api::Client::from_client( - http_client, - node.base_url(), - self.config.jwt_token.as_deref(), - ); - - fn is_fatal(e: &mgmt_api::Error) -> bool { - use mgmt_api::Error::*; - match e { - ReceiveBody(_) | ReceiveErrorBody(_) => false, - ApiError(StatusCode::SERVICE_UNAVAILABLE, _) - | ApiError(StatusCode::GATEWAY_TIMEOUT, _) - | ApiError(StatusCode::REQUEST_TIMEOUT, _) => false, - ApiError(_, _) => true, - } - } - - tracing::info!("Scanning shards on node {}...", node.id); - let description = format!("List locations on {}", node.id); - let response = backoff::retry( - || client.list_location_config(), - is_fatal, - 1, - 5, - &description, - &self.cancel, - ) - .await; - - (node.id, response) + tracing::info!("Scanning shards on node {node}..."); + let timeout = Duration::from_secs(5); + let response = node + .with_client_retries( + |client| async move { client.list_location_config().await }, + &self.config.jwt_token, + 1, + 5, + timeout, + &self.cancel, + ) + .await; + (node.get_id(), response) } }); } @@ -662,19 +643,9 @@ impl Service { .list_nodes() .await? .into_iter() - .map(|n| Node { - id: NodeId(n.node_id as u64), - // At startup we consider a node offline until proven otherwise. - availability: NodeAvailability::Offline, - scheduling: NodeSchedulingPolicy::from_str(&n.scheduling_policy) - .expect("Bad scheduling policy in DB"), - listen_http_addr: n.listen_http_addr, - listen_http_port: n.listen_http_port as u16, - listen_pg_addr: n.listen_pg_addr, - listen_pg_port: n.listen_pg_port as u16, - }) + .map(Node::from_persistent) .collect::>(); - let nodes: HashMap = nodes.into_iter().map(|n| (n.id, n)).collect(); + let nodes: HashMap = nodes.into_iter().map(|n| (n.get_id(), n)).collect(); tracing::info!("Loaded {} nodes from database.", nodes.len()); tracing::info!("Loading shards from database..."); @@ -701,15 +672,13 @@ impl Service { } for node_id in node_ids { tracing::info!("Creating node {} in scheduler for tests", node_id); - let node = Node { - id: NodeId(node_id as u64), - availability: NodeAvailability::Active, - scheduling: NodeSchedulingPolicy::Active, - listen_http_addr: "".to_string(), - listen_http_port: 123, - listen_pg_addr: "".to_string(), - listen_pg_port: 123, - }; + let node = Node::new( + NodeId(node_id as u64), + "".to_string(), + 123, + "".to_string(), + 123, + ); scheduler.node_upsert(&node); } @@ -975,6 +944,12 @@ impl Service { // Ordering: we must persist generation number updates before making them visible in the in-memory state let incremented_generations = self.persistence.re_attach(reattach_req.node_id).await?; + tracing::info!( + node_id=%reattach_req.node_id, + "Incremented {} tenant shards' generations", + incremented_generations.len() + ); + // Apply the updated generation to our in-memory state let mut locked = self.inner.write().unwrap(); @@ -987,7 +962,6 @@ impl Service { id: tenant_shard_id, gen: new_gen.into().unwrap(), }); - // Apply the new generation number to our in-memory state let shard_state = locked.tenants.get_mut(&tenant_shard_id); let Some(shard_state) = shard_state else { @@ -1023,6 +997,14 @@ impl Service { 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 }); } // TODO: cancel/restart any running reconciliation for this tenant, it might be trying @@ -1685,7 +1667,7 @@ impl Service { .map_err(|e| { ApiError::InternalServerError(anyhow::anyhow!( "Error doing time travel recovery for shard {tenant_shard_id} on node {}: {e}", - node.id + node )) })?; } @@ -1739,10 +1721,7 @@ impl Service { // Secondary downloads are always advisory: if something fails, we nevertheless report success, so that whoever // is calling us will proceed with whatever migration they're doing, albeit with a slightly less warm cache // than they had hoped for. - tracing::warn!( - "Ignoring tenant secondary download error from pageserver {}: {e}", - node.id, - ); + tracing::warn!("Ignoring tenant secondary download error from pageserver {node}: {e}",); } Ok(()) @@ -1780,13 +1759,11 @@ impl Service { // surface immediately as an error to our caller. let status = client.tenant_delete(tenant_shard_id).await.map_err(|e| { ApiError::InternalServerError(anyhow::anyhow!( - "Error deleting shard {tenant_shard_id} on node {}: {e}", - node.id + "Error deleting shard {tenant_shard_id} on node {node}: {e}", )) })?; tracing::info!( - "Shard {tenant_shard_id} on node {}, delete returned {}", - node.id, + "Shard {tenant_shard_id} on node {node}, delete returned {}", status ); if status == StatusCode::ACCEPTED { @@ -1885,10 +1862,9 @@ impl Service { create_req: TimelineCreateRequest, ) -> Result { tracing::info!( - "Creating timeline on shard {}/{}, attached to node {}", + "Creating timeline on shard {}/{}, attached to node {node}", tenant_shard_id, create_req.new_timeline_id, - node.id ); let client = mgmt_api::Client::new(node.base_url(), jwt.as_deref()); @@ -2012,10 +1988,7 @@ impl Service { jwt: Option, ) -> Result { tracing::info!( - "Deleting timeline on shard {}/{}, attached to node {}", - tenant_shard_id, - timeline_id, - node.id + "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()); @@ -2024,8 +1997,7 @@ impl Service { .await .map_err(|e| { ApiError::InternalServerError(anyhow::anyhow!( - "Error deleting timeline {timeline_id} on {tenant_shard_id} on node {}: {e}", - node.id + "Error deleting timeline {timeline_id} on {tenant_shard_id} on node {node}: {e}", )) }) } @@ -2126,14 +2098,7 @@ impl Service { .get(&node_id) .expect("Pageservers may not be deleted while referenced"); - result.push(TenantLocateResponseShard { - shard_id: *tenant_shard_id, - node_id, - listen_http_addr: node.listen_http_addr.clone(), - listen_http_port: node.listen_http_port, - listen_pg_addr: node.listen_pg_addr.clone(), - listen_pg_port: node.listen_pg_port, - }); + result.push(node.shard_location(*tenant_shard_id)); match &shard_params { None => { @@ -2324,7 +2289,7 @@ impl Service { // populate the correct generation as part of its transaction, to protect us // against racing with changes in the state of the parent. generation: None, - generation_pageserver: Some(target.node.id.0 as i64), + 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(), @@ -2526,10 +2491,10 @@ impl Service { ))); }; - if node.availability != NodeAvailability::Active { + if !node.is_available() { // Warn but proceed: the caller may intend to manually adjust the placement of // a shard even if the node is down, e.g. if intervening during an incident. - tracing::warn!("Migrating to an unavailable node ({})", node.id); + tracing::warn!("Migrating to unavailable node {node}"); } let Some(shard) = tenants.get_mut(&tenant_shard_id) else { @@ -2784,11 +2749,7 @@ impl Service { if let Some(node) = locked.nodes.get(®ister_req.node_id) { // Note that we do not do a total equality of the struct, because we don't require // the availability/scheduling states to agree for a POST to be idempotent. - if node.listen_http_addr == register_req.listen_http_addr - && node.listen_http_port == register_req.listen_http_port - && node.listen_pg_addr == register_req.listen_pg_addr - && node.listen_pg_port == register_req.listen_pg_port - { + if node.registration_match(®ister_req) { tracing::info!( "Node {} re-registered with matching address", register_req.node_id @@ -2812,16 +2773,14 @@ impl Service { // Ordering: we must persist the new node _before_ adding it to in-memory state. // This ensures that before we use it for anything or expose it via any external // API, it is guaranteed to be available after a restart. - let new_node = Node { - id: register_req.node_id, - listen_http_addr: register_req.listen_http_addr, - listen_http_port: register_req.listen_http_port, - listen_pg_addr: register_req.listen_pg_addr, - listen_pg_port: register_req.listen_pg_port, - scheduling: NodeSchedulingPolicy::Filling, - // TODO: we shouldn't really call this Active until we've heartbeated it. - availability: NodeAvailability::Active, - }; + let new_node = Node::new( + register_req.node_id, + register_req.listen_http_addr, + register_req.listen_http_port, + register_req.listen_pg_addr, + register_req.listen_pg_port, + ); + // TODO: idempotency if the node already exists in the database self.persistence.insert_node(&new_node).await?; @@ -2866,29 +2825,14 @@ impl Service { )); }; - let mut offline_transition = false; - let mut active_transition = false; - - if let Some(availability) = &config_req.availability { - match (availability, &node.availability) { - (NodeAvailability::Offline, NodeAvailability::Active) => { - tracing::info!("Node {} transition to offline", config_req.node_id); - offline_transition = true; - } - (NodeAvailability::Active, NodeAvailability::Offline) => { - tracing::info!("Node {} transition to active", config_req.node_id); - active_transition = true; - } - _ => { - tracing::info!("Node {} no change during config", config_req.node_id); - // No change - } - }; - node.availability = *availability; - } + let availability_transition = if let Some(availability) = &config_req.availability { + node.set_availability(*availability) + } else { + AvailabilityTransition::Unchanged + }; if let Some(scheduling) = config_req.scheduling { - node.scheduling = scheduling; + node.set_scheduling(scheduling); // TODO: once we have a background scheduling ticker for fill/drain, kick it // to wake up and start working. @@ -2899,74 +2843,80 @@ impl Service { let new_nodes = Arc::new(new_nodes); - if offline_transition { - let mut tenants_affected: usize = 0; - for (tenant_shard_id, tenant_state) in tenants { - if let Some(observed_loc) = - tenant_state.observed.locations.get_mut(&config_req.node_id) - { - // When a node goes offline, we set its observed configuration to None, indicating unknown: we will - // not assume our knowledge of the node's configuration is accurate until it comes back online - observed_loc.conf = None; - } + match availability_transition { + AvailabilityTransition::ToOffline => { + tracing::info!("Node {} transition to offline", config_req.node_id); + let mut tenants_affected: usize = 0; + for (tenant_shard_id, tenant_state) in tenants { + if let Some(observed_loc) = + tenant_state.observed.locations.get_mut(&config_req.node_id) + { + // When a node goes offline, we set its observed configuration to None, indicating unknown: we will + // not assume our knowledge of the node's configuration is accurate until it comes back online + observed_loc.conf = None; + } - if tenant_state.intent.demote_attached(config_req.node_id) { - tenant_state.sequence = tenant_state.sequence.next(); - match tenant_state.schedule(scheduler) { - Err(e) => { - // It is possible that some tenants will become unschedulable when too many pageservers - // go offline: in this case there isn't much we can do other than make the issue observable. - // TODO: give TenantState a scheduling error attribute to be queried later. - tracing::warn!(%tenant_shard_id, "Scheduling error when marking pageserver {} offline: {e}", config_req.node_id); - } - Ok(()) => { - if tenant_state - .maybe_reconcile( - result_tx.clone(), - &new_nodes, - &compute_hook, - &self.config, - &self.persistence, - &self.gate, - &self.cancel, - ) - .is_some() - { - tenants_affected += 1; - }; + if tenant_state.intent.demote_attached(config_req.node_id) { + tenant_state.sequence = tenant_state.sequence.next(); + match tenant_state.schedule(scheduler) { + Err(e) => { + // It is possible that some tenants will become unschedulable when too many pageservers + // go offline: in this case there isn't much we can do other than make the issue observable. + // TODO: give TenantState a scheduling error attribute to be queried later. + tracing::warn!(%tenant_shard_id, "Scheduling error when marking pageserver {} offline: {e}", config_req.node_id); + } + Ok(()) => { + if tenant_state + .maybe_reconcile( + result_tx.clone(), + &new_nodes, + &compute_hook, + &self.config, + &self.persistence, + &self.gate, + &self.cancel, + ) + .is_some() + { + tenants_affected += 1; + }; + } } } } + tracing::info!( + "Launched {} reconciler tasks for tenants affected by node {} going offline", + tenants_affected, + config_req.node_id + ) } - tracing::info!( - "Launched {} reconciler tasks for tenants affected by node {} going offline", - tenants_affected, - config_req.node_id - ) - } - - if active_transition { - // When a node comes back online, we must reconcile any tenant that has a None observed - // location on the node. - for tenant_state in locked.tenants.values_mut() { - if let Some(observed_loc) = - tenant_state.observed.locations.get_mut(&config_req.node_id) - { - if observed_loc.conf.is_none() { - tenant_state.maybe_reconcile( - result_tx.clone(), - &new_nodes, - &compute_hook, - &self.config, - &self.persistence, - &self.gate, - &self.cancel, - ); + AvailabilityTransition::ToActive => { + tracing::info!("Node {} transition to active", config_req.node_id); + // When a node comes back online, we must reconcile any tenant that has a None observed + // location on the node. + for tenant_state in locked.tenants.values_mut() { + if let Some(observed_loc) = + tenant_state.observed.locations.get_mut(&config_req.node_id) + { + if observed_loc.conf.is_none() { + tenant_state.maybe_reconcile( + result_tx.clone(), + &new_nodes, + &compute_hook, + &self.config, + &self.persistence, + &self.gate, + &self.cancel, + ); + } } } - } - // TODO: in the background, we should balance work back onto this pageserver + // TODO: in the background, we should balance work back onto this pageserver + } + AvailabilityTransition::Unchanged => { + tracing::info!("Node {} no change during config", config_req.node_id); + } } locked.nodes = new_nodes; diff --git a/control_plane/attachment_service/src/tenant_state.rs b/control_plane/attachment_service/src/tenant_state.rs index 33b7d578c7ca..ddb98665278d 100644 --- a/control_plane/attachment_service/src/tenant_state.rs +++ b/control_plane/attachment_service/src/tenant_state.rs @@ -1,7 +1,10 @@ -use std::{collections::HashMap, sync::Arc, time::Duration}; +use std::{ + collections::{HashMap, HashSet}, + sync::Arc, + time::Duration, +}; use crate::{metrics, persistence::TenantShardPersistence}; -use pageserver_api::controller_api::NodeAvailability; use pageserver_api::{ models::{LocationConfig, LocationConfigMode, TenantConfig}, shard::{ShardIdentity, TenantShardId}, @@ -370,7 +373,7 @@ impl TenantState { /// [`ObservedState`], even if it violates my [`PlacementPolicy`]. Call [`Self::schedule`] next, /// to get an intent state that complies with placement policy. The overall goal is to do scheduling /// in a way that makes use of any configured locations that already exist in the outside world. - pub(crate) fn intent_from_observed(&mut self) { + pub(crate) fn intent_from_observed(&mut self, scheduler: &mut Scheduler) { // Choose an attached location by filtering observed locations, and then sorting to get the highest // generation let mut attached_locs = self @@ -395,7 +398,7 @@ impl TenantState { attached_locs.sort_by_key(|i| i.1); if let Some((node_id, _gen)) = attached_locs.into_iter().last() { - self.intent.attached = Some(*node_id); + self.intent.set_attached(scheduler, Some(*node_id)); } // All remaining observed locations generate secondary intents. This includes None @@ -406,7 +409,7 @@ impl TenantState { // will take care of promoting one of these secondaries to be attached. self.observed.locations.keys().for_each(|node_id| { if Some(*node_id) != self.intent.attached { - self.intent.secondary.push(*node_id); + self.intent.push_secondary(scheduler, *node_id); } }); } @@ -564,7 +567,9 @@ impl TenantState { } } - fn dirty(&self) -> bool { + fn dirty(&self, nodes: &Arc>) -> bool { + let mut dirty_nodes = HashSet::new(); + if let Some(node_id) = self.intent.attached { // Maybe panic: it is a severe bug if we try to attach while generation is null. let generation = self @@ -575,7 +580,7 @@ impl TenantState { match self.observed.locations.get(&node_id) { Some(conf) if conf.conf.as_ref() == Some(&wanted_conf) => {} Some(_) | None => { - return true; + dirty_nodes.insert(node_id); } } } @@ -585,7 +590,7 @@ impl TenantState { match self.observed.locations.get(node_id) { Some(conf) if conf.conf.as_ref() == Some(&wanted_conf) => {} Some(_) | None => { - return true; + dirty_nodes.insert(*node_id); } } } @@ -593,17 +598,18 @@ impl TenantState { for node_id in self.observed.locations.keys() { if self.intent.attached != Some(*node_id) && !self.intent.secondary.contains(node_id) { // We have observed state that isn't part of our intent: need to clean it up. - return true; + dirty_nodes.insert(*node_id); } } - // Even if there is no pageserver work to be done, if we have a pending notification to computes, - // wake up a reconciler to send it. - if self.pending_compute_notification { - return true; - } + dirty_nodes.retain(|node_id| { + nodes + .get(node_id) + .map(|n| n.is_available()) + .unwrap_or(false) + }); - false + !dirty_nodes.is_empty() } #[allow(clippy::too_many_arguments)] @@ -625,15 +631,20 @@ impl TenantState { let node = pageservers .get(node_id) .expect("Nodes may not be removed while referenced"); - if observed_loc.conf.is_none() - && !matches!(node.availability, NodeAvailability::Offline) - { + if observed_loc.conf.is_none() && node.is_available() { dirty_observed = true; break; } } - if !self.dirty() && !dirty_observed { + let active_nodes_dirty = self.dirty(pageservers); + + // Even if there is no pageserver work to be done, if we have a pending notification to computes, + // wake up a reconciler to send it. + let do_reconcile = + active_nodes_dirty || dirty_observed || self.pending_compute_notification; + + if !do_reconcile { tracing::info!("Not dirty, no reconciliation needed."); return None; } @@ -663,6 +674,21 @@ impl TenantState { } } + // Build list of nodes from which the reconciler should detach + let mut detach = Vec::new(); + for node_id in self.observed.locations.keys() { + if self.intent.get_attached() != &Some(*node_id) + && !self.intent.secondary.contains(node_id) + { + detach.push( + pageservers + .get(node_id) + .expect("Intent references non-existent pageserver") + .clone(), + ) + } + } + // Reconcile in flight for a stale sequence? Our sequence's task will wait for it before // doing our sequence's work. let old_handle = self.reconciler.take(); @@ -677,14 +703,15 @@ impl TenantState { self.sequence = self.sequence.next(); let reconciler_cancel = cancel.child_token(); + let reconciler_intent = TargetState::from_intent(pageservers, &self.intent); let mut reconciler = Reconciler { tenant_shard_id: self.tenant_shard_id, shard: self.shard, generation: self.generation, - intent: TargetState::from_intent(&self.intent), + intent: reconciler_intent, + detach, config: self.config.clone(), observed: self.observed.clone(), - pageservers: pageservers.clone(), compute_hook: compute_hook.clone(), service_config: service_config.clone(), _gate_guard: gate_guard, @@ -819,7 +846,10 @@ impl TenantState { #[cfg(test)] pub(crate) mod tests { - use pageserver_api::shard::{ShardCount, ShardNumber}; + use pageserver_api::{ + controller_api::NodeAvailability, + shard::{ShardCount, ShardNumber}, + }; use utils::id::TenantId; use crate::scheduler::test_utils::make_test_nodes; @@ -878,7 +908,10 @@ pub(crate) mod tests { assert_eq!(tenant_state.intent.secondary.len(), 2); // Update the scheduler state to indicate the node is offline - nodes.get_mut(&attached_node_id).unwrap().availability = NodeAvailability::Offline; + nodes + .get_mut(&attached_node_id) + .unwrap() + .set_availability(NodeAvailability::Offline); scheduler.node_upsert(nodes.get(&attached_node_id).unwrap()); // Scheduling the node should promote the still-available secondary node to attached @@ -897,4 +930,54 @@ pub(crate) mod tests { Ok(()) } + + #[test] + fn intent_from_observed() -> anyhow::Result<()> { + let nodes = make_test_nodes(3); + let mut scheduler = Scheduler::new(nodes.values()); + + let mut tenant_state = make_test_tenant_shard(PlacementPolicy::Double(1)); + + tenant_state.observed.locations.insert( + NodeId(3), + ObservedStateLocation { + conf: Some(LocationConfig { + mode: LocationConfigMode::AttachedMulti, + generation: Some(2), + secondary_conf: None, + shard_number: tenant_state.shard.number.0, + shard_count: tenant_state.shard.count.literal(), + shard_stripe_size: tenant_state.shard.stripe_size.0, + tenant_conf: TenantConfig::default(), + }), + }, + ); + + tenant_state.observed.locations.insert( + NodeId(2), + ObservedStateLocation { + conf: Some(LocationConfig { + mode: LocationConfigMode::AttachedStale, + generation: Some(1), + secondary_conf: None, + shard_number: tenant_state.shard.number.0, + shard_count: tenant_state.shard.count.literal(), + shard_stripe_size: tenant_state.shard.stripe_size.0, + tenant_conf: TenantConfig::default(), + }), + }, + ); + + tenant_state.intent_from_observed(&mut scheduler); + + // The highest generationed attached location gets used as attached + assert_eq!(tenant_state.intent.attached, Some(NodeId(3))); + // Other locations get used as secondary + assert_eq!(tenant_state.intent.secondary, vec![NodeId(2)]); + + scheduler.consistency_check(nodes.values(), [&tenant_state].into_iter())?; + + tenant_state.intent.clear(&mut scheduler); + Ok(()) + } } diff --git a/pageserver/client/src/mgmt_api.rs b/pageserver/client/src/mgmt_api.rs index 4dde7bdf0baf..732eb951c9fd 100644 --- a/pageserver/client/src/mgmt_api.rs +++ b/pageserver/client/src/mgmt_api.rs @@ -7,7 +7,7 @@ use utils::{ pub mod util; -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct Client { mgmt_api_endpoint: String, authorization_header: Option, @@ -24,6 +24,9 @@ pub enum Error { #[error("pageserver API: {1}")] ApiError(StatusCode, String), + + #[error("Cancelled")] + Cancelled, } pub type Result = std::result::Result; @@ -287,6 +290,21 @@ impl Client { .map_err(Error::ReceiveBody) } + pub async fn get_location_config( + &self, + tenant_shard_id: TenantShardId, + ) -> Result> { + let path = format!( + "{}/v1/location_config/{tenant_shard_id}", + self.mgmt_api_endpoint + ); + self.request(Method::GET, &path, ()) + .await? + .json() + .await + .map_err(Error::ReceiveBody) + } + pub async fn timeline_create( &self, tenant_shard_id: TenantShardId, diff --git a/pageserver/src/http/routes.rs b/pageserver/src/http/routes.rs index 6aaf1ab27e46..eafad9ab7383 100644 --- a/pageserver/src/http/routes.rs +++ b/pageserver/src/http/routes.rs @@ -14,6 +14,7 @@ use hyper::header; use hyper::StatusCode; use hyper::{Body, Request, Response, Uri}; use metrics::launch_timestamp::LaunchTimestamp; +use pageserver_api::models::LocationConfig; use pageserver_api::models::LocationConfigListResponse; use pageserver_api::models::ShardParameters; use pageserver_api::models::TenantDetails; @@ -1519,6 +1520,29 @@ async fn list_location_config_handler( json_response(StatusCode::OK, result) } +async fn get_location_config_handler( + request: Request, + _cancel: CancellationToken, +) -> Result, ApiError> { + let state = get_state(&request); + let tenant_shard_id: TenantShardId = parse_request_param(&request, "tenant_shard_id")?; + let slot = state.tenant_manager.get(tenant_shard_id); + + let Some(slot) = slot else { + return Err(ApiError::NotFound( + anyhow::anyhow!("Tenant shard not found").into(), + )); + }; + + let result: Option = match slot { + TenantSlot::Attached(t) => Some(t.get_location_conf()), + TenantSlot::Secondary(s) => Some(s.get_location_conf()), + TenantSlot::InProgress(_) => None, + }; + + json_response(StatusCode::OK, result) +} + // Do a time travel recovery on the given tenant/tenant shard. Tenant needs to be detached // (from all pageservers) as it invalidates consistency assumptions. async fn tenant_time_travel_remote_storage_handler( @@ -2223,6 +2247,9 @@ pub fn make_router( .get("/v1/location_config", |r| { api_handler(r, list_location_config_handler) }) + .get("/v1/location_config/:tenant_id", |r| { + api_handler(r, get_location_config_handler) + }) .put( "/v1/tenant/:tenant_shard_id/time_travel_remote_storage", |r| api_handler(r, tenant_time_travel_remote_storage_handler), diff --git a/pageserver/src/tenant/mgr.rs b/pageserver/src/tenant/mgr.rs index 06b61d4631a5..fc08b3c82e27 100644 --- a/pageserver/src/tenant/mgr.rs +++ b/pageserver/src/tenant/mgr.rs @@ -1358,6 +1358,16 @@ impl TenantManager { } } + pub(crate) fn get(&self, tenant_shard_id: TenantShardId) -> Option { + let locked = self.tenants.read().unwrap(); + match &*locked { + TenantsMap::Initializing => None, + TenantsMap::Open(map) | TenantsMap::ShuttingDown(map) => { + map.get(&tenant_shard_id).cloned() + } + } + } + pub(crate) async fn delete_tenant( &self, tenant_shard_id: TenantShardId, From ce7a82db058cecdba996a210b5afea8451bfbc4a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Arpad=20M=C3=BCller?= Date: Thu, 7 Mar 2024 18:32:09 +0100 Subject: [PATCH 05/43] Update svg_fmt (#7049) Gets upstream PR https://github.com/nical/rust_debug/pull/3 , removes trailing "s from output. --- Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 167a2b21796c..5c48942d41aa 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5525,9 +5525,9 @@ checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" [[package]] name = "svg_fmt" -version = "0.4.1" +version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8fb1df15f412ee2e9dfc1c504260fa695c1c3f10fe9f4a6ee2d2184d7d6450e2" +checksum = "f83ba502a3265efb76efb89b0a2f7782ad6f2675015d4ce37e4b547dda42b499" [[package]] name = "syn" From 2fc89428c33508bee9fa5772c0c5c35ba3e38548 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Thu, 7 Mar 2024 09:12:06 -0900 Subject: [PATCH 06/43] Hopefully stabilize test_bad_connection.py (#6976) ## Problem It seems that even though we have a retry on basebackup, it still sometimes fails to fetch it with the failpoint enabled, resulting in a test error. ## Summary of changes If we fail to get the basebackup, disable the failpoint and try again. --- compute_tools/src/compute.rs | 8 ++++---- control_plane/src/endpoint.rs | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/compute_tools/src/compute.rs b/compute_tools/src/compute.rs index 5613e6c8681d..96ab4a06a58e 100644 --- a/compute_tools/src/compute.rs +++ b/compute_tools/src/compute.rs @@ -396,9 +396,9 @@ impl ComputeNode { // Gets the basebackup in a retry loop #[instrument(skip_all, fields(%lsn))] pub fn get_basebackup(&self, compute_state: &ComputeState, lsn: Lsn) -> Result<()> { - let mut retry_period_ms = 500; + let mut retry_period_ms = 500.0; let mut attempts = 0; - let max_attempts = 5; + let max_attempts = 10; loop { let result = self.try_get_basebackup(compute_state, lsn); match result { @@ -410,8 +410,8 @@ impl ComputeNode { "Failed to get basebackup: {} (attempt {}/{})", e, attempts, max_attempts ); - std::thread::sleep(std::time::Duration::from_millis(retry_period_ms)); - retry_period_ms *= 2; + std::thread::sleep(std::time::Duration::from_millis(retry_period_ms as u64)); + retry_period_ms *= 1.5; } Err(_) => { return result; diff --git a/control_plane/src/endpoint.rs b/control_plane/src/endpoint.rs index 10e4c5d69fec..ac0a8417aec9 100644 --- a/control_plane/src/endpoint.rs +++ b/control_plane/src/endpoint.rs @@ -656,7 +656,7 @@ impl Endpoint { // Wait for it to start let mut attempt = 0; const ATTEMPT_INTERVAL: Duration = Duration::from_millis(100); - const MAX_ATTEMPTS: u32 = 10 * 30; // Wait up to 30 s + const MAX_ATTEMPTS: u32 = 10 * 90; // Wait up to 1.5 min loop { attempt += 1; match self.get_status().await { From 02358b21a41311be2ee610bd461093a68b14222e Mon Sep 17 00:00:00 2001 From: Conrad Ludgate Date: Thu, 7 Mar 2024 18:23:19 +0000 Subject: [PATCH 07/43] update rustls (#7048) ## Summary of changes Update rustls from 0.21 to 0.22. reqwest/tonic/aws-smithy still use rustls 0.21. no upgrade route available yet. --- Cargo.lock | 293 +++++++++++++------ Cargo.toml | 10 +- libs/postgres_backend/tests/simple_select.rs | 19 +- proxy/src/bin/pg_sni_router.rs | 38 +-- proxy/src/config.rs | 54 ++-- proxy/src/proxy/tests.rs | 19 +- workspace_hack/Cargo.toml | 2 +- 7 files changed, 281 insertions(+), 154 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 5c48942d41aa..7fd9053f627c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -241,7 +241,7 @@ checksum = "16e62a023e7c117e27523144c5d2459f4397fcc3cab0085af8e2224f643a0193" dependencies = [ "proc-macro2", "quote", - "syn 2.0.32", + "syn 2.0.52", ] [[package]] @@ -252,7 +252,7 @@ checksum = "b9ccdd8f2a161be9bd5c023df56f1b2a0bd1d83872ae53b71a84a12c9bf6e842" dependencies = [ "proc-macro2", "quote", - "syn 2.0.32", + "syn 2.0.52", ] [[package]] @@ -626,7 +626,7 @@ dependencies = [ "once_cell", "pin-project-lite", "pin-utils", - "rustls", + "rustls 0.21.9", "tokio", "tracing", ] @@ -907,6 +907,16 @@ version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8c3c1a368f70d6cf7302d78f8f7093da241fb8e8807c05cc9e51a125895a6d5b" +[[package]] +name = "bcder" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c627747a6774aab38beb35990d88309481378558875a41da1a4b2e373c906ef0" +dependencies = [ + "bytes", + "smallvec", +] + [[package]] name = "bincode" version = "1.3.3" @@ -935,7 +945,7 @@ dependencies = [ "regex", "rustc-hash", "shlex", - "syn 2.0.32", + "syn 2.0.52", "which", ] @@ -986,9 +996,9 @@ checksum = "14c189c53d098945499cdfa7ecc63567cf3886b3332b312a5b4585d8d3a6a610" [[package]] name = "bytes" -version = "1.4.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89b2fd2a0dcf38d7971e2194b6b6eebab45ae01067456a7fd93d5547a61b70be" +checksum = "a2bd12c1caf447e69cd4528f47f94d203fd2582878ecb9e9465484c4148a8223" dependencies = [ "serde", ] @@ -1149,7 +1159,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.32", + "syn 2.0.52", ] [[package]] @@ -1574,7 +1584,7 @@ dependencies = [ "proc-macro2", "quote", "strsim", - "syn 2.0.32", + "syn 2.0.52", ] [[package]] @@ -1585,7 +1595,7 @@ checksum = "29a358ff9f12ec09c3e61fef9b5a9902623a695a46a917b07f269bff1445611a" dependencies = [ "darling_core", "quote", - "syn 2.0.32", + "syn 2.0.52", ] [[package]] @@ -1627,6 +1637,16 @@ dependencies = [ "zeroize", ] +[[package]] +name = "der" +version = "0.7.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fffa369a668c8af7dbf8b5e56c9f744fbd399949ed171606040001947de40b1c" +dependencies = [ + "const-oid", + "zeroize", +] + [[package]] name = "der-parser" version = "8.2.0" @@ -1681,7 +1701,7 @@ dependencies = [ "diesel_table_macro_syntax", "proc-macro2", "quote", - "syn 2.0.32", + "syn 2.0.52", ] [[package]] @@ -1701,7 +1721,7 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fc5557efc453706fed5e4fa85006fe9817c224c3f480a34c7e5959fd700921c5" dependencies = [ - "syn 2.0.32", + "syn 2.0.52", ] [[package]] @@ -1723,7 +1743,7 @@ checksum = "487585f4d0c6655fe74905e2504d8ad6908e4db67f744eb140876906c2f3175d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.32", + "syn 2.0.52", ] [[package]] @@ -1747,10 +1767,10 @@ version = "0.14.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "413301934810f597c1d19ca71c8710e99a3f1ba28a0d2ebc01551a2daeea3c5c" dependencies = [ - "der", + "der 0.6.1", "elliptic-curve", "rfc6979", - "signature", + "signature 1.6.4", ] [[package]] @@ -1767,7 +1787,7 @@ checksum = "e7bb888ab5300a19b8e5bceef25ac745ad065f3c9f7efc6de1b91958110891d3" dependencies = [ "base16ct", "crypto-bigint 0.4.9", - "der", + "der 0.6.1", "digest", "ff", "generic-array", @@ -1827,7 +1847,7 @@ dependencies = [ "darling", "proc-macro2", "quote", - "syn 2.0.32", + "syn 2.0.52", ] [[package]] @@ -2087,7 +2107,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.32", + "syn 2.0.52", ] [[package]] @@ -2470,10 +2490,10 @@ dependencies = [ "http 0.2.9", "hyper", "log", - "rustls", + "rustls 0.21.9", "rustls-native-certs", "tokio", - "tokio-rustls", + "tokio-rustls 0.24.0", ] [[package]] @@ -2711,7 +2731,7 @@ checksum = "5c7ea04a7c5c055c175f189b6dc6ba036fd62306b58c66c9f6389036c503a3f4" dependencies = [ "base64 0.21.1", "js-sys", - "pem 3.0.3", + "pem", "ring 0.17.6", "serde", "serde_json", @@ -3234,7 +3254,7 @@ checksum = "a948666b637a0f465e8564c73e89d4dde00d72d4d473cc972f390fc3dcee7d9c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.32", + "syn 2.0.52", ] [[package]] @@ -3716,7 +3736,7 @@ dependencies = [ "parquet", "proc-macro2", "quote", - "syn 2.0.32", + "syn 2.0.52", ] [[package]] @@ -3754,16 +3774,6 @@ version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "19b17cddbe7ec3f8bc800887bab5e717348c95ea2ca0b1bf0837fb964dc67099" -[[package]] -name = "pem" -version = "2.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b13fe415cdf3c8e44518e18a7c95a13431d9bdf6d15367d82b23c377fdd441a" -dependencies = [ - "base64 0.21.1", - "serde", -] - [[package]] name = "pem" version = "3.0.3" @@ -3825,7 +3835,7 @@ checksum = "39407670928234ebc5e6e580247dd567ad73a3578460c5990f9503df207e8f07" dependencies = [ "proc-macro2", "quote", - "syn 2.0.32", + "syn 2.0.52", ] [[package]] @@ -3846,8 +3856,8 @@ version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9eca2c590a5f85da82668fa685c09ce2888b9430e83299debf1f34b65fd4a4ba" dependencies = [ - "der", - "spki", + "der 0.6.1", + "spki 0.6.0", ] [[package]] @@ -3946,14 +3956,14 @@ dependencies = [ "futures", "once_cell", "pq_proto", - "rustls", - "rustls-pemfile", + "rustls 0.22.2", + "rustls-pemfile 2.1.1", "serde", "thiserror", "tokio", "tokio-postgres", "tokio-postgres-rustls", - "tokio-rustls", + "tokio-rustls 0.25.0", "tracing", "workspace_hack", ] @@ -4042,7 +4052,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3b69d39aab54d069e7f2fe8cb970493e7834601ca2d8c65fd7bbd183578080d1" dependencies = [ "proc-macro2", - "syn 2.0.32", + "syn 2.0.52", ] [[package]] @@ -4053,9 +4063,9 @@ checksum = "dc375e1527247fe1a97d8b7156678dfe7c1af2fc075c9a4db3690ecd2a148068" [[package]] name = "proc-macro2" -version = "1.0.66" +version = "1.0.78" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "18fb31db3f9bddb2ea821cde30a9f70117e3f119938b5ee630b7403aa6e2ead9" +checksum = "e2422ad645d89c99f8f3e6b88a9fdeca7fabeac836b1002371c4367c8f984aae" dependencies = [ "unicode-ident", ] @@ -4202,8 +4212,8 @@ dependencies = [ "routerify", "rstest", "rustc-hash", - "rustls", - "rustls-pemfile", + "rustls 0.22.2", + "rustls-pemfile 2.1.1", "scopeguard", "serde", "serde_json", @@ -4219,7 +4229,7 @@ dependencies = [ "tokio", "tokio-postgres", "tokio-postgres-rustls", - "tokio-rustls", + "tokio-rustls 0.25.0", "tokio-util", "tracing", "tracing-opentelemetry", @@ -4247,9 +4257,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.32" +version = "1.0.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "50f3b39ccfb720540debaa0164757101c08ecb8d326b15358ce76a62c7e85965" +checksum = "291ec9ab5efd934aaf503a6466c5d5251535d108ee747472c3977cc5acc868ef" dependencies = [ "proc-macro2", ] @@ -4370,12 +4380,12 @@ dependencies = [ [[package]] name = "rcgen" -version = "0.11.1" +version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4954fbc00dcd4d8282c987710e50ba513d351400dbdd00e803a05172a90d8976" +checksum = "48406db8ac1f3cbc7dcdb56ec355343817958a356ff430259bb07baf7607e1e1" dependencies = [ - "pem 2.0.1", - "ring 0.16.20", + "pem", + "ring 0.17.6", "time", "yasna", ] @@ -4393,15 +4403,15 @@ dependencies = [ "itoa", "percent-encoding", "pin-project-lite", - "rustls", + "rustls 0.21.9", "rustls-native-certs", - "rustls-pemfile", + "rustls-pemfile 1.0.2", "rustls-webpki 0.101.7", "ryu", "sha1_smol", "socket2 0.4.9", "tokio", - "tokio-rustls", + "tokio-rustls 0.24.0", "tokio-util", "url", ] @@ -4547,14 +4557,14 @@ dependencies = [ "once_cell", "percent-encoding", "pin-project-lite", - "rustls", - "rustls-pemfile", + "rustls 0.21.9", + "rustls-pemfile 1.0.2", "serde", "serde_json", "serde_urlencoded", "tokio", "tokio-native-tls", - "tokio-rustls", + "tokio-rustls 0.24.0", "tokio-util", "tower-service", "url", @@ -4720,7 +4730,7 @@ dependencies = [ "regex", "relative-path", "rustc_version", - "syn 2.0.32", + "syn 2.0.52", "unicode-ident", ] @@ -4804,6 +4814,20 @@ dependencies = [ "sct", ] +[[package]] +name = "rustls" +version = "0.22.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e87c9956bd9807afa1f77e0f7594af32566e830e088a5576d27c5b6f30f49d41" +dependencies = [ + "log", + "ring 0.17.6", + "rustls-pki-types", + "rustls-webpki 0.102.2", + "subtle", + "zeroize", +] + [[package]] name = "rustls-native-certs" version = "0.6.2" @@ -4811,7 +4835,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0167bac7a9f490495f3c33013e7722b53cb087ecbe082fb0c6387c96f634ea50" dependencies = [ "openssl-probe", - "rustls-pemfile", + "rustls-pemfile 1.0.2", "schannel", "security-framework", ] @@ -4825,6 +4849,22 @@ dependencies = [ "base64 0.21.1", ] +[[package]] +name = "rustls-pemfile" +version = "2.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f48172685e6ff52a556baa527774f61fcaa884f59daf3375c62a3f1cd2549dab" +dependencies = [ + "base64 0.21.1", + "rustls-pki-types", +] + +[[package]] +name = "rustls-pki-types" +version = "1.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5ede67b28608b4c60685c7d54122d4400d90f62b40caee7700e700380a390fa8" + [[package]] name = "rustls-webpki" version = "0.100.2" @@ -4845,6 +4885,17 @@ dependencies = [ "untrusted 0.9.0", ] +[[package]] +name = "rustls-webpki" +version = "0.102.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "faaa0a62740bedb9b2ef5afa303da42764c012f743917351dc9a237ea1663610" +dependencies = [ + "ring 0.17.6", + "rustls-pki-types", + "untrusted 0.9.0", +] + [[package]] name = "rustversion" version = "1.0.12" @@ -4887,7 +4938,7 @@ dependencies = [ "serde_with", "thiserror", "tokio", - "tokio-rustls", + "tokio-rustls 0.25.0", "tokio-stream", "tracing", "tracing-appender", @@ -5022,7 +5073,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3be24c1842290c45df0a7bf069e0c268a747ad05a192f2fd7dcfdbc1cba40928" dependencies = [ "base16ct", - "der", + "der 0.6.1", "generic-array", "pkcs8", "subtle", @@ -5066,7 +5117,7 @@ checksum = "2e95efd0cefa32028cdb9766c96de71d96671072f9fb494dc9fb84c0ef93e52b" dependencies = [ "httpdate", "reqwest", - "rustls", + "rustls 0.21.9", "sentry-backtrace", "sentry-contexts", "sentry-core", @@ -5188,7 +5239,7 @@ checksum = "aafe972d60b0b9bee71a91b92fee2d4fb3c9d7e8f6b179aa99f27203d99a4816" dependencies = [ "proc-macro2", "quote", - "syn 2.0.32", + "syn 2.0.52", ] [[package]] @@ -5269,7 +5320,7 @@ dependencies = [ "darling", "proc-macro2", "quote", - "syn 2.0.32", + "syn 2.0.52", ] [[package]] @@ -5355,6 +5406,15 @@ dependencies = [ "rand_core 0.6.4", ] +[[package]] +name = "signature" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77549399552de45a898a580c1b41d445bf730df867cc44e6c0233bbc4b8329de" +dependencies = [ + "rand_core 0.6.4", +] + [[package]] name = "simple_asn1" version = "0.6.2" @@ -5439,7 +5499,17 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "67cf02bbac7a337dc36e4f5a693db6c21e7863f45070f7064577eb4367a3212b" dependencies = [ "base64ct", - "der", + "der 0.6.1", +] + +[[package]] +name = "spki" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d91ed6c858b01f942cd56b37a94b3e0a1798290327d1236e4d9cf4eaca44d29d" +dependencies = [ + "base64ct", + "der 0.7.8", ] [[package]] @@ -5542,9 +5612,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.32" +version = "2.0.52" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "239814284fd6f1a4ffe4ca893952cdd93c224b6a1571c9a9eadd670295c0c9e2" +checksum = "b699d15b36d1f02c3e7c69f8ffef53de37aefae075d8488d4ba1a7788d574a07" dependencies = [ "proc-macro2", "quote", @@ -5659,22 +5729,22 @@ dependencies = [ [[package]] name = "thiserror" -version = "1.0.47" +version = "1.0.57" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97a802ec30afc17eee47b2855fc72e0c4cd62be9b4efe6591edde0ec5bd68d8f" +checksum = "1e45bcbe8ed29775f228095caf2cd67af7a4ccf756ebff23a306bf3e8b47b24b" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.47" +version = "1.0.57" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6bb623b56e39ab7dcd4b1b98bb6c8f8d907ed255b18de254088016b27a8ee19b" +checksum = "a953cb265bef375dae3de6663da4d3804eee9682ea80d8e2542529b73c531c81" dependencies = [ "proc-macro2", "quote", - "syn 2.0.32", + "syn 2.0.52", ] [[package]] @@ -5845,7 +5915,7 @@ checksum = "5b8a1e28f2deaa14e508979454cb3a223b10b938b45af148bc0986de36f1923b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.32", + "syn 2.0.52", ] [[package]] @@ -5883,16 +5953,17 @@ dependencies = [ [[package]] name = "tokio-postgres-rustls" -version = "0.10.0" +version = "0.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd5831152cb0d3f79ef5523b357319ba154795d64c7078b2daa95a803b54057f" +checksum = "0ea13f22eda7127c827983bdaf0d7fff9df21c8817bab02815ac277a21143677" dependencies = [ "futures", - "ring 0.16.20", - "rustls", + "ring 0.17.6", + "rustls 0.22.2", "tokio", "tokio-postgres", - "tokio-rustls", + "tokio-rustls 0.25.0", + "x509-certificate", ] [[package]] @@ -5901,7 +5972,18 @@ version = "0.24.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e0d409377ff5b1e3ca6437aa86c1eb7d40c134bfec254e44c830defa92669db5" dependencies = [ - "rustls", + "rustls 0.21.9", + "tokio", +] + +[[package]] +name = "tokio-rustls" +version = "0.25.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "775e0c0f0adb3a2f22a00c4745d728b479985fc15ee7ca6a2608388c5569860f" +dependencies = [ + "rustls 0.22.2", + "rustls-pki-types", "tokio", ] @@ -6016,9 +6098,9 @@ dependencies = [ "pin-project", "prost", "rustls-native-certs", - "rustls-pemfile", + "rustls-pemfile 1.0.2", "tokio", - "tokio-rustls", + "tokio-rustls 0.24.0", "tokio-stream", "tower", "tower-layer", @@ -6114,7 +6196,7 @@ checksum = "0f57e3ca2a01450b1a921183a9c9cbfda207fd822cef4ccb00a65402cbba7a74" dependencies = [ "proc-macro2", "quote", - "syn 2.0.32", + "syn 2.0.52", ] [[package]] @@ -6330,7 +6412,7 @@ dependencies = [ "base64 0.21.1", "log", "once_cell", - "rustls", + "rustls 0.21.9", "rustls-webpki 0.100.2", "url", "webpki-roots 0.23.1", @@ -6572,7 +6654,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.32", + "syn 2.0.52", "wasm-bindgen-shared", ] @@ -6606,7 +6688,7 @@ checksum = "e128beba882dd1eb6200e1dc92ae6c5dbaa4311aa7bb211ca035779e5efc39f8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.32", + "syn 2.0.52", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -6939,19 +7021,18 @@ dependencies = [ "regex-automata 0.4.3", "regex-syntax 0.8.2", "reqwest", - "ring 0.16.20", - "rustls", + "rustls 0.21.9", "scopeguard", "serde", "serde_json", "smallvec", "subtle", "syn 1.0.109", - "syn 2.0.32", + "syn 2.0.52", "time", "time-macros", "tokio", - "tokio-rustls", + "tokio-rustls 0.24.0", "tokio-util", "toml_datetime", "toml_edit", @@ -6962,11 +7043,31 @@ dependencies = [ "tungstenite", "url", "uuid", + "zeroize", "zstd", "zstd-safe", "zstd-sys", ] +[[package]] +name = "x509-certificate" +version = "0.23.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "66534846dec7a11d7c50a74b7cdb208b9a581cad890b7866430d438455847c85" +dependencies = [ + "bcder", + "bytes", + "chrono", + "der 0.7.8", + "hex", + "pem", + "ring 0.17.6", + "signature 2.2.0", + "spki 0.7.3", + "thiserror", + "zeroize", +] + [[package]] name = "x509-parser" version = "0.15.0" @@ -7025,7 +7126,7 @@ checksum = "b3c129550b3e6de3fd0ba67ba5c81818f9805e58b8d7fee80a3a59d2c9fc601a" dependencies = [ "proc-macro2", "quote", - "syn 2.0.32", + "syn 2.0.52", ] [[package]] @@ -7033,6 +7134,20 @@ name = "zeroize" version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2a0956f1ba7c7909bfb66c2e9e4124ab6f6482560f6628b5aaeba39207c9aad9" +dependencies = [ + "zeroize_derive", +] + +[[package]] +name = "zeroize_derive" +version = "1.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ce36e65b0d2999d2aafac989fb249189a141aee1f53c612c1f37d72631959f69" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.52", +] [[package]] name = "zstd" diff --git a/Cargo.toml b/Cargo.toml index 42deaac19b34..76f4ff041c09 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -129,8 +129,8 @@ reqwest-retry = "0.2.2" routerify = "3" rpds = "0.13" rustc-hash = "1.1.0" -rustls = "0.21" -rustls-pemfile = "1" +rustls = "0.22" +rustls-pemfile = "2" rustls-split = "0.3" scopeguard = "1.1" sysinfo = "0.29.2" @@ -159,8 +159,8 @@ tikv-jemalloc-ctl = "0.5" tokio = { version = "1.17", features = ["macros"] } tokio-epoll-uring = { git = "https://github.com/neondatabase/tokio-epoll-uring.git" , branch = "main" } tokio-io-timeout = "1.2.0" -tokio-postgres-rustls = "0.10.0" -tokio-rustls = "0.24" +tokio-postgres-rustls = "0.11.0" +tokio-rustls = "0.25" tokio-stream = "0.1" tokio-tar = "0.3" tokio-util = { version = "0.7.10", features = ["io", "rt"] } @@ -219,7 +219,7 @@ workspace_hack = { version = "0.1", path = "./workspace_hack/" } ## Build dependencies criterion = "0.5.1" -rcgen = "0.11" +rcgen = "0.12" rstest = "0.18" camino-tempfile = "1.0.2" tonic-build = "0.9" diff --git a/libs/postgres_backend/tests/simple_select.rs b/libs/postgres_backend/tests/simple_select.rs index e046fa526060..80df9db858a9 100644 --- a/libs/postgres_backend/tests/simple_select.rs +++ b/libs/postgres_backend/tests/simple_select.rs @@ -72,14 +72,19 @@ async fn simple_select() { } } -static KEY: Lazy = Lazy::new(|| { +static KEY: Lazy> = Lazy::new(|| { let mut cursor = Cursor::new(include_bytes!("key.pem")); - rustls::PrivateKey(rustls_pemfile::rsa_private_keys(&mut cursor).unwrap()[0].clone()) + let key = rustls_pemfile::rsa_private_keys(&mut cursor) + .next() + .unwrap() + .unwrap(); + rustls::pki_types::PrivateKeyDer::Pkcs1(key) }); -static CERT: Lazy = Lazy::new(|| { +static CERT: Lazy> = Lazy::new(|| { let mut cursor = Cursor::new(include_bytes!("cert.pem")); - rustls::Certificate(rustls_pemfile::certs(&mut cursor).unwrap()[0].clone()) + let cert = rustls_pemfile::certs(&mut cursor).next().unwrap().unwrap(); + cert }); // test that basic select with ssl works @@ -88,9 +93,8 @@ async fn simple_select_ssl() { let (client_sock, server_sock) = make_tcp_pair().await; let server_cfg = rustls::ServerConfig::builder() - .with_safe_defaults() .with_no_client_auth() - .with_single_cert(vec![CERT.clone()], KEY.clone()) + .with_single_cert(vec![CERT.clone()], KEY.clone_key()) .unwrap(); let tls_config = Some(Arc::new(server_cfg)); let pgbackend = @@ -102,10 +106,9 @@ async fn simple_select_ssl() { }); let client_cfg = rustls::ClientConfig::builder() - .with_safe_defaults() .with_root_certificates({ let mut store = rustls::RootCertStore::empty(); - store.add(&CERT).unwrap(); + store.add(CERT.clone()).unwrap(); store }) .with_no_client_auth(); diff --git a/proxy/src/bin/pg_sni_router.rs b/proxy/src/bin/pg_sni_router.rs index d5ab66d6aafe..385f7820cbff 100644 --- a/proxy/src/bin/pg_sni_router.rs +++ b/proxy/src/bin/pg_sni_router.rs @@ -10,6 +10,7 @@ use itertools::Itertools; use proxy::config::TlsServerEndPoint; use proxy::context::RequestMonitoring; use proxy::proxy::run_until_cancelled; +use rustls::pki_types::PrivateKeyDer; use tokio::net::TcpListener; use anyhow::{anyhow, bail, ensure, Context}; @@ -76,37 +77,40 @@ async fn main() -> anyhow::Result<()> { (Some(key_path), Some(cert_path)) => { let key = { let key_bytes = std::fs::read(key_path).context("TLS key file")?; - let mut keys = rustls_pemfile::pkcs8_private_keys(&mut &key_bytes[..]) - .context(format!("Failed to read TLS keys at '{key_path}'"))?; + + let mut keys = + rustls_pemfile::pkcs8_private_keys(&mut &key_bytes[..]).collect_vec(); ensure!(keys.len() == 1, "keys.len() = {} (should be 1)", keys.len()); - keys.pop().map(rustls::PrivateKey).unwrap() + PrivateKeyDer::Pkcs8( + keys.pop() + .unwrap() + .context(format!("Failed to read TLS keys at '{key_path}'"))?, + ) }; let cert_chain_bytes = std::fs::read(cert_path) .context(format!("Failed to read TLS cert file at '{cert_path}.'"))?; - let cert_chain = { + let cert_chain: Vec<_> = { rustls_pemfile::certs(&mut &cert_chain_bytes[..]) - .context(format!( - "Failed to read TLS certificate chain from bytes from file at '{cert_path}'." - ))? - .into_iter() - .map(rustls::Certificate) - .collect_vec() + .try_collect() + .with_context(|| { + format!("Failed to read TLS certificate chain from bytes from file at '{cert_path}'.") + })? }; // needed for channel bindings let first_cert = cert_chain.first().context("missing certificate")?; let tls_server_end_point = TlsServerEndPoint::new(first_cert)?; - let tls_config = rustls::ServerConfig::builder() - .with_safe_default_cipher_suites() - .with_safe_default_kx_groups() - .with_protocol_versions(&[&rustls::version::TLS13, &rustls::version::TLS12])? - .with_no_client_auth() - .with_single_cert(cert_chain, key)? - .into(); + let tls_config = rustls::ServerConfig::builder_with_protocol_versions(&[ + &rustls::version::TLS13, + &rustls::version::TLS12, + ]) + .with_no_client_auth() + .with_single_cert(cert_chain, key)? + .into(); (tls_config, tls_server_end_point) } diff --git a/proxy/src/config.rs b/proxy/src/config.rs index 9f276c3c247f..437ec9f401ed 100644 --- a/proxy/src/config.rs +++ b/proxy/src/config.rs @@ -1,6 +1,10 @@ use crate::{auth, rate_limiter::RateBucketInfo, serverless::GlobalConnPoolOptions}; use anyhow::{bail, ensure, Context, Ok}; -use rustls::{sign, Certificate, PrivateKey}; +use itertools::Itertools; +use rustls::{ + crypto::ring::sign, + pki_types::{CertificateDer, PrivateKeyDer}, +}; use sha2::{Digest, Sha256}; use std::{ collections::{HashMap, HashSet}, @@ -88,14 +92,14 @@ pub fn configure_tls( let cert_resolver = Arc::new(cert_resolver); - let config = rustls::ServerConfig::builder() - .with_safe_default_cipher_suites() - .with_safe_default_kx_groups() - // allow TLS 1.2 to be compatible with older client libraries - .with_protocol_versions(&[&rustls::version::TLS13, &rustls::version::TLS12])? - .with_no_client_auth() - .with_cert_resolver(cert_resolver.clone()) - .into(); + // allow TLS 1.2 to be compatible with older client libraries + let config = rustls::ServerConfig::builder_with_protocol_versions(&[ + &rustls::version::TLS13, + &rustls::version::TLS12, + ]) + .with_no_client_auth() + .with_cert_resolver(cert_resolver.clone()) + .into(); Ok(TlsConfig { config, @@ -133,14 +137,14 @@ pub enum TlsServerEndPoint { } impl TlsServerEndPoint { - pub fn new(cert: &Certificate) -> anyhow::Result { + pub fn new(cert: &CertificateDer) -> anyhow::Result { let sha256_oids = [ // I'm explicitly not adding MD5 or SHA1 here... They're bad. oid_registry::OID_SIG_ECDSA_WITH_SHA256, oid_registry::OID_PKCS1_SHA256WITHRSA, ]; - let pem = x509_parser::parse_x509_certificate(&cert.0) + let pem = x509_parser::parse_x509_certificate(cert) .context("Failed to parse PEM object from cerficiate")? .1; @@ -150,8 +154,7 @@ impl TlsServerEndPoint { let oid = pem.signature_algorithm.oid(); let alg = reg.get(oid); if sha256_oids.contains(oid) { - let tls_server_end_point: [u8; 32] = - Sha256::new().chain_update(&cert.0).finalize().into(); + let tls_server_end_point: [u8; 32] = Sha256::new().chain_update(cert).finalize().into(); info!(subject = %pem.subject, signature_algorithm = alg.map(|a| a.description()), tls_server_end_point = %base64::encode(tls_server_end_point), "determined channel binding"); Ok(Self::Sha256(tls_server_end_point)) } else { @@ -165,7 +168,7 @@ impl TlsServerEndPoint { } } -#[derive(Default)] +#[derive(Default, Debug)] pub struct CertResolver { certs: HashMap, TlsServerEndPoint)>, default: Option<(Arc, TlsServerEndPoint)>, @@ -185,11 +188,14 @@ impl CertResolver { let priv_key = { let key_bytes = std::fs::read(key_path) .context(format!("Failed to read TLS keys at '{key_path}'"))?; - let mut keys = rustls_pemfile::pkcs8_private_keys(&mut &key_bytes[..]) - .context(format!("Failed to parse TLS keys at '{key_path}'"))?; + let mut keys = rustls_pemfile::pkcs8_private_keys(&mut &key_bytes[..]).collect_vec(); ensure!(keys.len() == 1, "keys.len() = {} (should be 1)", keys.len()); - keys.pop().map(rustls::PrivateKey).unwrap() + PrivateKeyDer::Pkcs8( + keys.pop() + .unwrap() + .context(format!("Failed to parse TLS keys at '{key_path}'"))?, + ) }; let cert_chain_bytes = std::fs::read(cert_path) @@ -197,14 +203,10 @@ impl CertResolver { let cert_chain = { rustls_pemfile::certs(&mut &cert_chain_bytes[..]) + .try_collect() .with_context(|| { - format!( - "Failed to read TLS certificate chain from bytes from file at '{cert_path}'." - ) + format!("Failed to read TLS certificate chain from bytes from file at '{cert_path}'.") })? - .into_iter() - .map(rustls::Certificate) - .collect() }; self.add_cert(priv_key, cert_chain, is_default) @@ -212,15 +214,15 @@ impl CertResolver { pub fn add_cert( &mut self, - priv_key: PrivateKey, - cert_chain: Vec, + priv_key: PrivateKeyDer<'static>, + cert_chain: Vec>, is_default: bool, ) -> anyhow::Result<()> { let key = sign::any_supported_type(&priv_key).context("invalid private key")?; let first_cert = &cert_chain[0]; let tls_server_end_point = TlsServerEndPoint::new(first_cert)?; - let pem = x509_parser::parse_x509_certificate(&first_cert.0) + let pem = x509_parser::parse_x509_certificate(first_cert) .context("Failed to parse PEM object from cerficiate")? .1; diff --git a/proxy/src/proxy/tests.rs b/proxy/src/proxy/tests.rs index d866b1820f39..5d0340e85201 100644 --- a/proxy/src/proxy/tests.rs +++ b/proxy/src/proxy/tests.rs @@ -20,6 +20,7 @@ use crate::{http, sasl, scram}; use anyhow::{bail, Context}; use async_trait::async_trait; use rstest::rstest; +use rustls::pki_types; use tokio_postgres::config::SslMode; use tokio_postgres::tls::{MakeTlsConnect, NoTls}; use tokio_postgres_rustls::{MakeRustlsConnect, RustlsStream}; @@ -28,7 +29,11 @@ use tokio_postgres_rustls::{MakeRustlsConnect, RustlsStream}; fn generate_certs( hostname: &str, common_name: &str, -) -> anyhow::Result<(rustls::Certificate, rustls::Certificate, rustls::PrivateKey)> { +) -> anyhow::Result<( + pki_types::CertificateDer<'static>, + pki_types::CertificateDer<'static>, + pki_types::PrivateKeyDer<'static>, +)> { let ca = rcgen::Certificate::from_params({ let mut params = rcgen::CertificateParams::default(); params.is_ca = rcgen::IsCa::Ca(rcgen::BasicConstraints::Unconstrained); @@ -45,9 +50,9 @@ fn generate_certs( })?; Ok(( - rustls::Certificate(ca.serialize_der()?), - rustls::Certificate(cert.serialize_der_with_signer(&ca)?), - rustls::PrivateKey(cert.serialize_private_key_der()), + pki_types::CertificateDer::from(ca.serialize_der()?), + pki_types::CertificateDer::from(cert.serialize_der_with_signer(&ca)?), + pki_types::PrivateKeyDer::Pkcs8(cert.serialize_private_key_der().into()), )) } @@ -82,9 +87,8 @@ fn generate_tls_config<'a>( let tls_config = { let config = rustls::ServerConfig::builder() - .with_safe_defaults() .with_no_client_auth() - .with_single_cert(vec![cert.clone()], key.clone())? + .with_single_cert(vec![cert.clone()], key.clone_key())? .into(); let mut cert_resolver = CertResolver::new(); @@ -101,10 +105,9 @@ fn generate_tls_config<'a>( let client_config = { let config = rustls::ClientConfig::builder() - .with_safe_defaults() .with_root_certificates({ let mut store = rustls::RootCertStore::empty(); - store.add(&ca)?; + store.add(ca)?; store }) .with_no_client_auth(); diff --git a/workspace_hack/Cargo.toml b/workspace_hack/Cargo.toml index e808fabbe7db..8593b752c209 100644 --- a/workspace_hack/Cargo.toml +++ b/workspace_hack/Cargo.toml @@ -60,7 +60,6 @@ regex = { version = "1" } regex-automata = { version = "0.4", default-features = false, features = ["dfa-onepass", "hybrid", "meta", "nfa-backtrack", "perf-inline", "perf-literal", "unicode"] } regex-syntax = { version = "0.8" } reqwest = { version = "0.11", default-features = false, features = ["blocking", "default-tls", "json", "multipart", "rustls-tls", "stream"] } -ring = { version = "0.16" } rustls = { version = "0.21", features = ["dangerous_configuration"] } scopeguard = { version = "1" } serde = { version = "1", features = ["alloc", "derive"] } @@ -80,6 +79,7 @@ tracing-core = { version = "0.1" } tungstenite = { version = "0.20" } url = { version = "2", features = ["serde"] } uuid = { version = "1", features = ["serde", "v4", "v7"] } +zeroize = { version = "1", features = ["derive"] } zstd = { version = "0.13" } zstd-safe = { version = "7", default-features = false, features = ["arrays", "legacy", "std", "zdict_builder"] } zstd-sys = { version = "2", default-features = false, features = ["legacy", "std", "zdict_builder"] } From 0f05ef67e28fc0c26e0b1300edad82d4e054e24f Mon Sep 17 00:00:00 2001 From: Vlad Lazar Date: Thu, 7 Mar 2024 19:53:10 +0000 Subject: [PATCH 08/43] pageserver: revert open layer rolling revert (#6962) ## Problem We reverted https://github.com/neondatabase/neon/pull/6661 a few days ago. The change led to OOMs in benchmarks followed by large WAL reingests. The issue was that we removed [this code](https://github.com/neondatabase/neon/blob/d04af08567cc3ff94ff19a2f6b3f7a2a1e3c55d1/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs#L409-L417). That call may trigger a roll of the open layer due to the keepalive messages received from the safekeeper. Removing it meant that enforcing of checkpoint timeout became even more lax and led to using up large amounts of memory for the in memory layer indices. ## Summary of changes Piggyback on keep alive messages to enforce checkpoint timeout. This is a hack, but it's exactly what the current code is doing. ## Alternatives Christhian, Joonas and myself sketched out a timer based approach [here](https://github.com/neondatabase/neon/pull/6940). While discussing it further, it became obvious that's also a bit of a hack and not the desired end state. I chose not to take that further since it's not what we ultimately want and it'll be harder to rip out. Right now it's unclear what the ideal system behaviour is: * early flushing on memory pressure, or ... * detaching tenants on memory pressure --- pageserver/src/pgdatadir_mapping.rs | 17 +- pageserver/src/tenant.rs | 36 +- .../tenant/storage_layer/inmemory_layer.rs | 38 +- pageserver/src/tenant/timeline.rs | 375 +++++++++++++----- .../walreceiver/walreceiver_connection.rs | 36 +- test_runner/performance/test_layer_map.py | 4 +- 6 files changed, 322 insertions(+), 184 deletions(-) diff --git a/pageserver/src/pgdatadir_mapping.rs b/pageserver/src/pgdatadir_mapping.rs index 628aeb5a281c..727650a5a5ec 100644 --- a/pageserver/src/pgdatadir_mapping.rs +++ b/pageserver/src/pgdatadir_mapping.rs @@ -15,6 +15,7 @@ use crate::walrecord::NeonWalRecord; use anyhow::{ensure, Context}; use bytes::{Buf, Bytes, BytesMut}; use enum_map::Enum; +use itertools::Itertools; use pageserver_api::key::{ dbdir_key_range, is_rel_block_key, is_slru_block_key, rel_block_to_key, rel_dir_to_key, rel_key_range, rel_size_to_key, relmap_file_key, slru_block_to_key, slru_dir_to_key, @@ -1498,7 +1499,7 @@ impl<'a> DatadirModification<'a> { return Ok(()); } - let writer = self.tline.writer().await; + let mut writer = self.tline.writer().await; // Flush relation and SLRU data blocks, keep metadata. let mut retained_pending_updates = HashMap::<_, Vec<_>>::new(); @@ -1537,14 +1538,22 @@ impl<'a> DatadirModification<'a> { /// All the modifications in this atomic update are stamped by the specified LSN. /// pub async fn commit(&mut self, ctx: &RequestContext) -> anyhow::Result<()> { - let writer = self.tline.writer().await; + let mut writer = self.tline.writer().await; let pending_nblocks = self.pending_nblocks; self.pending_nblocks = 0; if !self.pending_updates.is_empty() { - writer.put_batch(&self.pending_updates, ctx).await?; - self.pending_updates.clear(); + // 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(); + + writer.put_batch(lsn_ordered_batch, ctx).await?; } if !self.pending_deletions.is_empty() { diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index 2f23e535faf2..4f4654422be5 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -3857,7 +3857,7 @@ mod tests { .create_test_timeline(TIMELINE_ID, Lsn(0x08), DEFAULT_PG_VERSION, &ctx) .await?; - let writer = tline.writer().await; + let mut writer = tline.writer().await; writer .put( *TEST_KEY, @@ -3869,7 +3869,7 @@ mod tests { writer.finish_write(Lsn(0x10)); drop(writer); - let writer = tline.writer().await; + let mut writer = tline.writer().await; writer .put( *TEST_KEY, @@ -3935,7 +3935,7 @@ mod tests { let tline = tenant .create_test_timeline(TIMELINE_ID, Lsn(0x10), DEFAULT_PG_VERSION, &ctx) .await?; - let writer = tline.writer().await; + let mut writer = tline.writer().await; #[allow(non_snake_case)] let TEST_KEY_A: Key = Key::from_hex("110000000033333333444444445500000001").unwrap(); @@ -3969,7 +3969,7 @@ mod tests { let newtline = tenant .get_timeline(NEW_TIMELINE_ID, true) .expect("Should have a local timeline"); - let new_writer = newtline.writer().await; + let mut new_writer = newtline.writer().await; new_writer .put(TEST_KEY_A, Lsn(0x40), &test_value("bar at 0x40"), &ctx) .await?; @@ -4001,7 +4001,7 @@ mod tests { ) -> anyhow::Result<()> { let mut lsn = start_lsn; { - let writer = tline.writer().await; + let mut writer = tline.writer().await; // Create a relation on the timeline writer .put( @@ -4026,7 +4026,7 @@ mod tests { } tline.freeze_and_flush().await?; { - let writer = tline.writer().await; + let mut writer = tline.writer().await; writer .put( *TEST_KEY, @@ -4389,7 +4389,7 @@ mod tests { .create_test_timeline(TIMELINE_ID, Lsn(0x08), DEFAULT_PG_VERSION, &ctx) .await?; - let writer = tline.writer().await; + let mut writer = tline.writer().await; writer .put( *TEST_KEY, @@ -4406,7 +4406,7 @@ mod tests { .compact(&CancellationToken::new(), EnumSet::empty(), &ctx) .await?; - let writer = tline.writer().await; + let mut writer = tline.writer().await; writer .put( *TEST_KEY, @@ -4423,7 +4423,7 @@ mod tests { .compact(&CancellationToken::new(), EnumSet::empty(), &ctx) .await?; - let writer = tline.writer().await; + let mut writer = tline.writer().await; writer .put( *TEST_KEY, @@ -4440,7 +4440,7 @@ mod tests { .compact(&CancellationToken::new(), EnumSet::empty(), &ctx) .await?; - let writer = tline.writer().await; + let mut writer = tline.writer().await; writer .put( *TEST_KEY, @@ -4497,7 +4497,7 @@ mod tests { for _ in 0..repeat { for _ in 0..key_count { test_key.field6 = blknum; - let writer = timeline.writer().await; + let mut writer = timeline.writer().await; writer .put( test_key, @@ -4690,7 +4690,7 @@ mod tests { current_lsn += 0x100; - let writer = current_timeline.writer().await; + let mut writer = current_timeline.writer().await; writer .put( gap_at_key, @@ -4729,7 +4729,7 @@ mod tests { current_lsn += 0x10; - let writer = child_timeline.writer().await; + let mut writer = child_timeline.writer().await; writer .put( current_key, @@ -4807,7 +4807,7 @@ mod tests { for blknum in 0..NUM_KEYS { lsn = Lsn(lsn.0 + 0x10); test_key.field6 = blknum as u32; - let writer = tline.writer().await; + let mut writer = tline.writer().await; writer .put( test_key, @@ -4828,7 +4828,7 @@ mod tests { lsn = Lsn(lsn.0 + 0x10); let blknum = thread_rng().gen_range(0..NUM_KEYS); test_key.field6 = blknum as u32; - let writer = tline.writer().await; + let mut writer = tline.writer().await; writer .put( test_key, @@ -4896,7 +4896,7 @@ mod tests { for blknum in 0..NUM_KEYS { lsn = Lsn(lsn.0 + 0x10); test_key.field6 = blknum as u32; - let writer = tline.writer().await; + let mut writer = tline.writer().await; writer .put( test_key, @@ -4925,7 +4925,7 @@ mod tests { lsn = Lsn(lsn.0 + 0x10); let blknum = thread_rng().gen_range(0..NUM_KEYS); test_key.field6 = blknum as u32; - let writer = tline.writer().await; + let mut writer = tline.writer().await; writer .put( test_key, @@ -5002,7 +5002,7 @@ mod tests { lsn = Lsn(lsn.0 + 0x10); let blknum = thread_rng().gen_range(0..NUM_KEYS); test_key.field6 = blknum as u32; - let writer = tline.writer().await; + let mut writer = tline.writer().await; writer .put( test_key, diff --git a/pageserver/src/tenant/storage_layer/inmemory_layer.rs b/pageserver/src/tenant/storage_layer/inmemory_layer.rs index e7da28b8d601..5f1db21d493b 100644 --- a/pageserver/src/tenant/storage_layer/inmemory_layer.rs +++ b/pageserver/src/tenant/storage_layer/inmemory_layer.rs @@ -336,32 +336,17 @@ impl InMemoryLayer { /// Common subroutine of the public put_wal_record() and put_page_image() functions. /// Adds the page version to the in-memory tree + pub(crate) async fn put_value( &self, key: Key, lsn: Lsn, - val: &Value, - ctx: &RequestContext, - ) -> Result<()> { - let mut inner = self.inner.write().await; - self.assert_writable(); - self.put_value_locked(&mut inner, key, lsn, val, ctx).await - } - - pub(crate) async fn put_values( - &self, - values: &HashMap>, + buf: &[u8], ctx: &RequestContext, ) -> Result<()> { let mut inner = self.inner.write().await; self.assert_writable(); - for (key, vals) in values { - for (lsn, val) in vals { - self.put_value_locked(&mut inner, *key, *lsn, val, ctx) - .await?; - } - } - Ok(()) + self.put_value_locked(&mut inner, key, lsn, buf, ctx).await } async fn put_value_locked( @@ -369,22 +354,16 @@ impl InMemoryLayer { locked_inner: &mut RwLockWriteGuard<'_, InMemoryLayerInner>, key: Key, lsn: Lsn, - val: &Value, + buf: &[u8], ctx: &RequestContext, ) -> Result<()> { trace!("put_value key {} at {}/{}", key, self.timeline_id, lsn); let off = { - // Avoid doing allocations for "small" values. - // In the regression test suite, the limit of 256 avoided allocations in 95% of cases: - // https://github.com/neondatabase/neon/pull/5056#discussion_r1301975061 - let mut buf = smallvec::SmallVec::<[u8; 256]>::new(); - buf.clear(); - val.ser_into(&mut buf)?; locked_inner .file .write_blob( - &buf, + buf, &RequestContextBuilder::extend(ctx) .page_content_kind(PageContentKind::InMemoryLayer) .build(), @@ -412,7 +391,12 @@ impl InMemoryLayer { pub async fn freeze(&self, end_lsn: Lsn) { let inner = self.inner.write().await; - assert!(self.start_lsn < end_lsn); + assert!( + self.start_lsn < end_lsn, + "{} >= {}", + self.start_lsn, + end_lsn + ); self.end_lsn.set(end_lsn).expect("end_lsn set only once"); for vec_map in inner.index.values() { diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 71a958206c5c..7004db1cb598 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -27,6 +27,18 @@ use pageserver_api::{ }; use rand::Rng; use serde_with::serde_as; +use storage_broker::BrokerClientChannel; +use tokio::{ + runtime::Handle, + sync::{oneshot, watch}, +}; +use tokio_util::sync::CancellationToken; +use tracing::*; +use utils::{ + bin_ser::BeSer, + sync::gate::{Gate, GateGuard}, +}; + use std::ops::{Deref, Range}; use std::pin::pin; use std::sync::atomic::Ordering as AtomicOrdering; @@ -41,14 +53,6 @@ use std::{ cmp::{max, min, Ordering}, ops::ControlFlow, }; -use storage_broker::BrokerClientChannel; -use tokio::{ - runtime::Handle, - sync::{oneshot, watch}, -}; -use tokio_util::sync::CancellationToken; -use tracing::*; -use utils::sync::gate::{Gate, GateGuard}; use crate::tenant::timeline::logical_size::CurrentLogicalSize; use crate::tenant::{ @@ -271,7 +275,7 @@ pub struct Timeline { /// Locked automatically by [`TimelineWriter`] and checkpointer. /// Must always be acquired before the layer map/individual layer lock /// to avoid deadlock. - write_lock: tokio::sync::Mutex<()>, + write_lock: tokio::sync::Mutex>, /// Used to avoid multiple `flush_loop` tasks running pub(super) flush_loop_state: Mutex, @@ -917,8 +921,6 @@ impl Timeline { seq: &Bytes, vec: &Bytes, ) { - use utils::bin_ser::BeSer; - if *key == AUX_FILES_KEY { // The value reconstruct of AUX_FILES_KEY from records is not deterministic // since it uses a hash map under the hood. Hence, deserialise both results @@ -1149,56 +1151,8 @@ impl Timeline { pub(crate) async fn writer(&self) -> TimelineWriter<'_> { TimelineWriter { tl: self, - _write_guard: self.write_lock.lock().await, - } - } - - /// Check if more than 'checkpoint_distance' of WAL has been accumulated in - /// the in-memory layer, and initiate flushing it if so. - /// - /// Also flush after a period of time without new data -- it helps - /// safekeepers to regard pageserver as caught up and suspend activity. - pub(crate) async fn check_checkpoint_distance(self: &Arc) -> anyhow::Result<()> { - let last_lsn = self.get_last_record_lsn(); - let open_layer_size = { - let guard = self.layers.read().await; - let layers = guard.layer_map(); - let Some(open_layer) = layers.open_layer.as_ref() else { - return Ok(()); - }; - open_layer.size().await? - }; - let last_freeze_at = self.last_freeze_at.load(); - let last_freeze_ts = *(self.last_freeze_ts.read().unwrap()); - let distance = last_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 - >= self.get_checkpoint_distance() as i128 * self.shard_identity.count.count() as i128) - || open_layer_size > self.get_checkpoint_distance() - || (distance > 0 && last_freeze_ts.elapsed() >= self.get_checkpoint_timeout()) - { - info!( - "check_checkpoint_distance {}, layer size {}, elapsed since last flush {:?}", - distance, - open_layer_size, - last_freeze_ts.elapsed() - ); - - self.freeze_inmem_layer(true).await; - self.last_freeze_at.store(last_lsn); - *(self.last_freeze_ts.write().unwrap()) = Instant::now(); - - // Wake up the layer flusher - self.flush_frozen_layers(); + write_guard: self.write_lock.lock().await, } - Ok(()) } pub(crate) fn activate( @@ -1635,7 +1589,7 @@ impl Timeline { layer_flush_start_tx, layer_flush_done_tx, - write_lock: tokio::sync::Mutex::new(()), + write_lock: tokio::sync::Mutex::new(None), gc_info: std::sync::RwLock::new(GcInfo { retain_lsns: Vec::new(), @@ -2961,43 +2915,6 @@ impl Timeline { Ok(layer) } - async fn put_value( - &self, - key: Key, - lsn: Lsn, - val: &Value, - ctx: &RequestContext, - ) -> anyhow::Result<()> { - //info!("PUT: key {} at {}", key, lsn); - let layer = self.get_layer_for_write(lsn).await?; - layer.put_value(key, lsn, val, ctx).await?; - Ok(()) - } - - async fn put_values( - &self, - values: &HashMap>, - ctx: &RequestContext, - ) -> anyhow::Result<()> { - // Pick the first LSN in the batch to get the layer to write to. - for lsns in values.values() { - if let Some((lsn, _)) = lsns.first() { - let layer = self.get_layer_for_write(*lsn).await?; - layer.put_values(values, ctx).await?; - break; - } - } - Ok(()) - } - - async fn put_tombstones(&self, tombstones: &[(Range, Lsn)]) -> anyhow::Result<()> { - if let Some((_, lsn)) = tombstones.first() { - let layer = self.get_layer_for_write(*lsn).await?; - layer.put_tombstones(tombstones).await?; - } - Ok(()) - } - pub(crate) fn finish_write(&self, new_lsn: Lsn) { assert!(new_lsn.is_aligned()); @@ -3008,14 +2925,20 @@ impl Timeline { async fn freeze_inmem_layer(&self, write_lock_held: bool) { // Freeze the current open in-memory layer. It will be written to disk on next // iteration. + let _write_guard = if write_lock_held { None } else { Some(self.write_lock.lock().await) }; + + self.freeze_inmem_layer_at(self.get_last_record_lsn()).await; + } + + async fn freeze_inmem_layer_at(&self, at: Lsn) { let mut guard = self.layers.write().await; guard - .try_freeze_in_memory_layer(self.get_last_record_lsn(), &self.last_freeze_at) + .try_freeze_in_memory_layer(at, &self.last_freeze_at) .await; } @@ -4392,13 +4315,43 @@ fn layer_traversal_error(msg: String, path: Vec) -> PageRecon PageReconstructError::from(msg) } +struct TimelineWriterState { + open_layer: Arc, + current_size: u64, + // Previous Lsn which passed through + prev_lsn: Option, + // Largest Lsn which passed through the current writer + max_lsn: Option, + // Cached details of the last freeze. Avoids going trough the atomic/lock on every put. + cached_last_freeze_at: Lsn, + cached_last_freeze_ts: Instant, +} + +impl TimelineWriterState { + fn new( + open_layer: Arc, + current_size: u64, + last_freeze_at: Lsn, + last_freeze_ts: Instant, + ) -> Self { + Self { + open_layer, + current_size, + prev_lsn: None, + max_lsn: None, + cached_last_freeze_at: last_freeze_at, + cached_last_freeze_ts: last_freeze_ts, + } + } +} + /// Various functions to mutate the timeline. // TODO Currently, Deref is used to allow easy access to read methods from this trait. // This is probably considered a bad practice in Rust and should be fixed eventually, // but will cause large code changes. pub(crate) struct TimelineWriter<'a> { tl: &'a Timeline, - _write_guard: tokio::sync::MutexGuard<'a, ()>, + write_guard: tokio::sync::MutexGuard<'a, Option>, } impl Deref for TimelineWriter<'_> { @@ -4409,31 +4362,239 @@ impl Deref for TimelineWriter<'_> { } } +impl Drop for TimelineWriter<'_> { + fn drop(&mut self) { + self.write_guard.take(); + } +} + +#[derive(PartialEq)] +enum OpenLayerAction { + Roll, + Open, + None, +} + impl<'a> TimelineWriter<'a> { /// Put a new page version that can be constructed from a WAL record /// /// This will implicitly extend the relation, if the page is beyond the /// current end-of-file. pub(crate) async fn put( - &self, + &mut self, key: Key, lsn: Lsn, value: &Value, ctx: &RequestContext, ) -> anyhow::Result<()> { - self.tl.put_value(key, lsn, value, ctx).await + // Avoid doing allocations for "small" values. + // In the regression test suite, the limit of 256 avoided allocations in 95% of cases: + // https://github.com/neondatabase/neon/pull/5056#discussion_r1301975061 + let mut buf = smallvec::SmallVec::<[u8; 256]>::new(); + value.ser_into(&mut buf)?; + let buf_size: u64 = buf.len().try_into().expect("oversized value buf"); + + let action = self.get_open_layer_action(lsn, buf_size); + let layer = self.handle_open_layer_action(lsn, action).await?; + let res = layer.put_value(key, lsn, &buf, ctx).await; + + if res.is_ok() { + // Update the current size only when the entire write was ok. + // In case of failures, we may have had partial writes which + // render the size tracking out of sync. That's ok because + // the checkpoint distance should be significantly smaller + // than the S3 single shot upload limit of 5GiB. + let state = self.write_guard.as_mut().unwrap(); + + state.current_size += buf_size; + state.prev_lsn = Some(lsn); + state.max_lsn = std::cmp::max(state.max_lsn, Some(lsn)); + } + + 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?; + } + + 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, + action: OpenLayerAction, + ) -> anyhow::Result<&Arc> { + match action { + OpenLayerAction::Roll => { + let freeze_at = self.write_guard.as_ref().unwrap().max_lsn.unwrap(); + self.roll_layer(freeze_at).await?; + self.open_layer(at).await?; + } + OpenLayerAction::Open => self.open_layer(at).await?, + OpenLayerAction::None => { + assert!(self.write_guard.is_some()); + } + } + + Ok(&self.write_guard.as_ref().unwrap().open_layer) + } + + async fn open_layer(&mut self, at: Lsn) -> anyhow::Result<()> { + let layer = self.tl.get_layer_for_write(at).await?; + let initial_size = 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( + layer, + initial_size, + last_freeze_at, + last_freeze_ts, + )); + + Ok(()) + } + + async fn roll_layer(&mut self, freeze_at: Lsn) -> anyhow::Result<()> { + assert!(self.write_guard.is_some()); + + self.tl.freeze_inmem_layer_at(freeze_at).await; + + let now = Instant::now(); + *(self.last_freeze_ts.write().unwrap()) = now; + + self.tl.flush_frozen_layers(); + + let current_size = self.write_guard.as_ref().unwrap().current_size; + if current_size > self.get_checkpoint_distance() { + warn!("Flushed oversized open layer with size {}", current_size) + } + + Ok(()) + } + + fn get_open_layer_action(&self, lsn: Lsn, new_value_size: u64) -> OpenLayerAction { + let state = &*self.write_guard; + let Some(state) = &state else { + return OpenLayerAction::Open; + }; + + if state.prev_lsn == Some(lsn) { + // Rolling mid LSN is not supported by downstream code. + // Hence, only roll at LSN boundaries. + return OpenLayerAction::None; + } + + if state.current_size == 0 { + // Don't roll empty layers + 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() + ); + + OpenLayerAction::Roll + } else { + OpenLayerAction::None + } + } + + /// Put a batch keys at the specified Lsns. + /// + /// The batch should be sorted by Lsn such that it's safe + /// to roll the open layer mid batch. pub(crate) async fn put_batch( - &self, - batch: &HashMap>, + &mut self, + batch: Vec<(Key, Lsn, Value)>, ctx: &RequestContext, ) -> anyhow::Result<()> { - self.tl.put_values(batch, ctx).await + for (key, lsn, val) in batch { + self.put(key, lsn, &val, ctx).await? + } + + Ok(()) } - pub(crate) async fn delete_batch(&self, batch: &[(Range, Lsn)]) -> anyhow::Result<()> { - self.tl.put_tombstones(batch).await + pub(crate) async fn delete_batch(&mut self, batch: &[(Range, Lsn)]) -> anyhow::Result<()> { + if let Some((_, lsn)) = batch.first() { + let action = self.get_open_layer_action(*lsn, 0); + let layer = self.handle_open_layer_action(*lsn, action).await?; + layer.put_tombstones(batch).await?; + } + + Ok(()) } /// Track the end of the latest digested WAL record. diff --git a/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs b/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs index 9cb53f46d134..8297ca6563ee 100644 --- a/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs +++ b/pageserver/src/tenant/timeline/walreceiver/walreceiver_connection.rs @@ -343,23 +343,6 @@ pub(super) async fn handle_walreceiver_connection( modification.commit(&ctx).await?; uncommitted_records = 0; filtered_records = 0; - - // - // We should check checkpoint distance after appending each ingest_batch_size bytes because otherwise - // layer size can become much larger than `checkpoint_distance`. - // It can append because wal-sender is sending WAL using 125kb chucks and some WAL records can cause writing large - // amount of data to key-value storage. So performing this check only after processing - // all WAL records in the chunk, can cause huge L0 layer files. - // - timeline - .check_checkpoint_distance() - .await - .with_context(|| { - format!( - "Failed to check checkpoint distance for timeline {}", - timeline.timeline_id - ) - })?; } } @@ -406,15 +389,16 @@ pub(super) async fn handle_walreceiver_connection( } } - timeline - .check_checkpoint_distance() - .await - .with_context(|| { - format!( - "Failed to check checkpoint distance for timeline {}", - timeline.timeline_id - ) - })?; + { + // 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 diff --git a/test_runner/performance/test_layer_map.py b/test_runner/performance/test_layer_map.py index 6bd0d85fa25c..9b20954d4514 100644 --- a/test_runner/performance/test_layer_map.py +++ b/test_runner/performance/test_layer_map.py @@ -17,10 +17,10 @@ def test_layer_map(neon_env_builder: NeonEnvBuilder, zenbenchmark): tenant, _ = env.neon_cli.create_tenant( conf={ "gc_period": "0s", - "checkpoint_distance": "8192", + "checkpoint_distance": "16384", "compaction_period": "1 s", "compaction_threshold": "1", - "compaction_target_size": "8192", + "compaction_target_size": "16384", } ) From 2c132e45cb624a39ac7f23ea78f082078277a450 Mon Sep 17 00:00:00 2001 From: Conrad Ludgate Date: Fri, 8 Mar 2024 07:56:23 +0000 Subject: [PATCH 09/43] proxy: do not store ephemeral endpoints in http pool (#6819) ## Problem For the ephemeral endpoint feature, it's not really too helpful to keep them around in the connection pool. This isn't really pressing but I think it's still a bit better this way. ## Summary of changes Add `is_ephemeral` function to `NeonOptions`. Allow `serverless::ConnInfo::endpoint_cache_key()` to return an `Option`. Handle that option appropriately --- proxy/src/proxy.rs | 5 +++++ proxy/src/serverless/conn_pool.rs | 30 +++++++++++++++++++++--------- 2 files changed, 26 insertions(+), 9 deletions(-) diff --git a/proxy/src/proxy.rs b/proxy/src/proxy.rs index aeba08bc4f54..7848fc2ac2cc 100644 --- a/proxy/src/proxy.rs +++ b/proxy/src/proxy.rs @@ -380,6 +380,11 @@ impl NeonOptions { Self::parse_from_iter(StartupMessageParams::parse_options_raw(options)) } + pub fn is_ephemeral(&self) -> bool { + // Currently, neon endpoint options are all reserved for ephemeral endpoints. + !self.0.is_empty() + } + fn parse_from_iter<'a>(options: impl Iterator) -> Self { let mut options = options .filter_map(neon_option) diff --git a/proxy/src/serverless/conn_pool.rs b/proxy/src/serverless/conn_pool.rs index 7d705ba04993..73f213d074d3 100644 --- a/proxy/src/serverless/conn_pool.rs +++ b/proxy/src/serverless/conn_pool.rs @@ -43,8 +43,13 @@ impl ConnInfo { (self.dbname.clone(), self.user_info.user.clone()) } - pub fn endpoint_cache_key(&self) -> EndpointCacheKey { - self.user_info.endpoint_cache_key() + pub fn endpoint_cache_key(&self) -> Option { + // We don't want to cache http connections for ephemeral endpoints. + if self.user_info.options.is_ephemeral() { + None + } else { + Some(self.user_info.endpoint_cache_key()) + } } } @@ -360,8 +365,11 @@ impl GlobalConnPool { conn_info: &ConnInfo, ) -> Result>, HttpConnError> { let mut client: Option> = None; + let Some(endpoint) = conn_info.endpoint_cache_key() else { + return Ok(None); + }; - let endpoint_pool = self.get_or_create_endpoint_pool(&conn_info.endpoint_cache_key()); + let endpoint_pool = self.get_or_create_endpoint_pool(&endpoint); if let Some(entry) = endpoint_pool .write() .get_conn_entry(conn_info.db_and_user()) @@ -455,8 +463,10 @@ pub fn poll_client( span.in_scope(|| { info!(%conn_info, %session_id, "new connection"); }); - let pool = - Arc::downgrade(&global_pool.get_or_create_endpoint_pool(&conn_info.endpoint_cache_key())); + let pool = match conn_info.endpoint_cache_key() { + Some(endpoint) => Arc::downgrade(&global_pool.get_or_create_endpoint_pool(&endpoint)), + None => Weak::new(), + }; let pool_clone = pool.clone(); let db_user = conn_info.db_and_user(); @@ -723,8 +733,9 @@ mod tests { dbname: "dbname".into(), password: "password".as_bytes().into(), }; - let ep_pool = - Arc::downgrade(&pool.get_or_create_endpoint_pool(&conn_info.endpoint_cache_key())); + let ep_pool = Arc::downgrade( + &pool.get_or_create_endpoint_pool(&conn_info.endpoint_cache_key().unwrap()), + ); { let mut client = Client::new(create_inner(), conn_info.clone(), ep_pool.clone()); assert_eq!(0, pool.get_global_connections_count()); @@ -780,8 +791,9 @@ mod tests { dbname: "dbname".into(), password: "password".as_bytes().into(), }; - let ep_pool = - Arc::downgrade(&pool.get_or_create_endpoint_pool(&conn_info.endpoint_cache_key())); + let ep_pool = Arc::downgrade( + &pool.get_or_create_endpoint_pool(&conn_info.endpoint_cache_key().unwrap()), + ); { let mut client = Client::new(create_inner(), conn_info.clone(), ep_pool.clone()); client.do_drop().unwrap()(); From 7329413705be0939b550553be2f40d4bb11a1a9b Mon Sep 17 00:00:00 2001 From: John Spray Date: Fri, 8 Mar 2024 15:34:53 +0000 Subject: [PATCH 10/43] storage controller: enable setting PlacementPolicy in tenant creation (#7037) ## Problem Tenants created via the storage controller have a `PlacementPolicy` that defines their HA/secondary/detach intent. For backward compat we can just set it to Single, for onboarding tenants using /location_conf it is automatically set to Double(1) if there are at least two pageservers, but for freshly created tenants we didn't have a way to specify it. This unblocks writing tests that create HA tenants on the storage controller and do failure injection testing. ## Summary of changes - Add optional fields to TenantCreateRequest for specifying PlacementPolicy. This request structure is used both on pageserver API and storage controller API, but this method is only meaningful for the storage controller (same as existing `shard_parameters` attribute). - Use the value from the creation request in tenant creation, if provided. --- control_plane/attachment_service/src/http.rs | 7 +-- control_plane/attachment_service/src/lib.rs | 25 +-------- .../attachment_service/src/persistence.rs | 11 ++-- .../attachment_service/src/service.rs | 55 ++++++++++--------- .../attachment_service/src/tenant_state.rs | 3 +- control_plane/src/bin/neon_local.rs | 9 ++- control_plane/src/pageserver.rs | 2 + libs/pageserver_api/src/controller_api.rs | 40 ++++++++++++++ libs/pageserver_api/src/models.rs | 6 ++ 9 files changed, 92 insertions(+), 66 deletions(-) diff --git a/control_plane/attachment_service/src/http.rs b/control_plane/attachment_service/src/http.rs index 384bdcef0c5f..7e4030b221b0 100644 --- a/control_plane/attachment_service/src/http.rs +++ b/control_plane/attachment_service/src/http.rs @@ -1,6 +1,5 @@ use crate::reconciler::ReconcileError; use crate::service::{Service, STARTUP_RECONCILE_TIMEOUT}; -use crate::PlacementPolicy; use hyper::{Body, Request, Response}; use hyper::{StatusCode, Uri}; use pageserver_api::models::{ @@ -119,13 +118,9 @@ async fn handle_tenant_create( let create_req = json_request::(&mut req).await?; - // TODO: enable specifying this. Using Single as a default helps legacy tests to work (they - // have no expectation of HA). - let placement_policy = PlacementPolicy::Single; - json_response( StatusCode::CREATED, - service.tenant_create(create_req, placement_policy).await?, + service.tenant_create(create_req).await?, ) } diff --git a/control_plane/attachment_service/src/lib.rs b/control_plane/attachment_service/src/lib.rs index 7ae7e264c7b1..796b465c102c 100644 --- a/control_plane/attachment_service/src/lib.rs +++ b/control_plane/attachment_service/src/lib.rs @@ -1,4 +1,4 @@ -use serde::{Deserialize, Serialize}; +use serde::Serialize; use utils::seqwait::MonotonicCounter; mod auth; @@ -13,23 +13,6 @@ mod schema; pub mod service; mod tenant_state; -#[derive(Clone, Serialize, Deserialize, Debug, PartialEq, Eq)] -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), - /// 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, - - /// Do not attach to any pageservers. This is appropriate for tenants that - /// have been idle for a long time, where we do not mind some delay in making - /// them available in future. - Detached, -} - #[derive(Ord, PartialOrd, Eq, PartialEq, Copy, Clone, Serialize)] struct Sequence(u64); @@ -66,9 +49,3 @@ impl Sequence { Sequence(self.0 + 1) } } - -impl Default for PlacementPolicy { - fn default() -> Self { - PlacementPolicy::Double(1) - } -} diff --git a/control_plane/attachment_service/src/persistence.rs b/control_plane/attachment_service/src/persistence.rs index d5c304385c15..d5c6d74ebed8 100644 --- a/control_plane/attachment_service/src/persistence.rs +++ b/control_plane/attachment_service/src/persistence.rs @@ -7,11 +7,9 @@ use self::split_state::SplitState; use camino::Utf8Path; use camino::Utf8PathBuf; use diesel::pg::PgConnection; -use diesel::{ - Connection, ExpressionMethods, Insertable, QueryDsl, QueryResult, Queryable, RunQueryDsl, - Selectable, SelectableHelper, -}; -use pageserver_api::controller_api::NodeSchedulingPolicy; +use diesel::prelude::*; +use diesel::Connection; +use pageserver_api::controller_api::{NodeSchedulingPolicy, PlacementPolicy}; use pageserver_api::models::TenantConfig; use pageserver_api::shard::{ShardCount, ShardNumber, TenantShardId}; use serde::{Deserialize, Serialize}; @@ -19,7 +17,6 @@ use utils::generation::Generation; use utils::id::{NodeId, TenantId}; use crate::node::Node; -use crate::PlacementPolicy; /// ## What do we store? /// @@ -210,7 +207,7 @@ impl Persistence { 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::default()) + tenant.placement_policy = serde_json::to_string(&PlacementPolicy::Single) .map_err(|e| DatabaseError::Logical(format!("Serialization error: {e}")))?; } } diff --git a/control_plane/attachment_service/src/service.rs b/control_plane/attachment_service/src/service.rs index f41c4f89b96b..556d6a682864 100644 --- a/control_plane/attachment_service/src/service.rs +++ b/control_plane/attachment_service/src/service.rs @@ -16,9 +16,9 @@ use futures::{stream::FuturesUnordered, StreamExt}; use hyper::StatusCode; use pageserver_api::{ controller_api::{ - NodeAvailability, NodeConfigureRequest, NodeRegisterRequest, TenantCreateResponse, - TenantCreateResponseShard, TenantLocateResponse, TenantShardMigrateRequest, - TenantShardMigrateResponse, + NodeAvailability, NodeConfigureRequest, NodeRegisterRequest, PlacementPolicy, + TenantCreateResponse, TenantCreateResponseShard, TenantLocateResponse, + TenantShardMigrateRequest, TenantShardMigrateResponse, }, models::TenantConfigRequest, }; @@ -57,7 +57,7 @@ use crate::{ IntentState, ObservedState, ObservedStateLocation, ReconcileResult, ReconcileWaitError, ReconcilerWaiter, TenantState, }, - PlacementPolicy, Sequence, + Sequence, }; // For operations that should be quick, like attaching a new tenant @@ -176,7 +176,7 @@ impl From for ApiError { #[allow(clippy::large_enum_variant)] enum TenantCreateOrUpdate { - Create((TenantCreateRequest, PlacementPolicy)), + Create(TenantCreateRequest), Update(Vec), } @@ -792,7 +792,7 @@ impl Service { shard_stripe_size: 0, generation: Some(0), generation_pageserver: None, - placement_policy: serde_json::to_string(&PlacementPolicy::default()).unwrap(), + placement_policy: serde_json::to_string(&PlacementPolicy::Single).unwrap(), config: serde_json::to_string(&TenantConfig::default()).unwrap(), splitting: SplitState::default(), }; @@ -1053,9 +1053,8 @@ impl Service { pub(crate) async fn tenant_create( &self, create_req: TenantCreateRequest, - placement_policy: PlacementPolicy, ) -> Result { - let (response, waiters) = self.do_tenant_create(create_req, placement_policy).await?; + let (response, waiters) = self.do_tenant_create(create_req).await?; self.await_waiters(waiters, SHORT_RECONCILE_TIMEOUT).await?; Ok(response) @@ -1064,8 +1063,13 @@ impl Service { pub(crate) async fn do_tenant_create( &self, create_req: TenantCreateRequest, - placement_policy: PlacementPolicy, ) -> 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); + // This service expects to handle sharding itself: it is an error to try and directly create // a particular shard here. let tenant_id = if !create_req.new_tenant_id.is_unsharded() { @@ -1339,22 +1343,20 @@ impl Service { TenantCreateOrUpdate::Create( // Synthesize a creation request - ( - TenantCreateRequest { - new_tenant_id: TenantShardId::unsharded(tenant_id), - generation, - shard_parameters: ShardParameters { - // Must preserve the incoming shard_count do distinguish unsharded (0) - // from single-sharded (1): this distinction appears in the S3 keys of the tenant. - count: req.tenant_id.shard_count, - // We only import un-sharded or single-sharded tenants, so stripe - // size can be made up arbitrarily here. - stripe_size: ShardParameters::DEFAULT_STRIPE_SIZE, - }, - config: req.config.tenant_conf, + TenantCreateRequest { + new_tenant_id: TenantShardId::unsharded(tenant_id), + generation, + shard_parameters: ShardParameters { + // Must preserve the incoming shard_count do distinguish unsharded (0) + // from single-sharded (1): this distinction appears in the S3 keys of the tenant. + count: req.tenant_id.shard_count, + // We only import un-sharded or single-sharded tenants, so stripe + // size can be made up arbitrarily here. + stripe_size: ShardParameters::DEFAULT_STRIPE_SIZE, }, - placement_policy, - ), + placement_policy: Some(placement_policy), + config: req.config.tenant_conf, + }, ) } else { TenantCreateOrUpdate::Update(updates) @@ -1393,9 +1395,8 @@ impl Service { stripe_size: None, }; let waiters = match create_or_update { - TenantCreateOrUpdate::Create((create_req, placement_policy)) => { - let (create_resp, waiters) = - self.do_tenant_create(create_req, placement_policy).await?; + TenantCreateOrUpdate::Create(create_req) => { + let (create_resp, waiters) = self.do_tenant_create(create_req).await?; result.shards = create_resp .shards .into_iter() diff --git a/control_plane/attachment_service/src/tenant_state.rs b/control_plane/attachment_service/src/tenant_state.rs index ddb98665278d..c775736b313b 100644 --- a/control_plane/attachment_service/src/tenant_state.rs +++ b/control_plane/attachment_service/src/tenant_state.rs @@ -5,6 +5,7 @@ use std::{ }; use crate::{metrics, persistence::TenantShardPersistence}; +use pageserver_api::controller_api::PlacementPolicy; use pageserver_api::{ models::{LocationConfig, LocationConfigMode, TenantConfig}, shard::{ShardIdentity, TenantShardId}, @@ -28,7 +29,7 @@ use crate::{ attached_location_conf, secondary_location_conf, ReconcileError, Reconciler, TargetState, }, scheduler::{ScheduleError, Scheduler}, - service, PlacementPolicy, Sequence, + service, Sequence, }; /// Serialization helper diff --git a/control_plane/src/bin/neon_local.rs b/control_plane/src/bin/neon_local.rs index 1feec5cd9bac..27abcb182aa8 100644 --- a/control_plane/src/bin/neon_local.rs +++ b/control_plane/src/bin/neon_local.rs @@ -15,7 +15,7 @@ use control_plane::pageserver::{PageServerNode, PAGESERVER_REMOTE_STORAGE_DIR}; use control_plane::safekeeper::SafekeeperNode; use control_plane::{broker, local_env}; use pageserver_api::controller_api::{ - NodeAvailability, NodeConfigureRequest, NodeSchedulingPolicy, + NodeAvailability, NodeConfigureRequest, NodeSchedulingPolicy, PlacementPolicy, }; use pageserver_api::models::{ ShardParameters, TenantCreateRequest, TimelineCreateRequest, TimelineInfo, @@ -435,6 +435,11 @@ async fn handle_tenant( let shard_stripe_size: Option = create_match.get_one::("shard-stripe-size").cloned(); + let placement_policy = match create_match.get_one::("placement-policy") { + Some(s) if !s.is_empty() => serde_json::from_str::(s)?, + _ => PlacementPolicy::Single, + }; + let tenant_conf = PageServerNode::parse_config(tenant_conf)?; // If tenant ID was not specified, generate one @@ -456,6 +461,7 @@ async fn handle_tenant( .map(ShardStripeSize) .unwrap_or(ShardParameters::DEFAULT_STRIPE_SIZE), }, + placement_policy: Some(placement_policy), config: tenant_conf, }) .await?; @@ -1562,6 +1568,7 @@ fn cli() -> Command { .help("Use this tenant in future CLI commands where tenant_id is needed, but not specified")) .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")) + .arg(Arg::new("placement-policy").value_parser(value_parser!(String)).long("placement-policy").action(ArgAction::Set).help("Placement policy shards in this tenant")) ) .subcommand(Command::new("set-default").arg(tenant_id_arg.clone().required(true)) .about("Set a particular tenant as default in future CLI commands where tenant_id is needed, but not specified")) diff --git a/control_plane/src/pageserver.rs b/control_plane/src/pageserver.rs index b2904c1191a3..ae1bd60c5229 100644 --- a/control_plane/src/pageserver.rs +++ b/control_plane/src/pageserver.rs @@ -429,6 +429,8 @@ impl PageServerNode { generation, config, shard_parameters: ShardParameters::default(), + // Placement policy is not meaningful for creations not done via storage controller + placement_policy: None, }; if !settings.is_empty() { bail!("Unrecognized tenant settings: {settings:?}") diff --git a/libs/pageserver_api/src/controller_api.rs b/libs/pageserver_api/src/controller_api.rs index 64b70a1a51b2..38e61239c5bc 100644 --- a/libs/pageserver_api/src/controller_api.rs +++ b/libs/pageserver_api/src/controller_api.rs @@ -125,5 +125,45 @@ impl From for String { } } +/// Controls how tenant shards are mapped to locations on pageservers, e.g. whether +/// 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), + /// 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, + + /// Do not attach to any pageservers. This is appropriate for tenants that + /// have been idle for a long time, where we do not mind some delay in making + /// them available in future. + Detached, +} + #[derive(Serialize, Deserialize, Debug)] pub struct TenantShardMigrateResponse {} + +#[cfg(test)] +mod test { + use super::*; + use serde_json; + + /// Check stability of PlacementPolicy's serialization + #[test] + fn placement_policy_encoding() -> anyhow::Result<()> { + let v = PlacementPolicy::Double(1); + let encoded = serde_json::to_string(&v)?; + assert_eq!(encoded, "{\"Double\":1}"); + assert_eq!(serde_json::from_str::(&encoded)?, v); + + let v = PlacementPolicy::Single; + let encoded = serde_json::to_string(&v)?; + assert_eq!(encoded, "\"Single\""); + assert_eq!(serde_json::from_str::(&encoded)?, v); + Ok(()) + } +} diff --git a/libs/pageserver_api/src/models.rs b/libs/pageserver_api/src/models.rs index 57497e38313e..fe5bbd1c06fe 100644 --- a/libs/pageserver_api/src/models.rs +++ b/libs/pageserver_api/src/models.rs @@ -21,6 +21,7 @@ use utils::{ lsn::Lsn, }; +use crate::controller_api::PlacementPolicy; use crate::{ reltag::RelTag, shard::{ShardCount, ShardStripeSize, TenantShardId}, @@ -242,6 +243,11 @@ pub struct TenantCreateRequest { #[serde(skip_serializing_if = "ShardParameters::is_unsharded")] pub shard_parameters: ShardParameters, + // This parameter is only meaningful in requests sent to the storage controller + #[serde(default)] + #[serde(skip_serializing_if = "Option::is_none")] + pub placement_policy: Option, + #[serde(flatten)] pub config: TenantConfig, // as we have a flattened field, we should reject all unknown fields in it } From 86e8c43ddf817c7e3ee112e5c399cc5d60b34f29 Mon Sep 17 00:00:00 2001 From: Anastasia Lubennikova Date: Fri, 8 Mar 2024 20:42:35 +0000 Subject: [PATCH 11/43] Add downgrade scripts for neon extension. (#7065) ## Problem When we start compute with newer version of extension (i.e. 1.2) and then rollback the release, downgrading the compute version, next compute start will try to update extension to the latest version available in neon.control (i.e. 1.1). Thus we need to provide downgrade scripts like neon--1.2--1.1.sql These scripts must revert the changes made by the upgrade scripts in the reverse order. This is necessary to ensure that the next upgrade will work correctly. In general, we need to write upgrade and downgrade scripts to be more robust and add IF EXISTS / CREATE OR REPLACE clauses to all statements (where applicable). ## Summary of changes Adds downgrade scripts. Adds test cases for extension downgrade/upgrade. fixes #7066 This is a follow-up for https://app.incident.io/neondb/incidents/167?tab=follow-ups Signed-off-by: Alex Chi Z Co-authored-by: Alex Chi Z Co-authored-by: Anastasia Lubennikova --- pgxn/neon/Makefile | 2 +- pgxn/neon/neon--1.1--1.0.sql | 6 +++++ pgxn/neon/neon--1.2--1.1.sql | 1 + pgxn/neon/neon--1.3--1.2.sql | 1 + test_runner/regress/test_neon_extension.py | 31 ++++++++++++++++++++++ 5 files changed, 40 insertions(+), 1 deletion(-) create mode 100644 pgxn/neon/neon--1.1--1.0.sql create mode 100644 pgxn/neon/neon--1.2--1.1.sql create mode 100644 pgxn/neon/neon--1.3--1.2.sql diff --git a/pgxn/neon/Makefile b/pgxn/neon/Makefile index 7ea767ec7495..0bcb9545a674 100644 --- a/pgxn/neon/Makefile +++ b/pgxn/neon/Makefile @@ -21,7 +21,7 @@ SHLIB_LINK_INTERNAL = $(libpq) SHLIB_LINK = -lcurl EXTENSION = neon -DATA = neon--1.0.sql neon--1.0--1.1.sql neon--1.1--1.2.sql neon--1.2--1.3.sql +DATA = neon--1.0.sql neon--1.0--1.1.sql neon--1.1--1.2.sql neon--1.2--1.3.sql neon--1.3--1.2.sql neon--1.2--1.1.sql neon--1.1--1.0.sql PGFILEDESC = "neon - cloud storage for PostgreSQL" EXTRA_CLEAN = \ diff --git a/pgxn/neon/neon--1.1--1.0.sql b/pgxn/neon/neon--1.1--1.0.sql new file mode 100644 index 000000000000..e83e3104e84b --- /dev/null +++ b/pgxn/neon/neon--1.1--1.0.sql @@ -0,0 +1,6 @@ +-- the order of operations is important here +-- because the view depends on the function + +DROP VIEW IF EXISTS neon_lfc_stats CASCADE; + +DROP FUNCTION IF EXISTS neon_get_lfc_stats CASCADE; diff --git a/pgxn/neon/neon--1.2--1.1.sql b/pgxn/neon/neon--1.2--1.1.sql new file mode 100644 index 000000000000..c9f6a40f737f --- /dev/null +++ b/pgxn/neon/neon--1.2--1.1.sql @@ -0,0 +1 @@ +DROP VIEW IF EXISTS NEON_STAT_FILE_CACHE CASCADE; diff --git a/pgxn/neon/neon--1.3--1.2.sql b/pgxn/neon/neon--1.3--1.2.sql new file mode 100644 index 000000000000..2733a15c7581 --- /dev/null +++ b/pgxn/neon/neon--1.3--1.2.sql @@ -0,0 +1 @@ +DROP FUNCTION IF EXISTS approximate_working_set_size(bool) CASCADE; diff --git a/test_runner/regress/test_neon_extension.py b/test_runner/regress/test_neon_extension.py index 1179a3afe969..e31e1cab512e 100644 --- a/test_runner/regress/test_neon_extension.py +++ b/test_runner/regress/test_neon_extension.py @@ -29,3 +29,34 @@ def test_neon_extension(neon_env_builder: NeonEnvBuilder): log.info(res) assert len(res) == 1 assert len(res[0]) == 5 + + +# Verify that the neon extension can be upgraded/downgraded. +def test_neon_extension_compatibility(neon_env_builder: NeonEnvBuilder): + env = neon_env_builder.init_start() + env.neon_cli.create_branch("test_neon_extension_compatibility") + + endpoint_main = env.endpoints.create("test_neon_extension_compatibility") + # don't skip pg_catalog updates - it runs CREATE EXTENSION neon + endpoint_main.respec(skip_pg_catalog_updates=False) + endpoint_main.start() + + with closing(endpoint_main.connect()) as conn: + with conn.cursor() as cur: + all_versions = ["1.3", "1.2", "1.1", "1.0"] + current_version = "1.3" + for idx, begin_version in enumerate(all_versions): + for target_version in all_versions[idx + 1 :]: + if current_version != begin_version: + cur.execute( + f"ALTER EXTENSION neon UPDATE TO '{begin_version}'; -- {current_version}->{begin_version}" + ) + current_version = begin_version + # downgrade + cur.execute( + f"ALTER EXTENSION neon UPDATE TO '{target_version}'; -- {begin_version}->{target_version}" + ) + # upgrade + cur.execute( + f"ALTER EXTENSION neon UPDATE TO '{begin_version}'; -- {target_version}->{begin_version}" + ) From 4834d22d2d99bb7f9726c1cac3176550cc404e38 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Fri, 8 Mar 2024 13:24:30 -0900 Subject: [PATCH 12/43] Revoke REPLICATION (#7052) ## Problem Currently users can cause problems with replication ## Summary of changes Don't let them replicate --- compute_tools/src/spec.rs | 16 ++++++++++++++-- test_runner/regress/test_migrations.py | 2 +- 2 files changed, 15 insertions(+), 3 deletions(-) diff --git a/compute_tools/src/spec.rs b/compute_tools/src/spec.rs index 84a5a263af2e..ba3a84cda84a 100644 --- a/compute_tools/src/spec.rs +++ b/compute_tools/src/spec.rs @@ -302,9 +302,9 @@ pub fn handle_roles(spec: &ComputeSpec, client: &mut Client) -> Result<()> { RoleAction::Create => { // This branch only runs when roles are created through the console, so it is // safe to add more permissions here. BYPASSRLS and REPLICATION are inherited - // from neon_superuser. + // from neon_superuser. (NOTE: REPLICATION has been removed from here for now). let mut query: String = format!( - "CREATE ROLE {} INHERIT CREATEROLE CREATEDB BYPASSRLS REPLICATION IN ROLE neon_superuser", + "CREATE ROLE {} INHERIT CREATEROLE CREATEDB BYPASSRLS IN ROLE neon_superuser", name.pg_quote() ); info!("running role create query: '{}'", &query); @@ -805,6 +805,18 @@ $$;"#, "", "", // Add new migrations below. + r#" +DO $$ +DECLARE + role_name TEXT; +BEGIN + FOR role_name IN SELECT rolname FROM pg_roles WHERE rolreplication IS TRUE + LOOP + RAISE NOTICE 'EXECUTING ALTER ROLE % NOREPLICATION', quote_ident(role_name); + EXECUTE 'ALTER ROLE ' || quote_ident(role_name) || ' NOREPLICATION'; + END LOOP; +END +$$;"#, ]; let mut query = "CREATE SCHEMA IF NOT EXISTS neon_migration"; diff --git a/test_runner/regress/test_migrations.py b/test_runner/regress/test_migrations.py index 3f626c5c7c37..526ae14b8715 100644 --- a/test_runner/regress/test_migrations.py +++ b/test_runner/regress/test_migrations.py @@ -15,7 +15,7 @@ def test_migrations(neon_simple_env: NeonEnv): endpoint.wait_for_migrations() - num_migrations = 8 + num_migrations = 9 with endpoint.cursor() as cur: cur.execute("SELECT id FROM neon_migration.migration_id") From 74d24582cfe67f4115b54d26e5fb787a221dcae4 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Sat, 9 Mar 2024 13:37:02 +0100 Subject: [PATCH 13/43] throttling: exclude throttled time from basebackup (fixup of #6953) (#7072) PR #6953 only excluded throttled time from the handle_pagerequests (aka smgr metrics). This PR implements the deduction for `basebackup ` queries. The other page_service methods either don't use Timeline::get or they aren't used in production. Found by manually inspecting in [staging logs](https://neonprod.grafana.net/explore?schemaVersion=1&panes=%7B%22wx8%22:%7B%22datasource%22:%22xHHYY0dVz%22,%22queries%22:%5B%7B%22refId%22:%22A%22,%22expr%22:%22%7Bhostname%3D%5C%22pageserver-0.eu-west-1.aws.neon.build%5C%22%7D%20%7C~%20%60git-env%7CERR%7CWARN%60%22,%22queryType%22:%22range%22,%22datasource%22:%7B%22type%22:%22loki%22,%22uid%22:%22xHHYY0dVz%22%7D,%22editorMode%22:%22code%22%7D%5D,%22range%22:%7B%22to%22:%221709919114642%22,%22from%22:%221709904430898%22%7D%7D%7D). --- libs/metrics/src/lib.rs | 1 - libs/metrics/src/metric_vec_duration.rs | 23 --------- pageserver/src/metrics.rs | 63 +++++++++++++++++++++++-- pageserver/src/page_service.rs | 50 ++++++++++---------- 4 files changed, 83 insertions(+), 54 deletions(-) delete mode 100644 libs/metrics/src/metric_vec_duration.rs diff --git a/libs/metrics/src/lib.rs b/libs/metrics/src/lib.rs index 744fc18e61df..22b0a18933f5 100644 --- a/libs/metrics/src/lib.rs +++ b/libs/metrics/src/lib.rs @@ -29,7 +29,6 @@ pub mod launch_timestamp; mod wrappers; pub use wrappers::{CountedReader, CountedWriter}; mod hll; -pub mod metric_vec_duration; pub use hll::{HyperLogLog, HyperLogLogVec}; #[cfg(target_os = "linux")] pub mod more_process_metrics; diff --git a/libs/metrics/src/metric_vec_duration.rs b/libs/metrics/src/metric_vec_duration.rs deleted file mode 100644 index e9a0a655706f..000000000000 --- a/libs/metrics/src/metric_vec_duration.rs +++ /dev/null @@ -1,23 +0,0 @@ -//! Helpers for observing duration on `HistogramVec` / `CounterVec` / `GaugeVec` / `MetricVec`. - -use std::{future::Future, time::Instant}; - -pub trait DurationResultObserver { - fn observe_result(&self, res: &Result, duration: std::time::Duration); -} - -pub async fn observe_async_block_duration_by_result< - T, - E, - F: Future>, - O: DurationResultObserver, ->( - observer: &O, - block: F, -) -> Result { - let start = Instant::now(); - let result = block.await; - let duration = start.elapsed(); - observer.observe_result(&result, duration); - result -} diff --git a/pageserver/src/metrics.rs b/pageserver/src/metrics.rs index ee62ee0367e8..27e754e9990a 100644 --- a/pageserver/src/metrics.rs +++ b/pageserver/src/metrics.rs @@ -1,5 +1,4 @@ use enum_map::EnumMap; -use metrics::metric_vec_duration::DurationResultObserver; use metrics::{ register_counter_vec, register_gauge_vec, register_histogram, register_histogram_vec, register_int_counter, register_int_counter_pair_vec, register_int_counter_vec, @@ -1283,11 +1282,65 @@ pub(crate) static BASEBACKUP_QUERY_TIME: Lazy = Lazy::new(| }) }); -impl DurationResultObserver for BasebackupQueryTime { - fn observe_result(&self, res: &Result, duration: std::time::Duration) { +pub(crate) struct BasebackupQueryTimeOngoingRecording<'a, 'c> { + parent: &'a BasebackupQueryTime, + ctx: &'c RequestContext, + start: std::time::Instant, +} + +impl BasebackupQueryTime { + pub(crate) fn start_recording<'c: 'a, 'a>( + &'a self, + ctx: &'c RequestContext, + ) -> BasebackupQueryTimeOngoingRecording<'_, '_> { + let start = Instant::now(); + match ctx.micros_spent_throttled.open() { + Ok(()) => (), + Err(error) => { + use utils::rate_limit::RateLimit; + static LOGGED: Lazy> = + Lazy::new(|| Mutex::new(RateLimit::new(Duration::from_secs(10)))); + let mut rate_limit = LOGGED.lock().unwrap(); + rate_limit.call(|| { + warn!(error, "error opening micros_spent_throttled; this message is logged at a global rate limit"); + }); + } + } + BasebackupQueryTimeOngoingRecording { + parent: self, + ctx, + start, + } + } +} + +impl<'a, 'c> BasebackupQueryTimeOngoingRecording<'a, 'c> { + pub(crate) fn observe(self, res: &Result) { + let elapsed = self.start.elapsed(); + let ex_throttled = self + .ctx + .micros_spent_throttled + .close_and_checked_sub_from(elapsed); + let ex_throttled = match ex_throttled { + Ok(ex_throttled) => ex_throttled, + Err(error) => { + use utils::rate_limit::RateLimit; + static LOGGED: Lazy> = + Lazy::new(|| Mutex::new(RateLimit::new(Duration::from_secs(10)))); + let mut rate_limit = LOGGED.lock().unwrap(); + rate_limit.call(|| { + warn!(error, "error deducting time spent throttled; this message is logged at a global rate limit"); + }); + elapsed + } + }; let label_value = if res.is_ok() { "ok" } else { "error" }; - let metric = self.0.get_metric_with_label_values(&[label_value]).unwrap(); - metric.observe(duration.as_secs_f64()); + let metric = self + .parent + .0 + .get_metric_with_label_values(&[label_value]) + .unwrap(); + metric.observe(ex_throttled.as_secs_f64()); } } diff --git a/pageserver/src/page_service.rs b/pageserver/src/page_service.rs index dacee41e6ecb..f3ceb7d3e6e5 100644 --- a/pageserver/src/page_service.rs +++ b/pageserver/src/page_service.rs @@ -1199,7 +1199,7 @@ impl PageServerHandler { prev_lsn: Option, full_backup: bool, gzip: bool, - ctx: RequestContext, + ctx: &RequestContext, ) -> Result<(), QueryError> where IO: AsyncRead + AsyncWrite + Send + Sync + Unpin, @@ -1214,7 +1214,7 @@ impl PageServerHandler { if let Some(lsn) = lsn { // Backup was requested at a particular LSN. Wait for it to arrive. info!("waiting for {}", lsn); - timeline.wait_lsn(lsn, &ctx).await?; + timeline.wait_lsn(lsn, ctx).await?; timeline .check_lsn_is_in_scope(lsn, &latest_gc_cutoff_lsn) .context("invalid basebackup lsn")?; @@ -1236,7 +1236,7 @@ impl PageServerHandler { lsn, prev_lsn, full_backup, - &ctx, + ctx, ) .await?; } else { @@ -1257,7 +1257,7 @@ impl PageServerHandler { lsn, prev_lsn, full_backup, - &ctx, + ctx, ) .await?; // shutdown the encoder to ensure the gzip footer is written @@ -1269,7 +1269,7 @@ impl PageServerHandler { lsn, prev_lsn, full_backup, - &ctx, + ctx, ) .await?; } @@ -1449,25 +1449,25 @@ where false }; - ::metrics::metric_vec_duration::observe_async_block_duration_by_result( - &*metrics::BASEBACKUP_QUERY_TIME, - async move { - self.handle_basebackup_request( - pgb, - tenant_id, - timeline_id, - lsn, - None, - false, - gzip, - ctx, - ) - .await?; - pgb.write_message_noflush(&BeMessage::CommandComplete(b"SELECT 1"))?; - Result::<(), QueryError>::Ok(()) - }, - ) - .await?; + let metric_recording = metrics::BASEBACKUP_QUERY_TIME.start_recording(&ctx); + let res = async { + self.handle_basebackup_request( + pgb, + tenant_id, + timeline_id, + lsn, + None, + false, + gzip, + &ctx, + ) + .await?; + pgb.write_message_noflush(&BeMessage::CommandComplete(b"SELECT 1"))?; + Result::<(), QueryError>::Ok(()) + } + .await; + metric_recording.observe(&res); + res?; } // return pair of prev_lsn and last_lsn else if query_string.starts_with("get_last_record_rlsn ") { @@ -1563,7 +1563,7 @@ where prev_lsn, true, false, - ctx, + &ctx, ) .await?; pgb.write_message_noflush(&BeMessage::CommandComplete(b"SELECT 1"))?; From b09d68633510bdb12b017fb01ac055ffe7298833 Mon Sep 17 00:00:00 2001 From: Joonas Koivunen Date: Sat, 9 Mar 2024 15:09:08 +0200 Subject: [PATCH 14/43] fix: on-demand downloads can outlive timeline shutdown (#7051) ## Problem Before this PR, it was possible that on-demand downloads were started after `Timeline::shutdown()`. For example, we have observed a walreceiver-connection-handler-initiated on-demand download that was started after `Timeline::shutdown()`s final `task_mgr::shutdown_tasks()` call. The underlying issue is that `task_mgr::shutdown_tasks()` isn't sticky, i.e., new tasks can be spawned during or after `task_mgr::shutdown_tasks()`. Cc: https://github.com/neondatabase/neon/issues/4175 in lieu of a more specific issue for task_mgr. We already decided we want to get rid of it anyways. Original investigation: https://neondb.slack.com/archives/C033RQ5SPDH/p1709824952465949 ## Changes - enter gate while downloading - use timeline cancellation token for cancelling download thereby, fixes #7054 Entering the gate might also remove recent "kept the gate from closing" in staging. --- libs/remote_storage/tests/test_real_s3.rs | 26 +++++++++++-------- pageserver/src/task_mgr.rs | 3 --- pageserver/src/tenant/storage_layer/layer.rs | 27 ++++++++------------ test_runner/regress/test_tenant_delete.py | 2 ++ test_runner/regress/test_timeline_delete.py | 4 ++- 5 files changed, 31 insertions(+), 31 deletions(-) diff --git a/libs/remote_storage/tests/test_real_s3.rs b/libs/remote_storage/tests/test_real_s3.rs index e927b40e805a..d8b9824d99c4 100644 --- a/libs/remote_storage/tests/test_real_s3.rs +++ b/libs/remote_storage/tests/test_real_s3.rs @@ -17,6 +17,7 @@ use remote_storage::{ }; use test_context::test_context; use test_context::AsyncTestContext; +use tokio::io::AsyncBufReadExt; use tokio_util::sync::CancellationToken; use tracing::info; @@ -484,32 +485,33 @@ async fn download_is_cancelled(ctx: &mut MaybeEnabledStorage) { )) .unwrap(); - let len = upload_large_enough_file(&ctx.client, &path, &cancel).await; + let file_len = upload_large_enough_file(&ctx.client, &path, &cancel).await; { - let mut stream = ctx + let stream = ctx .client .download(&path, &cancel) .await .expect("download succeeds") .download_stream; - let first = stream - .next() - .await - .expect("should have the first blob") - .expect("should have succeeded"); + let mut reader = std::pin::pin!(tokio_util::io::StreamReader::new(stream)); + + let first = reader.fill_buf().await.expect("should have the first blob"); - tracing::info!(len = first.len(), "downloaded first chunk"); + let len = first.len(); + tracing::info!(len, "downloaded first chunk"); assert!( - first.len() < len, + first.len() < file_len, "uploaded file is too small, we downloaded all on first chunk" ); + reader.consume(len); + cancel.cancel(); - let next = stream.next().await.expect("stream should have more"); + let next = reader.fill_buf().await; let e = next.expect_err("expected an error, but got a chunk?"); @@ -520,6 +522,10 @@ async fn download_is_cancelled(ctx: &mut MaybeEnabledStorage) { .is_some_and(|e| matches!(e, DownloadError::Cancelled)), "{inner:?}" ); + + let e = DownloadError::from(e); + + assert!(matches!(e, DownloadError::Cancelled), "{e:?}"); } let cancel = CancellationToken::new(); diff --git a/pageserver/src/task_mgr.rs b/pageserver/src/task_mgr.rs index adaa55c179c5..275a72c0b012 100644 --- a/pageserver/src/task_mgr.rs +++ b/pageserver/src/task_mgr.rs @@ -272,9 +272,6 @@ pub enum TaskKind { // Task that uploads a file to remote storage RemoteUploadTask, - // Task that downloads a file from remote storage - RemoteDownloadTask, - // task that handles the initial downloading of all tenants InitialLoad, diff --git a/pageserver/src/tenant/storage_layer/layer.rs b/pageserver/src/tenant/storage_layer/layer.rs index 6c46b83622cc..aabb13b15c7a 100644 --- a/pageserver/src/tenant/storage_layer/layer.rs +++ b/pageserver/src/tenant/storage_layer/layer.rs @@ -880,23 +880,18 @@ impl LayerInner { ) -> Result { debug_assert_current_span_has_tenant_and_timeline_id(); - let task_name = format!("download layer {}", self); - let (tx, rx) = tokio::sync::oneshot::channel(); - // this is sadly needed because of task_mgr::shutdown_tasks, otherwise we cannot - // block tenant::mgr::remove_tenant_from_memory. - let this: Arc = self.clone(); - crate::task_mgr::spawn( - &tokio::runtime::Handle::current(), - crate::task_mgr::TaskKind::RemoteDownloadTask, - Some(self.desc.tenant_shard_id), - Some(self.desc.timeline_id), - &task_name, - false, - async move { + let guard = timeline + .gate + .enter() + .map_err(|_| DownloadError::DownloadCancelled)?; + + tokio::task::spawn(async move { + + let _guard = guard; let client = timeline .remote_client @@ -906,7 +901,7 @@ impl LayerInner { let result = client.download_layer_file( &this.desc.filename(), &this.metadata(), - &crate::task_mgr::shutdown_token() + &timeline.cancel ) .await; @@ -929,7 +924,6 @@ impl LayerInner { tokio::select! { _ = tokio::time::sleep(backoff) => {}, - _ = crate::task_mgr::shutdown_token().cancelled_owned() => {}, _ = timeline.cancel.cancelled() => {}, }; @@ -959,11 +953,10 @@ impl LayerInner { } } } - - Ok(()) } .in_current_span(), ); + match rx.await { Ok((Ok(()), permit)) => { if let Some(reason) = self diff --git a/test_runner/regress/test_tenant_delete.py b/test_runner/regress/test_tenant_delete.py index c4b4e5fb7781..52de8890843b 100644 --- a/test_runner/regress/test_tenant_delete.py +++ b/test_runner/regress/test_tenant_delete.py @@ -190,6 +190,8 @@ def test_delete_tenant_exercise_crash_safety_failpoints( # So by ignoring these instead of waiting for empty upload queue # we execute more distinct code paths. '.*stopping left-over name="remote upload".*', + # an on-demand is cancelled by shutdown + ".*initial size calculation failed: downloading failed, possibly for shutdown", ] ) diff --git a/test_runner/regress/test_timeline_delete.py b/test_runner/regress/test_timeline_delete.py index 795110d90be5..96a5cc491a05 100644 --- a/test_runner/regress/test_timeline_delete.py +++ b/test_runner/regress/test_timeline_delete.py @@ -213,7 +213,9 @@ def test_delete_timeline_exercise_crash_safety_failpoints( # This happens when timeline remains are cleaned up during loading ".*Timeline dir entry become invalid.*", # In one of the branches we poll for tenant to become active. Polls can generate this log message: - f".*Tenant {env.initial_tenant} is not active*", + f".*Tenant {env.initial_tenant} is not active.*", + # an on-demand is cancelled by shutdown + ".*initial size calculation failed: downloading failed, possibly for shutdown", ] ) From d894d2b4501d40a15589093a85ab7b9f98491701 Mon Sep 17 00:00:00 2001 From: Roman Zaynetdinov Date: Mon, 11 Mar 2024 10:10:04 +0200 Subject: [PATCH 15/43] Export db size, deadlocks and changed row metrics (#7050) ## Problem We want to report metrics for the oldest user database. --- vm-image-spec.yaml | 45 +++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 45 insertions(+) diff --git a/vm-image-spec.yaml b/vm-image-spec.yaml index c1b7ad533a78..5b93088303f5 100644 --- a/vm-image-spec.yaml +++ b/vm-image-spec.yaml @@ -142,6 +142,51 @@ files: query: | select datname, state, count(*) as count from pg_stat_activity where state <> '' group by datname, state; + - metric_name: pg_stats_userdb + type: gauge + help: 'Stats for the oldest non-system db' + key_labels: + - datname + value_label: kind + values: + - db_size + - deadlocks + # Rows + - inserted + - updated + - deleted + # We export stats for only one non-system database. Without this limit + # it is too easy to abuse the system by creating lots of databases. + # We can try lifting this limit in the future after we understand the needs better. + query: | + select pg_database_size(datname) as db_size, deadlocks, + tup_inserted as inserted, tup_updated as updated, tup_deleted as deleted, + datname + from pg_stat_database + where datname IN ( + select datname + from pg_database + where datname <> 'postgres' and not datistemplate + order by oid + limit 1 + ); + + - metric_name: max_cluster_size + type: gauge + help: 'neon.max_cluster_size setting' + key_labels: + values: [max_cluster_size] + query: | + select setting::int as max_cluster_size from pg_settings where name = 'neon.max_cluster_size'; + + - metric_name: db_total_size + type: gauge + help: 'Size of all databases' + key_labels: + values: [total] + query: | + select sum(pg_database_size(datname)) as total from pg_database; + build: | # Build cgroup-tools # From cc5d6c66b35ba91020d859e8bf39e92f040d0254 Mon Sep 17 00:00:00 2001 From: Conrad Ludgate Date: Mon, 11 Mar 2024 08:20:09 +0000 Subject: [PATCH 16/43] proxy: categorise new cplane error message (#7057) ## Problem `422 Unprocessable Entity: compute time quota of non-primary branches is exceeded` being marked as a control plane error. ## Summary of changes Add the manual checks to make this a user error that should not be retried. --- proxy/src/console/provider.rs | 13 ++++++++++++- proxy/src/proxy/wake_compute.rs | 6 ++++++ 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/proxy/src/console/provider.rs b/proxy/src/console/provider.rs index 0b74cd90ccc9..860960627352 100644 --- a/proxy/src/console/provider.rs +++ b/proxy/src/console/provider.rs @@ -73,7 +73,7 @@ pub mod errors { // Status 406: endpoint is disabled (we don't allow connections). format!("{REQUEST_FAILED}: endpoint is disabled") } - http::StatusCode::LOCKED => { + http::StatusCode::LOCKED | http::StatusCode::UNPROCESSABLE_ENTITY => { // Status 423: project might be in maintenance mode (or bad state), or quotas exceeded. format!("{REQUEST_FAILED}: endpoint is temporary unavailable. check your quotas and/or contact our support") } @@ -91,6 +91,12 @@ pub mod errors { status: http::StatusCode::NOT_FOUND | http::StatusCode::NOT_ACCEPTABLE, .. } => crate::error::ErrorKind::User, + ApiError::Console { + status: http::StatusCode::UNPROCESSABLE_ENTITY, + text, + } if text.contains("compute time quota of non-primary branches is exceeded") => { + crate::error::ErrorKind::User + } ApiError::Console { status: http::StatusCode::LOCKED, text, @@ -120,6 +126,11 @@ pub mod errors { status: http::StatusCode::BAD_REQUEST, .. } => true, + // don't retry when quotas are exceeded + Self::Console { + status: http::StatusCode::UNPROCESSABLE_ENTITY, + ref text, + } => !text.contains("compute time quota of non-primary branches is exceeded"), // locked can be returned when the endpoint was in transition // or when quotas are exceeded. don't retry when quotas are exceeded Self::Console { diff --git a/proxy/src/proxy/wake_compute.rs b/proxy/src/proxy/wake_compute.rs index 2c593451b401..bfe4b7ec3a25 100644 --- a/proxy/src/proxy/wake_compute.rs +++ b/proxy/src/proxy/wake_compute.rs @@ -69,6 +69,12 @@ fn report_error(e: &WakeComputeError, retry: bool) { { "quota_exceeded" } + WakeComputeError::ApiError(ApiError::Console { + status: StatusCode::UNPROCESSABLE_ENTITY, + ref text, + }) if text.contains("compute time quota of non-primary branches is exceeded") => { + "quota_exceeded" + } WakeComputeError::ApiError(ApiError::Console { status: StatusCode::LOCKED, .. From f8483cc4a38a06da2481dee557237298d8dc147b Mon Sep 17 00:00:00 2001 From: John Spray Date: Mon, 11 Mar 2024 09:32:17 +0000 Subject: [PATCH 17/43] pageserver: update swagger for HA APIs (#7070) - The type of heatmap_period in tenant config was wrrong - Secondary download and heatmap upload endpoints weren't in swagger. --- pageserver/src/http/openapi_spec.yml | 55 +++++++++++++++++++++++++++- pageserver/src/tenant/config.rs | 1 + 2 files changed, 55 insertions(+), 1 deletion(-) diff --git a/pageserver/src/http/openapi_spec.yml b/pageserver/src/http/openapi_spec.yml index d924224a321f..6a070e213594 100644 --- a/pageserver/src/http/openapi_spec.yml +++ b/pageserver/src/http/openapi_spec.yml @@ -932,6 +932,59 @@ paths: schema: $ref: "#/components/schemas/ServiceUnavailableError" + /v1/tenant/{tenant_shard_id}/heatmap_upload: + parameters: + - name: tenant_shard_id + in: path + required: true + schema: + type: string + post: + description: | + If the location is in an attached mode, upload the current state to the remote heatmap + responses: + "200": + description: Success + "500": + description: Generic operation error + content: + application/json: + schema: + $ref: "#/components/schemas/Error" + "503": + description: Temporarily unavailable, please retry. + content: + application/json: + schema: + $ref: "#/components/schemas/ServiceUnavailableError" + + /v1/tenant/{tenant_shard_id}/secondary/download: + parameters: + - name: tenant_shard_id + in: path + required: true + schema: + type: string + post: + description: | + If the location is in secondary mode, download latest heatmap and layers + responses: + "200": + description: Success + "500": + description: Generic operation error + content: + application/json: + schema: + $ref: "#/components/schemas/Error" + "503": + description: Temporarily unavailable, please retry. + content: + application/json: + schema: + $ref: "#/components/schemas/ServiceUnavailableError" + + /v1/tenant/{tenant_id}/timeline/: parameters: @@ -1391,7 +1444,7 @@ components: trace_read_requests: type: boolean heatmap_period: - type: integer + type: string TenantConfigResponse: type: object properties: diff --git a/pageserver/src/tenant/config.rs b/pageserver/src/tenant/config.rs index 9464324413a6..57fc444cdd8e 100644 --- a/pageserver/src/tenant/config.rs +++ b/pageserver/src/tenant/config.rs @@ -354,6 +354,7 @@ pub struct TenantConf { /// If non-zero, the period between uploads of a heatmap from attached tenants. This /// may be disabled if a Tenant will not have secondary locations: only secondary /// locations will use the heatmap uploaded by attached locations. + #[serde(with = "humantime_serde")] pub heatmap_period: Duration, /// If true then SLRU segments are dowloaded on demand, if false SLRU segments are included in basebackup From 26ae7b0b3e2e4371d644d9bdfe9baca4dc98418e Mon Sep 17 00:00:00 2001 From: Joonas Koivunen Date: Mon, 11 Mar 2024 15:25:53 +0200 Subject: [PATCH 18/43] fix(metrics): reset TENANT_STATE metric on startup (#7084) Otherwise, it might happen that we never get to witness the same state on subsequent restarts, thus the time series will show the value from a few restarts ago. The actual case here was that "Activating" was showing `3` while I was doing tenant migration testing on staging. The number 3 was however from a startup that happened some time ago which had been interrupted by another deployment. --- pageserver/src/metrics.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pageserver/src/metrics.rs b/pageserver/src/metrics.rs index 27e754e9990a..74e91210fc59 100644 --- a/pageserver/src/metrics.rs +++ b/pageserver/src/metrics.rs @@ -2676,6 +2676,12 @@ pub fn preinitialize_metrics() { Lazy::force(&crate::tenant::storage_layer::layer::LAYER_IMPL_METRICS); Lazy::force(&disk_usage_based_eviction::METRICS); + for state_name in pageserver_api::models::TenantState::VARIANTS { + // initialize the metric for all gauges, otherwise the time series might seemingly show + // values from last restart. + TENANT_STATE_METRIC.with_label_values(&[state_name]).set(0); + } + // countervecs [&BACKGROUND_LOOP_PERIOD_OVERRUN_COUNT] .into_iter() From b4972d07d41fce43550dc5ceb63806c3cf7d8f8d Mon Sep 17 00:00:00 2001 From: John Spray Date: Mon, 11 Mar 2024 14:29:32 +0000 Subject: [PATCH 19/43] storage controller: refactor non-mutable members up into Service (#7086) result_tx and compute_hook were in ServiceState (i.e. behind a sync mutex), but didn't need to be. Moving them up into Service removes a bunch of boilerplate clones. While we're here, create a helper `Service::maybe_reconcile_shard` which avoids writing out all the `&self.` arguments to `TenantState::maybe_reconcile` everywhere we call it. --- .../attachment_service/src/service.rs | 149 +++++------------- .../attachment_service/src/tenant_state.rs | 3 +- 2 files changed, 40 insertions(+), 112 deletions(-) diff --git a/control_plane/attachment_service/src/service.rs b/control_plane/attachment_service/src/service.rs index 556d6a682864..f3d97c0dfbaf 100644 --- a/control_plane/attachment_service/src/service.rs +++ b/control_plane/attachment_service/src/service.rs @@ -83,16 +83,10 @@ struct ServiceState { nodes: Arc>, scheduler: Scheduler, - - compute_hook: Arc, - - result_tx: tokio::sync::mpsc::UnboundedSender, } impl ServiceState { fn new( - config: Config, - result_tx: tokio::sync::mpsc::UnboundedSender, nodes: HashMap, tenants: BTreeMap, scheduler: Scheduler, @@ -101,8 +95,6 @@ impl ServiceState { tenants, nodes: Arc::new(nodes), scheduler, - compute_hook: Arc::new(ComputeHook::new(config)), - result_tx, } } @@ -152,6 +144,8 @@ pub struct Service { inner: Arc>, config: Config, persistence: Arc, + compute_hook: Arc, + result_tx: tokio::sync::mpsc::UnboundedSender, // Process shutdown will fire this token cancel: CancellationToken, @@ -481,8 +475,6 @@ impl Service { notifications: Vec<(TenantShardId, NodeId, ShardStripeSize)>, deadline: Instant, ) -> HashSet { - let compute_hook = self.inner.read().unwrap().compute_hook.clone(); - let attempt_shards = notifications.iter().map(|i| i.0).collect::>(); let mut success_shards = HashSet::new(); @@ -490,7 +482,7 @@ impl Service { // in order to subsequently use .buffered() on the stream to execute with bounded parallelism. let mut stream = futures::stream::iter(notifications.into_iter()) .map(|(tenant_shard_id, node_id, stripe_size)| { - let compute_hook = compute_hook.clone(); + let compute_hook = self.compute_hook.clone(); let cancel = self.cancel.clone(); async move { if let Err(e) = compute_hook @@ -730,14 +722,12 @@ impl Service { let this = Arc::new(Self { inner: Arc::new(std::sync::RwLock::new(ServiceState::new( - config.clone(), - result_tx, - nodes, - tenants, - scheduler, + nodes, tenants, scheduler, ))), - config, + config: config.clone(), persistence, + compute_hook: Arc::new(ComputeHook::new(config)), + result_tx, startup_complete: startup_complete.clone(), cancel: CancellationToken::new(), gate: Gate::default(), @@ -1145,8 +1135,6 @@ impl Service { let (waiters, response_shards) = { let mut locked = self.inner.write().unwrap(); - let result_tx = locked.result_tx.clone(); - let compute_hook = locked.compute_hook.clone(); let (nodes, tenants, scheduler) = locked.parts_mut(); let mut response_shards = Vec::new(); @@ -1231,17 +1219,7 @@ impl Service { let waiters = tenants .range_mut(TenantShardId::tenant_range(tenant_id)) - .filter_map(|(_shard_id, shard)| { - shard.maybe_reconcile( - result_tx.clone(), - nodes, - &compute_hook, - &self.config, - &self.persistence, - &self.gate, - &self.cancel, - ) - }) + .filter_map(|(_shard_id, shard)| self.maybe_reconcile_shard(shard, nodes)) .collect::>(); (waiters, response_shards) }; @@ -1432,8 +1410,6 @@ impl Service { let mut waiters = Vec::new(); { let mut locked = self.inner.write().unwrap(); - let result_tx = locked.result_tx.clone(); - let compute_hook = locked.compute_hook.clone(); let (nodes, tenants, scheduler) = locked.parts_mut(); for ShardUpdate { @@ -1461,15 +1437,7 @@ impl Service { shard.schedule(scheduler)?; - let maybe_waiter = shard.maybe_reconcile( - result_tx.clone(), - nodes, - &compute_hook, - &self.config, - &self.persistence, - &self.gate, - &self.cancel, - ); + let maybe_waiter = self.maybe_reconcile_shard(shard, nodes); if let Some(waiter) = maybe_waiter { waiters.push(waiter); } @@ -1514,20 +1482,10 @@ impl Service { let waiters = { let mut waiters = Vec::new(); let mut locked = self.inner.write().unwrap(); - let result_tx = locked.result_tx.clone(); - let compute_hook = locked.compute_hook.clone(); let (nodes, tenants, _scheduler) = locked.parts_mut(); for (_shard_id, shard) in tenants.range_mut(TenantShardId::tenant_range(tenant_id)) { shard.config = config.clone(); - if let Some(waiter) = shard.maybe_reconcile( - result_tx.clone(), - nodes, - &compute_hook, - &self.config, - &self.persistence, - &self.gate, - &self.cancel, - ) { + if let Some(waiter) = self.maybe_reconcile_shard(shard, nodes) { waiters.push(waiter); } } @@ -2159,7 +2117,7 @@ impl Service { } // Validate input, and calculate which shards we will create - let (old_shard_count, targets, compute_hook) = + let (old_shard_count, targets) = { let locked = self.inner.read().unwrap(); @@ -2255,7 +2213,7 @@ impl Service { } } - (old_shard_count, targets, locked.compute_hook.clone()) + (old_shard_count, targets) }; // unwrap safety: we would have returned above if we didn't find at least one shard to split @@ -2451,7 +2409,8 @@ impl Service { // Send compute notifications for all the new shards let mut failed_notifications = Vec::new(); for (child_id, child_ps, stripe_size) in child_locations { - if let Err(e) = compute_hook + if let Err(e) = self + .compute_hook .notify(child_id, child_ps, stripe_size, &self.cancel) .await { @@ -2481,8 +2440,6 @@ impl Service { ) -> Result { let waiter = { let mut locked = self.inner.write().unwrap(); - let result_tx = locked.result_tx.clone(); - let compute_hook = locked.compute_hook.clone(); let (nodes, tenants, scheduler) = locked.parts_mut(); let Some(node) = nodes.get(&migrate_req.node_id) else { @@ -2542,15 +2499,7 @@ impl Service { shard.sequence = shard.sequence.next(); } - shard.maybe_reconcile( - result_tx, - nodes, - &compute_hook, - &self.config, - &self.persistence, - &self.gate, - &self.cancel, - ) + self.maybe_reconcile_shard(shard, nodes) }; if let Some(waiter) = waiter { @@ -2814,8 +2763,6 @@ impl Service { } let mut locked = self.inner.write().unwrap(); - let result_tx = locked.result_tx.clone(); - let compute_hook = locked.compute_hook.clone(); let (nodes, tenants, scheduler) = locked.parts_mut(); let mut new_nodes = (**nodes).clone(); @@ -2867,16 +2814,8 @@ impl Service { tracing::warn!(%tenant_shard_id, "Scheduling error when marking pageserver {} offline: {e}", config_req.node_id); } Ok(()) => { - if tenant_state - .maybe_reconcile( - result_tx.clone(), - &new_nodes, - &compute_hook, - &self.config, - &self.persistence, - &self.gate, - &self.cancel, - ) + if self + .maybe_reconcile_shard(tenant_state, &new_nodes) .is_some() { tenants_affected += 1; @@ -2900,15 +2839,7 @@ impl Service { tenant_state.observed.locations.get_mut(&config_req.node_id) { if observed_loc.conf.is_none() { - tenant_state.maybe_reconcile( - result_tx.clone(), - &new_nodes, - &compute_hook, - &self.config, - &self.persistence, - &self.gate, - &self.cancel, - ); + self.maybe_reconcile_shard(tenant_state, &new_nodes); } } } @@ -2937,22 +2868,12 @@ impl Service { tenant_id: TenantId, ) -> Result, anyhow::Error> { let mut waiters = Vec::new(); - let result_tx = locked.result_tx.clone(); - let compute_hook = locked.compute_hook.clone(); let (nodes, tenants, scheduler) = locked.parts_mut(); for (_tenant_shard_id, shard) in tenants.range_mut(TenantShardId::tenant_range(tenant_id)) { shard.schedule(scheduler)?; - if let Some(waiter) = shard.maybe_reconcile( - result_tx.clone(), - nodes, - &compute_hook, - &self.config, - &self.persistence, - &self.gate, - &self.cancel, - ) { + if let Some(waiter) = self.maybe_reconcile_shard(shard, nodes) { waiters.push(waiter); } } @@ -2987,28 +2908,34 @@ impl Service { Ok(()) } + /// Convenience wrapper around [`TenantState::maybe_reconcile`] that provides + /// all the references to parts of Self that are needed + fn maybe_reconcile_shard( + &self, + shard: &mut TenantState, + nodes: &Arc>, + ) -> Option { + shard.maybe_reconcile( + &self.result_tx, + nodes, + &self.compute_hook, + &self.config, + &self.persistence, + &self.gate, + &self.cancel, + ) + } + /// Check all tenants for pending reconciliation work, and reconcile those in need /// /// Returns how many reconciliation tasks were started fn reconcile_all(&self) -> usize { let mut locked = self.inner.write().unwrap(); - let result_tx = locked.result_tx.clone(); - let compute_hook = locked.compute_hook.clone(); let pageservers = locked.nodes.clone(); locked .tenants .iter_mut() - .filter_map(|(_tenant_shard_id, shard)| { - shard.maybe_reconcile( - result_tx.clone(), - &pageservers, - &compute_hook, - &self.config, - &self.persistence, - &self.gate, - &self.cancel, - ) - }) + .filter_map(|(_tenant_shard_id, shard)| self.maybe_reconcile_shard(shard, &pageservers)) .count() } diff --git a/control_plane/attachment_service/src/tenant_state.rs b/control_plane/attachment_service/src/tenant_state.rs index c775736b313b..3c91e09ac319 100644 --- a/control_plane/attachment_service/src/tenant_state.rs +++ b/control_plane/attachment_service/src/tenant_state.rs @@ -617,7 +617,7 @@ impl TenantState { #[instrument(skip_all, fields(tenant_id=%self.tenant_shard_id.tenant_id, shard_id=%self.tenant_shard_id.shard_slug()))] pub(crate) fn maybe_reconcile( &mut self, - result_tx: tokio::sync::mpsc::UnboundedSender, + result_tx: &tokio::sync::mpsc::UnboundedSender, pageservers: &Arc>, compute_hook: &Arc, service_config: &service::Config, @@ -729,6 +729,7 @@ impl TenantState { tenant_id=%reconciler.tenant_shard_id.tenant_id, shard_id=%reconciler.tenant_shard_id.shard_slug()); metrics::RECONCILER.spawned.inc(); + let result_tx = result_tx.clone(); let join_handle = tokio::task::spawn( async move { // Wait for any previous reconcile task to complete before we start From 2b0f3549f7dad4ed7c62f89fada39f4e2ae33d34 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 11 Mar 2024 15:35:59 +0100 Subject: [PATCH 20/43] default to tokio-epoll-uring in CI tests & on Linux (#7077) All of production is using it now as of https://github.com/neondatabase/aws/pull/1121 The change in `flaky_tests.py` resets the flakiness detection logic. The alternative would have been to repeat the choice of io engine in each test name, which would junk up the various test reports too much. --------- Co-authored-by: Alexander Bayandin --- .github/workflows/build_and_test.yml | 4 ++-- pageserver/src/config.rs | 4 ++++ scripts/flaky_tests.py | 10 +++++++--- test_runner/fixtures/parametrize.py | 9 ++++++--- 4 files changed, 19 insertions(+), 8 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 276c71c6e0de..810c61de2d1a 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -474,7 +474,7 @@ jobs: TEST_RESULT_CONNSTR: ${{ secrets.REGRESS_TEST_RESULT_CONNSTR_NEW }} CHECK_ONDISK_DATA_COMPATIBILITY: nonempty BUILD_TAG: ${{ needs.tag.outputs.build-tag }} - PAGESERVER_VIRTUAL_FILE_IO_ENGINE: std-fs + PAGESERVER_VIRTUAL_FILE_IO_ENGINE: tokio-epoll-uring PAGESERVER_GET_VECTORED_IMPL: vectored # Temporary disable this step until we figure out why it's so flaky @@ -554,7 +554,7 @@ jobs: VIP_VAP_ACCESS_TOKEN: "${{ secrets.VIP_VAP_ACCESS_TOKEN }}" PERF_TEST_RESULT_CONNSTR: "${{ secrets.PERF_TEST_RESULT_CONNSTR }}" TEST_RESULT_CONNSTR: "${{ secrets.REGRESS_TEST_RESULT_CONNSTR_NEW }}" - PAGESERVER_VIRTUAL_FILE_IO_ENGINE: std-fs + PAGESERVER_VIRTUAL_FILE_IO_ENGINE: tokio-epoll-uring # XXX: no coverage data handling here, since benchmarks are run on release builds, # while coverage is currently collected for the debug ones diff --git a/pageserver/src/config.rs b/pageserver/src/config.rs index 437387164d8b..4adcedafd1c9 100644 --- a/pageserver/src/config.rs +++ b/pageserver/src/config.rs @@ -83,6 +83,10 @@ pub mod defaults { pub const DEFAULT_INGEST_BATCH_SIZE: u64 = 100; + #[cfg(target_os = "linux")] + pub const DEFAULT_VIRTUAL_FILE_IO_ENGINE: &str = "tokio-epoll-uring"; + + #[cfg(not(target_os = "linux"))] pub const DEFAULT_VIRTUAL_FILE_IO_ENGINE: &str = "std-fs"; pub const DEFAULT_GET_VECTORED_IMPL: &str = "sequential"; diff --git a/scripts/flaky_tests.py b/scripts/flaky_tests.py index 61a97f520d44..4464f09c29c7 100755 --- a/scripts/flaky_tests.py +++ b/scripts/flaky_tests.py @@ -15,7 +15,8 @@ DISTINCT parent_suite, suite, name FROM results WHERE - started_at > CURRENT_DATE - INTERVAL '%s' day + started_at > CURRENT_DATE - INTERVAL '10' day + AND started_at > '2024-03-11 11:32:12.874+00' -- TODO(update the date in a separate PR): we switched the default PAGESERVER_VIRTUAL_FILE_IO_ENGINE to `tokio-epoll-uring` from `std-fs` on this date, we want to ignore the flaky tests for `std-fs` AND ( (status IN ('failed', 'broken') AND reference = 'refs/heads/main') OR flaky @@ -46,11 +47,14 @@ def main(args: argparse.Namespace): logging.error("cannot fetch flaky tests from the DB due to an error", exc) rows = [] - # If a test run has non-default PAGESERVER_VIRTUAL_FILE_IO_ENGINE (i.e. not empty, not std-fs), + # If a test run has non-default PAGESERVER_VIRTUAL_FILE_IO_ENGINE (i.e. not empty, not tokio-epoll-uring), # use it to parametrize test name along with build_type and pg_version # # See test_runner/fixtures/parametrize.py for details - if (io_engine := os.getenv("PAGESERVER_VIRTUAL_FILE_IO_ENGINE", "")) not in ("", "std-fs"): + if (io_engine := os.getenv("PAGESERVER_VIRTUAL_FILE_IO_ENGINE", "")) not in ( + "", + "tokio-epoll-uring", + ): pageserver_virtual_file_io_engine_parameter = f"-{io_engine}" else: pageserver_virtual_file_io_engine_parameter = "" diff --git a/test_runner/fixtures/parametrize.py b/test_runner/fixtures/parametrize.py index 57ca1932b0a7..b28da83508f3 100644 --- a/test_runner/fixtures/parametrize.py +++ b/test_runner/fixtures/parametrize.py @@ -46,9 +46,12 @@ def pytest_generate_tests(metafunc: Metafunc): metafunc.parametrize("pg_version", pg_versions, ids=map(lambda v: f"pg{v}", pg_versions)) - # A hacky way to parametrize tests only for `pageserver_virtual_file_io_engine=tokio-epoll-uring` - # And do not change test name for default `pageserver_virtual_file_io_engine=std-fs` to keep tests statistics - if (io_engine := os.getenv("PAGESERVER_VIRTUAL_FILE_IO_ENGINE", "")) not in ("", "std-fs"): + # A hacky way to parametrize tests only for `pageserver_virtual_file_io_engine=std-fs` + # And do not change test name for default `pageserver_virtual_file_io_engine=tokio-epoll-uring` to keep tests statistics + if (io_engine := os.getenv("PAGESERVER_VIRTUAL_FILE_IO_ENGINE", "")) not in ( + "", + "tokio-epoll-uring", + ): metafunc.parametrize("pageserver_virtual_file_io_engine", [io_engine]) # For performance tests, parametrize also by platform From 8224580f3e0517a9d5792d2ddae275c0e26377d6 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 11 Mar 2024 15:41:41 +0100 Subject: [PATCH 21/43] fix(tenant/timeline metrics): race condition during shutdown + recreation (#7064) Tenant::shutdown or Timeline::shutdown completes and becomes externally observable before the corresponding Tenant/Timeline object is dropped. For example, after observing a Tenant::shutdown to complete, we could attach the same tenant_id again. The shut down Tenant object might still be around at the time of the attach. The race is then the following: - old object's metrics are still around - new object uses with_label_values - old object calls remove_label_values The outcome is that the new object will have the metric objects (they're an Arc internall) but the metrics won't be part of the internal registry and hence they'll be missing in `/metrics`. Later, when the new object gets shut down and tries to remove_label_value, it will observe an error because the metric was already removed by the old object. Changes ------- This PR moves metric removal to `shutdown()`. An alternative design would be to multi-version the metrics using a distinguishing label, or, to use a better metrics crate that allows removing metrics from the registry through the locally held metric handle instead of interacting with the (globally shared) registry. refs https://github.com/neondatabase/neon/pull/7051 --- pageserver/src/metrics.rs | 4 +--- pageserver/src/tenant.rs | 7 ++----- pageserver/src/tenant/timeline.rs | 2 ++ 3 files changed, 5 insertions(+), 8 deletions(-) diff --git a/pageserver/src/metrics.rs b/pageserver/src/metrics.rs index 74e91210fc59..814b3e1f9611 100644 --- a/pageserver/src/metrics.rs +++ b/pageserver/src/metrics.rs @@ -2017,10 +2017,8 @@ impl TimelineMetrics { pub(crate) fn resident_physical_size_get(&self) -> u64 { self.resident_physical_size_gauge.get() } -} -impl Drop for TimelineMetrics { - fn drop(&mut self) { + pub(crate) fn shutdown(&self) { let tenant_id = &self.tenant_id; let timeline_id = &self.timeline_id; let shard_id = &self.shard_id; diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index 4f4654422be5..961995b2d610 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -1846,6 +1846,8 @@ impl Tenant { // Wait for any in-flight operations to complete self.gate.close().await; + remove_tenant_metrics(&self.tenant_shard_id); + Ok(()) } @@ -3557,11 +3559,6 @@ async fn run_initdb( Ok(()) } -impl Drop for Tenant { - fn drop(&mut self) { - remove_tenant_metrics(&self.tenant_shard_id); - } -} /// Dump contents of a layer file to stdout. pub async fn dump_layerfile_from_path( path: &Utf8Path, diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 7004db1cb598..c017d30f45de 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -1257,6 +1257,8 @@ impl Timeline { // Finally wait until any gate-holders are complete self.gate.close().await; + + self.metrics.shutdown(); } pub(crate) fn set_state(&self, new_state: TimelineState) { From 8c5b3100904ac24a102fd086c076790d2c688e39 Mon Sep 17 00:00:00 2001 From: Joonas Koivunen Date: Mon, 11 Mar 2024 17:54:06 +0200 Subject: [PATCH 22/43] fix: Layer delete on drop and eviction can outlive timeline shutdown (#7082) This is a follow-up to #7051 where `LayerInner::drop` and `LayerInner::evict_blocking` were not noticed to require a gate before the file deletion. The lack of entering a gate opens up a similar possibility of deleting a layer file which a newer Timeline instance has already checked out to be resident in a similar case as #7051. --- pageserver/src/tenant/storage_layer/layer.rs | 56 ++++++++++++-------- 1 file changed, 33 insertions(+), 23 deletions(-) diff --git a/pageserver/src/tenant/storage_layer/layer.rs b/pageserver/src/tenant/storage_layer/layer.rs index aabb13b15c7a..959065bc4c41 100644 --- a/pageserver/src/tenant/storage_layer/layer.rs +++ b/pageserver/src/tenant/storage_layer/layer.rs @@ -536,6 +536,18 @@ impl Drop for LayerInner { // carry this until we are finished for [`Layer::wait_drop`] support let _status = status; + let Some(timeline) = timeline.upgrade() else { + // no need to nag that timeline is gone: under normal situation on + // task_mgr::remove_tenant_from_memory the timeline is gone before we get dropped. + LAYER_IMPL_METRICS.inc_deletes_failed(DeleteFailed::TimelineGone); + return; + }; + + let Ok(_guard) = timeline.gate.enter() else { + LAYER_IMPL_METRICS.inc_deletes_failed(DeleteFailed::TimelineGone); + return; + }; + let removed = match std::fs::remove_file(path) { Ok(()) => true, Err(e) if e.kind() == std::io::ErrorKind::NotFound => { @@ -554,32 +566,26 @@ impl Drop for LayerInner { } }; - if let Some(timeline) = timeline.upgrade() { - if removed { - timeline.metrics.resident_physical_size_sub(file_size); - } - if let Some(remote_client) = timeline.remote_client.as_ref() { - let res = remote_client.schedule_deletion_of_unlinked(vec![(file_name, meta)]); - - if let Err(e) = res { - // test_timeline_deletion_with_files_stuck_in_upload_queue is good at - // demonstrating this deadlock (without spawn_blocking): stop will drop - // queued items, which will have ResidentLayer's, and those drops would try - // to re-entrantly lock the RemoteTimelineClient inner state. - if !timeline.is_active() { - tracing::info!("scheduling deletion on drop failed: {e:#}"); - } else { - tracing::warn!("scheduling deletion on drop failed: {e:#}"); - } - LAYER_IMPL_METRICS.inc_deletes_failed(DeleteFailed::DeleteSchedulingFailed); + if removed { + timeline.metrics.resident_physical_size_sub(file_size); + } + if let Some(remote_client) = timeline.remote_client.as_ref() { + let res = remote_client.schedule_deletion_of_unlinked(vec![(file_name, meta)]); + + if let Err(e) = res { + // test_timeline_deletion_with_files_stuck_in_upload_queue is good at + // demonstrating this deadlock (without spawn_blocking): stop will drop + // queued items, which will have ResidentLayer's, and those drops would try + // to re-entrantly lock the RemoteTimelineClient inner state. + if !timeline.is_active() { + tracing::info!("scheduling deletion on drop failed: {e:#}"); } else { - LAYER_IMPL_METRICS.inc_completed_deletes(); + tracing::warn!("scheduling deletion on drop failed: {e:#}"); } + LAYER_IMPL_METRICS.inc_deletes_failed(DeleteFailed::DeleteSchedulingFailed); + } else { + LAYER_IMPL_METRICS.inc_completed_deletes(); } - } else { - // no need to nag that timeline is gone: under normal situation on - // task_mgr::remove_tenant_from_memory the timeline is gone before we get dropped. - LAYER_IMPL_METRICS.inc_deletes_failed(DeleteFailed::TimelineGone); } }); } @@ -1095,6 +1101,10 @@ impl LayerInner { return Err(EvictionCancelled::TimelineGone); }; + 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 = { From 17a3c9036e4da341d9f1ca05316eefb3e7575232 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Mon, 11 Mar 2024 17:36:49 +0100 Subject: [PATCH 23/43] follow-up(#7077): adjust flaky-test-detection cutoff date for tokio-epoll-uring (#7090) Co-authored-by: Alexander Bayandin --- scripts/flaky_tests.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scripts/flaky_tests.py b/scripts/flaky_tests.py index 4464f09c29c7..853c67d218da 100755 --- a/scripts/flaky_tests.py +++ b/scripts/flaky_tests.py @@ -16,7 +16,7 @@ FROM results WHERE started_at > CURRENT_DATE - INTERVAL '10' day - AND started_at > '2024-03-11 11:32:12.874+00' -- TODO(update the date in a separate PR): we switched the default PAGESERVER_VIRTUAL_FILE_IO_ENGINE to `tokio-epoll-uring` from `std-fs` on this date, we want to ignore the flaky tests for `std-fs` + AND started_at > '2024-03-11 14:50:11.845+00' -- we switched the default PAGESERVER_VIRTUAL_FILE_IO_ENGINE to `tokio-epoll-uring` from `std-fs` on this date, we want to ignore the flaky tests for `std-fs` AND ( (status IN ('failed', 'broken') AND reference = 'refs/heads/main') OR flaky From 73a8c97ac8280cefd103871b7e20bce3aae35635 Mon Sep 17 00:00:00 2001 From: Alex Chi Z Date: Mon, 11 Mar 2024 13:49:58 -0400 Subject: [PATCH 24/43] fix: warnings when compiling neon extensions (#7053) proceeding https://github.com/neondatabase/neon/pull/7010, close https://github.com/neondatabase/neon/issues/6188 ## Summary of changes This pull request (should) fix all warnings except `-Wdeclaration-after-statement` in the neon extension compilation. --------- Signed-off-by: Alex Chi Z --- pgxn/hnsw/hnsw.c | 2 +- pgxn/neon/extension_server.c | 1 - pgxn/neon/neon.c | 4 ---- pgxn/neon/pagestore_smgr.c | 1 - pgxn/neon/walproposer_pg.c | 3 +-- pgxn/neon_test_utils/neontest.c | 2 -- pgxn/neon_walredo/walredoproc.c | 3 +++ 7 files changed, 5 insertions(+), 11 deletions(-) diff --git a/pgxn/hnsw/hnsw.c b/pgxn/hnsw/hnsw.c index 45bf78ed3bc1..e624cb831fc1 100644 --- a/pgxn/hnsw/hnsw.c +++ b/pgxn/hnsw/hnsw.c @@ -149,7 +149,7 @@ hnsw_check_available_memory(Size requested) struct sysinfo si; Size total; if (sysinfo(&si) < 0) - elog(ERROR, "Failed to get amount of RAM: %n"); + elog(ERROR, "Failed to get amount of RAM: %m"); total = si.totalram*si.mem_unit; if ((Size)NBuffers*BLCKSZ + requested >= total) diff --git a/pgxn/neon/extension_server.c b/pgxn/neon/extension_server.c index 1329e2d17ba8..e38af08f8963 100644 --- a/pgxn/neon/extension_server.c +++ b/pgxn/neon/extension_server.c @@ -38,7 +38,6 @@ neon_download_extension_file_http(const char *filename, bool is_library) CURLcode res; char *compute_ctl_url; - char *postdata; bool ret = false; if (handle == NULL) diff --git a/pgxn/neon/neon.c b/pgxn/neon/neon.c index 1f456d9a3f5e..6ede78a57692 100644 --- a/pgxn/neon/neon.c +++ b/pgxn/neon/neon.c @@ -95,7 +95,6 @@ get_num_snap_files_lsn_threshold(void) DIR *dirdesc; struct dirent *de; char *snap_path = "pg_logical/snapshots/"; - int cnt = 0; int lsns_allocated = 1024; int lsns_num = 0; XLogRecPtr *lsns; @@ -161,9 +160,6 @@ get_num_snap_files_lsn_threshold(void) PGDLLEXPORT void LogicalSlotsMonitorMain(Datum main_arg) { - TimestampTz now, - last_checked; - /* Establish signal handlers. */ pqsignal(SIGUSR1, procsignal_sigusr1_handler); pqsignal(SIGHUP, SignalHandlerForConfigReload); diff --git a/pgxn/neon/pagestore_smgr.c b/pgxn/neon/pagestore_smgr.c index 213e39632838..0256de2b9aa8 100644 --- a/pgxn/neon/pagestore_smgr.c +++ b/pgxn/neon/pagestore_smgr.c @@ -1888,7 +1888,6 @@ neon_zeroextend(SMgrRelation reln, ForkNumber forkNum, BlockNumber blocknum, int nblocks, bool skipFsync) { const PGAlignedBlock buffer = {0}; - BlockNumber curblocknum = blocknum; int remblocks = nblocks; XLogRecPtr lsn = 0; diff --git a/pgxn/neon/walproposer_pg.c b/pgxn/neon/walproposer_pg.c index 7f07913fa684..cf76a495b515 100644 --- a/pgxn/neon/walproposer_pg.c +++ b/pgxn/neon/walproposer_pg.c @@ -1026,7 +1026,7 @@ static void StartProposerReplication(WalProposer *wp, StartReplicationCmd *cmd) { XLogRecPtr FlushPtr; - TimeLineID currTLI; + __attribute__((unused)) TimeLineID currTLI; #if PG_VERSION_NUM < 150000 if (ThisTimeLineID == 0) @@ -1230,7 +1230,6 @@ WalProposerRecovery(WalProposer *wp, Safekeeper *sk) TimeLineID timeline; XLogRecPtr startpos; XLogRecPtr endpos; - uint64 download_range_mb; startpos = GetLogRepRestartLSN(wp); if (startpos == InvalidXLogRecPtr) diff --git a/pgxn/neon_test_utils/neontest.c b/pgxn/neon_test_utils/neontest.c index 7c618848e298..82ce5be9f695 100644 --- a/pgxn/neon_test_utils/neontest.c +++ b/pgxn/neon_test_utils/neontest.c @@ -182,8 +182,6 @@ test_consume_memory(PG_FUNCTION_ARGS) Datum test_release_memory(PG_FUNCTION_ARGS) { - TimestampTz start; - if (PG_ARGISNULL(0)) { if (consume_cxt) diff --git a/pgxn/neon_walredo/walredoproc.c b/pgxn/neon_walredo/walredoproc.c index 1fdd3801c64d..c4ab22636b91 100644 --- a/pgxn/neon_walredo/walredoproc.c +++ b/pgxn/neon_walredo/walredoproc.c @@ -220,6 +220,9 @@ enter_seccomp_mode(void) } #endif /* HAVE_LIBSECCOMP */ +PGDLLEXPORT void +WalRedoMain(int argc, char *argv[]); + /* * Entry point for the WAL redo process. * From 98723844ee86fb3392fd59d7a9f60545257cee03 Mon Sep 17 00:00:00 2001 From: Sasha Krassovsky Date: Mon, 11 Mar 2024 10:36:39 -0800 Subject: [PATCH 25/43] Don't return from inside PG_TRY (#7095) ## Problem Returning from PG_TRY is a bug, and we currently do that ## Summary of changes Make it break and then return false. This should also help stabilize test_bad_connection.py --- pgxn/neon/libpagestore.c | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/pgxn/neon/libpagestore.c b/pgxn/neon/libpagestore.c index a3543bca78d6..e31de3c6b5aa 100644 --- a/pgxn/neon/libpagestore.c +++ b/pgxn/neon/libpagestore.c @@ -316,6 +316,7 @@ pageserver_connect(shardno_t shard_no, int elevel) static uint64_t delay_us = MIN_RECONNECT_INTERVAL_USEC; TimestampTz now; uint64_t us_since_last_connect; + bool broke_from_loop = false; Assert(page_servers[shard_no].conn == NULL); @@ -418,7 +419,9 @@ pageserver_connect(shardno_t shard_no, int elevel) neon_shard_log(shard_no, elevel, "could not complete handshake with pageserver: %s", msg); - return false; + /* Returning from inside PG_TRY is bad, so we break/return later */ + broke_from_loop = true; + break; } } } @@ -431,6 +434,11 @@ pageserver_connect(shardno_t shard_no, int elevel) } PG_END_TRY(); + if (broke_from_loop) + { + return false; + } + neon_shard_log(shard_no, LOG, "libpagestore: connected to '%s'", connstr); page_servers[shard_no].conn = conn; page_servers[shard_no].wes = wes; From 0cf0731d8bd2dc55187697a4f3b4b523c7e927e1 Mon Sep 17 00:00:00 2001 From: Arseny Sher Date: Mon, 11 Mar 2024 12:19:15 +0300 Subject: [PATCH 26/43] SIGQUIT instead of SIGKILL prewarmed postgres. To avoid orphaned processes using wiped datadir with confusing logging. --- compute_tools/src/compute.rs | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/compute_tools/src/compute.rs b/compute_tools/src/compute.rs index 96ab4a06a58e..0fa315682d56 100644 --- a/compute_tools/src/compute.rs +++ b/compute_tools/src/compute.rs @@ -17,6 +17,7 @@ use chrono::{DateTime, Utc}; use futures::future::join_all; use futures::stream::FuturesUnordered; use futures::StreamExt; +use nix::unistd::Pid; use postgres::error::SqlState; use postgres::{Client, NoTls}; use tracing::{debug, error, info, instrument, warn}; @@ -722,8 +723,12 @@ impl ComputeNode { // Stop it when it's ready info!("waiting for postgres"); wait_for_postgres(&mut pg, Path::new(pgdata))?; - pg.kill()?; - info!("sent kill signal"); + // SIGQUIT orders postgres to exit immediately. We don't want to SIGKILL + // it to avoid orphaned processes prowling around while datadir is + // wiped. + let pm_pid = Pid::from_raw(pg.id() as i32); + kill(pm_pid, Signal::SIGQUIT)?; + info!("sent SIGQUIT signal"); pg.wait()?; info!("done prewarming"); From 74d09b78c740039bb0c86752bf6858b3a37c6c9c Mon Sep 17 00:00:00 2001 From: Heikki Linnakangas Date: Fri, 9 Feb 2024 22:01:20 +0200 Subject: [PATCH 27/43] Keep walproposer alive until shutdown checkpoint is safe on safekepeers The walproposer pretends to be a walsender in many ways. It has a WalSnd slot, it claims to be a walsender by calling MarkPostmasterChildWalSender() etc. But one different to real walsenders was that the postmaster still treated it as a bgworker rather than a walsender. The difference is that at shutdown, walsenders are not killed until the very end, after the checkpointer process has written the shutdown checkpoint and exited. As a result, the walproposer always got killed before the shutdown checkpoint was written, so the shutdown checkpoint never made it to safekeepers. That's fine in principle, we don't require a clean shutdown after all. But it also feels a bit silly not to stream the shutdown checkpoint. It could be useful for initializing hot standby mode in a read replica, for example. Change postmaster to treat background workers that have called MarkPostmasterChildWalSender() as walsenders. That unfortunately requires another small change in postgres core. After doing that, walproposers stay alive longer. However, it also means that the checkpointer will wait for the walproposer to switch to WALSNDSTATE_STOPPING state, when the checkpointer sends the PROCSIG_WALSND_INIT_STOPPING signal. We don't have the machinery in walproposer to receive and handle that signal reliably. Instead, we mark walproposer as being in WALSNDSTATE_STOPPING always. In commit 568f91420a, I assumed that shutdown will wait for all the remaining WAL to be streamed to safekeepers, but before this commit that was not true, and the test became flaky. This should make it stable again. Some tests wrongly assumed that no WAL could have been written between pg_current_wal_flush_lsn and quick pg stop after it. Fix them by introducing flush_ep_to_pageserver which first stops the endpoint and then waits till all committed WAL reaches the pageserver. In passing extract safekeeper http client to its own module. --- libs/walproposer/src/api_bindings.rs | 4 +- libs/walproposer/src/walproposer.rs | 2 +- pgxn/neon/walproposer.c | 23 +- pgxn/neon/walproposer.h | 6 +- pgxn/neon/walproposer_pg.c | 102 ++++++- .../tests/walproposer_sim/walproposer_api.rs | 15 +- test_runner/fixtures/neon_fixtures.py | 277 ++++-------------- test_runner/fixtures/safekeeper/__init__.py | 0 test_runner/fixtures/safekeeper/http.py | 227 ++++++++++++++ test_runner/fixtures/safekeeper/utils.py | 11 + test_runner/regress/test_layer_eviction.py | 14 +- .../regress/test_layers_from_future.py | 5 +- test_runner/regress/test_ondemand_download.py | 3 +- test_runner/regress/test_wal_acceptor.py | 39 ++- vendor/postgres-v14 | 2 +- vendor/postgres-v15 | 2 +- vendor/postgres-v16 | 2 +- vendor/revisions.json | 7 +- 18 files changed, 460 insertions(+), 281 deletions(-) create mode 100644 test_runner/fixtures/safekeeper/__init__.py create mode 100644 test_runner/fixtures/safekeeper/http.py create mode 100644 test_runner/fixtures/safekeeper/utils.py diff --git a/libs/walproposer/src/api_bindings.rs b/libs/walproposer/src/api_bindings.rs index 8317e2fa03b4..f5ed6ebb977d 100644 --- a/libs/walproposer/src/api_bindings.rs +++ b/libs/walproposer/src/api_bindings.rs @@ -324,11 +324,11 @@ extern "C" fn finish_sync_safekeepers(wp: *mut WalProposer, lsn: XLogRecPtr) { } } -extern "C" fn process_safekeeper_feedback(wp: *mut WalProposer, commit_lsn: XLogRecPtr) { +extern "C" fn process_safekeeper_feedback(wp: *mut WalProposer) { unsafe { let callback_data = (*(*wp).config).callback_data; let api = callback_data as *mut Box; - (*api).process_safekeeper_feedback(&mut (*wp), commit_lsn) + (*api).process_safekeeper_feedback(&mut (*wp)) } } diff --git a/libs/walproposer/src/walproposer.rs b/libs/walproposer/src/walproposer.rs index 13fade220ce4..734967da3f9c 100644 --- a/libs/walproposer/src/walproposer.rs +++ b/libs/walproposer/src/walproposer.rs @@ -142,7 +142,7 @@ pub trait ApiImpl { todo!() } - fn process_safekeeper_feedback(&self, _wp: &mut WalProposer, _commit_lsn: u64) { + fn process_safekeeper_feedback(&mut self, _wp: &mut WalProposer) { todo!() } diff --git a/pgxn/neon/walproposer.c b/pgxn/neon/walproposer.c index 10487636aeda..9ff049335299 100644 --- a/pgxn/neon/walproposer.c +++ b/pgxn/neon/walproposer.c @@ -1220,7 +1220,7 @@ PrepareAppendRequest(WalProposer *wp, AppendRequestHeader *req, XLogRecPtr begin req->epochStartLsn = wp->propEpochStartLsn; req->beginLsn = beginLsn; req->endLsn = endLsn; - req->commitLsn = GetAcknowledgedByQuorumWALPosition(wp); + req->commitLsn = wp->commitLsn; req->truncateLsn = wp->truncateLsn; req->proposerId = wp->greetRequest.proposerId; } @@ -1405,7 +1405,7 @@ static bool RecvAppendResponses(Safekeeper *sk) { WalProposer *wp = sk->wp; - XLogRecPtr minQuorumLsn; + XLogRecPtr newCommitLsn; bool readAnything = false; while (true) @@ -1444,18 +1444,19 @@ RecvAppendResponses(Safekeeper *sk) if (!readAnything) return sk->state == SS_ACTIVE; - HandleSafekeeperResponse(wp); - + /* update commit_lsn */ + newCommitLsn = GetAcknowledgedByQuorumWALPosition(wp); /* - * Also send the new commit lsn to all the safekeepers. + * Send the new value to all safekeepers. */ - minQuorumLsn = GetAcknowledgedByQuorumWALPosition(wp); - if (minQuorumLsn > wp->lastSentCommitLsn) + if (newCommitLsn > wp->commitLsn) { + wp->commitLsn = newCommitLsn; BroadcastAppendRequest(wp); - wp->lastSentCommitLsn = minQuorumLsn; } + HandleSafekeeperResponse(wp); + return sk->state == SS_ACTIVE; } @@ -1632,11 +1633,9 @@ GetDonor(WalProposer *wp, XLogRecPtr *donor_lsn) static void HandleSafekeeperResponse(WalProposer *wp) { - XLogRecPtr minQuorumLsn; XLogRecPtr candidateTruncateLsn; - minQuorumLsn = GetAcknowledgedByQuorumWALPosition(wp); - wp->api.process_safekeeper_feedback(wp, minQuorumLsn); + wp->api.process_safekeeper_feedback(wp); /* * Try to advance truncateLsn -- the last record flushed to all @@ -1649,7 +1648,7 @@ HandleSafekeeperResponse(WalProposer *wp) * can't commit entries from previous term' in Raft); 2) */ candidateTruncateLsn = CalculateMinFlushLsn(wp); - candidateTruncateLsn = Min(candidateTruncateLsn, minQuorumLsn); + candidateTruncateLsn = Min(candidateTruncateLsn, wp->commitLsn); if (candidateTruncateLsn > wp->truncateLsn) { wp->truncateLsn = candidateTruncateLsn; diff --git a/pgxn/neon/walproposer.h b/pgxn/neon/walproposer.h index 53820f6e1b84..bc674fd979c4 100644 --- a/pgxn/neon/walproposer.h +++ b/pgxn/neon/walproposer.h @@ -564,7 +564,7 @@ typedef struct walproposer_api * backpressure feedback and to confirm WAL persistence (has been commited * on the quorum of safekeepers). */ - void (*process_safekeeper_feedback) (WalProposer *wp, XLogRecPtr commitLsn); + void (*process_safekeeper_feedback) (WalProposer *wp); /* * Write a log message to the internal log processor. This is used only @@ -646,8 +646,8 @@ typedef struct WalProposer /* WAL has been generated up to this point */ XLogRecPtr availableLsn; - /* last commitLsn broadcasted to safekeepers */ - XLogRecPtr lastSentCommitLsn; + /* cached GetAcknowledgedByQuorumWALPosition result */ + XLogRecPtr commitLsn; ProposerGreeting greetRequest; diff --git a/pgxn/neon/walproposer_pg.c b/pgxn/neon/walproposer_pg.c index cf76a495b515..8eec2f02c1c7 100644 --- a/pgxn/neon/walproposer_pg.c +++ b/pgxn/neon/walproposer_pg.c @@ -68,6 +68,8 @@ static WalproposerShmemState *walprop_shared; static WalProposerConfig walprop_config; static XLogRecPtr sentPtr = InvalidXLogRecPtr; static const walproposer_api walprop_pg; +static volatile sig_atomic_t got_SIGUSR2 = false; +static bool reported_sigusr2 = false; static void nwp_shmem_startup_hook(void); static void nwp_register_gucs(void); @@ -101,6 +103,8 @@ static void add_nwr_event_set(Safekeeper *sk, uint32 events); static void update_nwr_event_set(Safekeeper *sk, uint32 events); static void rm_safekeeper_event_set(Safekeeper *to_remove, bool is_sk); +static void CheckGracefulShutdown(WalProposer *wp); + static XLogRecPtr GetLogRepRestartLSN(WalProposer *wp); static void @@ -492,6 +496,24 @@ walprop_pg_init_standalone_sync_safekeepers(void) BackgroundWorkerUnblockSignals(); } +/* + * We pretend to be a walsender process, and the lifecycle of a walsender is + * slightly different than other procesess. At shutdown, walsender processes + * stay alive until the very end, after the checkpointer has written the + * shutdown checkpoint. When the checkpointer exits, the postmaster sends all + * remaining walsender processes SIGUSR2. On receiving SIGUSR2, we try to send + * the remaining WAL, and then exit. This ensures that the checkpoint record + * reaches durable storage (in safekeepers), before the server shuts down + * completely. + */ +static void +walprop_sigusr2(SIGNAL_ARGS) +{ + got_SIGUSR2 = true; + + SetLatch(MyLatch); +} + static void walprop_pg_init_bgworker(void) { @@ -503,6 +525,7 @@ walprop_pg_init_bgworker(void) pqsignal(SIGUSR1, procsignal_sigusr1_handler); pqsignal(SIGHUP, SignalHandlerForConfigReload); pqsignal(SIGTERM, die); + pqsignal(SIGUSR2, walprop_sigusr2); BackgroundWorkerUnblockSignals(); @@ -1075,14 +1098,26 @@ StartProposerReplication(WalProposer *wp, StartReplicationCmd *cmd) #endif /* - * When we first start replication the standby will be behind the primary. - * For some applications, for example synchronous replication, it is - * important to have a clear state for this initial catchup mode, so we - * can trigger actions when we change streaming state later. We may stay - * in this state for a long time, which is exactly why we want to be able - * to monitor whether or not we are still here. + * XXX: Move straight to STOPPING state, skipping the STREAMING state. + * + * This is a bit weird. Normal walsenders stay in STREAMING state, until + * the checkpointer signals them that it is about to start writing the + * shutdown checkpoint. The walsenders acknowledge that they have received + * that signal by switching to STOPPING state. That tells the walsenders + * that they must not write any new WAL. + * + * However, we cannot easily intercept that signal from the checkpointer. + * It's sent by WalSndInitStopping(), using + * SendProcSignal(PROCSIGNAL_WALSND_INIT_STOPPING). It's received by + * HandleWalSndInitStopping, which sets a process-local got_STOPPING flag. + * However, that's all private to walsender.c. + * + * We don't need to do anything special upon receiving the signal, the + * walproposer doesn't write any WAL anyway, so we skip the STREAMING + * state and go directly to STOPPING mode. That way, the checkpointer + * won't wait for us. */ - WalSndSetState(WALSNDSTATE_CATCHUP); + WalSndSetState(WALSNDSTATE_STOPPING); /* * Don't allow a request to stream from a future point in WAL that hasn't @@ -1122,6 +1157,8 @@ StartProposerReplication(WalProposer *wp, StartReplicationCmd *cmd) static void WalSndLoop(WalProposer *wp) { + XLogRecPtr flushPtr; + /* Clear any already-pending wakeups */ ResetLatch(MyLatch); @@ -1130,9 +1167,6 @@ WalSndLoop(WalProposer *wp) CHECK_FOR_INTERRUPTS(); XLogBroadcastWalProposer(wp); - - if (MyWalSnd->state == WALSNDSTATE_CATCHUP) - WalSndSetState(WALSNDSTATE_STREAMING); WalProposerPoll(wp); } } @@ -1744,6 +1778,9 @@ walprop_pg_wait_event_set(WalProposer *wp, long timeout, Safekeeper **sk, uint32 { ConditionVariableCancelSleep(); ResetLatch(MyLatch); + + CheckGracefulShutdown(wp); + *events = WL_LATCH_SET; return 1; } @@ -1797,6 +1834,41 @@ walprop_pg_finish_sync_safekeepers(WalProposer *wp, XLogRecPtr lsn) exit(0); } +/* + * Like vanilla walsender, on sigusr2 send all remaining WAL and exit. + * + * Note that unlike sync-safekeepers waiting here is not reliable: we + * don't check that majority of safekeepers received and persisted + * commit_lsn -- only that walproposer reached it (which immediately + * broadcasts new value). Doing that without incurring redundant control + * file syncing would need wp -> sk protocol change. OTOH unlike + * sync-safekeepers which must bump commit_lsn or basebackup will fail, + * this catchup is important only for tests where safekeepers/network + * don't crash on their own. + */ +static void +CheckGracefulShutdown(WalProposer *wp) +{ + if (got_SIGUSR2) + { + if (!reported_sigusr2) + { + XLogRecPtr flushPtr = walprop_pg_get_flush_rec_ptr(wp); + + wpg_log(LOG, "walproposer will send and wait for remaining WAL between %X/%X and %X/%X", + LSN_FORMAT_ARGS(wp->commitLsn), LSN_FORMAT_ARGS(flushPtr)); + reported_sigusr2 = true; + } + + if (wp->commitLsn >= walprop_pg_get_flush_rec_ptr(wp)) + { + wpg_log(LOG, "walproposer sent all WAL up to %X/%X, exiting", + LSN_FORMAT_ARGS(wp->commitLsn)); + proc_exit(0); + } + } +} + /* * Choose most advanced PageserverFeedback and set it to *rf. */ @@ -1877,7 +1949,7 @@ CombineHotStanbyFeedbacks(HotStandbyFeedback *hs, WalProposer *wp) * None of that is functional in sync-safekeepers. */ static void -walprop_pg_process_safekeeper_feedback(WalProposer *wp, XLogRecPtr commitLsn) +walprop_pg_process_safekeeper_feedback(WalProposer *wp) { HotStandbyFeedback hsFeedback; XLogRecPtr oldDiskConsistentLsn; @@ -1892,10 +1964,10 @@ walprop_pg_process_safekeeper_feedback(WalProposer *wp, XLogRecPtr commitLsn) replication_feedback_set(&quorumFeedback.rf); SetZenithCurrentClusterSize(quorumFeedback.rf.currentClusterSize); - if (commitLsn > quorumFeedback.flushLsn || oldDiskConsistentLsn != quorumFeedback.rf.disk_consistent_lsn) + if (wp->commitLsn > quorumFeedback.flushLsn || oldDiskConsistentLsn != quorumFeedback.rf.disk_consistent_lsn) { - if (commitLsn > quorumFeedback.flushLsn) - quorumFeedback.flushLsn = commitLsn; + if (wp->commitLsn > quorumFeedback.flushLsn) + quorumFeedback.flushLsn = wp->commitLsn; /* * Advance the replication slot to commitLsn. WAL before it is @@ -1928,6 +2000,8 @@ walprop_pg_process_safekeeper_feedback(WalProposer *wp, XLogRecPtr commitLsn) XidFromFullTransactionId(hsFeedback.catalog_xmin), EpochFromFullTransactionId(hsFeedback.catalog_xmin)); } + + CheckGracefulShutdown(wp); } static XLogRecPtr diff --git a/safekeeper/tests/walproposer_sim/walproposer_api.rs b/safekeeper/tests/walproposer_sim/walproposer_api.rs index 746cac019e81..5c79e9082bcd 100644 --- a/safekeeper/tests/walproposer_sim/walproposer_api.rs +++ b/safekeeper/tests/walproposer_sim/walproposer_api.rs @@ -196,6 +196,7 @@ pub struct SimulationApi { safekeepers: RefCell>, disk: Arc, redo_start_lsn: Option, + last_logged_commit_lsn: u64, shmem: UnsafeCell, config: Config, event_set: RefCell>, @@ -228,6 +229,7 @@ impl SimulationApi { safekeepers: RefCell::new(sk_conns), disk: args.disk, redo_start_lsn: args.redo_start_lsn, + last_logged_commit_lsn: 0, shmem: UnsafeCell::new(walproposer::bindings::WalproposerShmemState { mutex: 0, feedback: PageserverFeedback { @@ -596,14 +598,11 @@ impl ApiImpl for SimulationApi { } } - fn process_safekeeper_feedback( - &self, - wp: &mut walproposer::bindings::WalProposer, - commit_lsn: u64, - ) { - debug!("process_safekeeper_feedback, commit_lsn={}", commit_lsn); - if commit_lsn > wp.lastSentCommitLsn { - self.os.log_event(format!("commit_lsn;{}", commit_lsn)); + fn process_safekeeper_feedback(&mut self, wp: &mut walproposer::bindings::WalProposer) { + debug!("process_safekeeper_feedback, commit_lsn={}", wp.commitLsn); + if wp.commitLsn > self.last_logged_commit_lsn { + self.os.log_event(format!("commit_lsn;{}", wp.commitLsn)); + self.last_logged_commit_lsn = wp.commitLsn; } } diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index b933d391abb8..018de975dc9f 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -15,11 +15,11 @@ import time import uuid from contextlib import closing, contextmanager -from dataclasses import dataclass, field +from dataclasses import dataclass from datetime import datetime from enum import Enum from fcntl import LOCK_EX, LOCK_UN, flock -from functools import cached_property +from functools import cached_property, partial from itertools import chain, product from pathlib import Path from types import TracebackType @@ -70,6 +70,8 @@ default_remote_storage, remote_storage_to_toml_inline_table, ) +from fixtures.safekeeper.http import SafekeeperHttpClient +from fixtures.safekeeper.utils import are_walreceivers_absent from fixtures.types import Lsn, TenantId, TenantShardId, TimelineId from fixtures.utils import ( ATTACHMENT_NAME_REGEX, @@ -2547,6 +2549,20 @@ def run_capture( ) return base_path + def get_pg_controldata_checkpoint_lsn(self, pgdata: str) -> Lsn: + """ + Run pg_controldata on given datadir and extract checkpoint lsn. + """ + + pg_controldata_path = os.path.join(self.pg_bin_path, "pg_controldata") + cmd = f"{pg_controldata_path} -D {pgdata}" + result = subprocess.run(cmd, capture_output=True, text=True, shell=True) + checkpoint_lsn = re.findall( + "Latest checkpoint location:\\s+([0-9A-F]+/[0-9A-F]+)", result.stdout + )[0] + log.info(f"last checkpoint at {checkpoint_lsn}") + return Lsn(checkpoint_lsn) + @pytest.fixture(scope="function") def pg_bin(test_output_dir: Path, pg_distrib_dir: Path, pg_version: PgVersion) -> PgBin: @@ -3565,220 +3581,6 @@ def list_segments(self, tenant_id, timeline_id) -> List[str]: return segments -# Walreceiver as returned by sk's timeline status endpoint. -@dataclass -class Walreceiver: - conn_id: int - state: str - - -@dataclass -class SafekeeperTimelineStatus: - acceptor_epoch: int - pg_version: int # Not exactly a PgVersion, safekeeper returns version as int, for example 150002 for 15.2 - flush_lsn: Lsn - commit_lsn: Lsn - timeline_start_lsn: Lsn - backup_lsn: Lsn - peer_horizon_lsn: Lsn - remote_consistent_lsn: Lsn - walreceivers: List[Walreceiver] - - -@dataclass -class SafekeeperMetrics: - # These are metrics from Prometheus which uses float64 internally. - # As a consequence, values may differ from real original int64s. - flush_lsn_inexact: Dict[Tuple[TenantId, TimelineId], int] = field(default_factory=dict) - commit_lsn_inexact: Dict[Tuple[TenantId, TimelineId], int] = field(default_factory=dict) - - -class SafekeeperHttpClient(requests.Session): - HTTPError = requests.HTTPError - - def __init__(self, port: int, auth_token: Optional[str] = None, is_testing_enabled=False): - super().__init__() - self.port = port - self.auth_token = auth_token - self.is_testing_enabled = is_testing_enabled - - if auth_token is not None: - self.headers["Authorization"] = f"Bearer {auth_token}" - - def check_status(self): - self.get(f"http://localhost:{self.port}/v1/status").raise_for_status() - - def is_testing_enabled_or_skip(self): - if not self.is_testing_enabled: - pytest.skip("safekeeper was built without 'testing' feature") - - def configure_failpoints(self, config_strings: Tuple[str, str] | List[Tuple[str, str]]): - self.is_testing_enabled_or_skip() - - if isinstance(config_strings, tuple): - pairs = [config_strings] - else: - pairs = config_strings - - log.info(f"Requesting config failpoints: {repr(pairs)}") - - res = self.put( - f"http://localhost:{self.port}/v1/failpoints", - json=[{"name": name, "actions": actions} for name, actions in pairs], - ) - log.info(f"Got failpoints request response code {res.status_code}") - res.raise_for_status() - res_json = res.json() - assert res_json is None - return res_json - - def debug_dump(self, params: Optional[Dict[str, str]] = None) -> Dict[str, Any]: - params = params or {} - res = self.get(f"http://localhost:{self.port}/v1/debug_dump", params=params) - res.raise_for_status() - res_json = json.loads(res.text) - assert isinstance(res_json, dict) - return res_json - - def patch_control_file( - self, - tenant_id: TenantId, - timeline_id: TimelineId, - patch: Dict[str, Any], - ) -> Dict[str, Any]: - res = self.patch( - f"http://localhost:{self.port}/v1/tenant/{tenant_id}/timeline/{timeline_id}/control_file", - json={ - "updates": patch, - "apply_fields": list(patch.keys()), - }, - ) - res.raise_for_status() - res_json = res.json() - assert isinstance(res_json, dict) - return res_json - - def pull_timeline(self, body: Dict[str, Any]) -> Dict[str, Any]: - res = self.post(f"http://localhost:{self.port}/v1/pull_timeline", json=body) - res.raise_for_status() - res_json = res.json() - assert isinstance(res_json, dict) - return res_json - - def copy_timeline(self, tenant_id: TenantId, timeline_id: TimelineId, body: Dict[str, Any]): - res = self.post( - f"http://localhost:{self.port}/v1/tenant/{tenant_id}/timeline/{timeline_id}/copy", - json=body, - ) - res.raise_for_status() - - def timeline_digest( - self, tenant_id: TenantId, timeline_id: TimelineId, from_lsn: Lsn, until_lsn: Lsn - ) -> Dict[str, Any]: - res = self.get( - f"http://localhost:{self.port}/v1/tenant/{tenant_id}/timeline/{timeline_id}/digest", - params={ - "from_lsn": str(from_lsn), - "until_lsn": str(until_lsn), - }, - ) - res.raise_for_status() - res_json = res.json() - assert isinstance(res_json, dict) - return res_json - - def timeline_create( - self, - tenant_id: TenantId, - timeline_id: TimelineId, - pg_version: int, # Not exactly a PgVersion, safekeeper returns version as int, for example 150002 for 15.2 - commit_lsn: Lsn, - ): - body = { - "tenant_id": str(tenant_id), - "timeline_id": str(timeline_id), - "pg_version": pg_version, - "commit_lsn": str(commit_lsn), - } - res = self.post(f"http://localhost:{self.port}/v1/tenant/timeline", json=body) - res.raise_for_status() - - def timeline_status( - self, tenant_id: TenantId, timeline_id: TimelineId - ) -> SafekeeperTimelineStatus: - res = self.get(f"http://localhost:{self.port}/v1/tenant/{tenant_id}/timeline/{timeline_id}") - res.raise_for_status() - resj = res.json() - walreceivers = [Walreceiver(wr["conn_id"], wr["status"]) for wr in resj["walreceivers"]] - return SafekeeperTimelineStatus( - acceptor_epoch=resj["acceptor_state"]["epoch"], - pg_version=resj["pg_info"]["pg_version"], - flush_lsn=Lsn(resj["flush_lsn"]), - commit_lsn=Lsn(resj["commit_lsn"]), - timeline_start_lsn=Lsn(resj["timeline_start_lsn"]), - backup_lsn=Lsn(resj["backup_lsn"]), - peer_horizon_lsn=Lsn(resj["peer_horizon_lsn"]), - remote_consistent_lsn=Lsn(resj["remote_consistent_lsn"]), - walreceivers=walreceivers, - ) - - def record_safekeeper_info(self, tenant_id: TenantId, timeline_id: TimelineId, body): - res = self.post( - f"http://localhost:{self.port}/v1/record_safekeeper_info/{tenant_id}/{timeline_id}", - json=body, - ) - res.raise_for_status() - - # only_local doesn't remove segments in the remote storage. - def timeline_delete( - self, tenant_id: TenantId, timeline_id: TimelineId, only_local: bool = False - ) -> Dict[Any, Any]: - res = self.delete( - f"http://localhost:{self.port}/v1/tenant/{tenant_id}/timeline/{timeline_id}", - params={ - "only_local": str(only_local).lower(), - }, - ) - res.raise_for_status() - res_json = res.json() - assert isinstance(res_json, dict) - return res_json - - def tenant_delete_force(self, tenant_id: TenantId) -> Dict[Any, Any]: - res = self.delete(f"http://localhost:{self.port}/v1/tenant/{tenant_id}") - res.raise_for_status() - res_json = res.json() - assert isinstance(res_json, dict) - return res_json - - def get_metrics_str(self) -> str: - request_result = self.get(f"http://localhost:{self.port}/metrics") - request_result.raise_for_status() - return request_result.text - - def get_metrics(self) -> SafekeeperMetrics: - all_metrics_text = self.get_metrics_str() - - metrics = SafekeeperMetrics() - for match in re.finditer( - r'^safekeeper_flush_lsn{tenant_id="([0-9a-f]+)",timeline_id="([0-9a-f]+)"} (\S+)$', - all_metrics_text, - re.MULTILINE, - ): - metrics.flush_lsn_inexact[(TenantId(match.group(1)), TimelineId(match.group(2)))] = int( - match.group(3) - ) - for match in re.finditer( - r'^safekeeper_commit_lsn{tenant_id="([0-9a-f]+)",timeline_id="([0-9a-f]+)"} (\S+)$', - all_metrics_text, - re.MULTILINE, - ): - metrics.commit_lsn_inexact[ - (TenantId(match.group(1)), TimelineId(match.group(2))) - ] = int(match.group(3)) - return metrics - - class S3Scrubber: def __init__(self, env: NeonEnvBuilder, log_dir: Optional[Path] = None): self.env = env @@ -4262,6 +4064,49 @@ def wait_for_last_flush_lsn( return min(results) +def flush_ep_to_pageserver( + env: NeonEnv, + ep: Endpoint, + tenant: TenantId, + timeline: TimelineId, + pageserver_id: Optional[int] = None, +) -> Lsn: + """ + Stop endpoint and wait until all committed WAL reaches the pageserver + (last_record_lsn). This is for use by tests which want everything written so + far to reach pageserver *and* expecting that no more data will arrive until + endpoint starts again, so unlike wait_for_last_flush_lsn it polls + safekeepers instead of compute to learn LSN. + + Returns the catch up LSN. + """ + ep.stop() + + commit_lsn: Lsn = Lsn(0) + # In principle in the absense of failures polling single sk would be enough. + for sk in env.safekeepers: + cli = sk.http_client() + # wait until compute connections are gone + wait_until(30, 0.5, partial(are_walreceivers_absent, cli, tenant, timeline)) + commit_lsn = max(cli.get_commit_lsn(tenant, timeline), commit_lsn) + + # Note: depending on WAL filtering implementation, probably most shards + # won't be able to reach commit_lsn (unless gaps are also ack'ed), so this + # is broken in sharded case. + shards = tenant_get_shards(env, tenant, pageserver_id) + for tenant_shard_id, pageserver in shards: + log.info( + f"flush_ep_to_pageserver: waiting for {commit_lsn} on shard {tenant_shard_id} on pageserver {pageserver.id})" + ) + waited = wait_for_last_record_lsn( + pageserver.http_client(), tenant_shard_id, timeline, commit_lsn + ) + + assert waited >= commit_lsn + + return commit_lsn + + def wait_for_wal_insert_lsn( env: NeonEnv, endpoint: Endpoint, diff --git a/test_runner/fixtures/safekeeper/__init__.py b/test_runner/fixtures/safekeeper/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/test_runner/fixtures/safekeeper/http.py b/test_runner/fixtures/safekeeper/http.py new file mode 100644 index 000000000000..b9c1986818b0 --- /dev/null +++ b/test_runner/fixtures/safekeeper/http.py @@ -0,0 +1,227 @@ +import json +import re +from dataclasses import dataclass, field +from typing import Any, Dict, List, Optional, Tuple, Union + +import pytest +import requests + +from fixtures.log_helper import log +from fixtures.types import Lsn, TenantId, TimelineId + + +# Walreceiver as returned by sk's timeline status endpoint. +@dataclass +class Walreceiver: + conn_id: int + state: str + + +@dataclass +class SafekeeperTimelineStatus: + acceptor_epoch: int + pg_version: int # Not exactly a PgVersion, safekeeper returns version as int, for example 150002 for 15.2 + flush_lsn: Lsn + commit_lsn: Lsn + timeline_start_lsn: Lsn + backup_lsn: Lsn + peer_horizon_lsn: Lsn + remote_consistent_lsn: Lsn + walreceivers: List[Walreceiver] + + +@dataclass +class SafekeeperMetrics: + # These are metrics from Prometheus which uses float64 internally. + # As a consequence, values may differ from real original int64s. + flush_lsn_inexact: Dict[Tuple[TenantId, TimelineId], int] = field(default_factory=dict) + commit_lsn_inexact: Dict[Tuple[TenantId, TimelineId], int] = field(default_factory=dict) + + +class SafekeeperHttpClient(requests.Session): + HTTPError = requests.HTTPError + + def __init__(self, port: int, auth_token: Optional[str] = None, is_testing_enabled=False): + super().__init__() + self.port = port + self.auth_token = auth_token + self.is_testing_enabled = is_testing_enabled + + if auth_token is not None: + self.headers["Authorization"] = f"Bearer {auth_token}" + + def check_status(self): + self.get(f"http://localhost:{self.port}/v1/status").raise_for_status() + + def is_testing_enabled_or_skip(self): + if not self.is_testing_enabled: + pytest.skip("safekeeper was built without 'testing' feature") + + def configure_failpoints(self, config_strings: Union[Tuple[str, str], List[Tuple[str, str]]]): + self.is_testing_enabled_or_skip() + + if isinstance(config_strings, tuple): + pairs = [config_strings] + else: + pairs = config_strings + + log.info(f"Requesting config failpoints: {repr(pairs)}") + + res = self.put( + f"http://localhost:{self.port}/v1/failpoints", + json=[{"name": name, "actions": actions} for name, actions in pairs], + ) + log.info(f"Got failpoints request response code {res.status_code}") + res.raise_for_status() + res_json = res.json() + assert res_json is None + return res_json + + def debug_dump(self, params: Optional[Dict[str, str]] = None) -> Dict[str, Any]: + params = params or {} + res = self.get(f"http://localhost:{self.port}/v1/debug_dump", params=params) + res.raise_for_status() + res_json = json.loads(res.text) + assert isinstance(res_json, dict) + return res_json + + def patch_control_file( + self, + tenant_id: TenantId, + timeline_id: TimelineId, + patch: Dict[str, Any], + ) -> Dict[str, Any]: + res = self.patch( + f"http://localhost:{self.port}/v1/tenant/{tenant_id}/timeline/{timeline_id}/control_file", + json={ + "updates": patch, + "apply_fields": list(patch.keys()), + }, + ) + res.raise_for_status() + res_json = res.json() + assert isinstance(res_json, dict) + return res_json + + def pull_timeline(self, body: Dict[str, Any]) -> Dict[str, Any]: + res = self.post(f"http://localhost:{self.port}/v1/pull_timeline", json=body) + res.raise_for_status() + res_json = res.json() + assert isinstance(res_json, dict) + return res_json + + def copy_timeline(self, tenant_id: TenantId, timeline_id: TimelineId, body: Dict[str, Any]): + res = self.post( + f"http://localhost:{self.port}/v1/tenant/{tenant_id}/timeline/{timeline_id}/copy", + json=body, + ) + res.raise_for_status() + + def timeline_digest( + self, tenant_id: TenantId, timeline_id: TimelineId, from_lsn: Lsn, until_lsn: Lsn + ) -> Dict[str, Any]: + res = self.get( + f"http://localhost:{self.port}/v1/tenant/{tenant_id}/timeline/{timeline_id}/digest", + params={ + "from_lsn": str(from_lsn), + "until_lsn": str(until_lsn), + }, + ) + res.raise_for_status() + res_json = res.json() + assert isinstance(res_json, dict) + return res_json + + def timeline_create( + self, + tenant_id: TenantId, + timeline_id: TimelineId, + pg_version: int, # Not exactly a PgVersion, safekeeper returns version as int, for example 150002 for 15.2 + commit_lsn: Lsn, + ): + body = { + "tenant_id": str(tenant_id), + "timeline_id": str(timeline_id), + "pg_version": pg_version, + "commit_lsn": str(commit_lsn), + } + res = self.post(f"http://localhost:{self.port}/v1/tenant/timeline", json=body) + res.raise_for_status() + + def timeline_status( + self, tenant_id: TenantId, timeline_id: TimelineId + ) -> SafekeeperTimelineStatus: + res = self.get(f"http://localhost:{self.port}/v1/tenant/{tenant_id}/timeline/{timeline_id}") + res.raise_for_status() + resj = res.json() + walreceivers = [Walreceiver(wr["conn_id"], wr["status"]) for wr in resj["walreceivers"]] + return SafekeeperTimelineStatus( + acceptor_epoch=resj["acceptor_state"]["epoch"], + pg_version=resj["pg_info"]["pg_version"], + flush_lsn=Lsn(resj["flush_lsn"]), + commit_lsn=Lsn(resj["commit_lsn"]), + timeline_start_lsn=Lsn(resj["timeline_start_lsn"]), + backup_lsn=Lsn(resj["backup_lsn"]), + peer_horizon_lsn=Lsn(resj["peer_horizon_lsn"]), + remote_consistent_lsn=Lsn(resj["remote_consistent_lsn"]), + walreceivers=walreceivers, + ) + + def get_commit_lsn(self, tenant_id: TenantId, timeline_id: TimelineId) -> Lsn: + return self.timeline_status(tenant_id, timeline_id).commit_lsn + + def record_safekeeper_info(self, tenant_id: TenantId, timeline_id: TimelineId, body): + res = self.post( + f"http://localhost:{self.port}/v1/record_safekeeper_info/{tenant_id}/{timeline_id}", + json=body, + ) + res.raise_for_status() + + # only_local doesn't remove segments in the remote storage. + def timeline_delete( + self, tenant_id: TenantId, timeline_id: TimelineId, only_local: bool = False + ) -> Dict[Any, Any]: + res = self.delete( + f"http://localhost:{self.port}/v1/tenant/{tenant_id}/timeline/{timeline_id}", + params={ + "only_local": str(only_local).lower(), + }, + ) + res.raise_for_status() + res_json = res.json() + assert isinstance(res_json, dict) + return res_json + + def tenant_delete_force(self, tenant_id: TenantId) -> Dict[Any, Any]: + res = self.delete(f"http://localhost:{self.port}/v1/tenant/{tenant_id}") + res.raise_for_status() + res_json = res.json() + assert isinstance(res_json, dict) + return res_json + + def get_metrics_str(self) -> str: + request_result = self.get(f"http://localhost:{self.port}/metrics") + request_result.raise_for_status() + return request_result.text + + def get_metrics(self) -> SafekeeperMetrics: + all_metrics_text = self.get_metrics_str() + + metrics = SafekeeperMetrics() + for match in re.finditer( + r'^safekeeper_flush_lsn{tenant_id="([0-9a-f]+)",timeline_id="([0-9a-f]+)"} (\S+)$', + all_metrics_text, + re.MULTILINE, + ): + metrics.flush_lsn_inexact[(TenantId(match.group(1)), TimelineId(match.group(2)))] = int( + match.group(3) + ) + for match in re.finditer( + r'^safekeeper_commit_lsn{tenant_id="([0-9a-f]+)",timeline_id="([0-9a-f]+)"} (\S+)$', + all_metrics_text, + re.MULTILINE, + ): + metrics.commit_lsn_inexact[ + (TenantId(match.group(1)), TimelineId(match.group(2))) + ] = int(match.group(3)) + return metrics diff --git a/test_runner/fixtures/safekeeper/utils.py b/test_runner/fixtures/safekeeper/utils.py new file mode 100644 index 000000000000..2818a493d6ee --- /dev/null +++ b/test_runner/fixtures/safekeeper/utils.py @@ -0,0 +1,11 @@ +from fixtures.log_helper import log +from fixtures.safekeeper.http import SafekeeperHttpClient +from fixtures.types import TenantId, TimelineId + + +def are_walreceivers_absent( + sk_http_cli: SafekeeperHttpClient, tenant_id: TenantId, timeline_id: TimelineId +): + status = sk_http_cli.timeline_status(tenant_id, timeline_id) + log.info(f"waiting for walreceivers to be gone, currently {status.walreceivers}") + return len(status.walreceivers) == 0 diff --git a/test_runner/regress/test_layer_eviction.py b/test_runner/regress/test_layer_eviction.py index efba2033fb92..7bbc0cc16006 100644 --- a/test_runner/regress/test_layer_eviction.py +++ b/test_runner/regress/test_layer_eviction.py @@ -4,12 +4,11 @@ from fixtures.log_helper import log from fixtures.neon_fixtures import ( NeonEnvBuilder, + flush_ep_to_pageserver, wait_for_last_flush_lsn, ) -from fixtures.pageserver.utils import wait_for_last_record_lsn, wait_for_upload +from fixtures.pageserver.utils import wait_for_upload from fixtures.remote_storage import RemoteStorageKind -from fixtures.types import Lsn -from fixtures.utils import query_scalar # Crates a few layers, ensures that we can evict them (removing locally but keeping track of them anyway) @@ -46,14 +45,15 @@ def test_basic_eviction( FROM generate_series(1, 5000000) g """ ) - current_lsn = Lsn(query_scalar(cur, "SELECT pg_current_wal_flush_lsn()")) - wait_for_last_record_lsn(client, tenant_id, timeline_id, current_lsn) + # stops the endpoint + current_lsn = flush_ep_to_pageserver(env, endpoint, tenant_id, timeline_id) + client.timeline_checkpoint(tenant_id, timeline_id) wait_for_upload(client, tenant_id, timeline_id, current_lsn) - # disable compute & sks to avoid on-demand downloads by walreceiver / getpage - endpoint.stop() + # stop sks to avoid on-demand downloads by walreceiver / getpage; endpoint + # has already been stopped by flush_ep_to_pageserver for sk in env.safekeepers: sk.stop() diff --git a/test_runner/regress/test_layers_from_future.py b/test_runner/regress/test_layers_from_future.py index 9da47b9fd33d..abdebb6d7959 100644 --- a/test_runner/regress/test_layers_from_future.py +++ b/test_runner/regress/test_layers_from_future.py @@ -1,7 +1,7 @@ import time from fixtures.log_helper import log -from fixtures.neon_fixtures import NeonEnvBuilder +from fixtures.neon_fixtures import NeonEnvBuilder, flush_ep_to_pageserver from fixtures.pageserver.types import ( DeltaLayerFileName, ImageLayerFileName, @@ -115,8 +115,7 @@ def get_future_layers(): ) == 0 ) - - endpoint.stop() + last_record_lsn = flush_ep_to_pageserver(env, endpoint, tenant_id, timeline_id) wait_for_upload_queue_empty(ps_http, tenant_id, timeline_id) diff --git a/test_runner/regress/test_ondemand_download.py b/test_runner/regress/test_ondemand_download.py index 8bbf50373e69..914f068afb7c 100644 --- a/test_runner/regress/test_ondemand_download.py +++ b/test_runner/regress/test_ondemand_download.py @@ -8,6 +8,7 @@ from fixtures.log_helper import log from fixtures.neon_fixtures import ( NeonEnvBuilder, + flush_ep_to_pageserver, last_flush_lsn_upload, wait_for_last_flush_lsn, ) @@ -517,7 +518,7 @@ def downloaded_bytes_and_count(pageserver_http: PageserverHttpClient) -> Tuple[i with endpoint.cursor() as cur: cur.execute("update a set id = -id") - wait_for_last_flush_lsn(env, endpoint, tenant_id, timeline_id) + flush_ep_to_pageserver(env, endpoint, tenant_id, timeline_id) pageserver_http.timeline_checkpoint(tenant_id, timeline_id) layers = pageserver_http.layer_map_info(tenant_id, timeline_id) diff --git a/test_runner/regress/test_wal_acceptor.py b/test_runner/regress/test_wal_acceptor.py index 3d7bba6153ba..2cac58dc1ae2 100644 --- a/test_runner/regress/test_wal_acceptor.py +++ b/test_runner/regress/test_wal_acceptor.py @@ -28,7 +28,6 @@ PgBin, PgProtocol, Safekeeper, - SafekeeperHttpClient, SafekeeperPort, last_flush_lsn_upload, ) @@ -46,6 +45,8 @@ default_remote_storage, s3_storage, ) +from fixtures.safekeeper.http import SafekeeperHttpClient +from fixtures.safekeeper.utils import are_walreceivers_absent from fixtures.types import Lsn, TenantId, TimelineId from fixtures.utils import get_dir_size, query_scalar, start_in_background @@ -1097,12 +1098,6 @@ def is_flush_lsn_aligned(sk_http_clis, tenant_id, timeline_id): return all([flush_lsns[0] == flsn for flsn in flush_lsns]) -def are_walreceivers_absent(sk_http_cli, tenant_id: TenantId, timeline_id: TimelineId): - status = sk_http_cli.timeline_status(tenant_id, timeline_id) - log.info(f"waiting for walreceivers to be gone, currently {status.walreceivers}") - return len(status.walreceivers) == 0 - - # Assert by xxd that WAL on given safekeepers is identical. No compute must be # running for this to be reliable. def cmp_sk_wal(sks: List[Safekeeper], tenant_id: TenantId, timeline_id: TimelineId): @@ -1347,6 +1342,36 @@ def test_peer_recovery(neon_env_builder: NeonEnvBuilder): endpoint.safe_psql("insert into t select generate_series(1,100), 'payload'") +# Test that when compute is terminated in fast (or smart) mode, walproposer is +# allowed to run and self terminate after shutdown checkpoint is written, so it +# commits it to safekeepers before exiting. This not required for correctness, +# but needed for tests using check_restored_datadir_content. +def test_wp_graceful_shutdown(neon_env_builder: NeonEnvBuilder, pg_bin: PgBin): + neon_env_builder.num_safekeepers = 1 + env = neon_env_builder.init_start() + + tenant_id = env.initial_tenant + timeline_id = env.neon_cli.create_branch("test_wp_graceful_shutdown") + ep = env.endpoints.create_start("test_wp_graceful_shutdown") + ep.safe_psql("create table t(key int, value text)") + ep.stop() + + # figure out checkpoint lsn + ckpt_lsn = pg_bin.get_pg_controldata_checkpoint_lsn(ep.pg_data_dir_path()) + + sk_http_cli = env.safekeepers[0].http_client() + commit_lsn = sk_http_cli.timeline_status(tenant_id, timeline_id).commit_lsn + # Note: this is in memory value. Graceful shutdown of walproposer currently + # doesn't guarantee persisted value, which is ok as we need it only for + # tests. Persisting it without risking too many cf flushes needs a wp -> sk + # protocol change. (though in reality shutdown sync-safekeepers does flush + # of cf, so most of the time persisted value wouldn't lag) + log.info(f"sk commit_lsn {commit_lsn}") + # note that ckpt_lsn is the *beginning* of checkpoint record, so commit_lsn + # must be actually higher + assert commit_lsn > ckpt_lsn, "safekeeper must have checkpoint record" + + class SafekeeperEnv: def __init__( self, diff --git a/vendor/postgres-v14 b/vendor/postgres-v14 index f49a962b9b37..b980d6f090c6 160000 --- a/vendor/postgres-v14 +++ b/vendor/postgres-v14 @@ -1 +1 @@ -Subproject commit f49a962b9b3715d6f47017d1dcf905c36f93ae5e +Subproject commit b980d6f090c676e55fb2c830fb2434f532f635c0 diff --git a/vendor/postgres-v15 b/vendor/postgres-v15 index e8b9a28006a5..56f32c0e7330 160000 --- a/vendor/postgres-v15 +++ b/vendor/postgres-v15 @@ -1 +1 @@ -Subproject commit e8b9a28006a550d7ca7cbb9bd0238eb9cd57bbd8 +Subproject commit 56f32c0e7330d17aaeee8bf211a73995180bd133 diff --git a/vendor/postgres-v16 b/vendor/postgres-v16 index 072697b2250d..90078947229a 160000 --- a/vendor/postgres-v16 +++ b/vendor/postgres-v16 @@ -1 +1 @@ -Subproject commit 072697b2250da3251af75887b577104554b9cd44 +Subproject commit 90078947229aa7f9ac5f7ed4527b2c7386d5332b diff --git a/vendor/revisions.json b/vendor/revisions.json index 1529d87bcb74..1941c235eed2 100644 --- a/vendor/revisions.json +++ b/vendor/revisions.json @@ -1,6 +1,5 @@ { - "postgres-v16": "072697b2250da3251af75887b577104554b9cd44", - "postgres-v15": "e8b9a28006a550d7ca7cbb9bd0238eb9cd57bbd8", - "postgres-v14": "f49a962b9b3715d6f47017d1dcf905c36f93ae5e" + "postgres-v16": "90078947229aa7f9ac5f7ed4527b2c7386d5332b", + "postgres-v15": "56f32c0e7330d17aaeee8bf211a73995180bd133", + "postgres-v14": "b980d6f090c676e55fb2c830fb2434f532f635c0" } - From 621ea2ec4465a76a60c1c77e947b31e5a0812dfb Mon Sep 17 00:00:00 2001 From: Heikki Linnakangas Date: Wed, 7 Feb 2024 19:58:08 +0200 Subject: [PATCH 28/43] tests: try to make restored-datadir comparison tests not flaky v2 This test occasionally fails with a difference in "pg_xact/0000" file between the local and restored datadirs. My hypothesis is that something changed in the database between the last explicit checkpoint and the shutdown. I suspect autovacuum, it could certainly create transactions. To fix, be more precise about the point in time that we compare. Shut down the endpoint first, then read the last LSN (i.e. the shutdown checkpoint's LSN), from the local disk with pg_controldata. And use exactly that LSN in the basebackup. Closes #559 --- test_runner/fixtures/neon_fixtures.py | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index 018de975dc9f..584d5fea48bf 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -3890,24 +3890,21 @@ def list_files_to_compare(pgdata_dir: Path) -> List[str]: # pg is the existing and running compute node, that we want to compare with a basebackup def check_restored_datadir_content(test_output_dir: Path, env: NeonEnv, endpoint: Endpoint): + pg_bin = PgBin(test_output_dir, env.pg_distrib_dir, env.pg_version) + # Get the timeline ID. We need it for the 'basebackup' command timeline_id = TimelineId(endpoint.safe_psql("SHOW neon.timeline_id")[0][0]) - # many tests already checkpoint, but do it just in case - with closing(endpoint.connect()) as conn: - with conn.cursor() as cur: - cur.execute("CHECKPOINT") - - # wait for pageserver to catch up - wait_for_last_flush_lsn(env, endpoint, endpoint.tenant_id, timeline_id) # stop postgres to ensure that files won't change endpoint.stop() + # Read the shutdown checkpoint's LSN + checkpoint_lsn = pg_bin.get_pg_controldata_checkpoint_lsn(endpoint.pg_data_dir_path()) + # Take a basebackup from pageserver restored_dir_path = env.repo_dir / f"{endpoint.endpoint_id}_restored_datadir" restored_dir_path.mkdir(exist_ok=True) - pg_bin = PgBin(test_output_dir, env.pg_distrib_dir, env.pg_version) psql_path = os.path.join(pg_bin.pg_bin_path, "psql") pageserver_id = env.attachment_service.locate(endpoint.tenant_id)[0]["node_id"] @@ -3915,7 +3912,7 @@ def check_restored_datadir_content(test_output_dir: Path, env: NeonEnv, endpoint {psql_path} \ --no-psqlrc \ postgres://localhost:{env.get_pageserver(pageserver_id).service_port.pg} \ - -c 'basebackup {endpoint.tenant_id} {timeline_id}' \ + -c 'basebackup {endpoint.tenant_id} {timeline_id} {checkpoint_lsn}' \ | tar -x -C {restored_dir_path} """ From 89cf714890237862eb3fd52f473e4dbe15cd6e4a Mon Sep 17 00:00:00 2001 From: John Spray Date: Tue, 12 Mar 2024 11:36:27 +0000 Subject: [PATCH 29/43] tests/neon_local: rename "attachment service" -> "storage controller" (#7087) Not a user-facing change, but can break any existing `.neon` directories created by neon_local, as the name of the database used by the storage controller changes. This PR changes all the locations apart from the path of `control_plane/attachment_service` (waiting for an opportune moment to do that one, because it's the most conflict-ish wrt ongoing PRs like #6676 ) --- Makefile | 2 +- control_plane/attachment_service/src/http.rs | 2 +- control_plane/attachment_service/src/main.rs | 6 - .../attachment_service/src/persistence.rs | 4 +- .../attachment_service/src/service.rs | 4 +- control_plane/src/bin/neon_local.rs | 86 +++++----- control_plane/src/endpoint.rs | 10 +- control_plane/src/lib.rs | 2 +- control_plane/src/local_env.rs | 12 +- control_plane/src/pageserver.rs | 8 +- ...hment_service.rs => storage_controller.rs} | 38 ++--- docs/authentication.md | 4 +- libs/pageserver_api/src/controller_api.rs | 2 - test_runner/fixtures/neon_fixtures.py | 108 ++++++------ .../fixtures/pageserver/many_tenants.py | 2 +- .../interactive/test_many_small_tenants.py | 2 +- .../pagebench/test_large_slru_basebackup.py | 2 +- ...er_max_throughput_getpage_at_latest_lsn.py | 2 +- test_runner/performance/test_bulk_insert.py | 4 +- .../regress/test_attach_tenant_config.py | 2 +- test_runner/regress/test_change_pageserver.py | 8 +- test_runner/regress/test_compatibility.py | 2 +- .../regress/test_layers_from_future.py | 2 +- test_runner/regress/test_neon_cli.py | 4 +- test_runner/regress/test_pageserver_api.py | 2 +- .../regress/test_pageserver_generations.py | 14 +- .../regress/test_pageserver_secondary.py | 10 +- test_runner/regress/test_remote_storage.py | 4 +- test_runner/regress/test_s3_restore.py | 4 +- test_runner/regress/test_sharding.py | 30 ++-- test_runner/regress/test_sharding_service.py | 156 +++++++++--------- test_runner/regress/test_timeline_size.py | 4 +- 32 files changed, 267 insertions(+), 275 deletions(-) rename control_plane/src/{attachment_service.rs => storage_controller.rs} (94%) diff --git a/Makefile b/Makefile index ea782cb369a0..f13f080f1a8a 100644 --- a/Makefile +++ b/Makefile @@ -51,7 +51,7 @@ CARGO_BUILD_FLAGS += $(filter -j1,$(MAKEFLAGS)) CARGO_CMD_PREFIX += $(if $(filter n,$(MAKEFLAGS)),,+) # Force cargo not to print progress bar CARGO_CMD_PREFIX += CARGO_TERM_PROGRESS_WHEN=never CI=1 -# Set PQ_LIB_DIR to make sure `attachment_service` get linked with bundled libpq (through diesel) +# Set PQ_LIB_DIR to make sure `storage_controller` get linked with bundled libpq (through diesel) CARGO_CMD_PREFIX += PQ_LIB_DIR=$(POSTGRES_INSTALL_DIR)/v16/lib # diff --git a/control_plane/attachment_service/src/http.rs b/control_plane/attachment_service/src/http.rs index 7e4030b221b0..27ba5bdb6534 100644 --- a/control_plane/attachment_service/src/http.rs +++ b/control_plane/attachment_service/src/http.rs @@ -30,7 +30,7 @@ use pageserver_api::controller_api::{ }; use pageserver_api::upcall_api::{ReAttachRequest, ValidateRequest}; -use control_plane::attachment_service::{AttachHookRequest, InspectRequest}; +use control_plane::storage_controller::{AttachHookRequest, InspectRequest}; /// State available to HTTP request handlers #[derive(Clone)] diff --git a/control_plane/attachment_service/src/main.rs b/control_plane/attachment_service/src/main.rs index d9acbc0abd65..333c3911e36c 100644 --- a/control_plane/attachment_service/src/main.rs +++ b/control_plane/attachment_service/src/main.rs @@ -1,9 +1,3 @@ -/// The attachment service mimics the aspects of the control plane API -/// that are required for a pageserver to operate. -/// -/// This enables running & testing pageservers without a full-blown -/// deployment of the Neon cloud platform. -/// use anyhow::{anyhow, Context}; use attachment_service::http::make_router; use attachment_service::metrics::preinitialize_metrics; diff --git a/control_plane/attachment_service/src/persistence.rs b/control_plane/attachment_service/src/persistence.rs index d5c6d74ebed8..aa0894583466 100644 --- a/control_plane/attachment_service/src/persistence.rs +++ b/control_plane/attachment_service/src/persistence.rs @@ -20,7 +20,7 @@ use crate::node::Node; /// ## What do we store? /// -/// The attachment service does not store most of its state durably. +/// The storage controller service does not store most of its state durably. /// /// The essential things to store durably are: /// - generation numbers, as these must always advance monotonically to ensure data safety. @@ -34,7 +34,7 @@ use crate::node::Node; /// /// ## Performance/efficiency /// -/// The attachment service does not go via the database for most things: there are +/// The storage controller service does not go via the database for most things: there are /// a couple of places where we must, and where efficiency matters: /// - Incrementing generation numbers: the Reconciler has to wait for this to complete /// before it can attach a tenant, so this acts as a bound on how fast things like diff --git a/control_plane/attachment_service/src/service.rs b/control_plane/attachment_service/src/service.rs index f3d97c0dfbaf..3f245b52559f 100644 --- a/control_plane/attachment_service/src/service.rs +++ b/control_plane/attachment_service/src/service.rs @@ -8,7 +8,7 @@ use std::{ }; use anyhow::Context; -use control_plane::attachment_service::{ +use control_plane::storage_controller::{ AttachHookRequest, AttachHookResponse, InspectRequest, InspectResponse, }; use diesel::result::DatabaseErrorKind; @@ -839,7 +839,7 @@ impl Service { tenant_state.generation = Some(new_generation); } else { // This is a detach notification. We must update placement policy to avoid re-attaching - // during background scheduling/reconciliation, or during attachment service restart. + // during background scheduling/reconciliation, or during storage controller restart. assert!(attach_req.node_id.is_none()); tenant_state.policy = PlacementPolicy::Detached; } diff --git a/control_plane/src/bin/neon_local.rs b/control_plane/src/bin/neon_local.rs index 27abcb182aa8..86b9c0085dcc 100644 --- a/control_plane/src/bin/neon_local.rs +++ b/control_plane/src/bin/neon_local.rs @@ -8,11 +8,11 @@ use anyhow::{anyhow, bail, Context, Result}; use clap::{value_parser, Arg, ArgAction, ArgMatches, Command, ValueEnum}; use compute_api::spec::ComputeMode; -use control_plane::attachment_service::AttachmentService; use control_plane::endpoint::ComputeControlPlane; use control_plane::local_env::{InitForceMode, LocalEnv}; use control_plane::pageserver::{PageServerNode, PAGESERVER_REMOTE_STORAGE_DIR}; use control_plane::safekeeper::SafekeeperNode; +use control_plane::storage_controller::StorageController; use control_plane::{broker, local_env}; use pageserver_api::controller_api::{ NodeAvailability, NodeConfigureRequest, NodeSchedulingPolicy, PlacementPolicy, @@ -138,7 +138,7 @@ fn main() -> Result<()> { "start" => rt.block_on(handle_start_all(sub_args, &env)), "stop" => rt.block_on(handle_stop_all(sub_args, &env)), "pageserver" => rt.block_on(handle_pageserver(sub_args, &env)), - "attachment_service" => rt.block_on(handle_attachment_service(sub_args, &env)), + "storage_controller" => rt.block_on(handle_storage_controller(sub_args, &env)), "safekeeper" => rt.block_on(handle_safekeeper(sub_args, &env)), "endpoint" => rt.block_on(handle_endpoint(sub_args, &env)), "mappings" => handle_mappings(sub_args, &mut env), @@ -445,14 +445,14 @@ async fn handle_tenant( // If tenant ID was not specified, generate one let tenant_id = parse_tenant_id(create_match)?.unwrap_or_else(TenantId::generate); - // We must register the tenant with the attachment service, so + // We must register the tenant with the storage controller, so // that when the pageserver restarts, it will be re-attached. - let attachment_service = AttachmentService::from_env(env); - attachment_service + let storage_controller = StorageController::from_env(env); + storage_controller .tenant_create(TenantCreateRequest { // Note that ::unsharded here isn't actually because the tenant is unsharded, its because the - // attachment service expecfs a shard-naive tenant_id in this attribute, and the TenantCreateRequest - // type is used both in attachment service (for creating tenants) and in pageserver (for creating shards) + // storage controller expecfs a shard-naive tenant_id in this attribute, and the TenantCreateRequest + // type is used both in storage controller (for creating tenants) and in pageserver (for creating shards) new_tenant_id: TenantShardId::unsharded(tenant_id), generation: None, shard_parameters: ShardParameters { @@ -476,9 +476,9 @@ async fn handle_tenant( .context("Failed to parse postgres version from the argument string")?; // FIXME: passing None for ancestor_start_lsn is not kosher in a sharded world: we can't have - // different shards picking different start lsns. Maybe we have to teach attachment service + // different shards picking different start lsns. Maybe we have to teach storage controller // to let shard 0 branch first and then propagate the chosen LSN to other shards. - attachment_service + storage_controller .tenant_timeline_create( tenant_id, TimelineCreateRequest { @@ -528,8 +528,8 @@ async fn handle_tenant( let new_pageserver = get_pageserver(env, matches)?; let new_pageserver_id = new_pageserver.conf.id; - let attachment_service = AttachmentService::from_env(env); - attachment_service + let storage_controller = StorageController::from_env(env); + storage_controller .tenant_migrate(tenant_shard_id, new_pageserver_id) .await?; @@ -543,8 +543,8 @@ async fn handle_tenant( let mut tenant_synthetic_size = None; - let attachment_service = AttachmentService::from_env(env); - for shard in attachment_service.tenant_locate(tenant_id).await?.shards { + 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)?); @@ -586,8 +586,8 @@ async fn handle_tenant( let tenant_id = get_tenant_id(matches, env)?; let shard_count: u8 = matches.get_one::("shard-count").cloned().unwrap_or(0); - let attachment_service = AttachmentService::from_env(env); - let result = attachment_service + let storage_controller = StorageController::from_env(env); + let result = storage_controller .tenant_split(tenant_id, shard_count) .await?; println!( @@ -613,7 +613,7 @@ async fn handle_timeline(timeline_match: &ArgMatches, env: &mut local_env::Local match timeline_match.subcommand() { Some(("list", list_match)) => { - // TODO(sharding): this command shouldn't have to specify a shard ID: we should ask the attachment service + // TODO(sharding): this command shouldn't have to specify a shard ID: we should ask the storage controller // where shard 0 is attached, and query there. let tenant_shard_id = get_tenant_shard_id(list_match, env)?; let timelines = pageserver.timeline_list(&tenant_shard_id).await?; @@ -633,7 +633,7 @@ async fn handle_timeline(timeline_match: &ArgMatches, env: &mut local_env::Local let new_timeline_id_opt = parse_timeline_id(create_match)?; let new_timeline_id = new_timeline_id_opt.unwrap_or(TimelineId::generate()); - let attachment_service = AttachmentService::from_env(env); + let storage_controller = StorageController::from_env(env); let create_req = TimelineCreateRequest { new_timeline_id, ancestor_timeline_id: None, @@ -641,7 +641,7 @@ async fn handle_timeline(timeline_match: &ArgMatches, env: &mut local_env::Local ancestor_start_lsn: None, pg_version: Some(pg_version), }; - let timeline_info = attachment_service + let timeline_info = storage_controller .tenant_timeline_create(tenant_id, create_req) .await?; @@ -730,7 +730,7 @@ async fn handle_timeline(timeline_match: &ArgMatches, env: &mut local_env::Local .transpose() .context("Failed to parse ancestor start Lsn from the request")?; let new_timeline_id = TimelineId::generate(); - let attachment_service = AttachmentService::from_env(env); + let storage_controller = StorageController::from_env(env); let create_req = TimelineCreateRequest { new_timeline_id, ancestor_timeline_id: Some(ancestor_timeline_id), @@ -738,7 +738,7 @@ async fn handle_timeline(timeline_match: &ArgMatches, env: &mut local_env::Local ancestor_start_lsn: start_lsn, pg_version: None, }; - let timeline_info = attachment_service + let timeline_info = storage_controller .tenant_timeline_create(tenant_id, create_req) .await?; @@ -767,7 +767,7 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re match sub_name { "list" => { - // TODO(sharding): this command shouldn't have to specify a shard ID: we should ask the attachment service + // TODO(sharding): this command shouldn't have to specify a shard ID: we should ask the storage controller // where shard 0 is attached, and query there. let tenant_shard_id = get_tenant_shard_id(sub_args, env)?; let timeline_infos = get_timeline_infos(env, &tenant_shard_id) @@ -952,21 +952,21 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re ( vec![(parsed.0, parsed.1.unwrap_or(5432))], // If caller is telling us what pageserver to use, this is not a tenant which is - // full managed by attachment service, therefore not sharded. + // full managed by storage controller, therefore not sharded. ShardParameters::DEFAULT_STRIPE_SIZE, ) } else { // Look up the currently attached location of the tenant, and its striping metadata, // to pass these on to postgres. - let attachment_service = AttachmentService::from_env(env); - let locate_result = attachment_service.tenant_locate(endpoint.tenant_id).await?; + let storage_controller = StorageController::from_env(env); + let locate_result = storage_controller.tenant_locate(endpoint.tenant_id).await?; let pageservers = locate_result .shards .into_iter() .map(|shard| { ( Host::parse(&shard.listen_pg_addr) - .expect("Attachment service reported bad hostname"), + .expect("Storage controller reported bad hostname"), shard.listen_pg_port, ) }) @@ -1015,8 +1015,8 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re pageserver.pg_connection_config.port(), )] } else { - let attachment_service = AttachmentService::from_env(env); - attachment_service + let storage_controller = StorageController::from_env(env); + storage_controller .tenant_locate(endpoint.tenant_id) .await? .shards @@ -1024,7 +1024,7 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re .map(|shard| { ( Host::parse(&shard.listen_pg_addr) - .expect("Attachment service reported malformed host"), + .expect("Storage controller reported malformed host"), shard.listen_pg_port, ) }) @@ -1144,8 +1144,8 @@ async fn handle_pageserver(sub_match: &ArgMatches, env: &local_env::LocalEnv) -> let scheduling = subcommand_args.get_one("scheduling"); let availability = subcommand_args.get_one("availability"); - let attachment_service = AttachmentService::from_env(env); - attachment_service + let storage_controller = StorageController::from_env(env); + storage_controller .node_configure(NodeConfigureRequest { node_id: pageserver.conf.id, scheduling: scheduling.cloned(), @@ -1170,11 +1170,11 @@ async fn handle_pageserver(sub_match: &ArgMatches, env: &local_env::LocalEnv) -> Ok(()) } -async fn handle_attachment_service( +async fn handle_storage_controller( sub_match: &ArgMatches, env: &local_env::LocalEnv, ) -> Result<()> { - let svc = AttachmentService::from_env(env); + let svc = StorageController::from_env(env); match sub_match.subcommand() { Some(("start", _start_match)) => { if let Err(e) = svc.start().await { @@ -1194,8 +1194,8 @@ async fn handle_attachment_service( exit(1); } } - Some((sub_name, _)) => bail!("Unexpected attachment_service subcommand '{}'", sub_name), - None => bail!("no attachment_service subcommand provided"), + Some((sub_name, _)) => bail!("Unexpected storage_controller subcommand '{}'", sub_name), + None => bail!("no storage_controller subcommand provided"), } Ok(()) } @@ -1280,11 +1280,11 @@ async fn handle_start_all(sub_match: &ArgMatches, env: &local_env::LocalEnv) -> broker::start_broker_process(env).await?; - // Only start the attachment service if the pageserver is configured to need it + // Only start the storage controller if the pageserver is configured to need it if env.control_plane_api.is_some() { - let attachment_service = AttachmentService::from_env(env); - if let Err(e) = attachment_service.start().await { - eprintln!("attachment_service start failed: {:#}", e); + let storage_controller = StorageController::from_env(env); + if let Err(e) = storage_controller.start().await { + eprintln!("storage_controller start failed: {:#}", e); try_stop_all(env, true).await; exit(1); } @@ -1356,9 +1356,9 @@ async fn try_stop_all(env: &local_env::LocalEnv, immediate: bool) { } if env.control_plane_api.is_some() { - let attachment_service = AttachmentService::from_env(env); - if let Err(e) = attachment_service.stop(immediate).await { - eprintln!("attachment service stop failed: {e:#}"); + let storage_controller = StorageController::from_env(env); + if let Err(e) = storage_controller.stop(immediate).await { + eprintln!("storage controller stop failed: {e:#}"); } } } @@ -1618,9 +1618,9 @@ fn cli() -> Command { ) ) .subcommand( - Command::new("attachment_service") + Command::new("storage_controller") .arg_required_else_help(true) - .about("Manage attachment_service") + .about("Manage storage_controller") .subcommand(Command::new("start").about("Start local pageserver").arg(pageserver_config_args.clone())) .subcommand(Command::new("stop").about("Stop local pageserver") .arg(stop_mode_arg.clone())) diff --git a/control_plane/src/endpoint.rs b/control_plane/src/endpoint.rs index ac0a8417aec9..646bc2e8bc39 100644 --- a/control_plane/src/endpoint.rs +++ b/control_plane/src/endpoint.rs @@ -57,9 +57,9 @@ use serde::{Deserialize, Serialize}; use url::Host; use utils::id::{NodeId, TenantId, TimelineId}; -use crate::attachment_service::AttachmentService; use crate::local_env::LocalEnv; use crate::postgresql_conf::PostgresConf; +use crate::storage_controller::StorageController; use compute_api::responses::{ComputeState, ComputeStatus}; use compute_api::spec::{Cluster, ComputeFeature, ComputeMode, ComputeSpec}; @@ -750,17 +750,17 @@ impl Endpoint { let postgresql_conf = self.read_postgresql_conf()?; spec.cluster.postgresql_conf = Some(postgresql_conf); - // If we weren't given explicit pageservers, query the attachment service + // If we weren't given explicit pageservers, query the storage controller if pageservers.is_empty() { - let attachment_service = AttachmentService::from_env(&self.env); - let locate_result = attachment_service.tenant_locate(self.tenant_id).await?; + let storage_controller = StorageController::from_env(&self.env); + let locate_result = storage_controller.tenant_locate(self.tenant_id).await?; pageservers = locate_result .shards .into_iter() .map(|shard| { ( Host::parse(&shard.listen_pg_addr) - .expect("Attachment service reported bad hostname"), + .expect("Storage controller reported bad hostname"), shard.listen_pg_port, ) }) diff --git a/control_plane/src/lib.rs b/control_plane/src/lib.rs index bb79d36bfca6..2af272f3885d 100644 --- a/control_plane/src/lib.rs +++ b/control_plane/src/lib.rs @@ -6,7 +6,6 @@ //! local installations. #![deny(clippy::undocumented_unsafe_blocks)] -pub mod attachment_service; mod background_process; pub mod broker; pub mod endpoint; @@ -14,3 +13,4 @@ pub mod local_env; pub mod pageserver; pub mod postgresql_conf; pub mod safekeeper; +pub mod storage_controller; diff --git a/control_plane/src/local_env.rs b/control_plane/src/local_env.rs index 03270723a62c..2e644894329c 100644 --- a/control_plane/src/local_env.rs +++ b/control_plane/src/local_env.rs @@ -72,13 +72,13 @@ pub struct LocalEnv { #[serde(default)] pub safekeepers: Vec, - // Control plane upcall API for pageserver: if None, we will not run attachment_service. If set, this will + // Control plane upcall API for pageserver: if None, we will not run storage_controller If set, this will // be propagated into each pageserver's configuration. #[serde(default)] pub control_plane_api: Option, - // Control plane upcall API for attachment service. If set, this will be propagated into the - // attachment service's configuration. + // Control plane upcall API for storage controller. If set, this will be propagated into the + // storage controller's configuration. #[serde(default)] pub control_plane_compute_hook_api: Option, @@ -227,10 +227,10 @@ impl LocalEnv { self.neon_distrib_dir.join("pageserver") } - pub fn attachment_service_bin(&self) -> PathBuf { - // Irrespective of configuration, attachment service binary is always + pub fn storage_controller_bin(&self) -> PathBuf { + // Irrespective of configuration, storage controller binary is always // run from the same location as neon_local. This means that for compatibility - // tests that run old pageserver/safekeeper, they still run latest attachment service. + // tests that run old pageserver/safekeeper, they still run latest storage controller. let neon_local_bin_dir = env::current_exe().unwrap().parent().unwrap().to_owned(); neon_local_bin_dir.join("storage_controller") } diff --git a/control_plane/src/pageserver.rs b/control_plane/src/pageserver.rs index ae1bd60c5229..021b9aca34b0 100644 --- a/control_plane/src/pageserver.rs +++ b/control_plane/src/pageserver.rs @@ -31,8 +31,8 @@ use utils::{ lsn::Lsn, }; -use crate::attachment_service::AttachmentService; use crate::local_env::PageServerConf; +use crate::storage_controller::StorageController; use crate::{background_process, local_env::LocalEnv}; /// Directory within .neon which will be used by default for LocalFs remote storage. @@ -111,7 +111,7 @@ impl PageServerNode { control_plane_api.as_str() )); - // Attachment service uses the same auth as pageserver: if JWT is enabled + // Storage controller uses the same auth as pageserver: if JWT is enabled // for us, we will also need it to talk to them. if matches!(self.conf.http_auth_type, AuthType::NeonJWT) { let jwt_token = self @@ -214,12 +214,12 @@ impl PageServerNode { // Register the node with the storage controller before starting pageserver: pageserver must be registered to // successfully call /re-attach and finish starting up. if register { - let attachment_service = AttachmentService::from_env(&self.env); + let storage_controller = StorageController::from_env(&self.env); let (pg_host, pg_port) = parse_host_port(&self.conf.listen_pg_addr).expect("Unable to parse listen_pg_addr"); let (http_host, http_port) = parse_host_port(&self.conf.listen_http_addr) .expect("Unable to parse listen_http_addr"); - attachment_service + storage_controller .node_register(NodeRegisterRequest { node_id: self.conf.id, listen_pg_addr: pg_host.to_string(), diff --git a/control_plane/src/attachment_service.rs b/control_plane/src/storage_controller.rs similarity index 94% rename from control_plane/src/attachment_service.rs rename to control_plane/src/storage_controller.rs index 5c975619855c..c505e677707d 100644 --- a/control_plane/src/attachment_service.rs +++ b/control_plane/src/storage_controller.rs @@ -24,7 +24,7 @@ use utils::{ id::{NodeId, TenantId}, }; -pub struct AttachmentService { +pub struct StorageController { env: LocalEnv, listen: String, path: Utf8PathBuf, @@ -36,7 +36,7 @@ pub struct AttachmentService { const COMMAND: &str = "storage_controller"; -const ATTACHMENT_SERVICE_POSTGRES_VERSION: u32 = 16; +const STORAGE_CONTROLLER_POSTGRES_VERSION: u32 = 16; #[derive(Serialize, Deserialize)] pub struct AttachHookRequest { @@ -59,7 +59,7 @@ pub struct InspectResponse { pub attachment: Option<(u32, NodeId)>, } -impl AttachmentService { +impl StorageController { pub fn from_env(env: &LocalEnv) -> Self { let path = Utf8PathBuf::from_path_buf(env.base_data_dir.clone()) .unwrap() @@ -136,27 +136,27 @@ impl AttachmentService { } fn pid_file(&self) -> Utf8PathBuf { - Utf8PathBuf::from_path_buf(self.env.base_data_dir.join("attachment_service.pid")) + Utf8PathBuf::from_path_buf(self.env.base_data_dir.join("storage_controller.pid")) .expect("non-Unicode path") } - /// PIDFile for the postgres instance used to store attachment service state + /// PIDFile for the postgres instance used to store storage controller state fn postgres_pid_file(&self) -> Utf8PathBuf { Utf8PathBuf::from_path_buf( self.env .base_data_dir - .join("attachment_service_postgres.pid"), + .join("storage_controller_postgres.pid"), ) .expect("non-Unicode path") } /// Find the directory containing postgres binaries, such as `initdb` and `pg_ctl` /// - /// This usually uses ATTACHMENT_SERVICE_POSTGRES_VERSION of postgres, but will fall back + /// This usually uses STORAGE_CONTROLLER_POSTGRES_VERSION of postgres, but will fall back /// to other versions if that one isn't found. Some automated tests create circumstances /// where only one version is available in pg_distrib_dir, such as `test_remote_extensions`. pub async fn get_pg_bin_dir(&self) -> anyhow::Result { - let prefer_versions = [ATTACHMENT_SERVICE_POSTGRES_VERSION, 15, 14]; + let prefer_versions = [STORAGE_CONTROLLER_POSTGRES_VERSION, 15, 14]; for v in prefer_versions { let path = Utf8PathBuf::from_path_buf(self.env.pg_bin_dir(v)?).unwrap(); @@ -189,7 +189,7 @@ impl AttachmentService { /// /// Returns the database url pub async fn setup_database(&self) -> anyhow::Result { - const DB_NAME: &str = "attachment_service"; + const DB_NAME: &str = "storage_controller"; let database_url = format!("postgresql://localhost:{}/{DB_NAME}", self.postgres_port); let pg_bin_dir = self.get_pg_bin_dir().await?; @@ -219,10 +219,10 @@ impl AttachmentService { } pub async fn start(&self) -> anyhow::Result<()> { - // Start a vanilla Postgres process used by the attachment service for persistence. + // Start a vanilla Postgres process used by the storage controller for persistence. let pg_data_path = Utf8PathBuf::from_path_buf(self.env.base_data_dir.clone()) .unwrap() - .join("attachment_service_db"); + .join("storage_controller_db"); let pg_bin_dir = self.get_pg_bin_dir().await?; let pg_log_path = pg_data_path.join("postgres.log"); @@ -245,7 +245,7 @@ impl AttachmentService { .await?; }; - println!("Starting attachment service database..."); + println!("Starting storage controller database..."); let db_start_args = [ "-w", "-D", @@ -256,7 +256,7 @@ impl AttachmentService { ]; background_process::start_process( - "attachment_service_db", + "storage_controller_db", &self.env.base_data_dir, pg_bin_dir.join("pg_ctl").as_std_path(), db_start_args, @@ -300,7 +300,7 @@ impl AttachmentService { background_process::start_process( COMMAND, &self.env.base_data_dir, - &self.env.attachment_service_bin(), + &self.env.storage_controller_bin(), args, [( "NEON_REPO_DIR".to_string(), @@ -322,10 +322,10 @@ impl AttachmentService { pub async fn stop(&self, immediate: bool) -> anyhow::Result<()> { background_process::stop_process(immediate, COMMAND, &self.pid_file())?; - let pg_data_path = self.env.base_data_dir.join("attachment_service_db"); + let pg_data_path = self.env.base_data_dir.join("storage_controller_db"); let pg_bin_dir = self.get_pg_bin_dir().await?; - println!("Stopping attachment service database..."); + println!("Stopping storage controller database..."); let pg_stop_args = ["-D", &pg_data_path.to_string_lossy(), "stop"]; let stop_status = Command::new(pg_bin_dir.join("pg_ctl")) .args(pg_stop_args) @@ -344,10 +344,10 @@ impl AttachmentService { // fine that stop failed. Otherwise it is an error that stop failed. const PG_STATUS_NOT_RUNNING: i32 = 3; if Some(PG_STATUS_NOT_RUNNING) == status_exitcode.code() { - println!("Attachment service data base is already stopped"); + println!("Storage controller database is already stopped"); return Ok(()); } else { - anyhow::bail!("Failed to stop attachment service database: {stop_status}") + anyhow::bail!("Failed to stop storage controller database: {stop_status}") } } @@ -368,7 +368,7 @@ impl AttachmentService { } } - /// Simple HTTP request wrapper for calling into attachment service + /// Simple HTTP request wrapper for calling into storage controller async fn dispatch( &self, method: hyper::Method, diff --git a/docs/authentication.md b/docs/authentication.md index faac7aa28eed..522c5481b46e 100644 --- a/docs/authentication.md +++ b/docs/authentication.md @@ -70,9 +70,9 @@ Should only be used e.g. for status check/tenant creation/list. Should only be used e.g. for status check. Currently also used for connection from any pageserver to any safekeeper. -"generations_api": Provides access to the upcall APIs served by the attachment service or the control plane. +"generations_api": Provides access to the upcall APIs served by the storage controller or the control plane. -"admin": Provides access to the control plane and admin APIs of the attachment service. +"admin": Provides access to the control plane and admin APIs of the storage controller. ### CLI CLI generates a key pair during call to `neon_local init` with the following commands: diff --git a/libs/pageserver_api/src/controller_api.rs b/libs/pageserver_api/src/controller_api.rs index 38e61239c5bc..c172354e9f85 100644 --- a/libs/pageserver_api/src/controller_api.rs +++ b/libs/pageserver_api/src/controller_api.rs @@ -88,8 +88,6 @@ impl FromStr for NodeAvailability { } } -/// FIXME: this is a duplicate of the type in the attachment_service crate, because the -/// type needs to be defined with diesel traits in there. #[derive(Serialize, Deserialize, Clone, Copy, Eq, PartialEq)] pub enum NodeSchedulingPolicy { Active, diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index 584d5fea48bf..234bfa8bf931 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -1014,24 +1014,24 @@ def __init__(self, config: NeonEnvBuilder): self.initial_tenant = config.initial_tenant self.initial_timeline = config.initial_timeline - # Find two adjacent ports for attachment service and its postgres DB. This + # Find two adjacent ports for storage controller and its postgres DB. This # loop would eventually throw from get_port() if we run out of ports (extremely # unlikely): usually we find two adjacent free ports on the first iteration. while True: - self.attachment_service_port = self.port_distributor.get_port() - attachment_service_pg_port = self.port_distributor.get_port() - if attachment_service_pg_port == self.attachment_service_port + 1: + self.storage_controller_port = self.port_distributor.get_port() + storage_controller_pg_port = self.port_distributor.get_port() + if storage_controller_pg_port == self.storage_controller_port + 1: break # The URL for the pageserver to use as its control_plane_api config - self.control_plane_api: str = f"http://127.0.0.1:{self.attachment_service_port}/upcall/v1" - # The base URL of the attachment service - self.attachment_service_api: str = f"http://127.0.0.1:{self.attachment_service_port}" + self.control_plane_api: str = f"http://127.0.0.1:{self.storage_controller_port}/upcall/v1" + # The base URL of the storage controller + self.storage_controller_api: str = f"http://127.0.0.1:{self.storage_controller_port}" # For testing this with a fake HTTP server, enable passing through a URL from config self.control_plane_compute_hook_api = config.control_plane_compute_hook_api - self.attachment_service: NeonAttachmentService = NeonAttachmentService( + self.storage_controller: NeonStorageController = NeonStorageController( self, config.auth_enabled ) @@ -1113,16 +1113,16 @@ def __init__(self, config: NeonEnvBuilder): self.neon_cli.init(cfg, force=config.config_init_force) def start(self): - # Attachment service starts first, so that pageserver /re-attach calls don't + # storage controller starts first, so that pageserver /re-attach calls don't # bounce through retries on startup - self.attachment_service.start() + self.storage_controller.start() - def attachment_service_ready(): - assert self.attachment_service.ready() is True + def storage_controller_ready(): + assert self.storage_controller.ready() is True - # Wait for attachment service readiness to prevent unnecessary post start-up + # Wait for storage controller readiness to prevent unnecessary post start-up # reconcile. - wait_until(30, 1, attachment_service_ready) + wait_until(30, 1, storage_controller_ready) # Start up broker, pageserver and all safekeepers futs = [] @@ -1153,7 +1153,7 @@ def stop(self, immediate=False, ps_assert_metric_no_errors=False): if ps_assert_metric_no_errors: pageserver.assert_no_metric_errors() pageserver.stop(immediate=immediate) - self.attachment_service.stop(immediate=immediate) + self.storage_controller.stop(immediate=immediate) self.broker.stop(immediate=immediate) @property @@ -1188,9 +1188,9 @@ def get_pageserver(self, id: Optional[int]) -> NeonPageserver: def get_tenant_pageserver(self, tenant_id: Union[TenantId, TenantShardId]): """ Get the NeonPageserver where this tenant shard is currently attached, according - to the attachment service. + to the storage controller. """ - meta = self.attachment_service.inspect(tenant_id) + meta = self.storage_controller.inspect(tenant_id) if meta is None: return None pageserver_id = meta[1] @@ -1697,12 +1697,12 @@ def init( res.check_returncode() return res - def attachment_service_start(self): - cmd = ["attachment_service", "start"] + def storage_controller_start(self): + cmd = ["storage_controller", "start"] return self.raw_cli(cmd) - def attachment_service_stop(self, immediate: bool): - cmd = ["attachment_service", "stop"] + def storage_controller_stop(self, immediate: bool): + cmd = ["storage_controller", "stop"] if immediate: cmd.extend(["-m", "immediate"]) return self.raw_cli(cmd) @@ -1942,14 +1942,14 @@ def dump_index_part(self, path: Path) -> IndexPartDump: return IndexPartDump.from_json(parsed) -class AttachmentServiceApiException(Exception): +class StorageControllerApiException(Exception): def __init__(self, message, status_code: int): super().__init__(message) self.message = message self.status_code = status_code -class NeonAttachmentService(MetricsGetter): +class NeonStorageController(MetricsGetter): def __init__(self, env: NeonEnv, auth_enabled: bool): self.env = env self.running = False @@ -1957,13 +1957,13 @@ def __init__(self, env: NeonEnv, auth_enabled: bool): def start(self): assert not self.running - self.env.neon_cli.attachment_service_start() + self.env.neon_cli.storage_controller_start() self.running = True return self - def stop(self, immediate: bool = False) -> "NeonAttachmentService": + def stop(self, immediate: bool = False) -> "NeonStorageController": if self.running: - self.env.neon_cli.attachment_service_stop(immediate) + self.env.neon_cli.storage_controller_stop(immediate) self.running = False return self @@ -1976,22 +1976,22 @@ def raise_api_exception(res: requests.Response): msg = res.json()["msg"] except: # noqa: E722 msg = "" - raise AttachmentServiceApiException(msg, res.status_code) from e + raise StorageControllerApiException(msg, res.status_code) from e def pageserver_api(self) -> PageserverHttpClient: """ - The attachment service implements a subset of the pageserver REST API, for mapping + The storage controller implements a subset of the pageserver REST API, for mapping per-tenant actions into per-shard actions (e.g. timeline creation). Tests should invoke those functions via the HttpClient, as an implicit check that these APIs remain compatible. """ auth_token = None if self.auth_enabled: auth_token = self.env.auth_keys.generate_token(scope=TokenScope.PAGE_SERVER_API) - return PageserverHttpClient(self.env.attachment_service_port, lambda: True, auth_token) + return PageserverHttpClient(self.env.storage_controller_port, lambda: True, auth_token) def request(self, method, *args, **kwargs) -> requests.Response: resp = requests.request(method, *args, **kwargs) - NeonAttachmentService.raise_api_exception(resp) + NeonStorageController.raise_api_exception(resp) return resp @@ -2004,15 +2004,15 @@ def headers(self, scope: Optional[TokenScope]) -> Dict[str, str]: return headers def get_metrics(self) -> Metrics: - res = self.request("GET", f"{self.env.attachment_service_api}/metrics") + res = self.request("GET", f"{self.env.storage_controller_api}/metrics") return parse_metrics(res.text) def ready(self) -> bool: status = None try: - resp = self.request("GET", f"{self.env.attachment_service_api}/ready") + resp = self.request("GET", f"{self.env.storage_controller_api}/ready") status = resp.status_code - except AttachmentServiceApiException as e: + except StorageControllerApiException as e: status = e.status_code if status == 503: @@ -2027,7 +2027,7 @@ def attach_hook_issue( ) -> int: response = self.request( "POST", - f"{self.env.attachment_service_api}/debug/v1/attach-hook", + f"{self.env.storage_controller_api}/debug/v1/attach-hook", json={"tenant_shard_id": str(tenant_shard_id), "node_id": pageserver_id}, headers=self.headers(TokenScope.ADMIN), ) @@ -2038,7 +2038,7 @@ def attach_hook_issue( def attach_hook_drop(self, tenant_shard_id: Union[TenantId, TenantShardId]): self.request( "POST", - f"{self.env.attachment_service_api}/debug/v1/attach-hook", + f"{self.env.storage_controller_api}/debug/v1/attach-hook", json={"tenant_shard_id": str(tenant_shard_id), "node_id": None}, headers=self.headers(TokenScope.ADMIN), ) @@ -2049,7 +2049,7 @@ def inspect(self, tenant_shard_id: Union[TenantId, TenantShardId]) -> Optional[t """ response = self.request( "POST", - f"{self.env.attachment_service_api}/debug/v1/inspect", + f"{self.env.storage_controller_api}/debug/v1/inspect", json={"tenant_shard_id": str(tenant_shard_id)}, headers=self.headers(TokenScope.ADMIN), ) @@ -2070,7 +2070,7 @@ def node_register(self, node: NeonPageserver): log.info(f"node_register({body})") self.request( "POST", - f"{self.env.attachment_service_api}/control/v1/node", + f"{self.env.storage_controller_api}/control/v1/node", json=body, headers=self.headers(TokenScope.ADMIN), ) @@ -2078,7 +2078,7 @@ def node_register(self, node: NeonPageserver): def node_list(self): response = self.request( "GET", - f"{self.env.attachment_service_api}/control/v1/node", + f"{self.env.storage_controller_api}/control/v1/node", headers=self.headers(TokenScope.ADMIN), ) return response.json() @@ -2088,7 +2088,7 @@ def node_configure(self, node_id, body: dict[str, Any]): body["node_id"] = node_id self.request( "PUT", - f"{self.env.attachment_service_api}/control/v1/node/{node_id}/config", + f"{self.env.storage_controller_api}/control/v1/node/{node_id}/config", json=body, headers=self.headers(TokenScope.ADMIN), ) @@ -2118,7 +2118,7 @@ def tenant_create( response = self.request( "POST", - f"{self.env.attachment_service_api}/v1/tenant", + f"{self.env.storage_controller_api}/v1/tenant", json=body, headers=self.headers(TokenScope.PAGE_SERVER_API), ) @@ -2130,7 +2130,7 @@ def locate(self, tenant_id: TenantId) -> list[dict[str, Any]]: """ response = self.request( "GET", - f"{self.env.attachment_service_api}/control/v1/tenant/{tenant_id}/locate", + f"{self.env.storage_controller_api}/control/v1/tenant/{tenant_id}/locate", headers=self.headers(TokenScope.ADMIN), ) body = response.json() @@ -2140,7 +2140,7 @@ def locate(self, tenant_id: TenantId) -> list[dict[str, Any]]: def tenant_shard_split(self, tenant_id: TenantId, shard_count: int) -> list[TenantShardId]: response = self.request( "PUT", - f"{self.env.attachment_service_api}/control/v1/tenant/{tenant_id}/shard_split", + f"{self.env.storage_controller_api}/control/v1/tenant/{tenant_id}/shard_split", json={"new_shard_count": shard_count}, headers=self.headers(TokenScope.ADMIN), ) @@ -2152,7 +2152,7 @@ def tenant_shard_split(self, tenant_id: TenantId, shard_count: int) -> list[Tena def tenant_shard_migrate(self, tenant_shard_id: TenantShardId, dest_ps_id: int): self.request( "PUT", - f"{self.env.attachment_service_api}/control/v1/tenant/{tenant_shard_id}/migrate", + f"{self.env.storage_controller_api}/control/v1/tenant/{tenant_shard_id}/migrate", json={"tenant_shard_id": str(tenant_shard_id), "node_id": dest_ps_id}, headers=self.headers(TokenScope.ADMIN), ) @@ -2165,12 +2165,12 @@ def consistency_check(self): """ self.request( "POST", - f"{self.env.attachment_service_api}/debug/v1/consistency_check", + f"{self.env.storage_controller_api}/debug/v1/consistency_check", headers=self.headers(TokenScope.ADMIN), ) - log.info("Attachment service passed consistency check") + log.info("storage controller passed consistency check") - def __enter__(self) -> "NeonAttachmentService": + def __enter__(self) -> "NeonStorageController": return self def __exit__( @@ -2401,7 +2401,7 @@ def tenant_attach( """ client = self.http_client() if generation is None: - generation = self.env.attachment_service.attach_hook_issue(tenant_id, self.id) + generation = self.env.storage_controller.attach_hook_issue(tenant_id, self.id) return client.tenant_attach( tenant_id, config, @@ -2410,14 +2410,14 @@ def tenant_attach( ) def tenant_detach(self, tenant_id: TenantId): - self.env.attachment_service.attach_hook_drop(tenant_id) + self.env.storage_controller.attach_hook_drop(tenant_id) client = self.http_client() return client.tenant_detach(tenant_id) def tenant_location_configure(self, tenant_id: TenantId, config: dict[str, Any], **kwargs): if config["mode"].startswith("Attached") and "generation" not in config: - config["generation"] = self.env.attachment_service.attach_hook_issue(tenant_id, self.id) + config["generation"] = self.env.storage_controller.attach_hook_issue(tenant_id, self.id) client = self.http_client() return client.tenant_location_conf(tenant_id, config, **kwargs) @@ -2441,14 +2441,14 @@ def tenant_create( generation: Optional[int] = None, ) -> TenantId: if generation is None: - generation = self.env.attachment_service.attach_hook_issue(tenant_id, self.id) + generation = self.env.storage_controller.attach_hook_issue(tenant_id, self.id) client = self.http_client(auth_token=auth_token) return client.tenant_create(tenant_id, conf, generation=generation) def tenant_load(self, tenant_id: TenantId): client = self.http_client() return client.tenant_load( - tenant_id, generation=self.env.attachment_service.attach_hook_issue(tenant_id, self.id) + tenant_id, generation=self.env.storage_controller.attach_hook_issue(tenant_id, self.id) ) @@ -3907,7 +3907,7 @@ def check_restored_datadir_content(test_output_dir: Path, env: NeonEnv, endpoint psql_path = os.path.join(pg_bin.pg_bin_path, "psql") - pageserver_id = env.attachment_service.locate(endpoint.tenant_id)[0]["node_id"] + pageserver_id = env.storage_controller.locate(endpoint.tenant_id)[0]["node_id"] cmd = rf""" {psql_path} \ --no-psqlrc \ @@ -3994,7 +3994,7 @@ def tenant_get_shards( us to figure out the shards for a tenant. If the caller provides `pageserver_id`, it will be used for all shards, even - if the shard is indicated by attachment service to be on some other pageserver. + if the shard is indicated by storage controller to be on some other pageserver. Caller should over the response to apply their per-pageserver action to each shard @@ -4010,7 +4010,7 @@ def tenant_get_shards( TenantShardId.parse(s["shard_id"]), override_pageserver or env.get_pageserver(s["node_id"]), ) - for s in env.attachment_service.locate(tenant_id) + for s in env.storage_controller.locate(tenant_id) ] else: # Assume an unsharded tenant diff --git a/test_runner/fixtures/pageserver/many_tenants.py b/test_runner/fixtures/pageserver/many_tenants.py index bbb4ccee5b57..f47a3ea04332 100644 --- a/test_runner/fixtures/pageserver/many_tenants.py +++ b/test_runner/fixtures/pageserver/many_tenants.py @@ -43,7 +43,7 @@ def single_timeline( log.info("detach template tenant form pageserver") env.pageserver.tenant_detach(template_tenant) env.pageserver.allowed_errors.append( - # tenant detach causes this because the underlying attach-hook removes the tenant from attachment_service entirely + # tenant detach causes this because the underlying attach-hook removes the tenant from storage controller entirely ".*Dropped remote consistent LSN updates.*", ) diff --git a/test_runner/performance/pageserver/interactive/test_many_small_tenants.py b/test_runner/performance/pageserver/interactive/test_many_small_tenants.py index 3fb28ace4676..0ff9c8fdaa98 100644 --- a/test_runner/performance/pageserver/interactive/test_many_small_tenants.py +++ b/test_runner/performance/pageserver/interactive/test_many_small_tenants.py @@ -56,7 +56,7 @@ def setup_template(env: NeonEnv): template_tenant, template_timeline = env.neon_cli.create_tenant(set_default=True) env.pageserver.tenant_detach(template_tenant) env.pageserver.allowed_errors.append( - # tenant detach causes this because the underlying attach-hook removes the tenant from attachment_service entirely + # tenant detach causes this because the underlying attach-hook removes the tenant from storage controller entirely ".*Dropped remote consistent LSN updates.*", ) env.pageserver.tenant_attach(template_tenant, config) diff --git a/test_runner/performance/pageserver/pagebench/test_large_slru_basebackup.py b/test_runner/performance/pageserver/pagebench/test_large_slru_basebackup.py index 921b7c5b7609..c98fa44b1a81 100644 --- a/test_runner/performance/pageserver/pagebench/test_large_slru_basebackup.py +++ b/test_runner/performance/pageserver/pagebench/test_large_slru_basebackup.py @@ -92,7 +92,7 @@ def setup_tenant_template(env: NeonEnv, n_txns: int): template_tenant, template_timeline = env.neon_cli.create_tenant(set_default=True) env.pageserver.tenant_detach(template_tenant) env.pageserver.allowed_errors.append( - # tenant detach causes this because the underlying attach-hook removes the tenant from attachment_service entirely + # tenant detach causes this because the underlying attach-hook removes the tenant from storage controller entirely ".*Dropped remote consistent LSN updates.*", ) env.pageserver.tenant_attach(template_tenant, config) diff --git a/test_runner/performance/pageserver/pagebench/test_pageserver_max_throughput_getpage_at_latest_lsn.py b/test_runner/performance/pageserver/pagebench/test_pageserver_max_throughput_getpage_at_latest_lsn.py index 8cd3569ea5de..1a0012397c70 100644 --- a/test_runner/performance/pageserver/pagebench/test_pageserver_max_throughput_getpage_at_latest_lsn.py +++ b/test_runner/performance/pageserver/pagebench/test_pageserver_max_throughput_getpage_at_latest_lsn.py @@ -114,7 +114,7 @@ def setup_tenant_template(env: NeonEnv, pg_bin: PgBin, scale: int): template_tenant, template_timeline = env.neon_cli.create_tenant(set_default=True) env.pageserver.tenant_detach(template_tenant) env.pageserver.allowed_errors.append( - # tenant detach causes this because the underlying attach-hook removes the tenant from attachment_service entirely + # tenant detach causes this because the underlying attach-hook removes the tenant from storage controller entirely ".*Dropped remote consistent LSN updates.*", ) env.pageserver.tenant_attach(template_tenant, config) diff --git a/test_runner/performance/test_bulk_insert.py b/test_runner/performance/test_bulk_insert.py index 72173dc2a729..9e3f6022375d 100644 --- a/test_runner/performance/test_bulk_insert.py +++ b/test_runner/performance/test_bulk_insert.py @@ -56,12 +56,12 @@ def measure_recovery_time(env: NeonCompare): # Delete the Tenant in the pageserver: this will drop local and remote layers, such that # when we "create" the Tenant again, we will replay the WAL from the beginning. # - # This is a "weird" thing to do, and can confuse the attachment service as we're re-using + # This is a "weird" thing to do, and can confuse the storage controller as we're re-using # the same tenant ID for a tenant that is logically different from the pageserver's point # of view, but the same as far as the safekeeper/WAL is concerned. To work around that, # we will explicitly create the tenant in the same generation that it was previously # attached in. - attach_status = env.env.attachment_service.inspect(tenant_shard_id=env.tenant) + attach_status = env.env.storage_controller.inspect(tenant_shard_id=env.tenant) assert attach_status is not None (attach_gen, _) = attach_status diff --git a/test_runner/regress/test_attach_tenant_config.py b/test_runner/regress/test_attach_tenant_config.py index 7fbce6a10c38..3058926b257f 100644 --- a/test_runner/regress/test_attach_tenant_config.py +++ b/test_runner/regress/test_attach_tenant_config.py @@ -137,7 +137,7 @@ def test_no_config(positive_env: NeonEnv, content_type: Optional[str]): ps_http.tenant_detach(tenant_id) assert tenant_id not in [TenantId(t["id"]) for t in ps_http.tenant_list()] - body = {"generation": env.attachment_service.attach_hook_issue(tenant_id, env.pageserver.id)} + body = {"generation": env.storage_controller.attach_hook_issue(tenant_id, env.pageserver.id)} ps_http.post( f"{ps_http.base_url}/v1/tenant/{tenant_id}/attach", diff --git a/test_runner/regress/test_change_pageserver.py b/test_runner/regress/test_change_pageserver.py index adb67a579e8b..97ab69049d00 100644 --- a/test_runner/regress/test_change_pageserver.py +++ b/test_runner/regress/test_change_pageserver.py @@ -85,9 +85,9 @@ def fetchone(): # the endpoint. Whereas the previous reconfiguration was like a healthy migration, this # is more like what happens in an unexpected pageserver failure. # - # Since we're dual-attached, need to tip-off attachment service to treat the one we're + # Since we're dual-attached, need to tip-off storage controller to treat the one we're # about to start as the attached pageserver - env.attachment_service.attach_hook_issue(env.initial_tenant, env.pageservers[0].id) + env.storage_controller.attach_hook_issue(env.initial_tenant, env.pageservers[0].id) env.pageservers[0].start() env.pageservers[1].stop() @@ -97,9 +97,9 @@ def fetchone(): assert fetchone() == (100000,) env.pageservers[0].stop() - # Since we're dual-attached, need to tip-off attachment service to treat the one we're + # Since we're dual-attached, need to tip-off storage controller to treat the one we're # about to start as the attached pageserver - env.attachment_service.attach_hook_issue(env.initial_tenant, env.pageservers[1].id) + env.storage_controller.attach_hook_issue(env.initial_tenant, env.pageservers[1].id) env.pageservers[1].start() # Test a (former) bug where a child process spins without updating its connection string diff --git a/test_runner/regress/test_compatibility.py b/test_runner/regress/test_compatibility.py index 0ea76d447e47..618ac637855e 100644 --- a/test_runner/regress/test_compatibility.py +++ b/test_runner/regress/test_compatibility.py @@ -133,7 +133,7 @@ def test_create_snapshot( for sk in env.safekeepers: sk.stop() env.pageserver.stop() - env.attachment_service.stop() + env.storage_controller.stop() # Directory `compatibility_snapshot_dir` is uploaded to S3 in a workflow, keep the name in sync with it compatibility_snapshot_dir = ( diff --git a/test_runner/regress/test_layers_from_future.py b/test_runner/regress/test_layers_from_future.py index abdebb6d7959..ca4295c5cb83 100644 --- a/test_runner/regress/test_layers_from_future.py +++ b/test_runner/regress/test_layers_from_future.py @@ -159,7 +159,7 @@ def get_future_layers(): time.sleep(1.1) # so that we can use change in pre_stat.st_mtime to detect overwrites def get_generation_number(): - attachment = env.attachment_service.inspect(tenant_id) + attachment = env.storage_controller.inspect(tenant_id) assert attachment is not None return attachment[0] diff --git a/test_runner/regress/test_neon_cli.py b/test_runner/regress/test_neon_cli.py index 16d120e24a87..cb69f0ef39ac 100644 --- a/test_runner/regress/test_neon_cli.py +++ b/test_runner/regress/test_neon_cli.py @@ -133,7 +133,7 @@ def test_cli_start_stop(neon_env_builder: NeonEnvBuilder): # Stop default ps/sk env.neon_cli.pageserver_stop(env.pageserver.id) env.neon_cli.safekeeper_stop() - env.neon_cli.attachment_service_stop(False) + env.neon_cli.storage_controller_stop(False) # Keep NeonEnv state up to date, it usually owns starting/stopping services env.pageserver.running = False @@ -175,7 +175,7 @@ def test_cli_start_stop_multi(neon_env_builder: NeonEnvBuilder): env.neon_cli.safekeeper_stop(neon_env_builder.safekeepers_id_start + 2) # Stop this to get out of the way of the following `start` - env.neon_cli.attachment_service_stop(False) + env.neon_cli.storage_controller_stop(False) # Default start res = env.neon_cli.raw_cli(["start"]) diff --git a/test_runner/regress/test_pageserver_api.py b/test_runner/regress/test_pageserver_api.py index e29db1e25255..877deee08f11 100644 --- a/test_runner/regress/test_pageserver_api.py +++ b/test_runner/regress/test_pageserver_api.py @@ -73,7 +73,7 @@ def check_client(env: NeonEnv, client: PageserverHttpClient): # create new tenant and check it is also there tenant_id = TenantId.generate() client.tenant_create( - tenant_id, generation=env.attachment_service.attach_hook_issue(tenant_id, env.pageserver.id) + tenant_id, generation=env.storage_controller.attach_hook_issue(tenant_id, env.pageserver.id) ) assert tenant_id in {TenantId(t["id"]) for t in client.tenant_list()} diff --git a/test_runner/regress/test_pageserver_generations.py b/test_runner/regress/test_pageserver_generations.py index 89fc48a49f82..d1acb9817e5d 100644 --- a/test_runner/regress/test_pageserver_generations.py +++ b/test_runner/regress/test_pageserver_generations.py @@ -203,7 +203,7 @@ def test_generations_upgrade(neon_env_builder: NeonEnvBuilder): env.broker.try_start() for sk in env.safekeepers: sk.start() - env.attachment_service.start() + env.storage_controller.start() env.pageserver.start(overrides=('--pageserver-config-override=control_plane_api=""',)) @@ -285,7 +285,7 @@ def test_deferred_deletion(neon_env_builder: NeonEnvBuilder): neon_env_builder.num_pageservers = 2 env = neon_env_builder.init_start(initial_tenant_conf=TENANT_CONF) - attached_to_id = env.attachment_service.locate(env.initial_tenant)[0]["node_id"] + attached_to_id = env.storage_controller.locate(env.initial_tenant)[0]["node_id"] main_pageserver = env.get_pageserver(attached_to_id) other_pageserver = [p for p in env.pageservers if p.id != attached_to_id][0] @@ -310,7 +310,7 @@ def test_deferred_deletion(neon_env_builder: NeonEnvBuilder): # Now advance the generation in the control plane: subsequent validations # from the running pageserver will fail. No more deletions should happen. - env.attachment_service.attach_hook_issue(env.initial_tenant, other_pageserver.id) + env.storage_controller.attach_hook_issue(env.initial_tenant, other_pageserver.id) generate_uploads_and_deletions(env, init=False, pageserver=main_pageserver) assert_deletion_queue(ps_http, lambda n: n > 0) @@ -366,7 +366,7 @@ def test_deletion_queue_recovery( neon_env_builder.num_pageservers = 2 env = neon_env_builder.init_start(initial_tenant_conf=TENANT_CONF) - attached_to_id = env.attachment_service.locate(env.initial_tenant)[0]["node_id"] + attached_to_id = env.storage_controller.locate(env.initial_tenant)[0]["node_id"] main_pageserver = env.get_pageserver(attached_to_id) other_pageserver = [p for p in env.pageservers if p.id != attached_to_id][0] @@ -428,7 +428,7 @@ def assert_header_written(): if keep_attachment == KeepAttachment.LOSE: some_other_pageserver = other_pageserver.id - env.attachment_service.attach_hook_issue(env.initial_tenant, some_other_pageserver) + env.storage_controller.attach_hook_issue(env.initial_tenant, some_other_pageserver) main_pageserver.start() @@ -494,7 +494,7 @@ def test_emergency_mode(neon_env_builder: NeonEnvBuilder, pg_bin: PgBin): ) # Simulate a major incident: the control plane goes offline - env.attachment_service.stop() + env.storage_controller.stop() # Remember how many validations had happened before the control plane went offline validated = get_deletion_queue_validated(ps_http) @@ -525,7 +525,7 @@ def test_emergency_mode(neon_env_builder: NeonEnvBuilder, pg_bin: PgBin): assert get_deletion_queue_executed(ps_http) == 0 # When the control plane comes back up, normal service should resume - env.attachment_service.start() + env.storage_controller.start() ps_http.deletion_queue_flush(execute=True) assert get_deletion_queue_depth(ps_http) == 0 diff --git a/test_runner/regress/test_pageserver_secondary.py b/test_runner/regress/test_pageserver_secondary.py index 8ba9d767ddf8..79145f61b3c1 100644 --- a/test_runner/regress/test_pageserver_secondary.py +++ b/test_runner/regress/test_pageserver_secondary.py @@ -157,7 +157,7 @@ def test_location_conf_churn(neon_env_builder: NeonEnvBuilder, seed: int): workload.churn_rows(rng.randint(128, 256), pageserver.id) workload.validate(pageserver.id) elif last_state_ps[0].startswith("Attached"): - # The `attachment_service` will only re-attach on startup when a pageserver was the + # The `storage_controller` will only re-attach on startup when a pageserver was the # holder of the latest generation: otherwise the pageserver will revert to detached # state if it was running attached with a stale generation last_state[pageserver.id] = ("Detached", None) @@ -182,12 +182,12 @@ def test_location_conf_churn(neon_env_builder: NeonEnvBuilder, seed: int): generation = last_state_ps[1] else: # Switch generations, while also jumping between attached states - generation = env.attachment_service.attach_hook_issue( + generation = env.storage_controller.attach_hook_issue( tenant_id, pageserver.id ) latest_attached = pageserver.id else: - generation = env.attachment_service.attach_hook_issue(tenant_id, pageserver.id) + generation = env.storage_controller.attach_hook_issue(tenant_id, pageserver.id) latest_attached = pageserver.id else: generation = None @@ -273,7 +273,7 @@ def test_live_migration(neon_env_builder: NeonEnvBuilder): # Encourage the new location to download while still in secondary mode pageserver_b.http_client().tenant_secondary_download(tenant_id) - migrated_generation = env.attachment_service.attach_hook_issue(tenant_id, pageserver_b.id) + migrated_generation = env.storage_controller.attach_hook_issue(tenant_id, pageserver_b.id) log.info(f"Acquired generation {migrated_generation} for destination pageserver") assert migrated_generation == initial_generation + 1 @@ -436,7 +436,7 @@ def test_secondary_downloads(neon_env_builder: NeonEnvBuilder): remote_storage_kind=RemoteStorageKind.MOCK_S3, ) env = neon_env_builder.init_start(initial_tenant_conf=TENANT_CONF) - assert env.attachment_service is not None + assert env.storage_controller is not None assert isinstance(env.pageserver_remote_storage, S3Storage) # Satisfy linter tenant_id = env.initial_tenant diff --git a/test_runner/regress/test_remote_storage.py b/test_runner/regress/test_remote_storage.py index 06c13cc07d43..05f769b0e3e8 100644 --- a/test_runner/regress/test_remote_storage.py +++ b/test_runner/regress/test_remote_storage.py @@ -169,7 +169,7 @@ def test_remote_storage_backup_and_restore( # Ensure that even though the tenant is broken, retrying the attachment fails with pytest.raises(Exception, match="Tenant state is Broken"): # Use same generation as in previous attempt - gen_state = env.attachment_service.inspect(tenant_id) + gen_state = env.storage_controller.inspect(tenant_id) assert gen_state is not None generation = gen_state[0] env.pageserver.tenant_attach(tenant_id, generation=generation) @@ -355,7 +355,7 @@ def churn_while_failpoints_active(result): env.pageserver.stop(immediate=True) env.endpoints.stop_all() - # We are about to forcibly drop local dirs. Attachment service will increment generation in re-attach before + # We are about to forcibly drop local dirs. Storage controller will increment generation in re-attach before # we later increment when actually attaching it again, leading to skipping a generation and potentially getting # these warnings if there was a durable but un-executed deletion list at time of restart. env.pageserver.allowed_errors.extend( diff --git a/test_runner/regress/test_s3_restore.py b/test_runner/regress/test_s3_restore.py index aaa33f0bcbd6..611bd1c2a280 100644 --- a/test_runner/regress/test_s3_restore.py +++ b/test_runner/regress/test_s3_restore.py @@ -80,7 +80,7 @@ def test_tenant_s3_restore( assert ( ps_http.get_metric_value("pageserver_tenant_manager_slots") == 0 ), "tenant removed before we deletion was issued" - env.attachment_service.attach_hook_drop(tenant_id) + env.storage_controller.attach_hook_drop(tenant_id) tenant_path = env.pageserver.tenant_dir(tenant_id) assert not tenant_path.exists() @@ -103,7 +103,7 @@ def test_tenant_s3_restore( tenant_id, timestamp=ts_before_deletion, done_if_after=ts_after_deletion ) - generation = env.attachment_service.attach_hook_issue(tenant_id, env.pageserver.id) + generation = env.storage_controller.attach_hook_issue(tenant_id, env.pageserver.id) ps_http.tenant_attach(tenant_id, generation=generation) env.pageserver.quiesce_tenants() diff --git a/test_runner/regress/test_sharding.py b/test_runner/regress/test_sharding.py index 57c8d1f8498d..1b96cd6a8091 100644 --- a/test_runner/regress/test_sharding.py +++ b/test_runner/regress/test_sharding.py @@ -43,7 +43,7 @@ def test_sharding_smoke( tenant_id = env.initial_tenant pageservers = dict((int(p.id), p) for p in env.pageservers) - shards = env.attachment_service.locate(tenant_id) + shards = env.storage_controller.locate(tenant_id) def get_sizes(): sizes = {} @@ -86,7 +86,7 @@ def get_sizes(): ) assert timelines == {env.initial_timeline, timeline_b} - env.attachment_service.consistency_check() + env.storage_controller.consistency_check() def test_sharding_split_unsharded( @@ -102,7 +102,7 @@ def test_sharding_split_unsharded( # Check that we created with an unsharded TenantShardId: this is the default, # but check it in case we change the default in future - assert env.attachment_service.inspect(TenantShardId(tenant_id, 0, 0)) is not None + assert env.storage_controller.inspect(TenantShardId(tenant_id, 0, 0)) is not None workload = Workload(env, tenant_id, timeline_id, branch_name="main") workload.init() @@ -110,15 +110,15 @@ def test_sharding_split_unsharded( workload.validate() # Split one shard into two - env.attachment_service.tenant_shard_split(tenant_id, shard_count=2) + env.storage_controller.tenant_shard_split(tenant_id, shard_count=2) # Check we got the shard IDs we expected - assert env.attachment_service.inspect(TenantShardId(tenant_id, 0, 2)) is not None - assert env.attachment_service.inspect(TenantShardId(tenant_id, 1, 2)) is not None + assert env.storage_controller.inspect(TenantShardId(tenant_id, 0, 2)) is not None + assert env.storage_controller.inspect(TenantShardId(tenant_id, 1, 2)) is not None workload.validate() - env.attachment_service.consistency_check() + env.storage_controller.consistency_check() def test_sharding_split_smoke( @@ -161,7 +161,7 @@ def test_sharding_split_smoke( workload.write_rows(256) # Note which pageservers initially hold a shard after tenant creation - pre_split_pageserver_ids = [loc["node_id"] for loc in env.attachment_service.locate(tenant_id)] + pre_split_pageserver_ids = [loc["node_id"] for loc in env.storage_controller.locate(tenant_id)] # For pageservers holding a shard, validate their ingest statistics # reflect a proper splitting of the WAL. @@ -213,9 +213,9 @@ def shards_on_disk(shard_ids): # Before split, old shards exist assert shards_on_disk(old_shard_ids) - env.attachment_service.tenant_shard_split(tenant_id, shard_count=split_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.attachment_service.locate(tenant_id)] + post_split_pageserver_ids = [loc["node_id"] for loc in env.storage_controller.locate(tenant_id)] # We should have split into 8 shards, on the same 4 pageservers we started on. assert len(post_split_pageserver_ids) == split_shard_count assert len(set(post_split_pageserver_ids)) == shard_count @@ -261,7 +261,7 @@ def shards_on_disk(shard_ids): # 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. - reconcile_ok = env.attachment_service.get_metric_value( + 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 @@ -269,19 +269,19 @@ 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.attachment_service.get_metric_value( + env.storage_controller.get_metric_value( "storage_controller_reconcile_complete_total", filter={"status": "cancel"} ) is None ) assert ( - env.attachment_service.get_metric_value( + env.storage_controller.get_metric_value( "storage_controller_reconcile_complete_total", filter={"status": "error"} ) is None ) - env.attachment_service.consistency_check() + env.storage_controller.consistency_check() # Validate pageserver state shards_exist: list[TenantShardId] = [] @@ -360,7 +360,7 @@ def test_sharding_ingest( huge_layer_count = 0 # Inspect the resulting layer map, count how many layers are undersized. - for shard in env.attachment_service.locate(tenant_id): + for shard in env.storage_controller.locate(tenant_id): pageserver = env.get_pageserver(shard["node_id"]) shard_id = shard["shard_id"] layer_map = pageserver.http_client().layer_map_info(shard_id, timeline_id) diff --git a/test_runner/regress/test_sharding_service.py b/test_runner/regress/test_sharding_service.py index aecc244a4723..6b7cd9d8290d 100644 --- a/test_runner/regress/test_sharding_service.py +++ b/test_runner/regress/test_sharding_service.py @@ -6,10 +6,10 @@ import pytest from fixtures.log_helper import log from fixtures.neon_fixtures import ( - AttachmentServiceApiException, NeonEnv, NeonEnvBuilder, PgBin, + StorageControllerApiException, TokenScope, ) from fixtures.pageserver.http import PageserverHttpClient @@ -36,7 +36,7 @@ def get_node_shard_counts(env: NeonEnv, tenant_ids): counts: defaultdict[str, int] = defaultdict(int) for tid in tenant_ids: - for shard in env.attachment_service.locate(tid): + for shard in env.storage_controller.locate(tid): counts[shard["node_id"]] += 1 return counts @@ -62,20 +62,20 @@ def test_sharding_service_smoke( # Start services by hand so that we can skip a pageserver (this will start + register later) env.broker.try_start() - env.attachment_service.start() + env.storage_controller.start() env.pageservers[0].start() env.pageservers[1].start() for sk in env.safekeepers: sk.start() # The pageservers we started should have registered with the sharding service on startup - nodes = env.attachment_service.node_list() + nodes = env.storage_controller.node_list() assert len(nodes) == 2 assert set(n["id"] for n in nodes) == {env.pageservers[0].id, env.pageservers[1].id} # Starting an additional pageserver should register successfully env.pageservers[2].start() - nodes = env.attachment_service.node_list() + nodes = env.storage_controller.node_list() assert len(nodes) == 3 assert set(n["id"] for n in nodes) == {ps.id for ps in env.pageservers} @@ -99,22 +99,22 @@ def test_sharding_service_smoke( # Creating and deleting timelines should work, using identical API to pageserver timeline_crud_tenant = next(iter(tenant_ids)) timeline_id = TimelineId.generate() - env.attachment_service.pageserver_api().timeline_create( + env.storage_controller.pageserver_api().timeline_create( pg_version=PgVersion.NOT_SET, tenant_id=timeline_crud_tenant, new_timeline_id=timeline_id ) - timelines = env.attachment_service.pageserver_api().timeline_list(timeline_crud_tenant) + timelines = env.storage_controller.pageserver_api().timeline_list(timeline_crud_tenant) assert len(timelines) == 2 assert timeline_id in set(TimelineId(t["timeline_id"]) for t in timelines) # virtual_ps_http.timeline_delete(tenant_id=timeline_crud_tenant, timeline_id=timeline_id) timeline_delete_wait_completed( - env.attachment_service.pageserver_api(), timeline_crud_tenant, timeline_id + env.storage_controller.pageserver_api(), timeline_crud_tenant, timeline_id ) - timelines = env.attachment_service.pageserver_api().timeline_list(timeline_crud_tenant) + timelines = env.storage_controller.pageserver_api().timeline_list(timeline_crud_tenant) assert len(timelines) == 1 assert timeline_id not in set(TimelineId(t["timeline_id"]) for t in timelines) # Marking a pageserver offline should migrate tenants away from it. - env.attachment_service.node_configure(env.pageservers[0].id, {"availability": "Offline"}) + env.storage_controller.node_configure(env.pageservers[0].id, {"availability": "Offline"}) def node_evacuated(node_id: int) -> None: counts = get_node_shard_counts(env, tenant_ids) @@ -124,7 +124,7 @@ def node_evacuated(node_id: int) -> None: # Marking pageserver active should not migrate anything to it # immediately - env.attachment_service.node_configure(env.pageservers[0].id, {"availability": "Active"}) + env.storage_controller.node_configure(env.pageservers[0].id, {"availability": "Active"}) time.sleep(1) assert get_node_shard_counts(env, tenant_ids)[env.pageservers[0].id] == 0 @@ -144,13 +144,13 @@ def node_evacuated(node_id: int) -> None: # Delete all the tenants for tid in tenant_ids: - tenant_delete_wait_completed(env.attachment_service.pageserver_api(), tid, 10) + tenant_delete_wait_completed(env.storage_controller.pageserver_api(), tid, 10) - env.attachment_service.consistency_check() + env.storage_controller.consistency_check() # Set a scheduling policy on one node, create all the tenants, observe # that the scheduling policy is respected. - env.attachment_service.node_configure(env.pageservers[1].id, {"scheduling": "Draining"}) + env.storage_controller.node_configure(env.pageservers[1].id, {"scheduling": "Draining"}) # Create some fresh tenants tenant_ids = set(TenantId.generate() for i in range(0, tenant_count)) @@ -163,7 +163,7 @@ def node_evacuated(node_id: int) -> None: assert counts[env.pageservers[0].id] == tenant_shard_count // 2 assert counts[env.pageservers[2].id] == tenant_shard_count // 2 - env.attachment_service.consistency_check() + env.storage_controller.consistency_check() def test_node_status_after_restart( @@ -173,28 +173,28 @@ def test_node_status_after_restart( env = neon_env_builder.init_start() # Initially we have two online pageservers - nodes = env.attachment_service.node_list() + nodes = env.storage_controller.node_list() assert len(nodes) == 2 env.pageservers[1].stop() - env.attachment_service.stop() - env.attachment_service.start() + env.storage_controller.stop() + env.storage_controller.start() def is_ready(): - assert env.attachment_service.ready() is True + assert env.storage_controller.ready() is True wait_until(30, 1, is_ready) # We loaded nodes from database on restart - nodes = env.attachment_service.node_list() + nodes = env.storage_controller.node_list() assert len(nodes) == 2 # We should still be able to create a tenant, because the pageserver which is still online # should have had its availabilty state set to Active. - env.attachment_service.tenant_create(TenantId.generate()) + env.storage_controller.tenant_create(TenantId.generate()) - env.attachment_service.consistency_check() + env.storage_controller.consistency_check() def test_sharding_service_passthrough( @@ -208,9 +208,9 @@ def test_sharding_service_passthrough( neon_env_builder.num_pageservers = 2 env = neon_env_builder.init_start() - # We will talk to attachment service as if it was a pageserver, using the pageserver + # We will talk to storage controller as if it was a pageserver, using the pageserver # HTTP client - client = PageserverHttpClient(env.attachment_service_port, lambda: True) + client = PageserverHttpClient(env.storage_controller_port, lambda: True) timelines = client.timeline_list(tenant_id=env.initial_tenant) assert len(timelines) == 1 @@ -221,22 +221,22 @@ def test_sharding_service_passthrough( } assert status["state"]["slug"] == "Active" - env.attachment_service.consistency_check() + env.storage_controller.consistency_check() def test_sharding_service_restart(neon_env_builder: NeonEnvBuilder): env = neon_env_builder.init_start() tenant_a = env.initial_tenant tenant_b = TenantId.generate() - env.attachment_service.tenant_create(tenant_b) + env.storage_controller.tenant_create(tenant_b) env.pageserver.tenant_detach(tenant_a) # TODO: extend this test to use multiple pageservers, and check that locations don't move around # on restart. - # Attachment service restart - env.attachment_service.stop() - env.attachment_service.start() + # Storage controller restart + env.storage_controller.stop() + env.storage_controller.start() observed = set(TenantId(tenant["id"]) for tenant in env.pageserver.http_client().tenant_list()) @@ -255,7 +255,7 @@ def test_sharding_service_restart(neon_env_builder: NeonEnvBuilder): assert tenant_a not in observed assert tenant_b in observed - env.attachment_service.consistency_check() + env.storage_controller.consistency_check() @pytest.mark.parametrize("warm_up", [True, False]) @@ -271,7 +271,7 @@ def test_sharding_service_onboarding(neon_env_builder: NeonEnvBuilder, warm_up: # Start services by hand so that we can skip registration on one of the pageservers env = neon_env_builder.init_configs() env.broker.try_start() - env.attachment_service.start() + env.storage_controller.start() # This is the pageserver where we'll initially create the tenant. Run it in emergency # mode so that it doesn't talk to storage controller, and do not register it. @@ -286,12 +286,12 @@ def test_sharding_service_onboarding(neon_env_builder: NeonEnvBuilder, warm_up: # will be attached after onboarding env.pageservers[1].start(register=True) dest_ps = env.pageservers[1] - virtual_ps_http = PageserverHttpClient(env.attachment_service_port, lambda: True) + virtual_ps_http = PageserverHttpClient(env.storage_controller_port, lambda: True) for sk in env.safekeepers: sk.start() - # Create a tenant directly via pageserver HTTP API, skipping the attachment service + # Create a tenant directly via pageserver HTTP API, skipping the storage controller tenant_id = TenantId.generate() generation = 123 origin_ps.http_client().tenant_create(tenant_id, generation=generation) @@ -324,7 +324,7 @@ def test_sharding_service_onboarding(neon_env_builder: NeonEnvBuilder, warm_up: virtual_ps_http.tenant_secondary_download(tenant_id) - # Call into attachment service to onboard the tenant + # Call into storage controller to onboard the tenant generation += 1 virtual_ps_http.tenant_location_conf( tenant_id, @@ -347,7 +347,7 @@ def test_sharding_service_onboarding(neon_env_builder: NeonEnvBuilder, warm_up: }, ) - # As if doing a live migration, call into the attachment service to + # As if doing a live migration, call into the storage controller to # set it to AttachedSingle: this is a no-op, but we test it because the # cloud control plane may call this for symmetry with live migration to # an individual pageserver @@ -375,8 +375,8 @@ def test_sharding_service_onboarding(neon_env_builder: NeonEnvBuilder, warm_up: assert dest_tenants[0]["generation"] == generation + 1 # The onboarded tenant should survive a restart of sharding service - env.attachment_service.stop() - env.attachment_service.start() + env.storage_controller.stop() + env.storage_controller.start() # The onboarded tenant should surviev a restart of pageserver dest_ps.stop() @@ -407,7 +407,7 @@ def test_sharding_service_onboarding(neon_env_builder: NeonEnvBuilder, warm_up: dest_tenant_conf_after = dest_ps.http_client().tenant_config(tenant_id) assert dest_tenant_conf_after.tenant_specific_overrides == modified_tenant_conf - env.attachment_service.consistency_check() + env.storage_controller.consistency_check() def test_sharding_service_compute_hook( @@ -419,7 +419,7 @@ def test_sharding_service_compute_hook( Test that the sharding service calls out to the configured HTTP endpoint on attachment changes """ - # We will run two pageserver to migrate and check that the attachment service sends notifications + # We will run two pageserver to migrate and check that the storage controller sends notifications # when migrating. neon_env_builder.num_pageservers = 2 (host, port) = httpserver_listen_address @@ -450,7 +450,7 @@ def handler(request: Request): } assert notifications[0] == expect - env.attachment_service.node_configure(env.pageservers[0].id, {"availability": "Offline"}) + env.storage_controller.node_configure(env.pageservers[0].id, {"availability": "Offline"}) def node_evacuated(node_id: int) -> None: counts = get_node_shard_counts(env, [env.initial_tenant]) @@ -473,8 +473,8 @@ def received_migration_notification(): wait_until(20, 0.25, received_migration_notification) # When we restart, we should re-emit notifications for all tenants - env.attachment_service.stop() - env.attachment_service.start() + env.storage_controller.stop() + env.storage_controller.start() def received_restart_notification(): assert len(notifications) == 3 @@ -483,7 +483,7 @@ def received_restart_notification(): wait_until(10, 1, received_restart_notification) # Splitting a tenant should cause its stripe size to become visible in the compute notification - env.attachment_service.tenant_shard_split(env.initial_tenant, shard_count=2) + env.storage_controller.tenant_shard_split(env.initial_tenant, shard_count=2) expect = { "tenant_id": str(env.initial_tenant), "stripe_size": 32768, @@ -499,7 +499,7 @@ def received_split_notification(): wait_until(10, 1, received_split_notification) - env.attachment_service.consistency_check() + env.storage_controller.consistency_check() def test_sharding_service_debug_apis(neon_env_builder: NeonEnvBuilder): @@ -512,55 +512,55 @@ def test_sharding_service_debug_apis(neon_env_builder: NeonEnvBuilder): env = neon_env_builder.init_start() tenant_id = TenantId.generate() - env.attachment_service.tenant_create(tenant_id, shard_count=2, shard_stripe_size=8192) + env.storage_controller.tenant_create(tenant_id, shard_count=2, shard_stripe_size=8192) # Check that the consistency check passes on a freshly setup system - env.attachment_service.consistency_check() + env.storage_controller.consistency_check() - # These APIs are intentionally not implemented as methods on NeonAttachmentService, as + # These APIs are intentionally not implemented as methods on NeonStorageController, as # they're just for use in unanticipated circumstances. # Initial tenant (1 shard) and the one we just created (2 shards) should be visible - response = env.attachment_service.request( + response = env.storage_controller.request( "GET", - f"{env.attachment_service_api}/debug/v1/tenant", - headers=env.attachment_service.headers(TokenScope.ADMIN), + f"{env.storage_controller_api}/debug/v1/tenant", + headers=env.storage_controller.headers(TokenScope.ADMIN), ) assert len(response.json()) == 3 # Scheduler should report the expected nodes and shard counts - response = env.attachment_service.request( - "GET", f"{env.attachment_service_api}/debug/v1/scheduler" + response = env.storage_controller.request( + "GET", f"{env.storage_controller_api}/debug/v1/scheduler" ) # Two nodes, in a dict of node_id->node assert len(response.json()["nodes"]) == 2 assert sum(v["shard_count"] for v in response.json()["nodes"].values()) == 3 assert all(v["may_schedule"] for v in response.json()["nodes"].values()) - response = env.attachment_service.request( + response = env.storage_controller.request( "POST", - f"{env.attachment_service_api}/debug/v1/node/{env.pageservers[1].id}/drop", - headers=env.attachment_service.headers(TokenScope.ADMIN), + f"{env.storage_controller_api}/debug/v1/node/{env.pageservers[1].id}/drop", + headers=env.storage_controller.headers(TokenScope.ADMIN), ) - assert len(env.attachment_service.node_list()) == 1 + assert len(env.storage_controller.node_list()) == 1 - response = env.attachment_service.request( + response = env.storage_controller.request( "POST", - f"{env.attachment_service_api}/debug/v1/tenant/{tenant_id}/drop", - headers=env.attachment_service.headers(TokenScope.ADMIN), + f"{env.storage_controller_api}/debug/v1/tenant/{tenant_id}/drop", + headers=env.storage_controller.headers(TokenScope.ADMIN), ) # Tenant drop should be reflected in dump output - response = env.attachment_service.request( + response = env.storage_controller.request( "GET", - f"{env.attachment_service_api}/debug/v1/tenant", - headers=env.attachment_service.headers(TokenScope.ADMIN), + f"{env.storage_controller_api}/debug/v1/tenant", + headers=env.storage_controller.headers(TokenScope.ADMIN), ) assert len(response.json()) == 1 # Check that the 'drop' APIs didn't leave things in a state that would fail a consistency check: they're # meant to be unclean wrt the pageserver state, but not leave a broken storage controller behind. - env.attachment_service.consistency_check() + env.storage_controller.consistency_check() def test_sharding_service_s3_time_travel_recovery( @@ -584,10 +584,10 @@ def test_sharding_service_s3_time_travel_recovery( neon_env_builder.num_pageservers = 1 env = neon_env_builder.init_start() - virtual_ps_http = PageserverHttpClient(env.attachment_service_port, lambda: True) + virtual_ps_http = PageserverHttpClient(env.storage_controller_port, lambda: True) tenant_id = TenantId.generate() - env.attachment_service.tenant_create( + env.storage_controller.tenant_create( tenant_id, shard_count=2, shard_stripe_size=8192, @@ -595,7 +595,7 @@ def test_sharding_service_s3_time_travel_recovery( ) # Check that the consistency check passes - env.attachment_service.consistency_check() + env.storage_controller.consistency_check() branch_name = "main" timeline_id = env.neon_cli.create_timeline( @@ -670,28 +670,28 @@ def test_sharding_service_s3_time_travel_recovery( with env.endpoints.create_start("main", tenant_id=tenant_id) as endpoint: endpoint.safe_psql("SELECT * FROM created_foo;") - env.attachment_service.consistency_check() + env.storage_controller.consistency_check() def test_sharding_service_auth(neon_env_builder: NeonEnvBuilder): neon_env_builder.auth_enabled = True env = neon_env_builder.init_start() - svc = env.attachment_service - api = env.attachment_service_api + svc = env.storage_controller + api = env.storage_controller_api tenant_id = TenantId.generate() body: Dict[str, Any] = {"new_tenant_id": str(tenant_id)} # No token with pytest.raises( - AttachmentServiceApiException, + StorageControllerApiException, match="Unauthorized: missing authorization header", ): - svc.request("POST", f"{env.attachment_service_api}/v1/tenant", json=body) + svc.request("POST", f"{env.storage_controller_api}/v1/tenant", json=body) # Token with incorrect scope with pytest.raises( - AttachmentServiceApiException, + StorageControllerApiException, match="Forbidden: JWT authentication error", ): svc.request("POST", f"{api}/v1/tenant", json=body, headers=svc.headers(TokenScope.ADMIN)) @@ -703,14 +703,14 @@ def test_sharding_service_auth(neon_env_builder: NeonEnvBuilder): # No token with pytest.raises( - AttachmentServiceApiException, + StorageControllerApiException, match="Unauthorized: missing authorization header", ): svc.request("GET", f"{api}/debug/v1/tenant") # Token with incorrect scope with pytest.raises( - AttachmentServiceApiException, + StorageControllerApiException, match="Forbidden: JWT authentication error", ): svc.request( @@ -719,14 +719,14 @@ def test_sharding_service_auth(neon_env_builder: NeonEnvBuilder): # No token with pytest.raises( - AttachmentServiceApiException, + StorageControllerApiException, match="Unauthorized: missing authorization header", ): svc.request("POST", f"{api}/upcall/v1/re-attach") # Token with incorrect scope with pytest.raises( - AttachmentServiceApiException, + StorageControllerApiException, match="Forbidden: JWT authentication error", ): svc.request( @@ -743,7 +743,7 @@ def test_sharding_service_tenant_conf(neon_env_builder: NeonEnvBuilder): env = neon_env_builder.init_start() tenant_id = env.initial_tenant - http = env.attachment_service.pageserver_api() + http = env.storage_controller.pageserver_api() default_value = "7days" new_value = "1h" @@ -769,4 +769,4 @@ def test_sharding_service_tenant_conf(neon_env_builder: NeonEnvBuilder): assert readback_ps.effective_config["pitr_interval"] == default_value assert "pitr_interval" not in readback_ps.tenant_specific_overrides - env.attachment_service.consistency_check() + env.storage_controller.consistency_check() diff --git a/test_runner/regress/test_timeline_size.py b/test_runner/regress/test_timeline_size.py index cbf7059c925b..205ca18050a8 100644 --- a/test_runner/regress/test_timeline_size.py +++ b/test_runner/regress/test_timeline_size.py @@ -1011,7 +1011,7 @@ def eager_tenant_is_active(): resp = client.tenant_status(eager_tenant) assert resp["state"]["slug"] == "Active" - gen = env.attachment_service.attach_hook_issue(eager_tenant, env.pageserver.id) + gen = env.storage_controller.attach_hook_issue(eager_tenant, env.pageserver.id) client.tenant_location_conf( eager_tenant, { @@ -1071,7 +1071,7 @@ def initial_tenant_is_active(): # attach, it will consume the only permit because logical size calculation # is paused. - gen = env.attachment_service.attach_hook_issue(lazy_tenant, env.pageserver.id) + gen = env.storage_controller.attach_hook_issue(lazy_tenant, env.pageserver.id) client.tenant_location_conf( lazy_tenant, { From 09699d4bd883d9e1753dcff22406d8455b5f133e Mon Sep 17 00:00:00 2001 From: Conrad Ludgate Date: Tue, 12 Mar 2024 11:52:00 +0000 Subject: [PATCH 30/43] proxy: cancel http queries on timeout (#7031) ## Problem On HTTP query timeout, we should try and cancel the current in-flight SQL query. ## Summary of changes Trigger a cancellation command in postgres once the timeout is reach --- proxy/src/serverless/conn_pool.rs | 9 +- proxy/src/serverless/sql_over_http.rs | 313 +++++++++++++++++--------- test_runner/fixtures/neon_fixtures.py | 6 + test_runner/regress/test_proxy.py | 32 +++ 4 files changed, 242 insertions(+), 118 deletions(-) diff --git a/proxy/src/serverless/conn_pool.rs b/proxy/src/serverless/conn_pool.rs index 73f213d074d3..901e30224bda 100644 --- a/proxy/src/serverless/conn_pool.rs +++ b/proxy/src/serverless/conn_pool.rs @@ -612,13 +612,6 @@ impl Client { let inner = inner.as_mut().expect("client inner should not be removed"); (&mut inner.inner, Discard { pool, conn_info }) } - - pub fn check_idle(&mut self, status: ReadyForQueryStatus) { - self.inner().1.check_idle(status) - } - pub fn discard(&mut self) { - self.inner().1.discard() - } } impl Discard<'_, C> { @@ -739,7 +732,7 @@ mod tests { { let mut client = Client::new(create_inner(), conn_info.clone(), ep_pool.clone()); assert_eq!(0, pool.get_global_connections_count()); - client.discard(); + client.inner().1.discard(); // Discard should not add the connection from the pool. assert_eq!(0, pool.get_global_connections_count()); } diff --git a/proxy/src/serverless/sql_over_http.rs b/proxy/src/serverless/sql_over_http.rs index 74af9852110c..20d9795b470b 100644 --- a/proxy/src/serverless/sql_over_http.rs +++ b/proxy/src/serverless/sql_over_http.rs @@ -1,6 +1,10 @@ +use std::pin::pin; use std::sync::Arc; use anyhow::bail; +use futures::future::select; +use futures::future::try_join; +use futures::future::Either; use futures::StreamExt; use hyper::body::HttpBody; use hyper::header; @@ -11,13 +15,16 @@ use hyper::StatusCode; use hyper::{Body, HeaderMap, Request}; use serde_json::json; use serde_json::Value; -use tokio::try_join; +use tokio::time; use tokio_postgres::error::DbError; use tokio_postgres::error::ErrorPosition; +use tokio_postgres::error::SqlState; use tokio_postgres::GenericClient; use tokio_postgres::IsolationLevel; +use tokio_postgres::NoTls; use tokio_postgres::ReadyForQueryStatus; use tokio_postgres::Transaction; +use tokio_util::sync::CancellationToken; use tracing::error; use tracing::info; use url::Url; @@ -194,108 +201,111 @@ pub async fn handle( request: Request, backend: Arc, ) -> Result, ApiError> { - let result = tokio::time::timeout( - config.http_config.request_timeout, - handle_inner(config, &mut ctx, request, backend), - ) - .await; + let cancel = CancellationToken::new(); + let cancel2 = cancel.clone(); + let handle = tokio::spawn(async move { + time::sleep(config.http_config.request_timeout).await; + cancel2.cancel(); + }); + + let result = handle_inner(cancel, config, &mut ctx, request, backend).await; + handle.abort(); + let mut response = match result { - Ok(r) => match r { - Ok(r) => { - ctx.set_success(); - r + Ok(Ok(r)) => { + ctx.set_success(); + r + } + Err(e) => { + // TODO: ctx.set_error_kind(e.get_error_type()); + + let mut message = format!("{:?}", e); + let db_error = e + .downcast_ref::() + .and_then(|e| e.as_db_error()); + fn get<'a, T: serde::Serialize>( + db: Option<&'a DbError>, + x: impl FnOnce(&'a DbError) -> T, + ) -> Value { + db.map(x) + .and_then(|t| serde_json::to_value(t).ok()) + .unwrap_or_default() } - Err(e) => { - // TODO: ctx.set_error_kind(e.get_error_type()); - - let mut message = format!("{:?}", e); - let db_error = e - .downcast_ref::() - .and_then(|e| e.as_db_error()); - fn get<'a, T: serde::Serialize>( - db: Option<&'a DbError>, - x: impl FnOnce(&'a DbError) -> T, - ) -> Value { - db.map(x) - .and_then(|t| serde_json::to_value(t).ok()) - .unwrap_or_default() - } - if let Some(db_error) = db_error { - db_error.message().clone_into(&mut message); - } - - let position = db_error.and_then(|db| db.position()); - let (position, internal_position, internal_query) = match position { - Some(ErrorPosition::Original(position)) => ( - Value::String(position.to_string()), - Value::Null, - Value::Null, - ), - Some(ErrorPosition::Internal { position, query }) => ( - Value::Null, - Value::String(position.to_string()), - Value::String(query.clone()), - ), - None => (Value::Null, Value::Null, Value::Null), - }; - - let code = get(db_error, |db| db.code().code()); - let severity = get(db_error, |db| db.severity()); - let detail = get(db_error, |db| db.detail()); - let hint = get(db_error, |db| db.hint()); - let where_ = get(db_error, |db| db.where_()); - let table = get(db_error, |db| db.table()); - let column = get(db_error, |db| db.column()); - let schema = get(db_error, |db| db.schema()); - let datatype = get(db_error, |db| db.datatype()); - let constraint = get(db_error, |db| db.constraint()); - let file = get(db_error, |db| db.file()); - let line = get(db_error, |db| db.line().map(|l| l.to_string())); - let routine = get(db_error, |db| db.routine()); - - error!( - ?code, - "sql-over-http per-client task finished with an error: {e:#}" - ); - // TODO: this shouldn't always be bad request. - json_response( - StatusCode::BAD_REQUEST, - json!({ - "message": message, - "code": code, - "detail": detail, - "hint": hint, - "position": position, - "internalPosition": internal_position, - "internalQuery": internal_query, - "severity": severity, - "where": where_, - "table": table, - "column": column, - "schema": schema, - "dataType": datatype, - "constraint": constraint, - "file": file, - "line": line, - "routine": routine, - }), - )? + if let Some(db_error) = db_error { + db_error.message().clone_into(&mut message); } - }, - Err(_) => { + + let position = db_error.and_then(|db| db.position()); + let (position, internal_position, internal_query) = match position { + Some(ErrorPosition::Original(position)) => ( + Value::String(position.to_string()), + Value::Null, + Value::Null, + ), + Some(ErrorPosition::Internal { position, query }) => ( + Value::Null, + Value::String(position.to_string()), + Value::String(query.clone()), + ), + None => (Value::Null, Value::Null, Value::Null), + }; + + let code = get(db_error, |db| db.code().code()); + let severity = get(db_error, |db| db.severity()); + let detail = get(db_error, |db| db.detail()); + let hint = get(db_error, |db| db.hint()); + let where_ = get(db_error, |db| db.where_()); + let table = get(db_error, |db| db.table()); + let column = get(db_error, |db| db.column()); + let schema = get(db_error, |db| db.schema()); + let datatype = get(db_error, |db| db.datatype()); + let constraint = get(db_error, |db| db.constraint()); + let file = get(db_error, |db| db.file()); + let line = get(db_error, |db| db.line().map(|l| l.to_string())); + let routine = get(db_error, |db| db.routine()); + + error!( + ?code, + "sql-over-http per-client task finished with an error: {e:#}" + ); + // TODO: this shouldn't always be bad request. + json_response( + StatusCode::BAD_REQUEST, + json!({ + "message": message, + "code": code, + "detail": detail, + "hint": hint, + "position": position, + "internalPosition": internal_position, + "internalQuery": internal_query, + "severity": severity, + "where": where_, + "table": table, + "column": column, + "schema": schema, + "dataType": datatype, + "constraint": constraint, + "file": file, + "line": line, + "routine": routine, + }), + )? + } + Ok(Err(Cancelled())) => { // TODO: when http error classification is done, distinguish between // timeout on sql vs timeout in proxy/cplane // ctx.set_error_kind(crate::error::ErrorKind::RateLimit); let message = format!( - "HTTP-Connection timed out, execution time exceeded {} seconds", - config.http_config.request_timeout.as_secs() + "Query cancelled, runtime exceeded. SQL queries over HTTP must not exceed {} seconds of runtime. Please consider using our websocket based connections", + config.http_config.request_timeout.as_secs_f64() ); error!(message); json_response( - StatusCode::GATEWAY_TIMEOUT, - json!({ "message": message, "code": StatusCode::GATEWAY_TIMEOUT.as_u16() }), + StatusCode::BAD_REQUEST, + json!({ "message": message, "code": SqlState::PROTOCOL_VIOLATION.code() }), )? } }; @@ -307,12 +317,15 @@ pub async fn handle( Ok(response) } +struct Cancelled(); + async fn handle_inner( + cancel: CancellationToken, config: &'static ProxyConfig, ctx: &mut RequestMonitoring, request: Request, backend: Arc, -) -> anyhow::Result> { +) -> Result, Cancelled>, anyhow::Error> { let _request_gauge = NUM_CONNECTION_REQUESTS_GAUGE .with_label_values(&[ctx.protocol]) .guard(); @@ -389,7 +402,18 @@ async fn handle_inner( }; // Run both operations in parallel - let (payload, mut client) = try_join!(fetch_and_process_request, authenticate_and_connect)?; + let (payload, mut client) = match select( + try_join( + pin!(fetch_and_process_request), + pin!(authenticate_and_connect), + ), + pin!(cancel.cancelled()), + ) + .await + { + Either::Left((result, _cancelled)) => result?, + Either::Right((_cancelled, _)) => return Ok(Err(Cancelled())), + }; let mut response = Response::builder() .status(StatusCode::OK) @@ -401,19 +425,60 @@ async fn handle_inner( let mut size = 0; let result = match payload { Payload::Single(stmt) => { - let (status, results) = - query_to_json(&*client, stmt, &mut 0, raw_output, default_array_mode) - .await - .map_err(|e| { - client.discard(); - e - })?; - client.check_idle(status); - results + let mut size = 0; + let (inner, mut discard) = client.inner(); + let cancel_token = inner.cancel_token(); + let query = pin!(query_to_json( + &*inner, + stmt, + &mut size, + raw_output, + default_array_mode + )); + let cancelled = pin!(cancel.cancelled()); + let res = select(query, cancelled).await; + match res { + Either::Left((Ok((status, results)), _cancelled)) => { + discard.check_idle(status); + results + } + Either::Left((Err(e), _cancelled)) => { + discard.discard(); + return Err(e); + } + Either::Right((_cancelled, query)) => { + if let Err(err) = cancel_token.cancel_query(NoTls).await { + tracing::error!(?err, "could not cancel query"); + } + match time::timeout(time::Duration::from_millis(100), query).await { + Ok(Ok((status, results))) => { + discard.check_idle(status); + results + } + Ok(Err(error)) => { + let db_error = error + .downcast_ref::() + .and_then(|e| e.as_db_error()); + + // if errored for some other reason, it might not be safe to return + if !db_error.is_some_and(|e| *e.code() == SqlState::QUERY_CANCELED) { + discard.discard(); + } + + return Ok(Err(Cancelled())); + } + Err(_timeout) => { + discard.discard(); + return Ok(Err(Cancelled())); + } + } + } + } } Payload::Batch(statements) => { info!("starting transaction"); let (inner, mut discard) = client.inner(); + let cancel_token = inner.cancel_token(); let mut builder = inner.build_transaction(); if let Some(isolation_level) = txn_isolation_level { builder = builder.isolation_level(isolation_level); @@ -433,6 +498,7 @@ async fn handle_inner( })?; let results = match query_batch( + cancel.child_token(), &transaction, statements, &mut size, @@ -441,7 +507,7 @@ async fn handle_inner( ) .await { - Ok(results) => { + Ok(Ok(results)) => { info!("commit"); let status = transaction.commit().await.map_err(|e| { // if we cannot commit - for now don't return connection to pool @@ -452,6 +518,15 @@ async fn handle_inner( discard.check_idle(status); results } + Ok(Err(Cancelled())) => { + if let Err(err) = cancel_token.cancel_query(NoTls).await { + tracing::error!(?err, "could not cancel query"); + } + // TODO: after cancelling, wait to see if we can get a status. maybe the connection is still safe. + discard.discard(); + + return Ok(Err(Cancelled())); + } Err(err) => { info!("rollback"); let status = transaction.rollback().await.map_err(|e| { @@ -499,26 +574,44 @@ async fn handle_inner( // moving this later in the stack is going to be a lot of effort and ehhhh metrics.record_egress(len as u64); - Ok(response) + Ok(Ok(response)) } async fn query_batch( + cancel: CancellationToken, transaction: &Transaction<'_>, queries: BatchQueryData, total_size: &mut usize, raw_output: bool, array_mode: bool, -) -> anyhow::Result> { +) -> anyhow::Result, Cancelled>> { let mut results = Vec::with_capacity(queries.queries.len()); let mut current_size = 0; for stmt in queries.queries { - // TODO: maybe we should check that the transaction bit is set here - let (_, values) = - query_to_json(transaction, stmt, &mut current_size, raw_output, array_mode).await?; - results.push(values); + let query = pin!(query_to_json( + transaction, + stmt, + &mut current_size, + raw_output, + array_mode + )); + let cancelled = pin!(cancel.cancelled()); + let res = select(query, cancelled).await; + match res { + // TODO: maybe we should check that the transaction bit is set here + Either::Left((Ok((_, values)), _cancelled)) => { + results.push(values); + } + Either::Left((Err(e), _cancelled)) => { + return Err(e); + } + Either::Right((_cancelled, _)) => { + return Ok(Err(Cancelled())); + } + } } *total_size += current_size; - Ok(results) + Ok(Ok(results)) } async fn query_to_json( diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index 234bfa8bf931..b7196a25561a 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -2859,6 +2859,7 @@ def __init__( self.auth_backend = auth_backend self.metric_collection_endpoint = metric_collection_endpoint self.metric_collection_interval = metric_collection_interval + self.http_timeout_seconds = 15 self._popen: Optional[subprocess.Popen[bytes]] = None def start(self) -> NeonProxy: @@ -2897,6 +2898,7 @@ def start(self) -> NeonProxy: *["--proxy", f"{self.host}:{self.proxy_port}"], *["--mgmt", f"{self.host}:{self.mgmt_port}"], *["--wss", f"{self.host}:{self.external_http_port}"], + *["--sql-over-http-timeout", f"{self.http_timeout_seconds}s"], *["-c", str(crt_path)], *["-k", str(key_path)], *self.auth_backend.extra_args(), @@ -2937,6 +2939,8 @@ def http_query(self, query, args, **kwargs): password = quote(kwargs["password"]) expected_code = kwargs.get("expected_code") + log.info(f"Executing http query: {query}") + connstr = f"postgresql://{user}:{password}@{self.domain}:{self.proxy_port}/postgres" response = requests.post( f"https://{self.domain}:{self.external_http_port}/sql", @@ -2959,6 +2963,8 @@ async def http2_query(self, query, args, **kwargs): password = kwargs["password"] expected_code = kwargs.get("expected_code") + log.info(f"Executing http2 query: {query}") + connstr = f"postgresql://{user}:{password}@{self.domain}:{self.proxy_port}/postgres" async with httpx.AsyncClient( http2=True, verify=str(self.test_output_dir / "proxy.crt") diff --git a/test_runner/regress/test_proxy.py b/test_runner/regress/test_proxy.py index 9905f120e1d8..078589d8ebe5 100644 --- a/test_runner/regress/test_proxy.py +++ b/test_runner/regress/test_proxy.py @@ -564,3 +564,35 @@ async def test_sql_over_http2(static_proxy: NeonProxy): "select 42 as answer", [], user="http", password="http", expected_code=200 ) assert resp["rows"] == [{"answer": 42}] + + +def test_sql_over_http_timeout_cancel(static_proxy: NeonProxy): + static_proxy.safe_psql("create role http with login password 'http' superuser") + + static_proxy.safe_psql("create table test_table ( id int primary key )") + + # insert into a table, with a unique constraint, after sleeping for n seconds + query = "WITH temp AS ( \ + SELECT pg_sleep($1) as sleep, $2::int as id \ + ) INSERT INTO test_table (id) SELECT id FROM temp" + + # expect to fail with timeout + res = static_proxy.http_query( + query, + [static_proxy.http_timeout_seconds + 1, 1], + user="http", + password="http", + expected_code=400, + ) + assert "Query cancelled, runtime exceeded" in res["message"], "HTTP query should time out" + + time.sleep(2) + + res = static_proxy.http_query(query, [1, 1], user="http", password="http", expected_code=200) + assert res["command"] == "INSERT", "HTTP query should insert" + assert res["rowCount"] == 1, "HTTP query should insert" + + res = static_proxy.http_query(query, [0, 1], user="http", password="http", expected_code=400) + assert ( + "duplicate key value violates unique constraint" in res["message"] + ), "HTTP query should conflict" From 580e136b2e67321970b95e0fb51d46d4a2bec550 Mon Sep 17 00:00:00 2001 From: Arthur Petukhovsky Date: Tue, 12 Mar 2024 13:14:02 +0100 Subject: [PATCH 31/43] Forward all backpressure feedback to compute (#7079) Previously we aggregated ps_feedback on each safekeeper and sent it to walproposer with every AppendResponse. This PR changes it to send ps_feedback to walproposer right after receiving it from pageserver, without aggregating it in memory. Also contains some preparations for implementing backpressure support for sharding. --- libs/utils/src/pageserver_feedback.rs | 6 ++ safekeeper/src/metrics.rs | 30 ++++++- safekeeper/src/receive_wal.rs | 98 ++++++++++++++++++----- safekeeper/src/safekeeper.rs | 14 ++-- safekeeper/src/send_wal.rs | 109 ++++++++++---------------- safekeeper/src/timeline.rs | 22 +++--- 6 files changed, 172 insertions(+), 107 deletions(-) diff --git a/libs/utils/src/pageserver_feedback.rs b/libs/utils/src/pageserver_feedback.rs index c9fbdde92847..bc8fa7362eaa 100644 --- a/libs/utils/src/pageserver_feedback.rs +++ b/libs/utils/src/pageserver_feedback.rs @@ -123,6 +123,12 @@ impl PageserverFeedback { rf.replytime = *PG_EPOCH - Duration::from_micros(-raw_time as u64); } } + b"shard_number" => { + let len = buf.get_i32(); + // TODO: this will be implemented in the next update, + // for now, we just skip the value. + buf.advance(len as usize); + } _ => { let len = buf.get_i32(); warn!( diff --git a/safekeeper/src/metrics.rs b/safekeeper/src/metrics.rs index f12e07963238..e541527b6aa2 100644 --- a/safekeeper/src/metrics.rs +++ b/safekeeper/src/metrics.rs @@ -140,6 +140,13 @@ pub static BROKER_ITERATION_TIMELINES: Lazy = Lazy::new(|| { ) .expect("Failed to register safekeeper_broker_iteration_timelines histogram vec") }); +pub static RECEIVED_PS_FEEDBACKS: Lazy = Lazy::new(|| { + register_int_counter!( + "safekeeper_received_ps_feedbacks_total", + "Number of pageserver feedbacks received" + ) + .expect("Failed to register safekeeper_received_ps_feedbacks_total counter") +}); pub const LABEL_UNKNOWN: &str = "unknown"; @@ -301,7 +308,8 @@ pub async fn time_io_closure>( #[derive(Clone)] pub struct FullTimelineInfo { pub ttid: TenantTimelineId, - pub ps_feedback: PageserverFeedback, + pub ps_feedback_count: u64, + pub last_ps_feedback: PageserverFeedback, pub wal_backup_active: bool, pub timeline_is_active: bool, pub num_computes: u32, @@ -327,6 +335,7 @@ pub struct TimelineCollector { remote_consistent_lsn: GenericGaugeVec, ps_last_received_lsn: GenericGaugeVec, feedback_last_time_seconds: GenericGaugeVec, + ps_feedback_count: GenericGaugeVec, timeline_active: GenericGaugeVec, wal_backup_active: GenericGaugeVec, connected_computes: IntGaugeVec, @@ -430,6 +439,15 @@ impl TimelineCollector { .unwrap(); descs.extend(feedback_last_time_seconds.desc().into_iter().cloned()); + let ps_feedback_count = GenericGaugeVec::new( + Opts::new( + "safekeeper_ps_feedback_count_total", + "Number of feedbacks received from the pageserver", + ), + &["tenant_id", "timeline_id"], + ) + .unwrap(); + let timeline_active = GenericGaugeVec::new( Opts::new( "safekeeper_timeline_active", @@ -538,6 +556,7 @@ impl TimelineCollector { remote_consistent_lsn, ps_last_received_lsn, feedback_last_time_seconds, + ps_feedback_count, timeline_active, wal_backup_active, connected_computes, @@ -570,6 +589,7 @@ impl Collector for TimelineCollector { self.remote_consistent_lsn.reset(); self.ps_last_received_lsn.reset(); self.feedback_last_time_seconds.reset(); + self.ps_feedback_count.reset(); self.timeline_active.reset(); self.wal_backup_active.reset(); self.connected_computes.reset(); @@ -646,9 +666,12 @@ impl Collector for TimelineCollector { self.ps_last_received_lsn .with_label_values(labels) - .set(tli.ps_feedback.last_received_lsn.0); + .set(tli.last_ps_feedback.last_received_lsn.0); + self.ps_feedback_count + .with_label_values(labels) + .set(tli.ps_feedback_count); if let Ok(unix_time) = tli - .ps_feedback + .last_ps_feedback .replytime .duration_since(SystemTime::UNIX_EPOCH) { @@ -679,6 +702,7 @@ impl Collector for TimelineCollector { mfs.extend(self.remote_consistent_lsn.collect()); mfs.extend(self.ps_last_received_lsn.collect()); mfs.extend(self.feedback_last_time_seconds.collect()); + mfs.extend(self.ps_feedback_count.collect()); mfs.extend(self.timeline_active.collect()); mfs.extend(self.wal_backup_active.collect()); mfs.extend(self.connected_computes.collect()); diff --git a/safekeeper/src/receive_wal.rs b/safekeeper/src/receive_wal.rs index 9ce9b049ba5d..015b53bb2e6f 100644 --- a/safekeeper/src/receive_wal.rs +++ b/safekeeper/src/receive_wal.rs @@ -36,11 +36,15 @@ use tokio::time::Instant; use tracing::*; use utils::id::TenantTimelineId; use utils::lsn::Lsn; +use utils::pageserver_feedback::PageserverFeedback; + +const DEFAULT_FEEDBACK_CAPACITY: usize = 8; /// Registry of WalReceivers (compute connections). Timeline holds it (wrapped /// in Arc). pub struct WalReceivers { mutex: Mutex, + pageserver_feedback_tx: tokio::sync::broadcast::Sender, } /// Id under which walreceiver is registered in shmem. @@ -48,8 +52,12 @@ type WalReceiverId = usize; impl WalReceivers { pub fn new() -> Arc { + let (pageserver_feedback_tx, _) = + tokio::sync::broadcast::channel(DEFAULT_FEEDBACK_CAPACITY); + Arc::new(WalReceivers { mutex: Mutex::new(WalReceiversShared { slots: Vec::new() }), + pageserver_feedback_tx, }) } @@ -116,6 +124,12 @@ impl WalReceivers { let mut shared = self.mutex.lock(); shared.slots[id] = None; } + + /// Broadcast pageserver feedback to connected walproposers. + pub fn broadcast_pageserver_feedback(&self, feedback: PageserverFeedback) { + // Err means there is no subscribers, it is fine. + let _ = self.pageserver_feedback_tx.send(feedback); + } } /// Only a few connections are expected (normally one), so store in Vec. @@ -197,17 +211,28 @@ impl SafekeeperPostgresHandler { // sends, so this avoids deadlocks. let mut pgb_reader = pgb.split().context("START_WAL_PUSH split")?; let peer_addr = *pgb.get_peer_addr(); - let network_reader = NetworkReader { + let mut network_reader = NetworkReader { ttid: self.ttid, conn_id: self.conn_id, pgb_reader: &mut pgb_reader, peer_addr, acceptor_handle: &mut acceptor_handle, }; - let res = tokio::select! { - // todo: add read|write .context to these errors - r = network_reader.run(msg_tx, msg_rx, reply_tx) => r, - r = network_write(pgb, reply_rx) => r, + + // Read first message and create timeline if needed. + let res = network_reader.read_first_message().await; + + let res = if let Ok((tli, next_msg)) = res { + let pageserver_feedback_rx: tokio::sync::broadcast::Receiver = + tli.get_walreceivers().pageserver_feedback_tx.subscribe(); + + tokio::select! { + // todo: add read|write .context to these errors + r = network_reader.run(msg_tx, msg_rx, reply_tx, tli.clone(), next_msg) => r, + r = network_write(pgb, reply_rx, pageserver_feedback_rx) => r, + } + } else { + res.map(|_| ()) }; // Join pg backend back. @@ -251,12 +276,9 @@ struct NetworkReader<'a, IO> { } impl<'a, IO: AsyncRead + AsyncWrite + Unpin> NetworkReader<'a, IO> { - async fn run( - self, - msg_tx: Sender, - msg_rx: Receiver, - reply_tx: Sender, - ) -> Result<(), CopyStreamHandlerEnd> { + async fn read_first_message( + &mut self, + ) -> Result<(Arc, ProposerAcceptorMessage), CopyStreamHandlerEnd> { // Receive information about server to create timeline, if not yet. let next_msg = read_message(self.pgb_reader).await?; let tli = match next_msg { @@ -278,9 +300,19 @@ impl<'a, IO: AsyncRead + AsyncWrite + Unpin> NetworkReader<'a, IO> { ))) } }; + Ok((tli, next_msg)) + } + async fn run( + self, + msg_tx: Sender, + msg_rx: Receiver, + reply_tx: Sender, + tli: Arc, + next_msg: ProposerAcceptorMessage, + ) -> Result<(), CopyStreamHandlerEnd> { *self.acceptor_handle = Some(WalAcceptor::spawn( - tli.clone(), + tli, msg_rx, reply_tx, Some(self.conn_id), @@ -320,18 +352,46 @@ async fn read_network_loop( async fn network_write( pgb_writer: &mut PostgresBackend, mut reply_rx: Receiver, + mut pageserver_feedback_rx: tokio::sync::broadcast::Receiver, ) -> Result<(), CopyStreamHandlerEnd> { let mut buf = BytesMut::with_capacity(128); + // storing append_response to inject PageserverFeedback into it + let mut last_append_response = None; + loop { - match reply_rx.recv().await { - Some(msg) => { - buf.clear(); - msg.serialize(&mut buf)?; - pgb_writer.write_message(&BeMessage::CopyData(&buf)).await?; + // trying to read either AcceptorProposerMessage or PageserverFeedback + let msg = tokio::select! { + reply = reply_rx.recv() => { + if let Some(msg) = reply { + if let AcceptorProposerMessage::AppendResponse(append_response) = &msg { + last_append_response = Some(append_response.clone()); + } + Some(msg) + } else { + return Ok(()); // chan closed, WalAcceptor terminated + } } - None => return Ok(()), // chan closed, WalAcceptor terminated - } + + feedback = pageserver_feedback_rx.recv() => + match (feedback, &last_append_response) { + (Ok(feedback), Some(append_response)) => { + // clone AppendResponse and inject PageserverFeedback into it + let mut append_response = append_response.clone(); + append_response.pageserver_feedback = Some(feedback); + Some(AcceptorProposerMessage::AppendResponse(append_response)) + } + _ => None, + } + }; + + let Some(msg) = msg else { + continue; + }; + + buf.clear(); + msg.serialize(&mut buf)?; + pgb_writer.write_message(&BeMessage::CopyData(&buf)).await?; } } diff --git a/safekeeper/src/safekeeper.rs b/safekeeper/src/safekeeper.rs index 84393d8dabfc..d7c8fa6955e2 100644 --- a/safekeeper/src/safekeeper.rs +++ b/safekeeper/src/safekeeper.rs @@ -321,7 +321,7 @@ pub struct AppendRequestHeader { } /// Report safekeeper state to proposer -#[derive(Debug, Serialize)] +#[derive(Debug, Serialize, Clone)] pub struct AppendResponse { // Current term of the safekeeper; if it is higher than proposer's, the // compute is out of date. @@ -334,7 +334,7 @@ pub struct AppendResponse { // a criterion for walproposer --sync mode exit pub commit_lsn: Lsn, pub hs_feedback: HotStandbyFeedback, - pub pageserver_feedback: PageserverFeedback, + pub pageserver_feedback: Option, } impl AppendResponse { @@ -344,7 +344,7 @@ impl AppendResponse { flush_lsn: Lsn(0), commit_lsn: Lsn(0), hs_feedback: HotStandbyFeedback::empty(), - pageserver_feedback: PageserverFeedback::empty(), + pageserver_feedback: None, } } } @@ -462,7 +462,11 @@ impl AcceptorProposerMessage { buf.put_u64_le(msg.hs_feedback.xmin); buf.put_u64_le(msg.hs_feedback.catalog_xmin); - msg.pageserver_feedback.serialize(buf); + // AsyncReadMessage in walproposer.c will not try to decode pageserver_feedback + // if it is not present. + if let Some(ref msg) = msg.pageserver_feedback { + msg.serialize(buf); + } } } @@ -681,7 +685,7 @@ where commit_lsn: self.state.commit_lsn, // will be filled by the upper code to avoid bothering safekeeper hs_feedback: HotStandbyFeedback::empty(), - pageserver_feedback: PageserverFeedback::empty(), + pageserver_feedback: None, }; trace!("formed AppendResponse {:?}", ar); ar diff --git a/safekeeper/src/send_wal.rs b/safekeeper/src/send_wal.rs index 4b887f36b770..7da5fd00b01e 100644 --- a/safekeeper/src/send_wal.rs +++ b/safekeeper/src/send_wal.rs @@ -2,6 +2,8 @@ //! with the "START_REPLICATION" message, and registry of walsenders. use crate::handler::SafekeeperPostgresHandler; +use crate::metrics::RECEIVED_PS_FEEDBACKS; +use crate::receive_wal::WalReceivers; use crate::safekeeper::{Term, TermLsn}; use crate::timeline::Timeline; use crate::wal_service::ConnectionId; @@ -21,7 +23,7 @@ use utils::failpoint_support; use utils::id::TenantTimelineId; use utils::pageserver_feedback::PageserverFeedback; -use std::cmp::{max, min}; +use std::cmp::min; use std::net::SocketAddr; use std::str; use std::sync::Arc; @@ -90,12 +92,14 @@ pub struct StandbyFeedback { /// WalSenders registry. Timeline holds it (wrapped in Arc). pub struct WalSenders { mutex: Mutex, + walreceivers: Arc, } impl WalSenders { - pub fn new() -> Arc { + pub fn new(walreceivers: Arc) -> Arc { Arc::new(WalSenders { mutex: Mutex::new(WalSendersShared::new()), + walreceivers, }) } @@ -151,22 +155,29 @@ impl WalSenders { .min() } - /// Get aggregated pageserver feedback. - pub fn get_ps_feedback(self: &Arc) -> PageserverFeedback { - self.mutex.lock().agg_ps_feedback + /// Returns total counter of pageserver feedbacks received and last feedback. + pub fn get_ps_feedback_stats(self: &Arc) -> (u64, PageserverFeedback) { + let shared = self.mutex.lock(); + (shared.ps_feedback_counter, shared.last_ps_feedback) } - /// Get aggregated pageserver and hot standby feedback (we send them to compute). - pub fn get_feedbacks(self: &Arc) -> (PageserverFeedback, HotStandbyFeedback) { - let shared = self.mutex.lock(); - (shared.agg_ps_feedback, shared.agg_hs_feedback) + /// Get aggregated hot standby feedback (we send it to compute). + pub fn get_hotstandby(self: &Arc) -> HotStandbyFeedback { + self.mutex.lock().agg_hs_feedback } /// Record new pageserver feedback, update aggregated values. fn record_ps_feedback(self: &Arc, id: WalSenderId, feedback: &PageserverFeedback) { let mut shared = self.mutex.lock(); shared.get_slot_mut(id).feedback = ReplicationFeedback::Pageserver(*feedback); - shared.update_ps_feedback(); + shared.last_ps_feedback = *feedback; + shared.ps_feedback_counter += 1; + drop(shared); + + RECEIVED_PS_FEEDBACKS.inc(); + + // send feedback to connected walproposers + self.walreceivers.broadcast_pageserver_feedback(*feedback); } /// Record standby reply. @@ -222,8 +233,10 @@ impl WalSenders { struct WalSendersShared { // aggregated over all walsenders value agg_hs_feedback: HotStandbyFeedback, - // aggregated over all walsenders value - agg_ps_feedback: PageserverFeedback, + // last feedback ever received from any pageserver, empty if none + last_ps_feedback: PageserverFeedback, + // total counter of pageserver feedbacks received + ps_feedback_counter: u64, slots: Vec>, } @@ -231,7 +244,8 @@ impl WalSendersShared { fn new() -> Self { WalSendersShared { agg_hs_feedback: HotStandbyFeedback::empty(), - agg_ps_feedback: PageserverFeedback::empty(), + last_ps_feedback: PageserverFeedback::empty(), + ps_feedback_counter: 0, slots: Vec::new(), } } @@ -276,37 +290,6 @@ impl WalSendersShared { } self.agg_hs_feedback = agg; } - - /// Update aggregated pageserver feedback. LSNs (last_received, - /// disk_consistent, remote_consistent) and reply timestamp are just - /// maximized; timeline_size if taken from feedback with highest - /// last_received lsn. This is generally reasonable, but we might want to - /// implement other policies once multiple pageservers start to be actively - /// used. - fn update_ps_feedback(&mut self) { - let init = PageserverFeedback::empty(); - let acc = - self.slots - .iter() - .flatten() - .fold(init, |mut acc, ws_state| match ws_state.feedback { - ReplicationFeedback::Pageserver(feedback) => { - if feedback.last_received_lsn > acc.last_received_lsn { - acc.current_timeline_size = feedback.current_timeline_size; - } - acc.last_received_lsn = - max(feedback.last_received_lsn, acc.last_received_lsn); - acc.disk_consistent_lsn = - max(feedback.disk_consistent_lsn, acc.disk_consistent_lsn); - acc.remote_consistent_lsn = - max(feedback.remote_consistent_lsn, acc.remote_consistent_lsn); - acc.replytime = max(feedback.replytime, acc.replytime); - acc - } - ReplicationFeedback::Standby(_) => acc, - }); - self.agg_ps_feedback = acc; - } } // Serialized is used only for pretty printing in json. @@ -443,7 +426,7 @@ impl SafekeeperPostgresHandler { }; let mut reply_reader = ReplyReader { reader, - ws_guard, + ws_guard: ws_guard.clone(), tli, }; @@ -452,6 +435,18 @@ impl SafekeeperPostgresHandler { r = sender.run() => r, r = reply_reader.run() => r, }; + + let ws_state = ws_guard + .walsenders + .mutex + .lock() + .get_slot(ws_guard.id) + .clone(); + info!( + "finished streaming to {}, feedback={:?}", + ws_state.addr, ws_state.feedback, + ); + // Join pg backend back. pgb.unsplit(reply_reader.reader)?; @@ -733,7 +728,6 @@ async fn wait_for_lsn( #[cfg(test)] mod tests { - use postgres_protocol::PG_EPOCH; use utils::id::{TenantId, TimelineId}; use super::*; @@ -792,27 +786,4 @@ mod tests { wss.update_hs_feedback(); assert_eq!(wss.agg_hs_feedback.xmin, 42); } - - // form pageserver feedback with given last_record_lsn / tli size and the - // rest set to dummy values. - fn ps_feedback(current_timeline_size: u64, last_received_lsn: Lsn) -> ReplicationFeedback { - ReplicationFeedback::Pageserver(PageserverFeedback { - current_timeline_size, - last_received_lsn, - disk_consistent_lsn: Lsn::INVALID, - remote_consistent_lsn: Lsn::INVALID, - replytime: *PG_EPOCH, - }) - } - - // test that ps aggregation works as expected - #[test] - fn test_ps_feedback() { - let mut wss = WalSendersShared::new(); - push_feedback(&mut wss, ps_feedback(8, Lsn(42))); - push_feedback(&mut wss, ps_feedback(4, Lsn(84))); - wss.update_ps_feedback(); - assert_eq!(wss.agg_ps_feedback.current_timeline_size, 4); - assert_eq!(wss.agg_ps_feedback.last_received_lsn, Lsn(84)); - } } diff --git a/safekeeper/src/timeline.rs b/safekeeper/src/timeline.rs index 9b7ab14218ef..4901b86acf40 100644 --- a/safekeeper/src/timeline.rs +++ b/safekeeper/src/timeline.rs @@ -402,6 +402,7 @@ impl Timeline { ))); let (cancellation_tx, cancellation_rx) = watch::channel(false); + let walreceivers = WalReceivers::new(); Ok(Timeline { ttid, wal_backup_launcher_tx, @@ -410,8 +411,8 @@ impl Timeline { term_flush_lsn_watch_tx, term_flush_lsn_watch_rx, mutex: Mutex::new(shared_state), - walsenders: WalSenders::new(), - walreceivers: WalReceivers::new(), + walsenders: WalSenders::new(walreceivers.clone()), + walreceivers, cancellation_rx, cancellation_tx, timeline_dir: conf.timeline_dir(&ttid), @@ -435,6 +436,7 @@ impl Timeline { let state = TimelinePersistentState::new(&ttid, server_info, vec![], commit_lsn, local_start_lsn); + let walreceivers = WalReceivers::new(); Ok(Timeline { ttid, wal_backup_launcher_tx, @@ -443,8 +445,8 @@ impl Timeline { term_flush_lsn_watch_tx, term_flush_lsn_watch_rx, mutex: Mutex::new(SharedState::create_new(conf, &ttid, state)?), - walsenders: WalSenders::new(), - walreceivers: WalReceivers::new(), + walsenders: WalSenders::new(walreceivers.clone()), + walreceivers, cancellation_rx, cancellation_tx, timeline_dir: conf.timeline_dir(&ttid), @@ -656,12 +658,9 @@ impl Timeline { let mut shared_state = self.write_shared_state().await; rmsg = shared_state.sk.process_msg(msg).await?; - // if this is AppendResponse, fill in proper pageserver and hot - // standby feedback. + // if this is AppendResponse, fill in proper hot standby feedback. if let Some(AcceptorProposerMessage::AppendResponse(ref mut resp)) = rmsg { - let (ps_feedback, hs_feedback) = self.walsenders.get_feedbacks(); - resp.hs_feedback = hs_feedback; - resp.pageserver_feedback = ps_feedback; + resp.hs_feedback = self.walsenders.get_hotstandby(); } commit_lsn = shared_state.sk.state.inmem.commit_lsn; @@ -898,12 +897,13 @@ impl Timeline { return None; } - let ps_feedback = self.walsenders.get_ps_feedback(); + let (ps_feedback_count, last_ps_feedback) = self.walsenders.get_ps_feedback_stats(); let state = self.write_shared_state().await; if state.active { Some(FullTimelineInfo { ttid: self.ttid, - ps_feedback, + ps_feedback_count, + last_ps_feedback, wal_backup_active: state.wal_backup_active, timeline_is_active: state.active, num_computes: self.walreceivers.get_num() as u32, From 1f7d54f9872482b4b181f93dee2e6d91173d0ef8 Mon Sep 17 00:00:00 2001 From: Conrad Ludgate Date: Tue, 12 Mar 2024 13:05:40 +0000 Subject: [PATCH 32/43] proxy refactor tls listener (#7056) ## Problem Now that we have tls-listener vendored, we can refactor and remove a lot of bloated code and make the whole flow a bit simpler ## Summary of changes 1. Remove dead code 2. Move the error handling to inside the `TlsListener` accept() function 3. Extract the peer_addr from the PROXY protocol header and log it with errors --- proxy/src/protocol2.rs | 8 +- proxy/src/serverless.rs | 30 +-- proxy/src/serverless/tls_listener.rs | 307 +++++++-------------------- 3 files changed, 90 insertions(+), 255 deletions(-) diff --git a/proxy/src/protocol2.rs b/proxy/src/protocol2.rs index 3a7aabca329a..f476cb9b37d8 100644 --- a/proxy/src/protocol2.rs +++ b/proxy/src/protocol2.rs @@ -17,7 +17,7 @@ use pin_project_lite::pin_project; use tokio::io::{AsyncRead, AsyncReadExt, AsyncWrite, ReadBuf}; use uuid::Uuid; -use crate::{metrics::NUM_CLIENT_CONNECTION_GAUGE, serverless::tls_listener::AsyncAccept}; +use crate::metrics::NUM_CLIENT_CONNECTION_GAUGE; pub struct ProxyProtocolAccept { pub incoming: AddrIncoming, @@ -331,15 +331,15 @@ impl AsyncRead for WithClientIp { } } -impl AsyncAccept for ProxyProtocolAccept { - type Connection = WithConnectionGuard>; +impl Accept for ProxyProtocolAccept { + type Conn = WithConnectionGuard>; type Error = io::Error; fn poll_accept( mut self: Pin<&mut Self>, cx: &mut Context<'_>, - ) -> Poll>> { + ) -> Poll>> { let conn = ready!(Pin::new(&mut self.incoming).poll_accept(cx)?); tracing::info!(protocol = self.protocol, "accepted new TCP connection"); let Some(conn) = conn else { diff --git a/proxy/src/serverless.rs b/proxy/src/serverless.rs index c81ae03b232b..68f68eaba11f 100644 --- a/proxy/src/serverless.rs +++ b/proxy/src/serverless.rs @@ -21,24 +21,19 @@ pub use reqwest_retry::{policies::ExponentialBackoff, RetryTransientMiddleware}; use tokio_util::task::TaskTracker; use crate::context::RequestMonitoring; -use crate::metrics::TLS_HANDSHAKE_FAILURES; use crate::protocol2::{ProxyProtocolAccept, WithClientIp, WithConnectionGuard}; use crate::rate_limiter::EndpointRateLimiter; use crate::serverless::backend::PoolingBackend; use crate::{cancellation::CancellationHandler, config::ProxyConfig}; -use futures::StreamExt; use hyper::{ - server::{ - accept, - conn::{AddrIncoming, AddrStream}, - }, + server::conn::{AddrIncoming, AddrStream}, Body, Method, Request, Response, }; use std::convert::Infallible; use std::net::IpAddr; +use std::sync::Arc; use std::task::Poll; -use std::{future::ready, sync::Arc}; use tls_listener::TlsListener; use tokio::net::TcpListener; use tokio_util::sync::CancellationToken; @@ -105,19 +100,12 @@ pub async fn task_main( let ws_connections = tokio_util::task::task_tracker::TaskTracker::new(); ws_connections.close(); // allows `ws_connections.wait to complete` - let tls_listener = TlsListener::new(tls_acceptor, addr_incoming).filter(|conn| { - if let Err(err) = conn { - error!( - protocol = "http", - "failed to accept TLS connection: {err:?}" - ); - TLS_HANDSHAKE_FAILURES.inc(); - ready(false) - } else { - info!(protocol = "http", "accepted new TLS connection"); - ready(true) - } - }); + let tls_listener = TlsListener::new( + tls_acceptor, + addr_incoming, + "http", + config.handshake_timeout, + ); let make_svc = hyper::service::make_service_fn( |stream: &tokio_rustls::server::TlsStream< @@ -174,7 +162,7 @@ pub async fn task_main( }, ); - hyper::Server::builder(accept::from_stream(tls_listener)) + hyper::Server::builder(tls_listener) .serve(make_svc) .with_graceful_shutdown(cancellation_token.cancelled()) .await?; diff --git a/proxy/src/serverless/tls_listener.rs b/proxy/src/serverless/tls_listener.rs index 6196ff393c5a..cce02e385054 100644 --- a/proxy/src/serverless/tls_listener.rs +++ b/proxy/src/serverless/tls_listener.rs @@ -1,186 +1,110 @@ use std::{ + convert::Infallible, pin::Pin, task::{Context, Poll}, time::Duration, }; -use futures::{Future, Stream, StreamExt}; +use hyper::server::{accept::Accept, conn::AddrStream}; use pin_project_lite::pin_project; -use thiserror::Error; use tokio::{ io::{AsyncRead, AsyncWrite}, task::JoinSet, time::timeout, }; +use tokio_rustls::{server::TlsStream, TlsAcceptor}; +use tracing::{info, warn}; -/// Default timeout for the TLS handshake. -pub const DEFAULT_HANDSHAKE_TIMEOUT: Duration = Duration::from_secs(10); - -/// Trait for TLS implementation. -/// -/// Implementations are provided by the rustls and native-tls features. -pub trait AsyncTls: Clone { - /// The type of the TLS stream created from the underlying stream. - type Stream: Send + 'static; - /// Error type for completing the TLS handshake - type Error: std::error::Error + Send + 'static; - /// Type of the Future for the TLS stream that is accepted. - type AcceptFuture: Future> + Send + 'static; - - /// Accept a TLS connection on an underlying stream - fn accept(&self, stream: C) -> Self::AcceptFuture; -} - -/// Asynchronously accept connections. -pub trait AsyncAccept { - /// The type of the connection that is accepted. - type Connection: AsyncRead + AsyncWrite; - /// The type of error that may be returned. - type Error; - - /// Poll to accept the next connection. - fn poll_accept( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll>>; - - /// Return a new `AsyncAccept` that stops accepting connections after - /// `ender` completes. - /// - /// Useful for graceful shutdown. - /// - /// See [examples/echo.rs](https://github.com/tmccombs/tls-listener/blob/main/examples/echo.rs) - /// for example of how to use. - fn until(self, ender: F) -> Until - where - Self: Sized, - { - Until { - acceptor: self, - ender, - } - } -} +use crate::{ + metrics::TLS_HANDSHAKE_FAILURES, + protocol2::{WithClientIp, WithConnectionGuard}, +}; pin_project! { - /// /// Wraps a `Stream` of connections (such as a TCP listener) so that each connection is itself /// encrypted using TLS. - /// - /// It is similar to: - /// - /// ```ignore - /// tcpListener.and_then(|s| tlsAcceptor.accept(s)) - /// ``` - /// - /// except that it has the ability to accept multiple transport-level connections - /// simultaneously while the TLS handshake is pending for other connections. - /// - /// By default, if a client fails the TLS handshake, that is treated as an error, and the - /// `TlsListener` will return an `Err`. If the `TlsListener` is passed directly to a hyper - /// [`Server`][1], then an invalid handshake can cause the server to stop accepting connections. - /// See [`http-stream.rs`][2] or [`http-low-level`][3] examples, for examples of how to avoid this. - /// - /// Note that if the maximum number of pending connections is greater than 1, the resulting - /// [`T::Stream`][4] connections may come in a different order than the connections produced by the - /// underlying listener. - /// - /// [1]: https://docs.rs/hyper/latest/hyper/server/struct.Server.html - /// [2]: https://github.com/tmccombs/tls-listener/blob/main/examples/http-stream.rs - /// [3]: https://github.com/tmccombs/tls-listener/blob/main/examples/http-low-level.rs - /// [4]: AsyncTls::Stream - /// - #[allow(clippy::type_complexity)] - pub struct TlsListener> { + pub(crate) struct TlsListener { #[pin] listener: A, - tls: T, - waiting: JoinSet, tokio::time::error::Elapsed>>, + tls: TlsAcceptor, + waiting: JoinSet>>, timeout: Duration, + protocol: &'static str, } } -/// Builder for `TlsListener`. -#[derive(Clone)] -pub struct Builder { - tls: T, - handshake_timeout: Duration, -} - -/// Wraps errors from either the listener or the TLS Acceptor -#[derive(Debug, Error)] -pub enum Error { - /// An error that arose from the listener ([AsyncAccept::Error]) - #[error("{0}")] - ListenerError(#[source] LE), - /// An error that occurred during the TLS accept handshake - #[error("{0}")] - TlsAcceptError(#[source] TE), -} - -impl TlsListener -where - T: AsyncTls, -{ +impl TlsListener { /// Create a `TlsListener` with default options. - pub fn new(tls: T, listener: A) -> Self { - builder(tls).listen(listener) + pub(crate) fn new( + tls: TlsAcceptor, + listener: A, + protocol: &'static str, + timeout: Duration, + ) -> Self { + TlsListener { + listener, + tls, + waiting: JoinSet::new(), + timeout, + protocol, + } } } -impl TlsListener +impl Accept for TlsListener where - A: AsyncAccept, + A: Accept>>, A::Error: std::error::Error, - T: AsyncTls, + A::Conn: AsyncRead + AsyncWrite + Unpin + Send + 'static, { - /// Accept the next connection - /// - /// This is essentially an alias to `self.next()` with a more domain-appropriate name. - pub async fn accept(&mut self) -> Option<::Item> - where - Self: Unpin, - { - self.next().await - } + type Conn = TlsStream; - /// Replaces the Tls Acceptor configuration, which will be used for new connections. - /// - /// This can be used to change the certificate used at runtime. - pub fn replace_acceptor(&mut self, acceptor: T) { - self.tls = acceptor; - } + type Error = Infallible; - /// Replaces the Tls Acceptor configuration from a pinned reference to `Self`. - /// - /// This is useful if your listener is `!Unpin`. - /// - /// This can be used to change the certificate used at runtime. - pub fn replace_acceptor_pin(self: Pin<&mut Self>, acceptor: T) { - *self.project().tls = acceptor; - } -} - -impl Stream for TlsListener -where - A: AsyncAccept, - A::Error: std::error::Error, - T: AsyncTls, -{ - type Item = Result>; - - fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + fn poll_accept( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll>> { let mut this = self.project(); loop { match this.listener.as_mut().poll_accept(cx) { Poll::Pending => break, - Poll::Ready(Some(Ok(conn))) => { - this.waiting - .spawn(timeout(*this.timeout, this.tls.accept(conn))); + Poll::Ready(Some(Ok(mut conn))) => { + let t = *this.timeout; + let tls = this.tls.clone(); + let protocol = *this.protocol; + this.waiting.spawn(async move { + let peer_addr = match conn.inner.wait_for_addr().await { + Ok(Some(addr)) => addr, + Err(e) => { + tracing::error!("failed to accept TCP connection: invalid PROXY protocol V2 header: {e:#}"); + return None; + } + Ok(None) => conn.inner.inner.remote_addr() + }; + + let accept = tls.accept(conn); + match timeout(t, accept).await { + Ok(Ok(conn)) => Some(conn), + // The handshake failed, try getting another connection from the queue + Ok(Err(e)) => { + TLS_HANDSHAKE_FAILURES.inc(); + warn!(%peer_addr, protocol, "failed to accept TLS connection: {e:?}"); + None + } + // The handshake timed out, try getting another connection from the queue + Err(_) => { + TLS_HANDSHAKE_FAILURES.inc(); + warn!(%peer_addr, protocol, "failed to accept TLS connection: timeout"); + None + } + } + }); } Poll::Ready(Some(Err(e))) => { - return Poll::Ready(Some(Err(Error::ListenerError(e)))); + tracing::error!("error accepting TCP connection: {e}"); + continue; } Poll::Ready(None) => return Poll::Ready(None), } @@ -188,96 +112,19 @@ where loop { return match this.waiting.poll_join_next(cx) { - Poll::Ready(Some(Ok(Ok(conn)))) => { - Poll::Ready(Some(conn.map_err(Error::TlsAcceptError))) + Poll::Ready(Some(Ok(Some(conn)))) => { + info!(protocol = this.protocol, "accepted new TLS connection"); + Poll::Ready(Some(Ok(conn))) } - // The handshake timed out, try getting another connection from the queue - Poll::Ready(Some(Ok(Err(_)))) => continue, - // The handshake panicked - Poll::Ready(Some(Err(e))) if e.is_panic() => { - std::panic::resume_unwind(e.into_panic()) + // The handshake failed to complete, try getting another connection from the queue + Poll::Ready(Some(Ok(None))) => continue, + // The handshake panicked or was cancelled. ignore and get another connection + Poll::Ready(Some(Err(e))) => { + tracing::warn!("handshake aborted: {e}"); + continue; } - // The handshake was externally aborted - Poll::Ready(Some(Err(_))) => unreachable!("handshake tasks are never aborted"), _ => Poll::Pending, }; } } } - -impl AsyncTls for tokio_rustls::TlsAcceptor { - type Stream = tokio_rustls::server::TlsStream; - type Error = std::io::Error; - type AcceptFuture = tokio_rustls::Accept; - - fn accept(&self, conn: C) -> Self::AcceptFuture { - tokio_rustls::TlsAcceptor::accept(self, conn) - } -} - -impl Builder { - /// Set the timeout for handshakes. - /// - /// If a timeout takes longer than `timeout`, then the handshake will be - /// aborted and the underlying connection will be dropped. - /// - /// Defaults to `DEFAULT_HANDSHAKE_TIMEOUT`. - pub fn handshake_timeout(&mut self, timeout: Duration) -> &mut Self { - self.handshake_timeout = timeout; - self - } - - /// Create a `TlsListener` from the builder - /// - /// Actually build the `TlsListener`. The `listener` argument should be - /// an implementation of the `AsyncAccept` trait that accepts new connections - /// that the `TlsListener` will encrypt using TLS. - pub fn listen(&self, listener: A) -> TlsListener - where - T: AsyncTls, - { - TlsListener { - listener, - tls: self.tls.clone(), - waiting: JoinSet::new(), - timeout: self.handshake_timeout, - } - } -} - -/// Create a new Builder for a TlsListener -/// -/// `server_config` will be used to configure the TLS sessions. -pub fn builder(tls: T) -> Builder { - Builder { - tls, - handshake_timeout: DEFAULT_HANDSHAKE_TIMEOUT, - } -} - -pin_project! { - /// See [`AsyncAccept::until`] - pub struct Until { - #[pin] - acceptor: A, - #[pin] - ender: E, - } -} - -impl AsyncAccept for Until { - type Connection = A::Connection; - type Error = A::Error; - - fn poll_accept( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll>> { - let this = self.project(); - - match this.ender.poll(cx) { - Poll::Pending => this.acceptor.poll_accept(cx), - Poll::Ready(_) => Poll::Ready(None), - } - } -} From 7ae8364b0b0746b335f1d6e7c0d409fc1a236ffe Mon Sep 17 00:00:00 2001 From: John Spray Date: Tue, 12 Mar 2024 14:47:12 +0000 Subject: [PATCH 33/43] storage controller: register nodes in re-attach request (#7040) ## Problem Currently we manually register nodes with the storage controller, and use a script during deploy to register with the cloud control plane. Rather than extend that script further, nodes should just register on startup. ## Summary of changes - Extend the re-attach request to include an optional NodeRegisterRequest - If the `register` field is set, handle it like a normal node registration before executing the normal re-attach work. - Update tests/neon_local that used to rely on doing an explicit register step that could be enabled/disabled. --------- Co-authored-by: Christian Schwarz --- .../attachment_service/src/service.rs | 4 ++ control_plane/src/bin/neon_local.rs | 13 ++--- control_plane/src/endpoint.rs | 5 +- control_plane/src/pageserver.rs | 48 ++++++++-------- libs/pageserver_api/src/upcall_api.rs | 9 ++- pageserver/src/config.rs | 27 ++++++++- pageserver/src/control_plane_client.rs | 55 ++++++++++++++++++- pageserver/src/deletion_queue.rs | 5 +- pageserver/src/tenant/mgr.rs | 2 +- test_runner/fixtures/neon_fixtures.py | 20 ++++--- test_runner/regress/test_compatibility.py | 2 +- .../regress/test_pageserver_generations.py | 4 +- test_runner/regress/test_sharding_service.py | 3 +- 13 files changed, 145 insertions(+), 52 deletions(-) diff --git a/control_plane/attachment_service/src/service.rs b/control_plane/attachment_service/src/service.rs index 3f245b52559f..a8498a39b50e 100644 --- a/control_plane/attachment_service/src/service.rs +++ b/control_plane/attachment_service/src/service.rs @@ -922,6 +922,10 @@ impl Service { &self, reattach_req: ReAttachRequest, ) -> Result { + if let Some(register_req) = reattach_req.register { + self.node_register(register_req).await?; + } + // Take a re-attach as indication that the node is available: this is a precursor to proper // heartbeating in https://github.com/neondatabase/neon/issues/6844 self.node_configure(NodeConfigureRequest { diff --git a/control_plane/src/bin/neon_local.rs b/control_plane/src/bin/neon_local.rs index 86b9c0085dcc..952229c4b7fe 100644 --- a/control_plane/src/bin/neon_local.rs +++ b/control_plane/src/bin/neon_local.rs @@ -1100,9 +1100,8 @@ fn get_pageserver(env: &local_env::LocalEnv, args: &ArgMatches) -> Result Result<()> { match sub_match.subcommand() { Some(("start", subcommand_args)) => { - let register = subcommand_args.get_one::("register").unwrap_or(&true); if let Err(e) = get_pageserver(env, subcommand_args)? - .start(&pageserver_config_overrides(subcommand_args), *register) + .start(&pageserver_config_overrides(subcommand_args)) .await { eprintln!("pageserver start failed: {e}"); @@ -1131,7 +1130,7 @@ async fn handle_pageserver(sub_match: &ArgMatches, env: &local_env::LocalEnv) -> } if let Err(e) = pageserver - .start(&pageserver_config_overrides(subcommand_args), false) + .start(&pageserver_config_overrides(subcommand_args)) .await { eprintln!("pageserver start failed: {e}"); @@ -1293,7 +1292,7 @@ async fn handle_start_all(sub_match: &ArgMatches, env: &local_env::LocalEnv) -> for ps_conf in &env.pageservers { let pageserver = PageServerNode::from_env(env, ps_conf); if let Err(e) = pageserver - .start(&pageserver_config_overrides(sub_match), true) + .start(&pageserver_config_overrides(sub_match)) .await { eprintln!("pageserver {} start failed: {:#}", ps_conf.id, e); @@ -1596,11 +1595,7 @@ fn cli() -> Command { .subcommand(Command::new("status")) .subcommand(Command::new("start") .about("Start local pageserver") - .arg(pageserver_config_args.clone()).arg(Arg::new("register") - .long("register") - .default_value("true").required(false) - .value_parser(value_parser!(bool)) - .value_name("register")) + .arg(pageserver_config_args.clone()) ) .subcommand(Command::new("stop") .about("Stop local pageserver") diff --git a/control_plane/src/endpoint.rs b/control_plane/src/endpoint.rs index 646bc2e8bc39..5206222961ce 100644 --- a/control_plane/src/endpoint.rs +++ b/control_plane/src/endpoint.rs @@ -774,7 +774,10 @@ impl Endpoint { spec.shard_stripe_size = stripe_size.map(|s| s.0 as usize); } - let client = reqwest::Client::new(); + let client = reqwest::Client::builder() + .timeout(Duration::from_secs(30)) + .build() + .unwrap(); let response = client .post(format!( "http://{}:{}/configure", diff --git a/control_plane/src/pageserver.rs b/control_plane/src/pageserver.rs index 021b9aca34b0..06ec942895f2 100644 --- a/control_plane/src/pageserver.rs +++ b/control_plane/src/pageserver.rs @@ -17,7 +17,6 @@ use std::time::Duration; use anyhow::{bail, Context}; use camino::Utf8PathBuf; use futures::SinkExt; -use pageserver_api::controller_api::NodeRegisterRequest; use pageserver_api::models::{ self, LocationConfig, ShardParameters, TenantHistorySize, TenantInfo, TimelineInfo, }; @@ -32,7 +31,6 @@ use utils::{ }; use crate::local_env::PageServerConf; -use crate::storage_controller::StorageController; use crate::{background_process, local_env::LocalEnv}; /// Directory within .neon which will be used by default for LocalFs remote storage. @@ -163,8 +161,8 @@ impl PageServerNode { .expect("non-Unicode path") } - pub async fn start(&self, config_overrides: &[&str], register: bool) -> anyhow::Result<()> { - self.start_node(config_overrides, false, register).await + pub async fn start(&self, config_overrides: &[&str]) -> anyhow::Result<()> { + self.start_node(config_overrides, false).await } fn pageserver_init(&self, config_overrides: &[&str]) -> anyhow::Result<()> { @@ -202,6 +200,28 @@ impl PageServerNode { String::from_utf8_lossy(&init_output.stderr), ); + // Write metadata file, used by pageserver on startup to register itself with + // the storage controller + let metadata_path = datadir.join("metadata.json"); + + let (_http_host, http_port) = + parse_host_port(&self.conf.listen_http_addr).expect("Unable to parse listen_http_addr"); + let http_port = http_port.unwrap_or(9898); + // Intentionally hand-craft JSON: this acts as an implicit format compat test + // in case the pageserver-side structure is edited, and reflects the real life + // situation: the metadata is written by some other script. + std::fs::write( + metadata_path, + serde_json::to_vec(&serde_json::json!({ + "host": "localhost", + "port": self.pg_connection_config.port(), + "http_host": "localhost", + "http_port": http_port, + })) + .unwrap(), + ) + .expect("Failed to write metadata file"); + Ok(()) } @@ -209,27 +229,7 @@ impl PageServerNode { &self, config_overrides: &[&str], update_config: bool, - register: bool, ) -> anyhow::Result<()> { - // Register the node with the storage controller before starting pageserver: pageserver must be registered to - // successfully call /re-attach and finish starting up. - if register { - let storage_controller = StorageController::from_env(&self.env); - let (pg_host, pg_port) = - parse_host_port(&self.conf.listen_pg_addr).expect("Unable to parse listen_pg_addr"); - let (http_host, http_port) = parse_host_port(&self.conf.listen_http_addr) - .expect("Unable to parse listen_http_addr"); - storage_controller - .node_register(NodeRegisterRequest { - node_id: self.conf.id, - listen_pg_addr: pg_host.to_string(), - listen_pg_port: pg_port.unwrap_or(5432), - listen_http_addr: http_host.to_string(), - listen_http_port: http_port.unwrap_or(80), - }) - .await?; - } - // TODO: using a thread here because start_process() is not async but we need to call check_status() let datadir = self.repo_path(); print!( diff --git a/libs/pageserver_api/src/upcall_api.rs b/libs/pageserver_api/src/upcall_api.rs index 0acc3a7bb0ae..5472948091cb 100644 --- a/libs/pageserver_api/src/upcall_api.rs +++ b/libs/pageserver_api/src/upcall_api.rs @@ -6,11 +6,18 @@ use serde::{Deserialize, Serialize}; use utils::id::NodeId; -use crate::shard::TenantShardId; +use crate::{controller_api::NodeRegisterRequest, shard::TenantShardId}; +/// Upcall message sent by the pageserver to the configured `control_plane_api` on +/// startup. #[derive(Serialize, Deserialize)] pub struct ReAttachRequest { pub node_id: NodeId, + + /// Optional inline self-registration: this is useful with the storage controller, + /// if the node already has a node_id set. + #[serde(skip_serializing_if = "Option::is_none", default)] + pub register: Option, } #[derive(Serialize, Deserialize)] diff --git a/pageserver/src/config.rs b/pageserver/src/config.rs index 4adcedafd1c9..845b20c8db34 100644 --- a/pageserver/src/config.rs +++ b/pageserver/src/config.rs @@ -7,8 +7,9 @@ use anyhow::{anyhow, bail, ensure, Context, Result}; use pageserver_api::shard::TenantShardId; use remote_storage::{RemotePath, RemoteStorageConfig}; +use serde; use serde::de::IntoDeserializer; -use std::env; +use std::{collections::HashMap, env}; use storage_broker::Uri; use utils::crashsafe::path_with_suffix_extension; use utils::id::ConnectionId; @@ -304,6 +305,26 @@ impl BuilderValue { } } +// Certain metadata (e.g. externally-addressable name, AZ) is delivered +// as a separate structure. This information is not neeed by the pageserver +// itself, it is only used for registering the pageserver with the control +// plane and/or storage controller. +// +#[derive(serde::Deserialize)] +pub(crate) struct NodeMetadata { + #[serde(rename = "host")] + pub(crate) postgres_host: String, + #[serde(rename = "port")] + pub(crate) postgres_port: u16, + pub(crate) http_host: String, + pub(crate) http_port: u16, + + // Deployment tools may write fields to the metadata file beyond what we + // use in this type: this type intentionally only names fields that require. + #[serde(flatten)] + pub(crate) other: HashMap, +} + // needed to simplify config construction struct PageServerConfigBuilder { listen_pg_addr: BuilderValue, @@ -761,6 +782,10 @@ impl PageServerConf { self.workdir.join("deletion") } + pub fn metadata_path(&self) -> Utf8PathBuf { + self.workdir.join("metadata.json") + } + pub fn deletion_list_path(&self, sequence: u64) -> Utf8PathBuf { // Encode a version in the filename, so that if we ever switch away from JSON we can // increment this. diff --git a/pageserver/src/control_plane_client.rs b/pageserver/src/control_plane_client.rs index 3fcf3a983b36..1b3d76335dbc 100644 --- a/pageserver/src/control_plane_client.rs +++ b/pageserver/src/control_plane_client.rs @@ -2,6 +2,7 @@ use std::collections::HashMap; use futures::Future; use pageserver_api::{ + controller_api::NodeRegisterRequest, shard::TenantShardId, upcall_api::{ ReAttachRequest, ReAttachResponse, ValidateRequest, ValidateRequestTenant, ValidateResponse, @@ -12,7 +13,10 @@ use tokio_util::sync::CancellationToken; use url::Url; use utils::{backoff, generation::Generation, id::NodeId}; -use crate::config::PageServerConf; +use crate::{ + config::{NodeMetadata, PageServerConf}, + virtual_file::on_fatal_io_error, +}; /// The Pageserver's client for using the control plane API: this is a small subset /// of the overall control plane API, for dealing with generations (see docs/rfcs/025-generation-numbers.md) @@ -32,6 +36,7 @@ pub enum RetryForeverError { pub trait ControlPlaneGenerationsApi { fn re_attach( &self, + conf: &PageServerConf, ) -> impl Future, RetryForeverError>> + Send; fn validate( &self, @@ -110,13 +115,59 @@ impl ControlPlaneClient { impl ControlPlaneGenerationsApi for ControlPlaneClient { /// Block until we get a successful response, or error out if we are shut down - async fn re_attach(&self) -> Result, RetryForeverError> { + async fn re_attach( + &self, + conf: &PageServerConf, + ) -> Result, RetryForeverError> { let re_attach_path = self .base_url .join("re-attach") .expect("Failed to build re-attach path"); + + // Include registration content in the re-attach request if a metadata file is readable + let metadata_path = conf.metadata_path(); + let register = match tokio::fs::read_to_string(&metadata_path).await { + Ok(metadata_str) => match serde_json::from_str::(&metadata_str) { + Ok(m) => { + // Since we run one time at startup, be generous in our logging and + // dump all metadata. + tracing::info!( + "Loaded node metadata: postgres {}:{}, http {}:{}, other fields: {:?}", + m.postgres_host, + m.postgres_port, + m.http_host, + m.http_port, + m.other + ); + + Some(NodeRegisterRequest { + node_id: conf.id, + listen_pg_addr: m.postgres_host, + listen_pg_port: m.postgres_port, + listen_http_addr: m.http_host, + listen_http_port: m.http_port, + }) + } + Err(e) => { + tracing::error!("Unreadable metadata in {metadata_path}: {e}"); + None + } + }, + Err(e) => { + if e.kind() == std::io::ErrorKind::NotFound { + // This is legal: we may have been deployed with some external script + // doing registration for us. + tracing::info!("Metadata file not found at {metadata_path}"); + } else { + on_fatal_io_error(&e, &format!("Loading metadata at {metadata_path}")) + } + None + } + }; + let request = ReAttachRequest { node_id: self.node_id, + register, }; fail::fail_point!("control-plane-client-re-attach"); diff --git a/pageserver/src/deletion_queue.rs b/pageserver/src/deletion_queue.rs index 313eb2663d7c..b6aea8fae8db 100644 --- a/pageserver/src/deletion_queue.rs +++ b/pageserver/src/deletion_queue.rs @@ -831,7 +831,10 @@ mod test { } impl ControlPlaneGenerationsApi for MockControlPlane { - async fn re_attach(&self) -> Result, RetryForeverError> { + async fn re_attach( + &self, + _conf: &PageServerConf, + ) -> Result, RetryForeverError> { unimplemented!() } async fn validate( diff --git a/pageserver/src/tenant/mgr.rs b/pageserver/src/tenant/mgr.rs index fc08b3c82e27..38274448b350 100644 --- a/pageserver/src/tenant/mgr.rs +++ b/pageserver/src/tenant/mgr.rs @@ -295,7 +295,7 @@ async fn init_load_generations( } else if let Some(client) = ControlPlaneClient::new(conf, cancel) { 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().await { + match client.re_attach(conf).await { Ok(tenants) => tenants, Err(RetryForeverError::ShuttingDown) => { anyhow::bail!("Shut down while waiting for control plane re-attach response") diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index b7196a25561a..975c6d865b0c 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -519,9 +519,9 @@ def init_configs(self, default_remote_storage_if_missing: bool = True) -> NeonEn self.env = NeonEnv(self) return self.env - def start(self): + def start(self, register_pageservers=False): assert self.env is not None, "environment is not already initialized, call init() first" - self.env.start() + self.env.start(register_pageservers=register_pageservers) def init_start( self, @@ -1112,7 +1112,7 @@ def __init__(self, config: NeonEnvBuilder): log.info(f"Config: {cfg}") self.neon_cli.init(cfg, force=config.config_init_force) - def start(self): + def start(self, register_pageservers=False): # storage controller starts first, so that pageserver /re-attach calls don't # bounce through retries on startup self.storage_controller.start() @@ -1124,6 +1124,11 @@ def storage_controller_ready(): # reconcile. wait_until(30, 1, storage_controller_ready) + if register_pageservers: + # Special case for forward compat tests, this can be removed later. + for pageserver in self.pageservers: + self.storage_controller.node_register(pageserver) + # Start up broker, pageserver and all safekeepers futs = [] with concurrent.futures.ThreadPoolExecutor( @@ -1712,10 +1717,8 @@ def pageserver_start( id: int, overrides: Tuple[str, ...] = (), extra_env_vars: Optional[Dict[str, str]] = None, - register: bool = True, ) -> "subprocess.CompletedProcess[str]": - register_str = "true" if register else "false" - start_args = ["pageserver", "start", f"--id={id}", *overrides, f"--register={register_str}"] + start_args = ["pageserver", "start", f"--id={id}", *overrides] storage = self.env.pageserver_remote_storage append_pageserver_param_overrides( params_to_update=start_args, @@ -2066,6 +2069,8 @@ def node_register(self, node: NeonPageserver): "node_id": int(node.id), "listen_http_addr": "localhost", "listen_http_port": node.service_port.http, + "listen_pg_addr": "localhost", + "listen_pg_port": node.service_port.pg, } log.info(f"node_register({body})") self.request( @@ -2233,7 +2238,6 @@ def start( self, overrides: Tuple[str, ...] = (), extra_env_vars: Optional[Dict[str, str]] = None, - register: bool = True, ) -> "NeonPageserver": """ Start the page server. @@ -2243,7 +2247,7 @@ def start( assert self.running is False self.env.neon_cli.pageserver_start( - self.id, overrides=overrides, extra_env_vars=extra_env_vars, register=register + self.id, overrides=overrides, extra_env_vars=extra_env_vars ) self.running = True return self diff --git a/test_runner/regress/test_compatibility.py b/test_runner/regress/test_compatibility.py index 618ac637855e..5f815d3e6c52 100644 --- a/test_runner/regress/test_compatibility.py +++ b/test_runner/regress/test_compatibility.py @@ -242,7 +242,7 @@ def test_forward_compatibility( # everything else: our test code is written for latest CLI args. env.neon_local_binpath = neon_local_binpath - neon_env_builder.start() + neon_env_builder.start(register_pageservers=True) check_neon_works( env, diff --git a/test_runner/regress/test_pageserver_generations.py b/test_runner/regress/test_pageserver_generations.py index d1acb9817e5d..3ca13a904d9d 100644 --- a/test_runner/regress/test_pageserver_generations.py +++ b/test_runner/regress/test_pageserver_generations.py @@ -205,6 +205,9 @@ def test_generations_upgrade(neon_env_builder: NeonEnvBuilder): sk.start() env.storage_controller.start() + # We will start a pageserver with no control_plane_api set, so it won't be able to self-register + env.storage_controller.node_register(env.pageserver) + env.pageserver.start(overrides=('--pageserver-config-override=control_plane_api=""',)) env.neon_cli.create_tenant( @@ -511,7 +514,6 @@ def test_emergency_mode(neon_env_builder: NeonEnvBuilder, pg_bin: PgBin): env.pageserver.stop() # Non-immediate: implicitly checking that shutdown doesn't hang waiting for CP env.pageserver.start( overrides=("--pageserver-config-override=control_plane_emergency_mode=true",), - register=False, ) # The pageserver should provide service to clients diff --git a/test_runner/regress/test_sharding_service.py b/test_runner/regress/test_sharding_service.py index 6b7cd9d8290d..7a0707b564ae 100644 --- a/test_runner/regress/test_sharding_service.py +++ b/test_runner/regress/test_sharding_service.py @@ -278,13 +278,12 @@ def test_sharding_service_onboarding(neon_env_builder: NeonEnvBuilder, warm_up: env.pageservers[0].allowed_errors.append(".*Emergency mode!.*") env.pageservers[0].start( overrides=("--pageserver-config-override=control_plane_emergency_mode=true",), - register=False, ) origin_ps = env.pageservers[0] # This is the pageserver managed by the sharding service, where the tenant # will be attached after onboarding - env.pageservers[1].start(register=True) + env.pageservers[1].start() dest_ps = env.pageservers[1] virtual_ps_http = PageserverHttpClient(env.storage_controller_port, lambda: True) From bac06ea1accf54ae09c87cdd1f62e10565279b01 Mon Sep 17 00:00:00 2001 From: Jure Bajic Date: Tue, 12 Mar 2024 17:32:47 +0100 Subject: [PATCH 34/43] pageserver: fix read path max lsn bug (#7007) ## Summary of changes The problem it fixes is when `request_lsn` is `u64::MAX-1` the `cont_lsn` becomes `u64::MAX` which is the same as `prev_lsn` which stops the loop. Closes https://github.com/neondatabase/neon/issues/6812 --- pageserver/src/tenant.rs | 24 ++++++++++++++++++++---- pageserver/src/tenant/timeline.rs | 28 +++++++++++++++------------- 2 files changed, 35 insertions(+), 17 deletions(-) diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index 961995b2d610..f0996328c0a4 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -4625,10 +4625,7 @@ mod tests { drop(guard); // Pick a big LSN such that we query over all the changes. - // Technically, u64::MAX - 1 is the largest LSN supported by the read path, - // but there seems to be a bug on the non-vectored search path which surfaces - // in that case. - let reads_lsn = Lsn(u64::MAX - 1000); + let reads_lsn = Lsn(u64::MAX - 1); for read in reads { info!("Doing vectored read on {:?}", read); @@ -5145,4 +5142,23 @@ mod tests { Ok(()) } + + #[tokio::test] + async fn test_read_at_max_lsn() -> anyhow::Result<()> { + let harness = TenantHarness::create("test_read_at_max_lsn")?; + let (tenant, ctx) = harness.load().await; + let tline = tenant + .create_test_timeline(TIMELINE_ID, Lsn(0x08), DEFAULT_PG_VERSION, &ctx) + .await?; + + let lsn = Lsn(0x10); + bulk_insert_compact_gc(tline.clone(), &ctx, lsn, 50, 10000).await?; + + let test_key = Key::from_hex("010000000033333333444444445500000000").unwrap(); + let read_lsn = Lsn(u64::MAX - 1); + + assert!(tline.get(test_key, read_lsn, &ctx).await.is_ok()); + + Ok(()) + } } diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index c017d30f45de..a733a3b1a7f5 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -2478,7 +2478,7 @@ impl Timeline { // 'prev_lsn' tracks the last LSN that we were at in our search. It's used // to check that each iteration make some progress, to break infinite // looping if something goes wrong. - let mut prev_lsn = Lsn(u64::MAX); + let mut prev_lsn = None; let mut result = ValueReconstructResult::Continue; let mut cont_lsn = Lsn(request_lsn.0 + 1); @@ -2498,18 +2498,20 @@ impl Timeline { MATERIALIZED_PAGE_CACHE_HIT.inc_by(1); return Ok(traversal_path); } - if prev_lsn <= cont_lsn { - // Didn't make any progress in last iteration. Error out to avoid - // getting stuck in the loop. - return Err(layer_traversal_error(format!( - "could not find layer with more data for key {} at LSN {}, request LSN {}, ancestor {}", - key, - Lsn(cont_lsn.0 - 1), - request_lsn, - timeline.ancestor_lsn - ), traversal_path)); + if let Some(prev) = prev_lsn { + if prev <= cont_lsn { + // Didn't make any progress in last iteration. Error out to avoid + // getting stuck in the loop. + return Err(layer_traversal_error(format!( + "could not find layer with more data for key {} at LSN {}, request LSN {}, ancestor {}", + key, + Lsn(cont_lsn.0 - 1), + request_lsn, + timeline.ancestor_lsn + ), traversal_path)); + } } - prev_lsn = cont_lsn; + prev_lsn = Some(cont_lsn); } ValueReconstructResult::Missing => { return Err(layer_traversal_error( @@ -2539,7 +2541,7 @@ impl Timeline { timeline_owned = timeline.get_ready_ancestor_timeline(ctx).await?; timeline = &*timeline_owned; - prev_lsn = Lsn(u64::MAX); + prev_lsn = None; continue 'outer; } From 1b41db8bddfc1a89569346e1036df74f34454a4c Mon Sep 17 00:00:00 2001 From: John Spray Date: Tue, 12 Mar 2024 20:41:08 +0000 Subject: [PATCH 35/43] pageserver: enable setting stripe size inline with split request. (#7093) ## Summary - Currently we can set stripe size at tenant creation, but it doesn't mean anything until we have multiple shards - When onboarding an existing tenant, it will always get a default shard stripe size, so we would like to be able to pick the actual stripe size at the point we split. ## Why do this inline with a split? The alternative to this change would be to have a separate endpoint on the storage controller for setting the stripe size on a tenant, and only permit writes to that endpoint when the tenant has only a single shard. That would work, but be a little bit more work for a client, and not appreciably simpler (instead of having a special argument to the split functions, we'd have a special separate endpoint, and a requirement that the controller must sync its config down to the pageserver before calling the split API). Either approach would work, but this one feels a bit more robust end-to-end: the split API is the _very last moment_ that the stripe size is mutable, so if we aim to set it before splitting, it makes sense to do it as part of the same operation. --- .../attachment_service/src/service.rs | 14 ++- control_plane/src/bin/neon_local.rs | 7 +- control_plane/src/storage_controller.rs | 8 +- libs/pageserver_api/src/models.rs | 7 ++ pageserver/src/http/routes.rs | 9 +- pageserver/src/tenant/mgr.rs | 28 +++++- test_runner/fixtures/neon_fixtures.py | 6 +- test_runner/fixtures/pageserver/http.py | 7 ++ test_runner/regress/test_sharding.py | 95 +++++++++++++++++++ 9 files changed, 168 insertions(+), 13 deletions(-) diff --git a/control_plane/attachment_service/src/service.rs b/control_plane/attachment_service/src/service.rs index a8498a39b50e..ea301d037206 100644 --- a/control_plane/attachment_service/src/service.rs +++ b/control_plane/attachment_service/src/service.rs @@ -2222,7 +2222,18 @@ impl Service { // unwrap safety: we would have returned above if we didn't find at least one shard to split let old_shard_count = old_shard_count.unwrap(); - let shard_ident = shard_ident.unwrap(); + let shard_ident = if let Some(new_stripe_size) = split_req.new_stripe_size { + // This ShardIdentity will be used as the template for all children, so this implicitly + // applies the new stripe size to the children. + let mut shard_ident = shard_ident.unwrap(); + if shard_ident.count.count() > 1 && shard_ident.stripe_size != new_stripe_size { + return Err(ApiError::BadRequest(anyhow::anyhow!("Attempted to change stripe size ({:?}->{new_stripe_size:?}) on a tenant with multiple shards", shard_ident.stripe_size))); + } + shard_ident.stripe_size = new_stripe_size; + shard_ident + } else { + shard_ident.unwrap() + }; let policy = policy.unwrap(); // FIXME: we have dropped self.inner lock, and not yet written anything to the database: another @@ -2314,6 +2325,7 @@ impl Service { *parent_id, TenantShardSplitRequest { new_shard_count: split_req.new_shard_count, + new_stripe_size: split_req.new_stripe_size, }, ) .await diff --git a/control_plane/src/bin/neon_local.rs b/control_plane/src/bin/neon_local.rs index 952229c4b7fe..6c722f36b4a5 100644 --- a/control_plane/src/bin/neon_local.rs +++ b/control_plane/src/bin/neon_local.rs @@ -585,10 +585,14 @@ async fn handle_tenant( 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) + .tenant_split(tenant_id, shard_count, shard_stripe_size) .await?; println!( "Split tenant {} into shards {}", @@ -1585,6 +1589,7 @@ fn cli() -> Command { .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( diff --git a/control_plane/src/storage_controller.rs b/control_plane/src/storage_controller.rs index c505e677707d..d7673f1b2603 100644 --- a/control_plane/src/storage_controller.rs +++ b/control_plane/src/storage_controller.rs @@ -10,7 +10,7 @@ use pageserver_api::{ TenantCreateRequest, TenantShardSplitRequest, TenantShardSplitResponse, TimelineCreateRequest, TimelineInfo, }, - shard::TenantShardId, + shard::{ShardStripeSize, TenantShardId}, }; use pageserver_client::mgmt_api::ResponseErrorMessageExt; use postgres_backend::AuthType; @@ -496,11 +496,15 @@ impl StorageController { &self, tenant_id: TenantId, new_shard_count: u8, + new_stripe_size: Option, ) -> anyhow::Result { self.dispatch( Method::PUT, format!("control/v1/tenant/{tenant_id}/shard_split"), - Some(TenantShardSplitRequest { new_shard_count }), + Some(TenantShardSplitRequest { + new_shard_count, + new_stripe_size, + }), ) .await } diff --git a/libs/pageserver_api/src/models.rs b/libs/pageserver_api/src/models.rs index fe5bbd1c06fe..a96cc0915894 100644 --- a/libs/pageserver_api/src/models.rs +++ b/libs/pageserver_api/src/models.rs @@ -198,6 +198,13 @@ pub struct TimelineCreateRequest { #[derive(Serialize, Deserialize)] pub struct TenantShardSplitRequest { pub new_shard_count: u8, + + // A tenant's stripe size is only meaningful the first time their shard count goes + // above 1: therefore during a split from 1->N shards, we may modify the stripe size. + // + // If this is set while the stripe count is being increased from an already >1 value, + // then the request will fail with 400. + pub new_stripe_size: Option, } #[derive(Serialize, Deserialize)] diff --git a/pageserver/src/http/routes.rs b/pageserver/src/http/routes.rs index eafad9ab7383..bb8b1bb7e5f6 100644 --- a/pageserver/src/http/routes.rs +++ b/pageserver/src/http/routes.rs @@ -1151,7 +1151,12 @@ async fn tenant_shard_split_handler( let new_shards = state .tenant_manager - .shard_split(tenant_shard_id, ShardCount::new(req.new_shard_count), &ctx) + .shard_split( + tenant_shard_id, + ShardCount::new(req.new_shard_count), + req.new_stripe_size, + &ctx, + ) .await .map_err(ApiError::InternalServerError)?; @@ -2247,7 +2252,7 @@ pub fn make_router( .get("/v1/location_config", |r| { api_handler(r, list_location_config_handler) }) - .get("/v1/location_config/:tenant_id", |r| { + .get("/v1/location_config/:tenant_shard_id", |r| { api_handler(r, get_location_config_handler) }) .put( diff --git a/pageserver/src/tenant/mgr.rs b/pageserver/src/tenant/mgr.rs index 38274448b350..26fcce1f387c 100644 --- a/pageserver/src/tenant/mgr.rs +++ b/pageserver/src/tenant/mgr.rs @@ -6,7 +6,9 @@ use futures::stream::StreamExt; use itertools::Itertools; use pageserver_api::key::Key; use pageserver_api::models::ShardParameters; -use pageserver_api::shard::{ShardCount, ShardIdentity, ShardNumber, TenantShardId}; +use pageserver_api::shard::{ + ShardCount, ShardIdentity, ShardNumber, ShardStripeSize, TenantShardId, +}; use rand::{distributions::Alphanumeric, Rng}; use std::borrow::Cow; use std::cmp::Ordering; @@ -1439,11 +1441,12 @@ impl TenantManager { &self, tenant_shard_id: TenantShardId, new_shard_count: ShardCount, + new_stripe_size: Option, ctx: &RequestContext, ) -> anyhow::Result> { let tenant = get_tenant(tenant_shard_id, true)?; - // Plan: identify what the new child shards will be + // Validate the incoming request if new_shard_count.count() <= tenant_shard_id.shard_count.count() { anyhow::bail!("Requested shard count is not an increase"); } @@ -1452,10 +1455,18 @@ impl TenantManager { anyhow::bail!("Requested split is not a power of two"); } - let parent_shard_identity = tenant.shard_identity; - let parent_tenant_conf = tenant.get_tenant_conf(); - let parent_generation = tenant.generation; + if let Some(new_stripe_size) = new_stripe_size { + if tenant.get_shard_stripe_size() != new_stripe_size + && tenant_shard_id.shard_count.count() > 1 + { + // This tenant already has multiple shards, it is illegal to try and change its stripe size + anyhow::bail!( + "Shard stripe size may not be modified once tenant has multiple shards" + ); + } + } + // Plan: identify what the new child shards will be let child_shards = tenant_shard_id.split(new_shard_count); tracing::info!( "Shard {} splits into: {}", @@ -1466,6 +1477,10 @@ impl TenantManager { .join(",") ); + let parent_shard_identity = tenant.shard_identity; + let parent_tenant_conf = tenant.get_tenant_conf(); + let parent_generation = tenant.generation; + // Phase 1: Write out child shards' remote index files, in the parent tenant's current generation if let Err(e) = tenant.split_prepare(&child_shards).await { // If [`Tenant::split_prepare`] fails, we must reload the tenant, because it might @@ -1515,6 +1530,9 @@ impl TenantManager { // Phase 3: Spawn the child shards for child_shard in &child_shards { let mut child_shard_identity = parent_shard_identity; + if let Some(new_stripe_size) = new_stripe_size { + child_shard_identity.stripe_size = new_stripe_size; + } child_shard_identity.count = child_shard.shard_count; child_shard_identity.number = child_shard.shard_number; diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index 975c6d865b0c..b3f460c7fe63 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -2142,11 +2142,13 @@ def locate(self, tenant_id: TenantId) -> list[dict[str, Any]]: shards: list[dict[str, Any]] = body["shards"] return shards - def tenant_shard_split(self, tenant_id: TenantId, shard_count: int) -> list[TenantShardId]: + def tenant_shard_split( + self, tenant_id: TenantId, shard_count: int, shard_stripe_size: Optional[int] = None + ) -> list[TenantShardId]: response = self.request( "PUT", f"{self.env.storage_controller_api}/control/v1/tenant/{tenant_id}/shard_split", - json={"new_shard_count": shard_count}, + json={"new_shard_count": shard_count, "new_stripe_size": shard_stripe_size}, headers=self.headers(TokenScope.ADMIN), ) body = response.json() diff --git a/test_runner/fixtures/pageserver/http.py b/test_runner/fixtures/pageserver/http.py index b8e20c451fff..6e082374d7eb 100644 --- a/test_runner/fixtures/pageserver/http.py +++ b/test_runner/fixtures/pageserver/http.py @@ -318,6 +318,13 @@ def tenant_list_locations(self): assert isinstance(res_json["tenant_shards"], list) return res_json + def tenant_get_location(self, tenant_id: TenantShardId): + res = self.get( + f"http://localhost:{self.port}/v1/location_config/{tenant_id}", + ) + self.verbose_error(res) + return res.json() + def tenant_delete(self, tenant_id: Union[TenantId, TenantShardId]): res = self.delete(f"http://localhost:{self.port}/v1/tenant/{tenant_id}") self.verbose_error(res) diff --git a/test_runner/regress/test_sharding.py b/test_runner/regress/test_sharding.py index 1b96cd6a8091..9309af066b8e 100644 --- a/test_runner/regress/test_sharding.py +++ b/test_runner/regress/test_sharding.py @@ -1,4 +1,5 @@ import os +from typing import Dict, List, Union import pytest from fixtures.log_helper import log @@ -8,7 +9,11 @@ ) from fixtures.remote_storage import s3_storage from fixtures.types import Lsn, TenantShardId, TimelineId +from fixtures.utils import wait_until from fixtures.workload import Workload +from pytest_httpserver import HTTPServer +from werkzeug.wrappers.request import Request +from werkzeug.wrappers.response import Response def test_sharding_smoke( @@ -310,6 +315,96 @@ def shards_on_disk(shard_ids): workload.validate() +@pytest.mark.parametrize("initial_stripe_size", [None, 65536]) +def test_sharding_split_stripe_size( + neon_env_builder: NeonEnvBuilder, + httpserver: HTTPServer, + httpserver_listen_address, + initial_stripe_size: int, +): + """ + Check that modifying stripe size inline with a shard split works as expected + """ + (host, port) = httpserver_listen_address + neon_env_builder.control_plane_compute_hook_api = f"http://{host}:{port}/notify" + neon_env_builder.num_pageservers = 1 + + # Set up fake HTTP notify endpoint: we will use this to validate that we receive + # the correct stripe size after split. + notifications = [] + + def handler(request: Request): + log.info(f"Notify request: {request}") + notifications.append(request.json) + return Response(status=200) + + httpserver.expect_request("/notify", method="PUT").respond_with_handler(handler) + + env = neon_env_builder.init_start( + initial_tenant_shard_count=1, initial_tenant_shard_stripe_size=initial_stripe_size + ) + tenant_id = env.initial_tenant + + assert len(notifications) == 1 + expect: Dict[str, Union[List[Dict[str, int]], str, None, int]] = { + "tenant_id": str(env.initial_tenant), + "stripe_size": None, + "shards": [{"node_id": int(env.pageservers[0].id), "shard_number": 0}], + } + assert notifications[0] == expect + + new_stripe_size = 2048 + env.storage_controller.tenant_shard_split( + tenant_id, shard_count=2, shard_stripe_size=new_stripe_size + ) + + # Check that we ended up with the stripe size that we expected, both on the pageserver + # and in the notifications to compute + assert len(notifications) == 2 + expect_after: Dict[str, Union[List[Dict[str, int]], str, None, int]] = { + "tenant_id": str(env.initial_tenant), + "stripe_size": new_stripe_size, + "shards": [ + {"node_id": int(env.pageservers[0].id), "shard_number": 0}, + {"node_id": int(env.pageservers[0].id), "shard_number": 1}, + ], + } + log.info(f"Got notification: {notifications[1]}") + assert notifications[1] == expect_after + + # Inspect the stripe size on the pageserver + shard_0_loc = ( + env.pageservers[0].http_client().tenant_get_location(TenantShardId(tenant_id, 0, 2)) + ) + assert shard_0_loc["shard_stripe_size"] == new_stripe_size + shard_1_loc = ( + env.pageservers[0].http_client().tenant_get_location(TenantShardId(tenant_id, 1, 2)) + ) + assert shard_1_loc["shard_stripe_size"] == new_stripe_size + + # Ensure stripe size survives a pageserver restart + env.pageservers[0].stop() + env.pageservers[0].start() + shard_0_loc = ( + env.pageservers[0].http_client().tenant_get_location(TenantShardId(tenant_id, 0, 2)) + ) + assert shard_0_loc["shard_stripe_size"] == new_stripe_size + shard_1_loc = ( + env.pageservers[0].http_client().tenant_get_location(TenantShardId(tenant_id, 1, 2)) + ) + assert shard_1_loc["shard_stripe_size"] == new_stripe_size + + # Ensure stripe size survives a storage controller restart + env.storage_controller.stop() + env.storage_controller.start() + + def assert_restart_notification(): + assert len(notifications) == 3 + assert notifications[2] == expect_after + + wait_until(10, 1, assert_restart_notification) + + @pytest.mark.skipif( # The quantity of data isn't huge, but debug can be _very_ slow, and the things we're # validating in this test don't benefit much from debug assertions. From 83855a907c93ff5c8435d4f1acf3e71a40f5c18f Mon Sep 17 00:00:00 2001 From: Conrad Ludgate Date: Wed, 13 Mar 2024 06:35:49 +0000 Subject: [PATCH 36/43] proxy http error classification (#7098) ## Problem Missing error classification for SQL-over-HTTP queries. Not respecting `UserFacingError` for SQL-over-HTTP queries. ## Summary of changes Adds error classification. Adds user facing errors. --- proxy/src/serverless/backend.rs | 25 +++ proxy/src/serverless/conn_pool.rs | 14 +- proxy/src/serverless/sql_over_http.rs | 239 +++++++++++++++++++------- 3 files changed, 204 insertions(+), 74 deletions(-) diff --git a/proxy/src/serverless/backend.rs b/proxy/src/serverless/backend.rs index 2e63ad6c9920..d0f155165d4a 100644 --- a/proxy/src/serverless/backend.rs +++ b/proxy/src/serverless/backend.rs @@ -12,6 +12,7 @@ use crate::{ CachedNodeInfo, }, context::RequestMonitoring, + error::{ErrorKind, ReportableError, UserFacingError}, proxy::connect_compute::ConnectMechanism, }; @@ -117,6 +118,30 @@ pub enum HttpConnError { WakeCompute(#[from] WakeComputeError), } +impl ReportableError for HttpConnError { + fn get_error_kind(&self) -> ErrorKind { + match self { + HttpConnError::ConnectionClosedAbruptly(_) => ErrorKind::Compute, + HttpConnError::ConnectionError(p) => p.get_error_kind(), + HttpConnError::GetAuthInfo(a) => a.get_error_kind(), + HttpConnError::AuthError(a) => a.get_error_kind(), + HttpConnError::WakeCompute(w) => w.get_error_kind(), + } + } +} + +impl UserFacingError for HttpConnError { + fn to_string_client(&self) -> String { + match self { + HttpConnError::ConnectionClosedAbruptly(_) => self.to_string(), + HttpConnError::ConnectionError(p) => p.to_string(), + HttpConnError::GetAuthInfo(c) => c.to_string_client(), + HttpConnError::AuthError(c) => c.to_string_client(), + HttpConnError::WakeCompute(c) => c.to_string_client(), + } + } +} + struct TokioMechanism { pool: Arc>, conn_info: ConnInfo, diff --git a/proxy/src/serverless/conn_pool.rs b/proxy/src/serverless/conn_pool.rs index 901e30224bda..c7e8eaef7635 100644 --- a/proxy/src/serverless/conn_pool.rs +++ b/proxy/src/serverless/conn_pool.rs @@ -119,16 +119,12 @@ impl EndpointConnPool { } } - fn put( - pool: &RwLock, - conn_info: &ConnInfo, - client: ClientInner, - ) -> anyhow::Result<()> { + fn put(pool: &RwLock, conn_info: &ConnInfo, client: ClientInner) { let conn_id = client.conn_id; if client.is_closed() { info!(%conn_id, "pool: throwing away connection '{conn_info}' because connection is closed"); - return Ok(()); + return; } let global_max_conn = pool.read().global_pool_size_max_conns; if pool @@ -138,7 +134,7 @@ impl EndpointConnPool { >= global_max_conn { info!(%conn_id, "pool: throwing away connection '{conn_info}' because pool is full"); - return Ok(()); + return; } // return connection to the pool @@ -172,8 +168,6 @@ impl EndpointConnPool { } else { info!(%conn_id, "pool: throwing away connection '{conn_info}' because pool is full, total_conns={total_conns}"); } - - Ok(()) } } @@ -653,7 +647,7 @@ impl Client { // return connection to the pool return Some(move || { let _span = current_span.enter(); - let _ = EndpointConnPool::put(&conn_pool, &conn_info, client); + EndpointConnPool::put(&conn_pool, &conn_info, client); }); } None diff --git a/proxy/src/serverless/sql_over_http.rs b/proxy/src/serverless/sql_over_http.rs index 20d9795b470b..86c278030f9e 100644 --- a/proxy/src/serverless/sql_over_http.rs +++ b/proxy/src/serverless/sql_over_http.rs @@ -1,11 +1,11 @@ use std::pin::pin; use std::sync::Arc; -use anyhow::bail; use futures::future::select; use futures::future::try_join; use futures::future::Either; use futures::StreamExt; +use futures::TryFutureExt; use hyper::body::HttpBody; use hyper::header; use hyper::http::HeaderName; @@ -37,9 +37,13 @@ use crate::auth::ComputeUserInfoParseError; use crate::config::ProxyConfig; use crate::config::TlsConfig; use crate::context::RequestMonitoring; +use crate::error::ErrorKind; +use crate::error::ReportableError; +use crate::error::UserFacingError; use crate::metrics::HTTP_CONTENT_LENGTH; use crate::metrics::NUM_CONNECTION_REQUESTS_GAUGE; use crate::proxy::NeonOptions; +use crate::serverless::backend::HttpConnError; use crate::DbName; use crate::RoleName; @@ -47,6 +51,7 @@ use super::backend::PoolingBackend; use super::conn_pool::ConnInfo; use super::json::json_to_pg_text; use super::json::pg_text_row_to_json; +use super::json::JsonConversionError; #[derive(serde::Deserialize)] #[serde(rename_all = "camelCase")] @@ -117,6 +122,18 @@ pub enum ConnInfoError { MalformedEndpoint, } +impl ReportableError for ConnInfoError { + fn get_error_kind(&self) -> ErrorKind { + ErrorKind::User + } +} + +impl UserFacingError for ConnInfoError { + fn to_string_client(&self) -> String { + self.to_string() + } +} + fn get_conn_info( ctx: &mut RequestMonitoring, headers: &HeaderMap, @@ -212,17 +229,41 @@ pub async fn handle( handle.abort(); let mut response = match result { - Ok(Ok(r)) => { + Ok(r) => { ctx.set_success(); r } - Err(e) => { - // TODO: ctx.set_error_kind(e.get_error_type()); + Err(e @ SqlOverHttpError::Cancelled(_)) => { + let error_kind = e.get_error_kind(); + ctx.set_error_kind(error_kind); - let mut message = format!("{:?}", e); - let db_error = e - .downcast_ref::() - .and_then(|e| e.as_db_error()); + let message = format!( + "Query cancelled, runtime exceeded. SQL queries over HTTP must not exceed {} seconds of runtime. Please consider using our websocket based connections", + config.http_config.request_timeout.as_secs_f64() + ); + + tracing::info!( + kind=error_kind.to_metric_label(), + error=%e, + msg=message, + "forwarding error to user" + ); + + json_response( + StatusCode::BAD_REQUEST, + json!({ "message": message, "code": SqlState::PROTOCOL_VIOLATION.code() }), + )? + } + Err(e) => { + let error_kind = e.get_error_kind(); + ctx.set_error_kind(error_kind); + + let mut message = e.to_string_client(); + let db_error = match &e { + SqlOverHttpError::ConnectCompute(HttpConnError::ConnectionError(e)) + | SqlOverHttpError::Postgres(e) => e.as_db_error(), + _ => None, + }; fn get<'a, T: serde::Serialize>( db: Option<&'a DbError>, x: impl FnOnce(&'a DbError) -> T, @@ -265,10 +306,13 @@ pub async fn handle( let line = get(db_error, |db| db.line().map(|l| l.to_string())); let routine = get(db_error, |db| db.routine()); - error!( - ?code, - "sql-over-http per-client task finished with an error: {e:#}" + tracing::info!( + kind=error_kind.to_metric_label(), + error=%e, + msg=message, + "forwarding error to user" ); + // TODO: this shouldn't always be bad request. json_response( StatusCode::BAD_REQUEST, @@ -293,21 +337,6 @@ pub async fn handle( }), )? } - Ok(Err(Cancelled())) => { - // TODO: when http error classification is done, distinguish between - // timeout on sql vs timeout in proxy/cplane - // ctx.set_error_kind(crate::error::ErrorKind::RateLimit); - - let message = format!( - "Query cancelled, runtime exceeded. SQL queries over HTTP must not exceed {} seconds of runtime. Please consider using our websocket based connections", - config.http_config.request_timeout.as_secs_f64() - ); - error!(message); - json_response( - StatusCode::BAD_REQUEST, - json!({ "message": message, "code": SqlState::PROTOCOL_VIOLATION.code() }), - )? - } }; response.headers_mut().insert( @@ -317,7 +346,93 @@ pub async fn handle( Ok(response) } -struct Cancelled(); +#[derive(Debug, thiserror::Error)] +pub enum SqlOverHttpError { + #[error("{0}")] + ReadPayload(#[from] ReadPayloadError), + #[error("{0}")] + ConnectCompute(#[from] HttpConnError), + #[error("{0}")] + ConnInfo(#[from] ConnInfoError), + #[error("request is too large (max is {MAX_REQUEST_SIZE} bytes)")] + RequestTooLarge, + #[error("response is too large (max is {MAX_RESPONSE_SIZE} bytes)")] + ResponseTooLarge, + #[error("invalid isolation level")] + InvalidIsolationLevel, + #[error("{0}")] + Postgres(#[from] tokio_postgres::Error), + #[error("{0}")] + JsonConversion(#[from] JsonConversionError), + #[error("{0}")] + Cancelled(SqlOverHttpCancel), +} + +impl ReportableError for SqlOverHttpError { + fn get_error_kind(&self) -> ErrorKind { + match self { + SqlOverHttpError::ReadPayload(e) => e.get_error_kind(), + SqlOverHttpError::ConnectCompute(e) => e.get_error_kind(), + SqlOverHttpError::ConnInfo(e) => e.get_error_kind(), + SqlOverHttpError::RequestTooLarge => ErrorKind::User, + SqlOverHttpError::ResponseTooLarge => ErrorKind::User, + SqlOverHttpError::InvalidIsolationLevel => ErrorKind::User, + SqlOverHttpError::Postgres(p) => p.get_error_kind(), + SqlOverHttpError::JsonConversion(_) => ErrorKind::Postgres, + SqlOverHttpError::Cancelled(c) => c.get_error_kind(), + } + } +} + +impl UserFacingError for SqlOverHttpError { + fn to_string_client(&self) -> String { + match self { + SqlOverHttpError::ReadPayload(p) => p.to_string(), + SqlOverHttpError::ConnectCompute(c) => c.to_string_client(), + SqlOverHttpError::ConnInfo(c) => c.to_string_client(), + SqlOverHttpError::RequestTooLarge => self.to_string(), + SqlOverHttpError::ResponseTooLarge => self.to_string(), + SqlOverHttpError::InvalidIsolationLevel => self.to_string(), + SqlOverHttpError::Postgres(p) => p.to_string(), + SqlOverHttpError::JsonConversion(_) => "could not parse postgres response".to_string(), + SqlOverHttpError::Cancelled(_) => self.to_string(), + } + } +} + +#[derive(Debug, thiserror::Error)] +pub enum ReadPayloadError { + #[error("could not read the HTTP request body: {0}")] + Read(#[from] hyper::Error), + #[error("could not parse the HTTP request body: {0}")] + Parse(#[from] serde_json::Error), +} + +impl ReportableError for ReadPayloadError { + fn get_error_kind(&self) -> ErrorKind { + match self { + ReadPayloadError::Read(_) => ErrorKind::ClientDisconnect, + ReadPayloadError::Parse(_) => ErrorKind::User, + } + } +} + +#[derive(Debug, thiserror::Error)] +pub enum SqlOverHttpCancel { + #[error("query was cancelled")] + Postgres, + #[error("query was cancelled while stuck trying to connect to the database")] + Connect, +} + +impl ReportableError for SqlOverHttpCancel { + fn get_error_kind(&self) -> ErrorKind { + match self { + SqlOverHttpCancel::Postgres => ErrorKind::RateLimit, + SqlOverHttpCancel::Connect => ErrorKind::ServiceRateLimit, + } + } +} async fn handle_inner( cancel: CancellationToken, @@ -325,7 +440,7 @@ async fn handle_inner( ctx: &mut RequestMonitoring, request: Request, backend: Arc, -) -> Result, Cancelled>, anyhow::Error> { +) -> Result, SqlOverHttpError> { let _request_gauge = NUM_CONNECTION_REQUESTS_GAUGE .with_label_values(&[ctx.protocol]) .guard(); @@ -358,7 +473,7 @@ async fn handle_inner( b"ReadUncommitted" => IsolationLevel::ReadUncommitted, b"ReadCommitted" => IsolationLevel::ReadCommitted, b"RepeatableRead" => IsolationLevel::RepeatableRead, - _ => bail!("invalid isolation level"), + _ => return Err(SqlOverHttpError::InvalidIsolationLevel), }), None => None, }; @@ -376,19 +491,16 @@ async fn handle_inner( // we don't have a streaming request support yet so this is to prevent OOM // from a malicious user sending an extremely large request body if request_content_length > MAX_REQUEST_SIZE { - return Err(anyhow::anyhow!( - "request is too large (max is {MAX_REQUEST_SIZE} bytes)" - )); + return Err(SqlOverHttpError::RequestTooLarge); } let fetch_and_process_request = async { - let body = hyper::body::to_bytes(request.into_body()) - .await - .map_err(anyhow::Error::from)?; + let body = hyper::body::to_bytes(request.into_body()).await?; info!(length = body.len(), "request payload read"); let payload: Payload = serde_json::from_slice(&body)?; - Ok::(payload) // Adjust error type accordingly - }; + Ok::(payload) // Adjust error type accordingly + } + .map_err(SqlOverHttpError::from); let authenticate_and_connect = async { let keys = backend.authenticate(ctx, &conn_info).await?; @@ -398,8 +510,9 @@ async fn handle_inner( // not strictly necessary to mark success here, // but it's just insurance for if we forget it somewhere else ctx.latency_timer.success(); - Ok::<_, anyhow::Error>(client) - }; + Ok::<_, HttpConnError>(client) + } + .map_err(SqlOverHttpError::from); // Run both operations in parallel let (payload, mut client) = match select( @@ -412,7 +525,9 @@ async fn handle_inner( .await { Either::Left((result, _cancelled)) => result?, - Either::Right((_cancelled, _)) => return Ok(Err(Cancelled())), + Either::Right((_cancelled, _)) => { + return Err(SqlOverHttpError::Cancelled(SqlOverHttpCancel::Connect)) + } }; let mut response = Response::builder() @@ -456,20 +571,24 @@ async fn handle_inner( results } Ok(Err(error)) => { - let db_error = error - .downcast_ref::() - .and_then(|e| e.as_db_error()); + let db_error = match &error { + SqlOverHttpError::ConnectCompute( + HttpConnError::ConnectionError(e), + ) + | SqlOverHttpError::Postgres(e) => e.as_db_error(), + _ => None, + }; // if errored for some other reason, it might not be safe to return if !db_error.is_some_and(|e| *e.code() == SqlState::QUERY_CANCELED) { discard.discard(); } - return Ok(Err(Cancelled())); + return Err(SqlOverHttpError::Cancelled(SqlOverHttpCancel::Postgres)); } Err(_timeout) => { discard.discard(); - return Ok(Err(Cancelled())); + return Err(SqlOverHttpError::Cancelled(SqlOverHttpCancel::Postgres)); } } } @@ -507,7 +626,7 @@ async fn handle_inner( ) .await { - Ok(Ok(results)) => { + Ok(results) => { info!("commit"); let status = transaction.commit().await.map_err(|e| { // if we cannot commit - for now don't return connection to pool @@ -518,14 +637,14 @@ async fn handle_inner( discard.check_idle(status); results } - Ok(Err(Cancelled())) => { + Err(SqlOverHttpError::Cancelled(_)) => { if let Err(err) = cancel_token.cancel_query(NoTls).await { tracing::error!(?err, "could not cancel query"); } // TODO: after cancelling, wait to see if we can get a status. maybe the connection is still safe. discard.discard(); - return Ok(Err(Cancelled())); + return Err(SqlOverHttpError::Cancelled(SqlOverHttpCancel::Postgres)); } Err(err) => { info!("rollback"); @@ -541,16 +660,10 @@ async fn handle_inner( }; if txn_read_only { - response = response.header( - TXN_READ_ONLY.clone(), - HeaderValue::try_from(txn_read_only.to_string())?, - ); + response = response.header(TXN_READ_ONLY.clone(), &HEADER_VALUE_TRUE); } if txn_deferrable { - response = response.header( - TXN_DEFERRABLE.clone(), - HeaderValue::try_from(txn_deferrable.to_string())?, - ); + response = response.header(TXN_DEFERRABLE.clone(), &HEADER_VALUE_TRUE); } if let Some(txn_isolation_level) = txn_isolation_level_raw { response = response.header(TXN_ISOLATION_LEVEL.clone(), txn_isolation_level); @@ -574,7 +687,7 @@ async fn handle_inner( // moving this later in the stack is going to be a lot of effort and ehhhh metrics.record_egress(len as u64); - Ok(Ok(response)) + Ok(response) } async fn query_batch( @@ -584,7 +697,7 @@ async fn query_batch( total_size: &mut usize, raw_output: bool, array_mode: bool, -) -> anyhow::Result, Cancelled>> { +) -> Result, SqlOverHttpError> { let mut results = Vec::with_capacity(queries.queries.len()); let mut current_size = 0; for stmt in queries.queries { @@ -606,12 +719,12 @@ async fn query_batch( return Err(e); } Either::Right((_cancelled, _)) => { - return Ok(Err(Cancelled())); + return Err(SqlOverHttpError::Cancelled(SqlOverHttpCancel::Postgres)); } } } *total_size += current_size; - Ok(Ok(results)) + Ok(results) } async fn query_to_json( @@ -620,7 +733,7 @@ async fn query_to_json( current_size: &mut usize, raw_output: bool, default_array_mode: bool, -) -> anyhow::Result<(ReadyForQueryStatus, Value)> { +) -> Result<(ReadyForQueryStatus, Value), SqlOverHttpError> { info!("executing query"); let query_params = data.params; let mut row_stream = std::pin::pin!(client.query_raw_txt(&data.query, query_params).await?); @@ -637,9 +750,7 @@ async fn query_to_json( // we don't have a streaming response support yet so this is to prevent OOM // from a malicious query (eg a cross join) if *current_size > MAX_RESPONSE_SIZE { - return Err(anyhow::anyhow!( - "response is too large (max is {MAX_RESPONSE_SIZE} bytes)" - )); + return Err(SqlOverHttpError::ResponseTooLarge); } } From 0554bee02251ebf0bfdebf115a2ffc10c675782d Mon Sep 17 00:00:00 2001 From: Anna Khanova <32508607+khanova@users.noreply.github.com> Date: Wed, 13 Mar 2024 15:45:19 +0400 Subject: [PATCH 37/43] proxy: Report warm cold start if connection is from the local cache (#7104) ## Problem * quotes in serialized string * no status if connection is from local cache ## Summary of changes * remove quotes * report warm if connection if from local cache --- proxy/src/console/provider/neon.rs | 5 ++++- proxy/src/context.rs | 4 ++++ proxy/src/context/parquet.rs | 14 ++++++++------ proxy/src/serverless/backend.rs | 2 ++ 4 files changed, 18 insertions(+), 7 deletions(-) diff --git a/proxy/src/console/provider/neon.rs b/proxy/src/console/provider/neon.rs index f3befa33e018..3088cffa5793 100644 --- a/proxy/src/console/provider/neon.rs +++ b/proxy/src/console/provider/neon.rs @@ -6,7 +6,9 @@ use super::{ ApiCaches, ApiLocks, AuthInfo, AuthSecret, CachedAllowedIps, CachedNodeInfo, CachedRoleSecret, NodeInfo, }; -use crate::{auth::backend::ComputeUserInfo, compute, http, scram}; +use crate::{ + auth::backend::ComputeUserInfo, compute, console::messages::ColdStartInfo, http, scram, +}; use crate::{ cache::Cached, context::RequestMonitoring, @@ -254,6 +256,7 @@ impl super::Api for Api { if permit.should_check_cache() { if let Some(cached) = self.caches.node_info.get(&key) { info!(key = &*key, "found cached compute node info"); + ctx.set_cold_start_info(ColdStartInfo::Warm); return Ok(cached); } } diff --git a/proxy/src/context.rs b/proxy/src/context.rs index 1b48e01358c3..40aa21083ff3 100644 --- a/proxy/src/context.rs +++ b/proxy/src/context.rs @@ -111,6 +111,10 @@ impl RequestMonitoring { ) } + pub fn set_cold_start_info(&mut self, info: ColdStartInfo) { + self.cold_start_info = Some(info); + } + pub fn set_project(&mut self, x: MetricsAuxInfo) { self.set_endpoint_id(x.endpoint_id); self.branch = Some(x.branch_id); diff --git a/proxy/src/context/parquet.rs b/proxy/src/context/parquet.rs index 1b1274b1969e..ba144bb7ba57 100644 --- a/proxy/src/context/parquet.rs +++ b/proxy/src/context/parquet.rs @@ -93,7 +93,7 @@ struct RequestData { /// Or if we make it to proxy_pass success: bool, /// Indicates if the cplane started the new compute node for this request. - cold_start_info: Option, + cold_start_info: Option<&'static str>, /// Tracks time from session start (HTTP request/libpq TCP handshake) /// Through to success/failure duration_us: u64, @@ -121,10 +121,12 @@ impl From for RequestData { region: value.region, error: value.error_kind.as_ref().map(|e| e.to_metric_label()), success: value.success, - cold_start_info: value - .cold_start_info - .as_ref() - .map(|x| serde_json::to_string(x).unwrap_or_default()), + cold_start_info: value.cold_start_info.as_ref().map(|x| match x { + crate::console::messages::ColdStartInfo::Unknown => "unknown", + crate::console::messages::ColdStartInfo::Warm => "warm", + crate::console::messages::ColdStartInfo::PoolHit => "pool_hit", + crate::console::messages::ColdStartInfo::PoolMiss => "pool_miss", + }), duration_us: SystemTime::from(value.first_packet) .elapsed() .unwrap_or_default() @@ -458,7 +460,7 @@ mod tests { region: "us-east-1", error: None, success: rng.gen(), - cold_start_info: Some("no".into()), + cold_start_info: Some("no"), duration_us: rng.gen_range(0..30_000_000), } } diff --git a/proxy/src/serverless/backend.rs b/proxy/src/serverless/backend.rs index d0f155165d4a..9b3ca8d447e8 100644 --- a/proxy/src/serverless/backend.rs +++ b/proxy/src/serverless/backend.rs @@ -9,6 +9,7 @@ use crate::{ config::ProxyConfig, console::{ errors::{GetAuthInfoError, WakeComputeError}, + messages::ColdStartInfo, CachedNodeInfo, }, context::RequestMonitoring, @@ -83,6 +84,7 @@ impl PoolingBackend { }; if let Some(client) = maybe_client { + ctx.set_cold_start_info(ColdStartInfo::Warm); return Ok(client); } let conn_id = uuid::Uuid::new_v4(); From b0aff04157866904e53f815e7fd389e2823abce9 Mon Sep 17 00:00:00 2001 From: Anna Khanova <32508607+khanova@users.noreply.github.com> Date: Wed, 13 Mar 2024 16:50:05 +0400 Subject: [PATCH 38/43] proxy: add new dimension to exclude cplane latency (#7011) ## Problem Currently cplane communication is a part of the latency monitoring. It doesn't allow to setup the proper alerting based on proxy latency. ## Summary of changes Added dimension to exclude cplane latency. --- proxy/src/auth/backend/hacks.rs | 13 +++-- proxy/src/auth/flow.rs | 2 +- proxy/src/console/provider/neon.rs | 4 ++ proxy/src/context.rs | 9 ++-- proxy/src/context/parquet.rs | 6 +-- proxy/src/metrics.rs | 79 +++++++++++++++++++++--------- proxy/src/proxy.rs | 2 +- 7 files changed, 79 insertions(+), 36 deletions(-) diff --git a/proxy/src/auth/backend/hacks.rs b/proxy/src/auth/backend/hacks.rs index 26cf7a01f275..f7241be4a9ca 100644 --- a/proxy/src/auth/backend/hacks.rs +++ b/proxy/src/auth/backend/hacks.rs @@ -25,13 +25,16 @@ pub async fn authenticate_cleartext( ctx.set_auth_method(crate::context::AuthMethod::Cleartext); // pause the timer while we communicate with the client - let _paused = ctx.latency_timer.pause(); + let paused = ctx.latency_timer.pause(crate::metrics::Waiting::Client); - let auth_outcome = AuthFlow::new(client) + let auth_flow = AuthFlow::new(client) .begin(auth::CleartextPassword(secret)) - .await? - .authenticate() .await?; + drop(paused); + // cleartext auth is only allowed to the ws/http protocol. + // If we're here, we already received the password in the first message. + // Scram protocol will be executed on the proxy side. + let auth_outcome = auth_flow.authenticate().await?; let keys = match auth_outcome { sasl::Outcome::Success(key) => key, @@ -56,7 +59,7 @@ pub async fn password_hack_no_authentication( ctx.set_auth_method(crate::context::AuthMethod::Cleartext); // pause the timer while we communicate with the client - let _paused = ctx.latency_timer.pause(); + let _paused = ctx.latency_timer.pause(crate::metrics::Waiting::Client); let payload = AuthFlow::new(client) .begin(auth::PasswordHack) diff --git a/proxy/src/auth/flow.rs b/proxy/src/auth/flow.rs index dce73138c619..788381b6c08e 100644 --- a/proxy/src/auth/flow.rs +++ b/proxy/src/auth/flow.rs @@ -143,7 +143,7 @@ impl AuthFlow<'_, S, Scram<'_>> { let Scram(secret, ctx) = self.state; // pause the timer while we communicate with the client - let _paused = ctx.latency_timer.pause(); + let _paused = ctx.latency_timer.pause(crate::metrics::Waiting::Client); // Initial client message contains the chosen auth method's name. let msg = self.stream.read_password_message().await?; diff --git a/proxy/src/console/provider/neon.rs b/proxy/src/console/provider/neon.rs index 3088cffa5793..3b2e0cc2042b 100644 --- a/proxy/src/console/provider/neon.rs +++ b/proxy/src/console/provider/neon.rs @@ -74,7 +74,9 @@ impl Api { info!(url = request.url().as_str(), "sending http request"); let start = Instant::now(); + let pause = ctx.latency_timer.pause(crate::metrics::Waiting::Cplane); let response = self.endpoint.execute(request).await?; + drop(pause); info!(duration = ?start.elapsed(), "received http response"); let body = match parse_body::(response).await { Ok(body) => body, @@ -134,7 +136,9 @@ impl Api { info!(url = request.url().as_str(), "sending http request"); let start = Instant::now(); + let pause = ctx.latency_timer.pause(crate::metrics::Waiting::Cplane); let response = self.endpoint.execute(request).await?; + drop(pause); info!(duration = ?start.elapsed(), "received http response"); let body = parse_body::(response).await?; diff --git a/proxy/src/context.rs b/proxy/src/context.rs index 40aa21083ff3..7ca830cdb4a1 100644 --- a/proxy/src/context.rs +++ b/proxy/src/context.rs @@ -15,11 +15,12 @@ use crate::{ BranchId, DbName, EndpointId, ProjectId, RoleName, }; +use self::parquet::RequestData; + pub mod parquet; -static LOG_CHAN: OnceCell> = OnceCell::new(); +static LOG_CHAN: OnceCell> = OnceCell::new(); -#[derive(Clone)] /// Context data for a single request to connect to a database. /// /// This data should **not** be used for connection logic, only for observability and limiting purposes. @@ -46,7 +47,7 @@ pub struct RequestMonitoring { // extra // This sender is here to keep the request monitoring channel open while requests are taking place. - sender: Option>, + sender: Option>, pub latency_timer: LatencyTimer, } @@ -172,7 +173,7 @@ impl RequestMonitoring { impl Drop for RequestMonitoring { fn drop(&mut self) { if let Some(tx) = self.sender.take() { - let _: Result<(), _> = tx.send(self.clone()); + let _: Result<(), _> = tx.send(RequestData::from(&*self)); } } } diff --git a/proxy/src/context/parquet.rs b/proxy/src/context/parquet.rs index ba144bb7ba57..a2be1c41861f 100644 --- a/proxy/src/context/parquet.rs +++ b/proxy/src/context/parquet.rs @@ -74,7 +74,7 @@ pub(crate) const FAILED_UPLOAD_MAX_RETRIES: u32 = 10; // * after each rowgroup write, we check the length of the file and upload to s3 if large enough #[derive(parquet_derive::ParquetRecordWriter)] -struct RequestData { +pub struct RequestData { region: &'static str, protocol: &'static str, /// Must be UTC. The derive macro doesn't like the timezones @@ -99,8 +99,8 @@ struct RequestData { duration_us: u64, } -impl From for RequestData { - fn from(value: RequestMonitoring) -> Self { +impl From<&RequestMonitoring> for RequestData { + fn from(value: &RequestMonitoring) -> Self { Self { session_id: value.session_id, peer_addr: value.peer_addr.to_string(), diff --git a/proxy/src/metrics.rs b/proxy/src/metrics.rs index 0477176c4560..02ebcd6aaaf3 100644 --- a/proxy/src/metrics.rs +++ b/proxy/src/metrics.rs @@ -7,7 +7,7 @@ use ::metrics::{ use metrics::{register_int_counter, register_int_counter_pair, IntCounter, IntCounterPair}; use once_cell::sync::Lazy; -use tokio::time; +use tokio::time::{self, Instant}; pub static NUM_DB_CONNECTIONS_GAUGE: Lazy = Lazy::new(|| { register_int_counter_pair_vec!( @@ -46,9 +46,9 @@ pub static COMPUTE_CONNECTION_LATENCY: Lazy = Lazy::new(|| { register_histogram_vec!( "proxy_compute_connection_latency_seconds", "Time it took for proxy to establish a connection to the compute endpoint", - // http/ws/tcp, true/false, true/false, success/failure - // 3 * 2 * 2 * 2 = 24 counters - &["protocol", "cache_miss", "pool_miss", "outcome"], + // http/ws/tcp, true/false, true/false, success/failure, client/client_and_cplane + // 3 * 2 * 2 * 2 * 2 = 48 counters + &["protocol", "cache_miss", "pool_miss", "outcome", "excluded"], // largest bucket = 2^16 * 0.5ms = 32s exponential_buckets(0.0005, 2.0, 16).unwrap(), ) @@ -161,12 +161,26 @@ pub static NUM_CANCELLATION_REQUESTS: Lazy = Lazy::new(|| { .unwrap() }); -#[derive(Clone)] +pub enum Waiting { + Cplane, + Client, + Compute, +} + +#[derive(Default)] +struct Accumulated { + cplane: time::Duration, + client: time::Duration, + compute: time::Duration, +} + pub struct LatencyTimer { // time since the stopwatch was started - start: Option, + start: time::Instant, + // time since the stopwatch was stopped + stop: Option, // accumulated time on the stopwatch - pub accumulated: std::time::Duration, + accumulated: Accumulated, // label data protocol: &'static str, cache_miss: bool, @@ -176,13 +190,16 @@ pub struct LatencyTimer { pub struct LatencyTimerPause<'a> { timer: &'a mut LatencyTimer, + start: time::Instant, + waiting_for: Waiting, } impl LatencyTimer { pub fn new(protocol: &'static str) -> Self { Self { - start: Some(time::Instant::now()), - accumulated: std::time::Duration::ZERO, + start: time::Instant::now(), + stop: None, + accumulated: Accumulated::default(), protocol, cache_miss: false, // by default we don't do pooling @@ -192,11 +209,12 @@ impl LatencyTimer { } } - pub fn pause(&mut self) -> LatencyTimerPause<'_> { - // stop the stopwatch and record the time that we have accumulated - let start = self.start.take().expect("latency timer should be started"); - self.accumulated += start.elapsed(); - LatencyTimerPause { timer: self } + pub fn pause(&mut self, waiting_for: Waiting) -> LatencyTimerPause<'_> { + LatencyTimerPause { + timer: self, + start: Instant::now(), + waiting_for, + } } pub fn cache_miss(&mut self) { @@ -209,9 +227,7 @@ impl LatencyTimer { pub fn success(&mut self) { // stop the stopwatch and record the time that we have accumulated - if let Some(start) = self.start.take() { - self.accumulated += start.elapsed(); - } + self.stop = Some(time::Instant::now()); // success self.outcome = "success"; @@ -220,23 +236,42 @@ impl LatencyTimer { impl Drop for LatencyTimerPause<'_> { fn drop(&mut self) { - // start the stopwatch again - self.timer.start = Some(time::Instant::now()); + let dur = self.start.elapsed(); + match self.waiting_for { + Waiting::Cplane => self.timer.accumulated.cplane += dur, + Waiting::Client => self.timer.accumulated.client += dur, + Waiting::Compute => self.timer.accumulated.compute += dur, + } } } impl Drop for LatencyTimer { fn drop(&mut self) { - let duration = - self.start.map(|start| start.elapsed()).unwrap_or_default() + self.accumulated; + let duration = self + .stop + .unwrap_or_else(time::Instant::now) + .duration_since(self.start); + // Excluding cplane communication from the accumulated time. + COMPUTE_CONNECTION_LATENCY + .with_label_values(&[ + self.protocol, + bool_to_str(self.cache_miss), + bool_to_str(self.pool_miss), + self.outcome, + "client", + ]) + .observe((duration.saturating_sub(self.accumulated.client)).as_secs_f64()); + // Exclude client and cplane communication from the accumulated time. + let accumulated_total = self.accumulated.client + self.accumulated.cplane; COMPUTE_CONNECTION_LATENCY .with_label_values(&[ self.protocol, bool_to_str(self.cache_miss), bool_to_str(self.pool_miss), self.outcome, + "client_and_cplane", ]) - .observe(duration.as_secs_f64()) + .observe((duration.saturating_sub(accumulated_total)).as_secs_f64()); } } diff --git a/proxy/src/proxy.rs b/proxy/src/proxy.rs index 7848fc2ac2cc..ab5bf5d49498 100644 --- a/proxy/src/proxy.rs +++ b/proxy/src/proxy.rs @@ -248,7 +248,7 @@ pub async fn handle_client( let tls = config.tls_config.as_ref(); - let pause = ctx.latency_timer.pause(); + let pause = ctx.latency_timer.pause(crate::metrics::Waiting::Client); let do_handshake = handshake(stream, mode.handshake_tls(tls)); let (mut stream, params) = match tokio::time::timeout(config.handshake_timeout, do_handshake).await?? { From 8a53d576e685b700c498d50588b4c0224711bed2 Mon Sep 17 00:00:00 2001 From: Joonas Koivunen Date: Wed, 13 Mar 2024 17:10:20 +0200 Subject: [PATCH 39/43] fix(metrics): time individual layer flush operations (#7109) Currently, the flushing operation could flush multiple frozen layers to the disk and store the aggregate time in the histogram. The result is a bimodal distribution with short and over 1000-second flushes. Change it so that we record how long one layer flush takes. --- pageserver/src/tenant/timeline.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index a733a3b1a7f5..f10df19b7bdc 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -2967,7 +2967,6 @@ impl Timeline { } trace!("waking up"); - let timer = self.metrics.flush_time_histo.start_timer(); let flush_counter = *layer_flush_start_rx.borrow(); let result = loop { if self.cancel.is_cancelled() { @@ -2978,6 +2977,8 @@ impl Timeline { return; } + let timer = self.metrics.flush_time_histo.start_timer(); + let layer_to_flush = { let guard = self.layers.read().await; guard.layer_map().frozen_layers.front().cloned() @@ -2999,13 +3000,12 @@ impl Timeline { break err; } } + timer.stop_and_record(); }; // Notify any listeners that we're done let _ = self .layer_flush_done_tx .send_replace((flush_counter, result)); - - timer.stop_and_record(); } } @@ -3073,6 +3073,7 @@ impl Timeline { ctx: &RequestContext, ) -> Result<(), FlushLayerError> { debug_assert_current_span_has_tenant_and_timeline_id(); + // As a special case, when we have just imported an image into the repository, // instead of writing out a L0 delta layer, we directly write out image layer // files instead. This is possible as long as *all* the data imported into the From 5309711691325c274fc34994c240feaa529cbef5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Arpad=20M=C3=BCller?= Date: Wed, 13 Mar 2024 17:30:29 +0100 Subject: [PATCH 40/43] Make tenant_id in TenantLocationConfigRequest optional (#7055) The `tenant_id` in `TenantLocationConfigRequest` in the `location_config` endpoint was only used in the storage controller/attachment service, and there it was only used for assertions and the creation part. --- control_plane/attachment_service/src/http.rs | 6 +++--- control_plane/attachment_service/src/service.rs | 13 +++++++------ libs/pageserver_api/src/models.rs | 2 +- pageserver/client/src/mgmt_api.rs | 2 +- pageserver/src/http/openapi_spec.yml | 7 ++++--- 5 files changed, 16 insertions(+), 14 deletions(-) diff --git a/control_plane/attachment_service/src/http.rs b/control_plane/attachment_service/src/http.rs index 27ba5bdb6534..515c287ea99d 100644 --- a/control_plane/attachment_service/src/http.rs +++ b/control_plane/attachment_service/src/http.rs @@ -174,14 +174,14 @@ async fn handle_tenant_location_config( service: Arc, mut req: Request, ) -> Result, ApiError> { - let tenant_id: TenantId = parse_request_param(&req, "tenant_id")?; + let tenant_shard_id: TenantShardId = parse_request_param(&req, "tenant_shard_id")?; check_permissions(&req, Scope::PageServerApi)?; let config_req = json_request::(&mut req).await?; json_response( StatusCode::OK, service - .tenant_location_config(tenant_id, config_req) + .tenant_location_config(tenant_shard_id, config_req) .await?, ) } @@ -587,7 +587,7 @@ pub fn make_router( .get("/v1/tenant/:tenant_id/config", |r| { tenant_service_handler(r, handle_tenant_config_get) }) - .put("/v1/tenant/:tenant_id/location_config", |r| { + .put("/v1/tenant/:tenant_shard_id/location_config", |r| { tenant_service_handler(r, handle_tenant_location_config) }) .put("/v1/tenant/:tenant_id/time_travel_remote_storage", |r| { diff --git a/control_plane/attachment_service/src/service.rs b/control_plane/attachment_service/src/service.rs index ea301d037206..1c4ede3d9d75 100644 --- a/control_plane/attachment_service/src/service.rs +++ b/control_plane/attachment_service/src/service.rs @@ -1262,6 +1262,7 @@ impl Service { let mut updates = Vec::new(); let mut locked = self.inner.write().unwrap(); let (nodes, tenants, _scheduler) = locked.parts_mut(); + let tenant_shard_id = TenantShardId::unsharded(tenant_id); // Use location config mode as an indicator of policy. let placement_policy = match req.config.mode { @@ -1326,12 +1327,10 @@ impl Service { TenantCreateOrUpdate::Create( // Synthesize a creation request TenantCreateRequest { - new_tenant_id: TenantShardId::unsharded(tenant_id), + new_tenant_id: tenant_shard_id, generation, shard_parameters: ShardParameters { - // Must preserve the incoming shard_count do distinguish unsharded (0) - // from single-sharded (1): this distinction appears in the S3 keys of the tenant. - count: req.tenant_id.shard_count, + count: tenant_shard_id.shard_count, // We only import un-sharded or single-sharded tenants, so stripe // size can be made up arbitrarily here. stripe_size: ShardParameters::DEFAULT_STRIPE_SIZE, @@ -1360,15 +1359,17 @@ impl Service { /// - Call with mode Detached to switch to PolicyMode::Detached pub(crate) async fn tenant_location_config( &self, - tenant_id: TenantId, + tenant_shard_id: TenantShardId, req: TenantLocationConfigRequest, ) -> Result { - if !req.tenant_id.is_unsharded() { + if !tenant_shard_id.is_unsharded() { return Err(ApiError::BadRequest(anyhow::anyhow!( "This API is for importing single-sharded or unsharded tenants" ))); } + let tenant_id = tenant_shard_id.tenant_id; + // First check if this is a creation or an update let create_or_update = self.tenant_location_config_prepare(tenant_id, req); diff --git a/libs/pageserver_api/src/models.rs b/libs/pageserver_api/src/models.rs index a96cc0915894..3aa84f89033b 100644 --- a/libs/pageserver_api/src/models.rs +++ b/libs/pageserver_api/src/models.rs @@ -426,7 +426,7 @@ pub struct StatusResponse { #[derive(Serialize, Deserialize, Debug)] #[serde(deny_unknown_fields)] pub struct TenantLocationConfigRequest { - pub tenant_id: TenantShardId, + pub tenant_id: Option, #[serde(flatten)] pub config: LocationConfig, // as we have a flattened field, we should reject all unknown fields in it } diff --git a/pageserver/client/src/mgmt_api.rs b/pageserver/client/src/mgmt_api.rs index 732eb951c9fd..2f22ebd54d4b 100644 --- a/pageserver/client/src/mgmt_api.rs +++ b/pageserver/client/src/mgmt_api.rs @@ -257,7 +257,7 @@ impl Client { lazy: bool, ) -> Result<()> { let req_body = TenantLocationConfigRequest { - tenant_id: tenant_shard_id, + tenant_id: Some(tenant_shard_id), config, }; diff --git a/pageserver/src/http/openapi_spec.yml b/pageserver/src/http/openapi_spec.yml index 6a070e213594..4823710fb5a0 100644 --- a/pageserver/src/http/openapi_spec.yml +++ b/pageserver/src/http/openapi_spec.yml @@ -567,9 +567,9 @@ paths: application/json: schema: $ref: "#/components/schemas/ServiceUnavailableError" - /v1/tenant/{tenant_id}/location_config: + /v1/tenant/{tenant_shard_id}/location_config: parameters: - - name: tenant_id + - name: tenant_shard_id in: path required: true schema: @@ -1367,10 +1367,11 @@ components: TenantLocationConfigRequest: type: object required: - - tenant_id + - mode properties: tenant_id: type: string + description: Not used, scheduled for removal. mode: type: string enum: ["AttachedSingle", "AttachedMulti", "AttachedStale", "Secondary", "Detached"] From 69338e53e3628cd0133b27d1c079f9deeaaea725 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Wed, 13 Mar 2024 18:49:17 +0100 Subject: [PATCH 41/43] throttling: fixup interactions with Timeline::get_vectored (#7089) ## Problem Before this PR, `Timeline::get_vectored` would be throttled twice if the sequential option was enabled or if validation was enabled. Also, `pageserver_get_vectored_seconds` included the time spent in the throttle, which turns out to be undesirable for what we use that metric for. ## Summary of changes Double-throttle: * Add `Timeline::get0` method which is unthrottled. * Use that method from within the `Timeline::get_vectored` code path. Metric: * return throttled time from `throttle()` method * deduct the value from the observed time * globally rate-limited logging of duration subtraction errors, like in all other places that do the throttled-time deduction from observations --- pageserver/src/bin/pageserver.rs | 2 ++ pageserver/src/metrics.rs | 2 +- pageserver/src/tenant/throttle.rs | 7 ++-- pageserver/src/tenant/timeline.rs | 59 +++++++++++++++++++++++++------ 4 files changed, 57 insertions(+), 13 deletions(-) diff --git a/pageserver/src/bin/pageserver.rs b/pageserver/src/bin/pageserver.rs index 2f172bd38490..59750897ff7e 100644 --- a/pageserver/src/bin/pageserver.rs +++ b/pageserver/src/bin/pageserver.rs @@ -1,3 +1,5 @@ +#![recursion_limit = "300"] + //! Main entry point for the Page Server executable. use std::env::{var, VarError}; diff --git a/pageserver/src/metrics.rs b/pageserver/src/metrics.rs index 814b3e1f9611..03537ddb05e9 100644 --- a/pageserver/src/metrics.rs +++ b/pageserver/src/metrics.rs @@ -167,7 +167,7 @@ impl GetVectoredLatency { pub(crate) static GET_VECTORED_LATENCY: Lazy = Lazy::new(|| { let inner = register_histogram_vec!( "pageserver_get_vectored_seconds", - "Time spent in get_vectored", + "Time spent in get_vectored, excluding time spent in timeline_get_throttle.", &["task_kind"], CRITICAL_OP_BUCKETS.into(), ) diff --git a/pageserver/src/tenant/throttle.rs b/pageserver/src/tenant/throttle.rs index 280773e9c305..f3f3d5e3aee5 100644 --- a/pageserver/src/tenant/throttle.rs +++ b/pageserver/src/tenant/throttle.rs @@ -130,10 +130,10 @@ where self.inner.load().config.steady_rps() } - pub async fn throttle(&self, ctx: &RequestContext, key_count: usize) { + pub async fn throttle(&self, ctx: &RequestContext, key_count: usize) -> Option { let inner = self.inner.load_full(); // clones the `Inner` Arc if !inner.task_kinds.contains(ctx.task_kind()) { - return; + return None; }; let start = std::time::Instant::now(); let mut did_throttle = false; @@ -170,6 +170,9 @@ where }); } } + Some(wait_time) + } else { + None } } } diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index f10df19b7bdc..d507a19de902 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -634,6 +634,8 @@ impl Timeline { /// If a remote layer file is needed, it is downloaded as part of this /// call. /// + /// This method enforces [`Self::timeline_get_throttle`] internally. + /// /// NOTE: It is considered an error to 'get' a key that doesn't exist. The /// abstraction above this needs to store suitable metadata to track what /// data exists with what keys, in separate metadata entries. If a @@ -644,18 +646,27 @@ impl Timeline { /// # Cancel-Safety /// /// This method is cancellation-safe. + #[inline(always)] pub(crate) async fn get( &self, key: Key, lsn: Lsn, ctx: &RequestContext, + ) -> Result { + self.timeline_get_throttle.throttle(ctx, 1).await; + self.get_impl(key, lsn, ctx).await + } + /// Not subject to [`Self::timeline_get_throttle`]. + async fn get_impl( + &self, + key: Key, + lsn: Lsn, + ctx: &RequestContext, ) -> Result { if !lsn.is_valid() { return Err(PageReconstructError::Other(anyhow::anyhow!("Invalid LSN"))); } - self.timeline_get_throttle.throttle(ctx, 1).await; - // This check is debug-only because of the cost of hashing, and because it's a double-check: we // already checked the key against the shard_identity when looking up the Timeline from // page_service. @@ -752,10 +763,6 @@ impl Timeline { return Err(GetVectoredError::Oversized(key_count)); } - self.timeline_get_throttle - .throttle(ctx, key_count as usize) - .await; - for range in &keyspace.ranges { let mut key = range.start; while key != range.end { @@ -772,11 +779,18 @@ impl Timeline { self.conf.get_vectored_impl ); - let _timer = crate::metrics::GET_VECTORED_LATENCY + let start = crate::metrics::GET_VECTORED_LATENCY .for_task_kind(ctx.task_kind()) - .map(|t| t.start_timer()); + .map(|metric| (metric, Instant::now())); - match self.conf.get_vectored_impl { + // start counting after throttle so that throttle time + // is always less than observation time + let throttled = self + .timeline_get_throttle + .throttle(ctx, key_count as usize) + .await; + + let res = match self.conf.get_vectored_impl { GetVectoredImpl::Sequential => { self.get_vectored_sequential_impl(keyspace, lsn, ctx).await } @@ -790,9 +804,33 @@ impl Timeline { vectored_res } + }; + + if let Some((metric, start)) = start { + let elapsed = start.elapsed(); + let ex_throttled = if let Some(throttled) = throttled { + elapsed.checked_sub(throttled) + } else { + Some(elapsed) + }; + + if let Some(ex_throttled) = ex_throttled { + metric.observe(ex_throttled.as_secs_f64()); + } else { + use utils::rate_limit::RateLimit; + static LOGGED: Lazy> = + Lazy::new(|| Mutex::new(RateLimit::new(Duration::from_secs(10)))); + let mut rate_limit = LOGGED.lock().unwrap(); + rate_limit.call(|| { + warn!("error deducting time spent throttled; this message is logged at a global rate limit"); + }); + } } + + res } + /// Not subject to [`Self::timeline_get_throttle`]. pub(super) async fn get_vectored_sequential_impl( &self, keyspace: KeySpace, @@ -803,7 +841,7 @@ impl Timeline { for range in keyspace.ranges { let mut key = range.start; while key != range.end { - let block = self.get(key, lsn, ctx).await; + let block = self.get_impl(key, lsn, ctx).await; use PageReconstructError::*; match block { @@ -853,6 +891,7 @@ impl Timeline { Ok(results) } + /// Not subject to [`Self::timeline_get_throttle`]. pub(super) async fn validate_get_vectored_impl( &self, vectored_res: &Result>, GetVectoredError>, From 3bd6551b36be636c7497ee774c65718320093bc3 Mon Sep 17 00:00:00 2001 From: Conrad Ludgate Date: Thu, 14 Mar 2024 08:20:56 +0000 Subject: [PATCH 42/43] proxy http cancellation safety (#7117) ## Problem hyper auto-cancels the request futures on connection close. `sql_over_http::handle` is not 'drop cancel safe', so we need to do some other work to make sure connections are queries in the right way. ## Summary of changes 1. tokio::spawn the request handler to resolve the initial cancel-safety issue 2. share a cancellation token, and cancel it when the request `Service` is dropped. 3. Add a new log span to be able to track the HTTP connection lifecycle. --- proxy/src/protocol2.rs | 18 ++++++- proxy/src/serverless.rs | 76 +++++++++++++++++++-------- proxy/src/serverless/sql_over_http.rs | 2 +- proxy/src/serverless/tls_listener.rs | 29 ++++------ test_runner/fixtures/neon_fixtures.py | 2 + test_runner/regress/test_proxy.py | 36 +++++++++++++ 6 files changed, 121 insertions(+), 42 deletions(-) diff --git a/proxy/src/protocol2.rs b/proxy/src/protocol2.rs index f476cb9b37d8..700c8c868118 100644 --- a/proxy/src/protocol2.rs +++ b/proxy/src/protocol2.rs @@ -341,7 +341,14 @@ impl Accept for ProxyProtocolAccept { cx: &mut Context<'_>, ) -> Poll>> { let conn = ready!(Pin::new(&mut self.incoming).poll_accept(cx)?); - tracing::info!(protocol = self.protocol, "accepted new TCP connection"); + + let conn_id = uuid::Uuid::new_v4(); + let span = tracing::info_span!("http_conn", ?conn_id); + { + let _enter = span.enter(); + tracing::info!("accepted new TCP connection"); + } + let Some(conn) = conn else { return Poll::Ready(None); }; @@ -354,6 +361,7 @@ impl Accept for ProxyProtocolAccept { .with_label_values(&[self.protocol]) .guard(), )), + span, }))) } } @@ -364,6 +372,14 @@ pin_project! { pub inner: T, pub connection_id: Uuid, pub gauge: Mutex>, + pub span: tracing::Span, + } + + impl PinnedDrop for WithConnectionGuard { + fn drop(this: Pin<&mut Self>) { + let _enter = this.span.enter(); + tracing::info!("HTTP connection closed") + } } } diff --git a/proxy/src/serverless.rs b/proxy/src/serverless.rs index 68f68eaba11f..be9f90acde7b 100644 --- a/proxy/src/serverless.rs +++ b/proxy/src/serverless.rs @@ -19,6 +19,7 @@ use rand::SeedableRng; pub use reqwest_middleware::{ClientWithMiddleware, Error}; pub use reqwest_retry::{policies::ExponentialBackoff, RetryTransientMiddleware}; use tokio_util::task::TaskTracker; +use tracing::instrument::Instrumented; use crate::context::RequestMonitoring; use crate::protocol2::{ProxyProtocolAccept, WithClientIp, WithConnectionGuard}; @@ -30,13 +31,12 @@ use hyper::{ Body, Method, Request, Response, }; -use std::convert::Infallible; use std::net::IpAddr; use std::sync::Arc; use std::task::Poll; use tls_listener::TlsListener; use tokio::net::TcpListener; -use tokio_util::sync::CancellationToken; +use tokio_util::sync::{CancellationToken, DropGuard}; use tracing::{error, info, warn, Instrument}; use utils::http::{error::ApiError, json::json_response}; @@ -100,12 +100,7 @@ pub async fn task_main( let ws_connections = tokio_util::task::task_tracker::TaskTracker::new(); ws_connections.close(); // allows `ws_connections.wait to complete` - let tls_listener = TlsListener::new( - tls_acceptor, - addr_incoming, - "http", - config.handshake_timeout, - ); + let tls_listener = TlsListener::new(tls_acceptor, addr_incoming, config.handshake_timeout); let make_svc = hyper::service::make_service_fn( |stream: &tokio_rustls::server::TlsStream< @@ -121,6 +116,11 @@ pub async fn task_main( .take() .expect("gauge should be set on connection start"); + // Cancel all current inflight HTTP requests if the HTTP connection is closed. + let http_cancellation_token = CancellationToken::new(); + let cancel_connection = http_cancellation_token.clone().drop_guard(); + + let span = conn.span.clone(); let client_addr = conn.inner.client_addr(); let remote_addr = conn.inner.inner.remote_addr(); let backend = backend.clone(); @@ -136,27 +136,43 @@ pub async fn task_main( Ok(MetricService::new( hyper::service::service_fn(move |req: Request| { let backend = backend.clone(); - let ws_connections = ws_connections.clone(); + let ws_connections2 = ws_connections.clone(); let endpoint_rate_limiter = endpoint_rate_limiter.clone(); let cancellation_handler = cancellation_handler.clone(); - - async move { - Ok::<_, Infallible>( - request_handler( + let http_cancellation_token = http_cancellation_token.child_token(); + + // `request_handler` is not cancel safe. It expects to be cancelled only at specific times. + // By spawning the future, we ensure it never gets cancelled until it decides to. + ws_connections.spawn( + async move { + // Cancel the current inflight HTTP request if the requets stream is closed. + // This is slightly different to `_cancel_connection` in that + // h2 can cancel individual requests with a `RST_STREAM`. + let _cancel_session = http_cancellation_token.clone().drop_guard(); + + let res = request_handler( req, config, backend, - ws_connections, + ws_connections2, cancellation_handler, peer_addr.ip(), endpoint_rate_limiter, + http_cancellation_token, ) .await - .map_or_else(|e| e.into_response(), |r| r), - ) - } + .map_or_else(|e| e.into_response(), |r| r); + + _cancel_session.disarm(); + + res + } + .in_current_span(), + ) }), gauge, + cancel_connection, + span, )) } }, @@ -176,11 +192,23 @@ pub async fn task_main( struct MetricService { inner: S, _gauge: IntCounterPairGuard, + _cancel: DropGuard, + span: tracing::Span, } impl MetricService { - fn new(inner: S, _gauge: IntCounterPairGuard) -> MetricService { - MetricService { inner, _gauge } + fn new( + inner: S, + _gauge: IntCounterPairGuard, + _cancel: DropGuard, + span: tracing::Span, + ) -> MetricService { + MetricService { + inner, + _gauge, + _cancel, + span, + } } } @@ -190,14 +218,16 @@ where { type Response = S::Response; type Error = S::Error; - type Future = S::Future; + type Future = Instrumented; fn poll_ready(&mut self, cx: &mut std::task::Context<'_>) -> Poll> { self.inner.poll_ready(cx) } fn call(&mut self, req: Request) -> Self::Future { - self.inner.call(req) + self.span + .in_scope(|| self.inner.call(req)) + .instrument(self.span.clone()) } } @@ -210,6 +240,8 @@ async fn request_handler( cancellation_handler: Arc, peer_addr: IpAddr, endpoint_rate_limiter: Arc, + // used to cancel in-flight HTTP requests. not used to cancel websockets + http_cancellation_token: CancellationToken, ) -> Result, ApiError> { let session_id = uuid::Uuid::new_v4(); @@ -253,7 +285,7 @@ async fn request_handler( let ctx = RequestMonitoring::new(session_id, peer_addr, "http", &config.region); let span = ctx.span.clone(); - sql_over_http::handle(config, ctx, request, backend) + sql_over_http::handle(config, ctx, request, backend, http_cancellation_token) .instrument(span) .await } else if request.uri().path() == "/sql" && request.method() == Method::OPTIONS { diff --git a/proxy/src/serverless/sql_over_http.rs b/proxy/src/serverless/sql_over_http.rs index 86c278030f9e..f675375ff1d6 100644 --- a/proxy/src/serverless/sql_over_http.rs +++ b/proxy/src/serverless/sql_over_http.rs @@ -217,8 +217,8 @@ pub async fn handle( mut ctx: RequestMonitoring, request: Request, backend: Arc, + cancel: CancellationToken, ) -> Result, ApiError> { - let cancel = CancellationToken::new(); let cancel2 = cancel.clone(); let handle = tokio::spawn(async move { time::sleep(config.http_config.request_timeout).await; diff --git a/proxy/src/serverless/tls_listener.rs b/proxy/src/serverless/tls_listener.rs index cce02e385054..33f194dd5932 100644 --- a/proxy/src/serverless/tls_listener.rs +++ b/proxy/src/serverless/tls_listener.rs @@ -13,7 +13,7 @@ use tokio::{ time::timeout, }; use tokio_rustls::{server::TlsStream, TlsAcceptor}; -use tracing::{info, warn}; +use tracing::{info, warn, Instrument}; use crate::{ metrics::TLS_HANDSHAKE_FAILURES, @@ -29,24 +29,17 @@ pin_project! { tls: TlsAcceptor, waiting: JoinSet>>, timeout: Duration, - protocol: &'static str, } } impl TlsListener { /// Create a `TlsListener` with default options. - pub(crate) fn new( - tls: TlsAcceptor, - listener: A, - protocol: &'static str, - timeout: Duration, - ) -> Self { + pub(crate) fn new(tls: TlsAcceptor, listener: A, timeout: Duration) -> Self { TlsListener { listener, tls, waiting: JoinSet::new(), timeout, - protocol, } } } @@ -73,7 +66,7 @@ where Poll::Ready(Some(Ok(mut conn))) => { let t = *this.timeout; let tls = this.tls.clone(); - let protocol = *this.protocol; + let span = conn.span.clone(); this.waiting.spawn(async move { let peer_addr = match conn.inner.wait_for_addr().await { Ok(Some(addr)) => addr, @@ -86,21 +79,24 @@ where let accept = tls.accept(conn); match timeout(t, accept).await { - Ok(Ok(conn)) => Some(conn), + Ok(Ok(conn)) => { + info!(%peer_addr, "accepted new TLS connection"); + Some(conn) + }, // The handshake failed, try getting another connection from the queue Ok(Err(e)) => { TLS_HANDSHAKE_FAILURES.inc(); - warn!(%peer_addr, protocol, "failed to accept TLS connection: {e:?}"); + warn!(%peer_addr, "failed to accept TLS connection: {e:?}"); None } // The handshake timed out, try getting another connection from the queue Err(_) => { TLS_HANDSHAKE_FAILURES.inc(); - warn!(%peer_addr, protocol, "failed to accept TLS connection: timeout"); + warn!(%peer_addr, "failed to accept TLS connection: timeout"); None } } - }); + }.instrument(span)); } Poll::Ready(Some(Err(e))) => { tracing::error!("error accepting TCP connection: {e}"); @@ -112,10 +108,7 @@ where loop { return match this.waiting.poll_join_next(cx) { - Poll::Ready(Some(Ok(Some(conn)))) => { - info!(protocol = this.protocol, "accepted new TLS connection"); - Poll::Ready(Some(Ok(conn))) - } + Poll::Ready(Some(Ok(Some(conn)))) => Poll::Ready(Some(Ok(conn))), // The handshake failed to complete, try getting another connection from the queue Poll::Ready(Some(Ok(None))) => continue, // The handshake panicked or was cancelled. ignore and get another connection diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index b3f460c7fe63..5b76e808d57a 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -2944,6 +2944,7 @@ def http_query(self, query, args, **kwargs): user = quote(kwargs["user"]) password = quote(kwargs["password"]) expected_code = kwargs.get("expected_code") + timeout = kwargs.get("timeout") log.info(f"Executing http query: {query}") @@ -2957,6 +2958,7 @@ def http_query(self, query, args, **kwargs): "Neon-Pool-Opt-In": "true", }, verify=str(self.test_output_dir / "proxy.crt"), + timeout=timeout, ) if expected_code is not None: diff --git a/test_runner/regress/test_proxy.py b/test_runner/regress/test_proxy.py index 078589d8ebe5..3e986a8f7b20 100644 --- a/test_runner/regress/test_proxy.py +++ b/test_runner/regress/test_proxy.py @@ -596,3 +596,39 @@ def test_sql_over_http_timeout_cancel(static_proxy: NeonProxy): assert ( "duplicate key value violates unique constraint" in res["message"] ), "HTTP query should conflict" + + +def test_sql_over_http_connection_cancel(static_proxy: NeonProxy): + static_proxy.safe_psql("create role http with login password 'http' superuser") + + static_proxy.safe_psql("create table test_table ( id int primary key )") + + # insert into a table, with a unique constraint, after sleeping for n seconds + query = "WITH temp AS ( \ + SELECT pg_sleep($1) as sleep, $2::int as id \ + ) INSERT INTO test_table (id) SELECT id FROM temp" + + try: + # The request should complete before the proxy HTTP timeout triggers. + # Timeout and cancel the request on the client side before the query completes. + static_proxy.http_query( + query, + [static_proxy.http_timeout_seconds - 1, 1], + user="http", + password="http", + timeout=2, + ) + except requests.exceptions.ReadTimeout: + pass + + # wait until the query _would_ have been complete + time.sleep(static_proxy.http_timeout_seconds) + + res = static_proxy.http_query(query, [1, 1], user="http", password="http", expected_code=200) + assert res["command"] == "INSERT", "HTTP query should insert" + assert res["rowCount"] == 1, "HTTP query should insert" + + res = static_proxy.http_query(query, [0, 1], user="http", password="http", expected_code=400) + assert ( + "duplicate key value violates unique constraint" in res["message"] + ), "HTTP query should conflict" From 44f42627dd32f29a3be9cfcd2d8c487c89642dc8 Mon Sep 17 00:00:00 2001 From: John Spray Date: Thu, 14 Mar 2024 09:11:57 +0000 Subject: [PATCH 43/43] pageserver/controller: error handling for shard splitting (#7074) ## Problem Shard splits worked, but weren't safe against failures (e.g. node crash during split) yet. Related: #6676 ## Summary of changes - Introduce async rwlocks at the scope of Tenant and Node: - exclusive tenant lock is used to protect splits - exclusive node lock is used to protect new reconciliation process that happens when setting node active - exclusive locks used in both cases when doing persistent updates (e.g. node scheduling conf) where the update to DB & in-memory state needs to be atomic. - Add failpoints to shard splitting in control plane and pageserver code. - Implement error handling in control plane for shard splits: this detaches child chards and ensures parent shards are re-attached. - Crash-safety for storage controller restarts requires little effort: we already reconcile with nodes over a storage controller restart, so as long as we reset any incomplete splits in the DB on restart (added in this PR), things are implicitly cleaned up. - Implement reconciliation with offline nodes before they transition to active: - (in this context reconciliation means something like startup_reconcile, not literally the Reconciler) - This covers cases where split abort cannot reach a node to clean it up: the cleanup will eventually happen when the node is marked active, as part of reconciliation. - This also covers the case where a node was unavailable when the storage controller started, but becomes available later: previously this allowed it to skip the startup reconcile. - Storage controller now terminates on panics. We only use panics for true "should never happen" assertions, and these cases can leave us in an un-usable state if we keep running (e.g. panicking in a shard split). In the unlikely event that we get into a crashloop as a result, we'll rely on kubernetes to back us off. - Add `test_sharding_split_failures` which exercises a variety of failure cases during shard split. --- Cargo.lock | 2 + control_plane/attachment_service/Cargo.toml | 2 + control_plane/attachment_service/src/http.rs | 5 + .../attachment_service/src/id_lock_map.rs | 54 ++ control_plane/attachment_service/src/lib.rs | 1 + control_plane/attachment_service/src/main.rs | 6 + control_plane/attachment_service/src/node.rs | 35 +- .../attachment_service/src/persistence.rs | 78 ++ .../attachment_service/src/reconciler.rs | 56 +- .../attachment_service/src/service.rs | 787 +++++++++++++++--- .../attachment_service/src/tenant_state.rs | 7 +- pageserver/src/http/routes.rs | 10 + pageserver/src/tenant/mgr.rs | 56 +- test_runner/conftest.py | 1 + test_runner/fixtures/compute_reconfigure.py | 62 ++ test_runner/fixtures/neon_fixtures.py | 17 + test_runner/fixtures/workload.py | 78 +- test_runner/regress/test_sharding.py | 340 +++++++- 18 files changed, 1446 insertions(+), 151 deletions(-) create mode 100644 control_plane/attachment_service/src/id_lock_map.rs create mode 100644 test_runner/fixtures/compute_reconfigure.py diff --git a/Cargo.lock b/Cargo.lock index 7fd9053f627c..45397eb4a2c9 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -282,8 +282,10 @@ dependencies = [ "control_plane", "diesel", "diesel_migrations", + "fail", "futures", "git-version", + "hex", "humantime", "hyper", "metrics", diff --git a/control_plane/attachment_service/Cargo.toml b/control_plane/attachment_service/Cargo.toml index a5fad7216ca1..f78f56c48039 100644 --- a/control_plane/attachment_service/Cargo.toml +++ b/control_plane/attachment_service/Cargo.toml @@ -19,8 +19,10 @@ aws-config.workspace = true aws-sdk-secretsmanager.workspace = true camino.workspace = true clap.workspace = true +fail.workspace = true futures.workspace = true git-version.workspace = true +hex.workspace = true hyper.workspace = true humantime.workspace = true once_cell.workspace = true diff --git a/control_plane/attachment_service/src/http.rs b/control_plane/attachment_service/src/http.rs index 515c287ea99d..d26652cc948c 100644 --- a/control_plane/attachment_service/src/http.rs +++ b/control_plane/attachment_service/src/http.rs @@ -10,7 +10,9 @@ use pageserver_api::shard::TenantShardId; use pageserver_client::mgmt_api; use std::sync::Arc; use std::time::{Duration, Instant}; +use tokio_util::sync::CancellationToken; use utils::auth::{Scope, SwappableJwtAuth}; +use utils::failpoint_support::failpoints_handler; use utils::http::endpoint::{auth_middleware, check_permission_with, request_span}; use utils::http::request::{must_get_query_param, parse_request_param}; use utils::id::{TenantId, TimelineId}; @@ -554,6 +556,9 @@ pub fn make_router( .post("/debug/v1/consistency_check", |r| { request_span(r, handle_consistency_check) }) + .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) }) diff --git a/control_plane/attachment_service/src/id_lock_map.rs b/control_plane/attachment_service/src/id_lock_map.rs new file mode 100644 index 000000000000..b03700b50cf8 --- /dev/null +++ b/control_plane/attachment_service/src/id_lock_map.rs @@ -0,0 +1,54 @@ +use std::{collections::HashMap, sync::Arc}; + +/// A map of locks covering some arbitrary identifiers. Useful if you have a collection of objects but don't +/// want to embed a lock in each one, or if your locking granularity is different to your object granularity. +/// For example, used in the storage controller where the objects are tenant shards, but sometimes locking +/// is needed at a tenant-wide granularity. +pub(crate) struct IdLockMap +where + T: Eq + PartialEq + std::hash::Hash, +{ + /// A synchronous lock for getting/setting the async locks that our callers will wait on. + entities: std::sync::Mutex>>>, +} + +impl IdLockMap +where + T: Eq + PartialEq + std::hash::Hash, +{ + pub(crate) fn shared( + &self, + key: T, + ) -> impl std::future::Future> { + let mut locked = self.entities.lock().unwrap(); + let entry = locked.entry(key).or_default(); + entry.clone().read_owned() + } + + pub(crate) fn exclusive( + &self, + key: T, + ) -> impl std::future::Future> { + let mut locked = self.entities.lock().unwrap(); + let entry = locked.entry(key).or_default(); + entry.clone().write_owned() + } + + /// Rather than building a lock guard that re-takes the [`Self::entities`] lock, we just do + /// periodic housekeeping to avoid the map growing indefinitely + pub(crate) fn housekeeping(&self) { + let mut locked = self.entities.lock().unwrap(); + locked.retain(|_k, lock| lock.try_write().is_err()) + } +} + +impl Default for IdLockMap +where + T: Eq + PartialEq + std::hash::Hash, +{ + fn default() -> Self { + Self { + entities: std::sync::Mutex::new(HashMap::new()), + } + } +} diff --git a/control_plane/attachment_service/src/lib.rs b/control_plane/attachment_service/src/lib.rs index 796b465c102c..a017bc1ecc4c 100644 --- a/control_plane/attachment_service/src/lib.rs +++ b/control_plane/attachment_service/src/lib.rs @@ -4,6 +4,7 @@ use utils::seqwait::MonotonicCounter; mod auth; mod compute_hook; pub mod http; +mod id_lock_map; pub mod metrics; mod node; pub mod persistence; diff --git a/control_plane/attachment_service/src/main.rs b/control_plane/attachment_service/src/main.rs index 333c3911e36c..fb7b363c3966 100644 --- a/control_plane/attachment_service/src/main.rs +++ b/control_plane/attachment_service/src/main.rs @@ -206,6 +206,12 @@ async fn migration_run(database_url: &str) -> anyhow::Result<()> { } fn main() -> anyhow::Result<()> { + let default_panic = std::panic::take_hook(); + std::panic::set_hook(Box::new(move |info| { + default_panic(info); + std::process::exit(1); + })); + tokio::runtime::Builder::new_current_thread() // We use spawn_blocking for database operations, so require approximately // as many blocking threads as we will open database connections. diff --git a/control_plane/attachment_service/src/node.rs b/control_plane/attachment_service/src/node.rs index 27b03608fa25..dda8a155c6f0 100644 --- a/control_plane/attachment_service/src/node.rs +++ b/control_plane/attachment_service/src/node.rs @@ -83,29 +83,38 @@ impl Node { } } - pub(crate) fn set_availability( - &mut self, - availability: NodeAvailability, - ) -> AvailabilityTransition { - use NodeAvailability::*; - let transition = match (self.availability, availability) { - (Offline, Active) => { + pub(crate) fn set_availability(&mut self, availability: NodeAvailability) { + match self.get_availability_transition(availability) { + AvailabilityTransition::ToActive => { // Give the node a new cancellation token, effectively resetting it to un-cancelled. Any // users of previously-cloned copies of the node will still see the old cancellation // state. For example, Reconcilers in flight will have to complete and be spawned // again to realize that the node has become available. self.cancel = CancellationToken::new(); - AvailabilityTransition::ToActive } - (Active, Offline) => { + AvailabilityTransition::ToOffline => { // Fire the node's cancellation token to cancel any in-flight API requests to it self.cancel.cancel(); - AvailabilityTransition::ToOffline } - _ => AvailabilityTransition::Unchanged, - }; + AvailabilityTransition::Unchanged => {} + } self.availability = availability; - transition + } + + /// Without modifying the availability of the node, convert the intended availability + /// into a description of the transition. + pub(crate) fn get_availability_transition( + &self, + availability: NodeAvailability, + ) -> AvailabilityTransition { + use AvailabilityTransition::*; + use NodeAvailability::*; + + match (self.availability, availability) { + (Offline, Active) => ToActive, + (Active, Offline) => ToOffline, + _ => Unchanged, + } } /// Whether we may send API requests to this node. diff --git a/control_plane/attachment_service/src/persistence.rs b/control_plane/attachment_service/src/persistence.rs index aa0894583466..3602cf8b1f21 100644 --- a/control_plane/attachment_service/src/persistence.rs +++ b/control_plane/attachment_service/src/persistence.rs @@ -11,6 +11,9 @@ use diesel::prelude::*; use diesel::Connection; use pageserver_api::controller_api::{NodeSchedulingPolicy, PlacementPolicy}; use pageserver_api::models::TenantConfig; +use pageserver_api::shard::ShardConfigError; +use pageserver_api::shard::ShardIdentity; +use pageserver_api::shard::ShardStripeSize; use pageserver_api::shard::{ShardCount, ShardNumber, TenantShardId}; use serde::{Deserialize, Serialize}; use utils::generation::Generation; @@ -72,6 +75,14 @@ pub(crate) enum DatabaseError { Logical(String), } +#[must_use] +pub(crate) enum AbortShardSplitStatus { + /// We aborted the split in the database by reverting to the parent shards + Aborted, + /// The split had already been persisted. + Complete, +} + pub(crate) type DatabaseResult = Result; impl Persistence { @@ -570,6 +581,51 @@ impl Persistence { }) .await } + + /// Used when the remote part of a shard split failed: we will revert the database state to have only + /// the parent shards, with SplitState::Idle. + pub(crate) async fn abort_shard_split( + &self, + split_tenant_id: TenantId, + 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)?; + + // 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 \ + 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)?; + + Ok(AbortShardSplitStatus::Aborted) + })?; + + Ok(aborted) + }) + .await + } } /// Parts of [`crate::tenant_state::TenantState`] that are stored durably @@ -604,6 +660,28 @@ pub(crate) struct TenantShardPersistence { pub(crate) config: String, } +impl TenantShardPersistence { + pub(crate) fn get_shard_identity(&self) -> Result { + if self.shard_count == 0 { + Ok(ShardIdentity::unsharded()) + } else { + Ok(ShardIdentity::new( + ShardNumber(self.shard_number as u8), + ShardCount::new(self.shard_count as u8), + ShardStripeSize(self.shard_stripe_size as u32), + )?) + } + } + + pub(crate) fn get_tenant_shard_id(&self) -> Result { + Ok(TenantShardId { + tenant_id: TenantId::from_str(self.tenant_id.as_str())?, + shard_number: ShardNumber(self.shard_number as u8), + shard_count: ShardCount::new(self.shard_count as u8), + }) + } +} + /// Parts of [`crate::node::Node`] that are stored durably #[derive(Serialize, Deserialize, Queryable, Selectable, Insertable, Eq, PartialEq)] #[diesel(table_name = crate::schema::nodes)] diff --git a/control_plane/attachment_service/src/reconciler.rs b/control_plane/attachment_service/src/reconciler.rs index 603da9bf022c..7f68a65c1543 100644 --- a/control_plane/attachment_service/src/reconciler.rs +++ b/control_plane/attachment_service/src/reconciler.rs @@ -1,5 +1,6 @@ use crate::persistence::Persistence; use crate::service; +use hyper::StatusCode; use pageserver_api::models::{ LocationConfig, LocationConfigMode, LocationConfigSecondary, TenantConfig, }; @@ -18,6 +19,8 @@ use crate::compute_hook::{ComputeHook, NotifyError}; use crate::node::Node; use crate::tenant_state::{IntentState, ObservedState, ObservedStateLocation}; +const DEFAULT_HEATMAP_PERIOD: &str = "60s"; + /// Object with the lifetime of the background reconcile task that is created /// for tenants which have a difference between their intent and observed states. pub(super) struct Reconciler { @@ -485,17 +488,29 @@ impl Reconciler { ) .await { - Some(Ok(observed)) => observed, + Some(Ok(observed)) => Some(observed), + Some(Err(mgmt_api::Error::ApiError(status, _msg))) + if status == StatusCode::NOT_FOUND => + { + None + } Some(Err(e)) => return Err(e.into()), None => return Err(ReconcileError::Cancel), }; tracing::info!("Scanned location configuration on {attached_node}: {observed_conf:?}"); - self.observed.locations.insert( - attached_node.get_id(), - ObservedStateLocation { - conf: observed_conf, - }, - ); + match observed_conf { + Some(conf) => { + // Pageserver returned a state: update it in observed. This may still be an indeterminate (None) state, + // if internally the pageserver's TenantSlot was being mutated (e.g. some long running API call is still running) + self.observed + .locations + .insert(attached_node.get_id(), ObservedStateLocation { conf }); + } + None => { + // Pageserver returned 404: we have confirmation that there is no state for this shard on that pageserver. + self.observed.locations.remove(&attached_node.get_id()); + } + } } Ok(()) @@ -525,7 +540,12 @@ impl Reconciler { ))); }; - let mut wanted_conf = attached_location_conf(generation, &self.shard, &self.config); + let mut wanted_conf = attached_location_conf( + generation, + &self.shard, + &self.config, + !self.intent.secondary.is_empty(), + ); match self.observed.locations.get(&node.get_id()) { Some(conf) if conf.conf.as_ref() == Some(&wanted_conf) => { // Nothing to do @@ -662,10 +682,26 @@ impl Reconciler { } } +/// We tweak the externally-set TenantConfig while configuring +/// locations, using our awareness of whether secondary locations +/// are in use to automatically enable/disable heatmap uploads. +fn ha_aware_config(config: &TenantConfig, has_secondaries: bool) -> TenantConfig { + let mut config = config.clone(); + if has_secondaries { + if config.heatmap_period.is_none() { + config.heatmap_period = Some(DEFAULT_HEATMAP_PERIOD.to_string()); + } + } else { + config.heatmap_period = None; + } + config +} + pub(crate) fn attached_location_conf( generation: Generation, shard: &ShardIdentity, config: &TenantConfig, + has_secondaries: bool, ) -> LocationConfig { LocationConfig { mode: LocationConfigMode::AttachedSingle, @@ -674,7 +710,7 @@ pub(crate) fn attached_location_conf( shard_number: shard.number.0, shard_count: shard.count.literal(), shard_stripe_size: shard.stripe_size.0, - tenant_conf: config.clone(), + tenant_conf: ha_aware_config(config, has_secondaries), } } @@ -689,6 +725,6 @@ pub(crate) fn secondary_location_conf( shard_number: shard.number.0, shard_count: shard.count.literal(), shard_stripe_size: shard.stripe_size.0, - tenant_conf: config.clone(), + tenant_conf: ha_aware_config(config, true), } } diff --git a/control_plane/attachment_service/src/service.rs b/control_plane/attachment_service/src/service.rs index 1c4ede3d9d75..1b85081666b9 100644 --- a/control_plane/attachment_service/src/service.rs +++ b/control_plane/attachment_service/src/service.rs @@ -7,6 +7,7 @@ use std::{ time::{Duration, Instant}, }; +use crate::{id_lock_map::IdLockMap, persistence::AbortShardSplitStatus}; use anyhow::Context; use control_plane::storage_controller::{ AttachHookRequest, AttachHookResponse, InspectRequest, InspectResponse, @@ -36,6 +37,7 @@ use pageserver_api::{ }, }; use pageserver_client::mgmt_api; +use tokio::sync::OwnedRwLockWriteGuard; use tokio_util::sync::CancellationToken; use tracing::instrument; use utils::{ @@ -147,6 +149,18 @@ pub struct Service { compute_hook: Arc, result_tx: tokio::sync::mpsc::UnboundedSender, + // Channel for background cleanup from failed operations that require cleanup, such as shard split + abort_tx: tokio::sync::mpsc::UnboundedSender, + + // Locking on a tenant granularity (covers all shards in the tenant): + // - Take exclusively for rare operations that mutate the tenant's persistent state (e.g. create/delete/split) + // - Take in shared mode for operations that need the set of shards to stay the same to complete reliably (e.g. timeline CRUD) + tenant_op_locks: IdLockMap, + + // Locking for node-mutating operations: take exclusively for operations that modify the node's persistent state, or + // that transition it to/from Active. + node_op_locks: IdLockMap, + // Process shutdown will fire this token cancel: CancellationToken, @@ -174,6 +188,27 @@ enum TenantCreateOrUpdate { Update(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 { + tenant_id: TenantId, + /// The target values from the request that failed + new_shard_count: ShardCount, + new_stripe_size: Option, + /// Until this abort op is complete, no other operations may be done on the tenant + _tenant_lock: tokio::sync::OwnedRwLockWriteGuard<()>, +} + +#[derive(thiserror::Error, Debug)] +enum TenantShardSplitAbortError { + #[error(transparent)] + Database(#[from] DatabaseError), + #[error(transparent)] + Remote(#[from] mgmt_api::Error), + #[error("Unavailable")] + Unavailable, +} + struct ShardUpdate { tenant_shard_id: TenantShardId, placement_policy: PlacementPolicy, @@ -627,8 +662,52 @@ impl Service { } } + async fn process_aborts( + &self, + mut abort_rx: tokio::sync::mpsc::UnboundedReceiver, + ) { + loop { + // Wait for the next result, or for cancellation + let op = tokio::select! { + r = abort_rx.recv() => { + match r { + Some(op) => {op}, + None => {break;} + } + } + _ = self.cancel.cancelled() => { + break; + } + }; + + // Retry until shutdown: we must keep this request object alive until it is properly + // processed, as it holds a lock guard that prevents other operations trying to do things + // to the tenant while it is in a weird part-split state. + while !self.cancel.is_cancelled() { + match self.abort_tenant_shard_split(&op).await { + Ok(_) => break, + Err(e) => { + tracing::warn!( + "Failed to abort shard split on {}, will retry: {e}", + op.tenant_id + ); + + // If a node is unavailable, we hope that it has been properly marked Offline + // when we retry, so that the abort op will succeed. If the abort op is failing + // for some other reason, we will keep retrying forever, or until a human notices + // and does something about it (either fixing a pageserver or restarting the controller). + tokio::time::timeout(Duration::from_secs(5), self.cancel.cancelled()) + .await + .ok(); + } + } + } + } + } + pub async fn spawn(config: Config, persistence: Arc) -> anyhow::Result> { let (result_tx, result_rx) = tokio::sync::mpsc::unbounded_channel(); + let (abort_tx, abort_rx) = tokio::sync::mpsc::unbounded_channel(); tracing::info!("Loading nodes from database..."); let nodes = persistence @@ -641,12 +720,62 @@ impl Service { tracing::info!("Loaded {} nodes from database.", nodes.len()); tracing::info!("Loading shards from database..."); - let tenant_shard_persistence = persistence.list_tenant_shards().await?; + let mut tenant_shard_persistence = persistence.list_tenant_shards().await?; tracing::info!( "Loaded {} shards from database.", tenant_shard_persistence.len() ); + // If any shard splits were in progress, reset the database state to abort them + let mut tenant_shard_count_min_max: HashMap = + HashMap::new(); + for tsp in &mut tenant_shard_persistence { + let shard = tsp.get_shard_identity()?; + let tenant_shard_id = tsp.get_tenant_shard_id()?; + let entry = tenant_shard_count_min_max + .entry(tenant_shard_id.tenant_id) + .or_insert_with(|| (shard.count, shard.count)); + entry.0 = std::cmp::min(entry.0, shard.count); + entry.1 = std::cmp::max(entry.1, shard.count); + } + + for (tenant_id, (count_min, count_max)) in tenant_shard_count_min_max { + if count_min != count_max { + // Aborting the split in the database and dropping the child shards is sufficient: the reconciliation in + // [`Self::startup_reconcile`] will implicitly drop the child shards on remote pageservers, or they'll + // be dropped later in [`Self::node_activate_reconcile`] if it isn't available right now. + tracing::info!("Aborting shard split {tenant_id} {count_min:?} -> {count_max:?}"); + let abort_status = persistence.abort_shard_split(tenant_id, count_max).await?; + + // We may never see the Complete status here: if the split was complete, we wouldn't have + // identified this tenant has having mismatching min/max counts. + assert!(matches!(abort_status, AbortShardSplitStatus::Aborted)); + + // Clear the splitting status in-memory, to reflect that we just aborted in the database + tenant_shard_persistence.iter_mut().for_each(|tsp| { + // Set idle split state on those shards that we will retain. + let tsp_tenant_id = TenantId::from_str(tsp.tenant_id.as_str()).unwrap(); + if tsp_tenant_id == tenant_id + && tsp.get_shard_identity().unwrap().count == count_min + { + tsp.splitting = SplitState::Idle; + } else if tsp_tenant_id == tenant_id { + // Leave the splitting state on the child shards: this will be used next to + // drop them. + tracing::info!( + "Shard {tsp_tenant_id} will be dropped after shard split abort", + ); + } + }); + + // Drop shards for this tenant which we didn't just mark idle (i.e. child shards of the aborted split) + tenant_shard_persistence.retain(|tsp| { + TenantId::from_str(tsp.tenant_id.as_str()).unwrap() != tenant_id + || tsp.splitting == SplitState::Idle + }); + } + } + let mut tenants = BTreeMap::new(); let mut scheduler = Scheduler::new(nodes.values()); @@ -676,21 +805,8 @@ impl Service { } } for tsp in tenant_shard_persistence { - let tenant_shard_id = TenantShardId { - tenant_id: TenantId::from_str(tsp.tenant_id.as_str())?, - shard_number: ShardNumber(tsp.shard_number as u8), - shard_count: ShardCount::new(tsp.shard_count as u8), - }; - let shard_identity = if tsp.shard_count == 0 { - ShardIdentity::unsharded() - } else { - ShardIdentity::new( - ShardNumber(tsp.shard_number as u8), - ShardCount::new(tsp.shard_count as u8), - ShardStripeSize(tsp.shard_stripe_size as u32), - )? - }; - + let tenant_shard_id = tsp.get_tenant_shard_id()?; + let shard_identity = tsp.get_shard_identity()?; // We will populate intent properly later in [`Self::startup_reconcile`], initially populate // it with what we can infer: the node for which a generation was most recently issued. let mut intent = IntentState::new(); @@ -728,9 +844,12 @@ impl Service { persistence, compute_hook: Arc::new(ComputeHook::new(config)), result_tx, + abort_tx, startup_complete: startup_complete.clone(), cancel: CancellationToken::new(), gate: Gate::default(), + tenant_op_locks: Default::default(), + node_op_locks: Default::default(), }); let result_task_this = this.clone(); @@ -741,6 +860,33 @@ impl Service { } }); + tokio::task::spawn({ + let this = this.clone(); + async move { + // Block shutdown until we're done (we must respect self.cancel) + if let Ok(_gate) = this.gate.enter() { + this.process_aborts(abort_rx).await + } + } + }); + + tokio::task::spawn({ + let this = this.clone(); + async move { + if let Ok(_gate) = this.gate.enter() { + loop { + tokio::select! { + _ = this.cancel.cancelled() => { + break; + }, + _ = tokio::time::sleep(Duration::from_secs(60)) => {} + }; + this.tenant_op_locks.housekeeping(); + } + } + } + }); + tokio::task::spawn({ let this = this.clone(); // We will block the [`Service::startup_complete`] barrier until [`Self::startup_reconcile`] @@ -889,6 +1035,7 @@ impl Service { tenant_state.generation.unwrap(), &tenant_state.shard, &tenant_state.config, + false, )), }, )]); @@ -918,6 +1065,118 @@ impl Service { } } + // When the availability state of a node transitions to active, we must do a full reconciliation + // of LocationConfigs on that node. This is because while a node was offline: + // - we might have proceeded through startup_reconcile without checking for extraneous LocationConfigs on this node + // - aborting a tenant shard split might have left rogue child shards behind on this node. + // + // This function must complete _before_ setting a `Node` to Active: once it is set to Active, other + // Reconcilers might communicate with the node, and these must not overlap with the work we do in + // this function. + // + // The reconciliation logic in here is very similar to what [`Self::startup_reconcile`] does, but + // for written for a single node rather than as a batch job for all nodes. + #[tracing::instrument(skip_all, fields(node_id=%node.get_id()))] + async fn node_activate_reconcile( + &self, + mut node: Node, + _lock: &OwnedRwLockWriteGuard<()>, + ) -> Result<(), ApiError> { + // This Node is a mutable local copy: we will set it active so that we can use its + // API client to reconcile with the node. The Node in [`Self::nodes`] will get updated + // later. + node.set_availability(NodeAvailability::Active); + + let configs = match node + .with_client_retries( + |client| async move { client.list_location_config().await }, + &self.config.jwt_token, + 1, + 5, + SHORT_RECONCILE_TIMEOUT, + &self.cancel, + ) + .await + { + None => { + // We're shutting down (the Node's cancellation token can't have fired, because + // we're the only scope that has a reference to it, and we didn't fire it). + return Err(ApiError::ShuttingDown); + } + Some(Err(e)) => { + // This node didn't succeed listing its locations: it may not proceed to active state + // as it is apparently unavailable. + return Err(ApiError::PreconditionFailed( + format!("Failed to query node location configs, cannot activate ({e})").into(), + )); + } + Some(Ok(configs)) => configs, + }; + tracing::info!("Loaded {} LocationConfigs", configs.tenant_shards.len()); + + let mut cleanup = Vec::new(); + { + let mut locked = self.inner.write().unwrap(); + + for (tenant_shard_id, observed_loc) in configs.tenant_shards { + let Some(tenant_state) = locked.tenants.get_mut(&tenant_shard_id) else { + cleanup.push(tenant_shard_id); + continue; + }; + tenant_state + .observed + .locations + .insert(node.get_id(), ObservedStateLocation { conf: observed_loc }); + } + } + + for tenant_shard_id in cleanup { + tracing::info!("Detaching {tenant_shard_id}"); + match node + .with_client_retries( + |client| async move { + let config = LocationConfig { + mode: LocationConfigMode::Detached, + generation: None, + secondary_conf: None, + shard_number: tenant_shard_id.shard_number.0, + shard_count: tenant_shard_id.shard_count.literal(), + shard_stripe_size: 0, + tenant_conf: models::TenantConfig::default(), + }; + client + .location_config(tenant_shard_id, config, None, false) + .await + }, + &self.config.jwt_token, + 1, + 5, + SHORT_RECONCILE_TIMEOUT, + &self.cancel, + ) + .await + { + None => { + // We're shutting down (the Node's cancellation token can't have fired, because + // we're the only scope that has a reference to it, and we didn't fire it). + return Err(ApiError::ShuttingDown); + } + Some(Err(e)) => { + // Do not let the node proceed to Active state if it is not responsive to requests + // to detach. This could happen if e.g. a shutdown bug in the pageserver is preventing + // detach completing: we should not let this node back into the set of nodes considered + // okay for scheduling. + return Err(ApiError::Conflict(format!( + "Node {node} failed to detach {tenant_shard_id}: {e}" + ))); + } + Some(Ok(_)) => {} + }; + } + + Ok(()) + } + pub(crate) async fn re_attach( &self, reattach_req: ReAttachRequest, @@ -926,15 +1185,6 @@ impl Service { self.node_register(register_req).await?; } - // Take a re-attach as indication that the node is available: this is a precursor to proper - // heartbeating in https://github.com/neondatabase/neon/issues/6844 - self.node_configure(NodeConfigureRequest { - node_id: reattach_req.node_id, - availability: Some(NodeAvailability::Active), - scheduling: None, - }) - .await?; - // Ordering: we must persist generation number updates before making them visible in the in-memory state let incremented_generations = self.persistence.re_attach(reattach_req.node_id).await?; @@ -946,6 +1196,7 @@ impl Service { // Apply the updated generation to our in-memory state let mut locked = self.inner.write().unwrap(); + let (nodes, tenants, _scheduler) = locked.parts_mut(); let mut response = ReAttachResponse { tenants: Vec::new(), @@ -957,7 +1208,7 @@ impl Service { gen: new_gen.into().unwrap(), }); // Apply the new generation number to our in-memory state - let shard_state = locked.tenants.get_mut(&tenant_shard_id); + 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 @@ -1008,6 +1259,25 @@ impl Service { // request in flight over the network: TODO handle that by making location_conf API refuse // to go backward in generations. } + + // We consider a node Active once we have composed a re-attach response, but we + // do not call [`Self::node_activate_reconcile`]: the handling of the re-attach response + // implicitly synchronizes the LocationConfigs on the node. + // + // Setting a node active unblocks any Reconcilers that might write to the location config API, + // but those requests will not be accepted by the node until it has finished processing + // the re-attach response. + if let Some(node) = nodes.get(&reattach_req.node_id) { + if !node.is_available() { + let mut new_nodes = (**nodes).clone(); + if let Some(node) = new_nodes.get_mut(&reattach_req.node_id) { + node.set_availability(NodeAvailability::Active); + } + let new_nodes = Arc::new(new_nodes); + *nodes = new_nodes; + } + } + Ok(response) } @@ -1048,6 +1318,12 @@ impl Service { &self, create_req: TenantCreateRequest, ) -> Result { + // Exclude any concurrent attempts to create/access the same tenant ID + let _tenant_lock = self + .tenant_op_locks + .exclusive(create_req.new_tenant_id.tenant_id) + .await; + let (response, waiters) = self.do_tenant_create(create_req).await?; self.await_waiters(waiters, SHORT_RECONCILE_TIMEOUT).await?; @@ -1362,16 +1638,20 @@ impl Service { tenant_shard_id: TenantShardId, req: TenantLocationConfigRequest, ) -> Result { + // We require an exclusive lock, because we are updating both persistent and in-memory state + let _tenant_lock = self + .tenant_op_locks + .exclusive(tenant_shard_id.tenant_id) + .await; + if !tenant_shard_id.is_unsharded() { return Err(ApiError::BadRequest(anyhow::anyhow!( "This API is for importing single-sharded or unsharded tenants" ))); } - let tenant_id = tenant_shard_id.tenant_id; - // First check if this is a creation or an update - let create_or_update = self.tenant_location_config_prepare(tenant_id, req); + let create_or_update = self.tenant_location_config_prepare(tenant_shard_id.tenant_id, req); let mut result = TenantLocationConfigResponse { shards: Vec::new(), @@ -1477,6 +1757,9 @@ impl Service { } pub(crate) async fn tenant_config_set(&self, req: TenantConfigRequest) -> Result<(), ApiError> { + // We require an exclusive lock, because we are updating persistent and in-memory state + let _tenant_lock = self.tenant_op_locks.exclusive(req.tenant_id).await; + let tenant_id = req.tenant_id; let config = req.config; @@ -1558,6 +1841,8 @@ impl Service { timestamp: Cow<'_, str>, done_if_after: Cow<'_, str>, ) -> Result<(), ApiError> { + let _tenant_lock = self.tenant_op_locks.exclusive(tenant_id).await; + let node = { let locked = self.inner.read().unwrap(); // Just a sanity check to prevent misuse: the API expects that the tenant is fully @@ -1643,6 +1928,8 @@ impl Service { &self, tenant_id: TenantId, ) -> Result<(), ApiError> { + let _tenant_lock = self.tenant_op_locks.shared(tenant_id).await; + // Acquire lock and yield the collection of shard-node tuples which we will send requests onward to let targets = { let locked = self.inner.read().unwrap(); @@ -1692,6 +1979,8 @@ impl Service { } pub(crate) async fn tenant_delete(&self, tenant_id: TenantId) -> Result { + let _tenant_lock = self.tenant_op_locks.exclusive(tenant_id).await; + self.ensure_attached_wait(tenant_id).await?; // TODO: refactor into helper @@ -1788,10 +2077,10 @@ impl Service { create_req.new_timeline_id, ); + let _tenant_lock = self.tenant_op_locks.shared(tenant_id).await; + self.ensure_attached_wait(tenant_id).await?; - // TODO: refuse to do this if shard splitting is in progress - // (https://github.com/neondatabase/neon/issues/6676) let mut targets = { let locked = self.inner.read().unwrap(); let mut targets = Vec::new(); @@ -1913,11 +2202,10 @@ impl Service { timeline_id: TimelineId, ) -> Result { tracing::info!("Deleting timeline {}/{}", tenant_id, timeline_id,); + let _tenant_lock = self.tenant_op_locks.shared(tenant_id).await; self.ensure_attached_wait(tenant_id).await?; - // TODO: refuse to do this if shard splitting is in progress - // (https://github.com/neondatabase/neon/issues/6676) let mut targets = { let locked = self.inner.read().unwrap(); let mut targets = Vec::new(); @@ -2106,10 +2394,306 @@ impl Service { }) } + #[instrument(skip_all, fields(tenant_id=%op.tenant_id))] + async fn abort_tenant_shard_split( + &self, + op: &TenantShardSplitAbort, + ) -> Result<(), TenantShardSplitAbortError> { + // Cleaning up a split: + // - Parent shards are not destroyed during a split, just detached. + // - Failed pageserver split API calls can leave the remote node with just the parent attached, + // just the children attached, or both. + // + // Therefore our work to do is to: + // 1. Clean up storage controller's internal state to just refer to parents, no children + // 2. Call out to pageservers to ensure that children are detached + // 3. Call out to pageservers to ensure that parents are attached. + // + // Crash safety: + // - If the storage controller stops running during this cleanup *after* clearing the splitting state + // from our database, then [`Self::startup_reconcile`] will regard child attachments as garbage + // and detach them. + // - TODO: If the storage controller stops running during this cleanup *before* clearing the splitting state + // from our database, then we will re-enter this cleanup routine on startup. + + let TenantShardSplitAbort { + tenant_id, + new_shard_count, + new_stripe_size, + .. + } = op; + + // First abort persistent state, if any exists. + match self + .persistence + .abort_shard_split(*tenant_id, *new_shard_count) + .await? + { + AbortShardSplitStatus::Aborted => { + // Proceed to roll back any child shards created on pageservers + } + AbortShardSplitStatus::Complete => { + // The split completed (we might hit that path if e.g. our database transaction + // to write the completion landed in the database, but we dropped connection + // before seeing the result). + // + // We must update in-memory state to reflect the successful split. + self.tenant_shard_split_commit_inmem( + *tenant_id, + *new_shard_count, + *new_stripe_size, + ); + return Ok(()); + } + } + + // Clean up in-memory state, and accumulate the list of child locations that need detaching + 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(); + + for (tenant_shard_id, shard) in + tenants.range_mut(TenantShardId::tenant_range(op.tenant_id)) + { + if shard.shard.count == op.new_shard_count { + // Surprising: the phase of [`Self::do_tenant_shard_split`] which inserts child shards in-memory + // is infallible, so if we got an error we shouldn't have got that far. + tracing::warn!( + "During split abort, child shard {tenant_shard_id} found in-memory" + ); + continue; + } + + // Add the children of this shard to this list of things to detach + if let Some(node_id) = shard.intent.get_attached() { + for child_id in tenant_shard_id.split(*new_shard_count) { + detach_locations.push(( + nodes + .get(node_id) + .expect("Intent references nonexistent node") + .clone(), + child_id, + )); + } + } else { + tracing::warn!( + "During split abort, shard {tenant_shard_id} has no attached location" + ); + } + + tracing::info!("Restoring parent shard {tenant_shard_id}"); + shard.splitting = SplitState::Idle; + self.maybe_reconcile_shard(shard, nodes); + } + + // We don't expect any new_shard_count shards to exist here, but drop them just in case + tenants.retain(|_id, s| s.shard.count != *new_shard_count); + + detach_locations + }; + + for (node, child_id) in detach_locations { + if !node.is_available() { + // An unavailable node cannot be cleaned up now: to avoid blocking forever, we will permit this, and + // rely on the reconciliation that happens when a node transitions to Active to clean up. Since we have + // removed child shards from our in-memory state and database, the reconciliation will implicitly remove + // them from the node. + tracing::warn!("Node {node} unavailable, can't clean up during split abort. It will be cleaned up when it is reactivated."); + continue; + } + + // Detach the remote child. If the pageserver split API call is still in progress, this call will get + // a 503 and retry, up to our limit. + tracing::info!("Detaching {child_id} on {node}..."); + match node + .with_client_retries( + |client| async move { + let config = LocationConfig { + mode: LocationConfigMode::Detached, + generation: None, + secondary_conf: None, + shard_number: child_id.shard_number.0, + shard_count: child_id.shard_count.literal(), + // Stripe size and tenant config don't matter when detaching + shard_stripe_size: 0, + tenant_conf: TenantConfig::default(), + }; + + client.location_config(child_id, config, None, false).await + }, + &self.config.jwt_token, + 1, + 10, + Duration::from_secs(5), + &self.cancel, + ) + .await + { + Some(Ok(_)) => {} + Some(Err(e)) => { + // We failed to communicate with the remote node. This is problematic: we may be + // leaving it with a rogue child shard. + tracing::warn!( + "Failed to detach child {child_id} from node {node} during abort" + ); + return Err(e.into()); + } + None => { + // Cancellation: we were shutdown or the node went offline. Shutdown is fine, we'll + // clean up on restart. The node going offline requires a retry. + return Err(TenantShardSplitAbortError::Unavailable); + } + }; + } + + tracing::info!("Successfully aborted split"); + Ok(()) + } + + /// Infallible final stage of [`Self::tenant_shard_split`]: update the contents + /// of the tenant map to reflect the child shards that exist after the split. + fn tenant_shard_split_commit_inmem( + &self, + tenant_id: TenantId, + new_shard_count: ShardCount, + new_stripe_size: Option, + ) -> ( + TenantShardSplitResponse, + Vec<(TenantShardId, NodeId, ShardStripeSize)>, + ) { + let mut response = TenantShardSplitResponse { + new_shards: Vec::new(), + }; + let mut child_locations = Vec::new(); + { + let mut locked = self.inner.write().unwrap(); + + let parent_ids = locked + .tenants + .range(TenantShardId::tenant_range(tenant_id)) + .map(|(shard_id, _)| *shard_id) + .collect::>(); + + let (_nodes, tenants, scheduler) = locked.parts_mut(); + for parent_id in parent_ids { + let child_ids = parent_id.split(new_shard_count); + + let (pageserver, generation, policy, parent_ident, config) = { + let mut old_state = tenants + .remove(&parent_id) + .expect("It was present, we just split it"); + + // A non-splitting state is impossible, because [`Self::tenant_shard_split`] holds + // a TenantId lock and passes it through to [`TenantShardSplitAbort`] in case of cleanup: + // nothing else can clear this. + assert!(matches!(old_state.splitting, SplitState::Splitting)); + + let old_attached = old_state.intent.get_attached().unwrap(); + old_state.intent.clear(scheduler); + let generation = old_state.generation.expect("Shard must have been attached"); + ( + old_attached, + generation, + old_state.policy, + old_state.shard, + old_state.config, + ) + }; + + for child in child_ids { + let mut child_shard = parent_ident; + child_shard.number = child.shard_number; + child_shard.count = child.shard_count; + if let Some(stripe_size) = new_stripe_size { + child_shard.stripe_size = stripe_size; + } + + let mut child_observed: HashMap = HashMap::new(); + child_observed.insert( + pageserver, + ObservedStateLocation { + conf: Some(attached_location_conf( + generation, + &child_shard, + &config, + matches!(policy, PlacementPolicy::Double(n) if n > 0), + )), + }, + ); + + let mut child_state = TenantState::new(child, child_shard, policy.clone()); + child_state.intent = IntentState::single(scheduler, Some(pageserver)); + child_state.observed = ObservedState { + locations: child_observed, + }; + child_state.generation = Some(generation); + child_state.config = config.clone(); + + // The child's TenantState::splitting is intentionally left at the default value of Idle, + // as at this point in the split process we have succeeded and this part is infallible: + // we will never need to do any special recovery from this state. + + child_locations.push((child, pageserver, child_shard.stripe_size)); + + if let Err(e) = child_state.schedule(scheduler) { + // This is not fatal, because we've implicitly already got an attached + // location for the child shard. Failure here just means we couldn't + // find a secondary (e.g. because cluster is overloaded). + tracing::warn!("Failed to schedule child shard {child}: {e}"); + } + + tenants.insert(child, child_state); + response.new_shards.push(child); + } + } + + (response, child_locations) + } + } + pub(crate) async fn tenant_shard_split( &self, tenant_id: TenantId, split_req: TenantShardSplitRequest, + ) -> Result { + // TODO: return 503 if we get stuck waiting for this lock + // (issue https://github.com/neondatabase/neon/issues/7108) + let _tenant_lock = self.tenant_op_locks.exclusive(tenant_id).await; + + 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; + + 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. + tracing::warn!("Enqueuing background abort of split on {tenant_id}"); + self.abort_tx + .send(TenantShardSplitAbort { + tenant_id, + new_shard_count, + new_stripe_size, + _tenant_lock, + }) + // Ignore error sending: that just means we're shutting down: aborts are ephemeral so it's fine to drop it. + .ok(); + Err(e) + } + } + } + + pub(crate) async fn do_tenant_shard_split( + &self, + tenant_id: TenantId, + split_req: TenantShardSplitRequest, ) -> Result { let mut policy = None; let mut shard_ident = None; @@ -2121,6 +2705,10 @@ impl Service { child_ids: Vec, } + fail::fail_point!("shard-split-validation", |_| Err(ApiError::BadRequest( + anyhow::anyhow!("failpoint") + ))); + // Validate input, and calculate which shards we will create let (old_shard_count, targets) = { @@ -2230,7 +2818,9 @@ impl Service { if shard_ident.count.count() > 1 && shard_ident.stripe_size != new_stripe_size { return Err(ApiError::BadRequest(anyhow::anyhow!("Attempted to change stripe size ({:?}->{new_stripe_size:?}) on a tenant with multiple shards", shard_ident.stripe_size))); } + shard_ident.stripe_size = new_stripe_size; + tracing::info!("applied stripe size {}", shard_ident.stripe_size.0); shard_ident } else { shard_ident.unwrap() @@ -2255,6 +2845,11 @@ impl Service { child_shard.number = child.shard_number; child_shard.count = child.shard_count; + tracing::info!( + "Create child shard persistence with stripe size {}", + shard_ident.stripe_size.0 + ); + this_child_tsps.push(TenantShardPersistence { tenant_id: child.tenant_id.to_string(), shard_number: child.shard_number.0 as i32, @@ -2293,6 +2888,9 @@ impl Service { _ => return Err(ApiError::InternalServerError(e.into())), } } + fail::fail_point!("shard-split-post-begin", |_| Err( + ApiError::InternalServerError(anyhow::anyhow!("failpoint")) + )); // Now that I have persisted the splitting state, apply it in-memory. This is infallible, so // callers may assume that if splitting is set in memory, then it was persisted, and if splitting @@ -2302,15 +2900,16 @@ impl Service { for target in &targets { if let Some(parent_shard) = locked.tenants.get_mut(&target.parent_id) { parent_shard.splitting = SplitState::Splitting; + // Put the observed state to None, to reflect that it is indeterminate once we start the + // split operation. + parent_shard + .observed + .locations + .insert(target.node.get_id(), ObservedStateLocation { conf: None }); } } } - // FIXME: we have now committed the shard split state to the database, so any subsequent - // failure needs to roll it back. We will later wrap this function in logic to roll back - // the split if it fails. - // (https://github.com/neondatabase/neon/issues/6676) - // TODO: issue split calls concurrently (this only matters once we're splitting // N>1 shards into M shards -- initially we're usually splitting 1 shard into N). @@ -2332,6 +2931,10 @@ impl Service { .await .map_err(|e| ApiError::Conflict(format!("Failed to split {}: {}", parent_id, e)))?; + fail::fail_point!("shard-split-post-remote", |_| Err(ApiError::Conflict( + "failpoint".to_string() + ))); + tracing::info!( "Split {} into {}", parent_id, @@ -2366,62 +2969,16 @@ impl Service { .complete_shard_split(tenant_id, old_shard_count) .await?; - // Replace all the shards we just split with their children: this phase is infallible. - let mut response = TenantShardSplitResponse { - new_shards: Vec::new(), - }; - let mut child_locations = Vec::new(); - { - let mut locked = self.inner.write().unwrap(); - let (_nodes, tenants, scheduler) = locked.parts_mut(); - for target in targets { - let SplitTarget { - parent_id, - node: _node, - child_ids, - } = target; - let (pageserver, generation, config) = { - let mut old_state = tenants - .remove(&parent_id) - .expect("It was present, we just split it"); - let old_attached = old_state.intent.get_attached().unwrap(); - old_state.intent.clear(scheduler); - let generation = old_state.generation.expect("Shard must have been attached"); - (old_attached, generation, old_state.config.clone()) - }; - - for child in child_ids { - let mut child_shard = shard_ident; - child_shard.number = child.shard_number; - child_shard.count = child.shard_count; - - let mut child_observed: HashMap = HashMap::new(); - child_observed.insert( - pageserver, - ObservedStateLocation { - conf: Some(attached_location_conf(generation, &child_shard, &config)), - }, - ); + fail::fail_point!("shard-split-post-complete", |_| Err( + ApiError::InternalServerError(anyhow::anyhow!("failpoint")) + )); - let mut child_state = TenantState::new(child, child_shard, policy.clone()); - child_state.intent = IntentState::single(scheduler, Some(pageserver)); - child_state.observed = ObservedState { - locations: child_observed, - }; - child_state.generation = Some(generation); - child_state.config = config.clone(); - - // The child's TenantState::splitting is intentionally left at the default value of Idle, - // as at this point in the split process we have succeeded and this part is infallible: - // we will never need to do any special recovery from this state. - - child_locations.push((child, pageserver, child_shard.stripe_size)); - - tenants.insert(child, child_state); - response.new_shards.push(child); - } - } - } + // 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, + ); // Send compute notifications for all the new shards let mut failed_notifications = Vec::new(); @@ -2710,6 +3267,8 @@ impl Service { &self, register_req: NodeRegisterRequest, ) -> Result<(), ApiError> { + let _node_lock = self.node_op_locks.exclusive(register_req.node_id).await; + // Pre-check for an already-existing node { let locked = self.inner.read().unwrap(); @@ -2771,6 +3330,8 @@ impl Service { &self, config_req: NodeConfigureRequest, ) -> Result<(), ApiError> { + let _node_lock = self.node_op_locks.exclusive(config_req.node_id).await; + if let Some(scheduling) = config_req.scheduling { // Scheduling is a persistent part of Node: we must write updates to the database before // applying them in memory @@ -2779,6 +3340,37 @@ impl Service { .await?; } + // If we're activating a node, then before setting it active we must reconcile any shard locations + // on that node, in case it is out of sync, e.g. due to being unavailable during controller startup, + // by calling [`Self::node_activate_reconcile`] + // + // The transition we calculate here remains valid later in the function because we hold the op lock on the node: + // nothing else can mutate its availability while we run. + let availability_transition = if let Some(input_availability) = config_req.availability { + let (activate_node, availability_transition) = { + let locked = self.inner.read().unwrap(); + let Some(node) = locked.nodes.get(&config_req.node_id) else { + return Err(ApiError::NotFound( + anyhow::anyhow!("Node {} not registered", config_req.node_id).into(), + )); + }; + + ( + node.clone(), + node.get_availability_transition(input_availability), + ) + }; + + if matches!(availability_transition, AvailabilityTransition::ToActive) { + self.node_activate_reconcile(activate_node, &_node_lock) + .await?; + } + availability_transition + } else { + AvailabilityTransition::Unchanged + }; + + // Apply changes from the request to our in-memory state for the Node let mut locked = self.inner.write().unwrap(); let (nodes, tenants, scheduler) = locked.parts_mut(); @@ -2790,11 +3382,9 @@ impl Service { )); }; - let availability_transition = if let Some(availability) = &config_req.availability { - node.set_availability(*availability) - } else { - AvailabilityTransition::Unchanged - }; + if let Some(availability) = &config_req.availability { + node.set_availability(*availability); + } if let Some(scheduling) = config_req.scheduling { node.set_scheduling(scheduling); @@ -2808,6 +3398,7 @@ impl Service { let new_nodes = Arc::new(new_nodes); + // Modify scheduling state for any Tenants that are affected by a change in the node's availability state. match availability_transition { AvailabilityTransition::ToOffline => { tracing::info!("Node {} transition to offline", config_req.node_id); diff --git a/control_plane/attachment_service/src/tenant_state.rs b/control_plane/attachment_service/src/tenant_state.rs index 3c91e09ac319..39e557616d9c 100644 --- a/control_plane/attachment_service/src/tenant_state.rs +++ b/control_plane/attachment_service/src/tenant_state.rs @@ -577,7 +577,12 @@ impl TenantState { .generation .expect("Attempted to enter attached state without a generation"); - let wanted_conf = attached_location_conf(generation, &self.shard, &self.config); + let wanted_conf = attached_location_conf( + generation, + &self.shard, + &self.config, + !self.intent.secondary.is_empty(), + ); match self.observed.locations.get(&node_id) { Some(conf) if conf.conf.as_ref() == Some(&wanted_conf) => {} Some(_) | None => { diff --git a/pageserver/src/http/routes.rs b/pageserver/src/http/routes.rs index bb8b1bb7e5f6..fc67f4cf8fca 100644 --- a/pageserver/src/http/routes.rs +++ b/pageserver/src/http/routes.rs @@ -2108,6 +2108,16 @@ where R: std::future::Future, ApiError>> + Send + 'static, H: FnOnce(Request, CancellationToken) -> R + Send + Sync + 'static, { + if request.uri() != &"/v1/failpoints".parse::().unwrap() { + fail::fail_point!("api-503", |_| Err(ApiError::ResourceUnavailable( + "failpoint".into() + ))); + + fail::fail_point!("api-500", |_| Err(ApiError::InternalServerError( + anyhow::anyhow!("failpoint") + ))); + } + // Spawn a new task to handle the request, to protect the handler from unexpected // async cancellations. Most pageserver functions are not async cancellation safe. // We arm a drop-guard, so that if Hyper drops the Future, we signal the task diff --git a/pageserver/src/tenant/mgr.rs b/pageserver/src/tenant/mgr.rs index 26fcce1f387c..7cf03d8fd65b 100644 --- a/pageserver/src/tenant/mgr.rs +++ b/pageserver/src/tenant/mgr.rs @@ -1443,6 +1443,35 @@ impl TenantManager { new_shard_count: ShardCount, new_stripe_size: Option, ctx: &RequestContext, + ) -> anyhow::Result> { + let r = self + .do_shard_split(tenant_shard_id, 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"); + 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}"); + } + } + } + + r + } + + pub(crate) async fn do_shard_split( + &self, + tenant_shard_id: TenantShardId, + new_shard_count: ShardCount, + new_stripe_size: Option, + ctx: &RequestContext, ) -> anyhow::Result> { let tenant = get_tenant(tenant_shard_id, true)?; @@ -1477,6 +1506,10 @@ impl TenantManager { .join(",") ); + fail::fail_point!("shard-split-pre-prepare", |_| Err(anyhow::anyhow!( + "failpoint" + ))); + let parent_shard_identity = tenant.shard_identity; let parent_tenant_conf = tenant.get_tenant_conf(); let parent_generation = tenant.generation; @@ -1490,6 +1523,10 @@ impl TenantManager { return Err(e); } + fail::fail_point!("shard-split-post-prepare", |_| Err(anyhow::anyhow!( + "failpoint" + ))); + self.resources.deletion_queue_client.flush_advisory(); // Phase 2: Put the parent shard to InProgress and grab a reference to the parent Tenant @@ -1511,11 +1548,16 @@ impl TenantManager { anyhow::bail!("Detached parent shard in the middle of split!") } }; - + fail::fail_point!("shard-split-pre-hardlink", |_| Err(anyhow::anyhow!( + "failpoint" + ))); // Optimization: hardlink layers from the parent into the children, so that they don't have to // re-download & duplicate the data referenced in their initial IndexPart self.shard_split_hardlink(parent, child_shards.clone()) .await?; + fail::fail_point!("shard-split-post-hardlink", |_| Err(anyhow::anyhow!( + "failpoint" + ))); // Take a snapshot of where the parent's WAL ingest had got to: we will wait for // child shards to reach this point. @@ -1555,6 +1597,10 @@ impl TenantManager { .await?; } + fail::fail_point!("shard-split-post-child-conf", |_| Err(anyhow::anyhow!( + "failpoint" + ))); + // Phase 4: wait for child chards WAL ingest to catch up to target LSN for child_shard_id in &child_shards { let child_shard_id = *child_shard_id; @@ -1587,6 +1633,10 @@ impl TenantManager { timeline.timeline_id, target_lsn ); + + fail::fail_point!("shard-split-lsn-wait", |_| Err(anyhow::anyhow!( + "failpoint" + ))); if let Err(e) = timeline.wait_lsn(*target_lsn, ctx).await { // Failure here might mean shutdown, in any case this part is an optimization // and we shouldn't hold up the split operation. @@ -1632,6 +1682,10 @@ impl TenantManager { }, ); + fail::fail_point!("shard-split-pre-finish", |_| Err(anyhow::anyhow!( + "failpoint" + ))); + parent_slot_guard.drop_old_value()?; // Phase 6: Release the InProgress on the parent shard diff --git a/test_runner/conftest.py b/test_runner/conftest.py index 200c9c37409b..4b0c9ac71d90 100644 --- a/test_runner/conftest.py +++ b/test_runner/conftest.py @@ -2,6 +2,7 @@ "fixtures.pg_version", "fixtures.parametrize", "fixtures.httpserver", + "fixtures.compute_reconfigure", "fixtures.neon_fixtures", "fixtures.benchmark_fixture", "fixtures.pg_stats", diff --git a/test_runner/fixtures/compute_reconfigure.py b/test_runner/fixtures/compute_reconfigure.py new file mode 100644 index 000000000000..9dd66fe636cf --- /dev/null +++ b/test_runner/fixtures/compute_reconfigure.py @@ -0,0 +1,62 @@ +import concurrent.futures +from typing import Any + +import pytest +from werkzeug.wrappers.request import Request +from werkzeug.wrappers.response import Response + +from fixtures.log_helper import log +from fixtures.types import TenantId + + +class ComputeReconfigure: + def __init__(self, server): + self.server = server + self.control_plane_compute_hook_api = f"http://{server.host}:{server.port}/notify-attach" + self.workloads = {} + + def register_workload(self, workload): + self.workloads[workload.tenant_id] = workload + + +@pytest.fixture(scope="function") +def compute_reconfigure_listener(make_httpserver): + """ + This fixture exposes an HTTP listener for the storage controller to submit + compute notifications to us, instead of updating neon_local endpoints itself. + + Although storage controller can use neon_local directly, this causes problems when + the test is also concurrently modifying endpoints. Instead, configure storage controller + to send notifications up to this test code, which will route all endpoint updates + through Workload, which has a mutex to make concurrent updates safe. + """ + server = make_httpserver + + self = ComputeReconfigure(server) + + # Do neon_local endpoint reconfiguration in the background so that we can + # accept a healthy rate of calls into notify-attach. + reconfigure_threads = concurrent.futures.ThreadPoolExecutor(max_workers=1) + + def handler(request: Request): + assert request.json is not None + body: dict[str, Any] = request.json + log.info(f"notify-attach request: {body}") + + try: + workload = self.workloads[TenantId(body["tenant_id"])] + except KeyError: + pass + else: + # This causes the endpoint to query storage controller for its location, which + # is redundant since we already have it here, but this avoids extending the + # neon_local CLI to take full lists of locations + reconfigure_threads.submit(lambda workload=workload: workload.reconfigure()) # type: ignore[no-any-return] + + return Response(status=200) + + self.server.expect_request("/notify-attach", method="PUT").respond_with_handler(handler) + + yield self + reconfigure_threads.shutdown() + server.clear() diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index 5b76e808d57a..16ebc1969801 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -2177,6 +2177,23 @@ def consistency_check(self): ) log.info("storage controller passed consistency check") + def configure_failpoints(self, config_strings: Tuple[str, str] | List[Tuple[str, str]]): + if isinstance(config_strings, tuple): + pairs = [config_strings] + else: + pairs = config_strings + + log.info(f"Requesting config failpoints: {repr(pairs)}") + + res = self.request( + "PUT", + f"{self.env.storage_controller_api}/debug/v1/failpoints", + json=[{"name": name, "actions": actions} for name, actions in pairs], + headers=self.headers(TokenScope.ADMIN), + ) + log.info(f"Got failpoints request response code {res.status_code}") + res.raise_for_status() + def __enter__(self) -> "NeonStorageController": return self diff --git a/test_runner/fixtures/workload.py b/test_runner/fixtures/workload.py index 1d5394dc1de1..e852281fcf43 100644 --- a/test_runner/fixtures/workload.py +++ b/test_runner/fixtures/workload.py @@ -1,3 +1,4 @@ +import threading from typing import Optional from fixtures.log_helper import log @@ -11,6 +12,10 @@ from fixtures.pageserver.utils import wait_for_last_record_lsn, wait_for_upload from fixtures.types import TenantId, TimelineId +# neon_local doesn't handle creating/modifying endpoints concurrently, so we use a mutex +# to ensure we don't do that: this enables running lots of Workloads in parallel safely. +ENDPOINT_LOCK = threading.Lock() + class Workload: """ @@ -41,17 +46,30 @@ def __init__( self._endpoint: Optional[Endpoint] = None + def reconfigure(self): + """ + Request the endpoint to reconfigure based on location reported by storage controller + """ + if self._endpoint is not None: + with ENDPOINT_LOCK: + self._endpoint.reconfigure() + def endpoint(self, pageserver_id: Optional[int] = None) -> Endpoint: - if self._endpoint is None: - self._endpoint = self.env.endpoints.create( - self.branch_name, - tenant_id=self.tenant_id, - pageserver_id=pageserver_id, - endpoint_id="ep-workload", - ) - self._endpoint.start(pageserver_id=pageserver_id) - else: - self._endpoint.reconfigure(pageserver_id=pageserver_id) + # We may be running alongside other Workloads for different tenants. Full TTID is + # obnoxiously long for use here, but a cut-down version is still unique enough for tests. + endpoint_id = f"ep-workload-{str(self.tenant_id)[0:4]}-{str(self.timeline_id)[0:4]}" + + with ENDPOINT_LOCK: + if self._endpoint is None: + self._endpoint = self.env.endpoints.create( + self.branch_name, + tenant_id=self.tenant_id, + pageserver_id=pageserver_id, + endpoint_id=endpoint_id, + ) + self._endpoint.start(pageserver_id=pageserver_id) + else: + self._endpoint.reconfigure(pageserver_id=pageserver_id) connstring = self._endpoint.safe_psql( "SELECT setting FROM pg_settings WHERE name='neon.pageserver_connstring'" @@ -94,7 +112,7 @@ def write_rows(self, n, pageserver_id: Optional[int] = None, upload: bool = True else: return False - def churn_rows(self, n, pageserver_id: Optional[int] = None, upload=True): + def churn_rows(self, n, pageserver_id: Optional[int] = None, upload=True, ingest=True): assert self.expect_rows >= n max_iters = 10 @@ -132,22 +150,28 @@ def churn_rows(self, n, pageserver_id: Optional[int] = None, upload=True): ] ) - for tenant_shard_id, pageserver in tenant_get_shards( - self.env, self.tenant_id, pageserver_id - ): - last_flush_lsn = wait_for_last_flush_lsn( - self.env, endpoint, self.tenant_id, self.timeline_id, pageserver_id=pageserver_id - ) - ps_http = pageserver.http_client() - wait_for_last_record_lsn(ps_http, tenant_shard_id, self.timeline_id, last_flush_lsn) - - if upload: - # force a checkpoint to trigger upload - ps_http.timeline_checkpoint(tenant_shard_id, self.timeline_id) - wait_for_upload(ps_http, tenant_shard_id, self.timeline_id, last_flush_lsn) - log.info(f"Churn: waiting for remote LSN {last_flush_lsn}") - else: - log.info(f"Churn: not waiting for upload, disk LSN {last_flush_lsn}") + if ingest: + # Wait for written data to be ingested by the pageserver + for tenant_shard_id, pageserver in tenant_get_shards( + self.env, self.tenant_id, pageserver_id + ): + last_flush_lsn = wait_for_last_flush_lsn( + self.env, + endpoint, + self.tenant_id, + self.timeline_id, + pageserver_id=pageserver_id, + ) + ps_http = pageserver.http_client() + wait_for_last_record_lsn(ps_http, tenant_shard_id, self.timeline_id, last_flush_lsn) + + if upload: + # Wait for written data to be uploaded to S3 (force a checkpoint to trigger upload) + ps_http.timeline_checkpoint(tenant_shard_id, self.timeline_id) + wait_for_upload(ps_http, tenant_shard_id, self.timeline_id, last_flush_lsn) + log.info(f"Churn: waiting for remote LSN {last_flush_lsn}") + else: + log.info(f"Churn: not waiting for upload, disk LSN {last_flush_lsn}") def validate(self, pageserver_id: Optional[int] = None): endpoint = self.endpoint(pageserver_id) diff --git a/test_runner/regress/test_sharding.py b/test_runner/regress/test_sharding.py index 9309af066b8e..bdb9990a5111 100644 --- a/test_runner/regress/test_sharding.py +++ b/test_runner/regress/test_sharding.py @@ -1,10 +1,14 @@ import os -from typing import Dict, List, Union +from typing import Dict, List, Optional, Union import pytest +import requests +from fixtures.compute_reconfigure import ComputeReconfigure from fixtures.log_helper import log from fixtures.neon_fixtures import ( + NeonEnv, NeonEnvBuilder, + StorageControllerApiException, tenant_get_shards, ) from fixtures.remote_storage import s3_storage @@ -495,3 +499,337 @@ def test_sharding_ingest( # Each shard may emit up to one huge layer, because initdb ingest doesn't respect checkpoint_distance. assert huge_layer_count <= shard_count + + +class Failure: + pageserver_id: Optional[int] + + def apply(self, env: NeonEnv): + raise NotImplementedError() + + def clear(self, env: NeonEnv): + """ + Clear the failure, in a way that should enable the system to proceed + to a totally clean state (all nodes online and reconciled) + """ + raise NotImplementedError() + + def expect_available(self): + raise NotImplementedError() + + def can_mitigate(self): + """Whether Self.mitigate is available for use""" + return False + + def mitigate(self, env: NeonEnv): + """ + Mitigate the failure in a way that should allow shard split to + complete and service to resume, but does not guarantee to leave + the whole world in a clean state (e.g. an Offline node might have + junk LocationConfigs on it) + """ + raise NotImplementedError() + + def fails_forward(self, env: NeonEnv): + """ + If true, this failure results in a state that eventualy completes the split. + """ + return False + + def expect_exception(self): + """ + How do we expect a call to the split API to fail? + """ + return StorageControllerApiException + + +class PageserverFailpoint(Failure): + def __init__(self, failpoint, pageserver_id, mitigate): + self.failpoint = failpoint + self.pageserver_id = pageserver_id + self._mitigate = mitigate + + def apply(self, env: NeonEnv): + pageserver = env.get_pageserver(self.pageserver_id) + pageserver.allowed_errors.extend( + [".*failpoint.*", ".*Resetting.*after shard split failure.*"] + ) + pageserver.http_client().configure_failpoints((self.failpoint, "return(1)")) + + def clear(self, env: NeonEnv): + pageserver = env.get_pageserver(self.pageserver_id) + pageserver.http_client().configure_failpoints((self.failpoint, "off")) + if self._mitigate: + env.storage_controller.node_configure(self.pageserver_id, {"availability": "Active"}) + + def expect_available(self): + return True + + def can_mitigate(self): + return self._mitigate + + def mitigate(self, env): + env.storage_controller.node_configure(self.pageserver_id, {"availability": "Offline"}) + + +class StorageControllerFailpoint(Failure): + def __init__(self, failpoint, action): + self.failpoint = failpoint + self.pageserver_id = None + self.action = action + + def apply(self, env: NeonEnv): + env.storage_controller.configure_failpoints((self.failpoint, self.action)) + + def clear(self, env: NeonEnv): + if "panic" in self.action: + log.info("Restarting storage controller after panic") + env.storage_controller.stop() + env.storage_controller.start() + else: + env.storage_controller.configure_failpoints((self.failpoint, "off")) + + def expect_available(self): + # Controller panics _do_ leave pageservers available, but our test code relies + # on using the locate API to update configurations in Workload, so we must skip + # these actions when the controller has been panicked. + return "panic" not in self.action + + def can_mitigate(self): + return False + + def fails_forward(self, env): + # Edge case: the very last failpoint that simulates a DB connection error, where + # the abort path will fail-forward and result in a complete split. + fail_forward = self.failpoint == "shard-split-post-complete" + + # If the failure was a panic, then if we expect split to eventually (after restart) + # complete, we must restart before checking that. + if fail_forward and "panic" in self.action: + log.info("Restarting storage controller after panic") + env.storage_controller.stop() + env.storage_controller.start() + + return fail_forward + + def expect_exception(self): + if "panic" in self.action: + return requests.exceptions.ConnectionError + else: + return StorageControllerApiException + + +class NodeKill(Failure): + def __init__(self, pageserver_id, mitigate): + self.pageserver_id = pageserver_id + self._mitigate = mitigate + + def apply(self, env: NeonEnv): + pageserver = env.get_pageserver(self.pageserver_id) + pageserver.stop(immediate=True) + + def clear(self, env: NeonEnv): + pageserver = env.get_pageserver(self.pageserver_id) + pageserver.start() + + def expect_available(self): + return False + + def mitigate(self, env): + env.storage_controller.node_configure(self.pageserver_id, {"availability": "Offline"}) + + +class CompositeFailure(Failure): + """ + Wrapper for failures in multiple components (e.g. a failpoint in the storage controller, *and* + stop a pageserver to interfere with rollback) + """ + + def __init__(self, failures: list[Failure]): + self.failures = failures + + self.pageserver_id = None + for f in failures: + if f.pageserver_id is not None: + self.pageserver_id = f.pageserver_id + break + + def apply(self, env: NeonEnv): + for f in self.failures: + f.apply(env) + + def clear(self, env): + for f in self.failures: + f.clear(env) + + def expect_available(self): + return all(f.expect_available() for f in self.failures) + + def mitigate(self, env): + for f in self.failures: + f.mitigate(env) + + def expect_exception(self): + expect = set(f.expect_exception() for f in self.failures) + + # We can't give a sensible response if our failures have different expectations + assert len(expect) == 1 + + return list(expect)[0] + + +@pytest.mark.parametrize( + "failure", + [ + PageserverFailpoint("api-500", 1, False), + NodeKill(1, False), + PageserverFailpoint("api-500", 1, True), + NodeKill(1, True), + PageserverFailpoint("shard-split-pre-prepare", 1, False), + PageserverFailpoint("shard-split-post-prepare", 1, False), + PageserverFailpoint("shard-split-pre-hardlink", 1, False), + PageserverFailpoint("shard-split-post-hardlink", 1, False), + PageserverFailpoint("shard-split-post-child-conf", 1, False), + PageserverFailpoint("shard-split-lsn-wait", 1, False), + PageserverFailpoint("shard-split-pre-finish", 1, False), + StorageControllerFailpoint("shard-split-validation", "return(1)"), + StorageControllerFailpoint("shard-split-post-begin", "return(1)"), + StorageControllerFailpoint("shard-split-post-remote", "return(1)"), + StorageControllerFailpoint("shard-split-post-complete", "return(1)"), + StorageControllerFailpoint("shard-split-validation", "panic(failpoint)"), + StorageControllerFailpoint("shard-split-post-begin", "panic(failpoint)"), + StorageControllerFailpoint("shard-split-post-remote", "panic(failpoint)"), + StorageControllerFailpoint("shard-split-post-complete", "panic(failpoint)"), + CompositeFailure( + [NodeKill(1, True), StorageControllerFailpoint("shard-split-post-begin", "return(1)")] + ), + CompositeFailure( + [NodeKill(1, False), StorageControllerFailpoint("shard-split-post-begin", "return(1)")] + ), + ], +) +def test_sharding_split_failures( + neon_env_builder: NeonEnvBuilder, + compute_reconfigure_listener: ComputeReconfigure, + failure: Failure, +): + neon_env_builder.num_pageservers = 4 + neon_env_builder.control_plane_compute_hook_api = ( + compute_reconfigure_listener.control_plane_compute_hook_api + ) + 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 + + 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 + ps.allowed_errors.append(".*Dropped remote consistent LSN updates.*") + + # Make sure the node we're failing has a shard on it, otherwise the test isn't testing anything + assert ( + failure.pageserver_id is None + or len( + env.get_pageserver(failure.pageserver_id) + .http_client() + .tenant_list_locations()["tenant_shards"] + ) + > 0 + ) + + workload = Workload(env, tenant_id, timeline_id) + workload.init() + workload.write_rows(100) + + # Put the environment into a failing state (exact meaning depends on `failure`) + failure.apply(env) + + with pytest.raises(failure.expect_exception()): + env.storage_controller.tenant_shard_split(tenant_id, shard_count=4) + + # We expect that the overall operation will fail, but some split requests + # 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 + for ps in env.pageservers: + if exclude_ps_id is not None and ps.id == exclude_ps_id: + continue + + 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}") + assert tenant_shard_id.shard_count == initial_shard_count + count += 1 + assert count == initial_shard_count + + def assert_split_done(exclude_ps_id=None) -> None: + count = 0 + for ps in env.pageservers: + if exclude_ps_id is not None and ps.id == exclude_ps_id: + continue + + 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}") + assert tenant_shard_id.shard_count == split_shard_count + count += 1 + assert count == split_shard_count + + def finish_split(): + # Having failed+rolled back, we should be able to split again + # No failures this time; it will succeed + env.storage_controller.tenant_shard_split(tenant_id, shard_count=split_shard_count) + + workload.churn_rows(10) + 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) + workload.validate() + + if failure.fails_forward(env): + log.info("Fail-forward failure, checking split eventually completes...") + # A failure type which results in eventual completion of the split + wait_until(30, 1, assert_split_done) + elif failure.can_mitigate(): + log.info("Mitigating failure...") + # Mitigation phase: we expect to be able to proceed with a successful shard split + failure.mitigate(env) + + # The split should appear to be rolled back from the point of view of all pageservers + # apart from the one that is offline + wait_until(30, 1, lambda: assert_rolled_back(exclude_ps_id=failure.pageserver_id)) + + finish_split() + wait_until(30, 1, lambda: assert_split_done(exclude_ps_id=failure.pageserver_id)) + + # Having cleared the failure, everything should converge to a pristine state + failure.clear(env) + wait_until(30, 1, assert_split_done) + else: + # Once we restore the faulty pageserver's API to good health, rollback should + # eventually complete. + log.info("Clearing failure...") + failure.clear(env) + + wait_until(30, 1, assert_rolled_back) + + # Having rolled back, the tenant should be working + workload.churn_rows(10) + workload.validate() + + # Splitting again should work, since we cleared the failure + finish_split() + assert_split_done() + + env.storage_controller.consistency_check()