437 Commits

Author SHA1 Message Date
Dmitrii Kovalkov
f68be2b5e2 safekeeper: https for management API (#11171)
## Problem

Storage controller uses unencrypted HTTP requests for safekeeper
management API.

- Closes: https://github.com/neondatabase/cloud/issues/24836

## Summary of changes

- Replace `hyper0::server::Server` with `http_utils::server::Server` in
safekeeper.
- Add HTTPS handler for safekeeper management API.
2025-03-14 11:41:22 +00:00
Arpad Müller
5359cf717c storcon: add API definitions for exclude_timeline and term_bump (#11197)
Adds API definitions for the safekeeper API endpoints `exclude_timeline`
and `term_bump`. Also does a bugfix to return the correct type from
`delete_timeline`.

Part of #8614
2025-03-14 00:00:37 +00:00
Arseny Sher
359c64c779 walproposer: pre generations refactoring (#11060)
## Problem

https://github.com/neondatabase/neon/issues/10851

## Summary of changes

Do some refactoring before making walproposer generations aware.

- Rename SS_VOTING to SS_WAIT_VOTING, SS_IDLE to SS_WAIT_ELECTED
- Continue to get rid of epochs: rename GetEpoch to GetLastLogTerm,
donorEpoch to donorLastLogTerm
- Instead of counting n_votes, n_connected, introduce explicit
WalProposerState (collecting terms / voting / elected). Refactor out
TermsCollected and VotesCollected; they will determine state transition
differently depending whether generations are enabled or not.

There is no new logic in this PR and thus no new tests.
2025-03-11 14:01:00 +00:00
Vlad Lazar
8c553297cb safekeeper: use max end lsn as start of next batch (#11152)
## Problem

Partial reads are still problematic. They are stored in the buffer of
the wal decoder and result in gaps being reported too eagerly on the
pageserver side.

## Summary of changes

Previously, we always used the start LSN of the chunk of WAL that was
just read. This patch switches to using the end LSN of the last record
that was decoded in the previous iteration.
2025-03-10 16:33:28 +00:00
Dmitrii Kovalkov
e876794ce5 storcon: use https safekeeper api (#11065)
## Problem

Storage controller uses http for requests to safekeeper management API.

Closes: https://github.com/neondatabase/cloud/issues/24835

## Summary of changes
- Add `use_https_safekeeper_api` option to storcon to use https api
- Use https for requests to safekeeper management API if this option is
enabled
- Add `ssl_ca_file` option to storcon for ability to specify custom root
CA certificate
2025-03-07 17:22:47 +00:00
Vlad Lazar
937876cbe2 safekeeper: don't skip empty records for shard zero (#11137)
## Problem

Shard zero needs to track the start LSN of the latest record
in adition to the LSN of the next record to ingest. The former
is included in basebackup persisted by the compute in WAL.

Previously, empty records were skipped for all shards. This caused
the prev LSN tracking on the PS to fall behind and led to logical
replication
issues.

## Summary of changes

Shard zero now receives emtpy interpreted records for LSN tracking
purposes.
A test is included too.
2025-03-07 15:52:01 +00:00
Vlad Lazar
6ca49b4d0c safekeeper: fix a gap tracking edge case (#11054)
The interpreted reader tracks a record aligned current position in the
WAL stream.
Partial reads move the stream internally, but not from the pov of the
interpreted WAL reader.

Hence, where new shards subscribe with a start position that matches the
reader's current position,
but we've also done some partial reads. This confuses the gap tracking.
To make it more robust,
update the current batch start to the min between the new start position
and its current value.

Since no record has been decoded yet (position matches), we can't have
lost it
2025-03-03 19:16:03 +00:00
John Spray
b953daa21f safekeeper: allow remote deletion to proceed after dropped requests (#11042)
## Problem

If a caller times out on safekeeper timeline deletion on a large
timeline, and waits a while before retrying, the deletion will not
progress while the retry is waiting. The net effect is very very slow
deletion as it only proceeds in 30 second bursts across 5 minute idle
periods.

Related: https://github.com/neondatabase/neon/issues/10265

## Summary of changes

- Run remote deletion in a background task
- Carry a watch::Receiver on the Timeline for other callers to join the
wait
- Restart deletion if the API is called again and the previous attempt
failed
2025-03-03 16:03:51 +00:00
Erik Grinaker
c7ff3c4c9b safekeeper: downgrade interpreted reader errors (#11034)
## Problem

This `critical!` could fire on IO errors, which is just noisy.

Resolves #11027.

## Summary of changes

Downgrade to error, except for decode errors. These could be either data
corruption or a bug, but seem worth investigating either way.
2025-02-28 14:06:56 +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
Arseny Sher
8b86cd1154 safekeeper: follow membership configuration rules (#10781)
## Problem

safekeepers must ignore walproposer messages with non matching
membership conf.

## Summary of changes

Make safekeepers reject vote request, proposer elected and append
request messages with non matching generation. Switch to the
configuration in the greeting message if it is higher.

In passing, fix one comment and WAL truncation.

Last part of https://github.com/neondatabase/neon/issues/9965
2025-02-27 06:13:30 +00:00
Arseny Sher
643a48210f safekeeper: exclude API (#10757)
## Problem

https://github.com/neondatabase/neon/pull/10241 added configuration
switch endpoint, but it didn't delete timeline if node was excluded.

## Summary of changes

Add separate /exclude API endpoint which similarly accepts membership
configuration where sk is supposed by be excluded. Implementation
deletes the timeline locally.

Some more small related tweaks:
- make mconf switch API PUT instead of POST as it is idempotent;
- return 409 if switch was refused instead of 200 with requested &
current;
- remove unused was_active flag from delete response;
- remove meaningless _force suffix from delete functions names;
- reuse timeline.rs delete_dir function in timelines_global_map instead
of its own copy.

part of https://github.com/neondatabase/neon/issues/9965
2025-02-26 19:26:33 +00:00
Arseny Sher
01581f3af5 safekeeper: drop json_ctrl (#10722)
## Problem

json_ctrl.rs is an obsolete attempt to have tests with fine control of
feeding messages into safekeeper superseded by desim framework.

## Summary of changes

Drop it.
2025-02-26 13:32:37 +00:00
Vlad Lazar
622a9def6f tests: use generated record lsn instead of hardcoded one (#10990)
... and start the initial reader with the correct lsn

Closes https://github.com/neondatabase/neon/issues/10978
2025-02-26 12:47:13 +00:00
Arpad Müller
920040e402 Update storage components to edition 2024 (#10919)
Updates storage components 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.

The PR has two commits:

* the first commit updates storage crates to edition 2024 and appeases
`cargo clippy` by changing code. i have accidentially ran the formatter
on some files that had other edits.
* the second commit performs a `cargo fmt`

I would recommend a closer review of the first commit and a less close
review of the second one (as it just runs `cargo fmt`).

part of https://github.com/neondatabase/neon/issues/10918
2025-02-25 23:51:37 +00:00
Arseny Sher
758f597280 compute <-> sk protocol v3 (#10647)
## Problem

As part of https://github.com/neondatabase/neon/issues/8614 we need to
pass membership configurations between compute and safekeepers.

## Summary of changes

Add version 3 of the protocol carrying membership configurations.
Greeting message in both sides gets full conf, and other messages
generation number only. Use protocol bump to include other accumulated
changes:
- stop packing whole structs on the wire as is;
- make the tag u8 instead of u64;
- send all ints in network order;
- drop proposer_uuid, we can pass it in START_WAL_PUSH and it wasn't
much useful anyway.
Per message changes, apart from mconf:
- ProposerGreeting: tenant / timeline id is sent now as hex cstring.
Remove proto version, it is passed outside in START_WAL_PUSH. Remove
postgres timeline, it is unused. Reorder fields a bit.
- AcceptorGreeting: reorder fields
- VoteResponse: timeline_start_lsn is removed. It can be taken from
first member of term history, and later we won't need it at all when all
timelines will be explicitly created. Vote itself is u8 instead of u64.
- ProposerElected: timeline_start_lsn is removed for the same reasons.
- AppendRequest: epoch_start_lsn removed, it is known from term history
in ProposerElected.

Both compute and sk are able to talk v2 and v3 to make rollbacks (in
case we need them) easier; neon.safekeeper_proto_version GUC sets the
client version. v2 code can be dropped later.

So far empty conf is passed everywhere, future PRs will handle them.

To test, add param to some tests choosing proto version; we want to test
both 2 and 3 until we fully migrate.

ref https://github.com/neondatabase/neon/issues/10326

---------

Co-authored-by: Arthur Petukhovsky <petuhovskiy@yandex.ru>
2025-02-25 11:56:05 +00:00
Vlad Lazar
0d9a45a475 safekeeper: invalidate start of interpreted batch on reader resets (#10951)
## Problem

The interpreted WAL reader tracks the start of the current logical
batch.
This needs to be invalidated when the reader is reset.

This bug caused a couple of WAL gap alerts in staging.

## Summary of changes

* Refactor to make it possible to write a reproducer
* Add repro unit test
* Fix by resetting the start with the reader

Related https://github.com/neondatabase/cloud/issues/23935
2025-02-25 10:21:35 +00:00
Heikki Linnakangas
a6f315c9c9 Remove unnecessary dependencies to synchronous 'postgres' crate (#10938)
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.
2025-02-24 09:40:25 +00:00
Vlad Lazar
34996416d6 pageserver: guard against WAL gaps in the interpreted protocol (#10858)
## 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.


![image](https://github.com/user-attachments/assets/8aed292e-5d62-46a3-9b01-fbf9dc25efe0)

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
2025-02-20 17:49:05 +00:00
Arpad Müller
9ba2a87e69 storcon: sk heartbeat fixes (#10891)
This PR does the following things:

* The initial heartbeat round blocks the storage controller from
becoming online again. If all safekeepers are unresponsive, this can
cause storage controller startup to be very slow. The original intent of
#10583 was that heartbeats don't affect normal functionality of the
storage controller. So add a short timeout to prevent it from impeding
storcon functionality.

* Fix the URL of the utilization endpoint.

* Don't send heartbeats to safekeepers which are decomissioned.

Part of https://github.com/neondatabase/neon/issues/9011

context: https://neondb.slack.com/archives/C033RQ5SPDH/p1739966807592589
2025-02-19 16:57:11 +00:00
Arpad Müller
caece02da7 move pull_timeline to safekeeper_api and add SafekeeperGeneration (#10863)
Preparations for a successor of #10440: 

* move `pull_timeline` to `safekeeper_api` and add it to
`SafekeeperClient`. we want to do `pull_timeline` on any creations that
we couldn't do initially.
* Add a `SafekeeperGeneration` type instead of relying on a type alias.
we want to maintain a safekeeper specific generation number now in the
storcon database. A separate type is important to make it impossible to
mix it up with the tenant's pageserver specific generation number. We
absolutely want to avoid that for correctness reasons. If someone mixes
up a safekeeper and pageserver id (both use the `NodeId` type), that's
bad but there is no wrong generations flying around.

part of #9011
2025-02-18 14:02:22 +00:00
Christian Schwarz
b992a1a62a page_service: include socket send & recv queue length in slow flush log mesage (#10823)
# 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.
2025-02-14 16:20:07 +00:00
Arpad Müller
5008324460 Fix utilization URL and ensure heartbeats work (#10811)
There was a typo in the name of the utilization endpoint URL, fix it.
Also, ensure that the heartbeat mechanism actually works.

Related: #10583, #10429

Part of #9011
2025-02-13 20:55:53 +00:00
Arseny Sher
98e18e9a54 Add s3 storage to test_s3_wal_replay (#10809)
## Problem

The test is flaky: WAL in remote storage appears to be corrupted. One of
hypotheses so far is that corruption is the result of local fs
implementation being non atomic, and safekeepers may concurrently PUT
the same segment. That's dubious though because by looking at local_fs
impl I'd expect then early EOF on segment read rather then observed
zeros in test failures, but other directions seem even less probable.

## Summary of changes

Let's add s3 backend as well and see if it is also flaky. Also add some
more logging around segments uploads.

ref https://github.com/neondatabase/neon/issues/10761
2025-02-13 18:05:15 +00:00
Arpad Müller
536bdb3209 storcon: track safekeepers in memory, send heartbeats to them (#10583)
In #9011, we want to schedule timelines to safekeepers. In order to do
such scheduling, we need information about how utilized a safekeeper is
and if it's available or not.

Therefore, send constant heartbeats to the safekeepers and try to figure
out if they are online or not.

Includes some code from #10440.
2025-02-13 11:06:30 +00:00
Heikki Linnakangas
635b67508b Split utils::http to separate crate (#10753)
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.
2025-02-11 22:06:53 +00:00
Arpad Müller
f5243992fa safekeeper: make timeline deletions a bit more verbose (#10721)
Make timeline deletion print the sub-steps, so that we can narrow down
some stuck timeline deletion issues we are observing.

https://neondb.slack.com/archives/C08C2G15M6U/p1738930694716009
2025-02-07 15:06:26 +00:00
Erik Grinaker
f4cfa725b8 pageserver: add a few critical errors (#10657)
## Problem

Following #10641, let's add a few critical errors.

Resolves #10094.

## Summary of changes

Adds the following critical errors:

* WAL sender read/decode failure.
* WAL record ingestion failure.
* WAL redo failure.
* Missing key during compaction.

We don't add an error for missing keys during GetPage requests, since
we've seen a handful of these in production recently, and the cause is
still unclear (most likely a benign race).
2025-02-06 10:30:27 +00:00
Vlad Lazar
95588dab98 safekeeper: fix wal fan-out shard subscription data race (#10677)
## Problem

[This select
arm](https://github.com/neondatabase/neon/blob/main/safekeeper/src/send_interpreted_wal.rs#L414)
runs when we want to attach a new reader to the current cursor.
It checks the current position of the cursor and resets it if required.

The current position of the cursor is updated in the [other select
arm](https://github.com/neondatabase/neon/blob/main/safekeeper/src/send_interpreted_wal.rs#L336-L345).
That runs when we get some WAL to send.

Now, what happens if we want to attach two shards consecutively to the
cursor?
Let's say [this select
arm](https://github.com/neondatabase/neon/blob/main/safekeeper/src/send_interpreted_wal.rs#L397)
runs twice in a row.

Let's assume cursor is currently at LSN X. First shard wants to attach
at position V
and the other one at W. Assume X > W > V.

First shard resets the stream to position V. Second shard comes in, 
sees stale cursor position X and resets it to W. This means that the 
first shard doesn't get wal in the [V, W) range.

## Summary of changes

Ultimately, this boils down to the current position not being kept in
sync with the reset of the WAL stream. This patch fixes the race by
updating it when resetting the WAL stream and adds a unit test repro.

Closes https://github.com/neondatabase/cloud/issues/23750
2025-02-06 09:24:28 +00:00
Arpad Müller
8ae6f656a6 Don't require partial backup semaphore capacity for deletions (#10628)
In the safekeeper, we block deletions on the timeline's gate closing,
and any `WalResidentTimeline` keeps the gate open (because it owns a
gate lock object). Thus, unless the `main_task` function of a partial
backup doesn't return, we can't delete the associated timeline.

In order to make these tasks exit early, we call the cancellation token
of the timeline upon its shutdown. However, the partial backup task
wasn't looking for the cancellation while waiting to acquire a partial
backup permit.

On a staging safekeeper we have been in a situation in the past where
the semaphore was already empty for a duration of many hours, rendering
all attempted deletions unable to proceed until a restart where the
semaphore was reset:
https://neondb.slack.com/archives/C03H1K0PGKH/p1738416586442029
2025-02-03 04:11:06 +00:00
Arpad Müller
c60b91369a Expose safekeeper APIs for creation and deletion (#10478)
Add APIs for timeline creation and deletion to the safekeeper client
crate. Going to be used later in #10440.

Split off from #10440.

Part of https://github.com/neondatabase/neon/issues/9011
2025-01-22 18:52:16 +00:00
Arpad Müller
b0f34099f9 Add safekeeper utilization endpoint (#10429)
Add an endpoint to obtain the utilization of a safekeeper. Future
changes to the storage controller can use this endpoint to find the most
suitable safekeepers for newly created timelines, analogously to how
it's done for pageservers already.

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

Part of https://github.com/neondatabase/neon/issues/9011
2025-01-17 21:43:52 +00:00
Arseny Sher
6fe4c6798f Add START_WAL_PUSH proto_version and allow_timeline_creation options. (#10406)
## Problem

As part of https://github.com/neondatabase/neon/issues/8614 we need to
pass options to START_WAL_PUSH.

## Summary of changes

Add two options. `allow_timeline_creation`, default true, disables
implicit timeline creation in the connection from compute. Eventually
such creation will be forbidden completely, but as we migrate to
configurations we need to support both: current mode and configurations
enabled where creation by compute is disabled.

`proto_version` specifies compute <-> sk protocol version. We have it
currently in the first greeting package also, but I plan to change tag
size from u64 to u8, which would make it hard to use. Command is more
appropriate place for it anyway.
2025-01-16 08:01:19 +00:00
Vlad Lazar
dbebede7bf safekeeper: fan out from single wal reader to multiple shards (#10190)
## Problem

Safekeepers currently decode and interpret WAL for each shard
separately.
This is wasteful in terms of CPU memory usage - we've seen this in
profiles.

## Summary of changes

Fan-out interpreted WAL to multiple shards. 
The basic is that wal decoding and interpretation happens in a separate
tokio task and senders
attach to it. Senders only receive batches concerning their shard and
only past the Lsn they've last seen.

Fan-out is gated behind the `wal_reader_fanout` safekeeper flag
(disabled by default for now).

When fan-out is enabled, it might be desirable to control the absolute
delta between the
current position and a new shard's desired position (i.e. how far behind
or ahead a shard may be).
`max_delta_for_fanout` is a new optional safekeeper flag which dictates
whether to create a new
WAL reader or attach to the existing one. By default, this behaviour is
disabled. Let's consider enabling
it if we spot the need for it in the field.

## Testing

Tests passed [here](https://github.com/neondatabase/neon/pull/10301)
with wal reader fanout enabled
as of
34f6a71718.

Related: https://github.com/neondatabase/neon/issues/9337
Epic: https://github.com/neondatabase/neon/issues/9329
2025-01-15 15:33:54 +00:00
Arseny Sher
05a71c7d6a safekeeper: add membership configuration switch endpoint (#10241)
## Problem

https://github.com/neondatabase/neon/issues/9965

## Summary of changes

Add to safekeeper http endpoint to switch membership configuration. Also
add it to python client for tests, and add simple test itself.
2025-01-15 14:16:04 +00:00
Vlad Lazar
1577430408 safekeeper: decode and interpret for multiple shards in one go (#10201)
## Problem

Currently, we call `InterpretedWalRecord::from_bytes_filtered`
from each shard. To serve multiple shards at the same time,
the API needs to allow for enquiring about multiple shards.

## Summary of changes

This commit tweaks it a pretty brute force way. Naively, we could
just generate the shard for a key, but pre and post split shards
may be subscribed at the same time, so doing it efficiently is more
complex.
2025-01-15 11:10:24 +00:00
Arseny Sher
2d0ea08524 Add safekeeper membership conf to control file. (#10196)
## Problem

https://github.com/neondatabase/neon/issues/9965

## Summary of changes

Add safekeeper membership configuration struct itself and storing it in
the control file. In passing also add creation timestamp to the control
file (there were cases where I wanted it in the past).

Remove obsolete unused PersistedPeerInfo struct from control file (still
keep it control_file_upgrade.rs to have it in old upgrade code).

Remove the binary representation of cfile in the roundtrip test.
Updating it is annoying, and we still test the actual roundtrip.

Also add configuration to timeline creation http request, currently used
only in one python test. In passing, slightly change LSNs meaning in the
request: normally start_lsn is passed (the same as ancestor_start_lsn in
similar pageserver call), but we allow specifying higher commit_lsn for
manual intervention if needed. Also when given LSN initialize
term_history with it.
2025-01-15 09:45:58 +00:00
Erik Grinaker
cd982a82ec pageserver,safekeeper: increase heap profiling frequency to 2 MB (#10362)
## Problem

Currently, the heap profiling frequency is every 1 MB allocated. Taking
a profile stack trace takes about 1 µs, and allocating 1 MB takes about
15 µs, so the overhead is about 6.7% which is a bit high. This is a
fixed cost regardless of whether heap profiles are actually accessed.

## Summary of changes

Increase the heap profiling sample frequency from 1 MB to 2 MB, which
reduces the overhead to about 3.3%. This seems acceptable, considering
performance-sensitive code will avoid allocations as far as possible
anyway.
2025-01-13 09:44:59 +00:00
Arseny Sher
030ab1c0e8 TLA+ spec for safekeeper membership change (#9966)
## Problem

We want to define the algorithm for safekeeper membership change.

## Summary of changes

Add spec for it, several models and logs of checking them.

ref https://github.com/neondatabase/neon/issues/8699
2025-01-09 12:26:17 +00:00
Erik Grinaker
237dae71a1 Revert "pageserver,safekeeper: disable heap profiling (#10268)" (#10303)
This reverts commit b33299dc37.

Heap profiles weren't the culprit after all.

Touches #10225.
2025-01-07 22:49:00 +00:00
Erik Grinaker
b33299dc37 pageserver,safekeeper: disable heap profiling (#10268)
## 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.
2025-01-03 15:21:31 +00:00
Vlad Lazar
628451d68e safekeeper: short-circuit interpreted wal sender (#10202)
## Problem

Safekeeper may currently send a batch to the pageserver even if it
hasn't decoded a new record.
I think this is quite unlikely in the field, but worth adressing.

## Summary of changes

Don't send anything if we haven't decoded a full record. Once this
merges and releases, the `InterpretedWalRecords` struct can be updated
to remove the Option wrapper for `next_record_lsn`.
2024-12-19 14:04:46 +00:00
Vlad Lazar
502d512fe2 safekeeper: lift benchmarking utils into safekeeper crate (#10200)
## Problem

The benchmarking utilities are also useful for testing. We want to write
tests in the safekeeper crate.

## Summary of changes

This commit lifts the utils to the safekeeper crate. They are compiled
if the benchmarking features is enabled or if in test mode.
2024-12-19 14:04:42 +00:00
Erik Grinaker
1668d39b7c safekeeper: fix typo in allowlist for /profile/heap (#10186) 2024-12-18 15:51:53 +00:00
Conrad Ludgate
6565fd4056 chore: fix clippy lints 2024-12-06 (#10138) 2024-12-16 15:33:21 +00:00
Arseny Sher
1ed0e52bc8 Extract safekeeper http client to separate crate. (#10140)
## Problem

We want to use safekeeper http client in storage controller and
neon_local.

## Summary of changes

Extract it to separate crate. No functional changes.
2024-12-16 12:07:24 +00:00
Arseny Sher
ce8eb089f3 Extract public sk types to safekeeper_api (#10137)
## Problem

We want to extract safekeeper http client to separate crate for use in
storage controller and neon_local. However, many types used in the API
are internal to safekeeper.

## Summary of changes

Move them to safekeeper_api crate. No functional changes.

ref https://github.com/neondatabase/neon/issues/9011
2024-12-13 14:06:27 +00:00
Evan Fleming
b593e51eae safekeeper: use arc for global timelines and config (#10051)
Hello! I was interested in potentially making some contributions to Neon
and looking through the issue backlog I found
[8200](https://github.com/neondatabase/neon/issues/8200) which seemed
like a good first issue to attempt to tackle. I see it was assigned a
while ago so apologies if I'm stepping on any toes with this PR. I also
apologize for the size of this PR. I'm not sure if there is a simple way
to reduce it given the footprint of the components being changed.

## Problem
This PR is attempting to address part of the problem outlined in issue
[8200](https://github.com/neondatabase/neon/issues/8200). Namely to
remove global static usage of timeline state in favour of
`Arc<GlobalTimelines>` and to replace wasteful clones of
`SafeKeeperConf` with `Arc<SafeKeeperConf>`. I did not opt to tackle
`RemoteStorage` in this PR to minimize the amount of changes as this PR
is already quite large. I also did not opt to introduce an
`SafekeeperApp` wrapper struct to similarly minimize changes but I can
tackle either or both of these omissions in this PR if folks would like.

## Summary of changes
- Remove static usage of `GlobalTimelines` in favour of
`Arc<GlobalTimelines>`
- Wrap `SafeKeeperConf` in `Arc` to avoid wasteful clones of the
underlying struct

## Some additional thoughts
- We seem to currently store `SafeKeeperConf` in `GlobalTimelines` and
then expose it through a public`get_global_config` function which
requires locking. This seems needlessly wasteful and based on observed
usage we could remove this public accessor and force consumers to
acquire `SafeKeeperConf` through the new Arc reference.
2024-12-09 21:09:20 +00:00
Erik Grinaker
699a213c5d Display reqwest error source (#10004)
## 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.
2024-12-04 13:05:53 +00:00
Erik Grinaker
dcb24ce170 safekeeper,pageserver: add heap profiling (#9778)
## 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.
2024-12-03 11:35:59 +00:00