mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-06 13:02:55 +00:00
pageserver/controller: error handling for shard splitting (#7074)
## Problem Shard splits worked, but weren't safe against failures (e.g. node crash during split) yet. Related: #6676 ## Summary of changes - Introduce async rwlocks at the scope of Tenant and Node: - exclusive tenant lock is used to protect splits - exclusive node lock is used to protect new reconciliation process that happens when setting node active - exclusive locks used in both cases when doing persistent updates (e.g. node scheduling conf) where the update to DB & in-memory state needs to be atomic. - Add failpoints to shard splitting in control plane and pageserver code. - Implement error handling in control plane for shard splits: this detaches child chards and ensures parent shards are re-attached. - Crash-safety for storage controller restarts requires little effort: we already reconcile with nodes over a storage controller restart, so as long as we reset any incomplete splits in the DB on restart (added in this PR), things are implicitly cleaned up. - Implement reconciliation with offline nodes before they transition to active: - (in this context reconciliation means something like startup_reconcile, not literally the Reconciler) - This covers cases where split abort cannot reach a node to clean it up: the cleanup will eventually happen when the node is marked active, as part of reconciliation. - This also covers the case where a node was unavailable when the storage controller started, but becomes available later: previously this allowed it to skip the startup reconcile. - Storage controller now terminates on panics. We only use panics for true "should never happen" assertions, and these cases can leave us in an un-usable state if we keep running (e.g. panicking in a shard split). In the unlikely event that we get into a crashloop as a result, we'll rely on kubernetes to back us off. - Add `test_sharding_split_failures` which exercises a variety of failure cases during shard split.
This commit is contained in:
@@ -19,8 +19,10 @@ aws-config.workspace = true
|
||||
aws-sdk-secretsmanager.workspace = true
|
||||
camino.workspace = true
|
||||
clap.workspace = true
|
||||
fail.workspace = true
|
||||
futures.workspace = true
|
||||
git-version.workspace = true
|
||||
hex.workspace = true
|
||||
hyper.workspace = true
|
||||
humantime.workspace = true
|
||||
once_cell.workspace = true
|
||||
|
||||
@@ -10,7 +10,9 @@ use pageserver_api::shard::TenantShardId;
|
||||
use pageserver_client::mgmt_api;
|
||||
use std::sync::Arc;
|
||||
use std::time::{Duration, Instant};
|
||||
use tokio_util::sync::CancellationToken;
|
||||
use utils::auth::{Scope, SwappableJwtAuth};
|
||||
use utils::failpoint_support::failpoints_handler;
|
||||
use utils::http::endpoint::{auth_middleware, check_permission_with, request_span};
|
||||
use utils::http::request::{must_get_query_param, parse_request_param};
|
||||
use utils::id::{TenantId, TimelineId};
|
||||
@@ -554,6 +556,9 @@ pub fn make_router(
|
||||
.post("/debug/v1/consistency_check", |r| {
|
||||
request_span(r, handle_consistency_check)
|
||||
})
|
||||
.put("/debug/v1/failpoints", |r| {
|
||||
request_span(r, |r| failpoints_handler(r, CancellationToken::new()))
|
||||
})
|
||||
.get("/control/v1/tenant/:tenant_id/locate", |r| {
|
||||
tenant_service_handler(r, handle_tenant_locate)
|
||||
})
|
||||
|
||||
54
control_plane/attachment_service/src/id_lock_map.rs
Normal file
54
control_plane/attachment_service/src/id_lock_map.rs
Normal file
@@ -0,0 +1,54 @@
|
||||
use std::{collections::HashMap, sync::Arc};
|
||||
|
||||
/// A map of locks covering some arbitrary identifiers. Useful if you have a collection of objects but don't
|
||||
/// want to embed a lock in each one, or if your locking granularity is different to your object granularity.
|
||||
/// For example, used in the storage controller where the objects are tenant shards, but sometimes locking
|
||||
/// is needed at a tenant-wide granularity.
|
||||
pub(crate) struct IdLockMap<T>
|
||||
where
|
||||
T: Eq + PartialEq + std::hash::Hash,
|
||||
{
|
||||
/// A synchronous lock for getting/setting the async locks that our callers will wait on.
|
||||
entities: std::sync::Mutex<std::collections::HashMap<T, Arc<tokio::sync::RwLock<()>>>>,
|
||||
}
|
||||
|
||||
impl<T> IdLockMap<T>
|
||||
where
|
||||
T: Eq + PartialEq + std::hash::Hash,
|
||||
{
|
||||
pub(crate) fn shared(
|
||||
&self,
|
||||
key: T,
|
||||
) -> impl std::future::Future<Output = tokio::sync::OwnedRwLockReadGuard<()>> {
|
||||
let mut locked = self.entities.lock().unwrap();
|
||||
let entry = locked.entry(key).or_default();
|
||||
entry.clone().read_owned()
|
||||
}
|
||||
|
||||
pub(crate) fn exclusive(
|
||||
&self,
|
||||
key: T,
|
||||
) -> impl std::future::Future<Output = tokio::sync::OwnedRwLockWriteGuard<()>> {
|
||||
let mut locked = self.entities.lock().unwrap();
|
||||
let entry = locked.entry(key).or_default();
|
||||
entry.clone().write_owned()
|
||||
}
|
||||
|
||||
/// Rather than building a lock guard that re-takes the [`Self::entities`] lock, we just do
|
||||
/// periodic housekeeping to avoid the map growing indefinitely
|
||||
pub(crate) fn housekeeping(&self) {
|
||||
let mut locked = self.entities.lock().unwrap();
|
||||
locked.retain(|_k, lock| lock.try_write().is_err())
|
||||
}
|
||||
}
|
||||
|
||||
impl<T> Default for IdLockMap<T>
|
||||
where
|
||||
T: Eq + PartialEq + std::hash::Hash,
|
||||
{
|
||||
fn default() -> Self {
|
||||
Self {
|
||||
entities: std::sync::Mutex::new(HashMap::new()),
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -4,6 +4,7 @@ use utils::seqwait::MonotonicCounter;
|
||||
mod auth;
|
||||
mod compute_hook;
|
||||
pub mod http;
|
||||
mod id_lock_map;
|
||||
pub mod metrics;
|
||||
mod node;
|
||||
pub mod persistence;
|
||||
|
||||
@@ -206,6 +206,12 @@ async fn migration_run(database_url: &str) -> anyhow::Result<()> {
|
||||
}
|
||||
|
||||
fn main() -> anyhow::Result<()> {
|
||||
let default_panic = std::panic::take_hook();
|
||||
std::panic::set_hook(Box::new(move |info| {
|
||||
default_panic(info);
|
||||
std::process::exit(1);
|
||||
}));
|
||||
|
||||
tokio::runtime::Builder::new_current_thread()
|
||||
// We use spawn_blocking for database operations, so require approximately
|
||||
// as many blocking threads as we will open database connections.
|
||||
|
||||
@@ -83,29 +83,38 @@ impl Node {
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn set_availability(
|
||||
&mut self,
|
||||
availability: NodeAvailability,
|
||||
) -> AvailabilityTransition {
|
||||
use NodeAvailability::*;
|
||||
let transition = match (self.availability, availability) {
|
||||
(Offline, Active) => {
|
||||
pub(crate) fn set_availability(&mut self, availability: NodeAvailability) {
|
||||
match self.get_availability_transition(availability) {
|
||||
AvailabilityTransition::ToActive => {
|
||||
// Give the node a new cancellation token, effectively resetting it to un-cancelled. Any
|
||||
// users of previously-cloned copies of the node will still see the old cancellation
|
||||
// state. For example, Reconcilers in flight will have to complete and be spawned
|
||||
// again to realize that the node has become available.
|
||||
self.cancel = CancellationToken::new();
|
||||
AvailabilityTransition::ToActive
|
||||
}
|
||||
(Active, Offline) => {
|
||||
AvailabilityTransition::ToOffline => {
|
||||
// Fire the node's cancellation token to cancel any in-flight API requests to it
|
||||
self.cancel.cancel();
|
||||
AvailabilityTransition::ToOffline
|
||||
}
|
||||
_ => AvailabilityTransition::Unchanged,
|
||||
};
|
||||
AvailabilityTransition::Unchanged => {}
|
||||
}
|
||||
self.availability = availability;
|
||||
transition
|
||||
}
|
||||
|
||||
/// Without modifying the availability of the node, convert the intended availability
|
||||
/// into a description of the transition.
|
||||
pub(crate) fn get_availability_transition(
|
||||
&self,
|
||||
availability: NodeAvailability,
|
||||
) -> AvailabilityTransition {
|
||||
use AvailabilityTransition::*;
|
||||
use NodeAvailability::*;
|
||||
|
||||
match (self.availability, availability) {
|
||||
(Offline, Active) => ToActive,
|
||||
(Active, Offline) => ToOffline,
|
||||
_ => Unchanged,
|
||||
}
|
||||
}
|
||||
|
||||
/// Whether we may send API requests to this node.
|
||||
|
||||
@@ -11,6 +11,9 @@ use diesel::prelude::*;
|
||||
use diesel::Connection;
|
||||
use pageserver_api::controller_api::{NodeSchedulingPolicy, PlacementPolicy};
|
||||
use pageserver_api::models::TenantConfig;
|
||||
use pageserver_api::shard::ShardConfigError;
|
||||
use pageserver_api::shard::ShardIdentity;
|
||||
use pageserver_api::shard::ShardStripeSize;
|
||||
use pageserver_api::shard::{ShardCount, ShardNumber, TenantShardId};
|
||||
use serde::{Deserialize, Serialize};
|
||||
use utils::generation::Generation;
|
||||
@@ -72,6 +75,14 @@ pub(crate) enum DatabaseError {
|
||||
Logical(String),
|
||||
}
|
||||
|
||||
#[must_use]
|
||||
pub(crate) enum AbortShardSplitStatus {
|
||||
/// We aborted the split in the database by reverting to the parent shards
|
||||
Aborted,
|
||||
/// The split had already been persisted.
|
||||
Complete,
|
||||
}
|
||||
|
||||
pub(crate) type DatabaseResult<T> = Result<T, DatabaseError>;
|
||||
|
||||
impl Persistence {
|
||||
@@ -570,6 +581,51 @@ impl Persistence {
|
||||
})
|
||||
.await
|
||||
}
|
||||
|
||||
/// Used when the remote part of a shard split failed: we will revert the database state to have only
|
||||
/// the parent shards, with SplitState::Idle.
|
||||
pub(crate) async fn abort_shard_split(
|
||||
&self,
|
||||
split_tenant_id: TenantId,
|
||||
new_shard_count: ShardCount,
|
||||
) -> DatabaseResult<AbortShardSplitStatus> {
|
||||
use crate::schema::tenant_shards::dsl::*;
|
||||
self.with_conn(move |conn| -> DatabaseResult<AbortShardSplitStatus> {
|
||||
let aborted = conn.transaction(|conn| -> DatabaseResult<AbortShardSplitStatus> {
|
||||
// Clear the splitting state on parent shards
|
||||
let updated = diesel::update(tenant_shards)
|
||||
.filter(tenant_id.eq(split_tenant_id.to_string()))
|
||||
.filter(shard_count.ne(new_shard_count.literal() as i32))
|
||||
.set((splitting.eq(0),))
|
||||
.execute(conn)?;
|
||||
|
||||
// Parent shards are already gone: we cannot abort.
|
||||
if updated == 0 {
|
||||
return Ok(AbortShardSplitStatus::Complete);
|
||||
}
|
||||
|
||||
// Sanity check: if parent shards were present, their cardinality should
|
||||
// be less than the number of child shards.
|
||||
if updated >= new_shard_count.count() as usize {
|
||||
return Err(DatabaseError::Logical(format!(
|
||||
"Unexpected parent shard count {updated} while aborting split to \
|
||||
count {new_shard_count:?} on tenant {split_tenant_id}"
|
||||
)));
|
||||
}
|
||||
|
||||
// Erase child shards
|
||||
diesel::delete(tenant_shards)
|
||||
.filter(tenant_id.eq(split_tenant_id.to_string()))
|
||||
.filter(shard_count.eq(new_shard_count.literal() as i32))
|
||||
.execute(conn)?;
|
||||
|
||||
Ok(AbortShardSplitStatus::Aborted)
|
||||
})?;
|
||||
|
||||
Ok(aborted)
|
||||
})
|
||||
.await
|
||||
}
|
||||
}
|
||||
|
||||
/// Parts of [`crate::tenant_state::TenantState`] that are stored durably
|
||||
@@ -604,6 +660,28 @@ pub(crate) struct TenantShardPersistence {
|
||||
pub(crate) config: String,
|
||||
}
|
||||
|
||||
impl TenantShardPersistence {
|
||||
pub(crate) fn get_shard_identity(&self) -> Result<ShardIdentity, ShardConfigError> {
|
||||
if self.shard_count == 0 {
|
||||
Ok(ShardIdentity::unsharded())
|
||||
} else {
|
||||
Ok(ShardIdentity::new(
|
||||
ShardNumber(self.shard_number as u8),
|
||||
ShardCount::new(self.shard_count as u8),
|
||||
ShardStripeSize(self.shard_stripe_size as u32),
|
||||
)?)
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn get_tenant_shard_id(&self) -> Result<TenantShardId, hex::FromHexError> {
|
||||
Ok(TenantShardId {
|
||||
tenant_id: TenantId::from_str(self.tenant_id.as_str())?,
|
||||
shard_number: ShardNumber(self.shard_number as u8),
|
||||
shard_count: ShardCount::new(self.shard_count as u8),
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
/// Parts of [`crate::node::Node`] that are stored durably
|
||||
#[derive(Serialize, Deserialize, Queryable, Selectable, Insertable, Eq, PartialEq)]
|
||||
#[diesel(table_name = crate::schema::nodes)]
|
||||
|
||||
@@ -1,5 +1,6 @@
|
||||
use crate::persistence::Persistence;
|
||||
use crate::service;
|
||||
use hyper::StatusCode;
|
||||
use pageserver_api::models::{
|
||||
LocationConfig, LocationConfigMode, LocationConfigSecondary, TenantConfig,
|
||||
};
|
||||
@@ -18,6 +19,8 @@ use crate::compute_hook::{ComputeHook, NotifyError};
|
||||
use crate::node::Node;
|
||||
use crate::tenant_state::{IntentState, ObservedState, ObservedStateLocation};
|
||||
|
||||
const DEFAULT_HEATMAP_PERIOD: &str = "60s";
|
||||
|
||||
/// Object with the lifetime of the background reconcile task that is created
|
||||
/// for tenants which have a difference between their intent and observed states.
|
||||
pub(super) struct Reconciler {
|
||||
@@ -485,17 +488,29 @@ impl Reconciler {
|
||||
)
|
||||
.await
|
||||
{
|
||||
Some(Ok(observed)) => observed,
|
||||
Some(Ok(observed)) => Some(observed),
|
||||
Some(Err(mgmt_api::Error::ApiError(status, _msg)))
|
||||
if status == StatusCode::NOT_FOUND =>
|
||||
{
|
||||
None
|
||||
}
|
||||
Some(Err(e)) => return Err(e.into()),
|
||||
None => return Err(ReconcileError::Cancel),
|
||||
};
|
||||
tracing::info!("Scanned location configuration on {attached_node}: {observed_conf:?}");
|
||||
self.observed.locations.insert(
|
||||
attached_node.get_id(),
|
||||
ObservedStateLocation {
|
||||
conf: observed_conf,
|
||||
},
|
||||
);
|
||||
match observed_conf {
|
||||
Some(conf) => {
|
||||
// Pageserver returned a state: update it in observed. This may still be an indeterminate (None) state,
|
||||
// if internally the pageserver's TenantSlot was being mutated (e.g. some long running API call is still running)
|
||||
self.observed
|
||||
.locations
|
||||
.insert(attached_node.get_id(), ObservedStateLocation { conf });
|
||||
}
|
||||
None => {
|
||||
// Pageserver returned 404: we have confirmation that there is no state for this shard on that pageserver.
|
||||
self.observed.locations.remove(&attached_node.get_id());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
@@ -525,7 +540,12 @@ impl Reconciler {
|
||||
)));
|
||||
};
|
||||
|
||||
let mut wanted_conf = attached_location_conf(generation, &self.shard, &self.config);
|
||||
let mut wanted_conf = attached_location_conf(
|
||||
generation,
|
||||
&self.shard,
|
||||
&self.config,
|
||||
!self.intent.secondary.is_empty(),
|
||||
);
|
||||
match self.observed.locations.get(&node.get_id()) {
|
||||
Some(conf) if conf.conf.as_ref() == Some(&wanted_conf) => {
|
||||
// Nothing to do
|
||||
@@ -662,10 +682,26 @@ impl Reconciler {
|
||||
}
|
||||
}
|
||||
|
||||
/// We tweak the externally-set TenantConfig while configuring
|
||||
/// locations, using our awareness of whether secondary locations
|
||||
/// are in use to automatically enable/disable heatmap uploads.
|
||||
fn ha_aware_config(config: &TenantConfig, has_secondaries: bool) -> TenantConfig {
|
||||
let mut config = config.clone();
|
||||
if has_secondaries {
|
||||
if config.heatmap_period.is_none() {
|
||||
config.heatmap_period = Some(DEFAULT_HEATMAP_PERIOD.to_string());
|
||||
}
|
||||
} else {
|
||||
config.heatmap_period = None;
|
||||
}
|
||||
config
|
||||
}
|
||||
|
||||
pub(crate) fn attached_location_conf(
|
||||
generation: Generation,
|
||||
shard: &ShardIdentity,
|
||||
config: &TenantConfig,
|
||||
has_secondaries: bool,
|
||||
) -> LocationConfig {
|
||||
LocationConfig {
|
||||
mode: LocationConfigMode::AttachedSingle,
|
||||
@@ -674,7 +710,7 @@ pub(crate) fn attached_location_conf(
|
||||
shard_number: shard.number.0,
|
||||
shard_count: shard.count.literal(),
|
||||
shard_stripe_size: shard.stripe_size.0,
|
||||
tenant_conf: config.clone(),
|
||||
tenant_conf: ha_aware_config(config, has_secondaries),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -689,6 +725,6 @@ pub(crate) fn secondary_location_conf(
|
||||
shard_number: shard.number.0,
|
||||
shard_count: shard.count.literal(),
|
||||
shard_stripe_size: shard.stripe_size.0,
|
||||
tenant_conf: config.clone(),
|
||||
tenant_conf: ha_aware_config(config, true),
|
||||
}
|
||||
}
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
@@ -577,7 +577,12 @@ impl TenantState {
|
||||
.generation
|
||||
.expect("Attempted to enter attached state without a generation");
|
||||
|
||||
let wanted_conf = attached_location_conf(generation, &self.shard, &self.config);
|
||||
let wanted_conf = attached_location_conf(
|
||||
generation,
|
||||
&self.shard,
|
||||
&self.config,
|
||||
!self.intent.secondary.is_empty(),
|
||||
);
|
||||
match self.observed.locations.get(&node_id) {
|
||||
Some(conf) if conf.conf.as_ref() == Some(&wanted_conf) => {}
|
||||
Some(_) | None => {
|
||||
|
||||
Reference in New Issue
Block a user