pageserver: remove bare mgr::get_tenant, mgr::list_tenants (#7237)

## Problem

This is a refactor.

This PR was a precursor to a much smaller change
e5bd602dc1,
where as I was writing it I found that we were not far from getting rid
of the last non-deprecated code paths that use `mgr::` scoped functions
to get at the TenantManager state.

We're almost done cleaning this up as per
https://github.com/neondatabase/neon/issues/5796. The only significant
remaining mgr:: item is `get_active_tenant_with_timeout`, which is
page_service's path for fetching tenants.

## Summary of changes

- Remove the bool argument to get_attached_tenant_shard: this was almost
always false from API use cases, and in cases when it was true, it was
readily replacable with an explicit check of the returned tenant's
status.
- Rather than letting the timeline eviction task query any tenant it
likes via `mgr::`, pass an `Arc<Tenant>` into the task. This is still an
ugly circular reference, but should eventually go away: either when we
switch to exclusively using disk usage eviction, or when we change
metadata storage to avoid the need to imitate layer accesses.
- Convert all the mgr::get_tenant call sites to use
TenantManager::get_attached_tenant_shard
- Move list_tenants into TenantManager.
This commit is contained in:
John Spray
2024-03-26 18:29:08 +00:00
committed by GitHub
parent 6814bb4b59
commit b3b7ce457c
13 changed files with 221 additions and 243 deletions

View File

@@ -600,33 +600,37 @@ fn start_pageserver(
None,
"consumption metrics collection",
true,
async move {
// first wait until background jobs are cleared to launch.
//
// this is because we only process active tenants and timelines, and the
// Timeline::get_current_logical_size will spawn the logical size calculation,
// which will not be rate-limited.
let cancel = task_mgr::shutdown_token();
{
let tenant_manager = tenant_manager.clone();
async move {
// first wait until background jobs are cleared to launch.
//
// this is because we only process active tenants and timelines, and the
// Timeline::get_current_logical_size will spawn the logical size calculation,
// which will not be rate-limited.
let cancel = task_mgr::shutdown_token();
tokio::select! {
_ = cancel.cancelled() => { return Ok(()); },
_ = background_jobs_barrier.wait() => {}
};
tokio::select! {
_ = cancel.cancelled() => { return Ok(()); },
_ = background_jobs_barrier.wait() => {}
};
pageserver::consumption_metrics::collect_metrics(
metric_collection_endpoint,
&conf.metric_collection_bucket,
conf.metric_collection_interval,
conf.cached_metric_collection_interval,
conf.synthetic_size_calculation_interval,
conf.id,
local_disk_storage,
cancel,
metrics_ctx,
)
.instrument(info_span!("metrics_collection"))
.await?;
Ok(())
pageserver::consumption_metrics::collect_metrics(
tenant_manager,
metric_collection_endpoint,
&conf.metric_collection_bucket,
conf.metric_collection_interval,
conf.cached_metric_collection_interval,
conf.synthetic_size_calculation_interval,
conf.id,
local_disk_storage,
cancel,
metrics_ctx,
)
.instrument(info_span!("metrics_collection"))
.await?;
Ok(())
}
},
);
}

View File

@@ -3,7 +3,9 @@
use crate::context::{DownloadBehavior, RequestContext};
use crate::task_mgr::{self, TaskKind, BACKGROUND_RUNTIME};
use crate::tenant::tasks::BackgroundLoopKind;
use crate::tenant::{mgr, LogicalSizeCalculationCause, PageReconstructError, Tenant};
use crate::tenant::{
mgr::TenantManager, LogicalSizeCalculationCause, PageReconstructError, Tenant,
};
use camino::Utf8PathBuf;
use consumption_metrics::EventType;
use pageserver_api::models::TenantState;
@@ -41,6 +43,7 @@ type Cache = HashMap<MetricsKey, (EventType, u64)>;
/// Main thread that serves metrics collection
#[allow(clippy::too_many_arguments)]
pub async fn collect_metrics(
tenant_manager: Arc<TenantManager>,
metric_collection_endpoint: &Url,
metric_collection_bucket: &Option<RemoteStorageConfig>,
metric_collection_interval: Duration,
@@ -67,15 +70,19 @@ pub async fn collect_metrics(
None,
"synthetic size calculation",
false,
async move {
calculate_synthetic_size_worker(
synthetic_size_calculation_interval,
&cancel,
&worker_ctx,
)
.instrument(info_span!("synthetic_size_worker"))
.await?;
Ok(())
{
let tenant_manager = tenant_manager.clone();
async move {
calculate_synthetic_size_worker(
tenant_manager,
synthetic_size_calculation_interval,
&cancel,
&worker_ctx,
)
.instrument(info_span!("synthetic_size_worker"))
.await?;
Ok(())
}
},
);
@@ -116,7 +123,7 @@ pub async fn collect_metrics(
let started_at = Instant::now();
// these are point in time, with variable "now"
let metrics = metrics::collect_all_metrics(&cached_metrics, &ctx).await;
let metrics = metrics::collect_all_metrics(&tenant_manager, &cached_metrics, &ctx).await;
let metrics = Arc::new(metrics);
@@ -271,6 +278,7 @@ async fn reschedule(
/// Caclculate synthetic size for each active tenant
async fn calculate_synthetic_size_worker(
tenant_manager: Arc<TenantManager>,
synthetic_size_calculation_interval: Duration,
cancel: &CancellationToken,
ctx: &RequestContext,
@@ -283,7 +291,7 @@ async fn calculate_synthetic_size_worker(
loop {
let started_at = Instant::now();
let tenants = match mgr::list_tenants().await {
let tenants = match tenant_manager.list_tenants() {
Ok(tenants) => tenants,
Err(e) => {
warn!("cannot get tenant list: {e:#}");
@@ -302,10 +310,14 @@ async fn calculate_synthetic_size_worker(
continue;
}
let Ok(tenant) = mgr::get_tenant(tenant_shard_id, true) else {
let Ok(tenant) = tenant_manager.get_attached_tenant_shard(tenant_shard_id) else {
continue;
};
if !tenant.is_active() {
continue;
}
// there is never any reason to exit calculate_synthetic_size_worker following any
// return value -- we don't need to care about shutdown because no tenant is found when
// pageserver is shut down.
@@ -343,9 +355,7 @@ async fn calculate_and_log(tenant: &Tenant, cancel: &CancellationToken, ctx: &Re
};
// this error can be returned if timeline is shutting down, but it does not
// mean the synthetic size worker should terminate. we do not need any checks
// in this function because `mgr::get_tenant` will error out after shutdown has
// progressed to shutting down tenants.
// mean the synthetic size worker should terminate.
let shutting_down = matches!(
e.downcast_ref::<PageReconstructError>(),
Some(PageReconstructError::Cancelled | PageReconstructError::AncestorStopping(_))

View File

@@ -1,3 +1,4 @@
use crate::tenant::mgr::TenantManager;
use crate::{context::RequestContext, tenant::timeline::logical_size::CurrentLogicalSize};
use chrono::{DateTime, Utc};
use consumption_metrics::EventType;
@@ -181,6 +182,7 @@ impl MetricsKey {
}
pub(super) async fn collect_all_metrics(
tenant_manager: &Arc<TenantManager>,
cached_metrics: &Cache,
ctx: &RequestContext,
) -> Vec<RawMetric> {
@@ -188,7 +190,7 @@ pub(super) async fn collect_all_metrics(
let started_at = std::time::Instant::now();
let tenants = match crate::tenant::mgr::list_tenants().await {
let tenants = match tenant_manager.list_tenants() {
Ok(tenants) => tenants,
Err(err) => {
tracing::error!("failed to list tenants: {:?}", err);
@@ -200,7 +202,8 @@ pub(super) async fn collect_all_metrics(
if state != TenantState::Active || !id.is_zero() {
None
} else {
crate::tenant::mgr::get_tenant(id, true)
tenant_manager
.get_attached_tenant_shard(id)
.ok()
.map(|tenant| (id.tenant_id, tenant))
}

View File

@@ -61,7 +61,6 @@ use crate::{
metrics::disk_usage_based_eviction::METRICS,
task_mgr::{self, TaskKind, BACKGROUND_RUNTIME},
tenant::{
self,
mgr::TenantManager,
remote_timeline_client::LayerFileMetadata,
secondary::SecondaryTenant,
@@ -814,8 +813,8 @@ async fn collect_eviction_candidates(
const LOG_DURATION_THRESHOLD: std::time::Duration = std::time::Duration::from_secs(10);
// get a snapshot of the list of tenants
let tenants = tenant::mgr::list_tenants()
.await
let tenants = tenant_manager
.list_tenants()
.context("get list of tenants")?;
// TODO: avoid listing every layer in every tenant: this loop can block the executor,
@@ -827,8 +826,12 @@ async fn collect_eviction_candidates(
if cancel.is_cancelled() {
return Ok(EvictionCandidates::Cancelled);
}
let tenant = match tenant::mgr::get_tenant(tenant_id, true) {
Ok(tenant) => tenant,
let tenant = match tenant_manager.get_attached_tenant_shard(tenant_id) {
Ok(tenant) if tenant.is_active() => tenant,
Ok(_) => {
debug!(tenant_id=%tenant_id.tenant_id, shard_id=%tenant_id.shard_slug(), "Tenant shard is not active");
continue;
}
Err(e) => {
// this can happen if tenant has lifecycle transition after we fetched it
debug!("failed to get tenant: {e:#}");

View File

@@ -49,8 +49,8 @@ use crate::task_mgr::TaskKind;
use crate::tenant::config::{LocationConf, TenantConfOpt};
use crate::tenant::mgr::GetActiveTenantError;
use crate::tenant::mgr::{
GetTenantError, SetNewTenantConfigError, TenantManager, TenantMapError, TenantMapInsertError,
TenantSlotError, TenantSlotUpsertError, TenantStateError,
GetTenantError, TenantManager, TenantMapError, TenantMapInsertError, TenantSlotError,
TenantSlotUpsertError, TenantStateError,
};
use crate::tenant::mgr::{TenantSlot, UpsertLocationError};
use crate::tenant::remote_timeline_client;
@@ -249,16 +249,11 @@ impl From<GetTenantError> for ApiError {
fn from(tse: GetTenantError) -> ApiError {
match tse {
GetTenantError::NotFound(tid) => ApiError::NotFound(anyhow!("tenant {}", tid).into()),
GetTenantError::Broken(reason) => {
ApiError::InternalServerError(anyhow!("tenant is broken: {}", reason))
}
GetTenantError::NotActive(_) => {
// Why is this not `ApiError::NotFound`?
// Because we must be careful to never return 404 for a tenant if it does
// in fact exist locally. If we did, the caller could draw the conclusion
// that it can attach the tenant to another PS and we'd be in split-brain.
//
// (We can produce this variant only in `mgr::get_tenant(..., active=true)` calls).
ApiError::ResourceUnavailable("Tenant not yet active".into())
}
GetTenantError::MapState(e) => ApiError::ResourceUnavailable(format!("{e}").into()),
@@ -269,6 +264,9 @@ impl From<GetTenantError> for ApiError {
impl From<GetActiveTenantError> for ApiError {
fn from(e: GetActiveTenantError) -> ApiError {
match e {
GetActiveTenantError::Broken(reason) => {
ApiError::InternalServerError(anyhow!("tenant is broken: {}", reason))
}
GetActiveTenantError::WillNotBecomeActive(_) => ApiError::Conflict(format!("{}", e)),
GetActiveTenantError::Cancelled => ApiError::ShuttingDown,
GetActiveTenantError::NotFound(gte) => gte.into(),
@@ -279,19 +277,6 @@ impl From<GetActiveTenantError> for ApiError {
}
}
impl From<SetNewTenantConfigError> for ApiError {
fn from(e: SetNewTenantConfigError) -> ApiError {
match e {
SetNewTenantConfigError::GetTenant(tid) => {
ApiError::NotFound(anyhow!("tenant {}", tid).into())
}
e @ (SetNewTenantConfigError::Persist(_) | SetNewTenantConfigError::Other(_)) => {
ApiError::InternalServerError(anyhow::Error::new(e))
}
}
}
}
impl From<crate::tenant::DeleteTimelineError> for ApiError {
fn from(value: crate::tenant::DeleteTimelineError) -> Self {
use crate::tenant::DeleteTimelineError::*;
@@ -495,7 +480,7 @@ async fn timeline_create_handler(
async {
let tenant = state
.tenant_manager
.get_attached_tenant_shard(tenant_shard_id, false)?;
.get_attached_tenant_shard(tenant_shard_id)?;
tenant.wait_to_become_active(ACTIVE_TENANT_TIMEOUT).await?;
@@ -584,7 +569,7 @@ async fn timeline_list_handler(
let response_data = async {
let tenant = state
.tenant_manager
.get_attached_tenant_shard(tenant_shard_id, false)?;
.get_attached_tenant_shard(tenant_shard_id)?;
tenant.wait_to_become_active(ACTIVE_TENANT_TIMEOUT).await?;
@@ -622,6 +607,7 @@ async fn timeline_preserve_initdb_handler(
let tenant_shard_id: TenantShardId = parse_request_param(&request, "tenant_shard_id")?;
let timeline_id: TimelineId = parse_request_param(&request, "timeline_id")?;
check_permission(&request, Some(tenant_shard_id.tenant_id))?;
let state = get_state(&request);
// Part of the process for disaster recovery from safekeeper-stored WAL:
// If we don't recover into a new timeline but want to keep the timeline ID,
@@ -629,7 +615,9 @@ async fn timeline_preserve_initdb_handler(
// location where timeline recreation cand find it.
async {
let tenant = mgr::get_tenant(tenant_shard_id, false)?;
let tenant = state
.tenant_manager
.get_attached_tenant_shard(tenant_shard_id)?;
let timeline = tenant
.get_timeline(timeline_id, false)
@@ -671,7 +659,7 @@ async fn timeline_detail_handler(
let timeline_info = async {
let tenant = state
.tenant_manager
.get_attached_tenant_shard(tenant_shard_id, false)?;
.get_attached_tenant_shard(tenant_shard_id)?;
tenant.wait_to_become_active(ACTIVE_TENANT_TIMEOUT).await?;
@@ -858,7 +846,7 @@ async fn timeline_delete_handler(
let tenant = state
.tenant_manager
.get_attached_tenant_shard(tenant_shard_id, false)
.get_attached_tenant_shard(tenant_shard_id)
.map_err(|e| {
match e {
// GetTenantError has a built-in conversion to ApiError, but in this context we don't
@@ -976,10 +964,11 @@ async fn tenant_list_handler(
_cancel: CancellationToken,
) -> Result<Response<Body>, ApiError> {
check_permission(&request, None)?;
let state = get_state(&request);
let response_data = mgr::list_tenants()
.instrument(info_span!("tenant_list"))
.await
let response_data = state
.tenant_manager
.list_tenants()
.map_err(|_| {
ApiError::ResourceUnavailable("Tenant map is initializing or shutting down".into())
})?
@@ -1002,9 +991,12 @@ async fn tenant_status(
) -> Result<Response<Body>, ApiError> {
let tenant_shard_id: TenantShardId = parse_request_param(&request, "tenant_shard_id")?;
check_permission(&request, Some(tenant_shard_id.tenant_id))?;
let state = get_state(&request);
let tenant_info = async {
let tenant = mgr::get_tenant(tenant_shard_id, false)?;
let tenant = state
.tenant_manager
.get_attached_tenant_shard(tenant_shard_id)?;
// Calculate total physical size of all timelines
let mut current_physical_size = 0;
@@ -1077,9 +1069,7 @@ async fn tenant_size_handler(
let inputs_only: Option<bool> = parse_query_param(&request, "inputs_only")?;
let retention_period: Option<u64> = parse_query_param(&request, "retention_period")?;
let headers = request.headers();
let ctx = RequestContext::new(TaskKind::MgmtRequest, DownloadBehavior::Download);
let tenant = mgr::get_tenant(tenant_shard_id, true)?;
let state = get_state(&request);
if !tenant_shard_id.is_zero() {
return Err(ApiError::BadRequest(anyhow!(
@@ -1087,6 +1077,12 @@ async fn tenant_size_handler(
)));
}
let ctx = RequestContext::new(TaskKind::MgmtRequest, DownloadBehavior::Download);
let tenant = state
.tenant_manager
.get_attached_tenant_shard(tenant_shard_id)?;
tenant.wait_to_become_active(ACTIVE_TENANT_TIMEOUT).await?;
// this can be long operation
let inputs = tenant
.gather_size_inputs(
@@ -1155,10 +1151,15 @@ async fn tenant_shard_split_handler(
let state = get_state(&request);
let ctx = RequestContext::new(TaskKind::MgmtRequest, DownloadBehavior::Warn);
let tenant = state
.tenant_manager
.get_attached_tenant_shard(tenant_shard_id)?;
tenant.wait_to_become_active(ACTIVE_TENANT_TIMEOUT).await?;
let new_shards = state
.tenant_manager
.shard_split(
tenant_shard_id,
tenant,
ShardCount::new(req.new_shard_count),
req.new_stripe_size,
&ctx,
@@ -1376,8 +1377,11 @@ async fn get_tenant_config_handler(
) -> Result<Response<Body>, ApiError> {
let tenant_shard_id: TenantShardId = parse_request_param(&request, "tenant_shard_id")?;
check_permission(&request, Some(tenant_shard_id.tenant_id))?;
let state = get_state(&request);
let tenant = mgr::get_tenant(tenant_shard_id, false)?;
let tenant = state
.tenant_manager
.get_attached_tenant_shard(tenant_shard_id)?;
let response = HashMap::from([
(
@@ -1405,15 +1409,31 @@ async fn update_tenant_config_handler(
let tenant_id = request_data.tenant_id;
check_permission(&request, Some(tenant_id))?;
let tenant_conf =
let new_tenant_conf =
TenantConfOpt::try_from(&request_data.config).map_err(ApiError::BadRequest)?;
let state = get_state(&request);
state
let tenant_shard_id = TenantShardId::unsharded(tenant_id);
let tenant = state
.tenant_manager
.set_new_tenant_config(tenant_conf, tenant_id)
.instrument(info_span!("tenant_config", %tenant_id))
.await?;
.get_attached_tenant_shard(tenant_shard_id)?;
tenant.wait_to_become_active(ACTIVE_TENANT_TIMEOUT).await?;
// This is a legacy API that only operates on attached tenants: the preferred
// API to use is the location_config/ endpoint, which lets the caller provide
// the full LocationConf.
let location_conf = LocationConf::attached_single(
new_tenant_conf.clone(),
tenant.get_generation(),
&ShardParameters::default(),
);
crate::tenant::Tenant::persist_tenant_config(state.conf, &tenant_shard_id, &location_conf)
.await
.map_err(ApiError::InternalServerError)?;
tenant.set_new_tenant_config(new_tenant_conf);
json_response(StatusCode::OK, ())
}
@@ -1637,10 +1657,12 @@ async fn handle_tenant_break(
) -> Result<Response<Body>, ApiError> {
let tenant_shard_id: TenantShardId = parse_request_param(&r, "tenant_shard_id")?;
let tenant = crate::tenant::mgr::get_tenant(tenant_shard_id, true)
.map_err(|_| ApiError::Conflict(String::from("no active tenant found")))?;
tenant.set_broken("broken from test".to_owned()).await;
let state = get_state(&r);
state
.tenant_manager
.get_attached_tenant_shard(tenant_shard_id)?
.set_broken("broken from test".to_owned())
.await;
json_response(StatusCode::OK, ())
}
@@ -1884,7 +1906,7 @@ async fn active_timeline_of_active_tenant(
tenant_shard_id: TenantShardId,
timeline_id: TimelineId,
) -> Result<Arc<Timeline>, ApiError> {
let tenant = tenant_manager.get_attached_tenant_shard(tenant_shard_id, false)?;
let tenant = tenant_manager.get_attached_tenant_shard(tenant_shard_id)?;
tenant.wait_to_become_active(ACTIVE_TENANT_TIMEOUT).await?;

View File

@@ -760,6 +760,7 @@ impl PageServerHandler {
let mut copyin_reader = pin!(StreamReader::new(self.copyin_stream(pgb, &tenant.cancel)));
timeline
.import_basebackup_from_tar(
tenant.clone(),
&mut copyin_reader,
base_lsn,
self.broker_client.clone(),

View File

@@ -1411,7 +1411,7 @@ impl Tenant {
/// the same timeline ID already exists, returns CreateTimelineError::AlreadyExists.
#[allow(clippy::too_many_arguments)]
pub(crate) async fn create_timeline(
&self,
self: &Arc<Tenant>,
new_timeline_id: TimelineId,
ancestor_timeline_id: Option<TimelineId>,
mut ancestor_start_lsn: Option<Lsn>,
@@ -1559,7 +1559,7 @@ impl Tenant {
})?;
}
loaded_timeline.activate(broker_client, None, ctx);
loaded_timeline.activate(self.clone(), broker_client, None, ctx);
Ok(loaded_timeline)
}
@@ -1731,7 +1731,12 @@ impl Tenant {
let mut activated_timelines = 0;
for timeline in timelines_to_activate {
timeline.activate(broker_client.clone(), background_jobs_can_start, ctx);
timeline.activate(
self.clone(),
broker_client.clone(),
background_jobs_can_start,
ctx,
);
activated_timelines += 1;
}
@@ -2063,7 +2068,12 @@ impl Tenant {
TenantState::Active { .. } => {
return Ok(());
}
TenantState::Broken { .. } | TenantState::Stopping { .. } => {
TenantState::Broken { reason, .. } => {
// This is fatal, and reported distinctly from the general case of "will never be active" because
// it's logically a 500 to external API users (broken is always a bug).
return Err(GetActiveTenantError::Broken(reason));
}
TenantState::Stopping { .. } => {
// There's no chance the tenant can transition back into ::Active
return Err(GetActiveTenantError::WillNotBecomeActive(current_state));
}

View File

@@ -4,7 +4,7 @@
use camino::{Utf8DirEntry, Utf8Path, Utf8PathBuf};
use itertools::Itertools;
use pageserver_api::key::Key;
use pageserver_api::models::{LocationConfigMode, ShardParameters};
use pageserver_api::models::LocationConfigMode;
use pageserver_api::shard::{
ShardCount, ShardIdentity, ShardNumber, ShardStripeSize, TenantShardId,
};
@@ -40,7 +40,6 @@ use crate::metrics::{TENANT, TENANT_MANAGER as METRICS};
use crate::task_mgr::{self, TaskKind};
use crate::tenant::config::{
AttachedLocationConfig, AttachmentMode, LocationConf, LocationMode, SecondaryLocationConfig,
TenantConfOpt,
};
use crate::tenant::delete::DeleteTenantFlow;
use crate::tenant::span::debug_assert_current_span_has_tenant_id;
@@ -889,16 +888,6 @@ async fn shutdown_all_tenants0(tenants: &std::sync::RwLock<TenantsMap>) {
// caller will log how long we took
}
#[derive(Debug, thiserror::Error)]
pub(crate) enum SetNewTenantConfigError {
#[error(transparent)]
GetTenant(#[from] GetTenantError),
#[error(transparent)]
Persist(anyhow::Error),
#[error(transparent)]
Other(anyhow::Error),
}
#[derive(thiserror::Error, Debug)]
pub(crate) enum UpsertLocationError {
#[error("Bad config request: {0}")]
@@ -924,32 +913,21 @@ impl TenantManager {
self.conf
}
/// Gets the attached tenant from the in-memory data, erroring if it's absent, in secondary mode, or is not fitting to the query.
/// `active_only = true` allows to query only tenants that are ready for operations, erroring on other kinds of tenants.
/// Gets the attached tenant from the in-memory data, erroring if it's absent, in secondary mode, or currently
/// undergoing a state change (i.e. slot is InProgress).
///
/// The return Tenant is not guaranteed to be active: check its status after obtaing it, or
/// use [`Tenant::wait_to_become_active`] before using it if you will do I/O on it.
pub(crate) fn get_attached_tenant_shard(
&self,
tenant_shard_id: TenantShardId,
active_only: bool,
) -> Result<Arc<Tenant>, GetTenantError> {
let locked = self.tenants.read().unwrap();
let peek_slot = tenant_map_peek_slot(&locked, &tenant_shard_id, TenantSlotPeekMode::Read)?;
match peek_slot {
Some(TenantSlot::Attached(tenant)) => match tenant.current_state() {
TenantState::Broken {
reason,
backtrace: _,
} if active_only => Err(GetTenantError::Broken(reason)),
TenantState::Active => Ok(Arc::clone(tenant)),
_ => {
if active_only {
Err(GetTenantError::NotActive(tenant_shard_id))
} else {
Ok(Arc::clone(tenant))
}
}
},
Some(TenantSlot::Attached(tenant)) => Ok(Arc::clone(tenant)),
Some(TenantSlot::InProgress(_)) => Err(GetTenantError::NotActive(tenant_shard_id)),
None | Some(TenantSlot::Secondary(_)) => {
Err(GetTenantError::NotFound(tenant_shard_id.tenant_id))
@@ -1442,7 +1420,8 @@ impl TenantManager {
.wait_to_become_active(activation_timeout)
.await
.map_err(|e| match e {
GetActiveTenantError::WillNotBecomeActive(_) => {
GetActiveTenantError::WillNotBecomeActive(_)
| GetActiveTenantError::Broken(_) => {
DeleteTenantError::InvalidState(tenant.current_state())
}
GetActiveTenantError::Cancelled => DeleteTenantError::Cancelled,
@@ -1469,29 +1448,30 @@ impl TenantManager {
result
}
#[instrument(skip_all, fields(tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(), new_shard_count=%new_shard_count.literal()))]
#[instrument(skip_all, fields(tenant_id=%tenant.get_tenant_shard_id().tenant_id, shard_id=%tenant.get_tenant_shard_id().shard_slug(), new_shard_count=%new_shard_count.literal()))]
pub(crate) async fn shard_split(
&self,
tenant_shard_id: TenantShardId,
tenant: Arc<Tenant>,
new_shard_count: ShardCount,
new_stripe_size: Option<ShardStripeSize>,
ctx: &RequestContext,
) -> anyhow::Result<Vec<TenantShardId>> {
let tenant_shard_id = *tenant.get_tenant_shard_id();
let r = self
.do_shard_split(tenant_shard_id, new_shard_count, new_stripe_size, ctx)
.do_shard_split(tenant, new_shard_count, new_stripe_size, ctx)
.await;
if r.is_err() {
// Shard splitting might have left the original shard in a partially shut down state (it
// stops the shard's remote timeline client). Reset it to ensure we leave things in
// a working state.
if self.get(tenant_shard_id).is_some() {
tracing::warn!("Resetting {tenant_shard_id} after shard split failure");
tracing::warn!("Resetting after shard split failure");
if let Err(e) = self.reset_tenant(tenant_shard_id, false, ctx).await {
// Log this error because our return value will still be the original error, not this one. This is
// a severe error: if this happens, we might be leaving behind a tenant that is not fully functional
// (e.g. has uploads disabled). We can't do anything else: if reset fails then shutting the tenant down or
// setting it broken probably won't help either.
tracing::error!("Failed to reset {tenant_shard_id}: {e}");
tracing::error!("Failed to reset: {e}");
}
}
}
@@ -1501,12 +1481,12 @@ impl TenantManager {
pub(crate) async fn do_shard_split(
&self,
tenant_shard_id: TenantShardId,
tenant: Arc<Tenant>,
new_shard_count: ShardCount,
new_stripe_size: Option<ShardStripeSize>,
ctx: &RequestContext,
) -> anyhow::Result<Vec<TenantShardId>> {
let tenant = get_tenant(tenant_shard_id, true)?;
let tenant_shard_id = *tenant.get_tenant_shard_id();
// Validate the incoming request
if new_shard_count.count() <= tenant_shard_id.shard_count.count() {
@@ -1552,7 +1532,6 @@ impl TenantManager {
// If [`Tenant::split_prepare`] fails, we must reload the tenant, because it might
// have been left in a partially-shut-down state.
tracing::warn!("Failed to prepare for split: {e}, reloading Tenant before returning");
self.reset_tenant(tenant_shard_id, false, ctx).await?;
return Err(e);
}
@@ -1950,38 +1929,23 @@ impl TenantManager {
removal_result
}
pub(crate) async fn set_new_tenant_config(
pub(crate) fn list_tenants(
&self,
new_tenant_conf: TenantConfOpt,
tenant_id: TenantId,
) -> Result<(), SetNewTenantConfigError> {
// Legacy API: does not support sharding
let tenant_shard_id = TenantShardId::unsharded(tenant_id);
info!("configuring tenant {tenant_id}");
let tenant = get_tenant(tenant_shard_id, true)?;
if !tenant.tenant_shard_id().shard_count.is_unsharded() {
// Note that we use ShardParameters::default below.
return Err(SetNewTenantConfigError::Other(anyhow::anyhow!(
"This API may only be used on single-sharded tenants, use the /location_config API for sharded tenants"
)));
}
// This is a legacy API that only operates on attached tenants: the preferred
// API to use is the location_config/ endpoint, which lets the caller provide
// the full LocationConf.
let location_conf = LocationConf::attached_single(
new_tenant_conf.clone(),
tenant.generation,
&ShardParameters::default(),
);
Tenant::persist_tenant_config(self.conf, &tenant_shard_id, &location_conf)
.await
.map_err(SetNewTenantConfigError::Persist)?;
tenant.set_new_tenant_config(new_tenant_conf);
Ok(())
) -> Result<Vec<(TenantShardId, TenantState, Generation)>, TenantMapListError> {
let tenants = TENANTS.read().unwrap();
let m = match &*tenants {
TenantsMap::Initializing => return Err(TenantMapListError::Initializing),
TenantsMap::Open(m) | TenantsMap::ShuttingDown(m) => m,
};
Ok(m.iter()
.filter_map(|(id, tenant)| match tenant {
TenantSlot::Attached(tenant) => {
Some((*id, tenant.current_state(), tenant.generation()))
}
TenantSlot::Secondary(_) => None,
TenantSlot::InProgress(_) => None,
})
.collect())
}
}
@@ -1994,51 +1958,12 @@ pub(crate) enum GetTenantError {
#[error("Tenant {0} is not active")]
NotActive(TenantShardId),
/// Broken is logically a subset of NotActive, but a distinct error is useful as
/// NotActive is usually a retryable state for API purposes, whereas Broken
/// is a stuck error state
#[error("Tenant is broken: {0}")]
Broken(String),
// Initializing or shutting down: cannot authoritatively say whether we have this tenant
#[error("Tenant map is not available: {0}")]
MapState(#[from] TenantMapError),
}
/// Gets the tenant from the in-memory data, erroring if it's absent or is not fitting to the query.
/// `active_only = true` allows to query only tenants that are ready for operations, erroring on other kinds of tenants.
///
/// This method is cancel-safe.
pub(crate) fn get_tenant(
tenant_shard_id: TenantShardId,
active_only: bool,
) -> Result<Arc<Tenant>, GetTenantError> {
let locked = TENANTS.read().unwrap();
let peek_slot = tenant_map_peek_slot(&locked, &tenant_shard_id, TenantSlotPeekMode::Read)?;
match peek_slot {
Some(TenantSlot::Attached(tenant)) => match tenant.current_state() {
TenantState::Broken {
reason,
backtrace: _,
} if active_only => Err(GetTenantError::Broken(reason)),
TenantState::Active => Ok(Arc::clone(tenant)),
_ => {
if active_only {
Err(GetTenantError::NotActive(tenant_shard_id))
} else {
Ok(Arc::clone(tenant))
}
}
},
Some(TenantSlot::InProgress(_)) => Err(GetTenantError::NotActive(tenant_shard_id)),
None | Some(TenantSlot::Secondary(_)) => {
Err(GetTenantError::NotFound(tenant_shard_id.tenant_id))
}
}
}
#[derive(thiserror::Error, Debug)]
pub(crate) enum GetActiveTenantError {
/// We may time out either while TenantSlot is InProgress, or while the Tenant
@@ -2062,6 +1987,12 @@ pub(crate) enum GetActiveTenantError {
/// Tenant exists, but is in a state that cannot become active (e.g. Stopping, Broken)
#[error("will not become active. Current state: {0}")]
WillNotBecomeActive(TenantState),
/// Broken is logically a subset of WillNotBecomeActive, but a distinct error is useful as
/// WillNotBecomeActive is a permitted error under some circumstances, whereas broken should
/// never happen.
#[error("Tenant is broken: {0}")]
Broken(String),
}
/// Get a [`Tenant`] in its active state. If the tenant_id is currently in [`TenantSlot::InProgress`]
@@ -2281,27 +2212,6 @@ pub(crate) enum TenantMapListError {
Initializing,
}
///
/// Get list of tenants, for the mgmt API
///
pub(crate) async fn list_tenants(
) -> Result<Vec<(TenantShardId, TenantState, Generation)>, TenantMapListError> {
let tenants = TENANTS.read().unwrap();
let m = match &*tenants {
TenantsMap::Initializing => return Err(TenantMapListError::Initializing),
TenantsMap::Open(m) | TenantsMap::ShuttingDown(m) => m,
};
Ok(m.iter()
.filter_map(|(id, tenant)| match tenant {
TenantSlot::Attached(tenant) => {
Some((*id, tenant.current_state(), tenant.generation()))
}
TenantSlot::Secondary(_) => None,
TenantSlot::InProgress(_) => None,
})
.collect())
}
#[derive(Debug, thiserror::Error)]
pub(crate) enum TenantMapInsertError {
#[error(transparent)]

View File

@@ -9,6 +9,7 @@ use crate::{
metrics::SECONDARY_MODE,
tenant::{
config::AttachmentMode,
mgr::GetTenantError,
mgr::TenantManager,
remote_timeline_client::remote_heatmap_path,
span::debug_assert_current_span_has_tenant_id,
@@ -292,8 +293,11 @@ impl JobGenerator<UploadPending, WriteInProgress, WriteComplete, UploadCommand>
"Starting heatmap write on command");
let tenant = self
.tenant_manager
.get_attached_tenant_shard(*tenant_shard_id, true)
.get_attached_tenant_shard(*tenant_shard_id)
.map_err(|e| anyhow::anyhow!(e))?;
if !tenant.is_active() {
return Err(GetTenantError::NotActive(*tenant_shard_id).into());
}
Ok(UploadPending {
// Ignore our state for last digest: this forces an upload even if nothing has changed

View File

@@ -1274,6 +1274,7 @@ impl Timeline {
pub(crate) fn activate(
self: &Arc<Self>,
parent: Arc<crate::tenant::Tenant>,
broker_client: BrokerClientChannel,
background_jobs_can_start: Option<&completion::Barrier>,
ctx: &RequestContext,
@@ -1284,7 +1285,7 @@ impl Timeline {
}
self.launch_wal_receiver(ctx, broker_client);
self.set_state(TimelineState::Active);
self.launch_eviction_task(background_jobs_can_start);
self.launch_eviction_task(parent, background_jobs_can_start);
}
/// Graceful shutdown, may do a lot of I/O as we flush any open layers to disk and then

View File

@@ -51,6 +51,7 @@ pub struct EvictionTaskTenantState {
impl Timeline {
pub(super) fn launch_eviction_task(
self: &Arc<Self>,
parent: Arc<Tenant>,
background_tasks_can_start: Option<&completion::Barrier>,
) {
let self_clone = Arc::clone(self);
@@ -72,14 +73,14 @@ impl Timeline {
_ = completion::Barrier::maybe_wait(background_tasks_can_start) => {}
};
self_clone.eviction_task(cancel).await;
self_clone.eviction_task(parent, cancel).await;
Ok(())
},
);
}
#[instrument(skip_all, fields(tenant_id = %self.tenant_shard_id.tenant_id, shard_id = %self.tenant_shard_id.shard_slug(), timeline_id = %self.timeline_id))]
async fn eviction_task(self: Arc<Self>, cancel: CancellationToken) {
async fn eviction_task(self: Arc<Self>, tenant: Arc<Tenant>, cancel: CancellationToken) {
use crate::tenant::tasks::random_init_delay;
// acquire the gate guard only once within a useful span
@@ -103,7 +104,7 @@ impl Timeline {
loop {
let policy = self.get_eviction_policy();
let cf = self
.eviction_iteration(&policy, &cancel, &guard, &ctx)
.eviction_iteration(&tenant, &policy, &cancel, &guard, &ctx)
.await;
match cf {
@@ -123,6 +124,7 @@ impl Timeline {
#[instrument(skip_all, fields(policy_kind = policy.discriminant_str()))]
async fn eviction_iteration(
self: &Arc<Self>,
tenant: &Tenant,
policy: &EvictionPolicy,
cancel: &CancellationToken,
gate: &GateGuard,
@@ -137,7 +139,7 @@ impl Timeline {
}
EvictionPolicy::LayerAccessThreshold(p) => {
match self
.eviction_iteration_threshold(p, cancel, gate, ctx)
.eviction_iteration_threshold(tenant, p, cancel, gate, ctx)
.await
{
ControlFlow::Break(()) => return ControlFlow::Break(()),
@@ -146,7 +148,11 @@ impl Timeline {
(p.period, p.threshold)
}
EvictionPolicy::OnlyImitiate(p) => {
if self.imitiate_only(p, cancel, gate, ctx).await.is_break() {
if self
.imitiate_only(tenant, p, cancel, gate, ctx)
.await
.is_break()
{
return ControlFlow::Break(());
}
(p.period, p.threshold)
@@ -175,6 +181,7 @@ impl Timeline {
async fn eviction_iteration_threshold(
self: &Arc<Self>,
tenant: &Tenant,
p: &EvictionPolicyLayerAccessThreshold,
cancel: &CancellationToken,
gate: &GateGuard,
@@ -193,7 +200,10 @@ impl Timeline {
_ = self.cancel.cancelled() => return ControlFlow::Break(()),
};
match self.imitate_layer_accesses(p, cancel, gate, ctx).await {
match self
.imitate_layer_accesses(tenant, p, cancel, gate, ctx)
.await
{
ControlFlow::Break(()) => return ControlFlow::Break(()),
ControlFlow::Continue(()) => (),
}
@@ -315,6 +325,7 @@ impl Timeline {
/// disk usage based eviction task.
async fn imitiate_only(
self: &Arc<Self>,
tenant: &Tenant,
p: &EvictionPolicyLayerAccessThreshold,
cancel: &CancellationToken,
gate: &GateGuard,
@@ -331,7 +342,8 @@ impl Timeline {
_ = self.cancel.cancelled() => return ControlFlow::Break(()),
};
self.imitate_layer_accesses(p, cancel, gate, ctx).await
self.imitate_layer_accesses(tenant, p, cancel, gate, ctx)
.await
}
/// If we evict layers but keep cached values derived from those layers, then
@@ -361,6 +373,7 @@ impl Timeline {
#[instrument(skip_all)]
async fn imitate_layer_accesses(
&self,
tenant: &Tenant,
p: &EvictionPolicyLayerAccessThreshold,
cancel: &CancellationToken,
gate: &GateGuard,
@@ -396,17 +409,11 @@ impl Timeline {
// Make one of the tenant's timelines draw the short straw and run the calculation.
// The others wait until the calculation is done so that they take into account the
// imitated accesses that the winner made.
let tenant = match crate::tenant::mgr::get_tenant(self.tenant_shard_id, true) {
Ok(t) => t,
Err(_) => {
return ControlFlow::Break(());
}
};
let mut state = tenant.eviction_task_tenant_state.lock().await;
match state.last_layer_access_imitation {
Some(ts) if ts.elapsed() < inter_imitate_period => { /* no need to run */ }
_ => {
self.imitate_synthetic_size_calculation_worker(&tenant, cancel, ctx)
self.imitate_synthetic_size_calculation_worker(tenant, cancel, ctx)
.await;
state.last_layer_access_imitation = Some(tokio::time::Instant::now());
}
@@ -480,7 +487,7 @@ impl Timeline {
#[instrument(skip_all)]
async fn imitate_synthetic_size_calculation_worker(
&self,
tenant: &Arc<Tenant>,
tenant: &Tenant,
cancel: &CancellationToken,
ctx: &RequestContext,
) {

View File

@@ -86,6 +86,7 @@ impl<'t> UninitializedTimeline<'t> {
/// Prepares timeline data by loading it from the basebackup archive.
pub(crate) async fn import_basebackup_from_tar(
self,
tenant: Arc<Tenant>,
copyin_read: &mut (impl tokio::io::AsyncRead + Send + Sync + Unpin),
base_lsn: Lsn,
broker_client: storage_broker::BrokerClientChannel,
@@ -114,7 +115,7 @@ impl<'t> UninitializedTimeline<'t> {
// All the data has been imported. Insert the Timeline into the tenant's timelines map
let tl = self.finish_creation()?;
tl.activate(broker_client, None, ctx);
tl.activate(tenant, broker_client, None, ctx);
Ok(tl)
}

View File

@@ -90,6 +90,8 @@ def test_location_conf_churn(neon_env_builder: NeonEnvBuilder, seed: int):
# this shutdown case is logged at WARN severity by the time it bubbles up to logical size calculation code
# WARN ...: initial size calculation failed: downloading failed, possibly for shutdown
".*downloading failed, possibly for shutdown",
# {tenant_id=... timeline_id=...}:handle_pagerequests:handle_get_page_at_lsn_request{rel=1664/0/1260 blkno=0 req_lsn=0/149F0D8}: error reading relation or page version: Not found: will not become active. Current state: Stopping\n'
".*page_service.*will not become active.*",
]
)