Revert "pageserver: revert flush backpressure (#8550) (#10135)" (#10270)

This reverts commit f3ecd5d76a.

It is
[suspected](https://neondb.slack.com/archives/C033RQ5SPDH/p1735907405716759)
to have caused significant read amplification in the [ingest
benchmark](https://neonprod.grafana.net/d/de3mupf4g68e8e/perf-test3a-ingest-benchmark?orgId=1&from=now-30d&to=now&timezone=utc&var-new_project_endpoint_id=ep-solitary-sun-w22bmut6&var-large_tenant_endpoint_id=ep-holy-bread-w203krzs)
(specifically during index creation).

We will revisit an intermediate improvement here to unblock [upload
parallelism](https://github.com/neondatabase/neon/issues/10096) before
properly addressing [compaction
backpressure](https://github.com/neondatabase/neon/issues/8390).
This commit is contained in:
Erik Grinaker
2025-01-03 16:38:51 +01:00
committed by GitHub
parent b33299dc37
commit 1393cc668b
5 changed files with 112 additions and 13 deletions

View File

@@ -3,7 +3,7 @@ use metrics::{
register_counter_vec, register_gauge_vec, register_histogram, register_histogram_vec,
register_int_counter, register_int_counter_pair_vec, register_int_counter_vec,
register_int_gauge, register_int_gauge_vec, register_uint_gauge, register_uint_gauge_vec,
Counter, CounterVec, GaugeVec, Histogram, HistogramVec, IntCounter, IntCounterPair,
Counter, CounterVec, Gauge, GaugeVec, Histogram, HistogramVec, IntCounter, IntCounterPair,
IntCounterPairVec, IntCounterVec, IntGauge, IntGaugeVec, UIntGauge, UIntGaugeVec,
};
use once_cell::sync::Lazy;
@@ -445,6 +445,15 @@ pub(crate) static WAIT_LSN_TIME: Lazy<Histogram> = Lazy::new(|| {
.expect("failed to define a metric")
});
static FLUSH_WAIT_UPLOAD_TIME: Lazy<GaugeVec> = Lazy::new(|| {
register_gauge_vec!(
"pageserver_flush_wait_upload_seconds",
"Time spent waiting for preceding uploads during layer flush",
&["tenant_id", "shard_id", "timeline_id"]
)
.expect("failed to define a metric")
});
static LAST_RECORD_LSN: Lazy<IntGaugeVec> = Lazy::new(|| {
register_int_gauge_vec!(
"pageserver_last_record_lsn",
@@ -2577,6 +2586,7 @@ pub(crate) struct TimelineMetrics {
shard_id: String,
timeline_id: String,
pub flush_time_histo: StorageTimeMetrics,
pub flush_wait_upload_time_gauge: Gauge,
pub compact_time_histo: StorageTimeMetrics,
pub create_images_time_histo: StorageTimeMetrics,
pub logical_size_histo: StorageTimeMetrics,
@@ -2622,6 +2632,9 @@ impl TimelineMetrics {
&shard_id,
&timeline_id,
);
let flush_wait_upload_time_gauge = FLUSH_WAIT_UPLOAD_TIME
.get_metric_with_label_values(&[&tenant_id, &shard_id, &timeline_id])
.unwrap();
let compact_time_histo = StorageTimeMetrics::new(
StorageTimeOperation::Compact,
&tenant_id,
@@ -2767,6 +2780,7 @@ impl TimelineMetrics {
shard_id,
timeline_id,
flush_time_histo,
flush_wait_upload_time_gauge,
compact_time_histo,
create_images_time_histo,
logical_size_histo,
@@ -2816,6 +2830,14 @@ impl TimelineMetrics {
self.resident_physical_size_gauge.get()
}
pub(crate) fn flush_wait_upload_time_gauge_add(&self, duration: f64) {
self.flush_wait_upload_time_gauge.add(duration);
crate::metrics::FLUSH_WAIT_UPLOAD_TIME
.get_metric_with_label_values(&[&self.tenant_id, &self.shard_id, &self.timeline_id])
.unwrap()
.add(duration);
}
pub(crate) fn shutdown(&self) {
let was_shutdown = self
.shutdown
@@ -2833,6 +2855,7 @@ impl TimelineMetrics {
let shard_id = &self.shard_id;
let _ = LAST_RECORD_LSN.remove_label_values(&[tenant_id, shard_id, timeline_id]);
let _ = DISK_CONSISTENT_LSN.remove_label_values(&[tenant_id, shard_id, timeline_id]);
let _ = FLUSH_WAIT_UPLOAD_TIME.remove_label_values(&[tenant_id, shard_id, timeline_id]);
let _ = STANDBY_HORIZON.remove_label_values(&[tenant_id, shard_id, timeline_id]);
{
RESIDENT_PHYSICAL_SIZE_GLOBAL.sub(self.resident_physical_size_get());

View File

@@ -144,15 +144,19 @@ use self::layer_manager::LayerManager;
use self::logical_size::LogicalSize;
use self::walreceiver::{WalReceiver, WalReceiverConf};
use super::config::TenantConf;
use super::remote_timeline_client::index::IndexPart;
use super::remote_timeline_client::RemoteTimelineClient;
use super::secondary::heatmap::{HeatMapLayer, HeatMapTimeline};
use super::storage_layer::{LayerFringe, LayerVisibilityHint, ReadableLayer};
use super::upload_queue::NotInitialized;
use super::GcError;
use super::{
debug_assert_current_span_has_tenant_and_timeline_id, AttachedTenantConf, MaybeOffloaded,
config::TenantConf, storage_layer::LayerVisibilityHint, upload_queue::NotInitialized,
MaybeOffloaded,
};
use super::{debug_assert_current_span_has_tenant_and_timeline_id, AttachedTenantConf};
use super::{remote_timeline_client::index::IndexPart, storage_layer::LayerFringe};
use super::{
remote_timeline_client::RemoteTimelineClient, remote_timeline_client::WaitCompletionError,
storage_layer::ReadableLayer,
};
use super::{
secondary::heatmap::{HeatMapLayer, HeatMapTimeline},
GcError,
};
#[cfg(test)]
@@ -3836,6 +3840,24 @@ impl Timeline {
// release lock on 'layers'
};
// Backpressure mechanism: wait with continuation of the flush loop until we have uploaded all layer files.
// This makes us refuse ingest until the new layers have been persisted to the remote
let start = Instant::now();
self.remote_client
.wait_completion()
.await
.map_err(|e| match e {
WaitCompletionError::UploadQueueShutDownOrStopped
| WaitCompletionError::NotInitialized(
NotInitialized::ShuttingDown | NotInitialized::Stopped,
) => FlushLayerError::Cancelled,
WaitCompletionError::NotInitialized(NotInitialized::Uninitialized) => {
FlushLayerError::Other(anyhow!(e).into())
}
})?;
let duration = start.elapsed().as_secs_f64();
self.metrics.flush_wait_upload_time_gauge_add(duration);
// FIXME: between create_delta_layer and the scheduling of the upload in `update_metadata_file`,
// a compaction can delete the file and then it won't be available for uploads any more.
// We still schedule the upload, resulting in an error, but ideally we'd somehow avoid this

View File

@@ -170,6 +170,7 @@ PAGESERVER_PER_TENANT_METRICS: tuple[str, ...] = (
"pageserver_evictions_with_low_residence_duration_total",
"pageserver_aux_file_estimated_size",
"pageserver_valid_lsn_lease_count",
"pageserver_flush_wait_upload_seconds",
counter("pageserver_tenant_throttling_count_accounted_start"),
counter("pageserver_tenant_throttling_count_accounted_finish"),
counter("pageserver_tenant_throttling_wait_usecs_sum"),

View File

@@ -19,7 +19,6 @@ from fixtures.pageserver.utils import wait_until_tenant_active
from fixtures.utils import query_scalar
from performance.test_perf_pgbench import get_scales_matrix
from requests import RequestException
from requests.exceptions import RetryError
# Test branch creation
@@ -177,8 +176,11 @@ def test_cannot_create_endpoint_on_non_uploaded_timeline(neon_env_builder: NeonE
env.neon_cli.mappings_map_branch(initial_branch, env.initial_tenant, env.initial_timeline)
with pytest.raises(RuntimeError, match="is not active, state: Loading"):
env.endpoints.create_start(initial_branch, tenant_id=env.initial_tenant)
with pytest.raises(RuntimeError, match="ERROR: Not found: Timeline"):
env.endpoints.create_start(
initial_branch, tenant_id=env.initial_tenant, basebackup_request_tries=2
)
ps_http.configure_failpoints(("before-upload-index-pausable", "off"))
finally:
env.pageserver.stop(immediate=True)
@@ -219,7 +221,10 @@ def test_cannot_branch_from_non_uploaded_branch(neon_env_builder: NeonEnvBuilder
branch_id = TimelineId.generate()
with pytest.raises(RetryError, match="too many 503 error responses"):
with pytest.raises(
PageserverApiException,
match="Cannot branch off the timeline that's not present in pageserver",
):
ps_http.timeline_create(
env.pg_version,
env.initial_tenant,

View File

@@ -784,6 +784,54 @@ def test_empty_branch_remote_storage_upload_on_restart(neon_env_builder: NeonEnv
create_thread.join()
def test_paused_upload_stalls_checkpoint(
neon_env_builder: NeonEnvBuilder,
):
"""
This test checks that checkpoints block on uploads to remote storage.
"""
neon_env_builder.enable_pageserver_remote_storage(RemoteStorageKind.LOCAL_FS)
env = neon_env_builder.init_start(
initial_tenant_conf={
# Set a small compaction threshold
"compaction_threshold": "3",
# Disable GC
"gc_period": "0s",
# disable PITR
"pitr_interval": "0s",
}
)
env.pageserver.allowed_errors.append(
f".*PUT.* path=/v1/tenant/{env.initial_tenant}/timeline.* request was dropped before completing"
)
tenant_id = env.initial_tenant
timeline_id = env.initial_timeline
client = env.pageserver.http_client()
layers_at_creation = client.layer_map_info(tenant_id, timeline_id)
deltas_at_creation = len(layers_at_creation.delta_layers())
assert (
deltas_at_creation == 1
), "are you fixing #5863? make sure we end up with 2 deltas at the end of endpoint lifecycle"
# Make new layer uploads get stuck.
# Note that timeline creation waits for the initial layers to reach remote storage.
# So at this point, the `layers_at_creation` are in remote storage.
client.configure_failpoints(("before-upload-layer-pausable", "pause"))
with env.endpoints.create_start("main", tenant_id=tenant_id) as endpoint:
# Build two tables with some data inside
endpoint.safe_psql("CREATE TABLE foo AS SELECT x FROM generate_series(1, 10000) g(x)")
wait_for_last_flush_lsn(env, endpoint, tenant_id, timeline_id)
with pytest.raises(ReadTimeout):
client.timeline_checkpoint(tenant_id, timeline_id, timeout=5)
client.configure_failpoints(("before-upload-layer-pausable", "off"))
def wait_upload_queue_empty(
client: PageserverHttpClient, tenant_id: TenantId, timeline_id: TimelineId
):