mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-08 14:02:55 +00:00
pageserver/controller: enable tenant deletion without attachment (#7957)
## Problem As described in #7952, the controller's attempt to reconcile a tenant before finally deleting it can get hung up waiting for the compute notification hook to accept updates. The fact that we try and reconcile a tenant at all during deletion is part of a more general design issue (#5080), where deletion was implemented as an operation on attached tenant, requiring the tenant to be attached in order to delete it, which is not in principle necessary. Closes: #7952 ## Summary of changes - In the pageserver deletion API, only do the traditional deletion path if the tenant is attached. If it's secondary, then tear down the secondary location, and then do a remote delete. If it's not attached at all, just do the remote delete. - In the storage controller, instead of ensuring a tenant is attached before deletion, do a best-effort detach of the tenant, and then call into some arbitrary pageserver to issue a deletion of remote content. The pageserver retains its existing delete behavior when invoked on attached locations. We can remove this later when all users of the API are updated to either do a detach-before-delete. This will enable removing the "weird" code paths during startup that sometimes load a tenant and then immediately delete it, and removing the deletion markers on tenants.
This commit is contained in:
@@ -81,8 +81,10 @@ paths:
|
||||
Attempts to delete specified tenant. 500, 503 and 409 errors should be retried until 404 is retrieved.
|
||||
404 means that deletion successfully finished"
|
||||
responses:
|
||||
"200":
|
||||
description: Tenant was successfully deleted, or was already not found.
|
||||
"404":
|
||||
description: Tenant not found. This is the success path.
|
||||
description: Tenant not found. This is a success result, equivalent to 200.
|
||||
content:
|
||||
application/json:
|
||||
schema:
|
||||
|
||||
@@ -1073,7 +1073,7 @@ async fn tenant_delete_handler(
|
||||
|
||||
let state = get_state(&request);
|
||||
|
||||
state
|
||||
let status = state
|
||||
.tenant_manager
|
||||
.delete_tenant(tenant_shard_id, ACTIVE_TENANT_TIMEOUT)
|
||||
.instrument(info_span!("tenant_delete_handler",
|
||||
@@ -1082,7 +1082,14 @@ async fn tenant_delete_handler(
|
||||
))
|
||||
.await?;
|
||||
|
||||
json_response(StatusCode::ACCEPTED, ())
|
||||
// Callers use 404 as success for deletions, for historical reasons.
|
||||
if status == StatusCode::NOT_FOUND {
|
||||
return Err(ApiError::NotFound(
|
||||
anyhow::anyhow!("Deletion complete").into(),
|
||||
));
|
||||
}
|
||||
|
||||
json_response(status, ())
|
||||
}
|
||||
|
||||
/// HTTP endpoint to query the current tenant_size of a tenant.
|
||||
|
||||
@@ -3,6 +3,7 @@
|
||||
|
||||
use camino::{Utf8DirEntry, Utf8Path, Utf8PathBuf};
|
||||
use futures::StreamExt;
|
||||
use hyper::StatusCode;
|
||||
use itertools::Itertools;
|
||||
use pageserver_api::key::Key;
|
||||
use pageserver_api::models::LocationConfigMode;
|
||||
@@ -54,6 +55,7 @@ use utils::generation::Generation;
|
||||
use utils::id::{TenantId, TimelineId};
|
||||
|
||||
use super::delete::DeleteTenantError;
|
||||
use super::remote_timeline_client::remote_tenant_path;
|
||||
use super::secondary::SecondaryTenant;
|
||||
use super::timeline::detach_ancestor::PreparedTimelineDetach;
|
||||
use super::TenantSharedResources;
|
||||
@@ -1369,7 +1371,7 @@ impl TenantManager {
|
||||
&self,
|
||||
tenant_shard_id: TenantShardId,
|
||||
activation_timeout: Duration,
|
||||
) -> Result<(), DeleteTenantError> {
|
||||
) -> Result<StatusCode, DeleteTenantError> {
|
||||
super::span::debug_assert_current_span_has_tenant_id();
|
||||
// We acquire a SlotGuard during this function to protect against concurrent
|
||||
// changes while the ::prepare phase of DeleteTenantFlow executes, but then
|
||||
@@ -1382,18 +1384,79 @@ impl TenantManager {
|
||||
//
|
||||
// See https://github.com/neondatabase/neon/issues/5080
|
||||
|
||||
let slot_guard =
|
||||
tenant_map_acquire_slot(&tenant_shard_id, TenantSlotAcquireMode::MustExist)?;
|
||||
// Tenant deletion can happen two ways:
|
||||
// - Legacy: called on an attached location. The attached Tenant object stays alive in Stopping
|
||||
// state until deletion is complete.
|
||||
// - New: called on a pageserver without an attached location. We proceed with deletion from
|
||||
// remote storage.
|
||||
//
|
||||
// See https://github.com/neondatabase/neon/issues/5080 for more context on this transition.
|
||||
|
||||
// unwrap is safe because we used MustExist mode when acquiring
|
||||
let tenant = match slot_guard.get_old_value().as_ref().unwrap() {
|
||||
TenantSlot::Attached(tenant) => tenant.clone(),
|
||||
_ => {
|
||||
// Express "not attached" as equivalent to "not found"
|
||||
return Err(DeleteTenantError::NotAttached);
|
||||
let slot_guard = tenant_map_acquire_slot(&tenant_shard_id, TenantSlotAcquireMode::Any)?;
|
||||
match &slot_guard.old_value {
|
||||
Some(TenantSlot::Attached(tenant)) => {
|
||||
// Legacy deletion flow: the tenant remains attached, goes to Stopping state, and
|
||||
// deletion will be resumed across restarts.
|
||||
let tenant = tenant.clone();
|
||||
return self
|
||||
.delete_tenant_attached(slot_guard, tenant, activation_timeout)
|
||||
.await;
|
||||
}
|
||||
Some(TenantSlot::Secondary(secondary_tenant)) => {
|
||||
secondary_tenant.shutdown().await;
|
||||
let local_tenant_directory = self.conf.tenant_path(&tenant_shard_id);
|
||||
let tmp_dir = safe_rename_tenant_dir(&local_tenant_directory)
|
||||
.await
|
||||
.with_context(|| {
|
||||
format!("local tenant directory {local_tenant_directory:?} rename")
|
||||
})?;
|
||||
spawn_background_purge(tmp_dir);
|
||||
}
|
||||
Some(TenantSlot::InProgress(_)) => unreachable!(),
|
||||
None => {}
|
||||
};
|
||||
|
||||
// Fall through: local state for this tenant is no longer present, proceed with remote delete
|
||||
let remote_path = remote_tenant_path(&tenant_shard_id);
|
||||
let keys = match self
|
||||
.resources
|
||||
.remote_storage
|
||||
.list(
|
||||
Some(&remote_path),
|
||||
remote_storage::ListingMode::NoDelimiter,
|
||||
None,
|
||||
&self.cancel,
|
||||
)
|
||||
.await
|
||||
{
|
||||
Ok(listing) => listing.keys,
|
||||
Err(remote_storage::DownloadError::Cancelled) => {
|
||||
return Err(DeleteTenantError::Cancelled)
|
||||
}
|
||||
Err(remote_storage::DownloadError::NotFound) => return Ok(StatusCode::NOT_FOUND),
|
||||
Err(other) => return Err(DeleteTenantError::Other(anyhow::anyhow!(other))),
|
||||
};
|
||||
|
||||
if keys.is_empty() {
|
||||
tracing::info!("Remote storage already deleted");
|
||||
} else {
|
||||
tracing::info!("Deleting {} keys from remote storage", keys.len());
|
||||
self.resources
|
||||
.remote_storage
|
||||
.delete_objects(&keys, &self.cancel)
|
||||
.await?;
|
||||
}
|
||||
|
||||
// Callers use 404 as success for deletions, for historical reasons.
|
||||
Ok(StatusCode::NOT_FOUND)
|
||||
}
|
||||
|
||||
async fn delete_tenant_attached(
|
||||
&self,
|
||||
slot_guard: SlotGuard,
|
||||
tenant: Arc<Tenant>,
|
||||
activation_timeout: Duration,
|
||||
) -> Result<StatusCode, DeleteTenantError> {
|
||||
match tenant.current_state() {
|
||||
TenantState::Broken { .. } | TenantState::Stopping { .. } => {
|
||||
// If deletion is already in progress, return success (the semantics of this
|
||||
@@ -1403,7 +1466,7 @@ impl TenantManager {
|
||||
// The `delete_progress` lock is held: deletion is already happening
|
||||
// in the bacckground
|
||||
slot_guard.revert();
|
||||
return Ok(());
|
||||
return Ok(StatusCode::ACCEPTED);
|
||||
}
|
||||
}
|
||||
_ => {
|
||||
@@ -1436,7 +1499,8 @@ impl TenantManager {
|
||||
|
||||
// The Tenant goes back into the map in Stopping state, it will eventually be removed by DeleteTenantFLow
|
||||
slot_guard.revert();
|
||||
result
|
||||
let () = result?;
|
||||
Ok(StatusCode::ACCEPTED)
|
||||
}
|
||||
|
||||
#[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()))]
|
||||
|
||||
@@ -142,52 +142,6 @@ async fn handle_tenant_create(
|
||||
)
|
||||
}
|
||||
|
||||
// For tenant and timeline deletions, which both implement an "initially return 202, then 404 once
|
||||
// we're done" semantic, we wrap with a retry loop to expose a simpler API upstream. This avoids
|
||||
// needing to track a "deleting" state for tenants.
|
||||
async fn deletion_wrapper<R, F>(service: Arc<Service>, f: F) -> Result<Response<Body>, ApiError>
|
||||
where
|
||||
R: std::future::Future<Output = Result<StatusCode, ApiError>> + Send + 'static,
|
||||
F: Fn(Arc<Service>) -> R + Send + Sync + 'static,
|
||||
{
|
||||
let started_at = Instant::now();
|
||||
// To keep deletion reasonably snappy for small tenants, initially check after 1 second if deletion
|
||||
// completed.
|
||||
let mut retry_period = Duration::from_secs(1);
|
||||
// On subsequent retries, wait longer.
|
||||
let max_retry_period = Duration::from_secs(5);
|
||||
// Enable callers with a 30 second request timeout to reliably get a response
|
||||
let max_wait = Duration::from_secs(25);
|
||||
|
||||
loop {
|
||||
let status = f(service.clone()).await?;
|
||||
match status {
|
||||
StatusCode::ACCEPTED => {
|
||||
tracing::info!("Deletion accepted, waiting to try again...");
|
||||
tokio::time::sleep(retry_period).await;
|
||||
retry_period = max_retry_period;
|
||||
}
|
||||
StatusCode::NOT_FOUND => {
|
||||
tracing::info!("Deletion complete");
|
||||
return json_response(StatusCode::OK, ());
|
||||
}
|
||||
_ => {
|
||||
tracing::warn!("Unexpected status {status}");
|
||||
return json_response(status, ());
|
||||
}
|
||||
}
|
||||
|
||||
let now = Instant::now();
|
||||
if now + retry_period > started_at + max_wait {
|
||||
tracing::info!("Deletion timed out waiting for 404");
|
||||
// REQUEST_TIMEOUT would be more appropriate, but CONFLICT is already part of
|
||||
// the pageserver's swagger definition for this endpoint, and has the same desired
|
||||
// effect of causing the control plane to retry later.
|
||||
return json_response(StatusCode::CONFLICT, ());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
async fn handle_tenant_location_config(
|
||||
service: Arc<Service>,
|
||||
mut req: Request<Body>,
|
||||
@@ -283,13 +237,17 @@ async fn handle_tenant_delete(
|
||||
let tenant_id: TenantId = parse_request_param(&req, "tenant_id")?;
|
||||
check_permissions(&req, Scope::PageServerApi)?;
|
||||
|
||||
deletion_wrapper(service, move |service| async move {
|
||||
service
|
||||
.tenant_delete(tenant_id)
|
||||
.await
|
||||
.and_then(map_reqwest_hyper_status)
|
||||
})
|
||||
.await
|
||||
let status_code = service
|
||||
.tenant_delete(tenant_id)
|
||||
.await
|
||||
.and_then(map_reqwest_hyper_status)?;
|
||||
|
||||
if status_code == StatusCode::NOT_FOUND {
|
||||
// The pageserver uses 404 for successful deletion, but we use 200
|
||||
json_response(StatusCode::OK, ())
|
||||
} else {
|
||||
json_response(status_code, ())
|
||||
}
|
||||
}
|
||||
|
||||
async fn handle_tenant_timeline_create(
|
||||
@@ -317,6 +275,51 @@ async fn handle_tenant_timeline_delete(
|
||||
|
||||
let timeline_id: TimelineId = parse_request_param(&req, "timeline_id")?;
|
||||
|
||||
// For timeline deletions, which both implement an "initially return 202, then 404 once
|
||||
// we're done" semantic, we wrap with a retry loop to expose a simpler API upstream.
|
||||
async fn deletion_wrapper<R, F>(service: Arc<Service>, f: F) -> Result<Response<Body>, ApiError>
|
||||
where
|
||||
R: std::future::Future<Output = Result<StatusCode, ApiError>> + Send + 'static,
|
||||
F: Fn(Arc<Service>) -> R + Send + Sync + 'static,
|
||||
{
|
||||
let started_at = Instant::now();
|
||||
// To keep deletion reasonably snappy for small tenants, initially check after 1 second if deletion
|
||||
// completed.
|
||||
let mut retry_period = Duration::from_secs(1);
|
||||
// On subsequent retries, wait longer.
|
||||
let max_retry_period = Duration::from_secs(5);
|
||||
// Enable callers with a 30 second request timeout to reliably get a response
|
||||
let max_wait = Duration::from_secs(25);
|
||||
|
||||
loop {
|
||||
let status = f(service.clone()).await?;
|
||||
match status {
|
||||
StatusCode::ACCEPTED => {
|
||||
tracing::info!("Deletion accepted, waiting to try again...");
|
||||
tokio::time::sleep(retry_period).await;
|
||||
retry_period = max_retry_period;
|
||||
}
|
||||
StatusCode::NOT_FOUND => {
|
||||
tracing::info!("Deletion complete");
|
||||
return json_response(StatusCode::OK, ());
|
||||
}
|
||||
_ => {
|
||||
tracing::warn!("Unexpected status {status}");
|
||||
return json_response(status, ());
|
||||
}
|
||||
}
|
||||
|
||||
let now = Instant::now();
|
||||
if now + retry_period > started_at + max_wait {
|
||||
tracing::info!("Deletion timed out waiting for 404");
|
||||
// REQUEST_TIMEOUT would be more appropriate, but CONFLICT is already part of
|
||||
// the pageserver's swagger definition for this endpoint, and has the same desired
|
||||
// effect of causing the control plane to retry later.
|
||||
return json_response(StatusCode::CONFLICT, ());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
deletion_wrapper(service, move |service| async move {
|
||||
service
|
||||
.tenant_timeline_delete(tenant_id, timeline_id)
|
||||
|
||||
@@ -2376,61 +2376,80 @@ impl Service {
|
||||
let _tenant_lock =
|
||||
trace_exclusive_lock(&self.tenant_op_locks, tenant_id, TenantOperations::Delete).await;
|
||||
|
||||
self.ensure_attached_wait(tenant_id).await?;
|
||||
|
||||
// TODO: refactor into helper
|
||||
let targets = {
|
||||
let locked = self.inner.read().unwrap();
|
||||
let mut targets = Vec::new();
|
||||
|
||||
// Detach all shards
|
||||
let (detach_waiters, shard_ids, node) = {
|
||||
let mut shard_ids = Vec::new();
|
||||
let mut detach_waiters = Vec::new();
|
||||
let mut locked = self.inner.write().unwrap();
|
||||
let (nodes, tenants, scheduler) = locked.parts_mut();
|
||||
for (tenant_shard_id, shard) in
|
||||
locked.tenants.range(TenantShardId::tenant_range(tenant_id))
|
||||
tenants.range_mut(TenantShardId::tenant_range(tenant_id))
|
||||
{
|
||||
let node_id = shard.intent.get_attached().ok_or_else(|| {
|
||||
ApiError::InternalServerError(anyhow::anyhow!("Shard not scheduled"))
|
||||
})?;
|
||||
let node = locked
|
||||
.nodes
|
||||
.get(&node_id)
|
||||
.expect("Pageservers may not be deleted while referenced");
|
||||
shard_ids.push(*tenant_shard_id);
|
||||
|
||||
targets.push((*tenant_shard_id, node.clone()));
|
||||
// Update the tenant's intent to remove all attachments
|
||||
shard.policy = PlacementPolicy::Detached;
|
||||
shard
|
||||
.schedule(scheduler, &mut ScheduleContext::default())
|
||||
.expect("De-scheduling is infallible");
|
||||
debug_assert!(shard.intent.get_attached().is_none());
|
||||
debug_assert!(shard.intent.get_secondary().is_empty());
|
||||
|
||||
if let Some(waiter) = self.maybe_reconcile_shard(shard, nodes) {
|
||||
detach_waiters.push(waiter);
|
||||
}
|
||||
}
|
||||
targets
|
||||
|
||||
// Pick an arbitrary node to use for remote deletions (does not have to be where the tenant
|
||||
// was attached, just has to be able to see the S3 content)
|
||||
let node_id = scheduler.schedule_shard(&[], &ScheduleContext::default())?;
|
||||
let node = nodes
|
||||
.get(&node_id)
|
||||
.expect("Pageservers may not be deleted while lock is active");
|
||||
(detach_waiters, shard_ids, node.clone())
|
||||
};
|
||||
|
||||
// Phase 1: delete on the pageservers
|
||||
let mut any_pending = false;
|
||||
for (tenant_shard_id, node) in targets {
|
||||
let client = PageserverClient::new(
|
||||
node.get_id(),
|
||||
node.base_url(),
|
||||
self.config.jwt_token.as_deref(),
|
||||
);
|
||||
// TODO: this, like many other places, requires proper retry handling for 503, timeout: those should not
|
||||
// surface immediately as an error to our caller.
|
||||
let status = client.tenant_delete(tenant_shard_id).await.map_err(|e| {
|
||||
ApiError::InternalServerError(anyhow::anyhow!(
|
||||
"Error deleting shard {tenant_shard_id} on node {node}: {e}",
|
||||
))
|
||||
})?;
|
||||
tracing::info!(
|
||||
"Shard {tenant_shard_id} on node {node}, delete returned {}",
|
||||
status
|
||||
);
|
||||
if status == StatusCode::ACCEPTED {
|
||||
any_pending = true;
|
||||
}
|
||||
if let Err(e) = self.await_waiters(detach_waiters, RECONCILE_TIMEOUT).await {
|
||||
// Failing to detach shouldn't hold up deletion, e.g. if a node is offline we should be able
|
||||
// to use some other node to run the remote deletion.
|
||||
tracing::warn!("Failed to detach some locations: {e}");
|
||||
}
|
||||
|
||||
if any_pending {
|
||||
// Caller should call us again later. When we eventually see 404s from
|
||||
// all the shards, we may proceed to delete our records of the tenant.
|
||||
tracing::info!(
|
||||
"Tenant {} has some shards pending deletion, returning 202",
|
||||
tenant_id
|
||||
);
|
||||
return Ok(StatusCode::ACCEPTED);
|
||||
let locations = shard_ids
|
||||
.into_iter()
|
||||
.map(|s| (s, node.clone()))
|
||||
.collect::<Vec<_>>();
|
||||
let results = self.tenant_for_shards_api(
|
||||
locations,
|
||||
|tenant_shard_id, client| async move { client.tenant_delete(tenant_shard_id).await },
|
||||
1,
|
||||
3,
|
||||
RECONCILE_TIMEOUT,
|
||||
&self.cancel,
|
||||
)
|
||||
.await;
|
||||
for result in results {
|
||||
match result {
|
||||
Ok(StatusCode::ACCEPTED) => {
|
||||
// This could happen if we failed detach above, and hit a pageserver where the tenant
|
||||
// is still attached: it will accept the deletion in the background
|
||||
tracing::warn!(
|
||||
"Unexpectedly still attached on {}, client should retry",
|
||||
node
|
||||
);
|
||||
return Ok(StatusCode::ACCEPTED);
|
||||
}
|
||||
Ok(_) => {}
|
||||
Err(mgmt_api::Error::Cancelled) => {
|
||||
return Err(ApiError::ShuttingDown);
|
||||
}
|
||||
Err(e) => {
|
||||
// This is unexpected: remote deletion should be infallible, unless the object store
|
||||
// at large is unavailable.
|
||||
tracing::error!("Error deleting via node {}: {e}", node);
|
||||
return Err(ApiError::InternalServerError(anyhow::anyhow!(e)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Fall through: deletion of the tenant on pageservers is complete, we may proceed to drop
|
||||
|
||||
@@ -7,6 +7,7 @@ from typing import Any, Dict, List, Union
|
||||
|
||||
import pytest
|
||||
from fixtures.common_types import TenantId, TenantShardId, TimelineId
|
||||
from fixtures.compute_reconfigure import ComputeReconfigure
|
||||
from fixtures.log_helper import log
|
||||
from fixtures.neon_fixtures import (
|
||||
NeonEnv,
|
||||
@@ -18,6 +19,8 @@ from fixtures.neon_fixtures import (
|
||||
from fixtures.pageserver.http import PageserverHttpClient
|
||||
from fixtures.pageserver.utils import (
|
||||
MANY_SMALL_LAYERS_TENANT_CONFIG,
|
||||
assert_prefix_empty,
|
||||
assert_prefix_not_empty,
|
||||
enable_remote_storage_versioning,
|
||||
list_prefix,
|
||||
remote_storage_delete_key,
|
||||
@@ -839,6 +842,86 @@ def test_storage_controller_tenant_conf(neon_env_builder: NeonEnvBuilder):
|
||||
env.storage_controller.consistency_check()
|
||||
|
||||
|
||||
def test_storage_controller_tenant_deletion(
|
||||
neon_env_builder: NeonEnvBuilder,
|
||||
compute_reconfigure_listener: ComputeReconfigure,
|
||||
):
|
||||
"""
|
||||
Validate that:
|
||||
- Deleting a tenant deletes all its shards
|
||||
- Deletion does not require the compute notification hook to be responsive
|
||||
- Deleting a tenant also removes all secondary locations
|
||||
"""
|
||||
neon_env_builder.num_pageservers = 4
|
||||
neon_env_builder.enable_pageserver_remote_storage(s3_storage())
|
||||
neon_env_builder.control_plane_compute_hook_api = (
|
||||
compute_reconfigure_listener.control_plane_compute_hook_api
|
||||
)
|
||||
|
||||
env = neon_env_builder.init_configs()
|
||||
env.start()
|
||||
|
||||
tenant_id = TenantId.generate()
|
||||
timeline_id = TimelineId.generate()
|
||||
env.neon_cli.create_tenant(
|
||||
tenant_id, timeline_id, shard_count=2, placement_policy='{"Attached":1}'
|
||||
)
|
||||
|
||||
# Ensure all the locations are configured, including secondaries
|
||||
env.storage_controller.reconcile_until_idle()
|
||||
|
||||
shard_ids = [
|
||||
TenantShardId.parse(shard["shard_id"]) for shard in env.storage_controller.locate(tenant_id)
|
||||
]
|
||||
|
||||
# Assert attachments all have local content
|
||||
for shard_id in shard_ids:
|
||||
pageserver = env.get_tenant_pageserver(shard_id)
|
||||
assert pageserver.tenant_dir(shard_id).exists()
|
||||
|
||||
# Assert all shards have some content in remote storage
|
||||
for shard_id in shard_ids:
|
||||
assert_prefix_not_empty(
|
||||
neon_env_builder.pageserver_remote_storage,
|
||||
prefix="/".join(
|
||||
(
|
||||
"tenants",
|
||||
str(shard_id),
|
||||
)
|
||||
),
|
||||
)
|
||||
|
||||
# Break the compute hook: we are checking that deletion does not depend on the compute hook being available
|
||||
def break_hook():
|
||||
raise RuntimeError("Unexpected call to compute hook")
|
||||
|
||||
compute_reconfigure_listener.register_on_notify(break_hook)
|
||||
|
||||
# No retry loop: deletion should complete in one shot without polling for 202 responses, because
|
||||
# it cleanly detaches all the shards first, and then deletes them in remote storage
|
||||
env.storage_controller.pageserver_api().tenant_delete(tenant_id)
|
||||
|
||||
# Assert no pageservers have any local content
|
||||
for pageserver in env.pageservers:
|
||||
for shard_id in shard_ids:
|
||||
assert not pageserver.tenant_dir(shard_id).exists()
|
||||
|
||||
for shard_id in shard_ids:
|
||||
assert_prefix_empty(
|
||||
neon_env_builder.pageserver_remote_storage,
|
||||
prefix="/".join(
|
||||
(
|
||||
"tenants",
|
||||
str(shard_id),
|
||||
)
|
||||
),
|
||||
)
|
||||
|
||||
# Assert the tenant is not visible in storage controller API
|
||||
with pytest.raises(StorageControllerApiException):
|
||||
env.storage_controller.tenant_describe(tenant_id)
|
||||
|
||||
|
||||
class Failure:
|
||||
pageserver_id: int
|
||||
|
||||
|
||||
@@ -54,9 +54,26 @@ def test_tenant_delete_smoke(
|
||||
|
||||
# first try to delete non existing tenant
|
||||
tenant_id = TenantId.generate()
|
||||
env.pageserver.allowed_errors.append(f".*NotFound: tenant {tenant_id}.*")
|
||||
with pytest.raises(PageserverApiException, match=f"NotFound: tenant {tenant_id}"):
|
||||
ps_http.tenant_delete(tenant_id=tenant_id)
|
||||
env.pageserver.allowed_errors.append(".*NotFound.*")
|
||||
env.pageserver.allowed_errors.append(".*simulated failure.*")
|
||||
|
||||
# Check that deleting a non-existent tenant gives the expected result: this is a loop because we
|
||||
# may need to retry on some remote storage errors injected by the test harness
|
||||
while True:
|
||||
try:
|
||||
ps_http.tenant_delete(tenant_id=tenant_id)
|
||||
except PageserverApiException as e:
|
||||
if e.status_code == 500:
|
||||
# This test uses failure injection, which can produce 500s as the pageserver expects
|
||||
# the object store to always be available, and the ListObjects during deletion is generally
|
||||
# an infallible operation
|
||||
assert "simulated failure of remote operation" in e.message
|
||||
elif e.status_code == 404:
|
||||
# This is our expected result: trying to erase a non-existent tenant gives us 404
|
||||
assert "NotFound" in e.message
|
||||
break
|
||||
else:
|
||||
raise
|
||||
|
||||
env.neon_cli.create_tenant(
|
||||
tenant_id=tenant_id,
|
||||
|
||||
Reference in New Issue
Block a user