## Problem
We hold the layer map for too long on occasion.
## Summary of changes
This should help us identify the places where it's happening from.
Related https://github.com/neondatabase/neon/issues/12182
## Problem
There is this TODO in code:
https://github.com/neondatabase/neon/blob/main/pageserver/src/tenant/mgr.rs#L300-L302
This is an old TODO by @jcsp.
## Summary of changes
This PR addresses the TODO. Specifically, it removes a global static
`TENANTS`. Instead the `TenantManager` now directly manages the tenant
map. Enhancing abstraction.
Essentially, this PR moves all module-level methods to inside the
implementation of `TenantManager`.
## Problem
part of https://github.com/neondatabase/neon/issues/11813
## Summary of changes
Collect pageserver hostname property so that we can use it in the
PostHog UI. Not sure if this is the best way to do that -- open to
suggestions.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
We are getting tenants with a lot of branches and num of timelines is a
good indicator of pageserver loads. I added this metrics to help us
better plan pageserver capacities.
## Summary of changes
Add `pageserver_timeline_states_count` with two labels: active +
offloaded.
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
If a shard split fails and must roll back, the tenant may hit a cold
start as the parent shard's files have already been removed from local
disk.
External contribution with minor adjustments, see
https://neondb.slack.com/archives/C08TE3203RQ/p1748246398269309.
## Summary of changes
Keep the parent shard's files on local disk until the split has been
committed, such that they are available if the spilt is rolled back. If
all else fails, the files will be removed on the next Pageserver
restart.
This should also be fine in a mixed version:
* New storcon, old Pageserver: the Pageserver will delete the files
during the split, storcon will log an error when the cleanup detach
fails.
* Old storcon, new Pageserver: the Pageserver will leave the parent's
files around until the next Pageserver restart.
The change looks good to me, but shard splits are delicate so I'd like
some extra eyes on this.
## Problem
- `test_basebackup_cache` fails in
https://github.com/neondatabase/neon/pull/11712 because after the
timelines on safekeepers are managed by storage controller, they do
contain proper start_lsn and the compute_ctl tool sends the first
basebackup request with this LSN.
- `Failed to prepare basebackup` log messages during timeline
initialization, because the timeline is not yet in the global timeline
map.
- Relates to https://github.com/neondatabase/cloud/issues/29353
## Summary of changes
- Account for `timeline_onto_safekeepers` storcon's option in the test.
- Do not trigger basebackup prepare during the timeline initialization.
## Problem
The new gRPC page service protocol supports client-side batches. The
current libpq protocol only does best-effort server-side batching.
To compare these approaches, Pagebench should support submitting
contiguous page batches, similar to how Postgres will submit them (e.g.
with prefetches or vectored reads).
## Summary of changes
Add a `--batch-size` parameter specifying the size of contiguous page
batches. One batch counts as 1 RPS and 1 queue depth.
For the libpq protocol, a batch is submitted as individual requests and
we rely on the server to batch them for us. This will give a realistic
comparison of how these would be processed in the wild (e.g. when
Postgres sends 100 prefetch requests).
This patch also adds some basic validation of responses.
## Problem
We support two ingest protocols on the pageserver: vanilla and
interpreted.
Interpreted has been the only protocol in use for a long time.
## Summary of changes
* Remove the ingest handling of the vanilla protocol
* Remove tenant and pageserver configuration for it
* Update all tests that tweaked the ingest protocol
## Compatibility
Backward compatibility:
* The new pageserver version can read the existing pageserver
configuration and it will ignore the unknown field.
* When the tenant config is read from the storcon db or from the
pageserver disk, the extra field will be ignored.
Forward compatiblity:
* Both the pageserver config and the tenant config map missing fields to
their default value.
I'm not aware of any tenant level override that was made for this knob.
## Problem
Part of https://github.com/neondatabase/neon/issues/11813
In PostHog UI, we need to create the properties before using them as a
filter. We report all variants automatically when we start the
pageserver. In the future, we can report all real tenants instead of
fake tenants (we do that now to save money + we don't need real tenants
in the UI).
## Summary of changes
* Collect `region`, `availability_zone`, `pageserver_id` properties and
use them in the feature evaluation.
* Report 10 fake tenants on each pageserver startup.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
The gRPC page service doesn't respect `get_vectored_concurrent_io` and
always uses sequential IO.
## Summary of changes
Spawn a sidecar task for concurrent IO when enabled.
Cancellation will be addressed separately.
## Problem
We want to repro an OOM situation, but large partial reads are required.
## Summary of Changes
Make the max partial read size configurable for import jobs.
## Problem
Part of https://github.com/neondatabase/neon/issues/11813
## Summary of changes
Add a counter on the feature evaluation outcome and we will set up
alerts for too many failed evaluations in the future.
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
Currently, `page_api` domain types validate message invariants both when
converting Protobuf → domain and domain → Protobuf. This is annoying for
clients, because they can't use stream combinators to convert streamed
requests (needed for hot path performance), and also performs the
validation twice in the common case.
Blocks #12099.
## Summary of changes
Only validate the Protobuf → domain type conversion, i.e. on the
receiver side, and make domain → Protobuf infallible. This is where it
matters -- the Protobuf types are less strict than the domain types, and
receivers should expect all sorts of junk from senders (they're not
required to validate anyway, and can just construct an invalid message
manually).
Also adds a missing `impl From<CheckRelExistsRequest> for
proto::CheckRelExistsRequest`.
## Problem
Setting `max_batch_size` to anything higher than
`Timeline::MAX_GET_VECTORED_KEYS` will cause runtime error. We should
rather fail fast at startup if this is the case.
## Summary of changes
* Create `max_get_vectored_keys` as a new configuration (default to 32);
* Validate `max_batch_size` against `max_get_vectored_keys` right at
config parsing and validation.
Closes https://github.com/neondatabase/neon/issues/11994
## Problem
The gRPC `page_api` domain types used smallvecs to avoid heap
allocations in the common case where a single page is requested.
However, this is pointless: the Protobuf types use a normal vec, and
converting a smallvec into a vec always causes a heap allocation anyway.
## Summary of changes
Use a normal `Vec` instead of a `SmallVec` in `page_api` domain types.
## Problem
We print a backtrace in an info level log every 10 seconds while waiting
for the import data to land in the bucket.
## Summary of changes
The backtrace is not useful. Remove it.
## Problem
fix https://github.com/neondatabase/neon/issues/12101; this is a quick
hack and we need better API in the future.
In `get_db_size`, we call `get_reldir_size` for every relation. However,
we do the same deserializing the reldir directory thing for every
relation. This creates huge CPU overhead.
## Summary of changes
Get and deserialize the reldir v1 key once and use it across all
get_rel_size requests.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
We should expose the page service over gRPC.
Requires #12093.
Touches #11728.
## Summary of changes
This patch adds an initial page service implementation over gRPC. It
ties in with the existing `PageServerHandler` request logic, to avoid
the implementations drifting apart for the core read path.
This is just a bare-bones functional implementation. Several important
aspects have been omitted, and will be addressed in follow-up PRs:
* Limited observability: minimal tracing, no logging, limited metrics
and timing, etc.
* Rate limiting will currently block.
* No performance optimization.
* No cancellation handling.
* No tests.
I've only done rudimentary testing of this, but Pagebench passes at
least.
## Problem
We need gRPC support in Pagebench to benchmark the new gRPC Pageserver
implementation.
Touches #11728.
## Summary of changes
Adds a `Client` trait to make the client transport swappable, and a gRPC
client via a `--protocol grpc` parameter. This must also specify the
connstring with the gRPC port:
```
pagebench get-page-latest-lsn --protocol grpc --page-service-connstring grpc://localhost:51051
```
The client is implemented using the raw Tonic-generated gRPC client, to
minimize client overhead.
## Problem
The page service logic asserts that a tracing span is present with
tenant/timeline/shard IDs. An initial gRPC page service implementation
thus requires a tracing span.
Touches https://github.com/neondatabase/neon/issues/11728.
## Summary of changes
Adds an `ObservabilityLayer` middleware that generates a tracing span
and decorates it with IDs from the gRPC metadata.
This is a minimal implementation to address the tracing span assertion.
It will be extended with additional observability in later PRs.
## Problem
If the wal receiver is cancelled, there's a 50% chance that it will
ingest yet more WAL.
## Summary of Changes
Always check cancellation first.
The expected operating range for the production NVMe drives is
in the range of 50 to 250us.
The bucket boundaries before this PR were not well suited
to reason about the utilization / queuing / latency variability
of those devices.
# Performance
There was some concern about perf impact of having so many buckets,
considering the impl does a linear search on each observe().
I added a benchmark and measured on relevant machines.
In any way, the PR is 40 buckets, so, won't make a meaningful
difference on production machines (im4gn.2xlarge),
going from 30ns -> 35ns.
## Problem
There's a bunch of TODOs in the import code.
## Summary of changes
1. Bound max import byte range to 128MiB. This might still be too high,
given the default job concurrency, but it needs to be balanced with
going back and forth to S3.
2. Prevent unsigned overflow when determining key range splits for
concurrent jobs
3. Use sharded ranges to estimate task size when splitting jobs
4. Bubble up errors that we might hit due to invalid data in the bucket
back to the storage controller.
5. Tweak the import bucket S3 client configuration.
This patch is a fixup for
- https://github.com/neondatabase/neon/pull/6788
Background
----------
That PR 6788 added artificial advancement of `disk_consistent_lsn`
and `remote_consistent_lsn` for shards that weren't written to
while other shards _were_ written to.
See the PR description for more context.
At the time of that PR, Pageservers shards were doing WAL filtering.
Nowadays, the WAL filtering happens in Safekeepers.
Shards learn about the WAL gaps via
`InterpretedWalRecords::next_record_lsn`.
The Bug
-------
That artificial advancement code also runs if the flush failed.
So, we advance the disk_consistent_lsn / remote_consistent_lsn,
without having the corresponding L0 to the `index_part.json`.
The frozen layer remains in the layer map until detach,
so we continue to serve data correctly.
We're not advancing flush loop variable `flushed_to_lsn` either,
so, subsequent flush requests will retry the flush and repair the
situation if they succeed.
But if there aren't any successful retries, eventually the tenant
will be detached and when it is attached somewhere else, the
`index_part.json` and therefore layer map...
1. ... does not contain the frozen layer that failed to flush and
2. ... won't re-ingest that WAL either because walreceiver
starts up with the advanced disk_consistent_lsn/remote_consistent_lsn.
The result is that the read path will have a gap in the reconstruct
data for the keys whose modifications were lost, resulting in
a) either walredo failure
b) or an incorrect page@lsn image if walredo doesn't error.
The Fix
-------
The fix is to only do the artificial advancement if `result.is_ok()`.
Misc
----
As an aside, I took some time to re-review the flush loop and its
callers.
I found one more bug related to error handling that I filed here:
- https://github.com/neondatabase/neon/issues/12025
## Problem
## Summary of changes
## Problem
Part of #11813. This pull request adds misc observability improvements
for the functionality.
## Summary of changes
* Info span for the PostHog feature background loop.
* New evaluate feature flag API.
* Put the request error into the error message.
* Log when feature flag gets updated.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## 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.
## 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
## 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>
## 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
## 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>
## 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
## 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>
## 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

## 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`.
## 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
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/29435https://github.com/neondatabase/neon/issues/11670
## 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>
## 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>
## 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.
## 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.
## 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.
## 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>
## 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).
## 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).
## 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>