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:
John Spray
2024-01-17 18:01:08 +00:00
committed by GitHub
parent 4cec95ba13
commit b6ec11ad78
38 changed files with 4162 additions and 922 deletions

26
Cargo.lock generated
View File

@@ -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"

View File

@@ -3,6 +3,7 @@ resolver = "2"
members = [
"compute_tools",
"control_plane",
"control_plane/attachment_service",
"pageserver",
"pageserver/ctl",
"pageserver/client",

View 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" }

View 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
}
}

View 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)
})
}

View 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)
}
}

View 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(())
}

View 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,
}
}
}

View 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,
}

View 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(),
}
}

View 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)
}
}

File diff suppressed because it is too large Load Diff

View 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,
})
}
}

View File

@@ -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
}
}

View File

@@ -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(())
}

View File

@@ -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")

View File

@@ -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!(

View File

@@ -14,4 +14,3 @@ pub mod local_env;
pub mod pageserver;
pub mod postgresql_conf;
pub mod safekeeper;
pub mod tenant_migration;

View File

@@ -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}")
}
}

View File

@@ -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?)
}
}

View File

@@ -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(())
}

View File

@@ -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?

View File

@@ -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};

View File

@@ -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,

View File

@@ -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,

View File

@@ -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);

View File

@@ -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()):

View File

@@ -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

View File

@@ -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)

View File

@@ -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,

View File

@@ -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(
[

View File

@@ -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

View File

@@ -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

View File

@@ -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,
{

View File

@@ -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

View File

@@ -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,
)

View File

@@ -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

View File

@@ -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