Commit Graph

3068 Commits

Author SHA1 Message Date
Erik Grinaker
f4150614d0 pageserver: don't pass config to PageHandler (#11973)
## Problem

The gRPC page service API will require decoupling the `PageHandler` from
the libpq protocol implementation. As preparation for this, avoid
passing in the entire server config to `PageHandler`, and instead
explicitly pass in the relevant fields.

Touches https://github.com/neondatabase/neon/issues/11728.

## Summary of changes

* Change `PageHandler` to take a `GetVectoredConcurrentIo` instead of
the entire config.
* Change `IoConcurrency::spawn_from_conf` to take a
`GetVectoredConcurrentIo`.
2025-05-19 15:47:40 +00:00
Erik Grinaker
38dbc5f67f pageserver/page_api: add binary Protobuf descriptor (#11968)
## Problem

A binary Protobuf schema descriptor can be used to expose an API
reflection service, which in turn allows convenient usage of e.g.
`grpcurl` against the gRPC server.

Touches #11728.

## Summary of changes

* Generate a binary schema descriptor as
`pageserver_page_api::proto::FILE_DESCRIPTOR_SET`.
* Opportunistically rename the Protobuf package from `page_service` to
`page_api`.
2025-05-19 11:17:45 +00:00
Erik Grinaker
cdb6479c8a pageserver: add gRPC page service schema (#11815)
## Problem

For the [communicator
project](https://github.com/neondatabase/company_projects/issues/352),
we want to move to gRPC for the page service protocol.

Touches #11728.

## Summary of changes

This patch adds an experimental gRPC Protobuf schema for the page
service. It is equivalent to the current page service, but with several
improvements, e.g.:

* Connection multiplexing.
* Reduced head-of-line blocking.
* Client-side batching.
* Explicit tenant shard routing.
* GetPage request classification (normal vs. prefetch).
* Explicit rate limiting ("slow down" response status).

The API is exposed as a new `pageserver/page_api` package. This is
separate from the `pageserver_api` package to reduce the dependency
footprint for the communicator. The longer-term plan is to also split
out e.g. the WAL ingestion service to a separate gRPC package, e.g.
`pageserver/wal_api`.

Subsequent PRs will: add Rust domain types for the Protobuf types,
expose a gRPC server, and implement the page service.

Preliminary prototype benchmarks of this gRPC API is within 10% of
baseline libpq performance. We'll do further benchmarking and
optimization as the implementation lands in `main` and is deployed to
staging.
2025-05-19 09:03:06 +00:00
Trung Dinh
e963129678 pagesteam_handle_batched_message -> pagestream_handle_batched_message (#11916)
## Problem
Found a typo in code.

## Summary of changes

Co-authored-by: Trung Dinh <tdinh@roblox.com>
Co-authored-by: Erik Grinaker <erik@neon.tech>
2025-05-17 22:30:29 +00:00
Heikki Linnakangas
55f91cf10b Update 'nix' package (#11948)
There were some incompatible changes. Most churn was from switching from
the now-deprecated fcntl:flock() function to
fcntl::Flock::lock(). The new function returns a guard object, while
with the old function, the lock was associated directly with the file
descriptor.

It's good to stay up-to-date in general, but the impetus to do this now
is that in https://github.com/neondatabase/neon/pull/11929, I want to
use some functions that were added only in the latest version of 'nix',
and it's nice to not have to build multiple versions. (Although,
different versions of 'nix' are still pulled in as indirect dependencies
from other packages)
2025-05-16 14:45:08 +00:00
Arpad Müller
2d247375b3 Update rust to 1.87.0 (#11938)
We keep the practice of keeping the compiler up to date, pointing to the
latest release. This is done by many other projects in the Rust
ecosystem as well.

The 1.87.0 release marks 10 years of Rust.

[Announcement blog
post](https://blog.rust-lang.org/2025/05/15/Rust-1.87.0/)

Prior update was in #11431
2025-05-16 12:21:24 +00:00
Christian Schwarz
a7ce323949 benchmarking: extend test_page_service_batching.py to cover concurrent IO + batching under random reads (#10466)
This PR commits the benchmarks I ran to qualify concurrent IO before we
released it.

Changes:
- Add `l0stack` fixture; a reusable abstraction for creating a stack of
L0 deltas
  each of which has 1 Value::Delta per page.
- Such a stack of L0 deltas is a good and understandable demo for
concurrent IO
because to reconstruct any page, $layer_stack_height` Values need to be
read.
  Before concurrent IO, the reads were sequential.
  With concurrent IO, they are executed concurrently.
- So, switch `test_latency` to use the l0stack.
- Teach `pagebench`, which is used by `test_latency`, to limit itself to
the blocks of the relation created by the l0stack abstraction.
- Additional parametrization of `test_latency` over dimensions
`ps_io_concurrency,l0_stack_height,queue_depth`
- Use better names for the tests to reflect what they do, leave
interpretation of the (now quite high-dimensional) results to the reader
  - `test_{throughput => postgres_seqscan}`
  - `test_{latency => random_reads}`
- Cut down on permutations to those we use in production. Runtime is
about 2min.

Refs
- concurrent IO epic https://github.com/neondatabase/neon/issues/9378 
- batching task: fixes https://github.com/neondatabase/neon/issues/9837

---------

Co-authored-by: Peter Bendel <peterbendel@neon.tech>
2025-05-15 17:48:13 +00:00
Vlad Lazar
31026d5a3c pageserver: support import schema evolution (#11935)
## Problem

Imports don't support schema evolution nicely. If we want to change the
stuff we keep in storcon,
we'd have to carry the old cruft around.

## Summary of changes

Version import progress. Note that the import progress version
determines the version of the import
job split and execution. This means that we can also use it as a
mechanism for deploying new import
implementations in the future.
2025-05-15 16:13:15 +00:00
Vlad Lazar
2621ce2daf pageserver: checkpoint import progress in the storage controller (#11862)
## Problem

Timeline imports do not have progress checkpointing. Any time that the
tenant is shut-down, all progress is lost
and the import restarts from the beginning when the tenant is
re-attached.

## Summary of changes

This PR adds progress checkpointing.


### Preliminaries

The **unit of work** is a `ChunkProcessingJob`. Each
`ChunkProcessingJob` deals with the import for a set of key ranges. The
job split is done by using an estimation of how many pages each job will
produce.

The planning stage must be **pure**: given a fixed set of contents in
the import bucket, it will always yield the same plan. This property is
enforced by checking that the hash of the plan is identical when
resuming from a checkpoint.

The storage controller tracks the progress of each shard in the import
in the database in the form of the **latest
job** that has has completed.

### Flow

This is the high level flow for the happy path:
1. On the first run of the import task, the import task queries storcon
for the progress and sees that none is recorded.
2. Execute the preparatory stage of the import
3. Import jobs start running concurrently in a `FuturesOrdered`. Every
time the checkpointing threshold of jobs has been reached, notify the
storage controller.
4. Tenant is detached and re-attached
5. Import task starts up again and gets the latest progress checkpoint
from the storage controller in the form of a job index.
6. The plan is computed again and we check that the hash matches with
the original plan.
7. Jobs are spawned from where the previous import task left off. Note
that we will not report progress after the completion of each job, so
some jobs might run twice.

Closes https://github.com/neondatabase/neon/issues/11568
Closes https://github.com/neondatabase/neon/issues/11664
2025-05-15 13:18:22 +00:00
Vlad Lazar
a703cd342b storage_controller: enforce generations in import upcalls (#11900)
## Problem

Import up-calls did not enforce the usage of the latest generation. The
import might have finished in one previous generation, but not in the
latest one. Hence, the controller might try to activate a timeline
before it is ready. In theory, that would be fine, but it's tricky to
reason about.

## Summary of Changes

Pageserver provides the current generation in the upcall to the storage
controller and the later validates the generation. If the generation is
stale, we return an error which stops progress of the import job. Note
that the import job will retry the upcall until the stale location is
detached.

I'll add some proper tests for this as part of the [checkpointing
PR](https://github.com/neondatabase/neon/pull/11862).

Closes https://github.com/neondatabase/neon/issues/11884
2025-05-15 10:02:11 +00:00
Alex Chi Z.
81fd652151 fix(pageserver): use better estimation for compaction memory usage (#11904)
## Problem

Hopefully resolves `test_gc_feedback` flakiness.

## Summary of changes

`accumulated_values` should not exceed 512MB to avoid OOM. Previously we
only use number of items, which is not a good estimation.

---------

Signed-off-by: Alex Chi Z <chi@neon.tech>
2025-05-14 08:32:55 +00:00
Vlad Lazar
045ae13e06 pageserver: make imports work with tenant shut downs (#11855)
## Problem

Lifetime of imported timelines (and implicitly the import background
task) has some shortcomings:
1. Timeline activation upon import completion is tricky. Previously, a
timeline that finished importing
after a tenant detach would not get activated and there's concerns about
the safety of activating
concurrently with shut-down.
2. Import jobs can prevent tenant shut down since they hold the tenant
gate

## Summary of Changes

Track the import tasks in memory and abort them explicitly on tenant
shutdown.

Integrate more closely with the storage controller:
1. When an import task has finished all of its jobs, it notifies the
storage controller, but **does not** mark the import as done in the
index_part. When all shards have finished importing, the storage
controller will call the `/activate_post_import` idempotent endpoint for
all of them. The handler, marks the import complete in index part,
resets the tenant if required and checks if the timeline is active yet.
2. Not directly related, but the import job now gets the starting state
from the storage controller instead of the import bucket. This paves the
way for progress checkpointing.

Related: https://github.com/neondatabase/neon/issues/11568
2025-05-13 17:49:49 +00:00
Christian Schwarz
79ddc803af feat(direct IO): runtime alignment validation; support config flag on macOS; default to DirectRw (#11868)
This PR adds a runtime validation mode to check adherence to alignment
and size-multiple requirements at the VirtualFile level.

This can help prevent alignment bugs from slipping into production
because test systems may have more lax requirements than production.
(This is not the case today, but it could change in the future).

It also allows catching O_DIRECT bugs on systems that don't have
O_DIRECT (macOS).
Consequently, we can now accept
`virtual_file_io_mode={direct,direct-rw}` on macOS now.
This has the side benefit of removing some annoying conditional
compilation around `IoMode`.

A third benefit is that it helped weed out size-multiple requirement
violation bugs in how the VirtualFile unit tests exercise read and write
APIs.
I seized the opportunity to trim these tests down to what actually
matters, i.e., exercising of the `OpenFiles` file descriptor cache.

Lastly, this PR flips the binary-built-in default to `DirectRw` so that
when running Python regress tests and benchmarks without specifying
`PAGESERVER_VIRTUAL_FILE_IO_MODE`, one gets the production behavior.

Refs
- fixes https://github.com/neondatabase/neon/issues/11676
2025-05-10 14:19:52 +00:00
Alex Chi Z.
93b964f829 fix(pageserver): do not do image compaction if it's below gc cutoff (#11872)
## Problem

We observe image compaction errors after gc-compaction finishes
compacting below the gc_cutoff. This is because `repartition` returns an
LSN below the gc horizon as we (likely) determined that `distance <=
self.repartition_threshold`.

I think it's better to keep the current behavior of when to trigger
compaction but we should skip image compaction if the returned LSN is
below the gc horizon.

## Summary of changes

If the repartition returns an invalid LSN, skip image compaction.

Signed-off-by: Alex Chi Z <chi@neon.tech>
2025-05-09 12:07:52 +00:00
Alex Chi Z.
d0dc65da12 fix(pageserver): give up gc-compaction if one key has too long history (#11869)
## Problem

The limitation we imposed last week
https://github.com/neondatabase/neon/pull/11709 is not enough to protect
excessive memory usage.

## Summary of changes

If a single key accumulated too much history, give up compaction. In the
future, we can make the `generate_key_retention` function take a stream
of keys instead of first accumulating them in memory, thus easily
support such long key history cases.

Signed-off-by: Alex Chi Z <chi@neon.tech>
2025-05-09 10:12:49 +00:00
Christian Schwarz
b37bb7d7ed pageserver: timeline shutdown: fully quiesce ingest path beforefreeze_and_flush (#11851)
# Problem 

Before this PR, timeline shutdown would
- cancel the walreceiver cancellation token subtree (child token of
Timeline::cancel)
- call freeze_and_flush
- Timeline::cancel.cancel()
- ... bunch of waiting for things ...
- Timeline::gate.close()

As noted by the comment that is deleted by this PR, this left a window
where, after freeze_and_flush, walreceiver could still be running and
ingest data into a new InMemoryLayer.

This presents a potential source of log noise during Timeline shutdown
where the InMemoryLayer created after the freeze_and_flush observes
that Timeline::cancel is cancelled, failing the ingest with some
anyhow::Error wrapping (deeply) a `FlushTaskError::Cancelled` instance
(`flush task cancelled` error message).

# Solution

It turns out that it is quite easy to shut down, not just cancel,
walreceiver completely
because the only subtask spawned by walreceiver connection manager is
the `handle_walreceiver_connection` task, which is properly shut down
and waited upon when the manager task observes cancellation and exits
its retry loop.

The alternative is to replace all the usage of `anyhow` on the ingest
path
with differentiated error types. A lot of busywork for little gain to
fix
a potential logging noise nuisance, so, not doing that for now.

# Correctness / Risk

We do not risk leaking walreceiver child tasks because existing
discipline
is to hold a gate guard.

We will prolong `Timeline::shutdown` to the degree that we're no longer
making
progress with the rest of shutdown while the walreceiver task hasn't yet
observed cancellation. In practice, this should be negligible.

`Timeline::shutdown` could fail to complete if there is a hidden
dependency
of walreceiver shutdown on some subsystem. The code certainly suggests
there
isn't, and I'm not aware of any such dependency. Anyway, impact will be
low
because we only shut down Timeline instances that are obsolete, either
because
there is a newer attachment at a different location, or because the
timeline
got deleted by the user. We would learn about this through stuck cplane
operations or stuck storcon reconciliations. We would be able to
mitigate by
cancelling such stuck operations/reconciliations and/or by rolling back
pageserver.

# Refs
- identified this while investigating
https://github.com/neondatabase/neon/issues/11762
- PR that _does_ fix a bunch _real_ `flush task cancelled` noise on the
compaction path: https://github.com/neondatabase/neon/pull/11853
2025-05-08 18:48:24 +00:00
Christian Schwarz
42d93031a1 fixup(#11819): broken macOS build (#11861)
refs
- fixes https://github.com/neondatabase/neon/issues/11860
2025-05-08 11:48:29 +00:00
Vlad Lazar
40f32ea326 pageserver: refactor import flow and add job concurrency limiting (#11816)
## Problem

Import code is one big block. Separating planning and execution will
help with reporting
progress of import to storcon (building block for resuming import).

## Summary of changes

Split up the import into planning and execution.
A concurrency limit driven by PS config is also added.
2025-05-08 09:19:14 +00:00
Christian Schwarz
1d1502bc16 fix(pageserver): flush task cancelled errors during timeline shutdown (#11853)
# Refs
- fixes https://github.com/neondatabase/neon/issues/11762

# Problem

PR #10993 introduced internal retries for BufferedWriter flushes.
PR #11052 added cancellation sensitivity to that retry loop.
That cancellation sensitivity is an error path that didn't exist before.

The result is that during timeline shutdown, after we
`Timeline::cancel`, compaction can now fail with error `flush task
cancelled`.
The problem with that:
1. We mis-classify this as an `error!`-worthy event.
2. This causes tests to become flaky because the error is not in global
`allowed_errors`.

Technically we also trip the `compaction_circuit_breaker` because the
resulting `CompactionError` is variant `::Other`.
But since this is Timeline shutdown, is doesn't matter practically
speaking.

# Solution / Changes

- Log the anyhow stack trace when classifying a compaction error as
`error!`.
  This was helpful to identify sources of `flush task cancelled` errors.
We only log at `error!` level in exceptional circumstances, so, it's ok
to have bit verbose logs.
- Introduce typed errors along the `BufferedWriter::write_*`=>
`BlobWriter::write_blob`
=> `{Delta,Image}LayerWriter::put_*` =>
`Split{Delta,Image}LayerWriter::put_{value,image}` chain.
- Proper mapping to `CompactionError`/`CreateImageLayersError` via new
`From` impls.

I am usually opposed to any magic `From` impls, but, it's how most of
the compaction code
works today.

# Testing

The symptoms are most prevalent in
`test_runner/regress/test_branch_and_gc.py::test_branch_and_gc`.
Before this PR, I was able to reproduce locally 1 or 2 times per 400
runs using
`DEFAULT_PG_VERSION=15 BUILD_TYPE=release poetry run pytest --count 400
-n 8`.
After this PR, it doesn't reproduce anymore after 2000 runs.

# Future Work

Technically the ingest path is also exposed to this new source of errors
because `InMemoryLayer` is backed by `BufferedWriter`.
But we haven't seen it occur in flaky tests yet.
Details and a fix in
- https://github.com/neondatabase/neon/pull/11851
2025-05-08 06:57:53 +00:00
Christian Schwarz
7eb85c56ac tokio-epoll-uring: avoid warn! noise due to ECANCELED during shutdowns (#11819)
# Problem

Before this PR, `test_pageserver_catchup_while_compute_down` would
occasionally fail due to scary-looking WARN log line

```
WARN ephemeral_file_buffered_writer{...}:flush_attempt{attempt=1}: \
 error flushing buffered writer buffer to disk, retrying after backoff err=Operation canceled (os error 125)
```

After lengthy investigation, the conclusion is that this is likely due
to a kernel bug related due to io_uring async workers (io-wq) and
signals.
The main indicator is that the error only ever happens in correlation
with pageserver shtudown when SIGTERM is received.
There is a fix that is merged in 6.14
kernels (`io-wq: backoff when retrying worker creation`).
However, even when I revert that patch, the issue is not reproducible
on 6.14, so, it remains a speculation.

It was ruled out that the ECANCELED is due to the executor thread
exiting before the async worker starts processing the operation.

# Solution

The workaround in this issue is to retry the operation on ECANCELED
once.
Retries are safe because the low-level io_engine operations are
idempotent.
(We don't use O_APPEND and I can't think of another flag that would make
 the APIs covered by this patch not idempotent.)

# Testing

With this PR, the warn! log no longer happens on [my reproducer
setup](https://github.com/neondatabase/neon/issues/11446#issuecomment-2843015111).
And the new rate-limited `info!`-level log line informing about the
internal retry shows up instead, as expected.

# Refs
- fixes https://github.com/neondatabase/neon/issues/11446
2025-05-08 06:33:29 +00:00
Vlad Lazar
3cf5e1386c pageserver: fix rough edges of pageserver tracing (#11842)
## Problem

There's a few rough edges around PS tracing.

## Summary of changes

* include compute request id in pageserver trace
* use the get page specific context for GET_REL_SIZE and GET_BATCH
* fix assertion in download layer trace


![image](https://github.com/user-attachments/assets/2ff6779c-7c2d-4102-8013-ada8203aa42f)
2025-05-07 10:13:26 +00:00
Alex Chi Z.
6827f2f58c fix(pageserver): only keep iter_with_options API, improve docs in gc-compact (#11804)
## Problem

Address comments in https://github.com/neondatabase/neon/pull/11709

## Summary of changes

- remove `iter` API, users always need to specify buffer size depending
on the expected memory usage.
- several doc improvements

---------

Signed-off-by: Alex Chi Z <chi@neon.tech>
Co-authored-by: Christian Schwarz <christian@neon.tech>
2025-05-06 12:27:16 +00:00
devin-ai-integration[bot]
baf425a2cd [pageserver/virtual_file] impr: Improve OpenOptions API ergonomics (#11789)
# Improve OpenOptions API ergonomics

Closes #11787

This PR improves the OpenOptions API ergonomics by:

1. Making OpenOptions methods take and return owned Self instead of &mut
self
2. Changing VirtualFile::open_with_options_v2 to take an owned
OpenOptions
3. Removing unnecessary .clone() and .to_owned() calls

These changes make the API more idiomatic Rust by leveraging the builder
pattern with owned values, which is cleaner and more ergonomic than the
previous approach.

Link to Devin run:
https://app.devin.ai/sessions/c2a4b24f7aca40a3b3777f4259bf8ee1
Requested by: christian@neon.tech

---------

Co-authored-by: Devin AI <158243242+devin-ai-integration[bot]@users.noreply.github.com>
Co-authored-by: christian@neon.tech <christian@neon.tech>
2025-05-05 13:06:37 +00:00
Vlad Lazar
16d594b7b3 pagectl: list layers for given key in decreasing LSN order (#11799)
Adds an extra key CLI arg to `pagectl layer list-layer`. When provided,
only layers with key ranges containing the key will be listed in
decreasing LSN order (indices are preserved for `dump-layer`).
2025-05-01 15:56:43 +00:00
Alex Chi Z.
e2db76b9be feat(pageserver): ondemand download reason observability (#11780)
## Problem

Part of https://github.com/neondatabase/neon/issues/11615

## Summary of changes

We don't understand the root cause of why we get resident size surge
every now and then. This patch adds observability for that, and in the
next week, we might have a better understanding of what's going on.

---------

Signed-off-by: Alex Chi Z <chi@neon.tech>
2025-04-30 16:04:00 +00:00
Alex Chi Z.
6b4b8e0d8b fix(pageserver): do not increase basebackup err counter when shutdown (#11778)
## Problem

We occasionally see basebackup errors alerts but there were no errors
logged. Looking at the code, the only codepath that will cause this is
shutting down.

## Summary of changes

Do not increase any counter (ok/err) when basebackup request gets
cancelled due to shutdowns.

Signed-off-by: Alex Chi Z <chi@neon.tech>
2025-04-30 15:50:12 +00:00
devin-ai-integration[bot]
1d06172d59 pageserver: remove resident size from billing metrics (#11699)
This is a rebase of PR #10739 by @henryliu2014 on the current main
branch.

## Problem

pageserver: remove resident size from billing metrics

Fixes #10388

## Summary of changes

The following changes have been made to remove resident size from
billing metrics:

* removed the metric "resident_size" and related codes in
consumption_metrics/metrics.rs
* removed the item of the description of metric "resident_size" in
consumption_metrics.md
* refactored the metric "resident_size" related test case

Requested by: John Spray (john@neon.tech)

---------

Co-authored-by: liuheqing <hq.liu@qq.com>
Co-authored-by: Devin AI <158243242+devin-ai-integration[bot]@users.noreply.github.com>
Co-authored-by: John Spray <john@neon.tech>
2025-04-29 18:34:56 +00:00
Vlad Lazar
768a580373 pageserver: add not modified since lsn to get page span (#11774)
It's useful when debugging.
2025-04-29 14:07:23 +00:00
Alex Chi Z.
c1ff7db187 fix(pageserver): consider tombstones in replorigin (#11752)
## Problem

We didn't consider tombstones in replorigin read path in the past. This
was fine because tombstones are stored as LSN::Invalid before we
universally define what the tombstone is for sparse keyspaces.

Now we remove non-inherited keys during detach ancestor and write the
universal tombstone "empty image". So we need to consider it across all
the read paths.

related: https://github.com/neondatabase/neon/pull/11299

## Summary of changes

Empty value gets ignored for replorigin scans.

---------

Signed-off-by: Alex Chi Z <chi@neon.tech>
2025-04-28 18:54:26 +00:00
John Spray
0482690534 pageserver: make control_plane_api & generations fully mandatory (#10715)
## Problem

We had retained the ability to run in a generation-less mode to support
test_generations_upgrade, which was replaced with a cleaner backward
compat test in https://github.com/neondatabase/neon/pull/10701

## Summary of changes

- Remove all the special cases for "if no generation" or "if no control
plane api"
- Make control_plane_api config mandatory

---------

Co-authored-by: Arpad Müller <arpad-m@users.noreply.github.com>
2025-04-28 17:24:55 +00:00
Erik Grinaker
606f14034e pageserver: improve pageserver_smgr_query_seconds buckets (#11680)
## Problem

The `pageserver_smgr_query_seconds` buckets are too coarse, using powers
of 10: 1 µs, 10 µs, 100 µs, 1 ms, 10 ms, 100 ms, 1 s, 10 s, 100 s. This
is one of our most crucial latency metrics, and needs better resolution.

Touches #11594.

## Summary of changes

This patch uses buckets with better resolution around 1 ms (the typical
latency):

* 0.6 ms
* 1 ms
* 3 ms
* 6 ms
* 10 ms
* 30 ms
* 100 ms
* 1 s
* 3 s

These will be the same as the compute's `compute_getpage_wait_seconds`,
to make them comparable across the compute and Pageserver:
https://github.com/neondatabase/flux-fleet/pull/579. We sacrifice
buckets above 3 s, since these can already be considered "too slow".

This does not change the previously used `CRITICAL_OP_BUCKETS`, which is
also used for other operations on different timescales (e.g. LSN waits).
We should consider replacing this with more appropriate buckets for
specific operations, since it covers a large span with low resolution.
2025-04-28 11:52:44 +00:00
Vlad Lazar
6f0046b688 storage_controller: ensure mutual exclusion for imports and shard splits (#11632)
## Problem

Shard splits break timeline imports.

## Summary of Changes

Ensure mutual exclusion for imports and shard splits.

On the shard split code path:
1. Right before shard splitting, check the database to ensure that
no-import is on-going for the tenant. Exclusion is guaranteed because
this validation is done while holding the exclusive tenant lock.
Timeline creation (and import creation implicitly) requires a shared
tenant lock.
2. When selecting a shard to split, use the in-mem state to exclude
shards with an on-going import. This is opportunistic since an import
might start after the check, but allows shard splits to make progres
instead of continously retrying to split the same shard.

On the timeline creation code path:
1. Check the in-memory splitting flag on all shards of the tenant. If
any of them are splitting, error out asking the client to retry. On the
happy path this is not required, due to the tenant lock set-up described
above, but it covers the case where we restart with a pending
shard-split.

Closes https://github.com/neondatabase/neon/issues/11567
2025-04-25 11:46:15 +00:00
Alex Chi Z.
5d91d4e843 fix(pageserver): reduce gc-compaction memory usage (#11709)
## Problem

close https://github.com/neondatabase/neon/issues/11694

We had the delta layer iterator and image layer iterator set to buffer
at most 8MB data. Note that 8MB is the compressed size, so it is
possible for those iterators contain more than 8MB data in memory.

For the recent OOM case, gc-compaction was running over 556 layers,
which means that we will have 556 active iterators. So in theory, it
could take up to 556*8=4448MB memory when the compaction is going on. If
images get compressed and the compression ratio is high (for that
tenant, we see 3x compression ratio across image layers), then that's
13344MB memory.

Also we have layer rewrites, which explains the memory taken by
gc-compaction itself (versus the iterators). We rewrite 424 out of 556
layers, and each of such rewrites need a pair of delta layer writer. So
we are buffering a lot of deltas in the memory.

The flamegraph shows that gc-compaction itself takes 6GB memory, delta
iterator 7GB, and image iterator 2GB, which can be explained by the
above theory.

## Summary of changes

- Reduce the buffer sizes.
- Estimate memory consumption and if it is too high.
- Also give up if the number of layers-to-rewrite is too high.

---------

Signed-off-by: Alex Chi Z <chi@neon.tech>
2025-04-25 08:45:31 +00:00
Christian Schwarz
8afb783708 feat: Direct IO for the pageserver write path (#11558)
# Problem

The Pageserver read path exclusively uses direct IO if
`virtual_file_io_mode=direct`.

The write path is half-finished. Here is what the various writing
components use:

|what|buffering|flags on <br/>`v_f_io_mode`<br/>=`buffered`|flags on
<br/>`virtual_file_io_mode`<br/>=`direct`|
|-|-|-|-|
|`DeltaLayerWriter`| BlobWriter<BUFFERED=true> | () | () |
|`ImageLayerWriter`| BlobWriter<BUFFERED=false> | () | () |
|`download_layer_file`|BufferedWriter|()|()|
|`InMemoryLayer`|BufferedWriter|()|O_DIRECT|


The vehicle towards direct IO support is `BufferedWriter` which
- largely takes care of O_DIRECT alignment & size-multiple requirements 
- double-buffering to mask latency

`DeltaLayerWriter`, `ImageLayerWriter` use `blob_io::BlobWriter` , which
has neither of these.

# Changes

## High-Level

At a high-level this PR makes the following primary changes:

- switch the two layer writer types to use `BufferedWriter` & make
sensitive to `virtual_file_io_mode` (via open_with_options_**v2**)
- make `download_layer_file` sensitive to `virtual_file_io_mode` (also
via open_with_options_**v2**)
- add `virtual_file_io_mode=direct-rw` as a feature gate
- we're hackish-ly piggybacking on OpenOptions's ask for write access
here
- this means with just `=direct` InMemoryLayer reads and writes no
longer uses O_DIRECT
- this is transitory and we'll remove the `direct-rw` variant once the
rollout is complete

(The `_v2` APIs for opening / creating VirtualFile are those that are
sensitive to `virtual_file_io_mode`)

The result is:

|what|uses <br/>`BufferedWriter`|flags on
<br/>`v_f_io_mode`<br/>=`buffered`|flags on
<br/>`v_f_io_mode`<br/>=`direct`|flags on
<br/>`v_f_io_mode`<br/>=`direct-rw`|
|-|-|-|-|-|
|`DeltaLayerWriter`| ~~Blob~~BufferedWriter | () | () |  O_DIRECT |
|`ImageLayerWriter`| ~~Blob~~BufferedWriter | () | () |  O_DIRECT |
|`download_layer_file`|BufferedWriter|()|()|O_DIRECT|
|`InMemoryLayer`|BufferedWriter|()|~~O_DIRECT~~()|O_DIRECT|


## Code-Level


The main change is:
- Switch `blob_io::BlobWriter` away from its own buffering method to use
`BufferedWriter`.

Additional prep for upholding `O_DIRECT` requirements:
- Layer writer `finish()` methods switched to use IoBufferMut for
guaranteed buffer address alignment. The size of the buffers is PAGE_SZ
and thereby implicitly assumed to fulfill O_DIRECT requirements.

For the hacky feature-gating via `=direct-rw`:
- Track `OpenOptions::write(true|false)` in a field; bunch of mechanical
churn.
- Consolidate the APIs in which we "open" or "create" VirtualFile for
better overview over which parts of the code use the `_v2` APIs.

Necessary refactorings & infra work:
- Add doc comments explaining how BufferedWriter ensures that writes are
compliant with O_DIRECT alignment & size constraints. This isn't new,
but should be spelled out.
- Add the concept of shutdown modes to `BufferedWriter::shutdown` to
make writer shutdown adhere to these constraints.
- The `PadThenTruncate` mode might not be necessary in practice because
I believe all layer files ever written are sized in multiples `PAGE_SZ`
and since `PAGE_SZ` is larger than the current alignment requirements
(512/4k depending on platform), it won't be necesary to pad.
- Some test (I believe `round_trip_test_compressed`?) required it though
- [ ] TODO: decide if we want to accept that complexity; if we do then
address TODO in the code to separate alignment requirement from buffer
capacity
- Add `set_len` (=`ftruncate`) VirtualFile operation to support the
above.
- Allow `BufferedWriter` to start at a non-zero offset (to make room for
the summary block).

Cleanups unlocked by this change:
- Remove non-positional APIs from VirtualFile (e.g. seek, write_full,
read_full)

Drive-by fixes:
- PR https://github.com/neondatabase/neon/pull/11585 aimed to run unit
tests for all `virtual_file_io_mode` combinations but didn't because of
a missing `_` in the env var.

# Performance

This section assesses this PR's impact on deployments with current
production setting (`=direct`) and anticipated impact of switching to
(`=direct-rw`).

For `DeltaLayerWriter`, `=direct` should remain unchanged to slightly
improved on throughput because the `BlobWriter`'s buffer had the same
size as the `BufferedWriter`'s buffer, but it didn't have the
double-buffering that `BufferedWriter` has.
The `=direct-rw` enables direct IO; throughput should not be suffering
because of double-buffering; benchmarks will show if this is true.

The `ImageLayerWriter` was previously not doing any buffering
(`BUFFERED=false`).
It went straight to issuing the IO operation to the underlying
VirtualFile and the buffering was done by the kernel.
The switch to `BufferedWriter` under `=direct` adds an additional memcpy
into the BufferedWriter's buffer.
We will win back that memcpy when enabling direct IO via `=direct-rw`.

A nice win from the switch to `BufferedWriter` is that ImageLayerWriter
performs >=16x fewer write operations to VirtualFile (the BlobWriter
performs one write per len field and one write per image value).
This should save low tens of microseconds of CPU overhead from doing all
these syscalls/io_uring operations, regardless of `=direct` or
`=direct-rw`.
Aside from problems with alignment, this write frequency without
double-buffering is prohibitive if we actually have to wait for the
disk, which is what will happen when we enable direct IO via
(`=direct-rw`).
Throughput should not be suffering because of BufferedWrite's
double-buffering; benchmarks will show if this is true.

`InMemoryLayer` at `=direct` will flip back to using buffered IO but
remain on BufferedWriter.
The buffered IO adds back one memcpy of CPU overhead.
Throughput should not suffer and will might improve on
not-memory-pressured Pageservers but let's remember that we're doing the
whole direct IO thing to eliminate global memory pressure as a source of
perf variability.

## bench_ingest

I reran `bench_ingest` on `im4gn.2xlarge` and `Hetzner AX102`.
Use `git diff` with `--word-diff` or similar to see the change.

General guidance on interpretation:
- immediate production impact of this PR without production config
change can be gauged by comparing the same `io_mode=Direct`
- end state of production switched over to `io_mode=DirectRw` can be
gauged by comparing old results' `io_mode=Direct` to new results'
`io_mode=DirectRw`

Given above guidance, on `im4gn.2xlarge`
- immediate impact is a significant improvement in all cases
- end state after switching has same significant improvements in all
cases
- ... except `ingest/io_mode=DirectRw volume_mib=128 key_size_bytes=8192
key_layout=Sequential write_delta=Yes` which only achieves `238 MiB/s`
instead of `253.43 MiB/s`
  - this is a 6% degradation
  - this workload is typical for image layer creation

# Refs
- epic https://github.com/neondatabase/neon/issues/9868
- stacked atop
  - preliminary refactor https://github.com/neondatabase/neon/pull/11549
- bench_ingest overhaul https://github.com/neondatabase/neon/pull/11667
- derived from https://github.com/neondatabase/neon/pull/10063

Co-authored-by: Yuchen Liang <yuchen@neon.tech>
2025-04-24 14:57:36 +00:00
Christian Schwarz
9c6ff3aa2b refactor(BufferedWriter): flush task owns the VirtualFile & abstraction for cleanup on drop (#11549)
Main change:

- `BufferedWriter` owns the `W`; no more `Arc<W>`
- We introduce auto-delete-on-drop wrappers for `VirtualFile`.
  - `TempVirtualFile` for write-only users
- `TempVirtualFileCoOwnedByEphemeralFileAndBufferedWriter` for
EphemeralFile which requires read access to the immutable prefix of the
file (see doc comments for details)
- Users of `BufferedWriter` hand it such a wrapped `VirtualFile`.
- The wrapped `VirtualFile` moves to the background flush task.
- On `BufferedWriter` shutdown, ownership moves back.
- Callers remove the wrapper (`disarm_into_inner()`) after doing final
touches, e.g., flushing index blocks and summary for delta/image layer
writers.

If the BufferedWriter isn't shut down properly via
`BufferedWriter::shutdown`, or if there is an error during final
touches, the wrapper type ensures that the file gets unlinked.

We store a GateGuard inside the wrapper to ensure that the Timeline is
still alive when unlinking on drop.

Rust doesn't have async drop yet, so, the unlinking happens using a
synchronous syscall.
NB we don't fsync the surrounding directory.
This is how it's been before this PR; I believe it is correct because
all of these files are temporary paths that get cleaned up on timeline
load.
Again, timeline load does not need to fsync because the next timeline
load will unlink again if the file reappears.

The auto-delete-on-drop can happen after a higher-level mechanism
retries.
Therefore, we switch all users to monotonically increasing, never-reused
temp file disambiguators.

The aspects pointed out in the last two paragraphs will receive further
cleanup in follow-up task
- https://github.com/neondatabase/neon/issues/11692

Drive-by changes:
- It turns out we can remove the two-pronged code in the layer file
download code.
No need to make this a separate PR because all of production already
uses `tokio-epoll-uring` with the buffered writer for many weeks.


Refs
- epic https://github.com/neondatabase/neon/issues/9868
- alternative to https://github.com/neondatabase/neon/pull/11544
2025-04-24 13:07:57 +00:00
Arpad Müller
c35d489539 versioning API for remote_storage (#11671)
Adds a versioning API to remote_storage. We want to use it in the
scrubber, both for tenant snapshot as well as for metadata checks.

for #8830
and for #11588
2025-04-24 11:41:48 +00:00
Vlad Lazar
3a50d95b6d storage_controller: coordinate imports across shards in the storage controller (#11345)
## Problem

Pageservers notify control plane directly when a shard import has
completed.
Control plane has to download the status of each shard from S3 and
figure out if everything is truly done,
before proceeding with branch activation.

Issues with this approach are:
* We can't control shard split behaviour on the storage controller side.
It's unsafe to split
during import.
* Control plane needs to know about shards and implement logic to check
all timelines are indeed ready.

## Summary of changes

In short, storage controller coordinates imports, and, only when
everything is done, notifies control plane.

Big rocks:
1. Store timeline imports in the storage controller database. Each
import stores the status of its shards in the database.
We hook into the timeline creation call as our entry point for this.
2. Pageservers get a new upcall endpoint to notify the storage
controller of shard import updates.
3. Storage controller handles these updates by updating persisted state.
If an update finalizes the import,
then poll pageservers until timeline activation, and, then, notify the
control plane that the import is complete.

Cplane side change with new endpoint is in
https://github.com/neondatabase/cloud/pull/26166

Closes https://github.com/neondatabase/neon/issues/11566
2025-04-24 11:26:06 +00:00
devin-ai-integration[bot]
1808dad269 Add --dev CLI flag to pageserver and safekeeper binaries (#11526)
# Add --dev CLI flag to pageserver and safekeeper binaries

This PR adds the `--dev` CLI flag to both the pageserver and safekeeper
binaries without implementing any functionality yet. This is a precursor
to PR #11517, which will implement the full functionality to require
authentication by default unless the `--dev` flag is specified.

## Changes
- Add `dev_mode` config field to pageserver binary
- Add `--dev` CLI flag to safekeeper binary

This PR is needed for forward compatibility tests to work properly, when
we try to merge #11517

Link to Devin run:
https://app.devin.ai/sessions/ad8231b4e2be430398072b6fc4e85d46
Requested by: John Spray (john@neon.tech)

---------

Co-authored-by: Devin AI <158243242+devin-ai-integration[bot]@users.noreply.github.com>
Co-authored-by: John Spray <john@neon.tech>
2025-04-24 10:45:40 +00:00
Christian Schwarz
51cdb570eb bench_ingest: general overhaul & add parametrization over virtual_file_io_mode (#11667)
Changes:
- clean up existing parametrization & criterion `BenchmarkId`
- additional parametrization over `virtual_file_io_mode`
- switch to `multi_thread` to be closer to production ([Slack
thread](https://neondb.slack.com/archives/C033RQ5SPDH/p1745339543093159))

Refs
- epic https://github.com/neondatabase/neon/issues/9868
- extracted from https://github.com/neondatabase/neon/pull/11558
2025-04-24 07:38:18 +00:00
Alex Chi Z.
ad3519ebcb fix(pageserver): report synthetic size = 1 if all tls offloaded (#11648)
## Problem

A quick workaround for https://github.com/neondatabase/neon/issues/11631

## Summary of changes

Report synthetic size == 1 if all timelines are offloaded.

Signed-off-by: Alex Chi Z <chi@neon.tech>
2025-04-22 14:28:22 +00:00
Vlad Lazar
cbf442292b pageserver: handle empty get vectored queries (#11652)
## Problem

If all batched requests are excluded from the query by
`Timeine::get_rel_page_at_lsn_batched` (e.g. because they are past the
end of the relation), the read path would panic since it doesn't expect
empty queries. This is a change in behaviour that was introduced with
the scattered query implementation.

## Summary of Changes

Handle empty queries explicitly.
2025-04-21 17:45:16 +00:00
Heikki Linnakangas
4d0c1e8b78 refactor: Extract some code in pagebench getpage command to function (#11563)
This makes it easier to add a different client implementation alongside
the current one. I started working on a new gRPC-based protocol to
replace the libpq protocol, which will introduce a new function like
`client_libpq`, but for the new protocol.

It's a little more readable with less indentation anyway.
2025-04-19 08:38:03 +00:00
Dmitrii Kovalkov
a0d844dfed pageserver + safekeeper: pass ssl ca certs to broker client (#11635)
## Problem
Pageservers and safakeepers do not pass CA certificates to broker
client, so the client do not trust locally issued certificates.
- Part of https://github.com/neondatabase/cloud/issues/27492

## Summary of changes
- Change `ssl_ca_certs` type in PS/SK's config to `Pem` which may be
converted to both `reqwest` and `tonic` certificates.
- Pass CA certificates to storage broker client in PS and SK
2025-04-18 06:27:23 +00:00
Alex Chi Z.
5073e46df4 feat(pageserver): use rfc3339 time and print ratio in gc-compact stats (#11638)
## Problem

follow-up on https://github.com/neondatabase/neon/pull/11601

## Summary of changes

- serialize the start/end time using rfc3339 time string
- compute the size ratio of the compaction

---------

Signed-off-by: Alex Chi Z <chi@neon.tech>
2025-04-18 05:28:01 +00:00
Christian Schwarz
2b041964b3 cover direct IO + concurrent IO in unit, regression & perf tests (#11585)
This mirrors the production config.

Thread that discusses the merits of this:
- https://neondb.slack.com/archives/C033RQ5SPDH/p1744742010740569

# Refs
- context
https://neondb.slack.com/archives/C04BLQ4LW7K/p1744724844844589?thread_ts=1744705831.014169&cid=C04BLQ4LW7K
- prep for https://github.com/neondatabase/neon/pull/11558 which adds
new io mode `direct-rw`

# Impact on CI turnaround time

Spot-checking impact on CI timings

- Baseline: [some recent main
commit](https://github.com/neondatabase/neon/actions/runs/14471549758/job/40587837475)
- Comparison: [this
commit](https://github.com/neondatabase/neon/actions/runs/14471945087/job/40589613274)
in this PR here

Impact on CI turnaround time

- Regression tests:
  - x64: very minor, sometimes better; likely in the noise
  - arm64: substantial  30min => 40min
- Benchmarks (x86 only I think): very minor; noise seems higher than
regress tests

---------

Signed-off-by: Alex Chi Z <chi@neon.tech>
Co-authored-by: Alex Chi Z. <4198311+skyzh@users.noreply.github.com>
Co-authored-by: Peter Bendel <peterbendel@neon.tech>
Co-authored-by: Alex Chi Z <chi@neon.tech>
2025-04-17 15:53:10 +00:00
John Spray
0a27973584 pageserver: rename Tenant to TenantShard (#11589)
## Problem

`Tenant` isn't really a whole tenant: it's just one shard of a tenant.

## Summary of changes

- Automated rename of Tenant to TenantShard
- Followup commit to change references in comments
2025-04-17 13:29:16 +00:00
Tristan Partin
9794f386f4 Make Postgres 17 the default version (#11619)
This is mostly a documentation update, but a few updates with regard to
neon_local, pageserver, and tests.

17 is our default for users in production, so dropping references to 16
makes sense.

Signed-off-by: Tristan Partin <tristan@neon.tech>

Signed-off-by: Tristan Partin <tristan@neon.tech>
2025-04-16 23:23:37 +00:00
Alex Chi Z.
cf2e695f49 feat(pageserver): gc-compaction meta statistics (#11601)
## Problem

We currently only have gc-compaction statistics for each single
sub-compaction job.

## Summary of changes

Add meta statistics across all sub-compaction jobs scheduled.

Signed-off-by: Alex Chi Z <chi@neon.tech>
2025-04-16 18:51:48 +00:00
Erik Grinaker
4af0b9b387 pageserver: don't recompress images in ImageLayerInner::filter() (#11592)
## Problem

During shard ancestor compaction, we currently recompress all page
images as we move them into a new layer file. This is expensive and
unnecessary.

Resolves #11562.
Requires #11607.

## Summary of changes

Pass through compressed page images in `ImageLayerInner::filter()`.
2025-04-16 17:10:15 +00:00
Erik Grinaker
46100717ad pageserver: add VectoredBlob::raw_with_header (#11607)
## Problem

To avoid recompressing page images during layer filtering, we need
access to the raw header and data from vectored reads such that we can
pass them through to the target layer.

Touches #11562.

## Summary of changes

Adds `VectoredBlob::raw_with_header()` to return a raw view of the
header+data, and updates `read()` to track it.

Also adds `blob_io::Header` with header metadata and decode logic, to
reuse for tests and assertions. This isn't yet widely used.
2025-04-16 15:38:10 +00:00