From 17116f2ea941d6d5e821e81dd5f17c88edc968f3 Mon Sep 17 00:00:00 2001 From: Christian Schwarz Date: Tue, 4 Jun 2024 18:16:23 +0200 Subject: [PATCH] fix(pageserver): abort on duplicate layers, before doing damage (#7799) fixes https://github.com/neondatabase/neon/issues/7790 (duplicating most of the issue description here for posterity) # Background From the time before always-authoritative `index_part.json`, we had to handle duplicate layers. See the RFC for an illustration of how duplicate layers could happen: https://github.com/neondatabase/neon/blob/a8e6d259cb49d1bf156dfc2215b92c04d1e8a08f/docs/rfcs/027-crash-consistent-layer-map-through-index-part.md?plain=1#L41-L50 As of #5198 , we should not be exposed to that problem anymore. # Problem 1 We still have 1. [code in Pageserver](https://github.com/neondatabase/neon/blob/82960b2175211c0f666b91b5258c5e2253a245c7/pageserver/src/tenant/timeline.rs#L4502-L4521) than handles duplicate layers 2. [tests in the test suite](https://github.com/neondatabase/neon/blob/d9dcbffac37ccd3331ec9adcd12fd20ce0ea31aa/test_runner/regress/test_duplicate_layers.py#L15) that demonstrates the problem using a failpoint However, the test in the test suite doesn't use the failpoint to induce a crash that could legitimately happen in production. What is does instead is to return early with an `Ok()`, so that the code in Pageserver that handles duplicate layers (item 1) actually gets exercised. That "return early" would be a bug in the routine if it happened in production. So, the tests in the test suite are tests for their own sake, but don't serve to actually regress-test any production behavior. # Problem 2 Further, if production code _did_ (it nowawdays doesn't!) create a duplicate layer, the code in Pageserver that handles the condition (item 1 above) is too little and too late: * the code handles it by discarding the newer `struct Layer`; that's good. * however, on disk, we have already overwritten the old with the new layer file * the fact that we do it atomically doesn't matter because ... * if the new layer file is not bit-identical, then we have a cache coherency problem * PS PageCache block cache: caches old bit battern * blob_io offsets stored in variables, based on pre-overwrite bit pattern / offsets * => reading based on these offsets from the new file might yield different data than before # Solution - Remove the test suite code pertaining to Problem 1 - Move & rename test suite code that actually tests RFC-27 crash-consistent layer map. - Remove the Pageserver code that handles duplicate layers too late (Problem 1) - Use `RENAME_NOREPLACE` to prevent over-rename the file during `.finish()`, bail with an error if it happens (Problem 2) - This bailing prevents the caller from even trying to insert into the layer map, as they don't even get a `struct Layer` at hand. - Add `abort`s in the place where we have the layer map lock and check for duplicates (Problem 2) - Note again, we can't reach there because we bail from `.finish()` much earlier in the code. - Share the logic to clean up after failed `.finish()` between image layers and delta layers (drive-by cleanup) - This exposed that test `image_layer_rewrite` was overwriting layer files in place. Fix the test. # Future Work This PR adds a new failure scenario that was previously "papered over" by the overwriting of layers: 1. Start a compaction that will produce 3 layers: A, B, C 2. Layer A is `finish()`ed successfully. 3. Layer B fails mid-way at some `put_value()`. 4. Compaction bails out, sleeps 20s. 5. Some disk space gets freed in the meantime. 6. Compaction wakes from sleep, another iteration starts, it attempts to write Layer A again. But the `.finish()` **fails because A already exists on disk**. The failure in step 5 is new with this PR, and it **causes the compaction to get stuck**. Before, it would silently overwrite the file and "successfully" complete the second iteration. The mitigation for this is to `/reset` the tenant. --- libs/utils/src/fs_ext.rs | 3 + libs/utils/src/fs_ext/rename_noreplace.rs | 109 ++++++++++++++++++ pageserver/src/tenant.rs | 35 ++++-- .../src/tenant/storage_layer/delta_layer.rs | 35 +++--- .../src/tenant/storage_layer/image_layer.rs | 92 ++++++++++++--- pageserver/src/tenant/storage_layer/layer.rs | 7 +- pageserver/src/tenant/timeline/compaction.rs | 42 ------- test_runner/regress/test_compaction.py | 2 +- ...y => test_pageserver_crash_consistency.py} | 50 ++------ .../regress/test_pageserver_restart.py | 5 - .../regress/test_pageserver_secondary.py | 4 - 11 files changed, 252 insertions(+), 132 deletions(-) create mode 100644 libs/utils/src/fs_ext/rename_noreplace.rs rename test_runner/regress/{test_duplicate_layers.py => test_pageserver_crash_consistency.py} (66%) diff --git a/libs/utils/src/fs_ext.rs b/libs/utils/src/fs_ext.rs index 90ba348a02..8e53d2c79b 100644 --- a/libs/utils/src/fs_ext.rs +++ b/libs/utils/src/fs_ext.rs @@ -3,6 +3,9 @@ use std::{fs, io, path::Path}; use anyhow::Context; +mod rename_noreplace; +pub use rename_noreplace::rename_noreplace; + pub trait PathExt { /// Returns an error if `self` is not a directory. fn is_empty_dir(&self) -> io::Result; diff --git a/libs/utils/src/fs_ext/rename_noreplace.rs b/libs/utils/src/fs_ext/rename_noreplace.rs new file mode 100644 index 0000000000..897e30d7f1 --- /dev/null +++ b/libs/utils/src/fs_ext/rename_noreplace.rs @@ -0,0 +1,109 @@ +use nix::NixPath; + +/// Rename a file without replacing an existing file. +/// +/// This is a wrapper around platform-specific APIs. +pub fn rename_noreplace( + src: &P1, + dst: &P2, +) -> nix::Result<()> { + { + #[cfg(target_os = "linux")] + { + nix::fcntl::renameat2( + None, + src, + None, + dst, + nix::fcntl::RenameFlags::RENAME_NOREPLACE, + ) + } + #[cfg(target_os = "macos")] + { + let res = src.with_nix_path(|src| { + dst.with_nix_path(|dst| + // SAFETY: `src` and `dst` are valid C strings as per the NixPath trait and they outlive the call to renamex_np. + unsafe { + nix::libc::renamex_np(src.as_ptr(), dst.as_ptr(), nix::libc::RENAME_EXCL) + }) + })??; + nix::errno::Errno::result(res).map(drop) + } + #[cfg(not(any(target_os = "linux", target_os = "macos")))] + { + std::compile_error!("OS does not support no-replace renames"); + } + } +} + +#[cfg(test)] +mod test { + use std::{fs, path::PathBuf}; + + use super::*; + + fn testdir() -> camino_tempfile::Utf8TempDir { + match crate::env::var("NEON_UTILS_RENAME_NOREPLACE_TESTDIR") { + Some(path) => { + let path: camino::Utf8PathBuf = path; + camino_tempfile::tempdir_in(path).unwrap() + } + None => camino_tempfile::tempdir().unwrap(), + } + } + + #[test] + fn test_absolute_paths() { + let testdir = testdir(); + println!("testdir: {}", testdir.path()); + + let src = testdir.path().join("src"); + let dst = testdir.path().join("dst"); + + fs::write(&src, b"").unwrap(); + fs::write(&dst, b"").unwrap(); + + let src = src.canonicalize().unwrap(); + assert!(src.is_absolute()); + let dst = dst.canonicalize().unwrap(); + assert!(dst.is_absolute()); + + let result = rename_noreplace(&src, &dst); + assert_eq!(result.unwrap_err(), nix::Error::EEXIST); + } + + #[test] + fn test_relative_paths() { + let testdir = testdir(); + println!("testdir: {}", testdir.path()); + + // this is fine because we run in nextest => process per test + std::env::set_current_dir(testdir.path()).unwrap(); + + let src = PathBuf::from("src"); + let dst = PathBuf::from("dst"); + + fs::write(&src, b"").unwrap(); + fs::write(&dst, b"").unwrap(); + + let result = rename_noreplace(&src, &dst); + assert_eq!(result.unwrap_err(), nix::Error::EEXIST); + } + + #[test] + fn test_works_when_not_exists() { + let testdir = testdir(); + println!("testdir: {}", testdir.path()); + + let src = testdir.path().join("src"); + let dst = testdir.path().join("dst"); + + fs::write(&src, b"content").unwrap(); + + rename_noreplace(src.as_std_path(), dst.as_std_path()).unwrap(); + assert_eq!( + "content", + String::from_utf8(std::fs::read(&dst).unwrap()).unwrap() + ); + } +} diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index 19a0f59b2a..60cd5c9695 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -3865,6 +3865,9 @@ pub(crate) mod harness { pub fn create_custom( test_name: &'static str, tenant_conf: TenantConf, + tenant_id: TenantId, + shard_identity: ShardIdentity, + generation: Generation, ) -> anyhow::Result { setup_logging(); @@ -3877,8 +3880,12 @@ pub(crate) mod harness { // OK in a test. let conf: &'static PageServerConf = Box::leak(Box::new(conf)); - let tenant_id = TenantId::generate(); - let tenant_shard_id = TenantShardId::unsharded(tenant_id); + let shard = shard_identity.shard_index(); + let tenant_shard_id = TenantShardId { + tenant_id, + shard_number: shard.shard_number, + shard_count: shard.shard_count, + }; fs::create_dir_all(conf.tenant_path(&tenant_shard_id))?; fs::create_dir_all(conf.timelines_path(&tenant_shard_id))?; @@ -3896,8 +3903,8 @@ pub(crate) mod harness { conf, tenant_conf, tenant_shard_id, - generation: Generation::new(0xdeadbeef), - shard: ShardIndex::unsharded(), + generation, + shard, remote_storage, remote_fs_dir, deletion_queue, @@ -3912,8 +3919,15 @@ pub(crate) mod harness { compaction_period: Duration::ZERO, ..TenantConf::default() }; - - Self::create_custom(test_name, tenant_conf) + let tenant_id = TenantId::generate(); + let shard = ShardIdentity::unsharded(); + Self::create_custom( + test_name, + tenant_conf, + tenant_id, + shard, + Generation::new(0xdeadbeef), + ) } pub fn span(&self) -> tracing::Span { @@ -4037,6 +4051,7 @@ mod tests { use tests::storage_layer::ValuesReconstructState; use tests::timeline::{GetVectoredError, ShutdownMode}; use utils::bin_ser::BeSer; + use utils::id::TenantId; static TEST_KEY: Lazy = Lazy::new(|| Key::from_slice(&hex!("010000000033333333444444445500000001"))); @@ -4936,7 +4951,13 @@ mod tests { ..TenantConf::default() }; - let harness = TenantHarness::create_custom("test_get_vectored_key_gap", tenant_conf)?; + let harness = TenantHarness::create_custom( + "test_get_vectored_key_gap", + tenant_conf, + TenantId::generate(), + ShardIdentity::unsharded(), + Generation::new(0xdeadbeef), + )?; let (tenant, ctx) = harness.load().await; let mut current_key = Key::from_hex("010000000033333333444444445500000000").unwrap(); diff --git a/pageserver/src/tenant/storage_layer/delta_layer.rs b/pageserver/src/tenant/storage_layer/delta_layer.rs index 1b3802840f..999e2e8679 100644 --- a/pageserver/src/tenant/storage_layer/delta_layer.rs +++ b/pageserver/src/tenant/storage_layer/delta_layer.rs @@ -478,6 +478,23 @@ impl DeltaLayerWriterInner { key_end: Key, timeline: &Arc, ctx: &RequestContext, + ) -> anyhow::Result { + let temp_path = self.path.clone(); + let result = self.finish0(key_end, timeline, ctx).await; + if result.is_err() { + tracing::info!(%temp_path, "cleaning up temporary file after error during writing"); + if let Err(e) = std::fs::remove_file(&temp_path) { + tracing::warn!(error=%e, %temp_path, "error cleaning up temporary layer file after error during writing"); + } + } + result + } + + async fn finish0( + self, + key_end: Key, + timeline: &Arc, + ctx: &RequestContext, ) -> anyhow::Result { let index_start_blk = ((self.blob_writer.size() + PAGE_SZ as u64 - 1) / PAGE_SZ as u64) as u32; @@ -651,19 +668,11 @@ impl DeltaLayerWriter { timeline: &Arc, ctx: &RequestContext, ) -> anyhow::Result { - let inner = self.inner.take().unwrap(); - let temp_path = inner.path.clone(); - let result = inner.finish(key_end, timeline, ctx).await; - // The delta layer files can sometimes be really large. Clean them up. - if result.is_err() { - tracing::warn!( - "Cleaning up temporary delta file {temp_path} after error during writing" - ); - if let Err(e) = std::fs::remove_file(&temp_path) { - tracing::warn!("Error cleaning up temporary delta layer file {temp_path}: {e:?}") - } - } - result + self.inner + .take() + .unwrap() + .finish(key_end, timeline, ctx) + .await } } diff --git a/pageserver/src/tenant/storage_layer/image_layer.rs b/pageserver/src/tenant/storage_layer/image_layer.rs index 8394b33f19..285618b146 100644 --- a/pageserver/src/tenant/storage_layer/image_layer.rs +++ b/pageserver/src/tenant/storage_layer/image_layer.rs @@ -917,26 +917,57 @@ impl Drop for ImageLayerWriter { #[cfg(test)] mod test { + use std::time::Duration; + use bytes::Bytes; use pageserver_api::{ key::Key, shard::{ShardCount, ShardIdentity, ShardNumber, ShardStripeSize}, }; - use utils::{id::TimelineId, lsn::Lsn}; + use utils::{ + generation::Generation, + id::{TenantId, TimelineId}, + lsn::Lsn, + }; - use crate::{tenant::harness::TenantHarness, DEFAULT_PG_VERSION}; + use crate::{ + tenant::{config::TenantConf, harness::TenantHarness}, + DEFAULT_PG_VERSION, + }; use super::ImageLayerWriter; #[tokio::test] async fn image_layer_rewrite() { - let harness = TenantHarness::create("test_image_layer_rewrite").unwrap(); - let (tenant, ctx) = harness.load().await; - + let tenant_conf = TenantConf { + gc_period: Duration::ZERO, + compaction_period: Duration::ZERO, + ..TenantConf::default() + }; + let tenant_id = TenantId::generate(); + let mut gen = Generation::new(0xdead0001); + let mut get_next_gen = || { + let ret = gen; + gen = gen.next(); + ret + }; // The LSN at which we will create an image layer to filter let lsn = Lsn(0xdeadbeef0000); - let timeline_id = TimelineId::generate(); + + // + // Create an unsharded parent with a layer. + // + + let harness = TenantHarness::create_custom( + "test_image_layer_rewrite--parent", + tenant_conf.clone(), + tenant_id, + ShardIdentity::unsharded(), + get_next_gen(), + ) + .unwrap(); + let (tenant, ctx) = harness.load().await; let timeline = tenant .create_test_timeline(timeline_id, lsn, DEFAULT_PG_VERSION, &ctx) .await @@ -971,9 +1002,47 @@ mod test { }; let original_size = resident.metadata().file_size; + // + // Create child shards and do the rewrite, exercising filter(). + // TODO: abstraction in TenantHarness for splits. + // + // Filter for various shards: this exercises cases like values at start of key range, end of key // range, middle of key range. - for shard_number in 0..4 { + let shard_count = ShardCount::new(4); + for shard_number in 0..shard_count.count() { + // + // mimic the shard split + // + let shard_identity = ShardIdentity::new( + ShardNumber(shard_number), + shard_count, + ShardStripeSize(0x8000), + ) + .unwrap(); + let harness = TenantHarness::create_custom( + Box::leak(Box::new(format!( + "test_image_layer_rewrite--child{}", + shard_identity.shard_slug() + ))), + tenant_conf.clone(), + tenant_id, + shard_identity, + // NB: in reality, the shards would each fork off their own gen number sequence from the parent. + // But here, all we care about is that the gen number is unique. + get_next_gen(), + ) + .unwrap(); + let (tenant, ctx) = harness.load().await; + let timeline = tenant + .create_test_timeline(timeline_id, lsn, DEFAULT_PG_VERSION, &ctx) + .await + .unwrap(); + + // + // use filter() and make assertions + // + let mut filtered_writer = ImageLayerWriter::new( harness.conf, timeline_id, @@ -985,15 +1054,6 @@ mod test { .await .unwrap(); - // TenantHarness gave us an unsharded tenant, but we'll use a sharded ShardIdentity - // to exercise filter() - let shard_identity = ShardIdentity::new( - ShardNumber(shard_number), - ShardCount::new(4), - ShardStripeSize(0x8000), - ) - .unwrap(); - let wrote_keys = resident .filter(&shard_identity, &mut filtered_writer, &ctx) .await diff --git a/pageserver/src/tenant/storage_layer/layer.rs b/pageserver/src/tenant/storage_layer/layer.rs index 1ec13882da..18f9ba4ef8 100644 --- a/pageserver/src/tenant/storage_layer/layer.rs +++ b/pageserver/src/tenant/storage_layer/layer.rs @@ -277,9 +277,10 @@ impl Layer { let downloaded = resident.expect("just initialized"); - // if the rename works, the path is as expected - // TODO: sync system call - std::fs::rename(temp_path, owner.local_path()) + // We never want to overwrite an existing file, so we use `RENAME_NOREPLACE`. + // TODO: this leaves the temp file in place if the rename fails, risking us running + // out of space. Should we clean it up here or does the calling context deal with this? + utils::fs_ext::rename_noreplace(temp_path.as_std_path(), owner.local_path().as_std_path()) .with_context(|| format!("rename temporary file as correct path for {owner}"))?; Ok(ResidentLayer { downloaded, owner }) diff --git a/pageserver/src/tenant/timeline/compaction.rs b/pageserver/src/tenant/timeline/compaction.rs index 15c77d0316..d8de6aee7c 100644 --- a/pageserver/src/tenant/timeline/compaction.rs +++ b/pageserver/src/tenant/timeline/compaction.rs @@ -421,48 +421,6 @@ impl Timeline { return Ok(CompactLevel0Phase1Result::default()); } - // This failpoint is used together with `test_duplicate_layers` integration test. - // It returns the compaction result exactly the same layers as input to compaction. - // We want to ensure that this will not cause any problem when updating the layer map - // after the compaction is finished. - // - // Currently, there are two rare edge cases that will cause duplicated layers being - // inserted. - // 1. The compaction job is inturrupted / did not finish successfully. Assume we have file 1, 2, 3, 4, which - // is compacted to 5, but the page server is shut down, next time we start page server we will get a layer - // map containing 1, 2, 3, 4, and 5, whereas 5 has the same content as 4. If we trigger L0 compation at this - // point again, it is likely that we will get a file 6 which has the same content and the key range as 5, - // and this causes an overwrite. This is acceptable because the content is the same, and we should do a - // layer replace instead of the normal remove / upload process. - // 2. The input workload pattern creates exactly n files that are sorted, non-overlapping and is of target file - // size length. Compaction will likely create the same set of n files afterwards. - // - // This failpoint is a superset of both of the cases. - if cfg!(feature = "testing") { - let active = (|| { - ::fail::fail_point!("compact-level0-phase1-return-same", |_| true); - false - })(); - - if active { - let mut new_layers = Vec::with_capacity(level0_deltas.len()); - for delta in &level0_deltas { - // we are just faking these layers as being produced again for this failpoint - new_layers.push( - delta - .download_and_keep_resident() - .await - .context("download layer for failpoint")?, - ); - } - tracing::info!("compact-level0-phase1-return-same"); // so that we can check if we hit the failpoint - return Ok(CompactLevel0Phase1Result { - new_layers, - deltas_to_compact: level0_deltas, - }); - } - } - // Gather the files to compact in this iteration. // // Start with the oldest Level 0 delta file, and collect any other diff --git a/test_runner/regress/test_compaction.py b/test_runner/regress/test_compaction.py index b2e4d35cb8..49dcb9b86a 100644 --- a/test_runner/regress/test_compaction.py +++ b/test_runner/regress/test_compaction.py @@ -238,7 +238,7 @@ def test_uploads_and_deletions( # https://github.com/neondatabase/neon/issues/7707 # https://github.com/neondatabase/neon/issues/7759 allowed_errors = [ - ".*duplicated L1 layer.*", + ".*/checkpoint.*rename temporary file as correct path for.*", # EEXIST ".*delta layer created with.*duplicate values.*", ".*assertion failed: self.lsn_range.start <= lsn.*", ".*HTTP request handler task panicked: task.*panicked.*", diff --git a/test_runner/regress/test_duplicate_layers.py b/test_runner/regress/test_pageserver_crash_consistency.py similarity index 66% rename from test_runner/regress/test_duplicate_layers.py rename to test_runner/regress/test_pageserver_crash_consistency.py index 0ebb99c712..3831d2f917 100644 --- a/test_runner/regress/test_duplicate_layers.py +++ b/test_runner/regress/test_pageserver_crash_consistency.py @@ -12,42 +12,14 @@ from fixtures.remote_storage import LocalFsStorage, RemoteStorageKind from requests.exceptions import ConnectionError -def test_duplicate_layers(neon_env_builder: NeonEnvBuilder, pg_bin: PgBin): - env = neon_env_builder.init_start() - pageserver_http = env.pageserver.http_client() - - # use a failpoint to return all L0s as L1s - message = ".*duplicated L1 layer layer=.*" - env.pageserver.allowed_errors.append(message) - - # Use aggressive compaction and checkpoint settings - tenant_id, _ = env.neon_cli.create_tenant( - conf={ - "checkpoint_distance": f"{1024 ** 2}", - "compaction_target_size": f"{1024 ** 2}", - "compaction_period": "5 s", - "compaction_threshold": "3", - } - ) - - pageserver_http.configure_failpoints(("compact-level0-phase1-return-same", "return")) - - endpoint = env.endpoints.create_start("main", tenant_id=tenant_id) - connstr = endpoint.connstr(options="-csynchronous_commit=off") - pg_bin.run_capture(["pgbench", "-i", "-s1", connstr]) - - time.sleep(10) # let compaction to be performed - env.pageserver.assert_log_contains("compact-level0-phase1-return-same") - - -def test_actually_duplicated_l1(neon_env_builder: NeonEnvBuilder, pg_bin: PgBin): +def test_local_only_layers_after_crash(neon_env_builder: NeonEnvBuilder, pg_bin: PgBin): """ - Test sets fail point at the end of first compaction phase: after - flushing new L1 layer but before deletion of L0 layers. + Test case for docs/rfcs/027-crash-consistent-layer-map-through-index-part.md. - The L1 used to be overwritten, but with crash-consistency via remote - index_part.json, we end up deleting the not yet uploaded L1 layer on - startup. + Simulate crash after compaction has written layers to disk + but before they have been uploaded/linked into remote index_part.json. + + Startup handles this situation by deleting the not yet uploaded L1 layer files. """ neon_env_builder.enable_pageserver_remote_storage(RemoteStorageKind.LOCAL_FS) @@ -126,13 +98,6 @@ def test_actually_duplicated_l1(neon_env_builder: NeonEnvBuilder, pg_bin: PgBin) # give time for log flush time.sleep(1) - message = f".*duplicated L1 layer layer={l1_found}" - found_msg = env.pageserver.log_contains(message) - # resident or evicted, it should not be overwritten, however it should had been non-existing at startup - assert ( - found_msg is None - ), "layer should had been removed during startup, did it live on as evicted?" - assert env.pageserver.layer_exists(tenant_id, timeline_id, l1_found), "the L1 reappears" wait_for_upload_queue_empty(pageserver_http, tenant_id, timeline_id) @@ -141,3 +106,6 @@ def test_actually_duplicated_l1(neon_env_builder: NeonEnvBuilder, pg_bin: PgBin) tenant_id, timeline_id, l1_found.to_str() ) assert uploaded.exists(), "the L1 is uploaded" + + +# TODO: same test for L0s produced by ingest. diff --git a/test_runner/regress/test_pageserver_restart.py b/test_runner/regress/test_pageserver_restart.py index 759e845927..4ce53df214 100644 --- a/test_runner/regress/test_pageserver_restart.py +++ b/test_runner/regress/test_pageserver_restart.py @@ -163,11 +163,6 @@ def test_pageserver_chaos( env = neon_env_builder.init_start(initial_tenant_shard_count=shard_count) - # these can happen, if we shutdown at a good time. to be fixed as part of #5172. - message = ".*duplicated L1 layer layer=.*" - for ps in env.pageservers: - ps.allowed_errors.append(message) - # Use a tiny checkpoint distance, to create a lot of layers quickly. # That allows us to stress the compaction and layer flushing logic more. tenant, _ = env.neon_cli.create_tenant( diff --git a/test_runner/regress/test_pageserver_secondary.py b/test_runner/regress/test_pageserver_secondary.py index 9b9bdb2b08..5bfa9cce8c 100644 --- a/test_runner/regress/test_pageserver_secondary.py +++ b/test_runner/regress/test_pageserver_secondary.py @@ -100,10 +100,6 @@ def test_location_conf_churn(neon_env_builder: NeonEnvBuilder, seed: int): ] ) - # these can happen, if we shutdown at a good time. to be fixed as part of #5172. - message = ".*duplicated L1 layer layer=.*" - ps.allowed_errors.append(message) - workload = Workload(env, tenant_id, timeline_id) workload.init(env.pageservers[0].id) workload.write_rows(256, env.pageservers[0].id)