Commit Graph

29 Commits

Author SHA1 Message Date
Christian Schwarz
e33e109403 fix(pageserver): buffered writer cancellation error handling (#12376)
## Problem

The problem has been well described in already-commited PR #11853.
tl;dr: BufferedWriter is sensitive to cancellation, which the previous
approach was not.

The write path was most affected (ingest & compaction), which was mostly
fixed in #11853:
it introduced `PutError` and mapped instances of `PutError` that were
due to cancellation of underlying buffered writer into
`CreateImageLayersError::Cancelled`.

However, there is a long tail of remaining errors that weren't caught by
#11853 that result in `CompactionError::Other`s, which we log with great
noise.

## Solution

The stack trace logging for CompactionError::Other added in #11853
allows us to chop away at that long tail using the following pattern:
- look at the stack trace
- from leaf up, identify the place where we incorrectly map from the
distinguished variant X indicating cancellation to an `anyhow::Error`
- follow that anyhow further up, ensuring it stays the same anyhow all
the way up in the `CompactionError::Other`
- since it stayed one anyhow chain all the way up, root_cause() will
yield us X
- so, in `log_compaction_error`, add an additional `downcast_ref` check
for X

This PR specifically adds checks for
- the flush task cancelling (FlushTaskError, BlobWriterError)
- opening of the layer writer (GateError)

That should cover all the reports in issues 
- https://github.com/neondatabase/cloud/issues/29434
- https://github.com/neondatabase/neon/issues/12162

## Refs
- follow-up to #11853
- fixup of / fixes https://github.com/neondatabase/neon/issues/11762
- fixes https://github.com/neondatabase/neon/issues/12162
- refs https://github.com/neondatabase/cloud/issues/29434
2025-06-27 15:26:00 +00:00
Christian Schwarz
aad410c8f1 improve ondemand-download latency observability (#11421)
## Problem

We don't have metrics to exactly quantify the end user impact of
on-demand downloads.

Perf tracing is underway (#11140) to supply us with high-resolution
*samples*.

But it will also be useful to have some aggregate per-timeline and
per-instance metrics that definitively contain all observations.

## Summary of changes

This PR consists of independent commits that should be reviewed
independently.

However, for convenience, we're going to merge them together.

- refactor(metrics): measure_remote_op can use async traits
- impr(pageserver metrics): task_kind dimension for
remote_timeline_client latency histo
  - implements https://github.com/neondatabase/cloud/issues/26800
- refs
https://github.com/neondatabase/cloud/issues/26193#issuecomment-2769705793
- use the opportunity to rename the metric and add a _global suffix;
checked grafana export, it's only used in two personal dashboards, one
of them mine, the other by Heikki
- log on-demand download latency for expensive-to-query but precise
ground truth
- metric for wall clock time spent waiting for on-demand downloads

## Refs

- refs https://github.com/neondatabase/cloud/issues/26800
- a bunch of minor investigations / incidents into latency outliers
2025-04-04 18:04:39 +00:00
Christian Schwarz
9fb77d6cdd buffered writer: add cancellation sensitivity (#11052)
In
-
https://github.com/neondatabase/neon/pull/10993#issuecomment-2690428336

I added infinite retries for buffered writer flush IOs, primarily to
gracefully handle ENOSPC but more generally so that the buffered writer
is not left in a state where reads from the surrounding InMemoryLayer
cause panics.

However, I didn't add cancellation sensitivity, which is concerning
because then there is no way to detach a timeline/tenant that is
encountering the write IO errors.
That’s a legitimate scenario in the case of some edge case bug. 
See the #10993 description for details.


This PR
- first makes flush loop infallible, enabled by infinite retries
- then adds sensitivity to `Timeline::cancel` to the flush loop, thereby
making it fallible in one specific way again
- finally fixes the InMemoryLayer/EphemeralFile/BufferedWriter
amalgamate to remain read-available after flush loop is cancelled.

The support for read-availability after cancellation is necessary so
that reads from the InMemoryLayer that are already queued up behind the
RwLock that wraps the BufferedWriter won't panic because of the
`mutable=None` that we leave behind in case the flush loop gets
cancelled.

# Alternatives

One might think that we can only ship the change for read-availability
if flush encounters an error, without the infinite retrying and/or
cancellation sensitivity complexity.

The problem with that is that read-availability sounds good but is
really quite useless, because we cannot ingest new WAL without a
writable InMemoryLayer. Thus, very soon after we transition to read-only
mode, reads from compute are going to wait anyway, but on `wait_lsn`
instead of the RwLock, because ingest isn't progressing.

Thus, having the infinite flush retries still makes more sense because
they're just "slowness" to the user, whereas wait_lsn is hard errors.
2025-03-18 18:48:43 +00:00
Arpad Müller
a22be5af72 Migrate the last crates to edition 2024 (#10998)
Migrates the remaining crates to edition 2024. We like to stay on the
latest edition if possible. There is no functional changes, however some
code changes had to be done to accommodate the edition's breaking
changes.

Like the previous migration PRs, this is comprised of three commits:

* the first does the edition update and makes `cargo check`/`cargo
clippy` pass. we had to update bindgen to make its output [satisfy the
requirements of edition
2024](https://doc.rust-lang.org/edition-guide/rust-2024/unsafe-extern.html)
* the second commit does a `cargo fmt` for the new style edition.
* the third commit reorders imports as a one-off change. As before, it
is entirely optional.

Part of #10918
2025-02-27 09:40:40 +00:00
Vlad Lazar
414ed82c1f pageserver: issue concurrent IO on the read path (#9353)
## Refs

- Epic: https://github.com/neondatabase/neon/issues/9378

Co-authored-by: Vlad Lazar <vlad@neon.tech>
Co-authored-by: Christian Schwarz <christian@neon.tech>

## Problem

The read path does its IOs sequentially.
This means that if N values need to be read to reconstruct a page,
we will do N IOs and getpage latency is `O(N*IoLatency)`.

## Solution

With this PR we gain the ability to issue IO concurrently within one
layer visit **and** to move on to the next layer without waiting for IOs
from the previous visit to complete.

This is an evolved version of the work done at the Lisbon hackathon,
cf https://github.com/neondatabase/neon/pull/9002.

## Design

### `will_init` now sourced from disk btree index keys

On the algorithmic level, the only change is that the
`get_values_reconstruct_data`
now sources `will_init` from the disk btree index key (which is
PS-page_cache'd), instead
of from the `Value`, which is only available after the IO completes.

### Concurrent IOs, Submission & Completion 

To separate IO submission from waiting for its completion, while
simultaneously
feature-gating the change, we introduce the notion of an `IoConcurrency`
struct
through which IO futures are "spawned".

An IO is an opaque future, and waiting for completions is handled
through
`tokio::sync::oneshot` channels.
The oneshot Receiver's take the place of the `img` and `records` fields
inside `VectoredValueReconstructState`.

When we're done visiting all the layers and submitting all the IOs along
the way
we concurrently `collect_pending_ios` for each value, which means
for each value there is a future that awaits all the oneshot receivers
and then calls into walredo to reconstruct the page image.
Walredo is now invoked concurrently for each value instead of
sequentially.
Walredo itself remains unchanged.

The spawned IO futures are driven to completion by a sidecar tokio task
that
is separate from the task that performs all the layer visiting and
spawning of IOs.
That tasks receives the IO futures via an unbounded mpsc channel and
drives them to completion inside a `FuturedUnordered`.

(The behavior from before this PR is available through
`IoConcurrency::Sequential`,
which awaits the IO futures in place, without "spawning" or "submitting"
them
anywhere.)

#### Alternatives Explored

A few words on the rationale behind having a sidecar *task* and what
alternatives were considered.

One option is to queue up all IO futures in a FuturesUnordered that is
polled
the first time when we `collect_pending_ios`.

Firstly, the IO futures are opaque, compiler-generated futures that need
to be polled at least once to submit their IO. "At least once" because
tokio-epoll-uring may not be able to submit the IO to the kernel on
first
poll right away.

Second, there are deadlocks if we don't drive the IO futures to
completion
independently of the spawning task.
The reason is that both the IO futures and the spawning task may hold
some
_and_ try to acquire _more_ shared limited resources.
For example, both spawning task and IO future may try to acquire
* a VirtualFile file descriptor cache slot async mutex (observed during
impl)
* a tokio-epoll-uring submission slot (observed during impl)
* a PageCache slot (currently this is not the case but we may move more
code into the IO futures in the future)

Another option is to spawn a short-lived `tokio::task` for each IO
future.
We implemented and benchmarked it during development, but found little
throughput improvement and moderate mean & tail latency degradation.
Concerns about pressure on the tokio scheduler made us discard this
variant.

The sidecar task could be obsoleted if the IOs were not arbitrary code
but a well-defined struct.
However,
1. the opaque futures approach taken in this PR allows leaving the
existing
   code unchanged, which
2. allows us to implement the `IoConcurrency::Sequential` mode for
feature-gating
   the change.

Once the new mode sidecar task implementation is rolled out everywhere,
and `::Sequential` removed, we can think about a descriptive submission
& completion interface.
The problems around deadlocks pointed out earlier will need to be solved
then.
For example, we could eliminate VirtualFile file descriptor cache and
tokio-epoll-uring slots.
The latter has been drafted in
https://github.com/neondatabase/tokio-epoll-uring/pull/63.

See the lengthy doc comment on `spawn_io()` for more details.

### Error handling

There are two error classes during reconstruct data retrieval:
* traversal errors: index lookup, move to next layer, and the like
* value read IO errors

A traversal error fails the entire get_vectored request, as before this
PR.
A value read error only fails that value.

In any case, we preserve the existing behavior that once
`get_vectored` returns, all IOs are done. Panics and failing
to poll `get_vectored` to completion will leave the IOs dangling,
which is safe but shouldn't happen, and so, a rate-limited
log statement will be emitted at warning level.
There is a doc comment on `collect_pending_ios` giving more code-level
details and rationale.

### Feature Gating

The new behavior is opt-in via pageserver config.
The `Sequential` mode is the default.
The only significant change in `Sequential` mode compared to before
this PR is the buffering of results in the `oneshot`s.

## Code-Level Changes

Prep work:
  * Make `GateGuard` clonable.

Core Feature:
* Traversal code: track  `will_init` in `BlobMeta` and source it from
the Delta/Image/InMemory layer index, instead of determining `will_init`
  after we've read the value. This avoids having to read the value to
  determine whether traversal can stop.
* Introduce `IoConcurrency` & its sidecar task.
  * `IoConcurrency` is the clonable handle.
  * It connects to the sidecar task via an `mpsc`.
* Plumb through `IoConcurrency` from high level code to the
  individual layer implementations' `get_values_reconstruct_data`.
  We piggy-back on the `ValuesReconstructState` for this.
   * The sidecar task should be long-lived, so, `IoConcurrency` needs
     to be rooted up "high" in the call stack.
   * Roots as of this PR:
     * `page_service`: outside of pagestream loop
     * `create_image_layers`: when it is called
     * `basebackup`(only auxfiles + replorigin + SLRU segments)
   * Code with no roots that uses `IoConcurrency::sequential`
     * any `Timeline::get` call
       * `collect_keyspace` is a good example
       * follow-up: https://github.com/neondatabase/neon/issues/10460
* `TimelineAdaptor` code used by the compaction simulator, unused in
practive
     * `ingest_xlog_dbase_create`
* Transform Delta/Image/InMemoryLayer to
  * do their values IO in a distinct `async {}` block
  * extend the residence of the Delta/Image layer until the IO is done
  * buffer their results in a `oneshot` channel instead of straight
    in `ValuesReconstructState` 
* the `oneshot` channel is wrapped in `OnDiskValueIo` /
`OnDiskValueIoWaiter`
    types that aid in expressiveness and are used to keep track of
    in-flight IOs so we can print warnings if we leave them dangling.
* Change `ValuesReconstructState` to hold the receiving end of the
 `oneshot` channel aka `OnDiskValueIoWaiter`.
* Change `get_vectored_impl` to `collect_pending_ios` and issue walredo
concurrently, in a `FuturesUnordered`.

Testing / Benchmarking:
* Support queue-depth in pagebench for manual benchmarkinng.
* Add test suite support for setting concurrency mode ps config
   field via a) an env var and b) via NeonEnvBuilder.
* Hacky helper to have sidecar-based IoConcurrency in tests.
   This will be cleaned up later.

More benchmarking will happen post-merge in nightly benchmarks, plus in
staging/pre-prod.
Some intermediate helpers for manual benchmarking have been preserved in
https://github.com/neondatabase/neon/pull/10466 and will be landed in
later PRs.
(L0 layer stack generator!)

Drive-By:
* test suite actually didn't enable batching by default because
`config.compatibility_neon_binpath` is always Truthy in our CI
environment
  => https://neondb.slack.com/archives/C059ZC138NR/p1737490501941309
* initial logical size calculation wasn't always polled to completion,
which was
  surfaced through the added WARN logs emitted when dropping a 
  `ValuesReconstructState` that still has inflight IOs.
* remove the timing histograms
`pageserver_getpage_get_reconstruct_data_seconds`
and `pageserver_getpage_reconstruct_seconds` because with planning,
value read
IO, and walredo happening concurrently, one can no longer attribute
latency
to any one of them; we'll revisit this when Vlad's work on
tracing/sampling
  through RequestContext lands.
* remove code related to `get_cached_lsn()`.
  The logic around this has been dead at runtime for a long time,
  ever since the removal of the materialized page cache in #8105.

## Testing

Unit tests use the sidecar task by default and run both modes in CI.
Python regression tests and benchmarks also use the sidecar task by
default.
We'll test more in staging and possibly preprod.

# Future Work

Please refer to the parent epic for the full plan.

The next step will be to fold the plumbing of IoConcurrency
into RequestContext so that the function signatures get cleaned up.

Once `Sequential` isn't used anymore, we can take the next
big leap which is replacing the opaque IOs with structs
that have well-defined semantics.

---------

Co-authored-by: Christian Schwarz <christian@neon.tech>
2025-01-22 15:30:23 +00:00
Christian Schwarz
cdd34dfc12 impr(utils/spsc_fold): add another test case (#10319)
Wondered about the case covered here while investigating #10309.
2025-01-10 19:19:48 +00:00
Christian Schwarz
db00eb41a1 fix(spsc_fold): potentially missing wakeup when send()ing in state SenderWaitsForReceiverToConsume (#10318)
# Problem

Before this PR, there were cases where send() in state
SenderWaitsForReceiverToConsume would never be woken up
by the receiver, because it never registered with `wake_sender`.

Example Scenario 1: we stop polling a send() future A that was waiting
for the receiver to consume. We drop A and create a new send() future B.
B would return Poll::Pending and never regsister a waker.

Example Scenario 2: a send() future A transitions from HasData
to SenderWaitsForReceiverToConsume. This registers the context X
with `wake_sender`. But before the Receiver consumes the data,
we poll A from a different context Y.
The state is still SenderWaitsForReceiverToConsume, but we wouldn't
register the new context with `wake_sender`.
When the Receiver comes around to consume and `wake_sender.notify()`s,
it wakes the old context X instead of Y.

# Fix

Register the waker in the case where we're polled in
state `SenderWaitsForReceiverToConsume`.

# Relation to #10309

I found this bug while investigating #10309.
There was never proof that this bug here is the root cause for #10309.
In the meantime we found a more probably hypothesis
for the root cause than what is being fixed here.
Regardless, let's walk through my thought process about
how it might have been relevant:

There (in page_service), Scenario 1 does not apply because
we poll the send() future to completion.

Scenario 2 (`tokio::join!`) also does not apply with the
current `tokio::join!()` impl, because it will just poll each
future every time, each with the same context.
Although if we ever used something like a FuturesUnordered anywhere,
that will be using a different context, so, in that case,
the bug might materialize.

Regarding tokio & spurious poll in general:
@conradludgate is not aware of any spurious wakeup cases in current
tokio,
but within a `tokio::join!()`, any wake meant for one future will poll
all
the futures, so that can appear as a spurious wake up to the N-1 futures
of the `tokio::join!()`.
2025-01-10 11:06:03 +00:00
Yuchen Liang
e6cd5050fc pageserver: make BufferedWriter do double-buffering (#9693)
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>
2024-12-04 16:54:56 +00:00
Christian Schwarz
aa4ec11af9 page_service: rewrite batching to work without a timeout (#9851)
# 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
2024-11-30 00:16:24 +00:00
Conrad Ludgate
b8304f90d6 2024 oct new clippy lints (#9448)
Fixes new lints from `cargo +nightly clippy` (`clippy 0.1.83 (798fb83f
2024-10-16)`)
2024-10-18 10:27:50 +01:00
Arpad Müller
cbcd4058ed Fix 1.82 clippy lint too_long_first_doc_paragraph (#8941)
Addresses the 1.82 beta clippy lint `too_long_first_doc_paragraph` by
adding newlines to the first sentence if it is short enough, and making
a short first sentence if there is the need.
2024-09-06 14:33:52 +02:00
Joonas Koivunen
fc78774f39 fix: EphemeralFiles can outlive their Timeline via enum LayerManager (#8229)
Ephemeral files cleanup on drop but did not delay shutdown, leading to
problems with restarting the tenant. The solution is as proposed:
- make ephemeral files carry the gate guard to delay `Timeline::gate`
closing
- flush in-memory layers and strong references to those on
`Timeline::shutdown`

The above are realized by making LayerManager an `enum` with `Open` and
`Closed` variants, and fail requests to modify `LayerMap`.

Additionally:

- fix too eager anyhow conversions in compaction
- unify how we freeze layers and handle errors
- optimize likely_resident_layers to read LayerFileManager hashmap
values instead of bouncing through LayerMap

Fixes: #7830
2024-08-07 17:50:09 +03:00
Joonas Koivunen
a3f5b83677 chore: lower gate guard drop logging threshold to 100ms (#7862)
We have some 1001ms cases, which do not yield gate guard context.
2024-05-24 14:07:58 +01:00
Joonas Koivunen
3df67bf4d7 fix(Layer): metric regression with too many canceled evictions (#7363)
#7030 introduced an annoying papercut, deeming a failure to acquire a
strong reference to `LayerInner` from `DownloadedLayer::drop` as a
canceled eviction. Most of the time, it wasn't that, but just timeline
deletion or tenant detach with the layer not wanting to be deleted or
evicted.

When a Layer is dropped as part of a normal shutdown, the `Layer` is
dropped first, and the `DownloadedLayer` the second. Because of this, we
cannot detect eviction being canceled from the `DownloadedLayer::drop`.
We can detect it from `LayerInner::drop`, which this PR adds.

Test case is added which before had 1 started eviction, 2 canceled. Now
it accurately finds 1 started, 1 canceled.
2024-04-18 15:27:58 +00:00
Joonas Koivunen
a95c41f463 fix(heavier_once_cell): take_and_deinit should take ownership (#7185)
Small fix to remove confusing `mut` bindings.

Builds upon #7175, split off from #7030. Cc: #5331.
2024-03-21 00:42:38 +02:00
Joonas Koivunen
59b6cce418 heavier_once_cell: add detached init support (#7135)
Aiming for the design where `heavier_once_cell::OnceCell` is initialized
by a future factory lead to awkwardness with how
`LayerInner::get_or_maybe_download` looks right now with the `loop`. The
loop helps with two situations:

- an eviction has been scheduled but has not yet happened, and a read
access should cancel the eviction
- a previous `LayerInner::get_or_maybe_download` that canceled a pending
eviction was canceled leaving the `heavier_once_cell::OnceCell`
uninitialized but needing repair by the next
`LayerInner::get_or_maybe_download`

By instead supporting detached initialization in
`heavier_once_cell::OnceCell` via an `OnceCell::get_or_detached_init`,
we can fix what the monolithic #7030 does:
- spawned off download task initializes the
`heavier_once_cell::OnceCell` regardless of the download starter being
canceled
- a canceled `LayerInner::get_or_maybe_download` no longer stops
eviction but can win it if not canceled

Split off from #7030.

Cc: #5331
2024-03-15 15:54:28 +00:00
Arpad Müller
82853cc1d1 Fix warnings and compile errors on nightly (#6886)
Nightly has added a bunch of compiler and linter warnings. There is also
two dependencies that fail compilation on latest nightly due to using
the old `stdsimd` feature name. This PR fixes them.
2024-03-01 17:14:19 +01:00
Joonas Koivunen
aeda82a010 fix(heavier_once_cell): assertion failure can be hit (#6722)
@problame noticed that the `tokio::sync::AcquireError` branch assertion
can be hit like in the added test. We haven't seen this yet in
production, but I'd prefer not to see it there. There `take_and_deinit`
is being used, but this race must be quite timing sensitive.

Rework of earlier: #6652.
2024-02-12 09:57:29 +00:00
Christian Schwarz
5779c7908a revert two recent heavier_once_cell changes (#6704)
This PR reverts

- https://github.com/neondatabase/neon/pull/6589
- https://github.com/neondatabase/neon/pull/6652

because there's a performance regression that's particularly visible at
high layer counts.

Most likely it's because the switch to RwLock inflates the 

```
    inner: heavier_once_cell::OnceCell<ResidentOrWantedEvicted>,
```

size from 48 to 88 bytes, which, by itself is almost a doubling of the
cache footprint, and probably the fact that it's now larger than a cache
line also doesn't help.

See this chat on the Neon discord for more context:

https://discord.com/channels/1176467419317940276/1204714372295958548/1205541184634617906

I'm reverting 6652 as well because it might also have perf implications,
and we're getting close to the next release. We should re-do its changes
after the next release, though.

cc @koivunej 
cc @ivaxer
2024-02-09 22:22:40 +00:00
Joonas Koivunen
9a31311990 fix(heavier_once_cell): assertion failure can be hit (#6652)
@problame noticed that the `tokio::sync::AcquireError` branch assertion
can be hit like in the first commit. We haven't seen this yet in
production, but I'd prefer not to see it there. There `take_and_deinit`
is being used, but this race must be quite timing sensitive.
2024-02-08 22:40:14 +02:00
Christian Schwarz
0de46fd6f2 heavier_once_cell: switch to tokio::sync::RwLock (#6589)
Using the RwLock reduces contention on the hot path.

Co-authored-by: Joonas Koivunen <joonas@neon.tech>
2024-02-06 14:04:15 +02:00
Joonas Koivunen
3d5fab127a rewrite Gate impl for better observability (#6542)
changes:
- two messages instead of message every second when gate was closing
- replace the gate name string by using a pointer
- slow GateGuards are likely to log who they were (see example)

example found in regress tests: <https://github.com/neondatabase/neon/pull/6542#issuecomment-1919009256>
2024-01-31 22:15:58 +00:00
John Spray
3c560d27a8 pageserver: implement secondary-mode downloads (#6123)
Follows on from #6050 , in which we upload heatmaps. Secondary locations
will now poll those heatmaps and download layers mentioned in the
heatmap.

TODO:
- [X] ~Unify/reconcile stats for behind-schedule execution with
warn_when_period_overrun
(https://github.com/neondatabase/neon/pull/6050#discussion_r1426560695)~
- [x] Give downloads their own concurrency config independent of uploads

Deferred optimizations:
- https://github.com/neondatabase/neon/issues/6199
- https://github.com/neondatabase/neon/issues/6200

Eviction will be the next PR:
- #5342
2024-01-05 12:29:20 +00:00
John Spray
20e9cf7d31 pageserver: tweaks to slow/hung task logging (#6098)
## Problem

- `shutdown_tasks` would log when a particular task was taking a long
time to shut down, but not when it eventually completed. That left one
uncertain as to whether the slow task was the source of a hang, or just
a precursor.

## Summary of changes

- Add a log line after a slow task shutdown
- Add an equivalent in Gate's `warn_if_stuck`, in case we ever need it.
This isn't related to the original issue but was noticed when checking
through these logging paths.
2023-12-12 07:19:59 +00:00
John Spray
40441f8ada pageserver: use Gate for stronger safety check in SlotGuard (#5793)
## Problem

#5711 and #5367 raced -- the `SlotGuard` type needs `Gate` to properly
enforce its invariant that we may not drop an `Arc<Tenant>` from a slot.

## Summary of changes

Replace the TODO with the intended check of Gate.
2023-11-08 13:00:11 +00:00
John Spray
6defa2b5d5 pageserver: add Gate as a partner to CancellationToken for safe shutdown of Tenant & Timeline (#5711)
## Problem

When shutting down a Tenant, it isn't just important to cause any
background tasks to stop. It's also important to wait until they have
stopped before declaring shutdown complete, in cases where we may re-use
the tenant's local storage for something else, such as running in
secondary mode, or creating a new tenant with the same ID.

## Summary of changes

A `Gate` class is added, inspired by
[seastar::gate](https://docs.seastar.io/master/classseastar_1_1gate.html).
For types that have an important lifetime that corresponds to some
physical resource, use of a Gate as well as a CancellationToken provides
a robust pattern for async requests & shutdown:
- Requests must always acquire the gate as long as they are using the
object
- Shutdown must set the cancellation token, and then `close()` the gate
to wait for requests in progress before returning.

This is not for memory safety: it's for expressing the difference
between "Arc<Tenant> exists", and "This tenant's files on disk are
eligible to be read/written".

- Both Tenant and Timeline get a Gate & CancellationToken.
- The Timeline gate is held during eviction of layers, and during
page_service requests.
- Existing cancellation support in page_service is refined to use the
timeline-scope cancellation token instead of a process-scope
cancellation token. This replaces the use of `task_mgr::associate_with`:
tasks no longer change their tenant/timelineidentity after being
spawned.

The Tenant's Gate is not yet used, but will be important for
Tenant-scoped operations in secondary mode, where we must ensure that
our secondary-mode downloads for a tenant are gated wrt the activity of
an attached Tenant.

This is part of a broader move away from using the global-state driven
`task_mgr` shutdown tokens:
- less global state where we rely on implicit knowledge of what task a
given function is running in, and more explicit references to the
cancellation token that a particular function/type will respect, making
shutdown easier to reason about.
- eventually avoid the big global TASKS mutex.

---------

Co-authored-by: Joonas Koivunen <joonas@neon.tech>
2023-11-06 12:39:20 +00:00
Joonas Koivunen
27bdbf5e36 chore(layer): restore logging, doc changes (#5766)
Some of the log messages were lost with the #4938. This PR adds some of
them back, most notably:

- starting to on-demand download
- successful completion of on-demand download
- ability to see when there were many waiters for the layer download
- "unexpectedly on-demand downloading ..." is now `info!`

Additionally some rare events are logged as error, which should never
happen.
2023-11-02 19:05:33 +00:00
Joonas Koivunen
2dca4c03fc feat(layer): cancellable get_or_maybe_download (#5744)
With the layer implementation as was done in #4938, it is possible via
cancellation to cause two concurrent downloads on the same path, due to
how `RemoteTimelineClient::download_remote_layer` does tempfiles. Thread
the init semaphore through the spawned task of downloading to make this
impossible to happen.
2023-11-02 08:06:32 +00:00
Joonas Koivunen
c508d3b5fa reimpl Layer, remove remote layer, trait Layer, trait PersistentLayer (#4938)
Implement a new `struct Layer` abstraction which manages downloadness
internally, requiring no LayerMap locking or rewriting to download or
evict providing a property "you have a layer, you can read it". The new
`struct Layer` provides ability to keep the file resident via a RAII
structure for new layers which still need to be uploaded. Previous
solution solved this `RemoteTimelineClient::wait_completion` which lead
to bugs like #5639. Evicting or the final local deletion after garbage
collection is done using Arc'd value `Drop`.

With a single `struct Layer` the closed open ended `trait Layer`, `trait
PersistentLayer` and `struct RemoteLayer` are removed following noting
that compaction could be simplified by simply not using any of the
traits in between: #4839.

The new `struct Layer` is a preliminary to remove
`Timeline::layer_removal_cs` documented in #4745.

Preliminaries: #4936, #4937, #5013, #5014, #5022, #5033, #5044, #5058,
#5059, #5061, #5074, #5103, epic #5172, #5645, #5649. Related split off:
#5057, #5134.
2023-10-26 12:36:38 +03:00