diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index 3ed4621112..7fdd047c96 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -473,6 +473,14 @@ pub(crate) enum ShutdownError { AlreadyStopping, } +struct DeletionGuard(OwnedMutexGuard); + +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, + 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, timeline_id: TimelineId, timeline: Arc, - _guard: OwnedMutexGuard, + 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()) }; diff --git a/pageserver/src/tenant/remote_timeline_client.rs b/pageserver/src/tenant/remote_timeline_client.rs index 2c84c59dcb..8db2bc4eb2 100644 --- a/pageserver/src/tenant/remote_timeline_client.rs +++ b/pageserver/src/tenant/remote_timeline_client.rs @@ -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, diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 855896c832..d42fdf5e55 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -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, diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index a8610e24df..64c71d2a59 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -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( diff --git a/test_runner/fixtures/pageserver/http.py b/test_runner/fixtures/pageserver/http.py index f258a3a24d..5c4f5177d0 100644 --- a/test_runner/fixtures/pageserver/http.py +++ b/test_runner/fixtures/pageserver/http.py @@ -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 ) diff --git a/test_runner/fixtures/pageserver/utils.py b/test_runner/fixtures/pageserver/utils.py index 83880abc77..ad89ebad00 100644 --- a/test_runner/fixtures/pageserver/utils.py +++ b/test_runner/fixtures/pageserver/utils.py @@ -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) diff --git a/test_runner/regress/test_compatibility.py b/test_runner/regress/test_compatibility.py index 2635dbd93c..61f86dc3ce 100644 --- a/test_runner/regress/test_compatibility.py +++ b/test_runner/regress/test_compatibility.py @@ -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'}"] diff --git a/test_runner/regress/test_import.py b/test_runner/regress/test_import.py index 5c3948b027..141c69b230 100644 --- a/test_runner/regress/test_import.py +++ b/test_runner/regress/test_import.py @@ -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) diff --git a/test_runner/regress/test_remote_storage.py b/test_runner/regress/test_remote_storage.py index 11ac9e2555..f2b954a822 100644 --- a/test_runner/regress/test_remote_storage.py +++ b/test_runner/regress/test_remote_storage.py @@ -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() diff --git a/test_runner/regress/test_tenant_size.py b/test_runner/regress/test_tenant_size.py index e9dcd1e5cd..a0f9f854ed 100644 --- a/test_runner/regress/test_tenant_size.py +++ b/test_runner/regress/test_tenant_size.py @@ -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 diff --git a/test_runner/regress/test_tenant_tasks.py b/test_runner/regress/test_tenant_tasks.py index 21e4af4127..75e5c2c91c 100644 --- a/test_runner/regress/test_tenant_tasks.py +++ b/test_runner/regress/test_tenant_tasks.py @@ -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() diff --git a/test_runner/regress/test_tenants.py b/test_runner/regress/test_tenants.py index 4a1d659be3..4dbfa8bc1f 100644 --- a/test_runner/regress/test_tenants.py +++ b/test_runner/regress/test_tenants.py @@ -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( diff --git a/test_runner/regress/test_timeline_delete.py b/test_runner/regress/test_timeline_delete.py index 28b15d03ca..ddd9ffd755 100644 --- a/test_runner/regress/test_timeline_delete.py +++ b/test_runner/regress/test_timeline_delete.py @@ -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, diff --git a/test_runner/regress/test_timeline_size.py b/test_runner/regress/test_timeline_size.py index 1460172afe..5bdbc18927 100644 --- a/test_runner/regress/test_timeline_size.py +++ b/test_runner/regress/test_timeline_size.py @@ -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) diff --git a/test_runner/regress/test_wal_acceptor.py b/test_runner/regress/test_wal_acceptor.py index 8b595596cb..a837501678 100644 --- a/test_runner/regress/test_wal_acceptor.py +++ b/test_runner/regress/test_wal_acceptor.py @@ -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: