mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-07 13:32:57 +00:00
refactor TenantState transitions (#4321)
This is preliminary work for/from #4220 (async `Layer::get_value_reconstruct_data`). The motivation is to avoid locking `Tenant::timelines` in places that can't be `async`, because in #4333 we want to convert Tenant::timelines from `std::sync::Mutex` to `tokio::sync::Mutex`. But, the changes here are useful in general because they clean up & document tenant state transitions. That also paves the way for #4350, which is an alternative to #4333 that refactors the pageserver code so that we can keep the `Tenant::timelines` mutex sync. This patch consists of the following core insights and changes: * spawn_load and spawn_attach own the tenant state until they're done * once load()/attach() calls are done ... * if they failed, transition them to Broken directly (we know that there's no background activity because we didn't call activate yet) * if they succeed, call activate. We can make it infallible. How? Later. * set_broken() and set_stopping() are changed to wait for spawn_load() / spawn_attach() to finish. * This sounds scary because it might hinder detach or shutdown, but actually, concurrent attach+detach, or attach+shutdown, or load+shutdown, or attach+shutdown were just racy before this PR. So, with this change, they're not anymore. In the future, we can add a `CancellationToken` stored in Tenant to cancel `load` and `attach` faster, i.e., make `spawn_load` / `spawn_attach` transition them to Broken state sooner. See the doc comments on TenantState for the state transitions that are now possible. It might seem scary, but actually, this patch reduces the possible state transitions. We introduce a new state `TenantState::Activating` to avoid grabbing the `Tenant::timelines` lock inside the `send_modify` closure. These were the humble beginnings of this PR (see Motivation section), and I think it's still the right thing to have this `Activating` state, even if we decide against async `Tenant::timelines` mutex. The reason is that `send_modify` locks internally, and by moving locking of Tenant::timelines out of the closure, the internal locking of `send_modify` becomes a leaf of the lock graph, and so, we eliminate deadlock risk. Co-authored-by: Joonas Koivunen <joonas@neon.tech>
This commit is contained in:
committed by
GitHub
parent
ccf653c1f4
commit
f4f300732a
@@ -18,7 +18,29 @@ use crate::reltag::RelTag;
|
||||
use anyhow::bail;
|
||||
use bytes::{BufMut, Bytes, BytesMut};
|
||||
|
||||
/// A state of a tenant in pageserver's memory.
|
||||
/// The state of a tenant in this pageserver.
|
||||
///
|
||||
/// ```mermaid
|
||||
/// stateDiagram-v2
|
||||
///
|
||||
/// [*] --> Loading: spawn_load()
|
||||
/// [*] --> Attaching: spawn_attach()
|
||||
///
|
||||
/// Loading --> Activating: activate()
|
||||
/// Attaching --> Activating: activate()
|
||||
/// Activating --> Active: infallible
|
||||
///
|
||||
/// Loading --> Broken: load() failure
|
||||
/// Attaching --> Broken: attach() failure
|
||||
///
|
||||
/// Active --> Stopping: set_stopping(), part of shutdown & detach
|
||||
/// Stopping --> Broken: late error in remove_tenant_from_memory
|
||||
///
|
||||
/// Broken --> [*]: ignore / detach / shutdown
|
||||
/// Stopping --> [*]: remove_from_memory complete
|
||||
///
|
||||
/// Active --> Broken: cfg(testing)-only tenant break point
|
||||
/// ```
|
||||
#[derive(
|
||||
Clone,
|
||||
PartialEq,
|
||||
@@ -26,40 +48,63 @@ use bytes::{BufMut, Bytes, BytesMut};
|
||||
serde::Serialize,
|
||||
serde::Deserialize,
|
||||
strum_macros::Display,
|
||||
strum_macros::EnumString,
|
||||
strum_macros::EnumVariantNames,
|
||||
strum_macros::AsRefStr,
|
||||
strum_macros::IntoStaticStr,
|
||||
)]
|
||||
#[serde(tag = "slug", content = "data")]
|
||||
pub enum TenantState {
|
||||
/// This tenant is being loaded from local disk
|
||||
/// This tenant is being loaded from local disk.
|
||||
///
|
||||
/// `set_stopping()` and `set_broken()` do not work in this state and wait for it to pass.
|
||||
Loading,
|
||||
/// This tenant is being downloaded from cloud storage.
|
||||
/// This tenant is being attached to the pageserver.
|
||||
///
|
||||
/// `set_stopping()` and `set_broken()` do not work in this state and wait for it to pass.
|
||||
Attaching,
|
||||
/// Tenant is fully operational
|
||||
/// The tenant is transitioning from Loading/Attaching to Active.
|
||||
///
|
||||
/// While in this state, the individual timelines are being activated.
|
||||
///
|
||||
/// `set_stopping()` and `set_broken()` do not work in this state and wait for it to pass.
|
||||
Activating(ActivatingFrom),
|
||||
/// The tenant has finished activating and is open for business.
|
||||
///
|
||||
/// Transitions out of this state are possible through `set_stopping()` and `set_broken()`.
|
||||
Active,
|
||||
/// A tenant is recognized by pageserver, but it is being detached or the
|
||||
/// The tenant is recognized by pageserver, but it is being detached or the
|
||||
/// system is being shut down.
|
||||
///
|
||||
/// Transitions out of this state are possible through `set_broken()`.
|
||||
Stopping,
|
||||
/// A tenant is recognized by the pageserver, but can no longer be used for
|
||||
/// any operations, because it failed to be activated.
|
||||
/// The tenant is recognized by the pageserver, but can no longer be used for
|
||||
/// any operations.
|
||||
///
|
||||
/// If the tenant fails to load or attach, it will transition to this state
|
||||
/// and it is guaranteed that no background tasks are running in its name.
|
||||
///
|
||||
/// The other way to transition into this state is from `Stopping` state
|
||||
/// through `set_broken()` called from `remove_tenant_from_memory()`. That happens
|
||||
/// if the cleanup future executed by `remove_tenant_from_memory()` fails.
|
||||
Broken { reason: String, backtrace: String },
|
||||
}
|
||||
|
||||
impl TenantState {
|
||||
pub fn attachment_status(&self) -> TenantAttachmentStatus {
|
||||
use TenantAttachmentStatus::*;
|
||||
|
||||
// Below TenantState::Activating is used as "transient" or "transparent" state for
|
||||
// attachment_status determining.
|
||||
match self {
|
||||
// The attach procedure writes the marker file before adding the Attaching tenant to the tenants map.
|
||||
// So, technically, we can return Attached here.
|
||||
// However, as soon as Console observes Attached, it will proceed with the Postgres-level health check.
|
||||
// But, our attach task might still be fetching the remote timelines, etc.
|
||||
// So, return `Maybe` while Attaching, making Console wait for the attach task to finish.
|
||||
Self::Attaching => Maybe,
|
||||
Self::Attaching | Self::Activating(ActivatingFrom::Attaching) => Maybe,
|
||||
// tenant mgr startup distinguishes attaching from loading via marker file.
|
||||
// If it's loading, there is no attach marker file, i.e., attach had finished in the past.
|
||||
Self::Loading => Attached,
|
||||
Self::Loading | Self::Activating(ActivatingFrom::Loading) => Attached,
|
||||
// We only reach Active after successful load / attach.
|
||||
// So, call atttachment status Attached.
|
||||
Self::Active => Attached,
|
||||
@@ -98,6 +143,15 @@ impl std::fmt::Debug for TenantState {
|
||||
}
|
||||
}
|
||||
|
||||
/// The only [`TenantState`] variants we could be `TenantState::Activating` from.
|
||||
#[derive(Clone, Copy, Debug, PartialEq, Eq, serde::Serialize, serde::Deserialize)]
|
||||
pub enum ActivatingFrom {
|
||||
/// Arrived to [`TenantState::Activating`] from [`TenantState::Loading`]
|
||||
Loading,
|
||||
/// Arrived to [`TenantState::Activating`] from [`TenantState::Attaching`]
|
||||
Attaching,
|
||||
}
|
||||
|
||||
/// A state of a timeline in pageserver's memory.
|
||||
#[derive(Debug, Clone, Copy, PartialEq, Eq, serde::Serialize, serde::Deserialize)]
|
||||
pub enum TimelineState {
|
||||
@@ -829,4 +883,55 @@ mod tests {
|
||||
err
|
||||
);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn tenantstatus_activating_serde() {
|
||||
let states = [
|
||||
TenantState::Activating(ActivatingFrom::Loading),
|
||||
TenantState::Activating(ActivatingFrom::Attaching),
|
||||
];
|
||||
let expected = "[{\"slug\":\"Activating\",\"data\":\"Loading\"},{\"slug\":\"Activating\",\"data\":\"Attaching\"}]";
|
||||
|
||||
let actual = serde_json::to_string(&states).unwrap();
|
||||
|
||||
assert_eq!(actual, expected);
|
||||
|
||||
let parsed = serde_json::from_str::<Vec<TenantState>>(&actual).unwrap();
|
||||
|
||||
assert_eq!(states.as_slice(), &parsed);
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn tenantstatus_activating_strum() {
|
||||
// tests added, because we use these for metrics
|
||||
let examples = [
|
||||
(line!(), TenantState::Loading, "Loading"),
|
||||
(line!(), TenantState::Attaching, "Attaching"),
|
||||
(
|
||||
line!(),
|
||||
TenantState::Activating(ActivatingFrom::Loading),
|
||||
"Activating",
|
||||
),
|
||||
(
|
||||
line!(),
|
||||
TenantState::Activating(ActivatingFrom::Attaching),
|
||||
"Activating",
|
||||
),
|
||||
(line!(), TenantState::Active, "Active"),
|
||||
(line!(), TenantState::Stopping, "Stopping"),
|
||||
(
|
||||
line!(),
|
||||
TenantState::Broken {
|
||||
reason: "Example".into(),
|
||||
backtrace: "Looooong backtrace".into(),
|
||||
},
|
||||
"Broken",
|
||||
),
|
||||
];
|
||||
|
||||
for (line, rendered, expected) in examples {
|
||||
let actual: &'static str = rendered.into();
|
||||
assert_eq!(actual, expected, "example on {line}");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -859,7 +859,7 @@ async fn handle_tenant_break(r: Request<Body>) -> Result<Response<Body>, ApiErro
|
||||
.await
|
||||
.map_err(|_| ApiError::Conflict(String::from("no active tenant found")))?;
|
||||
|
||||
tenant.set_broken("broken from test".to_owned());
|
||||
tenant.set_broken("broken from test".to_owned()).await;
|
||||
|
||||
json_response(StatusCode::OK, ())
|
||||
}
|
||||
|
||||
@@ -45,6 +45,7 @@ static ZERO_PAGE: bytes::Bytes = bytes::Bytes::from_static(&[0u8; 8192]);
|
||||
|
||||
pub use crate::metrics::preinitialize_metrics;
|
||||
|
||||
#[tracing::instrument]
|
||||
pub async fn shutdown_pageserver(exit_code: i32) {
|
||||
// Shut down the libpq endpoint task. This prevents new connections from
|
||||
// being accepted.
|
||||
|
||||
@@ -447,6 +447,11 @@ pub enum DeleteTimelineError {
|
||||
Other(#[from] anyhow::Error),
|
||||
}
|
||||
|
||||
pub enum SetStoppingError {
|
||||
AlreadyStopping,
|
||||
Broken,
|
||||
}
|
||||
|
||||
struct RemoteStartupData {
|
||||
index_part: IndexPart,
|
||||
remote_metadata: TimelineMetadata,
|
||||
@@ -645,16 +650,17 @@ impl Tenant {
|
||||
"attach tenant",
|
||||
false,
|
||||
async move {
|
||||
let doit = async {
|
||||
tenant_clone.attach(&ctx).await?;
|
||||
tenant_clone.activate(broker_client, &ctx)?;
|
||||
anyhow::Ok(())
|
||||
};
|
||||
match doit.await {
|
||||
Ok(_) => {}
|
||||
match tenant_clone.attach(&ctx).await {
|
||||
Ok(()) => {
|
||||
info!("attach finished, activating");
|
||||
tenant_clone.activate(broker_client, &ctx);
|
||||
}
|
||||
Err(e) => {
|
||||
tenant_clone.set_broken(e.to_string());
|
||||
error!("error attaching tenant: {:?}", e);
|
||||
error!("attach failed, setting tenant state to Broken: {:?}", e);
|
||||
tenant_clone.state.send_modify(|state| {
|
||||
assert_eq!(*state, TenantState::Attaching, "the attach task owns the tenant state until activation is complete");
|
||||
*state = TenantState::broken_from_reason(e.to_string());
|
||||
});
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
@@ -671,6 +677,8 @@ impl Tenant {
|
||||
///
|
||||
/// Background task that downloads all data for a tenant and brings it to Active state.
|
||||
///
|
||||
/// No background tasks are started as part of this routine.
|
||||
///
|
||||
async fn attach(self: &Arc<Tenant>, ctx: &RequestContext) -> anyhow::Result<()> {
|
||||
debug_assert_current_span_has_tenant_id();
|
||||
|
||||
@@ -920,20 +928,20 @@ impl Tenant {
|
||||
"initial tenant load",
|
||||
false,
|
||||
async move {
|
||||
let doit = async {
|
||||
tenant_clone.load(&ctx).await?;
|
||||
tenant_clone.activate(broker_client, &ctx)?;
|
||||
anyhow::Ok(())
|
||||
};
|
||||
match doit.await {
|
||||
Ok(()) => {}
|
||||
match tenant_clone.load(&ctx).await {
|
||||
Ok(()) => {
|
||||
info!("load finished, activating");
|
||||
tenant_clone.activate(broker_client, &ctx);
|
||||
}
|
||||
Err(err) => {
|
||||
tenant_clone.set_broken(err.to_string());
|
||||
error!("could not load tenant {tenant_id}: {err:?}");
|
||||
error!("load failed, setting tenant state to Broken: {err:?}");
|
||||
tenant_clone.state.send_modify(|state| {
|
||||
assert_eq!(*state, TenantState::Loading, "the loading task owns the tenant state until activation is complete");
|
||||
*state = TenantState::broken_from_reason(err.to_string());
|
||||
});
|
||||
}
|
||||
}
|
||||
info!("initial load for tenant {tenant_id} finished!");
|
||||
Ok(())
|
||||
Ok(())
|
||||
}
|
||||
.instrument({
|
||||
let span = tracing::info_span!(parent: None, "load", tenant_id=%tenant_id);
|
||||
@@ -951,6 +959,7 @@ impl Tenant {
|
||||
/// Background task to load in-memory data structures for this tenant, from
|
||||
/// files on disk. Used at pageserver startup.
|
||||
///
|
||||
/// No background tasks are started as part of this routine.
|
||||
async fn load(self: &Arc<Tenant>, ctx: &RequestContext) -> anyhow::Result<()> {
|
||||
debug_assert_current_span_has_tenant_id();
|
||||
|
||||
@@ -1657,130 +1666,191 @@ impl Tenant {
|
||||
}
|
||||
|
||||
/// Changes tenant status to active, unless shutdown was already requested.
|
||||
fn activate(
|
||||
self: &Arc<Self>,
|
||||
broker_client: BrokerClientChannel,
|
||||
ctx: &RequestContext,
|
||||
) -> anyhow::Result<()> {
|
||||
fn activate(self: &Arc<Self>, broker_client: BrokerClientChannel, ctx: &RequestContext) {
|
||||
debug_assert_current_span_has_tenant_id();
|
||||
|
||||
let mut result = Ok(());
|
||||
let mut activating = false;
|
||||
self.state.send_modify(|current_state| {
|
||||
use pageserver_api::models::ActivatingFrom;
|
||||
match &*current_state {
|
||||
TenantState::Active => {
|
||||
// activate() was called on an already Active tenant. Shouldn't happen.
|
||||
result = Err(anyhow::anyhow!("Tenant is already active"));
|
||||
TenantState::Activating(_) | TenantState::Active | TenantState::Broken { .. } | TenantState::Stopping => {
|
||||
panic!("caller is responsible for calling activate() only on Loading / Attaching tenants, got {state:?}", state = current_state);
|
||||
}
|
||||
TenantState::Broken { reason, .. } => {
|
||||
// This shouldn't happen either
|
||||
result = Err(anyhow::anyhow!(
|
||||
"Could not activate tenant because it is in broken state due to: {reason}",
|
||||
));
|
||||
TenantState::Loading => {
|
||||
*current_state = TenantState::Activating(ActivatingFrom::Loading);
|
||||
}
|
||||
TenantState::Stopping => {
|
||||
// The tenant was detached, or system shutdown was requested, while we were
|
||||
// loading or attaching the tenant.
|
||||
info!("Tenant is already in Stopping state, skipping activation");
|
||||
}
|
||||
TenantState::Loading | TenantState::Attaching => {
|
||||
*current_state = TenantState::Active;
|
||||
|
||||
debug!(tenant_id = %self.tenant_id, "Activating tenant");
|
||||
|
||||
let timelines_accessor = self.timelines.lock().unwrap();
|
||||
let not_broken_timelines = timelines_accessor
|
||||
.values()
|
||||
.filter(|timeline| timeline.current_state() != TimelineState::Broken);
|
||||
|
||||
// Spawn gc and compaction loops. The loops will shut themselves
|
||||
// down when they notice that the tenant is inactive.
|
||||
tasks::start_background_loops(self);
|
||||
|
||||
let mut activated_timelines = 0;
|
||||
|
||||
for timeline in not_broken_timelines {
|
||||
timeline.activate(broker_client.clone(), ctx);
|
||||
activated_timelines += 1;
|
||||
}
|
||||
|
||||
let elapsed = self.loading_started_at.elapsed();
|
||||
let total_timelines = timelines_accessor.len();
|
||||
|
||||
// log a lot of stuff, because some tenants sometimes suffer from user-visible
|
||||
// times to activate. see https://github.com/neondatabase/neon/issues/4025
|
||||
info!(
|
||||
since_creation_millis = elapsed.as_millis(),
|
||||
tenant_id = %self.tenant_id,
|
||||
activated_timelines,
|
||||
total_timelines,
|
||||
post_state = <&'static str>::from(&*current_state),
|
||||
"activation attempt finished"
|
||||
);
|
||||
TenantState::Attaching => {
|
||||
*current_state = TenantState::Activating(ActivatingFrom::Attaching);
|
||||
}
|
||||
}
|
||||
debug!(tenant_id = %self.tenant_id, "Activating tenant");
|
||||
activating = true;
|
||||
// Continue outside the closure. We need to grab timelines.lock()
|
||||
// and we plan to turn it into a tokio::sync::Mutex in a future patch.
|
||||
});
|
||||
result
|
||||
|
||||
if activating {
|
||||
let timelines_accessor = self.timelines.lock().unwrap();
|
||||
let not_broken_timelines = timelines_accessor
|
||||
.values()
|
||||
.filter(|timeline| timeline.current_state() != TimelineState::Broken);
|
||||
|
||||
// Spawn gc and compaction loops. The loops will shut themselves
|
||||
// down when they notice that the tenant is inactive.
|
||||
tasks::start_background_loops(self);
|
||||
|
||||
let mut activated_timelines = 0;
|
||||
|
||||
for timeline in not_broken_timelines {
|
||||
timeline.activate(broker_client.clone(), ctx);
|
||||
activated_timelines += 1;
|
||||
}
|
||||
|
||||
self.state.send_modify(move |current_state| {
|
||||
assert!(
|
||||
matches!(current_state, TenantState::Activating(_)),
|
||||
"set_stopping and set_broken wait for us to leave Activating state",
|
||||
);
|
||||
*current_state = TenantState::Active;
|
||||
|
||||
let elapsed = self.loading_started_at.elapsed();
|
||||
let total_timelines = timelines_accessor.len();
|
||||
|
||||
// log a lot of stuff, because some tenants sometimes suffer from user-visible
|
||||
// times to activate. see https://github.com/neondatabase/neon/issues/4025
|
||||
info!(
|
||||
since_creation_millis = elapsed.as_millis(),
|
||||
tenant_id = %self.tenant_id,
|
||||
activated_timelines,
|
||||
total_timelines,
|
||||
post_state = <&'static str>::from(&*current_state),
|
||||
"activation attempt finished"
|
||||
);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
/// Change tenant status to Stopping, to mark that it is being shut down
|
||||
pub fn set_stopping(&self) {
|
||||
self.state.send_modify(|current_state| {
|
||||
match current_state {
|
||||
TenantState::Active | TenantState::Loading | TenantState::Attaching => {
|
||||
*current_state = TenantState::Stopping;
|
||||
/// Change tenant status to Stopping, to mark that it is being shut down.
|
||||
///
|
||||
/// This function waits for the tenant to become active if it isn't already, before transitioning it into Stopping state.
|
||||
///
|
||||
/// This function is not cancel-safe!
|
||||
pub async fn set_stopping(&self) -> Result<(), SetStoppingError> {
|
||||
let mut rx = self.state.subscribe();
|
||||
|
||||
// FIXME: If the tenant is still Loading or Attaching, new timelines
|
||||
// might be created after this. That's harmless, as the Timelines
|
||||
// won't be accessible to anyone, when the Tenant is in Stopping
|
||||
// state.
|
||||
let timelines_accessor = self.timelines.lock().unwrap();
|
||||
let not_broken_timelines = timelines_accessor
|
||||
.values()
|
||||
.filter(|timeline| timeline.current_state() != TimelineState::Broken);
|
||||
for timeline in not_broken_timelines {
|
||||
timeline.set_state(TimelineState::Stopping);
|
||||
}
|
||||
}
|
||||
TenantState::Broken { reason, .. } => {
|
||||
info!("Cannot set tenant to Stopping state, it is in Broken state due to: {reason}");
|
||||
}
|
||||
TenantState::Stopping => {
|
||||
// The tenant was detached, or system shutdown was requested, while we were
|
||||
// loading or attaching the tenant.
|
||||
info!("Tenant is already in Stopping state");
|
||||
}
|
||||
// cannot stop before we're done activating, so wait out until we're done activating
|
||||
rx.wait_for(|state| match state {
|
||||
TenantState::Activating(_) | TenantState::Loading | TenantState::Attaching => {
|
||||
info!(
|
||||
"waiting for {} to turn Active|Broken|Stopping",
|
||||
<&'static str>::from(state)
|
||||
);
|
||||
false
|
||||
}
|
||||
TenantState::Active | TenantState::Broken { .. } | TenantState::Stopping {} => true,
|
||||
})
|
||||
.await
|
||||
.expect("cannot drop self.state while on a &self method");
|
||||
|
||||
// we now know we're done activating, let's see whether this task is the winner to transition into Stopping
|
||||
let mut err = None;
|
||||
let stopping = self.state.send_if_modified(|current_state| match current_state {
|
||||
TenantState::Activating(_) | TenantState::Loading | TenantState::Attaching => {
|
||||
unreachable!("we ensured above that we're done with activation, and, there is no re-activation")
|
||||
}
|
||||
TenantState::Active => {
|
||||
// FIXME: due to time-of-check vs time-of-use issues, it can happen that new timelines
|
||||
// are created after the transition to Stopping. That's harmless, as the Timelines
|
||||
// won't be accessible to anyone afterwards, because the Tenant is in Stopping state.
|
||||
*current_state = TenantState::Stopping;
|
||||
// Continue stopping outside the closure. We need to grab timelines.lock()
|
||||
// and we plan to turn it into a tokio::sync::Mutex in a future patch.
|
||||
true
|
||||
}
|
||||
TenantState::Broken { reason, .. } => {
|
||||
info!(
|
||||
"Cannot set tenant to Stopping state, it is in Broken state due to: {reason}"
|
||||
);
|
||||
err = Some(SetStoppingError::Broken);
|
||||
false
|
||||
}
|
||||
TenantState::Stopping => {
|
||||
info!("Tenant is already in Stopping state");
|
||||
err = Some(SetStoppingError::AlreadyStopping);
|
||||
false
|
||||
}
|
||||
});
|
||||
match (stopping, err) {
|
||||
(true, None) => {} // continue
|
||||
(false, Some(err)) => return Err(err),
|
||||
(true, Some(_)) => unreachable!(
|
||||
"send_if_modified closure must error out if not transitioning to Stopping"
|
||||
),
|
||||
(false, None) => unreachable!(
|
||||
"send_if_modified closure must return true if transitioning to Stopping"
|
||||
),
|
||||
}
|
||||
|
||||
let timelines_accessor = self.timelines.lock().unwrap();
|
||||
let not_broken_timelines = timelines_accessor
|
||||
.values()
|
||||
.filter(|timeline| timeline.current_state() != TimelineState::Broken);
|
||||
for timeline in not_broken_timelines {
|
||||
timeline.set_state(TimelineState::Stopping);
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn set_broken(&self, reason: String) {
|
||||
/// Method for tenant::mgr to transition us into Broken state in case of a late failure in
|
||||
/// `remove_tenant_from_memory`
|
||||
///
|
||||
/// This function waits for the tenant to become active if it isn't already, before transitioning it into Stopping state.
|
||||
///
|
||||
/// In tests, we also use this to set tenants to Broken state on purpose.
|
||||
pub(crate) async fn set_broken(&self, reason: String) {
|
||||
let mut rx = self.state.subscribe();
|
||||
|
||||
// The load & attach routines own the tenant state until it has reached `Active`.
|
||||
// So, wait until it's done.
|
||||
rx.wait_for(|state| match state {
|
||||
TenantState::Activating(_) | TenantState::Loading | TenantState::Attaching => {
|
||||
info!(
|
||||
"waiting for {} to turn Active|Broken|Stopping",
|
||||
<&'static str>::from(state)
|
||||
);
|
||||
false
|
||||
}
|
||||
TenantState::Active | TenantState::Broken { .. } | TenantState::Stopping {} => true,
|
||||
})
|
||||
.await
|
||||
.expect("cannot drop self.state while on a &self method");
|
||||
|
||||
// we now know we're done activating, let's see whether this task is the winner to transition into Broken
|
||||
self.state.send_modify(|current_state| {
|
||||
match *current_state {
|
||||
TenantState::Activating(_) | TenantState::Loading | TenantState::Attaching => {
|
||||
unreachable!("we ensured above that we're done with activation, and, there is no re-activation")
|
||||
}
|
||||
TenantState::Active => {
|
||||
// Broken tenants can currently only used for fatal errors that happen
|
||||
// while loading or attaching a tenant. A tenant that has already been
|
||||
// activated should never be marked as broken. We cope with it the best
|
||||
// we can, but it shouldn't happen.
|
||||
warn!("Changing Active tenant to Broken state, reason: {}", reason);
|
||||
*current_state = TenantState::broken_from_reason(reason);
|
||||
if cfg!(feature = "testing") {
|
||||
warn!("Changing Active tenant to Broken state, reason: {}", reason);
|
||||
*current_state = TenantState::broken_from_reason(reason);
|
||||
} else {
|
||||
unreachable!("not allowed to call set_broken on Active tenants in non-testing builds")
|
||||
}
|
||||
}
|
||||
TenantState::Broken { .. } => {
|
||||
// This shouldn't happen either
|
||||
warn!("Tenant is already in Broken state");
|
||||
}
|
||||
// This is the only "expected" path, any other path is a bug.
|
||||
TenantState::Stopping => {
|
||||
// This shouldn't happen either
|
||||
warn!(
|
||||
"Marking Stopping tenant as Broken state, reason: {}",
|
||||
reason
|
||||
);
|
||||
*current_state = TenantState::broken_from_reason(reason);
|
||||
}
|
||||
TenantState::Loading | TenantState::Attaching => {
|
||||
info!("Setting tenant as Broken state, reason: {}", reason);
|
||||
*current_state = TenantState::broken_from_reason(reason);
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@@ -1793,7 +1863,7 @@ impl Tenant {
|
||||
loop {
|
||||
let current_state = receiver.borrow_and_update().clone();
|
||||
match current_state {
|
||||
TenantState::Loading | TenantState::Attaching => {
|
||||
TenantState::Loading | TenantState::Attaching | TenantState::Activating(_) => {
|
||||
// in these states, there's a chance that we can reach ::Active
|
||||
receiver.changed().await.map_err(
|
||||
|_e: tokio::sync::watch::error::RecvError| {
|
||||
|
||||
@@ -10,6 +10,7 @@ use tokio::fs;
|
||||
use anyhow::Context;
|
||||
use once_cell::sync::Lazy;
|
||||
use tokio::sync::RwLock;
|
||||
use tokio::task::JoinSet;
|
||||
use tracing::*;
|
||||
|
||||
use remote_storage::GenericRemoteStorage;
|
||||
@@ -19,7 +20,9 @@ use crate::config::PageServerConf;
|
||||
use crate::context::{DownloadBehavior, RequestContext};
|
||||
use crate::task_mgr::{self, TaskKind};
|
||||
use crate::tenant::config::TenantConfOpt;
|
||||
use crate::tenant::{create_tenant_files, CreateTenantFilesMode, Tenant, TenantState};
|
||||
use crate::tenant::{
|
||||
create_tenant_files, CreateTenantFilesMode, SetStoppingError, Tenant, TenantState,
|
||||
};
|
||||
use crate::IGNORED_TENANT_FILE_NAME;
|
||||
|
||||
use utils::fs_ext::PathExt;
|
||||
@@ -222,6 +225,7 @@ pub fn schedule_local_tenant_processing(
|
||||
/// That could be easily misinterpreted by control plane, the consumer of the
|
||||
/// management API. For example, it could attach the tenant on a different pageserver.
|
||||
/// We would then be in split-brain once this pageserver restarts.
|
||||
#[instrument]
|
||||
pub async fn shutdown_all_tenants() {
|
||||
// Prevent new tenants from being created.
|
||||
let tenants_to_shut_down = {
|
||||
@@ -244,15 +248,65 @@ pub async fn shutdown_all_tenants() {
|
||||
}
|
||||
};
|
||||
|
||||
// Set tenant (and its timlines) to Stoppping state.
|
||||
//
|
||||
// Since we can only transition into Stopping state after activation is complete,
|
||||
// run it in a JoinSet so all tenants have a chance to stop before we get SIGKILLed.
|
||||
//
|
||||
// Transitioning tenants to Stopping state has a couple of non-obvious side effects:
|
||||
// 1. Lock out any new requests to the tenants.
|
||||
// 2. Signal cancellation to WAL receivers (we wait on it below).
|
||||
// 3. Signal cancellation for other tenant background loops.
|
||||
// 4. ???
|
||||
//
|
||||
// The waiting for the cancellation is not done uniformly.
|
||||
// We certainly wait for WAL receivers to shut down.
|
||||
// That is necessary so that no new data comes in before the freeze_and_flush.
|
||||
// But the tenant background loops are joined-on in our caller.
|
||||
// It's mesed up.
|
||||
let mut join_set = JoinSet::new();
|
||||
let mut tenants_to_freeze_and_flush = Vec::with_capacity(tenants_to_shut_down.len());
|
||||
for (_, tenant) in tenants_to_shut_down {
|
||||
if tenant.is_active() {
|
||||
// updates tenant state, forbidding new GC and compaction iterations from starting
|
||||
tenant.set_stopping();
|
||||
tenants_to_freeze_and_flush.push(tenant);
|
||||
for (tenant_id, tenant) in tenants_to_shut_down {
|
||||
join_set.spawn(
|
||||
async move {
|
||||
match tenant.set_stopping().await {
|
||||
Ok(()) => debug!("tenant successfully stopped"),
|
||||
Err(SetStoppingError::Broken) => {
|
||||
info!("tenant is broken, so stopping failed, freeze_and_flush is likely going to make noise as well");
|
||||
},
|
||||
Err(SetStoppingError::AlreadyStopping) => {
|
||||
// our task_mgr::shutdown_tasks are going to coalesce on that just fine
|
||||
}
|
||||
}
|
||||
|
||||
tenant
|
||||
}
|
||||
.instrument(info_span!("set_stopping", %tenant_id)),
|
||||
);
|
||||
}
|
||||
|
||||
let mut panicked = 0;
|
||||
|
||||
while let Some(res) = join_set.join_next().await {
|
||||
match res {
|
||||
Err(join_error) if join_error.is_cancelled() => {
|
||||
unreachable!("we are not cancelling any of the futures");
|
||||
}
|
||||
Err(join_error) if join_error.is_panic() => {
|
||||
// cannot really do anything, as this panic is likely a bug
|
||||
panicked += 1;
|
||||
}
|
||||
Err(join_error) => {
|
||||
warn!("unknown kind of JoinError: {join_error}");
|
||||
}
|
||||
Ok(tenant) => tenants_to_freeze_and_flush.push(tenant),
|
||||
}
|
||||
}
|
||||
|
||||
if panicked > 0 {
|
||||
warn!(panicked, "observed panicks while stopping tenants");
|
||||
}
|
||||
|
||||
// Shut down all existing walreceiver connections and stop accepting the new ones.
|
||||
task_mgr::shutdown_tasks(Some(TaskKind::WalReceiverManager), None, None).await;
|
||||
|
||||
@@ -264,12 +318,30 @@ pub async fn shutdown_all_tenants() {
|
||||
// should be no more activity in any of the repositories.
|
||||
//
|
||||
// On error, log it but continue with the shutdown for other tenants.
|
||||
|
||||
let mut join_set = tokio::task::JoinSet::new();
|
||||
|
||||
for tenant in tenants_to_freeze_and_flush {
|
||||
let tenant_id = tenant.tenant_id();
|
||||
debug!("shutdown tenant {tenant_id}");
|
||||
|
||||
if let Err(err) = tenant.freeze_and_flush().await {
|
||||
error!("Could not checkpoint tenant {tenant_id} during shutdown: {err:?}");
|
||||
join_set.spawn(
|
||||
async move {
|
||||
if let Err(err) = tenant.freeze_and_flush().await {
|
||||
warn!("Could not checkpoint tenant during shutdown: {err:?}");
|
||||
}
|
||||
}
|
||||
.instrument(info_span!("freeze_and_flush", %tenant_id)),
|
||||
);
|
||||
}
|
||||
|
||||
while let Some(next) = join_set.join_next().await {
|
||||
match next {
|
||||
Ok(()) => {}
|
||||
Err(join_error) if join_error.is_cancelled() => {
|
||||
unreachable!("no cancelling")
|
||||
}
|
||||
Err(join_error) if join_error.is_panic() => { /* reported already */ }
|
||||
Err(join_error) => warn!("unknown kind of JoinError: {join_error}"),
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -589,13 +661,23 @@ where
|
||||
{
|
||||
let tenants_accessor = TENANTS.write().await;
|
||||
match tenants_accessor.get(&tenant_id) {
|
||||
Some(tenant) => match tenant.current_state() {
|
||||
TenantState::Attaching
|
||||
| TenantState::Loading
|
||||
| TenantState::Broken { .. }
|
||||
| TenantState::Active => tenant.set_stopping(),
|
||||
TenantState::Stopping => return Err(TenantStateError::IsStopping(tenant_id)),
|
||||
},
|
||||
Some(tenant) => {
|
||||
let tenant = Arc::clone(tenant);
|
||||
// don't hold TENANTS lock while set_stopping waits for activation to finish
|
||||
drop(tenants_accessor);
|
||||
match tenant.set_stopping().await {
|
||||
Ok(()) => {
|
||||
// we won, continue stopping procedure
|
||||
}
|
||||
Err(SetStoppingError::Broken) => {
|
||||
// continue the procedure, let's hope the closure can deal with broken tenants
|
||||
}
|
||||
Err(SetStoppingError::AlreadyStopping) => {
|
||||
// the tenant is already stopping or broken, don't do anything
|
||||
return Err(TenantStateError::IsStopping(tenant_id));
|
||||
}
|
||||
}
|
||||
}
|
||||
None => return Err(TenantStateError::NotFound(tenant_id)),
|
||||
}
|
||||
}
|
||||
@@ -620,7 +702,7 @@ where
|
||||
let tenants_accessor = TENANTS.read().await;
|
||||
match tenants_accessor.get(&tenant_id) {
|
||||
Some(tenant) => {
|
||||
tenant.set_broken(e.to_string());
|
||||
tenant.set_broken(e.to_string()).await;
|
||||
}
|
||||
None => {
|
||||
warn!("Tenant {tenant_id} got removed from memory");
|
||||
|
||||
@@ -20,7 +20,7 @@ def test_broken_timeline(neon_env_builder: NeonEnvBuilder):
|
||||
".*is not active. Current state: Broken.*",
|
||||
".*will not become active. Current state: Broken.*",
|
||||
".*failed to load metadata.*",
|
||||
".*could not load tenant.*load local timeline.*",
|
||||
".*load failed.*load local timeline.*",
|
||||
]
|
||||
)
|
||||
|
||||
|
||||
@@ -140,7 +140,7 @@ def test_remote_storage_backup_and_restore(
|
||||
# This is before the failures injected by test_remote_failures, so it's a permanent error.
|
||||
pageserver_http.configure_failpoints(("storage-sync-list-remote-timelines", "return"))
|
||||
env.pageserver.allowed_errors.append(
|
||||
".*error attaching tenant: storage-sync-list-remote-timelines",
|
||||
".*attach failed.*: storage-sync-list-remote-timelines",
|
||||
)
|
||||
# Attach it. This HTTP request will succeed and launch a
|
||||
# background task to load the tenant. In that background task,
|
||||
|
||||
@@ -647,7 +647,9 @@ def test_ignored_tenant_stays_broken_without_metadata(
|
||||
metadata_removed = True
|
||||
assert metadata_removed, f"Failed to find metadata file in {tenant_timeline_dir}"
|
||||
|
||||
env.pageserver.allowed_errors.append(".*could not load tenant .*?: failed to load metadata.*")
|
||||
env.pageserver.allowed_errors.append(
|
||||
f".*{tenant_id}.*: load failed.*: failed to load metadata.*"
|
||||
)
|
||||
|
||||
# now, load it from the local files and expect it to be broken due to inability to load tenant files into memory
|
||||
pageserver_http.tenant_load(tenant_id=tenant_id)
|
||||
|
||||
@@ -22,6 +22,7 @@ from fixtures.neon_fixtures import (
|
||||
available_remote_storages,
|
||||
)
|
||||
from fixtures.types import Lsn, TenantId, TimelineId
|
||||
from fixtures.utils import wait_until
|
||||
from prometheus_client.samples import Sample
|
||||
|
||||
|
||||
@@ -308,9 +309,7 @@ def test_pageserver_with_empty_tenants(
|
||||
env.pageserver.allowed_errors.append(
|
||||
".*marking .* as locally complete, while it doesnt exist in remote index.*"
|
||||
)
|
||||
env.pageserver.allowed_errors.append(
|
||||
".*could not load tenant.*Failed to list timelines directory.*"
|
||||
)
|
||||
env.pageserver.allowed_errors.append(".*load failed.*Failed to list timelines directory.*")
|
||||
|
||||
client = env.pageserver.http_client()
|
||||
|
||||
@@ -341,9 +340,15 @@ def test_pageserver_with_empty_tenants(
|
||||
env.pageserver.start()
|
||||
|
||||
client = env.pageserver.http_client()
|
||||
tenants = client.tenant_list()
|
||||
|
||||
assert len(tenants) == 2
|
||||
def not_loading():
|
||||
tenants = client.tenant_list()
|
||||
assert len(tenants) == 2
|
||||
assert all(t["state"]["slug"] != "Loading" for t in tenants)
|
||||
|
||||
wait_until(10, 0.2, not_loading)
|
||||
|
||||
tenants = client.tenant_list()
|
||||
|
||||
[broken_tenant] = [t for t in tenants if t["id"] == str(tenant_without_timelines_dir)]
|
||||
assert (
|
||||
@@ -355,7 +360,7 @@ def test_pageserver_with_empty_tenants(
|
||||
broken_tenant_status["state"]["slug"] == "Broken"
|
||||
), f"Tenant {tenant_without_timelines_dir} without timelines dir should be broken"
|
||||
|
||||
assert env.pageserver.log_contains(".*Setting tenant as Broken state, reason:.*")
|
||||
assert env.pageserver.log_contains(".*load failed, setting tenant state to Broken:.*")
|
||||
|
||||
[loaded_tenant] = [t for t in tenants if t["id"] == str(tenant_with_empty_timelines)]
|
||||
assert (
|
||||
|
||||
Reference in New Issue
Block a user