## Problem
https://github.com/neondatabase/neon/pull/11140 introduces performance
tracing with OTEL
and a pageserver config which configures the sampling ratio of get page
requests.
Enabling a non-zero sampling ratio on a per region basis is too
aggressive and comes with perf
impact that isn't very well understood yet.
## Summary of changes
Add a `sampling_ratio` tenant level config which overrides the
pageserver level config.
Note that we do not cache the config and load it on every get page
request such that changes propagate
timely.
Note that I've had to remove the `SHARD_SELECTION` span to get this to
work. The tracing library doesn't
expose a neat way to drop a span if one realises it's not needed at
runtime.
Closes https://github.com/neondatabase/neon/issues/11392
Based on https://github.com/neondatabase/neon/pull/11139
## Problem
We want to export performance traces from the pageserver in OTEL format.
End goal is to see them in Grafana.
## Summary of changes
https://github.com/neondatabase/neon/pull/11139 introduces the
infrastructure required to run the otel collector alongside the
pageserver.
### Design
Requirements:
1. We'd like to avoid implementing our own performance tracing stack if
possible and use the `tracing` crate if possible.
2. Ideally, we'd like zero overhead of a sampling rate of zero and be a
be able to change the tracing config for a tenant on the fly.
3. We should leave the current span hierarchy intact. This includes
adding perf traces without modifying existing tracing.
To satisfy (3) (and (2) in part) a separate span hierarchy is used.
`RequestContext` gains an optional `perf_span` member
that's only set when the request was chosen by sampling. All perf span
related methods added to `RequestContext` are no-ops for requests that
are not sampled.
This on its own is not enough for (3), so performance spans use a
separate tracing subscriber. The `tracing` crate doesn't have great
support for this, so there's a fair amount of boilerplate to override
the subscriber at all points of the perf span lifecycle.
### Perf Impact
[Periodic
pagebench](https://neonprod.grafana.net/d/ddqtbfykfqfi8d/e904990?orgId=1&from=2025-02-08T14:15:59.362Z&to=2025-03-10T14:15:59.362Z&timezone=utc)
shows no statistically significant regression with a sample ratio of 0.
There's an annotation on the dashboard on 2025-03-06.
### Overview of changes:
1. Clean up the `RequestContext` API a bit. Namely, get rid of the
`RequestContext::extend` API and use the builder instead.
2. Add pageserver level configs for tracing: sampling ratio, otel
endpoint, etc.
3. Introduce some perf span tracking utilities and expose them via
`RequestContext`. We add a `tracing::Span` wrapper to be used for perf
spans and a `tracing::Instrumented` equivalent for it. See doc comments
for reason.
4. Set up OTEL tracing infra according to configuration. A separate
runtime is used for the collector.
5. Add perf traces to the read path.
## Refs
- epic https://github.com/neondatabase/neon/issues/9873
---------
Co-authored-by: Christian Schwarz <christian@neon.tech>
## Problem
Previously, L0 flushes would wait for uploads, as a simple form of
backpressure. However, this prevented flush pipelining and upload
parallelism. It has since been disabled by default and replaced by L0
compaction backpressure.
Touches https://github.com/neondatabase/cloud/issues/24664.
## Summary of changes
This patch removes L0 flush upload waits, along with the
`l0_flush_wait_upload`. This can't be merged until the setting has been
removed across the fleet.
## Problem
part of https://github.com/neondatabase/neon/issues/11279
## Summary of changes
The invisible flag is used to exclude a timeline from synthetic size
calculation. For the first step, let's persist this flag. Most of the
code are following the `is_archived` modification flow.
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
SSL certs are loaded only during start up. It doesn't allow the rotation
of short-lived certificates without server restart.
- Closes: https://github.com/neondatabase/cloud/issues/25525
## Summary of changes
- Implement `ReloadingCertificateResolver` which reloads certificates
from disk periodically.
## Problem
Pageservers use unencrypted HTTP requests for storage controller API.
- Closes: https://github.com/neondatabase/cloud/issues/25524
## Summary of changes
- Replace hyper0::server::Server with http_utils::server::Server in
storage controller.
- Add HTTPS handler for storage controller API.
- Support `ssl_ca_file` in pageserver.
## Problem
The pageserver upcall api was designed to work with control plane or the
storage controller.
We have completed the transition period and now the upcall api only
targets the storage controller.
## Summary of changes
Rename types accordingly and tweak some comments.
The only difference between
- `pageserver_api::models::TenantConfig` and
- `pageserver::tenant::config::TenantConfOpt`
at this point is that `TenantConfOpt` serializes with
`skip_serializing_if = Option::is_none`.
That is an efficiency improvement for all the places that currently
serde `models::TenantConfig` because new serializations will no longer
write `$fieldname: null` for each field that is `None` at runtime.
This should be particularly beneficial for Storcon, which stores
JSON-serialized `models::TenantConfig` in its DB.
# Behavior Changes
This PR changes the serialization behavior: we omit `None` fields
instead of serializing `$fieldname: null`).
So it's a data format change (see section on compatibility below).
And it changes API responses from Storcon and Pageserver.
## API Response Compatibility
Storcon returns the location description.
Afaik it is passed through into
- storcon_cli output
- storcon UI in console admin UI
These outputs will no longer contain `$fieldname: null` values,
which de-bloats the output (good).
But in storcon UI, it also serves as an editor "default", which
will be eliminated after a storcon with this PR is released.
## Data Format Compatibility
Backwards compat: new software reading old serialized data will
deserialize to the same runtime value because all the field types
are exactly the same and `skip_serializing_if` does not affect
deserialization.
Forward compat: old software reading data serialized by new software
will map absence fields in the serialized form to runtime value
`Option::None`. This is serde default behavior, see this playground
to convince yourself:
https://play.rust-lang.org/?version=stable&mode=debug&edition=2024&gist=f7f4e1a169959a3085b6158c022a05eb
The `serde(with="humantime_serde")` however behaves strangely:
if used on an `Option<Duration>`, it still requires the field to be
present,
unlike the serde default behavior shown in the previous paragraph.
The workaround is to set `serde(default)`.
Previously it was set on each individual field, but, we do have the
container attribute, so, set it there.
This requires deriving a `Default` impl, which, because all fields are
`Option`,
is non-magic.
See my notes here:
https://gist.github.com/problame/eddbc225a5d12617e9f2c6413e0cf799
# Future Work
We should have separate types (& crates) for
- runtime types configuration (e.g. PageServerConf::tenant_config,
AttachedLocationConf)
- `config-v1` file pageserver local disk file format
- `mgmt API`
- `pageserver.toml`
Right now they all use the same, which is convenient but makes it hard
to reason about compatibility breakage.
# Refs
- corresponding docs.neon.build PR
https://github.com/neondatabase/docs/pull/470
## Problem
This is already disabled in production, as it is replaced by L0 flush
delays. It will be removed in a later PR, once the config option is no
longer specified in production.
## Summary of changes
Disable `l0_flush_wait_upload` by default.
## Problem
`l0_flush_delay_threshold` has already been set to 30 in production for
a couple of weeks. Let's harmonize the default.
## Summary of changes
Update `DEFAULT_L0_FLUSH_DELAY_FACTOR` to 3 such that the default
`l0_flush_delay_threshold` is `3 * compaction_threshold`.
This differs from the production setting, which is hardcoded to 30 (with
`compaction_threshold` at 10), and is more appropriate for any tenants
that have custom `compaction_threshold` overrides.
## Problem
`compaction_l0_first` has already been enabled in production for a
couple of weeks.
## Summary of changes
Enable `compaction_l0_first` by default.
Also set `CompactFlags::NoYield` in `timeline_checkpoint_handler`, to
ensure explicitly requested compaction runs to completion. This endpoint
is mainly used in tests, and caused some flakiness where tests expected
compaction to complete.
## Problem
When the storage controller and Pageserver loads tenants from persisted
storage, it uses `ShardIdentity::unsharded()` for unsharded tenants.
However, this replaces the persisted stripe size of unsharded tenants
with the default stripe size.
This doesn't really matter for practical purposes, since the stripe size
is meaningless for unsharded tenants anyway, but can cause consistency
check failures if the persisted stripe size differs from the default.
This was seen in #11168, where we change the default stripe size.
Touches #11168.
## Summary of changes
Carry over the persisted stripe size from `TenantShardPersistence` for
unsharded tenants, and from `LocationConf` on Pageservers.
Also add bounds checks for type casts when loading persisted shard
metadata.
## Problem
This field was retained for backward compat only in
https://github.com/neondatabase/neon/pull/10707.
Once https://github.com/neondatabase/cloud/pull/25233 is released,
nothing external will be reading this field.
Internally, this was a mandatory field so storage controller is still
trying to decode it, so we must do this removal in two steps: this PR
makes the field optional, and after one release we can fully remove it.
Related: https://github.com/neondatabase/cloud/issues/24250
## Summary of changes
- Rename field to `_unused`
- Remove field from swagger
- Make field optional
## Problem
In #11122, we want to split shards once the logical size of the largest
timeline exceeds a split threshold. However, `get_top_tenants` currently
only returns `max_logical_size`, which tracks the max _total_ logical
size of a timeline across all shards.
This is problematic, because the storage controller needs to fetch a
list of N tenants that are eligible for splits, but the API doesn't
currently have a way to express this. For example, with a split
threshold of 1 GB, a tenant with `max_logical_size` of 4 GB is eligible
to split if it has 1 or 2 shards, but not if it already has 4 shards. We
need to express this in per-shard terms, otherwise the `get_top_tenants`
endpoint may end up only returning tenants that can't be split, blocking
splits entirely.
Touches https://github.com/neondatabase/neon/pull/11122.
Touches https://github.com/neondatabase/cloud/issues/22532.
## Summary of changes
Add `TenantShardItem::max_logical_size_per_shard` containing
`max_logical_size / shard_count`, and
`TenantSorting::MaxLogicalSizePerShard` to order and filter by it.
This PR extends the storcon with basic safekeeper management of
timelines, mainly timeline creation and deletion. We want to make the
storcon manage safekeepers in the future. Timeline creation is
controlled by the `--timelines-onto-safekeepers` flag.
1. it adds the `timelines` and `safekeeper_timeline_pending_ops` tables
to the storcon db
2. extend code for the timeline creation and deletion
4. it adds per-safekeeper reconciler tasks
TODO:
* maybe not immediately schedule reconciliations for deletions but have
a prior manual step
* tenant deletions
* add exclude API definitions (probably separate PR)
* how to choose safekeeper to do exclude on vs deletion? this can be a
bit hairy because the safekeeper might go offline in the meantime.
* error/failure case handling
* tests (cc test_explicit_timeline_creation from #11002)
* single safekeeper mode: we often only have one SK (in tests for
example)
* `notify-safekeepers` hook:
https://github.com/neondatabase/neon/issues/11163
TODOs implemented:
* cancellations of enqueued reconciliations on a per-timeline basis,
helpful if there is an ongoing deletion
* implement pending ops overwrite behavior
* load pending operations from db
RFC section for important reading:
[link](https://github.com/neondatabase/neon/blob/main/docs/rfcs/035-safekeeper-dynamic-membership-change.md#storage_controller-implementation)
Implements the bulk of #9011
Successor of #10440.
---------
Co-authored-by: Arseny Sher <sher-ars@yandex.ru>
## Problem
As part of the disaster recovery tool. Partly for
https://github.com/neondatabase/neon/issues/9114.
## Summary of changes
* Add a new pageserver API to force patch the fields in index_part and
modify the timeline internal structures.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
Storage controller uses http for requests to safekeeper management API.
Closes: https://github.com/neondatabase/cloud/issues/24835
## Summary of changes
- Add `use_https_safekeeper_api` option to storcon to use https api
- Use https for requests to safekeeper management API if this option is
enabled
- Add `ssl_ca_file` option to storcon for ability to specify custom root
CA certificate
## Problem
The current migration API does a live migration, but if the destination
doesn't already have a secondary, that live migration is unlikely to be
able to warm up a tenant properly within its timeout (full warmup of a
big tenant can take tens of minutes).
Background optimisation code knows how to do this gracefully by creating
a secondary first, but we don't currently give a human a way to trigger
that.
Closes: https://github.com/neondatabase/neon/issues/10540
## Summary of changes
- Add `prefererred_node` parameter to TenantShard, which is respected by
optimize_attachment
- Modify migration API to have optional prewarm=true mode, in which we
set preferred_node and call optimize_attachment, rather than directly
modifying intentstate
- Require override_scheduler=true flag if migrating somewhere that is a
less-than-optimal scheduling location (e.g. wrong AZ)
- Add `origin_node_id` to migration API so that callers can ensure
they're moving from where they think they're moving from
- Add tests for the above
The storcon_cli wrapper for this has a 'watch' mode that waits for
eventual cutover. This doesn't show the warmth of the secondary evolve
because we don't currently have an API for that in the controller, as
the passthrough API only targets attached locations, not secondaries. It
would be straightforward to add later as a dedicated endpoint for
getting secondary status, then extend the storcon_cli to consume that
and print a nice progress indicator.
## Problem
On unarchival, we update the previous heatmap with all visible layers.
When the primary generates a new heatmap it includes all those layers,
so the secondary will download them. Since they're not actually resident
on the primary (we didn't call the warm up API), they'll never be
evicted, so they remain in the heatmap.
This leads to oversized secondary locations like we saw in pre-prod.
## Summary of changes
Gate the loading of the previous heatmaps and the heatmap generation on
unarchival behind configuration
flags. They are disabled by default, but enabled in tests.
## Problem
part of https://github.com/neondatabase/neon/issues/9516
## Summary of changes
Similar to the aux v2 migration, we persist the relv2 migration status
into index_part, so that even the config item is set to false, we will
still read from the v2 storage to avoid loss of data.
Note that only the two variants `None` and
`Some(RelSizeMigration::Migrating)` are used for now. We don't have full
migration implemented so it will never be set to
`RelSizeMigration::Migrated`.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
Updates storage components to edition 2024. We like to stay on the
latest edition if possible. There is no functional changes, however some
code changes had to be done to accommodate the edition's breaking
changes.
The PR has two commits:
* the first commit updates storage crates to edition 2024 and appeases
`cargo clippy` by changing code. i have accidentially ran the formatter
on some files that had other edits.
* the second commit performs a `cargo fmt`
I would recommend a closer review of the first commit and a less close
review of the second one (as it just runs `cargo fmt`).
part of https://github.com/neondatabase/neon/issues/10918
## Problem
part of https://github.com/neondatabase/neon/issues/9114
## Summary of changes
Add the auto trigger for gc-compaction. It computes two values: L1 size
and L2 size. When L1 size >= initial trigger threshold, we will trigger
an initial gc-compaction. When l1_size / l2_size >=
gc_compaction_ratio_percent, we will trigger the "tiered" gc-compaction.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
The storage controller treats durations in the tenant config as strings.
These are loaded from the db.
The pageserver maps these durations to a seconds only format and we
always get a mismatch compared
to what's in the db.
## Summary of changes
Treat durations as durations inside the storage controller and not as
strings.
Nothing changes in the cross service API's themselves or the way things
are stored in the db.
I also added some logging which I would have made the investigation a
10min job:
1. Reason for why the reconciliation was spawned
2. Location config diff between the observed and wanted states
## Problem
The interpreted SK <-> PS protocol does not guard against gaps (neither
does the Vanilla one, but that's beside the point).
## Summary of changes
Extend the protocol to include the start LSN of the PG WAL section from
which the records were interpreted.
Validation is enabled via a config flag on the pageserver and works as
follows:
**Case 1**: `raw_wal_start_lsn` is smaller than the requested LSN
There can't be gaps here, but we check that the shard received records
which it hasn't seen before.
**Case 2**: `raw_wal_start_lsn` is equal to the requested LSN
This is the happy case. No gap and nothing to check
**Case 3**: `raw_wal_start_lsn` is greater than the requested LSN
This is a gap.
To make Case 3 work I had to bend the protocol a bit.
We read record chunks of WAL which aren't record aligned and feed them
to the decoder.
The picture below shows a shard which subscribes at a position somewhere
within Record 2.
We already have a wal reader which is below that position so we wait to
catch up.
We read some wal in Read 1 (all of Record 1 and some of Record 2). The
new shard doesn't
need Record 1 (it has already processed it according to the starting
position), but we read
past it's starting position. When we do Read 2, we decode Record 2 and
ship it off to the shard,
but the starting position of Read 2 is greater than the starting
position the shard requested.
This looks like a gap.

To make it work, we extend the protocol to send an empty
`InterpretedWalRecords` to shards
if the WAL the records originated from ends the requested start
position. On the pageserver,
that just updates the tracking LSNs in memory (no-op really). This gives
us a workaround for
the fake gap.
As a drive by, make `InterpretedWalRecords::next_record_lsn` mandatory
in the application level definition.
It's always included.
Related: https://github.com/neondatabase/cloud/issues/23935
## Problem
Storage controller uses unsecure http for pageserver API.
Closes: https://github.com/neondatabase/cloud/issues/23734
Closes: https://github.com/neondatabase/cloud/issues/24091
## Summary of changes
- Add an optional `listen_https_port` field to storage controller's Node
state and its API (RegisterNode/ListNodes/etc).
- Allow updating `listen_https_port` on node registration to gradually
add https port for all nodes.
- Add `use_https_pageserver_api` CLI option to storage controller to
enable https.
- Pageserver doesn't support https for now and always reports
`https_port=None`. This will be addressed in follow-up PR.
## Problem
We've seen the previous default of 50 cause OOMs. Compacting many L0
layers at once now has limited benefit, since the cost is mostly linear
anyway. This is already being reduced to 20 in production settings.
## Summary of changes
Reduce `DEFAULT_COMPACTION_UPPER_LIMIT` to 20.
Once released, let's remove the config overrides.
## Problem
In #10707 some new fields were introduced in TimelineInfo.
I forgot that we do not only use TimelineInfo for encoding, but also
decoding when the storage controller calls into a pageserver, so this
broke some calls from controller to pageserver while in a mixed-version
state.
## Summary of changes
- Make new fields have default behavior so that they are optional
## Problem
Part of https://github.com/neondatabase/neon/issues/9516
## Summary of changes
This patch adds the support for storing reldir in the sparse keyspace.
All logic are guarded with the `rel_size_v2_enabled` flag, so if it's
set to false, the code path is exactly the same as what's currently in
prod.
Note that we did not persist the `rel_size_v2_enabled` flag and the
logic around it will be implemented in the next patch. (i.e., what if we
enabled it, restart the pageserver, and then it gets set to false? we
should still read from v2 using the rel_size_v2_migration_status in the
index_part). The persistence logic I'll implement in the next patch will
disallow switching from v2->v1 via config item.
I also refactored the metrics so that it can work with the new reldir
store. However, this metric is not correctly computed for reldirs (see
the comments) before. With the refactor, the value will be computed only
when we have an initial value for the reldir size. The refactor keeps
the incorrectness of the computation when there are more than 1
database.
For the tests, we currently run all the tests with v2, and I'll set it
to false and add some v2-specific tests before merging, probably also
v1->v2 migration tests.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
Previously, when cutting over to cold secondary locations,
we would clobber the previous, good, heatmap with a cold one.
This is because heatmap generation used to include only resident layers.
Once this merges, we can add an endpoint which triggers full heatmap
hydration on attached locations to heal cold migrations.
## Summary of changes
With this patch, heatmap generation becomes additive. If we have a
heatmap from when this location was secondary, the new uploaded heatmap
will be the result of a reconciliation between the old one and the on
disk resident layers.
More concretely, when we have the previous heatmap:
1. Filter the previous heatmap and keep layers that are (a) present
in the current layer map, (b) visible, (c) not resident. Call this set
of layers `visible_non_resident`.
2. From the layer map, select all layers that are resident and visible.
Call this set of layers `resident`.
3. The new heatmap is the result of merging the two disjoint sets.
Related https://github.com/neondatabase/neon/issues/10541
## Problem
We expose `latest_gc_cutoff` in our API, and callers understandably were
using that to validate LSNs for branch creation. However, this is _not_
the true GC cutoff from a user's point of view: it's just the point at
which we last actually did GC. The actual cutoff used when validating
branch creations and page_service reads is the min() of latest_gc_cutoff
and the planned GC lsn in GcInfo.
Closes: https://github.com/neondatabase/neon/issues/10639
## Summary of changes
- Expose the more useful min() of GC cutoffs as `gc_cutoff_lsn` in the
API, so that the most obviously named field is really the one people
should use.
- Retain the ability to read the LSN at which GC was actually done, in
an `applied_gc_cutoff_lsn` field.
- Internally rename `latest_gc_cutoff_lsn` to `applied_gc_cutoff_lsn`
("latest" was a confusing name, as the value in GcInfo is more up to
date in terms of what a user experiences)
- Temporarily preserve the old `latest_gc_cutoff_lsn` field for compat
with control plane until we update it to use the new field.
---------
Co-authored-by: Arpad Müller <arpad-m@users.noreply.github.com>
## Problem
L0 compaction frequently gets starved out by other background tasks and
image/GC compaction. L0 compaction must be responsive to keep read
amplification under control.
Touches #10694.
Resolves#10689.
## Summary of changes
Use a separate semaphore for the L0-only compaction pass.
* Add a `CONCURRENT_L0_COMPACTION_TASKS` semaphore and
`BackgroundLoopKind::L0Compaction`.
* Add a setting `compaction_l0_semaphore` (default off via
`compaction_l0_first`).
* Use the L0 semaphore when doing an `OnlyL0Compaction` pass.
* Use the background semaphore when doing a regular compaction pass
(which includes an initial L0 pass).
* While waiting for the background semaphore, yield for L0 compaction if
triggered.
* Add `CompactFlags::NoYield` to disable L0 yielding, and set it for the
HTTP API route.
* Remove the old `use_compaction_semaphore` setting and
compaction-scoped semaphore.
* Remove the warning when waiting for a semaphore; it's noisy and we
have metrics.
## Problem
Image compaction can starve out L0 compaction if a tenant has several
timelines with L0 debt.
Touches #10694.
Requires #10740.
## Summary of changes
* Add an initial L0 compaction pass, in order of L0 count.
* Add a tenant option `compaction_l0_first` to control the L0 pass
(disabled by default).
* Add `CompactFlags::OnlyL0Compaction` to run an L0-only compaction
pass.
* Clean up the compaction iteration logic.
A later PR will use separate semaphores for the L0 and image compaction
passes to avoid cross-tenant L0 starvation. That PR will also make image
compaction yield if _any_ of the tenant's timelines have pending L0
compaction to further avoid starvation.
This changes the default value of the `timeline_offloading` pageserver
and tenant configs to true, now that offloading has been rolled out
without problems.
There is also a small fix in the tenant config merge function, where we
applied the `lazy_slru_download` value instead of `timeline_offloading`.
Related issue: https://github.com/neondatabase/cloud/issues/21353
## Problem
helps investigate https://github.com/neondatabase/neon/issues/10482
## Summary of changes
In debug mode and testing mode, we will record all files visited by a
read operation, and print it out when it errors.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
L0 compaction can get starved by other background tasks. It needs to be
responsive to avoid read amp blowing up during heavy write workloads.
Touches #10694.
## Summary of changes
Add a separate semaphore for compaction, configurable via
`use_compaction_semaphore` (disabled by default). This is primarily for
testing in staging; it needs further work (in particular to split
image/L0 compaction jobs) before it can be enabled.
## Problem
Image layer generation could block L0 compactions for a long time.
## Summary of changes
* Refactored the return value of `create_image_layers_for_*` functions
to make it self-explainable.
* Preempt image layer generation in `Try` mode if L0 piles up.
Note that we might potentially run into a state that only the beginning
part of the keyspace gets image coverage. In that case, we either need
to implement something to prioritize some keyspaces with image coverage,
or tune the image_creation_threshold to ensure that the frequency of
image creation could keep up with L0 compaction.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
Co-authored-by: Erik Grinaker <erik@neon.tech>
## Problem
Follow-up of the incident, we should not use the same bound on
lower/upper limit of compaction files. This patch adds an upper bound
limit, which is set to 50 for now.
## Summary of changes
Add `compaction_upper_limit`.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
Co-authored-by: Christian Schwarz <christian@neon.tech>
## Problem
We've seen the ingest connection manager get stuck shortly after a
migration.
## Summary of changes
A speculative mitigation is to use the same mechanism as get page
requests for kicking LSN ingest. The connection manager monitors
LSN waits and queries the broker if no updates are received for the
timeline.
Closes https://github.com/neondatabase/neon/issues/10351
## Problem
We need a setting to disable the flush upload wait, to test L0 flush
backpressure in staging.
## Summary of changes
Add `l0_flush_wait_upload` setting.
## Problem
There is no direct backpressure for compaction and L0 read
amplification. This allows a large buildup of compaction debt and read
amplification.
Resolves#5415.
Requires #10402.
## Summary of changes
Delay layer flushes based on the number of level 0 delta layers:
* `l0_flush_delay_threshold`: delay flushes such that they take 2x as
long (default `2 * compaction_threshold`).
* `l0_flush_stall_threshold`: stall flushes until level 0 delta layers
drop below threshold (default `4 * compaction_threshold`).
If either threshold is reached, ephemeral layer rolls also synchronously
wait for layer flushes to propagate this backpressure up into WAL
ingestion. This will bound the number of frozen layers to 1 once
backpressure kicks in, since all other frozen layers must flush before
the rolled layer.
## Analysis
This will significantly change the compute backpressure characteristics.
Recall the three compute backpressure knobs:
* `max_replication_write_lag`: 500 MB (based on Pageserver
`last_received_lsn`).
* `max_replication_flush_lag`: 10 GB (based on Pageserver
`disk_consistent_lsn`).
* `max_replication_apply_lag`: disabled (based on Pageserver
`remote_consistent_lsn`).
Previously, the Pageserver would keep ingesting WAL and build up
ephemeral layers and L0 layers until the compute hit
`max_replication_flush_lag` at 10 GB and began backpressuring. Now, once
we delay/stall WAL ingestion, the compute will begin backpressuring
after `max_replication_write_lag`, i.e. 500 MB. This is probably a good
thing (we're not building up a ton of compaction debt), but we should
consider tuning these settings.
`max_replication_flush_lag` probably doesn't serve a purpose anymore,
and we should consider removing it.
Furthermore, the removal of the upload barrier in #10402 will mean that
we no longer backpressure flushes based on S3 uploads, since
`max_replication_apply_lag` is disabled. We should consider enabling
this as well.
### When and what do we compact?
Default compaction settings:
* `compaction_threshold`: 10 L0 delta layers.
* `compaction_period`: 20 seconds (between each compaction loop check).
* `checkpoint_distance`: 256 MB (size of L0 delta layers).
* `l0_flush_delay_threshold`: 20 L0 delta layers.
* `l0_flush_stall_threshold`: 40 L0 delta layers.
Compaction characteristics:
* Minimum compaction volume: 10 layers * 256 MB = 2.5 GB.
* Additional compaction volume (assuming 128 MB/s WAL): 128 MB/s * 20
seconds = 2.5 GB (10 L0 layers).
* Required compaction bandwidth: 5.0 GB / 20 seconds = 256 MB/s.
### When do we hit `max_replication_write_lag`?
Depending on how fast compaction and flushes happens, the compute will
backpressure somewhere between `l0_flush_delay_threshold` or
`l0_flush_stall_threshold` + `max_replication_write_lag`.
* Minimum compute backpressure lag: 20 layers * 256 MB + 500 MB = 5.6 GB
* Maximum compute backpressure lag: 40 layers * 256 MB + 500 MB = 10.0
GB
This seems like a reasonable range to me.
## Refs
- Epic: https://github.com/neondatabase/neon/issues/9378
Co-authored-by: Vlad Lazar <vlad@neon.tech>
Co-authored-by: Christian Schwarz <christian@neon.tech>
## Problem
The read path does its IOs sequentially.
This means that if N values need to be read to reconstruct a page,
we will do N IOs and getpage latency is `O(N*IoLatency)`.
## Solution
With this PR we gain the ability to issue IO concurrently within one
layer visit **and** to move on to the next layer without waiting for IOs
from the previous visit to complete.
This is an evolved version of the work done at the Lisbon hackathon,
cf https://github.com/neondatabase/neon/pull/9002.
## Design
### `will_init` now sourced from disk btree index keys
On the algorithmic level, the only change is that the
`get_values_reconstruct_data`
now sources `will_init` from the disk btree index key (which is
PS-page_cache'd), instead
of from the `Value`, which is only available after the IO completes.
### Concurrent IOs, Submission & Completion
To separate IO submission from waiting for its completion, while
simultaneously
feature-gating the change, we introduce the notion of an `IoConcurrency`
struct
through which IO futures are "spawned".
An IO is an opaque future, and waiting for completions is handled
through
`tokio::sync::oneshot` channels.
The oneshot Receiver's take the place of the `img` and `records` fields
inside `VectoredValueReconstructState`.
When we're done visiting all the layers and submitting all the IOs along
the way
we concurrently `collect_pending_ios` for each value, which means
for each value there is a future that awaits all the oneshot receivers
and then calls into walredo to reconstruct the page image.
Walredo is now invoked concurrently for each value instead of
sequentially.
Walredo itself remains unchanged.
The spawned IO futures are driven to completion by a sidecar tokio task
that
is separate from the task that performs all the layer visiting and
spawning of IOs.
That tasks receives the IO futures via an unbounded mpsc channel and
drives them to completion inside a `FuturedUnordered`.
(The behavior from before this PR is available through
`IoConcurrency::Sequential`,
which awaits the IO futures in place, without "spawning" or "submitting"
them
anywhere.)
#### Alternatives Explored
A few words on the rationale behind having a sidecar *task* and what
alternatives were considered.
One option is to queue up all IO futures in a FuturesUnordered that is
polled
the first time when we `collect_pending_ios`.
Firstly, the IO futures are opaque, compiler-generated futures that need
to be polled at least once to submit their IO. "At least once" because
tokio-epoll-uring may not be able to submit the IO to the kernel on
first
poll right away.
Second, there are deadlocks if we don't drive the IO futures to
completion
independently of the spawning task.
The reason is that both the IO futures and the spawning task may hold
some
_and_ try to acquire _more_ shared limited resources.
For example, both spawning task and IO future may try to acquire
* a VirtualFile file descriptor cache slot async mutex (observed during
impl)
* a tokio-epoll-uring submission slot (observed during impl)
* a PageCache slot (currently this is not the case but we may move more
code into the IO futures in the future)
Another option is to spawn a short-lived `tokio::task` for each IO
future.
We implemented and benchmarked it during development, but found little
throughput improvement and moderate mean & tail latency degradation.
Concerns about pressure on the tokio scheduler made us discard this
variant.
The sidecar task could be obsoleted if the IOs were not arbitrary code
but a well-defined struct.
However,
1. the opaque futures approach taken in this PR allows leaving the
existing
code unchanged, which
2. allows us to implement the `IoConcurrency::Sequential` mode for
feature-gating
the change.
Once the new mode sidecar task implementation is rolled out everywhere,
and `::Sequential` removed, we can think about a descriptive submission
& completion interface.
The problems around deadlocks pointed out earlier will need to be solved
then.
For example, we could eliminate VirtualFile file descriptor cache and
tokio-epoll-uring slots.
The latter has been drafted in
https://github.com/neondatabase/tokio-epoll-uring/pull/63.
See the lengthy doc comment on `spawn_io()` for more details.
### Error handling
There are two error classes during reconstruct data retrieval:
* traversal errors: index lookup, move to next layer, and the like
* value read IO errors
A traversal error fails the entire get_vectored request, as before this
PR.
A value read error only fails that value.
In any case, we preserve the existing behavior that once
`get_vectored` returns, all IOs are done. Panics and failing
to poll `get_vectored` to completion will leave the IOs dangling,
which is safe but shouldn't happen, and so, a rate-limited
log statement will be emitted at warning level.
There is a doc comment on `collect_pending_ios` giving more code-level
details and rationale.
### Feature Gating
The new behavior is opt-in via pageserver config.
The `Sequential` mode is the default.
The only significant change in `Sequential` mode compared to before
this PR is the buffering of results in the `oneshot`s.
## Code-Level Changes
Prep work:
* Make `GateGuard` clonable.
Core Feature:
* Traversal code: track `will_init` in `BlobMeta` and source it from
the Delta/Image/InMemory layer index, instead of determining `will_init`
after we've read the value. This avoids having to read the value to
determine whether traversal can stop.
* Introduce `IoConcurrency` & its sidecar task.
* `IoConcurrency` is the clonable handle.
* It connects to the sidecar task via an `mpsc`.
* Plumb through `IoConcurrency` from high level code to the
individual layer implementations' `get_values_reconstruct_data`.
We piggy-back on the `ValuesReconstructState` for this.
* The sidecar task should be long-lived, so, `IoConcurrency` needs
to be rooted up "high" in the call stack.
* Roots as of this PR:
* `page_service`: outside of pagestream loop
* `create_image_layers`: when it is called
* `basebackup`(only auxfiles + replorigin + SLRU segments)
* Code with no roots that uses `IoConcurrency::sequential`
* any `Timeline::get` call
* `collect_keyspace` is a good example
* follow-up: https://github.com/neondatabase/neon/issues/10460
* `TimelineAdaptor` code used by the compaction simulator, unused in
practive
* `ingest_xlog_dbase_create`
* Transform Delta/Image/InMemoryLayer to
* do their values IO in a distinct `async {}` block
* extend the residence of the Delta/Image layer until the IO is done
* buffer their results in a `oneshot` channel instead of straight
in `ValuesReconstructState`
* the `oneshot` channel is wrapped in `OnDiskValueIo` /
`OnDiskValueIoWaiter`
types that aid in expressiveness and are used to keep track of
in-flight IOs so we can print warnings if we leave them dangling.
* Change `ValuesReconstructState` to hold the receiving end of the
`oneshot` channel aka `OnDiskValueIoWaiter`.
* Change `get_vectored_impl` to `collect_pending_ios` and issue walredo
concurrently, in a `FuturesUnordered`.
Testing / Benchmarking:
* Support queue-depth in pagebench for manual benchmarkinng.
* Add test suite support for setting concurrency mode ps config
field via a) an env var and b) via NeonEnvBuilder.
* Hacky helper to have sidecar-based IoConcurrency in tests.
This will be cleaned up later.
More benchmarking will happen post-merge in nightly benchmarks, plus in
staging/pre-prod.
Some intermediate helpers for manual benchmarking have been preserved in
https://github.com/neondatabase/neon/pull/10466 and will be landed in
later PRs.
(L0 layer stack generator!)
Drive-By:
* test suite actually didn't enable batching by default because
`config.compatibility_neon_binpath` is always Truthy in our CI
environment
=> https://neondb.slack.com/archives/C059ZC138NR/p1737490501941309
* initial logical size calculation wasn't always polled to completion,
which was
surfaced through the added WARN logs emitted when dropping a
`ValuesReconstructState` that still has inflight IOs.
* remove the timing histograms
`pageserver_getpage_get_reconstruct_data_seconds`
and `pageserver_getpage_reconstruct_seconds` because with planning,
value read
IO, and walredo happening concurrently, one can no longer attribute
latency
to any one of them; we'll revisit this when Vlad's work on
tracing/sampling
through RequestContext lands.
* remove code related to `get_cached_lsn()`.
The logic around this has been dead at runtime for a long time,
ever since the removal of the materialized page cache in #8105.
## Testing
Unit tests use the sidecar task by default and run both modes in CI.
Python regression tests and benchmarks also use the sidecar task by
default.
We'll test more in staging and possibly preprod.
# Future Work
Please refer to the parent epic for the full plan.
The next step will be to fold the plumbing of IoConcurrency
into RequestContext so that the function signatures get cleaned up.
Once `Sequential` isn't used anymore, we can take the next
big leap which is replacing the opaque IOs with structs
that have well-defined semantics.
---------
Co-authored-by: Christian Schwarz <christian@neon.tech>
## Problem
part of https://github.com/neondatabase/neon/issues/9114
The automatic trigger is already implemented at
https://github.com/neondatabase/neon/pull/10221 but I need to write some
tests and finish my experiments in staging before I can merge it with
confidence. Given that I have some other patches that will modify the
config items, I'd like to get the config items merged first to reduce
conflicts.
## Summary of changes
* add `l2_lsn` to index_part.json -- below that LSN, data have been
processed by gc-compaction
* add a set of gc-compaction auto trigger control items into the config
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
# Refs
- fixes https://github.com/neondatabase/neon/issues/10309
- fixup of batching design, first introduced in
https://github.com/neondatabase/neon/pull/9851
- refinement of https://github.com/neondatabase/neon/pull/8339
# Problem
`Tenant::shutdown` was occasionally taking many minutes (sometimes up to
20) in staging and prod if the
`page_service_pipelining.mode="concurrent-futures"` is enabled.
# Symptoms
The issue happens during shard migration between pageservers.
There is page_service unavailability and hence effectively downtime for
customers in the following case:
1. The source (state `AttachedStale`) gets stuck in `Tenant::shutdown`,
waiting for the gate to close.
2. Cplane/Storcon decides to transition the target `AttachedMulti` to
`AttachedSingle`.
3. That transition comes with a bump of the generation number, causing
the `PUT .../location_config` endpoint to do a full `Tenant::shutdown` /
`Tenant::attach` cycle for the target location.
4. That `Tenant::shutdown` on the target gets stuck, waiting for the
gate to close.
5. Eventually the gate closes (`close completed`), correlating with a
`page_service` connection handler logging that it's exiting because of a
network error (`Connection reset by peer` or `Broken pipe`).
While in (4):
- `Tenant::shutdown` is stuck waiting for all `Timeline::shutdown` calls
to complete.
So, really, this is a `Timeline::shutdown` bug.
- retries from Cplane/Storcon to complete above state transitions, fail
with errors related to the tenant mgr slot being in state
`TenantSlot::InProgress`, the tenant state being
`TenantState::Stopping`, and the timelines being in
`TimelineState::Stopping`, and the `Timeline::cancel` being cancelled.
- Existing (and/or new?) page_service connections log errors `error
reading relation or page version: Not found: Timed out waiting 30s for
tenant active state. Latest state: None`
# Root-Cause
After a lengthy investigation ([internal
write-up](https://www.notion.so/neondatabase/2025-01-09-batching-deadlock-Slow-Log-Analysis-in-Staging-176f189e00478050bc21c1a072157ca4?pvs=4))
I arrived at the following root cause.
The `spsc_fold` channel (`batch_tx`/`batch_rx`) that connects the
Batcher and Executor stages of the pipelined mode was storing a `Handle`
and thus `GateGuard` of the Timeline that was not shutting down.
The design assumption with pipelining was that this would always be a
short transient state.
However, that was incorrect: the Executor was stuck on writing/flushing
an earlier response into the connection to the client, i.e., socket
write being slow because of TCP backpressure.
The probable scenario of how we end up in that case:
1. Compute backend process sends a continuous stream of getpage prefetch
requests into the connection, but never reads the responses (why this
happens: see Appendix section).
2. Batch N is processed by Batcher and Executor, up to the point where
Executor starts flushing the response.
3. Batch N+1 is procssed by Batcher and queued in the `spsc_fold`.
4. Executor is still waiting for batch N flush to finish.
5. Batcher eventually hits the `TimeoutReader` error (10min).
From here on it waits on the
`spsc_fold.send(Err(QueryError(TimeoutReader_error)))`
which will never finish because the batch already inside the `spsc_fold`
is not
being read by the Executor, because the Executor is still stuck in the
flush.
(This state is not observable at our default `info` log level)
6. Eventually, Compute backend process is killed (`close()` on the
socket) or Compute as a whole gets killed (probably no clean TCP
shutdown happening in that case).
7. Eventually, Pageserver TCP stack learns about (6) through RST packets
and the Executor's flush() call fails with an error.
8. The Executor exits, dropping `cancel_batcher` and its end of the
spsc_fold.
This wakes Batcher, causing the `spsc_fold.send` to fail.
Batcher exits.
The pipeline shuts down as intended.
We return from `process_query` and log the `Connection reset by peer` or
`Broken pipe` error.
The following diagram visualizes the wait-for graph at (5)
```mermaid
flowchart TD
Batcher --spsc_fold.send(TimeoutReader_error)--> Executor
Executor --flush batch N responses--> socket.write_end
socket.write_end --wait for TCP window to move forward--> Compute
```
# Analysis
By holding the GateGuard inside the `spsc_fold` open, the pipelining
implementation
violated the principle established in
(https://github.com/neondatabase/neon/pull/8339).
That is, that `Handle`s must only be held across an await point if that
await point
is sensitive to the `<Handle as Deref<Target=Timeline>>::cancel` token.
In this case, we were holding the Handle inside the `spsc_fold` while
awaiting the
`pgb_writer.flush()` future.
One may jump to the conclusion that we should simply peek into the
spsc_fold to get
that Timeline cancel token and be sensitive to it during flush, then.
But that violates another principle of the design from
https://github.com/neondatabase/neon/pull/8339.
That is, that the page_service connection lifecycle and the Timeline
lifecycles must be completely decoupled.
Tt must be possible to shut down one shard without shutting down the
page_service connection, because on that single connection we might be
serving other shards attached to this pageserver.
(The current compute client opens separate connections per shard, but,
there are plans to change that.)
# Solution
This PR adds a `handle::WeakHandle` struct that does _not_ hold the
timeline gate open.
It must be `upgrade()`d to get a `handle::Handle`.
That `handle::Handle` _does_ hold the timeline gate open.
The batch queued inside the `spsc_fold` only holds a `WeakHandle`.
We only upgrade it while calling into the various `handle_` methods,
i.e., while interacting with the `Timeline` via `<Handle as
Deref<Target=Timeline>>`.
All that code has always been required to be (and is!) sensitive to
`Timeline::cancel`, and therefore we're guaranteed to bail from it
quickly when `Timeline::shutdown` starts.
We will drop the `Handle` immediately, before we start
`pgb_writer.flush()`ing the responses.
Thereby letting go of our hold on the `GateGuard`, allowing the timeline
shutdown to complete while the page_service handler remains intact.
# Code Changes
* Reproducer & Regression Test
* Developed and proven to reproduce the issue in
https://github.com/neondatabase/neon/pull/10399
* Add a `Test` message to the pagestream protocol (`cfg(feature =
"testing")`).
* Drive-by minimal improvement to the parsing code, we now have a
`PagestreamFeMessageTag`.
* Refactor `pageserver/client` to allow sending and receiving
`page_service` requests independently.
* Add a Rust helper binary to produce situation (4) from above
* Rationale: (4) and (5) are the same bug class, we're holding a gate
open while `flush()`ing.
* Add a Python regression test that uses the helper binary to
demonstrate the problem.
* Fix
* Introduce and use `WeakHandle` as explained earlier.
* Replace the `shut_down` atomic with two enum states for `HandleInner`,
wrapped in a `Mutex`.
* To make `WeakHandle::upgrade()` and `Handle::downgrade()`
cache-efficient:
* Wrap the `Types::Timeline` in an `Arc`
* Wrap the `GateGuard` in an `Arc`
* The separate `Arc`s enable uncontended cloning of the timeline
reference in `upgrade()` and `downgrade()`.
If instead we were `Arc<Timeline>::clone`, different connection handlers
would be hitting the same cache line on every upgrade()/downgrade(),
causing contention.
* Please read the udpated module-level comment in `mod handle`
module-level comment for details.
# Testing & Performance
The reproducer test that failed before the changes now passes, and
obviously other tests are passing as well.
We'll do more testing in staging, where the issue happens every ~4h if
chaos migrations are enabled in storcon.
Existing perf testing will be sufficient, no perf degradation is
expected.
It's a few more alloctations due to the added Arc's, but, they're low
frequency.
# Appendix: Why Compute Sometimes Doesn't Read Responses
Remember, the whole problem surfaced because flush() was slow because
Compute was not reading responses. Why is that?
In short, the way the compute works, it only advances the page_service
protocol processing when it has an interest in data, i.e., when the
pagestore smgr is called to return pages.
Thus, if compute issues a bunch of requests as part of prefetch but then
it turns out it can service the query without reading those pages, it
may very well happen that these messages stay in the TCP until the next
smgr read happens, either in that session, or possibly in another
session.
If there’s too many unread responses in the TCP, the pageserver kernel
is going to backpressure into userspace, resulting in our stuck flush().
All of this stems from the way vanilla Postgres does prefetching and
"async IO":
it issues `fadvise()` to make the kernel do the IO in the background,
buffering results in the kernel page cache.
It then consumes the results through synchronous `read()` system calls,
which hopefully will be fast because of the `fadvise()`.
If it turns out that some / all of the prefetch results are not needed,
Postgres will not be issuing those `read()` system calls.
The kernel will eventually react to that by reusing page cache pages
that hold completed prefetched data.
Uncompleted prefetch requests may or may not be processed -- it's up to
the kernel.
In Neon, the smgr + Pageserver together take on the role of the kernel
in above paragraphs.
In the current implementation, all prefetches are sent as GetPage
requests to Pageserver.
The responses are only processed in the places where vanilla Postgres
would do the synchronous `read()` system call.
If we never get to that, the responses are queued inside the TCP
connection, which, once buffers run full, will backpressure into
Pageserver's sending code, i.e., the `pgb_writer.flush()` that was the
root cause of the problems we're fixing in this PR.