Compare commits

...

18 Commits

Author SHA1 Message Date
Konstantin Knizhnik
efe6f63eba Use keepalives for compute to PS connection 2025-01-21 12:19:23 +02:00
Arpad Müller
2ab9f69825 Simplify pageserver_physical_gc function (#10104)
This simplifies the code in `pageserver_physical_gc` a little bit after
the feedback in #10007 that the code is too complicated.

Most importantly, we don't pass around `GcSummary` any more in a
complicated fashion, and we save on async stream-combinator-inception in
one place in favour of `try_stream!{}`.

Follow-up of #10007
2025-01-20 21:57:15 +00:00
Alex Chi Z.
2de2b26c62 feat(pageserver): add reldir migration configs (#10439)
## Problem

Part of #9516 per RFC at https://github.com/neondatabase/neon/pull/10412

## Summary of changes

Adding the necessary config items and index_part items for the large
relation count work.

---------

Signed-off-by: Alex Chi Z <chi@neon.tech>
2025-01-20 20:44:12 +00:00
Matthias van de Meent
e781cf6dd8 Compute/LFC: Apply limits consistently (#10449)
Otherwise we might hit ERRORs in otherwise safe situations (such as user
queries), which isn't a great user experience.

## Problem

https://github.com/neondatabase/neon/pull/10376

## Summary of changes

Instead of accepting internal errors as acceptable, we ensure we don't
exceed our allocated usage.
2025-01-20 18:29:21 +00:00
Christian Schwarz
72130d7d6c fix(page_service / handle): panic when parallel client disconnect & Timeline shutdown (#10445)
## Refs
- fixes https://github.com/neondatabase/neon/issues/10444

## Problem

We're seeing a panic `handles are only shut down once in their lifetime`
in our performance testbed.

## Hypothesis

Annotated code in
https://github.com/neondatabase/neon/issues/10444#issuecomment-2602286415.

```
T1: drop Cache, executes up to (1)
=> HandleInner is now in state ShutDown
T2: Timeline::shutdown => PerTimelineState::shutdown  executes shutdown() again => panics
```

Likely this snuck in the final touches of #10386 where I narrowed down
the locking rules.

## Summary of changes

Make duplicate shutdowns a no-op.
2025-01-20 17:51:30 +00:00
John Spray
2657b7ec75 rfcs: add sharded ingest RFC (#8754)
## Summary

Whereas currently we send all WAL to all pageserver shards, and each
shard filters out the data that it needs,
in this RFC we add a mechanism to filter the WAL on the safekeeper, so
that each shard receives
only the data it needs.

This will place some extra CPU load on the safekeepers, in exchange for
reducing the network bandwidth
for ingesting WAL back to scaling as O(1) with shard count, rather than
O(N_shards).

Touches #9329.

## Checklist before requesting a review

- [ ] I have performed a self-review of my code.
- [ ] If it is a core feature, I have added thorough tests.
- [ ] Do we need to implement analytics? if so did you add the relevant
metrics to the dashboard?
- [ ] If this PR requires public announcement, mark it with
/release-notes label and add several sentences in this section.

## Checklist before merging

- [ ] Do not forget to reformat commit message to not include the above
checklist

---------

Co-authored-by: Vlad Lazar <vlalazar.vlad@gmail.com>
Co-authored-by: Vlad Lazar <vlad@neon.tech>
2025-01-20 17:33:07 +00:00
Christian Schwarz
02fc58b878 impr(timeline handles): add more tests covering reference cyle (#10446)
The other test focus on the external interface usage while the tests
added in this PR add some testing around HandleInner's lifecycle,
ensuring we don't leak it once either connection gets dropped or
per-timeline-state is shut down explicitly.
2025-01-20 14:37:24 +00:00
Arpad Müller
b312a3c320 Move DeleteTimelineFlow::prepare to separate function and use enum (#10334)
It was requested by review in #10305 to use an enum or something like it
for distinguishing the different modes instead of two parameters,
because two flags allow four combinations, and two of them don't really
make sense/ aren't used.

follow-up of #10305
2025-01-20 12:50:44 +00:00
John Spray
7d761a9d22 storage controller: make chaos less disruptive to AZ locality (#10438)
## Problem

Since #9916 , the chaos code is actively fighting the optimizer: tenants
tend to be attached in their preferred AZ, so most chaos migrations were
moving them to a non-preferred AZ.

## Summary of changes

- When picking migrations, prefer to migrate things _toward_ their
preferred AZ when possible. Then pick shards to move the other way when
necessary.

The resulting behavior should be an alternating "back and forth" where
the chaos code migrates thiings away from home, and then migrates them
back on the next iteration.

The side effect will be that the chaos code actively helps to push
things into their home AZ. That's not contrary to its purpose though: we
mainly just want it to continuously migrate things to exercise
migration+notification code.
2025-01-20 09:47:23 +00:00
John Spray
8bdaee35f3 pageserver: safety checks on validity of uploaded indices (#10403)
## Problem

Occasionally, we encounter bugs in test environments that can be
detected at the point of uploading an index, but we proceed to upload it
anyway and leave a tenant in a broken state that's awkward to handle.

## Summary of changes

- Validate index when submitting it for upload, so that we can see the
issue quickly e.g. in an API invoking compaction
- Validate index before executing the upload, so that we have a hard
enforcement that any code path that tries to upload an index will not
overwrite a valid index with an invalid one.
2025-01-20 09:20:31 +00:00
Arpad Müller
b0f34099f9 Add safekeeper utilization endpoint (#10429)
Add an endpoint to obtain the utilization of a safekeeper. Future
changes to the storage controller can use this endpoint to find the most
suitable safekeepers for newly created timelines, analogously to how
it's done for pageservers already.

Initially we just want to assign by timeline count, then we can iterate
from there.

Part of https://github.com/neondatabase/neon/issues/9011
2025-01-17 21:43:52 +00:00
Vlad Lazar
6975228a76 pageserver: add initdb metrics (#10434)
## Problem

Initdb observability is poor.

## Summary of changes

Add some metrics so we can figure out which part, if any, is slow.

Closes https://github.com/neondatabase/neon/issues/10423
2025-01-17 14:51:33 +00:00
JC Grünhage
053abff71f Fix dependency on neon-image in promote-images-dev (#10437)
## Problem
871e8b325f failed CI on main because a job
ran to soon. This was caused by
ea84ec357f. While `promote-images-dev`
does not inherently need `neon-image`, a few jobs depending on
`promote-images-dev` do need it, and previously had it when it was
`promote-images`, which depended on `test-images`, which in turn
depended on `neon-image`.

## Summary of changes
To ensure jobs depending `docker.io/neondatabase/neon` images get them,
`promote-images-dev` gets the dependency to `neon-image` back which it
previously had transitively through `test-images`.
2025-01-17 14:21:30 +00:00
Tristan Partin
871e8b325f Use the request ID given by the control plane in compute_ctl (#10418)
Instead of generating our own request ID, we can just use the one
provided by the control plane. In the event, we get a request from a
client which doesn't set X-Request-ID, then we just generate one which
is useful for tracing purposes.

Signed-off-by: Tristan Partin <tristan@neon.tech>
2025-01-16 20:46:53 +00:00
Christian Schwarz
c47c5f4ace fix(page_service pipelining): tenant cannot shut down because gate kept open while flushing responses (#10386)
# Refs

- fixes https://github.com/neondatabase/neon/issues/10309
- fixup of batching design, first introduced in
https://github.com/neondatabase/neon/pull/9851
- refinement of https://github.com/neondatabase/neon/pull/8339

# Problem

`Tenant::shutdown` was occasionally taking many minutes (sometimes up to
20) in staging and prod if the
`page_service_pipelining.mode="concurrent-futures"` is enabled.

# Symptoms

The issue happens during shard migration between pageservers.
There is page_service unavailability and hence effectively downtime for
customers in the following case:
1. The source (state `AttachedStale`) gets stuck in `Tenant::shutdown`,
waiting for the gate to close.
2. Cplane/Storcon decides to transition the target `AttachedMulti` to
`AttachedSingle`.
3. That transition comes with a bump of the generation number, causing
the `PUT .../location_config` endpoint to do a full `Tenant::shutdown` /
`Tenant::attach` cycle for the target location.
4. That `Tenant::shutdown` on the target gets stuck, waiting for the
gate to close.
5. Eventually the gate closes (`close completed`), correlating with a
`page_service` connection handler logging that it's exiting because of a
network error (`Connection reset by peer` or `Broken pipe`).

While in (4):
- `Tenant::shutdown` is stuck waiting for all `Timeline::shutdown` calls
to complete.
  So, really, this is a `Timeline::shutdown` bug.
- retries from Cplane/Storcon to complete above state transitions, fail
with errors related to the tenant mgr slot being in state
`TenantSlot::InProgress`, the tenant state being
`TenantState::Stopping`, and the timelines being in
`TimelineState::Stopping`, and the `Timeline::cancel` being cancelled.
- Existing (and/or new?) page_service connections log errors `error
reading relation or page version: Not found: Timed out waiting 30s for
tenant active state. Latest state: None`

# Root-Cause

After a lengthy investigation ([internal
write-up](https://www.notion.so/neondatabase/2025-01-09-batching-deadlock-Slow-Log-Analysis-in-Staging-176f189e00478050bc21c1a072157ca4?pvs=4))
I arrived at the following root cause.

The `spsc_fold` channel (`batch_tx`/`batch_rx`) that connects the
Batcher and Executor stages of the pipelined mode was storing a `Handle`
and thus `GateGuard` of the Timeline that was not shutting down.
The design assumption with pipelining was that this would always be a
short transient state.
However, that was incorrect: the Executor was stuck on writing/flushing
an earlier response into the connection to the client, i.e., socket
write being slow because of TCP backpressure.

The probable scenario of how we end up in that case:
1. Compute backend process sends a continuous stream of getpage prefetch
requests into the connection, but never reads the responses (why this
happens: see Appendix section).
2. Batch N is processed by Batcher and Executor, up to the point where
Executor starts flushing the response.
3. Batch N+1 is procssed by Batcher and queued in the `spsc_fold`.
4. Executor is still waiting for batch N flush to finish.
5. Batcher eventually hits the `TimeoutReader` error (10min).
From here on it waits on the
`spsc_fold.send(Err(QueryError(TimeoutReader_error)))`
which will never finish because the batch already inside the `spsc_fold`
is not
being read by the Executor, because the Executor is still stuck in the
flush.
   (This state is not observable at our default `info` log level)
6. Eventually, Compute backend process is killed (`close()` on the
socket) or Compute as a whole gets killed (probably no clean TCP
shutdown happening in that case).
7. Eventually, Pageserver TCP stack learns about (6) through RST packets
and the Executor's flush() call fails with an error.
8. The Executor exits, dropping `cancel_batcher` and its end of the
spsc_fold.
   This wakes Batcher, causing the `spsc_fold.send` to fail.
   Batcher exits.
   The pipeline shuts down as intended.
We return from `process_query` and log the `Connection reset by peer` or
`Broken pipe` error.

The following diagram visualizes the wait-for graph at (5)

```mermaid
flowchart TD
   Batcher --spsc_fold.send(TimeoutReader_error)--> Executor
   Executor --flush batch N responses--> socket.write_end
   socket.write_end --wait for TCP window to move forward--> Compute
```

# Analysis

By holding the GateGuard inside the `spsc_fold` open, the pipelining
implementation
violated the principle established in
(https://github.com/neondatabase/neon/pull/8339).
That is, that `Handle`s must only be held across an await point if that
await point
is sensitive to the `<Handle as Deref<Target=Timeline>>::cancel` token.

In this case, we were holding the Handle inside the `spsc_fold` while
awaiting the
`pgb_writer.flush()` future.

One may jump to the conclusion that we should simply peek into the
spsc_fold to get
that Timeline cancel token and be sensitive to it during flush, then.

But that violates another principle of the design from
https://github.com/neondatabase/neon/pull/8339.
That is, that the page_service connection lifecycle and the Timeline
lifecycles must be completely decoupled.
Tt must be possible to shut down one shard without shutting down the
page_service connection, because on that single connection we might be
serving other shards attached to this pageserver.
(The current compute client opens separate connections per shard, but,
there are plans to change that.)

# Solution

This PR adds a `handle::WeakHandle` struct that does _not_ hold the
timeline gate open.
It must be `upgrade()`d to get a `handle::Handle`.
That `handle::Handle` _does_ hold the timeline gate open.

The batch queued inside the `spsc_fold` only holds a `WeakHandle`.
We only upgrade it while calling into the various `handle_` methods,
i.e., while interacting with the `Timeline` via `<Handle as
Deref<Target=Timeline>>`.
All that code has always been required to be (and is!) sensitive to
`Timeline::cancel`, and therefore we're guaranteed to bail from it
quickly when `Timeline::shutdown` starts.
We will drop the `Handle` immediately, before we start
`pgb_writer.flush()`ing the responses.
Thereby letting go of our hold on the `GateGuard`, allowing the timeline
shutdown to complete while the page_service handler remains intact.

# Code Changes

* Reproducer & Regression Test
* Developed and proven to reproduce the issue in
https://github.com/neondatabase/neon/pull/10399
* Add a `Test` message to the pagestream protocol (`cfg(feature =
"testing")`).
* Drive-by minimal improvement to the parsing code, we now have a
`PagestreamFeMessageTag`.
* Refactor `pageserver/client` to allow sending and receiving
`page_service` requests independently.
  * Add a Rust helper binary to produce situation (4) from above
* Rationale: (4) and (5) are the same bug class, we're holding a gate
open while `flush()`ing.
* Add a Python regression test that uses the helper binary to
demonstrate the problem.
* Fix
   * Introduce and use `WeakHandle` as explained earlier.
* Replace the `shut_down` atomic with two enum states for `HandleInner`,
wrapped in a `Mutex`.
* To make `WeakHandle::upgrade()` and `Handle::downgrade()`
cache-efficient:
     * Wrap the `Types::Timeline` in an `Arc`
     * Wrap the `GateGuard` in an `Arc`
* The separate `Arc`s enable uncontended cloning of the timeline
reference in `upgrade()` and `downgrade()`.
If instead we were `Arc<Timeline>::clone`, different connection handlers
would be hitting the same cache line on every upgrade()/downgrade(),
causing contention.
* Please read the udpated module-level comment in `mod handle`
module-level comment for details.

# Testing & Performance

The reproducer test that failed before the changes now passes, and
obviously other tests are passing as well.

We'll do more testing in staging, where the issue happens every ~4h if
chaos migrations are enabled in storcon.

Existing perf testing will be sufficient, no perf degradation is
expected.
It's a few more alloctations due to the added Arc's, but, they're low
frequency.

# Appendix: Why Compute Sometimes Doesn't Read Responses

Remember, the whole problem surfaced because flush() was slow because
Compute was not reading responses. Why is that?

In short, the way the compute works, it only advances the page_service
protocol processing when it has an interest in data, i.e., when the
pagestore smgr is called to return pages.

Thus, if compute issues a bunch of requests as part of prefetch but then
it turns out it can service the query without reading those pages, it
may very well happen that these messages stay in the TCP until the next
smgr read happens, either in that session, or possibly in another
session.

If there’s too many unread responses in the TCP, the pageserver kernel
is going to backpressure into userspace, resulting in our stuck flush().

All of this stems from the way vanilla Postgres does prefetching and
"async IO":
it issues `fadvise()` to make the kernel do the IO in the background,
buffering results in the kernel page cache.
It then consumes the results through synchronous `read()` system calls,
which hopefully will be fast because of the `fadvise()`.

If it turns out that some / all of the prefetch results are not needed,
Postgres will not be issuing those `read()` system calls.
The kernel will eventually react to that by reusing page cache pages
that hold completed prefetched data.
Uncompleted prefetch requests may or may not be processed -- it's up to
the kernel.

In Neon, the smgr + Pageserver together take on the role of the kernel
in above paragraphs.
In the current implementation, all prefetches are sent as GetPage
requests to Pageserver.
The responses are only processed in the places where vanilla Postgres
would do the synchronous `read()` system call.
If we never get to that, the responses are queued inside the TCP
connection, which, once buffers run full, will backpressure into
Pageserver's sending code, i.e., the `pgb_writer.flush()` that was the
root cause of the problems we're fixing in this PR.
2025-01-16 20:34:02 +00:00
Tristan Partin
b0838a68e5 Enable pgx_ulid on Postgres 17 (#10397)
The extension now supports Postgres 17. The release also seems to be
binary compatible with the previous version.

Signed-off-by: Tristan Partin <tristan@neon.tech>
2025-01-16 19:49:04 +00:00
John Spray
8f2ebc0684 tests: stabilize test_storage_controller_node_deletion (#10420)
## Problem

`test_storage_controller_node_deletion` sometimes failed because shards
were moving around during timeline creation, and neon_local isn't
tolerant of that. The movements were unexpected because the shards had
only just been created.

This was a regression from #9916

Closes: #10383 

## Summary of changes

- Make this test use multiple AZs -- this makes the storage controller's
scheduling reliably stable

Why this works: in #9916 , I made a simplifying assumption that we would
have multiple AZs to get nice stable scheduling -- it's much easier,
because each tenant has a well defined primary+secondary location when
they have an AZ preference and nodes have different AZs. Everything
still works if you don't have multiple AZs, but you just have this quirk
that sometimes the optimizer can disagree with initial scheduling, so
once in a while a shard moves after being created -- annoying for tests,
harmless IRL.
2025-01-16 19:00:16 +00:00
Vlad Lazar
3a285a046b pageserver: include node id when subscribing to SK (#10432)
## Problem

All pageserver have the same application name which makes it hard to
distinguish them.

## Summary of changes

Include the node id in the application name sent to the safekeeper. This
should gives us
more visibility in logs. There's a few metrics that will increase in
cardinality by `pageserver_count`,
but that's fine.
2025-01-16 18:51:56 +00:00
42 changed files with 2060 additions and 535 deletions

View File

@@ -824,7 +824,7 @@ jobs:
docker compose -f ./docker-compose/docker-compose.yml down
promote-images-dev:
needs: [ check-permissions, tag, vm-compute-node-image ]
needs: [ check-permissions, tag, vm-compute-node-image, neon-image ]
runs-on: ubuntu-22.04
permissions:

1
Cargo.lock generated
View File

@@ -1312,6 +1312,7 @@ dependencies = [
"tracing-utils",
"url",
"utils",
"uuid",
"vm_monitor",
"workspace_hack",
"zstd",

View File

@@ -995,24 +995,50 @@ RUN wget https://github.com/kelvich/pg_tiktoken/archive/9118dd4549b7d8c0bbc98e04
#########################################################################################
#
# Layer "pg-pgx-ulid-build"
# Compile "pgx_ulid" extension
# Compile "pgx_ulid" extension for v16 and below
#
#########################################################################################
FROM rust-extensions-build AS pg-pgx-ulid-build
ARG PG_VERSION
# doesn't support v17 yet
# https://github.com/pksunkara/pgx_ulid/pull/52
RUN case "${PG_VERSION}" in "v17") \
echo "pgx_ulid does not support pg17 as of the latest version (0.1.5)" && exit 0;; \
RUN case "${PG_VERSION}" in \
"v14" | "v15" | "v16") \
;; \
*) \
echo "skipping the version of pgx_ulid for $PG_VERSION" && exit 0 \
;; \
esac && \
wget https://github.com/pksunkara/pgx_ulid/archive/refs/tags/v0.1.5.tar.gz -O pgx_ulid.tar.gz && \
echo "9d1659a2da65af0133d5451c454de31b37364e3502087dadf579f790bc8bef17 pgx_ulid.tar.gz" | sha256sum --check && \
echo "9d1659a2da65af0133d5451c454de31b37364e3502087dadf579f790bc8bef17 pgx_ulid.tar.gz" | sha256sum --check && \
mkdir pgx_ulid-src && cd pgx_ulid-src && tar xzf ../pgx_ulid.tar.gz --strip-components=1 -C . && \
sed -i 's/pgrx = "^0.11.2"/pgrx = { version = "=0.11.3", features = [ "unsafe-postgres" ] }/g' Cargo.toml && \
sed -i 's/pgrx = "^0.11.2"/pgrx = { version = "0.11.3", features = [ "unsafe-postgres" ] }/g' Cargo.toml && \
cargo pgrx install --release && \
echo "trusted = true" >> /usr/local/pgsql/share/extension/ulid.control
echo 'trusted = true' >> /usr/local/pgsql/share/extension/ulid.control
#########################################################################################
#
# Layer "pg-pgx-ulid-pgrx12-build"
# Compile "pgx_ulid" extension for v17 and up
#
#########################################################################################
FROM rust-extensions-build-pgrx12 AS pg-pgx-ulid-pgrx12-build
ARG PG_VERSION
RUN case "${PG_VERSION}" in \
"v17") \
;; \
*) \
echo "skipping the version of pgx_ulid for $PG_VERSION" && exit 0 \
;; \
esac && \
wget https://github.com/pksunkara/pgx_ulid/archive/refs/tags/v0.2.0.tar.gz -O pgx_ulid.tar.gz && \
echo "cef6a9a2e5e7bd1a10a18989286586ee9e6c1c06005a4055cff190de41bf3e9f pgx_ulid.tar.gz" | sha256sum --check && \
mkdir pgx_ulid-src && cd pgx_ulid-src && tar xzf ../pgx_ulid.tar.gz --strip-components=1 -C . && \
sed -i 's/pgrx = "^0.12.7"/pgrx = { version = "0.12.9", features = [ "unsafe-postgres" ] }/g' Cargo.toml && \
cargo pgrx install --release && \
echo 'trusted = true' >> /usr/local/pgsql/share/extension/pgx_ulid.control
#########################################################################################
#
@@ -1157,6 +1183,7 @@ COPY --from=timescaledb-pg-build /usr/local/pgsql/ /usr/local/pgsql/
COPY --from=pg-hint-plan-pg-build /usr/local/pgsql/ /usr/local/pgsql/
COPY --from=pg-cron-pg-build /usr/local/pgsql/ /usr/local/pgsql/
COPY --from=pg-pgx-ulid-build /usr/local/pgsql/ /usr/local/pgsql/
COPY --from=pg-pgx-ulid-pgrx12-build /usr/local/pgsql/ /usr/local/pgsql/
COPY --from=pg-session-jwt-build /usr/local/pgsql/ /usr/local/pgsql/
COPY --from=rdkit-pg-build /usr/local/pgsql/ /usr/local/pgsql/
COPY --from=pg-uuidv7-pg-build /usr/local/pgsql/ /usr/local/pgsql/

View File

@@ -51,6 +51,7 @@ tracing-subscriber.workspace = true
tracing-utils.workspace = true
thiserror.workspace = true
url.workspace = true
uuid.workspace = true
prometheus.workspace = true
postgres_initdb.workspace = true

View File

@@ -1,15 +1,14 @@
use std::{
net::{IpAddr, Ipv6Addr, SocketAddr},
sync::{
atomic::{AtomicU64, Ordering},
Arc,
},
sync::Arc,
thread,
time::Duration,
};
use anyhow::Result;
use axum::{
extract::Request,
middleware::{self, Next},
response::{IntoResponse, Response},
routing::{get, post},
Router,
@@ -17,11 +16,9 @@ use axum::{
use http::StatusCode;
use tokio::net::TcpListener;
use tower::ServiceBuilder;
use tower_http::{
request_id::{MakeRequestId, PropagateRequestIdLayer, RequestId, SetRequestIdLayer},
trace::TraceLayer,
};
use tower_http::{request_id::PropagateRequestIdLayer, trace::TraceLayer};
use tracing::{debug, error, info, Span};
use uuid::Uuid;
use super::routes::{
check_writability, configure, database_schema, dbs_and_roles, extension_server, extensions,
@@ -34,30 +31,24 @@ async fn handle_404() -> Response {
StatusCode::NOT_FOUND.into_response()
}
#[derive(Clone, Default)]
struct ComputeMakeRequestId(Arc<AtomicU64>);
const X_REQUEST_ID: &str = "x-request-id";
impl MakeRequestId for ComputeMakeRequestId {
fn make_request_id<B>(
&mut self,
_request: &http::Request<B>,
) -> Option<tower_http::request_id::RequestId> {
let request_id = self
.0
.fetch_add(1, Ordering::SeqCst)
.to_string()
.parse()
.unwrap();
/// This middleware function allows compute_ctl to generate its own request ID
/// if one isn't supplied. The control plane will always send one as a UUID. The
/// neon Postgres extension on the other hand does not send one.
async fn maybe_add_request_id_header(mut request: Request, next: Next) -> Response {
let headers = request.headers_mut();
Some(RequestId::new(request_id))
if headers.get(X_REQUEST_ID).is_none() {
headers.append(X_REQUEST_ID, Uuid::new_v4().to_string().parse().unwrap());
}
next.run(request).await
}
/// Run the HTTP server and wait on it forever.
#[tokio::main]
async fn serve(port: u16, compute: Arc<ComputeNode>) {
const X_REQUEST_ID: &str = "x-request-id";
let mut app = Router::new()
.route("/check_writability", post(check_writability::is_writable))
.route("/configure", post(configure::configure))
@@ -82,9 +73,8 @@ async fn serve(port: u16, compute: Arc<ComputeNode>) {
.fallback(handle_404)
.layer(
ServiceBuilder::new()
.layer(SetRequestIdLayer::x_request_id(
ComputeMakeRequestId::default(),
))
// Add this middleware since we assume the request ID exists
.layer(middleware::from_fn(maybe_add_request_id_header))
.layer(
TraceLayer::new_for_http()
.on_request(|request: &http::Request<_>, _span: &Span| {

View File

@@ -418,6 +418,11 @@ impl PageServerNode {
.map(serde_json::from_str)
.transpose()
.context("parse `wal_receiver_protocol_override` from json")?,
rel_size_v2_enabled: settings
.remove("rel_size_v2_enabled")
.map(|x| x.parse::<bool>())
.transpose()
.context("Failed to parse 'rel_size_v2_enabled' as bool")?,
};
if !settings.is_empty() {
bail!("Unrecognized tenant settings: {settings:?}")

View File

@@ -0,0 +1,255 @@
#
Created on Aug 2024
Implemented on Jan 2025
## Summary
Data in large tenants is split up between multiple pageservers according to key hashes, as
introduced in the [sharding RFC](031-sharding-static.md) and [shard splitting RFC](032-shard-splitting.md).
Whereas currently we send all WAL to all pageserver shards, and each shard filters out the data that it needs,
in this RFC we add a mechanism to filter the WAL on the safekeeper, so that each shard receives
only the data it needs.
This will place some extra CPU load on the safekeepers, in exchange for reducing the network bandwidth
for ingesting WAL back to scaling as O(1) with shard count, rather than O(N_shards).
## Motivation
1. Large databases require higher shard counts. Whereas currently we run with up to 8 shards for tenants
with a few TB of storage, the next order of magnitude capacity increase will require tens of shards, such
that sending all WAL to all shards is impractical in terms of bandwidth.
2. For contemporary database sizes (~2TB), the pageserver is the bottleneck for ingest: since each
shard has to decode and process the whole WAL, sharding doesn't fully relieve this bottleneck. To achieve significantly higher ingest speeds, we need to filter the WAL earlier so that each pageserver
only has to process relevant parts.
## Non Goals (if relevant)
We do not seek to introduce multiple WALs per timeline, or to share the work of handling a timeline's
WAL across safekeepers (beyond simple 3x replication). This RFC may be thought of as an incremental
move of the ingestion bottleneck up the stack: instead of high write rates bottlenecking on the
pageserver, they will bottleneck on the safekeeper.
## Impacted components (e.g. pageserver, safekeeper, console, etc)
Safekeeper, pageserver.
There will be no control plane or storage controller coordination needed, as pageservers will directly
indicate their sharding parameters to the safekeeper when subscribing for WAL.
## Proposed implementation
Terminology:
- "Data pages" refers to postgres relation blocks, and SLRU blocks.
- "Metadata pages" refers to everything else the pageserver stores, such as relation sizes and
directories of relations.
### Phase 1: Refactor ingest
Currently, pageserver ingest code is structured approximately as follows:
1. `handle_walreceiver_connection` reads a stream of binary WAL records off a network
socket
2. `WalIngest::ingest_record` to translate the record into a series of page-level modifications
3. `DatadirModification` accumulates page updates from several `ingest_record` calls, and when
its `commit()` method is called, flushes these into a Timeline's open `InMemoryLayer`.
This process currently assumes access to a pageserver `Timeline` throughout `ingest_record` and
from `DatadirModification`, which is used to do read-modify-write cycles on metadata pages
such as relation sizes and the master DBDIR page. It also assumes that records are ingested
strictly one after the other: they cannot be ingested in parallel because each record assumes
that earlier records' changes have already been applied to `Timeline`.
This code will be refactored to disentangle the simple, fast decode of relation page writes
from the more complex logic for updating internal metadata. An intermediate representation
called `InterpretedWalRecords` will be introduced. This is similar to the internal state of
a `DatadirModification`, but does not require access to a Timeline. Instead of storing
metadata updates as materialized writes to pages, it will accumulate these as abstract operations,
for example rather than including a write to a relation size key, this structure will include
an operation that indicates "Update relation _foo_'s size to the max of its current value and
_bar_", such that these may be applied later to a real Timeline.
The `DatadirModification` will be aware of the `EphemeralFile` format, so that as it accumulates
simple page writes of relation blocks, it can write them directly into a buffer in the serialized
format. This will avoid the need to later deserialize/reserialize this data when passing the
structure between safekeeper and pageserver.
The new pipeline will be:
1. `handle_walreceiver_connection` reads a stream of binary WAL records off a network
2. A `InterpretedWalRecords` is generated from the incoming WAL records. This does not
require a reference to a Timeline.
3. The logic that is current spread between `WalIngest` and `DatadirModification` for updating
metadata will be refactored to consume the metadata operations from the `InterpretedWalRecords`
and turn them into literal writes to metadata pages. This part must be done sequentially.
4. The resulting buffer of metadata page writes is combined with the buffer of relation block
writes, and written into the `InMemoryLayer`.
Implemented in:
1. https://github.com/neondatabase/neon/pull/9472
2. https://github.com/neondatabase/neon/pull/9504
3. https://github.com/neondatabase/neon/pull/9524
### Phase 2: Decode & filter on safekeeper
In the previous phase, the ingest code was modified to be able to do most of its work without access to
a Timeline: this first stage of ingest simply converts a series of binary wal records into
a buffer of relation/SLRU page writes, and a buffer of abstract metadata writes.
The modified ingest code may be transplanted from pageserver to safekeeper (probably via a
shared crate). The safekeeper->pageserver network protocol is modified to:
- in subscription requests, send the `ShardIdentity` from the pageserver to the safekeeper
- in responses, transmit a `InterpretedWalRecords` instead of a raw `WalRecord`.
- use the `ShardIdentity` to filter the `ProcessedWalIngest` to relevant content for
the subscribing shard before transmitting it.
The overall behavior of the pageserver->safekeeper interaction remains the same, in terms of
consistent LSN feedback, and connection management. Only the payload of the subscriptions
changes, to express an LSN range of WAL as a filtered `ProcessedWalIngest` instead of the
raw data.
The ingest code on the pageserver can now skip the part where it does the first phase of
processing, as it will receive pre-processed, compressed data off the wire.
Note that `InterpretedWalRecord` batches multiple `InterpretedWalRecord(s)` in the same network
message. Safekeeper reads WAL in chunks of 16 blocks and then decodes as many Postgres WAL records
as possible. Each Postgres WAL record maps to one `InterpretedWalRecord` for potentially multiple shards.
Hence, the size of the batch is given by the number of Postgres WAL records that fit in 16 blocks.
The protocol needs to support evolution. Protobuf was chosen here with the view that, in the future,
we may migrate it to GRPC altogether
Implemented in:
1. https://github.com/neondatabase/neon/pull/9746
2. https://github.com/neondatabase/neon/pull/9821
### Phase 3: Fan out interpreted WAL
In the previous phase, the initial processing of WAL was moved to the safekeeper, but it is still
done once for each shard: this will generate O(N_shards) CPU work on the safekeeper (especially
when considering converting to Protobuf format and compression).
To avoid this, we fan-out WAL from one (tenant, timeline, shard) to all other shards subscribed on
the same safekeeper. Under normal operation, the WAL will be read from disk, decoded and interpreted
_only_ once per (safekeeper, timeline).
When the first shard of a sharded timeline subscribes to a given safekeeper a task is spawned
for the WAL reader (`InterpretedWalReader`). This task reads WAL, decodes, interprets it and sends
it to the sender (`InterpretedWalSender`). The sender is a future that is polled from the connection
task. When further shards subscribe on the safekeeper they will attach themselves to the existing WAL reader.
There's two cases to consider:
1. The shard's requested `start_lsn` is ahead of the current position of the WAL reader. In this case, the shard
will start receiving data when the reader reaches that LSN. The intuition here is that there's little to gain
by letting shards "front-run" since compute backpressure is based on the laggard LSN.
2. The shard's requested `start_lsn` is below the current position of the WAL reader. In this case, the WAL reader
gets reset to this requested position (same intuition). Special care is taken such that advanced shards do not receive
interpreted WAL records below their current position.
The approach above implies that there is at most one WAL reader per (tenant, timeline) on a given safekeeper at any point in time.
If this turns out to be operationally problematic, there's a trick we can deploy: `--max-delta-for-fanout` is an optional safekeeper
argument that controls the max absolute delta between a new shard and the current WAL position of the WAL reader. If the absolute
delta is above that value, a new reader is spawned. Note that there's currently no concurrency control on the number of WAL readers,
so it's recommended to use large values to avoid pushing CPU utilisation too high.
Unsharded tenants do not spawn a separate task for the interpreted WAL reader since there's no benefit to it. Instead they poll
the reader and sender concurrently from the connection task.
Shard splits are interesting here because it is the only case when the same shard might have two subscriptions at the same time.
This is handled by giving readers a unique identifier. Both shards will receive the same data while respecting their requested start
position.
Implemented in:
1. https://github.com/neondatabase/neon/pull/10190
## Deployment
Each phase shall be deployed independently. Special care should be taken around protocol changes.
## Observability Tips
* The safekeeper logs the protocol requested by the pageserver
along with the pageserver ID, tenant, timeline and shard: `starting streaming from`.
* There's metrics for the number of wal readers:
* `safekeeper_wal_readers{kind="task", target=~"pageserver.*"}` gives the number of wal reader tasks for each SK
* `safekeeper_wal_readers{kind="future", target=~"pageserver.*"}` gives the numer of wal readers polled inline by each SK
* `safekeeper_interpreted_wal_reader_tasks` gives the number of wal reader tasks per tenant, timeline
* Interesting log lines for the fan-out reader:
* `Spawning interpreted`: first shard creates the interpreted wal reader
* `Fanning out`: a subsequent shard attaches itself to an interpreted wal reader
* `Aborting interpreted`: all senders have finished and the reader task is being aborted
## Future Optimizations
This sections describes some improvement areas which may be revisited in the future.
### Buffering of Interpreted WAL
The interpreted WAL reader may buffer interpreted WAL records in user space to help with serving
subscriptions that are lagging behind the current position of the reader.
Counterpoints:
* Safekeepers serve many thousands of timelines and allocating a buffer for each might be wasteful,
especially given that it would go unused on the happy path.
* WAL is buffered in the kernel page cache. Usually we'd only pay the CPU cost of decoding and interpreting.
### Tweaking the Pagserver Safekeeper Selection Algorithm
We could make the pageserver aware of which safekeeper's already host shards for the timeline along
with their current WAL positions. The pageserver should then prefer safkeepers that are in the same
AZ _and_ already have a shard with a position close to the desired start position.
We currently run one safekeeper per AZ, so the point is mute until that changes.
### Pipelining first ingest phase
The first ingest phase is a stateless transformation of a binary WAL record into a pre-processed
output per shard. To put multiple CPUs to work, we may pipeline this processing up to some defined buffer
depth.
## Alternatives considered
### Give safekeepers enough state to fully decode WAL
In this RFC, we only do the first phase of ingest on the safekeeper, because this is
the phase that is stateless. Subsequent changes then happen on the pageserver, with
access to the `Timeline` state.
We could do more work on the safekeeper if we transmitted metadata state to the safekeeper
when subscribing to the WAL: for example, by telling the safekeeper all the relation sizes,
so that it could then generate all the metadata writes for relation sizes.
We avoid doing this for several reasons:
1. Complexity: it's a more invasive protocol change
2. Decoupling: having the safekeeper understand the `ProcessedWalIngest` already somewhat
infects it with knowledge of the pageserver, but this is mainly an abstract structure
that describes postgres writes. However, if we taught the safekeeper about the exact
way that pageserver deals with metadata keys, this would be a much tighter coupling.
3. Load: once the WAL has been processed to the point that it can be split between shards,
it is preferable to share out work on the remaining shards rather than adding extra CPU
load to the safekeeper.
### Do pre-processing on the compute instead of the safekeeper
Since our first stage of ingest is stateless, it could be done at any stage in the pipeline,
all the way up to the compute.
We choose not to do this, because it is useful for the safekeeper to store the raw WAL rather
than just the preprocessed WAL:
- The safekeeper still needs to be able to serve raw WAL back to postgres for e.g. physical replication
- It simplifies our paxos implementation to have the offset in the write log be literally
the same as the LSN
- Raw WAL must have a stable protocol since we might have to re-ingest it at arbitrary points in the future.
Storing raw WAL give us more flexibility to evolve the pageserver, safekeeper protocol.
### Do wal pre-processing on shard 0 or a separate service, send it to other shards from there
If we wanted to keep the safekeepers as entirely pure stores of raw WAL bytes, then
we could do the initial decode and shard-splitting in some other location:
- Shard 0 could subscribe to the full WAL and then send writes to other shards
- A new intermediate service between the safekeeper and pageserver could do the splitting.
So why not?
- Extra network hop from shard 0 to the final destination shard
- Clearly there is more infrastructure involved here compared with doing it inline on the safekeeper.
- Safekeepers already have very light CPU load: typical cloud instances shapes with appropriate
disks for the safekeepers effectively have "free" CPU resources.
- Doing extra work on shard 0 would complicate scheduling of shards on pageservers, because
shard 0 would have significantly higher CPU load under write workloads than other shards.

View File

@@ -301,6 +301,10 @@ pub struct TenantConfigToml {
pub timeline_offloading: bool,
pub wal_receiver_protocol_override: Option<PostgresClientProtocol>,
/// Enable rel_size_v2 for this tenant. Once enabled, the tenant will persist this information into
/// `index_part.json`, and it cannot be reversed.
pub rel_size_v2_enabled: Option<bool>,
}
pub mod defaults {
@@ -538,6 +542,7 @@ impl Default for TenantConfigToml {
lsn_lease_length_for_ts: LsnLease::DEFAULT_LENGTH_FOR_TS,
timeline_offloading: false,
wal_receiver_protocol_override: None,
rel_size_v2_enabled: None,
}
}
}

View File

@@ -33,7 +33,6 @@ use crate::{
reltag::RelTag,
shard::{ShardCount, ShardStripeSize, TenantShardId},
};
use anyhow::bail;
use bytes::{Buf, BufMut, Bytes, BytesMut};
/// The state of a tenant in this pageserver.
@@ -498,6 +497,8 @@ pub struct TenantConfigPatch {
pub timeline_offloading: FieldPatch<bool>,
#[serde(skip_serializing_if = "FieldPatch::is_noop")]
pub wal_receiver_protocol_override: FieldPatch<PostgresClientProtocol>,
#[serde(skip_serializing_if = "FieldPatch::is_noop")]
pub rel_size_v2_enabled: FieldPatch<bool>,
}
/// An alternative representation of `pageserver::tenant::TenantConf` with
@@ -529,6 +530,7 @@ pub struct TenantConfig {
pub lsn_lease_length_for_ts: Option<String>,
pub timeline_offloading: Option<bool>,
pub wal_receiver_protocol_override: Option<PostgresClientProtocol>,
pub rel_size_v2_enabled: Option<bool>,
}
impl TenantConfig {
@@ -558,6 +560,7 @@ impl TenantConfig {
mut lsn_lease_length_for_ts,
mut timeline_offloading,
mut wal_receiver_protocol_override,
mut rel_size_v2_enabled,
} = self;
patch.checkpoint_distance.apply(&mut checkpoint_distance);
@@ -602,6 +605,7 @@ impl TenantConfig {
patch
.wal_receiver_protocol_override
.apply(&mut wal_receiver_protocol_override);
patch.rel_size_v2_enabled.apply(&mut rel_size_v2_enabled);
Self {
checkpoint_distance,
@@ -628,6 +632,7 @@ impl TenantConfig {
lsn_lease_length_for_ts,
timeline_offloading,
wal_receiver_protocol_override,
rel_size_v2_enabled,
}
}
}
@@ -1400,6 +1405,8 @@ pub enum PagestreamFeMessage {
GetPage(PagestreamGetPageRequest),
DbSize(PagestreamDbSizeRequest),
GetSlruSegment(PagestreamGetSlruSegmentRequest),
#[cfg(feature = "testing")]
Test(PagestreamTestRequest),
}
// Wrapped in libpq CopyData
@@ -1411,6 +1418,22 @@ pub enum PagestreamBeMessage {
Error(PagestreamErrorResponse),
DbSize(PagestreamDbSizeResponse),
GetSlruSegment(PagestreamGetSlruSegmentResponse),
#[cfg(feature = "testing")]
Test(PagestreamTestResponse),
}
// Keep in sync with `pagestore_client.h`
#[repr(u8)]
enum PagestreamFeMessageTag {
Exists = 0,
Nblocks = 1,
GetPage = 2,
DbSize = 3,
GetSlruSegment = 4,
/* future tags above this line */
/// For testing purposes, not available in production.
#[cfg(feature = "testing")]
Test = 99,
}
// Keep in sync with `pagestore_client.h`
@@ -1422,7 +1445,28 @@ enum PagestreamBeMessageTag {
Error = 103,
DbSize = 104,
GetSlruSegment = 105,
/* future tags above this line */
/// For testing purposes, not available in production.
#[cfg(feature = "testing")]
Test = 199,
}
impl TryFrom<u8> for PagestreamFeMessageTag {
type Error = u8;
fn try_from(value: u8) -> Result<Self, u8> {
match value {
0 => Ok(PagestreamFeMessageTag::Exists),
1 => Ok(PagestreamFeMessageTag::Nblocks),
2 => Ok(PagestreamFeMessageTag::GetPage),
3 => Ok(PagestreamFeMessageTag::DbSize),
4 => Ok(PagestreamFeMessageTag::GetSlruSegment),
#[cfg(feature = "testing")]
99 => Ok(PagestreamFeMessageTag::Test),
_ => Err(value),
}
}
}
impl TryFrom<u8> for PagestreamBeMessageTag {
type Error = u8;
fn try_from(value: u8) -> Result<Self, u8> {
@@ -1433,6 +1477,8 @@ impl TryFrom<u8> for PagestreamBeMessageTag {
103 => Ok(PagestreamBeMessageTag::Error),
104 => Ok(PagestreamBeMessageTag::DbSize),
105 => Ok(PagestreamBeMessageTag::GetSlruSegment),
#[cfg(feature = "testing")]
199 => Ok(PagestreamBeMessageTag::Test),
_ => Err(value),
}
}
@@ -1550,6 +1596,20 @@ pub struct PagestreamDbSizeResponse {
pub db_size: i64,
}
#[cfg(feature = "testing")]
#[derive(Debug, PartialEq, Eq, Clone)]
pub struct PagestreamTestRequest {
pub hdr: PagestreamRequest,
pub batch_key: u64,
pub message: String,
}
#[cfg(feature = "testing")]
#[derive(Debug)]
pub struct PagestreamTestResponse {
pub req: PagestreamTestRequest,
}
// This is a cut-down version of TenantHistorySize from the pageserver crate, omitting fields
// that require pageserver-internal types. It is sufficient to get the total size.
#[derive(Serialize, Deserialize, Debug)]
@@ -1569,7 +1629,7 @@ impl PagestreamFeMessage {
match self {
Self::Exists(req) => {
bytes.put_u8(0);
bytes.put_u8(PagestreamFeMessageTag::Exists as u8);
bytes.put_u64(req.hdr.reqid);
bytes.put_u64(req.hdr.request_lsn.0);
bytes.put_u64(req.hdr.not_modified_since.0);
@@ -1580,7 +1640,7 @@ impl PagestreamFeMessage {
}
Self::Nblocks(req) => {
bytes.put_u8(1);
bytes.put_u8(PagestreamFeMessageTag::Nblocks as u8);
bytes.put_u64(req.hdr.reqid);
bytes.put_u64(req.hdr.request_lsn.0);
bytes.put_u64(req.hdr.not_modified_since.0);
@@ -1591,7 +1651,7 @@ impl PagestreamFeMessage {
}
Self::GetPage(req) => {
bytes.put_u8(2);
bytes.put_u8(PagestreamFeMessageTag::GetPage as u8);
bytes.put_u64(req.hdr.reqid);
bytes.put_u64(req.hdr.request_lsn.0);
bytes.put_u64(req.hdr.not_modified_since.0);
@@ -1603,7 +1663,7 @@ impl PagestreamFeMessage {
}
Self::DbSize(req) => {
bytes.put_u8(3);
bytes.put_u8(PagestreamFeMessageTag::DbSize as u8);
bytes.put_u64(req.hdr.reqid);
bytes.put_u64(req.hdr.request_lsn.0);
bytes.put_u64(req.hdr.not_modified_since.0);
@@ -1611,13 +1671,24 @@ impl PagestreamFeMessage {
}
Self::GetSlruSegment(req) => {
bytes.put_u8(4);
bytes.put_u8(PagestreamFeMessageTag::GetSlruSegment as u8);
bytes.put_u64(req.hdr.reqid);
bytes.put_u64(req.hdr.request_lsn.0);
bytes.put_u64(req.hdr.not_modified_since.0);
bytes.put_u8(req.kind);
bytes.put_u32(req.segno);
}
#[cfg(feature = "testing")]
Self::Test(req) => {
bytes.put_u8(PagestreamFeMessageTag::Test as u8);
bytes.put_u64(req.hdr.reqid);
bytes.put_u64(req.hdr.request_lsn.0);
bytes.put_u64(req.hdr.not_modified_since.0);
bytes.put_u64(req.batch_key);
let message = req.message.as_bytes();
bytes.put_u64(message.len() as u64);
bytes.put_slice(message);
}
}
bytes.into()
@@ -1645,56 +1716,66 @@ impl PagestreamFeMessage {
),
};
match msg_tag {
0 => Ok(PagestreamFeMessage::Exists(PagestreamExistsRequest {
hdr: PagestreamRequest {
reqid,
request_lsn,
not_modified_since,
},
rel: RelTag {
spcnode: body.read_u32::<BigEndian>()?,
match PagestreamFeMessageTag::try_from(msg_tag)
.map_err(|tag: u8| anyhow::anyhow!("invalid tag {tag}"))?
{
PagestreamFeMessageTag::Exists => {
Ok(PagestreamFeMessage::Exists(PagestreamExistsRequest {
hdr: PagestreamRequest {
reqid,
request_lsn,
not_modified_since,
},
rel: RelTag {
spcnode: body.read_u32::<BigEndian>()?,
dbnode: body.read_u32::<BigEndian>()?,
relnode: body.read_u32::<BigEndian>()?,
forknum: body.read_u8()?,
},
}))
}
PagestreamFeMessageTag::Nblocks => {
Ok(PagestreamFeMessage::Nblocks(PagestreamNblocksRequest {
hdr: PagestreamRequest {
reqid,
request_lsn,
not_modified_since,
},
rel: RelTag {
spcnode: body.read_u32::<BigEndian>()?,
dbnode: body.read_u32::<BigEndian>()?,
relnode: body.read_u32::<BigEndian>()?,
forknum: body.read_u8()?,
},
}))
}
PagestreamFeMessageTag::GetPage => {
Ok(PagestreamFeMessage::GetPage(PagestreamGetPageRequest {
hdr: PagestreamRequest {
reqid,
request_lsn,
not_modified_since,
},
rel: RelTag {
spcnode: body.read_u32::<BigEndian>()?,
dbnode: body.read_u32::<BigEndian>()?,
relnode: body.read_u32::<BigEndian>()?,
forknum: body.read_u8()?,
},
blkno: body.read_u32::<BigEndian>()?,
}))
}
PagestreamFeMessageTag::DbSize => {
Ok(PagestreamFeMessage::DbSize(PagestreamDbSizeRequest {
hdr: PagestreamRequest {
reqid,
request_lsn,
not_modified_since,
},
dbnode: body.read_u32::<BigEndian>()?,
relnode: body.read_u32::<BigEndian>()?,
forknum: body.read_u8()?,
},
})),
1 => Ok(PagestreamFeMessage::Nblocks(PagestreamNblocksRequest {
hdr: PagestreamRequest {
reqid,
request_lsn,
not_modified_since,
},
rel: RelTag {
spcnode: body.read_u32::<BigEndian>()?,
dbnode: body.read_u32::<BigEndian>()?,
relnode: body.read_u32::<BigEndian>()?,
forknum: body.read_u8()?,
},
})),
2 => Ok(PagestreamFeMessage::GetPage(PagestreamGetPageRequest {
hdr: PagestreamRequest {
reqid,
request_lsn,
not_modified_since,
},
rel: RelTag {
spcnode: body.read_u32::<BigEndian>()?,
dbnode: body.read_u32::<BigEndian>()?,
relnode: body.read_u32::<BigEndian>()?,
forknum: body.read_u8()?,
},
blkno: body.read_u32::<BigEndian>()?,
})),
3 => Ok(PagestreamFeMessage::DbSize(PagestreamDbSizeRequest {
hdr: PagestreamRequest {
reqid,
request_lsn,
not_modified_since,
},
dbnode: body.read_u32::<BigEndian>()?,
})),
4 => Ok(PagestreamFeMessage::GetSlruSegment(
}))
}
PagestreamFeMessageTag::GetSlruSegment => Ok(PagestreamFeMessage::GetSlruSegment(
PagestreamGetSlruSegmentRequest {
hdr: PagestreamRequest {
reqid,
@@ -1705,7 +1786,21 @@ impl PagestreamFeMessage {
segno: body.read_u32::<BigEndian>()?,
},
)),
_ => bail!("unknown smgr message tag: {:?}", msg_tag),
#[cfg(feature = "testing")]
PagestreamFeMessageTag::Test => Ok(PagestreamFeMessage::Test(PagestreamTestRequest {
hdr: PagestreamRequest {
reqid,
request_lsn,
not_modified_since,
},
batch_key: body.read_u64::<BigEndian>()?,
message: {
let len = body.read_u64::<BigEndian>()?;
let mut buf = vec![0; len as usize];
body.read_exact(&mut buf)?;
String::from_utf8(buf)?
},
})),
}
}
}
@@ -1748,6 +1843,15 @@ impl PagestreamBeMessage {
bytes.put_u32((resp.segment.len() / BLCKSZ as usize) as u32);
bytes.put(&resp.segment[..]);
}
#[cfg(feature = "testing")]
Self::Test(resp) => {
bytes.put_u8(Tag::Test as u8);
bytes.put_u64(resp.req.batch_key);
let message = resp.req.message.as_bytes();
bytes.put_u64(message.len() as u64);
bytes.put_slice(message);
}
}
}
PagestreamProtocolVersion::V3 => {
@@ -1816,6 +1920,18 @@ impl PagestreamBeMessage {
bytes.put_u32((resp.segment.len() / BLCKSZ as usize) as u32);
bytes.put(&resp.segment[..]);
}
#[cfg(feature = "testing")]
Self::Test(resp) => {
bytes.put_u8(Tag::Test as u8);
bytes.put_u64(resp.req.hdr.reqid);
bytes.put_u64(resp.req.hdr.request_lsn.0);
bytes.put_u64(resp.req.hdr.not_modified_since.0);
bytes.put_u64(resp.req.batch_key);
let message = resp.req.message.as_bytes();
bytes.put_u64(message.len() as u64);
bytes.put_slice(message);
}
}
}
}
@@ -1958,6 +2074,28 @@ impl PagestreamBeMessage {
segment: segment.into(),
})
}
#[cfg(feature = "testing")]
Tag::Test => {
let reqid = buf.read_u64::<BigEndian>()?;
let request_lsn = Lsn(buf.read_u64::<BigEndian>()?);
let not_modified_since = Lsn(buf.read_u64::<BigEndian>()?);
let batch_key = buf.read_u64::<BigEndian>()?;
let len = buf.read_u64::<BigEndian>()?;
let mut msg = vec![0; len as usize];
buf.read_exact(&mut msg)?;
let message = String::from_utf8(msg)?;
Self::Test(PagestreamTestResponse {
req: PagestreamTestRequest {
hdr: PagestreamRequest {
reqid,
request_lsn,
not_modified_since,
},
batch_key,
message,
},
})
}
};
let remaining = buf.into_inner();
if !remaining.is_empty() {
@@ -1977,6 +2115,8 @@ impl PagestreamBeMessage {
Self::Error(_) => "Error",
Self::DbSize(_) => "DbSize",
Self::GetSlruSegment(_) => "GetSlruSegment",
#[cfg(feature = "testing")]
Self::Test(_) => "Test",
}
}
}

View File

@@ -277,3 +277,8 @@ pub struct TimelineTermBumpResponse {
pub previous_term: u64,
pub current_term: u64,
}
#[derive(Debug, Clone, Deserialize, Serialize)]
pub struct SafekeeperUtilization {
pub timeline_count: u64,
}

View File

@@ -8,7 +8,7 @@ license.workspace = true
default = []
# Enables test-only APIs, incuding failpoints. In particular, enables the `fail_point!` macro,
# which adds some runtime cost to run tests on outage conditions
testing = ["fail/failpoints", "pageserver_api/testing", "wal_decoder/testing"]
testing = ["fail/failpoints", "pageserver_api/testing", "wal_decoder/testing", "pageserver_client/testing"]
[dependencies]
anyhow.workspace = true
@@ -114,3 +114,7 @@ harness = false
[[bench]]
name = "upload_queue"
harness = false
[[bin]]
name = "test_helper_slow_client_reads"
required-features = [ "testing" ]

View File

@@ -4,6 +4,9 @@ version = "0.1.0"
edition.workspace = true
license.workspace = true
[features]
testing = [ "pageserver_api/testing" ]
[dependencies]
pageserver_api.workspace = true
thiserror.workspace = true

View File

@@ -1,6 +1,9 @@
use std::pin::Pin;
use std::sync::{Arc, Mutex};
use futures::SinkExt;
use futures::{
stream::{SplitSink, SplitStream},
SinkExt, StreamExt,
};
use pageserver_api::{
models::{
PagestreamBeMessage, PagestreamFeMessage, PagestreamGetPageRequest,
@@ -10,7 +13,6 @@ use pageserver_api::{
};
use tokio::task::JoinHandle;
use tokio_postgres::CopyOutStream;
use tokio_stream::StreamExt;
use tokio_util::sync::CancellationToken;
use utils::{
id::{TenantId, TimelineId},
@@ -62,15 +64,28 @@ impl Client {
.client
.copy_both_simple(&format!("pagestream_v3 {tenant_id} {timeline_id}"))
.await?;
let (sink, stream) = copy_both.split(); // TODO: actually support splitting of the CopyBothDuplex so the lock inside this split adaptor goes away.
let Client {
cancel_on_client_drop,
conn_task,
client: _,
} = self;
let shared = Arc::new(Mutex::new(PagestreamShared::ConnTaskRunning(
ConnTaskRunning {
cancel_on_client_drop,
conn_task,
},
)));
Ok(PagestreamClient {
copy_both: Box::pin(copy_both),
conn_task,
cancel_on_client_drop,
sink: PagestreamSender {
shared: shared.clone(),
sink,
},
stream: PagestreamReceiver {
shared: shared.clone(),
stream,
},
shared,
})
}
@@ -97,7 +112,28 @@ impl Client {
/// Create using [`Client::pagestream`].
pub struct PagestreamClient {
copy_both: Pin<Box<tokio_postgres::CopyBothDuplex<bytes::Bytes>>>,
shared: Arc<Mutex<PagestreamShared>>,
sink: PagestreamSender,
stream: PagestreamReceiver,
}
pub struct PagestreamSender {
#[allow(dead_code)]
shared: Arc<Mutex<PagestreamShared>>,
sink: SplitSink<tokio_postgres::CopyBothDuplex<bytes::Bytes>, bytes::Bytes>,
}
pub struct PagestreamReceiver {
#[allow(dead_code)]
shared: Arc<Mutex<PagestreamShared>>,
stream: SplitStream<tokio_postgres::CopyBothDuplex<bytes::Bytes>>,
}
enum PagestreamShared {
ConnTaskRunning(ConnTaskRunning),
ConnTaskCancelledJoinHandleReturnedOrDropped,
}
struct ConnTaskRunning {
cancel_on_client_drop: Option<tokio_util::sync::DropGuard>,
conn_task: JoinHandle<()>,
}
@@ -110,11 +146,11 @@ pub struct RelTagBlockNo {
impl PagestreamClient {
pub async fn shutdown(self) {
let Self {
copy_both,
cancel_on_client_drop: cancel_conn_task,
conn_task,
} = self;
// The `copy_both` contains internal channel sender, the receiver of which is polled by `conn_task`.
shared,
sink,
stream,
} = { self };
// The `copy_both` split into `sink` and `stream` contains internal channel sender, the receiver of which is polled by `conn_task`.
// When `conn_task` observes the sender has been dropped, it sends a `FeMessage::CopyFail` into the connection.
// (see https://github.com/neondatabase/rust-postgres/blob/2005bf79573b8add5cf205b52a2b208e356cc8b0/tokio-postgres/src/copy_both.rs#L56).
//
@@ -131,27 +167,77 @@ impl PagestreamClient {
//
// NB: page_service doesn't have a use case to exit the `pagestream` mode currently.
// => https://github.com/neondatabase/neon/issues/6390
let _ = cancel_conn_task.unwrap();
let ConnTaskRunning {
cancel_on_client_drop,
conn_task,
} = {
let mut guard = shared.lock().unwrap();
match std::mem::replace(
&mut *guard,
PagestreamShared::ConnTaskCancelledJoinHandleReturnedOrDropped,
) {
PagestreamShared::ConnTaskRunning(conn_task_running) => conn_task_running,
PagestreamShared::ConnTaskCancelledJoinHandleReturnedOrDropped => unreachable!(),
}
};
let _ = cancel_on_client_drop.unwrap();
conn_task.await.unwrap();
drop(copy_both);
// Now drop the split copy_both.
drop(sink);
drop(stream);
}
pub fn split(self) -> (PagestreamSender, PagestreamReceiver) {
let Self {
shared: _,
sink,
stream,
} = self;
(sink, stream)
}
pub async fn getpage(
&mut self,
req: PagestreamGetPageRequest,
) -> anyhow::Result<PagestreamGetPageResponse> {
let req = PagestreamFeMessage::GetPage(req);
let req: bytes::Bytes = req.serialize();
// let mut req = tokio_util::io::ReaderStream::new(&req);
let mut req = tokio_stream::once(Ok(req));
self.getpage_send(req).await?;
self.getpage_recv().await
}
self.copy_both.send_all(&mut req).await?;
pub async fn getpage_send(&mut self, req: PagestreamGetPageRequest) -> anyhow::Result<()> {
self.sink.getpage_send(req).await
}
let next: Option<Result<bytes::Bytes, _>> = self.copy_both.next().await;
pub async fn getpage_recv(&mut self) -> anyhow::Result<PagestreamGetPageResponse> {
self.stream.getpage_recv().await
}
}
impl PagestreamSender {
// TODO: maybe make this impl Sink instead for better composability?
pub async fn send(&mut self, msg: PagestreamFeMessage) -> anyhow::Result<()> {
let msg = msg.serialize();
self.sink.send_all(&mut tokio_stream::once(Ok(msg))).await?;
Ok(())
}
pub async fn getpage_send(&mut self, req: PagestreamGetPageRequest) -> anyhow::Result<()> {
self.send(PagestreamFeMessage::GetPage(req)).await
}
}
impl PagestreamReceiver {
// TODO: maybe make this impl Stream instead for better composability?
pub async fn recv(&mut self) -> anyhow::Result<PagestreamBeMessage> {
let next: Option<Result<bytes::Bytes, _>> = self.stream.next().await;
let next: bytes::Bytes = next.unwrap()?;
PagestreamBeMessage::deserialize(next)
}
let msg = PagestreamBeMessage::deserialize(next)?;
match msg {
pub async fn getpage_recv(&mut self) -> anyhow::Result<PagestreamGetPageResponse> {
let next: PagestreamBeMessage = self.recv().await?;
match next {
PagestreamBeMessage::GetPage(p) => Ok(p),
PagestreamBeMessage::Error(e) => anyhow::bail!("Error: {:?}", e),
PagestreamBeMessage::Exists(_)
@@ -160,7 +246,14 @@ impl PagestreamClient {
| PagestreamBeMessage::GetSlruSegment(_) => {
anyhow::bail!(
"unexpected be message kind in response to getpage request: {}",
msg.kind()
next.kind()
)
}
#[cfg(feature = "testing")]
PagestreamBeMessage::Test(_) => {
anyhow::bail!(
"unexpected be message kind in response to getpage request: {}",
next.kind()
)
}
}

View File

@@ -0,0 +1,65 @@
use std::{
io::{stdin, stdout, Read, Write},
time::Duration,
};
use clap::Parser;
use pageserver_api::models::{PagestreamRequest, PagestreamTestRequest};
use utils::{
id::{TenantId, TimelineId},
lsn::Lsn,
};
#[derive(clap::Parser)]
struct Args {
connstr: String,
tenant_id: TenantId,
timeline_id: TimelineId,
}
#[tokio::main]
async fn main() -> anyhow::Result<()> {
let Args {
connstr,
tenant_id,
timeline_id,
} = Args::parse();
let client = pageserver_client::page_service::Client::new(connstr).await?;
let client = client.pagestream(tenant_id, timeline_id).await?;
let (mut sender, _receiver) = client.split();
eprintln!("filling the pipe");
let mut msg = 0;
loop {
msg += 1;
let fut = sender.send(pageserver_api::models::PagestreamFeMessage::Test(
PagestreamTestRequest {
hdr: PagestreamRequest {
reqid: 0,
request_lsn: Lsn(23),
not_modified_since: Lsn(23),
},
batch_key: 42,
message: format!("message {}", msg),
},
));
let Ok(res) = tokio::time::timeout(Duration::from_secs(10), fut).await else {
eprintln!("pipe seems full");
break;
};
let _: () = res?;
}
let n = stdout().write(b"R")?;
assert_eq!(n, 1);
stdout().flush()?;
eprintln!("waiting for signal to tell us to exit");
let mut buf = [0u8; 1];
stdin().read_exact(&mut buf)?;
eprintln!("termination signal received, exiting");
anyhow::Ok(())
}

View File

@@ -100,6 +100,32 @@ pub(crate) static VEC_READ_NUM_LAYERS_VISITED: Lazy<Histogram> = Lazy::new(|| {
.expect("failed to define a metric")
});
pub(crate) static CONCURRENT_INITDBS: Lazy<UIntGauge> = Lazy::new(|| {
register_uint_gauge!(
"pageserver_concurrent_initdb",
"Number of initdb processes running"
)
.expect("failed to define a metric")
});
pub(crate) static INITDB_SEMAPHORE_ACQUISITION_TIME: Lazy<Histogram> = Lazy::new(|| {
register_histogram!(
"pageserver_initdb_semaphore_seconds_global",
"Time spent getting a permit from the global initdb semaphore",
STORAGE_OP_BUCKETS.into()
)
.expect("failed to define metric")
});
pub(crate) static INITDB_RUN_TIME: Lazy<Histogram> = Lazy::new(|| {
register_histogram!(
"pageserver_initdb_seconds_global",
"Time spent performing initdb",
STORAGE_OP_BUCKETS.into()
)
.expect("failed to define metric")
});
// Metrics collected on operations on the storage repository.
#[derive(
Clone, Copy, enum_map::Enum, strum_macros::EnumString, strum_macros::Display, IntoStaticStr,
@@ -1463,6 +1489,8 @@ pub enum SmgrQueryType {
GetPageAtLsn,
GetDbSize,
GetSlruSegment,
#[cfg(feature = "testing")]
Test,
}
pub(crate) struct SmgrQueryTimePerTimeline {

View File

@@ -555,37 +555,52 @@ struct BatchedGetPageRequest {
timer: SmgrOpTimer,
}
#[cfg(feature = "testing")]
struct BatchedTestRequest {
req: models::PagestreamTestRequest,
timer: SmgrOpTimer,
}
/// NB: we only hold [`timeline::handle::WeakHandle`] inside this enum,
/// so that we don't keep the [`Timeline::gate`] open while the batch
/// is being built up inside the [`spsc_fold`] (pagestream pipelining).
enum BatchedFeMessage {
Exists {
span: Span,
timer: SmgrOpTimer,
shard: timeline::handle::Handle<TenantManagerTypes>,
shard: timeline::handle::WeakHandle<TenantManagerTypes>,
req: models::PagestreamExistsRequest,
},
Nblocks {
span: Span,
timer: SmgrOpTimer,
shard: timeline::handle::Handle<TenantManagerTypes>,
shard: timeline::handle::WeakHandle<TenantManagerTypes>,
req: models::PagestreamNblocksRequest,
},
GetPage {
span: Span,
shard: timeline::handle::Handle<TenantManagerTypes>,
shard: timeline::handle::WeakHandle<TenantManagerTypes>,
effective_request_lsn: Lsn,
pages: smallvec::SmallVec<[BatchedGetPageRequest; 1]>,
},
DbSize {
span: Span,
timer: SmgrOpTimer,
shard: timeline::handle::Handle<TenantManagerTypes>,
shard: timeline::handle::WeakHandle<TenantManagerTypes>,
req: models::PagestreamDbSizeRequest,
},
GetSlruSegment {
span: Span,
timer: SmgrOpTimer,
shard: timeline::handle::Handle<TenantManagerTypes>,
shard: timeline::handle::WeakHandle<TenantManagerTypes>,
req: models::PagestreamGetSlruSegmentRequest,
},
#[cfg(feature = "testing")]
Test {
span: Span,
shard: timeline::handle::WeakHandle<TenantManagerTypes>,
requests: Vec<BatchedTestRequest>,
},
RespondError {
span: Span,
error: BatchedPageStreamError,
@@ -606,6 +621,12 @@ impl BatchedFeMessage {
page.timer.observe_execution_start(at);
}
}
#[cfg(feature = "testing")]
BatchedFeMessage::Test { requests, .. } => {
for req in requests {
req.timer.observe_execution_start(at);
}
}
BatchedFeMessage::RespondError { .. } => {}
}
}
@@ -735,7 +756,7 @@ impl PageServerHandler {
BatchedFeMessage::Exists {
span,
timer,
shard,
shard: shard.downgrade(),
req,
}
}
@@ -754,7 +775,7 @@ impl PageServerHandler {
BatchedFeMessage::Nblocks {
span,
timer,
shard,
shard: shard.downgrade(),
req,
}
}
@@ -773,7 +794,7 @@ impl PageServerHandler {
BatchedFeMessage::DbSize {
span,
timer,
shard,
shard: shard.downgrade(),
req,
}
}
@@ -792,7 +813,7 @@ impl PageServerHandler {
BatchedFeMessage::GetSlruSegment {
span,
timer,
shard,
shard: shard.downgrade(),
req,
}
}
@@ -844,6 +865,7 @@ impl PageServerHandler {
)
.await?;
// We're holding the Handle
let effective_request_lsn = match Self::wait_or_get_last_lsn(
&shard,
req.hdr.request_lsn,
@@ -861,11 +883,27 @@ impl PageServerHandler {
};
BatchedFeMessage::GetPage {
span,
shard,
shard: shard.downgrade(),
effective_request_lsn,
pages: smallvec::smallvec![BatchedGetPageRequest { req, timer }],
}
}
#[cfg(feature = "testing")]
PagestreamFeMessage::Test(req) => {
let span = tracing::info_span!(parent: parent_span, "handle_test_request");
let shard = timeline_handles
.get(tenant_id, timeline_id, ShardSelector::Zero)
.instrument(span.clone()) // sets `shard_id` field
.await?;
let timer =
record_op_start_and_throttle(&shard, metrics::SmgrQueryType::Test, received_at)
.await?;
BatchedFeMessage::Test {
span,
shard: shard.downgrade(),
requests: vec![BatchedTestRequest { req, timer }],
}
}
};
Ok(Some(batched_msg))
}
@@ -907,9 +945,7 @@ impl PageServerHandler {
assert_eq!(accum_pages.len(), max_batch_size.get());
return false;
}
if (accum_shard.tenant_shard_id, accum_shard.timeline_id)
!= (this_shard.tenant_shard_id, this_shard.timeline_id)
{
if !accum_shard.is_same_handle_as(&this_shard) {
trace!(%accum_lsn, %this_lsn, "stopping batching because timeline object mismatch");
// TODO: we _could_ batch & execute each shard seperately (and in parallel).
// But the current logic for keeping responses in order does not support that.
@@ -928,6 +964,44 @@ impl PageServerHandler {
accum_pages.extend(this_pages);
Ok(())
}
#[cfg(feature = "testing")]
(
Ok(BatchedFeMessage::Test {
shard: accum_shard,
requests: accum_requests,
..
}),
BatchedFeMessage::Test {
shard: this_shard,
requests: this_requests,
..
},
) if (|| {
assert!(this_requests.len() == 1);
if accum_requests.len() >= max_batch_size.get() {
trace!(%max_batch_size, "stopping batching because of batch size");
assert_eq!(accum_requests.len(), max_batch_size.get());
return false;
}
if !accum_shard.is_same_handle_as(&this_shard) {
trace!("stopping batching because timeline object mismatch");
// TODO: we _could_ batch & execute each shard seperately (and in parallel).
// But the current logic for keeping responses in order does not support that.
return false;
}
let this_batch_key = this_requests[0].req.batch_key;
let accum_batch_key = accum_requests[0].req.batch_key;
if this_requests[0].req.batch_key != accum_requests[0].req.batch_key {
trace!(%accum_batch_key, %this_batch_key, "stopping batching because batch key changed");
return false;
}
true
})() =>
{
// ok to batch
accum_requests.extend(this_requests);
Ok(())
}
// something batched already but this message is unbatchable
(_, this_msg) => {
// by default, don't continue batching
@@ -969,7 +1043,7 @@ impl PageServerHandler {
fail::fail_point!("ps::handle-pagerequest-message::exists");
(
vec![self
.handle_get_rel_exists_request(&shard, &req, ctx)
.handle_get_rel_exists_request(&*shard.upgrade()?, &req, ctx)
.instrument(span.clone())
.await
.map(|msg| (msg, timer))
@@ -986,7 +1060,7 @@ impl PageServerHandler {
fail::fail_point!("ps::handle-pagerequest-message::nblocks");
(
vec![self
.handle_get_nblocks_request(&shard, &req, ctx)
.handle_get_nblocks_request(&*shard.upgrade()?, &req, ctx)
.instrument(span.clone())
.await
.map(|msg| (msg, timer))
@@ -1007,7 +1081,7 @@ impl PageServerHandler {
trace!(npages, "handling getpage request");
let res = self
.handle_get_page_at_lsn_request_batched(
&shard,
&*shard.upgrade()?,
effective_request_lsn,
pages,
ctx,
@@ -1029,7 +1103,7 @@ impl PageServerHandler {
fail::fail_point!("ps::handle-pagerequest-message::dbsize");
(
vec![self
.handle_db_size_request(&shard, &req, ctx)
.handle_db_size_request(&*shard.upgrade()?, &req, ctx)
.instrument(span.clone())
.await
.map(|msg| (msg, timer))
@@ -1046,7 +1120,7 @@ impl PageServerHandler {
fail::fail_point!("ps::handle-pagerequest-message::slrusegment");
(
vec![self
.handle_get_slru_segment_request(&shard, &req, ctx)
.handle_get_slru_segment_request(&*shard.upgrade()?, &req, ctx)
.instrument(span.clone())
.await
.map(|msg| (msg, timer))
@@ -1054,6 +1128,27 @@ impl PageServerHandler {
span,
)
}
#[cfg(feature = "testing")]
BatchedFeMessage::Test {
span,
shard,
requests,
} => {
fail::fail_point!("ps::handle-pagerequest-message::test");
(
{
let npages = requests.len();
trace!(npages, "handling getpage request");
let res = self
.handle_test_request_batch(&*shard.upgrade()?, requests, ctx)
.instrument(span.clone())
.await;
assert_eq!(res.len(), npages);
res
},
span,
)
}
BatchedFeMessage::RespondError { span, error } => {
// We've already decided to respond with an error, so we don't need to
// call the handler.
@@ -1791,6 +1886,51 @@ impl PageServerHandler {
))
}
// NB: this impl mimics what we do for batched getpage requests.
#[cfg(feature = "testing")]
#[instrument(skip_all, fields(shard_id))]
async fn handle_test_request_batch(
&mut self,
timeline: &Timeline,
requests: Vec<BatchedTestRequest>,
_ctx: &RequestContext,
) -> Vec<Result<(PagestreamBeMessage, SmgrOpTimer), BatchedPageStreamError>> {
// real requests would do something with the timeline
let mut results = Vec::with_capacity(requests.len());
for _req in requests.iter() {
tokio::task::yield_now().await;
results.push({
if timeline.cancel.is_cancelled() {
Err(PageReconstructError::Cancelled)
} else {
Ok(())
}
});
}
// TODO: avoid creating the new Vec here
Vec::from_iter(
requests
.into_iter()
.zip(results.into_iter())
.map(|(req, res)| {
res.map(|()| {
(
PagestreamBeMessage::Test(models::PagestreamTestResponse {
req: req.req.clone(),
}),
req.timer,
)
})
.map_err(|e| BatchedPageStreamError {
err: PageStreamError::from(e),
req: req.req.hdr,
})
}),
)
}
/// Note on "fullbackup":
/// Full basebackups should only be used for debugging purposes.
/// Originally, it was introduced to enable breaking storage format changes,
@@ -2406,6 +2546,14 @@ impl From<GetActiveTimelineError> for QueryError {
}
}
impl From<crate::tenant::timeline::handle::HandleUpgradeError> for QueryError {
fn from(e: crate::tenant::timeline::handle::HandleUpgradeError) -> Self {
match e {
crate::tenant::timeline::handle::HandleUpgradeError::ShutDown => QueryError::Shutdown,
}
}
}
fn set_tracing_field_shard_id(timeline: &Timeline) {
debug_assert_current_span_has_tenant_and_timeline_id_no_shard_id();
tracing::Span::current().record(

View File

@@ -95,6 +95,9 @@ use crate::deletion_queue::DeletionQueueError;
use crate::import_datadir;
use crate::is_uninit_mark;
use crate::l0_flush::L0FlushGlobalState;
use crate::metrics::CONCURRENT_INITDBS;
use crate::metrics::INITDB_RUN_TIME;
use crate::metrics::INITDB_SEMAPHORE_ACQUISITION_TIME;
use crate::metrics::TENANT;
use crate::metrics::{
remove_tenant_metrics, BROKEN_TENANTS_SET, CIRCUIT_BREAKERS_BROKEN, CIRCUIT_BREAKERS_UNBROKEN,
@@ -5347,8 +5350,17 @@ async fn run_initdb(
initdb_bin_path, initdb_target_dir, initdb_lib_dir,
);
let _permit = INIT_DB_SEMAPHORE.acquire().await;
let _permit = {
let _timer = INITDB_SEMAPHORE_ACQUISITION_TIME.start_timer();
INIT_DB_SEMAPHORE.acquire().await
};
CONCURRENT_INITDBS.inc();
scopeguard::defer! {
CONCURRENT_INITDBS.dec();
}
let _timer = INITDB_RUN_TIME.start_timer();
let res = postgres_initdb::do_run_initdb(postgres_initdb::RunInitdbArgs {
superuser: &conf.superuser,
locale: &conf.locale,
@@ -5463,6 +5475,7 @@ pub(crate) mod harness {
lsn_lease_length_for_ts: Some(tenant_conf.lsn_lease_length_for_ts),
timeline_offloading: Some(tenant_conf.timeline_offloading),
wal_receiver_protocol_override: tenant_conf.wal_receiver_protocol_override,
rel_size_v2_enabled: tenant_conf.rel_size_v2_enabled,
}
}
}

View File

@@ -357,6 +357,9 @@ pub struct TenantConfOpt {
#[serde(skip_serializing_if = "Option::is_none")]
pub wal_receiver_protocol_override: Option<PostgresClientProtocol>,
#[serde(skip_serializing_if = "Option::is_none")]
pub rel_size_v2_enabled: Option<bool>,
}
impl TenantConfOpt {
@@ -425,6 +428,7 @@ impl TenantConfOpt {
wal_receiver_protocol_override: self
.wal_receiver_protocol_override
.or(global_conf.wal_receiver_protocol_override),
rel_size_v2_enabled: self.rel_size_v2_enabled.or(global_conf.rel_size_v2_enabled),
}
}
@@ -454,6 +458,7 @@ impl TenantConfOpt {
mut lsn_lease_length_for_ts,
mut timeline_offloading,
mut wal_receiver_protocol_override,
mut rel_size_v2_enabled,
} = self;
patch.checkpoint_distance.apply(&mut checkpoint_distance);
@@ -522,6 +527,7 @@ impl TenantConfOpt {
patch
.wal_receiver_protocol_override
.apply(&mut wal_receiver_protocol_override);
patch.rel_size_v2_enabled.apply(&mut rel_size_v2_enabled);
Ok(Self {
checkpoint_distance,
@@ -548,6 +554,7 @@ impl TenantConfOpt {
lsn_lease_length_for_ts,
timeline_offloading,
wal_receiver_protocol_override,
rel_size_v2_enabled,
})
}
}
@@ -603,6 +610,7 @@ impl From<TenantConfOpt> for models::TenantConfig {
lsn_lease_length_for_ts: value.lsn_lease_length_for_ts.map(humantime),
timeline_offloading: value.timeline_offloading,
wal_receiver_protocol_override: value.wal_receiver_protocol_override,
rel_size_v2_enabled: value.rel_size_v2_enabled,
}
}
}

View File

@@ -382,6 +382,12 @@ pub(crate) struct RemoteTimelineClient {
cancel: CancellationToken,
}
impl Drop for RemoteTimelineClient {
fn drop(&mut self) {
debug!("dropping RemoteTimelineClient");
}
}
impl RemoteTimelineClient {
///
/// Create a remote storage client for given timeline
@@ -797,6 +803,12 @@ impl RemoteTimelineClient {
upload_queue.dirty.metadata.apply(update);
// Defense in depth: if we somehow generated invalid metadata, do not persist it.
upload_queue
.dirty
.validate()
.map_err(|e| anyhow::anyhow!(e))?;
self.schedule_index_upload(upload_queue);
Ok(())

View File

@@ -79,6 +79,24 @@ pub struct IndexPart {
/// when this flag is introduced.
#[serde(skip_serializing_if = "Option::is_none", default)]
pub(crate) last_aux_file_policy: Option<AuxFilePolicy>,
#[serde(skip_serializing_if = "Option::is_none", default)]
pub(crate) rel_size_migration: Option<RelSizeMigration>,
}
#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)]
#[serde(rename_all = "camelCase")]
pub enum RelSizeMigration {
/// The tenant is using the old rel_size format.
/// Note that this enum is persisted as `Option<RelSizeMigration>` in the index part, so
/// `None` is the same as `Some(RelSizeMigration::Legacy)`.
Legacy,
/// The tenant is migrating to the new rel_size format. Both old and new rel_size format are
/// persisted in the index part. The read path will read both formats and merge them.
Migrating,
/// The tenant has migrated to the new rel_size format. Only the new rel_size format is persisted
/// in the index part, and the read path will not read the old format.
Migrated,
}
impl IndexPart {
@@ -97,10 +115,11 @@ impl IndexPart {
/// - 8: added `archived_at`
/// - 9: +gc_blocking
/// - 10: +import_pgdata
const LATEST_VERSION: usize = 10;
/// - 11: +rel_size_migration
const LATEST_VERSION: usize = 11;
// Versions we may see when reading from a bucket.
pub const KNOWN_VERSIONS: &'static [usize] = &[1, 2, 3, 4, 5, 6, 7, 8, 9, 10];
pub const KNOWN_VERSIONS: &'static [usize] = &[1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11];
pub const FILE_NAME: &'static str = "index_part.json";
@@ -116,6 +135,7 @@ impl IndexPart {
gc_blocking: None,
last_aux_file_policy: None,
import_pgdata: None,
rel_size_migration: None,
}
}
@@ -152,6 +172,21 @@ impl IndexPart {
};
is_same_remote_layer_path(name, metadata, name, index_metadata)
}
/// Check for invariants in the index: this is useful when uploading an index to ensure that if
/// we encounter a bug, we do not persist buggy metadata.
pub(crate) fn validate(&self) -> Result<(), String> {
if self.import_pgdata.is_none()
&& self.metadata.ancestor_timeline().is_none()
&& self.layer_metadata.is_empty()
{
// Unless we're in the middle of a raw pgdata import, or this is a child timeline,the index must
// always have at least one layer.
return Err("Index has no ancestor and no layers".to_string());
}
Ok(())
}
}
/// Metadata gathered for each of the layer files.
@@ -401,6 +436,7 @@ mod tests {
gc_blocking: None,
last_aux_file_policy: None,
import_pgdata: None,
rel_size_migration: None,
};
let part = IndexPart::from_json_bytes(example.as_bytes()).unwrap();
@@ -446,6 +482,7 @@ mod tests {
gc_blocking: None,
last_aux_file_policy: None,
import_pgdata: None,
rel_size_migration: None,
};
let part = IndexPart::from_json_bytes(example.as_bytes()).unwrap();
@@ -492,6 +529,7 @@ mod tests {
gc_blocking: None,
last_aux_file_policy: None,
import_pgdata: None,
rel_size_migration: None,
};
let part = IndexPart::from_json_bytes(example.as_bytes()).unwrap();
@@ -541,6 +579,7 @@ mod tests {
gc_blocking: None,
last_aux_file_policy: None,
import_pgdata: None,
rel_size_migration: None,
};
let empty_layers_parsed = IndexPart::from_json_bytes(empty_layers_json.as_bytes()).unwrap();
@@ -585,6 +624,7 @@ mod tests {
gc_blocking: None,
last_aux_file_policy: None,
import_pgdata: None,
rel_size_migration: None,
};
let part = IndexPart::from_json_bytes(example.as_bytes()).unwrap();
@@ -632,6 +672,7 @@ mod tests {
gc_blocking: None,
last_aux_file_policy: None,
import_pgdata: None,
rel_size_migration: None,
};
let part = IndexPart::from_json_bytes(example.as_bytes()).unwrap();
@@ -684,6 +725,7 @@ mod tests {
gc_blocking: None,
last_aux_file_policy: Some(AuxFilePolicy::V2),
import_pgdata: None,
rel_size_migration: None,
};
let part = IndexPart::from_json_bytes(example.as_bytes()).unwrap();
@@ -741,6 +783,7 @@ mod tests {
gc_blocking: None,
last_aux_file_policy: Default::default(),
import_pgdata: None,
rel_size_migration: None,
};
let part = IndexPart::from_json_bytes(example.as_bytes()).unwrap();
@@ -799,6 +842,7 @@ mod tests {
gc_blocking: None,
last_aux_file_policy: Default::default(),
import_pgdata: None,
rel_size_migration: None,
};
let part = IndexPart::from_json_bytes(example.as_bytes()).unwrap();
@@ -862,6 +906,7 @@ mod tests {
last_aux_file_policy: Default::default(),
archived_at: None,
import_pgdata: None,
rel_size_migration: None,
};
let part = IndexPart::from_json_bytes(example.as_bytes()).unwrap();
@@ -937,7 +982,86 @@ mod tests {
started_at: parse_naive_datetime("2024-11-13T09:23:42.123000000"),
finished_at: parse_naive_datetime("2024-11-13T09:42:23.123000000"),
idempotency_key: import_pgdata::index_part_format::IdempotencyKey::new("specified-by-client-218a5213-5044-4562-a28d-d024c5f057f5".to_string()),
})))
}))),
rel_size_migration: None,
};
let part = IndexPart::from_json_bytes(example.as_bytes()).unwrap();
assert_eq!(part, expected);
}
#[test]
fn v11_rel_size_migration_is_parsed() {
let example = r#"{
"version": 11,
"layer_metadata":{
"000000000000000000000000000000000000-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF__0000000001696070-00000000016960E9": { "file_size": 25600000 },
"000000000000000000000000000000000000-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF__00000000016B59D8-00000000016B5A51": { "file_size": 9007199254741001 }
},
"disk_consistent_lsn":"0/16960E8",
"metadata": {
"disk_consistent_lsn": "0/16960E8",
"prev_record_lsn": "0/1696070",
"ancestor_timeline": "e45a7f37d3ee2ff17dc14bf4f4e3f52e",
"ancestor_lsn": "0/0",
"latest_gc_cutoff_lsn": "0/1696070",
"initdb_lsn": "0/1696070",
"pg_version": 14
},
"gc_blocking": {
"started_at": "2024-07-19T09:00:00.123",
"reasons": ["DetachAncestor"]
},
"import_pgdata": {
"V1": {
"Done": {
"idempotency_key": "specified-by-client-218a5213-5044-4562-a28d-d024c5f057f5",
"started_at": "2024-11-13T09:23:42.123",
"finished_at": "2024-11-13T09:42:23.123"
}
}
},
"rel_size_migration": "legacy"
}"#;
let expected = IndexPart {
version: 11,
layer_metadata: HashMap::from([
("000000000000000000000000000000000000-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF__0000000001696070-00000000016960E9".parse().unwrap(), LayerFileMetadata {
file_size: 25600000,
generation: Generation::none(),
shard: ShardIndex::unsharded()
}),
("000000000000000000000000000000000000-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF__00000000016B59D8-00000000016B5A51".parse().unwrap(), LayerFileMetadata {
file_size: 9007199254741001,
generation: Generation::none(),
shard: ShardIndex::unsharded()
})
]),
disk_consistent_lsn: "0/16960E8".parse::<Lsn>().unwrap(),
metadata: TimelineMetadata::new(
Lsn::from_str("0/16960E8").unwrap(),
Some(Lsn::from_str("0/1696070").unwrap()),
Some(TimelineId::from_str("e45a7f37d3ee2ff17dc14bf4f4e3f52e").unwrap()),
Lsn::INVALID,
Lsn::from_str("0/1696070").unwrap(),
Lsn::from_str("0/1696070").unwrap(),
14,
).with_recalculated_checksum().unwrap(),
deleted_at: None,
lineage: Default::default(),
gc_blocking: Some(GcBlocking {
started_at: parse_naive_datetime("2024-07-19T09:00:00.123000000"),
reasons: enumset::EnumSet::from_iter([GcBlockingReason::DetachAncestor]),
}),
last_aux_file_policy: Default::default(),
archived_at: None,
import_pgdata: Some(import_pgdata::index_part_format::Root::V1(import_pgdata::index_part_format::V1::Done(import_pgdata::index_part_format::Done{
started_at: parse_naive_datetime("2024-11-13T09:23:42.123000000"),
finished_at: parse_naive_datetime("2024-11-13T09:42:23.123000000"),
idempotency_key: import_pgdata::index_part_format::IdempotencyKey::new("specified-by-client-218a5213-5044-4562-a28d-d024c5f057f5".to_string()),
}))),
rel_size_migration: Some(RelSizeMigration::Legacy),
};
let part = IndexPart::from_json_bytes(example.as_bytes()).unwrap();

View File

@@ -40,6 +40,10 @@ pub(crate) async fn upload_index_part(
});
pausable_failpoint!("before-upload-index-pausable");
// Safety: refuse to persist invalid index metadata, to mitigate the impact of any bug that produces this
// (this should never happen)
index_part.validate().map_err(|e| anyhow::anyhow!(e))?;
// FIXME: this error comes too late
let serialized = index_part.to_json_bytes()?;
let serialized = Bytes::from(serialized);

View File

@@ -1,6 +1,6 @@
use std::time::UNIX_EPOCH;
use pageserver_api::key::CONTROLFILE_KEY;
use pageserver_api::key::{Key, CONTROLFILE_KEY};
use tokio::task::JoinSet;
use utils::{
completion::{self, Completion},
@@ -9,7 +9,10 @@ use utils::{
use super::failpoints::{Failpoint, FailpointKind};
use super::*;
use crate::{context::DownloadBehavior, tenant::storage_layer::LayerVisibilityHint};
use crate::{
context::DownloadBehavior,
tenant::{harness::test_img, storage_layer::LayerVisibilityHint},
};
use crate::{task_mgr::TaskKind, tenant::harness::TenantHarness};
/// Used in tests to advance a future to wanted await point, and not futher.
@@ -31,20 +34,51 @@ async fn smoke_test() {
let ctx = RequestContext::new(TaskKind::UnitTest, DownloadBehavior::Download);
let image_layers = vec![(
Lsn(0x40),
vec![(
Key::from_hex("620000000033333333444444445500000000").unwrap(),
test_img("foo"),
)],
)];
// Create a test timeline with one real layer, and one synthetic test layer. The synthetic
// one is only there so that we can GC the real one without leaving the timeline's metadata
// empty, which is an illegal state (see [`IndexPart::validate`]).
let timeline = tenant
.create_test_timeline(TimelineId::generate(), Lsn(0x10), 14, &ctx)
.create_test_timeline_with_layers(
TimelineId::generate(),
Lsn(0x10),
14,
&ctx,
Default::default(),
image_layers,
Lsn(0x100),
)
.await
.unwrap();
let layer = {
// Grab one of the timeline's layers to exercise in the test, and the other layer that is just
// there to avoid the timeline being illegally empty
let (layer, dummy_layer) = {
let mut layers = {
let layers = timeline.layers.read().await;
layers.likely_resident_layers().cloned().collect::<Vec<_>>()
};
assert_eq!(layers.len(), 1);
assert_eq!(layers.len(), 2);
layers.swap_remove(0)
layers.sort_by_key(|l| l.layer_desc().get_key_range().start);
let synthetic_layer = layers.pop().unwrap();
let real_layer = layers.pop().unwrap();
tracing::info!(
"real_layer={:?} ({}), synthetic_layer={:?} ({})",
real_layer,
real_layer.layer_desc().file_size,
synthetic_layer,
synthetic_layer.layer_desc().file_size
);
(real_layer, synthetic_layer)
};
// all layers created at pageserver are like `layer`, initialized with strong
@@ -173,10 +207,13 @@ async fn smoke_test() {
let rtc = &timeline.remote_client;
// Simulate GC removing our test layer.
{
let layers = &[layer];
let mut g = timeline.layers.write().await;
let layers = &[layer];
g.open_mut().unwrap().finish_gc_timeline(layers);
// this just updates the remote_physical_size for demonstration purposes
rtc.schedule_gc_update(layers).unwrap();
}
@@ -191,7 +228,10 @@ async fn smoke_test() {
rtc.wait_completion().await.unwrap();
assert_eq!(rtc.get_remote_physical_size(), 0);
assert_eq!(
rtc.get_remote_physical_size(),
dummy_layer.metadata().file_size
);
assert_eq!(0, LAYER_IMPL_METRICS.inits_cancelled.get())
}

View File

@@ -76,6 +76,7 @@ use std::{pin::pin, sync::OnceLock};
use crate::{
aux_file::AuxFileSizeEstimator,
page_service::TenantManagerTypes,
tenant::{
config::AttachmentMode,
layer_map::{LayerMap, SearchResult},
@@ -431,7 +432,7 @@ pub struct Timeline {
pub(crate) l0_flush_global_state: L0FlushGlobalState,
pub(crate) handles: handle::PerTimelineState<crate::page_service::TenantManagerTypes>,
pub(crate) handles: handle::PerTimelineState<TenantManagerTypes>,
pub(crate) attach_wal_lag_cooldown: Arc<OnceLock<WalLagCooldown>>,
@@ -4625,6 +4626,10 @@ impl Drop for Timeline {
}
}
}
info!(
"Timeline {} for tenant {} is being dropped",
self.timeline_id, self.tenant_shard_id.tenant_id
);
}
}
@@ -5673,9 +5678,17 @@ impl Timeline {
info!("force created image layer {}", image_layer.local_path());
{
let mut guard = self.layers.write().await;
guard.open_mut().unwrap().force_insert_layer(image_layer);
guard
.open_mut()
.unwrap()
.force_insert_layer(image_layer.clone());
}
// Update remote_timeline_client state to reflect existence of this layer
self.remote_client
.schedule_layer_file_upload(image_layer)
.unwrap();
Ok(())
}
@@ -5726,9 +5739,17 @@ impl Timeline {
info!("force created delta layer {}", delta_layer.local_path());
{
let mut guard = self.layers.write().await;
guard.open_mut().unwrap().force_insert_layer(delta_layer);
guard
.open_mut()
.unwrap()
.force_insert_layer(delta_layer.clone());
}
// Update remote_timeline_client state to reflect existence of this layer
self.remote_client
.schedule_layer_file_upload(delta_layer)
.unwrap();
Ok(())
}

View File

@@ -112,7 +112,7 @@ pub(super) async fn delete_local_timeline_directory(
}
/// It is important that this gets called when DeletionGuard is being held.
/// For more context see comments in [`DeleteTimelineFlow::prepare`]
/// For more context see comments in [`make_timeline_delete_guard`]
async fn remove_maybe_offloaded_timeline_from_tenant(
tenant: &Tenant,
timeline: &TimelineOrOffloaded,
@@ -193,10 +193,8 @@ impl DeleteTimelineFlow {
) -> Result<(), DeleteTimelineError> {
super::debug_assert_current_span_has_tenant_and_timeline_id();
let allow_offloaded_children = false;
let set_stopping = true;
let (timeline, mut guard) =
Self::prepare(tenant, timeline_id, allow_offloaded_children, set_stopping)?;
make_timeline_delete_guard(tenant, timeline_id, TimelineDeleteGuardKind::Delete)?;
guard.mark_in_progress()?;
@@ -333,75 +331,6 @@ impl DeleteTimelineFlow {
Ok(())
}
pub(super) fn prepare(
tenant: &Tenant,
timeline_id: TimelineId,
allow_offloaded_children: bool,
set_stopping: bool,
) -> Result<(TimelineOrOffloaded, DeletionGuard), DeleteTimelineError> {
// Note the interaction between this guard and deletion guard.
// Here we attempt to lock deletion guard when we're holding a lock on timelines.
// This is important because when you take into account `remove_timeline_from_tenant`
// we remove timeline from memory when we still hold the deletion guard.
// So here when timeline deletion is finished timeline wont be present in timelines map at all
// which makes the following sequence impossible:
// T1: get preempted right before the try_lock on `Timeline::delete_progress`
// T2: do a full deletion, acquire and drop `Timeline::delete_progress`
// T1: acquire deletion lock, do another `DeleteTimelineFlow::run`
// For more context see this discussion: `https://github.com/neondatabase/neon/pull/4552#discussion_r1253437346`
let timelines = tenant.timelines.lock().unwrap();
let timelines_offloaded = tenant.timelines_offloaded.lock().unwrap();
let timeline = match timelines.get(&timeline_id) {
Some(t) => TimelineOrOffloaded::Timeline(Arc::clone(t)),
None => match timelines_offloaded.get(&timeline_id) {
Some(t) => TimelineOrOffloaded::Offloaded(Arc::clone(t)),
None => return Err(DeleteTimelineError::NotFound),
},
};
// Ensure that there are no child timelines, because we are about to remove files,
// which will break child branches
let mut children = Vec::new();
if !allow_offloaded_children {
children.extend(timelines_offloaded.iter().filter_map(|(id, entry)| {
(entry.ancestor_timeline_id == Some(timeline_id)).then_some(*id)
}));
}
children.extend(timelines.iter().filter_map(|(id, entry)| {
(entry.get_ancestor_timeline_id() == Some(timeline_id)).then_some(*id)
}));
if !children.is_empty() {
return Err(DeleteTimelineError::HasChildren(children));
}
// Note that using try_lock here is important to avoid a deadlock.
// Here we take lock on timelines and then the deletion guard.
// At the end of the operation we're holding the guard and need to lock timelines map
// to remove the timeline from it.
// Always if you have two locks that are taken in different order this can result in a deadlock.
let delete_progress = Arc::clone(timeline.delete_progress());
let delete_lock_guard = match delete_progress.try_lock_owned() {
Ok(guard) => DeletionGuard(guard),
Err(_) => {
// Unfortunately if lock fails arc is consumed.
return Err(DeleteTimelineError::AlreadyInProgress(Arc::clone(
timeline.delete_progress(),
)));
}
};
if set_stopping {
if let TimelineOrOffloaded::Timeline(timeline) = &timeline {
timeline.set_state(TimelineState::Stopping);
}
}
Ok((timeline, delete_lock_guard))
}
fn schedule_background(
guard: DeletionGuard,
conf: &'static PageServerConf,
@@ -483,6 +412,80 @@ impl DeleteTimelineFlow {
}
}
#[derive(Copy, Clone, PartialEq, Eq)]
pub(super) enum TimelineDeleteGuardKind {
Offload,
Delete,
}
pub(super) fn make_timeline_delete_guard(
tenant: &Tenant,
timeline_id: TimelineId,
guard_kind: TimelineDeleteGuardKind,
) -> Result<(TimelineOrOffloaded, DeletionGuard), DeleteTimelineError> {
// Note the interaction between this guard and deletion guard.
// Here we attempt to lock deletion guard when we're holding a lock on timelines.
// This is important because when you take into account `remove_timeline_from_tenant`
// we remove timeline from memory when we still hold the deletion guard.
// So here when timeline deletion is finished timeline wont be present in timelines map at all
// which makes the following sequence impossible:
// T1: get preempted right before the try_lock on `Timeline::delete_progress`
// T2: do a full deletion, acquire and drop `Timeline::delete_progress`
// T1: acquire deletion lock, do another `DeleteTimelineFlow::run`
// For more context see this discussion: `https://github.com/neondatabase/neon/pull/4552#discussion_r1253437346`
let timelines = tenant.timelines.lock().unwrap();
let timelines_offloaded = tenant.timelines_offloaded.lock().unwrap();
let timeline = match timelines.get(&timeline_id) {
Some(t) => TimelineOrOffloaded::Timeline(Arc::clone(t)),
None => match timelines_offloaded.get(&timeline_id) {
Some(t) => TimelineOrOffloaded::Offloaded(Arc::clone(t)),
None => return Err(DeleteTimelineError::NotFound),
},
};
// Ensure that there are no child timelines, because we are about to remove files,
// which will break child branches
let mut children = Vec::new();
if guard_kind == TimelineDeleteGuardKind::Delete {
children.extend(timelines_offloaded.iter().filter_map(|(id, entry)| {
(entry.ancestor_timeline_id == Some(timeline_id)).then_some(*id)
}));
}
children.extend(timelines.iter().filter_map(|(id, entry)| {
(entry.get_ancestor_timeline_id() == Some(timeline_id)).then_some(*id)
}));
if !children.is_empty() {
return Err(DeleteTimelineError::HasChildren(children));
}
// Note that using try_lock here is important to avoid a deadlock.
// Here we take lock on timelines and then the deletion guard.
// At the end of the operation we're holding the guard and need to lock timelines map
// to remove the timeline from it.
// Always if you have two locks that are taken in different order this can result in a deadlock.
let delete_progress = Arc::clone(timeline.delete_progress());
let delete_lock_guard = match delete_progress.try_lock_owned() {
Ok(guard) => DeletionGuard(guard),
Err(_) => {
// Unfortunately if lock fails arc is consumed.
return Err(DeleteTimelineError::AlreadyInProgress(Arc::clone(
timeline.delete_progress(),
)));
}
};
if guard_kind == TimelineDeleteGuardKind::Delete {
if let TimelineOrOffloaded::Timeline(timeline) = &timeline {
timeline.set_state(TimelineState::Stopping);
}
}
Ok((timeline, delete_lock_guard))
}
pub(super) struct DeletionGuard(OwnedMutexGuard<DeleteTimelineFlow>);
impl Deref for DeletionGuard {

View File

@@ -32,54 +32,151 @@
//!
//! # Design
//!
//! ## Data Structures
//!
//! There are three user-facing data structures:
//! - `PerTimelineState`: a struct embedded into each Timeline struct. Lifetime == Timeline lifetime.
//! - `Cache`: a struct private to each connection handler; Lifetime == connection lifetime.
//! - `Handle`: a smart pointer that holds the Timeline gate open and derefs to `&Timeline`.
//! Lifetime: for a single request dispatch on the Timeline (i.e., one getpage request)
//! - `WeakHandle`: downgrade of a `Handle` that does not keep the gate open, but allows
//! trying to ugprade back to a `Handle`, guaranteeing it's the same `Timeline` *object*.
//!
//! The `Handle` is just a wrapper around an `Arc<HandleInner>`.
//! Internally, there is 0 or 1 `HandleInner` per `(Cache,Timeline)`.
//! Since Cache:Connection is 1:1, there is 0 or 1 `HandleInner` per `(Connection,Timeline)`.
//!
//! There is one long-lived `Arc<HandleInner>`, which is stored in the `PerTimelineState`.
//! The `Cache` stores a `Weak<HandleInner>` for each cached Timeline.
//! The `HandleInner` is allocated as a `Arc<Mutex<HandleInner>>` and
//! referenced weakly and strongly from various places which we are now illustrating.
//! For brevity, we will omit the `Arc<Mutex<>>` part in the following and instead
//! use `strong ref` and `weak ref` when referring to the `Arc<Mutex<HandleInner>>`
//! or `Weak<Mutex<HandleInner>>`, respectively.
//!
//! - The `Handle` is a strong ref.
//! - The `WeakHandle` is a weak ref.
//! - The `PerTimelineState` contains a `HashMap<CacheId, strong ref>`.
//! - The `Cache` is a `HashMap<unique identifier for the shard, weak ref>`.
//!
//! Lifetimes:
//! - `WeakHandle` and `Handle`: single pagestream request.
//! - `Cache`: single page service connection.
//! - `PerTimelineState`: lifetime of the Timeline object (i.e., i.e., till `Timeline::shutdown`).
//!
//! ## Request Handling Flow (= filling and using the `Cache``)
//!
//! To dispatch a request, the page service connection calls `Cache::get`.
//!
//! A cache miss means we consult the tenant manager for shard routing,
//! resulting in an `Arc<Timeline>`. We enter its gate _once_ and construct an
//! `Arc<HandleInner>`. We store a `Weak<HandleInner>` in the cache
//! and the `Arc<HandleInner>` in the `PerTimelineState`.
//! resulting in an `Arc<Timeline>`. We enter its gate _once_ and store it in the the
//! `Arc<Mutex<HandleInner>>>`. A weak ref is stored in the `Cache`
//! and a strong ref in the `PerTimelineState`.
//! A strong ref is returned wrapped in a `Handle`.
//!
//! For subsequent requests, `Cache::get` will perform a "fast path" shard routing
//! and find the `Weak<HandleInner>` in the cache.
//! We upgrade the `Weak<HandleInner>` to an `Arc<HandleInner>` and wrap it in the user-facing `Handle` type.
//! and find the weak ref in the cache.
//! We upgrade the weak ref to a strong ref and return it wrapped in a `Handle`.
//!
//! The request handler dispatches the request to the right `<Handle as Deref<Target = Timeline>>::$request_method`.
//! The pagestream processing is pipelined and involves a batching step.
//! While a request is batching, the `Handle` is downgraded to a `WeakHandle`.
//! When the batch is ready to be executed, the `WeakHandle` is upgraded back to a `Handle`
//! and the request handler dispatches the request to the right `<Handle as Deref<Target = Timeline>>::$request_method`.
//! It then drops the `Handle`, which drops the `Arc<HandleInner>`.
//!
//! # Memory Management / How The Reference Cycle Is Broken
//! # Performance
//!
//! The attentive reader may have noticed the strong reference cycle
//! from `Arc<HandleInner>` to `PerTimelineState` to `Arc<Timeline>`.
//! Remember from the introductory section:
//!
//! This cycle is intentional: while it exists, the `Cache` can upgrade its
//! `Weak<HandleInner>` to an `Arc<HandleInner>` in a single atomic operation.
//! > However, we want to avoid the overhead of entering the gate for every
//! > method invocation.
//!
//! Why do we want to avoid that?
//! Because the gate is a shared location in memory and entering it involves
//! bumping refcounts, which leads to cache contention if done frequently
//! from multiple cores in parallel.
//!
//! So, we only acquire the `GateGuard` once on `Cache` miss, and wrap it in an `Arc`.
//! That `Arc` is private to the `HandleInner` and hence to the connection.
//! (Review the "Data Structures" section if that is unclear to you.)
//!
//! A `WeakHandle` is a weak ref to the `HandleInner`.
//! When upgrading a `WeakHandle`, we upgrade to a strong ref to the `HandleInner` and
//! further acquire an additional strong ref to the `Arc<GateGuard>` inside it.
//! Again, this manipulation of ref counts is is cheap because `Arc` is private to the connection.
//!
//! When downgrading a `Handle` to a `WeakHandle`, we drop the `Arc<GateGuard>`.
//! Again, this is cheap because the `Arc` is private to the connection.
//!
//! In addition to the GateGuard, we need to provide `Deref<Target=Timeline>` impl.
//! For this, both `Handle` need infallible access to an `Arc<Timeline>`.
//! We could clone the `Arc<Timeline>` when upgrading a `WeakHandle`, but that would cause contention
//! on the shared memory location that trakcs the refcount of the `Arc<Timeline>`.
//! Instead, we wrap the `Arc<Timeline>` into another `Arc`.
//! so that we can clone it cheaply when upgrading a `WeakHandle`.
//!
//! # Shutdown
//!
//! The attentive reader may have noticed the following reference cycle around the `Arc<Timeline>`:
//!
//! ```text
//! Timeline --owns--> PerTimelineState --strong--> HandleInner --strong--> Timeline
//! ```
//!
//! Further, there is this cycle:
//!
//! ```text
//! Timeline --owns--> PerTimelineState --strong--> HandleInner --strong--> GateGuard --keepalive--> Timeline
//! ```
//!
//! The former cycle is a memory leak if not broken.
//! The latter cycle further prevents the Timeline from shutting down
//! because we certainly won't drop the Timeline while the GateGuard is alive.
//! Preventing shutdown is the whole point of this handle/cache system,
//! but when the Timeline needs to shut down, we need to break the cycle.
//!
//! The cycle is broken by either
//! - `PerTimelineState::shutdown` or
//! - dropping the `Cache`.
//! - Timeline shutdown (=> `PerTimelineState::shutdown`)
//! - Connection shutdown (=> dropping the `Cache`).
//!
//! Concurrently existing `Handle`s will extend the existence of the cycle.
//! Both transition the `HandleInner` from [`HandleInner::KeepingTimelineGateOpen`] to
//! [`HandleInner::ShutDown`], which drops the only long-lived strong ref to the
//! `Arc<GateGuard>`.
//!
//! `PerTimelineState::shutdown` drops all the `HandleInners` it contains,
//! thereby breaking the cycle.
//! It also initiates draining of already existing `Handle`s by
//! poisoning things so that no new `HandleInner`'s can be added
//! to the `PerTimelineState`, which will make subsequent `Cache::get` fail.
//!
//! Concurrently existing / already upgraded `Handle`s will extend the
//! lifetime of the `Arc<Mutex<HandleInner>>` and hence cycles.
//! However, since `Handle`s are short-lived and new `Handle`s are not
//! handed out after either `PerTimelineState::shutdown` or `Cache` drop,
//! that extension of the cycle is bounded.
//! handed out from `Cache::get` or `WeakHandle::upgrade` after
//! `PerTimelineState::shutdown`, that extension of the cycle is bounded.
//!
//! Concurrently existing `WeakHandle`s will fail to `upgrade()`:
//! while they will succeed in upgrading `Weak<Mutex<HandleInner>>`,
//! they will find the inner in state `HandleInner::ShutDown` state where the
//! `Arc<GateGuard>` and Timeline has already been dropped.
//!
//! Dropping the `Cache` undoes the registration of this `Cache`'s
//! `HandleInner`s from all the `PerTimelineState`s, i.e., it
//! removes the strong ref to each of its `HandleInner`s
//! from all the `PerTimelineState`.
//!
//! # Locking Rules
//!
//! To prevent deadlocks we:
//!
//! 1. Only ever hold one of the locks at a time.
//! 2. Don't add more than one Drop impl that locks on the
//! cycles above.
//!
//! As per (2), that impl is in `Drop for Cache`.
//!
//! # Fast Path for Shard Routing
//!
//! The `Cache` has a fast path for shard routing to avoid calling into
//! the tenant manager for every request.
//!
//! The `Cache` maintains a hash map of `ShardTimelineId` to `Weak<HandleInner>`.
//! The `Cache` maintains a hash map of `ShardTimelineId` to `WeakHandle`s.
//!
//! The current implementation uses the first entry in the hash map
//! to determine the `ShardParameters` and derive the correct
@@ -87,18 +184,18 @@
//!
//! It then looks up the hash map for that `ShardTimelineId := {ShardIndex,TimelineId}`.
//!
//! If the lookup is successful and the `Weak<HandleInner>` can be upgraded,
//! If the lookup is successful and the `WeakHandle` can be upgraded,
//! it's a hit.
//!
//! ## Cache invalidation
//!
//! The insight is that cache invalidation is sufficient and most efficiently done lazily.
//! The insight is that cache invalidation is sufficient and most efficiently if done lazily.
//! The only reasons why an entry in the cache can become stale are:
//! 1. The `PerTimelineState` / Timeline is shutting down e.g. because the shard is
//! being detached, timeline or shard deleted, or pageserver is shutting down.
//! 2. We're doing a shard split and new traffic should be routed to the child shards.
//!
//! Regarding (1), we will eventually fail to upgrade the `Weak<HandleInner>` once the
//! Regarding (1), we will eventually fail to upgrade the `WeakHandle` once the
//! timeline has shut down, and when that happens, we remove the entry from the cache.
//!
//! Regarding (2), the insight is that it is toally fine to keep dispatching requests
@@ -107,8 +204,6 @@
use std::collections::hash_map;
use std::collections::HashMap;
use std::sync::atomic::AtomicBool;
use std::sync::atomic::Ordering;
use std::sync::Arc;
use std::sync::Mutex;
use std::sync::Weak;
@@ -152,7 +247,7 @@ pub(crate) struct Cache<T: Types> {
map: Map<T>,
}
type Map<T> = HashMap<ShardTimelineId, Weak<HandleInner<T>>>;
type Map<T> = HashMap<ShardTimelineId, WeakHandle<T>>;
impl<T: Types> Default for Cache<T> {
fn default() -> Self {
@@ -170,12 +265,22 @@ pub(crate) struct ShardTimelineId {
}
/// See module-level comment.
pub(crate) struct Handle<T: Types>(Arc<HandleInner<T>>);
struct HandleInner<T: Types> {
shut_down: AtomicBool,
timeline: T::Timeline,
// The timeline's gate held open.
_gate_guard: utils::sync::gate::GateGuard,
pub(crate) struct Handle<T: Types> {
timeline: Arc<T::Timeline>,
#[allow(dead_code)] // the field exists to keep the gate open
gate_guard: Arc<utils::sync::gate::GateGuard>,
inner: Arc<Mutex<HandleInner<T>>>,
}
pub(crate) struct WeakHandle<T: Types> {
inner: Weak<Mutex<HandleInner<T>>>,
}
enum HandleInner<T: Types> {
KeepingTimelineGateOpen {
#[allow(dead_code)]
gate_guard: Arc<utils::sync::gate::GateGuard>,
timeline: Arc<T::Timeline>,
},
ShutDown,
}
/// Embedded in each [`Types::Timeline`] as the anchor for the only long-lived strong ref to `HandleInner`.
@@ -183,7 +288,8 @@ struct HandleInner<T: Types> {
/// See module-level comment for details.
pub struct PerTimelineState<T: Types> {
// None = shutting down
handles: Mutex<Option<HashMap<CacheId, Arc<HandleInner<T>>>>>,
#[allow(clippy::type_complexity)]
handles: Mutex<Option<HashMap<CacheId, Arc<Mutex<HandleInner<T>>>>>>,
}
impl<T: Types> Default for PerTimelineState<T> {
@@ -243,49 +349,24 @@ impl<T: Types> Cache<T> {
shard_selector: ShardSelector,
tenant_manager: &T::TenantManager,
) -> Result<Handle<T>, GetError<T>> {
// terminates because each iteration removes an element from the map
loop {
let handle = self
.get_impl(timeline_id, shard_selector, tenant_manager)
.await?;
if handle.0.shut_down.load(Ordering::Relaxed) {
let removed = self
.map
.remove(&handle.0.timeline.shard_timeline_id())
.expect("invariant of get_impl is that the returned handle is in the map");
assert!(
Weak::ptr_eq(&removed, &Arc::downgrade(&handle.0)),
"shard_timeline_id() incorrect?"
);
} else {
return Ok(handle);
}
}
}
#[instrument(level = "trace", skip_all)]
async fn get_impl(
&mut self,
timeline_id: TimelineId,
shard_selector: ShardSelector,
tenant_manager: &T::TenantManager,
) -> Result<Handle<T>, GetError<T>> {
let miss: ShardSelector = {
// terminates because when every iteration we remove an element from the map
let miss: ShardSelector = loop {
let routing_state = self.shard_routing(timeline_id, shard_selector);
match routing_state {
RoutingResult::FastPath(handle) => return Ok(handle),
RoutingResult::SlowPath(key) => match self.map.get(&key) {
Some(cached) => match cached.upgrade() {
Some(upgraded) => return Ok(Handle(upgraded)),
None => {
Ok(upgraded) => return Ok(upgraded),
Err(HandleUpgradeError::ShutDown) => {
// TODO: dedup with shard_routing()
trace!("handle cache stale");
self.map.remove(&key).unwrap();
ShardSelector::Known(key.shard_index)
continue;
}
},
None => ShardSelector::Known(key.shard_index),
None => break ShardSelector::Known(key.shard_index),
},
RoutingResult::NeedConsultTenantManager => shard_selector,
RoutingResult::NeedConsultTenantManager => break shard_selector,
}
};
self.get_miss(timeline_id, miss, tenant_manager).await
@@ -302,7 +383,7 @@ impl<T: Types> Cache<T> {
let Some((first_key, first_handle)) = self.map.iter().next() else {
return RoutingResult::NeedConsultTenantManager;
};
let Some(first_handle) = first_handle.upgrade() else {
let Ok(first_handle) = first_handle.upgrade() else {
// TODO: dedup with get()
trace!("handle cache stale");
let first_key_owned = *first_key;
@@ -310,7 +391,7 @@ impl<T: Types> Cache<T> {
continue;
};
let first_handle_shard_identity = first_handle.timeline.get_shard_identity();
let first_handle_shard_identity = first_handle.get_shard_identity();
let make_shard_index = |shard_num: ShardNumber| ShardIndex {
shard_number: shard_num,
shard_count: first_handle_shard_identity.count,
@@ -329,11 +410,11 @@ impl<T: Types> Cache<T> {
};
let first_handle_shard_timeline_id = ShardTimelineId {
shard_index: first_handle_shard_identity.shard_index(),
timeline_id: first_handle.timeline.shard_timeline_id().timeline_id,
timeline_id: first_handle.shard_timeline_id().timeline_id,
};
if need_shard_timeline_id == first_handle_shard_timeline_id {
return RoutingResult::FastPath(Handle(first_handle));
return RoutingResult::FastPath(first_handle);
} else {
return RoutingResult::SlowPath(need_shard_timeline_id);
}
@@ -357,23 +438,30 @@ impl<T: Types> Cache<T> {
ShardSelector::Known(idx) => assert_eq!(idx, &key.shard_index),
}
let gate_guard = match timeline.gate().enter() {
Ok(guard) => guard,
Err(_) => {
return Err(GetError::TimelineGateClosed);
}
};
trace!("creating new HandleInner");
let handle = Arc::new(
// TODO: global metric that keeps track of the number of live HandlerTimeline instances
// so we can identify reference cycle bugs.
HandleInner {
shut_down: AtomicBool::new(false),
_gate_guard: gate_guard,
timeline: timeline.clone(),
},
);
let handle = {
let handle_inner_arc = Arc::new(Mutex::new(HandleInner::KeepingTimelineGateOpen {
gate_guard: Arc::new(
// this enter() is expensive in production code because
// it hits the global Arc<Timeline>::gate refcounts
match timeline.gate().enter() {
Ok(guard) => guard,
Err(_) => {
return Err(GetError::TimelineGateClosed);
}
},
),
// this clone is expensive in production code because
// it hits the global Arc<Timeline>::clone refcounts
timeline: Arc::new(timeline.clone()),
}));
let handle_weak = WeakHandle {
inner: Arc::downgrade(&handle_inner_arc),
};
let handle = handle_weak
.upgrade()
.ok()
.expect("we just created it and it's not linked anywhere yet");
{
let mut lock_guard = timeline
.per_timeline_state()
.handles
@@ -381,7 +469,8 @@ impl<T: Types> Cache<T> {
.expect("mutex poisoned");
match &mut *lock_guard {
Some(per_timeline_state) => {
let replaced = per_timeline_state.insert(self.id, Arc::clone(&handle));
let replaced =
per_timeline_state.insert(self.id, Arc::clone(&handle_inner_arc));
assert!(replaced.is_none(), "some earlier code left a stale handle");
match self.map.entry(key) {
hash_map::Entry::Occupied(_o) => {
@@ -392,8 +481,7 @@ impl<T: Types> Cache<T> {
unreachable!()
}
hash_map::Entry::Vacant(v) => {
v.insert(Arc::downgrade(&handle));
handle
v.insert(handle_weak);
}
}
}
@@ -401,14 +489,62 @@ impl<T: Types> Cache<T> {
return Err(GetError::PerTimelineStateShutDown);
}
}
};
Ok(Handle(handle))
}
Ok(handle)
}
Err(e) => Err(GetError::TenantManager(e)),
}
}
}
pub(crate) enum HandleUpgradeError {
ShutDown,
}
impl<T: Types> WeakHandle<T> {
pub(crate) fn upgrade(&self) -> Result<Handle<T>, HandleUpgradeError> {
let Some(inner) = Weak::upgrade(&self.inner) else {
return Err(HandleUpgradeError::ShutDown);
};
let lock_guard = inner.lock().expect("poisoned");
match &*lock_guard {
HandleInner::KeepingTimelineGateOpen {
timeline,
gate_guard,
} => {
let gate_guard = Arc::clone(gate_guard);
let timeline = Arc::clone(timeline);
drop(lock_guard);
Ok(Handle {
timeline,
gate_guard,
inner,
})
}
HandleInner::ShutDown => Err(HandleUpgradeError::ShutDown),
}
}
pub(crate) fn is_same_handle_as(&self, other: &WeakHandle<T>) -> bool {
Weak::ptr_eq(&self.inner, &other.inner)
}
}
impl<T: Types> std::ops::Deref for Handle<T> {
type Target = T::Timeline;
fn deref(&self) -> &Self::Target {
&self.timeline
}
}
impl<T: Types> Handle<T> {
pub(crate) fn downgrade(&self) -> WeakHandle<T> {
WeakHandle {
inner: Arc::downgrade(&self.inner),
}
}
}
impl<T: Types> PerTimelineState<T> {
/// After this method returns, [`Cache::get`] will never again return a [`Handle`]
/// to the [`Types::Timeline`] that embeds this per-timeline state.
@@ -430,43 +566,62 @@ impl<T: Types> PerTimelineState<T> {
trace!("already shut down");
return;
};
for handle in handles.values() {
for handle_inner_arc in handles.values() {
// Make hits fail.
handle.shut_down.store(true, Ordering::Relaxed);
let mut lock_guard = handle_inner_arc.lock().expect("poisoned");
lock_guard.shutdown();
}
drop(handles);
}
}
impl<T: Types> std::ops::Deref for Handle<T> {
type Target = T::Timeline;
fn deref(&self) -> &Self::Target {
&self.0.timeline
}
}
#[cfg(test)]
impl<T: Types> Drop for HandleInner<T> {
fn drop(&mut self) {
trace!("HandleInner dropped");
}
}
// When dropping a [`Cache`], prune its handles in the [`PerTimelineState`] to break the reference cycle.
impl<T: Types> Drop for Cache<T> {
fn drop(&mut self) {
for (_, weak) in self.map.drain() {
if let Some(strong) = weak.upgrade() {
// handle is still being kept alive in PerTimelineState
let timeline = strong.timeline.per_timeline_state();
let mut handles = timeline.handles.lock().expect("mutex poisoned");
if let Some(handles) = &mut *handles {
let Some(removed) = handles.remove(&self.id) else {
// There could have been a shutdown inbetween us upgrading the weak and locking the mutex.
continue;
};
assert!(Arc::ptr_eq(&removed, &strong));
}
for (
_,
WeakHandle {
inner: handle_inner_weak,
},
) in self.map.drain()
{
let Some(handle_inner_arc) = handle_inner_weak.upgrade() else {
continue;
};
let Some(handle_timeline) = handle_inner_arc
// locking rules: drop lock before acquiring other lock below
.lock()
.expect("poisoned")
.shutdown()
else {
// Concurrent PerTimelineState::shutdown.
continue;
};
// Clean up per_timeline_state so the HandleInner allocation can be dropped.
let per_timeline_state = handle_timeline.per_timeline_state();
let mut handles_lock_guard = per_timeline_state.handles.lock().expect("mutex poisoned");
let Some(handles) = &mut *handles_lock_guard else {
continue;
};
let Some(removed_handle_inner_arc) = handles.remove(&self.id) else {
// Concurrent PerTimelineState::shutdown.
continue;
};
drop(handles_lock_guard); // locking rules!
assert!(Arc::ptr_eq(&removed_handle_inner_arc, &handle_inner_arc));
}
}
}
impl<T: Types> HandleInner<T> {
fn shutdown(&mut self) -> Option<Arc<T::Timeline>> {
match std::mem::replace(self, HandleInner::ShutDown) {
HandleInner::KeepingTimelineGateOpen { timeline, .. } => Some(timeline),
HandleInner::ShutDown => {
// Duplicate shutdowns are possible because both Cache::drop and PerTimelineState::shutdown
// may do it concurrently, but locking rules disallow holding per-timeline-state lock and
// the handle lock at the same time.
None
}
}
}
@@ -474,6 +629,8 @@ impl<T: Types> Drop for Cache<T> {
#[cfg(test)]
mod tests {
use std::sync::Weak;
use pageserver_api::{
key::{rel_block_to_key, Key, DBDIR_KEY},
models::ShardParameters,
@@ -583,39 +740,13 @@ mod tests {
//
// fill the cache
//
assert_eq!(
(Arc::strong_count(&shard0), Arc::weak_count(&shard0)),
(2, 1),
"strong: shard0, mgr; weak: myself"
);
let handle: Handle<_> = cache
.get(timeline_id, ShardSelector::Page(key), &mgr)
.await
.expect("we have the timeline");
let handle_inner_weak = Arc::downgrade(&handle.0);
assert!(Weak::ptr_eq(&handle.myself, &shard0.myself));
assert_eq!(
(
Weak::strong_count(&handle_inner_weak),
Weak::weak_count(&handle_inner_weak)
),
(2, 2),
"strong: handle, per_timeline_state, weak: handle_inner_weak, cache"
);
assert_eq!(cache.map.len(), 1);
assert_eq!(
(Arc::strong_count(&shard0), Arc::weak_count(&shard0)),
(3, 1),
"strong: handleinner(per_timeline_state), shard0, mgr; weak: myself"
);
drop(handle);
assert_eq!(
(Arc::strong_count(&shard0), Arc::weak_count(&shard0)),
(3, 1),
"strong: handleinner(per_timeline_state), shard0, mgr; weak: myself"
);
//
// demonstrate that Handle holds up gate closure
@@ -640,21 +771,11 @@ mod tests {
// SHUTDOWN
shard0.per_timeline_state.shutdown(); // keeping handle alive across shutdown
assert_eq!(
1,
Weak::strong_count(&handle_inner_weak),
"through local var handle"
);
assert_eq!(
cache.map.len(),
1,
"this is an implementation detail but worth pointing out: we can't clear the cache from shutdown(), it's cleared on first access after"
);
assert_eq!(
(Arc::strong_count(&shard0), Arc::weak_count(&shard0)),
(3, 1),
"strong: handleinner(via handle), shard0, mgr; weak: myself"
);
// this handle is perfectly usable
handle.getpage();
@@ -678,16 +799,6 @@ mod tests {
}
drop(handle);
assert_eq!(
0,
Weak::strong_count(&handle_inner_weak),
"the HandleInner destructor already ran"
);
assert_eq!(
(Arc::strong_count(&shard0), Arc::weak_count(&shard0)),
(2, 1),
"strong: shard0, mgr; weak: myself"
);
// closing gate succeeds after dropping handle
tokio::select! {
@@ -706,10 +817,8 @@ mod tests {
assert_eq!(cache.map.len(), 0);
// ensure all refs to shard0 are gone and we're not leaking anything
let myself = Weak::clone(&shard0.myself);
drop(shard0);
drop(mgr);
assert_eq!(Weak::strong_count(&myself), 0);
}
#[tokio::test]
@@ -948,15 +1057,11 @@ mod tests {
handle
};
handle.getpage();
used_handles.push(Arc::downgrade(&handle.0));
used_handles.push(Arc::downgrade(&handle.timeline));
}
// No handles exist, thus gates are closed and don't require shutdown
assert!(used_handles
.iter()
.all(|weak| Weak::strong_count(weak) == 0));
// ... thus the gate should close immediately, even without shutdown
// No handles exist, thus gates are closed and don't require shutdown.
// Thus the gate should close immediately, even without shutdown.
tokio::select! {
_ = shard0.gate.close() => { }
_ = tokio::time::sleep(FOREVER) => {
@@ -964,4 +1069,172 @@ mod tests {
}
}
}
#[tokio::test(start_paused = true)]
async fn test_weak_handles() {
crate::tenant::harness::setup_logging();
let timeline_id = TimelineId::generate();
let shard0 = Arc::new_cyclic(|myself| StubTimeline {
gate: Default::default(),
id: timeline_id,
shard: ShardIdentity::unsharded(),
per_timeline_state: PerTimelineState::default(),
myself: myself.clone(),
});
let mgr = StubManager {
shards: vec![shard0.clone()],
};
let refcount_start = Arc::strong_count(&shard0);
let key = DBDIR_KEY;
let mut cache = Cache::<TestTypes>::default();
let handle = cache
.get(timeline_id, ShardSelector::Page(key), &mgr)
.await
.expect("we have the timeline");
assert!(Weak::ptr_eq(&handle.myself, &shard0.myself));
let weak_handle = handle.downgrade();
drop(handle);
let upgraded_handle = weak_handle.upgrade().ok().expect("we can upgrade it");
// Start shutdown
shard0.per_timeline_state.shutdown();
// Upgrades during shutdown don't work, even if upgraded_handle exists.
weak_handle
.upgrade()
.err()
.expect("can't upgrade weak handle as soon as shutdown started");
// But upgraded_handle is still alive, so the gate won't close.
tokio::select! {
_ = shard0.gate.close() => {
panic!("handle is keeping gate open");
}
_ = tokio::time::sleep(FOREVER) => { }
}
// Drop the last handle.
drop(upgraded_handle);
// The gate should close now, despite there still being a weak_handle.
tokio::select! {
_ = shard0.gate.close() => { }
_ = tokio::time::sleep(FOREVER) => {
panic!("only strong handle is dropped and we shut down per-timeline-state")
}
}
// The weak handle still can't be upgraded.
weak_handle
.upgrade()
.err()
.expect("still shouldn't be able to upgrade the weak handle");
// There should be no strong references to the timeline object except the one on "stack".
assert_eq!(Arc::strong_count(&shard0), refcount_start);
}
#[tokio::test(start_paused = true)]
async fn test_reference_cycle_broken_when_cache_is_dropped() {
crate::tenant::harness::setup_logging();
let timeline_id = TimelineId::generate();
let shard0 = Arc::new_cyclic(|myself| StubTimeline {
gate: Default::default(),
id: timeline_id,
shard: ShardIdentity::unsharded(),
per_timeline_state: PerTimelineState::default(),
myself: myself.clone(),
});
let mgr = StubManager {
shards: vec![shard0.clone()],
};
let key = DBDIR_KEY;
let mut cache = Cache::<TestTypes>::default();
// helper to check if a handle is referenced by per_timeline_state
let per_timeline_state_refs_handle = |handle_weak: &Weak<Mutex<HandleInner<_>>>| {
let per_timeline_state = shard0.per_timeline_state.handles.lock().unwrap();
let per_timeline_state = per_timeline_state.as_ref().unwrap();
per_timeline_state
.values()
.any(|v| Weak::ptr_eq(&Arc::downgrade(v), handle_weak))
};
// Fill the cache.
let handle = cache
.get(timeline_id, ShardSelector::Page(key), &mgr)
.await
.expect("we have the timeline");
assert!(Weak::ptr_eq(&handle.myself, &shard0.myself));
let handle_inner_weak = Arc::downgrade(&handle.inner);
assert!(
per_timeline_state_refs_handle(&handle_inner_weak),
"we still hold `handle` _and_ haven't dropped `cache` yet"
);
// Drop the cache.
drop(cache);
assert!(
!(per_timeline_state_refs_handle(&handle_inner_weak)),
"nothing should reference the handle allocation anymore"
);
assert!(
Weak::upgrade(&handle_inner_weak).is_some(),
"the local `handle` still keeps the allocation alive"
);
// but obviously the cache is gone so no new allocations can be handed out.
// Drop handle.
drop(handle);
assert!(
Weak::upgrade(&handle_inner_weak).is_none(),
"the local `handle` is dropped, so the allocation should be dropped by now"
);
}
#[tokio::test(start_paused = true)]
async fn test_reference_cycle_broken_when_per_timeline_state_shutdown() {
crate::tenant::harness::setup_logging();
let timeline_id = TimelineId::generate();
let shard0 = Arc::new_cyclic(|myself| StubTimeline {
gate: Default::default(),
id: timeline_id,
shard: ShardIdentity::unsharded(),
per_timeline_state: PerTimelineState::default(),
myself: myself.clone(),
});
let mgr = StubManager {
shards: vec![shard0.clone()],
};
let key = DBDIR_KEY;
let mut cache = Cache::<TestTypes>::default();
let handle = cache
.get(timeline_id, ShardSelector::Page(key), &mgr)
.await
.expect("we have the timeline");
// grab a weak reference to the inner so can later try to Weak::upgrade it and assert that fails
let handle_inner_weak = Arc::downgrade(&handle.inner);
// drop the handle, obviously the lifetime of `inner` is at least as long as each strong reference to it
drop(handle);
assert!(Weak::upgrade(&handle_inner_weak).is_some(), "can still");
// Shutdown the per_timeline_state.
shard0.per_timeline_state.shutdown();
assert!(Weak::upgrade(&handle_inner_weak).is_none(), "can no longer");
// cache only contains Weak's, so, it can outlive the per_timeline_state without
// Drop explicitly solely to make this point.
drop(cache);
}
}

View File

@@ -2,10 +2,11 @@ use std::sync::Arc;
use pageserver_api::models::{TenantState, TimelineState};
use super::delete::{delete_local_timeline_directory, DeleteTimelineFlow, DeletionGuard};
use super::delete::{delete_local_timeline_directory, DeletionGuard};
use super::Timeline;
use crate::span::debug_assert_current_span_has_tenant_and_timeline_id;
use crate::tenant::remote_timeline_client::ShutdownIfArchivedError;
use crate::tenant::timeline::delete::{make_timeline_delete_guard, TimelineDeleteGuardKind};
use crate::tenant::{OffloadedTimeline, Tenant, TenantManifestError, TimelineOrOffloaded};
#[derive(thiserror::Error, Debug)]
@@ -36,13 +37,10 @@ pub(crate) async fn offload_timeline(
debug_assert_current_span_has_tenant_and_timeline_id();
tracing::info!("offloading archived timeline");
let allow_offloaded_children = true;
let set_stopping = false;
let (timeline, guard) = DeleteTimelineFlow::prepare(
let (timeline, guard) = make_timeline_delete_guard(
tenant,
timeline.timeline_id,
allow_offloaded_children,
set_stopping,
TimelineDeleteGuardKind::Offload,
)
.map_err(|e| OffloadError::Other(anyhow::anyhow!(e)))?;
@@ -106,7 +104,7 @@ pub(crate) async fn offload_timeline(
}
/// It is important that this gets called when DeletionGuard is being held.
/// For more context see comments in [`DeleteTimelineFlow::prepare`]
/// For more context see comments in [`make_timeline_delete_guard`]
///
/// Returns the strong count of the timeline `Arc`
fn remove_timeline_from_tenant(

View File

@@ -140,7 +140,7 @@ pub(super) async fn handle_walreceiver_connection(
let (replication_client, connection) = {
let mut config = wal_source_connconf.to_tokio_postgres_config();
config.application_name("pageserver");
config.application_name(format!("pageserver-{}", node.0).as_str());
config.replication_mode(tokio_postgres::config::ReplicationMode::Physical);
match time::timeout(connect_timeout, config.connect(postgres::NoTls)).await {
Ok(client_and_conn) => client_and_conn?,

View File

@@ -911,57 +911,85 @@ lfc_writev(NRelFileInfo rinfo, ForkNumber forkNum, BlockNumber blkno,
if (entry->access_count++ == 0)
dlist_delete(&entry->list_node);
}
else
/*-----------
* If the chunk wasn't already in the LFC then we have these
* options, in order of preference:
*
* Unless there is no space available, we can:
* 1. Use an entry from the `holes` list, and
* 2. Create a new entry.
* We can always, regardless of space in the LFC:
* 3. evict an entry from LRU, and
* 4. ignore the write operation (the least favorite option)
*/
else if (lfc_ctl->used < lfc_ctl->limit)
{
/*
* We have two choices if all cache pages are pinned (i.e. used in IO
* operations):
*
* 1) Wait until some of this operation is completed and pages is
* unpinned.
*
* 2) Allocate one more chunk, so that specified cache size is more
* recommendation than hard limit.
*
* As far as probability of such event (that all pages are pinned) is
* considered to be very very small: there are should be very large
* number of concurrent IO operations and them are limited by
* max_connections, we prefer not to complicate code and use second
* approach.
*/
if (lfc_ctl->used >= lfc_ctl->limit && !dlist_is_empty(&lfc_ctl->lru))
{
/* Cache overflow: evict least recently used chunk */
FileCacheEntry *victim = dlist_container(FileCacheEntry, list_node, dlist_pop_head_node(&lfc_ctl->lru));
for (int i = 0; i < BLOCKS_PER_CHUNK; i++)
{
lfc_ctl->used_pages -= (victim->bitmap[i >> 5] >> (i & 31)) & 1;
}
CriticalAssert(victim->access_count == 0);
entry->offset = victim->offset; /* grab victim's chunk */
hash_search_with_hash_value(lfc_hash, &victim->key, victim->hash, HASH_REMOVE, NULL);
neon_log(DEBUG2, "Swap file cache page");
}
else if (!dlist_is_empty(&lfc_ctl->holes))
if (!dlist_is_empty(&lfc_ctl->holes))
{
/* We can reuse a hole that was left behind when the LFC was shrunk previously */
FileCacheEntry *hole = dlist_container(FileCacheEntry, list_node, dlist_pop_head_node(&lfc_ctl->holes));
uint32 offset = hole->offset;
bool hole_found;
hash_search_with_hash_value(lfc_hash, &hole->key, hole->hash, HASH_REMOVE, &hole_found);
FileCacheEntry *hole = dlist_container(FileCacheEntry, list_node,
dlist_pop_head_node(&lfc_ctl->holes));
uint32 offset = hole->offset;
bool hole_found;
hash_search_with_hash_value(lfc_hash, &hole->key,
hole->hash, HASH_REMOVE, &hole_found);
CriticalAssert(hole_found);
lfc_ctl->used += 1;
entry->offset = offset; /* reuse the hole */
entry->offset = offset; /* reuse the hole */
}
else
{
lfc_ctl->used += 1;
entry->offset = lfc_ctl->size++; /* allocate new chunk at end
* of file */
entry->offset = lfc_ctl->size++;/* allocate new chunk at end
* of file */
}
}
/*
* We've already used up all allocated LFC entries.
*
* If we can clear an entry from the LRU, do that.
* If we can't (e.g. because all other slots are being accessed)
* then we will remove this entry from the hash and continue
* on to the next chunk, as we may not exceed the limit.
*/
else if (!dlist_is_empty(&lfc_ctl->lru))
{
/* Cache overflow: evict least recently used chunk */
FileCacheEntry *victim = dlist_container(FileCacheEntry, list_node,
dlist_pop_head_node(&lfc_ctl->lru));
for (int i = 0; i < BLOCKS_PER_CHUNK; i++)
{
lfc_ctl->used_pages -= (victim->bitmap[i >> 5] >> (i & 31)) & 1;
}
CriticalAssert(victim->access_count == 0);
entry->offset = victim->offset; /* grab victim's chunk */
hash_search_with_hash_value(lfc_hash, &victim->key,
victim->hash, HASH_REMOVE, NULL);
neon_log(DEBUG2, "Swap file cache page");
}
else
{
/* Can't add this chunk - we don't have the space for it */
hash_search_with_hash_value(lfc_hash, &entry->key, hash,
HASH_REMOVE, NULL);
/*
* We can't process this chunk due to lack of space in LFC,
* so skip to the next one
*/
LWLockRelease(lfc_lock);
blkno += blocks_in_chunk;
buf_offset += blocks_in_chunk;
nblocks -= blocks_in_chunk;
continue;
}
if (!found)
{
entry->access_count = 1;
entry->hash = hash;
memset(entry->bitmap, 0, sizeof entry->bitmap);

View File

@@ -373,8 +373,8 @@ pageserver_connect(shardno_t shard_no, int elevel)
{
case PS_Disconnected:
{
const char *keywords[3];
const char *values[3];
const char *keywords[4];
const char *values[4];
int n_pgsql_params;
TimestampTz now;
int64 us_since_last_attempt;
@@ -421,12 +421,14 @@ pageserver_connect(shardno_t shard_no, int elevel)
*/
keywords[0] = "dbname";
values[0] = connstr;
n_pgsql_params = 1;
keywords[1] = "keepalives";
values[1] = "1";
n_pgsql_params = 2;
if (neon_auth_token)
{
keywords[1] = "password";
values[1] = neon_auth_token;
keywords[2] = "password";
values[2] = neon_auth_token;
n_pgsql_params++;
}

View File

@@ -34,6 +34,8 @@ typedef enum
T_NeonGetPageRequest,
T_NeonDbSizeRequest,
T_NeonGetSlruSegmentRequest,
/* future tags above this line */
T_NeonTestRequest = 99, /* only in cfg(feature = "testing") */
/* pagestore -> pagestore_client */
T_NeonExistsResponse = 100,
@@ -42,6 +44,8 @@ typedef enum
T_NeonErrorResponse,
T_NeonDbSizeResponse,
T_NeonGetSlruSegmentResponse,
/* future tags above this line */
T_NeonTestResponse = 199, /* only in cfg(feature = "testing") */
} NeonMessageTag;
typedef uint64 NeonRequestId;

View File

@@ -102,6 +102,11 @@ impl Client {
self.get(&uri).await
}
pub async fn utilization(&self) -> Result<reqwest::Response> {
let uri = format!("{}/v1/utilization/", self.mgmt_api_endpoint);
self.get(&uri).await
}
async fn get<U: IntoUrl>(&self, uri: U) -> Result<reqwest::Response> {
self.request(Method::GET, uri, ()).await
}

View File

@@ -127,6 +127,13 @@ async fn timeline_create_handler(mut request: Request<Body>) -> Result<Response<
json_response(StatusCode::OK, ())
}
async fn utilization_handler(request: Request<Body>) -> Result<Response<Body>, ApiError> {
check_permission(&request, None)?;
let global_timelines = get_global_timelines(&request);
let utilization = global_timelines.get_timeline_counts();
json_response(StatusCode::OK, utilization)
}
/// List all (not deleted) timelines.
/// Note: it is possible to do the same with debug_dump.
async fn timeline_list_handler(request: Request<Body>) -> Result<Response<Body>, ApiError> {
@@ -620,6 +627,7 @@ pub fn make_router(
failpoints_handler(r, cancel).await
})
})
.get("/v1/uzilization", |r| request_span(r, utilization_handler))
.delete("/v1/tenant/:tenant_id", |r| {
request_span(r, tenant_delete_handler)
})

View File

@@ -13,6 +13,7 @@ use anyhow::{bail, Context, Result};
use camino::Utf8PathBuf;
use camino_tempfile::Utf8TempDir;
use safekeeper_api::membership::Configuration;
use safekeeper_api::models::SafekeeperUtilization;
use safekeeper_api::ServerInfo;
use serde::Serialize;
use std::collections::HashMap;
@@ -416,6 +417,20 @@ impl GlobalTimelines {
.collect()
}
/// Returns statistics about timeline counts
pub fn get_timeline_counts(&self) -> SafekeeperUtilization {
let global_lock = self.state.lock().unwrap();
let timeline_count = global_lock
.timelines
.values()
.filter(|t| match t {
GlobalMapTimeline::CreationInProgress => false,
GlobalMapTimeline::Timeline(t) => !t.is_cancelled(),
})
.count() as u64;
SafekeeperUtilization { timeline_count }
}
/// Returns all timelines belonging to a given tenant. Used for deleting all timelines of a tenant,
/// and that's why it can return cancelled timelines, to retry deleting them.
fn get_all_for_tenant(&self, tenant_id: TenantId) -> Vec<Arc<Timeline>> {

View File

@@ -1,11 +1,17 @@
use std::{sync::Arc, time::Duration};
use std::{
collections::{BTreeMap, HashMap},
sync::Arc,
time::Duration,
};
use pageserver_api::controller_api::ShardSchedulingPolicy;
use rand::seq::SliceRandom;
use rand::thread_rng;
use tokio_util::sync::CancellationToken;
use utils::id::NodeId;
use utils::shard::TenantShardId;
use super::Service;
use super::{Node, Scheduler, Service, TenantShard};
pub struct ChaosInjector {
service: Arc<Service>,
@@ -35,50 +41,86 @@ impl ChaosInjector {
}
}
/// If a shard has a secondary and attached location, then re-assign the secondary to be
/// attached and the attached to be secondary.
///
/// Only modifies tenants if they're in Active scheduling policy.
fn maybe_migrate_to_secondary(
&self,
tenant_shard_id: TenantShardId,
nodes: &Arc<HashMap<NodeId, Node>>,
tenants: &mut BTreeMap<TenantShardId, TenantShard>,
scheduler: &mut Scheduler,
) {
let shard = tenants
.get_mut(&tenant_shard_id)
.expect("Held lock between choosing ID and this get");
if !matches!(shard.get_scheduling_policy(), ShardSchedulingPolicy::Active) {
// Skip non-active scheduling policies, so that a shard with a policy like Pause can
// be pinned without being disrupted by us.
tracing::info!(
"Skipping shard {tenant_shard_id}: scheduling policy is {:?}",
shard.get_scheduling_policy()
);
return;
}
// Pick a secondary to promote
let Some(new_location) = shard
.intent
.get_secondary()
.choose(&mut thread_rng())
.cloned()
else {
tracing::info!(
"Skipping shard {tenant_shard_id}: no secondary location, can't migrate"
);
return;
};
let Some(old_location) = *shard.intent.get_attached() else {
tracing::info!("Skipping shard {tenant_shard_id}: currently has no attached location");
return;
};
tracing::info!("Injecting chaos: migrate {tenant_shard_id} {old_location}->{new_location}");
shard.intent.demote_attached(scheduler, old_location);
shard.intent.promote_attached(scheduler, new_location);
self.service.maybe_reconcile_shard(shard, nodes);
}
async fn inject_chaos(&mut self) {
// Pick some shards to interfere with
let batch_size = 128;
let mut inner = self.service.inner.write().unwrap();
let (nodes, tenants, scheduler) = inner.parts_mut();
let tenant_ids = tenants.keys().cloned().collect::<Vec<_>>();
let victims = tenant_ids.choose_multiple(&mut thread_rng(), batch_size);
for victim in victims {
let shard = tenants
.get_mut(victim)
.expect("Held lock between choosing ID and this get");
if !matches!(shard.get_scheduling_policy(), ShardSchedulingPolicy::Active) {
// Skip non-active scheduling policies, so that a shard with a policy like Pause can
// be pinned without being disrupted by us.
tracing::info!(
"Skipping shard {victim}: scheduling policy is {:?}",
shard.get_scheduling_policy()
);
continue;
// Prefer to migrate tenants that are currently outside their home AZ. This avoids the chaos injector
// continuously pushing tenants outside their home AZ: instead, we'll tend to cycle between picking some
// random tenants to move, and then on next chaos iteration moving them back, then picking some new
// random tenants on the next iteration.
let mut victims = Vec::with_capacity(batch_size);
for shard in tenants.values() {
if shard.is_attached_outside_preferred_az(nodes) {
victims.push(shard.tenant_shard_id);
}
// Pick a secondary to promote
let Some(new_location) = shard
.intent
.get_secondary()
.choose(&mut thread_rng())
.cloned()
else {
tracing::info!("Skipping shard {victim}: no secondary location, can't migrate");
continue;
};
if victims.len() >= batch_size {
break;
}
}
let Some(old_location) = *shard.intent.get_attached() else {
tracing::info!("Skipping shard {victim}: currently has no attached location");
continue;
};
let choose_random = batch_size.saturating_sub(victims.len());
tracing::info!("Injecting chaos: found {} shards to migrate back to home AZ, picking {choose_random} random shards to migrate", victims.len());
tracing::info!("Injecting chaos: migrate {victim} {old_location}->{new_location}");
let random_victims = tenant_ids.choose_multiple(&mut thread_rng(), choose_random);
victims.extend(random_victims);
shard.intent.demote_attached(scheduler, old_location);
shard.intent.promote_attached(scheduler, new_location);
self.service.maybe_reconcile_shard(shard, nodes);
for victim in victims {
self.maybe_migrate_to_secondary(victim, nodes, tenants, scheduler);
}
}
}

View File

@@ -1793,6 +1793,23 @@ impl TenantShard {
}
}
}
/// Returns true if the tenant shard is attached to a node that is outside the preferred AZ.
///
/// If the shard does not have a preferred AZ, returns false.
pub(crate) fn is_attached_outside_preferred_az(&self, nodes: &HashMap<NodeId, Node>) -> bool {
self.intent
.get_attached()
.map(|node_id| {
Some(
nodes
.get(&node_id)
.expect("referenced node exists")
.get_availability_zone_id(),
) == self.intent.preferred_az_id.as_ref()
})
.unwrap_or(false)
}
}
impl Drop for TenantShard {

View File

@@ -8,6 +8,8 @@ use crate::checks::{
};
use crate::metadata_stream::{stream_tenant_timelines, stream_tenants};
use crate::{init_remote, BucketConfig, NodeKind, RootTarget, TenantShardTimelineId, MAX_RETRIES};
use async_stream::try_stream;
use futures::future::Either;
use futures_util::{StreamExt, TryStreamExt};
use pageserver::tenant::remote_timeline_client::index::LayerFileMetadata;
use pageserver::tenant::remote_timeline_client::manifest::OffloadedTimelineManifest;
@@ -578,7 +580,7 @@ async fn gc_timeline(
target: &RootTarget,
mode: GcMode,
ttid: TenantShardTimelineId,
accumulator: &Arc<std::sync::Mutex<TenantRefAccumulator>>,
accumulator: &std::sync::Mutex<TenantRefAccumulator>,
tenant_manifest_info: Arc<Option<RemoteTenantManifestInfo>>,
) -> anyhow::Result<GcSummary> {
let mut summary = GcSummary::default();
@@ -721,9 +723,9 @@ pub async fn pageserver_physical_gc(
let remote_client = Arc::new(remote_client);
let tenants = if tenant_shard_ids.is_empty() {
futures::future::Either::Left(stream_tenants(&remote_client, &target))
Either::Left(stream_tenants(&remote_client, &target))
} else {
futures::future::Either::Right(futures::stream::iter(tenant_shard_ids.into_iter().map(Ok)))
Either::Right(futures::stream::iter(tenant_shard_ids.into_iter().map(Ok)))
};
// How many tenants to process in parallel. We need to be mindful of pageservers
@@ -731,16 +733,16 @@ pub async fn pageserver_physical_gc(
const CONCURRENCY: usize = 32;
// Accumulate information about each tenant for cross-shard GC step we'll do at the end
let accumulator = Arc::new(std::sync::Mutex::new(TenantRefAccumulator::default()));
let accumulator = std::sync::Mutex::new(TenantRefAccumulator::default());
// Accumulate information about how many manifests we have GCd
let manifest_gc_summary = std::sync::Mutex::new(GcSummary::default());
// Generate a stream of TenantTimelineId
enum GcSummaryOrContent<T> {
Content(T),
GcSummary(GcSummary),
}
let timelines = tenants.map_ok(|tenant_shard_id| {
let target_ref = &target;
let remote_client_ref = &remote_client;
let manifest_gc_summary_ref = &manifest_gc_summary;
async move {
let gc_manifest_result = gc_tenant_manifests(
remote_client_ref,
@@ -757,55 +759,48 @@ pub async fn pageserver_physical_gc(
(GcSummary::default(), None)
}
};
manifest_gc_summary_ref
.lock()
.unwrap()
.merge(summary_from_manifest);
let tenant_manifest_arc = Arc::new(tenant_manifest_opt);
let summary_from_manifest = Ok(GcSummaryOrContent::<(_, _)>::GcSummary(
summary_from_manifest,
));
stream_tenant_timelines(remote_client_ref, target_ref, tenant_shard_id)
.await
.map(|stream| {
stream
.zip(futures::stream::iter(std::iter::repeat(
tenant_manifest_arc,
)))
.map(|(ttid_res, tenant_manifest_arc)| {
ttid_res.map(move |ttid| {
GcSummaryOrContent::Content((ttid, tenant_manifest_arc))
})
})
.chain(futures::stream::iter([summary_from_manifest].into_iter()))
})
let mut timelines = Box::pin(
stream_tenant_timelines(remote_client_ref, target_ref, tenant_shard_id).await?,
);
Ok(try_stream! {
while let Some(ttid_res) = timelines.next().await {
let ttid = ttid_res?;
yield (ttid, tenant_manifest_arc.clone());
}
})
}
});
let timelines = std::pin::pin!(timelines.try_buffered(CONCURRENCY));
let timelines = timelines.try_flatten();
let mut summary = GcSummary::default();
// Drain futures for per-shard GC, populating accumulator as a side effect
{
let timelines = timelines.map_ok(|summary_or_ttid| match summary_or_ttid {
GcSummaryOrContent::Content((ttid, tenant_manifest_arc)) => {
futures::future::Either::Left(gc_timeline(
&remote_client,
&min_age,
&target,
mode,
ttid,
&accumulator,
tenant_manifest_arc,
))
}
GcSummaryOrContent::GcSummary(gc_summary) => {
futures::future::Either::Right(futures::future::ok(gc_summary))
}
let timelines = std::pin::pin!(timelines.try_buffered(CONCURRENCY));
let timelines = timelines.try_flatten();
let timelines = timelines.map_ok(|(ttid, tenant_manifest_arc)| {
gc_timeline(
&remote_client,
&min_age,
&target,
mode,
ttid,
&accumulator,
tenant_manifest_arc,
)
});
let mut timelines = std::pin::pin!(timelines.try_buffered(CONCURRENCY));
// Drain futures for per-shard GC, populating accumulator as a side effect
while let Some(i) = timelines.next().await {
summary.merge(i?);
}
}
// Streams are lazily evaluated, so only now do we have access to the inner object
summary.merge(manifest_gc_summary.into_inner().unwrap());
// Execute cross-shard GC, using the accumulator's full view of all the shards built in the per-shard GC
let Some(client) = controller_client else {
@@ -813,8 +808,7 @@ pub async fn pageserver_physical_gc(
return Ok(summary);
};
let (ancestor_shards, ancestor_refs) = Arc::into_inner(accumulator)
.unwrap()
let (ancestor_shards, ancestor_refs) = accumulator
.into_inner()
.unwrap()
.into_gc_ancestors(client, &mut summary)

View File

@@ -370,6 +370,7 @@ class NeonEnvBuilder:
pageserver_config_override: str | Callable[[dict[str, Any]], None] | None = None,
num_safekeepers: int = 1,
num_pageservers: int = 1,
num_azs: int = 1,
# Use non-standard SK ids to check for various parsing bugs
safekeepers_id_start: int = 0,
# fsync is disabled by default to make the tests go faster
@@ -401,6 +402,7 @@ class NeonEnvBuilder:
self.pageserver_config_override = pageserver_config_override
self.num_safekeepers = num_safekeepers
self.num_pageservers = num_pageservers
self.num_azs = num_azs
self.safekeepers_id_start = safekeepers_id_start
self.safekeepers_enable_fsync = safekeepers_enable_fsync
self.auth_enabled = auth_enabled
@@ -990,6 +992,7 @@ class NeonEnv:
self.endpoints = EndpointFactory(self)
self.safekeepers: list[Safekeeper] = []
self.pageservers: list[NeonPageserver] = []
self.num_azs = config.num_azs
self.broker = NeonBroker(self)
self.pageserver_remote_storage = config.pageserver_remote_storage
self.safekeepers_remote_storage = config.safekeepers_remote_storage
@@ -1090,14 +1093,21 @@ class NeonEnv:
http=self.port_distributor.get_port(),
)
# Availabilty zones may also be configured manually with `NeonEnvBuilder.pageserver_config_override`
if self.num_azs > 1:
# Round-robin assignment of AZ names like us-east-2a, us-east-2b, etc.
az_prefix = DEFAULT_AZ_ID[:-1]
availability_zone = f"{az_prefix}{chr(ord('a') + (ps_id - 1) % self.num_azs)}"
else:
availability_zone = DEFAULT_AZ_ID
ps_cfg: dict[str, Any] = {
"id": ps_id,
"listen_pg_addr": f"localhost:{pageserver_port.pg}",
"listen_http_addr": f"localhost:{pageserver_port.http}",
"pg_auth_type": pg_auth_type,
"http_auth_type": http_auth_type,
# Default which can be overriden with `NeonEnvBuilder.pageserver_config_override`
"availability_zone": DEFAULT_AZ_ID,
"availability_zone": availability_zone,
# Disable pageserver disk syncs in tests: when running tests concurrently, this avoids
# the pageserver taking a long time to start up due to syncfs flushing other tests' data
"no_sync": True,

View File

@@ -176,6 +176,7 @@ def test_fully_custom_config(positive_env: NeonEnv):
"type": "interpreted",
"args": {"format": "bincode", "compression": {"zstd": {"level": 1}}},
},
"rel_size_v2_enabled": True,
}
vps_http = env.storage_controller.pageserver_api()

View File

@@ -7,9 +7,78 @@ import threading
import time
import pytest
from fixtures.neon_fixtures import NeonEnvBuilder
from fixtures.neon_fixtures import NeonEnv, NeonEnvBuilder
from fixtures.utils import USE_LFC, query_scalar
"""
Test whether LFC doesn't error out when the LRU is empty, but the LFC is
already at its maximum size.
If we don't handle this safely, we might allocate more hash entries than
otherwise considered safe, thus causing ERRORs in hash_search(HASH_ENTER) once
we hit lfc->used >= lfc->limit.
"""
@pytest.mark.skipif(not USE_LFC, reason="LFC is disabled, skipping")
def test_local_file_cache_all_pinned(neon_simple_env: NeonEnv):
env = neon_simple_env
endpoint = env.endpoints.create_start(
"main",
config_lines=[
"neon.max_file_cache_size='1MB'",
"neon.file_cache_size_limit='1MB'",
],
)
top_cur = endpoint.connect().cursor()
stop = threading.Event()
n_rows = 10000
n_threads = 5
n_updates_per_connection = 1000
top_cur.execute("CREATE TABLE lfctest (id int4 PRIMARY KEY, n int) WITH (fillfactor=10)")
top_cur.execute(f"INSERT INTO lfctest SELECT g, 1 FROM generate_series(1, {n_rows}) g")
# Start threads that will perform random UPDATEs. Each UPDATE
# increments the counter on the row, so that we can check at the
# end that the sum of all the counters match the number of updates
# performed (plus the initial 1 on each row).
#
# Furthermore, each thread will reconnect between every 1000 updates.
def run_updates(n_updates_performed_q: queue.Queue[int]):
n_updates_performed = 0
conn = endpoint.connect()
cur = conn.cursor()
while not stop.is_set():
id = random.randint(1, n_rows)
cur.execute(f"UPDATE lfctest SET n = n + 1 WHERE id = {id}")
n_updates_performed += 1
if n_updates_performed % n_updates_per_connection == 0:
cur.close()
conn.close()
conn = endpoint.connect()
cur = conn.cursor()
n_updates_performed_q.put(n_updates_performed)
n_updates_performed_q: queue.Queue[int] = queue.Queue()
threads: list[threading.Thread] = []
for _i in range(n_threads):
thread = threading.Thread(target=run_updates, args=(n_updates_performed_q,), daemon=True)
thread.start()
threads.append(thread)
time.sleep(15)
stop.set()
n_updates_performed = 0
for thread in threads:
thread.join()
n_updates_performed += n_updates_performed_q.get()
assert query_scalar(top_cur, "SELECT SUM(n) FROM lfctest") == n_rows + n_updates_performed
@pytest.mark.skipif(not USE_LFC, reason="LFC is disabled, skipping")
def test_local_file_cache_unlink(neon_env_builder: NeonEnvBuilder):

View File

@@ -0,0 +1,60 @@
# NB: there are benchmarks that double-serve as tests inside the `performance` directory.
import subprocess
from pathlib import Path
import pytest
from fixtures.log_helper import log
from fixtures.neon_fixtures import NeonEnvBuilder
@pytest.mark.timeout(30) # test takes <20s if pageserver impl is correct
@pytest.mark.parametrize("kind", ["pageserver-stop", "tenant-detach"])
def test_slow_flush(neon_env_builder: NeonEnvBuilder, neon_binpath: Path, kind: str):
def patch_pageserver_toml(config):
config["page_service_pipelining"] = {
"mode": "pipelined",
"max_batch_size": 32,
"execution": "concurrent-futures",
}
neon_env_builder.pageserver_config_override = patch_pageserver_toml
env = neon_env_builder.init_start()
log.info("make flush appear slow")
log.info("sending requests until pageserver accepts no more")
# TODO: extract this into a helper, like subprocess_capture,
# so that we capture the stderr from the helper somewhere.
child = subprocess.Popen(
[
neon_binpath / "test_helper_slow_client_reads",
env.pageserver.connstr(),
str(env.initial_tenant),
str(env.initial_timeline),
],
bufsize=0, # unbuffered
stdin=subprocess.PIPE,
stdout=subprocess.PIPE,
)
assert child.stdout is not None
buf = child.stdout.read(1)
if len(buf) != 1:
raise Exception("unexpected EOF")
if buf != b"R":
raise Exception(f"unexpected data: {buf!r}")
log.info("helper reports pageserver accepts no more requests")
log.info(
"assuming pageserver connection handle is in a state where TCP has backpressured pageserver=>client response flush() into userspace"
)
if kind == "pageserver-stop":
log.info("try to shut down the pageserver cleanly")
env.pageserver.stop()
elif kind == "tenant-detach":
log.info("try to shut down the tenant")
env.pageserver.tenant_detach(env.initial_tenant)
else:
raise ValueError(f"unexpected kind: {kind}")
log.info("shutdown did not time out, test passed")

View File

@@ -2394,6 +2394,7 @@ def test_storage_controller_node_deletion(
Test that deleting a node works & properly reschedules everything that was on the node.
"""
neon_env_builder.num_pageservers = 3
neon_env_builder.num_azs = 3
env = neon_env_builder.init_configs()
env.start()
@@ -2407,6 +2408,9 @@ def test_storage_controller_node_deletion(
tid, placement_policy='{"Attached":1}', shard_count=shard_count_per_tenant
)
# Sanity check: initial creations should not leave the system in an unstable scheduling state
assert env.storage_controller.reconcile_all() == 0
victim = env.pageservers[-1]
# The procedure a human would follow is:

View File

@@ -194,7 +194,7 @@ def test_metrics_normal_work(neon_env_builder: NeonEnvBuilder):
io_metrics = query_all_safekeepers(
"safekeeper_pg_io_bytes_total",
{
"app_name": "pageserver",
"app_name": f"pageserver-{env.pageserver.id}",
"client_az": "test_ps_az",
"dir": io_direction,
"same_az": "false",