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:
John Spray
2024-03-14 09:11:57 +00:00
committed by GitHub
parent 3bd6551b36
commit 44f42627dd
18 changed files with 1445 additions and 150 deletions

View File

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

View File

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

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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