From b6ec11ad787fed21044a772f3a82cd3221413223 Mon Sep 17 00:00:00 2001 From: John Spray Date: Wed, 17 Jan 2024 18:01:08 +0000 Subject: [PATCH] control_plane: generalize attachment_service to handle sharding (#6251) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## Problem To test sharding, we need something to control it. We could write python code for doing this from the test runner, but this wouldn't be usable with neon_local run directly, and when we want to write tests with large number of shards/tenants, Rust is a better fit efficiently handling all the required state. This service enables automated tests to easily get a system with sharding/HA without the test itself having to set this all up by hand: existing tests can be run against sharded tenants just by setting a shard count when creating the tenant. ## Summary of changes Attachment service was previously a map of TenantId->TenantState, where the principal state stored for each tenant was the generation and the last attached pageserver. This enabled it to serve the re-attach and validate requests that the pageserver requires. In this PR, the scope of the service is extended substantially to do overall management of tenants in the pageserver, including tenant/timeline creation, live migration, evacuation of offline pageservers etc. This is done using synchronous code to make declarative changes to the tenant's intended state (`TenantState.policy` and `TenantState.intent`), which are then translated into calls into the pageserver by the `Reconciler`. Top level summary of modules within `control_plane/attachment_service/src`: - `tenant_state`: structure that represents one tenant shard. - `service`: implements the main high level such as tenant/timeline creation, marking a node offline, etc. - `scheduler`: for operations that need to pick a pageserver for a tenant, construct a scheduler and call into it. - `compute_hook`: receive notifications when a tenant shard is attached somewhere new. Once we have locations for all the shards in a tenant, emit an update to postgres configuration via the neon_local `LocalEnv`. - `http`: HTTP stubs. These mostly map to methods on `Service`, but are separated for readability and so that it'll be easier to adapt if/when we switch to another RPC layer. - `node`: structure that describes a pageserver node. The most important attribute of a node is its availability: marking a node offline causes tenant shards to reschedule away from it. This PR is a precursor to implementing the full sharding service for prod (#6342). What's the difference between this and a production-ready controller for pageservers? - JSON file persistence to be replaced with a database - Limited observability. - No concurrency limits. Marking a pageserver offline will try and migrate every tenant to a new pageserver concurrently, even if there are thousands. - Very simple scheduler that only knows to pick the pageserver with fewest tenants, and place secondary locations on a different pageserver than attached locations: it does not try to place shards for the same tenant on different pageservers. This matters little in tests, because picking the least-used pageserver usually results in round-robin placement. - Scheduler state is rebuilt exhaustively for each operation that requires a scheduler. - Relies on neon_local mechanisms for updating postgres: in production this would be something that flows through the real control plane. --------- Co-authored-by: Arpad MΓΌller --- Cargo.lock | 26 + Cargo.toml | 1 + control_plane/attachment_service/Cargo.toml | 32 + .../attachment_service/src/compute_hook.rs | 116 ++ control_plane/attachment_service/src/http.rs | 218 ++++ control_plane/attachment_service/src/lib.rs | 57 + control_plane/attachment_service/src/main.rs | 100 ++ control_plane/attachment_service/src/node.rs | 37 + .../attachment_service/src/persistence.rs | 272 ++++ .../attachment_service/src/reconciler.rs | 495 +++++++ .../attachment_service/src/scheduler.rs | 89 ++ .../attachment_service/src/service.rs | 1137 +++++++++++++++++ .../attachment_service/src/tenant_state.rs | 455 +++++++ control_plane/src/attachment_service.rs | 371 +++++- control_plane/src/bin/attachment_service.rs | 355 ----- control_plane/src/bin/neon_local.rs | 343 +++-- control_plane/src/endpoint.rs | 74 +- control_plane/src/lib.rs | 1 - control_plane/src/local_env.rs | 8 +- control_plane/src/pageserver.rs | 61 +- control_plane/src/tenant_migration.rs | 232 ---- libs/pageserver_api/src/models.rs | 8 +- libs/utils/src/id.rs | 8 + pageserver/client/src/mgmt_api.rs | 23 +- pageserver/src/tenant/mgr.rs | 16 +- .../src/tenant/remote_timeline_client.rs | 7 +- test_runner/fixtures/metrics.py | 1 + test_runner/fixtures/neon_fixtures.py | 282 +++- test_runner/fixtures/pageserver/http.py | 115 +- test_runner/fixtures/pageserver/utils.py | 22 +- test_runner/fixtures/workload.py | 38 +- test_runner/performance/test_bulk_insert.py | 2 +- test_runner/regress/test_broken_timeline.py | 7 +- .../regress/test_disk_usage_eviction.py | 19 +- .../regress/test_pageserver_generations.py | 20 +- test_runner/regress/test_tenant_relocation.py | 22 +- test_runner/regress/test_tenants.py | 4 +- test_runner/regress/test_timeline_size.py | 10 +- 38 files changed, 4162 insertions(+), 922 deletions(-) create mode 100644 control_plane/attachment_service/Cargo.toml create mode 100644 control_plane/attachment_service/src/compute_hook.rs create mode 100644 control_plane/attachment_service/src/http.rs create mode 100644 control_plane/attachment_service/src/lib.rs create mode 100644 control_plane/attachment_service/src/main.rs create mode 100644 control_plane/attachment_service/src/node.rs create mode 100644 control_plane/attachment_service/src/persistence.rs create mode 100644 control_plane/attachment_service/src/reconciler.rs create mode 100644 control_plane/attachment_service/src/scheduler.rs create mode 100644 control_plane/attachment_service/src/service.rs create mode 100644 control_plane/attachment_service/src/tenant_state.rs delete mode 100644 control_plane/src/bin/attachment_service.rs delete mode 100644 control_plane/src/tenant_migration.rs diff --git a/Cargo.lock b/Cargo.lock index d4ebdd3cc4..a5b4953624 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -270,6 +270,32 @@ dependencies = [ "critical-section", ] +[[package]] +name = "attachment_service" +version = "0.1.0" +dependencies = [ + "anyhow", + "camino", + "clap", + "control_plane", + "futures", + "git-version", + "hyper", + "metrics", + "pageserver_api", + "pageserver_client", + "postgres_backend", + "postgres_connection", + "serde", + "serde_json", + "thiserror", + "tokio", + "tokio-util", + "tracing", + "utils", + "workspace_hack", +] + [[package]] name = "autocfg" version = "1.1.0" diff --git a/Cargo.toml b/Cargo.toml index 2d8fbaffa8..5d5d2f4a55 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -3,6 +3,7 @@ resolver = "2" members = [ "compute_tools", "control_plane", + "control_plane/attachment_service", "pageserver", "pageserver/ctl", "pageserver/client", diff --git a/control_plane/attachment_service/Cargo.toml b/control_plane/attachment_service/Cargo.toml new file mode 100644 index 0000000000..2e2286dbab --- /dev/null +++ b/control_plane/attachment_service/Cargo.toml @@ -0,0 +1,32 @@ +[package] +name = "attachment_service" +version = "0.1.0" +edition.workspace = true +license.workspace = true + +[dependencies] +anyhow.workspace = true +camino.workspace = true +clap.workspace = true +futures.workspace = true +git-version.workspace = true +hyper.workspace = true +pageserver_api.workspace = true +pageserver_client.workspace = true +postgres_connection.workspace = true +serde.workspace = true +serde_json.workspace = true +thiserror.workspace = true +tokio.workspace = true +tokio-util.workspace = true +tracing.workspace = true + +# TODO: remove this after DB persistence is added, it is only used for +# a parsing function when loading pageservers from neon_local LocalEnv +postgres_backend.workspace = true + +utils = { path = "../../libs/utils/" } +metrics = { path = "../../libs/metrics/" } +control_plane = { path = ".." } +workspace_hack = { version = "0.1", path = "../../workspace_hack" } + diff --git a/control_plane/attachment_service/src/compute_hook.rs b/control_plane/attachment_service/src/compute_hook.rs new file mode 100644 index 0000000000..02617cd065 --- /dev/null +++ b/control_plane/attachment_service/src/compute_hook.rs @@ -0,0 +1,116 @@ +use std::collections::HashMap; + +use control_plane::endpoint::ComputeControlPlane; +use control_plane::local_env::LocalEnv; +use pageserver_api::shard::{ShardCount, ShardIndex, TenantShardId}; +use postgres_connection::parse_host_port; +use utils::id::{NodeId, TenantId}; + +pub(super) struct ComputeHookTenant { + shards: Vec<(ShardIndex, NodeId)>, +} + +impl ComputeHookTenant { + pub(super) async fn maybe_reconfigure(&mut self, tenant_id: TenantId) -> anyhow::Result<()> { + // Find the highest shard count and drop any shards that aren't + // for that shard count. + let shard_count = self.shards.iter().map(|(k, _v)| k.shard_count).max(); + let Some(shard_count) = shard_count else { + // No shards, nothing to do. + tracing::info!("ComputeHookTenant::maybe_reconfigure: no shards"); + return Ok(()); + }; + + self.shards.retain(|(k, _v)| k.shard_count == shard_count); + self.shards + .sort_by_key(|(shard, _node_id)| shard.shard_number); + + if self.shards.len() == shard_count.0 as usize || shard_count == ShardCount(0) { + // We have pageservers for all the shards: proceed to reconfigure compute + let env = match LocalEnv::load_config() { + Ok(e) => e, + Err(e) => { + tracing::warn!( + "Couldn't load neon_local config, skipping compute update ({e})" + ); + return Ok(()); + } + }; + let cplane = ComputeControlPlane::load(env.clone()) + .expect("Error loading compute control plane"); + + let compute_pageservers = self + .shards + .iter() + .map(|(_shard, node_id)| { + let ps_conf = env + .get_pageserver_conf(*node_id) + .expect("Unknown pageserver"); + let (pg_host, pg_port) = parse_host_port(&ps_conf.listen_pg_addr) + .expect("Unable to parse listen_pg_addr"); + (pg_host, pg_port.unwrap_or(5432)) + }) + .collect::>(); + + for (endpoint_name, endpoint) in &cplane.endpoints { + if endpoint.tenant_id == tenant_id && endpoint.status() == "running" { + tracing::info!("πŸ” Reconfiguring endpoint {}", endpoint_name,); + endpoint.reconfigure(compute_pageservers.clone()).await?; + } + } + } else { + tracing::info!( + "ComputeHookTenant::maybe_reconfigure: not enough shards ({}/{})", + self.shards.len(), + shard_count.0 + ); + } + + Ok(()) + } +} + +/// The compute hook is a destination for notifications about changes to tenant:pageserver +/// mapping. It aggregates updates for the shards in a tenant, and when appropriate reconfigures +/// the compute connection string. +pub(super) struct ComputeHook { + state: tokio::sync::Mutex>, +} + +impl ComputeHook { + pub(super) fn new() -> Self { + Self { + state: Default::default(), + } + } + + pub(super) async fn notify( + &self, + tenant_shard_id: TenantShardId, + node_id: NodeId, + ) -> anyhow::Result<()> { + tracing::info!("ComputeHook::notify: {}->{}", tenant_shard_id, node_id); + let mut locked = self.state.lock().await; + let entry = locked + .entry(tenant_shard_id.tenant_id) + .or_insert_with(|| ComputeHookTenant { shards: Vec::new() }); + + let shard_index = ShardIndex { + shard_count: tenant_shard_id.shard_count, + shard_number: tenant_shard_id.shard_number, + }; + + let mut set = false; + for (existing_shard, existing_node) in &mut entry.shards { + if *existing_shard == shard_index { + *existing_node = node_id; + set = true; + } + } + if !set { + entry.shards.push((shard_index, node_id)); + } + + entry.maybe_reconfigure(tenant_shard_id.tenant_id).await + } +} diff --git a/control_plane/attachment_service/src/http.rs b/control_plane/attachment_service/src/http.rs new file mode 100644 index 0000000000..30f6dd66ee --- /dev/null +++ b/control_plane/attachment_service/src/http.rs @@ -0,0 +1,218 @@ +use crate::reconciler::ReconcileError; +use crate::service::Service; +use hyper::{Body, Request, Response}; +use hyper::{StatusCode, Uri}; +use pageserver_api::models::{TenantCreateRequest, TimelineCreateRequest}; +use pageserver_api::shard::TenantShardId; +use std::sync::Arc; +use utils::auth::SwappableJwtAuth; +use utils::http::endpoint::{auth_middleware, request_span}; +use utils::http::request::parse_request_param; +use utils::id::TenantId; + +use utils::{ + http::{ + endpoint::{self}, + error::ApiError, + json::{json_request, json_response}, + RequestExt, RouterBuilder, + }, + id::NodeId, +}; + +use pageserver_api::control_api::{ReAttachRequest, ValidateRequest}; + +use control_plane::attachment_service::{ + AttachHookRequest, InspectRequest, NodeConfigureRequest, NodeRegisterRequest, + TenantShardMigrateRequest, +}; + +/// State available to HTTP request handlers +#[derive(Clone)] +pub struct HttpState { + service: Arc, + auth: Option>, + allowlist_routes: Vec, +} + +impl HttpState { + pub fn new(service: Arc, auth: Option>) -> Self { + let allowlist_routes = ["/status"] + .iter() + .map(|v| v.parse().unwrap()) + .collect::>(); + Self { + service, + auth, + allowlist_routes, + } + } +} + +#[inline(always)] +fn get_state(request: &Request) -> &HttpState { + request + .data::>() + .expect("unknown state type") + .as_ref() +} + +/// Pageserver calls into this on startup, to learn which tenants it should attach +async fn handle_re_attach(mut req: Request) -> Result, ApiError> { + let reattach_req = json_request::(&mut req).await?; + let state = get_state(&req); + json_response( + StatusCode::OK, + state + .service + .re_attach(reattach_req) + .await + .map_err(ApiError::InternalServerError)?, + ) +} + +/// Pageserver calls into this before doing deletions, to confirm that it still +/// holds the latest generation for the tenants with deletions enqueued +async fn handle_validate(mut req: Request) -> Result, ApiError> { + let validate_req = json_request::(&mut req).await?; + let state = get_state(&req); + json_response(StatusCode::OK, state.service.validate(validate_req)) +} + +/// Call into this before attaching a tenant to a pageserver, to acquire a generation number +/// (in the real control plane this is unnecessary, because the same program is managing +/// generation numbers and doing attachments). +async fn handle_attach_hook(mut req: Request) -> Result, ApiError> { + let attach_req = json_request::(&mut req).await?; + let state = get_state(&req); + + json_response( + StatusCode::OK, + state + .service + .attach_hook(attach_req) + .await + .map_err(ApiError::InternalServerError)?, + ) +} + +async fn handle_inspect(mut req: Request) -> Result, ApiError> { + let inspect_req = json_request::(&mut req).await?; + + let state = get_state(&req); + + json_response(StatusCode::OK, state.service.inspect(inspect_req)) +} + +async fn handle_tenant_create(mut req: Request) -> Result, ApiError> { + let create_req = json_request::(&mut req).await?; + let state = get_state(&req); + json_response( + StatusCode::OK, + state.service.tenant_create(create_req).await?, + ) +} + +async fn handle_tenant_timeline_create(mut req: Request) -> Result, ApiError> { + let tenant_id: TenantId = parse_request_param(&req, "tenant_id")?; + let create_req = json_request::(&mut req).await?; + + let state = get_state(&req); + json_response( + StatusCode::OK, + state + .service + .tenant_timeline_create(tenant_id, create_req) + .await?, + ) +} + +async fn handle_tenant_locate(req: Request) -> Result, ApiError> { + let tenant_id: TenantId = parse_request_param(&req, "tenant_id")?; + let state = get_state(&req); + + json_response(StatusCode::OK, state.service.tenant_locate(tenant_id)?) +} + +async fn handle_node_register(mut req: Request) -> Result, ApiError> { + let register_req = json_request::(&mut req).await?; + let state = get_state(&req); + state.service.node_register(register_req).await?; + json_response(StatusCode::OK, ()) +} + +async fn handle_node_configure(mut req: Request) -> Result, ApiError> { + let node_id: NodeId = parse_request_param(&req, "node_id")?; + let config_req = json_request::(&mut req).await?; + if node_id != config_req.node_id { + return Err(ApiError::BadRequest(anyhow::anyhow!( + "Path and body node_id differ" + ))); + } + let state = get_state(&req); + + json_response(StatusCode::OK, state.service.node_configure(config_req)?) +} + +async fn handle_tenant_shard_migrate(mut req: Request) -> Result, ApiError> { + let tenant_shard_id: TenantShardId = parse_request_param(&req, "tenant_shard_id")?; + let migrate_req = json_request::(&mut req).await?; + let state = get_state(&req); + json_response( + StatusCode::OK, + state + .service + .tenant_shard_migrate(tenant_shard_id, migrate_req) + .await?, + ) +} + +/// Status endpoint is just used for checking that our HTTP listener is up +async fn handle_status(_req: Request) -> Result, ApiError> { + json_response(StatusCode::OK, ()) +} + +impl From for ApiError { + fn from(value: ReconcileError) -> Self { + ApiError::Conflict(format!("Reconciliation error: {}", value)) + } +} + +pub fn make_router( + service: Arc, + auth: Option>, +) -> RouterBuilder { + let mut router = endpoint::make_router(); + if auth.is_some() { + router = router.middleware(auth_middleware(|request| { + let state = get_state(request); + if state.allowlist_routes.contains(request.uri()) { + None + } else { + state.auth.as_deref() + } + })) + } + + router + .data(Arc::new(HttpState::new(service, auth))) + .get("/status", |r| request_span(r, handle_status)) + .post("/re-attach", |r| request_span(r, handle_re_attach)) + .post("/validate", |r| request_span(r, handle_validate)) + .post("/attach-hook", |r| request_span(r, handle_attach_hook)) + .post("/inspect", |r| request_span(r, handle_inspect)) + .post("/node", |r| request_span(r, handle_node_register)) + .put("/node/:node_id/config", |r| { + request_span(r, handle_node_configure) + }) + .post("/tenant", |r| request_span(r, handle_tenant_create)) + .post("/tenant/:tenant_id/timeline", |r| { + request_span(r, handle_tenant_timeline_create) + }) + .get("/tenant/:tenant_id/locate", |r| { + request_span(r, handle_tenant_locate) + }) + .put("/tenant/:tenant_shard_id/migrate", |r| { + request_span(r, handle_tenant_shard_migrate) + }) +} diff --git a/control_plane/attachment_service/src/lib.rs b/control_plane/attachment_service/src/lib.rs new file mode 100644 index 0000000000..d8f996952a --- /dev/null +++ b/control_plane/attachment_service/src/lib.rs @@ -0,0 +1,57 @@ +use serde::{Deserialize, Serialize}; +use utils::seqwait::MonotonicCounter; + +mod compute_hook; +pub mod http; +mod node; +pub mod persistence; +mod reconciler; +mod scheduler; +pub mod service; +mod tenant_state; + +#[derive(Clone, Serialize, Deserialize)] +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), +} + +#[derive(Ord, PartialOrd, Eq, PartialEq, Copy, Clone)] +struct Sequence(u64); + +impl Sequence { + fn initial() -> Self { + Self(0) + } +} + +impl std::fmt::Display for Sequence { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "{}", self.0) + } +} + +impl MonotonicCounter for Sequence { + fn cnt_advance(&mut self, v: Sequence) { + assert!(*self <= v); + *self = v; + } + fn cnt_value(&self) -> Sequence { + *self + } +} + +impl Sequence { + fn next(&self) -> Sequence { + Sequence(self.0 + 1) + } +} + +impl Default for PlacementPolicy { + fn default() -> Self { + PlacementPolicy::Double(1) + } +} diff --git a/control_plane/attachment_service/src/main.rs b/control_plane/attachment_service/src/main.rs new file mode 100644 index 0000000000..ee2a22ee53 --- /dev/null +++ b/control_plane/attachment_service/src/main.rs @@ -0,0 +1,100 @@ +/// 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; +use attachment_service::http::make_router; +use attachment_service::persistence::Persistence; +use attachment_service::service::{Config, Service}; +use camino::Utf8PathBuf; +use clap::Parser; +use metrics::launch_timestamp::LaunchTimestamp; +use std::sync::Arc; +use utils::auth::{JwtAuth, SwappableJwtAuth}; +use utils::logging::{self, LogFormat}; +use utils::signals::{ShutdownSignals, Signal}; + +use utils::{project_build_tag, project_git_version, tcp_listener}; + +project_git_version!(GIT_VERSION); +project_build_tag!(BUILD_TAG); + +#[derive(Parser)] +#[command(author, version, about, long_about = None)] +#[command(arg_required_else_help(true))] +struct Cli { + /// Host and port to listen on, like `127.0.0.1:1234` + #[arg(short, long)] + listen: std::net::SocketAddr, + + /// Path to public key for JWT authentication of clients + #[arg(long)] + public_key: Option, + + /// Token for authenticating this service with the pageservers it controls + #[arg(short, long)] + jwt_token: Option, + + /// Path to the .json file to store state (will be created if it doesn't exist) + #[arg(short, long)] + path: Utf8PathBuf, +} + +#[tokio::main] +async fn main() -> anyhow::Result<()> { + let launch_ts = Box::leak(Box::new(LaunchTimestamp::generate())); + + logging::init( + LogFormat::Plain, + logging::TracingErrorLayerEnablement::Disabled, + logging::Output::Stdout, + )?; + + let args = Cli::parse(); + tracing::info!( + "version: {}, launch_timestamp: {}, build_tag {}, state at {}, listening on {}", + GIT_VERSION, + launch_ts.to_string(), + BUILD_TAG, + args.path, + args.listen + ); + + let config = Config { + jwt_token: args.jwt_token, + }; + + let persistence = Arc::new(Persistence::new(&args.path).await); + + let service = Service::spawn(config, persistence).await?; + + let http_listener = tcp_listener::bind(args.listen)?; + + let auth = if let Some(public_key_path) = &args.public_key { + let jwt_auth = JwtAuth::from_key_path(public_key_path)?; + Some(Arc::new(SwappableJwtAuth::new(jwt_auth))) + } else { + None + }; + let router = make_router(service, auth) + .build() + .map_err(|err| anyhow!(err))?; + let service = utils::http::RouterService::new(router).unwrap(); + let server = hyper::Server::from_tcp(http_listener)?.serve(service); + + tracing::info!("Serving on {0}", args.listen); + + tokio::task::spawn(server); + + ShutdownSignals::handle(|signal| match signal { + Signal::Interrupt | Signal::Terminate | Signal::Quit => { + tracing::info!("Got {}. Terminating", signal.name()); + // We're just a test helper: no graceful shutdown. + std::process::exit(0); + } + })?; + + Ok(()) +} diff --git a/control_plane/attachment_service/src/node.rs b/control_plane/attachment_service/src/node.rs new file mode 100644 index 0000000000..efd3f8f49b --- /dev/null +++ b/control_plane/attachment_service/src/node.rs @@ -0,0 +1,37 @@ +use control_plane::attachment_service::{NodeAvailability, NodeSchedulingPolicy}; +use utils::id::NodeId; + +#[derive(Clone)] +pub(crate) struct Node { + pub(crate) id: NodeId, + + pub(crate) availability: NodeAvailability, + pub(crate) scheduling: NodeSchedulingPolicy, + + pub(crate) listen_http_addr: String, + pub(crate) listen_http_port: u16, + + pub(crate) listen_pg_addr: String, + pub(crate) listen_pg_port: u16, +} + +impl Node { + pub(crate) fn base_url(&self) -> String { + format!("http://{}:{}", self.listen_http_addr, self.listen_http_port) + } + + /// Is this node elegible to have work scheduled onto it? + pub(crate) fn may_schedule(&self) -> bool { + match self.availability { + NodeAvailability::Active => {} + NodeAvailability::Offline => return false, + } + + match self.scheduling { + NodeSchedulingPolicy::Active => true, + NodeSchedulingPolicy::Draining => false, + NodeSchedulingPolicy::Filling => true, + NodeSchedulingPolicy::Pause => false, + } + } +} diff --git a/control_plane/attachment_service/src/persistence.rs b/control_plane/attachment_service/src/persistence.rs new file mode 100644 index 0000000000..58708be140 --- /dev/null +++ b/control_plane/attachment_service/src/persistence.rs @@ -0,0 +1,272 @@ +use std::{collections::HashMap, str::FromStr}; + +use camino::{Utf8Path, Utf8PathBuf}; +use control_plane::{ + attachment_service::{NodeAvailability, NodeSchedulingPolicy}, + local_env::LocalEnv, +}; +use pageserver_api::{ + models::TenantConfig, + shard::{ShardCount, ShardNumber, TenantShardId}, +}; +use postgres_connection::parse_host_port; +use serde::{Deserialize, Serialize}; +use utils::{ + generation::Generation, + id::{NodeId, TenantId}, +}; + +use crate::{node::Node, PlacementPolicy}; + +/// Placeholder for storage. This will be replaced with a database client. +pub struct Persistence { + state: std::sync::Mutex, +} + +// Top level state available to all HTTP handlers +#[derive(Serialize, Deserialize)] +struct PersistentState { + tenants: HashMap, + + #[serde(skip)] + path: Utf8PathBuf, +} + +/// A convenience for serializing the state inside a sync lock, and then +/// writing it to disk outside of the lock. This will go away when switching +/// to a database backend. +struct PendingWrite { + bytes: Vec, + path: Utf8PathBuf, +} + +impl PendingWrite { + async fn commit(&self) -> anyhow::Result<()> { + tokio::fs::write(&self.path, &self.bytes).await?; + + Ok(()) + } +} + +impl PersistentState { + fn save(&self) -> PendingWrite { + PendingWrite { + bytes: serde_json::to_vec(self).expect("Serialization error"), + path: self.path.clone(), + } + } + + async fn load(path: &Utf8Path) -> anyhow::Result { + let bytes = tokio::fs::read(path).await?; + let mut decoded = serde_json::from_slice::(&bytes)?; + decoded.path = path.to_owned(); + + for (tenant_id, tenant) in &mut decoded.tenants { + // Backward compat: an old attachments.json from before PR #6251, replace + // empty strings with proper defaults. + if tenant.tenant_id.is_empty() { + tenant.tenant_id = format!("{}", tenant_id); + tenant.config = serde_json::to_string(&TenantConfig::default())?; + tenant.placement_policy = serde_json::to_string(&PlacementPolicy::default())?; + } + } + + Ok(decoded) + } + + async fn load_or_new(path: &Utf8Path) -> Self { + match Self::load(path).await { + Ok(s) => { + tracing::info!("Loaded state file at {}", path); + s + } + Err(e) + if e.downcast_ref::() + .map(|e| e.kind() == std::io::ErrorKind::NotFound) + .unwrap_or(false) => + { + tracing::info!("Will create state file at {}", path); + Self { + tenants: HashMap::new(), + path: path.to_owned(), + } + } + Err(e) => { + panic!("Failed to load state from '{}': {e:#} (maybe your .neon/ dir was written by an older version?)", path) + } + } + } +} + +impl Persistence { + pub async fn new(path: &Utf8Path) -> Self { + let state = PersistentState::load_or_new(path).await; + Self { + state: std::sync::Mutex::new(state), + } + } + + /// When registering a node, persist it so that on next start we will be able to + /// iterate over known nodes to synchronize their tenant shard states with our observed state. + pub(crate) async fn insert_node(&self, _node: &Node) -> anyhow::Result<()> { + // TODO: node persitence will come with database backend + Ok(()) + } + + /// At startup, we populate the service's list of nodes, and use this list to call into + /// each node to do an initial reconciliation of the state of the world with our in-memory + /// observed state. + pub(crate) async fn list_nodes(&self) -> anyhow::Result> { + let env = LocalEnv::load_config()?; + // TODO: node persitence will come with database backend + + // XXX hack: enable test_backward_compatibility to work by populating our list of + // nodes from LocalEnv when it is not present in persistent storage. Otherwise at + // first startup in the compat test, we may have shards but no nodes. + let mut result = Vec::new(); + tracing::info!( + "Loaded {} pageserver nodes from LocalEnv", + env.pageservers.len() + ); + for ps_conf in env.pageservers { + let (pg_host, pg_port) = + parse_host_port(&ps_conf.listen_pg_addr).expect("Unable to parse listen_pg_addr"); + let (http_host, http_port) = parse_host_port(&ps_conf.listen_http_addr) + .expect("Unable to parse listen_http_addr"); + result.push(Node { + id: ps_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), + availability: NodeAvailability::Active, + scheduling: NodeSchedulingPolicy::Active, + }); + } + + Ok(result) + } + + /// At startup, we populate our map of tenant shards from persistent storage. + pub(crate) async fn list_tenant_shards(&self) -> anyhow::Result> { + let locked = self.state.lock().unwrap(); + Ok(locked.tenants.values().cloned().collect()) + } + + /// Tenants must be persisted before we schedule them for the first time. This enables us + /// to correctly retain generation monotonicity, and the externally provided placement policy & config. + pub(crate) async fn insert_tenant_shards( + &self, + shards: Vec, + ) -> anyhow::Result<()> { + let write = { + let mut locked = self.state.lock().unwrap(); + for shard in shards { + let tenant_shard_id = TenantShardId { + tenant_id: TenantId::from_str(shard.tenant_id.as_str())?, + shard_number: ShardNumber(shard.shard_number as u8), + shard_count: ShardCount(shard.shard_count as u8), + }; + + locked.tenants.insert(tenant_shard_id, shard); + } + locked.save() + }; + + write.commit().await?; + + Ok(()) + } + + /// Reconciler calls this immediately before attaching to a new pageserver, to acquire a unique, monotonically + /// advancing generation number. We also store the NodeId for which the generation was issued, so that in + /// [`Self::re_attach`] we can do a bulk UPDATE on the generations for that node. + pub(crate) async fn increment_generation( + &self, + tenant_shard_id: TenantShardId, + node_id: Option, + ) -> anyhow::Result { + let (write, gen) = { + let mut locked = self.state.lock().unwrap(); + let Some(shard) = locked.tenants.get_mut(&tenant_shard_id) else { + anyhow::bail!("Tried to increment generation of unknown shard"); + }; + + // If we're called with a None pageserver, we need only update the generation + // record to disassociate it with this pageserver, not actually increment the number, as + // the increment is guaranteed to happen the next time this tenant is attached. + if node_id.is_some() { + shard.generation += 1; + } + + shard.generation_pageserver = node_id; + let gen = Generation::new(shard.generation); + (locked.save(), gen) + }; + + write.commit().await?; + Ok(gen) + } + + pub(crate) async fn re_attach( + &self, + node_id: NodeId, + ) -> anyhow::Result> { + let (write, result) = { + let mut result = HashMap::new(); + let mut locked = self.state.lock().unwrap(); + for (tenant_shard_id, shard) in locked.tenants.iter_mut() { + if shard.generation_pageserver == Some(node_id) { + shard.generation += 1; + result.insert(*tenant_shard_id, Generation::new(shard.generation)); + } + } + + (locked.save(), result) + }; + + write.commit().await?; + Ok(result) + } + + // TODO: when we start shard splitting, we must durably mark the tenant so that + // on restart, we know that we must go through recovery (list shards that exist + // and pick up where we left off and/or revert to parent shards). + #[allow(dead_code)] + pub(crate) async fn begin_shard_split(&self, _tenant_id: TenantId) -> anyhow::Result<()> { + todo!(); + } + + // TODO: when we finish shard splitting, we must atomically clean up the old shards + // and insert the new shards, and clear the splitting marker. + #[allow(dead_code)] + pub(crate) async fn complete_shard_split(&self, _tenant_id: TenantId) -> anyhow::Result<()> { + todo!(); + } +} + +/// Parts of [`crate::tenant_state::TenantState`] that are stored durably +#[derive(Serialize, Deserialize, Clone)] +pub(crate) struct TenantShardPersistence { + #[serde(default)] + pub(crate) tenant_id: String, + #[serde(default)] + pub(crate) shard_number: i32, + #[serde(default)] + pub(crate) shard_count: i32, + #[serde(default)] + pub(crate) shard_stripe_size: i32, + + // Currently attached pageserver + #[serde(rename = "pageserver")] + pub(crate) generation_pageserver: Option, + + // Latest generation number: next time we attach, increment this + // and use the incremented number when attaching + pub(crate) generation: u32, + + #[serde(default)] + pub(crate) placement_policy: String, + #[serde(default)] + pub(crate) config: String, +} diff --git a/control_plane/attachment_service/src/reconciler.rs b/control_plane/attachment_service/src/reconciler.rs new file mode 100644 index 0000000000..b08339b3b4 --- /dev/null +++ b/control_plane/attachment_service/src/reconciler.rs @@ -0,0 +1,495 @@ +use crate::persistence::Persistence; +use crate::service; +use control_plane::attachment_service::NodeAvailability; +use pageserver_api::models::{ + LocationConfig, LocationConfigMode, LocationConfigSecondary, TenantConfig, +}; +use pageserver_api::shard::{ShardIdentity, TenantShardId}; +use pageserver_client::mgmt_api; +use std::collections::HashMap; +use std::sync::Arc; +use std::time::Duration; +use tokio_util::sync::CancellationToken; +use utils::generation::Generation; +use utils::id::{NodeId, TimelineId}; +use utils::lsn::Lsn; + +use crate::compute_hook::ComputeHook; +use crate::node::Node; +use crate::tenant_state::{IntentState, ObservedState, ObservedStateLocation}; + +/// 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 { + /// See [`crate::tenant_state::TenantState`] for the meanings of these fields: they are a snapshot + /// of a tenant's state from when we spawned a reconcile task. + pub(super) tenant_shard_id: TenantShardId, + pub(crate) shard: ShardIdentity, + pub(crate) generation: Generation, + pub(crate) intent: IntentState, + 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 + pub(crate) compute_hook: Arc, + + /// A means to abort background reconciliation: it is essential to + /// call this when something changes in the original TenantState that + /// will make this reconciliation impossible or unnecessary, for + /// example when a pageserver node goes offline, or the PlacementPolicy for + /// the tenant is changed. + pub(crate) cancel: CancellationToken, + + /// Access to persistent storage for updating generation numbers + pub(crate) persistence: Arc, +} + +#[derive(thiserror::Error, Debug)] +pub enum ReconcileError { + #[error(transparent)] + Other(#[from] anyhow::Error), +} + +impl Reconciler { + async fn location_config( + &mut self, + node_id: NodeId, + config: LocationConfig, + flush_ms: Option, + ) -> anyhow::Result<()> { + let node = self + .pageservers + .get(&node_id) + .expect("Pageserver may not be removed while referenced"); + + 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) + .await?; + tracing::info!("location_config({}) complete: {:?}", node_id, config); + + self.observed + .locations + .insert(node.id, ObservedStateLocation { conf: Some(config) }); + + Ok(()) + } + + 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) { + 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 + } + None | Some(_) => { + // Attached or uncertain: don't do a live migration, proceed + // with a general-case reconciliation + tracing::info!("maybe_live_migrate: destination is None or attached"); + return Ok(()); + } + } + } + None => { + // Our destination is not attached: maybe live migrate if some other + // node is currently attached. Fall through. + node_id + } + } + } else { + // No intent to be attached + tracing::info!("maybe_live_migrate: no attached intent"); + return Ok(()); + }; + + let mut origin = None; + 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; + } + } + } + } + + let Some(origin) = origin else { + tracing::info!("maybe_live_migrate: no origin found"); + return Ok(()); + }; + + // 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?; + + Ok(()) + } + + async fn get_lsns( + &self, + tenant_shard_id: TenantShardId, + node_id: &NodeId, + ) -> 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()); + + let timelines = client.timeline_list(&tenant_shard_id).await?; + Ok(timelines + .into_iter() + .map(|t| (t.timeline_id, t.last_record_lsn)) + .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 await_lsn( + &self, + tenant_shard_id: TenantShardId, + pageserver_id: &NodeId, + baseline: HashMap, + ) -> anyhow::Result<()> { + loop { + let latest = match self.get_lsns(tenant_shard_id, pageserver_id).await { + Ok(l) => l, + Err(e) => { + println!( + "πŸ•‘ Can't get LSNs on pageserver {} yet, waiting ({e})", + pageserver_id + ); + std::thread::sleep(Duration::from_millis(500)); + continue; + } + }; + + let mut any_behind: bool = false; + for (timeline_id, baseline_lsn) in &baseline { + match latest.get(timeline_id) { + Some(latest_lsn) => { + println!("πŸ•‘ LSN origin {baseline_lsn} vs destination {latest_lsn}"); + if latest_lsn < baseline_lsn { + any_behind = true; + } + } + None => { + // Expected timeline isn't yet visible on migration destination. + // (IRL we would have to account for timeline deletion, but this + // is just test helper) + any_behind = true; + } + } + } + + if !any_behind { + println!("βœ… LSN caught up. Proceeding..."); + break; + } else { + std::thread::sleep(Duration::from_millis(500)); + } + } + + Ok(()) + } + + pub async fn live_migrate( + &mut self, + origin_ps_id: NodeId, + dest_ps_id: NodeId, + ) -> anyhow::Result<()> { + // `maybe_live_migrate` is responsibble for sanity of inputs + assert!(origin_ps_id != dest_ps_id); + + fn build_location_config( + shard: &ShardIdentity, + config: &TenantConfig, + mode: LocationConfigMode, + generation: Option, + secondary_conf: Option, + ) -> LocationConfig { + LocationConfig { + mode, + generation: generation.map(|g| g.into().unwrap()), + secondary_conf, + tenant_conf: config.clone(), + shard_number: shard.number.0, + shard_count: shard.count.0, + shard_stripe_size: shard.stripe_size.0, + } + } + + tracing::info!( + "πŸ” Switching origin pageserver {} to stale mode", + origin_ps_id + ); + + // 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) + let stale_conf = build_location_config( + &self.shard, + &self.config, + LocationConfigMode::AttachedStale, + Some(self.generation), + None, + ); + self.location_config(origin_ps_id, stale_conf, Some(Duration::from_secs(10))) + .await?; + + let baseline_lsns = Some(self.get_lsns(self.tenant_shard_id, &origin_ps_id).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) = &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; + } + } + } + + // Increment generation before attaching to new pageserver + self.generation = self + .persistence + .increment_generation(self.tenant_shard_id, Some(dest_ps_id)) + .await?; + + let dest_conf = build_location_config( + &self.shard, + &self.config, + LocationConfigMode::AttachedMulti, + Some(self.generation), + None, + ); + + tracing::info!("πŸ” Attaching to pageserver {}", dest_ps_id); + self.location_config(dest_ps_id, dest_conf, None).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) + .await?; + } + + tracing::info!("πŸ” Notifying compute to use pageserver {}", dest_ps_id); + self.compute_hook + .notify(self.tenant_shard_id, dest_ps_id) + .await?; + + // Downgrade the origin to secondary. If the tenant's policy is PlacementPolicy::Single, then + // this location will be deleted in the general case reconciliation that runs after this. + let origin_secondary_conf = build_location_config( + &self.shard, + &self.config, + LocationConfigMode::Secondary, + None, + Some(LocationConfigSecondary { warm: true }), + ); + self.location_config(origin_ps_id, origin_secondary_conf.clone(), None) + .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, + ObservedStateLocation { + conf: Some(origin_secondary_conf), + }, + ); + + println!( + "πŸ” Switching to AttachedSingle mode on pageserver {}", + dest_ps_id + ); + let dest_final_conf = build_location_config( + &self.shard, + &self.config, + LocationConfigMode::AttachedSingle, + Some(self.generation), + None, + ); + self.location_config(dest_ps_id, dest_final_conf.clone(), None) + .await?; + self.observed.locations.insert( + dest_ps_id, + ObservedStateLocation { + conf: Some(dest_final_conf), + }, + ); + + println!("βœ… Migration complete"); + + Ok(()) + } + + /// Reconciling a tenant makes API calls to pageservers until the observed state + /// matches the intended state. + /// + /// First we apply special case handling (e.g. for live migrations), and then a + /// 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. + + // 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 { + let mut wanted_conf = + attached_location_conf(self.generation, &self.shard, &self.config); + match self.observed.locations.get(&node_id) { + Some(conf) if conf.conf.as_ref() == Some(&wanted_conf) => { + // Nothing to do + tracing::info!("Observed configuration already correct.") + } + _ => { + // In all cases other than a matching observed configuration, we will + // reconcile this location. This includes locations with different configurations, as well + // as locations with unknown (None) observed state. + self.generation = self + .persistence + .increment_generation(self.tenant_shard_id, Some(node_id)) + .await?; + wanted_conf.generation = self.generation.into(); + tracing::info!("Observed configuration requires update."); + self.location_config(node_id, wanted_conf, None).await?; + if let Err(e) = self + .compute_hook + .notify(self.tenant_shard_id, node_id) + .await + { + tracing::warn!( + "Failed to notify compute of newly attached pageserver {node_id}: {e}" + ); + } + } + } + } + + // 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 { + let wanted_conf = secondary_location_conf(&self.shard, &self.config); + match self.observed.locations.get(node_id) { + Some(conf) if conf.conf.as_ref() == Some(&wanted_conf) => { + // Nothing to do + tracing::info!(%node_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)) + } + } + } + + // 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; + } + + changes.push(( + *node_id, + LocationConfig { + mode: LocationConfigMode::Detached, + generation: None, + secondary_conf: None, + shard_number: self.shard.number.0, + shard_count: self.shard.count.0, + shard_stripe_size: self.shard.stripe_size.0, + tenant_conf: self.config.clone(), + }, + )); + } + + for (node_id, conf) in changes { + self.location_config(node_id, conf, None).await?; + } + + Ok(()) + } +} + +pub(crate) fn attached_location_conf( + generation: Generation, + shard: &ShardIdentity, + config: &TenantConfig, +) -> LocationConfig { + LocationConfig { + mode: LocationConfigMode::AttachedSingle, + generation: generation.into(), + secondary_conf: None, + shard_number: shard.number.0, + shard_count: shard.count.0, + shard_stripe_size: shard.stripe_size.0, + tenant_conf: config.clone(), + } +} + +pub(crate) fn secondary_location_conf( + shard: &ShardIdentity, + config: &TenantConfig, +) -> LocationConfig { + LocationConfig { + mode: LocationConfigMode::Secondary, + generation: None, + secondary_conf: Some(LocationConfigSecondary { warm: true }), + shard_number: shard.number.0, + shard_count: shard.count.0, + shard_stripe_size: shard.stripe_size.0, + tenant_conf: config.clone(), + } +} diff --git a/control_plane/attachment_service/src/scheduler.rs b/control_plane/attachment_service/src/scheduler.rs new file mode 100644 index 0000000000..1966a7ea2a --- /dev/null +++ b/control_plane/attachment_service/src/scheduler.rs @@ -0,0 +1,89 @@ +use pageserver_api::shard::TenantShardId; +use std::collections::{BTreeMap, HashMap}; +use utils::{http::error::ApiError, id::NodeId}; + +use crate::{node::Node, tenant_state::TenantState}; + +/// Scenarios in which we cannot find a suitable location for a tenant shard +#[derive(thiserror::Error, Debug)] +pub enum ScheduleError { + #[error("No pageservers found")] + NoPageservers, + #[error("No pageserver found matching constraint")] + ImpossibleConstraint, +} + +impl From for ApiError { + fn from(value: ScheduleError) -> Self { + ApiError::Conflict(format!("Scheduling error: {}", value)) + } +} + +pub(crate) struct Scheduler { + tenant_counts: HashMap, +} + +impl Scheduler { + pub(crate) fn new( + tenants: &BTreeMap, + nodes: &HashMap, + ) -> Self { + let mut tenant_counts = HashMap::new(); + for node_id in nodes.keys() { + tenant_counts.insert(*node_id, 0); + } + + for tenant in tenants.values() { + if let Some(ps) = tenant.intent.attached { + let entry = tenant_counts.entry(ps).or_insert(0); + *entry += 1; + } + } + + for (node_id, node) in nodes { + if !node.may_schedule() { + tenant_counts.remove(node_id); + } + } + + Self { tenant_counts } + } + + pub(crate) fn schedule_shard( + &mut self, + hard_exclude: &[NodeId], + ) -> Result { + if self.tenant_counts.is_empty() { + return Err(ScheduleError::NoPageservers); + } + + let mut tenant_counts: Vec<(NodeId, usize)> = self + .tenant_counts + .iter() + .filter_map(|(k, v)| { + if hard_exclude.contains(k) { + None + } else { + Some((*k, *v)) + } + }) + .collect(); + + // Sort by tenant count. Nodes with the same tenant count are sorted by ID. + tenant_counts.sort_by_key(|i| (i.1, i.0)); + + if tenant_counts.is_empty() { + // After applying constraints, no pageservers were left + return Err(ScheduleError::ImpossibleConstraint); + } + + for (node_id, count) in &tenant_counts { + tracing::info!("tenant_counts[{node_id}]={count}"); + } + + let node_id = tenant_counts.first().unwrap().0; + tracing::info!("scheduler selected node {node_id}"); + *self.tenant_counts.get_mut(&node_id).unwrap() += 1; + Ok(node_id) + } +} diff --git a/control_plane/attachment_service/src/service.rs b/control_plane/attachment_service/src/service.rs new file mode 100644 index 0000000000..5999d48fd9 --- /dev/null +++ b/control_plane/attachment_service/src/service.rs @@ -0,0 +1,1137 @@ +use std::{ + collections::{BTreeMap, HashMap}, + str::FromStr, + sync::Arc, + time::{Duration, Instant}, +}; + +use control_plane::attachment_service::{ + AttachHookRequest, AttachHookResponse, InspectRequest, InspectResponse, NodeAvailability, + NodeConfigureRequest, NodeRegisterRequest, NodeSchedulingPolicy, TenantCreateResponse, + TenantCreateResponseShard, TenantLocateResponse, TenantLocateResponseShard, + TenantShardMigrateRequest, TenantShardMigrateResponse, +}; +use hyper::StatusCode; +use pageserver_api::{ + control_api::{ + ReAttachRequest, ReAttachResponse, ReAttachResponseTenant, ValidateRequest, + ValidateResponse, ValidateResponseTenant, + }, + models, + models::{ + LocationConfig, LocationConfigMode, ShardParameters, TenantConfig, TenantCreateRequest, + TimelineCreateRequest, TimelineInfo, + }, + shard::{ShardCount, ShardIdentity, ShardNumber, ShardStripeSize, TenantShardId}, +}; +use pageserver_client::mgmt_api; +use utils::{ + generation::Generation, + http::error::ApiError, + id::{NodeId, TenantId}, + seqwait::SeqWait, +}; + +use crate::{ + compute_hook::ComputeHook, + node::Node, + persistence::{Persistence, TenantShardPersistence}, + scheduler::Scheduler, + tenant_state::{ + IntentState, ObservedState, ObservedStateLocation, ReconcileResult, ReconcileWaitError, + ReconcilerWaiter, TenantState, + }, + PlacementPolicy, Sequence, +}; + +const RECONCILE_TIMEOUT: Duration = Duration::from_secs(30); + +// Top level state available to all HTTP handlers +struct ServiceState { + tenants: BTreeMap, + + nodes: Arc>, + + compute_hook: Arc, + + result_tx: tokio::sync::mpsc::UnboundedSender, +} + +impl ServiceState { + fn new( + result_tx: tokio::sync::mpsc::UnboundedSender, + nodes: HashMap, + tenants: BTreeMap, + ) -> Self { + Self { + tenants, + nodes: Arc::new(nodes), + compute_hook: Arc::new(ComputeHook::new()), + result_tx, + } + } +} + +#[derive(Clone)] +pub struct Config { + // All pageservers managed by one instance of this service must have + // the same public key. + pub jwt_token: Option, +} + +pub struct Service { + inner: Arc>, + config: Config, + persistence: Arc, +} + +impl From for ApiError { + fn from(value: ReconcileWaitError) -> Self { + match value { + ReconcileWaitError::Shutdown => ApiError::ShuttingDown, + e @ ReconcileWaitError::Timeout(_) => ApiError::Timeout(format!("{e}").into()), + e @ ReconcileWaitError::Failed(..) => ApiError::InternalServerError(anyhow::anyhow!(e)), + } + } +} + +impl Service { + pub async fn spawn(config: Config, persistence: Arc) -> anyhow::Result> { + let (result_tx, mut result_rx) = tokio::sync::mpsc::unbounded_channel(); + + tracing::info!("Loading nodes from database..."); + let mut nodes = persistence.list_nodes().await?; + tracing::info!("Loaded {} nodes from database.", nodes.len()); + + tracing::info!("Loading shards from database..."); + let tenant_shard_persistence = persistence.list_tenant_shards().await?; + tracing::info!( + "Loaded {} shards from database.", + tenant_shard_persistence.len() + ); + + let mut tenants = BTreeMap::new(); + + 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(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(tsp.shard_count as u8), + ShardStripeSize(tsp.shard_stripe_size as u32), + )? + }; + let new_tenant = TenantState { + tenant_shard_id, + shard: shard_identity, + sequence: Sequence::initial(), + // Note that we load generation, but don't care about generation_pageserver. We will either end up finding + // our existing attached location and it will match generation_pageserver, or we will attach somewhere new + // and update generation_pageserver in the process. + generation: Generation::new(tsp.generation), + policy: serde_json::from_str(&tsp.placement_policy).unwrap(), + intent: IntentState::new(), + observed: ObservedState::new(), + config: serde_json::from_str(&tsp.config).unwrap(), + reconciler: None, + waiter: Arc::new(SeqWait::new(Sequence::initial())), + error_waiter: Arc::new(SeqWait::new(Sequence::initial())), + last_error: Arc::default(), + }; + + tenants.insert(tenant_shard_id, new_tenant); + } + + // For all tenant shards, a vector of observed states on nodes (where None means + // indeterminate, same as in [`ObservedStateLocation`]) + let mut observed = HashMap::new(); + + // TODO: issue these requests concurrently + for node in &mut nodes { + let client = mgmt_api::Client::new(node.base_url(), config.jwt_token.as_deref()); + + tracing::info!("Scanning shards on node {}...", node.id); + match client.list_location_config().await { + Err(e) => { + tracing::warn!("Could not contact pageserver {} ({e})", node.id); + // TODO: be more tolerant, apply a generous 5-10 second timeout + // TODO: setting a node to Offline is a dramatic thing to do, and can + // prevent neon_local from starting up (it starts this service before + // any pageservers are running). It may make sense to give nodes + // a Pending state to accomodate this situation, and allow (but deprioritize) + // scheduling on Pending nodes. + //node.availability = NodeAvailability::Offline; + } + Ok(listing) => { + tracing::info!( + "Received {} shard statuses from pageserver {}, setting it to Active", + listing.tenant_shards.len(), + node.id + ); + node.availability = NodeAvailability::Active; + + for (tenant_shard_id, conf_opt) in listing.tenant_shards { + observed.insert(tenant_shard_id, (node.id, conf_opt)); + } + } + } + } + + let mut cleanup = Vec::new(); + + // Populate intent and observed states for all tenants, based on reported state on pageservers + 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 }); + } + + // State of nodes is now frozen, transform to a HashMap. + let mut nodes: HashMap = nodes.into_iter().map(|n| (n.id, n)).collect(); + + // Populate each tenant's intent state + let mut scheduler = Scheduler::new(&tenants, &nodes); + for (tenant_shard_id, tenant_state) in tenants.iter_mut() { + tenant_state.intent_from_observed(); + if let Err(e) = tenant_state.schedule(&mut 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 + // to clients. + tracing::error!("Failed to schedule tenant {tenant_shard_id} at startup: {e}"); + } + } + + // Clean up any tenants that were found on pageservers but are not known to us. + for (tenant_shard_id, node_id) in cleanup { + // A node reported a tenant_shard_id which is unknown to us: detach it. + let node = nodes + .get_mut(&node_id) + .expect("Always exists: only known nodes are scanned"); + + let client = mgmt_api::Client::new(node.base_url(), config.jwt_token.as_deref()); + match client + .location_config( + tenant_shard_id, + LocationConfig { + mode: LocationConfigMode::Detached, + generation: None, + secondary_conf: None, + shard_number: tenant_shard_id.shard_number.0, + shard_count: tenant_shard_id.shard_count.0, + shard_stripe_size: 0, + tenant_conf: models::TenantConfig::default(), + }, + None, + ) + .await + { + Ok(()) => { + tracing::info!( + "Detached unknown shard {tenant_shard_id} on pageserver {node_id}" + ); + } + Err(e) => { + // Non-fatal error: leaving a tenant shard behind that we are not managing shouldn't + // break anything. + tracing::error!( + "Failed to detach unknkown shard {tenant_shard_id} on pageserver {node_id}: {e}" + ); + } + } + } + + let shard_count = tenants.len(); + let this = Arc::new(Self { + inner: Arc::new(std::sync::RwLock::new(ServiceState::new( + result_tx, nodes, tenants, + ))), + config, + persistence, + }); + + let result_task_this = this.clone(); + tokio::task::spawn(async move { + while let Some(result) = result_rx.recv().await { + tracing::info!( + "Reconcile result for sequence {}, ok={}", + result.sequence, + result.result.is_ok() + ); + let mut locked = result_task_this.inner.write().unwrap(); + let Some(tenant) = locked.tenants.get_mut(&result.tenant_shard_id) else { + // A reconciliation result might race with removing a tenant: drop results for + // tenants that aren't in our map. + continue; + }; + + // Usually generation should only be updated via this path, so the max() isn't + // needed, but it is used to handle out-of-band updates via. e.g. test hook. + tenant.generation = std::cmp::max(tenant.generation, result.generation); + + match result.result { + Ok(()) => { + for (node_id, loc) in &result.observed.locations { + if let Some(conf) = &loc.conf { + tracing::info!( + "Updating observed location {}: {:?}", + node_id, + conf + ); + } else { + tracing::info!("Setting observed location {} to None", node_id,) + } + } + tenant.observed = result.observed; + tenant.waiter.advance(result.sequence); + } + Err(e) => { + tracing::warn!( + "Reconcile error on tenant {}: {}", + tenant.tenant_shard_id, + e + ); + + // Ordering: populate last_error before advancing error_seq, + // so that waiters will see the correct error after waiting. + *(tenant.last_error.lock().unwrap()) = format!("{e}"); + tenant.error_waiter.advance(result.sequence); + + for (node_id, o) in result.observed.locations { + tenant.observed.locations.insert(node_id, o); + } + } + } + } + }); + + // Finally, now that the service is up and running, launch reconcile operations for any tenants + // which require it: under normal circumstances this should only include tenants that were in some + // transient state before we restarted. + let reconcile_tasks = this.reconcile_all(); + tracing::info!("Startup complete, spawned {reconcile_tasks} reconciliation tasks ({shard_count} shards total)"); + + Ok(this) + } + + pub(crate) async fn attach_hook( + &self, + attach_req: AttachHookRequest, + ) -> anyhow::Result { + // This is a test hook. To enable using it on tenants that were created directly with + // the pageserver API (not via this service), we will auto-create any missing tenant + // shards with default state. + let insert = { + let locked = self.inner.write().unwrap(); + !locked.tenants.contains_key(&attach_req.tenant_shard_id) + }; + + if insert { + let tsp = TenantShardPersistence { + tenant_id: attach_req.tenant_shard_id.tenant_id.to_string(), + shard_number: attach_req.tenant_shard_id.shard_number.0 as i32, + shard_count: attach_req.tenant_shard_id.shard_count.0 as i32, + shard_stripe_size: 0, + generation: 0, + generation_pageserver: None, + placement_policy: serde_json::to_string(&PlacementPolicy::default()).unwrap(), + config: serde_json::to_string(&TenantConfig::default()).unwrap(), + }; + + self.persistence.insert_tenant_shards(vec![tsp]).await?; + + let mut locked = self.inner.write().unwrap(); + locked.tenants.insert( + attach_req.tenant_shard_id, + TenantState::new( + attach_req.tenant_shard_id, + ShardIdentity::unsharded(), + PlacementPolicy::Single, + ), + ); + } + + let new_generation = if attach_req.node_id.is_some() { + Some( + self.persistence + .increment_generation(attach_req.tenant_shard_id, attach_req.node_id) + .await?, + ) + } else { + None + }; + + let mut locked = self.inner.write().unwrap(); + let tenant_state = locked + .tenants + .get_mut(&attach_req.tenant_shard_id) + .expect("Checked for existence above"); + + if let Some(new_generation) = new_generation { + tenant_state.generation = new_generation; + } + + if let Some(attaching_pageserver) = attach_req.node_id.as_ref() { + tracing::info!( + tenant_id = %attach_req.tenant_shard_id, + ps_id = %attaching_pageserver, + generation = ?tenant_state.generation, + "issuing", + ); + } else if let Some(ps_id) = tenant_state.intent.attached { + tracing::info!( + tenant_id = %attach_req.tenant_shard_id, + %ps_id, + generation = ?tenant_state.generation, + "dropping", + ); + } else { + tracing::info!( + tenant_id = %attach_req.tenant_shard_id, + "no-op: tenant already has no pageserver"); + } + tenant_state.intent.attached = attach_req.node_id; + + tracing::info!( + "attach_hook: tenant {} set generation {:?}, pageserver {}", + attach_req.tenant_shard_id, + tenant_state.generation, + attach_req.node_id.unwrap_or(utils::id::NodeId(0xfffffff)) + ); + + Ok(AttachHookResponse { + gen: attach_req + .node_id + .map(|_| tenant_state.generation.into().unwrap()), + }) + } + + pub(crate) fn inspect(&self, inspect_req: InspectRequest) -> InspectResponse { + let locked = self.inner.read().unwrap(); + + let tenant_state = locked.tenants.get(&inspect_req.tenant_shard_id); + + InspectResponse { + attachment: tenant_state.and_then(|s| { + s.intent + .attached + .map(|ps| (s.generation.into().unwrap(), ps)) + }), + } + } + + pub(crate) async fn re_attach( + &self, + reattach_req: ReAttachRequest, + ) -> anyhow::Result { + // 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?; + + // Apply the updated generation to our in-memory state + let mut locked = self.inner.write().unwrap(); + + let mut response = ReAttachResponse { + tenants: Vec::new(), + }; + + for (tenant_shard_id, new_gen) in incremented_generations { + response.tenants.push(ReAttachResponseTenant { + id: tenant_shard_id, + gen: new_gen.into().unwrap(), + }); + + // Apply the new generation number to our in-memory state + let shard_state = locked.tenants.get_mut(&tenant_shard_id); + let Some(shard_state) = shard_state else { + // Not fatal. This edge case requires a re-attach to happen + // between inserting a new tenant shard in to the database, and updating our in-memory + // state to know about the shard, _and_ that the state inserted to the database referenced + // a pageserver. Should never happen, but handle it rather than panicking, since it should + // be harmless. + tracing::error!( + "Shard {} is in database for node {} but not in-memory state", + tenant_shard_id, + reattach_req.node_id + ); + continue; + }; + + shard_state.generation = std::cmp::max(shard_state.generation, new_gen); + + // TODO: cancel/restart any running reconciliation for this tenant, it might be trying + // to call location_conf API with an old generation. Wait for cancellation to complete + // before responding to this request. Requires well implemented CancellationToken logic + // all the way to where we call location_conf. Even then, there can still be a location_conf + // request in flight over the network: TODO handle that by making location_conf API refuse + // to go backward in generations. + } + Ok(response) + } + + pub(crate) fn validate(&self, validate_req: ValidateRequest) -> ValidateResponse { + let locked = self.inner.read().unwrap(); + + let mut response = ValidateResponse { + tenants: Vec::new(), + }; + + for req_tenant in validate_req.tenants { + if let Some(tenant_state) = locked.tenants.get(&req_tenant.id) { + let valid = tenant_state.generation == Generation::new(req_tenant.gen); + tracing::info!( + "handle_validate: {}(gen {}): valid={valid} (latest {:?})", + req_tenant.id, + req_tenant.gen, + tenant_state.generation + ); + response.tenants.push(ValidateResponseTenant { + id: req_tenant.id, + valid, + }); + } + } + response + } + + pub(crate) async fn tenant_create( + &self, + create_req: TenantCreateRequest, + ) -> Result { + // Shard count 0 is valid: it means create a single shard (ShardCount(0) means "unsharded") + let literal_shard_count = if create_req.shard_parameters.is_unsharded() { + 1 + } else { + create_req.shard_parameters.count.0 + }; + + // 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.shard_count > ShardCount(1) { + return Err(ApiError::BadRequest(anyhow::anyhow!( + "Attempted to create a specific shard, this API is for creating the whole tenant" + ))); + } else { + create_req.new_tenant_id.tenant_id + }; + + tracing::info!( + "Creating tenant {}, shard_count={:?}", + create_req.new_tenant_id, + create_req.shard_parameters.count, + ); + + let create_ids = (0..literal_shard_count) + .map(|i| TenantShardId { + tenant_id, + shard_number: ShardNumber(i), + shard_count: create_req.shard_parameters.count, + }) + .collect::>(); + + // TODO: enable specifying this. Using Single as a default helps legacy tests to work (they + // have no expectation of HA). + let placement_policy: PlacementPolicy = PlacementPolicy::Single; + + // Ordering: we persist tenant shards before creating them on the pageserver. This enables a caller + // to clean up after themselves by issuing a tenant deletion if something goes wrong and we restart + // during the creation, rather than risking leaving orphan objects in S3. + let persist_tenant_shards = create_ids + .iter() + .map(|tenant_shard_id| TenantShardPersistence { + tenant_id: tenant_shard_id.tenant_id.to_string(), + shard_number: tenant_shard_id.shard_number.0 as i32, + shard_count: tenant_shard_id.shard_count.0 as i32, + shard_stripe_size: create_req.shard_parameters.stripe_size.0 as i32, + generation: 0, + generation_pageserver: None, + placement_policy: serde_json::to_string(&placement_policy).unwrap(), + config: serde_json::to_string(&create_req.config).unwrap(), + }) + .collect(); + self.persistence + .insert_tenant_shards(persist_tenant_shards) + .await + .map_err(|e| { + // TODO: distinguish primary key constraint (idempotent, OK), from other errors + ApiError::InternalServerError(anyhow::anyhow!(e)) + })?; + + let (waiters, response_shards) = { + let mut locked = self.inner.write().unwrap(); + + let mut response_shards = Vec::new(); + + let mut scheduler = Scheduler::new(&locked.tenants, &locked.nodes); + + for tenant_shard_id in create_ids { + tracing::info!("Creating shard {tenant_shard_id}..."); + + use std::collections::btree_map::Entry; + match locked.tenants.entry(tenant_shard_id) { + Entry::Occupied(mut entry) => { + tracing::info!( + "Tenant shard {tenant_shard_id} already exists while creating" + ); + + // TODO: schedule() should take an anti-affinity expression that pushes + // attached and secondary locations (independently) away frorm those + // pageservers also holding a shard for this tenant. + + entry.get_mut().schedule(&mut scheduler).map_err(|e| { + ApiError::Conflict(format!( + "Failed to schedule shard {tenant_shard_id}: {e}" + )) + })?; + + response_shards.push(TenantCreateResponseShard { + node_id: entry + .get() + .intent + .attached + .expect("We just set pageserver if it was None"), + generation: entry.get().generation.into().unwrap(), + }); + + continue; + } + Entry::Vacant(entry) => { + let mut state = TenantState::new( + tenant_shard_id, + ShardIdentity::from_params( + tenant_shard_id.shard_number, + &create_req.shard_parameters, + ), + placement_policy.clone(), + ); + + if let Some(create_gen) = create_req.generation { + state.generation = Generation::new(create_gen); + } + state.config = create_req.config.clone(); + + state.schedule(&mut scheduler).map_err(|e| { + ApiError::Conflict(format!( + "Failed to schedule shard {tenant_shard_id}: {e}" + )) + })?; + + response_shards.push(TenantCreateResponseShard { + node_id: state + .intent + .attached + .expect("We just set pageserver if it was None"), + generation: state.generation.into().unwrap(), + }); + entry.insert(state) + } + }; + } + + // Take a snapshot of pageservers + let pageservers = locked.nodes.clone(); + + let result_tx = locked.result_tx.clone(); + let compute_hook = locked.compute_hook.clone(); + + let waiters = locked + .tenants + .range_mut(TenantShardId::tenant_range(tenant_id)) + .filter_map(|(_shard_id, shard)| { + shard.maybe_reconcile( + result_tx.clone(), + &pageservers, + &compute_hook, + &self.config, + &self.persistence, + ) + }) + .collect::>(); + (waiters, response_shards) + }; + + let deadline = Instant::now().checked_add(Duration::from_secs(5)).unwrap(); + for waiter in waiters { + let timeout = deadline.duration_since(Instant::now()); + waiter.wait_timeout(timeout).await?; + } + Ok(TenantCreateResponse { + shards: response_shards, + }) + } + + pub(crate) async fn tenant_timeline_create( + &self, + tenant_id: TenantId, + mut create_req: TimelineCreateRequest, + ) -> Result { + let mut timeline_info = None; + + let ensure_waiters = { + let locked = self.inner.write().unwrap(); + tracing::info!( + "Creating timeline {}/{}, have {} pageservers", + tenant_id, + create_req.new_timeline_id, + locked.nodes.len() + ); + + self.ensure_attached(locked, tenant_id) + .map_err(ApiError::InternalServerError)? + }; + + let deadline = Instant::now().checked_add(Duration::from_secs(5)).unwrap(); + for waiter in ensure_waiters { + let timeout = deadline.duration_since(Instant::now()); + waiter.wait_timeout(timeout).await?; + } + + let targets = { + let locked = self.inner.read().unwrap(); + let mut targets = Vec::new(); + + for (tenant_shard_id, shard) in + locked.tenants.range(TenantShardId::tenant_range(tenant_id)) + { + let node_id = shard.intent.attached.ok_or_else(|| { + ApiError::InternalServerError(anyhow::anyhow!("Shard not scheduled")) + })?; + let node = locked + .nodes + .get(&node_id) + .expect("Pageservers may not be deleted while referenced"); + + targets.push((*tenant_shard_id, node.clone())); + } + targets + }; + + if targets.is_empty() { + return Err(ApiError::NotFound( + anyhow::anyhow!("Tenant not found").into(), + )); + } + + for (tenant_shard_id, node) in targets { + // TODO: issue shard timeline creates in parallel, once the 0th is done. + + let client = mgmt_api::Client::new(node.base_url(), self.config.jwt_token.as_deref()); + + tracing::info!( + "Creating timeline on shard {}/{}, attached to node {}", + tenant_shard_id, + create_req.new_timeline_id, + node.id + ); + + let shard_timeline_info = client + .timeline_create(tenant_shard_id, &create_req) + .await + .map_err(|e| match e { + mgmt_api::Error::ApiError(status, msg) + if status == StatusCode::INTERNAL_SERVER_ERROR + || status == StatusCode::NOT_ACCEPTABLE => + { + // TODO: handle more error codes, e.g. 503 should be passed through. Make a general wrapper + // for pass-through API calls. + ApiError::InternalServerError(anyhow::anyhow!(msg)) + } + _ => ApiError::Conflict(format!("Failed to create timeline: {e}")), + })?; + + if timeline_info.is_none() { + // If the caller specified an ancestor but no ancestor LSN, we are responsible for + // propagating the LSN chosen by the first shard to the other shards: it is important + // that all shards end up with the same ancestor_start_lsn. + if create_req.ancestor_timeline_id.is_some() + && create_req.ancestor_start_lsn.is_none() + { + create_req.ancestor_start_lsn = shard_timeline_info.ancestor_lsn; + } + + // We will return the TimelineInfo from the first shard + timeline_info = Some(shard_timeline_info); + } + } + Ok(timeline_info.expect("targets cannot be empty")) + } + + pub(crate) fn tenant_locate( + &self, + tenant_id: TenantId, + ) -> Result { + let locked = self.inner.read().unwrap(); + tracing::info!("Locating shards for tenant {tenant_id}"); + + // Take a snapshot of pageservers + let pageservers = locked.nodes.clone(); + + let mut result = Vec::new(); + let mut shard_params: Option = None; + + for (tenant_shard_id, shard) in locked.tenants.range(TenantShardId::tenant_range(tenant_id)) + { + let node_id = shard + .intent + .attached + .ok_or(ApiError::BadRequest(anyhow::anyhow!( + "Cannot locate a tenant that is not attached" + )))?; + + let node = pageservers + .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, + }); + + match &shard_params { + None => { + shard_params = Some(ShardParameters { + stripe_size: shard.shard.stripe_size, + count: shard.shard.count, + }); + } + Some(params) => { + if params.stripe_size != shard.shard.stripe_size { + // This should never happen. We enforce at runtime because it's simpler than + // adding an extra per-tenant data structure to store the things that should be the same + return Err(ApiError::InternalServerError(anyhow::anyhow!( + "Inconsistent shard stripe size parameters!" + ))); + } + } + } + } + + if result.is_empty() { + return Err(ApiError::NotFound( + anyhow::anyhow!("No shards for this tenant ID found").into(), + )); + } + let shard_params = shard_params.expect("result is non-empty, therefore this is set"); + tracing::info!( + "Located tenant {} with params {:?} on shards {}", + tenant_id, + shard_params, + result + .iter() + .map(|s| format!("{:?}", s)) + .collect::>() + .join(",") + ); + + Ok(TenantLocateResponse { + shards: result, + shard_params, + }) + } + + pub(crate) async fn tenant_shard_migrate( + &self, + tenant_shard_id: TenantShardId, + migrate_req: TenantShardMigrateRequest, + ) -> Result { + let waiter = { + let mut locked = self.inner.write().unwrap(); + + let result_tx = locked.result_tx.clone(); + let pageservers = locked.nodes.clone(); + let compute_hook = locked.compute_hook.clone(); + + let Some(shard) = locked.tenants.get_mut(&tenant_shard_id) else { + return Err(ApiError::NotFound( + anyhow::anyhow!("Tenant shard not found").into(), + )); + }; + + if shard.intent.attached == Some(migrate_req.node_id) { + // No-op case: we will still proceed to wait for reconciliation in case it is + // incomplete from an earlier update to the intent. + tracing::info!("Migrating: intent is unchanged {:?}", shard.intent); + } else { + let old_attached = shard.intent.attached; + + shard.intent.attached = Some(migrate_req.node_id); + match shard.policy { + PlacementPolicy::Single => { + shard.intent.secondary.clear(); + } + PlacementPolicy::Double(_n) => { + // If our new attached node was a secondary, it no longer should be. + shard.intent.secondary.retain(|s| s != &migrate_req.node_id); + + // If we were already attached to something, demote that to a secondary + if let Some(old_attached) = old_attached { + shard.intent.secondary.push(old_attached); + } + } + } + + tracing::info!("Migrating: new intent {:?}", shard.intent); + shard.sequence = shard.sequence.next(); + } + + shard.maybe_reconcile( + result_tx, + &pageservers, + &compute_hook, + &self.config, + &self.persistence, + ) + }; + + if let Some(waiter) = waiter { + waiter.wait_timeout(RECONCILE_TIMEOUT).await?; + } else { + tracing::warn!("Migration is a no-op"); + } + + Ok(TenantShardMigrateResponse {}) + } + + pub(crate) async fn node_register( + &self, + register_req: NodeRegisterRequest, + ) -> Result<(), ApiError> { + // Pre-check for an already-existing node + { + let locked = self.inner.read().unwrap(); + 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 + { + tracing::info!( + "Node {} re-registered with matching address", + register_req.node_id + ); + return Ok(()); + } else { + // TODO: decide if we want to allow modifying node addresses without removing and re-adding + // the node. Safest/simplest thing is to refuse it, and usually we deploy with + // a fixed address through the lifetime of a node. + tracing::warn!( + "Node {} tried to register with different address", + register_req.node_id + ); + return Err(ApiError::Conflict( + "Node is already registered with different address".to_string(), + )); + } + } + } + + // 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, + }; + // TODO: idempotency if the node already exists in the database + self.persistence + .insert_node(&new_node) + .await + .map_err(ApiError::InternalServerError)?; + + let mut locked = self.inner.write().unwrap(); + let mut new_nodes = (*locked.nodes).clone(); + + new_nodes.insert(register_req.node_id, new_node); + + locked.nodes = Arc::new(new_nodes); + + tracing::info!( + "Registered pageserver {}, now have {} pageservers", + register_req.node_id, + locked.nodes.len() + ); + Ok(()) + } + + pub(crate) fn node_configure(&self, config_req: NodeConfigureRequest) -> Result<(), ApiError> { + let mut locked = self.inner.write().unwrap(); + let result_tx = locked.result_tx.clone(); + let compute_hook = locked.compute_hook.clone(); + + let mut new_nodes = (*locked.nodes).clone(); + + let Some(node) = new_nodes.get_mut(&config_req.node_id) else { + return Err(ApiError::NotFound( + anyhow::anyhow!("Node not registered").into(), + )); + }; + + 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; + } + + if let Some(scheduling) = config_req.scheduling { + node.scheduling = scheduling; + + // TODO: once we have a background scheduling ticker for fill/drain, kick it + // to wake up and start working. + } + + let new_nodes = Arc::new(new_nodes); + + let mut scheduler = Scheduler::new(&locked.tenants, &new_nodes); + if offline_transition { + for (tenant_shard_id, tenant_state) in &mut locked.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.notify_offline(config_req.node_id) { + tenant_state.sequence = tenant_state.sequence.next(); + match tenant_state.schedule(&mut 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(()) => { + tenant_state.maybe_reconcile( + result_tx.clone(), + &new_nodes, + &compute_hook, + &self.config, + &self.persistence, + ); + } + } + } + } + } + + 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, + ); + } + } + } + + // TODO: in the background, we should balance work back onto this pageserver + } + + locked.nodes = new_nodes; + + Ok(()) + } + + /// Helper for methods that will try and call pageserver APIs for + /// a tenant, such as timeline CRUD: they cannot proceed unless the tenant + /// is attached somewhere. + fn ensure_attached( + &self, + mut locked: std::sync::RwLockWriteGuard<'_, ServiceState>, + 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 mut scheduler = Scheduler::new(&locked.tenants, &locked.nodes); + let pageservers = locked.nodes.clone(); + + for (_tenant_shard_id, shard) in locked + .tenants + .range_mut(TenantShardId::tenant_range(tenant_id)) + { + shard.schedule(&mut scheduler)?; + + if let Some(waiter) = shard.maybe_reconcile( + result_tx.clone(), + &pageservers, + &compute_hook, + &self.config, + &self.persistence, + ) { + waiters.push(waiter); + } + } + Ok(waiters) + } + + /// 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, + ) + }) + .count() + } +} diff --git a/control_plane/attachment_service/src/tenant_state.rs b/control_plane/attachment_service/src/tenant_state.rs new file mode 100644 index 0000000000..a907628eff --- /dev/null +++ b/control_plane/attachment_service/src/tenant_state.rs @@ -0,0 +1,455 @@ +use std::{collections::HashMap, sync::Arc, time::Duration}; + +use control_plane::attachment_service::NodeAvailability; +use pageserver_api::{ + models::{LocationConfig, LocationConfigMode, TenantConfig}, + shard::{ShardIdentity, TenantShardId}, +}; +use tokio::task::JoinHandle; +use tokio_util::sync::CancellationToken; +use utils::{ + generation::Generation, + id::NodeId, + seqwait::{SeqWait, SeqWaitError}, +}; + +use crate::{ + compute_hook::ComputeHook, + node::Node, + persistence::Persistence, + reconciler::{attached_location_conf, secondary_location_conf, ReconcileError, Reconciler}, + scheduler::{ScheduleError, Scheduler}, + service, PlacementPolicy, Sequence, +}; + +pub(crate) struct TenantState { + pub(crate) tenant_shard_id: TenantShardId, + + pub(crate) shard: ShardIdentity, + + // Runtime only: sequence used to coordinate when updating this object while + // with background reconcilers may be running. A reconciler runs to a particular + // sequence. + pub(crate) sequence: Sequence, + + // Latest generation number: next time we attach, increment this + // and use the incremented number when attaching + pub(crate) generation: Generation, + + // High level description of how the tenant should be set up. Provided + // externally. + pub(crate) policy: PlacementPolicy, + + // Low level description of exactly which pageservers should fulfil + // which role. Generated by `Self::schedule`. + pub(crate) intent: IntentState, + + // Low level description of how the tenant is configured on pageservers: + // if this does not match `Self::intent` then the tenant needs reconciliation + // with `Self::reconcile`. + pub(crate) observed: ObservedState, + + // Tenant configuration, passed through opaquely to the pageserver. Identical + // for all shards in a tenant. + pub(crate) config: TenantConfig, + + /// If a reconcile task is currently in flight, it may be joined here (it is + /// only safe to join if either the result has been received or the reconciler's + /// cancellation token has been fired) + pub(crate) reconciler: Option, + + /// Optionally wait for reconciliation to complete up to a particular + /// sequence number. + pub(crate) waiter: std::sync::Arc>, + + /// Indicates sequence number for which we have encountered an error reconciling. If + /// this advances ahead of [`Self::waiter`] then a reconciliation error has occurred, + /// and callers should stop waiting for `waiter` and propagate the error. + pub(crate) error_waiter: std::sync::Arc>, + + /// The most recent error from a reconcile on this tenant + /// TODO: generalize to an array of recent events + /// TOOD: use a ArcSwap instead of mutex for faster reads? + pub(crate) last_error: std::sync::Arc>, +} + +#[derive(Default, Clone, Debug)] +pub(crate) struct IntentState { + pub(crate) attached: Option, + pub(crate) secondary: Vec, +} + +#[derive(Default, Clone)] +pub(crate) struct ObservedState { + pub(crate) locations: HashMap, +} + +/// Our latest knowledge of how this tenant is configured in the outside world. +/// +/// Meaning: +/// * No instance of this type exists for a node: we are certain that we have nothing configured on that +/// node for this shard. +/// * Instance exists with conf==None: we *might* have some state on that node, but we don't know +/// what it is (e.g. we failed partway through configuring it) +/// * Instance exists with conf==Some: this tells us what we last successfully configured on this node, +/// and that configuration will still be present unless something external interfered. +#[derive(Clone)] +pub(crate) struct ObservedStateLocation { + /// If None, it means we do not know the status of this shard's location on this node, but + /// we know that we might have some state on this node. + pub(crate) conf: Option, +} +pub(crate) struct ReconcilerWaiter { + // For observability purposes, remember the ID of the shard we're + // waiting for. + pub(crate) tenant_shard_id: TenantShardId, + + seq_wait: std::sync::Arc>, + error_seq_wait: std::sync::Arc>, + error: std::sync::Arc>, + seq: Sequence, +} + +#[derive(thiserror::Error, Debug)] +pub enum ReconcileWaitError { + #[error("Timeout waiting for shard {0}")] + Timeout(TenantShardId), + #[error("shutting down")] + Shutdown, + #[error("Reconcile error on shard {0}: {1}")] + Failed(TenantShardId, String), +} + +impl ReconcilerWaiter { + pub(crate) async fn wait_timeout(&self, timeout: Duration) -> Result<(), ReconcileWaitError> { + tokio::select! { + result = self.seq_wait.wait_for_timeout(self.seq, timeout)=> { + result.map_err(|e| match e { + SeqWaitError::Timeout => ReconcileWaitError::Timeout(self.tenant_shard_id), + SeqWaitError::Shutdown => ReconcileWaitError::Shutdown + })?; + }, + result = self.error_seq_wait.wait_for(self.seq) => { + result.map_err(|e| match e { + SeqWaitError::Shutdown => ReconcileWaitError::Shutdown, + SeqWaitError::Timeout => unreachable!() + })?; + + return Err(ReconcileWaitError::Failed(self.tenant_shard_id, self.error.lock().unwrap().clone())) + } + } + + Ok(()) + } +} + +/// Having spawned a reconciler task, the tenant shard's state will carry enough +/// information to optionally cancel & await it later. +pub(crate) struct ReconcilerHandle { + sequence: Sequence, + handle: JoinHandle<()>, + cancel: CancellationToken, +} + +/// When a reconcile task completes, it sends this result object +/// to be applied to the primary TenantState. +pub(crate) struct ReconcileResult { + pub(crate) sequence: Sequence, + /// On errors, `observed` should be treated as an incompleted description + /// of state (i.e. any nodes present in the result should override nodes + /// present in the parent tenant state, but any unmentioned nodes should + /// not be removed from parent tenant state) + pub(crate) result: Result<(), ReconcileError>, + + pub(crate) tenant_shard_id: TenantShardId, + pub(crate) generation: Generation, + pub(crate) observed: ObservedState, +} + +impl IntentState { + pub(crate) fn new() -> Self { + Self { + attached: None, + secondary: vec![], + } + } + pub(crate) fn all_pageservers(&self) -> Vec { + let mut result = Vec::new(); + if let Some(p) = self.attached { + result.push(p) + } + + result.extend(self.secondary.iter().copied()); + + result + } + + /// When a node goes offline, we update intents to avoid using it + /// as their attached pageserver. + /// + /// Returns true if a change was made + pub(crate) fn notify_offline(&mut self, node_id: NodeId) -> bool { + if self.attached == Some(node_id) { + self.attached = None; + self.secondary.push(node_id); + true + } else { + false + } + } +} + +impl ObservedState { + pub(crate) fn new() -> Self { + Self { + locations: HashMap::new(), + } + } +} + +impl TenantState { + pub(crate) fn new( + tenant_shard_id: TenantShardId, + shard: ShardIdentity, + policy: PlacementPolicy, + ) -> Self { + Self { + tenant_shard_id, + policy, + intent: IntentState::default(), + generation: Generation::new(0), + shard, + observed: ObservedState::default(), + config: TenantConfig::default(), + reconciler: None, + sequence: Sequence(1), + waiter: Arc::new(SeqWait::new(Sequence(0))), + error_waiter: Arc::new(SeqWait::new(Sequence(0))), + last_error: Arc::default(), + } + } + + /// For use on startup when learning state from pageservers: generate my [`IntentState`] from my + /// [`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) { + // Choose an attached location by filtering observed locations, and then sorting to get the highest + // generation + let mut attached_locs = self + .observed + .locations + .iter() + .filter_map(|(node_id, l)| { + if let Some(conf) = &l.conf { + if conf.mode == LocationConfigMode::AttachedMulti + || conf.mode == LocationConfigMode::AttachedSingle + || conf.mode == LocationConfigMode::AttachedStale + { + Some((node_id, conf.generation)) + } else { + None + } + } else { + None + } + }) + .collect::>(); + + 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); + } + + // All remaining observed locations generate secondary intents. This includes None + // observations, as these may well have some local content on disk that is usable (this + // is an edge case that might occur if we restarted during a migration or other change) + self.observed.locations.keys().for_each(|node_id| { + if Some(*node_id) != self.intent.attached { + self.intent.secondary.push(*node_id); + } + }); + } + + pub(crate) fn schedule(&mut self, scheduler: &mut Scheduler) -> Result<(), ScheduleError> { + // TODO: before scheduling new nodes, check if any existing content in + // self.intent refers to pageservers that are offline, and pick other + // pageservers if so. + + // Build the set of pageservers already in use by this tenant, to avoid scheduling + // more work on the same pageservers we're already using. + let mut used_pageservers = self.intent.all_pageservers(); + let mut modified = false; + + use PlacementPolicy::*; + match self.policy { + Single => { + // Should have exactly one attached, and zero secondaries + if self.intent.attached.is_none() { + let node_id = scheduler.schedule_shard(&used_pageservers)?; + self.intent.attached = Some(node_id); + used_pageservers.push(node_id); + modified = true; + } + if !self.intent.secondary.is_empty() { + self.intent.secondary.clear(); + modified = true; + } + } + Double(secondary_count) => { + // Should have exactly one attached, and N secondaries + if self.intent.attached.is_none() { + let node_id = scheduler.schedule_shard(&used_pageservers)?; + self.intent.attached = Some(node_id); + used_pageservers.push(node_id); + modified = true; + } + + while self.intent.secondary.len() < secondary_count { + let node_id = scheduler.schedule_shard(&used_pageservers)?; + self.intent.secondary.push(node_id); + used_pageservers.push(node_id); + modified = true; + } + } + } + + if modified { + self.sequence.0 += 1; + } + + Ok(()) + } + + fn dirty(&self) -> bool { + if let Some(node_id) = self.intent.attached { + let wanted_conf = attached_location_conf(self.generation, &self.shard, &self.config); + match self.observed.locations.get(&node_id) { + Some(conf) if conf.conf.as_ref() == Some(&wanted_conf) => {} + Some(_) | None => { + return true; + } + } + } + + for node_id in &self.intent.secondary { + let wanted_conf = secondary_location_conf(&self.shard, &self.config); + match self.observed.locations.get(node_id) { + Some(conf) if conf.conf.as_ref() == Some(&wanted_conf) => {} + Some(_) | None => { + return true; + } + } + } + + false + } + + pub(crate) fn maybe_reconcile( + &mut self, + result_tx: tokio::sync::mpsc::UnboundedSender, + pageservers: &Arc>, + compute_hook: &Arc, + service_config: &service::Config, + persistence: &Arc, + ) -> Option { + // If there are any ambiguous observed states, and the nodes they refer to are available, + // we should reconcile to clean them up. + let mut dirty_observed = false; + for (node_id, observed_loc) in &self.observed.locations { + 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) + { + dirty_observed = true; + break; + } + } + + if !self.dirty() && !dirty_observed { + tracing::info!("Not dirty, no reconciliation needed."); + return None; + } + + // Reconcile already in flight for the current sequence? + if let Some(handle) = &self.reconciler { + if handle.sequence == self.sequence { + return Some(ReconcilerWaiter { + tenant_shard_id: self.tenant_shard_id, + seq_wait: self.waiter.clone(), + error_seq_wait: self.error_waiter.clone(), + error: self.last_error.clone(), + seq: self.sequence, + }); + } + } + + // 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(); + + let cancel = CancellationToken::new(); + let mut reconciler = Reconciler { + tenant_shard_id: self.tenant_shard_id, + shard: self.shard, + generation: self.generation, + intent: self.intent.clone(), + config: self.config.clone(), + observed: self.observed.clone(), + pageservers: pageservers.clone(), + compute_hook: compute_hook.clone(), + service_config: service_config.clone(), + cancel: cancel.clone(), + persistence: persistence.clone(), + }; + + let reconcile_seq = self.sequence; + + tracing::info!("Spawning Reconciler for sequence {}", self.sequence); + let join_handle = tokio::task::spawn(async move { + // Wait for any previous reconcile task to complete before we start + if let Some(old_handle) = old_handle { + old_handle.cancel.cancel(); + if let Err(e) = old_handle.handle.await { + // We can't do much with this other than log it: the task is done, so + // we may proceed with our work. + tracing::error!("Unexpected join error waiting for reconcile task: {e}"); + } + } + + // Early check for cancellation before doing any work + // TODO: wrap all remote API operations in cancellation check + // as well. + if reconciler.cancel.is_cancelled() { + return; + } + + let result = reconciler.reconcile().await; + result_tx + .send(ReconcileResult { + sequence: reconcile_seq, + result, + tenant_shard_id: reconciler.tenant_shard_id, + generation: reconciler.generation, + observed: reconciler.observed, + }) + .ok(); + }); + + self.reconciler = Some(ReconcilerHandle { + sequence: self.sequence, + handle: join_handle, + cancel, + }); + + Some(ReconcilerWaiter { + tenant_shard_id: self.tenant_shard_id, + seq_wait: self.waiter.clone(), + error_seq_wait: self.error_waiter.clone(), + error: self.last_error.clone(), + seq: self.sequence, + }) + } +} diff --git a/control_plane/src/attachment_service.rs b/control_plane/src/attachment_service.rs index 731c05809e..0a353d8b12 100644 --- a/control_plane/src/attachment_service.rs +++ b/control_plane/src/attachment_service.rs @@ -1,14 +1,27 @@ use crate::{background_process, local_env::LocalEnv}; -use anyhow::anyhow; use camino::Utf8PathBuf; -use serde::{Deserialize, Serialize}; -use std::{path::PathBuf, process::Child}; -use utils::id::{NodeId, TenantId}; +use hyper::Method; +use pageserver_api::{ + models::{ShardParameters, TenantCreateRequest, TimelineCreateRequest, TimelineInfo}, + shard::TenantShardId, +}; +use pageserver_client::mgmt_api::ResponseErrorMessageExt; +use postgres_backend::AuthType; +use postgres_connection::parse_host_port; +use serde::{de::DeserializeOwned, Deserialize, Serialize}; +use std::{path::PathBuf, process::Child, str::FromStr}; +use tracing::instrument; +use utils::{ + auth::{Claims, Scope}, + id::{NodeId, TenantId}, +}; pub struct AttachmentService { env: LocalEnv, listen: String, path: PathBuf, + jwt_token: Option, + public_key_path: Option, client: reqwest::Client, } @@ -16,7 +29,7 @@ const COMMAND: &str = "attachment_service"; #[derive(Serialize, Deserialize)] pub struct AttachHookRequest { - pub tenant_id: TenantId, + pub tenant_shard_id: TenantShardId, pub node_id: Option, } @@ -27,7 +40,7 @@ pub struct AttachHookResponse { #[derive(Serialize, Deserialize)] pub struct InspectRequest { - pub tenant_id: TenantId, + pub tenant_shard_id: TenantShardId, } #[derive(Serialize, Deserialize)] @@ -35,6 +48,125 @@ pub struct InspectResponse { pub attachment: Option<(u32, NodeId)>, } +#[derive(Serialize, Deserialize)] +pub struct TenantCreateResponseShard { + pub node_id: NodeId, + pub generation: u32, +} + +#[derive(Serialize, Deserialize)] +pub struct TenantCreateResponse { + pub shards: Vec, +} + +#[derive(Serialize, Deserialize)] +pub struct NodeRegisterRequest { + pub node_id: NodeId, + + pub listen_pg_addr: String, + pub listen_pg_port: u16, + + pub listen_http_addr: String, + pub listen_http_port: u16, +} + +#[derive(Serialize, Deserialize)] +pub struct NodeConfigureRequest { + pub node_id: NodeId, + + pub availability: Option, + pub scheduling: Option, +} + +#[derive(Serialize, Deserialize, Debug)] +pub struct TenantLocateResponseShard { + pub shard_id: TenantShardId, + pub node_id: NodeId, + + pub listen_pg_addr: String, + pub listen_pg_port: u16, + + pub listen_http_addr: String, + pub listen_http_port: u16, +} + +#[derive(Serialize, Deserialize)] +pub struct TenantLocateResponse { + pub shards: Vec, + pub shard_params: ShardParameters, +} + +/// Explicitly migrating a particular shard is a low level operation +/// TODO: higher level "Reschedule tenant" operation where the request +/// specifies some constraints, e.g. asking it to get off particular node(s) +#[derive(Serialize, Deserialize, Debug)] +pub struct TenantShardMigrateRequest { + pub tenant_shard_id: TenantShardId, + pub node_id: NodeId, +} + +#[derive(Serialize, Deserialize, Clone, Copy)] +pub enum NodeAvailability { + // Normal, happy state + Active, + // Offline: Tenants shouldn't try to attach here, but they may assume that their + // secondary locations on this node still exist. Newly added nodes are in this + // state until we successfully contact them. + Offline, +} + +impl FromStr for NodeAvailability { + type Err = anyhow::Error; + + fn from_str(s: &str) -> Result { + match s { + "active" => Ok(Self::Active), + "offline" => Ok(Self::Offline), + _ => Err(anyhow::anyhow!("Unknown availability state '{s}'")), + } + } +} + +/// 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)] +pub enum NodeSchedulingPolicy { + Active, + Filling, + Pause, + Draining, +} + +impl FromStr for NodeSchedulingPolicy { + type Err = anyhow::Error; + + fn from_str(s: &str) -> Result { + match s { + "active" => Ok(Self::Active), + "filling" => Ok(Self::Filling), + "pause" => Ok(Self::Pause), + "draining" => Ok(Self::Draining), + _ => Err(anyhow::anyhow!("Unknown scheduling state '{s}'")), + } + } +} + +impl From for String { + fn from(value: NodeSchedulingPolicy) -> String { + use NodeSchedulingPolicy::*; + match value { + Active => "active", + Filling => "filling", + Pause => "pause", + Draining => "draining", + } + .to_string() + } +} + +#[derive(Serialize, Deserialize, Debug)] +pub struct TenantShardMigrateResponse {} + impl AttachmentService { pub fn from_env(env: &LocalEnv) -> Self { let path = env.base_data_dir.join("attachments.json"); @@ -49,10 +181,34 @@ impl AttachmentService { listen_url.port().unwrap() ); + // Assume all pageservers have symmetric auth configuration: this service + // expects to use one JWT token to talk to all of them. + let ps_conf = env + .pageservers + .first() + .expect("Config is validated to contain at least one pageserver"); + let (jwt_token, public_key_path) = match ps_conf.http_auth_type { + AuthType::Trust => (None, None), + AuthType::NeonJWT => { + let jwt_token = env + .generate_auth_token(&Claims::new(None, Scope::PageServerApi)) + .unwrap(); + + // If pageserver auth is enabled, this implicitly enables auth for this service, + // using the same credentials. + let public_key_path = + camino::Utf8PathBuf::try_from(env.base_data_dir.join("auth_public_key.pem")) + .unwrap(); + (Some(jwt_token), Some(public_key_path)) + } + }; + Self { env: env.clone(), path, listen, + jwt_token, + public_key_path, client: reqwest::ClientBuilder::new() .build() .expect("Failed to construct http client"), @@ -67,72 +223,199 @@ impl AttachmentService { pub async fn start(&self) -> anyhow::Result { let path_str = self.path.to_string_lossy(); - background_process::start_process( + let mut args = vec!["-l", &self.listen, "-p", &path_str] + .into_iter() + .map(|s| s.to_string()) + .collect::>(); + if let Some(jwt_token) = &self.jwt_token { + args.push(format!("--jwt-token={jwt_token}")); + } + + if let Some(public_key_path) = &self.public_key_path { + args.push(format!("--public-key={public_key_path}")); + } + + let result = background_process::start_process( COMMAND, &self.env.base_data_dir, &self.env.attachment_service_bin(), - ["-l", &self.listen, "-p", &path_str], - [], + args, + [( + "NEON_REPO_DIR".to_string(), + self.env.base_data_dir.to_string_lossy().to_string(), + )], background_process::InitialPidFile::Create(self.pid_file()), - // TODO: a real status check - || async move { anyhow::Ok(true) }, + || async { + match self.status().await { + Ok(_) => Ok(true), + Err(_) => Ok(false), + } + }, ) - .await + .await; + + for ps_conf in &self.env.pageservers { + let (pg_host, pg_port) = + parse_host_port(&ps_conf.listen_pg_addr).expect("Unable to parse listen_pg_addr"); + let (http_host, http_port) = parse_host_port(&ps_conf.listen_http_addr) + .expect("Unable to parse listen_http_addr"); + self.node_register(NodeRegisterRequest { + node_id: ps_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?; + } + + result } pub fn stop(&self, immediate: bool) -> anyhow::Result<()> { background_process::stop_process(immediate, COMMAND, &self.pid_file()) } - - /// Call into the attach_hook API, for use before handing out attachments to pageservers - pub async fn attach_hook( + /// Simple HTTP request wrapper for calling into attachment service + async fn dispatch( &self, - tenant_id: TenantId, - pageserver_id: NodeId, - ) -> anyhow::Result> { - use hyper::StatusCode; - + method: hyper::Method, + path: String, + body: Option, + ) -> anyhow::Result + where + RQ: Serialize + Sized, + RS: DeserializeOwned + Sized, + { let url = self .env .control_plane_api .clone() .unwrap() - .join("attach-hook") + .join(&path) .unwrap(); + let mut builder = self.client.request(method, url); + if let Some(body) = body { + builder = builder.json(&body) + } + if let Some(jwt_token) = &self.jwt_token { + builder = builder.header( + reqwest::header::AUTHORIZATION, + format!("Bearer {jwt_token}"), + ); + } + + let response = builder.send().await?; + let response = response.error_from_body().await?; + + Ok(response + .json() + .await + .map_err(pageserver_client::mgmt_api::Error::ReceiveBody)?) + } + + /// Call into the attach_hook API, for use before handing out attachments to pageservers + #[instrument(skip(self))] + pub async fn attach_hook( + &self, + tenant_shard_id: TenantShardId, + pageserver_id: NodeId, + ) -> anyhow::Result> { let request = AttachHookRequest { - tenant_id, + tenant_shard_id, node_id: Some(pageserver_id), }; - let response = self.client.post(url).json(&request).send().await?; - if response.status() != StatusCode::OK { - return Err(anyhow!("Unexpected status {}", response.status())); - } + let response = self + .dispatch::<_, AttachHookResponse>( + Method::POST, + "attach-hook".to_string(), + Some(request), + ) + .await?; - let response = response.json::().await?; Ok(response.gen) } - pub async fn inspect(&self, tenant_id: TenantId) -> anyhow::Result> { - use hyper::StatusCode; + #[instrument(skip(self))] + pub async fn inspect( + &self, + tenant_shard_id: TenantShardId, + ) -> anyhow::Result> { + let request = InspectRequest { tenant_shard_id }; - let url = self - .env - .control_plane_api - .clone() - .unwrap() - .join("inspect") - .unwrap(); + let response = self + .dispatch::<_, InspectResponse>(Method::POST, "inspect".to_string(), Some(request)) + .await?; - let request = InspectRequest { tenant_id }; - - let response = self.client.post(url).json(&request).send().await?; - if response.status() != StatusCode::OK { - return Err(anyhow!("Unexpected status {}", response.status())); - } - - let response = response.json::().await?; Ok(response.attachment) } + + #[instrument(skip(self))] + pub async fn tenant_create( + &self, + req: TenantCreateRequest, + ) -> anyhow::Result { + self.dispatch(Method::POST, "tenant".to_string(), Some(req)) + .await + } + + #[instrument(skip(self))] + pub async fn tenant_locate(&self, tenant_id: TenantId) -> anyhow::Result { + self.dispatch::<(), _>(Method::GET, format!("tenant/{tenant_id}/locate"), None) + .await + } + + #[instrument(skip(self))] + pub async fn tenant_migrate( + &self, + tenant_shard_id: TenantShardId, + node_id: NodeId, + ) -> anyhow::Result { + self.dispatch( + Method::PUT, + format!("tenant/{tenant_shard_id}/migrate"), + Some(TenantShardMigrateRequest { + tenant_shard_id, + node_id, + }), + ) + .await + } + + #[instrument(skip_all, fields(node_id=%req.node_id))] + pub async fn node_register(&self, req: NodeRegisterRequest) -> anyhow::Result<()> { + self.dispatch::<_, ()>(Method::POST, "node".to_string(), Some(req)) + .await + } + + #[instrument(skip_all, fields(node_id=%req.node_id))] + pub async fn node_configure(&self, req: NodeConfigureRequest) -> anyhow::Result<()> { + self.dispatch::<_, ()>( + Method::PUT, + format!("node/{}/config", req.node_id), + Some(req), + ) + .await + } + + #[instrument(skip(self))] + pub async fn status(&self) -> anyhow::Result<()> { + self.dispatch::<(), ()>(Method::GET, "status".to_string(), None) + .await + } + + #[instrument(skip_all, fields(%tenant_id, timeline_id=%req.new_timeline_id))] + pub async fn tenant_timeline_create( + &self, + tenant_id: TenantId, + req: TimelineCreateRequest, + ) -> anyhow::Result { + self.dispatch( + Method::POST, + format!("tenant/{tenant_id}/timeline"), + Some(req), + ) + .await + } } diff --git a/control_plane/src/bin/attachment_service.rs b/control_plane/src/bin/attachment_service.rs deleted file mode 100644 index fb5e4f926f..0000000000 --- a/control_plane/src/bin/attachment_service.rs +++ /dev/null @@ -1,355 +0,0 @@ -/// 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; -use clap::Parser; -use hex::FromHex; -use hyper::StatusCode; -use hyper::{Body, Request, Response}; -use pageserver_api::shard::TenantShardId; -use serde::{Deserialize, Serialize}; -use std::path::{Path, PathBuf}; -use std::{collections::HashMap, sync::Arc}; -use utils::http::endpoint::request_span; -use utils::logging::{self, LogFormat}; -use utils::signals::{ShutdownSignals, Signal}; - -use utils::{ - http::{ - endpoint::{self}, - error::ApiError, - json::{json_request, json_response}, - RequestExt, RouterBuilder, - }, - id::{NodeId, TenantId}, - tcp_listener, -}; - -use pageserver_api::control_api::{ - ReAttachRequest, ReAttachResponse, ReAttachResponseTenant, ValidateRequest, ValidateResponse, - ValidateResponseTenant, -}; - -use control_plane::attachment_service::{ - AttachHookRequest, AttachHookResponse, InspectRequest, InspectResponse, -}; - -#[derive(Parser)] -#[command(author, version, about, long_about = None)] -#[command(arg_required_else_help(true))] -struct Cli { - /// Host and port to listen on, like `127.0.0.1:1234` - #[arg(short, long)] - listen: std::net::SocketAddr, - - /// Path to the .json file to store state (will be created if it doesn't exist) - #[arg(short, long)] - path: PathBuf, -} - -// The persistent state of each Tenant -#[derive(Serialize, Deserialize, Clone)] -struct TenantState { - // Currently attached pageserver - pageserver: Option, - - // Latest generation number: next time we attach, increment this - // and use the incremented number when attaching - generation: u32, -} - -fn to_hex_map(input: &HashMap, serializer: S) -> Result -where - S: serde::Serializer, - V: Clone + Serialize, -{ - let transformed = input.iter().map(|(k, v)| (hex::encode(k), v.clone())); - - transformed - .collect::>() - .serialize(serializer) -} - -fn from_hex_map<'de, D, V>(deserializer: D) -> Result, D::Error> -where - D: serde::de::Deserializer<'de>, - V: Deserialize<'de>, -{ - let hex_map = HashMap::::deserialize(deserializer)?; - hex_map - .into_iter() - .map(|(k, v)| { - TenantId::from_hex(k) - .map(|k| (k, v)) - .map_err(serde::de::Error::custom) - }) - .collect() -} - -// Top level state available to all HTTP handlers -#[derive(Serialize, Deserialize)] -struct PersistentState { - #[serde(serialize_with = "to_hex_map", deserialize_with = "from_hex_map")] - tenants: HashMap, - - #[serde(skip)] - path: PathBuf, -} - -impl PersistentState { - async fn save(&self) -> anyhow::Result<()> { - let bytes = serde_json::to_vec(self)?; - tokio::fs::write(&self.path, &bytes).await?; - - Ok(()) - } - - async fn load(path: &Path) -> anyhow::Result { - let bytes = tokio::fs::read(path).await?; - let mut decoded = serde_json::from_slice::(&bytes)?; - decoded.path = path.to_owned(); - Ok(decoded) - } - - async fn load_or_new(path: &Path) -> Self { - match Self::load(path).await { - Ok(s) => { - tracing::info!("Loaded state file at {}", path.display()); - s - } - Err(e) - if e.downcast_ref::() - .map(|e| e.kind() == std::io::ErrorKind::NotFound) - .unwrap_or(false) => - { - tracing::info!("Will create state file at {}", path.display()); - Self { - tenants: HashMap::new(), - path: path.to_owned(), - } - } - Err(e) => { - panic!("Failed to load state from '{}': {e:#} (maybe your .neon/ dir was written by an older version?)", path.display()) - } - } - } -} - -/// State available to HTTP request handlers -#[derive(Clone)] -struct State { - inner: Arc>, -} - -impl State { - fn new(persistent_state: PersistentState) -> State { - Self { - inner: Arc::new(tokio::sync::RwLock::new(persistent_state)), - } - } -} - -#[inline(always)] -fn get_state(request: &Request) -> &State { - request - .data::>() - .expect("unknown state type") - .as_ref() -} - -/// Pageserver calls into this on startup, to learn which tenants it should attach -async fn handle_re_attach(mut req: Request) -> Result, ApiError> { - let reattach_req = json_request::(&mut req).await?; - - let state = get_state(&req).inner.clone(); - let mut locked = state.write().await; - - let mut response = ReAttachResponse { - tenants: Vec::new(), - }; - for (t, state) in &mut locked.tenants { - if state.pageserver == Some(reattach_req.node_id) { - state.generation += 1; - response.tenants.push(ReAttachResponseTenant { - // TODO(sharding): make this shard-aware - id: TenantShardId::unsharded(*t), - gen: state.generation, - }); - } - } - - locked.save().await.map_err(ApiError::InternalServerError)?; - - json_response(StatusCode::OK, response) -} - -/// Pageserver calls into this before doing deletions, to confirm that it still -/// holds the latest generation for the tenants with deletions enqueued -async fn handle_validate(mut req: Request) -> Result, ApiError> { - let validate_req = json_request::(&mut req).await?; - - let locked = get_state(&req).inner.read().await; - - let mut response = ValidateResponse { - tenants: Vec::new(), - }; - - for req_tenant in validate_req.tenants { - // TODO(sharding): make this shard-aware - if let Some(tenant_state) = locked.tenants.get(&req_tenant.id.tenant_id) { - let valid = tenant_state.generation == req_tenant.gen; - tracing::info!( - "handle_validate: {}(gen {}): valid={valid} (latest {})", - req_tenant.id, - req_tenant.gen, - tenant_state.generation - ); - response.tenants.push(ValidateResponseTenant { - id: req_tenant.id, - valid, - }); - } - } - - json_response(StatusCode::OK, response) -} -/// Call into this before attaching a tenant to a pageserver, to acquire a generation number -/// (in the real control plane this is unnecessary, because the same program is managing -/// generation numbers and doing attachments). -async fn handle_attach_hook(mut req: Request) -> Result, ApiError> { - let attach_req = json_request::(&mut req).await?; - - let state = get_state(&req).inner.clone(); - let mut locked = state.write().await; - - let tenant_state = locked - .tenants - .entry(attach_req.tenant_id) - .or_insert_with(|| TenantState { - pageserver: attach_req.node_id, - generation: 0, - }); - - if let Some(attaching_pageserver) = attach_req.node_id.as_ref() { - tenant_state.generation += 1; - tracing::info!( - tenant_id = %attach_req.tenant_id, - ps_id = %attaching_pageserver, - generation = %tenant_state.generation, - "issuing", - ); - } else if let Some(ps_id) = tenant_state.pageserver { - tracing::info!( - tenant_id = %attach_req.tenant_id, - %ps_id, - generation = %tenant_state.generation, - "dropping", - ); - } else { - tracing::info!( - tenant_id = %attach_req.tenant_id, - "no-op: tenant already has no pageserver"); - } - tenant_state.pageserver = attach_req.node_id; - let generation = tenant_state.generation; - - tracing::info!( - "handle_attach_hook: tenant {} set generation {}, pageserver {}", - attach_req.tenant_id, - tenant_state.generation, - attach_req.node_id.unwrap_or(utils::id::NodeId(0xfffffff)) - ); - - locked.save().await.map_err(ApiError::InternalServerError)?; - - json_response( - StatusCode::OK, - AttachHookResponse { - gen: attach_req.node_id.map(|_| generation), - }, - ) -} - -async fn handle_inspect(mut req: Request) -> Result, ApiError> { - let inspect_req = json_request::(&mut req).await?; - - let state = get_state(&req).inner.clone(); - let locked = state.write().await; - let tenant_state = locked.tenants.get(&inspect_req.tenant_id); - - json_response( - StatusCode::OK, - InspectResponse { - attachment: tenant_state.and_then(|s| s.pageserver.map(|ps| (s.generation, ps))), - }, - ) -} - -async fn handle_tenant_create(mut req: Request) -> Result, ApiError> { - let inspect_req = json_request::(&mut req).await?; - - let state = get_state(&req).inner.clone(); - let locked = state.write().await; - let tenant_state = locked.tenants.get(&inspect_req.tenant_id); - - json_response( - StatusCode::OK, - InspectResponse { - attachment: tenant_state.and_then(|s| s.pageserver.map(|ps| (s.generation, ps))), - }, - ) -} - -fn make_router(persistent_state: PersistentState) -> RouterBuilder { - endpoint::make_router() - .data(Arc::new(State::new(persistent_state))) - .post("/re-attach", |r| request_span(r, handle_re_attach)) - .post("/validate", |r| request_span(r, handle_validate)) - .post("/attach-hook", |r| request_span(r, handle_attach_hook)) - .post("/inspect", |r| request_span(r, handle_inspect)) - .post("/tenant/:tenant_id", |r| { - request_span(r, handle_tenant_create) - }) -} - -#[tokio::main] -async fn main() -> anyhow::Result<()> { - logging::init( - LogFormat::Plain, - logging::TracingErrorLayerEnablement::Disabled, - logging::Output::Stdout, - )?; - - let args = Cli::parse(); - tracing::info!( - "Starting, state at {}, listening on {}", - args.path.to_string_lossy(), - args.listen - ); - - let persistent_state = PersistentState::load_or_new(&args.path).await; - - let http_listener = tcp_listener::bind(args.listen)?; - let router = make_router(persistent_state) - .build() - .map_err(|err| anyhow!(err))?; - let service = utils::http::RouterService::new(router).unwrap(); - let server = hyper::Server::from_tcp(http_listener)?.serve(service); - - tracing::info!("Serving on {0}", args.listen); - - tokio::task::spawn(server); - - ShutdownSignals::handle(|signal| match signal { - Signal::Interrupt | Signal::Terminate | Signal::Quit => { - tracing::info!("Got {}. Terminating", signal.name()); - // We're just a test helper: no graceful shutdown. - std::process::exit(0); - } - })?; - - Ok(()) -} diff --git a/control_plane/src/bin/neon_local.rs b/control_plane/src/bin/neon_local.rs index 23361f17f5..279c47398f 100644 --- a/control_plane/src/bin/neon_local.rs +++ b/control_plane/src/bin/neon_local.rs @@ -8,20 +8,24 @@ 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::attachment_service::{ + AttachmentService, NodeAvailability, NodeConfigureRequest, NodeSchedulingPolicy, +}; 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::tenant_migration::migrate_tenant; use control_plane::{broker, local_env}; -use pageserver_api::models::TimelineInfo; -use pageserver_api::shard::TenantShardId; +use pageserver_api::models::{ + ShardParameters, TenantCreateRequest, TimelineCreateRequest, TimelineInfo, +}; +use pageserver_api::shard::{ShardCount, ShardStripeSize, TenantShardId}; use pageserver_api::{ DEFAULT_HTTP_LISTEN_PORT as DEFAULT_PAGESERVER_HTTP_PORT, DEFAULT_PG_LISTEN_PORT as DEFAULT_PAGESERVER_PG_PORT, }; use postgres_backend::AuthType; +use postgres_connection::parse_host_port; use safekeeper_api::{ DEFAULT_HTTP_LISTEN_PORT as DEFAULT_SAFEKEEPER_HTTP_PORT, DEFAULT_PG_LISTEN_PORT as DEFAULT_SAFEKEEPER_PG_PORT, @@ -31,6 +35,7 @@ use std::path::PathBuf; use std::process::exit; use std::str::FromStr; use storage_broker::DEFAULT_LISTEN_ADDR as DEFAULT_BROKER_ADDR; +use url::Host; use utils::{ auth::{Claims, Scope}, id::{NodeId, TenantId, TenantTimelineId, TimelineId}, @@ -277,10 +282,10 @@ fn print_timeline( /// Connects to the pageserver to query this information. async fn get_timeline_infos( env: &local_env::LocalEnv, - tenant_id: &TenantId, + tenant_shard_id: &TenantShardId, ) -> Result> { Ok(get_default_pageserver(env) - .timeline_list(&TenantShardId::unsharded(*tenant_id)) + .timeline_list(tenant_shard_id) .await? .into_iter() .map(|timeline_info| (timeline_info.timeline_id, timeline_info)) @@ -298,6 +303,20 @@ fn get_tenant_id(sub_match: &ArgMatches, env: &local_env::LocalEnv) -> anyhow::R } } +// Helper function to parse --tenant_id option, for commands that accept a shard suffix +fn get_tenant_shard_id( + sub_match: &ArgMatches, + env: &local_env::LocalEnv, +) -> anyhow::Result { + if let Some(tenant_id_from_arguments) = parse_tenant_shard_id(sub_match).transpose() { + tenant_id_from_arguments + } else if let Some(default_id) = env.default_tenant_id { + Ok(TenantShardId::unsharded(default_id)) + } else { + anyhow::bail!("No tenant shard id. Use --tenant-id, or set a default tenant"); + } +} + fn parse_tenant_id(sub_match: &ArgMatches) -> anyhow::Result> { sub_match .get_one::("tenant-id") @@ -306,6 +325,14 @@ fn parse_tenant_id(sub_match: &ArgMatches) -> anyhow::Result> { .context("Failed to parse tenant id from the argument string") } +fn parse_tenant_shard_id(sub_match: &ArgMatches) -> anyhow::Result> { + sub_match + .get_one::("tenant-id") + .map(|id_str| TenantShardId::from_str(id_str)) + .transpose() + .context("Failed to parse tenant shard id from the argument string") +} + fn parse_timeline_id(sub_match: &ArgMatches) -> anyhow::Result> { sub_match .get_one::("timeline-id") @@ -394,47 +421,68 @@ async fn handle_tenant( Some(("create", create_match)) => { let tenant_conf: HashMap<_, _> = create_match .get_many::("config") - .map(|vals| vals.flat_map(|c| c.split_once(':')).collect()) + .map(|vals: clap::parser::ValuesRef<'_, String>| { + vals.flat_map(|c| c.split_once(':')).collect() + }) .unwrap_or_default(); + let shard_count: u8 = create_match + .get_one::("shard-count") + .cloned() + .unwrap_or(0); + + let shard_stripe_size: Option = + create_match.get_one::("shard-stripe-size").cloned(); + + let tenant_conf = PageServerNode::parse_config(tenant_conf)?; + // If tenant ID was not specified, generate one let tenant_id = parse_tenant_id(create_match)?.unwrap_or_else(TenantId::generate); - let generation = if env.control_plane_api.is_some() { - // We must register the tenant with the attachment service, so - // that when the pageserver restarts, it will be re-attached. - let attachment_service = AttachmentService::from_env(env); - attachment_service - .attach_hook(tenant_id, pageserver.conf.id) - .await? - } else { - None - }; - - pageserver - .tenant_create(tenant_id, generation, tenant_conf) + // We must register the tenant with the attachment service, so + // that when the pageserver restarts, it will be re-attached. + let attachment_service = AttachmentService::from_env(env); + attachment_service + .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) + new_tenant_id: TenantShardId::unsharded(tenant_id), + generation: None, + shard_parameters: ShardParameters { + count: ShardCount(shard_count), + stripe_size: shard_stripe_size + .map(ShardStripeSize) + .unwrap_or(ShardParameters::DEFAULT_STRIPE_SIZE), + }, + config: tenant_conf, + }) .await?; println!("tenant {tenant_id} successfully created on the pageserver"); // Create an initial timeline for the new tenant - let new_timeline_id = parse_timeline_id(create_match)?; + let new_timeline_id = + parse_timeline_id(create_match)?.unwrap_or(TimelineId::generate()); let pg_version = create_match .get_one::("pg-version") .copied() .context("Failed to parse postgres version from the argument string")?; - let timeline_info = pageserver - .timeline_create( + // 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 + // to let shard 0 branch first and then propagate the chosen LSN to other shards. + attachment_service + .tenant_timeline_create( tenant_id, - new_timeline_id, - None, - None, - Some(pg_version), - None, + TimelineCreateRequest { + new_timeline_id, + ancestor_timeline_id: None, + ancestor_start_lsn: None, + existing_initdb_timeline_id: None, + pg_version: Some(pg_version), + }, ) .await?; - let new_timeline_id = timeline_info.timeline_id; - let last_record_lsn = timeline_info.last_record_lsn; env.register_branch_mapping( DEFAULT_BRANCH_NAME.to_string(), @@ -442,9 +490,7 @@ async fn handle_tenant( new_timeline_id, )?; - println!( - "Created an initial timeline '{new_timeline_id}' at Lsn {last_record_lsn} for tenant: {tenant_id}", - ); + println!("Created an initial timeline '{new_timeline_id}' for tenant: {tenant_id}",); if create_match.get_flag("set-default") { println!("Setting tenant {tenant_id} as a default one"); @@ -471,14 +517,64 @@ async fn handle_tenant( println!("tenant {tenant_id} successfully configured on the pageserver"); } Some(("migrate", matches)) => { - let tenant_id = get_tenant_id(matches, env)?; + let tenant_shard_id = get_tenant_shard_id(matches, env)?; let new_pageserver = get_pageserver(env, matches)?; let new_pageserver_id = new_pageserver.conf.id; - migrate_tenant(env, tenant_id, new_pageserver).await?; - println!("tenant {tenant_id} migrated to {}", new_pageserver_id); - } + let attachment_service = AttachmentService::from_env(env); + attachment_service + .tenant_migrate(tenant_shard_id, new_pageserver_id) + .await?; + println!("tenant {tenant_shard_id} migrated to {}", new_pageserver_id); + } + Some(("status", matches)) => { + let tenant_id = get_tenant_id(matches, env)?; + + let mut shard_table = comfy_table::Table::new(); + shard_table.set_header(["Shard", "Pageserver", "Physical Size"]); + + let mut tenant_synthetic_size = None; + + let attachment_service = AttachmentService::from_env(env); + for shard in attachment_service.tenant_locate(tenant_id).await?.shards { + let pageserver = + PageServerNode::from_env(env, env.get_pageserver_conf(shard.node_id)?); + + let size = pageserver + .http_client + .tenant_details(shard.shard_id) + .await? + .tenant_info + .current_physical_size + .unwrap(); + + shard_table.add_row([ + format!("{}", shard.shard_id.shard_slug()), + format!("{}", shard.node_id.0), + format!("{} MiB", size / (1024 * 1024)), + ]); + + if shard.shard_id.is_zero() { + tenant_synthetic_size = + Some(pageserver.tenant_synthetic_size(shard.shard_id).await?); + } + } + + let Some(synthetic_size) = tenant_synthetic_size else { + bail!("Shard 0 not found") + }; + + let mut tenant_table = comfy_table::Table::new(); + tenant_table.add_row(["Tenant ID".to_string(), tenant_id.to_string()]); + tenant_table.add_row([ + "Synthetic size".to_string(), + format!("{} MiB", synthetic_size.size.unwrap_or(0) / (1024 * 1024)), + ]); + + println!("{tenant_table}"); + println!("{shard_table}"); + } Some((sub_name, _)) => bail!("Unexpected tenant subcommand '{}'", sub_name), None => bail!("no tenant subcommand provided"), } @@ -490,10 +586,10 @@ async fn handle_timeline(timeline_match: &ArgMatches, env: &mut local_env::Local match timeline_match.subcommand() { Some(("list", list_match)) => { - let tenant_id = get_tenant_id(list_match, env)?; - let timelines = pageserver - .timeline_list(&TenantShardId::unsharded(tenant_id)) - .await?; + // TODO(sharding): this command shouldn't have to specify a shard ID: we should ask the attachment service + // 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?; print_timelines_tree(timelines, env.timeline_name_mappings())?; } Some(("create", create_match)) => { @@ -508,18 +604,19 @@ async fn handle_timeline(timeline_match: &ArgMatches, env: &mut local_env::Local .context("Failed to parse postgres version from the argument string")?; let new_timeline_id_opt = parse_timeline_id(create_match)?; + let new_timeline_id = new_timeline_id_opt.unwrap_or(TimelineId::generate()); - let timeline_info = pageserver - .timeline_create( - tenant_id, - new_timeline_id_opt, - None, - None, - Some(pg_version), - None, - ) + let attachment_service = AttachmentService::from_env(env); + let create_req = TimelineCreateRequest { + new_timeline_id, + ancestor_timeline_id: None, + existing_initdb_timeline_id: None, + ancestor_start_lsn: None, + pg_version: Some(pg_version), + }; + let timeline_info = attachment_service + .tenant_timeline_create(tenant_id, create_req) .await?; - let new_timeline_id = timeline_info.timeline_id; let last_record_lsn = timeline_info.last_record_lsn; env.register_branch_mapping(new_branch_name.to_string(), tenant_id, new_timeline_id)?; @@ -577,7 +674,6 @@ async fn handle_timeline(timeline_match: &ArgMatches, env: &mut local_env::Local None, pg_version, ComputeMode::Primary, - DEFAULT_PAGESERVER_ID, )?; println!("Done"); } @@ -601,17 +697,18 @@ async fn handle_timeline(timeline_match: &ArgMatches, env: &mut local_env::Local .map(|lsn_str| Lsn::from_str(lsn_str)) .transpose() .context("Failed to parse ancestor start Lsn from the request")?; - let timeline_info = pageserver - .timeline_create( - tenant_id, - None, - start_lsn, - Some(ancestor_timeline_id), - None, - None, - ) + let new_timeline_id = TimelineId::generate(); + let attachment_service = AttachmentService::from_env(env); + let create_req = TimelineCreateRequest { + new_timeline_id, + ancestor_timeline_id: Some(ancestor_timeline_id), + existing_initdb_timeline_id: None, + ancestor_start_lsn: start_lsn, + pg_version: None, + }; + let timeline_info = attachment_service + .tenant_timeline_create(tenant_id, create_req) .await?; - let new_timeline_id = timeline_info.timeline_id; let last_record_lsn = timeline_info.last_record_lsn; @@ -638,8 +735,10 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re match sub_name { "list" => { - let tenant_id = get_tenant_id(sub_args, env)?; - let timeline_infos = get_timeline_infos(env, &tenant_id) + // TODO(sharding): this command shouldn't have to specify a shard ID: we should ask the attachment service + // 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) .await .unwrap_or_else(|e| { eprintln!("Failed to load timeline info: {}", e); @@ -664,7 +763,7 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re for (endpoint_id, endpoint) in cplane .endpoints .iter() - .filter(|(_, endpoint)| endpoint.tenant_id == tenant_id) + .filter(|(_, endpoint)| endpoint.tenant_id == tenant_shard_id.tenant_id) { let lsn_str = match endpoint.mode { ComputeMode::Static(lsn) => { @@ -683,7 +782,10 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re }; let branch_name = timeline_name_mappings - .get(&TenantTimelineId::new(tenant_id, endpoint.timeline_id)) + .get(&TenantTimelineId::new( + tenant_shard_id.tenant_id, + endpoint.timeline_id, + )) .map(|name| name.as_str()) .unwrap_or("?"); @@ -731,13 +833,6 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re .copied() .unwrap_or(false); - let pageserver_id = - if let Some(id_str) = sub_args.get_one::("endpoint-pageserver-id") { - NodeId(id_str.parse().context("while parsing pageserver id")?) - } else { - DEFAULT_PAGESERVER_ID - }; - let mode = match (lsn, hot_standby) { (Some(lsn), false) => ComputeMode::Static(lsn), (None, true) => ComputeMode::Replica, @@ -765,7 +860,6 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re http_port, pg_version, mode, - pageserver_id, )?; } "start" => { @@ -775,9 +869,11 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re let pageserver_id = if let Some(id_str) = sub_args.get_one::("endpoint-pageserver-id") { - NodeId(id_str.parse().context("while parsing pageserver id")?) + Some(NodeId( + id_str.parse().context("while parsing pageserver id")?, + )) } else { - DEFAULT_PAGESERVER_ID + None }; let remote_ext_config = sub_args.get_one::("remote-ext-config"); @@ -808,7 +904,38 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re endpoint.timeline_id, )?; - let ps_conf = env.get_pageserver_conf(pageserver_id)?; + let (pageservers, stripe_size) = if let Some(pageserver_id) = pageserver_id { + let conf = env.get_pageserver_conf(pageserver_id).unwrap(); + let parsed = parse_host_port(&conf.listen_pg_addr).expect("Bad config"); + ( + 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. + 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 pageservers = locate_result + .shards + .into_iter() + .map(|shard| { + ( + Host::parse(&shard.listen_pg_addr) + .expect("Attachment service reported bad hostname"), + shard.listen_pg_port, + ) + }) + .collect::>(); + let stripe_size = locate_result.shard_params.stripe_size; + + (pageservers, stripe_size) + }; + assert!(!pageservers.is_empty()); + + let ps_conf = env.get_pageserver_conf(DEFAULT_PAGESERVER_ID)?; let auth_token = if matches!(ps_conf.pg_auth_type, AuthType::NeonJWT) { let claims = Claims::new(Some(endpoint.tenant_id), Scope::Tenant); @@ -819,7 +946,13 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re println!("Starting existing endpoint {endpoint_id}..."); endpoint - .start(&auth_token, safekeepers, remote_ext_config) + .start( + &auth_token, + safekeepers, + pageservers, + remote_ext_config, + stripe_size.0 as usize, + ) .await?; } "reconfigure" => { @@ -830,15 +963,31 @@ async fn handle_endpoint(ep_match: &ArgMatches, env: &local_env::LocalEnv) -> Re .endpoints .get(endpoint_id.as_str()) .with_context(|| format!("postgres endpoint {endpoint_id} is not found"))?; - let pageserver_id = + let pageservers = if let Some(id_str) = sub_args.get_one::("endpoint-pageserver-id") { - Some(NodeId( - id_str.parse().context("while parsing pageserver id")?, - )) + let ps_id = NodeId(id_str.parse().context("while parsing pageserver id")?); + let pageserver = PageServerNode::from_env(env, env.get_pageserver_conf(ps_id)?); + vec![( + pageserver.pg_connection_config.host().clone(), + pageserver.pg_connection_config.port(), + )] } else { - None + let attachment_service = AttachmentService::from_env(env); + attachment_service + .tenant_locate(endpoint.tenant_id) + .await? + .shards + .into_iter() + .map(|shard| { + ( + Host::parse(&shard.listen_pg_addr) + .expect("Attachment service reported malformed host"), + shard.listen_pg_port, + ) + }) + .collect::>() }; - endpoint.reconfigure(pageserver_id).await?; + endpoint.reconfigure(pageservers).await?; } "stop" => { let endpoint_id = sub_args @@ -962,6 +1111,21 @@ async fn handle_pageserver(sub_match: &ArgMatches, env: &local_env::LocalEnv) -> } } + Some(("set-state", subcommand_args)) => { + let pageserver = get_pageserver(env, subcommand_args)?; + let scheduling = subcommand_args.get_one("scheduling"); + let availability = subcommand_args.get_one("availability"); + + let attachment_service = AttachmentService::from_env(env); + attachment_service + .node_configure(NodeConfigureRequest { + node_id: pageserver.conf.id, + scheduling: scheduling.cloned(), + availability: availability.cloned(), + }) + .await?; + } + Some(("status", subcommand_args)) => { match get_pageserver(env, subcommand_args)?.check_status().await { Ok(_) => println!("Page server is up and running"), @@ -1361,6 +1525,8 @@ fn cli() -> Command { .arg(pg_version_arg.clone()) .arg(Arg::new("set-default").long("set-default").action(ArgAction::SetTrue).required(false) .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")) ) .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")) @@ -1371,6 +1537,9 @@ fn cli() -> Command { .about("Migrate a tenant from one pageserver to another") .arg(tenant_id_arg.clone()) .arg(pageserver_id_arg.clone())) + .subcommand(Command::new("status") + .about("Human readable summary of the tenant's shards and attachment locations") + .arg(tenant_id_arg.clone())) ) .subcommand( Command::new("pageserver") @@ -1390,6 +1559,12 @@ fn cli() -> Command { .about("Restart local pageserver") .arg(pageserver_config_args.clone()) ) + .subcommand(Command::new("set-state") + .arg(Arg::new("availability").value_parser(value_parser!(NodeAvailability)).long("availability").action(ArgAction::Set).help("Availability state: offline,active")) + .arg(Arg::new("scheduling").value_parser(value_parser!(NodeSchedulingPolicy)).long("scheduling").action(ArgAction::Set).help("Scheduling state: draining,pause,filling,active")) + .about("Set scheduling or availability state of pageserver node") + .arg(pageserver_config_args.clone()) + ) ) .subcommand( Command::new("attachment_service") diff --git a/control_plane/src/endpoint.rs b/control_plane/src/endpoint.rs index d2843c18c0..43f8ea3b43 100644 --- a/control_plane/src/endpoint.rs +++ b/control_plane/src/endpoint.rs @@ -48,12 +48,12 @@ use anyhow::{anyhow, bail, Context, Result}; use compute_api::spec::RemoteExtSpec; use nix::sys::signal::kill; use nix::sys::signal::Signal; -use pageserver_api::models::ShardParameters; 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::pageserver::PageServerNode; use crate::postgresql_conf::PostgresConf; use compute_api::responses::{ComputeState, ComputeStatus}; @@ -70,7 +70,6 @@ pub struct EndpointConf { http_port: u16, pg_version: u32, skip_pg_catalog_updates: bool, - pageserver_id: NodeId, } // @@ -122,19 +121,14 @@ impl ComputeControlPlane { http_port: Option, pg_version: u32, mode: ComputeMode, - pageserver_id: NodeId, ) -> Result> { let pg_port = pg_port.unwrap_or_else(|| self.get_port()); let http_port = http_port.unwrap_or_else(|| self.get_port() + 1); - let pageserver = - PageServerNode::from_env(&self.env, self.env.get_pageserver_conf(pageserver_id)?); - let ep = Arc::new(Endpoint { endpoint_id: endpoint_id.to_owned(), pg_address: SocketAddr::new("127.0.0.1".parse().unwrap(), pg_port), http_address: SocketAddr::new("127.0.0.1".parse().unwrap(), http_port), env: self.env.clone(), - pageserver, timeline_id, mode, tenant_id, @@ -160,7 +154,6 @@ impl ComputeControlPlane { pg_port, pg_version, skip_pg_catalog_updates: true, - pageserver_id, })?, )?; std::fs::write( @@ -219,7 +212,6 @@ pub struct Endpoint { // These are not part of the endpoint as such, but the environment // the endpoint runs in. pub env: LocalEnv, - pageserver: PageServerNode, // Optimizations skip_pg_catalog_updates: bool, @@ -242,15 +234,11 @@ impl Endpoint { let conf: EndpointConf = serde_json::from_slice(&std::fs::read(entry.path().join("endpoint.json"))?)?; - let pageserver = - PageServerNode::from_env(env, env.get_pageserver_conf(conf.pageserver_id)?); - Ok(Endpoint { pg_address: SocketAddr::new("127.0.0.1".parse().unwrap(), conf.pg_port), http_address: SocketAddr::new("127.0.0.1".parse().unwrap(), conf.http_port), endpoint_id, env: env.clone(), - pageserver, timeline_id: conf.timeline_id, mode: conf.mode, tenant_id: conf.tenant_id, @@ -470,11 +458,21 @@ impl Endpoint { } } + fn build_pageserver_connstr(pageservers: &[(Host, u16)]) -> String { + pageservers + .iter() + .map(|(host, port)| format!("postgresql://no_user@{host}:{port}")) + .collect::>() + .join(",") + } + pub async fn start( &self, auth_token: &Option, safekeepers: Vec, + pageservers: Vec<(Host, u16)>, remote_ext_config: Option<&String>, + shard_stripe_size: usize, ) -> Result<()> { if self.status() == "running" { anyhow::bail!("The endpoint is already running"); @@ -488,13 +486,9 @@ impl Endpoint { std::fs::remove_dir_all(self.pgdata())?; } - let pageserver_connstring = { - let config = &self.pageserver.pg_connection_config; - let (host, port) = (config.host(), config.port()); + let pageserver_connstring = Self::build_pageserver_connstr(&pageservers); + assert!(!pageserver_connstring.is_empty()); - // NOTE: avoid spaces in connection string, because it is less error prone if we forward it somewhere. - format!("postgresql://no_user@{host}:{port}") - }; let mut safekeeper_connstrings = Vec::new(); if self.mode == ComputeMode::Primary { for sk_id in safekeepers { @@ -544,7 +538,7 @@ impl Endpoint { storage_auth_token: auth_token.clone(), remote_extensions, pgbouncer_settings: None, - shard_stripe_size: Some(ShardParameters::DEFAULT_STRIPE_SIZE.0 as usize), + shard_stripe_size: Some(shard_stripe_size), }; let spec_path = self.endpoint_path().join("spec.json"); std::fs::write(spec_path, serde_json::to_string_pretty(&spec)?)?; @@ -667,7 +661,7 @@ impl Endpoint { } } - pub async fn reconfigure(&self, pageserver_id: Option) -> Result<()> { + pub async fn reconfigure(&self, mut pageservers: Vec<(Host, u16)>) -> Result<()> { let mut spec: ComputeSpec = { let spec_path = self.endpoint_path().join("spec.json"); let file = std::fs::File::open(spec_path)?; @@ -677,25 +671,27 @@ impl Endpoint { let postgresql_conf = self.read_postgresql_conf()?; spec.cluster.postgresql_conf = Some(postgresql_conf); - if let Some(pageserver_id) = pageserver_id { - let endpoint_config_path = self.endpoint_path().join("endpoint.json"); - let mut endpoint_conf: EndpointConf = { - let file = std::fs::File::open(&endpoint_config_path)?; - serde_json::from_reader(file)? - }; - endpoint_conf.pageserver_id = pageserver_id; - std::fs::write( - endpoint_config_path, - serde_json::to_string_pretty(&endpoint_conf)?, - )?; - - let pageserver = - PageServerNode::from_env(&self.env, self.env.get_pageserver_conf(pageserver_id)?); - let ps_http_conf = &pageserver.pg_connection_config; - let (host, port) = (ps_http_conf.host(), ps_http_conf.port()); - spec.pageserver_connstring = Some(format!("postgresql://no_user@{host}:{port}")); + // If we weren't given explicit pageservers, query the attachment service + if pageservers.is_empty() { + let attachment_service = AttachmentService::from_env(&self.env); + let locate_result = attachment_service.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"), + shard.listen_pg_port, + ) + }) + .collect::>(); } + let pageserver_connstr = Self::build_pageserver_connstr(&pageservers); + assert!(!pageserver_connstr.is_empty()); + spec.pageserver_connstring = Some(pageserver_connstr); + let client = reqwest::Client::new(); let response = client .post(format!( diff --git a/control_plane/src/lib.rs b/control_plane/src/lib.rs index 52a0e20429..bb79d36bfc 100644 --- a/control_plane/src/lib.rs +++ b/control_plane/src/lib.rs @@ -14,4 +14,3 @@ pub mod local_env; pub mod pageserver; pub mod postgresql_conf; pub mod safekeeper; -pub mod tenant_migration; diff --git a/control_plane/src/local_env.rs b/control_plane/src/local_env.rs index c87875457c..4460fdd3a6 100644 --- a/control_plane/src/local_env.rs +++ b/control_plane/src/local_env.rs @@ -251,7 +251,13 @@ impl LocalEnv { if let Some(conf) = self.pageservers.iter().find(|node| node.id == id) { Ok(conf) } else { - bail!("could not find pageserver {id}") + let have_ids = self + .pageservers + .iter() + .map(|node| format!("{}:{}", node.id, node.listen_http_addr)) + .collect::>(); + let joined = have_ids.join(","); + bail!("could not find pageserver {id}, have ids {joined}") } } diff --git a/control_plane/src/pageserver.rs b/control_plane/src/pageserver.rs index 82e01172c0..18ccf6bd98 100644 --- a/control_plane/src/pageserver.rs +++ b/control_plane/src/pageserver.rs @@ -17,7 +17,9 @@ use std::time::Duration; use anyhow::{bail, Context}; use camino::Utf8PathBuf; use futures::SinkExt; -use pageserver_api::models::{self, LocationConfig, ShardParameters, TenantInfo, TimelineInfo}; +use pageserver_api::models::{ + self, LocationConfig, ShardParameters, TenantHistorySize, TenantInfo, TimelineInfo, +}; use pageserver_api::shard::TenantShardId; use pageserver_client::mgmt_api; use postgres_backend::AuthType; @@ -106,6 +108,16 @@ impl PageServerNode { "control_plane_api='{}'", control_plane_api.as_str() )); + + // Attachment service 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 + .env + .generate_auth_token(&Claims::new(None, Scope::PageServerApi)) + .unwrap(); + overrides.push(format!("control_plane_api_token='{}'", jwt_token)); + } } if !cli_overrides @@ -301,16 +313,8 @@ impl PageServerNode { pub async fn tenant_list(&self) -> mgmt_api::Result> { self.http_client.list_tenants().await } - - pub async fn tenant_create( - &self, - new_tenant_id: TenantId, - generation: Option, - settings: HashMap<&str, &str>, - ) -> anyhow::Result { - let mut settings = settings.clone(); - - let config = models::TenantConfig { + pub fn parse_config(mut settings: HashMap<&str, &str>) -> anyhow::Result { + let result = models::TenantConfig { checkpoint_distance: settings .remove("checkpoint_distance") .map(|x| x.parse::()) @@ -371,6 +375,20 @@ impl PageServerNode { .context("Failed to parse 'gc_feedback' as bool")?, heatmap_period: settings.remove("heatmap_period").map(|x| x.to_string()), }; + if !settings.is_empty() { + bail!("Unrecognized tenant settings: {settings:?}") + } else { + Ok(result) + } + } + + pub async fn tenant_create( + &self, + new_tenant_id: TenantId, + generation: Option, + settings: HashMap<&str, &str>, + ) -> anyhow::Result { + let config = Self::parse_config(settings.clone())?; let request = models::TenantCreateRequest { new_tenant_id: TenantShardId::unsharded(new_tenant_id), @@ -498,15 +516,13 @@ impl PageServerNode { pub async fn timeline_create( &self, - tenant_id: TenantId, - new_timeline_id: Option, + tenant_shard_id: TenantShardId, + new_timeline_id: TimelineId, ancestor_start_lsn: Option, ancestor_timeline_id: Option, pg_version: Option, existing_initdb_timeline_id: Option, ) -> anyhow::Result { - // If timeline ID was not specified, generate one - let new_timeline_id = new_timeline_id.unwrap_or(TimelineId::generate()); let req = models::TimelineCreateRequest { new_timeline_id, ancestor_start_lsn, @@ -514,7 +530,10 @@ impl PageServerNode { pg_version, existing_initdb_timeline_id, }; - Ok(self.http_client.timeline_create(tenant_id, &req).await?) + Ok(self + .http_client + .timeline_create(tenant_shard_id, &req) + .await?) } /// Import a basebackup prepared using either: @@ -592,4 +611,14 @@ impl PageServerNode { Ok(()) } + + pub async fn tenant_synthetic_size( + &self, + tenant_shard_id: TenantShardId, + ) -> anyhow::Result { + Ok(self + .http_client + .tenant_synthetic_size(tenant_shard_id) + .await?) + } } diff --git a/control_plane/src/tenant_migration.rs b/control_plane/src/tenant_migration.rs deleted file mode 100644 index 8f46abd54f..0000000000 --- a/control_plane/src/tenant_migration.rs +++ /dev/null @@ -1,232 +0,0 @@ -//! -//! Functionality for migrating tenants across pageservers: unlike most of neon_local, this code -//! isn't scoped to a particular physical service, as it needs to update compute endpoints to -//! point to the new pageserver. -//! -use crate::local_env::LocalEnv; -use crate::{ - attachment_service::AttachmentService, endpoint::ComputeControlPlane, - pageserver::PageServerNode, -}; -use pageserver_api::models::{ - LocationConfig, LocationConfigMode, LocationConfigSecondary, TenantConfig, -}; -use pageserver_api::shard::TenantShardId; -use std::collections::HashMap; -use std::time::Duration; -use utils::{ - id::{TenantId, TimelineId}, - lsn::Lsn, -}; - -/// Given an attached pageserver, retrieve the LSN for all timelines -async fn get_lsns( - tenant_id: TenantId, - pageserver: &PageServerNode, -) -> anyhow::Result> { - let timelines = pageserver - .timeline_list(&TenantShardId::unsharded(tenant_id)) - .await?; - Ok(timelines - .into_iter() - .map(|t| (t.timeline_id, t.last_record_lsn)) - .collect()) -} - -/// Wait for the timeline LSNs on `pageserver` to catch up with or overtake -/// `baseline`. -async fn await_lsn( - tenant_id: TenantId, - pageserver: &PageServerNode, - baseline: HashMap, -) -> anyhow::Result<()> { - loop { - let latest = match get_lsns(tenant_id, pageserver).await { - Ok(l) => l, - Err(_e) => { - println!( - "πŸ•‘ Waiting for pageserver {} to activate...", - pageserver.conf.id - ); - std::thread::sleep(Duration::from_millis(500)); - continue; - } - }; - - let mut any_behind: bool = false; - for (timeline_id, baseline_lsn) in &baseline { - match latest.get(timeline_id) { - Some(latest_lsn) => { - println!("πŸ•‘ LSN origin {baseline_lsn} vs destination {latest_lsn}"); - if latest_lsn < baseline_lsn { - any_behind = true; - } - } - None => { - // Expected timeline isn't yet visible on migration destination. - // (IRL we would have to account for timeline deletion, but this - // is just test helper) - any_behind = true; - } - } - } - - if !any_behind { - println!("βœ… LSN caught up. Proceeding..."); - break; - } else { - std::thread::sleep(Duration::from_millis(500)); - } - } - - Ok(()) -} - -/// This function spans multiple services, to demonstrate live migration of a tenant -/// between pageservers: -/// - Coordinate attach/secondary/detach on pageservers -/// - call into attachment_service for generations -/// - reconfigure compute endpoints to point to new attached pageserver -pub async fn migrate_tenant( - env: &LocalEnv, - tenant_id: TenantId, - dest_ps: PageServerNode, -) -> anyhow::Result<()> { - println!("πŸ€” Checking existing status..."); - let attachment_service = AttachmentService::from_env(env); - - fn build_location_config( - mode: LocationConfigMode, - generation: Option, - secondary_conf: Option, - ) -> LocationConfig { - LocationConfig { - mode, - generation, - secondary_conf, - tenant_conf: TenantConfig::default(), - shard_number: 0, - shard_count: 0, - shard_stripe_size: 0, - } - } - - let previous = attachment_service.inspect(tenant_id).await?; - let mut baseline_lsns = None; - if let Some((generation, origin_ps_id)) = &previous { - let origin_ps = PageServerNode::from_env(env, env.get_pageserver_conf(*origin_ps_id)?); - - if origin_ps_id == &dest_ps.conf.id { - println!("πŸ” Already attached to {origin_ps_id}, freshening..."); - let gen = attachment_service - .attach_hook(tenant_id, dest_ps.conf.id) - .await?; - let dest_conf = build_location_config(LocationConfigMode::AttachedSingle, gen, None); - dest_ps - .location_config(TenantShardId::unsharded(tenant_id), dest_conf, None) - .await?; - println!("βœ… Migration complete"); - return Ok(()); - } - - println!("πŸ” Switching origin pageserver {origin_ps_id} to stale mode"); - - let stale_conf = - build_location_config(LocationConfigMode::AttachedStale, Some(*generation), None); - origin_ps - .location_config( - TenantShardId::unsharded(tenant_id), - stale_conf, - Some(Duration::from_secs(10)), - ) - .await?; - - baseline_lsns = Some(get_lsns(tenant_id, &origin_ps).await?); - } - - println!( - "πŸ” Downloading latest layers to destination pageserver {}", - dest_ps.conf.id - ); - match dest_ps - .tenant_secondary_download(&TenantShardId::unsharded(tenant_id)) - .await - { - Ok(()) => {} - Err(_) => { - println!(" (skipping, destination wasn't in secondary mode)") - } - } - - let gen = attachment_service - .attach_hook(tenant_id, dest_ps.conf.id) - .await?; - let dest_conf = build_location_config(LocationConfigMode::AttachedMulti, gen, None); - - println!("πŸ” Attaching to pageserver {}", dest_ps.conf.id); - dest_ps - .location_config(TenantShardId::unsharded(tenant_id), dest_conf, None) - .await?; - - if let Some(baseline) = baseline_lsns { - println!("πŸ•‘ Waiting for LSN to catch up..."); - await_lsn(tenant_id, &dest_ps, baseline).await?; - } - - let cplane = ComputeControlPlane::load(env.clone())?; - for (endpoint_name, endpoint) in &cplane.endpoints { - if endpoint.tenant_id == tenant_id { - println!( - "πŸ” Reconfiguring endpoint {} to use pageserver {}", - endpoint_name, dest_ps.conf.id - ); - endpoint.reconfigure(Some(dest_ps.conf.id)).await?; - } - } - - for other_ps_conf in &env.pageservers { - if other_ps_conf.id == dest_ps.conf.id { - continue; - } - - let other_ps = PageServerNode::from_env(env, other_ps_conf); - let other_ps_tenants = other_ps.tenant_list().await?; - - // Check if this tenant is attached - let found = other_ps_tenants - .into_iter() - .map(|t| t.id) - .any(|i| i.tenant_id == tenant_id); - if !found { - continue; - } - - // Downgrade to a secondary location - let secondary_conf = build_location_config( - LocationConfigMode::Secondary, - None, - Some(LocationConfigSecondary { warm: true }), - ); - - println!( - "πŸ’€ Switching to secondary mode on pageserver {}", - other_ps.conf.id - ); - other_ps - .location_config(TenantShardId::unsharded(tenant_id), secondary_conf, None) - .await?; - } - - println!( - "πŸ” Switching to AttachedSingle mode on pageserver {}", - dest_ps.conf.id - ); - let dest_conf = build_location_config(LocationConfigMode::AttachedSingle, gen, None); - dest_ps - .location_config(TenantShardId::unsharded(tenant_id), dest_conf, None) - .await?; - - println!("βœ… Migration complete"); - - Ok(()) -} diff --git a/libs/pageserver_api/src/models.rs b/libs/pageserver_api/src/models.rs index d423888c8b..86d2c2a7ca 100644 --- a/libs/pageserver_api/src/models.rs +++ b/libs/pageserver_api/src/models.rs @@ -251,7 +251,7 @@ impl std::ops::Deref for TenantCreateRequest { /// An alternative representation of `pageserver::tenant::TenantConf` with /// simpler types. -#[derive(Serialize, Deserialize, Debug, Default)] +#[derive(Serialize, Deserialize, Debug, Default, Clone, Eq, PartialEq)] pub struct TenantConfig { pub checkpoint_distance: Option, pub checkpoint_timeout: Option, @@ -300,7 +300,7 @@ pub struct EvictionPolicyLayerAccessThreshold { /// A flattened analog of a `pagesever::tenant::LocationMode`, which /// lists out all possible states (and the virtual "Detached" state) /// in a flat form rather than using rust-style enums. -#[derive(Serialize, Deserialize, Debug)] +#[derive(Serialize, Deserialize, Debug, Clone, Eq, PartialEq)] pub enum LocationConfigMode { AttachedSingle, AttachedMulti, @@ -309,14 +309,14 @@ pub enum LocationConfigMode { Detached, } -#[derive(Serialize, Deserialize, Debug)] +#[derive(Serialize, Deserialize, Debug, Clone, Eq, PartialEq)] pub struct LocationConfigSecondary { pub warm: bool, } /// An alternative representation of `pageserver::tenant::LocationConf`, /// for use in external-facing APIs. -#[derive(Serialize, Deserialize, Debug)] +#[derive(Serialize, Deserialize, Debug, Clone, Eq, PartialEq)] pub struct LocationConfig { pub mode: LocationConfigMode, /// If attaching, in what generation? diff --git a/libs/utils/src/id.rs b/libs/utils/src/id.rs index 57dcc27719..0409001f4f 100644 --- a/libs/utils/src/id.rs +++ b/libs/utils/src/id.rs @@ -1,3 +1,4 @@ +use std::num::ParseIntError; use std::{fmt, str::FromStr}; use anyhow::Context; @@ -374,6 +375,13 @@ impl fmt::Display for NodeId { } } +impl FromStr for NodeId { + type Err = ParseIntError; + fn from_str(s: &str) -> Result { + Ok(NodeId(u64::from_str(s)?)) + } +} + #[cfg(test)] mod tests { use serde_assert::{Deserializer, Serializer, Token, Tokens}; diff --git a/pageserver/client/src/mgmt_api.rs b/pageserver/client/src/mgmt_api.rs index a0b934e48d..077c3909e1 100644 --- a/pageserver/client/src/mgmt_api.rs +++ b/pageserver/client/src/mgmt_api.rs @@ -28,8 +28,8 @@ pub enum Error { pub type Result = std::result::Result; -pub(crate) trait ResponseErrorMessageExt: Sized { - async fn error_from_body(self) -> Result; +pub trait ResponseErrorMessageExt: Sized { + fn error_from_body(self) -> impl std::future::Future> + Send; } impl ResponseErrorMessageExt for reqwest::Response { @@ -220,12 +220,12 @@ impl Client { pub async fn timeline_create( &self, - tenant_id: TenantId, + tenant_shard_id: TenantShardId, req: &TimelineCreateRequest, ) -> Result { let uri = format!( "{}/v1/tenant/{}/timeline", - self.mgmt_api_endpoint, tenant_id + self.mgmt_api_endpoint, tenant_shard_id ); self.request(Method::POST, &uri, req) .await? @@ -246,6 +246,21 @@ impl Client { .map_err(Error::ReceiveBody) } + pub async fn timeline_list( + &self, + tenant_shard_id: &TenantShardId, + ) -> Result> { + let uri = format!( + "{}/v1/tenant/{}/timeline", + self.mgmt_api_endpoint, tenant_shard_id + ); + self.get(&uri) + .await? + .json() + .await + .map_err(Error::ReceiveBody) + } + pub async fn tenant_synthetic_size( &self, tenant_shard_id: TenantShardId, diff --git a/pageserver/src/tenant/mgr.rs b/pageserver/src/tenant/mgr.rs index 4b21fb9a9c..50b895aca1 100644 --- a/pageserver/src/tenant/mgr.rs +++ b/pageserver/src/tenant/mgr.rs @@ -1067,12 +1067,26 @@ impl TenantManager { } LocationMode::Attached(_attach_config) => { let shard_identity = new_location_config.shard; + + // Testing hack: if we are configured with no control plane, then drop the generation + // from upserts. This enables creating generation-less tenants even though neon_local + // always uses generations when calling the location conf API. + let attached_conf = if cfg!(feature = "testing") { + let mut conf = AttachedTenantConf::try_from(new_location_config)?; + if self.conf.control_plane_api.is_none() { + conf.location.generation = Generation::none(); + } + conf + } else { + AttachedTenantConf::try_from(new_location_config)? + }; + let tenant = tenant_spawn( self.conf, tenant_shard_id, &tenant_path, self.resources.clone(), - AttachedTenantConf::try_from(new_location_config)?, + attached_conf, shard_identity, None, self.tenants, diff --git a/pageserver/src/tenant/remote_timeline_client.rs b/pageserver/src/tenant/remote_timeline_client.rs index ec2a6efef6..7935209252 100644 --- a/pageserver/src/tenant/remote_timeline_client.rs +++ b/pageserver/src/tenant/remote_timeline_client.rs @@ -182,7 +182,7 @@ pub(crate) mod download; pub mod index; -mod upload; +pub(crate) mod upload; use anyhow::Context; use camino::Utf8Path; @@ -691,7 +691,10 @@ impl RemoteTimelineClient { .insert(layer.layer_desc().filename(), metadata.clone()); upload_queue.latest_files_changes_since_metadata_upload_scheduled += 1; - info!("scheduled layer file upload {layer}"); + info!( + "scheduled layer file upload {layer} gen={:?} shard={:?}", + metadata.generation, metadata.shard + ); let op = UploadOp::UploadLayer(layer, metadata); self.calls_unfinished_metric_begin(&op); upload_queue.queued_operations.push_back(op); diff --git a/test_runner/fixtures/metrics.py b/test_runner/fixtures/metrics.py index a6a25da332..7c489bda67 100644 --- a/test_runner/fixtures/metrics.py +++ b/test_runner/fixtures/metrics.py @@ -16,6 +16,7 @@ class Metrics: def query_all(self, name: str, filter: Optional[Dict[str, str]] = None) -> List[Sample]: filter = filter or {} res = [] + for sample in self.metrics[name]: try: if all(sample.labels[k] == v for k, v in filter.items()): diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index c139fa215c..ffd93004d2 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -19,7 +19,7 @@ from functools import cached_property from itertools import chain, product from pathlib import Path from types import TracebackType -from typing import Any, Dict, Iterator, List, Optional, Tuple, Type, cast +from typing import Any, Dict, Iterator, List, Optional, Tuple, Type, Union, cast from urllib.parse import urlparse import asyncpg @@ -61,7 +61,7 @@ from fixtures.remote_storage import ( default_remote_storage, remote_storage_to_toml_inline_table, ) -from fixtures.types import Lsn, TenantId, TimelineId +from fixtures.types import Lsn, TenantId, TenantShardId, TimelineId from fixtures.utils import ( ATTACHMENT_NAME_REGEX, allure_add_grafana_links, @@ -495,6 +495,8 @@ class NeonEnvBuilder: self, initial_tenant_conf: Optional[Dict[str, str]] = None, default_remote_storage_if_missing: bool = True, + initial_tenant_shard_count: Optional[int] = None, + initial_tenant_shard_stripe_size: Optional[int] = None, ) -> NeonEnv: """ Default way to create and start NeonEnv. Also creates the initial_tenant with root initial_timeline. @@ -512,7 +514,11 @@ class NeonEnvBuilder: f"Services started, creating initial tenant {env.initial_tenant} and its initial timeline" ) initial_tenant, initial_timeline = env.neon_cli.create_tenant( - tenant_id=env.initial_tenant, conf=initial_tenant_conf, timeline_id=env.initial_timeline + tenant_id=env.initial_tenant, + conf=initial_tenant_conf, + timeline_id=env.initial_timeline, + shard_count=initial_tenant_shard_count, + shard_stripe_size=initial_tenant_shard_stripe_size, ) assert env.initial_tenant == initial_tenant assert env.initial_timeline == initial_timeline @@ -861,7 +867,9 @@ class NeonEnv: attachment_service_port = self.port_distributor.get_port() self.control_plane_api: str = f"http://127.0.0.1:{attachment_service_port}" - self.attachment_service: NeonAttachmentService = NeonAttachmentService(self) + self.attachment_service: NeonAttachmentService = NeonAttachmentService( + self, config.auth_enabled + ) # Create a config file corresponding to the options cfg: Dict[str, Any] = { @@ -983,6 +991,16 @@ class NeonEnv: raise RuntimeError(f"Pageserver with ID {id} not found") + 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. + """ + meta = self.attachment_service.inspect(tenant_id) + assert meta is not None, f"{tenant_id} attachment location not found" + pageserver_id = meta[1] + return self.get_pageserver(pageserver_id) + def get_safekeeper_connstrs(self) -> str: """Get list of safekeeper endpoints suitable for safekeepers GUC""" return ",".join(f"localhost:{wa.port.pg}" for wa in self.safekeepers) @@ -1226,15 +1244,29 @@ class AbstractNeonCli(abc.ABC): env_vars[var] = val # Intercept CalledProcessError and print more info - res = subprocess.run( - args, - env=env_vars, - check=False, - universal_newlines=True, - stdout=subprocess.PIPE, - stderr=subprocess.PIPE, - timeout=timeout, - ) + try: + res = subprocess.run( + args, + env=env_vars, + check=False, + universal_newlines=True, + stdout=subprocess.PIPE, + stderr=subprocess.PIPE, + timeout=timeout, + ) + except subprocess.TimeoutExpired as e: + if e.stderr: + stderr = e.stderr.decode(errors="replace") + else: + stderr = "" + + if e.stdout: + stdout = e.stdout.decode(errors="replace") + else: + stdout = "" + + log.warn(f"CLI timeout: stderr={stderr}, stdout={stdout}") + raise indent = " " if not res.returncode: @@ -1285,6 +1317,8 @@ class NeonCli(AbstractNeonCli): tenant_id: Optional[TenantId] = None, timeline_id: Optional[TimelineId] = None, conf: Optional[Dict[str, str]] = None, + shard_count: Optional[int] = None, + shard_stripe_size: Optional[int] = None, set_default: bool = False, ) -> Tuple[TenantId, TimelineId]: """ @@ -1312,6 +1346,12 @@ class NeonCli(AbstractNeonCli): if set_default: args.append("--set-default") + if shard_count is not None: + args.extend(["--shard-count", str(shard_count)]) + + if shard_stripe_size is not None: + args.extend(["--shard-stripe-size", str(shard_stripe_size)]) + res = self.raw_cli(args) res.check_returncode() return tenant_id, timeline_id @@ -1636,6 +1676,19 @@ class NeonCli(AbstractNeonCli): return self.raw_cli(args, check_return_code=True) + def tenant_migrate( + self, tenant_shard_id: TenantShardId, new_pageserver: int, timeout_secs: Optional[int] + ): + args = [ + "tenant", + "migrate", + "--tenant-id", + str(tenant_shard_id), + "--id", + str(new_pageserver), + ] + return self.raw_cli(args, check_return_code=True, timeout=timeout_secs) + def start(self, check_return_code=True) -> "subprocess.CompletedProcess[str]": return self.raw_cli(["start"], check_return_code=check_return_code) @@ -1684,9 +1737,10 @@ class Pagectl(AbstractNeonCli): class NeonAttachmentService: - def __init__(self, env: NeonEnv): + def __init__(self, env: NeonEnv, auth_enabled): self.env = env self.running = False + self.auth_enabled = auth_enabled def start(self): assert not self.running @@ -1700,27 +1754,50 @@ class NeonAttachmentService: self.running = False return self - def attach_hook_issue(self, tenant_id: TenantId, pageserver_id: int) -> int: - response = requests.post( + def request(self, method, *args, **kwargs) -> requests.Response: + kwargs["headers"] = self.headers() + return requests.request(method, *args, **kwargs) + + def headers(self) -> Dict[str, str]: + headers = {} + if self.auth_enabled: + jwt_token = self.env.auth_keys.generate_pageserver_token() + headers["Authorization"] = f"Bearer {jwt_token}" + + return headers + + def attach_hook_issue( + self, tenant_shard_id: Union[TenantId, TenantShardId], pageserver_id: int + ) -> int: + response = self.request( + "POST", f"{self.env.control_plane_api}/attach-hook", - json={"tenant_id": str(tenant_id), "node_id": pageserver_id}, + json={"tenant_shard_id": str(tenant_shard_id), "node_id": pageserver_id}, + headers=self.headers(), ) response.raise_for_status() gen = response.json()["gen"] assert isinstance(gen, int) return gen - def attach_hook_drop(self, tenant_id: TenantId): - response = requests.post( + def attach_hook_drop(self, tenant_shard_id: Union[TenantId, TenantShardId]): + response = self.request( + "POST", f"{self.env.control_plane_api}/attach-hook", - json={"tenant_id": str(tenant_id), "node_id": None}, + json={"tenant_shard_id": str(tenant_shard_id), "node_id": None}, + headers=self.headers(), ) response.raise_for_status() - def inspect(self, tenant_id: TenantId) -> Optional[tuple[int, int]]: - response = requests.post( + def inspect(self, tenant_shard_id: Union[TenantId, TenantShardId]) -> Optional[tuple[int, int]]: + """ + :return: 2-tuple of (generation, pageserver id), or None if unknown + """ + response = self.request( + "POST", f"{self.env.control_plane_api}/inspect", - json={"tenant_id": str(tenant_id)}, + json={"tenant_shard_id": str(tenant_shard_id)}, + headers=self.headers(), ) response.raise_for_status() json = response.json() @@ -1731,6 +1808,79 @@ class NeonAttachmentService: else: return None + def node_register(self, node: NeonPageserver): + body = { + "node_id": int(node.id), + "listen_http_addr": "localhost", + "listen_http_port": node.service_port.http, + } + log.info(f"node_register({body})") + self.request( + "POST", f"{self.env.control_plane_api}/node", json=body, headers=self.headers() + ).raise_for_status() + + def tenant_create( + self, + tenant_id: TenantId, + shard_count: Optional[int] = None, + shard_stripe_size: Optional[int] = None, + tenant_config: Optional[Dict[Any, Any]] = None, + ): + body: Dict[str, Any] = {"new_tenant_id": str(tenant_id)} + + if shard_count is not None: + shard_params = {"count": shard_count} + if shard_stripe_size is not None: + shard_params["stripe_size"] = shard_stripe_size + + body["shard_parameters"] = shard_params + + if tenant_config is not None: + for k, v in tenant_config.items(): + body[k] = v + + response = self.request("POST", f"{self.env.control_plane_api}/tenant", json=body) + response.raise_for_status() + log.info(f"tenant_create success: {response.json()}") + + def tenant_timeline_create(self, tenant_id: TenantId, timeline_id: TimelineId): + body: Dict[str, Any] = {"new_timeline_id": str(timeline_id)} + + response = self.request( + "POST", f"{self.env.control_plane_api}/tenant/{tenant_id}/timeline", json=body + ) + response.raise_for_status() + log.info(f"tenant_timeline_create success: {response.json()}") + + def locate(self, tenant_id: TenantId) -> list[dict[str, Any]]: + response = self.request("GET", f"{self.env.control_plane_api}/tenant/{tenant_id}/locate") + response.raise_for_status() + body = response.json() + shards: list[dict[str, Any]] = body["shards"] + return shards + + def tenant_shard_split(self, tenant_id: TenantId, shard_count: int) -> list[TenantShardId]: + response = self.request( + "PUT", + f"{self.env.control_plane_api}/tenant/{tenant_id}/shard_split", + json={"new_shard_count": shard_count}, + ) + response.raise_for_status() + body = response.json() + log.info(f"tenant_shard_split success: {body}") + shards: list[TenantShardId] = body["new_shards"] + return shards + + def tenant_shard_migrate(self, tenant_shard_id: TenantShardId, dest_ps_id: int): + response = self.request( + "PUT", + f"{self.env.control_plane_api}/tenant/{tenant_shard_id}/migrate", + json={"tenant_shard_id": str(tenant_shard_id), "node_id": dest_ps_id}, + ) + response.raise_for_status() + log.info(f"Migrated tenant {tenant_shard_id} to pageserver {dest_ps_id}") + assert self.env.get_tenant_pageserver(tenant_shard_id).id == dest_ps_id + def __enter__(self) -> "NeonAttachmentService": return self @@ -2831,7 +2981,7 @@ class Endpoint(PgProtocol): hot_standby=hot_standby, lsn=lsn, pageserver_id=pageserver_id, - ).start(remote_ext_config=remote_ext_config) + ).start(remote_ext_config=remote_ext_config, pageserver_id=pageserver_id) log.info(f"Postgres startup took {time.time() - started_at} seconds") @@ -3344,7 +3494,7 @@ def pytest_addoption(parser: Parser): SMALL_DB_FILE_NAME_REGEX: re.Pattern = re.compile( # type: ignore[type-arg] - r"config|config-v1|heatmap-v1|metadata|.+\.(?:toml|pid|json|sql)" + r"config|config-v1|heatmap-v1|metadata|.+\.(?:toml|pid|json|sql|conf)" ) @@ -3481,9 +3631,7 @@ 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, pageserver_id: Optional[int] = None -): +def check_restored_datadir_content(test_output_dir: Path, env: NeonEnv, endpoint: Endpoint): # Get the timeline ID. We need it for the 'basebackup' command timeline_id = TimelineId(endpoint.safe_psql("SHOW neon.timeline_id")[0][0]) @@ -3504,6 +3652,7 @@ def check_restored_datadir_content( 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"] cmd = rf""" {psql_path} \ --no-psqlrc \ @@ -3572,6 +3721,38 @@ def logical_replication_sync(subscriber: VanillaPostgres, publisher: Endpoint) - time.sleep(0.5) +def tenant_get_shards( + env: NeonEnv, tenant_id: TenantId, pageserver_id: Optional[int] +) -> list[tuple[TenantShardId, NeonPageserver]]: + """ + Helper for when you want to talk to one or more pageservers, and the + caller _might_ have specified a pageserver, or they might leave it to + 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. + + Caller should over the response to apply their per-pageserver action to + each shard + """ + if pageserver_id is not None: + override_pageserver = [p for p in env.pageservers if p.id == pageserver_id][0] + else: + override_pageserver = None + + if len(env.pageservers) > 1: + return [ + ( + TenantShardId.parse(s["shard_id"]), + override_pageserver or env.get_pageserver(s["node_id"]), + ) + for s in env.attachment_service.locate(tenant_id) + ] + else: + # Assume an unsharded tenant + return [(TenantShardId(tenant_id, 0, 0), override_pageserver or env.pageserver)] + + def wait_for_last_flush_lsn( env: NeonEnv, endpoint: Endpoint, @@ -3581,10 +3762,24 @@ def wait_for_last_flush_lsn( ) -> Lsn: """Wait for pageserver to catch up the latest flush LSN, returns the last observed lsn.""" + shards = tenant_get_shards(env, tenant, pageserver_id) + last_flush_lsn = Lsn(endpoint.safe_psql("SELECT pg_current_wal_flush_lsn()")[0][0]) - return wait_for_last_record_lsn( - env.get_pageserver(pageserver_id).http_client(), tenant, timeline, last_flush_lsn - ) + + results = [] + for tenant_shard_id, pageserver in shards: + log.info( + f"wait_for_last_flush_lsn: waiting for {last_flush_lsn} on shard {tenant_shard_id} on pageserver {pageserver.id})" + ) + waited = wait_for_last_record_lsn( + pageserver.http_client(), tenant_shard_id, timeline, last_flush_lsn + ) + + assert waited >= last_flush_lsn + results.append(waited) + + # Return the lowest LSN that has been ingested by all shards + return min(results) def wait_for_wal_insert_lsn( @@ -3596,9 +3791,16 @@ def wait_for_wal_insert_lsn( ) -> Lsn: """Wait for pageserver to catch up the latest flush LSN, returns the last observed lsn.""" last_flush_lsn = Lsn(endpoint.safe_psql("SELECT pg_current_wal_insert_lsn()")[0][0]) - return wait_for_last_record_lsn( - env.get_pageserver(pageserver_id).http_client(), tenant, timeline, last_flush_lsn - ) + result = None + for tenant_shard_id, pageserver in tenant_get_shards(env, tenant, pageserver_id): + shard_r = wait_for_last_record_lsn( + pageserver.http_client(), tenant_shard_id, timeline, last_flush_lsn + ) + if result is None: + result = shard_r + + assert result is not None + return result def fork_at_current_lsn( @@ -3632,11 +3834,13 @@ def last_flush_lsn_upload( last_flush_lsn = wait_for_last_flush_lsn( env, endpoint, tenant_id, timeline_id, pageserver_id=pageserver_id ) - ps_http = env.get_pageserver(pageserver_id).http_client() - wait_for_last_record_lsn(ps_http, tenant_id, timeline_id, last_flush_lsn) - # force a checkpoint to trigger upload - ps_http.timeline_checkpoint(tenant_id, timeline_id) - wait_for_upload(ps_http, tenant_id, timeline_id, last_flush_lsn) + shards = tenant_get_shards(env, tenant_id, pageserver_id) + for tenant_shard_id, pageserver in shards: + ps_http = pageserver.http_client() + wait_for_last_record_lsn(ps_http, tenant_shard_id, timeline_id, last_flush_lsn) + # force a checkpoint to trigger upload + ps_http.timeline_checkpoint(tenant_shard_id, timeline_id) + wait_for_upload(ps_http, tenant_shard_id, timeline_id, last_flush_lsn) return last_flush_lsn diff --git a/test_runner/fixtures/pageserver/http.py b/test_runner/fixtures/pageserver/http.py index b24de342f8..cfa2a2674d 100644 --- a/test_runner/fixtures/pageserver/http.py +++ b/test_runner/fixtures/pageserver/http.py @@ -4,7 +4,7 @@ import json import time from collections import defaultdict from dataclasses import dataclass -from typing import Any, Dict, List, Optional, Set, Tuple +from typing import Any, Dict, List, Optional, Set, Tuple, Union import requests from requests.adapters import HTTPAdapter @@ -13,7 +13,7 @@ from urllib3.util.retry import Retry from fixtures.log_helper import log from fixtures.metrics import Metrics, parse_metrics from fixtures.pg_version import PgVersion -from fixtures.types import Lsn, TenantId, TimelineId +from fixtures.types import Lsn, TenantId, TenantShardId, TimelineId from fixtures.utils import Fn @@ -211,7 +211,7 @@ class PageserverHttpClient(requests.Session): def tenant_create( self, - new_tenant_id: TenantId, + new_tenant_id: Union[TenantId, TenantShardId], conf: Optional[Dict[str, Any]] = None, generation: Optional[int] = None, ) -> TenantId: @@ -239,7 +239,7 @@ class PageserverHttpClient(requests.Session): def tenant_attach( self, - tenant_id: TenantId, + tenant_id: Union[TenantId, TenantShardId], config: None | Dict[str, Any] = None, config_null: bool = False, generation: Optional[int] = None, @@ -269,7 +269,7 @@ class PageserverHttpClient(requests.Session): res = self.post(f"http://localhost:{self.port}/v1/tenant/{tenant_id}/detach", params=params) self.verbose_error(res) - def tenant_reset(self, tenant_id: TenantId, drop_cache: bool): + def tenant_reset(self, tenant_id: Union[TenantId, TenantShardId], drop_cache: bool): params = {} if drop_cache: params["drop_cache"] = "true" @@ -278,7 +278,7 @@ class PageserverHttpClient(requests.Session): self.verbose_error(res) def tenant_location_conf( - self, tenant_id: TenantId, location_conf=dict[str, Any], flush_ms=None + self, tenant_id: Union[TenantId, TenantShardId], location_conf=dict[str, Any], flush_ms=None ): body = location_conf.copy() body["tenant_id"] = str(tenant_id) @@ -294,7 +294,7 @@ class PageserverHttpClient(requests.Session): ) self.verbose_error(res) - def tenant_delete(self, tenant_id: TenantId): + 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) return res @@ -310,27 +310,27 @@ class PageserverHttpClient(requests.Session): res = self.post(f"http://localhost:{self.port}/v1/tenant/{tenant_id}/ignore") self.verbose_error(res) - def tenant_status(self, tenant_id: TenantId) -> Dict[Any, Any]: + def tenant_status(self, tenant_id: Union[TenantId, TenantShardId]) -> Dict[Any, Any]: res = self.get(f"http://localhost:{self.port}/v1/tenant/{tenant_id}") self.verbose_error(res) res_json = res.json() assert isinstance(res_json, dict) return res_json - def tenant_config(self, tenant_id: TenantId) -> TenantConfig: + def tenant_config(self, tenant_id: Union[TenantId, TenantShardId]) -> TenantConfig: res = self.get(f"http://localhost:{self.port}/v1/tenant/{tenant_id}/config") self.verbose_error(res) return TenantConfig.from_json(res.json()) - def tenant_heatmap_upload(self, tenant_id: TenantId): + def tenant_heatmap_upload(self, tenant_id: Union[TenantId, TenantShardId]): res = self.post(f"http://localhost:{self.port}/v1/tenant/{tenant_id}/heatmap_upload") self.verbose_error(res) - def tenant_secondary_download(self, tenant_id: TenantId): + def tenant_secondary_download(self, tenant_id: Union[TenantId, TenantShardId]): res = self.post(f"http://localhost:{self.port}/v1/tenant/{tenant_id}/secondary/download") self.verbose_error(res) - def set_tenant_config(self, tenant_id: TenantId, config: dict[str, Any]): + def set_tenant_config(self, tenant_id: Union[TenantId, TenantShardId], config: dict[str, Any]): assert "tenant_id" not in config.keys() res = self.put( f"http://localhost:{self.port}/v1/tenant/config", @@ -352,10 +352,12 @@ class PageserverHttpClient(requests.Session): del current[key] self.set_tenant_config(tenant_id, current) - def tenant_size(self, tenant_id: TenantId) -> int: + def tenant_size(self, tenant_id: Union[TenantId, TenantShardId]) -> int: return self.tenant_size_and_modelinputs(tenant_id)[0] - def tenant_size_and_modelinputs(self, tenant_id: TenantId) -> Tuple[int, Dict[str, Any]]: + def tenant_size_and_modelinputs( + self, tenant_id: Union[TenantId, TenantShardId] + ) -> Tuple[int, Dict[str, Any]]: """ Returns the tenant size, together with the model inputs as the second tuple item. """ @@ -370,7 +372,7 @@ class PageserverHttpClient(requests.Session): assert isinstance(inputs, dict) return (size, inputs) - def tenant_size_debug(self, tenant_id: TenantId) -> str: + def tenant_size_debug(self, tenant_id: Union[TenantId, TenantShardId]) -> str: """ Returns the tenant size debug info, as an HTML string """ @@ -382,7 +384,7 @@ class PageserverHttpClient(requests.Session): def timeline_list( self, - tenant_id: TenantId, + tenant_id: Union[TenantId, TenantShardId], include_non_incremental_logical_size: bool = False, include_timeline_dir_layer_file_size_sum: bool = False, ) -> List[Dict[str, Any]]: @@ -403,7 +405,7 @@ class PageserverHttpClient(requests.Session): def timeline_create( self, pg_version: PgVersion, - tenant_id: TenantId, + tenant_id: Union[TenantId, TenantShardId], new_timeline_id: TimelineId, ancestor_timeline_id: Optional[TimelineId] = None, ancestor_start_lsn: Optional[Lsn] = None, @@ -437,7 +439,7 @@ class PageserverHttpClient(requests.Session): def timeline_detail( self, - tenant_id: TenantId, + tenant_id: Union[TenantId, TenantShardId], timeline_id: TimelineId, include_non_incremental_logical_size: bool = False, include_timeline_dir_layer_file_size_sum: bool = False, @@ -462,7 +464,9 @@ class PageserverHttpClient(requests.Session): assert isinstance(res_json, dict) return res_json - def timeline_delete(self, tenant_id: TenantId, timeline_id: TimelineId, **kwargs): + def timeline_delete( + self, tenant_id: Union[TenantId, TenantShardId], timeline_id: TimelineId, **kwargs + ): """ Note that deletion is not instant, it is scheduled and performed mostly in the background. So if you need to wait for it to complete use `timeline_delete_wait_completed`. @@ -476,7 +480,10 @@ class PageserverHttpClient(requests.Session): assert res_json is None def timeline_gc( - self, tenant_id: TenantId, timeline_id: TimelineId, gc_horizon: Optional[int] + self, + tenant_id: Union[TenantId, TenantShardId], + timeline_id: TimelineId, + gc_horizon: Optional[int], ) -> dict[str, Any]: """ Unlike most handlers, this will wait for the layers to be actually @@ -499,7 +506,10 @@ class PageserverHttpClient(requests.Session): return res_json def timeline_compact( - self, tenant_id: TenantId, timeline_id: TimelineId, force_repartition=False + self, + tenant_id: Union[TenantId, TenantShardId], + timeline_id: TimelineId, + force_repartition=False, ): self.is_testing_enabled_or_skip() query = {} @@ -518,7 +528,7 @@ class PageserverHttpClient(requests.Session): def timeline_get_lsn_by_timestamp( self, - tenant_id: TenantId, + tenant_id: Union[TenantId, TenantShardId], timeline_id: TimelineId, timestamp, version: Optional[int] = None, @@ -537,7 +547,9 @@ class PageserverHttpClient(requests.Session): res_json = res.json() return res_json - def timeline_get_timestamp_of_lsn(self, tenant_id: TenantId, timeline_id: TimelineId, lsn: Lsn): + def timeline_get_timestamp_of_lsn( + self, tenant_id: Union[TenantId, TenantShardId], timeline_id: TimelineId, lsn: Lsn + ): log.info(f"Requesting time range of lsn {lsn}, tenant {tenant_id}, timeline {timeline_id}") res = self.get( f"http://localhost:{self.port}/v1/tenant/{tenant_id}/timeline/{timeline_id}/get_timestamp_of_lsn?lsn={lsn}", @@ -547,7 +559,10 @@ class PageserverHttpClient(requests.Session): return res_json def timeline_checkpoint( - self, tenant_id: TenantId, timeline_id: TimelineId, force_repartition=False + self, + tenant_id: Union[TenantId, TenantShardId], + timeline_id: TimelineId, + force_repartition=False, ): self.is_testing_enabled_or_skip() query = {} @@ -566,7 +581,7 @@ class PageserverHttpClient(requests.Session): def timeline_spawn_download_remote_layers( self, - tenant_id: TenantId, + tenant_id: Union[TenantId, TenantShardId], timeline_id: TimelineId, max_concurrent_downloads: int, ) -> dict[str, Any]: @@ -585,7 +600,7 @@ class PageserverHttpClient(requests.Session): def timeline_poll_download_remote_layers_status( self, - tenant_id: TenantId, + tenant_id: Union[TenantId, TenantShardId], timeline_id: TimelineId, spawn_response: dict[str, Any], poll_state=None, @@ -607,7 +622,7 @@ class PageserverHttpClient(requests.Session): def timeline_download_remote_layers( self, - tenant_id: TenantId, + tenant_id: Union[TenantId, TenantShardId], timeline_id: TimelineId, max_concurrent_downloads: int, errors_ok=False, @@ -689,9 +704,37 @@ class PageserverHttpClient(requests.Session): assert len(results) == 1, f"metric {name} with given filters is not unique, got: {results}" return results[0].value + def get_metrics_values( + self, names: list[str], filter: Optional[Dict[str, str]] = None + ) -> Dict[str, float]: + """ + When fetching multiple named metrics, it is more efficient to use this + than to call `get_metric_value` repeatedly. + + Throws RuntimeError if no metrics matching `names` are found, or if + not all of `names` are found: this method is intended for loading sets + of metrics whose existence is coupled. + """ + metrics = self.get_metrics() + samples = [] + for name in names: + samples.extend(metrics.query_all(name, filter=filter)) + + result = {} + for sample in samples: + if sample.name in result: + raise RuntimeError(f"Multiple values found for {sample.name}") + result[sample.name] = sample.value + + if len(result) != len(names): + log.info(f"Metrics found: {metrics.metrics}") + raise RuntimeError(f"could not find all metrics {' '.join(names)}") + + return result + def layer_map_info( self, - tenant_id: TenantId, + tenant_id: Union[TenantId, TenantShardId], timeline_id: TimelineId, ) -> LayerMapInfo: res = self.get( @@ -700,7 +743,9 @@ class PageserverHttpClient(requests.Session): self.verbose_error(res) return LayerMapInfo.from_json(res.json()) - def download_layer(self, tenant_id: TenantId, timeline_id: TimelineId, layer_name: str): + def download_layer( + self, tenant_id: Union[TenantId, TenantShardId], timeline_id: TimelineId, layer_name: str + ): res = self.get( f"http://localhost:{self.port}/v1/tenant/{tenant_id}/timeline/{timeline_id}/layer/{layer_name}", ) @@ -708,14 +753,18 @@ class PageserverHttpClient(requests.Session): assert res.status_code == 200 - def download_all_layers(self, tenant_id: TenantId, timeline_id: TimelineId): + def download_all_layers( + self, tenant_id: Union[TenantId, TenantShardId], timeline_id: TimelineId + ): info = self.layer_map_info(tenant_id, timeline_id) for layer in info.historic_layers: if not layer.remote: continue self.download_layer(tenant_id, timeline_id, layer.layer_file_name) - def evict_layer(self, tenant_id: TenantId, timeline_id: TimelineId, layer_name: str): + def evict_layer( + self, tenant_id: Union[TenantId, TenantShardId], timeline_id: TimelineId, layer_name: str + ): res = self.delete( f"http://localhost:{self.port}/v1/tenant/{tenant_id}/timeline/{timeline_id}/layer/{layer_name}", ) @@ -723,7 +772,7 @@ class PageserverHttpClient(requests.Session): assert res.status_code in (200, 304) - def evict_all_layers(self, tenant_id: TenantId, timeline_id: TimelineId): + def evict_all_layers(self, tenant_id: Union[TenantId, TenantShardId], timeline_id: TimelineId): info = self.layer_map_info(tenant_id, timeline_id) for layer in info.historic_layers: self.evict_layer(tenant_id, timeline_id, layer.layer_file_name) @@ -736,7 +785,7 @@ class PageserverHttpClient(requests.Session): self.verbose_error(res) return res.json() - def tenant_break(self, tenant_id: TenantId): + def tenant_break(self, tenant_id: Union[TenantId, TenantShardId]): res = self.put(f"http://localhost:{self.port}/v1/tenant/{tenant_id}/break") self.verbose_error(res) diff --git a/test_runner/fixtures/pageserver/utils.py b/test_runner/fixtures/pageserver/utils.py index e7b78cfb9a..d0bb566408 100644 --- a/test_runner/fixtures/pageserver/utils.py +++ b/test_runner/fixtures/pageserver/utils.py @@ -1,12 +1,12 @@ import time -from typing import TYPE_CHECKING, Any, Dict, List, Optional +from typing import TYPE_CHECKING, Any, Dict, List, Optional, Union from mypy_boto3_s3.type_defs import ListObjectsV2OutputTypeDef, ObjectTypeDef from fixtures.log_helper import log from fixtures.pageserver.http import PageserverApiException, PageserverHttpClient from fixtures.remote_storage import RemoteStorageKind, S3Storage -from fixtures.types import Lsn, TenantId, TimelineId +from fixtures.types import Lsn, TenantId, TenantShardId, TimelineId from fixtures.utils import wait_until @@ -22,7 +22,9 @@ def assert_tenant_state( def remote_consistent_lsn( - pageserver_http: PageserverHttpClient, tenant: TenantId, timeline: TimelineId + pageserver_http: PageserverHttpClient, + tenant: Union[TenantId, TenantShardId], + timeline: TimelineId, ) -> Lsn: detail = pageserver_http.timeline_detail(tenant, timeline) @@ -39,7 +41,7 @@ def remote_consistent_lsn( def wait_for_upload( pageserver_http: PageserverHttpClient, - tenant: TenantId, + tenant: Union[TenantId, TenantShardId], timeline: TimelineId, lsn: Lsn, ): @@ -92,7 +94,7 @@ def wait_until_tenant_state( def wait_until_timeline_state( pageserver_http: PageserverHttpClient, - tenant_id: TenantId, + tenant_id: Union[TenantId, TenantShardId], timeline_id: TimelineId, expected_state: str, iterations: int, @@ -141,7 +143,9 @@ def wait_until_tenant_active( def last_record_lsn( - pageserver_http_client: PageserverHttpClient, tenant: TenantId, timeline: TimelineId + pageserver_http_client: PageserverHttpClient, + tenant: Union[TenantId, TenantShardId], + timeline: TimelineId, ) -> Lsn: detail = pageserver_http_client.timeline_detail(tenant, timeline) @@ -152,7 +156,7 @@ def last_record_lsn( def wait_for_last_record_lsn( pageserver_http: PageserverHttpClient, - tenant: TenantId, + tenant: Union[TenantId, TenantShardId], timeline: TimelineId, lsn: Lsn, ) -> Lsn: @@ -194,7 +198,7 @@ def wait_for_upload_queue_empty( def wait_timeline_detail_404( pageserver_http: PageserverHttpClient, - tenant_id: TenantId, + tenant_id: Union[TenantId, TenantShardId], timeline_id: TimelineId, iterations: int, interval: Optional[float] = None, @@ -219,7 +223,7 @@ def wait_timeline_detail_404( def timeline_delete_wait_completed( pageserver_http: PageserverHttpClient, - tenant_id: TenantId, + tenant_id: Union[TenantId, TenantShardId], timeline_id: TimelineId, iterations: int = 20, interval: Optional[float] = None, diff --git a/test_runner/fixtures/workload.py b/test_runner/fixtures/workload.py index 241531437c..30def1194d 100644 --- a/test_runner/fixtures/workload.py +++ b/test_runner/fixtures/workload.py @@ -5,6 +5,7 @@ from fixtures.neon_fixtures import ( Endpoint, NeonEnv, last_flush_lsn_upload, + tenant_get_shards, wait_for_last_flush_lsn, ) from fixtures.pageserver.utils import wait_for_last_record_lsn, wait_for_upload @@ -31,7 +32,7 @@ class Workload: self._endpoint: Optional[Endpoint] = None - def endpoint(self, pageserver_id: int) -> Endpoint: + def endpoint(self, pageserver_id: Optional[int] = None) -> Endpoint: if self._endpoint is None: self._endpoint = self.env.endpoints.create( "main", @@ -54,7 +55,7 @@ class Workload: if self._endpoint is not None: self._endpoint.stop() - def init(self, pageserver_id: int): + def init(self, pageserver_id: Optional[int] = None): endpoint = self.endpoint(pageserver_id) endpoint.safe_psql(f"CREATE TABLE {self.table} (id INTEGER PRIMARY KEY, val text);") @@ -63,7 +64,7 @@ class Workload: self.env, endpoint, self.tenant_id, self.timeline_id, pageserver_id=pageserver_id ) - def write_rows(self, n, pageserver_id): + def write_rows(self, n, pageserver_id: Optional[int] = None): endpoint = self.endpoint(pageserver_id) start = self.expect_rows end = start + n - 1 @@ -81,7 +82,7 @@ class Workload: self.env, endpoint, self.tenant_id, self.timeline_id, pageserver_id=pageserver_id ) - def churn_rows(self, n, pageserver_id, upload=True): + def churn_rows(self, n, pageserver_id: Optional[int] = None, upload=True): assert self.expect_rows >= n max_iters = 10 @@ -119,21 +120,24 @@ class Workload: ] ) - last_flush_lsn = wait_for_last_flush_lsn( - self.env, endpoint, self.tenant_id, self.timeline_id, pageserver_id=pageserver_id - ) - ps_http = self.env.get_pageserver(pageserver_id).http_client() - wait_for_last_record_lsn(ps_http, self.tenant_id, self.timeline_id, last_flush_lsn) + 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(self.tenant_id, self.timeline_id) - wait_for_upload(ps_http, self.tenant_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 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}") - def validate(self, pageserver_id): + def validate(self, pageserver_id: Optional[int] = None): endpoint = self.endpoint(pageserver_id) result = endpoint.safe_psql_many( [ diff --git a/test_runner/performance/test_bulk_insert.py b/test_runner/performance/test_bulk_insert.py index edc23b29ba..72173dc2a7 100644 --- a/test_runner/performance/test_bulk_insert.py +++ b/test_runner/performance/test_bulk_insert.py @@ -61,7 +61,7 @@ def measure_recovery_time(env: NeonCompare): # 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_id=env.tenant) + attach_status = env.env.attachment_service.inspect(tenant_shard_id=env.tenant) assert attach_status is not None (attach_gen, _) = attach_status diff --git a/test_runner/regress/test_broken_timeline.py b/test_runner/regress/test_broken_timeline.py index 4da0ba7b20..b046ed7f1b 100644 --- a/test_runner/regress/test_broken_timeline.py +++ b/test_runner/regress/test_broken_timeline.py @@ -10,6 +10,7 @@ from fixtures.neon_fixtures import ( NeonEnvBuilder, wait_for_last_flush_lsn, ) +from fixtures.pg_version import PgVersion from fixtures.types import TenantId, TimelineId @@ -126,7 +127,7 @@ def test_timeline_init_break_before_checkpoint(neon_env_builder: NeonEnvBuilder) # Introduce failpoint during timeline init (some intermediate files are on disk), before it's checkpointed. pageserver_http.configure_failpoints(("before-checkpoint-new-timeline", "return")) with pytest.raises(Exception, match="before-checkpoint-new-timeline"): - _ = env.neon_cli.create_timeline("test_timeline_init_break_before_checkpoint", tenant_id) + _ = pageserver_http.timeline_create(PgVersion.NOT_SET, tenant_id, TimelineId.generate()) # Restart the page server env.pageserver.restart(immediate=True) @@ -160,7 +161,7 @@ def test_timeline_init_break_before_checkpoint_recreate( ] ) - env.pageserver.tenant_create(env.initial_tenant) + env.neon_cli.create_tenant(env.initial_tenant) tenant_id = env.initial_tenant timelines_dir = env.pageserver.timeline_dir(tenant_id) @@ -216,7 +217,7 @@ def test_timeline_create_break_after_uninit_mark(neon_env_builder: NeonEnvBuilde # Introduce failpoint when creating a new timeline uninit mark, before any other files were created pageserver_http.configure_failpoints(("after-timeline-uninit-mark-creation", "return")) with pytest.raises(Exception, match="after-timeline-uninit-mark-creation"): - _ = env.neon_cli.create_timeline("test_timeline_create_break_after_uninit_mark", tenant_id) + _ = pageserver_http.timeline_create(PgVersion.NOT_SET, tenant_id, TimelineId.generate()) # Creating the timeline didn't finish. The other timelines on tenant should still be present and work normally. # "New" timeline is not present in the list, allowing pageserver to retry the same request diff --git a/test_runner/regress/test_disk_usage_eviction.py b/test_runner/regress/test_disk_usage_eviction.py index 0e678e7148..64654c41a2 100644 --- a/test_runner/regress/test_disk_usage_eviction.py +++ b/test_runner/regress/test_disk_usage_eviction.py @@ -16,7 +16,7 @@ from fixtures.neon_fixtures import ( from fixtures.pageserver.http import PageserverHttpClient from fixtures.pageserver.utils import wait_for_upload_queue_empty from fixtures.remote_storage import RemoteStorageKind -from fixtures.types import Lsn, TenantId, TimelineId +from fixtures.types import Lsn, TenantId, TenantShardId, TimelineId from fixtures.utils import wait_until GLOBAL_LRU_LOG_LINE = "tenant_min_resident_size-respecting LRU would not relieve pressure, evicting more following global LRU policy" @@ -214,9 +214,6 @@ def _eviction_env( env = neon_env_builder.init_configs() env.start() - # We will create all tenants on the 0th pageserver - pageserver_http = env.pageservers[0].http_client() - # allow because we are invoking this manually; we always warn on executing disk based eviction for ps in env.pageservers: ps.allowed_errors.append(r".* running disk usage based eviction due to pressure.*") @@ -244,7 +241,7 @@ def _eviction_env( with env.endpoints.create_start("main", tenant_id=tenant_id) as endpoint: pg_bin.run(["pgbench", "-i", f"-s{scale}", endpoint.connstr()]) - wait_for_last_flush_lsn(env, endpoint, tenant_id, timeline_id, pageserver_id=1) + wait_for_last_flush_lsn(env, endpoint, tenant_id, timeline_id) timelines.append((tenant_id, timeline_id)) @@ -255,6 +252,8 @@ def _eviction_env( # after stopping the safekeepers, we know that no new WAL will be coming in for tenant_id, timeline_id in timelines: + pageserver_http = env.get_tenant_pageserver(tenant_id).http_client() + pageserver_http.timeline_checkpoint(tenant_id, timeline_id) wait_for_upload_queue_empty(pageserver_http, tenant_id, timeline_id) tl_info = pageserver_http.timeline_detail(tenant_id, timeline_id) @@ -710,10 +709,20 @@ def test_secondary_mode_eviction(eviction_env_ha: EvictionEnv): tenant_ids = [t[0] for t in env.timelines] + # Set up a situation where one pageserver _only_ has secondary locations on it, + # so that when we release space we are sure it is via secondary locations. + log.info("Setting up secondary location...") ps_attached = env.neon_env.pageservers[0] ps_secondary = env.neon_env.pageservers[1] for tenant_id in tenant_ids: + # Migrate all attached tenants to the same pageserver, so that all the secondaries + # will run on the other pageserver. This is necessary because when we create tenants, + # they are spread over pageservers by default. + env.neon_env.attachment_service.tenant_shard_migrate( + TenantShardId(tenant_id, 0, 0), ps_attached.id + ) + ps_secondary.tenant_location_configure( tenant_id, { diff --git a/test_runner/regress/test_pageserver_generations.py b/test_runner/regress/test_pageserver_generations.py index 87a4fa01fc..dd55d737ac 100644 --- a/test_runner/regress/test_pageserver_generations.py +++ b/test_runner/regress/test_pageserver_generations.py @@ -62,6 +62,7 @@ def generate_uploads_and_deletions( tenant_id: Optional[TenantId] = None, timeline_id: Optional[TimelineId] = None, data: Optional[str] = None, + pageserver_id: Optional[int] = None, ): """ Using the environment's default tenant + timeline, generate a load pattern @@ -78,7 +79,9 @@ def generate_uploads_and_deletions( ps_http = env.pageserver.http_client() - with env.endpoints.create_start("main", tenant_id=tenant_id) as endpoint: + with env.endpoints.create_start( + "main", tenant_id=tenant_id, pageserver_id=pageserver_id + ) as endpoint: if init: endpoint.safe_psql("CREATE TABLE foo (id INTEGER PRIMARY KEY, val text)") last_flush_lsn_upload(env, endpoint, tenant_id, timeline_id) @@ -202,7 +205,7 @@ def test_generations_upgrade(neon_env_builder: NeonEnvBuilder): env.neon_cli.create_tenant( tenant_id=env.initial_tenant, conf=TENANT_CONF, timeline_id=env.initial_timeline ) - generate_uploads_and_deletions(env) + generate_uploads_and_deletions(env, pageserver_id=env.pageserver.id) def parse_generation_suffix(key): m = re.match(".+-([0-9a-zA-Z]{8})$", key) @@ -224,7 +227,7 @@ def test_generations_upgrade(neon_env_builder: NeonEnvBuilder): # Starting without the override that disabled control_plane_api env.pageserver.start() - generate_uploads_and_deletions(env, init=False) + generate_uploads_and_deletions(env, pageserver_id=env.pageserver.id, init=False) legacy_objects: list[str] = [] suffixed_objects = [] @@ -268,6 +271,7 @@ def test_deferred_deletion(neon_env_builder: NeonEnvBuilder): env = neon_env_builder.init_start(initial_tenant_conf=TENANT_CONF) some_other_pageserver = 1234 + ps_http = env.pageserver.http_client() generate_uploads_and_deletions(env) @@ -290,7 +294,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, some_other_pageserver) - generate_uploads_and_deletions(env, init=False) + generate_uploads_and_deletions(env, init=False, pageserver_id=env.pageserver.id) assert_deletion_queue(ps_http, lambda n: n > 0) queue_depth_before = get_deletion_queue_depth(ps_http) @@ -456,7 +460,7 @@ def test_emergency_mode(neon_env_builder: NeonEnvBuilder, pg_bin: PgBin): ps_http = env.pageserver.http_client() - generate_uploads_and_deletions(env) + generate_uploads_and_deletions(env, pageserver_id=env.pageserver.id) env.pageserver.allowed_errors.extend( [ @@ -473,7 +477,7 @@ def test_emergency_mode(neon_env_builder: NeonEnvBuilder, pg_bin: PgBin): # Remember how many validations had happened before the control plane went offline validated = get_deletion_queue_validated(ps_http) - generate_uploads_and_deletions(env, init=False) + generate_uploads_and_deletions(env, init=False, pageserver_id=env.pageserver.id) # The running pageserver should stop progressing deletions time.sleep(10) @@ -488,7 +492,7 @@ def test_emergency_mode(neon_env_builder: NeonEnvBuilder, pg_bin: PgBin): ) # The pageserver should provide service to clients - generate_uploads_and_deletions(env, init=False) + generate_uploads_and_deletions(env, init=False, pageserver_id=env.pageserver.id) # The pageserver should neither validate nor execute any deletions, it should have # loaded the DeletionLists from before though @@ -509,7 +513,7 @@ 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() - generate_uploads_and_deletions(env, init=False) + generate_uploads_and_deletions(env, init=False, pageserver_id=env.pageserver.id) ps_http.deletion_queue_flush(execute=True) assert get_deletion_queue_depth(ps_http) == 0 assert get_deletion_queue_validated(ps_http) > 0 diff --git a/test_runner/regress/test_tenant_relocation.py b/test_runner/regress/test_tenant_relocation.py index 1887bca23b..80b4fab1d3 100644 --- a/test_runner/regress/test_tenant_relocation.py +++ b/test_runner/regress/test_tenant_relocation.py @@ -67,6 +67,7 @@ def load(endpoint: Endpoint, stop_event: threading.Event, load_ok_event: threadi log.info("successfully recovered %s", inserted_ctr) failed = False load_ok_event.set() + log.info("load thread stopped") @@ -144,18 +145,14 @@ def check_timeline_attached( def switch_pg_to_new_pageserver( origin_ps: NeonPageserver, endpoint: Endpoint, - new_pageserver_port: int, + new_pageserver_id: int, tenant_id: TenantId, timeline_id: TimelineId, ) -> Path: + # We could reconfigure online with endpoint.reconfigure(), but this stop/start + # is needed to trigger the logic in load() to set its ok event after restart. endpoint.stop() - - pg_config_file_path = Path(endpoint.config_file_path()) - pg_config_file_path.open("a").write( - f"\nneon.pageserver_connstring = 'postgresql://no_user:@localhost:{new_pageserver_port}'" - ) - - endpoint.start() + endpoint.start(pageserver_id=new_pageserver_id) timeline_to_detach_local_path = origin_ps.timeline_dir(tenant_id, timeline_id) files_before_detach = os.listdir(timeline_to_detach_local_path) @@ -212,7 +209,7 @@ def test_tenant_relocation( env = neon_env_builder.init_start() - tenant_id = TenantId("74ee8b079a0e437eb0afea7d26a07209") + tenant_id = env.initial_tenant env.pageservers[0].allowed_errors.extend( [ @@ -236,8 +233,7 @@ def test_tenant_relocation( origin_http = origin_ps.http_client() destination_http = destination_ps.http_client() - _, initial_timeline_id = env.neon_cli.create_tenant(tenant_id) - log.info("tenant to relocate %s initial_timeline_id %s", tenant_id, initial_timeline_id) + log.info("tenant to relocate %s initial_timeline_id %s", tenant_id, env.initial_timeline) env.neon_cli.create_branch("test_tenant_relocation_main", tenant_id=tenant_id) ep_main = env.endpoints.create_start( @@ -380,7 +376,7 @@ def test_tenant_relocation( old_local_path_main = switch_pg_to_new_pageserver( origin_ps, ep_main, - destination_ps.service_port.pg, + destination_ps.id, tenant_id, timeline_id_main, ) @@ -388,7 +384,7 @@ def test_tenant_relocation( old_local_path_second = switch_pg_to_new_pageserver( origin_ps, ep_second, - destination_ps.service_port.pg, + destination_ps.id, tenant_id, timeline_id_second, ) diff --git a/test_runner/regress/test_tenants.py b/test_runner/regress/test_tenants.py index 5f2c1500d8..2ee2d8125a 100644 --- a/test_runner/regress/test_tenants.py +++ b/test_runner/regress/test_tenants.py @@ -214,14 +214,14 @@ def test_metrics_normal_work(neon_env_builder: NeonEnvBuilder): labels = ",".join([f'{key}="{value}"' for key, value in sample.labels.items()]) log.info(f"{sample.name}{{{labels}}} {sample.value}") - # Test that we gather tenant create metric + # Test that we gather tenant operations metrics storage_operation_metrics = [ "pageserver_storage_operations_seconds_global_bucket", "pageserver_storage_operations_seconds_global_sum", "pageserver_storage_operations_seconds_global_count", ] for metric in storage_operation_metrics: - value = ps_metrics.query_all(metric, filter={"operation": "create tenant"}) + value = ps_metrics.query_all(metric, filter={"operation": "layer flush"}) assert value diff --git a/test_runner/regress/test_timeline_size.py b/test_runner/regress/test_timeline_size.py index 92f2e72378..2e58a413e4 100644 --- a/test_runner/regress/test_timeline_size.py +++ b/test_runner/regress/test_timeline_size.py @@ -759,15 +759,7 @@ def test_ondemand_activation(neon_env_builder: NeonEnvBuilder): tenant_ids = {env.initial_tenant} for _i in range(0, n_tenants - 1): tenant_id = TenantId.generate() - env.pageserver.tenant_create(tenant_id) - - # Empty tenants are not subject to waiting for logical size calculations, because - # those hapen on timeline level - timeline_id = TimelineId.generate() - env.neon_cli.create_timeline( - new_branch_name="main", tenant_id=tenant_id, timeline_id=timeline_id - ) - + env.neon_cli.create_tenant(tenant_id) tenant_ids.add(tenant_id) # Restart pageserver with logical size calculations paused