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:
Dmitry Rodionov
2023-06-15 17:30:12 +03:00
committed by GitHub
parent 76413a0fb8
commit 472cc17b7a
15 changed files with 184 additions and 138 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@@ -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'}"]

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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