diff --git a/libs/utils/src/lib.rs b/libs/utils/src/lib.rs index 160e082833..b3e38cd9e1 100644 --- a/libs/utils/src/lib.rs +++ b/libs/utils/src/lib.rs @@ -73,6 +73,8 @@ pub mod completion; /// Reporting utilities pub mod error; +pub mod sync; + /// This is a shortcut to embed git sha into binaries and avoid copying the same build script to all packages /// /// we have several cases: diff --git a/libs/utils/src/sync.rs b/libs/utils/src/sync.rs new file mode 100644 index 0000000000..125eeca129 --- /dev/null +++ b/libs/utils/src/sync.rs @@ -0,0 +1 @@ +pub mod heavier_once_cell; diff --git a/libs/utils/src/sync/heavier_once_cell.rs b/libs/utils/src/sync/heavier_once_cell.rs new file mode 100644 index 0000000000..4d66a54c98 --- /dev/null +++ b/libs/utils/src/sync/heavier_once_cell.rs @@ -0,0 +1,306 @@ +use std::sync::{Arc, Mutex, MutexGuard}; +use tokio::sync::Semaphore; + +/// Custom design like [`tokio::sync::OnceCell`] but using [`OwnedSemaphorePermit`] instead of +/// `SemaphorePermit`, allowing use of `take` which does not require holding an outer mutex guard +/// for the duration of initialization. +/// +/// Has no unsafe, builds upon [`tokio::sync::Semaphore`] and [`std::sync::Mutex`]. +/// +/// [`OwnedSemaphorePermit`]: tokio::sync::OwnedSemaphorePermit +pub struct OnceCell { + inner: Mutex>, +} + +impl Default for OnceCell { + /// Create new uninitialized [`OnceCell`]. + fn default() -> Self { + Self { + inner: Default::default(), + } + } +} + +/// Semaphore is the current state: +/// - open semaphore means the value is `None`, not yet initialized +/// - closed semaphore means the value has been initialized +#[derive(Debug)] +struct Inner { + init_semaphore: Arc, + value: Option, +} + +impl Default for Inner { + fn default() -> Self { + Self { + init_semaphore: Arc::new(Semaphore::new(1)), + value: None, + } + } +} + +impl OnceCell { + /// Creates an already initialized `OnceCell` with the given value. + pub fn new(value: T) -> Self { + let sem = Semaphore::new(1); + sem.close(); + Self { + inner: Mutex::new(Inner { + init_semaphore: Arc::new(sem), + value: Some(value), + }), + } + } + + /// Returns a guard to an existing initialized value, or uniquely initializes the value before + /// returning the guard. + /// + /// Initializing might wait on any existing [`Guard::take_and_deinit`] deinitialization. + /// + /// Initialization is panic-safe and cancellation-safe. + pub async fn get_or_init(&self, factory: F) -> Result, E> + where + F: FnOnce() -> Fut, + Fut: std::future::Future>, + { + let sem = { + let guard = self.inner.lock().unwrap(); + if guard.value.is_some() { + return Ok(Guard(guard)); + } + guard.init_semaphore.clone() + }; + + let permit = sem.acquire_owned().await; + if permit.is_err() { + let guard = self.inner.lock().unwrap(); + assert!( + guard.value.is_some(), + "semaphore got closed, must be initialized" + ); + return Ok(Guard(guard)); + } else { + // now we try + let value = factory().await?; + + let mut guard = self.inner.lock().unwrap(); + assert!( + guard.value.is_none(), + "we won permit, must not be initialized" + ); + guard.value = Some(value); + guard.init_semaphore.close(); + Ok(Guard(guard)) + } + } + + /// Returns a guard to an existing initialized value, if any. + pub fn get(&self) -> Option> { + let guard = self.inner.lock().unwrap(); + if guard.value.is_some() { + Some(Guard(guard)) + } else { + None + } + } +} + +/// Uninteresting guard object to allow short-lived access to inspect or clone the held, +/// initialized value. +#[derive(Debug)] +pub struct Guard<'a, T>(MutexGuard<'a, Inner>); + +impl std::ops::Deref for Guard<'_, T> { + type Target = T; + + fn deref(&self) -> &Self::Target { + self.0 + .value + .as_ref() + .expect("guard is not created unless value has been initialized") + } +} + +impl std::ops::DerefMut for Guard<'_, T> { + fn deref_mut(&mut self) -> &mut Self::Target { + self.0 + .value + .as_mut() + .expect("guard is not created unless value has been initialized") + } +} + +impl<'a, T> Guard<'a, T> { + /// Take the current value, and a new permit for it's deinitialization. + /// + /// The permit will be on a semaphore part of the new internal value, and any following + /// [`OnceCell::get_or_init`] will wait on it to complete. + pub fn take_and_deinit(&mut self) -> (T, tokio::sync::OwnedSemaphorePermit) { + let mut swapped = Inner::default(); + let permit = swapped + .init_semaphore + .clone() + .try_acquire_owned() + .expect("we just created this"); + std::mem::swap(&mut *self.0, &mut swapped); + swapped + .value + .map(|v| (v, permit)) + .expect("guard is not created unless value has been initialized") + } +} + +#[cfg(test)] +mod tests { + use super::*; + use std::{ + convert::Infallible, + sync::atomic::{AtomicUsize, Ordering}, + time::Duration, + }; + + #[tokio::test] + async fn many_initializers() { + #[derive(Default, Debug)] + struct Counters { + factory_got_to_run: AtomicUsize, + future_polled: AtomicUsize, + winners: AtomicUsize, + } + + let initializers = 100; + + let cell = Arc::new(OnceCell::default()); + let counters = Arc::new(Counters::default()); + let barrier = Arc::new(tokio::sync::Barrier::new(initializers + 1)); + + let mut js = tokio::task::JoinSet::new(); + for i in 0..initializers { + js.spawn({ + let cell = cell.clone(); + let counters = counters.clone(); + let barrier = barrier.clone(); + + async move { + barrier.wait().await; + let won = { + let g = cell + .get_or_init(|| { + counters.factory_got_to_run.fetch_add(1, Ordering::Relaxed); + async { + counters.future_polled.fetch_add(1, Ordering::Relaxed); + Ok::<_, Infallible>(i) + } + }) + .await + .unwrap(); + + *g == i + }; + + if won { + counters.winners.fetch_add(1, Ordering::Relaxed); + } + } + }); + } + + barrier.wait().await; + + while let Some(next) = js.join_next().await { + next.expect("no panics expected"); + } + + let mut counters = Arc::try_unwrap(counters).unwrap(); + + assert_eq!(*counters.factory_got_to_run.get_mut(), 1); + assert_eq!(*counters.future_polled.get_mut(), 1); + assert_eq!(*counters.winners.get_mut(), 1); + } + + #[tokio::test(start_paused = true)] + async fn reinit_waits_for_deinit() { + // with the tokio::time paused, we will "sleep" for 1s while holding the reinitialization + let sleep_for = Duration::from_secs(1); + let initial = 42; + let reinit = 1; + let cell = Arc::new(OnceCell::new(initial)); + + let deinitialization_started = Arc::new(tokio::sync::Barrier::new(2)); + + let jh = tokio::spawn({ + let cell = cell.clone(); + let deinitialization_started = deinitialization_started.clone(); + async move { + let (answer, _permit) = cell.get().expect("initialized to value").take_and_deinit(); + assert_eq!(answer, initial); + + deinitialization_started.wait().await; + tokio::time::sleep(sleep_for).await; + } + }); + + deinitialization_started.wait().await; + + let started_at = tokio::time::Instant::now(); + cell.get_or_init(|| async { Ok::<_, Infallible>(reinit) }) + .await + .unwrap(); + + let elapsed = started_at.elapsed(); + assert!( + elapsed >= sleep_for, + "initialization should had taken at least the time time slept with permit" + ); + + jh.await.unwrap(); + + assert_eq!(*cell.get().unwrap(), reinit); + } + + #[tokio::test] + async fn initialization_attemptable_until_ok() { + let cell = OnceCell::default(); + + for _ in 0..10 { + cell.get_or_init(|| async { Err("whatever error") }) + .await + .unwrap_err(); + } + + let g = cell + .get_or_init(|| async { Ok::<_, Infallible>("finally success") }) + .await + .unwrap(); + assert_eq!(*g, "finally success"); + } + + #[tokio::test] + async fn initialization_is_cancellation_safe() { + let cell = OnceCell::default(); + + let barrier = tokio::sync::Barrier::new(2); + + let initializer = cell.get_or_init(|| async { + barrier.wait().await; + futures::future::pending::<()>().await; + + Ok::<_, Infallible>("never reached") + }); + + tokio::select! { + _ = initializer => { unreachable!("cannot complete; stuck in pending().await") }, + _ = barrier.wait() => {} + }; + + // now initializer is dropped + + assert!(cell.get().is_none()); + + let g = cell + .get_or_init(|| async { Ok::<_, Infallible>("now initialized") }) + .await + .unwrap(); + assert_eq!(*g, "now initialized"); + } +} diff --git a/pageserver/src/disk_usage_eviction_task.rs b/pageserver/src/disk_usage_eviction_task.rs index b4d73baa88..413c941bc4 100644 --- a/pageserver/src/disk_usage_eviction_task.rs +++ b/pageserver/src/disk_usage_eviction_task.rs @@ -60,7 +60,11 @@ use utils::serde_percent::Percent; use crate::{ config::PageServerConf, task_mgr::{self, TaskKind, BACKGROUND_RUNTIME}, - tenant::{self, storage_layer::PersistentLayer, timeline::EvictionError, Timeline}, + tenant::{ + self, + storage_layer::{AsLayerDesc, EvictionError, Layer}, + Timeline, + }, }; #[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)] @@ -108,7 +112,7 @@ pub fn launch_disk_usage_global_eviction_task( _ = background_jobs_barrier.wait() => { } }; - disk_usage_eviction_task(&state, task_config, storage, &conf.tenants_path(), cancel) + disk_usage_eviction_task(&state, task_config, &storage, &conf.tenants_path(), cancel) .await; Ok(()) }, @@ -121,7 +125,7 @@ pub fn launch_disk_usage_global_eviction_task( async fn disk_usage_eviction_task( state: &State, task_config: &DiskUsageEvictionTaskConfig, - storage: GenericRemoteStorage, + _storage: &GenericRemoteStorage, tenants_dir: &Utf8Path, cancel: CancellationToken, ) { @@ -145,14 +149,8 @@ async fn disk_usage_eviction_task( let start = Instant::now(); async { - let res = disk_usage_eviction_task_iteration( - state, - task_config, - &storage, - tenants_dir, - &cancel, - ) - .await; + let res = + disk_usage_eviction_task_iteration(state, task_config, tenants_dir, &cancel).await; match res { Ok(()) => {} @@ -183,13 +181,12 @@ pub trait Usage: Clone + Copy + std::fmt::Debug { async fn disk_usage_eviction_task_iteration( state: &State, task_config: &DiskUsageEvictionTaskConfig, - storage: &GenericRemoteStorage, tenants_dir: &Utf8Path, cancel: &CancellationToken, ) -> anyhow::Result<()> { let usage_pre = filesystem_level_usage::get(tenants_dir, task_config) .context("get filesystem-level disk usage before evictions")?; - let res = disk_usage_eviction_task_iteration_impl(state, storage, usage_pre, cancel).await; + let res = disk_usage_eviction_task_iteration_impl(state, usage_pre, cancel).await; match res { Ok(outcome) => { debug!(?outcome, "disk_usage_eviction_iteration finished"); @@ -273,7 +270,6 @@ struct LayerCount { pub async fn disk_usage_eviction_task_iteration_impl( state: &State, - storage: &GenericRemoteStorage, usage_pre: U, cancel: &CancellationToken, ) -> anyhow::Result> { @@ -330,9 +326,10 @@ pub async fn disk_usage_eviction_task_iteration_impl( // If we get far enough in the list that we start to evict layers that are below // the tenant's min-resident-size threshold, print a warning, and memorize the disk // usage at that point, in 'usage_planned_min_resident_size_respecting'. - let mut batched: HashMap<_, Vec>> = HashMap::new(); + let mut batched: HashMap<_, Vec<_>> = HashMap::new(); let mut warned = None; let mut usage_planned = usage_pre; + let mut max_batch_size = 0; for (i, (partition, candidate)) in candidates.into_iter().enumerate() { if !usage_planned.has_pressure() { debug!( @@ -349,10 +346,18 @@ pub async fn disk_usage_eviction_task_iteration_impl( usage_planned.add_available_bytes(candidate.layer.layer_desc().file_size); - batched - .entry(TimelineKey(candidate.timeline)) - .or_default() - .push(candidate.layer); + // FIXME: batching makes no sense anymore because of no layermap locking, should just spawn + // tasks to evict all seen layers until we have evicted enough + + let batch = batched.entry(TimelineKey(candidate.timeline)).or_default(); + + // semaphore will later be used to limit eviction concurrency, and we can express at + // most u32 number of permits. unlikely we would have u32::MAX layers to be evicted, + // but fail gracefully by not making batches larger. + if batch.len() < u32::MAX as usize { + batch.push(candidate.layer); + max_batch_size = max_batch_size.max(batch.len()); + } } let usage_planned = match warned { @@ -369,69 +374,101 @@ pub async fn disk_usage_eviction_task_iteration_impl( // phase2: evict victims batched by timeline - // After the loop, `usage_assumed` is the post-eviction usage, - // according to internal accounting. - let mut usage_assumed = usage_pre; - let mut evictions_failed = LayerCount::default(); + let mut js = tokio::task::JoinSet::new(); + + // ratelimit to 1k files or any higher max batch size + let limit = Arc::new(tokio::sync::Semaphore::new(1000.max(max_batch_size))); + for (timeline, batch) in batched { let tenant_id = timeline.tenant_id; let timeline_id = timeline.timeline_id; - let batch_size = batch.len(); + let batch_size = + u32::try_from(batch.len()).expect("batch size limited to u32::MAX during partitioning"); + + // I dislike naming of `available_permits` but it means current total amount of permits + // because permits can be added + assert!(batch_size as usize <= limit.available_permits()); debug!(%timeline_id, "evicting batch for timeline"); - async { - let results = timeline.evict_layers(storage, &batch, cancel.clone()).await; + let evict = { + let limit = limit.clone(); + let cancel = cancel.clone(); + async move { + let mut evicted_bytes = 0; + let mut evictions_failed = LayerCount::default(); - match results { - Err(e) => { - warn!("failed to evict batch: {:#}", e); - } - Ok(results) => { - assert_eq!(results.len(), batch.len()); - for (result, layer) in results.into_iter().zip(batch.iter()) { - let file_size = layer.layer_desc().file_size; - match result { - Some(Ok(())) => { - usage_assumed.add_available_bytes(file_size); - } - Some(Err(EvictionError::CannotEvictRemoteLayer)) => { - unreachable!("get_local_layers_for_disk_usage_eviction finds only local layers") - } - Some(Err(EvictionError::FileNotFound)) => { - evictions_failed.file_sizes += file_size; - evictions_failed.count += 1; - } - Some(Err( - e @ EvictionError::LayerNotFound(_) - | e @ EvictionError::StatFailed(_), - )) => { - let e = utils::error::report_compact_sources(&e); - warn!(%layer, "failed to evict layer: {e}"); - evictions_failed.file_sizes += file_size; - evictions_failed.count += 1; - } - Some(Err(EvictionError::MetadataInconsistency(detail))) => { - warn!(%layer, "failed to evict layer: {detail}"); - evictions_failed.file_sizes += file_size; - evictions_failed.count += 1; - } - None => { - assert!(cancel.is_cancelled()); - return; + let Ok(_permit) = limit.acquire_many_owned(batch_size).await else { + // semaphore closing means cancelled + return (evicted_bytes, evictions_failed); + }; + + let results = timeline.evict_layers(&batch, &cancel).await; + + match results { + Ok(results) => { + assert_eq!(results.len(), batch.len()); + for (result, layer) in results.into_iter().zip(batch.iter()) { + let file_size = layer.layer_desc().file_size; + match result { + Some(Ok(())) => { + evicted_bytes += file_size; + } + Some(Err(EvictionError::NotFound | EvictionError::Downloaded)) => { + evictions_failed.file_sizes += file_size; + evictions_failed.count += 1; + } + None => { + assert!(cancel.is_cancelled()); + } } } } + Err(e) => { + warn!("failed to evict batch: {:#}", e); + } } + (evicted_bytes, evictions_failed) } } - .instrument(tracing::info_span!("evict_batch", %tenant_id, %timeline_id, batch_size)) - .await; + .instrument(tracing::info_span!("evict_batch", %tenant_id, %timeline_id, batch_size)); - if cancel.is_cancelled() { + js.spawn(evict); + + // spwaning multiple thousands of these is essentially blocking, so give already spawned a + // chance of making progress + tokio::task::yield_now().await; + } + + let join_all = async move { + // After the evictions, `usage_assumed` is the post-eviction usage, + // according to internal accounting. + let mut usage_assumed = usage_pre; + let mut evictions_failed = LayerCount::default(); + + while let Some(res) = js.join_next().await { + match res { + Ok((evicted_bytes, failed)) => { + usage_assumed.add_available_bytes(evicted_bytes); + evictions_failed.file_sizes += failed.file_sizes; + evictions_failed.count += failed.count; + } + Err(je) if je.is_cancelled() => unreachable!("not used"), + Err(je) if je.is_panic() => { /* already logged */ } + Err(je) => tracing::error!("unknown JoinError: {je:?}"), + } + } + (usage_assumed, evictions_failed) + }; + + let (usage_assumed, evictions_failed) = tokio::select! { + tuple = join_all => { tuple }, + _ = cancel.cancelled() => { + // close the semaphore to stop any pending acquires + limit.close(); return Ok(IterationOutcome::Cancelled); } - } + }; Ok(IterationOutcome::Finished(IterationOutcomeFinished { before: usage_pre, @@ -446,7 +483,7 @@ pub async fn disk_usage_eviction_task_iteration_impl( #[derive(Clone)] struct EvictionCandidate { timeline: Arc, - layer: Arc, + layer: Layer, last_activity_ts: SystemTime, } diff --git a/pageserver/src/http/routes.rs b/pageserver/src/http/routes.rs index 54c5cdf133..2c46d733d6 100644 --- a/pageserver/src/http/routes.rs +++ b/pageserver/src/http/routes.rs @@ -1205,7 +1205,7 @@ async fn timeline_compact_handler( timeline .compact(&cancel, &ctx) .await - .map_err(ApiError::InternalServerError)?; + .map_err(|e| ApiError::InternalServerError(e.into()))?; json_response(StatusCode::OK, ()) } .instrument(info_span!("manual_compaction", %tenant_id, %timeline_id)) @@ -1230,7 +1230,7 @@ async fn timeline_checkpoint_handler( timeline .compact(&cancel, &ctx) .await - .map_err(ApiError::InternalServerError)?; + .map_err(|e| ApiError::InternalServerError(e.into()))?; json_response(StatusCode::OK, ()) } @@ -1500,11 +1500,11 @@ async fn disk_usage_eviction_run( let state = get_state(&r); - let Some(storage) = state.remote_storage.clone() else { + if state.remote_storage.as_ref().is_none() { return Err(ApiError::InternalServerError(anyhow::anyhow!( "remote storage not configured, cannot run eviction iteration" ))); - }; + } let state = state.disk_usage_eviction_state.clone(); @@ -1522,7 +1522,6 @@ async fn disk_usage_eviction_run( async move { let res = crate::disk_usage_eviction_task::disk_usage_eviction_task_iteration_impl( &state, - &storage, usage, &child_cancel, ) diff --git a/pageserver/src/metrics.rs b/pageserver/src/metrics.rs index 267e632312..3e15be67dc 100644 --- a/pageserver/src/metrics.rs +++ b/pageserver/src/metrics.rs @@ -1404,7 +1404,7 @@ impl TimelineMetrics { crate::metrics::RESIDENT_PHYSICAL_SIZE_GLOBAL.add(sz); } - pub fn resident_physical_size_get(&self) -> u64 { + pub(crate) fn resident_physical_size_get(&self) -> u64 { self.resident_physical_size_gauge.get() } } diff --git a/pageserver/src/tenant.rs b/pageserver/src/tenant.rs index f8895c32dc..f177a17afc 100644 --- a/pageserver/src/tenant.rs +++ b/pageserver/src/tenant.rs @@ -138,9 +138,7 @@ pub(crate) mod timeline; pub mod size; pub(crate) use timeline::span::debug_assert_current_span_has_tenant_and_timeline_id; -pub use timeline::{ - LocalLayerInfoForDiskUsageEviction, LogicalSizeCalculationCause, PageReconstructError, Timeline, -}; +pub(crate) use timeline::{LogicalSizeCalculationCause, PageReconstructError, Timeline}; // re-export for use in remote_timeline_client.rs pub use crate::tenant::metadata::save_metadata; @@ -4328,6 +4326,7 @@ mod tests { #[tokio::test] async fn delta_layer_dumping() -> anyhow::Result<()> { + use storage_layer::AsLayerDesc; let (tenant, ctx) = TenantHarness::create("test_layer_dumping")?.load().await; let tline = tenant .create_test_timeline(TIMELINE_ID, Lsn(0x10), DEFAULT_PG_VERSION, &ctx) @@ -4335,16 +4334,18 @@ mod tests { make_some_layers(tline.as_ref(), Lsn(0x20), &ctx).await?; let layer_map = tline.layers.read().await; - let level0_deltas = layer_map.layer_map().get_level0_deltas()?; + let level0_deltas = layer_map + .layer_map() + .get_level0_deltas()? + .into_iter() + .map(|desc| layer_map.get_from_desc(&desc)) + .collect::>(); assert!(!level0_deltas.is_empty()); for delta in level0_deltas { - let delta = layer_map.get_from_desc(&delta); // Ensure we are dumping a delta layer here - let delta = delta.downcast_delta_layer().unwrap(); - - delta.dump(false, &ctx).await.unwrap(); + assert!(delta.layer_desc().is_delta); delta.dump(true, &ctx).await.unwrap(); } diff --git a/pageserver/src/tenant/layer_map.rs b/pageserver/src/tenant/layer_map.rs index b8185da010..9b6225501f 100644 --- a/pageserver/src/tenant/layer_map.rs +++ b/pageserver/src/tenant/layer_map.rs @@ -639,147 +639,10 @@ impl LayerMap { } println!("historic_layers:"); - for layer in self.iter_historic_layers() { - layer.dump(verbose, ctx)?; + for desc in self.iter_historic_layers() { + desc.dump(); } println!("End dump LayerMap"); Ok(()) } } - -#[cfg(test)] -mod tests { - use super::LayerMap; - use crate::tenant::storage_layer::LayerFileName; - use std::str::FromStr; - use std::sync::Arc; - - mod l0_delta_layers_updated { - - use crate::tenant::{ - storage_layer::{AsLayerDesc, PersistentLayerDesc}, - timeline::layer_manager::LayerFileManager, - }; - - use super::*; - - struct LayerObject(PersistentLayerDesc); - - impl AsLayerDesc for LayerObject { - fn layer_desc(&self) -> &PersistentLayerDesc { - &self.0 - } - } - - impl LayerObject { - fn new(desc: PersistentLayerDesc) -> Self { - LayerObject(desc) - } - } - - type TestLayerFileManager = LayerFileManager; - - #[test] - fn for_full_range_delta() { - // l0_delta_layers are used by compaction, and should observe all buffered updates - l0_delta_layers_updated_scenario( - "000000000000000000000000000000000000-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF__0000000053423C21-0000000053424D69", - true - ) - } - - #[test] - fn for_non_full_range_delta() { - // has minimal uncovered areas compared to l0_delta_layers_updated_on_insert_replace_remove_for_full_range_delta - l0_delta_layers_updated_scenario( - "000000000000000000000000000000000001-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFE__0000000053423C21-0000000053424D69", - // because not full range - false - ) - } - - #[test] - fn for_image() { - l0_delta_layers_updated_scenario( - "000000000000000000000000000000000000-000000000000000000000000000000010000__0000000053424D69", - // code only checks if it is a full range layer, doesn't care about images, which must - // mean we should in practice never have full range images - false - ) - } - - #[test] - fn replacing_missing_l0_is_notfound() { - // original impl had an oversight, and L0 was an anyhow::Error. anyhow::Error should - // however only happen for precondition failures. - - let layer = "000000000000000000000000000000000000-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF__0000000053423C21-0000000053424D69"; - let layer = LayerFileName::from_str(layer).unwrap(); - let layer = PersistentLayerDesc::from(layer); - - // same skeletan construction; see scenario below - let not_found = Arc::new(LayerObject::new(layer.clone())); - let new_version = Arc::new(LayerObject::new(layer)); - - // after the immutable storage state refactor, the replace operation - // will not use layer map any more. We keep it here for consistency in test cases - // and can remove it in the future. - let _map = LayerMap::default(); - - let mut mapping = TestLayerFileManager::new(); - - mapping - .replace_and_verify(not_found, new_version) - .unwrap_err(); - } - - fn l0_delta_layers_updated_scenario(layer_name: &str, expected_l0: bool) { - let name = LayerFileName::from_str(layer_name).unwrap(); - let skeleton = PersistentLayerDesc::from(name); - - let remote = Arc::new(LayerObject::new(skeleton.clone())); - let downloaded = Arc::new(LayerObject::new(skeleton)); - - let mut map = LayerMap::default(); - let mut mapping = LayerFileManager::new(); - - // two disjoint Arcs in different lifecycle phases. even if it seems they must be the - // same layer, we use LayerMap::compare_arced_layers as the identity of layers. - assert_eq!(remote.layer_desc(), downloaded.layer_desc()); - - let expected_in_counts = (1, usize::from(expected_l0)); - - map.batch_update() - .insert_historic(remote.layer_desc().clone()); - mapping.insert(remote.clone()); - assert_eq!( - count_layer_in(&map, remote.layer_desc()), - expected_in_counts - ); - - mapping - .replace_and_verify(remote, downloaded.clone()) - .expect("name derived attributes are the same"); - assert_eq!( - count_layer_in(&map, downloaded.layer_desc()), - expected_in_counts - ); - - map.batch_update().remove_historic(downloaded.layer_desc()); - assert_eq!(count_layer_in(&map, downloaded.layer_desc()), (0, 0)); - } - - fn count_layer_in(map: &LayerMap, layer: &PersistentLayerDesc) -> (usize, usize) { - let historic = map - .iter_historic_layers() - .filter(|x| x.key() == layer.key()) - .count(); - let l0s = map - .get_level0_deltas() - .expect("why does this return a result"); - let l0 = l0s.iter().filter(|x| x.key() == layer.key()).count(); - - (historic, l0) - } - } -} diff --git a/pageserver/src/tenant/par_fsync.rs b/pageserver/src/tenant/par_fsync.rs index 3b1526e910..3acb0fb431 100644 --- a/pageserver/src/tenant/par_fsync.rs +++ b/pageserver/src/tenant/par_fsync.rs @@ -57,8 +57,7 @@ pub fn par_fsync(paths: &[Utf8PathBuf]) -> io::Result<()> { fsync_in_thread_pool(paths) } -/// Parallel fsync asynchronously. If number of files are less than PARALLEL_PATH_THRESHOLD, fsync is done in the current -/// execution thread. Otherwise, we will spawn_blocking and run it in tokio. +/// Parallel fsync asynchronously. pub async fn par_fsync_async(paths: &[Utf8PathBuf]) -> io::Result<()> { const MAX_CONCURRENT_FSYNC: usize = 64; let mut next = paths.iter().peekable(); diff --git a/pageserver/src/tenant/remote_timeline_client.rs b/pageserver/src/tenant/remote_timeline_client.rs index e8415ee9f3..245649c3ad 100644 --- a/pageserver/src/tenant/remote_timeline_client.rs +++ b/pageserver/src/tenant/remote_timeline_client.rs @@ -167,8 +167,6 @@ //! - download their remote [`IndexPart`]s //! - create `Timeline` struct and a `RemoteTimelineClient` //! - initialize the client's upload queue with its `IndexPart` -//! - create [`RemoteLayer`](super::storage_layer::RemoteLayer) instances -//! for layers that are referenced by `IndexPart` but not present locally //! - schedule uploads for layers that are only present locally. //! - if the remote `IndexPart`'s metadata was newer than the metadata in //! the local filesystem, write the remote metadata to the local filesystem @@ -211,8 +209,7 @@ mod upload; use anyhow::Context; use camino::Utf8Path; use chrono::{NaiveDateTime, Utc}; -// re-export these -pub use download::{is_temp_download_file, list_remote_timelines}; + use scopeguard::ScopeGuard; use tokio_util::sync::CancellationToken; use utils::backoff::{ @@ -237,7 +234,7 @@ use crate::metrics::{ }; use crate::task_mgr::shutdown_token; use crate::tenant::debug_assert_current_span_has_tenant_and_timeline_id; -use crate::tenant::remote_timeline_client::index::LayerFileMetadata; +use crate::tenant::storage_layer::AsLayerDesc; use crate::tenant::upload_queue::Delete; use crate::tenant::TIMELINES_SEGMENT_NAME; use crate::{ @@ -255,10 +252,13 @@ use utils::id::{TenantId, TimelineId}; use self::index::IndexPart; -use super::storage_layer::LayerFileName; +use super::storage_layer::{Layer, LayerFileName, ResidentLayer}; use super::upload_queue::SetDeletedFlagProgress; use super::Generation; +pub(crate) use download::{is_temp_download_file, list_remote_timelines}; +pub(crate) use index::LayerFileMetadata; + // Occasional network issues and such can cause remote operations to fail, and // that's expected. If a download fails, we log it at info-level, and retry. // But after FAILED_DOWNLOAD_WARN_THRESHOLD retries, we start to log it at WARN @@ -629,13 +629,12 @@ impl RemoteTimelineClient { /// pub(crate) fn schedule_layer_file_upload( self: &Arc, - layer_file_name: &LayerFileName, - layer_metadata: &LayerFileMetadata, + layer: ResidentLayer, ) -> anyhow::Result<()> { let mut guard = self.upload_queue.lock().unwrap(); let upload_queue = guard.initialized_mut()?; - self.schedule_layer_file_upload0(upload_queue, layer_file_name, layer_metadata); + self.schedule_layer_file_upload0(upload_queue, layer); self.launch_queued_tasks(upload_queue); Ok(()) } @@ -643,18 +642,19 @@ impl RemoteTimelineClient { fn schedule_layer_file_upload0( self: &Arc, upload_queue: &mut UploadQueueInitialized, - layer_file_name: &LayerFileName, - layer_metadata: &LayerFileMetadata, + layer: ResidentLayer, ) { + let metadata = layer.metadata(); + upload_queue .latest_files - .insert(layer_file_name.clone(), layer_metadata.clone()); + .insert(layer.layer_desc().filename(), metadata.clone()); upload_queue.latest_files_changes_since_metadata_upload_scheduled += 1; - let op = UploadOp::UploadLayer(layer_file_name.clone(), layer_metadata.clone()); + info!("scheduled layer file upload {layer}"); + let op = UploadOp::UploadLayer(layer, metadata); self.calls_unfinished_metric_begin(&op); upload_queue.queued_operations.push_back(op); - info!("scheduled layer file upload {layer_file_name}"); } /// Launch a delete operation in the background. @@ -667,13 +667,13 @@ impl RemoteTimelineClient { /// successfully. pub fn schedule_layer_file_deletion( self: &Arc, - names: Vec, + names: &[LayerFileName], ) -> anyhow::Result<()> { let mut guard = self.upload_queue.lock().unwrap(); let upload_queue = guard.initialized_mut()?; let with_generations = - self.schedule_unlinking_of_layers_from_index_part0(upload_queue, &names); + self.schedule_unlinking_of_layers_from_index_part0(upload_queue, names.iter().cloned()); self.schedule_deletion_of_unlinked0(upload_queue, with_generations); @@ -687,17 +687,17 @@ impl RemoteTimelineClient { /// /// The files will be leaked in remote storage unless [`Self::schedule_deletion_of_unlinked`] /// is invoked on them. - #[allow(unused)] // will be used by PR#4938 - pub(crate) fn schedule_unlinking_of_layers_from_index_part( - self: &Arc, - names: Vec, - ) -> anyhow::Result<()> { + pub(crate) fn schedule_gc_update(self: &Arc, gc_layers: &[Layer]) -> anyhow::Result<()> { let mut guard = self.upload_queue.lock().unwrap(); let upload_queue = guard.initialized_mut()?; // just forget the return value; after uploading the next index_part.json, we can consider - // the layer files as "dangling". this is fine however. - self.schedule_unlinking_of_layers_from_index_part0(upload_queue, &names); + // the layer files as "dangling". this is fine, at worst case we create work for the + // scrubber. + + let names = gc_layers.iter().map(|x| x.layer_desc().filename()); + + self.schedule_unlinking_of_layers_from_index_part0(upload_queue, names); self.launch_queued_tasks(upload_queue); @@ -706,26 +706,28 @@ impl RemoteTimelineClient { /// Update the remote index file, removing the to-be-deleted files from the index, /// allowing scheduling of actual deletions later. - fn schedule_unlinking_of_layers_from_index_part0( + fn schedule_unlinking_of_layers_from_index_part0( self: &Arc, upload_queue: &mut UploadQueueInitialized, - names: &[LayerFileName], - ) -> Vec<(LayerFileName, Generation)> { + names: I, + ) -> Vec<(LayerFileName, Generation)> + where + I: IntoIterator, + { // Deleting layers doesn't affect the values stored in TimelineMetadata, // so we don't need update it. Just serialize it. let metadata = upload_queue.latest_metadata.clone(); // Decorate our list of names with each name's generation, dropping - // makes that are unexpectedly missing from our metadata. + // names that are unexpectedly missing from our metadata. let with_generations: Vec<_> = names - .iter() + .into_iter() .filter_map(|name| { - // Remove from latest_files, learning the file's remote generation in the process - let meta = upload_queue.latest_files.remove(name); + let meta = upload_queue.latest_files.remove(&name); if let Some(meta) = meta { upload_queue.latest_files_changes_since_metadata_upload_scheduled += 1; - Some((name.to_owned(), meta.generation)) + Some((name, meta.generation)) } else { // This can only happen if we forgot to to schedule the file upload // before scheduling the delete. Log it because it is a rare/strange @@ -748,8 +750,7 @@ impl RemoteTimelineClient { } /// Schedules deletion for layer files which have previously been unlinked from the - /// `index_part.json` with [`Self::schedule_unlinking_of_layers_from_index_part`]. - #[allow(unused)] // will be used by Layer::drop in PR#4938 + /// `index_part.json` with [`Self::schedule_gc_update`] or [`Self::schedule_compaction_update`]. pub(crate) fn schedule_deletion_of_unlinked( self: &Arc, layers: Vec<(LayerFileName, Generation)>, @@ -784,18 +785,20 @@ impl RemoteTimelineClient { /// `compacted_from` represent the L0 names which have been `compacted_to` L1 layers. pub(crate) fn schedule_compaction_update( self: &Arc, - compacted_from: &[LayerFileName], - compacted_to: &[(LayerFileName, LayerFileMetadata)], + compacted_from: &[Layer], + compacted_to: &[ResidentLayer], ) -> anyhow::Result<()> { let mut guard = self.upload_queue.lock().unwrap(); let upload_queue = guard.initialized_mut()?; - for (name, m) in compacted_to { - self.schedule_layer_file_upload0(upload_queue, name, m); + for layer in compacted_to { + self.schedule_layer_file_upload0(upload_queue, layer.clone()); } + let names = compacted_from.iter().map(|x| x.layer_desc().filename()); + let with_generations = - self.schedule_unlinking_of_layers_from_index_part0(upload_queue, compacted_from); + self.schedule_unlinking_of_layers_from_index_part0(upload_queue, names); self.schedule_deletion_of_unlinked0(upload_queue, with_generations); self.launch_queued_tasks(upload_queue); @@ -1170,16 +1173,12 @@ impl RemoteTimelineClient { } let upload_result: anyhow::Result<()> = match &task.op { - UploadOp::UploadLayer(ref layer_file_name, ref layer_metadata) => { - let path = self - .conf - .timeline_path(&self.tenant_id, &self.timeline_id) - .join(layer_file_name.file_name()); - + UploadOp::UploadLayer(ref layer, ref layer_metadata) => { + let path = layer.local_path(); upload::upload_timeline_layer( self.conf, &self.storage_impl, - &path, + path, layer_metadata, self.generation, ) @@ -1496,13 +1495,6 @@ impl RemoteTimelineClient { } } } - - pub(crate) fn get_layer_metadata( - &self, - name: &LayerFileName, - ) -> anyhow::Result> { - self.upload_queue.lock().unwrap().get_layer_metadata(name) - } } pub fn remote_timelines_path(tenant_id: &TenantId) -> RemotePath { @@ -1590,6 +1582,7 @@ mod tests { context::RequestContext, tenant::{ harness::{TenantHarness, TIMELINE_ID}, + storage_layer::Layer, Generation, Tenant, Timeline, }, DEFAULT_PG_VERSION, @@ -1758,32 +1751,29 @@ mod tests { let generation = harness.generation; // Create a couple of dummy files, schedule upload for them - let layer_file_name_1: LayerFileName = "000000000000000000000000000000000000-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF__00000000016B59D8-00000000016B5A51".parse().unwrap(); - let layer_file_name_2: LayerFileName = "000000000000000000000000000000000000-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF__00000000016B59D9-00000000016B5A52".parse().unwrap(); - let layer_file_name_3: LayerFileName = "000000000000000000000000000000000000-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF__00000000016B59DA-00000000016B5A53".parse().unwrap(); - let content_1 = dummy_contents("foo"); - let content_2 = dummy_contents("bar"); - let content_3 = dummy_contents("baz"); - for (filename, content) in [ - (&layer_file_name_1, &content_1), - (&layer_file_name_2, &content_2), - (&layer_file_name_3, &content_3), - ] { - std::fs::write(timeline_path.join(filename.file_name()), content).unwrap(); - } + let layers = [ + ("000000000000000000000000000000000000-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF__00000000016B59D8-00000000016B5A51".parse().unwrap(), dummy_contents("foo")), + ("000000000000000000000000000000000000-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF__00000000016B59D9-00000000016B5A52".parse().unwrap(), dummy_contents("bar")), + ("000000000000000000000000000000000000-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF__00000000016B59DA-00000000016B5A53".parse().unwrap(), dummy_contents("baz")) + ] + .into_iter() + .map(|(name, contents): (LayerFileName, Vec)| { + std::fs::write(timeline_path.join(name.file_name()), &contents).unwrap(); + + Layer::for_resident( + harness.conf, + &timeline, + name, + LayerFileMetadata::new(contents.len() as u64, generation), + ) + }).collect::>(); client - .schedule_layer_file_upload( - &layer_file_name_1, - &LayerFileMetadata::new(content_1.len() as u64, generation), - ) + .schedule_layer_file_upload(layers[0].clone()) .unwrap(); client - .schedule_layer_file_upload( - &layer_file_name_2, - &LayerFileMetadata::new(content_2.len() as u64, generation), - ) + .schedule_layer_file_upload(layers[1].clone()) .unwrap(); // Check that they are started immediately, not queued @@ -1837,38 +1827,42 @@ mod tests { .collect(), &[ &initial_layer.file_name(), - &layer_file_name_1.file_name(), - &layer_file_name_2.file_name(), + &layers[0].layer_desc().filename().file_name(), + &layers[1].layer_desc().filename().file_name(), ], ); assert_eq!(index_part.metadata, metadata); // Schedule upload and then a deletion. Check that the deletion is queued client - .schedule_layer_file_upload( - &layer_file_name_3, - &LayerFileMetadata::new(content_3.len() as u64, generation), - ) + .schedule_layer_file_upload(layers[2].clone()) .unwrap(); + + // this is no longer consistent with how deletion works with Layer::drop, but in this test + // keep using schedule_layer_file_deletion because we don't have a way to wait for the + // spawn_blocking started by the drop. client - .schedule_layer_file_deletion([layer_file_name_1.clone()].to_vec()) + .schedule_layer_file_deletion(&[layers[0].layer_desc().filename()]) .unwrap(); { let mut guard = client.upload_queue.lock().unwrap(); let upload_queue = guard.initialized_mut().unwrap(); // Deletion schedules upload of the index file, and the file deletion itself - assert!(upload_queue.queued_operations.len() == 2); - assert!(upload_queue.inprogress_tasks.len() == 1); - assert!(upload_queue.num_inprogress_layer_uploads == 1); - assert!(upload_queue.num_inprogress_deletions == 0); - assert!(upload_queue.latest_files_changes_since_metadata_upload_scheduled == 0); + assert_eq!(upload_queue.queued_operations.len(), 2); + assert_eq!(upload_queue.inprogress_tasks.len(), 1); + assert_eq!(upload_queue.num_inprogress_layer_uploads, 1); + assert_eq!(upload_queue.num_inprogress_deletions, 0); + assert_eq!( + upload_queue.latest_files_changes_since_metadata_upload_scheduled, + 0 + ); } assert_remote_files( &[ &initial_layer.file_name(), - &layer_file_name_1.file_name(), - &layer_file_name_2.file_name(), + &layers[0].layer_desc().filename().file_name(), + &layers[1].layer_desc().filename().file_name(), "index_part.json", ], &remote_timeline_dir, @@ -1882,8 +1876,8 @@ mod tests { assert_remote_files( &[ &initial_layer.file_name(), - &layer_file_name_2.file_name(), - &layer_file_name_3.file_name(), + &layers[1].layer_desc().filename().file_name(), + &layers[2].layer_desc().filename().file_name(), "index_part.json", ], &remote_timeline_dir, @@ -1912,6 +1906,13 @@ mod tests { ) .unwrap(); + let layer_file_1 = Layer::for_resident( + harness.conf, + &timeline, + layer_file_name_1.clone(), + LayerFileMetadata::new(content_1.len() as u64, harness.generation), + ); + #[derive(Debug, PartialEq, Clone, Copy)] struct BytesStartedFinished { started: Option, @@ -1947,10 +1948,7 @@ mod tests { let actual_a = get_bytes_started_stopped(); client - .schedule_layer_file_upload( - &layer_file_name_1, - &LayerFileMetadata::new(content_1.len() as u64, harness.generation), - ) + .schedule_layer_file_upload(layer_file_1.clone()) .unwrap(); let actual_b = get_bytes_started_stopped(); diff --git a/pageserver/src/tenant/remote_timeline_client/upload.rs b/pageserver/src/tenant/remote_timeline_client/upload.rs index 03ba137566..0a37a8f283 100644 --- a/pageserver/src/tenant/remote_timeline_client/upload.rs +++ b/pageserver/src/tenant/remote_timeline_client/upload.rs @@ -72,6 +72,8 @@ pub(super) async fn upload_timeline_layer<'a>( // upload. However, a nonexistent file can also be indicative of // something worse, like when a file is scheduled for upload before // it has been written to disk yet. + // + // This is tested against `test_compaction_delete_before_upload` info!(path = %source_path, "File to upload doesn't exist. Likely the file has been deleted and an upload is not required any more."); return Ok(()); } diff --git a/pageserver/src/tenant/storage_layer.rs b/pageserver/src/tenant/storage_layer.rs index b3aacb20d2..3b2a61dcba 100644 --- a/pageserver/src/tenant/storage_layer.rs +++ b/pageserver/src/tenant/storage_layer.rs @@ -4,26 +4,21 @@ pub mod delta_layer; mod filename; mod image_layer; mod inmemory_layer; +mod layer; mod layer_desc; -mod remote_layer; -use crate::config::PageServerConf; use crate::context::{AccessStatsBehavior, RequestContext}; -use crate::repository::Key; use crate::task_mgr::TaskKind; use crate::walrecord::NeonWalRecord; -use anyhow::Result; use bytes::Bytes; -use camino::Utf8PathBuf; use enum_map::EnumMap; use enumset::EnumSet; use once_cell::sync::Lazy; -use pageserver_api::models::LayerAccessKind; use pageserver_api::models::{ - HistoricLayerInfo, LayerResidenceEvent, LayerResidenceEventReason, LayerResidenceStatus, + LayerAccessKind, LayerResidenceEvent, LayerResidenceEventReason, LayerResidenceStatus, }; use std::ops::Range; -use std::sync::{Arc, Mutex}; +use std::sync::Mutex; use std::time::{Duration, SystemTime, UNIX_EPOCH}; use tracing::warn; use utils::history_buffer::HistoryBufferWithDropCounter; @@ -39,7 +34,8 @@ pub use filename::{DeltaFileName, ImageFileName, LayerFileName}; pub use image_layer::{ImageLayer, ImageLayerWriter}; pub use inmemory_layer::InMemoryLayer; pub use layer_desc::{PersistentLayerDesc, PersistentLayerKey}; -pub use remote_layer::RemoteLayer; + +pub(crate) use layer::{EvictionError, Layer, ResidentLayer}; pub fn range_overlaps(a: &Range, b: &Range) -> bool where @@ -74,7 +70,7 @@ pub struct ValueReconstructState { pub img: Option<(Lsn, Bytes)>, } -/// Return value from Layer::get_page_reconstruct_data +/// Return value from [`Layer::get_value_reconstruct_data`] #[derive(Clone, Copy, Debug)] pub enum ValueReconstructResult { /// Got all the data needed to reconstruct the requested page @@ -179,26 +175,6 @@ impl LayerAccessStats { new } - /// Creates a clone of `self` and records `new_status` in the clone. - /// - /// The `new_status` is not recorded in `self`. - /// - /// See [`record_residence_event`] for why you need to do this while holding the layer map lock. - /// - /// [`record_residence_event`]: Self::record_residence_event - pub(crate) fn clone_for_residence_change( - &self, - new_status: LayerResidenceStatus, - ) -> LayerAccessStats { - let clone = { - let inner = self.0.lock().unwrap(); - inner.clone() - }; - let new = LayerAccessStats(Mutex::new(clone)); - new.record_residence_event(new_status, LayerResidenceEventReason::ResidenceChange); - new - } - /// Record a change in layer residency. /// /// Recording the event must happen while holding the layer map lock to @@ -321,95 +297,12 @@ impl LayerAccessStats { } } -/// Supertrait of the [`Layer`] trait that captures the bare minimum interface -/// required by [`LayerMap`](super::layer_map::LayerMap). -/// -/// All layers should implement a minimal `std::fmt::Debug` without tenant or -/// timeline names, because those are known in the context of which the layers -/// are used in (timeline). -#[async_trait::async_trait] -pub trait Layer: std::fmt::Debug + std::fmt::Display + Send + Sync + 'static { - /// - /// Return data needed to reconstruct given page at LSN. - /// - /// It is up to the caller to collect more data from previous layer and - /// perform WAL redo, if necessary. - /// - /// See PageReconstructResult for possible return values. The collected data - /// is appended to reconstruct_data; the caller should pass an empty struct - /// on first call, or a struct with a cached older image of the page if one - /// is available. If this returns ValueReconstructResult::Continue, look up - /// the predecessor layer and call again with the same 'reconstruct_data' to - /// collect more data. - async fn get_value_reconstruct_data( - &self, - key: Key, - lsn_range: Range, - reconstruct_data: &mut ValueReconstructState, - ctx: &RequestContext, - ) -> Result; -} - /// Get a layer descriptor from a layer. pub trait AsLayerDesc { /// Get the layer descriptor. fn layer_desc(&self) -> &PersistentLayerDesc; } -/// A Layer contains all data in a "rectangle" consisting of a range of keys and -/// range of LSNs. -/// -/// There are two kinds of layers, in-memory and on-disk layers. In-memory -/// layers are used to ingest incoming WAL, and provide fast access to the -/// recent page versions. On-disk layers are stored as files on disk, and are -/// immutable. This trait presents the common functionality of in-memory and -/// on-disk layers. -/// -/// Furthermore, there are two kinds of on-disk layers: delta and image layers. -/// A delta layer contains all modifications within a range of LSNs and keys. -/// An image layer is a snapshot of all the data in a key-range, at a single -/// LSN. -pub trait PersistentLayer: Layer + AsLayerDesc { - /// File name used for this layer, both in the pageserver's local filesystem - /// state as well as in the remote storage. - fn filename(&self) -> LayerFileName { - self.layer_desc().filename() - } - - // Path to the layer file in the local filesystem. - // `None` for `RemoteLayer`. - fn local_path(&self) -> Option; - - /// Permanently remove this layer from disk. - fn delete_resident_layer_file(&self) -> Result<()>; - - fn downcast_remote_layer(self: Arc) -> Option> { - None - } - - fn downcast_delta_layer(self: Arc) -> Option> { - None - } - - fn is_remote_layer(&self) -> bool { - false - } - - fn info(&self, reset: LayerAccessStatsReset) -> HistoricLayerInfo; - - fn access_stats(&self) -> &LayerAccessStats; -} - -pub fn downcast_remote_layer( - layer: &Arc, -) -> Option> { - if layer.is_remote_layer() { - Arc::clone(layer).downcast_remote_layer() - } else { - None - } -} - pub mod tests { use super::*; @@ -447,19 +340,6 @@ pub mod tests { } } -/// Helper enum to hold a PageServerConf, or a path -/// -/// This is used by DeltaLayer and ImageLayer. Normally, this holds a reference to the -/// global config, and paths to layer files are constructed using the tenant/timeline -/// path from the config. But in the 'pagectl' binary, we need to construct a Layer -/// struct for a file on disk, without having a page server running, so that we have no -/// config. In that case, we use the Path variant to hold the full path to the file on -/// disk. -enum PathOrConf { - Path(Utf8PathBuf), - Conf(&'static PageServerConf), -} - /// Range wrapping newtype, which uses display to render Debug. /// /// Useful with `Key`, which has too verbose `{:?}` for printing multiple layers. diff --git a/pageserver/src/tenant/storage_layer/delta_layer.rs b/pageserver/src/tenant/storage_layer/delta_layer.rs index 55fb491b65..4ccce3d6bd 100644 --- a/pageserver/src/tenant/storage_layer/delta_layer.rs +++ b/pageserver/src/tenant/storage_layer/delta_layer.rs @@ -34,18 +34,17 @@ use crate::repository::{Key, Value, KEY_SIZE}; use crate::tenant::blob_io::BlobWriter; use crate::tenant::block_io::{BlockBuf, BlockCursor, BlockLease, BlockReader, FileBlockReader}; use crate::tenant::disk_btree::{DiskBtreeBuilder, DiskBtreeReader, VisitDirection}; -use crate::tenant::storage_layer::{ - PersistentLayer, ValueReconstructResult, ValueReconstructState, -}; +use crate::tenant::storage_layer::{Layer, ValueReconstructResult, ValueReconstructState}; +use crate::tenant::Timeline; use crate::virtual_file::VirtualFile; use crate::{walrecord, TEMP_FILE_SUFFIX}; use crate::{DELTA_FILE_MAGIC, STORAGE_FORMAT_VERSION}; use anyhow::{bail, ensure, Context, Result}; use camino::{Utf8Path, Utf8PathBuf}; -use pageserver_api::models::{HistoricLayerInfo, LayerAccessKind}; +use pageserver_api::models::LayerAccessKind; use rand::{distributions::Alphanumeric, Rng}; use serde::{Deserialize, Serialize}; -use std::fs::{self, File}; +use std::fs::File; use std::io::SeekFrom; use std::ops::Range; use std::os::unix::fs::FileExt; @@ -59,10 +58,7 @@ use utils::{ lsn::Lsn, }; -use super::{ - AsLayerDesc, DeltaFileName, Layer, LayerAccessStats, LayerAccessStatsReset, PathOrConf, - PersistentLayerDesc, -}; +use super::{AsLayerDesc, LayerAccessStats, PersistentLayerDesc, ResidentLayer}; /// /// Header stored in the beginning of the file @@ -182,20 +178,12 @@ impl DeltaKey { } } -/// DeltaLayer is the in-memory data structure associated with an on-disk delta -/// file. -/// -/// We keep a DeltaLayer in memory for each file, in the LayerMap. If a layer -/// is in "loaded" state, we have a copy of the index in memory, in 'inner'. -/// Otherwise the struct is just a placeholder for a file that exists on disk, -/// and it needs to be loaded before using it in queries. +/// This is used only from `pagectl`. Within pageserver, all layers are +/// [`crate::tenant::storage_layer::Layer`], which can hold a [`DeltaLayerInner`]. pub struct DeltaLayer { - path_or_conf: PathOrConf, - + path: Utf8PathBuf, pub desc: PersistentLayerDesc, - access_stats: LayerAccessStats, - inner: OnceCell>, } @@ -212,6 +200,8 @@ impl std::fmt::Debug for DeltaLayer { } } +/// `DeltaLayerInner` is the in-memory data structure associated with an on-disk delta +/// file. pub struct DeltaLayerInner { // values copied from summary index_start_blk: u32, @@ -221,12 +211,6 @@ pub struct DeltaLayerInner { file: FileBlockReader, } -impl AsRef for DeltaLayerInner { - fn as_ref(&self) -> &DeltaLayerInner { - self - } -} - impl std::fmt::Debug for DeltaLayerInner { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { f.debug_struct("DeltaLayerInner") @@ -236,19 +220,6 @@ impl std::fmt::Debug for DeltaLayerInner { } } -#[async_trait::async_trait] -impl Layer for DeltaLayer { - async fn get_value_reconstruct_data( - &self, - key: Key, - lsn_range: Range, - reconstruct_state: &mut ValueReconstructState, - ctx: &RequestContext, - ) -> anyhow::Result { - self.get_value_reconstruct_data(key, lsn_range, reconstruct_state, ctx) - .await - } -} /// Boilerplate to implement the Layer trait, always use layer_desc for persistent layers. impl std::fmt::Display for DeltaLayer { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { @@ -262,40 +233,9 @@ impl AsLayerDesc for DeltaLayer { } } -impl PersistentLayer for DeltaLayer { - fn downcast_delta_layer(self: Arc) -> Option> { - Some(self) - } - - fn local_path(&self) -> Option { - self.local_path() - } - - fn delete_resident_layer_file(&self) -> Result<()> { - self.delete_resident_layer_file() - } - - fn info(&self, reset: LayerAccessStatsReset) -> HistoricLayerInfo { - self.info(reset) - } - - fn access_stats(&self) -> &LayerAccessStats { - self.access_stats() - } -} - impl DeltaLayer { pub(crate) async fn dump(&self, verbose: bool, ctx: &RequestContext) -> Result<()> { - println!( - "----- delta layer for ten {} tli {} keys {}-{} lsn {}-{} size {} ----", - self.desc.tenant_id, - self.desc.timeline_id, - self.desc.key_range.start, - self.desc.key_range.end, - self.desc.lsn_range.start, - self.desc.lsn_range.end, - self.desc.file_size, - ); + self.desc.dump(); if !verbose { return Ok(()); @@ -303,119 +243,7 @@ impl DeltaLayer { let inner = self.load(LayerAccessKind::Dump, ctx).await?; - println!( - "index_start_blk: {}, root {}", - inner.index_start_blk, inner.index_root_blk - ); - - let file = &inner.file; - let tree_reader = DiskBtreeReader::<_, DELTA_KEY_SIZE>::new( - inner.index_start_blk, - inner.index_root_blk, - file, - ); - - tree_reader.dump().await?; - - let keys = DeltaLayerInner::load_keys(&inner, ctx).await?; - - // A subroutine to dump a single blob - async fn dump_blob(val: ValueRef<'_>, ctx: &RequestContext) -> Result { - let buf = val.reader.read_blob(val.blob_ref.pos(), ctx).await?; - let val = Value::des(&buf)?; - let desc = match val { - Value::Image(img) => { - format!(" img {} bytes", img.len()) - } - Value::WalRecord(rec) => { - let wal_desc = walrecord::describe_wal_record(&rec)?; - format!( - " rec {} bytes will_init: {} {}", - buf.len(), - rec.will_init(), - wal_desc - ) - } - }; - Ok(desc) - } - - for entry in keys { - let DeltaEntry { key, lsn, val, .. } = entry; - let desc = match dump_blob(val, ctx).await { - Ok(desc) => desc, - Err(err) => { - let err: anyhow::Error = err; - format!("ERROR: {err}") - } - }; - println!(" key {key} at {lsn}: {desc}"); - } - - Ok(()) - } - - pub(crate) async fn get_value_reconstruct_data( - &self, - key: Key, - lsn_range: Range, - reconstruct_state: &mut ValueReconstructState, - ctx: &RequestContext, - ) -> anyhow::Result { - ensure!(lsn_range.start >= self.desc.lsn_range.start); - - ensure!(self.desc.key_range.contains(&key)); - - let inner = self - .load(LayerAccessKind::GetValueReconstructData, ctx) - .await?; - inner - .get_value_reconstruct_data(key, lsn_range, reconstruct_state, ctx) - .await - } - - pub(crate) fn local_path(&self) -> Option { - Some(self.path()) - } - - pub(crate) fn delete_resident_layer_file(&self) -> Result<()> { - // delete underlying file - fs::remove_file(self.path())?; - Ok(()) - } - - pub(crate) fn info(&self, reset: LayerAccessStatsReset) -> HistoricLayerInfo { - let layer_file_name = self.layer_desc().filename().file_name(); - let lsn_range = self.layer_desc().lsn_range.clone(); - - let access_stats = self.access_stats.as_api_model(reset); - - HistoricLayerInfo::Delta { - layer_file_name, - layer_file_size: self.desc.file_size, - lsn_start: lsn_range.start, - lsn_end: lsn_range.end, - remote: false, - access_stats, - } - } - - pub(crate) fn access_stats(&self) -> &LayerAccessStats { - &self.access_stats - } - - fn path_for( - path_or_conf: &PathOrConf, - tenant_id: &TenantId, - timeline_id: &TimelineId, - fname: &DeltaFileName, - ) -> Utf8PathBuf { - match path_or_conf { - PathOrConf::Path(path) => path.clone(), - PathOrConf::Conf(conf) => conf - .timeline_path(tenant_id, timeline_id) - .join(fname.to_string()), - } + inner.dump(ctx).await } fn temp_path_for( @@ -461,52 +289,21 @@ impl DeltaLayer { async fn load_inner(&self, ctx: &RequestContext) -> Result> { let path = self.path(); - let summary = match &self.path_or_conf { - PathOrConf::Conf(_) => Some(Summary::from(self)), - PathOrConf::Path(_) => None, - }; + let loaded = DeltaLayerInner::load(&path, None, ctx).await?; - let loaded = DeltaLayerInner::load(&path, summary, ctx).await?; + // not production code + let actual_filename = path.file_name().unwrap().to_owned(); + let expected_filename = self.layer_desc().filename().file_name(); - if let PathOrConf::Path(ref path) = self.path_or_conf { - // not production code - - let actual_filename = path.file_name().unwrap().to_owned(); - let expected_filename = self.filename().file_name(); - - if actual_filename != expected_filename { - println!("warning: filename does not match what is expected from in-file summary"); - println!("actual: {:?}", actual_filename); - println!("expected: {:?}", expected_filename); - } + if actual_filename != expected_filename { + println!("warning: filename does not match what is expected from in-file summary"); + println!("actual: {:?}", actual_filename); + println!("expected: {:?}", expected_filename); } Ok(Arc::new(loaded)) } - /// Create a DeltaLayer struct representing an existing file on disk. - pub fn new( - conf: &'static PageServerConf, - timeline_id: TimelineId, - tenant_id: TenantId, - filename: &DeltaFileName, - file_size: u64, - access_stats: LayerAccessStats, - ) -> DeltaLayer { - DeltaLayer { - path_or_conf: PathOrConf::Conf(conf), - desc: PersistentLayerDesc::new_delta( - tenant_id, - timeline_id, - filename.key_range.clone(), - filename.lsn_range.clone(), - file_size, - ), - access_stats, - inner: OnceCell::new(), - } - } - /// Create a DeltaLayer struct representing an existing file on disk. /// /// This variant is only used for debugging purposes, by the 'pagectl' binary. @@ -520,7 +317,7 @@ impl DeltaLayer { .context("get file metadata to determine size")?; Ok(DeltaLayer { - path_or_conf: PathOrConf::Path(path.to_path_buf()), + path: path.to_path_buf(), desc: PersistentLayerDesc::new_delta( summary.tenant_id, summary.timeline_id, @@ -533,29 +330,9 @@ impl DeltaLayer { }) } - fn layer_name(&self) -> DeltaFileName { - self.desc.delta_file_name() - } /// Path to the layer file in pageserver workdir. - pub fn path(&self) -> Utf8PathBuf { - Self::path_for( - &self.path_or_conf, - &self.desc.tenant_id, - &self.desc.timeline_id, - &self.layer_name(), - ) - } - /// Loads all keys stored in the layer. Returns key, lsn, value size and value reference. - /// - /// The value can be obtained via the [`ValueRef::load`] function. - pub(crate) async fn load_keys(&self, ctx: &RequestContext) -> Result>> { - let inner = self - .load(LayerAccessKind::KeyIter, ctx) - .await - .context("load delta layer keys")?; - DeltaLayerInner::load_keys(inner, ctx) - .await - .context("Layer index is corrupted") + fn path(&self) -> Utf8PathBuf { + self.path.clone() } } @@ -660,7 +437,7 @@ impl DeltaLayerWriterInner { /// /// Finish writing the delta layer. /// - async fn finish(self, key_end: Key) -> anyhow::Result { + async fn finish(self, key_end: Key, timeline: &Arc) -> anyhow::Result { let index_start_blk = ((self.blob_writer.size() + PAGE_SZ as u64 - 1) / PAGE_SZ as u64) as u32; @@ -717,37 +494,21 @@ impl DeltaLayerWriterInner { // Note: Because we opened the file in write-only mode, we cannot // reuse the same VirtualFile for reading later. That's why we don't // set inner.file here. The first read will have to re-open it. - let layer = DeltaLayer { - path_or_conf: PathOrConf::Conf(self.conf), - desc: PersistentLayerDesc::new_delta( - self.tenant_id, - self.timeline_id, - self.key_start..key_end, - self.lsn_range.clone(), - metadata.len(), - ), - access_stats: LayerAccessStats::empty_will_record_residence_event_later(), - inner: OnceCell::new(), - }; + + let desc = PersistentLayerDesc::new_delta( + self.tenant_id, + self.timeline_id, + self.key_start..key_end, + self.lsn_range.clone(), + metadata.len(), + ); // fsync the file file.sync_all().await?; - // Rename the file to its final name - // - // Note: This overwrites any existing file. There shouldn't be any. - // FIXME: throw an error instead? - let final_path = DeltaLayer::path_for( - &PathOrConf::Conf(self.conf), - &self.tenant_id, - &self.timeline_id, - &DeltaFileName { - key_range: self.key_start..key_end, - lsn_range: self.lsn_range, - }, - ); - std::fs::rename(self.path, &final_path)?; - trace!("created delta layer {final_path}"); + let layer = Layer::finish_creating(self.conf, timeline, desc, &self.path)?; + + trace!("created delta layer {}", layer.local_path()); Ok(layer) } @@ -828,8 +589,12 @@ impl DeltaLayerWriter { /// /// Finish writing the delta layer. /// - pub async fn finish(mut self, key_end: Key) -> anyhow::Result { - self.inner.take().unwrap().finish(key_end).await + pub(crate) async fn finish( + mut self, + key_end: Key, + timeline: &Arc, + ) -> anyhow::Result { + self.inner.take().unwrap().finish(key_end, timeline).await } } @@ -967,15 +732,17 @@ impl DeltaLayerInner { } } - pub(super) async fn load_keys<'a, 'b, T: AsRef + Clone>( - this: &'a T, - ctx: &'b RequestContext, + pub(super) async fn load_keys<'a>( + &'a self, + ctx: &RequestContext, ) -> Result>> { - let dl = this.as_ref(); - let file = &dl.file; + let file = &self.file; - let tree_reader = - DiskBtreeReader::<_, DELTA_KEY_SIZE>::new(dl.index_start_blk, dl.index_root_blk, file); + let tree_reader = DiskBtreeReader::<_, DELTA_KEY_SIZE>::new( + self.index_start_blk, + self.index_root_blk, + file, + ); let mut all_keys: Vec> = Vec::new(); @@ -988,7 +755,7 @@ impl DeltaLayerInner { let val_ref = ValueRef { blob_ref: BlobRef(value), reader: BlockCursor::new(crate::tenant::block_io::BlockReaderRef::Adapter( - Adapter(dl), + Adapter(self), )), }; let pos = BlobRef(value).pos(); @@ -1015,10 +782,61 @@ impl DeltaLayerInner { if let Some(last) = all_keys.last_mut() { // Last key occupies all space till end of value storage, // which corresponds to beginning of the index - last.size = dl.index_start_blk as u64 * PAGE_SZ as u64 - last.size; + last.size = self.index_start_blk as u64 * PAGE_SZ as u64 - last.size; } Ok(all_keys) } + + pub(super) async fn dump(&self, ctx: &RequestContext) -> anyhow::Result<()> { + println!( + "index_start_blk: {}, root {}", + self.index_start_blk, self.index_root_blk + ); + + let file = &self.file; + let tree_reader = DiskBtreeReader::<_, DELTA_KEY_SIZE>::new( + self.index_start_blk, + self.index_root_blk, + file, + ); + + tree_reader.dump().await?; + + let keys = self.load_keys(ctx).await?; + + async fn dump_blob(val: ValueRef<'_>, ctx: &RequestContext) -> anyhow::Result { + let buf = val.reader.read_blob(val.blob_ref.pos(), ctx).await?; + let val = Value::des(&buf)?; + let desc = match val { + Value::Image(img) => { + format!(" img {} bytes", img.len()) + } + Value::WalRecord(rec) => { + let wal_desc = walrecord::describe_wal_record(&rec)?; + format!( + " rec {} bytes will_init: {} {}", + buf.len(), + rec.will_init(), + wal_desc + ) + } + }; + Ok(desc) + } + + for entry in keys { + let DeltaEntry { key, lsn, val, .. } = entry; + let desc = match dump_blob(val, ctx).await { + Ok(desc) => desc, + Err(err) => { + format!("ERROR: {err}") + } + }; + println!(" key {key} at {lsn}: {desc}"); + } + + Ok(()) + } } /// A set of data associated with a delta layer key and its value @@ -1058,3 +876,9 @@ impl> Adapter { self.0.as_ref().file.read_blk(blknum, ctx).await } } + +impl AsRef for DeltaLayerInner { + fn as_ref(&self) -> &DeltaLayerInner { + self + } +} diff --git a/pageserver/src/tenant/storage_layer/image_layer.rs b/pageserver/src/tenant/storage_layer/image_layer.rs index 94138a0786..b7b9ca69b0 100644 --- a/pageserver/src/tenant/storage_layer/image_layer.rs +++ b/pageserver/src/tenant/storage_layer/image_layer.rs @@ -31,21 +31,23 @@ use crate::tenant::blob_io::BlobWriter; use crate::tenant::block_io::{BlockBuf, BlockReader, FileBlockReader}; use crate::tenant::disk_btree::{DiskBtreeBuilder, DiskBtreeReader, VisitDirection}; use crate::tenant::storage_layer::{ - LayerAccessStats, PersistentLayer, ValueReconstructResult, ValueReconstructState, + LayerAccessStats, ValueReconstructResult, ValueReconstructState, }; +use crate::tenant::Timeline; use crate::virtual_file::VirtualFile; use crate::{IMAGE_FILE_MAGIC, STORAGE_FORMAT_VERSION, TEMP_FILE_SUFFIX}; use anyhow::{bail, ensure, Context, Result}; use bytes::Bytes; use camino::{Utf8Path, Utf8PathBuf}; use hex; -use pageserver_api::models::{HistoricLayerInfo, LayerAccessKind}; +use pageserver_api::models::LayerAccessKind; use rand::{distributions::Alphanumeric, Rng}; use serde::{Deserialize, Serialize}; -use std::fs::{self, File}; +use std::fs::File; use std::io::SeekFrom; use std::ops::Range; use std::os::unix::prelude::FileExt; +use std::sync::Arc; use tokio::sync::OnceCell; use tracing::*; @@ -56,7 +58,7 @@ use utils::{ }; use super::filename::ImageFileName; -use super::{AsLayerDesc, Layer, LayerAccessStatsReset, PathOrConf, PersistentLayerDesc}; +use super::{AsLayerDesc, Layer, PersistentLayerDesc, ResidentLayer}; /// /// Header stored in the beginning of the file @@ -114,22 +116,14 @@ impl Summary { } } -/// ImageLayer is the in-memory data structure associated with an on-disk image -/// file. -/// -/// We keep an ImageLayer in memory for each file, in the LayerMap. If a layer -/// is in "loaded" state, we have a copy of the index in memory, in 'inner'. -/// Otherwise the struct is just a placeholder for a file that exists on disk, -/// and it needs to be loaded before using it in queries. +/// This is used only from `pagectl`. Within pageserver, all layers are +/// [`crate::tenant::storage_layer::Layer`], which can hold an [`ImageLayerInner`]. pub struct ImageLayer { - path_or_conf: PathOrConf, - + path: Utf8PathBuf, pub desc: PersistentLayerDesc, // This entry contains an image of all pages as of this LSN, should be the same as desc.lsn pub lsn: Lsn, - access_stats: LayerAccessStats, - inner: OnceCell, } @@ -146,6 +140,8 @@ impl std::fmt::Debug for ImageLayer { } } +/// ImageLayer is the in-memory data structure associated with an on-disk image +/// file. pub struct ImageLayerInner { // values copied from summary index_start_blk: u32, @@ -166,73 +162,11 @@ impl std::fmt::Debug for ImageLayerInner { } } -#[async_trait::async_trait] -impl Layer for ImageLayer { - /// Look up given page in the file - async fn get_value_reconstruct_data( - &self, - key: Key, - lsn_range: Range, - reconstruct_state: &mut ValueReconstructState, - ctx: &RequestContext, - ) -> anyhow::Result { - self.get_value_reconstruct_data(key, lsn_range, reconstruct_state, ctx) - .await - } -} - -/// Boilerplate to implement the Layer trait, always use layer_desc for persistent layers. -impl std::fmt::Display for ImageLayer { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "{}", self.layer_desc().short_id()) - } -} - -impl AsLayerDesc for ImageLayer { - fn layer_desc(&self) -> &PersistentLayerDesc { - &self.desc - } -} - -impl PersistentLayer for ImageLayer { - fn local_path(&self) -> Option { - self.local_path() - } - - fn delete_resident_layer_file(&self) -> Result<()> { - self.delete_resident_layer_file() - } - - fn info(&self, reset: LayerAccessStatsReset) -> HistoricLayerInfo { - self.info(reset) - } - - fn access_stats(&self) -> &LayerAccessStats { - self.access_stats() - } -} - -impl ImageLayer { - pub(crate) async fn dump(&self, verbose: bool, ctx: &RequestContext) -> Result<()> { - println!( - "----- image layer for ten {} tli {} key {}-{} at {} is_incremental {} size {} ----", - self.desc.tenant_id, - self.desc.timeline_id, - self.desc.key_range.start, - self.desc.key_range.end, - self.lsn, - self.desc.is_incremental(), - self.desc.file_size - ); - - if !verbose { - return Ok(()); - } - - let inner = self.load(LayerAccessKind::Dump, ctx).await?; - let file = &inner.file; +impl ImageLayerInner { + pub(super) async fn dump(&self, ctx: &RequestContext) -> anyhow::Result<()> { + let file = &self.file; let tree_reader = - DiskBtreeReader::<_, KEY_SIZE>::new(inner.index_start_blk, inner.index_root_blk, file); + DiskBtreeReader::<_, KEY_SIZE>::new(self.index_start_blk, self.index_root_blk, file); tree_reader.dump().await?; @@ -250,69 +184,36 @@ impl ImageLayer { Ok(()) } +} - pub(crate) async fn get_value_reconstruct_data( - &self, - key: Key, - lsn_range: Range, - reconstruct_state: &mut ValueReconstructState, - ctx: &RequestContext, - ) -> anyhow::Result { - assert!(self.desc.key_range.contains(&key)); - assert!(lsn_range.start >= self.lsn); - assert!(lsn_range.end >= self.lsn); - - let inner = self - .load(LayerAccessKind::GetValueReconstructData, ctx) - .await?; - inner - .get_value_reconstruct_data(key, reconstruct_state, ctx) - .await - // FIXME: makes no sense to dump paths - .with_context(|| format!("read {}", self.path())) +/// Boilerplate to implement the Layer trait, always use layer_desc for persistent layers. +impl std::fmt::Display for ImageLayer { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.layer_desc().short_id()) } +} - pub(crate) fn local_path(&self) -> Option { - Some(self.path()) +impl AsLayerDesc for ImageLayer { + fn layer_desc(&self) -> &PersistentLayerDesc { + &self.desc } +} + +impl ImageLayer { + pub(crate) async fn dump(&self, verbose: bool, ctx: &RequestContext) -> Result<()> { + self.desc.dump(); + + if !verbose { + return Ok(()); + } + + let inner = self.load(LayerAccessKind::Dump, ctx).await?; + + inner.dump(ctx).await?; - pub(crate) fn delete_resident_layer_file(&self) -> Result<()> { - // delete underlying file - fs::remove_file(self.path())?; Ok(()) } - pub(crate) fn info(&self, reset: LayerAccessStatsReset) -> HistoricLayerInfo { - let layer_file_name = self.layer_desc().filename().file_name(); - let lsn_start = self.layer_desc().image_layer_lsn(); - - HistoricLayerInfo::Image { - layer_file_name, - layer_file_size: self.desc.file_size, - lsn_start, - remote: false, - access_stats: self.access_stats.as_api_model(reset), - } - } - - pub(crate) fn access_stats(&self) -> &LayerAccessStats { - &self.access_stats - } - - fn path_for( - path_or_conf: &PathOrConf, - timeline_id: TimelineId, - tenant_id: TenantId, - fname: &ImageFileName, - ) -> Utf8PathBuf { - match path_or_conf { - PathOrConf::Path(path) => path.to_path_buf(), - PathOrConf::Conf(conf) => conf - .timeline_path(&tenant_id, &timeline_id) - .join(fname.to_string()), - } - } - fn temp_path_for( conf: &PageServerConf, timeline_id: TimelineId, @@ -348,54 +249,21 @@ impl ImageLayer { async fn load_inner(&self, ctx: &RequestContext) -> Result { let path = self.path(); - let expected_summary = match &self.path_or_conf { - PathOrConf::Conf(_) => Some(Summary::from(self)), - PathOrConf::Path(_) => None, - }; + let loaded = ImageLayerInner::load(&path, self.desc.image_layer_lsn(), None, ctx).await?; - let loaded = - ImageLayerInner::load(&path, self.desc.image_layer_lsn(), expected_summary, ctx) - .await?; + // not production code + let actual_filename = path.file_name().unwrap().to_owned(); + let expected_filename = self.layer_desc().filename().file_name(); - if let PathOrConf::Path(ref path) = self.path_or_conf { - // not production code - let actual_filename = path.file_name().unwrap().to_owned(); - let expected_filename = self.filename().file_name(); - - if actual_filename != expected_filename { - println!("warning: filename does not match what is expected from in-file summary"); - println!("actual: {:?}", actual_filename); - println!("expected: {:?}", expected_filename); - } + if actual_filename != expected_filename { + println!("warning: filename does not match what is expected from in-file summary"); + println!("actual: {:?}", actual_filename); + println!("expected: {:?}", expected_filename); } Ok(loaded) } - /// Create an ImageLayer struct representing an existing file on disk - pub fn new( - conf: &'static PageServerConf, - timeline_id: TimelineId, - tenant_id: TenantId, - filename: &ImageFileName, - file_size: u64, - access_stats: LayerAccessStats, - ) -> ImageLayer { - ImageLayer { - path_or_conf: PathOrConf::Conf(conf), - desc: PersistentLayerDesc::new_img( - tenant_id, - timeline_id, - filename.key_range.clone(), - filename.lsn, - file_size, - ), // Now we assume image layer ALWAYS covers the full range. This may change in the future. - lsn: filename.lsn, - access_stats, - inner: OnceCell::new(), - } - } - /// Create an ImageLayer struct representing an existing file on disk. /// /// This variant is only used for debugging purposes, by the 'pagectl' binary. @@ -407,7 +275,7 @@ impl ImageLayer { .metadata() .context("get file metadata to determine size")?; Ok(ImageLayer { - path_or_conf: PathOrConf::Path(path.to_path_buf()), + path: path.to_path_buf(), desc: PersistentLayerDesc::new_img( summary.tenant_id, summary.timeline_id, @@ -421,18 +289,8 @@ impl ImageLayer { }) } - fn layer_name(&self) -> ImageFileName { - self.desc.image_file_name() - } - - /// Path to the layer file in pageserver workdir. - pub fn path(&self) -> Utf8PathBuf { - Self::path_for( - &self.path_or_conf, - self.desc.timeline_id, - self.desc.tenant_id, - &self.layer_name(), - ) + fn path(&self) -> Utf8PathBuf { + self.path.clone() } } @@ -604,7 +462,7 @@ impl ImageLayerWriterInner { /// /// Finish writing the image layer. /// - async fn finish(self) -> anyhow::Result { + async fn finish(self, timeline: &Arc) -> anyhow::Result { let index_start_blk = ((self.blob_writer.size() + PAGE_SZ as u64 - 1) / PAGE_SZ as u64) as u32; @@ -658,33 +516,14 @@ impl ImageLayerWriterInner { // Note: Because we open the file in write-only mode, we cannot // reuse the same VirtualFile for reading later. That's why we don't // set inner.file here. The first read will have to re-open it. - let layer = ImageLayer { - path_or_conf: PathOrConf::Conf(self.conf), - desc, - lsn: self.lsn, - access_stats: LayerAccessStats::empty_will_record_residence_event_later(), - inner: OnceCell::new(), - }; // fsync the file file.sync_all().await?; - // Rename the file to its final name - // - // Note: This overwrites any existing file. There shouldn't be any. - // FIXME: throw an error instead? - let final_path = ImageLayer::path_for( - &PathOrConf::Conf(self.conf), - self.timeline_id, - self.tenant_id, - &ImageFileName { - key_range: self.key_range.clone(), - lsn: self.lsn, - }, - ); - std::fs::rename(self.path, final_path)?; + // FIXME: why not carry the virtualfile here, it supports renaming? + let layer = Layer::finish_creating(self.conf, timeline, desc, &self.path)?; - trace!("created image layer {}", layer.path()); + trace!("created image layer {}", layer.local_path()); Ok(layer) } @@ -746,8 +585,11 @@ impl ImageLayerWriter { /// /// Finish writing the image layer. /// - pub async fn finish(mut self) -> anyhow::Result { - self.inner.take().unwrap().finish().await + pub(crate) async fn finish( + mut self, + timeline: &Arc, + ) -> anyhow::Result { + self.inner.take().unwrap().finish(timeline).await } } diff --git a/pageserver/src/tenant/storage_layer/inmemory_layer.rs b/pageserver/src/tenant/storage_layer/inmemory_layer.rs index 764dc2c64e..1275250bf0 100644 --- a/pageserver/src/tenant/storage_layer/inmemory_layer.rs +++ b/pageserver/src/tenant/storage_layer/inmemory_layer.rs @@ -10,11 +10,12 @@ use crate::repository::{Key, Value}; use crate::tenant::block_io::BlockReader; use crate::tenant::ephemeral_file::EphemeralFile; use crate::tenant::storage_layer::{ValueReconstructResult, ValueReconstructState}; +use crate::tenant::Timeline; use crate::walrecord; use anyhow::{ensure, Result}; use pageserver_api::models::InMemoryLayerInfo; use std::collections::HashMap; -use std::sync::OnceLock; +use std::sync::{Arc, OnceLock}; use tracing::*; use utils::{ bin_ser::BeSer, @@ -28,7 +29,7 @@ use std::fmt::Write as _; use std::ops::Range; use tokio::sync::RwLock; -use super::{DeltaLayer, DeltaLayerWriter, Layer}; +use super::{DeltaLayerWriter, ResidentLayer}; pub struct InMemoryLayer { conf: &'static PageServerConf, @@ -207,20 +208,6 @@ impl InMemoryLayer { } } -#[async_trait::async_trait] -impl Layer for InMemoryLayer { - async fn get_value_reconstruct_data( - &self, - key: Key, - lsn_range: Range, - reconstruct_data: &mut ValueReconstructState, - ctx: &RequestContext, - ) -> Result { - self.get_value_reconstruct_data(key, lsn_range, reconstruct_data, ctx) - .await - } -} - impl std::fmt::Display for InMemoryLayer { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { let end_lsn = self.end_lsn_or_max(); @@ -229,17 +216,13 @@ impl std::fmt::Display for InMemoryLayer { } impl InMemoryLayer { - /// /// Get layer size. - /// pub async fn size(&self) -> Result { let inner = self.inner.read().await; Ok(inner.file.len()) } - /// /// Create a new, empty, in-memory layer - /// pub async fn create( conf: &'static PageServerConf, timeline_id: TimelineId, @@ -331,7 +314,11 @@ impl InMemoryLayer { /// Write this frozen in-memory layer to disk. /// /// Returns a new delta layer with all the same data as this in-memory layer - pub(crate) async fn write_to_disk(&self, ctx: &RequestContext) -> Result { + pub(crate) async fn write_to_disk( + &self, + timeline: &Arc, + ctx: &RequestContext, + ) -> Result { // Grab the lock in read-mode. We hold it over the I/O, but because this // layer is not writeable anymore, no one should be trying to acquire the // write lock on it, so we shouldn't block anyone. There's one exception @@ -376,7 +363,8 @@ impl InMemoryLayer { } } - let delta_layer = delta_layer_writer.finish(Key::MAX).await?; + // MAX is used here because we identify L0 layers by full key range + let delta_layer = delta_layer_writer.finish(Key::MAX, timeline).await?; Ok(delta_layer) } } diff --git a/pageserver/src/tenant/storage_layer/layer.rs b/pageserver/src/tenant/storage_layer/layer.rs new file mode 100644 index 0000000000..759f6d664c --- /dev/null +++ b/pageserver/src/tenant/storage_layer/layer.rs @@ -0,0 +1,1481 @@ +use anyhow::Context; +use camino::{Utf8Path, Utf8PathBuf}; +use pageserver_api::models::{ + HistoricLayerInfo, LayerAccessKind, LayerResidenceEventReason, LayerResidenceStatus, +}; +use std::ops::Range; +use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering}; +use std::sync::{Arc, Weak}; +use std::time::SystemTime; +use tracing::Instrument; +use utils::lsn::Lsn; +use utils::sync::heavier_once_cell; + +use crate::config::PageServerConf; +use crate::context::RequestContext; +use crate::repository::Key; +use crate::tenant::{remote_timeline_client::LayerFileMetadata, RemoteTimelineClient, Timeline}; + +use super::delta_layer::{self, DeltaEntry}; +use super::image_layer; +use super::{ + AsLayerDesc, LayerAccessStats, LayerAccessStatsReset, LayerFileName, PersistentLayerDesc, + ValueReconstructResult, ValueReconstructState, +}; + +use utils::generation::Generation; + +/// A Layer contains all data in a "rectangle" consisting of a range of keys and +/// range of LSNs. +/// +/// There are two kinds of layers, in-memory and on-disk layers. In-memory +/// layers are used to ingest incoming WAL, and provide fast access to the +/// recent page versions. On-disk layers are stored as files on disk, and are +/// immutable. This type represents the on-disk kind while in-memory kind are represented by +/// [`InMemoryLayer`]. +/// +/// Furthermore, there are two kinds of on-disk layers: delta and image layers. +/// A delta layer contains all modifications within a range of LSNs and keys. +/// An image layer is a snapshot of all the data in a key-range, at a single +/// LSN. +/// +/// This type models the on-disk layers, which can be evicted and on-demand downloaded. +/// +/// [`InMemoryLayer`]: super::inmemory_layer::InMemoryLayer +#[derive(Clone)] +pub(crate) struct Layer(Arc); + +impl std::fmt::Display for Layer { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.layer_desc().short_id()) + } +} + +impl std::fmt::Debug for Layer { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self) + } +} + +impl AsLayerDesc for Layer { + fn layer_desc(&self) -> &PersistentLayerDesc { + self.0.layer_desc() + } +} + +impl Layer { + /// Creates a layer value for a file we know to not be resident. + pub(crate) fn for_evicted( + conf: &'static PageServerConf, + timeline: &Arc, + file_name: LayerFileName, + metadata: LayerFileMetadata, + ) -> Self { + let desc = PersistentLayerDesc::from_filename( + timeline.tenant_id, + timeline.timeline_id, + file_name, + metadata.file_size(), + ); + + let access_stats = LayerAccessStats::for_loading_layer(LayerResidenceStatus::Evicted); + + let owner = Layer(Arc::new(LayerInner::new( + conf, + timeline, + access_stats, + desc, + None, + metadata.generation, + ))); + + debug_assert!(owner.0.needs_download_blocking().unwrap().is_some()); + + owner + } + + /// Creates a Layer value for a file we know to be resident in timeline directory. + pub(crate) fn for_resident( + conf: &'static PageServerConf, + timeline: &Arc, + file_name: LayerFileName, + metadata: LayerFileMetadata, + ) -> ResidentLayer { + let desc = PersistentLayerDesc::from_filename( + timeline.tenant_id, + timeline.timeline_id, + file_name, + metadata.file_size(), + ); + + let access_stats = LayerAccessStats::for_loading_layer(LayerResidenceStatus::Resident); + + let mut resident = None; + + let owner = Layer(Arc::new_cyclic(|owner| { + let inner = Arc::new(DownloadedLayer { + owner: owner.clone(), + kind: tokio::sync::OnceCell::default(), + }); + resident = Some(inner.clone()); + + LayerInner::new( + conf, + timeline, + access_stats, + desc, + Some(inner), + metadata.generation, + ) + })); + + let downloaded = resident.expect("just initialized"); + + debug_assert!(owner.0.needs_download_blocking().unwrap().is_none()); + + timeline + .metrics + .resident_physical_size_add(metadata.file_size()); + + ResidentLayer { downloaded, owner } + } + + /// Creates a Layer value for freshly written out new layer file by renaming it from a + /// temporary path. + pub(crate) fn finish_creating( + conf: &'static PageServerConf, + timeline: &Arc, + desc: PersistentLayerDesc, + temp_path: &Utf8Path, + ) -> anyhow::Result { + let mut resident = None; + + let owner = Layer(Arc::new_cyclic(|owner| { + let inner = Arc::new(DownloadedLayer { + owner: owner.clone(), + kind: tokio::sync::OnceCell::default(), + }); + resident = Some(inner.clone()); + let access_stats = LayerAccessStats::empty_will_record_residence_event_later(); + access_stats.record_residence_event( + LayerResidenceStatus::Resident, + LayerResidenceEventReason::LayerCreate, + ); + LayerInner::new( + conf, + timeline, + access_stats, + desc, + Some(inner), + timeline.generation, + ) + })); + + let downloaded = resident.expect("just initialized"); + + // if the rename works, the path is as expected + std::fs::rename(temp_path, owner.local_path()) + .with_context(|| format!("rename temporary file as correct path for {owner}"))?; + + Ok(ResidentLayer { downloaded, owner }) + } + + /// Requests the layer to be evicted and waits for this to be done. + /// + /// If the file is not resident, an [`EvictionError::NotFound`] is returned. + /// + /// If for a bad luck or blocking of the executor, we miss the actual eviction and the layer is + /// re-downloaded, [`EvictionError::Downloaded`] is returned. + /// + /// Technically cancellation safe, but cancelling might shift the viewpoint of what generation + /// of download-evict cycle on retry. + pub(crate) async fn evict_and_wait( + &self, + rtc: &RemoteTimelineClient, + ) -> Result<(), EvictionError> { + self.0.evict_and_wait(rtc).await + } + + /// Delete the layer file when the `self` gets dropped, also try to schedule a remote index upload + /// then. + /// + /// On drop, this will cause a call to [`RemoteTimelineClient::schedule_deletion_of_unlinked`]. + /// This means that the unlinking by [gc] or [compaction] must have happened strictly before + /// the value this is called on gets dropped. + /// + /// This is ensured by both of those methods accepting references to Layer. + /// + /// [gc]: [`RemoteTimelineClient::schedule_gc_update`] + /// [compaction]: [`RemoteTimelineClient::schedule_compaction_update`] + pub(crate) fn garbage_collect_on_drop(&self) { + self.0.garbage_collect_on_drop(); + } + + /// Return data needed to reconstruct given page at LSN. + /// + /// It is up to the caller to collect more data from the previous layer and + /// perform WAL redo, if necessary. + pub(crate) async fn get_value_reconstruct_data( + &self, + key: Key, + lsn_range: Range, + reconstruct_data: &mut ValueReconstructState, + ctx: &RequestContext, + ) -> anyhow::Result { + use anyhow::ensure; + + let layer = self.0.get_or_maybe_download(true, Some(ctx)).await?; + self.0 + .access_stats + .record_access(LayerAccessKind::GetValueReconstructData, ctx); + + if self.layer_desc().is_delta { + ensure!(lsn_range.start >= self.layer_desc().lsn_range.start); + ensure!(self.layer_desc().key_range.contains(&key)); + } else { + ensure!(self.layer_desc().key_range.contains(&key)); + ensure!(lsn_range.start >= self.layer_desc().image_layer_lsn()); + ensure!(lsn_range.end >= self.layer_desc().image_layer_lsn()); + } + + layer + .get_value_reconstruct_data(key, lsn_range, reconstruct_data, &self.0, ctx) + .await + } + + /// Download the layer if evicted. + /// + /// Will not error when the layer is already downloaded. + pub(crate) async fn download(&self) -> anyhow::Result<()> { + self.0.get_or_maybe_download(true, None).await?; + Ok(()) + } + + /// Assuming the layer is already downloaded, returns a guard which will prohibit eviction + /// while the guard exists. + /// + /// Returns None if the layer is currently evicted. + pub(crate) async fn keep_resident(&self) -> anyhow::Result> { + let downloaded = match self.0.get_or_maybe_download(false, None).await { + Ok(d) => d, + // technically there are a lot of possible errors, but in practice it should only be + // DownloadRequired which is tripped up. could work to improve this situation + // statically later. + Err(DownloadError::DownloadRequired) => return Ok(None), + Err(e) => return Err(e.into()), + }; + + Ok(Some(ResidentLayer { + downloaded, + owner: self.clone(), + })) + } + + /// Downloads if necessary and creates a guard, which will keep this layer from being evicted. + pub(crate) async fn download_and_keep_resident(&self) -> anyhow::Result { + let downloaded = self.0.get_or_maybe_download(true, None).await?; + + Ok(ResidentLayer { + downloaded, + owner: self.clone(), + }) + } + + pub(crate) fn info(&self, reset: LayerAccessStatsReset) -> HistoricLayerInfo { + self.0.info(reset) + } + + pub(crate) fn access_stats(&self) -> &LayerAccessStats { + &self.0.access_stats + } + + pub(crate) fn local_path(&self) -> &Utf8Path { + &self.0.path + } + + pub(crate) fn metadata(&self) -> LayerFileMetadata { + self.0.metadata() + } + + /// Traditional debug dumping facility + #[allow(unused)] + pub(crate) async fn dump(&self, verbose: bool, ctx: &RequestContext) -> anyhow::Result<()> { + self.0.desc.dump(); + + if verbose { + // for now, unconditionally download everything, even if that might not be wanted. + let l = self.0.get_or_maybe_download(true, Some(ctx)).await?; + l.dump(&self.0, ctx).await? + } + + Ok(()) + } +} + +/// The download-ness ([`DownloadedLayer`]) can be either resident or wanted evicted. +/// +/// However when we want something evicted, we cannot evict it right away as there might be current +/// reads happening on it. For example: it has been searched from [`LayerMap::search`] but not yet +/// read with [`Layer::get_value_reconstruct_data`]. +/// +/// [`LayerMap::search`]: crate::tenant::layer_map::LayerMap::search +enum ResidentOrWantedEvicted { + Resident(Arc), + WantedEvicted(Weak), +} + +impl ResidentOrWantedEvicted { + fn get(&self) -> Option> { + match self { + ResidentOrWantedEvicted::Resident(strong) => Some(strong.clone()), + ResidentOrWantedEvicted::WantedEvicted(weak) => match weak.upgrade() { + Some(strong) => { + LAYER_IMPL_METRICS.inc_raced_wanted_evicted_accesses(); + Some(strong) + } + None => None, + }, + } + } + /// When eviction is first requested, drop down to holding a [`Weak`]. + /// + /// Returns `true` if this was the first time eviction was requested. + fn downgrade(&mut self) -> &Weak { + let _was_first = match self { + ResidentOrWantedEvicted::Resident(strong) => { + let weak = Arc::downgrade(strong); + *self = ResidentOrWantedEvicted::WantedEvicted(weak); + // returning the weak is not useful, because the drop could had already ran with + // the replacement above, and that will take care of cleaning the Option we are in + true + } + ResidentOrWantedEvicted::WantedEvicted(_) => false, + }; + + match self { + ResidentOrWantedEvicted::WantedEvicted(ref weak) => weak, + _ => unreachable!("just wrote wanted evicted"), + } + } +} + +struct LayerInner { + /// Only needed to check ondemand_download_behavior_treat_error_as_warn and creation of + /// [`Self::path`]. + conf: &'static PageServerConf, + + /// Full path to the file; unclear if this should exist anymore. + path: Utf8PathBuf, + + desc: PersistentLayerDesc, + + /// Timeline access is needed for remote timeline client and metrics. + timeline: Weak, + + /// Cached knowledge of [`Timeline::remote_client`] being `Some`. + have_remote_client: bool, + + access_stats: LayerAccessStats, + + /// This custom OnceCell is backed by std mutex, but only held for short time periods. + /// Initialization and deinitialization are done while holding a permit. + inner: heavier_once_cell::OnceCell, + + /// Do we want to garbage collect this when `LayerInner` is dropped + wanted_garbage_collected: AtomicBool, + + /// Do we want to evict this layer as soon as possible? After being set to `true`, all accesses + /// will try to downgrade [`ResidentOrWantedEvicted`], which will eventually trigger + /// [`LayerInner::on_downloaded_layer_drop`]. + wanted_evicted: AtomicBool, + + /// Version is to make sure we will in fact only evict a file if no new download has been + /// started. + version: AtomicUsize, + + /// Allow subscribing to when the layer actually gets evicted. + status: tokio::sync::broadcast::Sender, + + /// Counter for exponential backoff with the download + consecutive_failures: AtomicUsize, + + /// The generation of this Layer. + /// + /// For loaded layers (resident or evicted) this comes from [`LayerFileMetadata::generation`], + /// for created layers from [`Timeline::generation`]. + generation: Generation, +} + +impl std::fmt::Display for LayerInner { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.layer_desc().short_id()) + } +} + +impl AsLayerDesc for LayerInner { + fn layer_desc(&self) -> &PersistentLayerDesc { + &self.desc + } +} + +#[derive(Debug, Clone, Copy)] +enum Status { + Evicted, + Downloaded, +} + +impl Drop for LayerInner { + fn drop(&mut self) { + if !*self.wanted_garbage_collected.get_mut() { + // should we try to evict if the last wish was for eviction? + // feels like there's some hazard of overcrowding near shutdown near by, but we don't + // run drops during shutdown (yet) + return; + } + + let span = tracing::info_span!(parent: None, "layer_gc", tenant_id = %self.layer_desc().tenant_id, timeline_id = %self.layer_desc().timeline_id, layer = %self); + + let path = std::mem::take(&mut self.path); + let file_name = self.layer_desc().filename(); + let gen = self.generation; + let file_size = self.layer_desc().file_size; + let timeline = self.timeline.clone(); + + crate::task_mgr::BACKGROUND_RUNTIME.spawn_blocking(move || { + let _g = span.entered(); + + let removed = match std::fs::remove_file(path) { + Ok(()) => true, + Err(e) if e.kind() == std::io::ErrorKind::NotFound => { + // until we no longer do detaches by removing all local files before removing the + // tenant from the global map, we will always get these errors even if we knew what + // is the latest state. + // + // we currently do not track the latest state, so we'll also end up here on evicted + // layers. + false + } + Err(e) => { + tracing::error!("failed to remove garbage collected layer: {e}"); + LAYER_IMPL_METRICS.inc_gc_removes_failed(); + false + } + }; + + if let Some(timeline) = timeline.upgrade() { + if removed { + timeline.metrics.resident_physical_size_sub(file_size); + } + if let Some(remote_client) = timeline.remote_client.as_ref() { + let res = remote_client.schedule_deletion_of_unlinked(vec![(file_name, gen)]); + + if let Err(e) = res { + // test_timeline_deletion_with_files_stuck_in_upload_queue is good at + // demonstrating this deadlock (without spawn_blocking): stop will drop + // queued items, which will have ResidentLayer's, and those drops would try + // to re-entrantly lock the RemoteTimelineClient inner state. + if !timeline.is_active() { + tracing::info!("scheduling deletion on drop failed: {e:#}"); + } else { + tracing::warn!("scheduling deletion on drop failed: {e:#}"); + } + LAYER_IMPL_METRICS.inc_gcs_failed(GcFailed::DeleteSchedulingFailed); + } else { + LAYER_IMPL_METRICS.inc_completed_gcs(); + } + } + } else { + // no need to nag that timeline is gone: under normal situation on + // task_mgr::remove_tenant_from_memory the timeline is gone before we get dropped. + LAYER_IMPL_METRICS.inc_gcs_failed(GcFailed::TimelineGone); + } + }); + } +} + +impl LayerInner { + fn new( + conf: &'static PageServerConf, + timeline: &Arc, + access_stats: LayerAccessStats, + desc: PersistentLayerDesc, + downloaded: Option>, + generation: Generation, + ) -> Self { + let path = conf + .timeline_path(&timeline.tenant_id, &timeline.timeline_id) + .join(desc.filename().to_string()); + + LayerInner { + conf, + path, + desc, + timeline: Arc::downgrade(timeline), + have_remote_client: timeline.remote_client.is_some(), + access_stats, + wanted_garbage_collected: AtomicBool::new(false), + wanted_evicted: AtomicBool::new(false), + inner: if let Some(inner) = downloaded { + heavier_once_cell::OnceCell::new(ResidentOrWantedEvicted::Resident(inner)) + } else { + heavier_once_cell::OnceCell::default() + }, + version: AtomicUsize::new(0), + status: tokio::sync::broadcast::channel(1).0, + consecutive_failures: AtomicUsize::new(0), + generation, + } + } + + fn garbage_collect_on_drop(&self) { + let res = self.wanted_garbage_collected.compare_exchange( + false, + true, + Ordering::Release, + Ordering::Relaxed, + ); + + if res.is_ok() { + LAYER_IMPL_METRICS.inc_started_gcs(); + } + } + + pub(crate) async fn evict_and_wait( + &self, + _: &RemoteTimelineClient, + ) -> Result<(), EvictionError> { + use tokio::sync::broadcast::error::RecvError; + + assert!(self.have_remote_client); + + let mut rx = self.status.subscribe(); + + let res = + self.wanted_evicted + .compare_exchange(false, true, Ordering::Release, Ordering::Relaxed); + + if res.is_ok() { + LAYER_IMPL_METRICS.inc_started_evictions(); + } + + if self.get().is_none() { + // it was not evictable in the first place + // our store to the wanted_evicted does not matter; it will be reset by next download + return Err(EvictionError::NotFound); + } + + match rx.recv().await { + Ok(Status::Evicted) => Ok(()), + Ok(Status::Downloaded) => Err(EvictionError::Downloaded), + Err(RecvError::Closed) => { + unreachable!("sender cannot be dropped while we are in &self method") + } + Err(RecvError::Lagged(_)) => { + // this is quite unlikely, but we are blocking a lot in the async context, so + // we might be missing this because we are stuck on a LIFO slot on a thread + // which is busy blocking for a 1TB database create_image_layers. + // + // use however late (compared to the initial expressing of wanted) as the + // "outcome" now + match self.get() { + Some(_) => Err(EvictionError::Downloaded), + None => Ok(()), + } + } + } + } + + /// Should be cancellation safe, but cancellation is troublesome together with the spawned + /// download. + async fn get_or_maybe_download( + self: &Arc, + allow_download: bool, + ctx: Option<&RequestContext>, + ) -> Result, DownloadError> { + loop { + let download = move || async move { + // disable any scheduled but not yet running eviction deletions for this + self.version.fetch_add(1, Ordering::Relaxed); + + // no need to make the evict_and_wait wait for the actual download to complete + drop(self.status.send(Status::Downloaded)); + + let timeline = self + .timeline + .upgrade() + .ok_or_else(|| DownloadError::TimelineShutdown)?; + + let can_ever_evict = timeline.remote_client.as_ref().is_some(); + + // check if we really need to be downloaded; could have been already downloaded by a + // cancelled previous attempt. + let needs_download = self + .needs_download() + .await + .map_err(DownloadError::PreStatFailed)?; + + if let Some(reason) = needs_download { + // only reset this after we've decided we really need to download. otherwise it'd + // be impossible to mark cancelled downloads for eviction, like one could imagine + // we would like to do for prefetching which was not needed. + self.wanted_evicted.store(false, Ordering::Release); + + if !can_ever_evict { + return Err(DownloadError::NoRemoteStorage); + } + + tracing::debug!(%reason, "downloading layer"); + + if let Some(ctx) = ctx { + self.check_expected_download(ctx)?; + } + + if !allow_download { + // this does look weird, but for LayerInner the "downloading" means also changing + // internal once related state ... + return Err(DownloadError::DownloadRequired); + } + + self.spawn_download_and_wait(timeline).await?; + } else { + // the file is present locally, probably by a previous but cancelled call to + // get_or_maybe_download. alternatively we might be running without remote storage. + LAYER_IMPL_METRICS.inc_init_needed_no_download(); + } + + let res = Arc::new(DownloadedLayer { + owner: Arc::downgrade(self), + kind: tokio::sync::OnceCell::default(), + }); + + self.access_stats.record_residence_event( + LayerResidenceStatus::Resident, + LayerResidenceEventReason::ResidenceChange, + ); + + Ok(ResidentOrWantedEvicted::Resident(res)) + }; + + let locked = self.inner.get_or_init(download).await?; + + if let Some(strong) = Self::get_or_apply_evictedness(Some(locked), &self.wanted_evicted) + { + return Ok(strong); + } + + // the situation in which we might need to retry is that our init was ready + // immediatedly, but the DownloadedLayer had been dropped BUT failed to complete + // Self::evict_blocking + LAYER_IMPL_METRICS.inc_retried_get_or_maybe_download(); + } + } + + /// Nag or fail per RequestContext policy + fn check_expected_download(&self, ctx: &RequestContext) -> Result<(), DownloadError> { + use crate::context::DownloadBehavior::*; + let b = ctx.download_behavior(); + match b { + Download => Ok(()), + Warn | Error => { + tracing::warn!( + "unexpectedly on-demand downloading remote layer {self} for task kind {:?}", + ctx.task_kind() + ); + crate::metrics::UNEXPECTED_ONDEMAND_DOWNLOADS.inc(); + + let really_error = + matches!(b, Error) && !self.conf.ondemand_download_behavior_treat_error_as_warn; + + if really_error { + // this check is only probablistic, seems like flakyness footgun + Err(DownloadError::ContextAndConfigReallyDeniesDownloads) + } else { + Ok(()) + } + } + } + } + + /// Actual download, at most one is executed at the time. + async fn spawn_download_and_wait( + self: &Arc, + timeline: Arc, + ) -> Result<(), DownloadError> { + let task_name = format!("download layer {}", self); + + let (tx, rx) = tokio::sync::oneshot::channel(); + // this is sadly needed because of task_mgr::shutdown_tasks, otherwise we cannot + // block tenant::mgr::remove_tenant_from_memory. + + let this: Arc = self.clone(); + crate::task_mgr::spawn( + &tokio::runtime::Handle::current(), + crate::task_mgr::TaskKind::RemoteDownloadTask, + Some(self.desc.tenant_id), + Some(self.desc.timeline_id), + &task_name, + false, + async move { + let client = timeline + .remote_client + .as_ref() + .expect("checked above with have_remote_client"); + + let result = client.download_layer_file( + &this.desc.filename(), + &this.metadata(), + ) + .await; + + let result = match result { + Ok(size) => { + timeline.metrics.resident_physical_size_add(size); + Ok(()) + } + Err(e) => { + Err(e) + } + }; + + if let Err(res) = tx.send(result) { + match res { + Ok(()) => { + // our caller is cancellation safe so this is fine; if someone + // else requests the layer, they'll find it already downloaded + // or redownload. + // + // however, could be that we should consider marking the layer + // for eviction? alas, cannot: because only DownloadedLayer + // will handle that. + tracing::info!("layer file download completed after requester had cancelled"); + LAYER_IMPL_METRICS.inc_download_completed_without_requester(); + }, + Err(e) => { + // our caller is cancellation safe, but we might be racing with + // another attempt to initialize. before we have cancellation + // token support: these attempts should converge regardless of + // their completion order. + tracing::error!("layer file download failed, and additionally failed to communicate this to caller: {e:?}"); + LAYER_IMPL_METRICS.inc_download_failed_without_requester(); + } + } + } + + Ok(()) + } + .in_current_span(), + ); + match rx.await { + Ok(Ok(())) => { + if let Some(reason) = self + .needs_download() + .await + .map_err(DownloadError::PostStatFailed)? + { + // this is really a bug in needs_download or remote timeline client + panic!("post-condition failed: needs_download returned {reason:?}"); + } + + self.consecutive_failures.store(0, Ordering::Relaxed); + + Ok(()) + } + Ok(Err(e)) => { + let consecutive_failures = + self.consecutive_failures.fetch_add(1, Ordering::Relaxed); + tracing::error!(consecutive_failures, "layer file download failed: {e:#}"); + let backoff = utils::backoff::exponential_backoff_duration_seconds( + consecutive_failures.min(u32::MAX as usize) as u32, + 1.5, + 60.0, + ); + let backoff = std::time::Duration::from_secs_f64(backoff); + + tokio::time::sleep(backoff).await; + Err(DownloadError::DownloadFailed) + } + Err(_gone) => Err(DownloadError::DownloadCancelled), + } + } + + /// Access the current state without waiting for the file to be downloaded. + /// + /// Requires that we've initialized to state which is respective to the + /// actual residency state. + fn get(&self) -> Option> { + let locked = self.inner.get(); + Self::get_or_apply_evictedness(locked, &self.wanted_evicted) + } + + fn get_or_apply_evictedness( + guard: Option>, + wanted_evicted: &AtomicBool, + ) -> Option> { + if let Some(mut x) = guard { + if let Some(won) = x.get() { + // there are no guarantees that we will always get to observe a concurrent call + // to evict + if wanted_evicted.load(Ordering::Acquire) { + x.downgrade(); + } + return Some(won); + } + } + + None + } + + async fn needs_download(&self) -> Result, std::io::Error> { + match tokio::fs::metadata(&self.path).await { + Ok(m) => Ok(self.is_file_present_and_good_size(&m).err()), + Err(e) if e.kind() == std::io::ErrorKind::NotFound => Ok(Some(NeedsDownload::NotFound)), + Err(e) => Err(e), + } + } + + fn needs_download_blocking(&self) -> Result, std::io::Error> { + match self.path.metadata() { + Ok(m) => Ok(self.is_file_present_and_good_size(&m).err()), + Err(e) if e.kind() == std::io::ErrorKind::NotFound => Ok(Some(NeedsDownload::NotFound)), + Err(e) => Err(e), + } + } + + fn is_file_present_and_good_size(&self, m: &std::fs::Metadata) -> Result<(), NeedsDownload> { + // in future, this should include sha2-256 validation of the file. + if !m.is_file() { + Err(NeedsDownload::NotFile) + } else if m.len() != self.desc.file_size { + Err(NeedsDownload::WrongSize { + actual: m.len(), + expected: self.desc.file_size, + }) + } else { + Ok(()) + } + } + + fn info(&self, reset: LayerAccessStatsReset) -> HistoricLayerInfo { + let layer_file_name = self.desc.filename().file_name(); + + let remote = self.get().is_none(); + + let access_stats = self.access_stats.as_api_model(reset); + + if self.desc.is_delta { + let lsn_range = &self.desc.lsn_range; + + HistoricLayerInfo::Delta { + layer_file_name, + layer_file_size: self.desc.file_size, + lsn_start: lsn_range.start, + lsn_end: lsn_range.end, + remote, + access_stats, + } + } else { + let lsn = self.desc.image_layer_lsn(); + + HistoricLayerInfo::Image { + layer_file_name, + layer_file_size: self.desc.file_size, + lsn_start: lsn, + remote, + access_stats, + } + } + } + + /// `DownloadedLayer` is being dropped, so it calls this method. + fn on_downloaded_layer_drop(self: Arc) { + let gc = self.wanted_garbage_collected.load(Ordering::Acquire); + let evict = self.wanted_evicted.load(Ordering::Acquire); + let can_evict = self.have_remote_client; + + if gc { + // do nothing now, only in LayerInner::drop + } else if can_evict && evict { + let version = self.version.load(Ordering::Relaxed); + + let span = tracing::info_span!(parent: None, "layer_evict", tenant_id = %self.desc.tenant_id, timeline_id = %self.desc.timeline_id, layer=%self); + + // downgrade for queueing, in case there's a tear down already ongoing we should not + // hold it alive. + let this = Arc::downgrade(&self); + drop(self); + + crate::task_mgr::BACKGROUND_RUNTIME.spawn_blocking(move || { + let _g = span.entered(); + + // if LayerInner is already dropped here, do nothing because the garbage collection + // has already ran while we were in queue + let Some(this) = this.upgrade() else { + LAYER_IMPL_METRICS.inc_eviction_cancelled(EvictionCancelled::LayerGone); + return; + }; + this.evict_blocking(version); + }); + } + } + + fn evict_blocking(&self, version: usize) { + match self.evict_blocking0(version) { + Ok(()) => LAYER_IMPL_METRICS.inc_completed_evictions(), + Err(reason) => LAYER_IMPL_METRICS.inc_eviction_cancelled(reason), + } + } + + fn evict_blocking0(&self, version: usize) -> Result<(), EvictionCancelled> { + // deleted or detached timeline, don't do anything. + let Some(timeline) = self.timeline.upgrade() else { + return Err(EvictionCancelled::TimelineGone); + }; + + // to avoid starting a new download while we evict, keep holding on to the + // permit. + let _permit = { + let maybe_downloaded = self.inner.get(); + + if version != self.version.load(Ordering::Relaxed) { + // downloadness-state has advanced, we might no longer be the latest eviction + // work; don't do anything. + // + // this is possible to get to by having: + // + // 1. wanted_evicted.store(true) + // 2. ResidentOrWantedEvicted::downgrade + // 3. DownloadedLayer::drop + // 4. LayerInner::get_or_maybe_download + // 5. LayerInner::evict_blocking + return Err(EvictionCancelled::VersionCheckFailed); + } + + // free the DownloadedLayer allocation + match maybe_downloaded.map(|mut g| g.take_and_deinit()) { + Some((taken, permit)) => { + assert!(matches!(taken, ResidentOrWantedEvicted::WantedEvicted(_))); + permit + } + None => { + unreachable!("we do the version checking for this exact reason") + } + } + }; + + self.access_stats.record_residence_event( + LayerResidenceStatus::Evicted, + LayerResidenceEventReason::ResidenceChange, + ); + + let res = match capture_mtime_and_remove(&self.path) { + Ok(local_layer_mtime) => { + let duration = SystemTime::now().duration_since(local_layer_mtime); + match duration { + Ok(elapsed) => { + timeline + .metrics + .evictions_with_low_residence_duration + .read() + .unwrap() + .observe(elapsed); + tracing::info!( + residence_millis = elapsed.as_millis(), + "evicted layer after known residence period" + ); + } + Err(_) => { + tracing::info!("evicted layer after unknown residence period"); + } + } + timeline + .metrics + .resident_physical_size_sub(self.desc.file_size); + + Ok(()) + } + Err(e) if e.kind() == std::io::ErrorKind::NotFound => { + tracing::info!("failed to evict file from disk, it was already gone"); + Err(EvictionCancelled::FileNotFound) + } + Err(e) => { + tracing::warn!("failed to evict file from disk: {e:#}"); + Err(EvictionCancelled::RemoveFailed) + } + }; + + // we are still holding the permit, so no new spawn_download_and_wait can happen + drop(self.status.send(Status::Evicted)); + + res + } + + fn metadata(&self) -> LayerFileMetadata { + LayerFileMetadata::new(self.desc.file_size, self.generation) + } +} + +fn capture_mtime_and_remove(path: &Utf8Path) -> Result { + let m = path.metadata()?; + let local_layer_mtime = m.modified()?; + std::fs::remove_file(path)?; + Ok(local_layer_mtime) +} + +#[derive(Debug, thiserror::Error)] +pub(crate) enum EvictionError { + #[error("layer was already evicted")] + NotFound, + + /// Evictions must always lose to downloads in races, and this time it happened. + #[error("layer was downloaded instead")] + Downloaded, +} + +/// Error internal to the [`LayerInner::get_or_maybe_download`] +#[derive(Debug, thiserror::Error)] +enum DownloadError { + #[error("timeline has already shutdown")] + TimelineShutdown, + #[error("no remote storage configured")] + NoRemoteStorage, + #[error("context denies downloading")] + ContextAndConfigReallyDeniesDownloads, + #[error("downloading is really required but not allowed by this method")] + DownloadRequired, + /// Why no error here? Because it will be reported by page_service. We should had also done + /// retries already. + #[error("downloading evicted layer file failed")] + DownloadFailed, + #[error("downloading failed, possibly for shutdown")] + DownloadCancelled, + #[error("pre-condition: stat before download failed")] + PreStatFailed(#[source] std::io::Error), + #[error("post-condition: stat after download failed")] + PostStatFailed(#[source] std::io::Error), +} + +#[derive(Debug, PartialEq)] +pub(crate) enum NeedsDownload { + NotFound, + NotFile, + WrongSize { actual: u64, expected: u64 }, +} + +impl std::fmt::Display for NeedsDownload { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + NeedsDownload::NotFound => write!(f, "file was not found"), + NeedsDownload::NotFile => write!(f, "path is not a file"), + NeedsDownload::WrongSize { actual, expected } => { + write!(f, "file size mismatch {actual} vs. {expected}") + } + } + } +} + +/// Existence of `DownloadedLayer` means that we have the file locally, and can later evict it. +pub(crate) struct DownloadedLayer { + owner: Weak, + kind: tokio::sync::OnceCell>, +} + +impl std::fmt::Debug for DownloadedLayer { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("DownloadedLayer") + // owner omitted because it is always "Weak" + .field("kind", &self.kind) + .finish() + } +} + +impl Drop for DownloadedLayer { + fn drop(&mut self) { + if let Some(owner) = self.owner.upgrade() { + owner.on_downloaded_layer_drop(); + } else { + // no need to do anything, we are shutting down + } + } +} + +impl DownloadedLayer { + /// Initializes the `DeltaLayerInner` or `ImageLayerInner` within [`LayerKind`], or fails to + /// initialize it permanently. + /// + /// `owner` parameter is a strong reference at the same `LayerInner` as the + /// `DownloadedLayer::owner` would be when upgraded. Given how this method ends up called, + /// we will always have the LayerInner on the callstack, so we can just use it. + async fn get<'a>( + &'a self, + owner: &Arc, + ctx: &RequestContext, + ) -> anyhow::Result<&'a LayerKind> { + let init = || async { + assert_eq!( + Weak::as_ptr(&self.owner), + Arc::as_ptr(owner), + "these are the same, just avoiding the upgrade" + ); + + // there is nothing async here, but it should be async + let res = if owner.desc.is_delta { + let summary = Some(delta_layer::Summary::expected( + owner.desc.tenant_id, + owner.desc.timeline_id, + owner.desc.key_range.clone(), + owner.desc.lsn_range.clone(), + )); + delta_layer::DeltaLayerInner::load(&owner.path, summary, ctx) + .await + .map(LayerKind::Delta) + } else { + let lsn = owner.desc.image_layer_lsn(); + let summary = Some(image_layer::Summary::expected( + owner.desc.tenant_id, + owner.desc.timeline_id, + owner.desc.key_range.clone(), + lsn, + )); + image_layer::ImageLayerInner::load(&owner.path, lsn, summary, ctx) + .await + .map(LayerKind::Image) + } + // this will be a permanent failure + .context("load layer"); + + if res.is_err() { + LAYER_IMPL_METRICS.inc_permanent_loading_failures(); + } + res + }; + self.kind.get_or_init(init).await.as_ref().map_err(|e| { + // errors are not clonabled, cannot but stringify + // test_broken_timeline matches this string + anyhow::anyhow!("layer loading failed: {e:#}") + }) + } + + async fn get_value_reconstruct_data( + &self, + key: Key, + lsn_range: Range, + reconstruct_data: &mut ValueReconstructState, + owner: &Arc, + ctx: &RequestContext, + ) -> anyhow::Result { + use LayerKind::*; + + match self.get(owner, ctx).await? { + Delta(d) => { + d.get_value_reconstruct_data(key, lsn_range, reconstruct_data, ctx) + .await + } + Image(i) => { + i.get_value_reconstruct_data(key, reconstruct_data, ctx) + .await + } + } + } + + async fn dump(&self, owner: &Arc, ctx: &RequestContext) -> anyhow::Result<()> { + use LayerKind::*; + match self.get(owner, ctx).await? { + Delta(d) => d.dump(ctx).await?, + Image(i) => i.dump(ctx).await?, + } + + Ok(()) + } +} + +/// Wrapper around an actual layer implementation. +#[derive(Debug)] +enum LayerKind { + Delta(delta_layer::DeltaLayerInner), + Image(image_layer::ImageLayerInner), +} + +/// Guard for forcing a layer be resident while it exists. +#[derive(Clone)] +pub(crate) struct ResidentLayer { + owner: Layer, + downloaded: Arc, +} + +impl std::fmt::Display for ResidentLayer { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.owner) + } +} + +impl std::fmt::Debug for ResidentLayer { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.owner) + } +} + +impl ResidentLayer { + /// Release the eviction guard, converting back into a plain [`Layer`]. + pub(crate) fn drop_eviction_guard(self) -> Layer { + self.into() + } + + /// Loads all keys stored in the layer. Returns key, lsn and value size. + pub(crate) async fn load_keys<'a>( + &'a self, + ctx: &RequestContext, + ) -> anyhow::Result>> { + use LayerKind::*; + + let owner = &self.owner.0; + + match self.downloaded.get(owner, ctx).await? { + Delta(ref d) => { + owner + .access_stats + .record_access(LayerAccessKind::KeyIter, ctx); + + // this is valid because the DownloadedLayer::kind is a OnceCell, not a + // Mutex, so we cannot go and deinitialize the value with OnceCell::take + // while it's being held. + delta_layer::DeltaLayerInner::load_keys(d, ctx) + .await + .context("Layer index is corrupted") + } + Image(_) => anyhow::bail!("cannot load_keys on a image layer"), + } + } + + pub(crate) fn local_path(&self) -> &Utf8Path { + &self.owner.0.path + } + + pub(crate) fn access_stats(&self) -> &LayerAccessStats { + self.owner.access_stats() + } + + pub(crate) fn metadata(&self) -> LayerFileMetadata { + self.owner.metadata() + } +} + +impl AsLayerDesc for ResidentLayer { + fn layer_desc(&self) -> &PersistentLayerDesc { + self.owner.layer_desc() + } +} + +impl AsRef for ResidentLayer { + fn as_ref(&self) -> &Layer { + &self.owner + } +} + +/// Allow slimming down if we don't want the `2*usize` with eviction candidates? +impl From for Layer { + fn from(value: ResidentLayer) -> Self { + value.owner + } +} + +use metrics::{IntCounter, IntCounterVec}; + +struct LayerImplMetrics { + started_evictions: IntCounter, + completed_evictions: IntCounter, + cancelled_evictions: IntCounterVec, + + started_gcs: IntCounter, + completed_gcs: IntCounter, + failed_gcs: IntCounterVec, + + rare_counters: IntCounterVec, +} + +impl Default for LayerImplMetrics { + fn default() -> Self { + let evictions = metrics::register_int_counter_vec!( + "pageserver_layer_evictions_count", + "Evictions started and completed in the Layer implementation", + &["state"] + ) + .unwrap(); + + let started_evictions = evictions + .get_metric_with_label_values(&["started"]) + .unwrap(); + let completed_evictions = evictions + .get_metric_with_label_values(&["completed"]) + .unwrap(); + + let cancelled_evictions = metrics::register_int_counter_vec!( + "pageserver_layer_cancelled_evictions_count", + "Different reasons for evictions to have been cancelled or failed", + &["reason"] + ) + .unwrap(); + + let gcs = metrics::register_int_counter_vec!( + "pageserver_layer_gcs_count", + "Garbage collections started and completed in the Layer implementation", + &["state"] + ) + .unwrap(); + + let started_gcs = gcs.get_metric_with_label_values(&["pending"]).unwrap(); + let completed_gcs = gcs.get_metric_with_label_values(&["completed"]).unwrap(); + + let failed_gcs = metrics::register_int_counter_vec!( + "pageserver_layer_failed_gcs_count", + "Different reasons for garbage collections to have failed", + &["reason"] + ) + .unwrap(); + + let rare_counters = metrics::register_int_counter_vec!( + "pageserver_layer_assumed_rare_count", + "Times unexpected or assumed rare event happened", + &["event"] + ) + .unwrap(); + + Self { + started_evictions, + completed_evictions, + cancelled_evictions, + + started_gcs, + completed_gcs, + failed_gcs, + + rare_counters, + } + } +} + +impl LayerImplMetrics { + fn inc_started_evictions(&self) { + self.started_evictions.inc(); + } + fn inc_completed_evictions(&self) { + self.completed_evictions.inc(); + } + fn inc_eviction_cancelled(&self, reason: EvictionCancelled) { + self.cancelled_evictions + .get_metric_with_label_values(&[reason.as_str()]) + .unwrap() + .inc() + } + + fn inc_started_gcs(&self) { + self.started_gcs.inc(); + } + fn inc_completed_gcs(&self) { + self.completed_gcs.inc(); + } + fn inc_gcs_failed(&self, reason: GcFailed) { + self.failed_gcs + .get_metric_with_label_values(&[reason.as_str()]) + .unwrap() + .inc(); + } + + /// Counted separatedly from failed gcs because we will complete the gc attempt regardless of + /// failure to delete local file. + fn inc_gc_removes_failed(&self) { + self.rare_counters + .get_metric_with_label_values(&["gc_remove_failed"]) + .unwrap() + .inc(); + } + + /// Expected rare because requires a race with `evict_blocking` and + /// `get_or_maybe_download`. + fn inc_retried_get_or_maybe_download(&self) { + self.rare_counters + .get_metric_with_label_values(&["retried_gomd"]) + .unwrap() + .inc(); + } + + /// Expected rare because cancellations are unexpected + fn inc_download_completed_without_requester(&self) { + self.rare_counters + .get_metric_with_label_values(&["download_completed_without"]) + .unwrap() + .inc(); + } + + /// Expected rare because cancellations are unexpected + fn inc_download_failed_without_requester(&self) { + self.rare_counters + .get_metric_with_label_values(&["download_failed_without"]) + .unwrap() + .inc(); + } + + /// The Weak in ResidentOrWantedEvicted::WantedEvicted was successfully upgraded. + /// + /// If this counter is always zero, we should replace ResidentOrWantedEvicted type with an + /// Option. + fn inc_raced_wanted_evicted_accesses(&self) { + self.rare_counters + .get_metric_with_label_values(&["raced_wanted_evicted"]) + .unwrap() + .inc(); + } + + /// These are only expected for [`Self::inc_download_completed_without_requester`] amount when + /// running with remote storage. + fn inc_init_needed_no_download(&self) { + self.rare_counters + .get_metric_with_label_values(&["init_needed_no_download"]) + .unwrap() + .inc(); + } + + /// Expected rare because all layer files should be readable and good + fn inc_permanent_loading_failures(&self) { + self.rare_counters + .get_metric_with_label_values(&["permanent_loading_failure"]) + .unwrap() + .inc(); + } +} + +enum EvictionCancelled { + LayerGone, + TimelineGone, + VersionCheckFailed, + FileNotFound, + RemoveFailed, +} + +impl EvictionCancelled { + fn as_str(&self) -> &'static str { + match self { + EvictionCancelled::LayerGone => "layer_gone", + EvictionCancelled::TimelineGone => "timeline_gone", + EvictionCancelled::VersionCheckFailed => "version_check_fail", + EvictionCancelled::FileNotFound => "file_not_found", + EvictionCancelled::RemoveFailed => "remove_failed", + } + } +} + +enum GcFailed { + TimelineGone, + DeleteSchedulingFailed, +} + +impl GcFailed { + fn as_str(&self) -> &'static str { + match self { + GcFailed::TimelineGone => "timeline_gone", + GcFailed::DeleteSchedulingFailed => "delete_scheduling_failed", + } + } +} + +static LAYER_IMPL_METRICS: once_cell::sync::Lazy = + once_cell::sync::Lazy::new(LayerImplMetrics::default); diff --git a/pageserver/src/tenant/storage_layer/layer_desc.rs b/pageserver/src/tenant/storage_layer/layer_desc.rs index 6d6ab801d8..2e0b0b3e64 100644 --- a/pageserver/src/tenant/storage_layer/layer_desc.rs +++ b/pageserver/src/tenant/storage_layer/layer_desc.rs @@ -1,4 +1,3 @@ -use anyhow::Result; use core::fmt::Display; use std::ops::Range; use utils::{ @@ -6,7 +5,7 @@ use utils::{ lsn::Lsn, }; -use crate::{context::RequestContext, repository::Key}; +use crate::repository::Key; use super::{DeltaFileName, ImageFileName, LayerFileName}; @@ -100,6 +99,22 @@ impl PersistentLayerDesc { } } + pub fn from_filename( + tenant_id: TenantId, + timeline_id: TimelineId, + filename: LayerFileName, + file_size: u64, + ) -> Self { + match filename { + LayerFileName::Image(i) => { + Self::new_img(tenant_id, timeline_id, i.key_range, i.lsn, file_size) + } + LayerFileName::Delta(d) => { + Self::new_delta(tenant_id, timeline_id, d.key_range, d.lsn_range, file_size) + } + } + } + /// Get the LSN that the image layer covers. pub fn image_layer_lsn(&self) -> Lsn { assert!(!self.is_delta); @@ -173,21 +188,31 @@ impl PersistentLayerDesc { self.is_delta } - pub fn dump(&self, _verbose: bool, _ctx: &RequestContext) -> Result<()> { - println!( - "----- layer for ten {} tli {} keys {}-{} lsn {}-{} is_delta {} is_incremental {} size {} ----", - self.tenant_id, - self.timeline_id, - self.key_range.start, - self.key_range.end, - self.lsn_range.start, - self.lsn_range.end, - self.is_delta, - self.is_incremental(), - self.file_size, - ); - - Ok(()) + pub fn dump(&self) { + if self.is_delta { + println!( + "----- delta layer for ten {} tli {} keys {}-{} lsn {}-{} is_incremental {} size {} ----", + self.tenant_id, + self.timeline_id, + self.key_range.start, + self.key_range.end, + self.lsn_range.start, + self.lsn_range.end, + self.is_incremental(), + self.file_size, + ); + } else { + println!( + "----- image layer for ten {} tli {} key {}-{} at {} is_incremental {} size {} ----", + self.tenant_id, + self.timeline_id, + self.key_range.start, + self.key_range.end, + self.image_layer_lsn(), + self.is_incremental(), + self.file_size + ); + } } pub fn file_size(&self) -> u64 { diff --git a/pageserver/src/tenant/storage_layer/remote_layer.rs b/pageserver/src/tenant/storage_layer/remote_layer.rs deleted file mode 100644 index 94bebb5f6e..0000000000 --- a/pageserver/src/tenant/storage_layer/remote_layer.rs +++ /dev/null @@ -1,216 +0,0 @@ -//! A RemoteLayer is an in-memory placeholder for a layer file that exists -//! in remote storage. -//! -use crate::config::PageServerConf; -use crate::context::RequestContext; -use crate::repository::Key; -use crate::tenant::remote_timeline_client::index::LayerFileMetadata; -use crate::tenant::storage_layer::{Layer, ValueReconstructResult, ValueReconstructState}; -use crate::tenant::timeline::layer_manager::LayerManager; -use anyhow::{bail, Result}; -use camino::Utf8PathBuf; -use pageserver_api::models::HistoricLayerInfo; -use std::ops::Range; -use std::sync::Arc; - -use utils::{ - id::{TenantId, TimelineId}, - lsn::Lsn, -}; - -use super::filename::{DeltaFileName, ImageFileName}; -use super::{ - AsLayerDesc, DeltaLayer, ImageLayer, LayerAccessStats, LayerAccessStatsReset, - LayerResidenceStatus, PersistentLayer, PersistentLayerDesc, -}; - -/// RemoteLayer is a not yet downloaded [`ImageLayer`] or -/// [`DeltaLayer`]. -/// -/// RemoteLayer might be downloaded on-demand during operations which are -/// allowed download remote layers and during which, it gets replaced with a -/// concrete `DeltaLayer` or `ImageLayer`. -/// -/// See: [`crate::context::RequestContext`] for authorization to download -pub struct RemoteLayer { - pub desc: PersistentLayerDesc, - - pub layer_metadata: LayerFileMetadata, - - access_stats: LayerAccessStats, - - pub(crate) ongoing_download: Arc, - - /// Has `LayerMap::replace` failed for this (true) or not (false). - /// - /// Used together with [`ongoing_download`] semaphore in `Timeline::download_remote_layer`. - /// The field is used to mark a RemoteLayer permanently (until restart or ignore+load) - /// unprocessable, because a LayerMap::replace failed. - /// - /// It is very unlikely to accumulate these in the Timeline's LayerMap, but having this avoids - /// a possible fast loop between `Timeline::get_reconstruct_data` and - /// `Timeline::download_remote_layer`, which also logs. - /// - /// [`ongoing_download`]: Self::ongoing_download - pub(crate) download_replacement_failure: std::sync::atomic::AtomicBool, -} - -impl std::fmt::Debug for RemoteLayer { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("RemoteLayer") - .field("file_name", &self.desc.filename()) - .field("layer_metadata", &self.layer_metadata) - .field("is_incremental", &self.desc.is_incremental()) - .finish() - } -} - -#[async_trait::async_trait] -impl Layer for RemoteLayer { - async fn get_value_reconstruct_data( - &self, - _key: Key, - _lsn_range: Range, - _reconstruct_state: &mut ValueReconstructState, - _ctx: &RequestContext, - ) -> Result { - Err(anyhow::anyhow!("layer {self} needs to be downloaded")) - } -} - -/// Boilerplate to implement the Layer trait, always use layer_desc for persistent layers. -impl std::fmt::Display for RemoteLayer { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "{}", self.layer_desc().short_id()) - } -} - -impl AsLayerDesc for RemoteLayer { - fn layer_desc(&self) -> &PersistentLayerDesc { - &self.desc - } -} - -impl PersistentLayer for RemoteLayer { - fn local_path(&self) -> Option { - None - } - - fn delete_resident_layer_file(&self) -> Result<()> { - bail!("remote layer has no layer file"); - } - - fn downcast_remote_layer<'a>(self: Arc) -> Option> { - Some(self) - } - - fn is_remote_layer(&self) -> bool { - true - } - - fn info(&self, reset: LayerAccessStatsReset) -> HistoricLayerInfo { - let layer_file_name = self.layer_desc().filename().file_name(); - let lsn_range = self.layer_desc().lsn_range.clone(); - - if self.desc.is_delta { - HistoricLayerInfo::Delta { - layer_file_name, - layer_file_size: self.layer_metadata.file_size(), - lsn_start: lsn_range.start, - lsn_end: lsn_range.end, - remote: true, - access_stats: self.access_stats.as_api_model(reset), - } - } else { - HistoricLayerInfo::Image { - layer_file_name, - layer_file_size: self.layer_metadata.file_size(), - lsn_start: lsn_range.start, - remote: true, - access_stats: self.access_stats.as_api_model(reset), - } - } - } - - fn access_stats(&self) -> &LayerAccessStats { - &self.access_stats - } -} - -impl RemoteLayer { - pub fn new_img( - tenantid: TenantId, - timelineid: TimelineId, - fname: &ImageFileName, - layer_metadata: &LayerFileMetadata, - access_stats: LayerAccessStats, - ) -> RemoteLayer { - RemoteLayer { - desc: PersistentLayerDesc::new_img( - tenantid, - timelineid, - fname.key_range.clone(), - fname.lsn, - layer_metadata.file_size(), - ), - layer_metadata: layer_metadata.clone(), - ongoing_download: Arc::new(tokio::sync::Semaphore::new(1)), - download_replacement_failure: std::sync::atomic::AtomicBool::default(), - access_stats, - } - } - - pub fn new_delta( - tenantid: TenantId, - timelineid: TimelineId, - fname: &DeltaFileName, - layer_metadata: &LayerFileMetadata, - access_stats: LayerAccessStats, - ) -> RemoteLayer { - RemoteLayer { - desc: PersistentLayerDesc::new_delta( - tenantid, - timelineid, - fname.key_range.clone(), - fname.lsn_range.clone(), - layer_metadata.file_size(), - ), - layer_metadata: layer_metadata.clone(), - ongoing_download: Arc::new(tokio::sync::Semaphore::new(1)), - download_replacement_failure: std::sync::atomic::AtomicBool::default(), - access_stats, - } - } - - /// Create a Layer struct representing this layer, after it has been downloaded. - pub(crate) fn create_downloaded_layer( - &self, - _layer_map_lock_held_witness: &LayerManager, - conf: &'static PageServerConf, - file_size: u64, - ) -> Arc { - if self.desc.is_delta { - let fname = self.desc.delta_file_name(); - Arc::new(DeltaLayer::new( - conf, - self.desc.timeline_id, - self.desc.tenant_id, - &fname, - file_size, - self.access_stats - .clone_for_residence_change(LayerResidenceStatus::Resident), - )) - } else { - let fname = self.desc.image_file_name(); - Arc::new(ImageLayer::new( - conf, - self.desc.timeline_id, - self.desc.tenant_id, - &fname, - file_size, - self.access_stats - .clone_for_residence_change(LayerResidenceStatus::Resident), - )) - } - } -} diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index 91aad10210..278290eef4 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -11,18 +11,16 @@ use anyhow::{anyhow, bail, ensure, Context, Result}; use bytes::Bytes; use camino::{Utf8Path, Utf8PathBuf}; use fail::fail_point; -use futures::StreamExt; use itertools::Itertools; use pageserver_api::models::{ - DownloadRemoteLayersTaskInfo, DownloadRemoteLayersTaskSpawnRequest, - DownloadRemoteLayersTaskState, LayerMapInfo, LayerResidenceEventReason, LayerResidenceStatus, - TimelineState, + DownloadRemoteLayersTaskInfo, DownloadRemoteLayersTaskSpawnRequest, LayerMapInfo, TimelineState, }; -use remote_storage::GenericRemoteStorage; use serde_with::serde_as; use storage_broker::BrokerClientChannel; -use tokio::runtime::Handle; -use tokio::sync::{oneshot, watch, TryAcquireError}; +use tokio::{ + runtime::Handle, + sync::{oneshot, watch, TryAcquireError}, +}; use tokio_util::sync::CancellationToken; use tracing::*; use utils::id::TenantTimelineId; @@ -39,10 +37,11 @@ use crate::context::{ AccessStatsBehavior, DownloadBehavior, RequestContext, RequestContextBuilder, }; use crate::deletion_queue::DeletionQueueClient; -use crate::tenant::remote_timeline_client::index::LayerFileMetadata; use crate::tenant::storage_layer::delta_layer::DeltaEntry; use crate::tenant::storage_layer::{ - DeltaLayerWriter, ImageLayerWriter, InMemoryLayer, LayerAccessStats, LayerFileName, RemoteLayer, + AsLayerDesc, DeltaLayerWriter, EvictionError, ImageLayerWriter, InMemoryLayer, Layer, + LayerAccessStatsReset, LayerFileName, ResidentLayer, ValueReconstructResult, + ValueReconstructState, }; use crate::tenant::tasks::{BackgroundLoopKind, RateLimitError}; use crate::tenant::timeline::logical_size::CurrentLogicalSize; @@ -50,14 +49,12 @@ use crate::tenant::{ layer_map::{LayerMap, SearchResult}, metadata::{save_metadata, TimelineMetadata}, par_fsync, - storage_layer::{PersistentLayer, ValueReconstructResult, ValueReconstructState}, }; use crate::config::PageServerConf; use crate::keyspace::{KeyPartitioning, KeySpace, KeySpaceRandomAccum}; use crate::metrics::{ TimelineMetrics, MATERIALIZED_PAGE_CACHE_HIT, MATERIALIZED_PAGE_CACHE_HIT_DIRECT, - UNEXPECTED_ONDEMAND_DOWNLOADS, }; use crate::pgdatadir_mapping::LsnForTimestamp; use crate::pgdatadir_mapping::{is_rel_fsm_block_key, is_rel_vm_block_key}; @@ -93,9 +90,6 @@ use self::walreceiver::{WalReceiver, WalReceiverConf}; use super::config::TenantConf; use super::remote_timeline_client::index::IndexPart; use super::remote_timeline_client::RemoteTimelineClient; -use super::storage_layer::{ - AsLayerDesc, DeltaLayer, ImageLayer, LayerAccessStatsReset, PersistentLayerDesc, -}; use super::{debug_assert_current_span_has_tenant_and_timeline_id, AttachedTenantConf}; #[derive(Debug, PartialEq, Eq, Clone, Copy)] @@ -161,7 +155,7 @@ pub struct Timeline { /// /// This duplicates the generation stored in LocationConf, but that structure is mutable: /// this copy enforces the invariant that generatio doesn't change during a Tenant's lifetime. - generation: Generation, + pub(crate) generation: Generation, pub pg_version: u32, @@ -677,13 +671,11 @@ impl Timeline { } /// Outermost timeline compaction operation; downloads needed layers. - pub async fn compact( + pub(crate) async fn compact( self: &Arc, cancel: &CancellationToken, ctx: &RequestContext, - ) -> anyhow::Result<()> { - const ROUNDS: usize = 2; - + ) -> Result<(), CompactionError> { // this wait probably never needs any "long time spent" logging, because we already nag if // compaction task goes over it's period (20s) which is quite often in production. let _permit = match super::tasks::concurrent_background_tasks_rate_limit( @@ -705,87 +697,6 @@ impl Timeline { return Ok(()); } - // retry two times to allow first round to find layers which need to be downloaded, then - // download them, then retry compaction - for round in 0..ROUNDS { - // should we error out with the most specific error? - let last_round = round == ROUNDS - 1; - - let res = self.compact_inner(ctx).await; - - // If `create_image_layers' or `compact_level0` scheduled any - // uploads or deletions, but didn't update the index file yet, - // do it now. - // - // This isn't necessary for correctness, the remote state is - // consistent without the uploads and deletions, and we would - // update the index file on next flush iteration too. But it - // could take a while until that happens. - // - // Additionally, only do this once before we return from this function. - if last_round || res.is_ok() { - if let Some(remote_client) = &self.remote_client { - remote_client.schedule_index_upload_for_file_changes()?; - } - } - - let rls = match res { - Ok(()) => return Ok(()), - Err(CompactionError::DownloadRequired(rls)) if !last_round => { - // this can be done at most one time before exiting, waiting - rls - } - Err(CompactionError::DownloadRequired(rls)) => { - anyhow::bail!("Compaction requires downloading multiple times (last was {} layers), possibly battling against eviction", rls.len()) - } - Err(CompactionError::ShuttingDown) => { - return Ok(()); - } - Err(CompactionError::Other(e)) => { - return Err(e); - } - }; - - // this path can be visited in the second round of retrying, if first one found that we - // must first download some remote layers - let total = rls.len(); - - let mut downloads = rls - .into_iter() - .map(|rl| self.download_remote_layer(rl)) - .collect::>(); - - let mut failed = 0; - - loop { - tokio::select! { - _ = cancel.cancelled() => anyhow::bail!("Cancelled while downloading remote layers"), - res = downloads.next() => { - match res { - Some(Ok(())) => {}, - Some(Err(e)) => { - warn!("Downloading remote layer for compaction failed: {e:#}"); - failed += 1; - } - None => break, - } - } - } - } - - if failed != 0 { - anyhow::bail!("{failed} out of {total} layers failed to download, retrying later"); - } - - // if everything downloaded fine, lets try again - } - - unreachable!("retry loop exits") - } - - /// Compaction which might need to be retried after downloading remote layers. - async fn compact_inner(self: &Arc, ctx: &RequestContext) -> Result<(), CompactionError> { - // // High level strategy for compaction / image creation: // // 1. First, calculate the desired "partitioning" of the @@ -830,6 +741,7 @@ impl Timeline { // Define partitioning schema if needed + // FIXME: the match should only cover repartitioning, not the next steps match self .repartition( self.get_last_record_lsn(), @@ -846,13 +758,13 @@ impl Timeline { // 2. Create new image layers for partitions that have been modified // "enough". - let layer_paths_to_upload = self + let layers = self .create_image_layers(&partitioning, lsn, false, &image_ctx) .await .map_err(anyhow::Error::from)?; if let Some(remote_client) = &self.remote_client { - for (path, layer_metadata) in layer_paths_to_upload { - remote_client.schedule_layer_file_upload(&path, &layer_metadata)?; + for layer in layers { + remote_client.schedule_layer_file_upload(layer)?; } } @@ -861,6 +773,13 @@ impl Timeline { self.compact_level0(layer_removal_cs.clone(), target_file_size, ctx) .await?; timer.stop_and_record(); + + if let Some(remote_client) = &self.remote_client { + // should any new image layer been created, not uploading index_part will + // result in a mismatch between remote_physical_size and layermap calculated + // size, which will fail some tests, but should not be an issue otherwise. + remote_client.schedule_index_upload_for_file_changes()?; + } } Err(err) => { // no partitioning? This is normal, if the timeline was just created @@ -1116,14 +1035,13 @@ impl Timeline { let Some(layer) = self.find_layer(layer_file_name).await else { return Ok(None); }; - let Some(remote_layer) = layer.downcast_remote_layer() else { - return Ok(Some(false)); - }; + if self.remote_client.is_none() { return Ok(Some(false)); } - self.download_remote_layer(remote_layer).await?; + layer.download().await?; + Ok(Some(true)) } @@ -1133,6 +1051,13 @@ impl Timeline { let Some(local_layer) = self.find_layer(layer_file_name).await else { return Ok(None); }; + + let Some(local_layer) = local_layer.keep_resident().await? else { + return Ok(Some(false)); + }; + + let local_layer: Layer = local_layer.into(); + let remote_client = self .remote_client .as_ref() @@ -1140,7 +1065,7 @@ impl Timeline { let cancel = CancellationToken::new(); let results = self - .evict_layer_batch(remote_client, &[local_layer], cancel) + .evict_layer_batch(remote_client, &[local_layer], &cancel) .await?; assert_eq!(results.len(), 1); let result: Option> = results.into_iter().next().unwrap(); @@ -1152,31 +1077,22 @@ impl Timeline { } /// Evict a batch of layers. - /// - /// GenericRemoteStorage reference is required as a (witness)[witness_article] for "remote storage is configured." - /// - /// [witness_article]: https://willcrichton.net/rust-api-type-patterns/witnesses.html pub(crate) async fn evict_layers( &self, - _: &GenericRemoteStorage, - layers_to_evict: &[Arc], - cancel: CancellationToken, + layers_to_evict: &[Layer], + cancel: &CancellationToken, ) -> anyhow::Result>>> { - let remote_client = self.remote_client.clone().expect( - "GenericRemoteStorage is configured, so timeline must have RemoteTimelineClient", - ); + let remote_client = self + .remote_client + .as_ref() + .context("timeline must have RemoteTimelineClient")?; - self.evict_layer_batch(&remote_client, layers_to_evict, cancel) + self.evict_layer_batch(remote_client, layers_to_evict, cancel) .await } /// Evict multiple layers at once, continuing through errors. /// - /// Try to evict the given `layers_to_evict` by - /// - /// 1. Replacing the given layer object in the layer map with a corresponding [`RemoteLayer`] object. - /// 2. Deleting the now unreferenced layer file from disk. - /// /// The `remote_client` should be this timeline's `self.remote_client`. /// We make the caller provide it so that they are responsible for handling the case /// where someone wants to evict the layer but no remote storage is configured. @@ -1185,17 +1101,15 @@ impl Timeline { /// If `Err()` is returned, no eviction was attempted. /// Each position of `Ok(results)` corresponds to the layer in `layers_to_evict`. /// Meaning of each `result[i]`: - /// - `Some(Err(...))` if layer replacement failed for an unexpected reason - /// - `Some(Ok(true))` if everything went well. - /// - `Some(Ok(false))` if there was an expected reason why the layer could not be replaced, e.g.: - /// - evictee was not yet downloaded + /// - `Some(Err(...))` if layer replacement failed for some reason /// - replacement failed for an expectable reason (e.g., layer removed by GC before we grabbed all locks) + /// - `Some(Ok(()))` if everything went well. /// - `None` if no eviction attempt was made for the layer because `cancel.is_cancelled() == true`. async fn evict_layer_batch( &self, remote_client: &Arc, - layers_to_evict: &[Arc], - cancel: CancellationToken, + layers_to_evict: &[Layer], + cancel: &CancellationToken, ) -> anyhow::Result>>> { // ensure that the layers have finished uploading // (don't hold the layer_removal_cs while we do it, we're not removing anything yet) @@ -1205,7 +1119,7 @@ impl Timeline { .context("wait for layer upload ops to complete")?; // now lock out layer removal (compaction, gc, timeline deletion) - let layer_removal_guard = self.layer_removal_cs.lock().await; + let _layer_removal_guard = self.layer_removal_cs.lock().await; { // to avoid racing with detach and delete_timeline @@ -1216,163 +1130,40 @@ impl Timeline { ); } - // start the batch update - let mut guard = self.layers.write().await; let mut results = Vec::with_capacity(layers_to_evict.len()); - - for l in layers_to_evict.iter() { - let res = if cancel.is_cancelled() { - None - } else { - Some(self.evict_layer_batch_impl(&layer_removal_guard, l, &mut guard)) - }; - results.push(res); + for _ in 0..layers_to_evict.len() { + results.push(None); } - // commit the updates & release locks - drop_wlock(guard); - drop(layer_removal_guard); + let mut js = tokio::task::JoinSet::new(); + + for (i, l) in layers_to_evict.iter().enumerate() { + js.spawn({ + let l = l.to_owned(); + let remote_client = remote_client.clone(); + async move { (i, l.evict_and_wait(&remote_client).await) } + }); + } + + let join = async { + while let Some(next) = js.join_next().await { + match next { + Ok((i, res)) => results[i] = Some(res), + Err(je) if je.is_cancelled() => unreachable!("not used"), + Err(je) if je.is_panic() => { /* already logged */ } + Err(je) => tracing::error!("unknown JoinError: {je:?}"), + } + } + }; + + tokio::select! { + _ = cancel.cancelled() => {}, + _ = join => {} + } assert_eq!(results.len(), layers_to_evict.len()); Ok(results) } - - fn evict_layer_batch_impl( - &self, - _layer_removal_cs: &tokio::sync::MutexGuard<'_, ()>, - local_layer: &Arc, - layer_mgr: &mut LayerManager, - ) -> Result<(), EvictionError> { - if local_layer.is_remote_layer() { - return Err(EvictionError::CannotEvictRemoteLayer); - } - - let layer_file_size = local_layer.layer_desc().file_size; - - let local_layer_mtime = local_layer - .local_path() - .expect("local layer should have a local path") - .metadata() - // when the eviction fails because we have already deleted the layer in compaction for - // example, a NotFound error bubbles up from here. - .map_err(|e| { - if e.kind() == std::io::ErrorKind::NotFound { - EvictionError::FileNotFound - } else { - EvictionError::StatFailed(e) - } - })? - .modified() - .map_err(EvictionError::StatFailed)?; - - let local_layer_residence_duration = - match SystemTime::now().duration_since(local_layer_mtime) { - Err(e) => { - warn!(layer = %local_layer, "layer mtime is in the future: {}", e); - None - } - Ok(delta) => Some(delta), - }; - - // RemoteTimelineClient holds the metadata on layers' remote generations, so - // query it to construct a RemoteLayer. - let layer_metadata = self - .remote_client - .as_ref() - .expect("Eviction is not called without remote storage") - .get_layer_metadata(&local_layer.filename()) - .map_err(EvictionError::LayerNotFound)? - .ok_or_else(|| { - EvictionError::LayerNotFound(anyhow::anyhow!("Layer not in remote metadata")) - })?; - if layer_metadata.file_size() != layer_file_size { - return Err(EvictionError::MetadataInconsistency(format!( - "Layer size {layer_file_size} doesn't match remote metadata file size {}", - layer_metadata.file_size() - ))); - } - - let new_remote_layer = Arc::new(match local_layer.filename() { - LayerFileName::Image(image_name) => RemoteLayer::new_img( - self.tenant_id, - self.timeline_id, - &image_name, - &layer_metadata, - local_layer - .access_stats() - .clone_for_residence_change(LayerResidenceStatus::Evicted), - ), - LayerFileName::Delta(delta_name) => RemoteLayer::new_delta( - self.tenant_id, - self.timeline_id, - &delta_name, - &layer_metadata, - local_layer - .access_stats() - .clone_for_residence_change(LayerResidenceStatus::Evicted), - ), - }); - - assert_eq!(local_layer.layer_desc(), new_remote_layer.layer_desc()); - - layer_mgr - .replace_and_verify(local_layer.clone(), new_remote_layer) - .map_err(EvictionError::LayerNotFound)?; - - if let Err(e) = local_layer.delete_resident_layer_file() { - // this should never happen, because of layer_removal_cs usage and above stat - // access for mtime - error!("failed to remove layer file on evict after replacement: {e:#?}"); - } - // Always decrement the physical size gauge, even if we failed to delete the file. - // Rationale: we already replaced the layer with a remote layer in the layer map, - // and any subsequent download_remote_layer will - // 1. overwrite the file on disk and - // 2. add the downloaded size to the resident size gauge. - // - // If there is no re-download, and we restart the pageserver, then load_layer_map - // will treat the file as a local layer again, count it towards resident size, - // and it'll be like the layer removal never happened. - // The bump in resident size is perhaps unexpected but overall a robust behavior. - self.metrics.resident_physical_size_sub(layer_file_size); - self.metrics.evictions.inc(); - - if let Some(delta) = local_layer_residence_duration { - self.metrics - .evictions_with_low_residence_duration - .read() - .unwrap() - .observe(delta); - info!(layer=%local_layer, residence_millis=delta.as_millis(), "evicted layer after known residence period"); - } else { - info!(layer=%local_layer, "evicted layer after unknown residence period"); - } - - Ok(()) - } -} - -#[derive(Debug, thiserror::Error)] -pub(crate) enum EvictionError { - #[error("cannot evict a remote layer")] - CannotEvictRemoteLayer, - /// Most likely the to-be evicted layer has been deleted by compaction or gc which use the same - /// locks, so they got to execute before the eviction. - #[error("file backing the layer has been removed already")] - FileNotFound, - #[error("stat failed")] - StatFailed(#[source] std::io::Error), - /// In practice, this can be a number of things, but lets assume it means only this. - /// - /// This case includes situations such as the Layer was evicted and redownloaded in between, - /// because the file existed before an replacement attempt was made but now the Layers are - /// different objects in memory. - #[error("layer was no longer part of LayerMap")] - LayerNotFound(#[source] anyhow::Error), - - /// This should never happen - #[error("Metadata inconsistency")] - MetadataInconsistency(String), } /// Number of times we will compute partition within a checkpoint distance. @@ -1709,11 +1500,12 @@ impl Timeline { // Scan timeline directory and create ImageFileName and DeltaFilename // structs representing all files on disk let timeline_path = self.conf.timeline_path(&self.tenant_id, &self.timeline_id); - let (conf, tenant_id, timeline_id) = (self.conf, self.tenant_id, self.timeline_id); + let conf = self.conf; let span = tracing::Span::current(); // Copy to move into the task we're about to spawn let generation = self.generation; + let this = self.myself.upgrade().expect("&self method holds the arc"); let (loaded_layers, needs_cleanup, total_physical_size) = tokio::task::spawn_blocking({ move || { @@ -1807,44 +1599,16 @@ impl Timeline { Image(i) => assert!(i.lsn <= disk_consistent_lsn), } - let status = match &decision { - UseLocal(_) => LayerResidenceStatus::Resident, - Evicted(_) | UseRemote { .. } => LayerResidenceStatus::Evicted, - }; + tracing::debug!(layer=%name, ?decision, "applied"); - tracing::debug!(layer=%name, ?decision, ?status, "applied"); - - let stats = LayerAccessStats::for_loading_layer(status); - - let layer: Arc = match (name, &decision) { - (Delta(d), UseLocal(m)) => { + let layer = match decision { + UseLocal(m) => { total_physical_size += m.file_size(); - Arc::new(DeltaLayer::new( - conf, - timeline_id, - tenant_id, - &d, - m.file_size(), - stats, - )) + Layer::for_resident(conf, &this, name, m).drop_eviction_guard() } - (Image(i), UseLocal(m)) => { - total_physical_size += m.file_size(); - Arc::new(ImageLayer::new( - conf, - timeline_id, - tenant_id, - &i, - m.file_size(), - stats, - )) + Evicted(remote) | UseRemote { remote, .. } => { + Layer::for_evicted(conf, &this, name, remote) } - (Delta(d), Evicted(remote) | UseRemote { remote, .. }) => Arc::new( - RemoteLayer::new_delta(tenant_id, timeline_id, &d, remote, stats), - ), - (Image(i), Evicted(remote) | UseRemote { remote, .. }) => Arc::new( - RemoteLayer::new_img(tenant_id, timeline_id, &i, remote, stats), - ), }; loaded_layers.push(layer); @@ -1861,7 +1625,7 @@ impl Timeline { guard.initialize_local_layers(loaded_layers, disk_consistent_lsn + 1); if let Some(rtc) = self.remote_client.as_ref() { - rtc.schedule_layer_file_deletion(needs_cleanup)?; + rtc.schedule_layer_file_deletion(&needs_cleanup)?; rtc.schedule_index_upload_for_file_changes()?; // Tenant::create_timeline will wait for these uploads to happen before returning, or // on retry. @@ -1871,7 +1635,6 @@ impl Timeline { "loaded layer map with {} layers at {}, total physical size: {}", num_layers, disk_consistent_lsn, total_physical_size ); - self.metrics.resident_physical_size_add(total_physical_size); timer.stop_and_record(); Ok(()) @@ -2195,7 +1958,7 @@ impl Timeline { } } - async fn find_layer(&self, layer_file_name: &str) -> Option> { + async fn find_layer(&self, layer_file_name: &str) -> Option { let guard = self.layers.read().await; for historic_layer in guard.layer_map().iter_historic_layers() { let historic_layer_name = historic_layer.filename().file_name(); @@ -2214,20 +1977,9 @@ trait TraversalLayerExt { fn traversal_id(&self) -> TraversalId; } -impl TraversalLayerExt for Arc { +impl TraversalLayerExt for Layer { fn traversal_id(&self) -> TraversalId { - let timeline_id = self.layer_desc().timeline_id; - match self.local_path() { - Some(local_path) => { - debug_assert!(local_path.to_string().contains(&format!("{}", timeline_id)), - "need timeline ID to uniquely identify the layer when traversal crosses ancestor boundary", - ); - format!("{local_path}") - } - None => { - format!("remote {}/{self}", timeline_id) - } - } + self.local_path().to_string() } } @@ -2386,174 +2138,107 @@ impl Timeline { continue 'outer; } - #[allow(clippy::never_loop)] // see comment at bottom of this loop - 'layer_map_search: loop { - let remote_layer = { - let guard = timeline.layers.read().await; - let layers = guard.layer_map(); + let guard = timeline.layers.read().await; + let layers = guard.layer_map(); - // Check the open and frozen in-memory layers first, in order from newest - // to oldest. - if let Some(open_layer) = &layers.open_layer { - let start_lsn = open_layer.get_lsn_range().start; - if cont_lsn > start_lsn { - //info!("CHECKING for {} at {} on open layer {}", key, cont_lsn, open_layer.filename().display()); - // Get all the data needed to reconstruct the page version from this layer. - // But if we have an older cached page image, no need to go past that. - let lsn_floor = max(cached_lsn + 1, start_lsn); - result = match open_layer - .get_value_reconstruct_data( - key, - lsn_floor..cont_lsn, - reconstruct_state, - ctx, - ) - .await - { - Ok(result) => result, - Err(e) => return Err(PageReconstructError::from(e)), - }; - cont_lsn = lsn_floor; - // metrics: open_layer does not count as fs access, so we are not updating `read_count` - traversal_path.push(( - result, - cont_lsn, - Box::new({ - let open_layer = Arc::clone(open_layer); - move || open_layer.traversal_id() - }), - )); - continue 'outer; - } - } - for frozen_layer in layers.frozen_layers.iter().rev() { - let start_lsn = frozen_layer.get_lsn_range().start; - if cont_lsn > start_lsn { - //info!("CHECKING for {} at {} on frozen layer {}", key, cont_lsn, frozen_layer.filename().display()); - let lsn_floor = max(cached_lsn + 1, start_lsn); - result = match frozen_layer - .get_value_reconstruct_data( - key, - lsn_floor..cont_lsn, - reconstruct_state, - ctx, - ) - .await - { - Ok(result) => result, - Err(e) => return Err(PageReconstructError::from(e)), - }; - cont_lsn = lsn_floor; - // metrics: open_layer does not count as fs access, so we are not updating `read_count` - traversal_path.push(( - result, - cont_lsn, - Box::new({ - let frozen_layer = Arc::clone(frozen_layer); - move || frozen_layer.traversal_id() - }), - )); - continue 'outer; - } - } - - if let Some(SearchResult { lsn_floor, layer }) = layers.search(key, cont_lsn) { - let layer = guard.get_from_desc(&layer); - // If it's a remote layer, download it and retry. - if let Some(remote_layer) = - super::storage_layer::downcast_remote_layer(&layer) - { - // TODO: push a breadcrumb to 'traversal_path' to record the fact that - // we downloaded / would need to download this layer. - remote_layer // download happens outside the scope of `layers` guard object - } else { - // Get all the data needed to reconstruct the page version from this layer. - // But if we have an older cached page image, no need to go past that. - let lsn_floor = max(cached_lsn + 1, lsn_floor); - result = match layer - .get_value_reconstruct_data( - key, - lsn_floor..cont_lsn, - reconstruct_state, - ctx, - ) - .await - { - Ok(result) => result, - Err(e) => return Err(PageReconstructError::from(e)), - }; - cont_lsn = lsn_floor; - *read_count += 1; - traversal_path.push(( - result, - cont_lsn, - Box::new({ - let layer = Arc::clone(&layer); - move || layer.traversal_id() - }), - )); - continue 'outer; - } - } else if timeline.ancestor_timeline.is_some() { - // Nothing on this timeline. Traverse to parent - result = ValueReconstructResult::Continue; - cont_lsn = Lsn(timeline.ancestor_lsn.0 + 1); - continue 'outer; - } else { - // Nothing found - result = ValueReconstructResult::Missing; - continue 'outer; - } - }; - // Download the remote_layer and replace it in the layer map. - // For that, we need to release the mutex. Otherwise, we'd deadlock. - // - // The control flow is so weird here because `drop(layers)` inside - // the if stmt above is not enough for current rustc: it requires - // that the layers lock guard is not in scope across the download - // await point. - let remote_layer_as_persistent: Arc = - Arc::clone(&remote_layer) as Arc; - let id = remote_layer_as_persistent.traversal_id(); - info!( - "need remote layer {} for task kind {:?}", - id, - ctx.task_kind() - ); - - // The next layer doesn't exist locally. Need to download it. - // (The control flow is a bit complicated here because we must drop the 'layers' - // lock before awaiting on the Future.) - match ( - ctx.download_behavior(), - self.conf.ondemand_download_behavior_treat_error_as_warn, - ) { - (DownloadBehavior::Download, _) => { - info!( - "on-demand downloading remote layer {id} for task kind {:?}", - ctx.task_kind() - ); - timeline.download_remote_layer(remote_layer).await?; - continue 'layer_map_search; - } - (DownloadBehavior::Warn, _) | (DownloadBehavior::Error, true) => { - warn!( - "unexpectedly on-demand downloading remote layer {} for task kind {:?}", - id, - ctx.task_kind() - ); - UNEXPECTED_ONDEMAND_DOWNLOADS.inc(); - timeline.download_remote_layer(remote_layer).await?; - continue 'layer_map_search; - } - (DownloadBehavior::Error, false) => { - return Err(PageReconstructError::NeedsDownload( - TenantTimelineId::new(self.tenant_id, self.timeline_id), - remote_layer.filename(), - )) - } + // Check the open and frozen in-memory layers first, in order from newest + // to oldest. + if let Some(open_layer) = &layers.open_layer { + let start_lsn = open_layer.get_lsn_range().start; + if cont_lsn > start_lsn { + //info!("CHECKING for {} at {} on open layer {}", key, cont_lsn, open_layer.filename().display()); + // Get all the data needed to reconstruct the page version from this layer. + // But if we have an older cached page image, no need to go past that. + let lsn_floor = max(cached_lsn + 1, start_lsn); + result = match open_layer + .get_value_reconstruct_data( + key, + lsn_floor..cont_lsn, + reconstruct_state, + ctx, + ) + .await + { + Ok(result) => result, + Err(e) => return Err(PageReconstructError::from(e)), + }; + cont_lsn = lsn_floor; + // metrics: open_layer does not count as fs access, so we are not updating `read_count` + traversal_path.push(( + result, + cont_lsn, + Box::new({ + let open_layer = Arc::clone(open_layer); + move || open_layer.traversal_id() + }), + )); + continue 'outer; } } + for frozen_layer in layers.frozen_layers.iter().rev() { + let start_lsn = frozen_layer.get_lsn_range().start; + if cont_lsn > start_lsn { + //info!("CHECKING for {} at {} on frozen layer {}", key, cont_lsn, frozen_layer.filename().display()); + let lsn_floor = max(cached_lsn + 1, start_lsn); + result = match frozen_layer + .get_value_reconstruct_data( + key, + lsn_floor..cont_lsn, + reconstruct_state, + ctx, + ) + .await + { + Ok(result) => result, + Err(e) => return Err(PageReconstructError::from(e)), + }; + cont_lsn = lsn_floor; + // metrics: open_layer does not count as fs access, so we are not updating `read_count` + traversal_path.push(( + result, + cont_lsn, + Box::new({ + let frozen_layer = Arc::clone(frozen_layer); + move || frozen_layer.traversal_id() + }), + )); + continue 'outer; + } + } + + if let Some(SearchResult { lsn_floor, layer }) = layers.search(key, cont_lsn) { + let layer = guard.get_from_desc(&layer); + // Get all the data needed to reconstruct the page version from this layer. + // But if we have an older cached page image, no need to go past that. + let lsn_floor = max(cached_lsn + 1, lsn_floor); + result = match layer + .get_value_reconstruct_data(key, lsn_floor..cont_lsn, reconstruct_state, ctx) + .await + { + Ok(result) => result, + Err(e) => return Err(PageReconstructError::from(e)), + }; + cont_lsn = lsn_floor; + *read_count += 1; + traversal_path.push(( + result, + cont_lsn, + Box::new({ + let layer = layer.to_owned(); + move || layer.traversal_id() + }), + )); + continue 'outer; + } else if timeline.ancestor_timeline.is_some() { + // Nothing on this timeline. Traverse to parent + result = ValueReconstructResult::Continue; + cont_lsn = Lsn(timeline.ancestor_lsn.0 + 1); + continue 'outer; + } else { + // Nothing found + result = ValueReconstructResult::Missing; + continue 'outer; + } } } @@ -2742,7 +2427,7 @@ impl Timeline { // files instead. This is possible as long as *all* the data imported into the // repository have the same LSN. let lsn_range = frozen_layer.get_lsn_range(); - let (layer_paths_to_upload, delta_layer_to_add) = + let (layers_to_upload, delta_layer_to_add) = if lsn_range.start == self.initdb_lsn && lsn_range.end == Lsn(self.initdb_lsn.0 + 1) { #[cfg(test)] match &mut *self.flush_loop_state.lock().unwrap() { @@ -2785,10 +2470,9 @@ impl Timeline { // We will remove frozen layer and add delta layer in one atomic operation later. let layer = self.create_delta_layer(&frozen_layer, ctx).await?; ( - HashMap::from([( - layer.filename(), - LayerFileMetadata::new(layer.layer_desc().file_size, self.generation), - )]), + // FIXME: even though we have a single image and single delta layer assumption + // we push them to vec + vec![layer.clone()], Some(layer), ) }; @@ -2802,25 +2486,14 @@ impl Timeline { let metadata = { let mut guard = self.layers.write().await; - if let Some(ref l) = delta_layer_to_add { - // TODO: move access stats, metrics update, etc. into layer manager. - l.access_stats().record_residence_event( - LayerResidenceStatus::Resident, - LayerResidenceEventReason::LayerCreate, - ); + guard.finish_flush_l0_layer(delta_layer_to_add.as_ref(), &frozen_layer, &self.metrics); - // update metrics - let sz = l.layer_desc().file_size; - self.metrics.record_new_file_metrics(sz); - } - - guard.finish_flush_l0_layer(delta_layer_to_add, &frozen_layer); if disk_consistent_lsn != old_disk_consistent_lsn { assert!(disk_consistent_lsn > old_disk_consistent_lsn); self.disk_consistent_lsn.store(disk_consistent_lsn); // Schedule remote uploads that will reflect our new disk_consistent_lsn - Some(self.schedule_uploads(disk_consistent_lsn, layer_paths_to_upload)?) + Some(self.schedule_uploads(disk_consistent_lsn, layers_to_upload)?) } else { None } @@ -2855,7 +2528,7 @@ impl Timeline { fn schedule_uploads( &self, disk_consistent_lsn: Lsn, - layer_paths_to_upload: HashMap, + layers_to_upload: impl IntoIterator, ) -> anyhow::Result { // We can only save a valid 'prev_record_lsn' value on disk if we // flushed *all* in-memory changes to disk. We only track @@ -2894,8 +2567,8 @@ impl Timeline { )); if let Some(remote_client) = &self.remote_client { - for (path, layer_metadata) in layer_paths_to_upload { - remote_client.schedule_layer_file_upload(&path, &layer_metadata)?; + for layer in layers_to_upload { + remote_client.schedule_layer_file_upload(layer)?; } remote_client.schedule_index_upload_for_metadata_update(&metadata)?; } @@ -2906,9 +2579,9 @@ impl Timeline { async fn update_metadata_file( &self, disk_consistent_lsn: Lsn, - layer_paths_to_upload: HashMap, + layers_to_upload: impl IntoIterator, ) -> anyhow::Result<()> { - let metadata = self.schedule_uploads(disk_consistent_lsn, layer_paths_to_upload)?; + let metadata = self.schedule_uploads(disk_consistent_lsn, layers_to_upload)?; save_metadata(self.conf, &self.tenant_id, &self.timeline_id, &metadata) .await @@ -2923,10 +2596,9 @@ impl Timeline { self: &Arc, frozen_layer: &Arc, ctx: &RequestContext, - ) -> anyhow::Result { + ) -> anyhow::Result { let span = tracing::info_span!("blocking"); - let new_delta: DeltaLayer = tokio::task::spawn_blocking({ - let _g = span.entered(); + let new_delta: ResidentLayer = tokio::task::spawn_blocking({ let self_clone = Arc::clone(self); let frozen_layer = Arc::clone(frozen_layer); let ctx = ctx.attached_child(); @@ -2936,8 +2608,10 @@ impl Timeline { // as long as the write path is still sync and the read impl // is still not fully async. Otherwise executor threads would // be blocked. - let new_delta = Handle::current().block_on(frozen_layer.write_to_disk(&ctx))?; - let new_delta_path = new_delta.path(); + let _g = span.entered(); + let new_delta = + Handle::current().block_on(frozen_layer.write_to_disk(&self_clone, &ctx))?; + let new_delta_path = new_delta.local_path().to_owned(); // Sync it to disk. // @@ -2954,6 +2628,8 @@ impl Timeline { // 3. rename to the final name // 4. fsync the parent directory. // Note that (1),(2),(3) today happen inside write_to_disk(). + // + // FIXME: the writer already fsyncs all data, only rename needs to be fsynced here par_fsync::par_fsync(&[new_delta_path]).context("fsync of delta layer")?; par_fsync::par_fsync(&[self_clone .conf @@ -2964,7 +2640,8 @@ impl Timeline { } }) .await - .context("spawn_blocking")??; + .context("spawn_blocking") + .and_then(|x| x)?; Ok(new_delta) } @@ -3081,14 +2758,14 @@ impl Timeline { #[tracing::instrument(skip_all, fields(%lsn, %force))] async fn create_image_layers( - &self, + self: &Arc, partitioning: &KeyPartitioning, lsn: Lsn, force: bool, ctx: &RequestContext, - ) -> Result, PageReconstructError> { + ) -> Result, PageReconstructError> { let timer = self.metrics.create_images_time_histo.start_timer(); - let mut image_layers: Vec = Vec::new(); + let mut image_layers = Vec::new(); // We need to avoid holes between generated image layers. // Otherwise LayerMap::image_layer_exists will return false if key range of some layer is covered by more than one @@ -3152,7 +2829,7 @@ impl Timeline { key = key.next(); } } - let image_layer = image_layer_writer.finish().await?; + let image_layer = image_layer_writer.finish(self).await?; image_layers.push(image_layer); } } @@ -3174,7 +2851,7 @@ impl Timeline { // and fsync them all in parallel. let all_paths = image_layers .iter() - .map(|layer| layer.path()) + .map(|layer| layer.local_path().to_owned()) .collect::>(); par_fsync::par_fsync_async(&all_paths) @@ -3185,63 +2862,32 @@ impl Timeline { .await .context("fsync of timeline dir")?; - let mut layer_paths_to_upload = HashMap::with_capacity(image_layers.len()); - let mut guard = self.layers.write().await; - let timeline_path = self.conf.timeline_path(&self.tenant_id, &self.timeline_id); - for l in &image_layers { - let path = l.filename(); - let metadata = timeline_path - .join(path.file_name()) - .metadata() - .with_context(|| format!("reading metadata of layer file {}", path.file_name()))?; - - layer_paths_to_upload.insert( - path, - LayerFileMetadata::new(metadata.len(), self.generation), - ); - - // update metrics - self.metrics.record_new_file_metrics(metadata.len()); - let l = Arc::new(l); - l.access_stats().record_residence_event( - LayerResidenceStatus::Resident, - LayerResidenceEventReason::LayerCreate, - ); - } - guard.track_new_image_layers(image_layers); + // FIXME: we could add the images to be uploaded *before* returning from here, but right + // now they are being scheduled outside of write lock + guard.track_new_image_layers(&image_layers, &self.metrics); drop_wlock(guard); timer.stop_and_record(); - Ok(layer_paths_to_upload) + Ok(image_layers) } } #[derive(Default)] struct CompactLevel0Phase1Result { - new_layers: Vec>, - deltas_to_compact: Vec>, + new_layers: Vec, + deltas_to_compact: Vec, } /// Top-level failure to compact. -#[derive(Debug)] -enum CompactionError { - /// L0 compaction requires layers to be downloaded. - /// - /// This should not happen repeatedly, but will be retried once by top-level - /// `Timeline::compact`. - DownloadRequired(Vec>), - /// The timeline or pageserver is shutting down +#[derive(Debug, thiserror::Error)] +pub(crate) enum CompactionError { + #[error("The timeline or pageserver is shutting down")] ShuttingDown, /// Compaction cannot be done right now; page reconstruction and so on. - Other(anyhow::Error), -} - -impl From for CompactionError { - fn from(value: anyhow::Error) -> Self { - CompactionError::Other(value) - } + #[error(transparent)] + Other(#[from] anyhow::Error), } #[serde_as] @@ -3366,13 +3012,11 @@ impl TryFrom for CompactLevel0Phase1Stats { } impl Timeline { - /// Level0 files first phase of compaction, explained in the [`compact_inner`] comment. + /// Level0 files first phase of compaction, explained in the [`Self::compact`] comment. /// /// This method takes the `_layer_removal_cs` guard to highlight it required downloads are /// returned as an error. If the `layer_removal_cs` boundary is changed not to be taken in the /// start of level0 files compaction, the on-demand download should be revisited as well. - /// - /// [`compact_inner`]: Self::compact_inner async fn compact_level0_phase1( self: &Arc, _layer_removal_cs: Arc>, @@ -3417,19 +3061,30 @@ impl Timeline { // size length. Compaction will likely create the same set of n files afterwards. // // This failpoint is a superset of both of the cases. - fail_point!("compact-level0-phase1-return-same", |_| { - println!("compact-level0-phase1-return-same"); // so that we can check if we hit the failpoint - Ok(CompactLevel0Phase1Result { - new_layers: level0_deltas - .iter() - .map(|x| x.clone().downcast_delta_layer().unwrap()) - .collect(), - deltas_to_compact: level0_deltas - .iter() - .map(|x| x.layer_desc().clone().into()) - .collect(), - }) - }); + 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. // @@ -3449,14 +3104,18 @@ impl Timeline { let first_level0_delta = level0_deltas_iter.next().unwrap(); let mut prev_lsn_end = first_level0_delta.layer_desc().lsn_range.end; - let mut deltas_to_compact = vec![Arc::clone(first_level0_delta)]; + let mut deltas_to_compact = Vec::with_capacity(level0_deltas.len()); + + // FIXME: downloading while holding layer_removal_cs is not great, but we will remove that + // soon + deltas_to_compact.push(first_level0_delta.download_and_keep_resident().await?); for l in level0_deltas_iter { let lsn_range = &l.layer_desc().lsn_range; if lsn_range.start != prev_lsn_end { break; } - deltas_to_compact.push(Arc::clone(l)); + deltas_to_compact.push(l.download_and_keep_resident().await?); prev_lsn_end = lsn_range.end; } let lsn_range = Range { @@ -3469,24 +3128,6 @@ impl Timeline { end: deltas_to_compact.last().unwrap().layer_desc().lsn_range.end, }; - let remotes = deltas_to_compact - .iter() - .filter(|l| l.is_remote_layer()) - .inspect(|l| info!("compact requires download of {l}")) - .map(|l| { - l.clone() - .downcast_remote_layer() - .expect("just checked it is remote layer") - }) - .collect::>(); - - if !remotes.is_empty() { - // caller is holding the lock to layer_removal_cs, and we don't want to download while - // holding that; in future download_remote_layer might take it as well. this is - // regardless of earlier image creation downloading on-demand, while holding the lock. - return Err(CompactionError::DownloadRequired(remotes)); - } - info!( "Starting Level0 compaction in LSN range {}-{} for {} layers ({} deltas in total)", lsn_range.start, @@ -3519,23 +3160,19 @@ impl Timeline { let mut all_keys = Vec::new(); - let downcast_deltas: Vec<_> = deltas_to_compact - .iter() - .map(|l| l.clone().downcast_delta_layer().expect("delta layer")) - .collect(); - for dl in downcast_deltas.iter() { - // TODO: replace this with an await once we fully go async - all_keys.extend(DeltaLayer::load_keys(dl, ctx).await?); + for l in deltas_to_compact.iter() { + all_keys.extend(l.load_keys(ctx).await?); } + // FIXME: should spawn_blocking the rest of this function + // The current stdlib sorting implementation is designed in a way where it is // particularly fast where the slice is made up of sorted sub-ranges. all_keys.sort_by_key(|DeltaEntry { key, lsn, .. }| (*key, *lsn)); stats.read_lock_held_key_sort_micros = stats.read_lock_held_prerequisites_micros.till_now(); - for DeltaEntry { key: next_key, .. } in all_keys.iter() { - let next_key = *next_key; + for &DeltaEntry { key: next_key, .. } in all_keys.iter() { if let Some(prev_key) = prev { // just first fast filter if next_key.to_i128() - prev_key.to_i128() >= min_hole_range { @@ -3695,13 +3332,13 @@ impl Timeline { || contains_hole { // ... if so, flush previous layer and prepare to write new one - new_layers.push(Arc::new( + new_layers.push( writer .take() .unwrap() - .finish(prev_key.unwrap().next()) + .finish(prev_key.unwrap().next(), self) .await?, - )); + ); writer = None; if contains_hole { @@ -3749,7 +3386,7 @@ impl Timeline { prev_key = Some(key); } if let Some(writer) = writer { - new_layers.push(Arc::new(writer.finish(prev_key.unwrap().next()).await?)); + new_layers.push(writer.finish(prev_key.unwrap().next(), self).await?); } // Sync layers @@ -3767,10 +3404,17 @@ impl Timeline { ); } } - let mut layer_paths: Vec = new_layers.iter().map(|l| l.path()).collect(); + + // FIXME: the writer already fsyncs all data, only rename needs to be fsynced here + let mut layer_paths: Vec = new_layers + .iter() + .map(|l| l.local_path().to_owned()) + .collect(); // Fsync all the layer files and directory using multiple threads to // minimize latency. + // + // FIXME: spawn_blocking above for this par_fsync::par_fsync(&layer_paths).context("fsync all new layers")?; par_fsync::par_fsync(&[self.conf.timeline_path(&self.tenant_id, &self.timeline_id)]) @@ -3801,8 +3445,8 @@ impl Timeline { new_layers, deltas_to_compact: deltas_to_compact .into_iter() - .map(|x| Arc::new(x.layer_desc().clone())) - .collect(), + .map(|x| x.drop_eviction_guard()) + .collect::>(), }) } @@ -3863,74 +3507,45 @@ impl Timeline { } let mut guard = self.layers.write().await; - let mut new_layer_paths = HashMap::with_capacity(new_layers.len()); - // In some rare cases, we may generate a file with exactly the same key range / LSN as before the compaction. - // We should move to numbering the layer files instead of naming them using key range / LSN some day. But for - // now, we just skip the file to avoid unintentional modification to files on the disk and in the layer map. let mut duplicated_layers = HashSet::new(); - let mut uploaded_layers = Vec::with_capacity(new_layers.len()); - let mut insert_layers = Vec::new(); - let mut remove_layers = Vec::new(); + let mut insert_layers = Vec::with_capacity(new_layers.len()); for l in &new_layers { - let new_delta_path = l.path(); - - let metadata = new_delta_path.metadata().with_context(|| { - format!("read file metadata for new created layer {new_delta_path}") - })?; - - uploaded_layers.push(( - l.filename(), - LayerFileMetadata::new(metadata.len(), self.generation), - )); - - // update metrics, including the timeline's physical size - self.metrics.record_new_file_metrics(metadata.len()); - - new_layer_paths.insert( - new_delta_path, - LayerFileMetadata::new(metadata.len(), self.generation), - ); - l.access_stats().record_residence_event( - LayerResidenceStatus::Resident, - LayerResidenceEventReason::LayerCreate, - ); - let l = l.to_owned() as Arc; - if guard.contains(&l) { + if guard.contains(l.as_ref()) { + // expected in tests tracing::error!(layer=%l, "duplicated L1 layer"); + + // good ways to cause a duplicate: we repeatedly error after taking the writelock + // `guard` on self.layers. as of writing this, there are no error returns except + // for compact_level0_phase1 creating an L0, which does not happen in practice + // because we have not implemented L0 => L0 compaction. duplicated_layers.insert(l.layer_desc().key()); + } else if LayerMap::is_l0(l.layer_desc()) { + return Err(CompactionError::Other(anyhow!("compaction generates a L0 layer file as output, which will cause infinite compaction."))); } else { - if LayerMap::is_l0(l.layer_desc()) { - return Err(CompactionError::Other(anyhow!("compaction generates a L0 layer file as output, which will cause infinite compaction."))); - } - insert_layers.push(l); + insert_layers.push(l.clone()); } } - // Now that we have reshuffled the data to set of new delta layers, we can - // delete the old ones - let mut layer_names_to_delete = Vec::with_capacity(deltas_to_compact.len()); - for ldesc in deltas_to_compact { - if duplicated_layers.contains(&ldesc.key()) { - // skip duplicated layers, they will not be removed; we have already overwritten them - // with new layers in the compaction phase 1. - continue; - } - layer_names_to_delete.push(ldesc.filename()); - remove_layers.push(guard.get_from_desc(&ldesc)); - } + let remove_layers = { + let mut deltas_to_compact = deltas_to_compact; + // only remove those inputs which were not outputs + deltas_to_compact.retain(|l| !duplicated_layers.contains(&l.layer_desc().key())); + deltas_to_compact + }; + // deletion will happen later, the layer file manager calls garbage_collect_on_drop guard.finish_compact_l0( - layer_removal_cs, - remove_layers, - insert_layers, + &layer_removal_cs, + &remove_layers, + &insert_layers, &self.metrics, - )?; + ); if let Some(remote_client) = self.remote_client.as_ref() { - remote_client.schedule_compaction_update(&layer_names_to_delete, &uploaded_layers)?; + remote_client.schedule_compaction_update(&remove_layers, &new_layers)?; } drop_wlock(guard); @@ -4235,7 +3850,7 @@ impl Timeline { l.filename(), l.is_incremental(), ); - layers_to_remove.push(Arc::clone(&l)); + layers_to_remove.push(l); } self.wanted_image_layers .lock() @@ -4245,32 +3860,28 @@ impl Timeline { if !layers_to_remove.is_empty() { // Persist the new GC cutoff value in the metadata file, before // we actually remove anything. - self.update_metadata_file(self.disk_consistent_lsn.load(), HashMap::new()) + // + // This does not in fact have any effect as we no longer consider local metadata unless + // running without remote storage. + self.update_metadata_file(self.disk_consistent_lsn.load(), None) .await?; - // Actually delete the layers from disk and remove them from the map. - // (couldn't do this in the loop above, because you cannot modify a collection - // while iterating it. BTreeMap::retain() would be another option) - let mut layer_names_to_delete = Vec::with_capacity(layers_to_remove.len()); let gc_layers = layers_to_remove .iter() .map(|x| guard.get_from_desc(x)) - .collect(); - for doomed_layer in layers_to_remove { - layer_names_to_delete.push(doomed_layer.filename()); - result.layers_removed += 1; + .collect::>(); + + result.layers_removed = gc_layers.len() as u64; + + if let Some(remote_client) = self.remote_client.as_ref() { + remote_client.schedule_gc_update(&gc_layers)?; } - let apply = guard.finish_gc_timeline(layer_removal_cs, gc_layers, &self.metrics)?; + + guard.finish_gc_timeline(&layer_removal_cs, gc_layers); if result.layers_removed != 0 { fail_point!("after-timeline-gc-removed-layers"); } - - if let Some(remote_client) = &self.remote_client { - remote_client.schedule_layer_file_deletion(layer_names_to_delete)?; - } - - apply.flush(); } info!( @@ -4367,155 +3978,15 @@ impl Timeline { } } - /// Download a layer file from remote storage and insert it into the layer map. - /// - /// It's safe to call this function for the same layer concurrently. In that case: - /// - If the layer has already been downloaded, `OK(...)` is returned. - /// - If the layer is currently being downloaded, we wait until that download succeeded / failed. - /// - If it succeeded, we return `Ok(...)`. - /// - If it failed, we or another concurrent caller will initiate a new download attempt. - /// - /// Download errors are classified and retried if appropriate by the underlying RemoteTimelineClient function. - /// It has an internal limit for the maximum number of retries and prints appropriate log messages. - /// If we exceed the limit, it returns an error, and this function passes it through. - /// The caller _could_ retry further by themselves by calling this function again, but _should not_ do it. - /// The reason is that they cannot distinguish permanent errors from temporary ones, whereas - /// the underlying RemoteTimelineClient can. - /// - /// There is no internal timeout or slowness detection. - /// If the caller has a deadline or needs a timeout, they can simply stop polling: - /// we're **cancellation-safe** because the download happens in a separate task_mgr task. - /// So, the current download attempt will run to completion even if we stop polling. - #[instrument(skip_all, fields(layer=%remote_layer))] - pub async fn download_remote_layer( - &self, - remote_layer: Arc, - ) -> anyhow::Result<()> { - span::debug_assert_current_span_has_tenant_and_timeline_id(); - - use std::sync::atomic::Ordering::Relaxed; - - let permit = match Arc::clone(&remote_layer.ongoing_download) - .acquire_owned() - .await - { - Ok(permit) => permit, - Err(_closed) => { - if remote_layer.download_replacement_failure.load(Relaxed) { - // this path will be hit often, in case there are upper retries. however - // hitting this error will prevent a busy loop between get_reconstruct_data and - // download, so an error is prefered. - // - // TODO: we really should poison the timeline, but panicking is not yet - // supported. Related: https://github.com/neondatabase/neon/issues/3621 - anyhow::bail!("an earlier download succeeded but LayerMap::replace failed") - } else { - info!("download of layer has already finished"); - return Ok(()); - } - } - }; - - let (sender, receiver) = tokio::sync::oneshot::channel(); - // Spawn a task so that download does not outlive timeline when we detach tenant / delete timeline. - let self_clone = self.myself.upgrade().expect("timeline is gone"); - task_mgr::spawn( - &tokio::runtime::Handle::current(), - TaskKind::RemoteDownloadTask, - Some(self.tenant_id), - Some(self.timeline_id), - &format!("download layer {}", remote_layer), - false, - async move { - let remote_client = self_clone.remote_client.as_ref().unwrap(); - - // Does retries + exponential back-off internally. - // When this fails, don't layer further retry attempts here. - let result = remote_client - .download_layer_file(&remote_layer.filename(), &remote_layer.layer_metadata) - .await; - - if let Ok(size) = &result { - info!("layer file download finished"); - - // XXX the temp file is still around in Err() case - // and consumes space until we clean up upon pageserver restart. - self_clone.metrics.resident_physical_size_add(*size); - - // Download complete. Replace the RemoteLayer with the corresponding - // Delta- or ImageLayer in the layer map. - let mut guard = self_clone.layers.write().await; - let new_layer = - remote_layer.create_downloaded_layer(&guard, self_clone.conf, *size); - { - let l: Arc = remote_layer.clone(); - let failure = match guard.replace_and_verify(l, new_layer) { - Ok(()) => false, - Err(e) => { - // this is a precondition failure, the layer filename derived - // attributes didn't match up, which doesn't seem likely. - error!("replacing downloaded layer into layermap failed: {e:#?}"); - true - } - }; - - if failure { - // mark the remote layer permanently failed; the timeline is most - // likely unusable after this. sadly we cannot just poison the layermap - // lock with panic, because that would create an issue with shutdown. - // - // this does not change the retry semantics on failed downloads. - // - // use of Relaxed is valid because closing of the semaphore gives - // happens-before and wakes up any waiters; we write this value before - // and any waiters (or would be waiters) will load it after closing - // semaphore. - // - // See: https://github.com/neondatabase/neon/issues/3533 - remote_layer - .download_replacement_failure - .store(true, Relaxed); - } - } - drop_wlock(guard); - - info!("on-demand download successful"); - - // Now that we've inserted the download into the layer map, - // close the semaphore. This will make other waiters for - // this download return Ok(()). - assert!(!remote_layer.ongoing_download.is_closed()); - remote_layer.ongoing_download.close(); - } else { - // Keep semaphore open. We'll drop the permit at the end of the function. - error!( - "layer file download failed: {:?}", - result.as_ref().unwrap_err() - ); - } - - // Don't treat it as an error if the task that triggered the download - // is no longer interested in the result. - sender.send(result.map(|_sz| ())).ok(); - - // In case we failed and there are other waiters, this will make one - // of them retry the download in a new task. - // XXX: This resets the exponential backoff because it's a new call to - // download_layer file. - drop(permit); - - Ok(()) - } - .in_current_span(), - ); - - receiver.await.context("download task cancelled")? - } - - pub async fn spawn_download_all_remote_layers( + pub(crate) async fn spawn_download_all_remote_layers( self: Arc, request: DownloadRemoteLayersTaskSpawnRequest, ) -> Result { + use pageserver_api::models::DownloadRemoteLayersTaskState; + + // this is not really needed anymore; it has tests which really check the return value from + // http api. it would be better not to maintain this anymore. + let mut status_guard = self.download_all_remote_layers_task_info.write().unwrap(); if let Some(st) = &*status_guard { match &st.state { @@ -4574,21 +4045,17 @@ impl Timeline { self: &Arc, request: DownloadRemoteLayersTaskSpawnRequest, ) { - let mut downloads = Vec::new(); - { + use pageserver_api::models::DownloadRemoteLayersTaskState; + + let remaining = { let guard = self.layers.read().await; - let layers = guard.layer_map(); - layers + guard + .layer_map() .iter_historic_layers() - .map(|l| guard.get_from_desc(&l)) - .filter_map(|l| l.downcast_remote_layer()) - .map(|l| self.download_remote_layer(l)) - .for_each(|dl| downloads.push(dl)) - } - let total_layer_count = downloads.len(); - // limit download concurrency as specified in request - let downloads = futures::stream::iter(downloads); - let mut downloads = downloads.buffer_unordered(request.max_concurrent_downloads.get()); + .map(|desc| guard.get_from_desc(&desc)) + .collect::>() + }; + let total_layer_count = remaining.len(); macro_rules! lock_status { ($st:ident) => { @@ -4611,29 +4078,58 @@ impl Timeline { lock_status!(st); st.total_layer_count = total_layer_count as u64; } + + let mut remaining = remaining.into_iter(); + let mut have_remaining = true; + let mut js = tokio::task::JoinSet::new(); + + let cancel = task_mgr::shutdown_token(); + + let limit = request.max_concurrent_downloads; + loop { - tokio::select! { - dl = downloads.next() => { - lock_status!(st); - match dl { - None => break, - Some(Ok(())) => { - st.successful_download_count += 1; - }, - Some(Err(e)) => { - error!(error = %e, "layer download failed"); - st.failed_download_count += 1; - } + while js.len() < limit.get() && have_remaining && !cancel.is_cancelled() { + let Some(next) = remaining.next() else { + have_remaining = false; + break; + }; + + let span = tracing::info_span!("download", layer = %next); + + js.spawn( + async move { + let res = next.download().await; + (next, res) } - } - _ = task_mgr::shutdown_watcher() => { - // Kind of pointless to watch for shutdowns here, - // as download_remote_layer spawns other task_mgr tasks internally. - lock_status!(st); - st.state = DownloadRemoteLayersTaskState::ShutDown; + .instrument(span), + ); + } + + while let Some(res) = js.join_next().await { + match res { + Ok((_, Ok(_))) => { + lock_status!(st); + st.successful_download_count += 1; + } + Ok((layer, Err(e))) => { + tracing::error!(%layer, "download failed: {e:#}"); + lock_status!(st); + st.failed_download_count += 1; + } + Err(je) if je.is_cancelled() => unreachable!("not used here"), + Err(je) if je.is_panic() => { + lock_status!(st); + st.failed_download_count += 1; + } + Err(je) => tracing::warn!("unknown joinerror: {je:?}"), } } + + if js.is_empty() && (!have_remaining || cancel.is_cancelled()) { + break; + } } + { lock_status!(st); st.state = DownloadRemoteLayersTaskState::Completed; @@ -4648,15 +4144,15 @@ impl Timeline { } } -pub struct DiskUsageEvictionInfo { +pub(crate) struct DiskUsageEvictionInfo { /// Timeline's largest layer (remote or resident) pub max_layer_size: Option, /// Timeline's resident layers pub resident_layers: Vec, } -pub struct LocalLayerInfoForDiskUsageEviction { - pub layer: Arc, +pub(crate) struct LocalLayerInfoForDiskUsageEviction { + pub layer: Layer, pub last_activity_ts: SystemTime, } @@ -4666,8 +4162,14 @@ impl std::fmt::Debug for LocalLayerInfoForDiskUsageEviction { // having to allocate a string to this is bad, but it will rarely be formatted let ts = chrono::DateTime::::from(self.last_activity_ts); let ts = ts.to_rfc3339_opts(chrono::SecondsFormat::Nanos, true); + struct DisplayIsDebug<'a, T>(&'a T); + impl<'a, T: std::fmt::Display> std::fmt::Debug for DisplayIsDebug<'a, T> { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.0) + } + } f.debug_struct("LocalLayerInfoForDiskUsageEviction") - .field("layer", &self.layer) + .field("layer", &DisplayIsDebug(&self.layer)) .field("last_activity", &ts) .finish() } @@ -4694,9 +4196,16 @@ impl Timeline { let l = guard.get_from_desc(&l); - if l.is_remote_layer() { - continue; - } + let l = match l.keep_resident().await { + Ok(Some(l)) => l, + Ok(None) => continue, + Err(e) => { + // these should not happen, but we cannot make them statically impossible right + // now. + tracing::warn!(layer=%l, "failed to keep the layer resident: {e:#}"); + continue; + } + }; let last_activity_ts = l.access_stats().latest_activity().unwrap_or_else(|| { // We only use this fallback if there's an implementation error. @@ -4706,7 +4215,7 @@ impl Timeline { }); resident_layers.push(LocalLayerInfoForDiskUsageEviction { - layer: l, + layer: l.drop_eviction_guard(), last_activity_ts, }); } @@ -4829,43 +4338,13 @@ fn rename_to_backup(path: &Utf8Path) -> anyhow::Result<()> { bail!("couldn't find an unused backup number for {:?}", path) } -/// Similar to `Arc::ptr_eq`, but only compares the object pointers, not vtables. -/// -/// Returns `true` if the two `Arc` point to the same layer, false otherwise. -/// -/// If comparing persistent layers, ALWAYS compare the layer descriptor key. -#[inline(always)] -pub fn compare_arced_layers(left: &Arc, right: &Arc) -> bool { - // "dyn Trait" objects are "fat pointers" in that they have two components: - // - pointer to the object - // - pointer to the vtable - // - // rust does not provide a guarantee that these vtables are unique, but however - // `Arc::ptr_eq` as of writing (at least up to 1.67) uses a comparison where both the - // pointer and the vtable need to be equal. - // - // See: https://github.com/rust-lang/rust/issues/103763 - // - // A future version of rust will most likely use this form below, where we cast each - // pointer into a pointer to unit, which drops the inaccessible vtable pointer, making it - // not affect the comparison. - // - // See: https://github.com/rust-lang/rust/pull/106450 - let left = Arc::as_ptr(left) as *const (); - let right = Arc::as_ptr(right) as *const (); - - left == right -} - #[cfg(test)] mod tests { - use std::sync::Arc; - use utils::{id::TimelineId, lsn::Lsn}; - use crate::tenant::{harness::TenantHarness, storage_layer::PersistentLayer}; - - use super::{EvictionError, Timeline}; + use crate::tenant::{ + harness::TenantHarness, storage_layer::Layer, timeline::EvictionError, Timeline, + }; #[tokio::test] async fn two_layer_eviction_attempts_at_the_same_time() { @@ -4885,22 +4364,29 @@ mod tests { .expect("just configured this"); let layer = find_some_layer(&timeline).await; + let layer = layer + .keep_resident() + .await + .expect("no download => no downloading errors") + .expect("should had been resident") + .drop_eviction_guard(); let cancel = tokio_util::sync::CancellationToken::new(); let batch = [layer]; let first = { - let cancel = cancel.clone(); + let cancel = cancel.child_token(); async { + let cancel = cancel; timeline - .evict_layer_batch(&rc, &batch, cancel) + .evict_layer_batch(&rc, &batch, &cancel) .await .unwrap() } }; let second = async { timeline - .evict_layer_batch(&rc, &batch, cancel) + .evict_layer_batch(&rc, &batch, &cancel) .await .unwrap() }; @@ -4909,76 +4395,21 @@ mod tests { let (first, second) = (only_one(first), only_one(second)); + let res = batch[0].keep_resident().await; + assert!(matches!(res, Ok(None)), "{res:?}"); + match (first, second) { - (Ok(()), Err(EvictionError::FileNotFound)) - | (Err(EvictionError::FileNotFound), Ok(())) => { - // one of the evictions gets to do it, - // other one gets FileNotFound. all is good. + (Ok(()), Ok(())) => { + // because there are no more timeline locks being taken on eviction path, we can + // witness all three outcomes here. + } + (Ok(()), Err(EvictionError::NotFound)) | (Err(EvictionError::NotFound), Ok(())) => { + // if one completes before the other, this is fine just as well. } other => unreachable!("unexpected {:?}", other), } } - #[tokio::test] - async fn layer_eviction_aba_fails() { - let harness = TenantHarness::create("layer_eviction_aba_fails").unwrap(); - - let ctx = any_context(); - let tenant = harness.try_load(&ctx).await.unwrap(); - let timeline = tenant - .create_test_timeline(TimelineId::generate(), Lsn(0x10), 14, &ctx) - .await - .unwrap(); - - let _e = tracing::info_span!("foobar", tenant_id = %tenant.tenant_id, timeline_id = %timeline.timeline_id).entered(); - - let rc = timeline.remote_client.clone().unwrap(); - - // TenantHarness allows uploads to happen given GenericRemoteStorage is configured - let layer = find_some_layer(&timeline).await; - - let cancel = tokio_util::sync::CancellationToken::new(); - let batch = [layer]; - - let first = { - let cancel = cancel.clone(); - async { - timeline - .evict_layer_batch(&rc, &batch, cancel) - .await - .unwrap() - } - }; - - // lets imagine this is stuck somehow, still referencing the original `Arc` - let second = { - let cancel = cancel.clone(); - async { - timeline - .evict_layer_batch(&rc, &batch, cancel) - .await - .unwrap() - } - }; - - // while it's stuck, we evict and end up redownloading it - only_one(first.await).expect("eviction succeeded"); - - let layer = find_some_layer(&timeline).await; - let layer = layer.downcast_remote_layer().unwrap(); - timeline.download_remote_layer(layer).await.unwrap(); - - let res = only_one(second.await); - - assert!( - matches!(res, Err(EvictionError::LayerNotFound(_))), - "{res:?}" - ); - - // no more specific asserting, outside of preconds this is the only valid replacement - // failure - } - fn any_context() -> crate::context::RequestContext { use crate::context::*; use crate::task_mgr::*; @@ -4993,7 +4424,7 @@ mod tests { .expect("no cancellation") } - async fn find_some_layer(timeline: &Timeline) -> Arc { + async fn find_some_layer(timeline: &Timeline) -> Layer { let layers = timeline.layers.read().await; let desc = layers .layer_map() diff --git a/pageserver/src/tenant/timeline/eviction_task.rs b/pageserver/src/tenant/timeline/eviction_task.rs index 38da993deb..dc5c71bbe1 100644 --- a/pageserver/src/tenant/timeline/eviction_task.rs +++ b/pageserver/src/tenant/timeline/eviction_task.rs @@ -29,7 +29,6 @@ use crate::{ task_mgr::{self, TaskKind, BACKGROUND_RUNTIME}, tenant::{ config::{EvictionPolicy, EvictionPolicyLayerAccessThreshold}, - storage_layer::PersistentLayer, tasks::{BackgroundLoopKind, RateLimitError}, timeline::EvictionError, LogicalSizeCalculationCause, Tenant, @@ -210,15 +209,26 @@ impl Timeline { // NB: all the checks can be invalidated as soon as we release the layer map lock. // We don't want to hold the layer map lock during eviction. // So, we just need to deal with this. - let candidates: Vec> = { + let candidates: Vec<_> = { let guard = self.layers.read().await; let layers = guard.layer_map(); let mut candidates = Vec::new(); for hist_layer in layers.iter_historic_layers() { let hist_layer = guard.get_from_desc(&hist_layer); - if hist_layer.is_remote_layer() { - continue; - } + + // guard against eviction while we inspect it; it might be that eviction_task and + // disk_usage_eviction_task both select the same layers to be evicted, and + // seemingly free up double the space. both succeeding is of no consequence. + let guard = match hist_layer.keep_resident().await { + Ok(Some(l)) => l, + Ok(None) => continue, + Err(e) => { + // these should not happen, but we cannot make them statically impossible right + // now. + tracing::warn!(layer=%hist_layer, "failed to keep the layer resident: {e:#}"); + continue; + } + }; let last_activity_ts = hist_layer.access_stats().latest_activity().unwrap_or_else(|| { // We only use this fallback if there's an implementation error. @@ -249,7 +259,7 @@ impl Timeline { } }; if no_activity_for > p.threshold { - candidates.push(hist_layer) + candidates.push(guard.drop_eviction_guard()) } } candidates @@ -268,7 +278,7 @@ impl Timeline { }; let results = match self - .evict_layer_batch(remote_client, &candidates[..], cancel.clone()) + .evict_layer_batch(remote_client, &candidates, cancel) .await { Err(pre_err) => { @@ -279,7 +289,7 @@ impl Timeline { Ok(results) => results, }; assert_eq!(results.len(), candidates.len()); - for (l, result) in candidates.iter().zip(results) { + for result in results { match result { None => { stats.skipped_for_shutdown += 1; @@ -287,24 +297,10 @@ impl Timeline { Some(Ok(())) => { stats.evicted += 1; } - Some(Err(EvictionError::CannotEvictRemoteLayer)) => { - stats.not_evictable += 1; - } - Some(Err(EvictionError::FileNotFound)) => { + Some(Err(EvictionError::NotFound | EvictionError::Downloaded)) => { // compaction/gc removed the file while we were waiting on layer_removal_cs stats.not_evictable += 1; } - Some(Err( - e @ EvictionError::LayerNotFound(_) | e @ EvictionError::StatFailed(_), - )) => { - let e = utils::error::report_compact_sources(&e); - warn!(layer = %l, "failed to evict layer: {e}"); - stats.not_evictable += 1; - } - Some(Err(EvictionError::MetadataInconsistency(detail))) => { - warn!(layer = %l, "failed to evict layer: {detail}"); - stats.not_evictable += 1; - } } } if stats.candidates == stats.not_evictable { diff --git a/pageserver/src/tenant/timeline/layer_manager.rs b/pageserver/src/tenant/timeline/layer_manager.rs index 0a387bd779..e4991e0865 100644 --- a/pageserver/src/tenant/timeline/layer_manager.rs +++ b/pageserver/src/tenant/timeline/layer_manager.rs @@ -12,27 +12,16 @@ use crate::{ tenant::{ layer_map::{BatchedUpdates, LayerMap}, storage_layer::{ - AsLayerDesc, DeltaLayer, ImageLayer, InMemoryLayer, PersistentLayer, - PersistentLayerDesc, PersistentLayerKey, + AsLayerDesc, InMemoryLayer, Layer, PersistentLayerDesc, PersistentLayerKey, + ResidentLayer, }, - timeline::compare_arced_layers, }, }; /// Provides semantic APIs to manipulate the layer map. pub(crate) struct LayerManager { layer_map: LayerMap, - layer_fmgr: LayerFileManager, -} - -/// After GC, the layer map changes will not be applied immediately. Users should manually apply the changes after -/// scheduling deletes in remote client. -pub(crate) struct ApplyGcResultGuard<'a>(BatchedUpdates<'a>); - -impl ApplyGcResultGuard<'_> { - pub(crate) fn flush(self) { - self.0.flush(); - } + layer_fmgr: LayerFileManager, } impl LayerManager { @@ -43,7 +32,7 @@ impl LayerManager { } } - pub(crate) fn get_from_desc(&self, desc: &PersistentLayerDesc) -> Arc { + pub(crate) fn get_from_desc(&self, desc: &PersistentLayerDesc) -> Layer { self.layer_fmgr.get_from_desc(desc) } @@ -55,21 +44,12 @@ impl LayerManager { &self.layer_map } - /// Replace layers in the layer file manager, used in evictions and layer downloads. - pub(crate) fn replace_and_verify( - &mut self, - expected: Arc, - new: Arc, - ) -> Result<()> { - self.layer_fmgr.replace_and_verify(expected, new) - } - /// Called from `load_layer_map`. Initialize the layer manager with: /// 1. all on-disk layers /// 2. next open layer (with disk disk_consistent_lsn LSN) pub(crate) fn initialize_local_layers( &mut self, - on_disk_layers: Vec>, + on_disk_layers: Vec, next_open_layer_at: Lsn, ) { let mut updates = self.layer_map.batch_update(); @@ -164,10 +144,19 @@ impl LayerManager { } /// Add image layers to the layer map, called from `create_image_layers`. - pub(crate) fn track_new_image_layers(&mut self, image_layers: Vec) { + pub(crate) fn track_new_image_layers( + &mut self, + image_layers: &[ResidentLayer], + metrics: &TimelineMetrics, + ) { let mut updates = self.layer_map.batch_update(); for layer in image_layers { - Self::insert_historic_layer(Arc::new(layer), &mut updates, &mut self.layer_fmgr); + Self::insert_historic_layer(layer.as_ref().clone(), &mut updates, &mut self.layer_fmgr); + + // record these here instead of Layer::finish_creating because otherwise partial + // failure with create_image_layers would balloon up the physical size gauge. downside + // is that all layers need to be created before metrics are updated. + metrics.record_new_file_metrics(layer.layer_desc().file_size); } updates.flush(); } @@ -175,76 +164,71 @@ impl LayerManager { /// Flush a frozen layer and add the written delta layer to the layer map. pub(crate) fn finish_flush_l0_layer( &mut self, - delta_layer: Option, + delta_layer: Option<&ResidentLayer>, frozen_layer_for_check: &Arc, + metrics: &TimelineMetrics, ) { - let l = self.layer_map.frozen_layers.pop_front(); - let mut updates = self.layer_map.batch_update(); + let inmem = self + .layer_map + .frozen_layers + .pop_front() + .expect("there must be a inmem layer to flush"); - // Only one thread may call this function at a time (for this - // timeline). If two threads tried to flush the same frozen + // Only one task may call this function at a time (for this + // timeline). If two tasks tried to flush the same frozen // layer to disk at the same time, that would not work. - assert!(compare_arced_layers(&l.unwrap(), frozen_layer_for_check)); + assert_eq!(Arc::as_ptr(&inmem), Arc::as_ptr(frozen_layer_for_check)); - if let Some(delta_layer) = delta_layer { - Self::insert_historic_layer(Arc::new(delta_layer), &mut updates, &mut self.layer_fmgr); + if let Some(l) = delta_layer { + let mut updates = self.layer_map.batch_update(); + Self::insert_historic_layer(l.as_ref().clone(), &mut updates, &mut self.layer_fmgr); + metrics.record_new_file_metrics(l.layer_desc().file_size); + updates.flush(); } - updates.flush(); } /// Called when compaction is completed. pub(crate) fn finish_compact_l0( &mut self, - layer_removal_cs: Arc>, - compact_from: Vec>, - compact_to: Vec>, + layer_removal_cs: &Arc>, + compact_from: &[Layer], + compact_to: &[ResidentLayer], metrics: &TimelineMetrics, - ) -> Result<()> { + ) { let mut updates = self.layer_map.batch_update(); for l in compact_to { - Self::insert_historic_layer(l, &mut updates, &mut self.layer_fmgr); + Self::insert_historic_layer(l.as_ref().clone(), &mut updates, &mut self.layer_fmgr); + metrics.record_new_file_metrics(l.layer_desc().file_size); } for l in compact_from { - // NB: the layer file identified by descriptor `l` is guaranteed to be present - // in the LayerFileManager because compaction kept holding `layer_removal_cs` the entire - // time, even though we dropped `Timeline::layers` inbetween. - Self::delete_historic_layer( - layer_removal_cs.clone(), - l, - &mut updates, - metrics, - &mut self.layer_fmgr, - )?; + Self::delete_historic_layer(layer_removal_cs, l, &mut updates, &mut self.layer_fmgr); } updates.flush(); - Ok(()) } /// Called when garbage collect the timeline. Returns a guard that will apply the updates to the layer map. pub(crate) fn finish_gc_timeline( &mut self, - layer_removal_cs: Arc>, - gc_layers: Vec>, - metrics: &TimelineMetrics, - ) -> Result { + layer_removal_cs: &Arc>, + gc_layers: Vec, + ) { let mut updates = self.layer_map.batch_update(); for doomed_layer in gc_layers { Self::delete_historic_layer( - layer_removal_cs.clone(), - doomed_layer, + layer_removal_cs, + &doomed_layer, &mut updates, - metrics, &mut self.layer_fmgr, - )?; // FIXME: schedule succeeded deletions in timeline.rs `gc_timeline` instead of in batch? + ); } - Ok(ApplyGcResultGuard(updates)) + updates.flush() } /// Helper function to insert a layer into the layer map and file manager. fn insert_historic_layer( - layer: Arc, + layer: Layer, updates: &mut BatchedUpdates<'_>, - mapping: &mut LayerFileManager, + mapping: &mut LayerFileManager, ) { updates.insert_historic(layer.layer_desc().clone()); mapping.insert(layer); @@ -254,17 +238,12 @@ impl LayerManager { /// Remote storage is not affected by this operation. fn delete_historic_layer( // we cannot remove layers otherwise, since gc and compaction will race - _layer_removal_cs: Arc>, - layer: Arc, + _layer_removal_cs: &Arc>, + layer: &Layer, updates: &mut BatchedUpdates<'_>, - metrics: &TimelineMetrics, - mapping: &mut LayerFileManager, - ) -> anyhow::Result<()> { + mapping: &mut LayerFileManager, + ) { let desc = layer.layer_desc(); - if !layer.is_remote_layer() { - layer.delete_resident_layer_file()?; - metrics.resident_physical_size_sub(desc.file_size); - } // TODO Removing from the bottom of the layer map is expensive. // Maybe instead discard all layer map historic versions that @@ -273,21 +252,18 @@ impl LayerManager { // map index without actually rebuilding the index. updates.remove_historic(desc); mapping.remove(layer); - - Ok(()) + layer.garbage_collect_on_drop(); } - pub(crate) fn contains(&self, layer: &Arc) -> bool { + pub(crate) fn contains(&self, layer: &Layer) -> bool { self.layer_fmgr.contains(layer) } } -pub(crate) struct LayerFileManager( - HashMap>, -); +pub(crate) struct LayerFileManager(HashMap); -impl LayerFileManager { - fn get_from_desc(&self, desc: &PersistentLayerDesc) -> Arc { +impl LayerFileManager { + fn get_from_desc(&self, desc: &PersistentLayerDesc) -> T { // The assumption for the `expect()` is that all code maintains the following invariant: // A layer's descriptor is present in the LayerMap => the LayerFileManager contains a layer for the descriptor. self.0 @@ -297,14 +273,14 @@ impl LayerFileManager { .clone() } - pub(crate) fn insert(&mut self, layer: Arc) { + pub(crate) fn insert(&mut self, layer: T) { let present = self.0.insert(layer.layer_desc().key(), layer.clone()); if present.is_some() && cfg!(debug_assertions) { panic!("overwriting a layer: {:?}", layer.layer_desc()) } } - pub(crate) fn contains(&self, layer: &Arc) -> bool { + pub(crate) fn contains(&self, layer: &T) -> bool { self.0.contains_key(&layer.layer_desc().key()) } @@ -312,7 +288,7 @@ impl LayerFileManager { Self(HashMap::new()) } - pub(crate) fn remove(&mut self, layer: Arc) { + pub(crate) fn remove(&mut self, layer: &T) { let present = self.0.remove(&layer.layer_desc().key()); if present.is_none() && cfg!(debug_assertions) { panic!( @@ -321,39 +297,4 @@ impl LayerFileManager { ) } } - - pub(crate) fn replace_and_verify(&mut self, expected: Arc, new: Arc) -> Result<()> { - let key = expected.layer_desc().key(); - let other = new.layer_desc().key(); - - let expected_l0 = LayerMap::is_l0(expected.layer_desc()); - let new_l0 = LayerMap::is_l0(new.layer_desc()); - - fail::fail_point!("layermap-replace-notfound", |_| anyhow::bail!( - "layermap-replace-notfound" - )); - - anyhow::ensure!( - key == other, - "expected and new layer have different keys: {key:?} != {other:?}" - ); - - anyhow::ensure!( - expected_l0 == new_l0, - "one layer is l0 while the other is not: {expected_l0} != {new_l0}" - ); - - if let Some(layer) = self.0.get_mut(&key) { - anyhow::ensure!( - compare_arced_layers(&expected, layer), - "another layer was found instead of expected, expected={expected:?}, new={new:?}", - expected = Arc::as_ptr(&expected), - new = Arc::as_ptr(layer), - ); - *layer = new; - Ok(()) - } else { - anyhow::bail!("layer was not found"); - } - } } diff --git a/pageserver/src/tenant/upload_queue.rs b/pageserver/src/tenant/upload_queue.rs index 2db67d071a..5959819548 100644 --- a/pageserver/src/tenant/upload_queue.rs +++ b/pageserver/src/tenant/upload_queue.rs @@ -1,4 +1,5 @@ use super::storage_layer::LayerFileName; +use super::storage_layer::ResidentLayer; use super::Generation; use crate::tenant::metadata::TimelineMetadata; use crate::tenant::remote_timeline_client::index::IndexPart; @@ -203,18 +204,6 @@ impl UploadQueue { UploadQueue::Stopped(stopped) => Ok(stopped), } } - - pub(crate) fn get_layer_metadata( - &self, - name: &LayerFileName, - ) -> anyhow::Result> { - match self { - UploadQueue::Stopped(_) | UploadQueue::Uninitialized => { - anyhow::bail!("queue is in state {}", self.as_str()) - } - UploadQueue::Initialized(inner) => Ok(inner.latest_files.get(name).cloned()), - } - } } /// An in-progress upload or delete task. @@ -237,7 +226,7 @@ pub(crate) struct Delete { #[derive(Debug)] pub(crate) enum UploadOp { /// Upload a layer file - UploadLayer(LayerFileName, LayerFileMetadata), + UploadLayer(ResidentLayer, LayerFileMetadata), /// Upload the metadata file UploadMetadata(IndexPart, Lsn), @@ -252,13 +241,13 @@ pub(crate) enum UploadOp { impl std::fmt::Display for UploadOp { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { match self { - UploadOp::UploadLayer(path, metadata) => { + UploadOp::UploadLayer(layer, metadata) => { write!( f, "UploadLayer({}, size={:?}, gen={:?})", - path.file_name(), + layer, metadata.file_size(), - metadata.generation, + metadata.generation ) } UploadOp::UploadMetadata(_, lsn) => { diff --git a/test_runner/fixtures/neon_fixtures.py b/test_runner/fixtures/neon_fixtures.py index 1a7db18b07..0c0c06aa86 100644 --- a/test_runner/fixtures/neon_fixtures.py +++ b/test_runner/fixtures/neon_fixtures.py @@ -1616,7 +1616,7 @@ class NeonPageserver(PgProtocol): ".*wait for layer upload ops to complete.*", # .*Caused by:.*wait_completion aborted because upload queue was stopped ".*gc_loop.*Gc failed, retrying in.*timeline is Stopping", # When gc checks timeline state after acquiring layer_removal_cs ".*gc_loop.*Gc failed, retrying in.*: Cannot run GC iteration on inactive tenant", # Tenant::gc precondition - ".*compaction_loop.*Compaction failed, retrying in.*timeline is Stopping", # When compaction checks timeline state after acquiring layer_removal_cs + ".*compaction_loop.*Compaction failed, retrying in.*timeline or pageserver is shutting down", # When compaction checks timeline state after acquiring layer_removal_cs ".*query handler for 'pagestream.*failed: Timeline .* was not found", # postgres reconnects while timeline_delete doesn't hold the tenant's timelines.lock() ".*query handler for 'pagestream.*failed: Timeline .* is not active", # timeline delete in progress ".*task iteration took longer than the configured period.*", diff --git a/test_runner/regress/test_attach_tenant_config.py b/test_runner/regress/test_attach_tenant_config.py index 892e40b19b..10cffb1d6c 100644 --- a/test_runner/regress/test_attach_tenant_config.py +++ b/test_runner/regress/test_attach_tenant_config.py @@ -19,7 +19,7 @@ def positive_env(neon_env_builder: NeonEnvBuilder) -> NeonEnv: # eviction might be the first one after an attach to access the layers env.pageserver.allowed_errors.append( - ".*unexpectedly on-demand downloading remote layer remote.* for task kind Eviction" + ".*unexpectedly on-demand downloading remote layer .* for task kind Eviction" ) assert isinstance(env.pageserver_remote_storage, LocalFsStorage) return env diff --git a/test_runner/regress/test_broken_timeline.py b/test_runner/regress/test_broken_timeline.py index c80f2d8360..48accdb43d 100644 --- a/test_runner/regress/test_broken_timeline.py +++ b/test_runner/regress/test_broken_timeline.py @@ -20,7 +20,7 @@ def test_broken_timeline(neon_env_builder: NeonEnvBuilder): env.pageserver.allowed_errors.extend( [ - ".*Failed to load delta layer.*", + ".*layer loading failed:.*", ".*could not find data for key.*", ".*is not active. Current state: Broken.*", ".*will not become active. Current state: Broken.*", @@ -87,7 +87,7 @@ def test_broken_timeline(neon_env_builder: NeonEnvBuilder): # Second timeline will fail during basebackup, because the local layer file is corrupt. # It will fail when we try to read (and reconstruct) a page from it, ergo the error message. # (We don't check layer file contents on startup, when loading the timeline) - with pytest.raises(Exception, match="Failed to load delta layer") as err: + with pytest.raises(Exception, match="layer loading failed:") as err: pg2.start() log.info( f"As expected, compute startup failed for timeline {tenant2}/{timeline2} with corrupt layers: {err}" diff --git a/test_runner/regress/test_layer_eviction.py b/test_runner/regress/test_layer_eviction.py index 3ab3700866..6a6273760c 100644 --- a/test_runner/regress/test_layer_eviction.py +++ b/test_runner/regress/test_layer_eviction.py @@ -247,34 +247,34 @@ def test_gc_of_remote_layers(neon_env_builder: NeonEnvBuilder): ps_http.evict_all_layers(tenant_id, timeline_id) def ensure_resident_and_remote_size_metrics(): - log.info("ensure that all the layers are gone") resident_layers = list(env.pageserver.timeline_dir(tenant_id, timeline_id).glob("*-*_*")) # we have disabled all background loops, so, this should hold - assert len(resident_layers) == 0 + assert len(resident_layers) == 0, "ensure that all the layers are gone" info = ps_http.layer_map_info(tenant_id, timeline_id) log.info("layer map dump: %s", info) - log.info("ensure that resident_physical_size metric is zero") resident_physical_size_metric = ps_http.get_timeline_metric( tenant_id, timeline_id, "pageserver_resident_physical_size" ) - assert resident_physical_size_metric == 0 - log.info("ensure that resident_physical_size metric corresponds to layer map dump") + assert ( + resident_physical_size_metric == 0 + ), "ensure that resident_physical_size metric is zero" assert resident_physical_size_metric == sum( - [layer.layer_file_size or 0 for layer in info.historic_layers if not layer.remote] - ) + layer.layer_file_size or 0 for layer in info.historic_layers if not layer.remote + ), "ensure that resident_physical_size metric corresponds to layer map dump" - log.info("ensure that remote_physical_size metric matches layer map") remote_physical_size_metric = ps_http.get_timeline_metric( tenant_id, timeline_id, "pageserver_remote_physical_size" ) - log.info("ensure that remote_physical_size metric corresponds to layer map dump") assert remote_physical_size_metric == sum( layer.layer_file_size or 0 for layer in info.historic_layers if layer.remote - ) + ), "ensure that remote_physical_size metric corresponds to layer map dump" log.info("before runnning GC, ensure that remote_physical size is zero") + # leaving index_part.json upload from successful compaction out will show + # up here as a mismatch between remove_physical_size and summed up layermap + # size ensure_resident_and_remote_size_metrics() log.info("run GC") diff --git a/test_runner/regress/test_ondemand_download.py b/test_runner/regress/test_ondemand_download.py index 33de7445a4..a4cd42b6c3 100644 --- a/test_runner/regress/test_ondemand_download.py +++ b/test_runner/regress/test_ondemand_download.py @@ -12,13 +12,12 @@ from fixtures.neon_fixtures import ( last_flush_lsn_upload, wait_for_last_flush_lsn, ) -from fixtures.pageserver.http import PageserverApiException, PageserverHttpClient +from fixtures.pageserver.http import PageserverHttpClient from fixtures.pageserver.utils import ( assert_tenant_state, wait_for_last_record_lsn, wait_for_upload, wait_for_upload_queue_empty, - wait_until_tenant_state, ) from fixtures.remote_storage import RemoteStorageKind, available_remote_storages from fixtures.types import Lsn @@ -384,7 +383,7 @@ def test_download_remote_layers_api( env.pageserver.start(extra_env_vars={"FAILPOINTS": "remote-storage-download-pre-rename=return"}) env.pageserver.allowed_errors.extend( [ - f".*download_all_remote_layers.*{tenant_id}.*{timeline_id}.*layer download failed.*remote-storage-download-pre-rename failpoint", + ".*download failed: downloading evicted layer file failed.*", f".*initial size calculation.*{tenant_id}.*{timeline_id}.*Failed to calculate logical size", ] ) @@ -637,56 +636,5 @@ def test_compaction_downloads_on_demand_with_image_creation(neon_env_builder: Ne assert dict(kinds_after) == {"Delta": 4, "Image": 1} -def test_ondemand_download_failure_to_replace(neon_env_builder: NeonEnvBuilder): - """ - Make sure that we fail on being unable to replace a RemoteLayer instead of for example livelocking. - - See: https://github.com/neondatabase/neon/issues/3533 - """ - - neon_env_builder.enable_pageserver_remote_storage(RemoteStorageKind.LOCAL_FS) - - # disable gc and compaction via default tenant config because config is lost while detaching - # so that compaction will not be the one to download the layer but the http handler is - neon_env_builder.pageserver_config_override = ( - """tenant_config={gc_period = "0s", compaction_period = "0s"}""" - ) - - env = neon_env_builder.init_start() - - tenant_id = env.initial_tenant - timeline_id = env.initial_timeline - assert timeline_id is not None - - pageserver_http = env.pageserver.http_client() - - # remove layers so that they will be redownloaded - pageserver_http.tenant_detach(tenant_id) - pageserver_http.tenant_attach(tenant_id) - - wait_until_tenant_state(pageserver_http, tenant_id, "Active", 5) - pageserver_http.configure_failpoints(("layermap-replace-notfound", "return")) - - # requesting details with non-incremental size should trigger a download of the only layer - # this will need to be adjusted if an index for logical sizes is ever implemented - with pytest.raises(PageserverApiException): - # PageserverApiException is expected because of the failpoint (timeline_detail building does something) - # ReadTimeout can happen on our busy CI, but it should not, because there is no more busylooping - # but should it be added back, we would wait for 15s here. - pageserver_http.timeline_detail(tenant_id, timeline_id, True, timeout=15) - - actual_message = ".* ERROR .*layermap-replace-notfound" - assert env.pageserver.log_contains(actual_message) is not None - env.pageserver.allowed_errors.append(actual_message) - - env.pageserver.allowed_errors.append( - ".* ERROR .*Error processing HTTP request: InternalServerError\\(get local timeline info" - ) - # this might get to run and attempt on-demand, but not always - env.pageserver.allowed_errors.append(".* ERROR .*Task 'initial size calculation'") - - # if the above returned, then we didn't have a livelock, and all is well - - def stringify(conf: Dict[str, Any]) -> Dict[str, str]: return dict(map(lambda x: (x[0], str(x[1])), conf.items())) diff --git a/test_runner/regress/test_remote_storage.py b/test_runner/regress/test_remote_storage.py index f4bf9207b0..f201c735e1 100644 --- a/test_runner/regress/test_remote_storage.py +++ b/test_runner/regress/test_remote_storage.py @@ -586,7 +586,7 @@ def test_timeline_deletion_with_files_stuck_in_upload_queue( log.info("sending delete request") checkpoint_allowed_to_fail.set() env.pageserver.allowed_errors.append( - ".* ERROR .*Error processing HTTP request: InternalServerError\\(timeline is Stopping" + ".* ERROR .*Error processing HTTP request: InternalServerError\\(The timeline or pageserver is shutting down" ".* ERROR .*[Cc]ould not flush frozen layer.*" ) diff --git a/test_runner/regress/test_tenants_with_remote_storage.py b/test_runner/regress/test_tenants_with_remote_storage.py index 30a36d0eca..0169335a70 100644 --- a/test_runner/regress/test_tenants_with_remote_storage.py +++ b/test_runner/regress/test_tenants_with_remote_storage.py @@ -227,9 +227,7 @@ def test_tenant_redownloads_truncated_file_on_startup( assert isinstance(env.pageserver_remote_storage, LocalFsStorage) - env.pageserver.allowed_errors.append( - ".*removing local file .* because it has unexpected length.*" - ) + env.pageserver.allowed_errors.append(".*removing local file .* because .*") # FIXME: Are these expected? env.pageserver.allowed_errors.append(