## Problem
close https://github.com/neondatabase/neon/issues/10192
## Summary of changes
* `find_gc_time_cutoff` takes `now` parameter so that all branches
compute the cutoff based on the same start time, avoiding races.
* gc-compaction uses a single `get_gc_compaction_watermark` function to
get the safe LSN to compact.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
Co-authored-by: Arpad Müller <arpad-m@users.noreply.github.com>
## Problem
It's not legal to modify layers that are referenced by the current layer
index. Assert this in the upload queue, as preparation for upload queue
reordering.
Touches #10096.
## Summary of changes
Add a debug assertion that the upload queue does not modify layers
referenced by the current index.
I could be convinced that this should be a plain assertion, but will be
conservative for now.
## Problem
Since enabling continuous profiling in staging, we've seen frequent seg
faults. This is suspected to be because jemalloc and pprof-rs take a
stack trace at the same time, and the handlers aren't signal safe.
jemalloc does this probabilistically on every allocation, regardless of
whether someone is taking a heap profile, which means that any CPU
profile has a chance to cause a seg fault.
Touches #10225.
## Summary of changes
For now, just disable heap profiles -- CPU profiles are more important,
and we need to be able to take them without risking a crash.
There is a race condition between `Tenant::shutdown`'s `defuse_for_drop`
loop and `offload_timeline`, where timeline offloading can insert into a
tenant that is in the process of shutting down, in fact so far
progressed that the `defuse_for_drop` has already been called.
This prevents warn log lines of the form:
```
offloaded timeline <hash> was dropped without having cleaned it up at the ancestor
```
The solution piggybacks on the `offloaded_timelines` lock: both the
defuse loop and the offloaded timeline insertion need to acquire the
lock, and we know that the defuse loop only runs after the tenant has
set its `TenantState` to `Stopping`.
So if we hold the `offloaded_timelines` lock, and know that the
`TenantState` is not `Stopping`, then we know that the defuse loop has
not ran yet, and holding the lock ensures that it doesn't start running
while we are inserting the offloaded timeline.
Fixes#10070
## Problem
In https://github.com/neondatabase/neon/pull/9897 we temporarily
disabled the layer valid check because the current one only considers
the end result of all compaction algorithms, but partial gc-compaction
would temporarily produce an "invalid" layer map.
part of https://github.com/neondatabase/neon/issues/9114
## Summary of changes
Allow LSN splits to overlap in the slow path check. Currently, the valid
check is only used in storage scrubber (background job) and during
gc-compaction (without taking layer lock). Therefore, it's fine for such
checks to be a little bit inefficient but more accurate.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
Co-authored-by: Arpad Müller <arpad-m@users.noreply.github.com>
## Problem
We cannot get the size of the compaction queue and access the info.
Part of #9114
## Summary of changes
* Add an API endpoint to get the compaction queue.
* gc_compaction test case now waits until the compaction finishes.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
part of https://github.com/neondatabase/neon/issues/9114
In https://github.com/neondatabase/neon/pull/10127 we fixed the race,
but we didn't add the errors to the allowlist.
## Summary of changes
* Allow repartition errors in the gc-compaction smoke test.
I think it might be worth to refactor the code to allow multiple threads
getting a copy of repartition status (i.e., using Rcu) in the future.
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
Changes in #9786 were functionally complete but missed some edges that
made testing less robust than it should have been:
- `is_key_disposable` didn't consider SLRU dir keys disposable
- Timeline `init_empty` was always creating SLRU dir keys on all shards
The result was that when we had a bug
(https://github.com/neondatabase/neon/pull/10080), it wasn't apparent in
tests, because one would only encounter the issue if running on a
long-lived timeline with enough compaction to drop the initially created
empty SLRU dir keys, _and_ some CLog truncation going on.
Closes: https://github.com/neondatabase/cloud/issues/21516
## Summary of changes
- Update is_key_global and init_empty to handle SLRU dir keys properly
-- the only functional impact is that we avoid writing some spurious
keys in shards >0, but this makes testing much more robust.
- Make `test_clog_truncate` explicitly use a sharded tenant
The net result is that if one reverts #10080, then tests fail (i.e. this
PR is a reproducer for the issue)
## Problem
In #8550, we made the flush loop wait for uploads after every layer.
This was to avoid unbounded buildup of uploads, and to reduce compaction
debt. However, the approach has several problems:
* It prevents upload parallelism.
* It prevents flush and upload pipelining.
* It slows down ingestion even when there is no need to backpressure.
* It does not directly backpressure WAL ingestion (only via
`disk_consistent_lsn`), and will build up in-memory layers.
* It does not directly backpressure based on compaction debt and read
amplification.
An alternative solution to these problems is proposed in #8390.
In the meanwhile, we revert the change to reduce the impact on ingest
throughput. This does reintroduce some risk of unbounded
upload/compaction buildup. Until
https://github.com/neondatabase/neon/issues/8390, this can be addressed
in other ways:
* Use `max_replication_apply_lag` (aka `remote_consistent_lsn`), which
will more directly limit upload debt.
* Shard the tenant, which will spread the flush/upload work across more
Pageservers and move the bottleneck to Safekeeper.
Touches #10095.
## Summary of changes
Remove waiting on the upload queue in the flush loop.
## Problem
close https://github.com/neondatabase/neon/issues/10124
gc-compaction split_gc_jobs is holding the repartition lock for too long
time.
## Summary of changes
* Ensure split_gc_compaction_jobs drops the repartition lock once it
finishes cloning the structures.
* Update comments.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
Cplane and storage controller tenant config changes are not additive.
Any change overrides all existing tenant configs. This would be fine if
both did client side patching, but that's not the case.
Once this merges, we must update cplane to use the PATCH endpoint.
## Summary of changes
### High Level
Allow for patching of tenant configuration with a `PATCH
/v1/tenant/config` endpoint.
It takes the same data as it's PUT counterpart. For example the payload
below will update `gc_period` and unset `compaction_period`. All other
fields are left in their original state.
```
{
"tenant_id": "1234",
"gc_period": "10s",
"compaction_period": null
}
```
### Low Level
* PS and storcon gain `PATCH /v1/tenant/config` endpoints. PS endpoint
is only used for cplane managed instances.
* `storcon_cli` is updated to have separate commands for
`set-tenant-config` and `patch-tenant-config`
Related https://github.com/neondatabase/cloud/issues/21043
## Problem
We get slru truncation commands on non-zero shards.
Compaction will drop the slru dir keys and ingest will fail when
receiving such records.
https://github.com/neondatabase/neon/pull/10080 fixed it for clog, but
not for multixact.
## Summary of changes
Only truncate multixact slrus on shard zero. I audited the rest of the
ingest code and it looks
fine from this pov.
## Problem
With pipelining enabled, the time a request spends in the batcher stage
counts towards the smgr op latency.
If pipelining is disabled, that time is not accounted for.
In practice, this results in a jump in smgr getpage latencies in various
dashboards and degrades the internal SLO.
## Solution
In a similar vein to #10042 and with a similar rationale, this PR stops
counting the time spent in batcher stage towards smgr op latency.
The smgr op latency metric is reduced to the actual execution time.
Time spent in batcher stage is tracked in a separate histogram.
I expect to remove that histogram after batching rollout is complete,
but it will be helpful in the meantime to reason about the rollout.
## Problem
In #9786 we stop storing SLRUs on non-zero shards.
However, there was one code path during ingest that still tries to
enumerate SLRU relations on all shards. This fails if it sees a tenant
who has never seen any write to an SLRU, or who has done such thorough
compaction+GC that it has dropped its SLRU directory key.
## Summary of changes
- Avoid trying to list SLRU relations on nonzero shards
## Problem
close https://github.com/neondatabase/cloud/issues/19671
```
Timeline -----------------------------
^ last GC happened LSN
^ original retention period setting = 24hr
> refresh-gc-info updates the gc_info
^ planned cutoff (gc_info)
^ customer set retention to 48hr, and it's still within the last GC LSN
^1 ^2 we have two choices: (1) update the planned cutoff to
move backwards, or (2) keep the current one
```
In this patch, we decided to keep the current cutoff instead of moving
back the gc_info to avoid races. In the future, we could allow the
planned gc cutoff to go back once cplane sends a retention_history
tenant config update, but this requires a careful revisit of the code.
## Summary of changes
Ensure that GC cutoffs never go back if retention settings get changed.
Signed-off-by: Alex Chi Z <chi@neon.tech>
Azure has a different per-request limit of 256 items for bulk deletion
compared to the number of 1000 on AWS. Therefore, we need to support
multiple values. Due to `GenericRemoteStorage`, we can't add an
associated constant, but it has to be a function.
The PR replaces the `MAX_KEYS_PER_DELETE` constant with a function of
the same name, implemented on both the `RemoteStorage` trait as well as
on `GenericRemoteStorage`.
The value serves as hint of how many objects to pass to the
`delete_objects` function.
Reading:
* https://learn.microsoft.com/en-us/rest/api/storageservices/blob-batch
* https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html
Part of #7931
## Problem
close https://github.com/neondatabase/neon/issues/10049, close
https://github.com/neondatabase/neon/issues/10030, close
https://github.com/neondatabase/neon/issues/8861
part of https://github.com/neondatabase/neon/issues/9114
The legacy gc process calls `get_latest_gc_cutoff`, which uses a Rcu
different than the gc_info struct. In the gc_compaction_smoke test case,
the "latest" cutoff could be lower than the gc_info struct, causing
gc-compaction to collect data that could be accessed by
`latest_gc_cutoff`. Technically speaking, there's nothing wrong with
gc-compaction using gc_info without considering latest_gc_cutoff,
because gc_info is the source of truth. But anyways, let's fix it.
## Summary of changes
* gc-compaction uses `latest_gc_cutoff` instead of gc_info to determine
the gc horizon.
* if a gc-compaction is scheduled via tenant compaction iteration, it
will take the gc_block lock to avoid racing with functionalities like
detach ancestor (if it's triggered via manual compaction API without
scheduling, then it won't take the lock)
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
Co-authored-by: Arpad Müller <arpad-m@users.noreply.github.com>
## Problem
In #9962 I changed the smgr metrics to include time spent on flush.
It isn't under our (=storage team's) control how long that flush takes
because the client can stop reading requests.
## Summary of changes
Stop the timer as soon as we've buffered up the response in the
`pgb_writer`.
Track flush time in a separate metric.
---------
Co-authored-by: Yuchen Liang <70461588+yliang412@users.noreply.github.com>
## Problem
part of https://github.com/neondatabase/neon/issues/9114, stacked PR
over #9809
The compaction scheduler now schedules partial compaction jobs.
## Summary of changes
* Add the compaction job splitter based on size.
* Schedule subcompactions using the compaction scheduler.
* Test subcompaction scheduler in the smoke regress test.
* Temporarily disable layer map checks
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
With the current metrics we can't identify which shards are ingesting
data at any given time.
## Summary of changes
Add a metric for the number of wal records received for processing by
each shard. This is per (tenant, timeline, shard).
We've seen cases where stray keys end up on the wrong shard. This
shouldn't happen. Add debug assertions to prevent this. In release
builds, we should be lenient in order to handle changing key ownership
policies.
Touches #9914.
## Problem
There's no metrics for disk consistent LSN and remote LSN. This stuff is
useful when looking at ingest performance.
## Summary of changes
Two per timeline metrics are added: `pageserver_disk_consistent_lsn` and
`pageserver_projected_remote_consistent_lsn`. I went for the projected
remote lsn instead of the visible one
because that more closely matches remote storage write tput. Ideally we
would have both, but these metrics are expensive.
## Problem
I'm writing an ingest benchmark in #9812. To time S3 uploads, I need to
schedule a flush of the Pageserver's in-memory layer, but don't actually
want to wait around for it to complete (which will take a minute).
## Summary of changes
Add a parameter `wait_until_flush` (default `true`) for
`timeline/checkpoint` to control whether to wait for the flush to
complete.
## Problem
FSM pages are managed like regular relation pages, and owned by a single
shard. However, when truncating the FSM relation the last FSM page was
zeroed out on all shards. This is unnecessary and potentially confusing.
The superfluous keys will be removed during compactions, as they do not
belong on these shards.
Resolves#10027.
## Summary of changes
Only zero out the truncated FSM page on the owning shard.
## Problem
part of https://github.com/neondatabase/neon/issues/9114
gc-compaction can take a long time. This patch adds support for
scheduling a gc-compaction job. The compaction loop will first handle
L0->L1 compaction, and then gc compaction. The scheduled jobs are stored
in a non-persistent queue within the tenant structure.
This will be the building block for the partial compaction trigger -- if
the system determines that we need to do a gc compaction, it will
partition the keyspace and schedule several jobs. Each of these jobs
will run for a short amount of time (i.e, 1 min). L0 compaction will be
prioritized over gc compaction.
## Summary of changes
* Add compaction scheduler in tenant.
* Run scheduled compaction in integration tests.
* Change the manual compaction API to allow schedule a compaction
instead of immediately doing it.
* Add LSN upper bound as gc-compaction parameter. If we schedule partial
compactions, gc_cutoff might move across different runs. Therefore, we
need to pass a pre-determined gc_cutoff beforehand. (TODO: support LSN
lower bound so that we can compact arbitrary "rectangle" in the layer
map)
* Refactor the gc_compaction internal interface.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
Co-authored-by: Christian Schwarz <christian@neon.tech>
Closes#9387.
## Problem
`BufferedWriter` cannot proceed while the owned buffer is flushing to
disk. We want to implement double buffering so that the flush can happen
in the background. See #9387.
## Summary of changes
- Maintain two owned buffers in `BufferedWriter`.
- The writer is in charge of copying the data into owned, aligned
buffer, once full, submit it to the flush task.
- The flush background task is in charge of flushing the owned buffer to
disk, and returned the buffer to the writer for reuse.
- The writer and the flush background task communicate through a
bi-directional channel.
For in-memory layer, we also need to be able to read from the buffered
writer in `get_values_reconstruct_data`. To handle this case, we did the
following
- Use replace `VirtualFile::write_all` with `VirtualFile::write_all_at`,
and use `Arc` to share it between writer and background task.
- leverage `IoBufferMut::freeze` to get a cheaply clonable `IoBuffer`,
one clone will be submitted to the channel, the other clone will be
saved within the writer to serve reads. When we want to reuse the
buffer, we can invoke `IoBuffer::into_mut`, which gives us back the
mutable aligned buffer.
- InMemoryLayer reads is now aware of the maybe_flushed part of the
buffer.
**Caveat**
- We removed the owned version of write, because this interface does not
work well with buffer alignment. The result is that without direct IO
enabled,
[`download_object`](a439d57050/pageserver/src/tenant/remote_timeline_client/download.rs (L243))
does one more memcpy than before this PR due to the switch to use
`_borrowed` version of the write.
- "Bypass aligned part of write" could be implemented later to avoid
large amount of memcpy.
**Testing**
- use an oneshot channel based control mechanism to make flush behavior
deterministic in test.
- test reading from `EphemeralFile` when the last submitted buffer is
not flushed, in-progress, and done flushing to disk.
## Performance
We see performance improvement for small values, and regression on big
values, likely due to being CPU bound + disk write latency.
[Results](https://www.notion.so/neondatabase/Benchmarking-New-BufferedWriter-11-20-2024-143f189e0047805ba99acda89f984d51?pvs=4)
## 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
---------
Signed-off-by: Yuchen Liang <yuchen@neon.tech>
Co-authored-by: Christian Schwarz <christian@neon.tech>
## Problem
Reqwest errors don't include details about the inner source error. This
means that we get opaque errors like:
```
receive body: error sending request for url (http://localhost:9898/v1/location_config)
```
Instead of the more helpful:
```
receive body: error sending request for url (http://localhost:9898/v1/location_config): operation timed out
```
Touches #9801.
## Summary of changes
Include the source error for `reqwest::Error` wherever it's displayed.
## Problem
During deploys, we see a lot of 500 errors due to heapmap uploads for
inactive tenants. These should be 503s instead.
Resolves#9574.
## Summary of changes
Make the secondary tenant scheduler use `ApiError` rather than
`anyhow::Error`, to propagate the tenant error and convert it to an
appropriate status code.
Support tenant manifests in the storage scrubber:
* list the manifests, order them by generation
* delete all manifests except for the two most recent generations
* for the latest manifest: try parsing it.
I've tested this patch by running the against a staging bucket and it
successfully deleted stuff (and avoided deleting the latest two
generations).
In follow-up work, we might want to also check some invariants of the
manifest, as mentioned in #8088.
Part of #9386
Part of #8088
---------
Co-authored-by: Christian Schwarz <christian@neon.tech>
## Problem
The Pageserver signal handler would only respond to a single signal and
initiate shutdown. Subsequent signals were ignored. This meant that a
`SIGQUIT` sent after a `SIGTERM` had no effect (e.g. in the case of a
slow or stalled shutdown). The `test_runner` uses this to force shutdown
if graceful shutdown is slow.
Touches #9740.
## Summary of changes
Keep responding to signals after the initial shutdown signal has been
received.
Arguably, the `test_runner` should also use `SIGKILL` rather than
`SIGQUIT` in this case, but it seems reasonable to respond to `SIGQUIT`
regardless.
## Problem
We saw a peculiar case where a pageserver apparently got a 0-tenant
response to `/re-attach` but we couldn't see the request landing on a
storage controller. It was hard to confirm retrospectively that the
pageserver was configured properly at the moment it sent the request.
## Summary of changes
- Log the URL to which we are sending the request
- Log the NodeId and metadata that we sent
## Problem
In the batching PR
- https://github.com/neondatabase/neon/pull/9870
I stopped deducting the time-spent-in-throttle fro latency metrics,
i.e.,
- smgr latency metrics (`SmgrOpTimer`)
- basebackup latency (+scan latency, which I think is part of
basebackup).
The reason for stopping the deduction was that with the introduction of
batching, the trick with tracking time-spent-in-throttle inside
RequestContext and swap-replacing it from the `impl Drop for
SmgrOpTimer` no longer worked with >1 requests in a batch.
However, deducting time-spent-in-throttle is desirable because our
internal latency SLO definition does not account for throttling.
## Summary of changes
- Redefine throttling to be a page_service pagestream request throttle
instead of a throttle for repository `Key` reads through `Timeline::get`
/ `Timeline::get_vectored`.
- This means reads done by `basebackup` are no longer subject to any
throttle.
- The throttle applies after batching, before handling of the request.
- Drive-by fix: make throttle sensitive to cancellation.
- Rename metric label `kind` from `timeline_get` to `pagestream` to
reflect the new scope of throttling.
To avoid config format breakage, we leave the config field named
`timeline_get_throttle` and ignore the `task_kinds` field.
This will be cleaned up in a future PR.
## Trade-Offs
Ideally, we would apply the throttle before reading a request off the
connection, so that we queue the minimal amount of work inside the
process.
However, that's not possible because we need to do shard routing.
The redefinition of the throttle to limit pagestream request rate
instead of repository `Key` rate comes with several downsides:
- We're no longer able to use the throttle mechanism for other other
tasks, e.g. image layer creation.
However, in practice, we never used that capability anyways.
- We no longer throttle basebackup.
## Problem
We don't have good observability for memory usage. This would be useful
e.g. to debug OOM incidents or optimize performance or resource usage.
We would also like to use continuous profiling with e.g. [Grafana Cloud
Profiles](https://grafana.com/products/cloud/profiles-for-continuous-profiling/)
(see https://github.com/neondatabase/cloud/issues/14888).
This PR is intended as a proof of concept, to try it out in staging and
drive further discussions about profiling more broadly.
Touches https://github.com/neondatabase/neon/issues/9534.
Touches https://github.com/neondatabase/cloud/issues/14888.
Depends on #9779.
Depends on #9780.
## Summary of changes
Adds a HTTP route `/profile/heap` that takes a heap profile and returns
it. Query parameters:
* `format`: output format (`jemalloc` or `pprof`; default `pprof`).
Unlike CPU profiles (see #9764), heap profiles are not symbolized and
require the original binary to translate addresses to function names. To
make this work with Grafana, we'll probably have to symbolize the
process server-side -- this is left as future work, as is other output
formats like SVG.
Heap profiles don't work on macOS due to limitations in jemalloc.
This PR
- fixes smgr metrics https://github.com/neondatabase/neon/issues/9925
- adds an additional startup log line logging the current batching
config
- adds a histogram of batch sizes global and per-tenant
- adds a metric exposing the current batching config
The issue described #9925 is that before this PR, request latency was
only observed *after* batching.
This means that smgr latency metrics (most importantly getpage latency)
don't account for
- `wait_lsn` time
- time spent waiting for batch to fill up / the executor stage to pick
up the batch.
The fix is to use a per-request batching timer, like we did before the
initial batching PR.
We funnel those timers through the entire request lifecycle.
I noticed that even before the initial batching changes, we weren't
accounting for the time spent writing & flushing the response to the
wire.
This PR drive-by fixes that deficiency by dropping the timers at the
very end of processing the batch, i.e., after the `pgb.flush()` call.
I was **unable to maintain the behavior that we deduct
time-spent-in-throttle from various latency metrics.
The reason is that we're using a *single* counter in `RequestContext` to
track micros spent in throttle.
But there are *N* metrics timers in the batch, one per request.
As a consequence, the practice of consuming the counter in the drop
handler of each timer no longer works because all but the first timer
will encounter error `close() called on closed state`.
A failed attempt to maintain the current behavior can be found in
https://github.com/neondatabase/neon/pull/9951.
So, this PR remvoes the deduction behavior from all metrics.
I started a discussion on Slack about it the implications this has for
our internal SLO calculation:
https://neondb.slack.com/archives/C033RQ5SPDH/p1732910861704029
# Refs
- fixes https://github.com/neondatabase/neon/issues/9925
- sub-issue https://github.com/neondatabase/neon/issues/9377
- epic: https://github.com/neondatabase/neon/issues/9376
# Problem
The timeout-based batching adds latency to unbatchable workloads.
We can choose a short batching timeout (e.g. 10us) but that requires
high-resolution timers, which tokio doesn't have.
I thoroughly explored options to use OS timers (see
[this](https://github.com/neondatabase/neon/pull/9822) abandoned PR).
In short, it's not an attractive option because any timer implementation
adds non-trivial overheads.
# Solution
The insight is that, in the steady state of a batchable workload, the
time we spend in `get_vectored` will be hundreds of microseconds anyway.
If we prepare the next batch concurrently to `get_vectored`, we will
have a sizeable batch ready once `get_vectored` of the current batch is
done and do not need an explicit timeout.
This can be reasonably described as **pipelining of the protocol
handler**.
# Implementation
We model the sub-protocol handler for pagestream requests
(`handle_pagrequests`) as two futures that form a pipeline:
2. Batching: read requests from the connection and fill the current
batch
3. Execution: `take` the current batch, execute it using `get_vectored`,
and send the response.
The Reading and Batching stage are connected through a new type of
channel called `spsc_fold`.
See the long comment in the `handle_pagerequests_pipelined` for details.
# Changes
- Refactor `handle_pagerequests`
- separate functions for
- reading one protocol message; produces a `BatchedFeMessage` with just
one page request in it
- batching; tried to merge an incoming `BatchedFeMessage` into an
existing `BatchedFeMessage`; returns `None` on success and returns back
the incoming message in case merging isn't possible
- execution of a batched message
- unify the timeline handle acquisition & request span construction; it
now happen in the function that reads the protocol message
- Implement serial and pipelined model
- serial: what we had before any of the batching changes
- read one protocol message
- execute protocol messages
- pipelined: the design described above
- optionality for execution of the pipeline: either via concurrent
futures vs tokio tasks
- Pageserver config
- remove batching timeout field
- add ability to configure pipelining mode
- add ability to limit max batch size for pipelined configurations
(required for the rollout, cf
https://github.com/neondatabase/cloud/issues/20620 )
- ability to configure execution mode
- Tests
- remove `batch_timeout` parametrization
- rename `test_getpage_merge_smoke` to `test_throughput`
- add parametrization to test different max batch sizes and execution
moes
- rename `test_timer_precision` to `test_latency`
- rename the test case file to `test_page_service_batching.py`
- better descriptions of what the tests actually do
## On the holding The `TimelineHandle` in the pending batch
While batching, we hold the `TimelineHandle` in the pending batch.
Therefore, the timeline will not finish shutting down while we're
batching.
This is not a problem in practice because the concurrently ongoing
`get_vectored` call will fail quickly with an error indicating that the
timeline is shutting down.
This results in the Execution stage returning a `QueryError::Shutdown`,
which causes the pipeline / entire page service connection to shut down.
This drops all references to the
`Arc<Mutex<Option<Box<BatchedFeMessage>>>>` object, thereby dropping the
contained `TimelineHandle`s.
- => fixes https://github.com/neondatabase/neon/issues/9850
# Performance
Local run of the benchmarks, results in [this empty
commit](1cf5b1463f)
in the PR branch.
Key take-aways:
* `concurrent-futures` and `tasks` deliver identical `batching_factor`
* tail latency impact unknown, cf
https://github.com/neondatabase/neon/issues/9837
* `concurrent-futures` has higher throughput than `tasks` in all
workloads (=lower `time` metric)
* In unbatchable workloads, `concurrent-futures` has 5% higher
`CPU-per-throughput` than that of `tasks`, and 15% higher than that of
`serial`.
* In batchable-32 workload, `concurrent-futures` has 8% lower
`CPU-per-throughput` than that of `tasks` (comparison to tput of
`serial` is irrelevant)
* in unbatchable workloads, mean and tail latencies of
`concurrent-futures` is practically identical to `serial`, whereas
`tasks` adds 20-30us of overhead
Overall, `concurrent-futures` seems like a slightly more attractive
choice.
# Rollout
This change is disabled-by-default.
Rollout plan:
- https://github.com/neondatabase/cloud/issues/20620
# Refs
- epic: https://github.com/neondatabase/neon/issues/9376
- this sub-task: https://github.com/neondatabase/neon/issues/9377
- the abandoned attempt to improve batching timeout resolution:
https://github.com/neondatabase/neon/pull/9820
- closes https://github.com/neondatabase/neon/issues/9850
- fixes https://github.com/neondatabase/neon/issues/9835
## Problem
It appears that the Azure storage API tends to hang TCP connections more
than S3 does.
Currently we use a 2 minute timeout for all downloads. This is large
because sometimes the objects we download are large. However, waiting 2
minutes when doing something like downloading a manifest on tenant
attach is problematic, because when someone is doing a "create tenant,
create timeline" workflow, that 2 minutes is long enough for them
reasonably to give up creating that timeline.
Rather than propagate oversized timeouts further up the stack, we should
use a different timeout for objects that we expect to be small.
Closes: https://github.com/neondatabase/neon/issues/9836
## Summary of changes
- Add a `small_timeout` configuration attribute to remote storage,
defaulting to 30 seconds (still a very generous period to do something
like download an index)
- Add a DownloadKind parameter to DownloadOpts, so that callers can
indicate whether they expect the object to be small or large.
- In the azure client, use small timeout for HEAD requests, and for GET
requests if DownloadKind::Small is used.
- Use DownloadKind::Small for manifests, indices, and heatmap downloads.
This PR intentionally does not make the equivalent change to the S3
client, to reduce blast radius in case this has unexpected consequences
(we could accomplish the same thing by editing lots of configs, but just
skipping the code is simpler for right now)
## Problem
For the interpreted proto the pageserver is not returning the correct
LSN
in replies to keep alive requests. This is because the interpreted
protocol arm
was not updating `last_rec_lsn`.
## Summary of changes
* Return correct LSN in keep-alive responses
* Fix shard field in wal sender traces
## Problem
close https://github.com/neondatabase/neon/issues/9859
## Summary of changes
Ensure that the deletion queue gets fully flushed (i.e., the deletion
lists get applied) during a graceful shutdown.
It is still possible that an incomplete shutdown would leave deletion
list behind and cause race upon the next startup, but we assume this
will unlikely happen, and even if it happened, the pageserver should
already be at a tainted state and the tenant should be moved to a new
tenant with a new generation number.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
When ingesting implicit `ClearVmBits` operations, we silently drop the
writes if the relation or page is unknown. There are implicit
assumptions around VM pages wrt. explicit/implicit updates, sharding,
and relation sizes, which can possibly drop writes incorrectly. Adding a
few metrics will allow us to investigate further and tighten up the
logic.
Touches #9855.
## Summary of changes
Add a `pageserver_wal_ingest_clear_vm_bits_unknown` metric to record
dropped `ClearVmBits` writes.
Also add comments clarifying the behavior of relation sizes on non-zero
shards.
## Problem
We don't have any observability for the relation size cache. We have
seen cache misses cause significant performance impact with high
relation counts.
Touches #9855.
## Summary of changes
Adds the following metrics:
* `pageserver_relsize_cache_entries`
* `pageserver_relsize_cache_hits`
* `pageserver_relsize_cache_misses`
* `pageserver_relsize_cache_misses_old`
## Problem
https://github.com/neondatabase/neon/pull/9746 lifted decoding and
interpretation of WAL to the safekeeper.
This reduced the ingested amount on the pageservers by around 10x for a
tenant with 8 shards, but doubled
the ingested amount for single sharded tenants.
Also, https://github.com/neondatabase/neon/pull/9746 uses bincode which
doesn't support schema evolution.
Technically the schema can be evolved, but it's very cumbersome.
## Summary of changes
This patch set addresses both problems by adding protobuf support for
the interpreted wal records and adding compression support. Compressed
protobuf reduced the ingested amount by 100x on the 32 shards
`test_sharded_ingest` case (compared to non-interpreted proto). For the
1 shard case the reduction is 5x.
Sister change to `rust-postgres` is
[here](https://github.com/neondatabase/rust-postgres/pull/33).
## Links
Related: https://github.com/neondatabase/neon/issues/9336
Epic: https://github.com/neondatabase/neon/issues/9329