Commit Graph

3091 Commits

Author SHA1 Message Date
Vlad Lazar
8a6fc6fd8c pageserver: hook importing timelines up into disk usage eviction (#12038)
## Problem

Disk usage eviction isn't sensitive to layers of imported timelines.

## Summary of changes

Hook importing timelines up into eviction and add a test for it.
I don't think we need any special eviction logic for this. These layers
will all be visible and
their access time will be their creation time. Hence, we'll remove
covered layers first
and get to the imported layers if there's still disk pressure.
2025-05-29 13:01:10 +00:00
Vlad Lazar
51639cd6af pageserver: allow for deletion of importing timelines (#12033)
## Problem

Importing timelines can't currently be deleted. This is problematic
because:
1. Cplane cannot delete failed imports and we leave the timeline behind.
2. The flow does not support user driven cancellation of the import

## Summary of changes

On the pageserver: I've taken the path of least resistance, extended
`TimelineOrOffloaded`
with a new variant and added handling in the right places. I'm open to
thoughts here,
but I think it turned out better than I was envisioning.

On the storage controller: Again, fairly simple business: when a DELETE
timeline request is
received, we remove the import from the DB and stop any finalization
tasks/futures. In order
to stop finalizations, we track them in-memory. For each finalizing
import, we associate a gate
and a cancellation token.

Note that we delete the entry from the database before cancelling any
finalizations. This is such
that a concurrent request can't progress the import into finalize state
and race with the deletion.
This concern about deleting an import with on-going finalization is
theoretical in the near future.
We are only going to delete importing timelines after the storage
controller reports the failure to
cplane. Alas, the design works for user driven cancellation too.

Closes https://github.com/neondatabase/neon/issues/11897
2025-05-29 11:13:52 +00:00
Alex Chi Z.
9e4cf52949 pageserver: reduce concurrency for gc-compaction (#12054)
## Problem

Temporarily reduce the concurrency of gc-compaction to 1 job at a time.
We are going to roll out in the largest AWS region next week. Having one
job running at a time makes it easier to identify what tenant causes
problem if it's not running well and pause gc-compaction for that
specific tenant.

(We can make this configurable via pageserver config in the future!)

## Summary of changes

Reduce `CONCURRENT_GC_COMPACTION_TASKS` from 2 to 1.

Signed-off-by: Alex Chi Z <chi@neon.tech>
2025-05-29 09:32:19 +00:00
Vlad Lazar
eadabeddb8 pageserver: use the same job size throughout the import lifetime (#12026)
## Problem

Import planning takes a job size limit as its input. Previously, the job
size came from a pageserver config field. This field may change while
imports are in progress. If this happens, plans will no longer be
identical and the import would fail permanently.

## Summary of Changes

Bake the job size into the import progress reported to the storage
controller. For new imports, use the value from the pagesever config,
and, for existing imports, use the value present in the shard progress.

This value is identical for all shards, but we want it to be versioned
since future versions of the planner might split the jobs up
differently. Hence, it ends up in `ShardImportProgress`.

Closes https://github.com/neondatabase/neon/issues/11983
2025-05-28 15:19:41 +00:00
Alex Chi Z.
67ddf1de28 feat(pageserver): create image layers at L0-L1 boundary (#12023)
## Problem

Previous attempt https://github.com/neondatabase/neon/pull/10548 caused
some issues in staging and we reverted it. This is a re-attempt to
address https://github.com/neondatabase/neon/issues/11063.

Currently we create image layers at latest record LSN. We would create
"future image layers" (i.e., image layers with LSN larger than disk
consistent LSN) that need special handling at startup. We also waste a
lot of read operations to reconstruct from L0 layers while we could have
compacted all of the L0 layers and operate on a flat level of historic
layers.

## Summary of changes

* Run repartition at L0-L1 boundary.
* Roll out with feature flags.
* Piggyback a change that downgrades "image layer creating below
gc_cutoff" to debug level.

---------

Signed-off-by: Alex Chi Z <chi@neon.tech>
2025-05-28 07:00:52 +00:00
Nikita Kalyanov
541fcd8d2f chore: expose new mark_invisible API in openAPI spec for use in cplane (#12032)
## Problem
There is a new API that I plan to use. We generate client from the spec
so it should be in the spec
## Summary of changes
Document the existing API in openAPI format
2025-05-28 03:39:59 +00:00
Alex Chi Z.
f0bb93a9c9 feat(pageserver): support evaluate boolean flags (#12024)
## Problem

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

## Summary of changes

* Support evaluate boolean flags.
* Add docs on how to handle errors.
* Add test cases based on real PostHog config.

Signed-off-by: Alex Chi Z <chi@neon.tech>
2025-05-27 14:29:15 +00:00
Vlad Lazar
30adf8e2bd pageserver: add tracing spans for time spent in batch and flushing (#12012)
## Problem

We have some gaps in our traces. This indicates missing spans.

## Summary of changes

This PR adds two new spans:
* WAIT_EXECUTOR: time a batched request spends in the batch waiting to
be picked up
* FLUSH_RESPONSE: time a get page request spends flushing the response
to the compute


![image](https://github.com/user-attachments/assets/41b3ddb8-438d-4375-9da3-da341fc0916a)
2025-05-27 13:57:53 +00:00
Erik Grinaker
5d538a9503 page_api: tweak errors (#12019)
## Problem

The page API gRPC errors need a few tweaks to integrate better with the
GetPage machinery.

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

## Summary of changes

* Add `GetPageStatus::InternalError` for internal server errors.
* Rename `GetPageStatus::Invalid` to `InvalidRequest` for clarity.
* Rename `status` and `GetPageStatus` to `status_code` and
`GetPageStatusCode`.
* Add an `Into<tonic::Status>` implementation for `ProtocolError`.
2025-05-27 12:06:51 +00:00
Vlad Lazar
9657fbc194 pageserver: add and stabilize import chaos test (#11982)
## Problem

Test coverage of timeline imports is lacking.

## Summary of changes

This PR adds a chaos import test. It runs an import while injecting
various chaos events
in the environment. All the commits that follow the test fix various
issues that were surfaced by it.

Closes https://github.com/neondatabase/neon/issues/10191
2025-05-27 09:52:59 +00:00
Arpad Müller
3e86008e66 read-only timelines (#12015)
Support timeline creations on the storage controller to opt out from
their creation on the safekeepers, introducing the read-only timelines
concept. Read only timelines:

* will never receive WAL of their own, so it's fine to not create them
on the safekeepers
* the property is non-transitive. children of read-only timelines aren't
neccessarily read-only themselves.

This feature can be used for snapshots, to prevent the safekeepers from
being overloaded by empty timelines that won't ever get written to. In
the current world, this is not a problem, because timelines are created
implicitly by the compute connecting to a safekeeper that doesn't have
the timeline yet. In the future however, where the storage controller
creates timelines eagerly, we should watch out for that.

We represent read-only timelines in the storage controller database so
that we ensure that they never touch the safekeepers at all. Especially
we don't want them to cause a mess during the importing process of the
timelines from the cplane to the storcon database.

In a hypothetical future where we have a feature to detach timelines
from safekeepers, we'll either need to find a way to distinguish the
two, or if not, asking safekeepers to list the (empty) timeline prefix
and delete everything from it isn't a big issue either.

This patch will unconditionally hit the new safekeeper timeline creation
path for read-only timelines, without them needing the
`--timelines-onto-safekeepers` flag enabled. This is done because it's
lower risk (no safekeepers or computes involved at all) and gives us
some initial way to verify at least some parts of that code in prod.

https://github.com/neondatabase/cloud/issues/29435
https://github.com/neondatabase/neon/issues/11670
2025-05-26 23:23:58 +00:00
Alex Chi Z.
dc953de85d feat(pageserver): integrate PostHog with gc-compaction rollout (#11917)
## Problem

part of https://github.com/neondatabase/neon/issues/11813

## Summary of changes

* Integrate feature store with tenant structure.
* gc-compaction picks up the current strategy from the feature store.
* We only log them for now for testing purpose. They will not be used
until we have more patches to support different strategies defined in
PostHog.
* We don't support property-based evaulation for now; it will be
implemented later.
* Evaluating result of the feature flag is not cached -- it's not
efficient and cannot be used on hot path right now.
* We don't report the evaluation result back to PostHog right now.

I plan to enable it in staging once we get the patch merged.

---------

Signed-off-by: Alex Chi Z <chi@neon.tech>
2025-05-26 13:09:37 +00:00
Alex Chi Z.
841517ee37 fix(pageserver): do not increase basebackup err counter when reconnect (#12016)
## Problem

We see unexpected basebackup error alerts in the alert channel.

https://github.com/neondatabase/neon/pull/11778 only fixed the alerts
for shutdown errors. However, another path is that tenant shutting down
while waiting LSN -> WaitLsnError::BadState -> QueryError::Reconnect.
Therefore, the reconnect error should also be discarded from the
ok/error counter.

## Summary of changes

Do not increase ok/err counter for reconnect errors.

---------

Signed-off-by: Alex Chi Z <chi@neon.tech>
2025-05-26 11:31:27 +00:00
Erik Grinaker
7cd0defaf0 page_api: add Rust domain types (#11999)
## Problem

For the gRPC Pageserver API, we should convert the Protobuf types to
stricter, canonical Rust types.

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

## Summary of changes

Adds Rust domain types that mirror the Protobuf types, with conversion
and validation.
2025-05-26 11:01:36 +00:00
Erik Grinaker
a082f9814a pageserver: add gRPC authentication (#12010)
## Problem

We need authentication for the gRPC server.

Requires #11972.
Touches #11728.

## Summary of changes

Add two request interceptors that decode the tenant/timeline/shard
metadata and authenticate the JWT token against them.
2025-05-26 10:24:45 +00:00
Erik Grinaker
ec991877f4 pageserver: add gRPC server (#11972)
## Problem

We want to expose the page service over gRPC, for use with the
communicator.

Requires #11995.
Touches #11728.

## Summary of changes

This patch wires up a gRPC server in the Pageserver, using Tonic. It
does not yet implement the actual page service.

* Adds `listen_grpc_addr` and `grpc_auth_type` config options (disabled
by default).
* Enables gRPC by default with `neon_local`.
* Stub implementation of `page_api.PageService`, returning unimplemented
errors.
* gRPC reflection service for use with e.g. `grpcurl`.

Subsequent PRs will implement the actual page service, including
authentication and observability.

Notably, TLS support is not yet implemented. Certificate reloading
requires us to reimplement the entire Tonic gRPC server.
2025-05-26 08:27:48 +00:00
Dmitrii Kovalkov
136eaeb74a pageserver: basebackup cache (hackathon project) (#11989)
## Problem
Basebackup cache is on the hot path of compute startup and is generated
on every request (may be slow).

- Issue: https://github.com/neondatabase/cloud/issues/29353

## Summary of changes
- Add `BasebackupCache` which stores basebackups on local disk.
- Basebackup prepare requests are triggered by
`XLOG_CHECKPOINT_SHUTDOWN` records in the log.
- Limit the size of the cache by number of entries.
- Add `basebackup_cache_enabled` feature flag to TenantConfig.
- Write tests for the cache

## Not implemented yet
- Limit the size of the cache by total size in bytes

---------

Co-authored-by: Aleksandr Sarantsev <aleksandr@neon.tech>
2025-05-22 12:45:00 +00:00
Erik Grinaker
211b824d62 pageserver: add branch-local consumption metrics (#11852)
## Problem

For billing, we'd like per-branch consumption metrics.

Requires https://github.com/neondatabase/neon/pull/11984.
Resolves https://github.com/neondatabase/cloud/issues/28155.

## Summary of changes

This patch adds two new consumption metrics:

* `written_size_since_parent`: `written_size - ancestor_lsn`
* `pitr_history_size_since_parent`: `written_size - max(pitr_cutoff,
ancestor_lsn)`

Note that `pitr_history_size_since_parent` will not be emitted until the
PITR cutoff has been computed, and may or may not increase ~immediately
when a user increases their PITR window (depending on how much history
we have available and whether the tenant is restarted/migrated).
2025-05-22 12:26:32 +00:00
Erik Grinaker
95a5f749c8 pageserver: use an Option for GcCutoffs::time (#11984)
## Problem

It is not currently possible to disambiguate a timeline with an
uninitialized PITR cutoff from one that was created within the PITR
window -- both of these have `GcCutoffs::time == Lsn(0)`. For billing
metrics, we need to disambiguate these to avoid accidentally billing the
entire history when a tenant is initially loaded.

Touches https://github.com/neondatabase/cloud/issues/28155.

## Summary of changes

Make `GcCutoffs::time` an `Option<Lsn>`, and only set it to `Some` when
initialized. A `pitr_interval` of 0 will yield `Some(last_record_lsn)`.

This PR takes a conservative approach, and mostly retains the old
behavior of consumers by using `unwrap_or_default()` to yield 0 when
uninitialized, to avoid accidentally introducing bugs -- except in cases
where there is high confidence that the change is beneficial (e.g. for
the `pageserver_pitr_history_size` Prometheus metric and to return early
during GC).
2025-05-21 15:42:11 +00:00
Arpad Müller
136cf1979b Add metric for number of offloaded timelines (#11976)
We want to keep track of the number of offloaded timelines. It's a
per-tenant shard metric because each shard makes offloading decisions on
its own.
2025-05-21 11:28:22 +00:00
Vlad Lazar
08bb72e516 pageserver: allow in-mem reads to be planned during writes (#11937)
## Problem

Get page tracing revealed situations where planning an in-memory layer
is taking around 150ms. Upon investigation, the culprit is the inner
in-mem layer file lock. A batch being written holds the write lock and a
read being planned wants the read lock. See [this
trace](https://neonprod.grafana.net/explore?schemaVersion=1&panes=%7B%22j61%22:%7B%22datasource%22:%22JMfY_5TVz%22,%22queries%22:%5B%7B%22refId%22:%22traceId%22,%22queryType%22:%22traceql%22,%22query%22:%22412ec4522fe1750798aca54aec2680ac%22,%22datasource%22:%7B%22type%22:%22tempo%22,%22uid%22:%22JMfY_5TVz%22%7D,%22limit%22:20,%22tableType%22:%22traces%22,%22metricsQueryType%22:%22range%22%7D%5D,%22range%22:%7B%22to%22:%221746702606349%22,%22from%22:%221746681006349%22%7D,%22panelsState%22:%7B%22trace%22:%7B%22spanId%22:%2291e9f1879c9bccc0%22%7D%7D%7D,%226d0%22:%7B%22datasource%22:%22JMfY_5TVz%22,%22queries%22:%5B%7B%22refId%22:%22traceId%22,%22queryType%22:%22traceql%22,%22query%22:%2220a4757706b16af0e1fbab83f9d2e925%22,%22datasource%22:%7B%22type%22:%22tempo%22,%22uid%22:%22JMfY_5TVz%22%7D,%22limit%22:20,%22tableType%22:%22traces%22,%22metricsQueryType%22:%22range%22%7D%5D,%22range%22:%7B%22to%22:%221746702614807%22,%22from%22:%221746681014807%22%7D,%22panelsState%22:%7B%22trace%22:%7B%22spanId%22:%2260e7825512bc2a6b%22%7D%7D%7D%7D)
for example.

## Summary of changes

Lift the index into its own RwLock such that we can at least plan during
write IO.

I tried to be smarter in
https://github.com/neondatabase/neon/pull/11866: arc swap + structurally
shared datastructure
and that killed ingest perf for small keys.

## Benchmarking

* No statistically significant difference for rust inget benchmarks when
compared to main.
2025-05-21 11:08:49 +00:00
Alexander Sarantcev
6f4f3691a5 pageserver: Add tracing endpoint correctness check in config validation (#11970)
## Problem

When using an incorrect endpoint string - `"localhost:4317"`, it's a
runtime error, but it can be a config error
- Closes: https://github.com/neondatabase/neon/issues/11394

## Summary of changes

Add config parse time check via `request::Url::parse` validation.

---------

Co-authored-by: Aleksandr Sarantsev <ephemeralsad@gmail.com>
2025-05-21 09:03:26 +00:00
Konstantin Knizhnik
2e3dc9a8c2 Add rel_size_replica_cache (#11889)
## Problem

See 
Discussion:
https://neondb.slack.com/archives/C033RQ5SPDH/p1746645666075799
Issue: https://github.com/neondatabase/cloud/issues/28609

Relation size cache is not correctly updated at PS in case of replicas.

## Summary of changes

1. Have two caches for relation size in timeline:
`rel_size_primary_cache` and `rel_size_replica_cache`.
2. `rel_size_primary_cache` is actually what we have now. The only
difference is that it is not updated in `get_rel_size`, only by WAL
ingestion
3. `rel_size_replica_cache` has limited size (LruCache) and it's key is
`(Lsn,RelTag)` . It is updated in `get_rel_size`. Only strict LSN
matches are accepted as cache hit.

---------

Co-authored-by: Konstantin Knizhnik <knizhnik@neon.tech>
2025-05-20 15:38:27 +00:00
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