## Problem
We recently added slow GetPage request logging. However, this
unintentionally included the flush time when logging (which we already
have separate logging for). It also logs at WARN level, which is a bit
aggressive since we see this fire quite frequently.
Follows https://github.com/neondatabase/neon/pull/10906.
## Summary of changes
* Only log the request execution time, not the flush time.
* Extract a `pagestream_dispatch_batched_message()` helper.
* Rename `warn_slow()` to `log_slow()` and downgrade to INFO.
We've seen some cases in production where a compute doesn't get a
response to a pageserver request for several minutes, or even more. We
haven't found the root cause for that yet, but whatever the reason is,
it seems overly optimistic to think that if the pageserver hasn't
responded for 2 minutes, we'd get a response if we just wait patiently a
little longer. More likely, the pageserver is dead or there's some kind
of a network glitch so that the TCP connection is dead, or at least
stuck for a long time. Either way, it's better to disconnect and
reconnect. I set the default timeout to 2 minutes, which should be
enough for any GetPage request under normal circumstances, even if the
pageserver has to download several layer files from remote storage.
Make the disconnect timeout configurable. Also make the "log interval",
after which we print a message to the log configurable, so that if you
change the disconnect timeout, you can set the log timeout
correspondingly. The default log interval is still 10 s. The new GUCs
are called "neon.pageserver_response_log_timeout" and
"neon.pageserver_response_disconnect_timeout".
Includes a basic test for the log and disconnect timeouts.
Implements issue #10857
For archived timelines, we would like to prevent all non-pageserver
issued getpage requests, as users are not supposed to send these.
Instead, they should unarchive a timeline before issuing any external
read traffic.
As this is non-trivial to do, at least prevent launches of new computes,
by errorring on basebackup requests for archived timelines. In #10688,
we started issuing a warning instead of an error, because an error would
mean a stuck project. Now after we can confirm the the warning is not
present in the logs for about a week, we can issue errors.
Follow-up of #10688
Related: #9548
## Problem
https://github.com/neondatabase/neon/pull/10788 introduced an API for
warming up attached locations
by downloading all layers in the heatmap. We intend to use it for
warming up timelines after unarchival too,
but it doesn't work. Any heatmap generated after the unarchival will not
include our timeline, so we've lost
all those layers.
## Summary of changes
Generate a cheeky heatmap on unarchival. It includes all the visible
layers.
Use that as the `PreviousHeatmap` which inputs into actual heatmap
generation.
Closes: https://github.com/neondatabase/neon/issues/10541
The synchronous 'postgres' crate is just a wrapper around the async
'tokio_postgres' crate. Some places were unnecessarily using the
re-exported NoTls and Error from the synchronous 'postgres' crate, even
though they were otherwise using the 'tokio_postgres' crate. Tidy up by
using the tokio_postgres types directly.
## Problem
The storage controller treats durations in the tenant config as strings.
These are loaded from the db.
The pageserver maps these durations to a seconds only format and we
always get a mismatch compared
to what's in the db.
## Summary of changes
Treat durations as durations inside the storage controller and not as
strings.
Nothing changes in the cross service API's themselves or the way things
are stored in the db.
I also added some logging which I would have made the investigation a
10min job:
1. Reason for why the reconciliation was spawned
2. Location config diff between the observed and wanted states
## Problem
This should be one last fix for
https://github.com/neondatabase/neon/issues/10517.
## Summary of changes
If a keyspace is empty, we might produce a gc-compaction job which
covers no layer files. We should avoid generating such jobs so that the
gc-compaction image layer can cover the full key range.
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
We don't have good observability for "stuck" getpage requests.
Resolves https://github.com/neondatabase/cloud/issues/23808.
## Summary of changes
Log a periodic warning (every 30 seconds) if GetPage request execution
is slow to complete, to aid in debugging stuck GetPage requests.
This does not cover response flushing (we have separate logging for
that), nor reading the request from the socket and batching it (expected
to be insignificant and not straightforward to handle with the current
protocol).
This costs 95 nanoseconds on the happy path when awaiting a
`tokio::task::yield_now()`:
```
warn_slow/enabled=false time: [45.716 ns 46.116 ns 46.687 ns]
warn_slow/enabled=true time: [141.53 ns 141.83 ns 142.18 ns]
```
## Problem
The interpreted SK <-> PS protocol does not guard against gaps (neither
does the Vanilla one, but that's beside the point).
## Summary of changes
Extend the protocol to include the start LSN of the PG WAL section from
which the records were interpreted.
Validation is enabled via a config flag on the pageserver and works as
follows:
**Case 1**: `raw_wal_start_lsn` is smaller than the requested LSN
There can't be gaps here, but we check that the shard received records
which it hasn't seen before.
**Case 2**: `raw_wal_start_lsn` is equal to the requested LSN
This is the happy case. No gap and nothing to check
**Case 3**: `raw_wal_start_lsn` is greater than the requested LSN
This is a gap.
To make Case 3 work I had to bend the protocol a bit.
We read record chunks of WAL which aren't record aligned and feed them
to the decoder.
The picture below shows a shard which subscribes at a position somewhere
within Record 2.
We already have a wal reader which is below that position so we wait to
catch up.
We read some wal in Read 1 (all of Record 1 and some of Record 2). The
new shard doesn't
need Record 1 (it has already processed it according to the starting
position), but we read
past it's starting position. When we do Read 2, we decode Record 2 and
ship it off to the shard,
but the starting position of Read 2 is greater than the starting
position the shard requested.
This looks like a gap.

To make it work, we extend the protocol to send an empty
`InterpretedWalRecords` to shards
if the WAL the records originated from ends the requested start
position. On the pageserver,
that just updates the tracking LSNs in memory (no-op really). This gives
us a workaround for
the fake gap.
As a drive by, make `InterpretedWalRecords::next_record_lsn` mandatory
in the application level definition.
It's always included.
Related: https://github.com/neondatabase/cloud/issues/23935
## Problem
Storage controller uses unsecure http for pageserver API.
Closes: https://github.com/neondatabase/cloud/issues/23734
Closes: https://github.com/neondatabase/cloud/issues/24091
## Summary of changes
- Add an optional `listen_https_port` field to storage controller's Node
state and its API (RegisterNode/ListNodes/etc).
- Allow updating `listen_https_port` on node registration to gradually
add https port for all nodes.
- Add `use_https_pageserver_api` CLI option to storage controller to
enable https.
- Pageserver doesn't support https for now and always reports
`https_port=None`. This will be addressed in follow-up PR.
## Problem
A simpler version of https://github.com/neondatabase/neon/pull/10812
## Summary of changes
Image layer creation will be preempted by L0 accumulated on other
timelines. We stop image layer generation if there's a pending L0
compaction request.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
Read errors during repartition should be a critical error.
## Summary of changes
<del>We only have one call site</del> We have two call sites of
`repartition` where one of them is during the initial image upload
optimization and another is during image layer creation, so I added a
`critical!` here instead of inside `collect_keyspace`.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
The usual workflow for me to debug read path errors in staging is:
download the tenant to my laptop, import, and then run some read tests.
With this patch, we can do this directly over staging pageservers.
## Summary of changes
* Add a new `touchpagelsn` API that does a page read but does not return
page info back.
* Allow read from latest record LSN from get/touchpagelsn
* Add read_debug config in the context.
* The read path will read the context config to decide whether to enable
read path tracing or not.
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
We log image compaction stats even when no image compaction happened.
This is logged every 10 seconds for every timeline.
## Summary of changes
Only log when we actually performed any image compaction.
## Problem
We lack an API for warming up attached locations based on the heatmap
contents.
This is problematic in two places:
1. If we manually migrate and cut over while the secondary is still cold
2. When we re-attach a previously offloaded tenant
## Summary of changes
https://github.com/neondatabase/neon/pull/10597 made heatmap generation
additive
across migrations, so we won't clobber it a after a cold migration. This
allows us to implement:
1. An endpoint for downloading all missing heatmap layers on the
pageserver:
`/v1/tenant/:tenant_shard_id/timeline/:timeline_id/download_heatmap_layers`.
Only one such operation per timeline is allowed at any given time. The
granularity is tenant shard.
2. An endpoint to the storage controller to trigger the downloads on the
pageserver:
`/v1/tenant/:tenant_shard_id/timeline/:timeline_id/download_heatmap_layers`.
This works both at
tenant and tenant shard level. If an unsharded tenant id is provided,
the operation is started on
all shards, otherwise only the specified shard.
3. A storcon cli command. Again, tenant and tenant-shard level
granularities are supported.
Cplane will call into storcon and trigger the downloads for all shards.
When we want to rescue a migration, we will use storcon cli targeting
the specific tenant shard.
Related: https://github.com/neondatabase/neon/issues/10541
## Problem
ref https://github.com/neondatabase/neon/issues/10517
## Summary of changes
For some reasons the job split algorithm decides to have different image
coverage range for two compactions before/after restart. So we remove
the subcompaction key range and let it generate an image covering the
full range, which should make the test more stable.
Also slightly tuned the logging span.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
This reverts commit 443c8d0b4b.
## Problem
We observe a massive amount of compaction errors.
## Summary of changes
If the tenant did not write any L1 layers (i.e., they accumulate L0
layers where number of them is below L0 threshold), image creation will
always fail. Therefore, it's not correct to simply use the
disk_consistent_lsn or L0/L1 boundary for the image creation.
## Problem
The current `pageserver_layers_per_read` histogram buckets don't
represent the current reality very well. For the percentiles we care
about (e.g. p50 and p99), we often see fairly high read amp, especially
during ingestion, and anything below 4 can be considered very good.
## Summary of changes
Change the per-timeline read amp histogram buckets to `[4.0, 8.0, 16.0,
32.0, 64.0, 128.0, 256.0]`.
## Problem
`SmgrOpFlushInProgress::measure()` takes a `socket_fd` argument which is
only used on Linux. This causes linter warnings on macOS.
Touches #10823.
## Summary of changes
Add a noop use of `socket_fd` on non-Linux branch.
## Problem
Part of https://github.com/neondatabase/neon/issues/9516
## Summary of changes
This patch adds the support for storing reldir in the sparse keyspace.
All logic are guarded with the `rel_size_v2_enabled` flag, so if it's
set to false, the code path is exactly the same as what's currently in
prod.
Note that we did not persist the `rel_size_v2_enabled` flag and the
logic around it will be implemented in the next patch. (i.e., what if we
enabled it, restart the pageserver, and then it gets set to false? we
should still read from v2 using the rel_size_v2_migration_status in the
index_part). The persistence logic I'll implement in the next patch will
disallow switching from v2->v1 via config item.
I also refactored the metrics so that it can work with the new reldir
store. However, this metric is not correctly computed for reldirs (see
the comments) before. With the refactor, the value will be computed only
when we have an initial value for the reldir size. The refactor keeps
the incorrectness of the computation when there are more than 1
database.
For the tests, we currently run all the tests with v2, and I'll set it
to false and add some v2-specific tests before merging, probably also
v1->v2 migration tests.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
I noticed the opportunity to simplify here while working on
https://github.com/neondatabase/neon/pull/9353 .
The only difference is the zero-fill behavior: if one reads past rel
size,
`get_rel_page_at_lsn` returns a zeroed page whereas `Timeline::get`
returns an error.
However, the `endblk` is at most rel size large, because `nblocks` is eq
`get_rel_size`, see a few lines above this change.
We're using the same LSN (`self.lsn`) for everything, so there is no
chance of non-determinism.
Refs:
- Slack discussion debating correctness:
https://neondb.slack.com/archives/C033RQ5SPDH/p1737457010607119
# Summary
In
- https://github.com/neondatabase/neon/pull/10813
we added slow flush logging but it didn't log the TCP send & recv queue
length.
This PR adds that data to the log message.
I believe the implementation to be safe & correct right now, but it's
brittle and thus this PR should be reverted or improved upon once the
investigation is over.
Refs:
- stacked atop https://github.com/neondatabase/neon/pull/10813
- context:
https://neondb.slack.com/archives/C08DE6Q9C3B/p1739464533762049?thread_ts=1739462628.361019&cid=C08DE6Q9C3B
- improves https://github.com/neondatabase/neon/issues/10668
- part of https://github.com/neondatabase/cloud/issues/23515
# How It Works
The trouble is two-fold:
1. getting to the raw socket file descriptor through the many Rust types
that wrap it and
2. integrating with the `measure()` function
Rust wraps it in types to model file descriptor lifetimes and ownership,
and usually one can get access using `as_raw_fd()`.
However, we `split()` the stream and the resulting
[`tokio::io::WriteHalf`](https://docs.rs/tokio/latest/tokio/io/struct.WriteHalf.html)
.
Check the PR commit history for my attempts to do it.
My solution is to get the socket fd before we wrap it in our protocol
types, and to store that fd in the new `PostgresBackend::socket_fd`
field.
I believe it's safe because the lifetime of `PostgresBackend::socket_fd`
value == the lifetime of the `TcpStream` that wrap and store in
`PostgresBackend::framed`.
Specifically, the only place that close()s the socket is the `impl Drop
for TcpStream`.
I think the protocol stack calls `TcpStream::shutdown()`, but, that
doesn't `close()` the file descriptor underneath.
Regarding integration with the `measure()` function, the trouble is that
`flush_fut` is currently a generic `Future` type. So, we just pass in
the `socket_fd` as a separate argument.
A clean implementation would convert the `pgb_writer.flush()` to a named
future that provides an accessor for the socket fd while not being
polled.
I tried (see PR history), but failed to break through the `WriteHalf`.
# Testing
Tested locally by running
```
./target/debug/pagebench get-page-latest-lsn --num-clients=1000 --queue-depth=1000
```
in one terminal, waiting a bit, then
```
pkill -STOP pagebench
```
then wait for slow logs to show up in `pageserver.log`.
Pick one of the slow log message's port pairs, e.g., `127.0.0.1:39500`,
and then checking sockstat output
```
ss -ntp | grep '127.0.0.1:39500'
```
to ensure that send & recv queue size match those in the log message.
The logic might seem a bit intricate / over-optimized, but I recently
spent time benchmarking this code path in the context of a nightly
pagebench regression
(https://github.com/neondatabase/cloud/issues/21759)
and I want to avoid regressing it any further.
Ideally would also log the socket send & recv queue length like we do on
the compute side in
- https://github.com/neondatabase/neon/pull/10673
But that is proving difficult due to the Rust abstractions that wrap the
socket fd.
Work in progress on that is happening in
- https://github.com/neondatabase/neon/pull/10823
Regarding production impact, I am worried at a theoretical level that
the additional logging may cause a downward spiral in the case where a
pageserver is slow to flush because there is not enough CPU. The logging
would consume more CPU and thereby slow down flushes even more. However,
I don't think this matters practically speaking.
# Refs
- context:
https://neondb.slack.com/archives/C08DE6Q9C3B/p1739464533762049?thread_ts=1739462628.361019&cid=C08DE6Q9C3B
- fixes https://github.com/neondatabase/neon/issues/10668
- part of https://github.com/neondatabase/cloud/issues/23515
# Testing
Tested locally by running
```
./target/debug/pagebench get-page-latest-lsn --num-clients=1000 --queue-depth=1000
```
in one terminal, waiting a bit, then
```
pkill -STOP pagebench
```
then wait for slow logs to show up in `pageserver.log`.
To see that the completion log message is logged, run
```
pkill -CONT pagebench
```
PR #10305 makes sure that there is no *actual* race, i.e. we will never
attempt to offload a timeline that has just been unarchived, or similar.
However, if a timeline has been unarchived and has children that are
unarchived too, we will get an error log line. Such races can occur as
in compaction we check if the timeline can be offloaded way before we
attempt to offload it: the result might change in the meantime.
This patch checks if the delete guard can't be obtained because the
timeline has unarchived children, and if yes, it does another check for
whether the timeline has become unarchived or not. If it is unarchived,
it just prints an info log msg and integrates itself into the error
suppression logic of the compaction calling into it.
If you squint at it really closely, there is still a possible race in
which we print an error log, but this one is unlikely because the
timeline and its children need to be archived right after the check for
whether the timeline has any unarchived children, and right before the
check whether the timeline is archived. Archival involves a network
operation while nothing between these two checks does that, so it's very
unlikely to happen in real life.
https://github.com/neondatabase/cloud/issues/23979#issuecomment-2651265729
Before this PR, an IO error returned from the kernel, e.g., due to a bad
disk, would get bubbled up, all the way to a user-visible query failing.
This is against the IO error handling policy where we have established
and is hence being rectified in this PR.
[[(internal Policy document
link)]](bef44149f7/src/storage/handling_io_and_logical_errors.md (L33-L35))
The practice on the write path seems to be that we call
`maybe_fatal_err()` or `fatal_err()` fairly high up the stack.
That is, regardless of whether std::fs, tokio::fs, or VirtualFile is
used to perform the IO.
For the read path, I choose a centralized approach in this PR by
checking for errors as close to the kernel interface as possible.
I believe this is better for long-term consistency.
To mitigate the problem of missing context if we abort so far down in
the stack, the `on_fatal_io_error` now captures and logs a backtrace.
I grepped the pageserver code base for `fs::read` to convince myself
that all non-VirtualFile reads already handle IO errors according to
policy.
Refs
- fixes https://github.com/neondatabase/neon/issues/10454
## Problem
We didn't catch all client errors causing alerts.
## Summary of changes
Client errors should be wrapped with ClientError so that it doesn't fire
alerts.
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
Previously, when cutting over to cold secondary locations,
we would clobber the previous, good, heatmap with a cold one.
This is because heatmap generation used to include only resident layers.
Once this merges, we can add an endpoint which triggers full heatmap
hydration on attached locations to heal cold migrations.
## Summary of changes
With this patch, heatmap generation becomes additive. If we have a
heatmap from when this location was secondary, the new uploaded heatmap
will be the result of a reconciliation between the old one and the on
disk resident layers.
More concretely, when we have the previous heatmap:
1. Filter the previous heatmap and keep layers that are (a) present
in the current layer map, (b) visible, (c) not resident. Call this set
of layers `visible_non_resident`.
2. From the layer map, select all layers that are resident and visible.
Call this set of layers `resident`.
3. The new heatmap is the result of merging the two disjoint sets.
Related https://github.com/neondatabase/neon/issues/10541
## Problem
We expose `latest_gc_cutoff` in our API, and callers understandably were
using that to validate LSNs for branch creation. However, this is _not_
the true GC cutoff from a user's point of view: it's just the point at
which we last actually did GC. The actual cutoff used when validating
branch creations and page_service reads is the min() of latest_gc_cutoff
and the planned GC lsn in GcInfo.
Closes: https://github.com/neondatabase/neon/issues/10639
## Summary of changes
- Expose the more useful min() of GC cutoffs as `gc_cutoff_lsn` in the
API, so that the most obviously named field is really the one people
should use.
- Retain the ability to read the LSN at which GC was actually done, in
an `applied_gc_cutoff_lsn` field.
- Internally rename `latest_gc_cutoff_lsn` to `applied_gc_cutoff_lsn`
("latest" was a confusing name, as the value in GcInfo is more up to
date in terms of what a user experiences)
- Temporarily preserve the old `latest_gc_cutoff_lsn` field for compat
with control plane until we update it to use the new field.
---------
Co-authored-by: Arpad Müller <arpad-m@users.noreply.github.com>
## Problem
L0 compaction frequently gets starved out by other background tasks and
image/GC compaction. L0 compaction must be responsive to keep read
amplification under control.
Touches #10694.
Resolves#10689.
## Summary of changes
Use a separate semaphore for the L0-only compaction pass.
* Add a `CONCURRENT_L0_COMPACTION_TASKS` semaphore and
`BackgroundLoopKind::L0Compaction`.
* Add a setting `compaction_l0_semaphore` (default off via
`compaction_l0_first`).
* Use the L0 semaphore when doing an `OnlyL0Compaction` pass.
* Use the background semaphore when doing a regular compaction pass
(which includes an initial L0 pass).
* While waiting for the background semaphore, yield for L0 compaction if
triggered.
* Add `CompactFlags::NoYield` to disable L0 yielding, and set it for the
HTTP API route.
* Remove the old `use_compaction_semaphore` setting and
compaction-scoped semaphore.
* Remove the warning when waiting for a semaphore; it's noisy and we
have metrics.
## Problem
When image compaction yields for L0 compaction, it may not immediately
schedule L0 compaction, because it just goes on to compact the next
pending timeline.
Touches #10694.
Requires #10744.
## Summary of changes
Extend `CompactionOutcome` with `YieldForL0` and `Skipped` variants, and
immediately schedule an L0 compaction pass in the `YieldForL0` case.
## Problem
Image compaction can starve out L0 compaction if a tenant has several
timelines with L0 debt.
Touches #10694.
Requires #10740.
## Summary of changes
* Add an initial L0 compaction pass, in order of L0 count.
* Add a tenant option `compaction_l0_first` to control the L0 pass
(disabled by default).
* Add `CompactFlags::OnlyL0Compaction` to run an L0-only compaction
pass.
* Clean up the compaction iteration logic.
A later PR will use separate semaphores for the L0 and image compaction
passes to avoid cross-tenant L0 starvation. That PR will also make image
compaction yield if _any_ of the tenant's timelines have pending L0
compaction to further avoid starvation.
Avoids compiling the crate and its dependencies into binaries that don't
need them. Shrinks the compute_ctl binary from about 31MB to 28MB in the
release-line-debug-size-lto profile.
This changes the default value of the `timeline_offloading` pageserver
and tenant configs to true, now that offloading has been rolled out
without problems.
There is also a small fix in the tenant config merge function, where we
applied the `lazy_slru_download` value instead of `timeline_offloading`.
Related issue: https://github.com/neondatabase/cloud/issues/21353
# Problem
Say we have a batch of 10 responses to send out.
Then, even with
- #10728
we've still only called observe_execution_end_flush_start for the first
3 responses.
The remaining 7 response timers are still ticking.
When compute now closes the connection, the waiting flush fails with an
error and we `drop()` the remaining 7 responses' smgr op timers. The
`impl Drop for SmgrOpTimer` will observe an execution time that includes
the flush time.
In practice, this is supsected to produce the `+Inf` observations in the
smgr op latency histogram we've seen since the introduction of
pipelining, even after shipping #10728.
refs:
- fixup of https://github.com/neondatabase/neon/pull/10042
- fixup of https://github.com/neondatabase/neon/pull/10728
- fixes https://github.com/neondatabase/neon/issues/10754
## Problem
close https://github.com/neondatabase/neon/issues/10213
`range_search` only returns the top-most layers that may satisfy the
search, so it doesn't include all layers that might be accessed (the
user needs to recursively call this function). We need to retrieve the
full layer map and find overlaps in order to have a correct heuristics
of the job split.
## Summary of changes
Retrieve all layers and find overlaps instead of doing `range_search`.
The patch also reduces the time holding the layer map read guard.
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
The compaction loop currently runs periodically, which can cause it to
wait for up to 20 seconds before starting L0 compaction by default.
Also, when we later separate the semaphores for L0 compaction and image
compaction, we want to give up waiting for the image compaction
semaphore if L0 compaction is needed on any timeline.
Touches #10694.
## Summary of changes
Notify the compaction loop when an L0 flush (on any timeline) exceeds
`compaction_threshold`.
Also do some opportunistic cleanups in the area.
## Problem
helps investigate https://github.com/neondatabase/neon/issues/10482
## Summary of changes
In debug mode and testing mode, we will record all files visited by a
read operation, and print it out when it errors.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
Reduce the read amplification when doing `repartition`.
## Summary of changes
Compute the L0-L1 boundary LSN and do repartition here.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
# Problem
walredo shutdown is done in the compaction task. Let's move it to tenant
housekeeping.
# Summary of changes
* Rename "ingest housekeeping" to "tenant housekeeping".
* Move walredo shutdown into tenant housekeeping.
* Add a constant `WALREDO_IDLE_TIMEOUT` set to 3 minutes (previously 10x
compaction threshold).
This patch does a bunch of superficial cleanups of `tenant::tasks` to
avoid noise in subsequent PRs. There are no functional changes.
PS: enable "hide whitespace" when reviewing, due to the unindentation of
large async blocks.
# Problem
Before this PR, the `shard_id` field was missing when page_service logs
a reconstruct error.
This was caused by batching-related refactorings.
Example from staging:
```
2025-01-30T07:10:04.346022Z ERROR page_service_conn_main{peer_addr=...}:process_query{tenant_id=... timeline_id=...}:handle_pagerequests:request:handle_get_page_at_lsn_request_batched{req_lsn=FFFFFFFF/FFFFFFFF}: error reading relation or page version: Read error: whole vectored get request failed because one or more of the requested keys were missing: could not find data for key ...
```
# Changes
Delay creation of the handler-specific span until after shard routing
This also avoids the need for the record() call in the pagestream hot
path.
# Testing
Manual testing with a failpoint that is part of this PR's history but
will be squashed away.
# Refs
- fixes https://github.com/neondatabase/neon/issues/10599