mirror of
https://github.com/neondatabase/neon.git
synced 2026-01-07 05:22:56 +00:00
propagate lock guard to background deletion task (#4495)
## Problem 1. During the rollout we got a panic: "timeline that we were deleting was concurrently removed from 'timelines' map" that was caused by lock guard not being propagated to the background part of the deletion. Existing test didnt catch it because failpoint that was used for verification was placed earlier prior to background task spawning. 2. When looking at surrounding code one more bug was detected. We removed timeline from the map before deletion is finished, which breaks client retry logic, because it will indicate 404 before actual deletion is completed which can lead to client stopping its retry poll earlier. ## Summary of changes 1. Carry the lock guard over to background deletion. Ensure existing test case fails without applied patch (second deletion becomes stuck without it, which eventually leads to a test failure). 2. Move delete_all call earlier so timeline is removed from the map is the last thing done during deletion. Additionally I've added timeline_id to the `update_gc_info` span, because `debug_assert_current_span_has_tenant_and_timeline_id` in `download_remote_layer` was firing when `update_gc_info` lead to on-demand downloads via `find_lsn_for_timestamp` (caught by @problame). This is not directly related to the PR but fixes possible flakiness. Another smaller set of changes involves deletion wrapper used in python tests. Now there is a simpler wrapper that waits for deletions to complete `timeline_delete_wait_completed`. Most of the test_delete_timeline.py tests make negative tests, i.e., "does ps_http.timeline_delete() fail in this and that scenario". These can be left alone. Other places when we actually do the deletions, we need to use the helper that polls for completion. Discussion https://neondb.slack.com/archives/C03F5SM1N02/p1686668007396639 resolves #4496 --------- Co-authored-by: Christian Schwarz <christian@neon.tech>
This commit is contained in:
@@ -473,6 +473,14 @@ pub(crate) enum ShutdownError {
|
||||
AlreadyStopping,
|
||||
}
|
||||
|
||||
struct DeletionGuard(OwnedMutexGuard<bool>);
|
||||
|
||||
impl DeletionGuard {
|
||||
fn is_deleted(&self) -> bool {
|
||||
*self.0
|
||||
}
|
||||
}
|
||||
|
||||
impl Tenant {
|
||||
/// Yet another helper for timeline initialization.
|
||||
/// Contains the common part of `load_local_timeline` and `load_remote_timeline`.
|
||||
@@ -1138,7 +1146,11 @@ impl Tenant {
|
||||
)
|
||||
.context("create_timeline_struct")?;
|
||||
|
||||
let guard = Arc::clone(&timeline.delete_lock).lock_owned().await;
|
||||
let guard = DeletionGuard(
|
||||
Arc::clone(&timeline.delete_lock)
|
||||
.try_lock_owned()
|
||||
.expect("cannot happen because we're the only owner"),
|
||||
);
|
||||
|
||||
// Note: here we even skip populating layer map. Timeline is essentially uninitialized.
|
||||
// RemoteTimelineClient is the only functioning part.
|
||||
@@ -1549,6 +1561,7 @@ impl Tenant {
|
||||
&self,
|
||||
timeline_id: TimelineId,
|
||||
timeline: Arc<Timeline>,
|
||||
guard: DeletionGuard,
|
||||
) -> anyhow::Result<()> {
|
||||
{
|
||||
// Grab the layer_removal_cs lock, and actually perform the deletion.
|
||||
@@ -1621,6 +1634,25 @@ impl Tenant {
|
||||
Err(anyhow::anyhow!("failpoint: timeline-delete-after-rm"))?
|
||||
});
|
||||
|
||||
if let Some(remote_client) = &timeline.remote_client {
|
||||
remote_client.delete_all().await.context("delete_all")?
|
||||
};
|
||||
|
||||
// Have a failpoint that can use the `pause` failpoint action.
|
||||
// We don't want to block the executor thread, hence, spawn_blocking + await.
|
||||
if cfg!(feature = "testing") {
|
||||
tokio::task::spawn_blocking({
|
||||
let current = tracing::Span::current();
|
||||
move || {
|
||||
let _entered = current.entered();
|
||||
tracing::info!("at failpoint in_progress_delete");
|
||||
fail::fail_point!("in_progress_delete");
|
||||
}
|
||||
})
|
||||
.await
|
||||
.expect("spawn_blocking");
|
||||
}
|
||||
|
||||
{
|
||||
// Remove the timeline from the map.
|
||||
let mut timelines = self.timelines.lock().unwrap();
|
||||
@@ -1641,12 +1673,7 @@ impl Tenant {
|
||||
drop(timelines);
|
||||
}
|
||||
|
||||
let remote_client = match &timeline.remote_client {
|
||||
Some(remote_client) => remote_client,
|
||||
None => return Ok(()),
|
||||
};
|
||||
|
||||
remote_client.delete_all().await?;
|
||||
drop(guard);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
@@ -1694,23 +1721,18 @@ impl Tenant {
|
||||
timeline = Arc::clone(timeline_entry.get());
|
||||
|
||||
// Prevent two tasks from trying to delete the timeline at the same time.
|
||||
//
|
||||
// XXX: We should perhaps return an HTTP "202 Accepted" to signal that the caller
|
||||
// needs to poll until the operation has finished. But for now, we return an
|
||||
// error, because the control plane knows to retry errors.
|
||||
|
||||
delete_lock_guard =
|
||||
Arc::clone(&timeline.delete_lock)
|
||||
.try_lock_owned()
|
||||
.map_err(|_| {
|
||||
DeletionGuard(Arc::clone(&timeline.delete_lock).try_lock_owned().map_err(
|
||||
|_| {
|
||||
DeleteTimelineError::Other(anyhow::anyhow!(
|
||||
"timeline deletion is already in progress"
|
||||
))
|
||||
})?;
|
||||
},
|
||||
)?);
|
||||
|
||||
// If another task finished the deletion just before we acquired the lock,
|
||||
// return success.
|
||||
if *delete_lock_guard {
|
||||
if delete_lock_guard.is_deleted() {
|
||||
return Ok(());
|
||||
}
|
||||
|
||||
@@ -1784,7 +1806,7 @@ impl Tenant {
|
||||
self: Arc<Self>,
|
||||
timeline_id: TimelineId,
|
||||
timeline: Arc<Timeline>,
|
||||
_guard: OwnedMutexGuard<bool>,
|
||||
guard: DeletionGuard,
|
||||
) {
|
||||
let tenant_id = self.tenant_id;
|
||||
let timeline_clone = Arc::clone(&timeline);
|
||||
@@ -1797,7 +1819,7 @@ impl Tenant {
|
||||
"timeline_delete",
|
||||
false,
|
||||
async move {
|
||||
if let Err(err) = self.delete_timeline(timeline_id, timeline).await {
|
||||
if let Err(err) = self.delete_timeline(timeline_id, timeline, guard).await {
|
||||
error!("Error: {err:#}");
|
||||
timeline_clone.set_broken(err.to_string())
|
||||
};
|
||||
|
||||
@@ -753,22 +753,18 @@ impl RemoteTimelineClient {
|
||||
|
||||
// Have a failpoint that can use the `pause` failpoint action.
|
||||
// We don't want to block the executor thread, hence, spawn_blocking + await.
|
||||
#[cfg(feature = "testing")]
|
||||
tokio::task::spawn_blocking({
|
||||
let current = tracing::Span::current();
|
||||
move || {
|
||||
let _entered = current.entered();
|
||||
tracing::info!(
|
||||
"at failpoint persist_index_part_with_deleted_flag_after_set_before_upload_pause"
|
||||
);
|
||||
fail::fail_point!(
|
||||
"persist_index_part_with_deleted_flag_after_set_before_upload_pause"
|
||||
);
|
||||
}
|
||||
})
|
||||
.await
|
||||
.expect("spawn_blocking");
|
||||
|
||||
if cfg!(feature = "testing") {
|
||||
tokio::task::spawn_blocking({
|
||||
let current = tracing::Span::current();
|
||||
move || {
|
||||
let _entered = current.entered();
|
||||
tracing::info!("at failpoint persist_deleted_index_part");
|
||||
fail::fail_point!("persist_deleted_index_part");
|
||||
}
|
||||
})
|
||||
.await
|
||||
.expect("spawn_blocking");
|
||||
}
|
||||
upload::upload_index_part(
|
||||
self.conf,
|
||||
&self.storage_impl,
|
||||
|
||||
@@ -3791,6 +3791,7 @@ impl Timeline {
|
||||
/// for example. The caller should hold `Tenant::gc_cs` lock to ensure
|
||||
/// that.
|
||||
///
|
||||
#[instrument(skip_all, fields(timline_id=%self.timeline_id))]
|
||||
pub(super) async fn update_gc_info(
|
||||
&self,
|
||||
retain_lsns: Vec<Lsn>,
|
||||
|
||||
@@ -1631,6 +1631,8 @@ class NeonPageserver(PgProtocol):
|
||||
r".*ERROR.*ancestor timeline \S+ is being stopped",
|
||||
# this is expected given our collaborative shutdown approach for the UploadQueue
|
||||
".*Compaction failed, retrying in .*: queue is in state Stopped.*",
|
||||
# Pageserver timeline deletion should be polled until it gets 404, so ignore it globally
|
||||
".*Error processing HTTP request: NotFound: Timeline .* was not found",
|
||||
]
|
||||
|
||||
def start(
|
||||
|
||||
@@ -342,6 +342,11 @@ class PageserverHttpClient(requests.Session):
|
||||
return res_json
|
||||
|
||||
def timeline_delete(self, tenant_id: TenantId, timeline_id: TimelineId, **kwargs):
|
||||
"""
|
||||
Note that deletion is not instant, it is scheduled and performed mostly in the background.
|
||||
So if you need to wait for it to complete use `timeline_delete_wait_completed`.
|
||||
For longer description consult with pageserver openapi spec.
|
||||
"""
|
||||
res = self.delete(
|
||||
f"http://localhost:{self.port}/v1/tenant/{tenant_id}/timeline/{timeline_id}", **kwargs
|
||||
)
|
||||
|
||||
@@ -193,19 +193,30 @@ def wait_for_upload_queue_empty(
|
||||
time.sleep(0.2)
|
||||
|
||||
|
||||
def assert_timeline_detail_404(
|
||||
def wait_timeline_detail_404(
|
||||
pageserver_http: PageserverHttpClient, tenant_id: TenantId, timeline_id: TimelineId
|
||||
):
|
||||
last_exc = None
|
||||
for _ in range(2):
|
||||
time.sleep(0.250)
|
||||
try:
|
||||
data = pageserver_http.timeline_detail(tenant_id, timeline_id)
|
||||
log.error(f"detail {data}")
|
||||
except PageserverApiException as e:
|
||||
log.debug(e)
|
||||
if e.status_code == 404:
|
||||
return
|
||||
|
||||
last_exc = e
|
||||
|
||||
raise last_exc or RuntimeError(f"Timeline wasnt deleted in time, state: {data['state']}")
|
||||
|
||||
|
||||
def timeline_delete_wait_completed(
|
||||
pageserver_http: PageserverHttpClient,
|
||||
tenant_id: TenantId,
|
||||
timeline_id: TimelineId,
|
||||
**delete_args,
|
||||
):
|
||||
"""Asserts that timeline_detail returns 404, or dumps the detail."""
|
||||
try:
|
||||
data = pageserver_http.timeline_detail(tenant_id, timeline_id)
|
||||
log.error(f"detail {data}")
|
||||
except PageserverApiException as e:
|
||||
log.error(e)
|
||||
if e.status_code == 404:
|
||||
return
|
||||
else:
|
||||
raise
|
||||
raise Exception("detail succeeded (it should return 404)")
|
||||
pageserver_http.timeline_delete(tenant_id=tenant_id, timeline_id=timeline_id, **delete_args)
|
||||
wait_timeline_detail_404(pageserver_http, tenant_id, timeline_id)
|
||||
|
||||
@@ -15,7 +15,11 @@ from fixtures.neon_fixtures import (
|
||||
PortDistributor,
|
||||
)
|
||||
from fixtures.pageserver.http import PageserverHttpClient
|
||||
from fixtures.pageserver.utils import wait_for_last_record_lsn, wait_for_upload
|
||||
from fixtures.pageserver.utils import (
|
||||
timeline_delete_wait_completed,
|
||||
wait_for_last_record_lsn,
|
||||
wait_for_upload,
|
||||
)
|
||||
from fixtures.pg_version import PgVersion
|
||||
from fixtures.types import Lsn
|
||||
from pytest import FixtureRequest
|
||||
@@ -417,7 +421,7 @@ def check_neon_works(
|
||||
)
|
||||
|
||||
shutil.rmtree(repo_dir / "local_fs_remote_storage")
|
||||
pageserver_http.timeline_delete(tenant_id, timeline_id)
|
||||
timeline_delete_wait_completed(pageserver_http, tenant_id, timeline_id)
|
||||
pageserver_http.timeline_create(pg_version, tenant_id, timeline_id)
|
||||
pg_bin.run(
|
||||
["pg_dumpall", f"--dbname={connstr}", f"--file={test_output_dir / 'dump-from-wal.sql'}"]
|
||||
|
||||
@@ -14,7 +14,11 @@ from fixtures.neon_fixtures import (
|
||||
NeonEnvBuilder,
|
||||
PgBin,
|
||||
)
|
||||
from fixtures.pageserver.utils import wait_for_last_record_lsn, wait_for_upload
|
||||
from fixtures.pageserver.utils import (
|
||||
timeline_delete_wait_completed,
|
||||
wait_for_last_record_lsn,
|
||||
wait_for_upload,
|
||||
)
|
||||
from fixtures.types import Lsn, TenantId, TimelineId
|
||||
from fixtures.utils import subprocess_capture
|
||||
|
||||
@@ -151,7 +155,7 @@ def test_import_from_vanilla(test_output_dir, pg_bin, vanilla_pg, neon_env_build
|
||||
".*files not bound to index_file.json, proceeding with their deletion.*"
|
||||
)
|
||||
|
||||
client.timeline_delete(tenant, timeline)
|
||||
timeline_delete_wait_completed(client, tenant, timeline)
|
||||
|
||||
# Importing correct backup works
|
||||
import_tar(base_tar, wal_tar)
|
||||
|
||||
@@ -20,7 +20,7 @@ from fixtures.neon_fixtures import (
|
||||
)
|
||||
from fixtures.pageserver.http import PageserverApiException, PageserverHttpClient
|
||||
from fixtures.pageserver.utils import (
|
||||
assert_timeline_detail_404,
|
||||
timeline_delete_wait_completed,
|
||||
wait_for_last_record_lsn,
|
||||
wait_for_upload,
|
||||
wait_until_tenant_active,
|
||||
@@ -597,14 +597,11 @@ def test_timeline_deletion_with_files_stuck_in_upload_queue(
|
||||
env.pageserver.allowed_errors.append(
|
||||
".* ERROR .*Error processing HTTP request: InternalServerError\\(timeline is Stopping"
|
||||
)
|
||||
client.timeline_delete(tenant_id, timeline_id)
|
||||
|
||||
env.pageserver.allowed_errors.append(f".*Timeline {tenant_id}/{timeline_id} was not found.*")
|
||||
env.pageserver.allowed_errors.append(
|
||||
".*files not bound to index_file.json, proceeding with their deletion.*"
|
||||
)
|
||||
|
||||
wait_until(2, 0.5, lambda: assert_timeline_detail_404(client, tenant_id, timeline_id))
|
||||
timeline_delete_wait_completed(client, tenant_id, timeline_id)
|
||||
|
||||
assert not timeline_path.exists()
|
||||
|
||||
|
||||
@@ -11,6 +11,7 @@ from fixtures.neon_fixtures import (
|
||||
wait_for_wal_insert_lsn,
|
||||
)
|
||||
from fixtures.pageserver.http import PageserverHttpClient
|
||||
from fixtures.pageserver.utils import timeline_delete_wait_completed
|
||||
from fixtures.pg_version import PgVersion, xfail_on_postgres
|
||||
from fixtures.types import Lsn, TenantId, TimelineId
|
||||
|
||||
@@ -628,12 +629,12 @@ def test_get_tenant_size_with_multiple_branches(
|
||||
size_debug_file_before.write(size_debug)
|
||||
|
||||
# teardown, delete branches, and the size should be going down
|
||||
http_client.timeline_delete(tenant_id, first_branch_timeline_id)
|
||||
timeline_delete_wait_completed(http_client, tenant_id, first_branch_timeline_id)
|
||||
|
||||
size_after_deleting_first = http_client.tenant_size(tenant_id)
|
||||
assert size_after_deleting_first < size_after_thinning_branch
|
||||
|
||||
http_client.timeline_delete(tenant_id, second_branch_timeline_id)
|
||||
timeline_delete_wait_completed(http_client, tenant_id, second_branch_timeline_id)
|
||||
size_after_deleting_second = http_client.tenant_size(tenant_id)
|
||||
assert size_after_deleting_second < size_after_deleting_first
|
||||
|
||||
|
||||
@@ -1,6 +1,10 @@
|
||||
from fixtures.log_helper import log
|
||||
from fixtures.neon_fixtures import NeonEnvBuilder
|
||||
from fixtures.pageserver.utils import assert_tenant_state, wait_until_tenant_active
|
||||
from fixtures.pageserver.utils import (
|
||||
assert_tenant_state,
|
||||
timeline_delete_wait_completed,
|
||||
wait_until_tenant_active,
|
||||
)
|
||||
from fixtures.types import TenantId, TimelineId
|
||||
from fixtures.utils import wait_until
|
||||
|
||||
@@ -24,7 +28,7 @@ def test_tenant_tasks(neon_env_builder: NeonEnvBuilder):
|
||||
def delete_all_timelines(tenant: TenantId):
|
||||
timelines = [TimelineId(t["timeline_id"]) for t in client.timeline_list(tenant)]
|
||||
for t in timelines:
|
||||
client.timeline_delete(tenant, t)
|
||||
timeline_delete_wait_completed(client, tenant, t)
|
||||
|
||||
# Create tenant, start compute
|
||||
tenant, _ = env.neon_cli.create_tenant()
|
||||
|
||||
@@ -21,6 +21,7 @@ from fixtures.neon_fixtures import (
|
||||
RemoteStorageKind,
|
||||
available_remote_storages,
|
||||
)
|
||||
from fixtures.pageserver.utils import timeline_delete_wait_completed
|
||||
from fixtures.types import Lsn, TenantId, TimelineId
|
||||
from fixtures.utils import wait_until
|
||||
from prometheus_client.samples import Sample
|
||||
@@ -318,9 +319,10 @@ def test_pageserver_with_empty_tenants(
|
||||
client.tenant_create(tenant_with_empty_timelines)
|
||||
temp_timelines = client.timeline_list(tenant_with_empty_timelines)
|
||||
for temp_timeline in temp_timelines:
|
||||
client.timeline_delete(
|
||||
tenant_with_empty_timelines, TimelineId(temp_timeline["timeline_id"])
|
||||
timeline_delete_wait_completed(
|
||||
client, tenant_with_empty_timelines, TimelineId(temp_timeline["timeline_id"])
|
||||
)
|
||||
|
||||
files_in_timelines_dir = sum(
|
||||
1
|
||||
for _p in Path.iterdir(
|
||||
|
||||
@@ -17,9 +17,10 @@ from fixtures.neon_fixtures import (
|
||||
)
|
||||
from fixtures.pageserver.http import PageserverApiException
|
||||
from fixtures.pageserver.utils import (
|
||||
assert_timeline_detail_404,
|
||||
timeline_delete_wait_completed,
|
||||
wait_for_last_record_lsn,
|
||||
wait_for_upload,
|
||||
wait_timeline_detail_404,
|
||||
wait_until_tenant_active,
|
||||
wait_until_timeline_state,
|
||||
)
|
||||
@@ -83,7 +84,7 @@ def test_timeline_delete(neon_simple_env: NeonEnv):
|
||||
wait_until(
|
||||
number_of_iterations=3,
|
||||
interval=0.2,
|
||||
func=lambda: ps_http.timeline_delete(env.initial_tenant, leaf_timeline_id),
|
||||
func=lambda: timeline_delete_wait_completed(ps_http, env.initial_tenant, leaf_timeline_id),
|
||||
)
|
||||
|
||||
assert not timeline_path.exists()
|
||||
@@ -94,16 +95,16 @@ def test_timeline_delete(neon_simple_env: NeonEnv):
|
||||
match=f"Timeline {env.initial_tenant}/{leaf_timeline_id} was not found",
|
||||
) as exc:
|
||||
ps_http.timeline_detail(env.initial_tenant, leaf_timeline_id)
|
||||
|
||||
# FIXME leaves tenant without timelines, should we prevent deletion of root timeline?
|
||||
wait_until(
|
||||
number_of_iterations=3,
|
||||
interval=0.2,
|
||||
func=lambda: ps_http.timeline_delete(env.initial_tenant, parent_timeline_id),
|
||||
)
|
||||
|
||||
assert exc.value.status_code == 404
|
||||
|
||||
wait_until(
|
||||
number_of_iterations=3,
|
||||
interval=0.2,
|
||||
func=lambda: timeline_delete_wait_completed(
|
||||
ps_http, env.initial_tenant, parent_timeline_id
|
||||
),
|
||||
)
|
||||
|
||||
# Check that we didn't pick up the timeline again after restart.
|
||||
# See https://github.com/neondatabase/neon/issues/3560
|
||||
env.pageserver.stop(immediate=True)
|
||||
@@ -143,7 +144,6 @@ def test_delete_timeline_post_rm_failure(
|
||||
ps_http.configure_failpoints((failpoint_name, "return"))
|
||||
|
||||
ps_http.timeline_delete(env.initial_tenant, env.initial_timeline)
|
||||
|
||||
timeline_info = wait_until_timeline_state(
|
||||
pageserver_http=ps_http,
|
||||
tenant_id=env.initial_tenant,
|
||||
@@ -165,13 +165,7 @@ def test_delete_timeline_post_rm_failure(
|
||||
|
||||
# this should succeed
|
||||
# this also checks that delete can be retried even when timeline is in Broken state
|
||||
ps_http.timeline_delete(env.initial_tenant, env.initial_timeline, timeout=2)
|
||||
with pytest.raises(PageserverApiException) as e:
|
||||
ps_http.timeline_detail(env.initial_tenant, env.initial_timeline)
|
||||
|
||||
assert e.value.status_code == 404
|
||||
|
||||
env.pageserver.allowed_errors.append(f".*NotFound: Timeline.*{env.initial_timeline}.*")
|
||||
timeline_delete_wait_completed(ps_http, env.initial_tenant, env.initial_timeline)
|
||||
env.pageserver.allowed_errors.append(
|
||||
f".*{env.initial_timeline}.*timeline directory not found, proceeding anyway.*"
|
||||
)
|
||||
@@ -247,13 +241,7 @@ def test_timeline_resurrection_on_attach(
|
||||
pass
|
||||
|
||||
# delete new timeline
|
||||
ps_http.timeline_delete(tenant_id=tenant_id, timeline_id=branch_timeline_id)
|
||||
|
||||
env.pageserver.allowed_errors.append(
|
||||
f".*Timeline {tenant_id}/{branch_timeline_id} was not found.*"
|
||||
)
|
||||
|
||||
wait_until(2, 0.5, lambda: assert_timeline_detail_404(ps_http, tenant_id, branch_timeline_id))
|
||||
timeline_delete_wait_completed(ps_http, tenant_id=tenant_id, timeline_id=branch_timeline_id)
|
||||
|
||||
##### Stop the pageserver instance, erase all its data
|
||||
env.endpoints.stop_all()
|
||||
@@ -338,7 +326,6 @@ def test_timeline_delete_fail_before_local_delete(neon_env_builder: NeonEnvBuild
|
||||
)
|
||||
|
||||
ps_http.timeline_delete(env.initial_tenant, leaf_timeline_id)
|
||||
|
||||
timeline_info = wait_until_timeline_state(
|
||||
pageserver_http=ps_http,
|
||||
tenant_id=env.initial_tenant,
|
||||
@@ -357,12 +344,15 @@ def test_timeline_delete_fail_before_local_delete(neon_env_builder: NeonEnvBuild
|
||||
# Wait for tenant to finish loading.
|
||||
wait_until_tenant_active(ps_http, tenant_id=env.initial_tenant, iterations=10, period=1)
|
||||
|
||||
env.pageserver.allowed_errors.append(
|
||||
f".*Timeline {env.initial_tenant}/{leaf_timeline_id} was not found.*"
|
||||
)
|
||||
wait_until(
|
||||
2, 0.5, lambda: assert_timeline_detail_404(ps_http, env.initial_tenant, leaf_timeline_id)
|
||||
)
|
||||
try:
|
||||
data = ps_http.timeline_detail(env.initial_tenant, leaf_timeline_id)
|
||||
log.debug(f"detail {data}")
|
||||
except PageserverApiException as e:
|
||||
log.debug(e)
|
||||
if e.status_code != 404:
|
||||
raise
|
||||
else:
|
||||
raise Exception("detail succeeded (it should return 404)")
|
||||
|
||||
assert (
|
||||
not leaf_timeline_path.exists()
|
||||
@@ -389,13 +379,8 @@ def test_timeline_delete_fail_before_local_delete(neon_env_builder: NeonEnvBuild
|
||||
assert env.initial_timeline is not None
|
||||
|
||||
for timeline_id in (intermediate_timeline_id, env.initial_timeline):
|
||||
ps_http.timeline_delete(env.initial_tenant, timeline_id)
|
||||
|
||||
env.pageserver.allowed_errors.append(
|
||||
f".*Timeline {env.initial_tenant}/{timeline_id} was not found.*"
|
||||
)
|
||||
wait_until(
|
||||
2, 0.5, lambda: assert_timeline_detail_404(ps_http, env.initial_tenant, timeline_id)
|
||||
timeline_delete_wait_completed(
|
||||
ps_http, tenant_id=env.initial_tenant, timeline_id=timeline_id
|
||||
)
|
||||
|
||||
assert_prefix_empty(
|
||||
@@ -419,23 +404,27 @@ def test_timeline_delete_fail_before_local_delete(neon_env_builder: NeonEnvBuild
|
||||
)
|
||||
|
||||
|
||||
def test_concurrent_timeline_delete_if_first_stuck_at_index_upload(
|
||||
neon_env_builder: NeonEnvBuilder,
|
||||
@pytest.mark.parametrize(
|
||||
"stuck_failpoint",
|
||||
["persist_deleted_index_part", "in_progress_delete"],
|
||||
)
|
||||
def test_concurrent_timeline_delete_stuck_on(
|
||||
neon_env_builder: NeonEnvBuilder, stuck_failpoint: str
|
||||
):
|
||||
"""
|
||||
If we're stuck uploading the index file with the is_delete flag,
|
||||
eventually console will hand up and retry.
|
||||
If we're still stuck at the retry time, ensure that the retry
|
||||
fails with status 500, signalling to console that it should retry
|
||||
later.
|
||||
Ideally, timeline_delete should return 202 Accepted and require
|
||||
console to poll for completion, but, that would require changing
|
||||
the API contract.
|
||||
If delete is stuck console will eventually retry deletion.
|
||||
So we need to be sure that these requests wont interleave with each other.
|
||||
In this tests we check two places where we can spend a lot of time.
|
||||
This is a regression test because there was a bug when DeletionGuard wasnt propagated
|
||||
to the background task.
|
||||
|
||||
Ensure that when retry comes if we're still stuck request will get an immediate error response,
|
||||
signalling to console that it should retry later.
|
||||
"""
|
||||
|
||||
neon_env_builder.enable_remote_storage(
|
||||
remote_storage_kind=RemoteStorageKind.MOCK_S3,
|
||||
test_name="test_concurrent_timeline_delete_if_first_stuck_at_index_upload",
|
||||
test_name=f"concurrent_timeline_delete_stuck_on_{stuck_failpoint}",
|
||||
)
|
||||
|
||||
env = neon_env_builder.init_start()
|
||||
@@ -445,13 +434,14 @@ def test_concurrent_timeline_delete_if_first_stuck_at_index_upload(
|
||||
ps_http = env.pageserver.http_client()
|
||||
|
||||
# make the first call sleep practically forever
|
||||
failpoint_name = "persist_index_part_with_deleted_flag_after_set_before_upload_pause"
|
||||
ps_http.configure_failpoints((failpoint_name, "pause"))
|
||||
ps_http.configure_failpoints((stuck_failpoint, "pause"))
|
||||
|
||||
def first_call(result_queue):
|
||||
try:
|
||||
log.info("first call start")
|
||||
ps_http.timeline_delete(env.initial_tenant, child_timeline_id, timeout=10)
|
||||
timeline_delete_wait_completed(
|
||||
ps_http, env.initial_tenant, child_timeline_id, timeout=10
|
||||
)
|
||||
log.info("first call success")
|
||||
result_queue.put("success")
|
||||
except Exception:
|
||||
@@ -466,7 +456,7 @@ def test_concurrent_timeline_delete_if_first_stuck_at_index_upload(
|
||||
|
||||
def first_call_hit_failpoint():
|
||||
assert env.pageserver.log_contains(
|
||||
f".*{child_timeline_id}.*at failpoint {failpoint_name}"
|
||||
f".*{child_timeline_id}.*at failpoint {stuck_failpoint}"
|
||||
)
|
||||
|
||||
wait_until(50, 0.1, first_call_hit_failpoint)
|
||||
@@ -484,8 +474,12 @@ def test_concurrent_timeline_delete_if_first_stuck_at_index_upload(
|
||||
)
|
||||
log.info("second call failed as expected")
|
||||
|
||||
# ensure it is not 404 and stopping
|
||||
detail = ps_http.timeline_detail(env.initial_tenant, child_timeline_id)
|
||||
assert detail["state"] == "Stopping"
|
||||
|
||||
# by now we know that the second call failed, let's ensure the first call will finish
|
||||
ps_http.configure_failpoints((failpoint_name, "off"))
|
||||
ps_http.configure_failpoints((stuck_failpoint, "off"))
|
||||
|
||||
result = first_call_result.get()
|
||||
assert result == "success"
|
||||
@@ -498,8 +492,10 @@ def test_concurrent_timeline_delete_if_first_stuck_at_index_upload(
|
||||
|
||||
def test_delete_timeline_client_hangup(neon_env_builder: NeonEnvBuilder):
|
||||
"""
|
||||
If the client hangs up before we start the index part upload but after we mark it
|
||||
If the client hangs up before we start the index part upload but after deletion is scheduled
|
||||
we mark it
|
||||
deleted in local memory, a subsequent delete_timeline call should be able to do
|
||||
|
||||
another delete timeline operation.
|
||||
|
||||
This tests cancel safety up to the given failpoint.
|
||||
@@ -515,12 +511,18 @@ def test_delete_timeline_client_hangup(neon_env_builder: NeonEnvBuilder):
|
||||
|
||||
ps_http = env.pageserver.http_client()
|
||||
|
||||
failpoint_name = "persist_index_part_with_deleted_flag_after_set_before_upload_pause"
|
||||
failpoint_name = "persist_deleted_index_part"
|
||||
ps_http.configure_failpoints((failpoint_name, "pause"))
|
||||
|
||||
with pytest.raises(requests.exceptions.Timeout):
|
||||
ps_http.timeline_delete(env.initial_tenant, child_timeline_id, timeout=2)
|
||||
|
||||
env.pageserver.allowed_errors.append(
|
||||
f".*{child_timeline_id}.*timeline deletion is already in progress.*"
|
||||
)
|
||||
with pytest.raises(PageserverApiException, match="timeline deletion is already in progress"):
|
||||
ps_http.timeline_delete(env.initial_tenant, child_timeline_id, timeout=2)
|
||||
|
||||
# make sure the timeout was due to the failpoint
|
||||
at_failpoint_log_message = f".*{child_timeline_id}.*at failpoint {failpoint_name}.*"
|
||||
|
||||
@@ -552,12 +554,7 @@ def test_delete_timeline_client_hangup(neon_env_builder: NeonEnvBuilder):
|
||||
wait_until(50, 0.1, first_request_finished)
|
||||
|
||||
# check that the timeline is gone
|
||||
notfound_message = f"Timeline {env.initial_tenant}/{child_timeline_id} was not found"
|
||||
env.pageserver.allowed_errors.append(".*" + notfound_message)
|
||||
with pytest.raises(PageserverApiException, match=notfound_message) as exc:
|
||||
ps_http.timeline_detail(env.initial_tenant, child_timeline_id)
|
||||
|
||||
assert exc.value.status_code == 404
|
||||
wait_timeline_detail_404(ps_http, env.initial_tenant, child_timeline_id)
|
||||
|
||||
|
||||
@pytest.mark.parametrize(
|
||||
@@ -616,12 +613,7 @@ def test_timeline_delete_works_for_remote_smoke(
|
||||
for timeline_id in reversed(timeline_ids):
|
||||
# note that we need to finish previous deletion before scheduling next one
|
||||
# otherwise we can get an "HasChildren" error if deletion is not fast enough (real_s3)
|
||||
ps_http.timeline_delete(tenant_id=tenant_id, timeline_id=timeline_id)
|
||||
|
||||
env.pageserver.allowed_errors.append(
|
||||
f".*Timeline {env.initial_tenant}/{timeline_id} was not found.*"
|
||||
)
|
||||
wait_until(2, 0.5, lambda: assert_timeline_detail_404(ps_http, tenant_id, timeline_id))
|
||||
timeline_delete_wait_completed(ps_http, tenant_id=tenant_id, timeline_id=timeline_id)
|
||||
|
||||
assert_prefix_empty(
|
||||
neon_env_builder,
|
||||
|
||||
@@ -24,6 +24,7 @@ from fixtures.neon_fixtures import (
|
||||
from fixtures.pageserver.http import PageserverApiException, PageserverHttpClient
|
||||
from fixtures.pageserver.utils import (
|
||||
assert_tenant_state,
|
||||
timeline_delete_wait_completed,
|
||||
wait_for_upload_queue_empty,
|
||||
wait_until_tenant_active,
|
||||
)
|
||||
@@ -272,7 +273,7 @@ def test_timeline_initial_logical_size_calculation_cancellation(
|
||||
if deletion_method == "tenant_detach":
|
||||
client.tenant_detach(tenant_id)
|
||||
elif deletion_method == "timeline_delete":
|
||||
client.timeline_delete(tenant_id, timeline_id)
|
||||
timeline_delete_wait_completed(client, tenant_id, timeline_id)
|
||||
delete_timeline_success.put(True)
|
||||
except PageserverApiException:
|
||||
delete_timeline_success.put(False)
|
||||
|
||||
@@ -31,7 +31,11 @@ from fixtures.neon_fixtures import (
|
||||
SafekeeperPort,
|
||||
available_remote_storages,
|
||||
)
|
||||
from fixtures.pageserver.utils import wait_for_last_record_lsn, wait_for_upload
|
||||
from fixtures.pageserver.utils import (
|
||||
timeline_delete_wait_completed,
|
||||
wait_for_last_record_lsn,
|
||||
wait_for_upload,
|
||||
)
|
||||
from fixtures.pg_version import PgVersion
|
||||
from fixtures.types import Lsn, TenantId, TimelineId
|
||||
from fixtures.utils import get_dir_size, query_scalar, start_in_background
|
||||
@@ -548,15 +552,15 @@ def test_s3_wal_replay(neon_env_builder: NeonEnvBuilder, remote_storage_kind: Re
|
||||
f"sk_id={sk.id} to flush {last_lsn}",
|
||||
)
|
||||
|
||||
ps_cli = env.pageserver.http_client()
|
||||
pageserver_lsn = Lsn(ps_cli.timeline_detail(tenant_id, timeline_id)["last_record_lsn"])
|
||||
ps_http = env.pageserver.http_client()
|
||||
pageserver_lsn = Lsn(ps_http.timeline_detail(tenant_id, timeline_id)["last_record_lsn"])
|
||||
lag = last_lsn - pageserver_lsn
|
||||
log.info(
|
||||
f"Pageserver last_record_lsn={pageserver_lsn}; flush_lsn={last_lsn}; lag before replay is {lag / 1024}kb"
|
||||
)
|
||||
|
||||
endpoint.stop_and_destroy()
|
||||
ps_cli.timeline_delete(tenant_id, timeline_id)
|
||||
timeline_delete_wait_completed(ps_http, tenant_id, timeline_id)
|
||||
|
||||
# Also delete and manually create timeline on safekeepers -- this tests
|
||||
# scenario of manual recovery on different set of safekeepers.
|
||||
@@ -583,7 +587,7 @@ def test_s3_wal_replay(neon_env_builder: NeonEnvBuilder, remote_storage_kind: Re
|
||||
shutil.copy(f_partial_saved, f_partial_path)
|
||||
|
||||
# recreate timeline on pageserver from scratch
|
||||
ps_cli.timeline_create(
|
||||
ps_http.timeline_create(
|
||||
pg_version=PgVersion(pg_version),
|
||||
tenant_id=tenant_id,
|
||||
new_timeline_id=timeline_id,
|
||||
@@ -598,7 +602,7 @@ def test_s3_wal_replay(neon_env_builder: NeonEnvBuilder, remote_storage_kind: Re
|
||||
if elapsed > wait_lsn_timeout:
|
||||
raise RuntimeError("Timed out waiting for WAL redo")
|
||||
|
||||
tenant_status = ps_cli.tenant_status(tenant_id)
|
||||
tenant_status = ps_http.tenant_status(tenant_id)
|
||||
if tenant_status["state"]["slug"] == "Loading":
|
||||
log.debug(f"Tenant {tenant_id} is still loading, retrying")
|
||||
else:
|
||||
|
||||
Reference in New Issue
Block a user