The final patch for partial compaction, part of
https://github.com/neondatabase/neon/issues/9114, close
https://github.com/neondatabase/neon/issues/8921 (note that we didn't
implement parallel compaction or compaction scheduler for partial
compaction -- currently this needs to be scheduled by using a Python
script to split the keyspace, and in the future, automatically split
based on the key partitioning when the pageserver wants to trigger a
gc-compaction)
## Summary of changes
* Update the layer selection algorithm to use the same selection as full
compaction (everything intersect/below gc horizon)
* Update the layer selection algorithm to also generate a list of delta
layers that need to be rewritten
* Add the logic to rewrite delta layers and add them back to the layer
map
* Update test case to do partial compaction on deltas
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
Removes some unnecessary initdb arguments, and fixes Neon for MacOS
since it doesn't seem to ship a C.UTF-8 locale.
Signed-off-by: Tristan Partin <tristan@neon.tech>
Perf benchmarks produce a lot of layers.
## Summary of changes
Bumping the threshold and ignore the warning.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
We wish to stop using admin tokens in the infra repo, but step down
requests use the admin token.
## Summary of Changes
Introduce a new "ControllerPeer" scope and use it for step-down requests.
It is possible at the point we shutdown the timeline, there are
still layer files we did not upload.
## Summary of changes
* If the queue is not empty, avoid offloading.
* Shutdown the timeline gracefully using the flush mode to
ensure all local files are uploaded before deleting the timeline
directory.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
We saw pageserver OOMs
https://github.com/neondatabase/cloud/issues/19715 for tenants doing
large writes. Add log lines around in-memory layers to hopefully collect
some info during my on-call shift next week.
## Summary of changes
* Estimate in-memory size of an in-mem layer.
* Print frozen layer number if there are too many layers accumulated in
memory.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
Right now, our environments create databases with the C locale, which is
really unfortunate for users who have data stored in other languages
that they want to analyze. For instance, show_trgm on Hebrew text
currently doesn't work in staging or production.
I don't envision this being the final solution. I think this is just a
way to set a known value so the pageserver doesn't use its parent
environment. The final solution to me is exposing initdb parameters to
users in the console. Then they could use a different locale or encoding
if they so chose.
Signed-off-by: Tristan Partin <tristan@neon.tech>
In INC-317
https://neondb.slack.com/archives/C033RQ5SPDH/p1730815677932209, we saw
an interesting series of operations that would remove valid layer files
existing in the layer map.
* Timeline A starts compaction and generates an image layer Z but not
uploading it yet.
* Timeline B/C starts ancestor detaching (which should not affect
timeline A)
* The tenant gets restarted as part of the ancestor detaching process,
without increasing the generation number.
* Timeline A reloads, discovering the layer Z is a future layer, and
schedules a **deletion into the deletion queue**. This means that the
file will be deleted any time in the future.
* Timeline A starts compaction and generates layer Z again, adding it to
the layer map. Note that because we don't bump generation number during
ancestor detach, it has the same filename + generation number as the
original Z.
* Timeline A deletes layer Z from s3 + disk, and now we have a dangling
reference in the layer map, blocking all
compaction/logical_size_calculation process.
## Summary of changes
* We wait until all layers to be uploaded before shutting down the
tenants in `Flush` mode.
* Ancestor detach restarts now use this mode.
* Ancestor detach also waits for remote queue completion before starting
the detaching process.
* The patch ensures that we don't have any future image layer (or
something similar) after restart, but not fixing the underlying problem
around generation numbers.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
In test environments, the `syncfs` that the pageserver does on startup
can take a long time, as other tests running concurrently might have
many gigabytes of dirty pages.
## Summary of changes
- Add a `no_sync` option to the pageserver's config.
- Skip syncfs on startup if this is set
- A subsequent PR (https://github.com/neondatabase/neon/pull/9678) will
enable this by default in tests. We need to wait until after the next
release to avoid breaking compat tests, which would fail if we set
no_sync & use an old pageserver binary.
Q: Why is this a different mechanism than safekeeper, which as a
--no-sync CLI?
A: Because the way we manage pageservers in neon_local depends on the
pageserver.toml containing the full configuration, whereas safekeepers
have a config file which is neon-local-specific and can drive a CLI
flag.
Q: Why is the option no_sync rather than sync?
A: For boolean configs with a dangerous value, it's preferable to make
"false" the safe option, so that any downstream future config tooling
that might have a "booleans are false by default" behavior (e.g. golang
structs) is safe by default.
Q: Why only skip the syncfs, and not all fsyncs?
A: Skipping all fsyncs would require more code changes, and the most
acute problem isn't fsyncs themselves (these just slow down a running
test), it's the syncfs (which makes a pageserver startup slow as a
result of _other_ tests)
ref https://github.com/neondatabase/neon/issues/9441
The metrics from LR publisher testing project: ~300KB aux key deltas per
256MB files. Therefore, I think we can do compaction more aggressively
as these deltas are small and compaction can reduce layer download
latency. We also have a read path perf fix
https://github.com/neondatabase/neon/pull/9631 but I'd still combine the
read path fix with the reduce of the compaction threshold.
## Summary of changes
* reduce metadata compaction threshold
* use num of L1 delta layers as an indicator for metadata compaction
* dump more logs
Signed-off-by: Alex Chi Z <chi@neon.tech>
We need to use the shard associated with the layer file, not the shard
associated with our current tenant shard ID.
Due to shard splits, the shard IDs can refer to older files.
close https://github.com/neondatabase/neon/issues/9667
In https://github.com/neondatabase/neon/issues/9441, the tenant has a
lot of aux keys spread in multiple aux files. The perf tool shows that a
significant amount of time is spent on remove_overlapping_keys. For
sparse keyspaces, we don't need to report missing key errors anyways,
and it's very likely that we will need to read all layers intersecting
with the key range. Therefore, this patch adds a new fast path for
sparse keyspace reads that we do not track `unmapped_keyspace` in a
fine-grained way. We only modify it when we find an image layer.
In debug mode, it was ~5min to read the aux files for a dump of the
tenant, and now it's only 8s, that's a 60x speedup.
## Summary of changes
* Do not add sparse keys into `keys_done` so that remove_overlapping
does nothing.
* Allow `ValueReconstructSituation::Complete` to be updated again in
`ValuesReconstructState::update_key` for sparse keyspaces.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
https://github.com/neondatabase/neon/pull/9524 split the decoding and
interpretation step from ingestion.
The output of the first phase is a `wal_decoder::models::InterpretedWalRecord`.
Before this patch set that struct contained a list of `Value` instances.
We wish to lift the decoding and interpretation step to the safekeeper,
but it would be nice if the safekeeper gave us a batch containing the raw data instead of actual values.
## Summary of changes
Main goal here is to make `InterpretedWalRecord` hold a raw buffer which
contains pre-serialized Values.
For this we do:
1. Add a `SerializedValueBatch` type. This is `inmemory_layer::SerializedBatch` with some
extra functionality for extension, observing values for shard 0 and tests.
2. Replace `inmemory_layer::SerializedBatch` with `SerializedValueBatch`
3. Make `DatadirModification` maintain a `SerializedValueBatch`.
### `DatadirModification` changes
`DatadirModification` now maintains a `SerializedValueBatch` and extends
it as new WAL records come in (to avoid flushing to disk on every
record).
In turn, this cascaded into a number of modifications to
`DatadirModification`:
1. Replace `pending_data_pages` and `pending_zero_data_pages` with `pending_data_batch`.
2. Removal of `pending_zero_data_pages` and its cousin `on_wal_record_end`
3. Rename `pending_bytes` to `pending_metadata_bytes` since this is what it tracks now.
4. Adapting of various utility methods like `len`, `approx_pending_bytes` and `has_dirty_data_pages`.
Removal of `pending_zero_data_pages` and the optimisation associated
with it ((1) and (2)) deserves more detail.
Previously all zero data pages went through `pending_zero_data_pages`.
We wrote zero data pages when filling gaps caused by relation extension
(case A) and when handling special wal records (case B). If it happened
that the same WAL record contained a non zero write for an entry in
`pending_zero_data_pages` we skipped the zero write.
Case A: We handle this differently now. When ingesting the
`SerialiezdValueBatch` associated with one PG WAL record, we identify the gaps and fill the
them in one go. Essentially, we move from a per key process (gaps were filled after each
new key), and replace it with a per record process. Hence, the optimisation is not
required anymore.
Case B: When the handling of a special record needs to zero out a key,
it just adds that to the current batch. I inspected the code, and I
don't think the optimisation kicked in here.
## Problem
Tenant operations may return `409 Conflict` if the tenant is shutting
down. This status code is not retried by the control plane, causing
user-facing errors during pageserver restarts. Operations should instead
return `503 Service Unavailable`, which may be retried for idempotent
operations.
## Summary of changes
Convert
`GetActiveTenantError::WillNotBecomeActive(TenantState::Stopping)` to
`ApiError::ShuttingDown` rather than `ApiError::Conflict`. This error is
returned by `Tenant::wait_to_become_active` in most (all?)
tenant/timeline-related HTTP routes.
## Problem
In https://github.com/neondatabase/neon/pull/9589, timeline offload code
is modified to return an explicit error type rather than propagating
anyhow::Error. One of the 'Other' cases there is I/O errors from local
timeline deletion, which shouldn't need to exist, because our policy is
not to try and continue running if the local disk gives us errors.
## Summary of changes
- Make `delete_local_timeline_directory` and use `.fatal_err(` on I/O
errors
---------
Co-authored-by: Erik Grinaker <erik@neon.tech>
## Problem
https://neondb.slack.com/archives/C04DGM6SMTM/p1727872045252899
See https://github.com/neondatabase/neon/issues/9240
## Summary of changes
Add `!page_is_new` check before assigning page lsn.
## Checklist before requesting a review
- [ ] I have performed a self-review of my code.
- [ ] If it is a core feature, I have added thorough tests.
- [ ] Do we need to implement analytics? if so did you add the relevant
metrics to the dashboard?
- [ ] If this PR requires public announcement, mark it with
/release-notes label and add several sentences in this section.
## Checklist before merging
- [ ] Do not forget to reformat commit message to not include the above
checklist
---------
Co-authored-by: Konstantin Knizhnik <knizhnik@neon.tech>
## Problem
The final part of https://github.com/neondatabase/neon/issues/9543 will
be a chaos test that creates/deletes/archives/offloads timelines while
restarting pageservers and migrating tenants. Developing that test
showed up a few places where we log errors during normal shutdown.
## Summary of changes
- UninitializedTimeline's drop should log at info severity: this is a
normal code path when some part of timeline creation encounters a
cancellation `?` path.
- When offloading and finding a `RemoteTimelineClient` in a
non-initialized state, this is not an error and should not be logged as
such.
- The `offload_timeline` function returned an anyhow error, so callers
couldn't gracefully pick out cancellation errors from real errors:
update this to have a structured error type and use it throughout.
## Problem
Decoding and ingestion are still coupled in `pageserver::WalIngest`.
## Summary of changes
A new type is added to `wal_decoder::models`, InterpretedWalRecord. This
type contains everything that the pageserver requires in order to ingest
a WAL record. The highlights are the `metadata_record` which is an
optional special record type to be handled and `blocks` which stores
key, value pairs to be persisted to storage.
This type is produced by
`wal_decoder::models::InterpretedWalRecord::from_bytes` from a raw PG
wal record.
The rest of this commit separates decoding and interpretation of the PG
WAL record from its application in `WalIngest::ingest_record`.
Related: https://github.com/neondatabase/neon/issues/9335
Epic: https://github.com/neondatabase/neon/issues/9329
If we delete a timeline that has childen, those children will have their
data corrupted. Therefore, extend the already existing safety check to
offloaded timelines as well.
Part of #8088
In https://github.com/neondatabase/neon/issues/9032, I would like to
eventually add a `generation` field to the consumption metrics cache.
The current encoding is not backward compatible and it is hard to add
another field into the cache. Therefore, this patch refactors the format
to store "field -> value", and it's easier to maintain backward/forward
compatibility with this new format.
## Summary of changes
* Add `NewRawMetric` as the new format.
* Add upgrade path. When opening the disk cache, the codepath first
inspects the `version` field, and decide how to decode.
* Refactor metrics generation code and tests.
* Add tests on upgrade / compatibility with the old format.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
Constructing a remote client is no big deal. Yes, it means an extra
download from S3 but it's not that expensive. This simplifies code paths
and scenarios to test. This unifies timelines that have been recently
offloaded with timelines that have been offloaded in an earlier
invocation of the process.
Part of #8088
Disallow a request for timeline ancestor detach if either the to be
detached timeline, or any of the to be reparented timelines are
offloaded or archived.
In theory we could support timelines that are archived but not
offloaded, but archived timelines are at the risk of being offloaded, so
we treat them like offloaded timelines. As for offloaded timelines, any
code to "support" them would amount to unoffloading them, at which point
we can just demand to have the timelines be unarchived.
Part of #8088
## Problem
When tenant manifest objects are written without a generation suffix,
concurrently attached pageservers may stamp on each others writes of the
manifest and cause undefined behavior.
Closes: #9543
## Summary of changes
- Use download_generation_object helper when reading manifests, to
search for the most recent generation
- Use Tenant::generation as the generation suffix when writing
manifests.
This patch contains various improvements for the pagectl tool.
## Summary of changes
* Rewrite layer name parsing: LayerName now supports all variants we use
now.
* Drop pagectl's own layer parsing function, use LayerName in the
pageserver crate.
* Support image layer dumping in the layer dump command using
ImageLayer::dump, drop the original implementation.
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
See https://github.com/neondatabase/neon/pull/9458
This PR separates PS related changes in #9458 from compute_ctl changes
to enforce that PS is deployed before compute.
## Summary of changes
This PR adds handlings of `--replica` parameters of backebackup to page
server.
## Checklist before requesting a review
- [ ] I have performed a self-review of my code.
- [ ] If it is a core feature, I have added thorough tests.
- [ ] Do we need to implement analytics? if so did you add the relevant
metrics to the dashboard?
- [ ] If this PR requires public announcement, mark it with
/release-notes label and add several sentences in this section.
## Checklist before merging
- [ ] Do not forget to reformat commit message to not include the above
checklist
---------
Co-authored-by: Konstantin Knizhnik <knizhnik@neon.tech>
## Problem
Uploads of the tenant manifest could race between different tasks,
resulting in unexpected results in remote storage.
Closes: https://github.com/neondatabase/neon/issues/9556
## Summary of changes
- Create a central function for uploads that takes a tokio::sync::Mutex
- Store the latest upload in that Mutex, so that when there is lots of
concurrency (e.g. archive 20 timelines at once) we can coalesce their
manifest writes somewhat.
## Problem
Indices used to be the only kind of object where we had to search across
generations to find the most recent one. As of
https://github.com/neondatabase/neon/issues/9543, manifests will need
the same treatment.
## Summary of changes
- Refactor download_index_part to a generic download_generation_object
function, which will be usable for downloading manifest objects as well.
As pointed out in
https://github.com/neondatabase/neon/pull/9489#discussion_r1814699683 ,
we currently didn't support deletion for offloaded timelines after the
timeline has been loaded from the manifest instead of having been
offloaded.
This was because the upload queue hasn't been initialized yet. This PR
thus initializes the timeline and shuts it down immediately.
Part of #8088
## Problem
We wish to have high level WAL decoding logic in `wal_decoder::decoder`
module.
## Summary of Changes
For this we need the `Value` and `NeonWalRecord` types accessible there, so:
1. Move `Value` and `NeonWalRecord` to `pageserver::value` and
`pageserver::record` respectively.
2. Get rid of `pageserver::repository` (follow up from (1))
3. Move PG specific WAL record types to `postgres_ffi::walrecord`. In
theory they could live in `wal_decoder`, but it would create a circular
dependency between `wal_decoder` and `postgres_ffi`. Long term it makes
sense for those types to be PG version specific, so that will work out nicely.
4. Move higher level WAL record types (to be ingested by pageserver)
into `wal_decoder::models`
Related: https://github.com/neondatabase/neon/issues/9335
Epic: https://github.com/neondatabase/neon/issues/9329
Currently, all callers of `unoffload_timeline` ensure that the tenant
the unoffload operation is called on is active. We rely on it being
active as we activate the timeline below and don't want to race with the
activation code of the tenant (in the worst case, activating a timeline
twice).
Therefore, add this assertion.
Part of #8088
## Problem
Part of https://github.com/neondatabase/neon/issues/8623
## Summary of changes
Removed all aux-v1 config processing code. Note that we persisted it
into the index part file, so we cannot really remove the field from
index part. I also kept the config item within the tenant config, but we
will not read it any more.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
In complement to
https://github.com/neondatabase/tokio-epoll-uring/pull/56.
## Problem
We want to make tokio-epoll-uring slots waiters queue depth observable
via Prometheus.
## Summary of changes
- Add `pageserver_tokio_epoll_uring_slots_submission_queue_depth`
metrics as a `Histogram`.
- Each thread-local tokio-epoll-uring system is given a `LocalHistogram`
to observe the metrics.
- Keep a list of `Arc<ThreadLocalMetrics>` used on-demand to flush data
to the shared histogram.
- Extend `Collector::collect` to report
`pageserver_tokio_epoll_uring_slots_submission_queue_depth`.
Signed-off-by: Yuchen Liang <yuchen@neon.tech>
Co-authored-by: Christian Schwarz <christian@neon.tech>
This PR does two things:
1. Obtain a `TimelineCreateGuard` object in `unoffload_timeline`. This
prevents two unoffload tasks from racing with each other. While they
already obtain locks for `timelines` and `offloaded_timelines`, they
aren't sufficient, as we have already constructed an entire timeline at
that point. We shouldn't ever have two `Timeline` objects in the same
process at the same time.
2. don't allow timeline creations for timelines that have been
offloaded. Obviously they already exist, so we should not allow
creation. the previous logic only looked at the timelines list.
Part of #8088
## Problem
We have some known N^2 behaviors when it comes to large relation counts,
due to the monolithic encoding and full rewrites of of RelDirectory each
time a relation is added. Ordinarily our backpressure mechanisms give
"slow but steady" performance when creating/dropping/truncating
relations. However, in the case of a transaction abort, it is possible
for a single WAL record to drop an unbounded number of relations. The
results in an unavailable compute, as when it sends one of these
records, it can stall the pageserver's ingest for many minutes, even
though the compute only sent a small amount of WAL.
Closes https://github.com/neondatabase/neon/issues/9505
## Summary of changes
- Rewrite relation-dropping code to do one read/modify/write cycle of
RelDirectory, instead of doing it separately for each relation in a
loop.
- Add a test for the bug scenario encountered:
`test_tx_abort_with_many_relations`
The test has ~40s runtime on my workstation. About 1 second of that is
the part where we wait for ingest to catch up after a rollback, the rest
is the slowness of creating and truncating a large number of relations.
---------
Co-authored-by: Heikki Linnakangas <heikki@neon.tech>
# Context
In the PGDATA import code
(https://github.com/neondatabase/neon/pull/9218) I add a third way to
create timelines, namely, by importing from a copy of a vanilla PGDATA
directory in object storage.
For idempotency, I'm using the PGDATA object storage location
specification, which is stored in the IndexPart for the entire lifespan
of the timeline. When loading the timeline from remote storage, that
value gets stored inside `struct Timeline` and timeline creation
compares the creation argument with that value to determine idempotency
of the request.
# Changes
This PR refactors the existing idempotency handling of Timeline
bootstrap and branching such that we simply compare the
`CreateTimelineIdempotency` struct, using the derive-generated
`PartialEq` implementation.
Also, by spelling idempotency out in the type names, I find it adds a
lot of clarity.
The pathway to idempotency via requester-provided idempotency key also
becomes very straight-forward, if we ever want to do this in the future.
# Refs
* platform context: https://github.com/neondatabase/neon/pull/9218
* product context: https://github.com/neondatabase/cloud/issues/17507
* stacks on top of https://github.com/neondatabase/neon/pull/9366
This PR adds a pageserver mgmt API to scan a layer file for disposable
keys.
It hooks it up to the sharding compaction test, demonstrating that we're
not filtering out all disposable keys.
This is extracted from PGDATA import
(https://github.com/neondatabase/neon/pull/9218)
where I do the filtering of layer files based on `is_key_disposable`.
Before, we didn't copy over the `index-part.json` of offloaded timelines
to the new shard's location, resulting in the new shard not knowing the
timeline even exists.
In #9444, we copy over the manifest, but we also need to do this for
`index-part.json`.
As the operations to do are mostly the same between offloaded and
non-offloaded timelines, we can iterate over all of them in the same
loop, after the introduction of a `TimelineOrOffloadedArcRef` type to
generalize over the two cases. This is analogous to the deletion code
added in #8907.
The added test also ensures that the sharded archival config endpoint
works, something that has not yet been ensured by tests.
Part of #8088
## Problem
https://github.com/neondatabase/neon/pull/9492 added a metric to track
the total count of block gaps filled on rel extend. More context is
needed to understand when this happens. The current theory is that it
may only happen on pg 14 and pg 15 since they do not WAL log relation extends.
## Summary of Changes
A rate limited log is added.
# Problem
Timeline creation can either be bootstrap or branch.
The distinction is made based on whether the `ancestor_*` fields are
present or not.
In the PGDATA import code
(https://github.com/neondatabase/neon/pull/9218), I add a third variant
to timeline creation.
# Solution
The above pushed me to refactor the code in Pageserver to distinguish
the different creation requests through enum variants.
There is no externally observable effect from this change.
On the implementation level, a notable change is that the acquisition of
the `TimelineCreationGuard` happens later than before. This is necessary
so that we have everything in place to construct the
`CreateTimelineIdempotency`. Notably, this moves the acquisition of the
creation guard _after_ the acquisition of the `gc_cs` lock in the case
of branching. This might appear as if we're at risk of holding `gc_cs`
longer than before this PR, but, even before this PR, we were holding
`gc_cs` until after the `wait_completion()` that makes the timeline
creation durable in S3 returns. I don't see any deadlock risk with
reversing the lock acquisition order.
As a drive-by change, I found that the `create_timeline()` function in
`neon_local` is unused, so I removed it.
# Refs
* platform context: https://github.com/neondatabase/neon/pull/9218
* product context: https://github.com/neondatabase/cloud/issues/17507
* next PR stacked atop this one:
https://github.com/neondatabase/neon/pull/9501
## Problem
WAL ingest couples decoding of special records with their handling
(updates to the storage engine mostly).
This is a roadblock for our plan to move WAL filtering (and implicitly
decoding) to safekeepers since they cannot
do writes to the storage engine.
## Summary of changes
This PR decouples the decoding of the special WAL records from their
application. The changes are done in place
and I've done my best to refrain from refactorings and attempted to
preserve the original code as much as possible.
Related: https://github.com/neondatabase/neon/issues/9335
Epic: https://github.com/neondatabase/neon/issues/9329
part of https://github.com/neondatabase/neon/issues/9114,
https://github.com/neondatabase/neon/issues/8836,
https://github.com/neondatabase/neon/issues/8362
The split layer writer code can be used in a more general way: the
caller puts unfinished writers into the batch layer writer and let batch
layer writer to ensure the atomicity of the layer produces.
## Summary of changes
* Add batch layer writer, which atomically finishes the layers.
`BatchLayerWriter::finish` is simply a copy-paste from previous split
layer writers.
* Refactor split writers to use the batch layer writer.
* The current split writer tests cover all code path of batch layer
writer.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
Filling the gap in with zeroes is annoying for sharded ingest. We are
not sure it even happens in reality.
## Summary of Changes
Add one global counter which tracks how many such gap blocks we filled
on relation extends. We can add more metrics once we understand the
scope.