mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-05 20:42:54 +00:00
control_plane: generalize attachment_service to handle sharding (#6251)
## 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 <arpad-m@users.noreply.github.com>
This commit is contained in:
26
Cargo.lock
generated
26
Cargo.lock
generated
@@ -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"
|
||||
|
||||
@@ -3,6 +3,7 @@ resolver = "2"
|
||||
members = [
|
||||
"compute_tools",
|
||||
"control_plane",
|
||||
"control_plane/attachment_service",
|
||||
"pageserver",
|
||||
"pageserver/ctl",
|
||||
"pageserver/client",
|
||||
|
||||
32
control_plane/attachment_service/Cargo.toml
Normal file
32
control_plane/attachment_service/Cargo.toml
Normal file
@@ -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" }
|
||||
|
||||
116
control_plane/attachment_service/src/compute_hook.rs
Normal file
116
control_plane/attachment_service/src/compute_hook.rs
Normal file
@@ -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::<Vec<_>>();
|
||||
|
||||
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<HashMap<TenantId, ComputeHookTenant>>,
|
||||
}
|
||||
|
||||
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
|
||||
}
|
||||
}
|
||||
218
control_plane/attachment_service/src/http.rs
Normal file
218
control_plane/attachment_service/src/http.rs
Normal file
@@ -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<crate::service::Service>,
|
||||
auth: Option<Arc<SwappableJwtAuth>>,
|
||||
allowlist_routes: Vec<Uri>,
|
||||
}
|
||||
|
||||
impl HttpState {
|
||||
pub fn new(service: Arc<crate::service::Service>, auth: Option<Arc<SwappableJwtAuth>>) -> Self {
|
||||
let allowlist_routes = ["/status"]
|
||||
.iter()
|
||||
.map(|v| v.parse().unwrap())
|
||||
.collect::<Vec<_>>();
|
||||
Self {
|
||||
service,
|
||||
auth,
|
||||
allowlist_routes,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[inline(always)]
|
||||
fn get_state(request: &Request<Body>) -> &HttpState {
|
||||
request
|
||||
.data::<Arc<HttpState>>()
|
||||
.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<Body>) -> Result<Response<Body>, ApiError> {
|
||||
let reattach_req = json_request::<ReAttachRequest>(&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<Body>) -> Result<Response<Body>, ApiError> {
|
||||
let validate_req = json_request::<ValidateRequest>(&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<Body>) -> Result<Response<Body>, ApiError> {
|
||||
let attach_req = json_request::<AttachHookRequest>(&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<Body>) -> Result<Response<Body>, ApiError> {
|
||||
let inspect_req = json_request::<InspectRequest>(&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<Body>) -> Result<Response<Body>, ApiError> {
|
||||
let create_req = json_request::<TenantCreateRequest>(&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<Body>) -> Result<Response<Body>, ApiError> {
|
||||
let tenant_id: TenantId = parse_request_param(&req, "tenant_id")?;
|
||||
let create_req = json_request::<TimelineCreateRequest>(&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<Body>) -> Result<Response<Body>, 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<Body>) -> Result<Response<Body>, ApiError> {
|
||||
let register_req = json_request::<NodeRegisterRequest>(&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<Body>) -> Result<Response<Body>, ApiError> {
|
||||
let node_id: NodeId = parse_request_param(&req, "node_id")?;
|
||||
let config_req = json_request::<NodeConfigureRequest>(&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<Body>) -> Result<Response<Body>, ApiError> {
|
||||
let tenant_shard_id: TenantShardId = parse_request_param(&req, "tenant_shard_id")?;
|
||||
let migrate_req = json_request::<TenantShardMigrateRequest>(&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<Body>) -> Result<Response<Body>, ApiError> {
|
||||
json_response(StatusCode::OK, ())
|
||||
}
|
||||
|
||||
impl From<ReconcileError> for ApiError {
|
||||
fn from(value: ReconcileError) -> Self {
|
||||
ApiError::Conflict(format!("Reconciliation error: {}", value))
|
||||
}
|
||||
}
|
||||
|
||||
pub fn make_router(
|
||||
service: Arc<Service>,
|
||||
auth: Option<Arc<SwappableJwtAuth>>,
|
||||
) -> RouterBuilder<hyper::Body, ApiError> {
|
||||
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)
|
||||
})
|
||||
}
|
||||
57
control_plane/attachment_service/src/lib.rs
Normal file
57
control_plane/attachment_service/src/lib.rs
Normal file
@@ -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<Sequence> 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)
|
||||
}
|
||||
}
|
||||
100
control_plane/attachment_service/src/main.rs
Normal file
100
control_plane/attachment_service/src/main.rs
Normal file
@@ -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<camino::Utf8PathBuf>,
|
||||
|
||||
/// Token for authenticating this service with the pageservers it controls
|
||||
#[arg(short, long)]
|
||||
jwt_token: Option<String>,
|
||||
|
||||
/// 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(())
|
||||
}
|
||||
37
control_plane/attachment_service/src/node.rs
Normal file
37
control_plane/attachment_service/src/node.rs
Normal file
@@ -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,
|
||||
}
|
||||
}
|
||||
}
|
||||
272
control_plane/attachment_service/src/persistence.rs
Normal file
272
control_plane/attachment_service/src/persistence.rs
Normal file
@@ -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<PersistentState>,
|
||||
}
|
||||
|
||||
// Top level state available to all HTTP handlers
|
||||
#[derive(Serialize, Deserialize)]
|
||||
struct PersistentState {
|
||||
tenants: HashMap<TenantShardId, TenantShardPersistence>,
|
||||
|
||||
#[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<u8>,
|
||||
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<Self> {
|
||||
let bytes = tokio::fs::read(path).await?;
|
||||
let mut decoded = serde_json::from_slice::<Self>(&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::<std::io::Error>()
|
||||
.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<Vec<Node>> {
|
||||
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<Vec<TenantShardPersistence>> {
|
||||
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<TenantShardPersistence>,
|
||||
) -> 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<NodeId>,
|
||||
) -> anyhow::Result<Generation> {
|
||||
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<HashMap<TenantShardId, Generation>> {
|
||||
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<NodeId>,
|
||||
|
||||
// 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,
|
||||
}
|
||||
495
control_plane/attachment_service/src/reconciler.rs
Normal file
495
control_plane/attachment_service/src/reconciler.rs
Normal file
@@ -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<HashMap<NodeId, Node>>,
|
||||
|
||||
/// A hook to notify the running postgres instances when we change the location
|
||||
/// of a tenant
|
||||
pub(crate) compute_hook: Arc<ComputeHook>,
|
||||
|
||||
/// 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<Persistence>,
|
||||
}
|
||||
|
||||
#[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<Duration>,
|
||||
) -> 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<HashMap<TimelineId, Lsn>> {
|
||||
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<TimelineId, Lsn>,
|
||||
) -> 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<Generation>,
|
||||
secondary_conf: Option<LocationConfigSecondary>,
|
||||
) -> 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(),
|
||||
}
|
||||
}
|
||||
89
control_plane/attachment_service/src/scheduler.rs
Normal file
89
control_plane/attachment_service/src/scheduler.rs
Normal file
@@ -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<ScheduleError> for ApiError {
|
||||
fn from(value: ScheduleError) -> Self {
|
||||
ApiError::Conflict(format!("Scheduling error: {}", value))
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) struct Scheduler {
|
||||
tenant_counts: HashMap<NodeId, usize>,
|
||||
}
|
||||
|
||||
impl Scheduler {
|
||||
pub(crate) fn new(
|
||||
tenants: &BTreeMap<TenantShardId, TenantState>,
|
||||
nodes: &HashMap<NodeId, Node>,
|
||||
) -> 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<NodeId, ScheduleError> {
|
||||
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)
|
||||
}
|
||||
}
|
||||
1137
control_plane/attachment_service/src/service.rs
Normal file
1137
control_plane/attachment_service/src/service.rs
Normal file
File diff suppressed because it is too large
Load Diff
455
control_plane/attachment_service/src/tenant_state.rs
Normal file
455
control_plane/attachment_service/src/tenant_state.rs
Normal file
@@ -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<ReconcilerHandle>,
|
||||
|
||||
/// Optionally wait for reconciliation to complete up to a particular
|
||||
/// sequence number.
|
||||
pub(crate) waiter: std::sync::Arc<SeqWait<Sequence, Sequence>>,
|
||||
|
||||
/// 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<SeqWait<Sequence, Sequence>>,
|
||||
|
||||
/// 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<std::sync::Mutex<String>>,
|
||||
}
|
||||
|
||||
#[derive(Default, Clone, Debug)]
|
||||
pub(crate) struct IntentState {
|
||||
pub(crate) attached: Option<NodeId>,
|
||||
pub(crate) secondary: Vec<NodeId>,
|
||||
}
|
||||
|
||||
#[derive(Default, Clone)]
|
||||
pub(crate) struct ObservedState {
|
||||
pub(crate) locations: HashMap<NodeId, ObservedStateLocation>,
|
||||
}
|
||||
|
||||
/// 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<LocationConfig>,
|
||||
}
|
||||
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<SeqWait<Sequence, Sequence>>,
|
||||
error_seq_wait: std::sync::Arc<SeqWait<Sequence, Sequence>>,
|
||||
error: std::sync::Arc<std::sync::Mutex<String>>,
|
||||
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<NodeId> {
|
||||
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::<Vec<_>>();
|
||||
|
||||
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<ReconcileResult>,
|
||||
pageservers: &Arc<HashMap<NodeId, Node>>,
|
||||
compute_hook: &Arc<ComputeHook>,
|
||||
service_config: &service::Config,
|
||||
persistence: &Arc<Persistence>,
|
||||
) -> Option<ReconcilerWaiter> {
|
||||
// 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,
|
||||
})
|
||||
}
|
||||
}
|
||||
@@ -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<String>,
|
||||
public_key_path: Option<Utf8PathBuf>,
|
||||
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<NodeId>,
|
||||
}
|
||||
|
||||
@@ -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<TenantCreateResponseShard>,
|
||||
}
|
||||
|
||||
#[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<NodeAvailability>,
|
||||
pub scheduling: Option<NodeSchedulingPolicy>,
|
||||
}
|
||||
|
||||
#[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<TenantLocateResponseShard>,
|
||||
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<Self, Self::Err> {
|
||||
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<Self, Self::Err> {
|
||||
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<NodeSchedulingPolicy> 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<Child> {
|
||||
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::<Vec<_>>();
|
||||
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<RQ, RS>(
|
||||
&self,
|
||||
tenant_id: TenantId,
|
||||
pageserver_id: NodeId,
|
||||
) -> anyhow::Result<Option<u32>> {
|
||||
use hyper::StatusCode;
|
||||
|
||||
method: hyper::Method,
|
||||
path: String,
|
||||
body: Option<RQ>,
|
||||
) -> anyhow::Result<RS>
|
||||
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<Option<u32>> {
|
||||
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::<AttachHookResponse>().await?;
|
||||
Ok(response.gen)
|
||||
}
|
||||
|
||||
pub async fn inspect(&self, tenant_id: TenantId) -> anyhow::Result<Option<(u32, NodeId)>> {
|
||||
use hyper::StatusCode;
|
||||
#[instrument(skip(self))]
|
||||
pub async fn inspect(
|
||||
&self,
|
||||
tenant_shard_id: TenantShardId,
|
||||
) -> anyhow::Result<Option<(u32, NodeId)>> {
|
||||
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::<InspectResponse>().await?;
|
||||
Ok(response.attachment)
|
||||
}
|
||||
|
||||
#[instrument(skip(self))]
|
||||
pub async fn tenant_create(
|
||||
&self,
|
||||
req: TenantCreateRequest,
|
||||
) -> anyhow::Result<TenantCreateResponse> {
|
||||
self.dispatch(Method::POST, "tenant".to_string(), Some(req))
|
||||
.await
|
||||
}
|
||||
|
||||
#[instrument(skip(self))]
|
||||
pub async fn tenant_locate(&self, tenant_id: TenantId) -> anyhow::Result<TenantLocateResponse> {
|
||||
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<TenantShardMigrateResponse> {
|
||||
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<TimelineInfo> {
|
||||
self.dispatch(
|
||||
Method::POST,
|
||||
format!("tenant/{tenant_id}/timeline"),
|
||||
Some(req),
|
||||
)
|
||||
.await
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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<NodeId>,
|
||||
|
||||
// Latest generation number: next time we attach, increment this
|
||||
// and use the incremented number when attaching
|
||||
generation: u32,
|
||||
}
|
||||
|
||||
fn to_hex_map<S, V>(input: &HashMap<TenantId, V>, serializer: S) -> Result<S::Ok, S::Error>
|
||||
where
|
||||
S: serde::Serializer,
|
||||
V: Clone + Serialize,
|
||||
{
|
||||
let transformed = input.iter().map(|(k, v)| (hex::encode(k), v.clone()));
|
||||
|
||||
transformed
|
||||
.collect::<HashMap<String, V>>()
|
||||
.serialize(serializer)
|
||||
}
|
||||
|
||||
fn from_hex_map<'de, D, V>(deserializer: D) -> Result<HashMap<TenantId, V>, D::Error>
|
||||
where
|
||||
D: serde::de::Deserializer<'de>,
|
||||
V: Deserialize<'de>,
|
||||
{
|
||||
let hex_map = HashMap::<String, V>::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<TenantId, TenantState>,
|
||||
|
||||
#[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<Self> {
|
||||
let bytes = tokio::fs::read(path).await?;
|
||||
let mut decoded = serde_json::from_slice::<Self>(&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::<std::io::Error>()
|
||||
.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<tokio::sync::RwLock<PersistentState>>,
|
||||
}
|
||||
|
||||
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<Body>) -> &State {
|
||||
request
|
||||
.data::<Arc<State>>()
|
||||
.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<Body>) -> Result<Response<Body>, ApiError> {
|
||||
let reattach_req = json_request::<ReAttachRequest>(&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<Body>) -> Result<Response<Body>, ApiError> {
|
||||
let validate_req = json_request::<ValidateRequest>(&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<Body>) -> Result<Response<Body>, ApiError> {
|
||||
let attach_req = json_request::<AttachHookRequest>(&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<Body>) -> Result<Response<Body>, ApiError> {
|
||||
let inspect_req = json_request::<InspectRequest>(&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<Body>) -> Result<Response<Body>, ApiError> {
|
||||
let inspect_req = json_request::<InspectRequest>(&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<hyper::Body, ApiError> {
|
||||
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(())
|
||||
}
|
||||
@@ -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<HashMap<TimelineId, TimelineInfo>> {
|
||||
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<TenantShardId> {
|
||||
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<Option<TenantId>> {
|
||||
sub_match
|
||||
.get_one::<String>("tenant-id")
|
||||
@@ -306,6 +325,14 @@ fn parse_tenant_id(sub_match: &ArgMatches) -> anyhow::Result<Option<TenantId>> {
|
||||
.context("Failed to parse tenant id from the argument string")
|
||||
}
|
||||
|
||||
fn parse_tenant_shard_id(sub_match: &ArgMatches) -> anyhow::Result<Option<TenantShardId>> {
|
||||
sub_match
|
||||
.get_one::<String>("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<Option<TimelineId>> {
|
||||
sub_match
|
||||
.get_one::<String>("timeline-id")
|
||||
@@ -394,47 +421,68 @@ async fn handle_tenant(
|
||||
Some(("create", create_match)) => {
|
||||
let tenant_conf: HashMap<_, _> = create_match
|
||||
.get_many::<String>("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::<u8>("shard-count")
|
||||
.cloned()
|
||||
.unwrap_or(0);
|
||||
|
||||
let shard_stripe_size: Option<u32> =
|
||||
create_match.get_one::<u32>("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::<u32>("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::<String>("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::<String>("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::<String>("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::<Vec<_>>();
|
||||
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::<String>("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::<Vec<_>>()
|
||||
};
|
||||
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")
|
||||
|
||||
@@ -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<u16>,
|
||||
pg_version: u32,
|
||||
mode: ComputeMode,
|
||||
pageserver_id: NodeId,
|
||||
) -> Result<Arc<Endpoint>> {
|
||||
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::<Vec<_>>()
|
||||
.join(",")
|
||||
}
|
||||
|
||||
pub async fn start(
|
||||
&self,
|
||||
auth_token: &Option<String>,
|
||||
safekeepers: Vec<NodeId>,
|
||||
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<NodeId>) -> 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::<Vec<_>>();
|
||||
}
|
||||
|
||||
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!(
|
||||
|
||||
@@ -14,4 +14,3 @@ pub mod local_env;
|
||||
pub mod pageserver;
|
||||
pub mod postgresql_conf;
|
||||
pub mod safekeeper;
|
||||
pub mod tenant_migration;
|
||||
|
||||
@@ -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::<Vec<_>>();
|
||||
let joined = have_ids.join(",");
|
||||
bail!("could not find pageserver {id}, have ids {joined}")
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -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<Vec<TenantInfo>> {
|
||||
self.http_client.list_tenants().await
|
||||
}
|
||||
|
||||
pub async fn tenant_create(
|
||||
&self,
|
||||
new_tenant_id: TenantId,
|
||||
generation: Option<u32>,
|
||||
settings: HashMap<&str, &str>,
|
||||
) -> anyhow::Result<TenantId> {
|
||||
let mut settings = settings.clone();
|
||||
|
||||
let config = models::TenantConfig {
|
||||
pub fn parse_config(mut settings: HashMap<&str, &str>) -> anyhow::Result<models::TenantConfig> {
|
||||
let result = models::TenantConfig {
|
||||
checkpoint_distance: settings
|
||||
.remove("checkpoint_distance")
|
||||
.map(|x| x.parse::<u64>())
|
||||
@@ -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<u32>,
|
||||
settings: HashMap<&str, &str>,
|
||||
) -> anyhow::Result<TenantId> {
|
||||
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<TimelineId>,
|
||||
tenant_shard_id: TenantShardId,
|
||||
new_timeline_id: TimelineId,
|
||||
ancestor_start_lsn: Option<Lsn>,
|
||||
ancestor_timeline_id: Option<TimelineId>,
|
||||
pg_version: Option<u32>,
|
||||
existing_initdb_timeline_id: Option<TimelineId>,
|
||||
) -> anyhow::Result<TimelineInfo> {
|
||||
// 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<TenantHistorySize> {
|
||||
Ok(self
|
||||
.http_client
|
||||
.tenant_synthetic_size(tenant_shard_id)
|
||||
.await?)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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<HashMap<TimelineId, Lsn>> {
|
||||
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<TimelineId, Lsn>,
|
||||
) -> 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<u32>,
|
||||
secondary_conf: Option<LocationConfigSecondary>,
|
||||
) -> 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(())
|
||||
}
|
||||
@@ -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<u64>,
|
||||
pub checkpoint_timeout: Option<String>,
|
||||
@@ -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?
|
||||
|
||||
@@ -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<Self, Self::Err> {
|
||||
Ok(NodeId(u64::from_str(s)?))
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use serde_assert::{Deserializer, Serializer, Token, Tokens};
|
||||
|
||||
@@ -28,8 +28,8 @@ pub enum Error {
|
||||
|
||||
pub type Result<T> = std::result::Result<T, Error>;
|
||||
|
||||
pub(crate) trait ResponseErrorMessageExt: Sized {
|
||||
async fn error_from_body(self) -> Result<Self>;
|
||||
pub trait ResponseErrorMessageExt: Sized {
|
||||
fn error_from_body(self) -> impl std::future::Future<Output = Result<Self>> + 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<TimelineInfo> {
|
||||
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<Vec<TimelineInfo>> {
|
||||
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,
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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()):
|
||||
|
||||
@@ -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
|
||||
|
||||
|
||||
|
||||
@@ -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)
|
||||
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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(
|
||||
[
|
||||
|
||||
@@ -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
|
||||
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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,
|
||||
{
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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,
|
||||
)
|
||||
|
||||
@@ -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
|
||||
|
||||
|
||||
|
||||
@@ -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
|
||||
|
||||
Reference in New Issue
Block a user