diff --git a/pageserver/src/tenant/timeline.rs b/pageserver/src/tenant/timeline.rs index c5eda44b7d..d3c8c5f66c 100644 --- a/pageserver/src/tenant/timeline.rs +++ b/pageserver/src/tenant/timeline.rs @@ -282,10 +282,12 @@ pub struct Timeline { pub(super) flush_loop_state: Mutex, /// layer_flush_start_tx can be used to wake up the layer-flushing task. - /// The value is a counter, incremented every time a new flush cycle is requested. - /// The flush cycle counter is sent back on the layer_flush_done channel when - /// the flush finishes. You can use that to wait for the flush to finish. - layer_flush_start_tx: tokio::sync::watch::Sender, + /// - The u64 value is a counter, incremented every time a new flush cycle is requested. + /// The flush cycle counter is sent back on the layer_flush_done channel when + /// the flush finishes. You can use that to wait for the flush to finish. + /// - The LSN is updated to max() of its current value and the latest disk_consistent_lsn + /// read by whoever sends an update + layer_flush_start_tx: tokio::sync::watch::Sender<(u64, Lsn)>, /// to be notified when layer flushing has finished, subscribe to the layer_flush_done channel layer_flush_done_tx: tokio::sync::watch::Sender<(u64, Result<(), FlushLayerError>)>, @@ -1169,8 +1171,8 @@ impl Timeline { /// Flush to disk all data that was written with the put_* functions #[instrument(skip(self), fields(tenant_id=%self.tenant_shard_id.tenant_id, shard_id=%self.tenant_shard_id.shard_slug(), timeline_id=%self.timeline_id))] pub(crate) async fn freeze_and_flush(&self) -> anyhow::Result<()> { - self.freeze_inmem_layer(false).await; - self.flush_frozen_layers_and_wait().await + let to_lsn = self.freeze_inmem_layer(false).await; + self.flush_frozen_layers_and_wait(to_lsn).await } /// If there is no writer, and conditions for rolling the latest layer are met, then freeze it. @@ -1190,7 +1192,39 @@ impl Timeline { }; let Some(open_layer) = &layers_guard.layer_map().open_layer else { - // No open layer, no work to do. + // If there is no open layer, we have no layer freezing to do. However, we might need to generate + // some updates to disk_consistent_lsn and remote_consistent_lsn, in case we ingested some WAL regions + // that didn't result in writes to this shard. + + // Must not hold the layers lock while waiting for a flush. + drop(layers_guard); + + let last_record_lsn = self.get_last_record_lsn(); + let disk_consistent_lsn = self.get_disk_consistent_lsn(); + if last_record_lsn > disk_consistent_lsn { + // We have no open layer, but disk_consistent_lsn is behind the last record: this indicates + // we are a sharded tenant and have skipped some WAL + let last_freeze_ts = *self.last_freeze_ts.read().unwrap(); + if last_freeze_ts.elapsed() >= self.get_checkpoint_timeout() { + // This should be somewhat rare, so we log it at INFO level. + // + // We checked for checkpoint timeout so that a shard without any + // data ingested (yet) doesn't write a remote index as soon as it + // sees its LSN advance: we only do this if we've been layer-less + // for some time. + tracing::info!( + "Advancing disk_consistent_lsn past WAL ingest gap {} -> {}", + disk_consistent_lsn, + last_record_lsn + ); + + // The flush loop will update remote consistent LSN as well as disk consistent LSN. + self.flush_frozen_layers_and_wait(last_record_lsn) + .await + .ok(); + } + } + return; }; @@ -1769,7 +1803,7 @@ impl Timeline { let disk_consistent_lsn = metadata.disk_consistent_lsn(); let (state, _) = watch::channel(state); - let (layer_flush_start_tx, _) = tokio::sync::watch::channel(0); + let (layer_flush_start_tx, _) = tokio::sync::watch::channel((0, disk_consistent_lsn)); let (layer_flush_done_tx, _) = tokio::sync::watch::channel((0, Ok(()))); let evictions_low_residence_duration_metric_threshold = { @@ -3174,7 +3208,9 @@ impl Timeline { self.last_record_lsn.advance(new_lsn); } - async fn freeze_inmem_layer(&self, write_lock_held: bool) { + /// Whether there was a layer to freeze or not, return the value of get_last_record_lsn + /// before we attempted the freeze: this guarantees that ingested data is frozen up to this lsn (inclusive). + async fn freeze_inmem_layer(&self, write_lock_held: bool) -> Lsn { // Freeze the current open in-memory layer. It will be written to disk on next // iteration. @@ -3184,7 +3220,9 @@ impl Timeline { Some(self.write_lock.lock().await) }; - self.freeze_inmem_layer_at(self.get_last_record_lsn()).await; + let to_lsn = self.get_last_record_lsn(); + self.freeze_inmem_layer_at(to_lsn).await; + to_lsn } async fn freeze_inmem_layer_at(&self, at: Lsn) { @@ -3197,7 +3235,7 @@ impl Timeline { /// Layer flusher task's main loop. async fn flush_loop( self: &Arc, - mut layer_flush_start_rx: tokio::sync::watch::Receiver, + mut layer_flush_start_rx: tokio::sync::watch::Receiver<(u64, Lsn)>, ctx: &RequestContext, ) { info!("started flush loop"); @@ -3210,7 +3248,11 @@ impl Timeline { _ = layer_flush_start_rx.changed() => {} } trace!("waking up"); - let flush_counter = *layer_flush_start_rx.borrow(); + let (flush_counter, frozen_to_lsn) = *layer_flush_start_rx.borrow(); + + // The highest LSN to which we flushed in the loop over frozen layers + let mut flushed_to_lsn = Lsn(0); + let result = loop { if self.cancel.is_cancelled() { info!("dropping out of flush loop for timeline shutdown"); @@ -3231,7 +3273,9 @@ impl Timeline { break Ok(()); }; match self.flush_frozen_layer(layer_to_flush, ctx).await { - Ok(()) => {} + Ok(this_layer_to_lsn) => { + flushed_to_lsn = std::cmp::max(flushed_to_lsn, this_layer_to_lsn); + } Err(FlushLayerError::Cancelled) => { info!("dropping out of flush loop for timeline shutdown"); return; @@ -3240,11 +3284,36 @@ impl Timeline { FlushLayerError::Other(_) | FlushLayerError::CreateImageLayersError(_), ) => { error!("could not flush frozen layer: {err:?}"); - break err; + break err.map(|_| ()); } } timer.stop_and_record(); }; + + // Unsharded tenants should never advance their LSN beyond the end of the + // highest layer they write: such gaps between layer data and the frozen LSN + // are only legal on sharded tenants. + debug_assert!( + self.shard_identity.count.count() > 1 + || flushed_to_lsn >= frozen_to_lsn + || !flushed_to_lsn.is_valid() + ); + + if flushed_to_lsn < frozen_to_lsn && self.shard_identity.count.count() > 1 { + // If our layer flushes didn't carry disk_consistent_lsn up to the `to_lsn` advertised + // to us via layer_flush_start_rx, then advance it here. + // + // This path is only taken for tenants with multiple shards: single sharded tenants should + // never encounter a gap in the wal. + let old_disk_consistent_lsn = self.disk_consistent_lsn.load(); + tracing::debug!("Advancing disk_consistent_lsn across layer gap {old_disk_consistent_lsn}->{frozen_to_lsn}"); + if self.set_disk_consistent_lsn(frozen_to_lsn) { + if let Err(e) = self.schedule_uploads(frozen_to_lsn, vec![]) { + tracing::warn!("Failed to schedule metadata upload after updating disk_consistent_lsn: {e}"); + } + } + } + // Notify any listeners that we're done let _ = self .layer_flush_done_tx @@ -3252,7 +3321,13 @@ impl Timeline { } } - async fn flush_frozen_layers_and_wait(&self) -> anyhow::Result<()> { + /// Request the flush loop to write out all frozen layers up to `to_lsn` as Delta L0 files to disk. + /// The caller is responsible for the freezing, e.g., [`Self::freeze_inmem_layer`]. + /// + /// `last_record_lsn` may be higher than the highest LSN of a frozen layer: if this is the case, + /// it means no data will be written between the top of the highest frozen layer and to_lsn, + /// e.g. because this tenant shard has ingested up to to_lsn and not written any data locally for that part of the WAL. + async fn flush_frozen_layers_and_wait(&self, last_record_lsn: Lsn) -> anyhow::Result<()> { let mut rx = self.layer_flush_done_tx.subscribe(); // Increment the flush cycle counter and wake up the flush task. @@ -3266,9 +3341,10 @@ impl Timeline { anyhow::bail!("cannot flush frozen layers when flush_loop is not running, state is {flush_loop_state:?}") } - self.layer_flush_start_tx.send_modify(|counter| { + self.layer_flush_start_tx.send_modify(|(counter, lsn)| { my_flush_request = *counter + 1; *counter = my_flush_request; + *lsn = std::cmp::max(last_record_lsn, *lsn); }); loop { @@ -3305,16 +3381,22 @@ impl Timeline { } fn flush_frozen_layers(&self) { - self.layer_flush_start_tx.send_modify(|val| *val += 1); + self.layer_flush_start_tx.send_modify(|(counter, lsn)| { + *counter += 1; + + *lsn = std::cmp::max(*lsn, Lsn(self.last_freeze_at.load().0 - 1)); + }); } /// Flush one frozen in-memory layer to disk, as a new delta layer. + /// + /// Return value is the last lsn (inclusive) of the layer that was frozen. #[instrument(skip_all, fields(layer=%frozen_layer))] async fn flush_frozen_layer( self: &Arc, frozen_layer: Arc, ctx: &RequestContext, - ) -> Result<(), FlushLayerError> { + ) -> Result { debug_assert_current_span_has_tenant_and_timeline_id(); // As a special case, when we have just imported an image into the repository, @@ -3389,7 +3471,6 @@ impl Timeline { } let disk_consistent_lsn = Lsn(lsn_range.end.0 - 1); - let old_disk_consistent_lsn = self.disk_consistent_lsn.load(); // The new on-disk layers are now in the layer map. We can remove the // in-memory layer from the map now. The flushed layer is stored in @@ -3403,10 +3484,7 @@ impl Timeline { guard.finish_flush_l0_layer(delta_layer_to_add.as_ref(), &frozen_layer, &self.metrics); - if disk_consistent_lsn != old_disk_consistent_lsn { - assert!(disk_consistent_lsn > old_disk_consistent_lsn); - self.disk_consistent_lsn.store(disk_consistent_lsn); - + if self.set_disk_consistent_lsn(disk_consistent_lsn) { // Schedule remote uploads that will reflect our new disk_consistent_lsn self.schedule_uploads(disk_consistent_lsn, layers_to_upload)?; } @@ -3423,7 +3501,22 @@ impl Timeline { // This failpoint is used by another test case `test_pageserver_recovery`. fail_point!("flush-frozen-exit"); - Ok(()) + Ok(Lsn(lsn_range.end.0 - 1)) + } + + /// Return true if the value changed + /// + /// This function must only be used from the layer flush task, and may not be called concurrently. + fn set_disk_consistent_lsn(&self, new_value: Lsn) -> bool { + // We do a simple load/store cycle: that's why this function isn't safe for concurrent use. + let old_value = self.disk_consistent_lsn.load(); + if new_value != old_value { + assert!(new_value >= old_value); + self.disk_consistent_lsn.store(new_value); + true + } else { + false + } } /// Update metadata file diff --git a/pageserver/src/tenant/timeline/layer_manager.rs b/pageserver/src/tenant/timeline/layer_manager.rs index d54dc1642c..64edcc5e40 100644 --- a/pageserver/src/tenant/timeline/layer_manager.rs +++ b/pageserver/src/tenant/timeline/layer_manager.rs @@ -120,9 +120,10 @@ impl LayerManager { /// Called from `freeze_inmem_layer`, returns true if successfully frozen. pub(crate) async fn try_freeze_in_memory_layer( &mut self, - Lsn(last_record_lsn): Lsn, + lsn: Lsn, last_freeze_at: &AtomicLsn, ) { + let Lsn(last_record_lsn) = lsn; let end_lsn = Lsn(last_record_lsn + 1); if let Some(open_layer) = &self.layer_map.open_layer { @@ -135,8 +136,11 @@ impl LayerManager { self.layer_map.frozen_layers.push_back(open_layer_rc); self.layer_map.open_layer = None; self.layer_map.next_open_layer_at = Some(end_lsn); - last_freeze_at.store(end_lsn); } + + // Even if there was no layer to freeze, advance last_freeze_at to last_record_lsn+1: this + // accounts for regions in the LSN range where we might have ingested no data due to sharding. + last_freeze_at.store(end_lsn); } /// Add image layers to the layer map, called from `create_image_layers`. diff --git a/test_runner/fixtures/workload.py b/test_runner/fixtures/workload.py index ab8717de54..4ebc02e6fd 100644 --- a/test_runner/fixtures/workload.py +++ b/test_runner/fixtures/workload.py @@ -85,6 +85,11 @@ class Workload: if self._endpoint is not None: self._endpoint.stop() + def stop(self): + if self._endpoint is not None: + self._endpoint.stop() + self._endpoint = None + def init(self, pageserver_id: Optional[int] = None): endpoint = self.endpoint(pageserver_id) diff --git a/test_runner/regress/test_sharding.py b/test_runner/regress/test_sharding.py index 2699654f80..bca11bbbe7 100644 --- a/test_runner/regress/test_sharding.py +++ b/test_runner/regress/test_sharding.py @@ -11,7 +11,9 @@ from fixtures.neon_fixtures import ( NeonEnv, NeonEnvBuilder, StorageControllerApiException, + last_flush_lsn_upload, tenant_get_shards, + wait_for_last_flush_lsn, ) from fixtures.remote_storage import s3_storage from fixtures.types import Lsn, TenantId, TenantShardId, TimelineId @@ -466,13 +468,11 @@ def test_sharding_split_stripe_size( os.getenv("BUILD_TYPE") == "debug", reason="Avoid running bulkier ingest tests in debug mode", ) -def test_sharding_ingest( +def test_sharding_ingest_layer_sizes( neon_env_builder: NeonEnvBuilder, ): """ - Check behaviors related to ingest: - - That we generate properly sized layers - - TODO: that updates to remote_consistent_lsn are made correctly via safekeepers + Check that when ingesting data to a sharded tenant, we properly respect layer size limts. """ # Set a small stripe size and checkpoint distance, so that we can exercise rolling logic @@ -503,6 +503,7 @@ def test_sharding_ingest( workload.write_rows(4096, upload=False) workload.write_rows(4096, upload=False) workload.write_rows(4096, upload=False) + workload.validate() small_layer_count = 0 @@ -515,7 +516,9 @@ def test_sharding_ingest( shard_id = shard["shard_id"] layer_map = pageserver.http_client().layer_map_info(shard_id, timeline_id) - for layer in layer_map.historic_layers: + historic_layers = sorted(layer_map.historic_layers, key=lambda layer: layer.lsn_start) + + for layer in historic_layers: assert layer.layer_file_size is not None if layer.layer_file_size < expect_layer_size // 2: classification = "Small" @@ -552,6 +555,93 @@ def test_sharding_ingest( assert huge_layer_count <= shard_count +def test_sharding_ingest_gaps( + neon_env_builder: NeonEnvBuilder, +): + """ + Check ingest behavior when the incoming data results in some shards having gaps where + no data is ingested: they should advance their disk_consistent_lsn and remote_consistent_lsn + even if they aren't writing out layers. + """ + + # Set a small stripe size and checkpoint distance, so that we can exercise rolling logic + # without writing a lot of data. + expect_layer_size = 131072 + checkpoint_interval_secs = 5 + TENANT_CONF = { + # small checkpointing and compaction targets to ensure we generate many upload operations + "checkpoint_distance": f"{expect_layer_size}", + "compaction_target_size": f"{expect_layer_size}", + # Set a short checkpoint interval as we will wait for uploads to happen + "checkpoint_timeout": f"{checkpoint_interval_secs}s", + # Background checkpointing is done from compaction loop, so set that interval short too + "compaction_period": "1s", + } + shard_count = 4 + neon_env_builder.num_pageservers = shard_count + env = neon_env_builder.init_start( + initial_tenant_conf=TENANT_CONF, + initial_tenant_shard_count=shard_count, + initial_tenant_shard_stripe_size=128, + ) + tenant_id = env.initial_tenant + timeline_id = env.initial_timeline + + # Just a few writes: we aim to produce a situation where some shards are skipping + # ingesting some records and thereby won't have layer files that advance their + # consistent LSNs, to exercise the code paths that explicitly handle this case by + # advancing consistent LSNs in the background if there is no open layer. + workload = Workload(env, tenant_id, timeline_id) + workload.init() + workload.write_rows(128, upload=False) + workload.churn_rows(128, upload=False) + + # Checkpoint, so that we won't get a background checkpoint happening during the next step + workload.endpoint().safe_psql("checkpoint") + # Freeze + flush, so that subsequent writes will start from a position of no open layers + last_flush_lsn_upload(env, workload.endpoint(), tenant_id, timeline_id) + + # This write is tiny: at least some of the shards should find they don't have any + # data to ingest. This will exercise how they handle that. + workload.churn_rows(1, upload=False) + + # The LSN that has reached pageservers, but may not have been flushed to historic layers yet + expect_lsn = wait_for_last_flush_lsn(env, workload.endpoint(), tenant_id, timeline_id) + + # Don't leave the endpoint running, we don't want it writing in the background + workload.stop() + + log.info(f"Waiting for shards' consistent LSNs to reach {expect_lsn}") + + shards = tenant_get_shards(env, tenant_id, None) + + def assert_all_disk_consistent(): + """ + Assert that all the shards' disk_consistent_lsns have reached expect_lsn + """ + for tenant_shard_id, pageserver in shards: + timeline_detail = pageserver.http_client().timeline_detail(tenant_shard_id, timeline_id) + log.info(f"{tenant_shard_id} (ps {pageserver.id}) detail: {timeline_detail}") + assert Lsn(timeline_detail["disk_consistent_lsn"]) >= expect_lsn + + # We set a short checkpoint timeout: expect things to get frozen+flushed within that + wait_until(checkpoint_interval_secs * 3, 1, assert_all_disk_consistent) + + def assert_all_remote_consistent(): + """ + Assert that all the shards' remote_consistent_lsns have reached expect_lsn + """ + for tenant_shard_id, pageserver in shards: + timeline_detail = pageserver.http_client().timeline_detail(tenant_shard_id, timeline_id) + log.info(f"{tenant_shard_id} (ps {pageserver.id}) detail: {timeline_detail}") + assert Lsn(timeline_detail["remote_consistent_lsn"]) >= expect_lsn + + # We set a short checkpoint timeout: expect things to get frozen+flushed within that + wait_until(checkpoint_interval_secs * 3, 1, assert_all_remote_consistent) + + workload.validate() + + class Failure: pageserver_id: Optional[int] @@ -795,6 +885,8 @@ def test_sharding_split_failures( ".*Reconcile error: receive body: error sending request for url.*", # Node offline cases will fail inside reconciler when detaching secondaries ".*Reconcile error on shard.*: receive body: error sending request for url.*", + # While parent shard's client is stopped during split, flush loop updating LSNs will emit this warning + ".*Failed to schedule metadata upload after updating disk_consistent_lsn.*", ] )