Currently using gc blocking and unblocking with storage controller
managed pageservers is painful. Implement the API on storage controller.
Fixes: #8893
Implement the timeline specific `archival_config` endpoint also in the
storage controller.
It's mostly a copy-paste of the detach handler: the task is the same: do
the same operation on all shards.
Part of #8088.
Removes additional async_trait usages from safekeeper and neon_local.
Also removes now redundant dependencies of the `async_trait` crate.
cc earlier work: #6305, #6464, #7303, #7342, #7212, #8296
Part of #8130, closes#8719.
## Problem
Currently, vectored blob io only coalesce blocks if they are immediately
adjacent to each other. When we switch to Direct IO, we need a way to
coalesce blobs that are within the dio-aligned boundary but has gap
between them.
## Summary of changes
- Introduces a `VectoredReadCoalesceMode` for `VectoredReadPlanner` and
`StreamingVectoredReadPlanner` which has two modes:
- `AdjacentOnly` (current implementation)
- `Chunked(<alignment requirement>)`
- New `ChunkedVectorBuilder` that considers batching `dio-align`-sized
read, the start and end of the vectored read will respect
`stx_dio_offset_align` / `stx_dio_mem_align` (`vectored_read.start` and
`vectored_read.blobs_at.first().start_offset` will be two different
value).
- Since we break the assumption that blobs within single `VectoredRead`
are next to each other (implicit end offset), we start to store blob end
offsets in the `VectoredRead`.
- Adapted existing tests to run in both `VectoredReadCoalesceMode`.
- The io alignment can also be live configured at runtime.
Signed-off-by: Yuchen Liang <yuchen@neon.tech>
Currently storage controller does not support forwarding timeline detach
ancestor requests to pageservers. Add support for forwarding `PUT
.../:tenant_id/timelines/:timeline_id/detach_ancestor`. Implement the
support mostly as is, because the timeline detach ancestor will be made
(mostly) idempotent in future PR.
Cc: #6994
I want to fix bugs in `page_service`
([issue](https://github.com/neondatabase/neon/issues/7427)) and the
`import basebackup` / `import wal` stand in the way / make the
refactoring more complicated.
We don't use these methods anyway in practice, but, there have been some
objections to removing the functionality completely.
So, this PR preserves the existing functionality but moves it into the
HTTP management API.
Note that I don't try to fix existing bugs in the code, specifically not
fixing
* it only ever worked correctly for unsharded tenants
* it doesn't clean up on error
All errors are mapped to `ApiError::InternalServerError`.
## Problem
For some time, we have created tenants with calls to location_conf. The
legacy "POST /v1/tenant" path was only used in some tests.
## Summary of changes
- Remove the API
- Relocate TenantCreateRequest to the controller API file (this used to
be used in both pageserver and controller APIs)
- Rewrite tenant_create test helper to use location_config API, as
control plane and storage controller do
- Update docker-compose test script to create tenants with
location_config API (this small commit is also present in
https://github.com/neondatabase/neon/pull/7947)
part of https://github.com/neondatabase/neon/issues/7462
## Summary of changes
This pull request adds two APIs to the pageserver management API:
list_aux_files and ingest_aux_files. The aux file pagebench is intended
to be used on an empty timeline because the data do not go through the
safekeeper. LSNs are advanced by 8 for each ingestion, to avoid
invariant checks inside the pageserver.
For now, I only care about space amplification / read amplification, so
the bench is designed in a very simple way: ingest 10000 files, and I
will manually dump the layer map to analyze.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
Currently tenants are only split into multiple shards if a human being
calls the API to do it.
Issue: #7388
## Summary of changes
- Add a pageserver API for returning the top tenants by size
- Add a step to the controller's background loop where if there is no
reconciliation or optimization to be done, it looks for things to split.
- Add a test that runs pgbench on many tenants concurrently, and checks
that splitting happens as expected as tenants grow, without interrupting
the client I/O.
This PR is quite basic: there is a tasklist in
https://github.com/neondatabase/neon/issues/7388 for further work. This
PR is meant to be safe (off by default), and sufficient to enable our
staging environment to run lots of sharded tenants without a human
having to set them up.
## Problem
The logic in Service::optimize_all would sometimes choose to migrate a
tenant to a secondary location that was only recently created, resulting
in Reconciler::live_migrate hitting its 5 minute timeout warming up the
location, and proceeding to attach a tenant to a location that doesn't
have a warm enough local set of layer files for good performance.
Closes: #7532
## Summary of changes
- Add a pageserver API for checking download progress of a secondary
location
- During `optimize_all`, connect to pageservers of candidate
optimization secondary locations, and check they are warm.
- During shard split, do heatmap uploads and start secondary downloads,
so that the new shards' secondary locations start downloading ASAP,
rather than waiting minutes for background downloads to kick in.
I have intentionally not implemented this by continuously reading the
status of locations, to avoid dealing with the scale challenge of
efficiently polling & updating 10k-100k locations status. If we
implement that in the future, then this code can be simplified to act
based on latest state of a location rather than fetching it inline
during optimize_all.
## Problem
Sometimes we have test data in the form of S3 contents that we would
like to run live in a neon_local environment.
## Summary of changes
- Add a storage controller API that imports an existing tenant.
Currently this is equivalent to doing a create with a high generation
number, but in future this would be something smarter to probe S3 to
find the shards in a tenant and find generation numbers.
- Add a `neon_local` command that invokes the import API, and then
inspects timelines in the newly attached tenant to create matching
branches.
In the old protocol version, the client sent with each request:
- latest: bool. If true, the client requested the latest page
version, and the 'lsn' was just a hint of when the page was last
modified
- lsn: Lsn, the page version to return
This protocol didn't allow requesting a page at a particular
non-latest LSN and *also* sending a hint on when the page was last
modified. That put a read only compute into an awkward position where
it had to either request each page at the replay-LSN, which could be
very close to the last LSN written in the primary and therefore
require the pageserver to wait for it to arrive, or an older LSN which
could already be garbage collected in the pageserver, resulting in an
error. The new protocol version fixes that by allowing a read only
compute to send both LSNs.
To use the new protocol version, use "pagestream_v2" command instead
of just "pagestream". The old protocol version is still supported, for
compatibility with old computes (and in fact there is no client
support yet, it is added by the next commit).
## Problem
The API client was written around the same time as some of the server
APIs changed from TenantId to TenantShardId
Closes: https://github.com/neondatabase/neon/issues/6154
## Summary of changes
- Refactor mgmt_api timeline_info and keyspace methods to use
TenantShardId to match the server
This doesn't make pagebench sharding aware, but it paves the way to do
so later.
## Problem
The existing secondary download API relied on the caller to wait as long
as it took to complete -- for large shards that could be a long time, so
typical clients that might have a baked-in ~30s timeout would have a
problem.
## Summary of changes
- Take a `wait_ms` query parameter to instruct the pageserver how long
to wait: if the download isn't complete in this duration, then 201 is
returned instead of 200.
- For both 200 and 201 responses, include response body describing
download progress, in terms of layers and bytes. This is sufficient for
the caller to track how much data is being transferred and log/present
that status.
- In storage controller live migrations, use this API to apply a much
longer outer timeout, with smaller individual per-request timeouts, and
log the progress of the downloads.
- Add a test that injects layer download delays to exercise the new
behavior
# Problem
On-demand downloads are still using `tokio::fs`, which we know is
inefficient.
# Changes
- Add `pagebench ondemand-download-churn` to quantify on-demand download
throughput
- Requires dumping layer map, which required making `history_buffer`
impl `Deserialize`
- Implement an equivalent of `tokio::io::copy_buf` for owned buffers =>
`owned_buffers_io` module and children.
- Make layer file download sensitive to `io_engine::get()`, using
VirtualFile + above copy loop
- For this, I had to move some code into the `retry_download`, e.g.,
`sync_all()` call.
Drive-by:
- fix missing escaping in `scripts/ps_ec2_setup_instance_store`
- if we failed in retry_download to create a file, we'd try to remove
it, encounter `NotFound`, and `abort()` the process using
`on_fatal_io_error`. This PR adds treats `NotFound` as a success.
# Testing
Functional
- The copy loop is generic & unit tested.
Performance
- Used the `ondemand-download-churn` benchmark to manually test against
real S3.
- Results (public Notion page):
https://neondatabase.notion.site/Benchmarking-tokio-epoll-uring-on-demand-downloads-2024-04-15-newer-code-03c0fdc475c54492b44d9627b6e4e710?pvs=4
- Performance is equivalent at low concurrency. Jumpier situation at
high concurrency, but, still less CPU / throughput with
tokio-epoll-uring.
- It’s a win.
# Future Work
Turn the manual performance testing described in the above results
document into a performance regression test:
https://github.com/neondatabase/neon/issues/7146
## Problem
If a pageserver was offline when the storage controller started, there
was no mechanism to update the
storage controller state when the pageserver becomes active.
## Summary of changes
* Add a heartbeater module. The heartbeater must be driven by an
external loop.
* Integrate the heartbeater into the service.
- Extend the types used by the service and scheduler to keep track of a
nodes' utilisation score.
- Add a background loop to drive the heartbeater and update the state
based on the deltas it generated
- Do an initial round of heartbeats at start-up
The `tenant_id` in `TenantLocationConfigRequest` in the
`location_config` endpoint was only used in the storage
controller/attachment service, and there it was only used for assertions
and the creation part.
## Problem
Closes: https://github.com/neondatabase/neon/issues/6847
Closes: https://github.com/neondatabase/neon/issues/7006
## Summary of changes
- Pageserver API calls are wrapped in timeout/retry logic: this prevents
a reconciler getting hung on a pageserver API hang, and prevents
reconcilers having to totally retry if one API call returns a retryable
error (e.g. 503).
- Add a cancellation token to `Node`, so that when we mark a node
offline we will cancel any API calls in progress to that node, and avoid
issuing any more API calls to that offline node.
- If the dirty locations of a shard are all on offline nodes, then don't
spawn a reconciler
- In re-attach, if we have no observed state object for a tenant then
construct one with conf: None (which means "unknown"). Then in
Reconciler, implement a TODO for scanning such locations before running,
so that we will avoid spuriously incrementing a generation in the case
of a node that was offline while we started (this is the case that
tripped up #7006)
- Refactoring: make Node contents private (and thereby guarantee that
updates to availability mode reliably update the cancellation token.)
- Refactoring: don't pass the whole map of nodes into Reconciler (and
thereby remove a bunch of .expect() calls)
Some of this was discovered/tested with a new failure injection test
that will come in a separate PR, once it is stable enough for CI.
## Problem
If large numbers of shards are attached to a pageserver concurrently,
for example after another node fails, it can cause excessive I/O queue
depths due to all the newly attached shards trying to calculate logical
sizes concurrently.
#6907 added the `lazy` flag to handle this.
## Summary of changes
- Use `lazy=true` from all /location_config calls in the storage
controller Reconciler.
The sharding service didn't have support for S3 disaster recovery.
This PR adds a new endpoint to the attachment service, which is slightly
different from the endpoint on the pageserver, in that it takes the
shard count history of the tenant as json parameters: we need to do
time travel recovery for both the shard count at the target time and the
shard count at the current moment in time, as well as the past shard
counts that either still reference.
Fixes#6604, part of https://github.com/neondatabase/cloud/issues/8233
---------
Co-authored-by: John Spray <john@neon.tech>
- Automatically set a node's availability to Active if it is responsive
in startup_reconcile
- Impose a 5s timeout of HTTP request to list location conf, so that an
unresponsive node can't hang it for minutes
- Do several retries if the request fails with a retryable error, to be
tolerant of concurrent pageserver & storage controller restarts
- Add a readiness hook for use with k8s so that we can tell when the
startup reconciliaton is done and the service is fully ready to do work.
- Add /metrics to the list of un-authenticated endpoints (this is
unrelated but we're touching the line in this PR already, and it fixes
auth error spam in deployed container.)
- A test for the above.
Closes: #6670
## Problem
One doesn't know at tenant creation time how large the tenant will grow.
We need to be able to dynamically adjust the shard count at runtime.
This is implemented as "splitting" of shards into smaller child shards,
which cover a subset of the keyspace that the parent covered.
Refer to RFC: https://github.com/neondatabase/neon/pull/6358
Part of epic: #6278
## Summary of changes
This PR implements the happy path (does not cleanly recover from a crash
mid-split, although won't lose any data), without any optimizations
(e.g. child shards re-download their own copies of layers that the
parent shard already had on local disk)
- Add `/v1/tenant/:tenant_shard_id/shard_split` API to pageserver: this
copies the shard's index to the child shards' paths, instantiates child
`Tenant` object, and tears down parent `Tenant` object.
- Add `splitting` column to `tenant_shards` table. This is written into
an existing migration because we haven't deployed yet, so don't need to
cleanly upgrade.
- Add `/control/v1/tenant/:tenant_id/shard_split` API to
attachment_service,
- Add `test_sharding_split_smoke` test. This covers the happy path:
future PRs will add tests that exercise failure cases.
This PR adds an API to live-reconfigure the VirtualFile io engine.
It also adds a flag to `pagebench get-page-latest-lsn`, which is where I
found this functionality to be useful: it helps compare the io engines
in a benchmark without re-compiling a release build, which took ~50s on
the i3en.3xlarge where I was doing the benchmark.
Switching the IO engine is completely safe at runtime.
## Problem
See https://github.com/neondatabase/cloud/issues/8673
## Summary of changes
Download missed SLRU segments from page server
## Checklist before requesting a review
- [ ] I have performed a self-review of my code.
- [ ] If it is a core feature, I have added thorough tests.
- [ ] Do we need to implement analytics? if so did you add the relevant
metrics to the dashboard?
- [ ] If this PR requires public announcement, mark it with
/release-notes label and add several sentences in this section.
## Checklist before merging
- [ ] Do not forget to reformat commit message to not include the above
checklist
---------
Co-authored-by: Konstantin Knizhnik <knizhnik@neon.tech>
Co-authored-by: Heikki Linnakangas <heikki@neon.tech>
Depends on: https://github.com/neondatabase/neon/pull/6468
## Problem
The sharding service will be used as a "virtual pageserver" by the
control plane -- so it needs the set of pageserver APIs that the control
plane uses, and to present them under identical URLs, including prefix
(/v1).
## Summary of changes
- Add missing APIs:
- Tenant deletion
- Timeline deletion
- Node list (used in test now, later in tools)
- `/location_config` API (for migrating tenants into the sharding
service)
- Rework attachment service URLs:
- `/v1` prefix is used for pageserver-compatible APIs
- `/upcall/v1` prefix is used for APIs that are called by the pageserver
(re-attach and validate)
- `/debug/v1` prefix is used for endpoints that are for testing
- `/control/v1` prefix is used for new sharding service APIs that do not
mimic a pageserver API, such as registering and configuring nodes.
- Add test_sharding_service. The sharding service already had some
collateral coverage from its use in general tests, but this is the first
dedicated testing for it.
## Problem
To test sharding, we need something to control it. We could write python
code for doing this from the test runner, but this wouldn't be usable
with neon_local run directly, and when we want to write tests with large
number of shards/tenants, Rust is a better fit efficiently handling all
the required state.
This service enables automated tests to easily get a system with
sharding/HA without the test itself having to set this all up by hand:
existing tests can be run against sharded tenants just by setting a
shard count when creating the tenant.
## Summary of changes
Attachment service was previously a map of TenantId->TenantState, where
the principal state stored for each tenant was the generation and the
last attached pageserver. This enabled it to serve the re-attach and
validate requests that the pageserver requires.
In this PR, the scope of the service is extended substantially to do
overall management of tenants in the pageserver, including
tenant/timeline creation, live migration, evacuation of offline
pageservers etc. This is done using synchronous code to make declarative
changes to the tenant's intended state (`TenantState.policy` and
`TenantState.intent`), which are then translated into calls into the
pageserver by the `Reconciler`.
Top level summary of modules within
`control_plane/attachment_service/src`:
- `tenant_state`: structure that represents one tenant shard.
- `service`: implements the main high level such as tenant/timeline
creation, marking a node offline, etc.
- `scheduler`: for operations that need to pick a pageserver for a
tenant, construct a scheduler and call into it.
- `compute_hook`: receive notifications when a tenant shard is attached
somewhere new. Once we have locations for all the shards in a tenant,
emit an update to postgres configuration via the neon_local `LocalEnv`.
- `http`: HTTP stubs. These mostly map to methods on `Service`, but are
separated for readability and so that it'll be easier to adapt if/when
we switch to another RPC layer.
- `node`: structure that describes a pageserver node. The most important
attribute of a node is its availability: marking a node offline causes
tenant shards to reschedule away from it.
This PR is a precursor to implementing the full sharding service for
prod (#6342). What's the difference between this and a production-ready
controller for pageservers?
- JSON file persistence to be replaced with a database
- Limited observability.
- No concurrency limits. Marking a pageserver offline will try and
migrate every tenant to a new pageserver concurrently, even if there are
thousands.
- Very simple scheduler that only knows to pick the pageserver with
fewest tenants, and place secondary locations on a different pageserver
than attached locations: it does not try to place shards for the same
tenant on different pageservers. This matters little in tests, because
picking the least-used pageserver usually results in round-robin
placement.
- Scheduler state is rebuilt exhaustively for each operation that
requires a scheduler.
- Relies on neon_local mechanisms for updating postgres: in production
this would be something that flows through the real control plane.
---------
Co-authored-by: Arpad Müller <arpad-m@users.noreply.github.com>
## Problem
The `/v1/tenant` listing API only applies to attached tenants.
For an external service to implement a global reconciliation of its list
of shards vs. what's on the pageserver, we need a full view of what's in
TenantManager, including secondary tenant locations, and InProgress
locations.
Dependency of https://github.com/neondatabase/neon/pull/6251
## Summary of changes
- Add methods to Tenant and SecondaryTenant to reconstruct the
LocationConf used to create them.
- Add `GET /v1/location_config` API
The theme of the changes in this PR is that they're enablers for #6251
which are superficial struct/api changes.
This is a spinoff from #6251:
- Various APIs + clients thereof take TenantShardId rather than TenantId
- The creation API gets a ShardParameters member, which may be used to
configure shard count and stripe size. This enables the attachment
service to present a "virtual pageserver" creation endpoint that creates
multiple shards.
- The attachment service will use tenant size information to drive shard
splitting. Make a version of `TenantHistorySize` that is usable for
decoding these API responses.
- ComputeSpec includes a shard stripe size.
## Problem
Previously, `GET /v1/tenant/:tenant_id/timeline` and `GET
/v1/tenant/:tenant_id/timeline/:timeline_id`
would bump the priority of the background task which computes the
initial logical size by cancelling
the wait on the synchronisation semaphore. However, the request would
still return an approximate
logical size. It's undesirable to force background work for a status
request.
## Summary of changes
This PR updates the priority used by the timeline status request such
that they don't do priority boosting
by default anymore. An optional query parameter,
`force-await-initial-logical-size`, is added for both
mentioned endpoints. When set to true, it will skip the concurrency
limiting semaphore and wait
for the background task to complete before returning the exact logical
size.
In order to exercise this behaviour in a test I had to add an extra
failpoint. If you think it's too intrusive,
it can be removed.
Also fixeda small bug where the cancellation of a download is reported as an
opaque download failure upstream. This caused `test_location_conf_churn`
to fail at teardown due to a WARN log line.
Closes https://github.com/neondatabase/neon/issues/6168
This uses the [newly stable](https://blog.rust-lang.org/2023/12/21/async-fn-rpit-in-traits.html)
async trait feature for three internal traits. One requires `Send`
bounds to be present so uses `impl Future<...> + Send` instead.
Advantages:
* less macro usage
* no extra boxing
Disadvantages:
* impl syntax needed for `Send` bounds is a bit more verbose (but only
required in one place)
This PR adds a component-level benchmarking utility for pageserver.
Its name is `pagebench`.
The problem solved by `pagebench` is that we want to put Pageserver
under high load.
This isn't easily achieved with `pgbench` because it needs to go through
a compute, which has signficant performance overhead compared to
accessing Pageserver directly.
Further, compute has its own performance optimizations (most
importantly: caches). Instead of designing a compute-facing workload
that defeats those internal optimizations, `pagebench` simply bypasses
them by accessing pageserver directly.
Supported benchmarks:
* getpage@latest_lsn
* basebackup
* triggering logical size calculation
This code has no automated users yet.
A performance regression test for getpage@latest_lsn will be added in a
later PR.
part of https://github.com/neondatabase/neon/issues/5771
Part of getpage@lsn benchmark epic:
https://github.com/neondatabase/neon/issues/5771
This allows getting the list of tenants and timelines without triggering
initial logical size calculation by requesting the timeline details API
response, which would skew our results.
Part of getpage@lsn benchmark epic:
https://github.com/neondatabase/neon/issues/5771
This PR moves the control plane's spread-all-over-the-place client for
the pageserver management API into a separate module within the
pageserver crate.
I need that client to be async in my benchmarking work, so, this PR
switches to the async version of `reqwest`.
That is also the right direction generally IMO.
The switch to async in turn mandated converting most of the
`control_plane/` code to async.
Note that some of the client methods should be taking `TenantShardId`
instead of `TenantId`, but, none of the callers seem to be
sharding-aware.
Leaving that for another time:
https://github.com/neondatabase/neon/issues/6154