Compare commits

..

89 Commits

Author SHA1 Message Date
John Spray
5664eadb17 dirty 2024-08-05 18:39:26 +00:00
John Spray
1a99aa4834 utils: use SmallVec in VecMap 2024-08-05 18:22:41 +00:00
John Spray
35df89dcfe update ingest_bench 2024-08-05 17:57:41 +00:00
John Spray
28280683ab enable bulk_ingest 2024-08-05 17:49:02 +00:00
John Spray
eed100b21e wip 2024-08-05 17:49:02 +00:00
John Spray
4bd26c54e5 wip 2024-08-05 17:49:02 +00:00
John Spray
e461a711d0 Soft limit on the size of monolithic serialization stage 2024-08-05 17:49:02 +00:00
John Spray
93dacd75aa Refactor InMemoryLayer put_batch code 2024-08-05 17:49:02 +00:00
John Spray
06428e856e Remove unused singular puts 2024-08-05 17:49:01 +00:00
John Spray
fce68fe84e pageserver: avoid a no-longer-needed sort during ingest 2024-08-05 17:47:58 +00:00
John Spray
fd60904376 pageserver: batch ephemeral layer writes during ingest 2024-08-05 17:47:58 +00:00
John Spray
bf3e767b35 update split_writer for merge 2024-08-05 17:43:08 +00:00
John Spray
513cafd72b Merge remote-tracking branch 'upstream/main' into jcsp/ingest-bench 2024-08-05 17:38:10 +00:00
John Spray
c2d5395a00 clean up temp dir 2024-08-05 12:36:15 +00:00
John Spray
d152a57c29 s/field3/field6/ 2024-08-05 12:23:15 +00:00
John Spray
a8be0f3376 add a doc comment 2024-08-05 12:18:06 +00:00
Alexander Bayandin
bd845c7587 CI(trigger-e2e-tests): wait for promote-images job from the last commit (#8592)
## Problem

We don't trigger e2e tests for draft PRs, but we do trigger them once a
PR is in the "Ready for review" state.
Sometimes, a PR can be marked as "Ready for review" before we finish
image building. In such cases, triggering e2e tests fails.

## Summary of changes
- Make `trigger-e2e-tests` job poll status of `promote-images` job from
the build-and-test workflow for the last commit. And trigger only if the
status is `success`
- Remove explicit image checking from the workflow
- Add `concurrency` for `triggere-e2e-tests` workflow to make it
possible to cancel jobs in progress (if PR moves from "Draft" to "Ready
for review" several times in a row)
2024-08-05 12:25:23 +01:00
Konstantin Knizhnik
f63c8e5a8c Update Postgres versions to use smgrexists() instead of access() to check if Oid is used (#8597)
## Problem

PR #7992 was merged without correspondent changes in Postgres submodules
and this is why test_oid_overflow.py is failed now.

## Summary of changes

Bump Postgres versions

## Checklist before requesting a review

- [ ] I have performed a self-review of my code.
- [ ] If it is a core feature, I have added thorough tests.
- [ ] Do we need to implement analytics? if so did you add the relevant
metrics to the dashboard?
- [ ] If this PR requires public announcement, mark it with
/release-notes label and add several sentences in this section.

## Checklist before merging

- [ ] Do not forget to reformat commit message to not include the above
checklist

Co-authored-by: Konstantin Knizhnik <knizhnik@neon.tech>
2024-08-05 14:24:54 +03:00
Alex Chi Z.
200fa56b04 feat(pageserver): support split delta layers (#8599)
part of https://github.com/neondatabase/neon/issues/8002

Similar to https://github.com/neondatabase/neon/pull/8574, we add
auto-split support for delta layers. Tests are reused from image layer
split writers.


---------

Signed-off-by: Alex Chi Z <chi@neon.tech>
2024-08-05 10:30:49 +00:00
dotdister
0f3dac265b safekeeper: remove unused partial_backup_enabled option (#8547)
## Problem
There is an unused safekeeper option `partial_backup_enabled`.

`partial_backup_enabled` was implemented in #6530, but this option was
always turned into enabled in #8022.

If you intended to keep this option for a specific reason, I will close
this PR.

## Summary of changes
I removed an unused safekeeper option `partial_backup_enabled`.
2024-08-05 09:23:59 +02:00
Alex Chi Z.
1dc496a2c9 feat(pageserver): support auto split layers based on size (#8574)
part of https://github.com/neondatabase/neon/issues/8002

## Summary of changes

Add a `SplitImageWriter` that automatically splits image layer based on
estimated target image layer size. This does not consider compression
and we might need a better metrics.

---------

Signed-off-by: Alex Chi Z <chi@neon.tech>
Co-authored-by: Arpad Müller <arpad-m@users.noreply.github.com>
2024-08-05 06:55:36 +01:00
Alex Chi Z.
6814bdd30b fix(pageserver): deadlock in gc-compaction (#8590)
We need both compaction and gc lock for gc-compaction. The lock order
should be the same everywhere, otherwise there could be a deadlock where
A waits for B and B waits for A.

We also had a double-lock issue. The compaction lock gets acquired in
the outer `compact` function. Note that the unit tests directly call
`compact_with_gc`, and therefore not triggering the issue.

## Summary of changes

Ensure all places acquire compact lock and then gc lock. Remove an extra
compact lock acqusition.

---------

Signed-off-by: Alex Chi Z <chi@neon.tech>
2024-08-03 00:52:04 +01:00
John Spray
0a667bc8ef tests: add test_historic_storage_formats (#8423)
## Problem

Currently, our backward compatibility tests only look one release back.
That means, for example, that when we switch on image layer compression
by default, we'll test reading of uncompressed layers for one release,
and then stop doing it. When we make an index_part.json format change,
we'll test against the old format for a week, then stop (unless we write
separate unit tests for each old format).

The reality in the field is that data in old formats will continue to
exist for weeks/months/years. When we make major format changes, we
should retain examples of the old format data, and continuously verify
that the latest code can still read them.

This test uses contents from a new path in the public S3 bucket,
`compatibility-data-snapshots/`. It is populated by hand. The first
important artifact is one from before we switch on compression, so that
we will keep testing reads of uncompressed data. We will generate more
artifacts ahead of other key changes, like when we update remote storage
format for archival timelines.

Closes: https://github.com/neondatabase/cloud/issues/15576
2024-08-02 18:28:23 +01:00
Arthur Petukhovsky
f3acfb2d80 Improve safekeepers eviction rate limiting (#8456)
This commit tries to fix regular load spikes on staging, caused by too
many eviction and partial upload operations running at the same time.
Usually it was hapenning after restart, for partial backup the load was
delayed.
- Add a semaphore for evictions (2 permits by default)
- Rename `resident_since` to `evict_not_before` and smooth out the curve
by using random duration
- Use random duration in partial uploads as well

related to https://github.com/neondatabase/neon/issues/6338
some discussion in
https://neondb.slack.com/archives/C033RQ5SPDH/p1720601531744029
2024-08-02 15:26:46 +01:00
Arpad Müller
8c828c586e Wait for completion of the upload queue in flush_frozen_layer (#8550)
Makes `flush_frozen_layer` add a barrier to the upload queue and makes
it wait for that barrier to be reached until it lets the flushing be
completed.

This gives us backpressure and ensures that writes can't build up in an
unbounded fashion.

Fixes #7317
2024-08-02 13:07:12 +02:00
John Spray
2334fed762 storage_controller: start adding chaos hooks (#7946)
Chaos injection bridges the gap between automated testing (where we do
lots of different things with small, short-lived tenants), and staging
(where we do many fewer things, but with larger, long-lived tenants).

This PR adds a first type of chaos which isn't really very chaotic: it's
live migration of tenants between healthy pageservers. This nevertheless
provides continuous checks that things like clean, prompt shutdown of
tenants works for realistically deployed pageservers with realistically
large tenants.
2024-08-02 09:37:44 +01:00
John Spray
c53799044d pageserver: refine how we delete timelines after shard split (#8436)
## Problem

Previously, when we do a timeline deletion, shards will delete layers
that belong to an ancestor. That is not a correctness issue, because
when we delete a timeline, we're always deleting it from all shards, and
destroying data for that timeline is clearly fine.

However, there exists a race where one shard might start doing this
deletion while another shard has not yet received the deletion request,
and might try to access an ancestral layer. This creates ambiguity over
the "all layers referenced by my index should always exist" invariant,
which is important to detecting and reporting corruption.

Now that we have a GC mode for clearing up ancestral layers, we can rely
on that to clean up such layers, and avoid deleting them right away.
This makes things easier to reason about: there are now no cases where a
shard will delete a layer that belongs to a ShardIndex other than
itself.

## Summary of changes

- Modify behavior of RemoteTimelineClient::delete_all
- Add `test_scrubber_physical_gc_timeline_deletion` to exercise this
case
- Tweak AWS SDK config in the scrubber to enable retries. Motivated by
seeing the test for this feature encounter some transient "service
error" S3 errors (which are probably nothing to do with the changes in
this PR)
2024-08-02 08:00:46 +01:00
Alexander Bayandin
e7477855b7 test_runner: don't create artifacts if Allure is not enabled (#8580)
## Problem

`allure_attach_from_dir` method might create `tar.zst` archives even
if `--alluredir` is not set (i.e. Allure results collection is disabled)

## Summary of changes
- Don't run `allure_attach_from_dir` if `--alluredir`  is not set
2024-08-01 15:55:43 +00:00
John Spray
5dcfe1c4b8 pageserver: downgrade an assertion to debug 2024-08-01 15:43:21 +00:00
John Spray
ae7d635098 pageserver: add ingest bench 2024-08-01 15:43:21 +00:00
John Spray
137cbb4db4 pageserver: refactor DeltaLayerWriter to not need a Timeline 2024-08-01 15:43:21 +00:00
John Spray
74eda0b0b7 pageserver: make bench'able methods public 2024-08-01 15:00:01 +00:00
Alex Chi Z.
f4a668a27d fix(pageserver): skip existing layers for btm-gc-compaction (#8498)
part of https://github.com/neondatabase/neon/issues/8002

Due to the limitation of the current layer map implementation, we cannot
directly replace a layer. It's interpreted as an insert and a deletion,
and there will be file exist error when renaming the newly-created layer
to replace the old layer. We work around that by changing the end key of
the image layer. A long-term fix would involve a refactor around the
layer file naming. For delta layers, we simply skip layers with the same
key range produced, though it is possible to add an extra key as an
alternative solution.

* The image layer range for the layers generated from gc-compaction will
be Key::MIN..(Key..MAX-1), to avoid being recognized as an L0 delta
layer.
* Skip existing layers if it turns out that we need to generate a layer
with the same persistent key in the same generation.

Note that it is possible that the newly-generated layer has different
content from the existing layer. For example, when the user drops a
retain_lsn, the compaction could have combined or dropped some records,
therefore creating a smaller layer than the existing one. We discard the
"optimized" layer for now because we cannot deal with such rewrites
within the same generation.


---------

Signed-off-by: Alex Chi Z <chi@neon.tech>
Co-authored-by: Christian Schwarz <christian@neon.tech>
2024-08-01 15:00:06 +01:00
Alex Chi Z.
970f2923b2 storage-scrubber: log version on start (#8571)
Helps us better identify which version of storage scrubber is running.

---------

Signed-off-by: Alex Chi Z <chi@neon.tech>
2024-08-01 13:52:34 +00:00
John Spray
1678dea20f pageserver: add layer visibility calculation (#8511)
## Problem

We recently added a "visibility" state to layers, but nothing
initializes it.

Part of:
- #8398 

## Summary of changes

- Add a dependency on `range-set-blaze`, which is used as a fast
incrementally updated alternative to KeySpace. We could also use this to
replace the internals of KeySpaceRandomAccum if we wanted to. Writing a
type that does this kind of "BtreeMap & merge overlapping entries" thing
isn't super complicated, but no reason to write this ourselves when
there's a third party impl available.
- Add a function to layermap to calculate visibilities for each layer
- Add a function to Timeline to call into layermap and then apply these
visibilities to the Layer objects.
- Invoke the calculation during startup, after image layer creations,
and when removing branches. Branch removal and image layer creation are
the two ways that a layer can go from Visible to Covered.
- Add unit test & benchmark for the visibility calculation
- Expose `pageserver_visible_physical_size` metric, which should always
be <= `pageserver_remote_physical_size`.
- This metric will feed into the /v1/utilization endpoint later: the
visible size indicates how much space we would like to use on this
pageserver for this tenant.
- When `pageserver_visible_physical_size` is greater than
`pageserver_resident_physical_size`, this is a sign that the tenant has
long-idle branches, which result in layers that are visible in
principle, but not used in practice.

This does not keep visibility hints up to date in all cases:
particularly, when creating a child timeline, any previously covered
layers will not get marked Visible until they are accessed.

Updates after image layer creation could be implemented as more of a
special case, but this would require more new code: the existing depth
calculation code doesn't maintain+yield the list of deltas that would be
covered by an image layer.

## Performance

This operation is done rarely (at startup and at timeline deletion), so
needs to be efficient but not ultra-fast.

There is a new `visibility` bench that measures runtime for a synthetic
100k layers case (`sequential`) and a real layer map (`real_map`) with
~26k layers.

The benchmark shows runtimes of single digit milliseconds (on a ryzen
7950). This confirms that the runtime shouldn't be a problem at startup
(as we already incur S3-level latencies there), but that it's slow
enough that we definitely shouldn't call it more often than necessary,
and it may be worthwhile to optimize further later (things like: when
removing a branch, only bother scanning layers below the branchpoint)

```
visibility/sequential   time:   [4.5087 ms 4.5894 ms 4.6775 ms]
                        change: [+2.0826% +3.9097% +5.8995%] (p = 0.00 < 0.05)
                        Performance has regressed.
Found 24 outliers among 100 measurements (24.00%)
  2 (2.00%) high mild
  22 (22.00%) high severe
min: 0/1696070, max: 93/1C0887F0
visibility/real_map     time:   [7.0796 ms 7.0832 ms 7.0871 ms]
                        change: [+0.3900% +0.4505% +0.5164%] (p = 0.00 < 0.05)
                        Change within noise threshold.
Found 4 outliers among 100 measurements (4.00%)
  3 (3.00%) high mild
  1 (1.00%) high severe
min: 0/1696070, max: 93/1C0887F0
visibility/real_map_many_branches
                        time:   [4.5285 ms 4.5355 ms 4.5434 ms]
                        change: [-1.0012% -0.8004% -0.5969%] (p = 0.00 < 0.05)
                        Change within noise threshold.
```
2024-08-01 09:25:35 +00:00
Arpad Müller
163f2eaf79 Reduce linux-raw-sys duplication (#8577)
Before, we had four versions of linux-raw-sys in our dependency graph:

```
  linux-raw-sys@0.1.4
  linux-raw-sys@0.3.8
  linux-raw-sys@0.4.13
  linux-raw-sys@0.6.4
```

now it's only two:

```
  linux-raw-sys@0.4.13
  linux-raw-sys@0.6.4
```

The changes in this PR are minimal. In order to get to its state one
only has to update procfs in Cargo.toml to 0.16 and do `cargo update -p
tempfile -p is-terminal -p prometheus`.
2024-08-01 08:22:21 +00:00
Christian Schwarz
980d506bda pageserver: shutdown all walredo managers 8s into shutdown (#8572)
# Motivation

The working theory for hung systemd during PS deploy
(https://github.com/neondatabase/cloud/issues/11387) is that leftover
walredo processes trigger a race condition.

In https://github.com/neondatabase/neon/pull/8150 I arranged that a
clean Tenant shutdown does actually kill its walredo processes.

But many prod machines don't manage to shut down all their tenants until
the 10s systemd timeout hits and, presumably, triggers the race
condition in systemd / the Linux kernel that causes the frozen systemd

# Solution

This PR bolts on a rather ugly mechanism to shut down tenant managers
out of order 8s after we've received the SIGTERM from systemd.

# Changes

- add a global registry of `Weak<WalRedoManager>`
- add a special thread spawned during `shutdown_pageserver` that sleeps
for 8s, then shuts down all redo managers in the registry and prevents
new redo managers from being created
- propagate the new failure mode of tenant spawning throughout the code
base
- make sure shut down tenant manager results in
PageReconstructError::Cancelled so that if Timeline::get calls come in
after the shutdown, they do the right thing
2024-08-01 07:57:09 +02:00
Alex Chi Z.
d6c79b77df test(pageserver): add test_gc_feedback_with_snapshots (#8474)
should be working after https://github.com/neondatabase/neon/pull/8328
gets merged. Part of https://github.com/neondatabase/neon/issues/8002

adds a new perf benchmark case that ensures garbages can be collected
with branches

---------

Signed-off-by: Alex Chi Z <chi@neon.tech>
2024-07-31 17:55:19 -04:00
Alexander Bayandin
3350daeb9a CI(create-test-report): fix missing benchmark results in Allure report (#8540)
## Problem

In https://github.com/neondatabase/neon/pull/8241 I've accidentally
removed `create-test-report` dependency on `benchmarks` job

## Summary of changes
- Run `create-test-report` after `benchmarks` job
2024-07-31 19:47:59 +01:00
Arpad Müller
939d50a41c storage_scrubber: migrate FindGarbage to remote_storage (#8548)
Uses the newly added APIs from #8541 named `stream_tenants_generic` and
`stream_objects_with_retries` and extends them with
`list_objects_with_retries_generic` and
`stream_tenant_timelines_generic` to migrate the `find-garbage` command
of the scrubber to `GenericRemoteStorage`.

Part of https://github.com/neondatabase/neon/issues/7547
2024-07-31 18:24:42 +00:00
John Spray
2f9ada13c4 controller: simplify reconciler generation increment logic (#8560)
## Problem

This code was confusing, untested and covered:
- an impossible case, where intent state is AttacheStale (we never do
this)
- a rare edge case (going from AttachedMulti to Attached), which we were
not testing, and in any case the pageserver internally does the same
Tenant reset in this transition as it would do if we incremented
generation.

Closes: https://github.com/neondatabase/neon/issues/8367

## Summary of changes

- Simplify the logic to only skip incrementing the generation if the
location already has the expected generation and the exact same mode.
2024-07-31 18:37:47 +01:00
Cihan Demirci
ff51b565d3 cicd: change Azure storage details [2/2] (#8562)
Change Azure storage configuration to point to updated variables/secrets.

Also update subscription id variable.
2024-07-31 17:42:10 +01:00
Tristan Partin
5e0409de95 Fix negative replication delay metric
In some cases, we can get a negative metric for replication_delay_bytes.
My best guess from all the research I've done is that we evaluate
pg_last_wal_receive_lsn() before pg_last_wal_replay_lsn(), and that by
the time everything is said and done, the replay LSN has advanced past
the receive LSN. In this case, our lag can effectively be modeled as
0 due to the speed of the WAL reception and replay.
2024-07-31 10:16:58 -05:00
Christian Schwarz
4e3b70e308 refactor(page_service): Timeline gate guard holding + cancellation + shutdown (#8339)
Since the introduction of sharding, the protocol handling loop in
`handle_pagerequests` cannot know anymore which concrete
`Tenant`/`Timeline` object any of the incoming `PagestreamFeMessage`
resolves to.
In fact, one message might resolve to one `Tenant`/`Timeline` while
the next one may resolve to another one.

To avoid going to tenant manager, we added the `shard_timelines` which
acted as an ever-growing cache that held timeline gate guards open for
the lifetime of the connection.
The consequence of holding the gate guards open was that we had to be
sensitive to every cached `Timeline::cancel` on each interaction with
the network connection, so that Timeline shutdown would not have to wait
for network connection interaction.

We can do better than that, meaning more efficiency & better
abstraction.
I proposed a sketch for it in

* https://github.com/neondatabase/neon/pull/8286

and this PR implements an evolution of that sketch.

The main idea is is that `mod page_service` shall be solely concerned
with the following:
1. receiving requests by speaking the protocol / pagestream subprotocol
2. dispatching the request to a corresponding method on the correct
shard/`Timeline` object
3. sending response by speaking the protocol / pagestream subprotocol.

The cancellation sensitivity responsibilities are clear cut:
* while in `page_service` code, sensitivity to page_service cancellation
is sufficient
* while in `Timeline` code, sensitivity to `Timeline::cancel` is
sufficient

To enforce these responsibilities, we introduce the notion of a
`timeline::handle::Handle` to a `Timeline` object that is checked out
from a `timeline::handle::Cache` for **each request**.
The `Handle` derefs to `Timeline` and is supposed to be used for a
single async method invocation on `Timeline`.
See the lengthy doc comment in `mod handle` for details of the design.
2024-07-31 17:05:45 +02:00
Alex Chi Z.
61a65f61f3 feat(pageserver): support btm-gc-compaction for child branches (#8519)
part of https://github.com/neondatabase/neon/issues/8002

For child branches, we will pull the image of the modified keys from the
parant into the child branch, which creates a full history for
generating key retention. If there are not enough delta keys, the image
won't be wrote eventually, and we will only keep the deltas inside the
child branch. We could avoid the wasteful work to pull the image from
the parent if we can know the number of deltas in advance, in the future
(currently we always pull image for all modified keys in the child
branch)


---------

Signed-off-by: Alex Chi Z <chi@neon.tech>
2024-07-31 15:48:48 +01:00
Alexander Bayandin
d21246c8bd CI(regress-tests): run less regression tests (#8561)
## Problem
We run regression tests on `release` & `debug` builds for each of the
three supported Postgres versions (6 in total).
With upcoming ARM support and Postgres 17, the number of jobs will jump
to 16, which is a lot.

See the internal discussion here:
https://neondb.slack.com/archives/C033A2WE6BZ/p1722365908404329

## Summary of changes
- Run `regress-tests` job in debug builds only with the latest Postgres
version
- Do not do `debug` builds on release branches
2024-07-31 15:10:27 +01:00
Christian Schwarz
4825b0fec3 compaction_level0_phase1: bypass PS PageCache for data blocks (#8543)
part of https://github.com/neondatabase/neon/issues/8184

# Problem

We want to bypass PS PageCache for all data block reads, but
`compact_level0_phase1` currently uses `ValueRef::load` to load the WAL
records from delta layers.
Internally, that maps to `FileBlockReader:read_blk` which hits the
PageCache
[here](e78341e1c2/pageserver/src/tenant/block_io.rs (L229-L236)).

# Solution

This PR adds a mode for `compact_level0_phase1` that uses the
`MergeIterator` for reading the `Value`s from the delta layer files.

`MergeIterator` is a streaming k-merge that uses vectored blob_io under
the hood, which bypasses the PS PageCache for data blocks.

Other notable changes:
* change the `DiskBtreeReader::into_stream` to buffer the node, instead
of holding a `PageCache` `PageReadGuard`.
* Without this, we run out of page cache slots in
`test_pageserver_compaction_smoke`.
* Generally, `PageReadGuard`s aren't supposed to be held across await
points, so, this is a general bugfix.

# Testing / Validation / Performance

`MergeIterator` has not yet been used in production; it's being
developed as part of
* https://github.com/neondatabase/neon/issues/8002

Therefore, this PR adds a validation mode that compares the existing
approach's value iterator with the new approach's stream output, item by
item.
If they're not identical, we log a warning / fail the unit/regression
test.
To avoid flooding the logs, we apply a global rate limit of once per 10
seconds.
In any case, we use the existing approach's value.

Expected performance impact that will be monitored in staging / nightly
benchmarks / eventually pre-prod:
* with validation:
  * increased CPU usage
  * ~doubled VirtualFile read bytes/second metric
* no change in disk IO usage because the kernel page cache will likely
have the pages buffered on the second read
* without validation:
* slightly higher DRAM usage because each iterator participating in the
k-merge has a dedicated buffer (as opposed to before, where compactions
would rely on the PS PageCaceh as a shared evicting buffer)
* less disk IO if previously there were repeat PageCache misses (likely
case on a busy production Pageserver)
* lower CPU usage: PageCache out of the picture, fewer syscalls are made
(vectored blob io batches reads)

# Rollout

The new code is used with validation mode enabled-by-default.
This gets us validation everywhere by default, specifically in
- Rust unit tests
- Python tests
- Nightly pagebench (shouldn't really matter)
- Staging

Before the next release, I'll merge the following aws.git PR that
configures prod to continue using the existing behavior:

* https://github.com/neondatabase/aws/pull/1663

# Interactions With Other Features

This work & rollout should complete before Direct IO is enabled because
Direct IO would double the IOPS & latency for each compaction read
(#8240).

# Future Work

The streaming k-merge's memory usage is proportional to the amount of
memory per participating layer.

But `compact_level0_phase1` still loads all keys into memory for
`all_keys_iter`.
Thus, it continues to have active memory usage proportional to the
number of keys involved in the compaction.

Future work should replace `all_keys_iter` with a streaming keys
iterator.
This PR has a draft in its first commit, which I later reverted because
it's not necessary to achieve the goal of this PR / issue #8184.
2024-07-31 14:17:59 +02:00
Cihan Demirci
a4df3c8488 cicd: change Azure storage details [1/2] (#8553)
Change Azure storage configuration to point to new variables/secrets. They have
the `_NEW` suffix in order not to disrupt any tests while we complete the
switch.
2024-07-30 19:34:15 +00:00
Christian Schwarz
d95b46f3f3 cleanup(compact_level0_phase1): some commentary and wrapping into block expressions (#8544)
Byproduct of scouting done for
https://github.com/neondatabase/neon/issues/8184

refs https://github.com/neondatabase/neon/issues/8184
2024-07-30 18:13:18 +02:00
Yuchen Liang
85bef9f05d feat(scrubber): post scan_metadata results to storage controller (#8502)
Part of #8128, followup to #8480. closes #8421. 

Enable scrubber to optionally post metadata scan health results to
storage controller.

Signed-off-by: Yuchen Liang <yuchen@neon.tech>
2024-07-30 16:07:34 +01:00
Yuchen Liang
e374d6778e feat(storcon): store scrubber metadata scan result (#8480)
Part of #8128, followed by #8502.

## Problem

Currently we lack mechanism to alert unhealthy `scan_metadata` status if
we start running this scrubber command as part of a cronjob. With the
storage controller client introduced to storage scrubber in #8196, it is
viable to set up alert by storing health status in the storage
controller database.

We intentionally do not store the full output to the database as the
json blobs potentially makes the table really huge. Instead, only a
health status and a timestamp recording the last time metadata health
status is posted on a tenant shard.

Signed-off-by: Yuchen Liang <yuchen@neon.tech>
2024-07-30 14:32:00 +01:00
Anton Chaporgin
9ceaf9a986 [neon/acr] impr: push to ACR while building images (#8545)
This tests the ability to push into ACR using OIDC. Proved it worked by running slightly modified YAML.
In `promote-images` we push the following images `neon compute-tools {vm-,}compute-node-{v14,v15,v16}` into `neoneastus2`.

https://github.com/neondatabase/cloud/issues/14640
2024-07-30 14:15:53 +01:00
Alexander Bayandin
f72fe68626 CI(benchmarking): make neonvm default provisioner (#8538)
## Problem

We don't allow regular end-users to use `k8s-pod` provisioner, 
but we still use it in nightly benchmarks

## Summary of changes
- Remove `provisioner` input from `neon-create-project` action, use
`k8s-neonvm` as a default provioner
- Change `neon-` platform prefix to `neonvm-`
- Remove `neon-captest-freetier` and `neon-captest-new` as we already
have their `neonvm` counterparts
2024-07-30 13:38:23 +01:00
Arpad Müller
9fabdda2dc scrubber: add remote_storage based listing APIs and use them in find-large-objects (#8541)
Add two new functions `stream_objects_with_retries` and
`stream_tenants_generic` and use them in the `find-large-objects`
subcommand, migrating it to `remote_storage`.

Also adds the `size` field to the `ListingObject` struct.

Part of #7547
2024-07-30 09:00:37 +00:00
Arpad Müller
1c7b06c988 Add metrics for input data considered and taken for compression (#8522)
If compression is enabled, we currently try compressing each image
larger than a specific size and if the compressed version is smaller, we
write that one, otherwise we use the uncompressed image. However, this
might sometimes be a wasteful process, if there is a substantial amount
of images that don't compress well.

The compression metrics added in #8420
`pageserver_compression_image_in_bytes_total` and
`pageserver_compression_image_out_bytes_total` are well designed for
answering the question how space efficient the total compression process
is end-to-end, which helps one to decide whether to enable it or not.

To answer the question of how much waste there is in terms of trial
compression, so CPU time, we add two metrics:

* one about the images that have been trial-compressed (considered), and
* one about the images where the compressed image has actually been
written (chosen).

There is different ways of weighting them, like for example one could
look at the count, or the compressed data. But the main contributor to
compression CPU usage is amount of data processed, so we weight the
images by their *uncompressed* size. In other words, the two metrics
are:

* `pageserver_compression_image_in_bytes_considered`
* `pageserver_compression_image_in_bytes_chosen`

Part of #5431
2024-07-30 09:59:15 +02:00
John Spray
52b02d95c8 scrubber: enable cleaning up garbage tenants from known deletion bugs, add object age safety check (#8461)
## Problem

Old storage buckets can contain a lot of tenants that aren't known to
the control plane at all, because they belonged to test jobs that get
their control plane state cleaned up shortly after running.

In general, it's somewhat unsafe to purge these, as it's hard to
distinguish "control plane doesn't know about this, so it's garbage"
from "control plane said it didn't know about this, which is a bug in
the scrubber, control plane, or API URL configured".

However, the most common case is that we see only a small husk of a
tenant in S3 from a specific old behavior of the software, for example:
- We had a bug where heatmaps weren't deleted on tenant delete
- When WAL DR was first deployed, we didn't delete initdb.tar.zst on
tenant deletion

## Summary of changes

- Add a KnownBug variant for the garbage reason
- Include such cases in the "safe" deletion mode (`--mode=deleted`)
- Add code that inspects tenants missing in control plane to identify
cases of known bugs (this is kind of slow, but should go away once we've
cleaned all these up)
- Add an additional `-min-age` safety check similar to physical GC,
where even if everything indicates objects aren't needed, we won't
delete something that has been modified too recently.

---------

Co-authored-by: Yuchen Liang <70461588+yliang412@users.noreply.github.com>
Co-authored-by: Arpad Müller <arpad-m@users.noreply.github.com>
2024-07-29 17:50:44 +01:00
Christian Schwarz
4be58522fb l0_flush: use mode=direct by default => coverage in automated tests (#8534)
Testing in staging and pre-prod has been [going

well](https://github.com/neondatabase/neon/issues/7418#issuecomment-2255474917).

This PR enables mode=direct by default, thereby providing additional
coverage in the automated tests:
- Rust tests
- Integration tests
- Nightly pagebench (likely irrelevant because it's read-only)

Production deployments continue to use `mode=page-cache` for the time
being: https://github.com/neondatabase/aws/pull/1655

refs https://github.com/neondatabase/neon/issues/7418
2024-07-29 16:49:22 +02:00
Christian Schwarz
d09dad0ea2 pageserver: fail if id is present in pageserver.toml (#8489)
Overall plan:
https://www.notion.so/neondatabase/Rollout-Plan-simplified-pageserver-initialization-f935ae02b225444e8a41130b7d34e4ea?pvs=4

---

`identity.toml` is the authoritative place for `id` as of
https://github.com/neondatabase/neon/pull/7766

refs https://github.com/neondatabase/neon/issues/7736
2024-07-29 15:16:32 +01:00
John Spray
5775662276 pageserver: fix return code from secondary_download_handler (#8508)
## Problem

The secondary download HTTP API is meant to return 200 if the download
is complete, and 202 if it is still in progress. In #8198 the download
implementation was changed to drop out with success early if it
over-runs a time budget, which resulted in 200 responses for incomplete
downloads.

This breaks storcon_cli's "tenant-warmup" command, which uses the OK
status to indicate download complete.

## Summary of changes

- Only return 200 if we get an Ok() _and_ the progress stats indicate
the download is complete.
2024-07-29 15:05:30 +01:00
Joonas Koivunen
bdfc9ca7e9 test: deflake test_duplicate_creation (#8536)
By including comparison of `remote_consistent_lsn_visible` we risk
flakyness coming from outside of timeline creation. Mask out the
`remote_consistent_lsn_visible` for the comparison.

Evidence:
https://neon-github-public-dev.s3.amazonaws.com/reports/pr-8489/10142336315/index.html#suites/ffbb7f9930a77115316b58ff32b7c719/89ff0270bf58577a
2024-07-29 13:41:06 +01:00
a-masterov
1d8cf5b3a9 Add a test for clickhouse as a logical replication consumer (#8408)
## Problem

We need to test logical replication with 3rd-party tools regularly. 

## Summary of changes

Added a test using ClickHouse as a client

Co-authored-by: Alexander Bayandin <alexander@neon.tech>
2024-07-29 14:35:12 +02:00
Arpad Müller
859f019185 Adopt list_streaming in tenant deletion (#8504)
Uses the Stream based `list_streaming` function added by #8457 in tenant
deletion, as suggested in https://github.com/neondatabase/neon/pull/7932#issuecomment-2150480180 .

We don't have to worry about retries, as the function is wrapped inside
an outer retry block. If there is a retryable error either during the
listing or during deletion, we just do a fresh start.

Also adds `+ Send` bounds as they are required by the
`delete_tenant_remote` function.
2024-07-29 12:05:18 +02:00
Alexander Bayandin
da6bdff893 test_runner: fix user_property usage in benchmarks (#8531)
## Problem

After https://github.com/neondatabase/neon/pull/7990 `regress_test` job
started to fail with an error:
```
...
File "/__w/neon/neon/test_runner/fixtures/benchmark_fixture.py", line 485, in pytest_terminal_summary
  terminalreporter.write(f"{test_report.head_line}.{recorded_property['name']}: ")
TypeError: 'bool' object is not subscriptable
```

https://github.com/neondatabase/neon/actions/runs/10125750938/job/28002582582

It happens because the current implementation doesn't expect pytest's
`user_properties` can be used for anything else but benchmarks (and
https://github.com/neondatabase/neon/pull/7990 started to use it for
tracking `preserve_database_files` parameter)

## Summary of changes
- Make NeonBenchmarker use only records with`neon_benchmarker_` prefix
2024-07-29 11:00:33 +01:00
Christian Schwarz
2416da337e safekeeper: include application name in wal service root span (#8525)
For IDENTIFY_SYSTEM in particular, application name gives away whether
the client is
* walproposer => Some(wal_proposer_recovery)
* safekeeper => Some(safekeeper)
* pageserver => Some(pageserver)

Context:
https://neondb.slack.com/archives/C06SJG60FRB/p1721987794673429?thread_ts=1721981056.451599&cid=C06SJG60FRB
2024-07-28 20:36:59 +01:00
Alexander Bayandin
6cad0455b0 CI(test_runner): Upload all test artifacts if preserve_database_files is enabled (#7990)
## Problem

There's a `NeonEnvBuilder#preserve_database_files` parameter that allows
you to keep database files for debugging purposes (by default, files get
cleaned up), but there's no way to get these files from a CI run.
This PR adds handling of `NeonEnvBuilder#preserve_database_files` and
adds the compressed test output directory to Allure reports (for tests
with this parameter enabled).

Ref https://github.com/neondatabase/neon/issues/6967

## Summary of changes
- Compress and add the whole test output directory to Allure reports
- Currently works only with `neon_env_builder` fixture
- Remove `preserve_database_files = True` from sharding tests as
unneeded

---------

Co-authored-by: Christian Schwarz <christian@neon.tech>
2024-07-27 20:01:10 +01:00
Arpad Müller
b5e95f68b5 Persist archival information (#8479)
Persists whether a timeline is archived or not in `index_part.json`. We
only return success if the upload has actually worked successfully.

Also introduces a new `index_part.json` version number.

Fixes #8459

Part of #8088
2024-07-27 02:32:05 +00:00
Alex Chi Z.
dd40b19db4 fix(pageserver): give L0 compaction priorities over image layer creation (#8443)
close https://github.com/neondatabase/neon/issues/8435

## Summary of changes

If L0 compaction did not include all L0 layers, skip image generation.

There are multiple possible solutions to the original issue, i.e., an
alternative is to wrap the partial L0 compaction in a loop until it
compacts all L0 layers. However, considering that we should weight all
tenants equally, the current solution can ensure everyone gets a chance
to run compaction, and those who write too much won't get a chance to
create image layers. This creates a natural backpressure feedback that
they get a slower read due to no image layers are created, slowing down
their writes, and eventually compaction could keep up with their writes
+ generate image layers.

Consider deployment, we should add an alert on "skipping image layer
generation", so that we won't run into the case that image layers are
not generated => incidents again.

---------

Signed-off-by: Alex Chi Z <chi@neon.tech>
2024-07-26 18:09:55 +00:00
Christian Schwarz
68241f5a3e raise wait_lsn timeout from 60s to 300s (#8529)
Problem
-------

wait_lsn timeouts result in a user-facing errors like

```
$ /tmp/neon/pg_install/v16/bin/pgbench -s3424 -i -I dtGvp user=neondb_owner dbname=neondb host=ep-tiny-wave-w23owa37.eastus2.azure.neon.build sslmode=require options='-cstatement_timeout=0 '

dropping old tables...
NOTICE:  table "pgbench_accounts" does not exist, skipping
NOTICE:  table "pgbench_branches" does not exist, skipping
NOTICE:  table "pgbench_history" does not exist, skipping
NOTICE:  table "pgbench_tellers" does not exist, skipping
creating tables...
generating data (server-side)...
vacuuming...
pgbench: error: query failed: ERROR:  [NEON_SMGR] [shard 0] could not read block 214338 in rel 1663/16389/16839.0 from page server at lsn C/E1C12828
DETAIL:  page server returned error: LSN timeout: Timed out while waiting for WAL record at LSN C/E1418528 to arrive, last_record_lsn 6/999D9CA8 disk consistent LSN=6/999D9CA8, WalReceiver status:  (update 2024-07-25 08:30:07): connecting to node 25, safekeeper candidates (id|update_time|commit_lsn): [(21|08:30:16|C/E1C129E0), (23|08:30:16|C/E1C129E0), (25|08:30:17|C/E1C129E0)]
CONTEXT:  while scanning block 214338 of relation "public.pgbench_accounts"
pgbench: detail: Query was: vacuum analyze pgbench_accounts
```

Solution
--------

Its better to be slow than to fail the queries.
If the app has a deadline, it can use `statement_timeout`.

In the long term, we want to eliminate wait_lsn timeout.

In the short term (this PR), we bump the wait_lsn timeout to
a larger value to reduce the frequency at which these wait_lsn timeouts
occur.

We will observe SLOs and specifically
`pageserver_wait_lsn_seconds_bucket`
before we eliminate the timeout completely.
2024-07-26 16:44:57 +01:00
Christian Schwarz
8154e88732 refactor(layer load API): all errors are permanent (#8527)
I am not aware of a case of "transient" VirtualFile errors as mentioned
in https://github.com/neondatabase/neon/pull/5880

Private DM with Joonas discussing this:
https://neondb.slack.com/archives/D049K7HJ9JM/p1721836424615799
2024-07-26 15:48:44 +01:00
Em Sharnoff
240ba7e10c Fix sql-exporter-autoscaling for pg < 16 (#8523)
The lfc_approximate_working_set_size_windows query was failing on pg14
and pg15 with

  pq: subquery in FROM must have an alias

Because aliases in that position became optional only in pg16.

Some context here: https://neondb.slack.com/archives/C04DGM6SMTM/p1721970322601679?thread_ts=1721921122.528849
2024-07-26 15:08:13 +01:00
Vlad Lazar
7a796a9963 storcon: introduce step down primitive (#8512)
## Problem
We are missing the step-down primitive required to implement rolling
restarts of the storage controller.

## Summary of changes
Add `/control/v1/step_down` endpoint which puts the storage controller
into a state where it rejects
all API requests apart from `/control/v1/step_down`, `/status` and
`/metrics`. When receiving the request,
storage controller cancels all pending reconciles and waits for them to
exit gracefully. The response contains
a snapshot of the in-memory observed state.

Related:
* https://github.com/neondatabase/cloud/issues/14701
* https://github.com/neondatabase/neon/issues/7797
* https://github.com/neondatabase/neon/pull/8310
2024-07-26 14:54:09 +01:00
John Spray
eddfd62333 CODEOWNERS: collapse safekeepers into storage (#8510)
## Problem

- The intersection of "safekeepers" and "storage" is just one person
2024-07-26 13:29:59 +00:00
Vlad Lazar
cdaa2816e7 pageserver: make vectored get the default read path for the pageserver (#8384)
## Problem
Vectored get is already enabled in all prod regions without validation.
The pageserver defaults
are out of sync however.

## Summary of changes
Update the pageserver defaults to match the prod config. Also means that
when running tests locally,
people don't have to use the env vars to get the prod config.
2024-07-26 14:19:52 +01:00
John Spray
3cecbfc04d .github: reduce test concurrency (#8444)
## Problem

This is an experiment to see if 16x concurrency is actually helping, or
if it's just giving us very noisy results. If the total runtime with a
lower concurrency is similar, then a lower concurrency is preferable to
reduce the impact of resource-hungry tests running concurrently.
2024-07-26 11:55:37 +01:00
John Spray
65868258d2 tests: checkpoint instead of compact in test_sharding_split_compaction (#8473)
## Problem

This test relies on writing image layers before the split. It can fail
to do so durably if the image layers are written ahead of the remote
consistent LSN, so we should have been doing a checkpoint rather than
just a compaction
2024-07-26 11:03:44 +01:00
Arpad Müller
bb2a3f9b02 Update Rust to 1.80.0 (#8518)
We keep the practice of keeping the compiler up to date, pointing to the
latest release. This is done by many other projects in the Rust ecosystem as well.

[Release notes](https://github.com/rust-lang/rust/blob/master/RELEASES.md#version-180-2024-07-25).

Prior update was in #8048
2024-07-26 11:17:33 +02:00
John Spray
6711087ddf remote_storage: expose last_modified in listings (#8497)
## Problem

The scrubber would like to check the highest mtime in a tenant's objects
as a safety check during purges. It recently switched to use
GenericRemoteStorage, so we need to expose that in the listing methods.

## Summary of changes

- In Listing.keys, return a ListingObject{} including a last_modified
field, instead of a RemotePath

---------

Co-authored-by: Arpad Müller <arpad-m@users.noreply.github.com>
2024-07-26 10:57:52 +03:00
Andrey Taranik
8182bfdf01 Using own registry to cache layers in docker build workflows (#8521)
## Problem

follow up for #8475 

## Summary of changes

Using own private docker registry in `cache-from` and `cache-to`
settings in docker build-push actions
2024-07-26 08:55:57 +01:00
Arpad Müller
8e02db1ab9 Handle NotInitialized::ShuttingDown error in shard split (#8506)
There is a race condition between timeline shutdown and the split task.
Timeline shutdown first shuts down the upload queue, and only then fires
the cancellation token. A parallel running timeline split operation
might thus encounter a cancelled upload queue before the cancellation
token is fired, and print a noisy error.

Fix this by mapping `anyhow::Error{ NotInitialized::ShuttingDown }) to
`FlushLayerError::Cancelled` instead of `FlushLayerError::Other(_)`.


Fixes #8496
2024-07-26 02:16:10 +02:00
Mihai Bojin
857a1823fe Update links in synthetic-size.md (#8501) 2024-07-26 01:14:12 +01:00
Anastasia Lubennikova
9bfa180f2e Update pgrx to v 0.11.3 (#8515)
update pg_jsonschema extension to v 0.3.1
update pg_graphql extension to v1.5.7
update pgx_ulid extension to v0.1.5
update pg_tiktoken extension, patch Cargo.toml to use new pgrx
2024-07-25 21:21:58 +01:00
Alex Chi Z.
bea0468f1f fix(pageserver): allow incomplete history in btm-gc-compaction (#8500)
This pull request (should) fix the failure of test_gc_feedback. See the
explanation in the newly-added test case.

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

Allow incomplete history for the compaction algorithm.

Signed-off-by: Alex Chi Z <chi@neon.tech>
2024-07-25 12:56:37 -04:00
Vlad Lazar
3977e0a7a3 storcon: shutdown with clean observed state (#8494)
## Problem
Storcon shutdown did not produce a clean observed state. This is not a
problem at the moment, but we will need to stop all reconciles with
clean observed state for rolling restarts.

I tried to test this by collecting the observed state during shutdown
and comparing it with the in-memory observed
state, but it doesn't work because a lot of tests use the cursed attach
hook to create tenants directly through the ps.

## Summary of Changes
Rework storcon shutdown as follows:
* Reconcilers get a separate cancellation token which is a child token
of the global `Service::cancel`.
* Reconcilers get a separate gate
* Add a mechanism to drain the reconciler result queue before
* Put all of this together into a clean shutdown sequence

Related https://github.com/neondatabase/cloud/issues/14701
2024-07-25 15:13:34 +01:00
John Spray
775c0c8892 tests: adjust threshold in test_partial_evict_tenant (#8509)
## Problem

This test was destabilized by
https://github.com/neondatabase/neon/pull/8431. The threshold is
arbitrary & failures are still quite close to it. At a high level the
test is asserting "eviction was approximately fair to these tenants",
which appears to still be the case when the abs diff between ratios is
slightly higher at ~0.6-0.7.

## Summary of changes

- Change threshold from 0.06 to 0.065. Based on the last ~10 failures
that should be sufficient.
2024-07-25 15:00:42 +01:00
John Spray
24ea9f9f60 tests: always scrub on test exit when using S3Storage (#8437)
## Problem

Currently, tests may have a scrub during teardown if they ask for it,
but most tests don't request it. To detect "unknown unknowns", let's run
it at the end of every test where possible. This is similar to asserting
that there are no errors in the log at the end of tests.

## Summary of changes

- Remove explicit `enable_scrub_on_exit`
- Always scrub if remote storage is an S3Storage.
2024-07-25 14:19:38 +01:00
Vlad Lazar
9c5ad21341 storcon: make heartbeats restart aware (#8222)
## Problem
Re-attach blocks the pageserver http server from starting up. Hence, it
can't reply to heartbeats
until that's done. This makes the storage controller mark the node
off-line (not good). We worked
around this by setting the interval after which nodes are marked offline
to 5 minutes. This isn't a
long term solution.

## Summary of changes
* Introduce a new `NodeAvailability` state: `WarmingUp`. This state
models the following time interval:
* From receiving the re-attach request until the pageserver replies to
the first heartbeat post re-attach
* The heartbeat delta generator becomes aware of this state and uses a
separate longer interval
* Flag `max-warming-up-interval` now models the longer timeout and
`max-offline-interval` the shorter one to
match the names of the states

Closes https://github.com/neondatabase/neon/issues/7552
2024-07-25 14:09:12 +01:00
Peter Bendel
f76a4e0ad2 Temporarily remove week-end test for res-aurora from pgbench-compare benchmarking runs (#8493)
## Problem

The rds-aurora endpoint connection cannot be reached from GitHub action
runners.
Temporarily remove this DBMS from the pgbench comparison runs.

## Summary of changes

On Saturday we normally run Neon in comparison with AWS RDS-Postgres and
AWS RDS-Aurora.
Remove Aurora until we have a working setup
2024-07-25 09:51:20 +01:00
Christian Schwarz
a1256b2a67 fix: remote timeline client shutdown trips circuit breaker (#8495)
Before this PR

1.The circuit breaker would trip on CompactionError::Shutdown. That's
wrong, we want to ignore those cases.
2. remote timeline client shutdown would not be mapped to
CompactionError::Shutdown in all circumstances.

We observed this in staging, see
https://neondb.slack.com/archives/C033RQ5SPDH/p1721829745384449

This PR fixes (1) with a simple `match` statement, and (2) by switching
a bunch of `anyhow` usage over to distinguished errors that ultimately
get mapped to `CompactionError::Shutdown`.

I removed the implicit `#[from]` conversion from `anyhow::Error` to
`CompactionError::Other` to discover all the places that were mapping
remote timeline client shutdown to `anyhow::Error`.

In my opinion `#[from]` is an antipattern and we should avoid it,
especially for `anyhow::Error`. If some callee is going to return
anyhow, the very least the caller should to is to acknowledge, through a
`map_err(MyError::Other)` that they're conflating different failure
reasons.
2024-07-25 09:44:31 +01:00
Christian Schwarz
d57412aaab followup(#8359): pre-initialize circuitbreaker metrics (#8491) 2024-07-25 10:24:28 +02:00
161 changed files with 8544 additions and 2223 deletions

View File

@@ -14,11 +14,8 @@ inputs:
api_host:
description: 'Neon API host'
default: console-stage.neon.build
provisioner:
description: 'k8s-pod or k8s-neonvm'
default: 'k8s-pod'
compute_units:
description: '[Min, Max] compute units; Min and Max are used for k8s-neonvm with autoscaling, for k8s-pod values Min and Max should be equal'
description: '[Min, Max] compute units'
default: '[1, 1]'
outputs:
@@ -37,10 +34,6 @@ runs:
# A shell without `set -x` to not to expose password/dsn in logs
shell: bash -euo pipefail {0}
run: |
if [ "${PROVISIONER}" == "k8s-pod" ] && [ "${MIN_CU}" != "${MAX_CU}" ]; then
echo >&2 "For k8s-pod provisioner MIN_CU should be equal to MAX_CU"
fi
project=$(curl \
"https://${API_HOST}/api/v2/projects" \
--fail \
@@ -52,7 +45,7 @@ runs:
\"name\": \"Created by actions/neon-project-create; GITHUB_RUN_ID=${GITHUB_RUN_ID}\",
\"pg_version\": ${POSTGRES_VERSION},
\"region_id\": \"${REGION_ID}\",
\"provisioner\": \"${PROVISIONER}\",
\"provisioner\": \"k8s-neonvm\",
\"autoscaling_limit_min_cu\": ${MIN_CU},
\"autoscaling_limit_max_cu\": ${MAX_CU},
\"settings\": { }
@@ -75,6 +68,5 @@ runs:
API_KEY: ${{ inputs.api_key }}
REGION_ID: ${{ inputs.region_id }}
POSTGRES_VERSION: ${{ inputs.postgres_version }}
PROVISIONER: ${{ inputs.provisioner }}
MIN_CU: ${{ fromJSON(inputs.compute_units)[0] }}
MAX_CU: ${{ fromJSON(inputs.compute_units)[1] }}

View File

@@ -131,8 +131,8 @@ runs:
exit 1
fi
if [[ "${{ inputs.run_in_parallel }}" == "true" ]]; then
# -n16 uses sixteen processes to run tests via pytest-xdist
EXTRA_PARAMS="-n16 $EXTRA_PARAMS"
# -n sets the number of parallel processes that pytest-xdist will run
EXTRA_PARAMS="-n12 $EXTRA_PARAMS"
# --dist=loadgroup points tests marked with @pytest.mark.xdist_group
# to the same worker to make @pytest.mark.order work with xdist

View File

@@ -19,6 +19,10 @@ on:
description: 'debug or release'
required: true
type: string
pg-versions:
description: 'a json array of postgres versions to run regression tests on'
required: true
type: string
defaults:
run:
@@ -254,7 +258,7 @@ jobs:
strategy:
fail-fast: false
matrix:
pg_version: [ v14, v15, v16 ]
pg_version: ${{ fromJson(inputs.pg-versions) }}
steps:
- uses: actions/checkout@v4
with:
@@ -278,14 +282,11 @@ jobs:
CHECK_ONDISK_DATA_COMPATIBILITY: nonempty
BUILD_TAG: ${{ inputs.build-tag }}
PAGESERVER_VIRTUAL_FILE_IO_ENGINE: tokio-epoll-uring
PAGESERVER_GET_VECTORED_IMPL: vectored
PAGESERVER_GET_IMPL: vectored
PAGESERVER_VALIDATE_VEC_GET: true
# Temporary disable this step until we figure out why it's so flaky
# Ref https://github.com/neondatabase/neon/issues/4540
- name: Merge and upload coverage data
if: |
false &&
inputs.build-type == 'debug' && matrix.pg_version == 'v14'
inputs.build-type == 'debug' && matrix.pg_version == 'v16'
uses: ./.github/actions/save-coverage-data

View File

@@ -63,11 +63,9 @@ jobs:
- DEFAULT_PG_VERSION: 16
PLATFORM: "neon-staging"
region_id: ${{ github.event.inputs.region_id || 'aws-us-east-2' }}
provisioner: 'k8s-pod'
- DEFAULT_PG_VERSION: 16
PLATFORM: "azure-staging"
region_id: 'azure-eastus2'
provisioner: 'k8s-neonvm'
env:
TEST_PG_BENCH_DURATIONS_MATRIX: "300"
TEST_PG_BENCH_SCALES_MATRIX: "10,100"
@@ -100,7 +98,6 @@ jobs:
region_id: ${{ matrix.region_id }}
postgres_version: ${{ env.DEFAULT_PG_VERSION }}
api_key: ${{ secrets.NEON_STAGING_API_KEY }}
provisioner: ${{ matrix.provisioner }}
- name: Run benchmark
uses: ./.github/actions/run-python-test-set
@@ -216,11 +213,11 @@ jobs:
# Create matrices for the benchmarking jobs, so we run benchmarks on rds only once a week (on Saturday)
#
# Available platforms:
# - neon-captest-new: Freshly created project (1 CU)
# - neon-captest-freetier: Use freetier-sized compute (0.25 CU)
# - neonvm-captest-new: Freshly created project (1 CU)
# - neonvm-captest-freetier: Use freetier-sized compute (0.25 CU)
# - neonvm-captest-azure-new: Freshly created project (1 CU) in azure region
# - neonvm-captest-azure-freetier: Use freetier-sized compute (0.25 CU) in azure region
# - neon-captest-reuse: Reusing existing project
# - neonvm-captest-reuse: Reusing existing project
# - rds-aurora: Aurora Postgres Serverless v2 with autoscaling from 0.5 to 2 ACUs
# - rds-postgres: RDS Postgres db.m5.large instance (2 vCPU, 8 GiB) with gp3 EBS storage
env:
@@ -245,24 +242,21 @@ jobs:
"'"$region_id_default"'"
],
"platform": [
"neon-captest-new",
"neon-captest-reuse",
"neonvm-captest-new",
"neonvm-captest-reuse",
"neonvm-captest-new"
],
"db_size": [ "10gb" ],
"include": [{ "pg_version": 16, "region_id": "'"$region_id_default"'", "platform": "neon-captest-freetier", "db_size": "3gb" },
{ "pg_version": 16, "region_id": "'"$region_id_default"'", "platform": "neon-captest-new", "db_size": "50gb" },
{ "pg_version": 16, "region_id": "'"$region_id_default"'", "platform": "neonvm-captest-freetier", "db_size": "3gb" },
"include": [{ "pg_version": 16, "region_id": "'"$region_id_default"'", "platform": "neonvm-captest-freetier", "db_size": "3gb" },
{ "pg_version": 16, "region_id": "'"$region_id_default"'", "platform": "neonvm-captest-new", "db_size": "50gb" },
{ "pg_version": 16, "region_id": "azure-eastus2", "platform": "neonvm-azure-captest-freetier", "db_size": "3gb" },
{ "pg_version": 16, "region_id": "azure-eastus2", "platform": "neonvm-azure-captest-new", "db_size": "10gb" },
{ "pg_version": 16, "region_id": "azure-eastus2", "platform": "neonvm-azure-captest-new", "db_size": "50gb" },
{ "pg_version": 16, "region_id": "azure-eastus2", "platform": "neonvm-azure-captest-freetier", "db_size": "3gb" },
{ "pg_version": 16, "region_id": "azure-eastus2", "platform": "neonvm-azure-captest-new", "db_size": "10gb" },
{ "pg_version": 16, "region_id": "azure-eastus2", "platform": "neonvm-azure-captest-new", "db_size": "50gb" },
{ "pg_version": 16, "region_id": "'"$region_id_default"'", "platform": "neonvm-captest-sharding-reuse", "db_size": "50gb" }]
}'
if [ "$(date +%A)" = "Saturday" ]; then
matrix=$(echo "$matrix" | jq '.include += [{ "pg_version": 14, "region_id": "'"$region_id_default"'", "platform": "rds-postgres", "db_size": "10gb"},
{ "pg_version": 14, "region_id": "'"$region_id_default"'", "platform": "rds-aurora", "db_size": "50gb"}]')
matrix=$(echo "$matrix" | jq '.include += [{ "pg_version": 14, "region_id": "'"$region_id_default"'", "platform": "rds-postgres", "db_size": "10gb"}]')
fi
echo "matrix=$(echo "$matrix" | jq --compact-output '.')" >> $GITHUB_OUTPUT
@@ -272,7 +266,7 @@ jobs:
run: |
matrix='{
"platform": [
"neon-captest-reuse"
"neonvm-captest-reuse"
]
}'
@@ -288,7 +282,7 @@ jobs:
run: |
matrix='{
"platform": [
"neon-captest-reuse"
"neonvm-captest-reuse"
],
"scale": [
"10"
@@ -339,7 +333,7 @@ jobs:
prefix: latest
- name: Create Neon Project
if: contains(fromJson('["neon-captest-new", "neon-captest-freetier", "neonvm-captest-new", "neonvm-captest-freetier", "neonvm-azure-captest-freetier", "neonvm-azure-captest-new"]'), matrix.platform)
if: contains(fromJson('["neonvm-captest-new", "neonvm-captest-freetier", "neonvm-azure-captest-freetier", "neonvm-azure-captest-new"]'), matrix.platform)
id: create-neon-project
uses: ./.github/actions/neon-project-create
with:
@@ -347,19 +341,18 @@ jobs:
postgres_version: ${{ env.DEFAULT_PG_VERSION }}
api_key: ${{ secrets.NEON_STAGING_API_KEY }}
compute_units: ${{ (contains(matrix.platform, 'captest-freetier') && '[0.25, 0.25]') || '[1, 1]' }}
provisioner: ${{ (contains(matrix.platform, 'neonvm-') && 'k8s-neonvm') || 'k8s-pod' }}
- name: Set up Connection String
id: set-up-connstr
run: |
case "${PLATFORM}" in
neon-captest-reuse)
neonvm-captest-reuse)
CONNSTR=${{ secrets.BENCHMARK_CAPTEST_CONNSTR }}
;;
neonvm-captest-sharding-reuse)
CONNSTR=${{ secrets.BENCHMARK_CAPTEST_SHARDING_CONNSTR }}
;;
neon-captest-new | neon-captest-freetier | neonvm-captest-new | neonvm-captest-freetier | neonvm-azure-captest-new | neonvm-azure-captest-freetier)
neonvm-captest-new | neonvm-captest-freetier | neonvm-azure-captest-new | neonvm-azure-captest-freetier)
CONNSTR=${{ steps.create-neon-project.outputs.dsn }}
;;
rds-aurora)
@@ -443,9 +436,9 @@ jobs:
fail-fast: false
matrix:
include:
- PLATFORM: "neon-captest-pgvector"
- PLATFORM: "neonvm-captest-pgvector"
- PLATFORM: "azure-captest-pgvector"
env:
TEST_PG_BENCH_DURATIONS_MATRIX: "15m"
TEST_PG_BENCH_SCALES_MATRIX: "1"
@@ -487,7 +480,7 @@ jobs:
id: set-up-connstr
run: |
case "${PLATFORM}" in
neon-captest-pgvector)
neonvm-captest-pgvector)
CONNSTR=${{ secrets.BENCHMARK_PGVECTOR_CONNSTR }}
;;
azure-captest-pgvector)
@@ -586,7 +579,7 @@ jobs:
id: set-up-connstr
run: |
case "${PLATFORM}" in
neon-captest-reuse)
neonvm-captest-reuse)
CONNSTR=${{ secrets.BENCHMARK_CAPTEST_CLICKBENCH_10M_CONNSTR }}
;;
rds-aurora)
@@ -596,7 +589,7 @@ jobs:
CONNSTR=${{ secrets.BENCHMARK_RDS_POSTGRES_CLICKBENCH_10M_CONNSTR }}
;;
*)
echo >&2 "Unknown PLATFORM=${PLATFORM}. Allowed only 'neon-captest-reuse', 'rds-aurora', or 'rds-postgres'"
echo >&2 "Unknown PLATFORM=${PLATFORM}. Allowed only 'neonvm-captest-reuse', 'rds-aurora', or 'rds-postgres'"
exit 1
;;
esac
@@ -673,7 +666,7 @@ jobs:
- name: Get Connstring Secret Name
run: |
case "${PLATFORM}" in
neon-captest-reuse)
neonvm-captest-reuse)
ENV_PLATFORM=CAPTEST_TPCH
;;
rds-aurora)
@@ -683,7 +676,7 @@ jobs:
ENV_PLATFORM=RDS_AURORA_TPCH
;;
*)
echo >&2 "Unknown PLATFORM=${PLATFORM}. Allowed only 'neon-captest-reuse', 'rds-aurora', or 'rds-postgres'"
echo >&2 "Unknown PLATFORM=${PLATFORM}. Allowed only 'neonvm-captest-reuse', 'rds-aurora', or 'rds-postgres'"
exit 1
;;
esac
@@ -760,7 +753,7 @@ jobs:
id: set-up-connstr
run: |
case "${PLATFORM}" in
neon-captest-reuse)
neonvm-captest-reuse)
CONNSTR=${{ secrets.BENCHMARK_USER_EXAMPLE_CAPTEST_CONNSTR }}
;;
rds-aurora)
@@ -770,7 +763,7 @@ jobs:
CONNSTR=${{ secrets.BENCHMARK_USER_EXAMPLE_RDS_POSTGRES_CONNSTR }}
;;
*)
echo >&2 "Unknown PLATFORM=${PLATFORM}. Allowed only 'neon-captest-reuse', 'rds-aurora', or 'rds-postgres'"
echo >&2 "Unknown PLATFORM=${PLATFORM}. Allowed only 'neonvm-captest-reuse', 'rds-aurora', or 'rds-postgres'"
exit 1
;;
esac

View File

@@ -72,6 +72,12 @@ jobs:
username: ${{ secrets.NEON_DOCKERHUB_USERNAME }}
password: ${{ secrets.NEON_DOCKERHUB_PASSWORD }}
- uses: docker/login-action@v3
with:
registry: cache.neon.build
username: ${{ secrets.NEON_CI_DOCKERCACHE_USERNAME }}
password: ${{ secrets.NEON_CI_DOCKERCACHE_PASSWORD }}
- uses: docker/build-push-action@v6
with:
context: .
@@ -79,8 +85,8 @@ jobs:
push: true
pull: true
file: Dockerfile.build-tools
cache-from: type=registry,ref=neondatabase/build-tools:cache-${{ matrix.arch }}
cache-to: ${{ github.ref_name == 'main' && format('type=registry,ref=neondatabase/build-tools:cache-{0},mode=max', matrix.arch) || '' }}
cache-from: type=registry,ref=cache.neon.build/build-tools:cache-${{ matrix.arch }}
cache-to: ${{ github.ref_name == 'main' && format('type=registry,ref=cache.neon.build/build-tools:cache-{0},mode=max', matrix.arch) || '' }}
tags: neondatabase/build-tools:${{ inputs.image-tag }}-${{ matrix.arch }}
- name: Remove custom docker config directory

View File

@@ -203,7 +203,8 @@ jobs:
fail-fast: false
matrix:
arch: [ x64 ]
build-type: [ debug, release ]
# Do not build or run tests in debug for release branches
build-type: ${{ fromJson((startsWith(github.ref_name, 'release' && github.event_name == 'push')) && '["release"]' || '["debug", "release"]') }}
include:
- build-type: release
arch: arm64
@@ -213,6 +214,8 @@ jobs:
build-tools-image: ${{ needs.build-build-tools-image.outputs.image }}
build-tag: ${{ needs.tag.outputs.build-tag }}
build-type: ${{ matrix.build-type }}
# Run tests on all Postgres versions in release builds and only on the latest version in debug builds
pg-versions: ${{ matrix.build-type == 'release' && '["v14", "v15", "v16"]' || '["v16"]' }}
secrets: inherit
# Keep `benchmarks` job outside of `build-and-test-locally` workflow to make job failures non-blocking
@@ -286,9 +289,6 @@ jobs:
PERF_TEST_RESULT_CONNSTR: "${{ secrets.PERF_TEST_RESULT_CONNSTR }}"
TEST_RESULT_CONNSTR: "${{ secrets.REGRESS_TEST_RESULT_CONNSTR_NEW }}"
PAGESERVER_VIRTUAL_FILE_IO_ENGINE: tokio-epoll-uring
PAGESERVER_GET_VECTORED_IMPL: vectored
PAGESERVER_GET_IMPL: vectored
PAGESERVER_VALIDATE_VEC_GET: false
# XXX: no coverage data handling here, since benchmarks are run on release builds,
# while coverage is currently collected for the debug ones
@@ -309,7 +309,7 @@ jobs:
SLACK_BOT_TOKEN: ${{ secrets.SLACK_BOT_TOKEN }}
create-test-report:
needs: [ check-permissions, build-and-test-locally, coverage-report, build-build-tools-image ]
needs: [ check-permissions, build-and-test-locally, coverage-report, build-build-tools-image, benchmarks ]
if: ${{ !cancelled() && contains(fromJSON('["skipped", "success"]'), needs.check-permissions.result) }}
outputs:
report-url: ${{ steps.create-allure-report.outputs.report-url }}
@@ -499,6 +499,12 @@ jobs:
username: ${{ secrets.NEON_DOCKERHUB_USERNAME }}
password: ${{ secrets.NEON_DOCKERHUB_PASSWORD }}
- uses: docker/login-action@v3
with:
registry: cache.neon.build
username: ${{ secrets.NEON_CI_DOCKERCACHE_USERNAME }}
password: ${{ secrets.NEON_CI_DOCKERCACHE_PASSWORD }}
- uses: docker/build-push-action@v6
with:
context: .
@@ -510,9 +516,8 @@ jobs:
push: true
pull: true
file: Dockerfile
cache-from: type=registry,ref=neondatabase/neon:cache-${{ matrix.arch }}
# 23.07.2024 temporarily disable cache saving in the registry as it is very slow
# cache-to: ${{ github.ref_name == 'main' && format('type=registry,ref=neondatabase/neon:cache-{0},mode=max', matrix.arch) || '' }}
cache-from: type=registry,ref=cache.neon.build/neon:cache-${{ matrix.arch }}
cache-to: ${{ github.ref_name == 'main' && format('type=registry,ref=cache.neon.build/neon:cache-{0},mode=max', matrix.arch) || '' }}
tags: |
neondatabase/neon:${{ needs.tag.outputs.build-tag }}-${{ matrix.arch }}
@@ -591,6 +596,12 @@ jobs:
username: ${{ secrets.AWS_ACCESS_KEY_DEV }}
password: ${{ secrets.AWS_SECRET_KEY_DEV }}
- uses: docker/login-action@v3
with:
registry: cache.neon.build
username: ${{ secrets.NEON_CI_DOCKERCACHE_USERNAME }}
password: ${{ secrets.NEON_CI_DOCKERCACHE_PASSWORD }}
- name: Build compute-node image
uses: docker/build-push-action@v6
with:
@@ -604,9 +615,8 @@ jobs:
push: true
pull: true
file: Dockerfile.compute-node
cache-from: type=registry,ref=neondatabase/compute-node-${{ matrix.version }}:cache-${{ matrix.arch }}
# 23.07.2024 temporarily disable cache saving in the registry as it is very slow
# cache-to: ${{ github.ref_name == 'main' && format('type=registry,ref=neondatabase/compute-node-{0}:cache-{1},mode=max', matrix.version, matrix.arch) || '' }}
cache-from: type=registry,ref=cache.neon.build/compute-node-${{ matrix.version }}:cache-${{ matrix.arch }}
cache-to: ${{ github.ref_name == 'main' && format('type=registry,ref=cache.neon.build/compute-node-{0}:cache-{1},mode=max', matrix.version, matrix.arch) || '' }}
tags: |
neondatabase/compute-node-${{ matrix.version }}:${{ needs.tag.outputs.build-tag }}-${{ matrix.arch }}
@@ -625,9 +635,8 @@ jobs:
pull: true
file: Dockerfile.compute-node
target: neon-pg-ext-test
cache-from: type=registry,ref=neondatabase/neon-test-extensions-${{ matrix.version }}:cache-${{ matrix.arch }}
# 23.07.2024 temporarily disable cache saving in the registry as it is very slow
# cache-to: ${{ github.ref_name == 'main' && format('type=registry,ref=neondatabase/neon-test-extensions-{0}:cache-{1},mode=max', matrix.version, matrix.arch) || '' }}
cache-from: type=registry,ref=cache.neon.build/neon-test-extensions-${{ matrix.version }}:cache-${{ matrix.arch }}
cache-to: ${{ github.ref_name == 'main' && format('type=registry,ref=cache.neon.build/neon-test-extensions-{0}:cache-{1},mode=max', matrix.version, matrix.arch) || '' }}
tags: |
neondatabase/neon-test-extensions-${{ matrix.version }}:${{needs.tag.outputs.build-tag}}-${{ matrix.arch }}
@@ -827,6 +836,9 @@ jobs:
rm -rf .docker-custom
promote-images:
permissions:
contents: read # This is required for actions/checkout
id-token: write # This is required for Azure Login to work.
needs: [ check-permissions, tag, test-images, vm-compute-node-image ]
runs-on: ubuntu-22.04
@@ -853,6 +865,28 @@ jobs:
neondatabase/vm-compute-node-${version}:${{ needs.tag.outputs.build-tag }}
done
- name: Azure login
if: github.ref_name == 'main'
uses: azure/login@6c251865b4e6290e7b78be643ea2d005bc51f69a # @v2.1.1
with:
client-id: ${{ secrets.AZURE_DEV_CLIENT_ID }}
tenant-id: ${{ secrets.AZURE_TENANT_ID }}
subscription-id: ${{ secrets.AZURE_DEV_SUBSCRIPTION_ID }}
- name: Login to ACR
if: github.ref_name == 'main'
run: |
az acr login --name=neoneastus2
- name: Copy docker images to ACR-dev
if: github.ref_name == 'main'
run: |
for image in neon compute-tools {vm-,}compute-node-{v14,v15,v16}; do
docker buildx imagetools create \
-t neoneastus2.azurecr.io/neondatabase/${image}:${{ needs.tag.outputs.build-tag }} \
neondatabase/${image}:${{ needs.tag.outputs.build-tag }}
done
- name: Add latest tag to images
if: github.ref_name == 'main'
run: |

View File

@@ -13,6 +13,7 @@ on:
paths:
- '.github/workflows/pg-clients.yml'
- 'test_runner/pg_clients/**'
- 'test_runner/logical_repl/**'
- 'poetry.lock'
workflow_dispatch:
@@ -49,6 +50,77 @@ jobs:
image-tag: ${{ needs.check-build-tools-image.outputs.image-tag }}
secrets: inherit
test-logical-replication:
needs: [ build-build-tools-image ]
runs-on: ubuntu-22.04
container:
image: ${{ needs.build-build-tools-image.outputs.image }}
credentials:
username: ${{ secrets.NEON_DOCKERHUB_USERNAME }}
password: ${{ secrets.NEON_DOCKERHUB_PASSWORD }}
options: --init --user root
services:
clickhouse:
image: clickhouse/clickhouse-server:24.6.3.64
ports:
- 9000:9000
- 8123:8123
steps:
- uses: actions/checkout@v4
- name: Download Neon artifact
uses: ./.github/actions/download
with:
name: neon-${{ runner.os }}-${{ runner.arch }}-release-artifact
path: /tmp/neon/
prefix: latest
- name: Create Neon Project
id: create-neon-project
uses: ./.github/actions/neon-project-create
with:
api_key: ${{ secrets.NEON_STAGING_API_KEY }}
postgres_version: ${{ env.DEFAULT_PG_VERSION }}
- name: Run tests
uses: ./.github/actions/run-python-test-set
with:
build_type: remote
test_selection: logical_repl
run_in_parallel: false
extra_params: -m remote_cluster
pg_version: ${{ env.DEFAULT_PG_VERSION }}
env:
BENCHMARK_CONNSTR: ${{ steps.create-neon-project.outputs.dsn }}
- name: Delete Neon Project
if: always()
uses: ./.github/actions/neon-project-delete
with:
project_id: ${{ steps.create-neon-project.outputs.project_id }}
api_key: ${{ secrets.NEON_STAGING_API_KEY }}
- name: Create Allure report
if: ${{ !cancelled() }}
id: create-allure-report
uses: ./.github/actions/allure-report-generate
with:
store-test-results-into-db: true
env:
REGRESS_TEST_RESULT_CONNSTR_NEW: ${{ secrets.REGRESS_TEST_RESULT_CONNSTR_NEW }}
- name: Post to a Slack channel
if: github.event.schedule && failure()
uses: slackapi/slack-github-action@v1
with:
channel-id: "C06KHQVQ7U3" # on-call-qa-staging-stream
slack-message: |
Testing the logical replication: <${{ github.server_url }}/${{ github.repository }}/actions/runs/${{ github.run_id }}|${{ job.status }}> (<${{ steps.create-allure-report.outputs.report-url }}|test report>)
env:
SLACK_BOT_TOKEN: ${{ secrets.SLACK_BOT_TOKEN }}
test-postgres-client-libs:
needs: [ build-build-tools-image ]
runs-on: ubuntu-22.04

View File

@@ -10,11 +10,13 @@ defaults:
run:
shell: bash -euxo pipefail {0}
concurrency:
group: ${{ github.workflow }}-${{ github.ref_name }}-${{ github.ref_name == 'main' && github.sha || 'anysha' }}
cancel-in-progress: true
env:
# A concurrency group that we use for e2e-tests runs, matches `concurrency.group` above with `github.repository` as a prefix
E2E_CONCURRENCY_GROUP: ${{ github.repository }}-e2e-tests-${{ github.ref_name }}-${{ github.ref_name == 'main' && github.sha || 'anysha' }}
AWS_ACCESS_KEY_ID: ${{ secrets.AWS_ACCESS_KEY_DEV }}
AWS_SECRET_ACCESS_KEY: ${{ secrets.AWS_SECRET_KEY_DEV }}
jobs:
cancel-previous-e2e-tests:
@@ -64,19 +66,35 @@ jobs:
needs: [ tag ]
runs-on: ubuntu-22.04
env:
EVENT_ACTION: ${{ github.event.action }}
GH_TOKEN: ${{ secrets.CI_ACCESS_TOKEN }}
TAG: ${{ needs.tag.outputs.build-tag }}
steps:
- name: check if ecr image are present
env:
AWS_ACCESS_KEY_ID: ${{ secrets.AWS_ACCESS_KEY_DEV }}
AWS_SECRET_ACCESS_KEY: ${{ secrets.AWS_SECRET_KEY_DEV }}
- name: Wait for `promote-images` job to finish
# It's important to have a timeout here, the script in the step can run infinitely
timeout-minutes: 60
run: |
for REPO in neon compute-tools compute-node-v14 vm-compute-node-v14 compute-node-v15 vm-compute-node-v15 compute-node-v16 vm-compute-node-v16; do
OUTPUT=$(aws ecr describe-images --repository-name ${REPO} --region eu-central-1 --query "imageDetails[?imageTags[?contains(@, '${TAG}')]]" --output text)
if [ "$OUTPUT" == "" ]; then
echo "$REPO with image tag $TAG not found" >> $GITHUB_OUTPUT
exit 1
fi
if [ "${GITHUB_EVENT_NAME}" != "pull_request" ] || [ "${EVENT_ACTION}" != "ready_for_review" ]; then
exit 0
fi
# For PRs we use the run id as the tag
BUILD_AND_TEST_RUN_ID=${TAG}
while true; do
conclusion=$(gh run --repo ${GITHUB_REPOSITORY} view ${BUILD_AND_TEST_RUN_ID} --json jobs --jq '.jobs[] | select(.name == "promote-images") | .conclusion')
case "$conclusion" in
success)
break
;;
failure | cancelled | skipped)
echo "The 'promote-images' job didn't succeed: '${conclusion}'. Exiting..."
exit 1
;;
*)
echo "The 'promote-images' hasn't succeed yet. Waiting..."
sleep 60
;;
esac
done
- name: Set e2e-platforms

View File

@@ -1,13 +1,13 @@
/compute_tools/ @neondatabase/control-plane @neondatabase/compute
/storage_controller @neondatabase/storage
/libs/pageserver_api/ @neondatabase/storage
/libs/postgres_ffi/ @neondatabase/compute @neondatabase/safekeepers
/libs/postgres_ffi/ @neondatabase/compute @neondatabase/storage
/libs/remote_storage/ @neondatabase/storage
/libs/safekeeper_api/ @neondatabase/safekeepers
/libs/safekeeper_api/ @neondatabase/storage
/libs/vm_monitor/ @neondatabase/autoscaling
/pageserver/ @neondatabase/storage
/pgxn/ @neondatabase/compute
/pgxn/neon/ @neondatabase/compute @neondatabase/safekeepers
/pgxn/neon/ @neondatabase/compute @neondatabase/storage
/proxy/ @neondatabase/proxy
/safekeeper/ @neondatabase/safekeepers
/safekeeper/ @neondatabase/storage
/vendor/ @neondatabase/compute

197
Cargo.lock generated
View File

@@ -1418,7 +1418,7 @@ dependencies = [
"clap",
"criterion-plot",
"is-terminal",
"itertools",
"itertools 0.10.5",
"num-traits",
"once_cell",
"oorandom",
@@ -1439,7 +1439,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "6b50826342786a51a89e2da3a28f1c32b06e387201bc2d19791f622c673706b1"
dependencies = [
"cast",
"itertools",
"itertools 0.10.5",
]
[[package]]
@@ -1672,6 +1672,7 @@ checksum = "62d6dcd069e7b5fe49a302411f759d4cf1cf2c27fe798ef46fb8baefc053dd2b"
dependencies = [
"bitflags 2.4.1",
"byteorder",
"chrono",
"diesel_derives",
"itoa",
"pq-sys",
@@ -2133,6 +2134,12 @@ dependencies = [
"slab",
]
[[package]]
name = "gen_ops"
version = "0.4.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "304de19db7028420975a296ab0fcbbc8e69438c4ed254a1e41e2a7f37d5f0e0a"
[[package]]
name = "generic-array"
version = "0.14.7"
@@ -2709,17 +2716,6 @@ version = "3.0.4"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02"
[[package]]
name = "io-lifetimes"
version = "1.0.11"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "eae7b9aee968036d54dce06cebaefd919e4472e753296daccd6d344e3e2df0c2"
dependencies = [
"hermit-abi",
"libc",
"windows-sys 0.48.0",
]
[[package]]
name = "io-uring"
version = "0.6.2"
@@ -2738,14 +2734,13 @@ checksum = "8f518f335dce6725a761382244631d86cf0ccb2863413590b31338feb467f9c3"
[[package]]
name = "is-terminal"
version = "0.4.7"
version = "0.4.12"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "adcf93614601c8129ddf72e2d5633df827ba6551541c6d8c59520a371475be1f"
checksum = "f23ff5ef2b80d608d61efee834934d862cd92461afc0560dedf493e4c033738b"
dependencies = [
"hermit-abi",
"io-lifetimes",
"rustix 0.37.25",
"windows-sys 0.48.0",
"libc",
"windows-sys 0.52.0",
]
[[package]]
@@ -2757,6 +2752,15 @@ dependencies = [
"either",
]
[[package]]
name = "itertools"
version = "0.12.1"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "ba291022dbbd398a455acf126c1e341954079855bc60dfdda641363bd6922569"
dependencies = [
"either",
]
[[package]]
name = "itoa"
version = "1.0.6"
@@ -2871,18 +2875,6 @@ version = "0.2.8"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "4ec2a862134d2a7d32d7983ddcdd1c4923530833c9f2ea1a44fc5fa473989058"
[[package]]
name = "linux-raw-sys"
version = "0.1.4"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "f051f77a7c8e6957c0696eac88f26b0117e54f52d3fc682ab19397a8812846a4"
[[package]]
name = "linux-raw-sys"
version = "0.3.8"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "ef53942eb7bf7ff43a617b3e2c1c4a5ecf5944a7c1bc12d7ee39bbb15e5c1519"
[[package]]
name = "linux-raw-sys"
version = "0.4.13"
@@ -3000,7 +2992,7 @@ checksum = "7c4b80445aeb08e832d87bf1830049a924cdc1d6b7ef40b6b9b365bff17bf8ec"
dependencies = [
"libc",
"measured",
"procfs 0.16.0",
"procfs",
]
[[package]]
@@ -3045,7 +3037,7 @@ dependencies = [
"measured",
"measured-process",
"once_cell",
"procfs 0.14.2",
"procfs",
"prometheus",
"rand 0.8.5",
"rand_distr",
@@ -3574,7 +3566,7 @@ dependencies = [
"humantime",
"humantime-serde",
"hyper 0.14.26",
"itertools",
"itertools 0.10.5",
"leaky-bucket",
"md5",
"metrics",
@@ -3592,8 +3584,9 @@ dependencies = [
"postgres_connection",
"postgres_ffi",
"pq_proto",
"procfs 0.14.2",
"procfs",
"rand 0.8.5",
"range-set-blaze",
"regex",
"remote_storage",
"reqwest 0.12.4",
@@ -3644,7 +3637,7 @@ dependencies = [
"hex",
"humantime",
"humantime-serde",
"itertools",
"itertools 0.10.5",
"postgres_ffi",
"rand 0.8.5",
"serde",
@@ -3702,7 +3695,7 @@ dependencies = [
"hex-literal",
"humantime",
"humantime-serde",
"itertools",
"itertools 0.10.5",
"metrics",
"once_cell",
"pageserver_api",
@@ -3967,7 +3960,7 @@ dependencies = [
[[package]]
name = "postgres"
version = "0.19.4"
source = "git+https://github.com/neondatabase/rust-postgres.git?branch=neon#20031d7a9ee1addeae6e0968e3899ae6bf01cee2"
source = "git+https://github.com/neondatabase/rust-postgres.git?branch=neon#cff6927e4f58b1af6ecc2ee7279df1f2ff537295"
dependencies = [
"bytes",
"fallible-iterator",
@@ -3980,7 +3973,7 @@ dependencies = [
[[package]]
name = "postgres-protocol"
version = "0.6.4"
source = "git+https://github.com/neondatabase/rust-postgres.git?branch=neon#20031d7a9ee1addeae6e0968e3899ae6bf01cee2"
source = "git+https://github.com/neondatabase/rust-postgres.git?branch=neon#cff6927e4f58b1af6ecc2ee7279df1f2ff537295"
dependencies = [
"base64 0.20.0",
"byteorder",
@@ -3999,7 +3992,7 @@ dependencies = [
[[package]]
name = "postgres-types"
version = "0.2.4"
source = "git+https://github.com/neondatabase/rust-postgres.git?branch=neon#20031d7a9ee1addeae6e0968e3899ae6bf01cee2"
source = "git+https://github.com/neondatabase/rust-postgres.git?branch=neon#cff6927e4f58b1af6ecc2ee7279df1f2ff537295"
dependencies = [
"bytes",
"fallible-iterator",
@@ -4034,7 +4027,7 @@ name = "postgres_connection"
version = "0.1.0"
dependencies = [
"anyhow",
"itertools",
"itertools 0.10.5",
"once_cell",
"postgres",
"tokio-postgres",
@@ -4092,7 +4085,7 @@ version = "0.1.0"
dependencies = [
"byteorder",
"bytes",
"itertools",
"itertools 0.10.5",
"pin-project-lite",
"postgres-protocol",
"rand 0.8.5",
@@ -4138,21 +4131,6 @@ dependencies = [
"unicode-ident",
]
[[package]]
name = "procfs"
version = "0.14.2"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "b1de8dacb0873f77e6aefc6d71e044761fcc68060290f5b1089fcdf84626bb69"
dependencies = [
"bitflags 1.3.2",
"byteorder",
"chrono",
"flate2",
"hex",
"lazy_static",
"rustix 0.36.16",
]
[[package]]
name = "procfs"
version = "0.16.0"
@@ -4160,10 +4138,12 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "731e0d9356b0c25f16f33b5be79b1c57b562f141ebfcdb0ad8ac2c13a24293b4"
dependencies = [
"bitflags 2.4.1",
"chrono",
"flate2",
"hex",
"lazy_static",
"procfs-core",
"rustix 0.38.28",
"rustix",
]
[[package]]
@@ -4173,14 +4153,15 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "2d3554923a69f4ce04c4a754260c338f505ce22642d3830e049a399fc2059a29"
dependencies = [
"bitflags 2.4.1",
"chrono",
"hex",
]
[[package]]
name = "prometheus"
version = "0.13.3"
version = "0.13.4"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "449811d15fbdf5ceb5c1144416066429cf82316e2ec8ce0c1f6f8a02e7bbcf8c"
checksum = "3d33c28a30771f7f96db69893f78b857f7450d7e0237e9c8fc6427a81bae7ed1"
dependencies = [
"cfg-if",
"fnv",
@@ -4188,7 +4169,7 @@ dependencies = [
"libc",
"memchr",
"parking_lot 0.12.1",
"procfs 0.14.2",
"procfs",
"thiserror",
]
@@ -4210,7 +4191,7 @@ checksum = "119533552c9a7ffacc21e099c24a0ac8bb19c2a2a3f363de84cd9b844feab270"
dependencies = [
"bytes",
"heck 0.4.1",
"itertools",
"itertools 0.10.5",
"lazy_static",
"log",
"multimap",
@@ -4231,7 +4212,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "e5d2d8d10f3c6ded6da8b05b5fb3b8a5082514344d56c9f871412d29b4e075b4"
dependencies = [
"anyhow",
"itertools",
"itertools 0.10.5",
"proc-macro2",
"quote",
"syn 1.0.109",
@@ -4288,7 +4269,7 @@ dependencies = [
"hyper-util",
"indexmap 2.0.1",
"ipnet",
"itertools",
"itertools 0.10.5",
"lasso",
"md5",
"measured",
@@ -4464,6 +4445,18 @@ dependencies = [
"rand_core 0.5.1",
]
[[package]]
name = "range-set-blaze"
version = "0.1.16"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "8421b5d459262eabbe49048d362897ff3e3830b44eac6cfe341d6acb2f0f13d2"
dependencies = [
"gen_ops",
"itertools 0.12.1",
"num-integer",
"num-traits",
]
[[package]]
name = "rayon"
version = "1.7.0"
@@ -4632,7 +4625,7 @@ dependencies = [
"humantime",
"humantime-serde",
"hyper 0.14.26",
"itertools",
"itertools 0.10.5",
"metrics",
"once_cell",
"pin-project-lite",
@@ -4942,34 +4935,6 @@ dependencies = [
"nom",
]
[[package]]
name = "rustix"
version = "0.36.16"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "6da3636faa25820d8648e0e31c5d519bbb01f72fdf57131f0f5f7da5fed36eab"
dependencies = [
"bitflags 1.3.2",
"errno",
"io-lifetimes",
"libc",
"linux-raw-sys 0.1.4",
"windows-sys 0.45.0",
]
[[package]]
name = "rustix"
version = "0.37.25"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "d4eb579851244c2c03e7c24f501c3432bed80b8f720af1d6e5b0e0f01555a035"
dependencies = [
"bitflags 1.3.2",
"errno",
"io-lifetimes",
"libc",
"linux-raw-sys 0.3.8",
"windows-sys 0.48.0",
]
[[package]]
name = "rustix"
version = "0.38.28"
@@ -5718,6 +5683,7 @@ dependencies = [
"aws-config",
"bytes",
"camino",
"chrono",
"clap",
"control_plane",
"diesel",
@@ -5728,7 +5694,7 @@ dependencies = [
"hex",
"humantime",
"hyper 0.14.26",
"itertools",
"itertools 0.10.5",
"lasso",
"measured",
"metrics",
@@ -5737,6 +5703,7 @@ dependencies = [
"pageserver_client",
"postgres_connection",
"r2d2",
"rand 0.8.5",
"reqwest 0.12.4",
"routerify",
"scopeguard",
@@ -5792,9 +5759,10 @@ dependencies = [
"either",
"futures",
"futures-util",
"git-version",
"hex",
"humantime",
"itertools",
"itertools 0.10.5",
"once_cell",
"pageserver",
"pageserver_api",
@@ -5971,15 +5939,15 @@ dependencies = [
[[package]]
name = "tempfile"
version = "3.5.0"
version = "3.9.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "b9fbec84f381d5795b08656e4912bec604d162bff9291d6189a78f4c8ab87998"
checksum = "01ce4141aa927a6d1bd34a041795abd0db1cccba5d5f24b009f694bdf3a1f3fa"
dependencies = [
"cfg-if",
"fastrand 1.9.0",
"redox_syscall 0.3.5",
"rustix 0.37.25",
"windows-sys 0.45.0",
"fastrand 2.0.0",
"redox_syscall 0.4.1",
"rustix",
"windows-sys 0.52.0",
]
[[package]]
@@ -6218,7 +6186,7 @@ dependencies = [
[[package]]
name = "tokio-postgres"
version = "0.7.7"
source = "git+https://github.com/neondatabase/rust-postgres.git?branch=neon#20031d7a9ee1addeae6e0968e3899ae6bf01cee2"
source = "git+https://github.com/neondatabase/rust-postgres.git?branch=neon#cff6927e4f58b1af6ecc2ee7279df1f2ff537295"
dependencies = [
"async-trait",
"byteorder",
@@ -6794,6 +6762,7 @@ dependencies = [
"serde_path_to_error",
"serde_with",
"signal-hook",
"smallvec",
"strum",
"strum_macros",
"thiserror",
@@ -7176,15 +7145,6 @@ dependencies = [
"windows_x86_64_msvc 0.42.2",
]
[[package]]
name = "windows-sys"
version = "0.45.0"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "75283be5efb2831d37ea142365f009c02ec203cd29a3ebecbc093d52315b66d0"
dependencies = [
"windows-targets 0.42.2",
]
[[package]]
name = "windows-sys"
version = "0.48.0"
@@ -7203,21 +7163,6 @@ dependencies = [
"windows-targets 0.52.4",
]
[[package]]
name = "windows-targets"
version = "0.42.2"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "8e5180c00cd44c9b1c88adb3693291f1cd93605ded80c250a75d472756b4d071"
dependencies = [
"windows_aarch64_gnullvm 0.42.2",
"windows_aarch64_msvc 0.42.2",
"windows_i686_gnu 0.42.2",
"windows_i686_msvc 0.42.2",
"windows_x86_64_gnu 0.42.2",
"windows_x86_64_gnullvm 0.42.2",
"windows_x86_64_msvc 0.42.2",
]
[[package]]
name = "windows-targets"
version = "0.48.0"
@@ -7447,7 +7392,7 @@ dependencies = [
"hmac",
"hyper 0.14.26",
"indexmap 1.9.3",
"itertools",
"itertools 0.10.5",
"libc",
"log",
"memchr",

View File

@@ -126,7 +126,7 @@ parquet = { version = "51.0.0", default-features = false, features = ["zstd"] }
parquet_derive = "51.0.0"
pbkdf2 = { version = "0.12.1", features = ["simple", "std"] }
pin-project-lite = "0.2"
procfs = "0.14"
procfs = "0.16"
prometheus = {version = "0.13", default-features=false, features = ["process"]} # removes protobuf dependency
prost = "0.11"
rand = "0.8"

View File

@@ -192,7 +192,7 @@ WORKDIR /home/nonroot
# Rust
# Please keep the version of llvm (installed above) in sync with rust llvm (`rustc --version --verbose | grep LLVM`)
ENV RUSTC_VERSION=1.79.0
ENV RUSTC_VERSION=1.80.0
ENV RUSTUP_HOME="/home/nonroot/.rustup"
ENV PATH="/home/nonroot/.cargo/bin:${PATH}"
RUN curl -sSO https://static.rust-lang.org/rustup/dist/$(uname -m)-unknown-linux-gnu/rustup-init && whoami && \

View File

@@ -657,7 +657,7 @@ RUN curl -sSO https://static.rust-lang.org/rustup/dist/$(uname -m)-unknown-linux
chmod +x rustup-init && \
./rustup-init -y --no-modify-path --profile minimal --default-toolchain stable && \
rm rustup-init && \
cargo install --locked --version 0.10.2 cargo-pgrx && \
cargo install --locked --version 0.11.3 cargo-pgrx && \
/bin/bash -c 'cargo pgrx init --pg${PG_VERSION:1}=/usr/local/pgsql/bin/pg_config'
USER root
@@ -672,10 +672,15 @@ USER root
FROM rust-extensions-build AS pg-jsonschema-pg-build
ARG PG_VERSION
RUN wget https://github.com/supabase/pg_jsonschema/archive/refs/tags/v0.2.0.tar.gz -O pg_jsonschema.tar.gz && \
echo "9118fc508a6e231e7a39acaa6f066fcd79af17a5db757b47d2eefbe14f7794f0 pg_jsonschema.tar.gz" | sha256sum --check && \
RUN wget https://github.com/supabase/pg_jsonschema/archive/refs/tags/v0.3.1.tar.gz -O pg_jsonschema.tar.gz && \
echo "61df3db1ed83cf24f6aa39c826f8818bfa4f0bd33b587fd6b2b1747985642297 pg_jsonschema.tar.gz" | sha256sum --check && \
mkdir pg_jsonschema-src && cd pg_jsonschema-src && tar xzf ../pg_jsonschema.tar.gz --strip-components=1 -C . && \
sed -i 's/pgrx = "0.10.2"/pgrx = { version = "0.10.2", features = [ "unsafe-postgres" ] }/g' Cargo.toml && \
# see commit 252b3685a27a0f4c31a0f91e983c6314838e89e8
# `unsafe-postgres` feature allows to build pgx extensions
# against postgres forks that decided to change their ABI name (like us).
# With that we can build extensions without forking them and using stock
# pgx. As this feature is new few manual version bumps were required.
sed -i 's/pgrx = "0.11.3"/pgrx = { version = "0.11.3", features = [ "unsafe-postgres" ] }/g' Cargo.toml && \
cargo pgrx install --release && \
echo "trusted = true" >> /usr/local/pgsql/share/extension/pg_jsonschema.control
@@ -689,10 +694,10 @@ RUN wget https://github.com/supabase/pg_jsonschema/archive/refs/tags/v0.2.0.tar.
FROM rust-extensions-build AS pg-graphql-pg-build
ARG PG_VERSION
RUN wget https://github.com/supabase/pg_graphql/archive/refs/tags/v1.4.0.tar.gz -O pg_graphql.tar.gz && \
echo "bd8dc7230282b3efa9ae5baf053a54151ed0e66881c7c53750e2d0c765776edc pg_graphql.tar.gz" | sha256sum --check && \
RUN wget https://github.com/supabase/pg_graphql/archive/refs/tags/v1.5.7.tar.gz -O pg_graphql.tar.gz && \
echo "2b3e567a5b31019cb97ae0e33263c1bcc28580be5a444ac4c8ece5c4be2aea41 pg_graphql.tar.gz" | sha256sum --check && \
mkdir pg_graphql-src && cd pg_graphql-src && tar xzf ../pg_graphql.tar.gz --strip-components=1 -C . && \
sed -i 's/pgrx = "=0.10.2"/pgrx = { version = "0.10.2", features = [ "unsafe-postgres" ] }/g' Cargo.toml && \
sed -i 's/pgrx = "=0.11.3"/pgrx = { version = "0.11.3", features = [ "unsafe-postgres" ] }/g' Cargo.toml && \
cargo pgrx install --release && \
# it's needed to enable extension because it uses untrusted C language
sed -i 's/superuser = false/superuser = true/g' /usr/local/pgsql/share/extension/pg_graphql.control && \
@@ -712,6 +717,9 @@ ARG PG_VERSION
RUN wget https://github.com/kelvich/pg_tiktoken/archive/26806147b17b60763039c6a6878884c41a262318.tar.gz -O pg_tiktoken.tar.gz && \
echo "e64e55aaa38c259512d3e27c572da22c4637418cf124caba904cd50944e5004e pg_tiktoken.tar.gz" | sha256sum --check && \
mkdir pg_tiktoken-src && cd pg_tiktoken-src && tar xzf ../pg_tiktoken.tar.gz --strip-components=1 -C . && \
# TODO update pgrx version in the pg_tiktoken repo and remove this line
sed -i 's/pgrx = { version = "=0.10.2",/pgrx = { version = "0.11.3",/g' Cargo.toml && \
sed -i 's/pgrx-tests = "=0.10.2"/pgrx-tests = "0.11.3"/g' Cargo.toml && \
cargo pgrx install --release && \
echo "trusted = true" >> /usr/local/pgsql/share/extension/pg_tiktoken.control
@@ -725,14 +733,10 @@ RUN wget https://github.com/kelvich/pg_tiktoken/archive/26806147b17b60763039c6a6
FROM rust-extensions-build AS pg-pgx-ulid-build
ARG PG_VERSION
RUN wget https://github.com/pksunkara/pgx_ulid/archive/refs/tags/v0.1.3.tar.gz -O pgx_ulid.tar.gz && \
echo "ee5db82945d2d9f2d15597a80cf32de9dca67b897f605beb830561705f12683c pgx_ulid.tar.gz" | sha256sum --check && \
RUN wget https://github.com/pksunkara/pgx_ulid/archive/refs/tags/v0.1.5.tar.gz -O pgx_ulid.tar.gz && \
echo "9d1659a2da65af0133d5451c454de31b37364e3502087dadf579f790bc8bef17 pgx_ulid.tar.gz" | sha256sum --check && \
mkdir pgx_ulid-src && cd pgx_ulid-src && tar xzf ../pgx_ulid.tar.gz --strip-components=1 -C . && \
echo "******************* Apply a patch for Postgres 16 support; delete in the next release ******************" && \
wget https://github.com/pksunkara/pgx_ulid/commit/f84954cf63fc8c80d964ac970d9eceed3c791196.patch && \
patch -p1 < f84954cf63fc8c80d964ac970d9eceed3c791196.patch && \
echo "********************************************************************************************************" && \
sed -i 's/pgrx = "=0.10.2"/pgrx = { version = "=0.10.2", features = [ "unsafe-postgres" ] }/g' Cargo.toml && \
sed -i 's/pgrx = "^0.11.2"/pgrx = { version = "=0.11.3", features = [ "unsafe-postgres" ] }/g' Cargo.toml && \
cargo pgrx install --release && \
echo "trusted = true" >> /usr/local/pgsql/share/extension/ulid.control

View File

@@ -4,6 +4,11 @@ version = "0.1.0"
edition.workspace = true
license.workspace = true
[features]
default = []
# Enables test specific features.
testing = []
[dependencies]
anyhow.workspace = true
async-compression.workspace = true

View File

@@ -400,7 +400,15 @@ impl ComputeNode {
pub fn get_basebackup(&self, compute_state: &ComputeState, lsn: Lsn) -> Result<()> {
let mut retry_period_ms = 500.0;
let mut attempts = 0;
let max_attempts = 10;
const DEFAULT_ATTEMPTS: u16 = 10;
#[cfg(feature = "testing")]
let max_attempts = if let Ok(v) = env::var("NEON_COMPUTE_TESTING_BASEBACKUP_RETRIES") {
u16::from_str(&v).unwrap()
} else {
DEFAULT_ATTEMPTS
};
#[cfg(not(feature = "testing"))]
let max_attempts = DEFAULT_ATTEMPTS;
loop {
let result = self.try_get_basebackup(compute_state, lsn);
match result {

View File

@@ -289,7 +289,7 @@ fn fill_remote_storage_secrets_vars(mut cmd: &mut Command) -> &mut Command {
fn fill_env_vars_prefixed_neon(mut cmd: &mut Command) -> &mut Command {
for (var, val) in std::env::vars() {
if var.starts_with("NEON_PAGESERVER_") {
if var.starts_with("NEON_") {
cmd = cmd.env(var, val);
}
}

View File

@@ -21,7 +21,9 @@ use pageserver_api::config::{
DEFAULT_HTTP_LISTEN_PORT as DEFAULT_PAGESERVER_HTTP_PORT,
DEFAULT_PG_LISTEN_PORT as DEFAULT_PAGESERVER_PG_PORT,
};
use pageserver_api::controller_api::{PlacementPolicy, TenantCreateRequest};
use pageserver_api::controller_api::{
NodeAvailabilityWrapper, PlacementPolicy, TenantCreateRequest,
};
use pageserver_api::models::{ShardParameters, TimelineCreateRequest, TimelineInfo};
use pageserver_api::shard::{ShardCount, ShardStripeSize, TenantShardId};
use postgres_backend::AuthType;
@@ -1250,9 +1252,70 @@ async fn handle_start_all(
exit(1);
}
}
neon_start_status_check(env, retry_timeout).await?;
Ok(())
}
async fn neon_start_status_check(
env: &local_env::LocalEnv,
retry_timeout: &Duration,
) -> anyhow::Result<()> {
const RETRY_INTERVAL: Duration = Duration::from_millis(100);
const NOTICE_AFTER_RETRIES: Duration = Duration::from_secs(5);
if env.control_plane_api.is_none() {
return Ok(());
}
let storcon = StorageController::from_env(env);
let retries = retry_timeout.as_millis() / RETRY_INTERVAL.as_millis();
let notice_after_retries = retry_timeout.as_millis() / NOTICE_AFTER_RETRIES.as_millis();
println!("\nRunning neon status check");
for retry in 0..retries {
if retry == notice_after_retries {
println!("\nNeon status check has not passed yet, continuing to wait")
}
let mut passed = true;
let mut nodes = storcon.node_list().await?;
let mut pageservers = env.pageservers.clone();
if nodes.len() != pageservers.len() {
continue;
}
nodes.sort_by_key(|ps| ps.id);
pageservers.sort_by_key(|ps| ps.id);
for (idx, pageserver) in pageservers.iter().enumerate() {
let node = &nodes[idx];
if node.id != pageserver.id {
passed = false;
break;
}
if !matches!(node.availability, NodeAvailabilityWrapper::Active) {
passed = false;
break;
}
}
if passed {
println!("\nNeon started and passed status check");
return Ok(());
}
tokio::time::sleep(RETRY_INTERVAL).await;
}
anyhow::bail!("\nNeon passed status check")
}
async fn handle_stop_all(sub_match: &ArgMatches, env: &local_env::LocalEnv) -> Result<()> {
let immediate =
sub_match.get_one::<String>("stop-mode").map(|s| s.as_str()) == Some("immediate");

View File

@@ -151,7 +151,10 @@ pub struct NeonBroker {
pub struct NeonStorageControllerConf {
/// Heartbeat timeout before marking a node offline
#[serde(with = "humantime_serde")]
pub max_unavailable: Duration,
pub max_offline: Duration,
#[serde(with = "humantime_serde")]
pub max_warming_up: Duration,
/// Threshold for auto-splitting a tenant into shards
pub split_threshold: Option<u64>,
@@ -159,14 +162,16 @@ pub struct NeonStorageControllerConf {
impl NeonStorageControllerConf {
// Use a shorter pageserver unavailability interval than the default to speed up tests.
const DEFAULT_MAX_UNAVAILABLE_INTERVAL: std::time::Duration =
std::time::Duration::from_secs(10);
const DEFAULT_MAX_OFFLINE_INTERVAL: std::time::Duration = std::time::Duration::from_secs(10);
const DEFAULT_MAX_WARMING_UP_INTERVAL: std::time::Duration = std::time::Duration::from_secs(30);
}
impl Default for NeonStorageControllerConf {
fn default() -> Self {
Self {
max_unavailable: Self::DEFAULT_MAX_UNAVAILABLE_INTERVAL,
max_offline: Self::DEFAULT_MAX_OFFLINE_INTERVAL,
max_warming_up: Self::DEFAULT_MAX_WARMING_UP_INTERVAL,
split_threshold: None,
}
}
@@ -509,7 +514,6 @@ impl LocalEnv {
#[derive(serde::Serialize, serde::Deserialize)]
// (allow unknown fields, unlike PageServerConf)
struct PageserverConfigTomlSubset {
id: NodeId,
listen_pg_addr: String,
listen_http_addr: String,
pg_auth_type: AuthType,
@@ -521,18 +525,30 @@ impl LocalEnv {
.with_context(|| format!("read {:?}", config_toml_path))?,
)
.context("parse pageserver.toml")?;
let identity_toml_path = dentry.path().join("identity.toml");
#[derive(serde::Serialize, serde::Deserialize)]
struct IdentityTomlSubset {
id: NodeId,
}
let identity_toml: IdentityTomlSubset = toml_edit::de::from_str(
&std::fs::read_to_string(&identity_toml_path)
.with_context(|| format!("read {:?}", identity_toml_path))?,
)
.context("parse identity.toml")?;
let PageserverConfigTomlSubset {
id: config_toml_id,
listen_pg_addr,
listen_http_addr,
pg_auth_type,
http_auth_type,
} = config_toml;
let IdentityTomlSubset {
id: identity_toml_id,
} = identity_toml;
let conf = PageServerConf {
id: {
anyhow::ensure!(
config_toml_id == id,
"id mismatch: config_toml.id={config_toml_id} id={id}",
identity_toml_id == id,
"id mismatch: identity.toml:id={identity_toml_id} pageserver_(.*) id={id}",
);
id
},

View File

@@ -127,10 +127,13 @@ impl PageServerNode {
}
// Apply the user-provided overrides
overrides.push(
toml_edit::ser::to_string_pretty(&conf)
.expect("we deserialized this from toml earlier"),
);
overrides.push({
let mut doc =
toml_edit::ser::to_document(&conf).expect("we deserialized this from toml earlier");
// `id` is written out to `identity.toml` instead of `pageserver.toml`
doc.remove("id").expect("it's part of the struct");
doc.to_string()
});
// Turn `overrides` into a toml document.
// TODO: above code is legacy code, it should be refactored to use toml_edit directly.

View File

@@ -5,8 +5,9 @@ use crate::{
use camino::{Utf8Path, Utf8PathBuf};
use pageserver_api::{
controller_api::{
NodeConfigureRequest, NodeRegisterRequest, TenantCreateRequest, TenantCreateResponse,
TenantLocateResponse, TenantShardMigrateRequest, TenantShardMigrateResponse,
NodeConfigureRequest, NodeDescribeResponse, NodeRegisterRequest, TenantCreateRequest,
TenantCreateResponse, TenantLocateResponse, TenantShardMigrateRequest,
TenantShardMigrateResponse,
},
models::{
TenantShardSplitRequest, TenantShardSplitResponse, TimelineCreateRequest, TimelineInfo,
@@ -353,8 +354,10 @@ impl StorageController {
"--dev",
"--database-url",
&database_url,
"--max-unavailable-interval",
&humantime::Duration::from(self.config.max_unavailable).to_string(),
"--max-offline-interval",
&humantime::Duration::from(self.config.max_offline).to_string(),
"--max-warming-up-interval",
&humantime::Duration::from(self.config.max_warming_up).to_string(),
]
.into_iter()
.map(|s| s.to_string())
@@ -625,6 +628,15 @@ impl StorageController {
.await
}
pub async fn node_list(&self) -> anyhow::Result<Vec<NodeDescribeResponse>> {
self.dispatch::<(), Vec<NodeDescribeResponse>>(
Method::GET,
"control/v1/node".to_string(),
None,
)
.await
}
#[instrument(skip(self))]
pub async fn ready(&self) -> anyhow::Result<()> {
self.dispatch::<(), ()>(Method::GET, "ready".to_string(), None)

View File

@@ -21,9 +21,9 @@ implementation where we keep more data than we would need to, do not
change the synthetic size or incur any costs to the user.
The synthetic size is calculated for the whole project. It is not
straightforward to attribute size to individual branches. See "What is
the size of an individual branch?" for discussion on those
difficulties.
straightforward to attribute size to individual branches. See [What is
the size of an individual branch?](#what-is-the-size-of-an-individual-branch)
for a discussion of those difficulties.
The synthetic size is designed to:
@@ -40,8 +40,9 @@ The synthetic size is designed to:
- logical size is the size of a branch *at a given point in
time*. It's the total size of all tables in all databases, as you
see with "\l+" in psql for example, plus the Postgres SLRUs and some
small amount of metadata. NOTE that currently, Neon does not include
the SLRUs and metadata in the logical size. See comment to `get_current_logical_size_non_incremental()`.
small amount of metadata. Note that currently, Neon does not include
the SLRUs and metadata in the logical size. Refer to the comment in
[`get_current_logical_size_non_incremental()`](/pageserver/src/pgdatadir_mapping.rs#L813-L814).
- a "point in time" is defined as an LSN value. You can convert a
timestamp to an LSN, but the storage internally works with LSNs.

View File

@@ -1,4 +1,6 @@
use std::collections::HashSet;
use std::str::FromStr;
use std::time::{Duration, Instant};
/// Request/response types for the storage controller
/// API (`/control/v1` prefix). Implemented by the server
@@ -150,11 +152,16 @@ impl UtilizationScore {
}
}
#[derive(Serialize, Deserialize, Clone, Copy, Debug)]
#[derive(Serialize, Clone, Copy, Debug)]
#[serde(into = "NodeAvailabilityWrapper")]
pub enum NodeAvailability {
// Normal, happy state
Active(UtilizationScore),
// Node is warming up, but we expect it to become available soon. Covers
// the time span between the re-attach response being composed on the storage controller
// and the first successful heartbeat after the processing of the re-attach response
// finishes on the pageserver.
WarmingUp(Instant),
// Offline: Tenants shouldn't try to attach here, but they may assume that their
// secondary locations on this node still exist. Newly added nodes are in this
// state until we successfully contact them.
@@ -164,7 +171,10 @@ pub enum NodeAvailability {
impl PartialEq for NodeAvailability {
fn eq(&self, other: &Self) -> bool {
use NodeAvailability::*;
matches!((self, other), (Active(_), Active(_)) | (Offline, Offline))
matches!(
(self, other),
(Active(_), Active(_)) | (Offline, Offline) | (WarmingUp(_), WarmingUp(_))
)
}
}
@@ -176,6 +186,7 @@ impl Eq for NodeAvailability {}
#[derive(Serialize, Deserialize, Clone, Copy, Debug)]
pub enum NodeAvailabilityWrapper {
Active,
WarmingUp,
Offline,
}
@@ -185,6 +196,7 @@ impl From<NodeAvailabilityWrapper> for NodeAvailability {
// Assume the worst utilisation score to begin with. It will later be updated by
// the heartbeats.
NodeAvailabilityWrapper::Active => NodeAvailability::Active(UtilizationScore::worst()),
NodeAvailabilityWrapper::WarmingUp => NodeAvailability::WarmingUp(Instant::now()),
NodeAvailabilityWrapper::Offline => NodeAvailability::Offline,
}
}
@@ -194,6 +206,7 @@ impl From<NodeAvailability> for NodeAvailabilityWrapper {
fn from(val: NodeAvailability) -> Self {
match val {
NodeAvailability::Active(_) => NodeAvailabilityWrapper::Active,
NodeAvailability::WarmingUp(_) => NodeAvailabilityWrapper::WarmingUp,
NodeAvailability::Offline => NodeAvailabilityWrapper::Offline,
}
}
@@ -282,6 +295,42 @@ pub enum PlacementPolicy {
#[derive(Serialize, Deserialize, Debug)]
pub struct TenantShardMigrateResponse {}
/// Metadata health record posted from scrubber.
#[derive(Serialize, Deserialize, Debug)]
pub struct MetadataHealthRecord {
pub tenant_shard_id: TenantShardId,
pub healthy: bool,
pub last_scrubbed_at: chrono::DateTime<chrono::Utc>,
}
#[derive(Serialize, Deserialize, Debug)]
pub struct MetadataHealthUpdateRequest {
pub healthy_tenant_shards: HashSet<TenantShardId>,
pub unhealthy_tenant_shards: HashSet<TenantShardId>,
}
#[derive(Serialize, Deserialize, Debug)]
pub struct MetadataHealthUpdateResponse {}
#[derive(Serialize, Deserialize, Debug)]
pub struct MetadataHealthListUnhealthyResponse {
pub unhealthy_tenant_shards: Vec<TenantShardId>,
}
#[derive(Serialize, Deserialize, Debug)]
pub struct MetadataHealthListOutdatedRequest {
#[serde(with = "humantime_serde")]
pub not_scrubbed_for: Duration,
}
#[derive(Serialize, Deserialize, Debug)]
pub struct MetadataHealthListOutdatedResponse {
pub health_records: Vec<MetadataHealthRecord>,
}
#[cfg(test)]
mod test {
use super::*;

View File

@@ -144,20 +144,7 @@ impl PgConnectionConfig {
// implement and this function is hardly a bottleneck. The function is only called around
// establishing a new connection.
#[allow(unstable_name_collisions)]
config.options(
&self
.options
.iter()
.map(|s| {
if s.contains(['\\', ' ']) {
Cow::Owned(s.replace('\\', "\\\\").replace(' ', "\\ "))
} else {
Cow::Borrowed(s.as_str())
}
})
.intersperse(Cow::Borrowed(" ")) // TODO: use impl from std once it's stabilized
.collect::<String>(),
);
config.options(&encode_options(&self.options));
}
config
}
@@ -178,6 +165,21 @@ impl PgConnectionConfig {
}
}
#[allow(unstable_name_collisions)]
fn encode_options(options: &[String]) -> String {
options
.iter()
.map(|s| {
if s.contains(['\\', ' ']) {
Cow::Owned(s.replace('\\', "\\\\").replace(' ', "\\ "))
} else {
Cow::Borrowed(s.as_str())
}
})
.intersperse(Cow::Borrowed(" ")) // TODO: use impl from std once it's stabilized
.collect::<String>()
}
impl fmt::Display for PgConnectionConfig {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
// The password is intentionally hidden and not part of this display string.
@@ -206,7 +208,7 @@ impl fmt::Debug for PgConnectionConfig {
#[cfg(test)]
mod tests_pg_connection_config {
use crate::PgConnectionConfig;
use crate::{encode_options, PgConnectionConfig};
use once_cell::sync::Lazy;
use url::Host;
@@ -255,18 +257,12 @@ mod tests_pg_connection_config {
#[test]
fn test_with_options() {
let cfg = PgConnectionConfig::new_host_port(STUB_HOST.clone(), 123).extend_options([
"hello",
"world",
"with space",
"and \\ backslashes",
let options = encode_options(&[
"hello".to_owned(),
"world".to_owned(),
"with space".to_owned(),
"and \\ backslashes".to_owned(),
]);
assert_eq!(cfg.host(), &*STUB_HOST);
assert_eq!(cfg.port(), 123);
assert_eq!(cfg.raw_address(), "stub.host.example:123");
assert_eq!(
cfg.to_tokio_postgres_config().get_options(),
Some("hello world with\\ space and\\ \\\\\\ backslashes")
);
assert_eq!(options, "hello world with\\ space and\\ \\\\\\ backslashes");
}
}

View File

@@ -29,7 +29,7 @@ use anyhow::{bail, Result};
use bytes::{Bytes, BytesMut};
/// Equivalent to sizeof(ControlFileData) in C
const SIZEOF_CONTROLDATA: usize = std::mem::size_of::<ControlFileData>();
const SIZEOF_CONTROLDATA: usize = size_of::<ControlFileData>();
impl ControlFileData {
/// Compute the offset of the `crc` field within the `ControlFileData` struct.

View File

@@ -31,7 +31,7 @@ pub const SMGR_TRUNCATE_FSM: u32 = 0x0004;
//
// Assumes 8 byte alignment
const SIZEOF_PAGE_HEADER_DATA: usize = std::mem::size_of::<PageHeaderData>();
const SIZEOF_PAGE_HEADER_DATA: usize = size_of::<PageHeaderData>();
pub const MAXALIGN_SIZE_OF_PAGE_HEADER_DATA: usize = (SIZEOF_PAGE_HEADER_DATA + 7) & !7;
//
@@ -191,7 +191,7 @@ pub const XLR_RMGR_INFO_MASK: u8 = 0xF0;
pub const XLOG_TBLSPC_CREATE: u8 = 0x00;
pub const XLOG_TBLSPC_DROP: u8 = 0x10;
pub const SIZEOF_XLOGRECORD: u32 = std::mem::size_of::<XLogRecord>() as u32;
pub const SIZEOF_XLOGRECORD: u32 = size_of::<XLogRecord>() as u32;
//
// from xlogrecord.h

View File

@@ -42,9 +42,9 @@ pub const XLP_FIRST_IS_CONTRECORD: u16 = 0x0001;
pub const XLP_REM_LEN_OFFS: usize = 2 + 2 + 4 + 8;
pub const XLOG_RECORD_CRC_OFFS: usize = 4 + 4 + 8 + 1 + 1 + 2;
pub const XLOG_SIZE_OF_XLOG_SHORT_PHD: usize = std::mem::size_of::<XLogPageHeaderData>();
pub const XLOG_SIZE_OF_XLOG_LONG_PHD: usize = std::mem::size_of::<XLogLongPageHeaderData>();
pub const XLOG_SIZE_OF_XLOG_RECORD: usize = std::mem::size_of::<XLogRecord>();
pub const XLOG_SIZE_OF_XLOG_SHORT_PHD: usize = size_of::<XLogPageHeaderData>();
pub const XLOG_SIZE_OF_XLOG_LONG_PHD: usize = size_of::<XLogLongPageHeaderData>();
pub const XLOG_SIZE_OF_XLOG_RECORD: usize = size_of::<XLogRecord>();
#[allow(clippy::identity_op)]
pub const SIZE_OF_XLOG_RECORD_DATA_HEADER_SHORT: usize = 1 * 2;
@@ -311,7 +311,7 @@ impl XLogLongPageHeaderData {
}
}
pub const SIZEOF_CHECKPOINT: usize = std::mem::size_of::<CheckPoint>();
pub const SIZEOF_CHECKPOINT: usize = size_of::<CheckPoint>();
impl CheckPoint {
pub fn encode(&self) -> Result<Bytes, SerializeError> {

View File

@@ -178,7 +178,7 @@ pub fn test_find_end_of_wal_last_crossing_segment() {
/// currently 1024.
#[test]
pub fn test_update_next_xid() {
let checkpoint_buf = [0u8; std::mem::size_of::<CheckPoint>()];
let checkpoint_buf = [0u8; size_of::<CheckPoint>()];
let mut checkpoint = CheckPoint::decode(&checkpoint_buf).unwrap();
checkpoint.nextXid = FullTransactionId { value: 10 };
@@ -204,7 +204,7 @@ pub fn test_update_next_xid() {
#[test]
pub fn test_update_next_multixid() {
let checkpoint_buf = [0u8; std::mem::size_of::<CheckPoint>()];
let checkpoint_buf = [0u8; size_of::<CheckPoint>()];
let mut checkpoint = CheckPoint::decode(&checkpoint_buf).unwrap();
// simple case

View File

@@ -33,6 +33,7 @@ use tracing::debug;
use utils::backoff;
use crate::metrics::{start_measuring_requests, AttemptOutcome, RequestKind};
use crate::ListingObject;
use crate::{
config::AzureConfig, error::Cancelled, ConcurrencyLimiter, Download, DownloadError, Listing,
ListingMode, RemotePath, RemoteStorage, StorageMetadata, TimeTravelError, TimeoutOrCancel,
@@ -352,7 +353,12 @@ impl RemoteStorage for AzureBlobStorage {
let blob_iter = entry
.blobs
.blobs()
.map(|k| self.name_to_relative_path(&k.name));
.map(|k| ListingObject{
key: self.name_to_relative_path(&k.name),
last_modified: k.properties.last_modified.into(),
size: k.properties.content_length,
}
);
for key in blob_iter {
res.keys.push(key);

View File

@@ -144,15 +144,23 @@ impl RemotePath {
///
/// The WithDelimiter mode will populate `prefixes` and `keys` in the result. The
/// NoDelimiter mode will only populate `keys`.
#[derive(Copy, Clone)]
pub enum ListingMode {
WithDelimiter,
NoDelimiter,
}
#[derive(PartialEq, Eq, Debug)]
pub struct ListingObject {
pub key: RemotePath,
pub last_modified: SystemTime,
pub size: u64,
}
#[derive(Default)]
pub struct Listing {
pub prefixes: Vec<RemotePath>,
pub keys: Vec<RemotePath>,
pub keys: Vec<ListingObject>,
}
/// Storage (potentially remote) API to manage its state.
@@ -188,7 +196,7 @@ pub trait RemoteStorage: Send + Sync + 'static {
mode: ListingMode,
max_keys: Option<NonZeroU32>,
cancel: &CancellationToken,
) -> impl Stream<Item = Result<Listing, DownloadError>>;
) -> impl Stream<Item = Result<Listing, DownloadError>> + Send;
async fn list(
&self,
@@ -201,7 +209,7 @@ pub trait RemoteStorage: Send + Sync + 'static {
let mut combined = stream.next().await.expect("At least one item required")?;
while let Some(list) = stream.next().await {
let list = list?;
combined.keys.extend_from_slice(&list.keys);
combined.keys.extend(list.keys.into_iter());
combined.prefixes.extend_from_slice(&list.prefixes);
}
Ok(combined)
@@ -345,10 +353,10 @@ impl<Other: RemoteStorage> GenericRemoteStorage<Arc<Other>> {
mode: ListingMode,
max_keys: Option<NonZeroU32>,
cancel: &'a CancellationToken,
) -> impl Stream<Item = Result<Listing, DownloadError>> + 'a {
) -> impl Stream<Item = Result<Listing, DownloadError>> + 'a + Send {
match self {
Self::LocalFs(s) => Box::pin(s.list_streaming(prefix, mode, max_keys, cancel))
as Pin<Box<dyn Stream<Item = Result<Listing, DownloadError>>>>,
as Pin<Box<dyn Stream<Item = Result<Listing, DownloadError>> + Send>>,
Self::AwsS3(s) => Box::pin(s.list_streaming(prefix, mode, max_keys, cancel)),
Self::AzureBlob(s) => Box::pin(s.list_streaming(prefix, mode, max_keys, cancel)),
Self::Unreliable(s) => Box::pin(s.list_streaming(prefix, mode, max_keys, cancel)),

View File

@@ -23,8 +23,8 @@ use tokio_util::{io::ReaderStream, sync::CancellationToken};
use utils::crashsafe::path_with_suffix_extension;
use crate::{
Download, DownloadError, Listing, ListingMode, RemotePath, TimeTravelError, TimeoutOrCancel,
REMOTE_STORAGE_PREFIX_SEPARATOR,
Download, DownloadError, Listing, ListingMode, ListingObject, RemotePath, TimeTravelError,
TimeoutOrCancel, REMOTE_STORAGE_PREFIX_SEPARATOR,
};
use super::{RemoteStorage, StorageMetadata};
@@ -357,19 +357,29 @@ impl RemoteStorage for LocalFs {
.list_recursive(prefix)
.await
.map_err(DownloadError::Other)?;
let keys = keys
let objects = keys
.into_iter()
.filter(|k| {
.filter_map(|k| {
let path = k.with_base(&self.storage_root);
!path.is_dir()
if path.is_dir() {
None
} else {
Some(ListingObject {
key: k.clone(),
// LocalFs is just for testing, so just specify a dummy time
last_modified: SystemTime::now(),
size: 0,
})
}
})
.collect();
if let ListingMode::NoDelimiter = mode {
result.keys = keys;
result.keys = objects;
} else {
let mut prefixes = HashSet::new();
for key in keys {
for object in objects {
let key = object.key;
// If the part after the prefix includes a "/", take only the first part and put it in `prefixes`.
let relative_key = if let Some(prefix) = prefix {
let mut prefix = prefix.clone();
@@ -398,9 +408,12 @@ impl RemoteStorage for LocalFs {
.to_owned();
prefixes.insert(first_part);
} else {
result
.keys
.push(RemotePath::from_string(&relative_key).unwrap());
result.keys.push(ListingObject {
key: RemotePath::from_string(&relative_key).unwrap(),
// LocalFs is just for testing
last_modified: SystemTime::now(),
size: 0,
});
}
}
result.prefixes = prefixes
@@ -950,7 +963,11 @@ mod fs_tests {
.await?;
assert!(listing.prefixes.is_empty());
assert_eq!(
listing.keys.into_iter().collect::<HashSet<_>>(),
listing
.keys
.into_iter()
.map(|o| o.key)
.collect::<HashSet<_>>(),
HashSet::from([uncle.clone(), child.clone(), child_sibling.clone()])
);
@@ -975,7 +992,7 @@ mod fs_tests {
)
.await?;
assert_eq!(
listing.keys,
listing.keys.into_iter().map(|o| o.key).collect::<Vec<_>>(),
[RemotePath::from_string("uncle").unwrap()].to_vec()
);
assert_eq!(
@@ -992,7 +1009,7 @@ mod fs_tests {
&cancel,
)
.await?;
assert_eq!(listing.keys, [].to_vec());
assert_eq!(listing.keys, vec![]);
assert_eq!(
listing.prefixes,
[RemotePath::from_string("grandparent").unwrap()].to_vec()
@@ -1007,7 +1024,7 @@ mod fs_tests {
&cancel,
)
.await?;
assert_eq!(listing.keys, [].to_vec());
assert_eq!(listing.keys, vec![]);
assert_eq!(
listing.prefixes,
[RemotePath::from_string("grandparent").unwrap()].to_vec()
@@ -1040,7 +1057,7 @@ mod fs_tests {
&cancel,
)
.await?;
assert_eq!(listing.keys, [].to_vec());
assert_eq!(listing.keys, vec![]);
let mut found_prefixes = listing.prefixes.clone();
found_prefixes.sort();

View File

@@ -44,8 +44,9 @@ use crate::{
error::Cancelled,
metrics::{start_counting_cancelled_wait, start_measuring_requests},
support::PermitCarrying,
ConcurrencyLimiter, Download, DownloadError, Listing, ListingMode, RemotePath, RemoteStorage,
TimeTravelError, TimeoutOrCancel, MAX_KEYS_PER_DELETE, REMOTE_STORAGE_PREFIX_SEPARATOR,
ConcurrencyLimiter, Download, DownloadError, Listing, ListingMode, ListingObject, RemotePath,
RemoteStorage, TimeTravelError, TimeoutOrCancel, MAX_KEYS_PER_DELETE,
REMOTE_STORAGE_PREFIX_SEPARATOR,
};
use crate::metrics::AttemptOutcome;
@@ -548,9 +549,29 @@ impl RemoteStorage for S3Bucket {
let mut result = Listing::default();
for object in keys {
let object_path = object.key().expect("response does not contain a key");
let remote_path = self.s3_object_to_relative_path(object_path);
result.keys.push(remote_path);
let key = object.key().expect("response does not contain a key");
let key = self.s3_object_to_relative_path(key);
let last_modified = match object.last_modified.map(SystemTime::try_from) {
Some(Ok(t)) => t,
Some(Err(_)) => {
tracing::warn!("Remote storage last_modified {:?} for {} is out of bounds",
object.last_modified, key
);
SystemTime::now()
},
None => {
SystemTime::now()
}
};
let size = object.size.unwrap_or(0) as u64;
result.keys.push(ListingObject{
key,
last_modified,
size,
});
if let Some(mut mk) = max_keys {
assert!(mk > 0);
mk -= 1;

View File

@@ -114,7 +114,7 @@ impl RemoteStorage for UnreliableWrapper {
mode: ListingMode,
max_keys: Option<NonZeroU32>,
cancel: &CancellationToken,
) -> impl Stream<Item = Result<Listing, DownloadError>> {
) -> impl Stream<Item = Result<Listing, DownloadError>> + Send {
async_stream::stream! {
self.attempt(RemoteOp::ListPrefixes(prefix.cloned()))
.map_err(DownloadError::Other)?;

View File

@@ -156,6 +156,7 @@ async fn list_no_delimiter_works(
.context("client list root files failure")?
.keys
.into_iter()
.map(|o| o.key)
.collect::<HashSet<_>>();
assert_eq!(
root_files,
@@ -182,6 +183,7 @@ async fn list_no_delimiter_works(
.context("client list nested files failure")?
.keys
.into_iter()
.map(|o| o.key)
.collect::<HashSet<_>>();
let trim_remote_blobs: HashSet<_> = ctx
.remote_blobs

View File

@@ -81,6 +81,7 @@ async fn s3_time_travel_recovery_works(ctx: &mut MaybeEnabledStorage) -> anyhow:
.context("list root files failure")?
.keys
.into_iter()
.map(|o| o.key)
.collect::<HashSet<_>>(),
)
}

View File

@@ -35,6 +35,7 @@ routerify.workspace = true
serde.workspace = true
serde_json.workspace = true
signal-hook.workspace = true
smallvec.workspace = true
thiserror.workspace = true
tokio.workspace = true
tokio-tar.workspace = true

View File

@@ -18,20 +18,20 @@ const STORAGE_TOKEN_ALGORITHM: Algorithm = Algorithm::EdDSA;
#[derive(Debug, Serialize, Deserialize, Clone, Copy, PartialEq)]
#[serde(rename_all = "lowercase")]
pub enum Scope {
// Provides access to all data for a specific tenant (specified in `struct Claims` below)
/// Provides access to all data for a specific tenant (specified in `struct Claims` below)
// TODO: join these two?
Tenant,
// Provides blanket access to all tenants on the pageserver plus pageserver-wide APIs.
// Should only be used e.g. for status check/tenant creation/list.
/// Provides blanket access to all tenants on the pageserver plus pageserver-wide APIs.
/// Should only be used e.g. for status check/tenant creation/list.
PageServerApi,
// Provides blanket access to all data on the safekeeper plus safekeeper-wide APIs.
// Should only be used e.g. for status check.
// Currently also used for connection from any pageserver to any safekeeper.
/// Provides blanket access to all data on the safekeeper plus safekeeper-wide APIs.
/// Should only be used e.g. for status check.
/// Currently also used for connection from any pageserver to any safekeeper.
SafekeeperData,
// The scope used by pageservers in upcalls to storage controller and cloud control plane
/// The scope used by pageservers in upcalls to storage controller and cloud control plane
#[serde(rename = "generations_api")]
GenerationsApi,
// Allows access to control plane managment API and some storage controller endpoints.
/// Allows access to control plane managment API and some storage controller endpoints.
Admin,
/// Allows access to storage controller APIs used by the scrubber, to interrogate the state

View File

@@ -1,11 +1,15 @@
use std::{alloc::Layout, cmp::Ordering, ops::RangeBounds};
use smallvec::SmallVec;
#[derive(Clone, Copy, Debug, PartialEq, Eq)]
pub enum VecMapOrdering {
Greater,
GreaterOrEqual,
}
const INLINE_ELEMENTS: usize = 1;
/// Ordered map datastructure implemented in a Vec.
/// Append only - can only add keys that are larger than the
/// current max key.
@@ -13,7 +17,7 @@ pub enum VecMapOrdering {
/// during `VecMap` construction.
#[derive(Clone, Debug)]
pub struct VecMap<K, V> {
data: Vec<(K, V)>,
data: SmallVec<[(K, V); INLINE_ELEMENTS]>,
ordering: VecMapOrdering,
}
@@ -37,14 +41,14 @@ pub enum VecMapError {
impl<K: Ord, V> VecMap<K, V> {
pub fn new(ordering: VecMapOrdering) -> Self {
Self {
data: Vec::new(),
data: Default::default(),
ordering,
}
}
pub fn with_capacity(capacity: usize, ordering: VecMapOrdering) -> Self {
Self {
data: Vec::with_capacity(capacity),
data: SmallVec::with_capacity(capacity),
ordering,
}
}
@@ -95,6 +99,10 @@ impl<K: Ord, V> VecMap<K, V> {
Ok(delta_size)
}
pub fn append_fast(&mut self, key: K, value: V) {
self.data.push((key, value))
}
/// Update the maximum key value pair or add a new key value pair to the map.
/// If `key` is not respective of the `self` ordering no updates or additions
/// will occur and `InvalidKey` error will be returned.
@@ -135,11 +143,11 @@ impl<K: Ord, V> VecMap<K, V> {
(
VecMap {
data: self.data[..split_idx].to_vec(),
data: SmallVec::from(&self.data[..split_idx]),
ordering: self.ordering,
},
VecMap {
data: self.data[split_idx..].to_vec(),
data: SmallVec::from(&self.data[split_idx..]),
ordering: self.ordering,
},
)
@@ -186,7 +194,10 @@ impl<K: Ord, V> VecMap<K, V> {
/// Instrument an operation on the underlying [`Vec`].
/// Will panic if the operation decreases capacity.
/// Returns the increase in memory usage caused by the op.
fn instrument_vec_op(&mut self, op: impl FnOnce(&mut Vec<(K, V)>)) -> usize {
fn instrument_vec_op(
&mut self,
op: impl FnOnce(&mut SmallVec<[(K, V); INLINE_ELEMENTS]>),
) -> usize {
let old_cap = self.data.capacity();
op(&mut self.data);
let new_cap = self.data.capacity();
@@ -226,7 +237,7 @@ impl<K: Ord, V> VecMap<K, V> {
impl<K: Ord, V> IntoIterator for VecMap<K, V> {
type Item = (K, V);
type IntoIter = std::vec::IntoIter<(K, V)>;
type IntoIter = smallvec::IntoIter<[(K, V); INLINE_ELEMENTS]>;
fn into_iter(self) -> Self::IntoIter {
self.data.into_iter()

View File

@@ -49,6 +49,7 @@ postgres_backend.workspace = true
postgres-protocol.workspace = true
postgres-types.workspace = true
rand.workspace = true
range-set-blaze = { version = "0.1.16", features = ["alloc"] }
regex.workspace = true
scopeguard.workspace = true
serde.workspace = true
@@ -107,3 +108,7 @@ harness = false
[[bench]]
name = "bench_walredo"
harness = false
[[bench]]
name = "bench_ingest"
harness = false

View File

@@ -0,0 +1,250 @@
use std::{env, num::NonZeroUsize};
use bytes::Bytes;
use camino::Utf8PathBuf;
use criterion::{criterion_group, criterion_main, Criterion};
use pageserver::{
config::PageServerConf,
context::{DownloadBehavior, RequestContext},
l0_flush::{L0FlushConfig, L0FlushGlobalState},
page_cache,
repository::Value,
task_mgr::TaskKind,
tenant::storage_layer::{InMemoryLayer, SerializedBatch},
virtual_file::{self, api::IoEngineKind},
};
use pageserver_api::{key::Key, shard::TenantShardId};
use utils::{
bin_ser::BeSer,
id::{TenantId, TimelineId},
};
// A very cheap hash for generating non-sequential keys.
fn murmurhash32(mut h: u32) -> u32 {
h ^= h >> 16;
h = h.wrapping_mul(0x85ebca6b);
h ^= h >> 13;
h = h.wrapping_mul(0xc2b2ae35);
h ^= h >> 16;
h
}
enum KeyLayout {
/// Sequential unique keys
Sequential,
/// Random unique keys
Random,
/// Random keys, but only use the bits from the mask of them
RandomReuse(u32),
}
enum WriteDelta {
Yes,
No,
}
async fn ingest(
conf: &'static PageServerConf,
put_size: usize,
put_count: usize,
key_layout: KeyLayout,
write_delta: WriteDelta,
) -> anyhow::Result<()> {
let mut lsn = utils::lsn::Lsn(1000);
let mut key = Key::from_i128(0x0);
let timeline_id = TimelineId::generate();
let tenant_id = TenantId::generate();
let tenant_shard_id = TenantShardId::unsharded(tenant_id);
tokio::fs::create_dir_all(conf.timeline_path(&tenant_shard_id, &timeline_id)).await?;
let ctx = RequestContext::new(TaskKind::DebugTool, DownloadBehavior::Error);
let layer = InMemoryLayer::create(conf, timeline_id, tenant_shard_id, lsn, &ctx).await?;
let value = Value::Image(Bytes::from(vec![0u8; put_size]));
let ctx = RequestContext::new(
pageserver::task_mgr::TaskKind::WalReceiverConnectionHandler,
pageserver::context::DownloadBehavior::Download,
);
let batch_pages = 10000;
let mut batch_values = vec![];
for i in 0..put_count {
lsn += put_size as u64;
// Generate lots of keys within a single relation, which simulates the typical bulk ingest case: people
// usually care the most about write performance when they're blasting a huge batch of data into a huge table.
match key_layout {
KeyLayout::Sequential => {
// Use sequential order to illustrate the experience a user is likely to have
// when ingesting bulk data.
key.field6 = i as u32;
}
KeyLayout::Random => {
// Use random-order keys to avoid giving a false advantage to data structures that are
// faster when inserting on the end.
key.field6 = murmurhash32(i as u32);
}
KeyLayout::RandomReuse(mask) => {
// Use low bits only, to limit cardinality
key.field6 = murmurhash32(i as u32) & mask;
}
}
batch_values.push((key, lsn, value.clone()));
if batch_values.len() >= batch_pages {
let write_batch = std::mem::take(&mut batch_values);
let batch = SerializedBatch::from_values(write_batch);
layer.put_batch(&batch, &ctx).await?;
}
}
if !batch_values.is_empty() {
let batch = SerializedBatch::from_values(vec![(key, lsn, value.clone())]);
layer.put_batch(&batch, &ctx).await?;
}
layer.freeze(lsn + 1).await;
if matches!(write_delta, WriteDelta::Yes) {
let l0_flush_state = L0FlushGlobalState::new(L0FlushConfig::Direct {
max_concurrency: NonZeroUsize::new(1).unwrap(),
});
let (_desc, path) = layer
.write_to_disk(&ctx, None, l0_flush_state.inner())
.await?
.unwrap();
tokio::fs::remove_file(path).await?;
}
Ok(())
}
/// Wrapper to instantiate a tokio runtime
fn ingest_main(
conf: &'static PageServerConf,
put_size: usize,
put_count: usize,
key_layout: KeyLayout,
write_delta: WriteDelta,
) {
let runtime = tokio::runtime::Builder::new_current_thread()
.enable_all()
.build()
.unwrap();
runtime.block_on(async move {
let r = ingest(conf, put_size, put_count, key_layout, write_delta).await;
if let Err(e) = r {
panic!("{e:?}");
}
});
}
/// Declare a series of benchmarks for the Pageserver's ingest write path.
///
/// This benchmark does not include WAL decode: it starts at InMemoryLayer::put_value, and ends either
/// at freezing the ephemeral layer, or writing the ephemeral layer out to an L0 (depending on whether WriteDelta is set).
///
/// Genuine disk I/O is used, so expect results to differ depending on storage. However, when running on
/// a fast disk, CPU is the bottleneck at time of writing.
fn criterion_benchmark(c: &mut Criterion) {
let temp_dir_parent: Utf8PathBuf = env::current_dir().unwrap().try_into().unwrap();
let temp_dir = camino_tempfile::tempdir_in(temp_dir_parent).unwrap();
eprintln!("Data directory: {}", temp_dir.path());
let conf: &'static PageServerConf = Box::leak(Box::new(
pageserver::config::PageServerConf::dummy_conf(temp_dir.path().to_path_buf()),
));
virtual_file::init(16384, IoEngineKind::TokioEpollUring);
page_cache::init(conf.page_cache_size);
{
let mut group = c.benchmark_group("ingest-small-values");
let put_size = 100usize;
let put_count = 128 * 1024 * 1024 / put_size;
group.throughput(criterion::Throughput::Bytes((put_size * put_count) as u64));
group.sample_size(10);
group.bench_function("ingest 128MB/100b seq", |b| {
b.iter(|| {
ingest_main(
conf,
put_size,
put_count,
KeyLayout::Sequential,
WriteDelta::Yes,
)
})
});
group.bench_function("ingest 128MB/100b rand", |b| {
b.iter(|| {
ingest_main(
conf,
put_size,
put_count,
KeyLayout::Random,
WriteDelta::Yes,
)
})
});
group.bench_function("ingest 128MB/100b rand-1024keys", |b| {
b.iter(|| {
ingest_main(
conf,
put_size,
put_count,
KeyLayout::RandomReuse(0x3ff),
WriteDelta::Yes,
)
})
});
group.bench_function("ingest 128MB/100b seq, no delta", |b| {
b.iter(|| {
ingest_main(
conf,
put_size,
put_count,
KeyLayout::Sequential,
WriteDelta::No,
)
})
});
}
{
let mut group = c.benchmark_group("ingest-big-values");
let put_size = 8192usize;
let put_count = 128 * 1024 * 1024 / put_size;
group.throughput(criterion::Throughput::Bytes((put_size * put_count) as u64));
group.sample_size(10);
group.bench_function("ingest 128MB/8k seq", |b| {
b.iter(|| {
ingest_main(
conf,
put_size,
put_count,
KeyLayout::Sequential,
WriteDelta::Yes,
)
})
});
group.bench_function("ingest 128MB/8k seq, no delta", |b| {
b.iter(|| {
ingest_main(
conf,
put_size,
put_count,
KeyLayout::Sequential,
WriteDelta::No,
)
})
});
}
}
criterion_group!(benches, criterion_benchmark);
criterion_main!(benches);

View File

@@ -1,3 +1,4 @@
use criterion::measurement::WallTime;
use pageserver::keyspace::{KeyPartitioning, KeySpace};
use pageserver::repository::Key;
use pageserver::tenant::layer_map::LayerMap;
@@ -15,7 +16,11 @@ use utils::id::{TenantId, TimelineId};
use utils::lsn::Lsn;
use criterion::{black_box, criterion_group, criterion_main, Criterion};
use criterion::{black_box, criterion_group, criterion_main, BenchmarkGroup, Criterion};
fn fixture_path(relative: &str) -> PathBuf {
PathBuf::from(env!("CARGO_MANIFEST_DIR")).join(relative)
}
fn build_layer_map(filename_dump: PathBuf) -> LayerMap {
let mut layer_map = LayerMap::default();
@@ -109,7 +114,7 @@ fn uniform_key_partitioning(layer_map: &LayerMap, _lsn: Lsn) -> KeyPartitioning
// between each test run.
fn bench_from_captest_env(c: &mut Criterion) {
// TODO consider compressing this file
let layer_map = build_layer_map(PathBuf::from("benches/odd-brook-layernames.txt"));
let layer_map = build_layer_map(fixture_path("benches/odd-brook-layernames.txt"));
let queries: Vec<(Key, Lsn)> = uniform_query_pattern(&layer_map);
// Test with uniform query pattern
@@ -139,7 +144,7 @@ fn bench_from_captest_env(c: &mut Criterion) {
fn bench_from_real_project(c: &mut Criterion) {
// Init layer map
let now = Instant::now();
let layer_map = build_layer_map(PathBuf::from("benches/odd-brook-layernames.txt"));
let layer_map = build_layer_map(fixture_path("benches/odd-brook-layernames.txt"));
println!("Finished layer map init in {:?}", now.elapsed());
// Choose uniformly distributed queries
@@ -242,7 +247,72 @@ fn bench_sequential(c: &mut Criterion) {
group.finish();
}
fn bench_visibility_with_map(
group: &mut BenchmarkGroup<WallTime>,
layer_map: LayerMap,
read_points: Vec<Lsn>,
bench_name: &str,
) {
group.bench_function(bench_name, |b| {
b.iter(|| black_box(layer_map.get_visibility(read_points.clone())));
});
}
// Benchmark using synthetic data. Arrange image layers on stacked diagonal lines.
fn bench_visibility(c: &mut Criterion) {
let mut group = c.benchmark_group("visibility");
{
// Init layer map. Create 100_000 layers arranged in 1000 diagonal lines.
let now = Instant::now();
let mut layer_map = LayerMap::default();
let mut updates = layer_map.batch_update();
for i in 0..100_000 {
let i32 = (i as u32) % 100;
let zero = Key::from_hex("000000000000000000000000000000000000").unwrap();
let layer = PersistentLayerDesc::new_img(
TenantShardId::unsharded(TenantId::generate()),
TimelineId::generate(),
zero.add(10 * i32)..zero.add(10 * i32 + 1),
Lsn(i),
0,
);
updates.insert_historic(layer);
}
updates.flush();
println!("Finished layer map init in {:?}", now.elapsed());
let mut read_points = Vec::new();
for i in (0..100_000).step_by(1000) {
read_points.push(Lsn(i));
}
bench_visibility_with_map(&mut group, layer_map, read_points, "sequential");
}
{
let layer_map = build_layer_map(fixture_path("benches/odd-brook-layernames.txt"));
let read_points = vec![Lsn(0x1C760FA190)];
bench_visibility_with_map(&mut group, layer_map, read_points, "real_map");
let layer_map = build_layer_map(fixture_path("benches/odd-brook-layernames.txt"));
let read_points = vec![
Lsn(0x1C760FA190),
Lsn(0x000000931BEAD539),
Lsn(0x000000931BF63011),
Lsn(0x000000931B33AE68),
Lsn(0x00000038E67ABFA0),
Lsn(0x000000931B33AE68),
Lsn(0x000000914E3F38F0),
Lsn(0x000000931B33AE68),
];
bench_visibility_with_map(&mut group, layer_map, read_points, "real_map_many_branches");
}
group.finish();
}
criterion_group!(group_1, bench_from_captest_env);
criterion_group!(group_2, bench_from_real_project);
criterion_group!(group_3, bench_sequential);
criterion_main!(group_1, group_2, group_3);
criterion_group!(group_4, bench_visibility);
criterion_main!(group_1, group_2, group_3, group_4);

View File

@@ -17,11 +17,9 @@ use pageserver::config::PageserverIdentity;
use pageserver::control_plane_client::ControlPlaneClient;
use pageserver::disk_usage_eviction_task::{self, launch_disk_usage_global_eviction_task};
use pageserver::metrics::{STARTUP_DURATION, STARTUP_IS_LOADING};
use pageserver::task_mgr::WALRECEIVER_RUNTIME;
use pageserver::task_mgr::{COMPUTE_REQUEST_RUNTIME, WALRECEIVER_RUNTIME};
use pageserver::tenant::{secondary, TenantSharedResources};
use pageserver::{
CancellableTask, ConsumptionMetricsTasks, HttpEndpointListener, LibpqEndpointListener,
};
use pageserver::{CancellableTask, ConsumptionMetricsTasks, HttpEndpointListener};
use remote_storage::GenericRemoteStorage;
use tokio::signal::unix::SignalKind;
use tokio::time::Instant;
@@ -31,11 +29,9 @@ use tracing::*;
use metrics::set_build_info_metric;
use pageserver::{
config::PageServerConf,
context::{DownloadBehavior, RequestContext},
deletion_queue::DeletionQueue,
http, page_cache, page_service, task_mgr,
task_mgr::TaskKind,
task_mgr::{BACKGROUND_RUNTIME, COMPUTE_REQUEST_RUNTIME, MGMT_REQUEST_RUNTIME},
task_mgr::{BACKGROUND_RUNTIME, MGMT_REQUEST_RUNTIME},
tenant::mgr,
virtual_file,
};
@@ -129,6 +125,7 @@ fn main() -> anyhow::Result<()> {
info!(?conf.virtual_file_io_engine, "starting with virtual_file IO engine");
info!(?conf.get_impl, "starting with get page implementation");
info!(?conf.get_vectored_impl, "starting with vectored get page implementation");
info!(?conf.compact_level0_phase1_value_access, "starting with setting for compact_level0_phase1_value_access");
let tenants_path = conf.tenants_path();
if !tenants_path.exists() {
@@ -593,30 +590,13 @@ fn start_pageserver(
// Spawn a task to listen for libpq connections. It will spawn further tasks
// for each connection. We created the listener earlier already.
let libpq_listener = {
let cancel = CancellationToken::new();
let libpq_ctx = RequestContext::todo_child(
TaskKind::LibpqEndpointListener,
// listener task shouldn't need to download anything. (We will
// create a separate sub-contexts for each connection, with their
// own download behavior. This context is used only to listen and
// accept connections.)
DownloadBehavior::Error,
);
let task = COMPUTE_REQUEST_RUNTIME.spawn(task_mgr::exit_on_panic_or_error(
"libpq listener",
page_service::libpq_listener_main(
tenant_manager.clone(),
pg_auth,
pageserver_listener,
conf.pg_auth_type,
libpq_ctx,
cancel.clone(),
),
));
LibpqEndpointListener(CancellableTask { task, cancel })
};
let page_service = page_service::spawn(conf, tenant_manager.clone(), pg_auth, {
let _entered = COMPUTE_REQUEST_RUNTIME.enter(); // TcpListener::from_std requires it
pageserver_listener
.set_nonblocking(true)
.context("set listener to nonblocking")?;
tokio::net::TcpListener::from_std(pageserver_listener).context("create tokio listener")?
});
let mut shutdown_pageserver = Some(shutdown_pageserver.drop_guard());
@@ -644,7 +624,7 @@ fn start_pageserver(
shutdown_pageserver.take();
pageserver::shutdown_pageserver(
http_endpoint_listener,
libpq_listener,
page_service,
consumption_metrics_tasks,
disk_usage_eviction_task,
&tenant_manager,

View File

@@ -29,6 +29,7 @@ use utils::{
logging::LogFormat,
};
use crate::tenant::timeline::compaction::CompactL0Phase1ValueAccess;
use crate::tenant::vectored_blob_io::MaxVectoredReadBytes;
use crate::tenant::{config::TenantConfOpt, timeline::GetImpl};
use crate::tenant::{TENANTS_SEGMENT_NAME, TIMELINES_SEGMENT_NAME};
@@ -52,7 +53,7 @@ pub mod defaults {
use pageserver_api::models::ImageCompressionAlgorithm;
pub use storage_broker::DEFAULT_ENDPOINT as BROKER_DEFAULT_ENDPOINT;
pub const DEFAULT_WAIT_LSN_TIMEOUT: &str = "60 s";
pub const DEFAULT_WAIT_LSN_TIMEOUT: &str = "300 s";
pub const DEFAULT_WAL_REDO_TIMEOUT: &str = "60 s";
pub const DEFAULT_SUPERUSER: &str = "cloud_admin";
@@ -83,16 +84,16 @@ pub mod defaults {
#[cfg(not(target_os = "linux"))]
pub const DEFAULT_VIRTUAL_FILE_IO_ENGINE: &str = "std-fs";
pub const DEFAULT_GET_VECTORED_IMPL: &str = "sequential";
pub const DEFAULT_GET_VECTORED_IMPL: &str = "vectored";
pub const DEFAULT_GET_IMPL: &str = "legacy";
pub const DEFAULT_GET_IMPL: &str = "vectored";
pub const DEFAULT_MAX_VECTORED_READ_BYTES: usize = 128 * 1024; // 128 KiB
pub const DEFAULT_IMAGE_COMPRESSION: ImageCompressionAlgorithm =
ImageCompressionAlgorithm::Disabled;
pub const DEFAULT_VALIDATE_VECTORED_GET: bool = true;
pub const DEFAULT_VALIDATE_VECTORED_GET: bool = false;
pub const DEFAULT_EPHEMERAL_BYTES_PER_MEMORY_KB: usize = 0;
@@ -295,6 +296,10 @@ pub struct PageServerConf {
pub ephemeral_bytes_per_memory_kb: usize,
pub l0_flush: L0FlushConfig,
/// This flag is temporary and will be removed after gradual rollout.
/// See <https://github.com/neondatabase/neon/issues/8184>.
pub compact_level0_phase1_value_access: CompactL0Phase1ValueAccess,
}
/// We do not want to store this in a PageServerConf because the latter may be logged
@@ -356,8 +361,6 @@ struct PageServerConfigBuilder {
auth_validation_public_key_path: BuilderValue<Option<Utf8PathBuf>>,
remote_storage_config: BuilderValue<Option<RemoteStorageConfig>>,
id: BuilderValue<NodeId>,
broker_endpoint: BuilderValue<Uri>,
broker_keepalive_interval: BuilderValue<Duration>,
@@ -403,14 +406,13 @@ struct PageServerConfigBuilder {
ephemeral_bytes_per_memory_kb: BuilderValue<usize>,
l0_flush: BuilderValue<L0FlushConfig>,
compact_level0_phase1_value_access: BuilderValue<CompactL0Phase1ValueAccess>,
}
impl PageServerConfigBuilder {
fn new(node_id: NodeId) -> Self {
let mut this = Self::default();
this.id(node_id);
this
fn new() -> Self {
Self::default()
}
#[inline(always)]
@@ -438,7 +440,6 @@ impl PageServerConfigBuilder {
pg_auth_type: Set(AuthType::Trust),
auth_validation_public_key_path: Set(None),
remote_storage_config: Set(None),
id: NotSet,
broker_endpoint: Set(storage_broker::DEFAULT_ENDPOINT
.parse()
.expect("failed to parse default broker endpoint")),
@@ -496,6 +497,7 @@ impl PageServerConfigBuilder {
validate_vectored_get: Set(DEFAULT_VALIDATE_VECTORED_GET),
ephemeral_bytes_per_memory_kb: Set(DEFAULT_EPHEMERAL_BYTES_PER_MEMORY_KB),
l0_flush: Set(L0FlushConfig::default()),
compact_level0_phase1_value_access: Set(CompactL0Phase1ValueAccess::default()),
}
}
}
@@ -568,10 +570,6 @@ impl PageServerConfigBuilder {
self.broker_keepalive_interval = BuilderValue::Set(broker_keepalive_interval)
}
pub fn id(&mut self, node_id: NodeId) {
self.id = BuilderValue::Set(node_id)
}
pub fn log_format(&mut self, log_format: LogFormat) {
self.log_format = BuilderValue::Set(log_format)
}
@@ -683,7 +681,11 @@ impl PageServerConfigBuilder {
self.l0_flush = BuilderValue::Set(value);
}
pub fn build(self) -> anyhow::Result<PageServerConf> {
pub fn compact_level0_phase1_value_access(&mut self, value: CompactL0Phase1ValueAccess) {
self.compact_level0_phase1_value_access = BuilderValue::Set(value);
}
pub fn build(self, id: NodeId) -> anyhow::Result<PageServerConf> {
let default = Self::default_values();
macro_rules! conf {
@@ -716,7 +718,6 @@ impl PageServerConfigBuilder {
pg_auth_type,
auth_validation_public_key_path,
remote_storage_config,
id,
broker_endpoint,
broker_keepalive_interval,
log_format,
@@ -741,9 +742,11 @@ impl PageServerConfigBuilder {
image_compression,
ephemeral_bytes_per_memory_kb,
l0_flush,
compact_level0_phase1_value_access,
}
CUSTOM LOGIC
{
id: id,
// TenantConf is handled separately
default_tenant_conf: TenantConf::default(),
concurrent_tenant_warmup: ConfigurableSemaphore::new({
@@ -893,7 +896,7 @@ impl PageServerConf {
toml: &Document,
workdir: &Utf8Path,
) -> anyhow::Result<Self> {
let mut builder = PageServerConfigBuilder::new(node_id);
let mut builder = PageServerConfigBuilder::new();
builder.workdir(workdir.to_owned());
let mut t_conf = TenantConfOpt::default();
@@ -924,8 +927,6 @@ impl PageServerConf {
"tenant_config" => {
t_conf = TenantConfOpt::try_from(item.to_owned()).context(format!("failed to parse: '{key}'"))?;
}
"id" => {}, // Ignoring `id` field in pageserver.toml - using identity.toml as the source of truth
// Logging is not set up yet, so we can't do it.
"broker_endpoint" => builder.broker_endpoint(parse_toml_string(key, item)?.parse().context("failed to parse broker endpoint")?),
"broker_keepalive_interval" => builder.broker_keepalive_interval(parse_toml_duration(key, item)?),
"log_format" => builder.log_format(
@@ -1014,11 +1015,14 @@ impl PageServerConf {
"l0_flush" => {
builder.l0_flush(utils::toml_edit_ext::deserialize_item(item).context("l0_flush")?)
}
"compact_level0_phase1_value_access" => {
builder.compact_level0_phase1_value_access(utils::toml_edit_ext::deserialize_item(item).context("compact_level0_phase1_value_access")?)
}
_ => bail!("unrecognized pageserver option '{key}'"),
}
}
let mut conf = builder.build().context("invalid config")?;
let mut conf = builder.build(node_id).context("invalid config")?;
if conf.http_auth_type == AuthType::NeonJWT || conf.pg_auth_type == AuthType::NeonJWT {
let auth_validation_public_key_path = conf
@@ -1098,6 +1102,7 @@ impl PageServerConf {
validate_vectored_get: defaults::DEFAULT_VALIDATE_VECTORED_GET,
ephemeral_bytes_per_memory_kb: defaults::DEFAULT_EPHEMERAL_BYTES_PER_MEMORY_KB,
l0_flush: L0FlushConfig::default(),
compact_level0_phase1_value_access: CompactL0Phase1ValueAccess::default(),
}
}
}
@@ -1255,7 +1260,6 @@ max_file_descriptors = 333
# initial superuser role name to use when creating a new tenant
initial_superuser_name = 'zzzz'
id = 10
metric_collection_interval = '222 s'
metric_collection_endpoint = 'http://localhost:80/metrics'
@@ -1272,9 +1276,8 @@ background_task_maximum_delay = '334 s'
let (workdir, pg_distrib_dir) = prepare_fs(&tempdir)?;
let broker_endpoint = storage_broker::DEFAULT_ENDPOINT;
// we have to create dummy values to overcome the validation errors
let config_string = format!(
"pg_distrib_dir='{pg_distrib_dir}'\nid=10\nbroker_endpoint = '{broker_endpoint}'",
);
let config_string =
format!("pg_distrib_dir='{pg_distrib_dir}'\nbroker_endpoint = '{broker_endpoint}'",);
let toml = config_string.parse()?;
let parsed_config = PageServerConf::parse_and_validate(NodeId(10), &toml, &workdir)
@@ -1341,6 +1344,7 @@ background_task_maximum_delay = '334 s'
image_compression: defaults::DEFAULT_IMAGE_COMPRESSION,
ephemeral_bytes_per_memory_kb: defaults::DEFAULT_EPHEMERAL_BYTES_PER_MEMORY_KB,
l0_flush: L0FlushConfig::default(),
compact_level0_phase1_value_access: CompactL0Phase1ValueAccess::default(),
},
"Correct defaults should be used when no config values are provided"
);
@@ -1415,6 +1419,7 @@ background_task_maximum_delay = '334 s'
image_compression: defaults::DEFAULT_IMAGE_COMPRESSION,
ephemeral_bytes_per_memory_kb: defaults::DEFAULT_EPHEMERAL_BYTES_PER_MEMORY_KB,
l0_flush: L0FlushConfig::default(),
compact_level0_phase1_value_access: CompactL0Phase1ValueAccess::default(),
},
"Should be able to parse all basic config values correctly"
);
@@ -1579,7 +1584,6 @@ broker_endpoint = '{broker_endpoint}'
r#"pg_distrib_dir = "{pg_distrib_dir}"
metric_collection_endpoint = "http://sample.url"
metric_collection_interval = "10min"
id = 222
[disk_usage_based_eviction]
max_usage_pct = 80
@@ -1649,7 +1653,6 @@ threshold = "20m"
r#"pg_distrib_dir = "{pg_distrib_dir}"
metric_collection_endpoint = "http://sample.url"
metric_collection_interval = "10min"
id = 222
[tenant_config]
evictions_low_residence_duration_metric_threshold = "20m"

View File

@@ -171,14 +171,14 @@ impl ControlPlaneGenerationsApi for ControlPlaneClient {
register,
};
fail::fail_point!("control-plane-client-re-attach");
let response: ReAttachResponse = self.retry_http_forever(&re_attach_path, request).await?;
tracing::info!(
"Received re-attach response with {} tenants",
response.tenants.len()
);
failpoint_support::sleep_millis_async!("control-plane-client-re-attach");
Ok(response
.tenants
.into_iter()

View File

@@ -296,6 +296,11 @@ impl From<GetActiveTenantError> for ApiError {
GetActiveTenantError::WaitForActiveTimeout { .. } => {
ApiError::ResourceUnavailable(format!("{}", e).into())
}
GetActiveTenantError::SwitchedTenant => {
// in our HTTP handlers, this error doesn't happen
// TODO: separate error types
ApiError::ResourceUnavailable("switched tenant".into())
}
}
}
}
@@ -1650,7 +1655,9 @@ async fn timeline_compact_handler(
.await
.map_err(|e| ApiError::InternalServerError(e.into()))?;
if wait_until_uploaded {
timeline.remote_client.wait_completion().await.map_err(ApiError::InternalServerError)?;
timeline.remote_client.wait_completion().await
// XXX map to correct ApiError for the cases where it's due to shutdown
.context("wait completion").map_err(ApiError::InternalServerError)?;
}
json_response(StatusCode::OK, ())
}
@@ -1709,7 +1716,9 @@ async fn timeline_checkpoint_handler(
}
if wait_until_uploaded {
timeline.remote_client.wait_completion().await.map_err(ApiError::InternalServerError)?;
timeline.remote_client.wait_completion().await
// XXX map to correct ApiError for the cases where it's due to shutdown
.context("wait completion").map_err(ApiError::InternalServerError)?;
}
json_response(StatusCode::OK, ())
@@ -2125,14 +2134,24 @@ async fn secondary_download_handler(
let timeout = wait.unwrap_or(Duration::MAX);
let status = match tokio::time::timeout(
let result = tokio::time::timeout(
timeout,
state.secondary_controller.download_tenant(tenant_shard_id),
)
.await
{
// Download job ran to completion.
Ok(Ok(())) => StatusCode::OK,
.await;
let progress = secondary_tenant.progress.lock().unwrap().clone();
let status = match result {
Ok(Ok(())) => {
if progress.layers_downloaded >= progress.layers_total {
// Download job ran to completion
StatusCode::OK
} else {
// Download dropped out without errors because it ran out of time budget
StatusCode::ACCEPTED
}
}
// Edge case: downloads aren't usually fallible: things like a missing heatmap are considered
// okay. We could get an error here in the unlikely edge case that the tenant
// was detached between our check above and executing the download job.
@@ -2142,8 +2161,6 @@ async fn secondary_download_handler(
Err(_) => StatusCode::ACCEPTED,
};
let progress = secondary_tenant.progress.lock().unwrap().clone();
json_response(status, progress)
}

View File

@@ -2,19 +2,29 @@ use std::{num::NonZeroUsize, sync::Arc};
use crate::tenant::ephemeral_file;
#[derive(Default, Debug, PartialEq, Eq, Clone, serde::Deserialize)]
#[derive(Debug, PartialEq, Eq, Clone, serde::Deserialize)]
#[serde(tag = "mode", rename_all = "kebab-case", deny_unknown_fields)]
pub enum L0FlushConfig {
#[default]
PageCached,
#[serde(rename_all = "snake_case")]
Direct { max_concurrency: NonZeroUsize },
Direct {
max_concurrency: NonZeroUsize,
},
}
impl Default for L0FlushConfig {
fn default() -> Self {
Self::Direct {
// TODO: using num_cpus results in different peak memory usage on different instance types.
max_concurrency: NonZeroUsize::new(usize::max(1, num_cpus::get())).unwrap(),
}
}
}
#[derive(Clone)]
pub struct L0FlushGlobalState(Arc<Inner>);
pub(crate) enum Inner {
pub enum Inner {
PageCached,
Direct { semaphore: tokio::sync::Semaphore },
}
@@ -30,7 +40,7 @@ impl L0FlushGlobalState {
}
}
pub(crate) fn inner(&self) -> &Arc<Inner> {
pub fn inner(&self) -> &Arc<Inner> {
&self.0
}
}

View File

@@ -12,6 +12,8 @@ pub mod disk_usage_eviction_task;
pub mod http;
pub mod import_datadir;
pub mod l0_flush;
use futures::{stream::FuturesUnordered, StreamExt};
pub use pageserver_api::keyspace;
use tokio_util::sync::CancellationToken;
pub mod aux_file;
@@ -30,14 +32,13 @@ pub mod walingest;
pub mod walrecord;
pub mod walredo;
use crate::task_mgr::TaskKind;
use camino::Utf8Path;
use deletion_queue::DeletionQueue;
use tenant::{
mgr::{BackgroundPurges, TenantManager},
secondary,
};
use tracing::info;
use tracing::{info, info_span};
/// Current storage format version
///
@@ -63,7 +64,6 @@ pub struct CancellableTask {
pub cancel: CancellationToken,
}
pub struct HttpEndpointListener(pub CancellableTask);
pub struct LibpqEndpointListener(pub CancellableTask);
pub struct ConsumptionMetricsTasks(pub CancellableTask);
pub struct DiskUsageEvictionTask(pub CancellableTask);
impl CancellableTask {
@@ -77,7 +77,7 @@ impl CancellableTask {
#[allow(clippy::too_many_arguments)]
pub async fn shutdown_pageserver(
http_listener: HttpEndpointListener,
libpq_listener: LibpqEndpointListener,
page_service: page_service::Listener,
consumption_metrics_worker: ConsumptionMetricsTasks,
disk_usage_eviction_task: Option<DiskUsageEvictionTask>,
tenant_manager: &TenantManager,
@@ -87,10 +87,83 @@ pub async fn shutdown_pageserver(
exit_code: i32,
) {
use std::time::Duration;
// If the orderly shutdown below takes too long, we still want to make
// sure that all walredo processes are killed and wait()ed on by us, not systemd.
//
// (Leftover walredo processes are the hypothesized trigger for the systemd freezes
// that we keep seeing in prod => https://github.com/neondatabase/cloud/issues/11387.
//
// We use a thread instead of a tokio task because the background runtime is likely busy
// with the final flushing / uploads. This activity here has priority, and due to lack
// of scheduling priority feature sin the tokio scheduler, using a separate thread is
// an effective priority booster.
let walredo_extraordinary_shutdown_thread_span = {
let span = info_span!(parent: None, "walredo_extraordinary_shutdown_thread");
span.follows_from(tracing::Span::current());
span
};
let walredo_extraordinary_shutdown_thread_cancel = CancellationToken::new();
let walredo_extraordinary_shutdown_thread = std::thread::spawn({
let walredo_extraordinary_shutdown_thread_cancel =
walredo_extraordinary_shutdown_thread_cancel.clone();
move || {
let rt = tokio::runtime::Builder::new_current_thread()
.enable_all()
.build()
.unwrap();
let _entered = rt.enter();
let _entered = walredo_extraordinary_shutdown_thread_span.enter();
if let Ok(()) = rt.block_on(tokio::time::timeout(
Duration::from_secs(8),
walredo_extraordinary_shutdown_thread_cancel.cancelled(),
)) {
info!("cancellation requested");
return;
}
let managers = tenant::WALREDO_MANAGERS
.lock()
.unwrap()
// prevents new walredo managers from being inserted
.take()
.expect("only we take()");
// Use FuturesUnordered to get in queue early for each manager's
// heavier_once_cell semaphore wait list.
// Also, for idle tenants that for some reason haven't
// shut down yet, it's quite likely that we're not going
// to get Poll::Pending once.
let mut futs: FuturesUnordered<_> = managers
.into_iter()
.filter_map(|(_, mgr)| mgr.upgrade())
.map(|mgr| async move { tokio::task::unconstrained(mgr.shutdown()).await })
.collect();
info!(count=%futs.len(), "built FuturesUnordered");
let mut last_log_at = std::time::Instant::now();
#[derive(Debug, Default)]
struct Results {
initiated: u64,
already: u64,
}
let mut results = Results::default();
while let Some(we_initiated) = rt.block_on(futs.next()) {
if we_initiated {
results.initiated += 1;
} else {
results.already += 1;
}
if last_log_at.elapsed() > Duration::from_millis(100) {
info!(remaining=%futs.len(), ?results, "progress");
last_log_at = std::time::Instant::now();
}
}
info!(?results, "done");
}
});
// Shut down the libpq endpoint task. This prevents new connections from
// being accepted.
timed(
libpq_listener.0.shutdown(),
let remaining_connections = timed(
page_service.stop_accepting(),
"shutdown LibpqEndpointListener",
Duration::from_secs(1),
)
@@ -108,7 +181,7 @@ pub async fn shutdown_pageserver(
// Shut down any page service tasks: any in-progress work for particular timelines or tenants
// should already have been canclled via mgr::shutdown_all_tenants
timed(
task_mgr::shutdown_tasks(Some(TaskKind::PageRequestHandler), None, None),
remaining_connections.shutdown(),
"shutdown PageRequestHandlers",
Duration::from_secs(1),
)
@@ -162,6 +235,12 @@ pub async fn shutdown_pageserver(
Duration::from_secs(1),
)
.await;
info!("cancel & join walredo_extraordinary_shutdown_thread");
walredo_extraordinary_shutdown_thread_cancel.cancel();
walredo_extraordinary_shutdown_thread.join().unwrap();
info!("walredo_extraordinary_shutdown_thread done");
info!("Shut down successfully completed");
std::process::exit(exit_code);
}

View File

@@ -525,6 +525,15 @@ static RESIDENT_PHYSICAL_SIZE: Lazy<UIntGaugeVec> = Lazy::new(|| {
.expect("failed to define a metric")
});
static VISIBLE_PHYSICAL_SIZE: Lazy<UIntGaugeVec> = Lazy::new(|| {
register_uint_gauge_vec!(
"pageserver_visible_physical_size",
"The size of the layer files present in the pageserver's filesystem.",
&["tenant_id", "shard_id", "timeline_id"]
)
.expect("failed to define a metric")
});
pub(crate) static RESIDENT_PHYSICAL_SIZE_GLOBAL: Lazy<UIntGauge> = Lazy::new(|| {
register_uint_gauge!(
"pageserver_resident_physical_size_global",
@@ -613,7 +622,23 @@ pub(crate) static CIRCUIT_BREAKERS_UNBROKEN: Lazy<IntCounter> = Lazy::new(|| {
pub(crate) static COMPRESSION_IMAGE_INPUT_BYTES: Lazy<IntCounter> = Lazy::new(|| {
register_int_counter!(
"pageserver_compression_image_in_bytes_total",
"Size of uncompressed data written into image layers"
"Size of data written into image layers before compression"
)
.expect("failed to define a metric")
});
pub(crate) static COMPRESSION_IMAGE_INPUT_BYTES_CONSIDERED: Lazy<IntCounter> = Lazy::new(|| {
register_int_counter!(
"pageserver_compression_image_in_bytes_considered",
"Size of potentially compressible data written into image layers before compression"
)
.expect("failed to define a metric")
});
pub(crate) static COMPRESSION_IMAGE_INPUT_BYTES_CHOSEN: Lazy<IntCounter> = Lazy::new(|| {
register_int_counter!(
"pageserver_compression_image_in_bytes_chosen",
"Size of data whose compressed form was written into image layers"
)
.expect("failed to define a metric")
});
@@ -2188,6 +2213,7 @@ pub(crate) struct TimelineMetrics {
pub(crate) layer_count_delta: UIntGauge,
pub standby_horizon_gauge: IntGauge,
pub resident_physical_size_gauge: UIntGauge,
pub visible_physical_size_gauge: UIntGauge,
/// copy of LayeredTimeline.current_logical_size
pub current_logical_size_gauge: UIntGauge,
pub aux_file_size_gauge: IntGauge,
@@ -2310,6 +2336,9 @@ impl TimelineMetrics {
let resident_physical_size_gauge = RESIDENT_PHYSICAL_SIZE
.get_metric_with_label_values(&[&tenant_id, &shard_id, &timeline_id])
.unwrap();
let visible_physical_size_gauge = VISIBLE_PHYSICAL_SIZE
.get_metric_with_label_values(&[&tenant_id, &shard_id, &timeline_id])
.unwrap();
// TODO: we shouldn't expose this metric
let current_logical_size_gauge = CURRENT_LOGICAL_SIZE
.get_metric_with_label_values(&[&tenant_id, &shard_id, &timeline_id])
@@ -2364,6 +2393,7 @@ impl TimelineMetrics {
layer_count_delta,
standby_horizon_gauge,
resident_physical_size_gauge,
visible_physical_size_gauge,
current_logical_size_gauge,
aux_file_size_gauge,
directory_entries_count_gauge,
@@ -2415,6 +2445,7 @@ impl TimelineMetrics {
RESIDENT_PHYSICAL_SIZE_GLOBAL.sub(self.resident_physical_size_get());
let _ = RESIDENT_PHYSICAL_SIZE.remove_label_values(&[tenant_id, shard_id, timeline_id]);
}
let _ = VISIBLE_PHYSICAL_SIZE.remove_label_values(&[tenant_id, shard_id, timeline_id]);
let _ = CURRENT_LOGICAL_SIZE.remove_label_values(&[tenant_id, shard_id, timeline_id]);
if let Some(metric) = Lazy::get(&DIRECTORY_ENTRIES_COUNT) {
let _ = metric.remove_label_values(&[tenant_id, shard_id, timeline_id]);
@@ -3104,6 +3135,8 @@ pub fn preinitialize_metrics() {
&tokio_epoll_uring::THREAD_LOCAL_LAUNCH_SUCCESSES,
&REMOTE_ONDEMAND_DOWNLOADED_LAYERS,
&REMOTE_ONDEMAND_DOWNLOADED_BYTES,
&CIRCUIT_BREAKERS_BROKEN,
&CIRCUIT_BREAKERS_UNBROKEN,
]
.into_iter()
.for_each(|c| {

File diff suppressed because it is too large Load Diff

View File

@@ -15,7 +15,6 @@ use crate::{aux_file, repository::*};
use anyhow::{ensure, Context};
use bytes::{Buf, Bytes, BytesMut};
use enum_map::Enum;
use itertools::Itertools;
use pageserver_api::key::{
dbdir_key_range, rel_block_to_key, rel_dir_to_key, rel_key_range, rel_size_to_key,
relmap_file_key, repl_origin_key, repl_origin_key_range, slru_block_to_key, slru_dir_to_key,
@@ -37,7 +36,6 @@ use tokio_util::sync::CancellationToken;
use tracing::{debug, info, trace, warn};
use utils::bin_ser::DeserializeError;
use utils::pausable_failpoint;
use utils::vec_map::{VecMap, VecMapOrdering};
use utils::{bin_ser::BeSer, lsn::Lsn};
/// Max delta records appended to the AUX_FILES_KEY (for aux v1). The write path will write a full image once this threshold is reached.
@@ -174,6 +172,7 @@ impl Timeline {
pending_deletions: Vec::new(),
pending_nblocks: 0,
pending_directory_entries: Vec::new(),
pending_bytes: 0,
lsn,
}
}
@@ -1058,14 +1057,26 @@ pub struct DatadirModification<'a> {
/// For special "directory" keys that store key-value maps, track the size of the map
/// if it was updated in this modification.
pending_directory_entries: Vec<(DirectoryKind, usize)>,
/// An **approximation** of how large our EphemeralFile write will be when committed.
pending_bytes: usize,
}
impl<'a> DatadirModification<'a> {
// When a DatadirModification is committed, we do a monolithic serialization of all its contents. WAL records can
// contain multiple pages, so the pageserver's record-based batch size isn't sufficient to bound this allocation: we
// additionally specify a limit on how much payload a DatadirModification may contain before it should be committed.
pub(crate) const MAX_PENDING_BYTES: usize = 8 * 1024 * 1024;
/// Get the current lsn
pub(crate) fn get_lsn(&self) -> Lsn {
self.lsn
}
pub(crate) fn approx_pending_bytes(&self) -> usize {
self.pending_bytes
}
/// Set the current lsn
pub(crate) fn set_lsn(&mut self, lsn: Lsn) -> anyhow::Result<()> {
ensure!(
@@ -1793,11 +1804,12 @@ impl<'a> DatadirModification<'a> {
// Flush relation and SLRU data blocks, keep metadata.
let mut retained_pending_updates = HashMap::<_, Vec<_>>::new();
for (key, values) in self.pending_updates.drain() {
let mut write_batch = Vec::new();
for (lsn, value) in values {
if key.is_rel_block_key() || key.is_slru_block_key() {
// This bails out on first error without modifying pending_updates.
// That's Ok, cf this function's doc comment.
writer.put(key, lsn, &value, ctx).await?;
write_batch.push((key, lsn, value));
} else {
retained_pending_updates
.entry(key)
@@ -1805,9 +1817,11 @@ impl<'a> DatadirModification<'a> {
.push((lsn, value));
}
}
writer.put_batch(write_batch, ctx).await?;
}
self.pending_updates = retained_pending_updates;
self.pending_bytes = 0;
if pending_nblocks != 0 {
writer.update_current_logical_size(pending_nblocks * i64::from(BLCKSZ));
@@ -1833,17 +1847,20 @@ impl<'a> DatadirModification<'a> {
self.pending_nblocks = 0;
if !self.pending_updates.is_empty() {
// The put_batch call below expects expects the inputs to be sorted by Lsn,
// so we do that first.
let lsn_ordered_batch: VecMap<Lsn, (Key, Value)> = VecMap::from_iter(
self.pending_updates
.drain()
.map(|(key, vals)| vals.into_iter().map(move |(lsn, val)| (lsn, (key, val))))
.kmerge_by(|lhs, rhs| lhs.0 < rhs.0),
VecMapOrdering::GreaterOrEqual,
);
// Ordering: the items in this batch do not need to be in any global order, but values for
// a particular Key must be in Lsn order relative to one another. InMemoryLayer relies on
// this to do efficient updates to its index.
let batch: Vec<(Key, Lsn, Value)> = self
.pending_updates
.drain()
.flat_map(|(key, values)| {
values
.into_iter()
.map(move |(lsn, value)| (key, lsn, value))
})
.collect::<Vec<_>>();
writer.put_batch(lsn_ordered_batch, ctx).await?;
writer.put_batch(batch, ctx).await?;
}
if !self.pending_deletions.is_empty() {
@@ -1868,6 +1885,8 @@ impl<'a> DatadirModification<'a> {
writer.update_directory_entries_count(kind, count as u64);
}
self.pending_bytes = 0;
Ok(())
}
@@ -1918,6 +1937,10 @@ impl<'a> DatadirModification<'a> {
return;
}
}
self.pending_bytes += match &val {
Value::Image(inner) => inner.len(),
Value::WalRecord(_) => 100, // Rough approximation of typical serialized WalRecord size.
};
values.push((self.lsn, val));
}

View File

@@ -8,8 +8,7 @@ use std::time::Duration;
pub use pageserver_api::key::{Key, KEY_SIZE};
/// A 'value' stored for a one Key.
#[derive(Debug, Clone, Serialize, Deserialize)]
#[cfg_attr(test, derive(PartialEq))]
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)]
pub enum Value {
/// An Image value contains a full copy of the value
Image(Bytes),

View File

@@ -33,6 +33,7 @@ use remote_storage::GenericRemoteStorage;
use remote_storage::TimeoutOrCancel;
use std::collections::BTreeMap;
use std::fmt;
use std::sync::Weak;
use std::time::SystemTime;
use storage_broker::BrokerClientChannel;
use tokio::io::BufReader;
@@ -102,8 +103,7 @@ use std::fmt::Debug;
use std::fmt::Display;
use std::fs;
use std::fs::File;
use std::sync::atomic::AtomicU64;
use std::sync::atomic::Ordering;
use std::sync::atomic::{AtomicU64, Ordering};
use std::sync::Arc;
use std::sync::Mutex;
use std::time::{Duration, Instant};
@@ -313,14 +313,66 @@ impl std::fmt::Debug for Tenant {
}
pub(crate) enum WalRedoManager {
Prod(PostgresRedoManager),
Prod(WalredoManagerId, PostgresRedoManager),
#[cfg(test)]
Test(harness::TestRedoManager),
}
impl From<PostgresRedoManager> for WalRedoManager {
fn from(mgr: PostgresRedoManager) -> Self {
Self::Prod(mgr)
#[derive(thiserror::Error, Debug)]
#[error("pageserver is shutting down")]
pub(crate) struct GlobalShutDown;
impl WalRedoManager {
pub(crate) fn new(mgr: PostgresRedoManager) -> Result<Arc<Self>, GlobalShutDown> {
let id = WalredoManagerId::next();
let arc = Arc::new(Self::Prod(id, mgr));
let mut guard = WALREDO_MANAGERS.lock().unwrap();
match &mut *guard {
Some(map) => {
map.insert(id, Arc::downgrade(&arc));
Ok(arc)
}
None => Err(GlobalShutDown),
}
}
}
impl Drop for WalRedoManager {
fn drop(&mut self) {
match self {
Self::Prod(id, _) => {
let mut guard = WALREDO_MANAGERS.lock().unwrap();
if let Some(map) = &mut *guard {
map.remove(id).expect("new() registers, drop() unregisters");
}
}
#[cfg(test)]
Self::Test(_) => {
// Not applicable to test redo manager
}
}
}
}
/// Global registry of all walredo managers so that [`crate::shutdown_pageserver`] can shut down
/// the walredo processes outside of the regular order.
///
/// This is necessary to work around a systemd bug where it freezes if there are
/// walredo processes left => <https://github.com/neondatabase/cloud/issues/11387>
#[allow(clippy::type_complexity)]
pub(crate) static WALREDO_MANAGERS: once_cell::sync::Lazy<
Mutex<Option<HashMap<WalredoManagerId, Weak<WalRedoManager>>>>,
> = once_cell::sync::Lazy::new(|| Mutex::new(Some(HashMap::new())));
#[derive(PartialEq, Eq, Hash, Clone, Copy, Debug)]
pub(crate) struct WalredoManagerId(u64);
impl WalredoManagerId {
pub fn next() -> Self {
static NEXT: std::sync::atomic::AtomicU64 = std::sync::atomic::AtomicU64::new(1);
let id = NEXT.fetch_add(1, std::sync::atomic::Ordering::Relaxed);
if id == 0 {
panic!("WalredoManagerId::new() returned 0, indicating wraparound, risking it's no longer unique");
}
Self(id)
}
}
@@ -332,19 +384,20 @@ impl From<harness::TestRedoManager> for WalRedoManager {
}
impl WalRedoManager {
pub(crate) async fn shutdown(&self) {
pub(crate) async fn shutdown(&self) -> bool {
match self {
Self::Prod(mgr) => mgr.shutdown().await,
Self::Prod(_, mgr) => mgr.shutdown().await,
#[cfg(test)]
Self::Test(_) => {
// Not applicable to test redo manager
true
}
}
}
pub(crate) fn maybe_quiesce(&self, idle_timeout: Duration) {
match self {
Self::Prod(mgr) => mgr.maybe_quiesce(idle_timeout),
Self::Prod(_, mgr) => mgr.maybe_quiesce(idle_timeout),
#[cfg(test)]
Self::Test(_) => {
// Not applicable to test redo manager
@@ -364,7 +417,7 @@ impl WalRedoManager {
pg_version: u32,
) -> Result<bytes::Bytes, walredo::Error> {
match self {
Self::Prod(mgr) => {
Self::Prod(_, mgr) => {
mgr.request_redo(key, lsn, base_img, records, pg_version)
.await
}
@@ -378,7 +431,7 @@ impl WalRedoManager {
pub(crate) fn status(&self) -> Option<WalRedoManagerStatus> {
match self {
WalRedoManager::Prod(m) => Some(m.status()),
WalRedoManager::Prod(_, m) => Some(m.status()),
#[cfg(test)]
WalRedoManager::Test(_) => None,
}
@@ -387,6 +440,8 @@ impl WalRedoManager {
#[derive(Debug, thiserror::Error, PartialEq, Eq)]
pub enum GetTimelineError {
#[error("Timeline is shutting down")]
ShuttingDown,
#[error("Timeline {tenant_id}/{timeline_id} is not active, state: {state:?}")]
NotActive {
tenant_id: TenantShardId,
@@ -676,11 +731,9 @@ impl Tenant {
init_order: Option<InitializationOrder>,
mode: SpawnMode,
ctx: &RequestContext,
) -> Arc<Tenant> {
let wal_redo_manager = Arc::new(WalRedoManager::from(PostgresRedoManager::new(
conf,
tenant_shard_id,
)));
) -> Result<Arc<Tenant>, GlobalShutDown> {
let wal_redo_manager =
WalRedoManager::new(PostgresRedoManager::new(conf, tenant_shard_id))?;
let TenantSharedResources {
broker_client,
@@ -879,7 +932,7 @@ impl Tenant {
}
.instrument(tracing::info_span!(parent: None, "attach", tenant_id=%tenant_shard_id.tenant_id, shard_id=%tenant_shard_id.shard_slug(), gen=?generation)),
);
tenant
Ok(tenant)
}
#[instrument(skip_all)]
@@ -1227,11 +1280,29 @@ impl Tenant {
Ok(timeline_preloads)
}
pub async fn apply_timeline_archival_config(
pub(crate) async fn apply_timeline_archival_config(
&self,
_timeline_id: TimelineId,
_config: TimelineArchivalState,
timeline_id: TimelineId,
state: TimelineArchivalState,
) -> anyhow::Result<()> {
let timeline = self
.get_timeline(timeline_id, false)
.context("Cannot apply timeline archival config to inexistent timeline")?;
let upload_needed = timeline
.remote_client
.schedule_index_upload_for_timeline_archival_state(state)?;
if upload_needed {
const MAX_WAIT: Duration = Duration::from_secs(10);
let Ok(v) =
tokio::time::timeout(MAX_WAIT, timeline.remote_client.wait_completion()).await
else {
tracing::warn!("reached timeout for waiting on upload queue");
bail!("reached timeout for upload queue flush");
};
v?;
}
Ok(())
}
@@ -1563,7 +1634,7 @@ impl Tenant {
self: Arc<Self>,
timeline_id: TimelineId,
) -> Result<(), DeleteTimelineError> {
DeleteTimelineFlow::run(&self, timeline_id, false).await?;
DeleteTimelineFlow::run(&self, timeline_id).await?;
Ok(())
}
@@ -1616,21 +1687,23 @@ impl Tenant {
/// This function is periodically called by compactor task.
/// Also it can be explicitly requested per timeline through page server
/// api's 'compact' command.
///
/// Returns whether we have pending compaction task.
async fn compaction_iteration(
&self,
cancel: &CancellationToken,
ctx: &RequestContext,
) -> anyhow::Result<(), timeline::CompactionError> {
) -> Result<bool, timeline::CompactionError> {
// Don't start doing work during shutdown, or when broken, we do not need those in the logs
if !self.is_active() {
return Ok(());
return Ok(false);
}
{
let conf = self.tenant_conf.load();
if !conf.location.may_delete_layers_hint() || !conf.location.may_upload_layers_hint() {
info!("Skipping compaction in location state {:?}", conf.location);
return Ok(());
return Ok(false);
}
}
@@ -1657,20 +1730,24 @@ impl Tenant {
// Before doing any I/O work, check our circuit breaker
if self.compaction_circuit_breaker.lock().unwrap().is_broken() {
info!("Skipping compaction due to previous failures");
return Ok(());
return Ok(false);
}
let mut has_pending_task = false;
for (timeline_id, timeline) in &timelines_to_compact {
timeline
has_pending_task |= timeline
.compact(cancel, EnumSet::empty(), ctx)
.instrument(info_span!("compact_timeline", %timeline_id))
.await
.map_err(|e| {
self.compaction_circuit_breaker
.lock()
.unwrap()
.fail(&CIRCUIT_BREAKERS_BROKEN, &e);
e
.inspect_err(|e| match e {
timeline::CompactionError::ShuttingDown => (),
timeline::CompactionError::Other(e) => {
self.compaction_circuit_breaker
.lock()
.unwrap()
.fail(&CIRCUIT_BREAKERS_BROKEN, e);
}
})?;
}
@@ -1679,7 +1756,7 @@ impl Tenant {
.unwrap()
.success(&CIRCUIT_BREAKERS_UNBROKEN);
Ok(())
Ok(has_pending_task)
}
// Call through to all timelines to freeze ephemeral layers if needed. Usually
@@ -4568,7 +4645,7 @@ mod tests {
let layer_map = tline.layers.read().await;
let level0_deltas = layer_map
.layer_map()
.get_level0_deltas()?
.get_level0_deltas()
.into_iter()
.map(|desc| layer_map.get_from_desc(&desc))
.collect::<Vec<_>>();
@@ -5787,7 +5864,7 @@ mod tests {
.read()
.await
.layer_map()
.get_level0_deltas()?
.get_level0_deltas()
.len();
tline.compact(&cancel, EnumSet::empty(), &ctx).await?;
@@ -5797,7 +5874,7 @@ mod tests {
.read()
.await
.layer_map()
.get_level0_deltas()?
.get_level0_deltas()
.len();
assert!(after_num_l0_delta_files < before_num_l0_delta_files, "after_num_l0_delta_files={after_num_l0_delta_files}, before_num_l0_delta_files={before_num_l0_delta_files}");
@@ -6886,7 +6963,11 @@ mod tests {
vec![
// Image layer at GC horizon
PersistentLayerKey {
key_range: Key::MIN..Key::MAX,
key_range: {
let mut key = Key::MAX;
key.field6 -= 1;
Key::MIN..key
},
lsn_range: Lsn(0x30)..Lsn(0x31),
is_delta: false
},
@@ -6905,6 +6986,15 @@ mod tests {
]
);
// increase GC horizon and compact again
{
// Update GC info
let mut guard = tline.gc_info.write().unwrap();
guard.cutoffs.time = Lsn(0x40);
guard.cutoffs.space = Lsn(0x40);
}
tline.compact_with_gc(&cancel, &ctx).await.unwrap();
Ok(())
}
@@ -7256,6 +7346,15 @@ mod tests {
);
}
// increase GC horizon and compact again
{
// Update GC info
let mut guard = tline.gc_info.write().unwrap();
guard.cutoffs.time = Lsn(0x40);
guard.cutoffs.space = Lsn(0x40);
}
tline.compact_with_gc(&cancel, &ctx).await.unwrap();
Ok(())
}
@@ -7307,7 +7406,9 @@ mod tests {
(
key,
Lsn(0x80),
Value::WalRecord(NeonWalRecord::wal_append(";0x80")),
Value::Image(Bytes::copy_from_slice(
b"0x10;0x20;0x30;0x40;0x50;0x60;0x70;0x80",
)),
),
(
key,
@@ -7322,6 +7423,7 @@ mod tests {
Lsn(0x60),
&[Lsn(0x20), Lsn(0x40), Lsn(0x50)],
3,
None,
)
.await
.unwrap();
@@ -7369,7 +7471,9 @@ mod tests {
),
(
Lsn(0x80),
Value::WalRecord(NeonWalRecord::wal_append(";0x80")),
Value::Image(Bytes::copy_from_slice(
b"0x10;0x20;0x30;0x40;0x50;0x60;0x70;0x80",
)),
),
(
Lsn(0x90),
@@ -7378,7 +7482,226 @@ mod tests {
]),
};
assert_eq!(res, expected_res);
// TODO: more tests with mixed image + delta, adding with k-merge test cases; e2e compaction test
// We expect GC-compaction to run with the original GC. This would create a situation that
// the original GC algorithm removes some delta layers b/c there are full image coverage,
// therefore causing some keys to have an incomplete history below the lowest retain LSN.
// For example, we have
// ```plain
// init delta @ 0x10, image @ 0x20, delta @ 0x30 (gc_horizon), image @ 0x40.
// ```
// Now the GC horizon moves up, and we have
// ```plain
// init delta @ 0x10, image @ 0x20, delta @ 0x30, image @ 0x40 (gc_horizon)
// ```
// The original GC algorithm kicks in, and removes delta @ 0x10, image @ 0x20.
// We will end up with
// ```plain
// delta @ 0x30, image @ 0x40 (gc_horizon)
// ```
// Now we run the GC-compaction, and this key does not have a full history.
// We should be able to handle this partial history and drop everything before the
// gc_horizon image.
let history = vec![
(
key,
Lsn(0x20),
Value::WalRecord(NeonWalRecord::wal_append(";0x20")),
),
(
key,
Lsn(0x30),
Value::WalRecord(NeonWalRecord::wal_append(";0x30")),
),
(
key,
Lsn(0x40),
Value::Image(Bytes::copy_from_slice(b"0x10;0x20;0x30;0x40")),
),
(
key,
Lsn(0x50),
Value::WalRecord(NeonWalRecord::wal_append(";0x50")),
),
(
key,
Lsn(0x60),
Value::WalRecord(NeonWalRecord::wal_append(";0x60")),
),
(
key,
Lsn(0x70),
Value::WalRecord(NeonWalRecord::wal_append(";0x70")),
),
(
key,
Lsn(0x80),
Value::Image(Bytes::copy_from_slice(
b"0x10;0x20;0x30;0x40;0x50;0x60;0x70;0x80",
)),
),
(
key,
Lsn(0x90),
Value::WalRecord(NeonWalRecord::wal_append(";0x90")),
),
];
let res = tline
.generate_key_retention(key, &history, Lsn(0x60), &[Lsn(0x40), Lsn(0x50)], 3, None)
.await
.unwrap();
let expected_res = KeyHistoryRetention {
below_horizon: vec![
(
Lsn(0x40),
KeyLogAtLsn(vec![(
Lsn(0x40),
Value::Image(Bytes::copy_from_slice(b"0x10;0x20;0x30;0x40")),
)]),
),
(
Lsn(0x50),
KeyLogAtLsn(vec![(
Lsn(0x50),
Value::WalRecord(NeonWalRecord::wal_append(";0x50")),
)]),
),
(
Lsn(0x60),
KeyLogAtLsn(vec![(
Lsn(0x60),
Value::WalRecord(NeonWalRecord::wal_append(";0x60")),
)]),
),
],
above_horizon: KeyLogAtLsn(vec![
(
Lsn(0x70),
Value::WalRecord(NeonWalRecord::wal_append(";0x70")),
),
(
Lsn(0x80),
Value::Image(Bytes::copy_from_slice(
b"0x10;0x20;0x30;0x40;0x50;0x60;0x70;0x80",
)),
),
(
Lsn(0x90),
Value::WalRecord(NeonWalRecord::wal_append(";0x90")),
),
]),
};
assert_eq!(res, expected_res);
// In case of branch compaction, the branch itself does not have the full history, and we need to provide
// the ancestor image in the test case.
let history = vec![
(
key,
Lsn(0x20),
Value::WalRecord(NeonWalRecord::wal_append(";0x20")),
),
(
key,
Lsn(0x30),
Value::WalRecord(NeonWalRecord::wal_append(";0x30")),
),
(
key,
Lsn(0x40),
Value::WalRecord(NeonWalRecord::wal_append(";0x40")),
),
(
key,
Lsn(0x70),
Value::WalRecord(NeonWalRecord::wal_append(";0x70")),
),
];
let res = tline
.generate_key_retention(
key,
&history,
Lsn(0x60),
&[],
3,
Some((key, Lsn(0x10), Bytes::copy_from_slice(b"0x10"))),
)
.await
.unwrap();
let expected_res = KeyHistoryRetention {
below_horizon: vec![(
Lsn(0x60),
KeyLogAtLsn(vec![(
Lsn(0x60),
Value::Image(Bytes::copy_from_slice(b"0x10;0x20;0x30;0x40")), // use the ancestor image to reconstruct the page
)]),
)],
above_horizon: KeyLogAtLsn(vec![(
Lsn(0x70),
Value::WalRecord(NeonWalRecord::wal_append(";0x70")),
)]),
};
assert_eq!(res, expected_res);
let history = vec![
(
key,
Lsn(0x20),
Value::WalRecord(NeonWalRecord::wal_append(";0x20")),
),
(
key,
Lsn(0x40),
Value::WalRecord(NeonWalRecord::wal_append(";0x40")),
),
(
key,
Lsn(0x60),
Value::WalRecord(NeonWalRecord::wal_append(";0x60")),
),
(
key,
Lsn(0x70),
Value::WalRecord(NeonWalRecord::wal_append(";0x70")),
),
];
let res = tline
.generate_key_retention(
key,
&history,
Lsn(0x60),
&[Lsn(0x30)],
3,
Some((key, Lsn(0x10), Bytes::copy_from_slice(b"0x10"))),
)
.await
.unwrap();
let expected_res = KeyHistoryRetention {
below_horizon: vec![
(
Lsn(0x30),
KeyLogAtLsn(vec![(
Lsn(0x20),
Value::WalRecord(NeonWalRecord::wal_append(";0x20")),
)]),
),
(
Lsn(0x60),
KeyLogAtLsn(vec![(
Lsn(0x60),
Value::Image(Bytes::copy_from_slice(b"0x10;0x20;0x40;0x60")),
)]),
),
],
above_horizon: KeyLogAtLsn(vec![(
Lsn(0x70),
Value::WalRecord(NeonWalRecord::wal_append(";0x70")),
)]),
};
assert_eq!(res, expected_res);
Ok(())
}
@@ -7536,6 +7859,10 @@ mod tests {
];
let verify_result = || async {
let gc_horizon = {
let gc_info = tline.gc_info.read().unwrap();
gc_info.cutoffs.time
};
for idx in 0..10 {
assert_eq!(
tline
@@ -7546,7 +7873,7 @@ mod tests {
);
assert_eq!(
tline
.get(get_key(idx as u32), Lsn(0x30), &ctx)
.get(get_key(idx as u32), gc_horizon, &ctx)
.await
.unwrap(),
&expected_result_at_gc_horizon[idx]
@@ -7572,6 +7899,205 @@ mod tests {
let cancel = CancellationToken::new();
tline.compact_with_gc(&cancel, &ctx).await.unwrap();
verify_result().await;
// compact again
tline.compact_with_gc(&cancel, &ctx).await.unwrap();
verify_result().await;
// increase GC horizon and compact again
{
// Update GC info
let mut guard = tline.gc_info.write().unwrap();
guard.cutoffs.time = Lsn(0x38);
guard.cutoffs.space = Lsn(0x38);
}
tline.compact_with_gc(&cancel, &ctx).await.unwrap();
verify_result().await; // no wals between 0x30 and 0x38, so we should obtain the same result
// not increasing the GC horizon and compact again
tline.compact_with_gc(&cancel, &ctx).await.unwrap();
verify_result().await;
Ok(())
}
#[tokio::test]
async fn test_simple_bottom_most_compaction_on_branch() -> anyhow::Result<()> {
let harness = TenantHarness::create("test_simple_bottom_most_compaction_on_branch").await?;
let (tenant, ctx) = harness.load().await;
fn get_key(id: u32) -> Key {
let mut key = Key::from_hex("000000000033333333444444445500000000").unwrap();
key.field6 = id;
key
}
let img_layer = (0..10)
.map(|id| (get_key(id), Bytes::from(format!("value {id}@0x10"))))
.collect_vec();
let delta1 = vec![
(
get_key(1),
Lsn(0x20),
Value::WalRecord(NeonWalRecord::wal_append("@0x20")),
),
(
get_key(2),
Lsn(0x30),
Value::WalRecord(NeonWalRecord::wal_append("@0x30")),
),
(
get_key(3),
Lsn(0x28),
Value::WalRecord(NeonWalRecord::wal_append("@0x28")),
),
(
get_key(3),
Lsn(0x30),
Value::WalRecord(NeonWalRecord::wal_append("@0x30")),
),
(
get_key(3),
Lsn(0x40),
Value::WalRecord(NeonWalRecord::wal_append("@0x40")),
),
];
let delta2 = vec![
(
get_key(5),
Lsn(0x20),
Value::WalRecord(NeonWalRecord::wal_append("@0x20")),
),
(
get_key(6),
Lsn(0x20),
Value::WalRecord(NeonWalRecord::wal_append("@0x20")),
),
];
let delta3 = vec![
(
get_key(8),
Lsn(0x48),
Value::WalRecord(NeonWalRecord::wal_append("@0x48")),
),
(
get_key(9),
Lsn(0x48),
Value::WalRecord(NeonWalRecord::wal_append("@0x48")),
),
];
let parent_tline = tenant
.create_test_timeline_with_layers(
TIMELINE_ID,
Lsn(0x10),
DEFAULT_PG_VERSION,
&ctx,
vec![], // delta layers
vec![(Lsn(0x18), img_layer)], // image layers
Lsn(0x18),
)
.await?;
parent_tline.add_extra_test_dense_keyspace(KeySpace::single(get_key(0)..get_key(10)));
let branch_tline = tenant
.branch_timeline_test_with_layers(
&parent_tline,
NEW_TIMELINE_ID,
Some(Lsn(0x18)),
&ctx,
vec![
DeltaLayerTestDesc::new_with_inferred_key_range(Lsn(0x20)..Lsn(0x48), delta1),
DeltaLayerTestDesc::new_with_inferred_key_range(Lsn(0x20)..Lsn(0x48), delta2),
DeltaLayerTestDesc::new_with_inferred_key_range(Lsn(0x48)..Lsn(0x50), delta3),
], // delta layers
vec![], // image layers
Lsn(0x50),
)
.await?;
branch_tline.add_extra_test_dense_keyspace(KeySpace::single(get_key(0)..get_key(10)));
{
// Update GC info
let mut guard = parent_tline.gc_info.write().unwrap();
*guard = GcInfo {
retain_lsns: vec![(Lsn(0x18), branch_tline.timeline_id)],
cutoffs: GcCutoffs {
time: Lsn(0x10),
space: Lsn(0x10),
},
leases: Default::default(),
within_ancestor_pitr: false,
};
}
{
// Update GC info
let mut guard = branch_tline.gc_info.write().unwrap();
*guard = GcInfo {
retain_lsns: vec![(Lsn(0x40), branch_tline.timeline_id)],
cutoffs: GcCutoffs {
time: Lsn(0x50),
space: Lsn(0x50),
},
leases: Default::default(),
within_ancestor_pitr: false,
};
}
let expected_result_at_gc_horizon = [
Bytes::from_static(b"value 0@0x10"),
Bytes::from_static(b"value 1@0x10@0x20"),
Bytes::from_static(b"value 2@0x10@0x30"),
Bytes::from_static(b"value 3@0x10@0x28@0x30@0x40"),
Bytes::from_static(b"value 4@0x10"),
Bytes::from_static(b"value 5@0x10@0x20"),
Bytes::from_static(b"value 6@0x10@0x20"),
Bytes::from_static(b"value 7@0x10"),
Bytes::from_static(b"value 8@0x10@0x48"),
Bytes::from_static(b"value 9@0x10@0x48"),
];
let expected_result_at_lsn_40 = [
Bytes::from_static(b"value 0@0x10"),
Bytes::from_static(b"value 1@0x10@0x20"),
Bytes::from_static(b"value 2@0x10@0x30"),
Bytes::from_static(b"value 3@0x10@0x28@0x30@0x40"),
Bytes::from_static(b"value 4@0x10"),
Bytes::from_static(b"value 5@0x10@0x20"),
Bytes::from_static(b"value 6@0x10@0x20"),
Bytes::from_static(b"value 7@0x10"),
Bytes::from_static(b"value 8@0x10"),
Bytes::from_static(b"value 9@0x10"),
];
let verify_result = || async {
for idx in 0..10 {
assert_eq!(
branch_tline
.get(get_key(idx as u32), Lsn(0x50), &ctx)
.await
.unwrap(),
&expected_result_at_gc_horizon[idx]
);
assert_eq!(
branch_tline
.get(get_key(idx as u32), Lsn(0x40), &ctx)
.await
.unwrap(),
&expected_result_at_lsn_40[idx]
);
}
};
verify_result().await;
let cancel = CancellationToken::new();
branch_tline.compact_with_gc(&cancel, &ctx).await.unwrap();
verify_result().await;

View File

@@ -28,6 +28,12 @@ use crate::virtual_file::VirtualFile;
use std::cmp::min;
use std::io::{Error, ErrorKind};
#[derive(Copy, Clone, Debug)]
pub struct CompressionInfo {
pub written_compressed: bool,
pub compressed_size: Option<usize>,
}
impl<'a> BlockCursor<'a> {
/// Read a blob into a new buffer.
pub async fn read_blob(
@@ -273,8 +279,10 @@ impl<const BUFFERED: bool> BlobWriter<BUFFERED> {
srcbuf: B,
ctx: &RequestContext,
) -> (B::Buf, Result<u64, Error>) {
self.write_blob_maybe_compressed(srcbuf, ctx, ImageCompressionAlgorithm::Disabled)
.await
let (buf, res) = self
.write_blob_maybe_compressed(srcbuf, ctx, ImageCompressionAlgorithm::Disabled)
.await;
(buf, res.map(|(off, _compression_info)| off))
}
/// Write a blob of data. Returns the offset that it was written to,
@@ -284,8 +292,12 @@ impl<const BUFFERED: bool> BlobWriter<BUFFERED> {
srcbuf: B,
ctx: &RequestContext,
algorithm: ImageCompressionAlgorithm,
) -> (B::Buf, Result<u64, Error>) {
) -> (B::Buf, Result<(u64, CompressionInfo), Error>) {
let offset = self.offset;
let mut compression_info = CompressionInfo {
written_compressed: false,
compressed_size: None,
};
let len = srcbuf.bytes_init();
@@ -328,7 +340,9 @@ impl<const BUFFERED: bool> BlobWriter<BUFFERED> {
encoder.write_all(&slice[..]).await.unwrap();
encoder.shutdown().await.unwrap();
let compressed = encoder.into_inner();
compression_info.compressed_size = Some(compressed.len());
if compressed.len() < len {
compression_info.written_compressed = true;
let compressed_len = compressed.len();
compressed_buf = Some(compressed);
(BYTE_ZSTD, compressed_len, slice.into_inner())
@@ -359,7 +373,7 @@ impl<const BUFFERED: bool> BlobWriter<BUFFERED> {
} else {
self.write_all(srcbuf, ctx).await
};
(srcbuf, res.map(|_| offset))
(srcbuf, res.map(|_| (offset, compression_info)))
}
}
@@ -416,12 +430,14 @@ pub(crate) mod tests {
let mut wtr = BlobWriter::<BUFFERED>::new(file, 0);
for blob in blobs.iter() {
let (_, res) = if compression {
wtr.write_blob_maybe_compressed(
blob.clone(),
ctx,
ImageCompressionAlgorithm::Zstd { level: Some(1) },
)
.await
let res = wtr
.write_blob_maybe_compressed(
blob.clone(),
ctx,
ImageCompressionAlgorithm::Zstd { level: Some(1) },
)
.await;
(res.0, res.1.map(|(off, _)| off))
} else {
wtr.write_blob(blob.clone(), ctx).await
};

View File

@@ -296,13 +296,19 @@ where
let mut stack = Vec::new();
stack.push((self.root_blk, None));
let block_cursor = self.reader.block_cursor();
let mut node_buf = [0_u8; PAGE_SZ];
while let Some((node_blknum, opt_iter)) = stack.pop() {
// Locate the node.
let node_buf = block_cursor
// Read the node, through the PS PageCache, into local variable `node_buf`.
// We could keep the page cache read guard alive, but, at the time of writing,
// we run quite small PS PageCache s => can't risk running out of
// PageCache space because this stream isn't consumed fast enough.
let page_read_guard = block_cursor
.read_blk(self.start_blk + node_blknum, ctx)
.await?;
node_buf.copy_from_slice(page_read_guard.as_ref());
drop(page_read_guard); // drop page cache read guard early
let node = OnDiskNode::deparse(node_buf.as_ref())?;
let node = OnDiskNode::deparse(&node_buf)?;
let prefix_len = node.prefix_len as usize;
let suffix_len = node.suffix_len as usize;
@@ -345,6 +351,7 @@ where
Either::Left(idx..node.num_children.into())
};
// idx points to the first match now. Keep going from there
while let Some(idx) = iter.next() {
let key_off = idx * suffix_len;

View File

@@ -79,6 +79,8 @@ impl EphemeralFile {
self.rw.read_blk(blknum, ctx).await
}
#[cfg(test)]
// This is a test helper: outside of tests, we are always written do via a pre-serialized batch.
pub(crate) async fn write_blob(
&mut self,
srcbuf: &[u8],
@@ -86,17 +88,28 @@ impl EphemeralFile {
) -> Result<u64, io::Error> {
let pos = self.rw.bytes_written();
// Write the length field
if srcbuf.len() < 0x80 {
// short one-byte length header
let len_buf = [srcbuf.len() as u8];
let mut len_bytes = std::io::Cursor::new(Vec::new());
crate::tenant::storage_layer::inmemory_layer::SerializedBatch::write_blob_length(
srcbuf.len(),
&mut len_bytes,
);
let len_bytes = len_bytes.into_inner();
self.rw.write_all_borrowed(&len_buf, ctx).await?;
} else {
let mut len_buf = u32::to_be_bytes(srcbuf.len() as u32);
len_buf[0] |= 0x80;
self.rw.write_all_borrowed(&len_buf, ctx).await?;
}
// Write the length field
self.rw.write_all_borrowed(&len_bytes, ctx).await?;
// Write the payload
self.rw.write_all_borrowed(srcbuf, ctx).await?;
Ok(pos)
}
pub(crate) async fn write_raw(
&mut self,
srcbuf: &[u8],
ctx: &RequestContext,
) -> Result<u64, io::Error> {
let pos = self.rw.bytes_written();
// Write the payload
self.rw.write_all_borrowed(srcbuf, ctx).await?;

View File

@@ -51,7 +51,8 @@ use crate::keyspace::KeyPartitioning;
use crate::repository::Key;
use crate::tenant::storage_layer::InMemoryLayer;
use anyhow::Result;
use pageserver_api::keyspace::KeySpaceAccum;
use pageserver_api::keyspace::{KeySpace, KeySpaceAccum};
use range_set_blaze::{CheckSortedDisjoint, RangeSetBlaze};
use std::collections::{HashMap, VecDeque};
use std::iter::Peekable;
use std::ops::Range;
@@ -61,7 +62,7 @@ use utils::lsn::Lsn;
use historic_layer_coverage::BufferedHistoricLayerCoverage;
pub use historic_layer_coverage::LayerKey;
use super::storage_layer::PersistentLayerDesc;
use super::storage_layer::{LayerVisibilityHint, PersistentLayerDesc};
///
/// LayerMap tracks what layers exist on a timeline.
@@ -845,8 +846,8 @@ impl LayerMap {
}
/// Return all L0 delta layers
pub fn get_level0_deltas(&self) -> Result<Vec<Arc<PersistentLayerDesc>>> {
Ok(self.l0_delta_layers.to_vec())
pub fn get_level0_deltas(&self) -> Vec<Arc<PersistentLayerDesc>> {
self.l0_delta_layers.to_vec()
}
/// debugging function to print out the contents of the layer map
@@ -871,11 +872,183 @@ impl LayerMap {
println!("End dump LayerMap");
Ok(())
}
/// `read_points` represent the tip of a timeline and any branch points, i.e. the places
/// where we expect to serve reads.
///
/// This function is O(N) and should be called infrequently. The caller is responsible for
/// looking up and updating the Layer objects for these layer descriptors.
pub fn get_visibility(
&self,
mut read_points: Vec<Lsn>,
) -> (
Vec<(Arc<PersistentLayerDesc>, LayerVisibilityHint)>,
KeySpace,
) {
// This is like a KeySpace, but this type is intended for efficient unions with image layer ranges, whereas
// KeySpace is intended to be composed statically and iterated over.
struct KeyShadow {
// Map of range start to range end
inner: RangeSetBlaze<i128>,
}
impl KeyShadow {
fn new() -> Self {
Self {
inner: Default::default(),
}
}
fn contains(&self, range: Range<Key>) -> bool {
let range_incl = range.start.to_i128()..=range.end.to_i128() - 1;
self.inner.is_superset(&RangeSetBlaze::from_sorted_disjoint(
CheckSortedDisjoint::from([range_incl]),
))
}
/// Add the input range to the keys covered by self.
///
/// Return true if inserting this range covered some keys that were previously not covered
fn cover(&mut self, insert: Range<Key>) -> bool {
let range_incl = insert.start.to_i128()..=insert.end.to_i128() - 1;
self.inner.ranges_insert(range_incl)
}
fn reset(&mut self) {
self.inner = Default::default();
}
fn to_keyspace(&self) -> KeySpace {
let mut accum = KeySpaceAccum::new();
for range_incl in self.inner.ranges() {
let range = Range {
start: Key::from_i128(*range_incl.start()),
end: Key::from_i128(range_incl.end() + 1),
};
accum.add_range(range)
}
accum.to_keyspace()
}
}
// The 'shadow' will be updated as we sweep through the layers: an image layer subtracts from the shadow,
// and a ReadPoint
read_points.sort_by_key(|rp| rp.0);
let mut shadow = KeyShadow::new();
// We will interleave all our read points and layers into a sorted collection
enum Item {
ReadPoint { lsn: Lsn },
Layer(Arc<PersistentLayerDesc>),
}
let mut items = Vec::with_capacity(self.historic.len() + read_points.len());
items.extend(self.iter_historic_layers().map(Item::Layer));
items.extend(
read_points
.into_iter()
.map(|rp| Item::ReadPoint { lsn: rp }),
);
// Ordering: we want to iterate like this:
// 1. Highest LSNs first
// 2. Consider images before deltas if they end at the same LSNs (images cover deltas)
// 3. Consider ReadPoints before image layers if they're at the same LSN (readpoints make that image visible)
items.sort_by_key(|item| {
std::cmp::Reverse(match item {
Item::Layer(layer) => {
if layer.is_delta() {
(Lsn(layer.get_lsn_range().end.0 - 1), 0)
} else {
(layer.image_layer_lsn(), 1)
}
}
Item::ReadPoint { lsn } => (*lsn, 2),
})
});
let mut results = Vec::with_capacity(self.historic.len());
let mut maybe_covered_deltas: Vec<Arc<PersistentLayerDesc>> = Vec::new();
for item in items {
let (reached_lsn, is_readpoint) = match &item {
Item::ReadPoint { lsn } => (lsn, true),
Item::Layer(layer) => (&layer.lsn_range.start, false),
};
maybe_covered_deltas.retain(|d| {
if *reached_lsn >= d.lsn_range.start && is_readpoint {
// We encountered a readpoint within the delta layer: it is visible
results.push((d.clone(), LayerVisibilityHint::Visible));
false
} else if *reached_lsn < d.lsn_range.start {
// We passed the layer's range without encountering a read point: it is not visible
results.push((d.clone(), LayerVisibilityHint::Covered));
false
} else {
// We're still in the delta layer: continue iterating
true
}
});
match item {
Item::ReadPoint { lsn: _lsn } => {
// TODO: propagate the child timeline's shadow from their own run of this function, so that we don't have
// to assume that the whole key range is visible at the branch point.
shadow.reset();
}
Item::Layer(layer) => {
let visibility = if layer.is_delta() {
if shadow.contains(layer.get_key_range()) {
// If a layer isn't visible based on current state, we must defer deciding whether
// it is truly not visible until we have advanced past the delta's range: we might
// encounter another branch point within this delta layer's LSN range.
maybe_covered_deltas.push(layer);
continue;
} else {
LayerVisibilityHint::Visible
}
} else {
let modified = shadow.cover(layer.get_key_range());
if modified {
// An image layer in a region which wasn't fully covered yet: this layer is visible, but layers below it will be covered
LayerVisibilityHint::Visible
} else {
// An image layer in a region that was already covered
LayerVisibilityHint::Covered
}
};
results.push((layer, visibility));
}
}
}
// Drain any remaining maybe_covered deltas
results.extend(
maybe_covered_deltas
.into_iter()
.map(|d| (d, LayerVisibilityHint::Covered)),
);
(results, shadow.to_keyspace())
}
}
#[cfg(test)]
mod tests {
use pageserver_api::keyspace::KeySpace;
use crate::tenant::{storage_layer::LayerName, IndexPart};
use pageserver_api::{
key::DBDIR_KEY,
keyspace::{KeySpace, KeySpaceRandomAccum},
};
use std::{collections::HashMap, path::PathBuf};
use utils::{
id::{TenantId, TimelineId},
shard::TenantShardId,
};
use super::*;
@@ -1002,4 +1175,299 @@ mod tests {
}
}
}
#[test]
fn layer_visibility_basic() {
// A simple synthetic input, as a smoke test.
let tenant_shard_id = TenantShardId::unsharded(TenantId::generate());
let timeline_id = TimelineId::generate();
let mut layer_map = LayerMap::default();
let mut updates = layer_map.batch_update();
const FAKE_LAYER_SIZE: u64 = 1024;
let inject_delta = |updates: &mut BatchedUpdates,
key_start: i128,
key_end: i128,
lsn_start: u64,
lsn_end: u64| {
let desc = PersistentLayerDesc::new_delta(
tenant_shard_id,
timeline_id,
Range {
start: Key::from_i128(key_start),
end: Key::from_i128(key_end),
},
Range {
start: Lsn(lsn_start),
end: Lsn(lsn_end),
},
1024,
);
updates.insert_historic(desc.clone());
desc
};
let inject_image =
|updates: &mut BatchedUpdates, key_start: i128, key_end: i128, lsn: u64| {
let desc = PersistentLayerDesc::new_img(
tenant_shard_id,
timeline_id,
Range {
start: Key::from_i128(key_start),
end: Key::from_i128(key_end),
},
Lsn(lsn),
FAKE_LAYER_SIZE,
);
updates.insert_historic(desc.clone());
desc
};
//
// Construct our scenario: the following lines go in backward-LSN order, constructing the various scenarios
// we expect to handle. You can follow these examples through in the same order as they would be processed
// by the function under test.
//
let mut read_points = vec![Lsn(1000)];
// A delta ahead of any image layer
let ahead_layer = inject_delta(&mut updates, 10, 20, 101, 110);
// An image layer is visible and covers some layers beneath itself
let visible_covering_img = inject_image(&mut updates, 5, 25, 99);
// A delta layer covered by the image layer: should be covered
let covered_delta = inject_delta(&mut updates, 10, 20, 90, 100);
// A delta layer partially covered by an image layer: should be visible
let partially_covered_delta = inject_delta(&mut updates, 1, 7, 90, 100);
// A delta layer not covered by an image layer: should be visible
let not_covered_delta = inject_delta(&mut updates, 1, 4, 90, 100);
// An image layer covered by the image layer above: should be covered
let covered_image = inject_image(&mut updates, 10, 20, 89);
// An image layer partially covered by an image layer: should be visible
let partially_covered_image = inject_image(&mut updates, 1, 7, 89);
// An image layer not covered by an image layer: should be visible
let not_covered_image = inject_image(&mut updates, 1, 4, 89);
// A read point: this will make subsequent layers below here visible, even if there are
// more recent layers covering them.
read_points.push(Lsn(80));
// A delta layer covered by an earlier image layer, but visible to a readpoint below that covering layer
let covered_delta_below_read_point = inject_delta(&mut updates, 10, 20, 70, 79);
// A delta layer whose end LSN is covered, but where a read point is present partway through its LSN range:
// the read point should make it visible, even though its end LSN is covered
let covering_img_between_read_points = inject_image(&mut updates, 10, 20, 69);
let covered_delta_between_read_points = inject_delta(&mut updates, 10, 15, 67, 69);
read_points.push(Lsn(65));
let covered_delta_intersects_read_point = inject_delta(&mut updates, 15, 20, 60, 69);
let visible_img_after_last_read_point = inject_image(&mut updates, 10, 20, 65);
updates.flush();
let (layer_visibilities, shadow) = layer_map.get_visibility(read_points);
let layer_visibilities = layer_visibilities.into_iter().collect::<HashMap<_, _>>();
assert_eq!(
layer_visibilities.get(&ahead_layer),
Some(&LayerVisibilityHint::Visible)
);
assert_eq!(
layer_visibilities.get(&visible_covering_img),
Some(&LayerVisibilityHint::Visible)
);
assert_eq!(
layer_visibilities.get(&covered_delta),
Some(&LayerVisibilityHint::Covered)
);
assert_eq!(
layer_visibilities.get(&partially_covered_delta),
Some(&LayerVisibilityHint::Visible)
);
assert_eq!(
layer_visibilities.get(&not_covered_delta),
Some(&LayerVisibilityHint::Visible)
);
assert_eq!(
layer_visibilities.get(&covered_image),
Some(&LayerVisibilityHint::Covered)
);
assert_eq!(
layer_visibilities.get(&partially_covered_image),
Some(&LayerVisibilityHint::Visible)
);
assert_eq!(
layer_visibilities.get(&not_covered_image),
Some(&LayerVisibilityHint::Visible)
);
assert_eq!(
layer_visibilities.get(&covered_delta_below_read_point),
Some(&LayerVisibilityHint::Visible)
);
assert_eq!(
layer_visibilities.get(&covering_img_between_read_points),
Some(&LayerVisibilityHint::Visible)
);
assert_eq!(
layer_visibilities.get(&covered_delta_between_read_points),
Some(&LayerVisibilityHint::Covered)
);
assert_eq!(
layer_visibilities.get(&covered_delta_intersects_read_point),
Some(&LayerVisibilityHint::Visible)
);
assert_eq!(
layer_visibilities.get(&visible_img_after_last_read_point),
Some(&LayerVisibilityHint::Visible)
);
// Shadow should include all the images below the last read point
let expected_shadow = KeySpace {
ranges: vec![Key::from_i128(10)..Key::from_i128(20)],
};
assert_eq!(shadow, expected_shadow);
}
fn fixture_path(relative: &str) -> PathBuf {
PathBuf::from(env!("CARGO_MANIFEST_DIR")).join(relative)
}
#[test]
fn layer_visibility_realistic() {
// Load a large example layermap
let index_raw = std::fs::read_to_string(fixture_path(
"test_data/indices/mixed_workload/index_part.json",
))
.unwrap();
let index: IndexPart = serde_json::from_str::<IndexPart>(&index_raw).unwrap();
let tenant_id = TenantId::generate();
let tenant_shard_id = TenantShardId::unsharded(tenant_id);
let timeline_id = TimelineId::generate();
let mut layer_map = LayerMap::default();
let mut updates = layer_map.batch_update();
for (layer_name, layer_metadata) in index.layer_metadata {
let layer_desc = match layer_name {
LayerName::Image(layer_name) => PersistentLayerDesc {
key_range: layer_name.key_range.clone(),
lsn_range: layer_name.lsn_as_range(),
tenant_shard_id,
timeline_id,
is_delta: false,
file_size: layer_metadata.file_size,
},
LayerName::Delta(layer_name) => PersistentLayerDesc {
key_range: layer_name.key_range,
lsn_range: layer_name.lsn_range,
tenant_shard_id,
timeline_id,
is_delta: true,
file_size: layer_metadata.file_size,
},
};
updates.insert_historic(layer_desc);
}
updates.flush();
let read_points = vec![index.metadata.disk_consistent_lsn()];
let (layer_visibilities, shadow) = layer_map.get_visibility(read_points);
for (layer_desc, visibility) in &layer_visibilities {
tracing::info!("{layer_desc:?}: {visibility:?}");
eprintln!("{layer_desc:?}: {visibility:?}");
}
// The shadow should be non-empty, since there were some image layers
assert!(!shadow.ranges.is_empty());
// At least some layers should be marked covered
assert!(layer_visibilities
.iter()
.any(|i| matches!(i.1, LayerVisibilityHint::Covered)));
let layer_visibilities = layer_visibilities.into_iter().collect::<HashMap<_, _>>();
// Brute force validation: a layer should be marked covered if and only if there are image layers above it in LSN order which cover it
for (layer_desc, visible) in &layer_visibilities {
let mut coverage = KeySpaceRandomAccum::new();
let mut covered_by = Vec::new();
for other_layer in layer_map.iter_historic_layers() {
if &other_layer == layer_desc {
continue;
}
if !other_layer.is_delta()
&& other_layer.image_layer_lsn() >= Lsn(layer_desc.get_lsn_range().end.0 - 1)
&& other_layer.key_range.start <= layer_desc.key_range.end
&& layer_desc.key_range.start <= other_layer.key_range.end
{
coverage.add_range(other_layer.get_key_range());
covered_by.push((*other_layer).clone());
}
}
let coverage = coverage.to_keyspace();
let expect_visible = if coverage.ranges.len() == 1
&& coverage.contains(&layer_desc.key_range.start)
&& coverage.contains(&Key::from_i128(layer_desc.key_range.end.to_i128() - 1))
{
LayerVisibilityHint::Covered
} else {
LayerVisibilityHint::Visible
};
if expect_visible != *visible {
eprintln!(
"Layer {}..{} @ {}..{} (delta={}) is {visible:?}, should be {expect_visible:?}",
layer_desc.key_range.start,
layer_desc.key_range.end,
layer_desc.lsn_range.start,
layer_desc.lsn_range.end,
layer_desc.is_delta()
);
if expect_visible == LayerVisibilityHint::Covered {
eprintln!("Covered by:");
for other in covered_by {
eprintln!(
" {}..{} @ {}",
other.get_key_range().start,
other.get_key_range().end,
other.image_layer_lsn()
);
}
if let Some(range) = coverage.ranges.first() {
eprintln!(
"Total coverage from contributing layers: {}..{}",
range.start, range.end
);
} else {
eprintln!(
"Total coverage from contributing layers: {:?}",
coverage.ranges
);
}
}
}
assert_eq!(expect_visible, *visible);
}
// Sanity: the layer that holds latest data for the DBDIR key should always be visible
// (just using this key as a key that will always exist for any layermap fixture)
let dbdir_layer = layer_map
.search(DBDIR_KEY, index.metadata.disk_consistent_lsn())
.unwrap();
assert!(matches!(
layer_visibilities.get(&dbdir_layer.layer).unwrap(),
LayerVisibilityHint::Visible
));
}
}

View File

@@ -521,6 +521,10 @@ impl<Value: Clone> BufferedHistoricLayerCoverage<Value> {
Ok(&self.historic_coverage)
}
pub(crate) fn len(&self) -> usize {
self.layers.len()
}
}
#[test]

View File

@@ -111,7 +111,7 @@ impl TryFrom<&TimelineMetadataBodyV2> for TimelineMetadataHeader {
#[error("re-serializing for crc32 failed")]
struct Crc32CalculationFailed(#[source] utils::bin_ser::SerializeError);
const METADATA_HDR_SIZE: usize = std::mem::size_of::<TimelineMetadataHeader>();
const METADATA_HDR_SIZE: usize = size_of::<TimelineMetadataHeader>();
#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)]
struct TimelineMetadataBodyV2 {

View File

@@ -55,7 +55,7 @@ use utils::id::{TenantId, TimelineId};
use super::remote_timeline_client::remote_tenant_path;
use super::secondary::SecondaryTenant;
use super::timeline::detach_ancestor::PreparedTimelineDetach;
use super::TenantSharedResources;
use super::{GlobalShutDown, TenantSharedResources};
/// For a tenant that appears in TenantsMap, it may either be
/// - `Attached`: has a full Tenant object, is elegible to service
@@ -116,8 +116,6 @@ pub(crate) enum ShardSelector {
/// Only return the 0th shard, if it is present. If a non-0th shard is present,
/// ignore it.
Zero,
/// Pick the first shard we find for the TenantId
First,
/// Pick the shard that holds this key
Page(Key),
/// The shard ID is known: pick the given shard
@@ -667,17 +665,20 @@ pub async fn init_tenant_mgr(
let tenant_dir_path = conf.tenant_path(&tenant_shard_id);
let shard_identity = location_conf.shard;
let slot = match location_conf.mode {
LocationMode::Attached(attached_conf) => TenantSlot::Attached(tenant_spawn(
conf,
tenant_shard_id,
&tenant_dir_path,
resources.clone(),
AttachedTenantConf::new(location_conf.tenant_conf, attached_conf),
shard_identity,
Some(init_order.clone()),
SpawnMode::Lazy,
&ctx,
)),
LocationMode::Attached(attached_conf) => TenantSlot::Attached(
tenant_spawn(
conf,
tenant_shard_id,
&tenant_dir_path,
resources.clone(),
AttachedTenantConf::new(location_conf.tenant_conf, attached_conf),
shard_identity,
Some(init_order.clone()),
SpawnMode::Lazy,
&ctx,
)
.expect("global shutdown during init_tenant_mgr cannot happen"),
),
LocationMode::Secondary(secondary_conf) => {
info!(
tenant_id = %tenant_shard_id.tenant_id,
@@ -725,7 +726,7 @@ fn tenant_spawn(
init_order: Option<InitializationOrder>,
mode: SpawnMode,
ctx: &RequestContext,
) -> Arc<Tenant> {
) -> Result<Arc<Tenant>, GlobalShutDown> {
// All these conditions should have been satisfied by our caller: the tenant dir exists, is a well formed
// path, and contains a configuration file. Assertions that do synchronous I/O are limited to debug mode
// to avoid impacting prod runtime performance.
@@ -1192,7 +1193,10 @@ impl TenantManager {
None,
spawn_mode,
ctx,
);
)
.map_err(|_: GlobalShutDown| {
UpsertLocationError::Unavailable(TenantMapError::ShuttingDown)
})?;
TenantSlot::Attached(tenant)
}
@@ -1313,7 +1317,7 @@ impl TenantManager {
None,
SpawnMode::Eager,
ctx,
);
)?;
slot_guard.upsert(TenantSlot::Attached(tenant))?;
@@ -1384,33 +1388,32 @@ impl TenantManager {
tenant_shard_id: TenantShardId,
) -> Result<(), DeleteTenantError> {
let remote_path = remote_tenant_path(&tenant_shard_id);
let keys = match self
.resources
.remote_storage
.list(
Some(&remote_path),
remote_storage::ListingMode::NoDelimiter,
None,
&self.cancel,
)
.await
{
Ok(listing) => listing.keys,
Err(remote_storage::DownloadError::Cancelled) => {
return Err(DeleteTenantError::Cancelled)
}
Err(remote_storage::DownloadError::NotFound) => return Ok(()),
Err(other) => return Err(DeleteTenantError::Other(anyhow::anyhow!(other))),
};
let mut keys_stream = self.resources.remote_storage.list_streaming(
Some(&remote_path),
remote_storage::ListingMode::NoDelimiter,
None,
&self.cancel,
);
while let Some(chunk) = keys_stream.next().await {
let keys = match chunk {
Ok(listing) => listing.keys,
Err(remote_storage::DownloadError::Cancelled) => {
return Err(DeleteTenantError::Cancelled)
}
Err(remote_storage::DownloadError::NotFound) => return Ok(()),
Err(other) => return Err(DeleteTenantError::Other(anyhow::anyhow!(other))),
};
if keys.is_empty() {
tracing::info!("Remote storage already deleted");
} else {
tracing::info!("Deleting {} keys from remote storage", keys.len());
self.resources
.remote_storage
.delete_objects(&keys, &self.cancel)
.await?;
if keys.is_empty() {
tracing::info!("Remote storage already deleted");
} else {
tracing::info!("Deleting {} keys from remote storage", keys.len());
let keys = keys.into_iter().map(|o| o.key).collect::<Vec<_>>();
self.resources
.remote_storage
.delete_objects(&keys, &self.cancel)
.await?;
}
}
Ok(())
@@ -2048,7 +2051,7 @@ impl TenantManager {
None,
SpawnMode::Eager,
ctx,
);
)?;
slot_guard.upsert(TenantSlot::Attached(tenant))?;
@@ -2089,7 +2092,6 @@ impl TenantManager {
};
match selector {
ShardSelector::First => return ShardResolveResult::Found(tenant.clone()),
ShardSelector::Zero if slot.0.shard_number == ShardNumber(0) => {
return ShardResolveResult::Found(tenant.clone())
}
@@ -2171,6 +2173,9 @@ pub(crate) enum GetActiveTenantError {
/// never happen.
#[error("Tenant is broken: {0}")]
Broken(String),
#[error("reconnect to switch tenant id")]
SwitchedTenant,
}
#[derive(Debug, thiserror::Error)]

View File

@@ -187,7 +187,7 @@ use camino::Utf8Path;
use chrono::{NaiveDateTime, Utc};
pub(crate) use download::download_initdb_tar_zst;
use pageserver_api::models::AuxFilePolicy;
use pageserver_api::models::{AuxFilePolicy, TimelineArchivalState};
use pageserver_api::shard::{ShardIndex, TenantShardId};
use scopeguard::ScopeGuard;
use tokio_util::sync::CancellationToken;
@@ -287,6 +287,14 @@ pub enum PersistIndexPartWithDeletedFlagError {
Other(#[from] anyhow::Error),
}
#[derive(Debug, thiserror::Error)]
pub enum WaitCompletionError {
#[error(transparent)]
NotInitialized(NotInitialized),
#[error("wait_completion aborted because upload queue was stopped")]
UploadQueueShutDownOrStopped,
}
/// A client for accessing a timeline's data in remote storage.
///
/// This takes care of managing the number of connections, and balancing them
@@ -449,6 +457,17 @@ impl RemoteTimelineClient {
.unwrap_or(false)
}
/// Returns whether the timeline is archived.
/// Return None if the remote index_part hasn't been downloaded yet.
pub(crate) fn is_archived(&self) -> Option<bool> {
self.upload_queue
.lock()
.unwrap()
.initialized_mut()
.map(|q| q.clean.0.archived_at.is_some())
.ok()
}
fn update_remote_physical_size_gauge(&self, current_remote_index_part: Option<&IndexPart>) {
let size: u64 = if let Some(current_remote_index_part) = current_remote_index_part {
current_remote_index_part
@@ -609,7 +628,7 @@ impl RemoteTimelineClient {
Ok(())
}
/// Launch an index-file upload operation in the background, with only aux_file_policy flag updated.
/// Launch an index-file upload operation in the background, with only the `aux_file_policy` flag updated.
pub(crate) fn schedule_index_upload_for_aux_file_policy_update(
self: &Arc<Self>,
last_aux_file_policy: Option<AuxFilePolicy>,
@@ -620,6 +639,48 @@ impl RemoteTimelineClient {
self.schedule_index_upload(upload_queue)?;
Ok(())
}
/// Launch an index-file upload operation in the background, with only the `archived_at` field updated.
///
/// Returns whether it is required to wait for the queue to be empty to ensure that the change is uploaded,
/// so either if the change is already sitting in the queue, but not commited yet, or the change has not
/// been in the queue yet.
pub(crate) fn schedule_index_upload_for_timeline_archival_state(
self: &Arc<Self>,
state: TimelineArchivalState,
) -> anyhow::Result<bool> {
let mut guard = self.upload_queue.lock().unwrap();
let upload_queue = guard.initialized_mut()?;
/// Returns Some(_) if a change is needed, and Some(true) if it's a
/// change needed to set archived_at.
fn need_change(
archived_at: &Option<NaiveDateTime>,
state: TimelineArchivalState,
) -> Option<bool> {
match (archived_at, state) {
(Some(_), TimelineArchivalState::Archived)
| (None, TimelineArchivalState::Unarchived) => {
// Nothing to do
tracing::info!("intended state matches present state");
None
}
(None, TimelineArchivalState::Archived) => Some(true),
(Some(_), TimelineArchivalState::Unarchived) => Some(false),
}
}
let need_upload_scheduled = need_change(&upload_queue.dirty.archived_at, state);
if let Some(archived_at_set) = need_upload_scheduled {
let intended_archived_at = archived_at_set.then(|| Utc::now().naive_utc());
upload_queue.dirty.archived_at = intended_archived_at;
self.schedule_index_upload(upload_queue)?;
}
let need_wait = need_change(&upload_queue.clean.0.archived_at, state).is_some();
Ok(need_wait)
}
///
/// Launch an index-file upload operation in the background, if necessary.
///
@@ -630,7 +691,7 @@ impl RemoteTimelineClient {
///
/// Like schedule_index_upload_for_metadata_update(), this merely adds
/// the upload to the upload queue and returns quickly.
pub fn schedule_index_upload_for_file_changes(self: &Arc<Self>) -> anyhow::Result<()> {
pub fn schedule_index_upload_for_file_changes(self: &Arc<Self>) -> Result<(), NotInitialized> {
let mut guard = self.upload_queue.lock().unwrap();
let upload_queue = guard.initialized_mut()?;
@@ -645,7 +706,7 @@ impl RemoteTimelineClient {
fn schedule_index_upload(
self: &Arc<Self>,
upload_queue: &mut UploadQueueInitialized,
) -> anyhow::Result<()> {
) -> Result<(), NotInitialized> {
let disk_consistent_lsn = upload_queue.dirty.metadata.disk_consistent_lsn();
// fix up the duplicated field
upload_queue.dirty.disk_consistent_lsn = disk_consistent_lsn;
@@ -653,7 +714,7 @@ impl RemoteTimelineClient {
// make sure it serializes before doing it in perform_upload_task so that it doesn't
// look like a retryable error
let void = std::io::sink();
serde_json::to_writer(void, &upload_queue.dirty).context("serialize index_part.json")?;
serde_json::to_writer(void, &upload_queue.dirty).expect("serialize index_part.json");
let index_part = &upload_queue.dirty;
@@ -699,7 +760,9 @@ impl RemoteTimelineClient {
self.schedule_barrier0(upload_queue)
};
Self::wait_completion0(receiver).await
Self::wait_completion0(receiver)
.await
.context("wait completion")
}
/// Schedules uploading a new version of `index_part.json` with the given layers added,
@@ -732,7 +795,9 @@ impl RemoteTimelineClient {
barrier
};
Self::wait_completion0(barrier).await
Self::wait_completion0(barrier)
.await
.context("wait completion")
}
/// Launch an upload operation in the background; the file is added to be included in next
@@ -740,7 +805,7 @@ impl RemoteTimelineClient {
pub(crate) fn schedule_layer_file_upload(
self: &Arc<Self>,
layer: ResidentLayer,
) -> anyhow::Result<()> {
) -> Result<(), NotInitialized> {
let mut guard = self.upload_queue.lock().unwrap();
let upload_queue = guard.initialized_mut()?;
@@ -826,7 +891,7 @@ impl RemoteTimelineClient {
self: &Arc<Self>,
upload_queue: &mut UploadQueueInitialized,
names: I,
) -> anyhow::Result<Vec<(LayerName, LayerFileMetadata)>>
) -> Result<Vec<(LayerName, LayerFileMetadata)>, NotInitialized>
where
I: IntoIterator<Item = LayerName>,
{
@@ -952,7 +1017,7 @@ impl RemoteTimelineClient {
self: &Arc<Self>,
compacted_from: &[Layer],
compacted_to: &[ResidentLayer],
) -> anyhow::Result<()> {
) -> Result<(), NotInitialized> {
let mut guard = self.upload_queue.lock().unwrap();
let upload_queue = guard.initialized_mut()?;
@@ -969,10 +1034,12 @@ impl RemoteTimelineClient {
}
/// Wait for all previously scheduled uploads/deletions to complete
pub(crate) async fn wait_completion(self: &Arc<Self>) -> anyhow::Result<()> {
pub(crate) async fn wait_completion(self: &Arc<Self>) -> Result<(), WaitCompletionError> {
let receiver = {
let mut guard = self.upload_queue.lock().unwrap();
let upload_queue = guard.initialized_mut()?;
let upload_queue = guard
.initialized_mut()
.map_err(WaitCompletionError::NotInitialized)?;
self.schedule_barrier0(upload_queue)
};
@@ -981,9 +1048,9 @@ impl RemoteTimelineClient {
async fn wait_completion0(
mut receiver: tokio::sync::watch::Receiver<()>,
) -> anyhow::Result<()> {
) -> Result<(), WaitCompletionError> {
if receiver.changed().await.is_err() {
anyhow::bail!("wait_completion aborted because upload queue was stopped");
return Err(WaitCompletionError::UploadQueueShutDownOrStopped);
}
Ok(())
@@ -1311,6 +1378,18 @@ impl RemoteTimelineClient {
.dirty
.layer_metadata
.drain()
.filter(|(_file_name, meta)| {
// Filter out layers that belonged to an ancestor shard. Since we are deleting the whole timeline from
// all shards anyway, we _could_ delete these, but
// - it creates a potential race if other shards are still
// using the layers while this shard deletes them.
// - it means that if we rolled back the shard split, the ancestor shards would be in a state where
// these timelines are present but corrupt (their index exists but some layers don't)
//
// These layers will eventually be cleaned up by the scrubber when it does physical GC.
meta.shard.shard_number == self.tenant_shard_id.shard_number
&& meta.shard.shard_count == self.tenant_shard_id.shard_count
})
.map(|(file_name, meta)| {
remote_layer_path(
&self.tenant_shard_id.tenant_id,
@@ -1366,12 +1445,13 @@ impl RemoteTimelineClient {
// marker via its deleted_at attribute
let latest_index = remaining
.iter()
.filter(|p| {
p.object_name()
.filter(|o| {
o.key
.object_name()
.map(|n| n.starts_with(IndexPart::FILE_NAME))
.unwrap_or(false)
})
.filter_map(|path| parse_remote_index_path(path.clone()).map(|gen| (path, gen)))
.filter_map(|o| parse_remote_index_path(o.key.clone()).map(|gen| (o.key.clone(), gen)))
.max_by_key(|i| i.1)
.map(|i| i.0.clone())
.unwrap_or(
@@ -1382,14 +1462,12 @@ impl RemoteTimelineClient {
let remaining_layers: Vec<RemotePath> = remaining
.into_iter()
.filter(|p| {
if p == &latest_index {
return false;
.filter_map(|o| {
if o.key == latest_index || o.key.object_name() == Some(INITDB_PRESERVED_PATH) {
None
} else {
Some(o.key)
}
if p.object_name() == Some(INITDB_PRESERVED_PATH) {
return false;
}
true
})
.inspect(|path| {
if let Some(name) = path.object_name() {

View File

@@ -295,10 +295,11 @@ where
};
}
for key in listing.keys {
let object_name = key
for object in listing.keys {
let object_name = object
.key
.object_name()
.ok_or_else(|| anyhow::anyhow!("object name for key {key}"))?;
.ok_or_else(|| anyhow::anyhow!("object name for key {}", object.key))?;
other_prefixes.insert(object_name.to_string());
}
@@ -459,7 +460,7 @@ pub(crate) async fn download_index_part(
// is <= our own. See "Finding the remote indices for timelines" in docs/rfcs/025-generation-numbers.md
let max_previous_generation = indices
.into_iter()
.filter_map(parse_remote_index_path)
.filter_map(|o| parse_remote_index_path(o.key))
.filter(|g| g <= &my_generation)
.max();

View File

@@ -32,6 +32,10 @@ pub struct IndexPart {
#[serde(skip_serializing_if = "Option::is_none")]
pub deleted_at: Option<NaiveDateTime>,
#[serde(default)]
#[serde(skip_serializing_if = "Option::is_none")]
pub archived_at: Option<NaiveDateTime>,
/// Per layer file name metadata, which can be present for a present or missing layer file.
///
/// Older versions of `IndexPart` will not have this property or have only a part of metadata
@@ -80,10 +84,11 @@ impl IndexPart {
/// - 5: lineage was added
/// - 6: last_aux_file_policy is added.
/// - 7: metadata_bytes is no longer written, but still read
const LATEST_VERSION: usize = 7;
/// - 8: added `archived_at`
const LATEST_VERSION: usize = 8;
// Versions we may see when reading from a bucket.
pub const KNOWN_VERSIONS: &'static [usize] = &[1, 2, 3, 4, 5, 6, 7];
pub const KNOWN_VERSIONS: &'static [usize] = &[1, 2, 3, 4, 5, 6, 7, 8];
pub const FILE_NAME: &'static str = "index_part.json";
@@ -94,6 +99,7 @@ impl IndexPart {
disk_consistent_lsn: metadata.disk_consistent_lsn(),
metadata,
deleted_at: None,
archived_at: None,
lineage: Default::default(),
last_aux_file_policy: None,
}
@@ -284,6 +290,7 @@ mod tests {
disk_consistent_lsn: "0/16960E8".parse::<Lsn>().unwrap(),
metadata: TimelineMetadata::from_bytes(&[113,11,159,210,0,54,0,4,0,0,0,0,1,105,96,232,1,0,0,0,0,1,105,96,112,0,0,0,0,0,0,0,0,0,0,0,0,0,1,105,96,112,0,0,0,0,1,105,96,112,0,0,0,14,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0]).unwrap(),
deleted_at: None,
archived_at: None,
lineage: Lineage::default(),
last_aux_file_policy: None,
};
@@ -326,6 +333,7 @@ mod tests {
disk_consistent_lsn: "0/16960E8".parse::<Lsn>().unwrap(),
metadata: TimelineMetadata::from_bytes(&[113,11,159,210,0,54,0,4,0,0,0,0,1,105,96,232,1,0,0,0,0,1,105,96,112,0,0,0,0,0,0,0,0,0,0,0,0,0,1,105,96,112,0,0,0,0,1,105,96,112,0,0,0,14,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0]).unwrap(),
deleted_at: None,
archived_at: None,
lineage: Lineage::default(),
last_aux_file_policy: None,
};
@@ -369,6 +377,7 @@ mod tests {
disk_consistent_lsn: "0/16960E8".parse::<Lsn>().unwrap(),
metadata: TimelineMetadata::from_bytes(&[113,11,159,210,0,54,0,4,0,0,0,0,1,105,96,232,1,0,0,0,0,1,105,96,112,0,0,0,0,0,0,0,0,0,0,0,0,0,1,105,96,112,0,0,0,0,1,105,96,112,0,0,0,14,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0]).unwrap(),
deleted_at: Some(parse_naive_datetime("2023-07-31T09:00:00.123000000")),
archived_at: None,
lineage: Lineage::default(),
last_aux_file_policy: None,
};
@@ -415,6 +424,7 @@ mod tests {
])
.unwrap(),
deleted_at: None,
archived_at: None,
lineage: Lineage::default(),
last_aux_file_policy: None,
};
@@ -456,6 +466,7 @@ mod tests {
disk_consistent_lsn: "0/16960E8".parse::<Lsn>().unwrap(),
metadata: TimelineMetadata::from_bytes(&[113,11,159,210,0,54,0,4,0,0,0,0,1,105,96,232,1,0,0,0,0,1,105,96,112,0,0,0,0,0,0,0,0,0,0,0,0,0,1,105,96,112,0,0,0,0,1,105,96,112,0,0,0,14,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0]).unwrap(),
deleted_at: Some(parse_naive_datetime("2023-07-31T09:00:00.123000000")),
archived_at: None,
lineage: Lineage::default(),
last_aux_file_policy: None,
};
@@ -496,6 +507,7 @@ mod tests {
disk_consistent_lsn: Lsn::from_str("0/15A7618").unwrap(),
metadata: TimelineMetadata::from_bytes(&[226,88,25,241,0,46,0,4,0,0,0,0,1,90,118,24,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,78,244,32,0,0,0,0,1,78,244,32,0,0,0,16,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0]).unwrap(),
deleted_at: None,
archived_at: None,
lineage: Lineage {
reparenting_history_truncated: false,
reparenting_history: vec![TimelineId::from_str("e1bfd8c633d713d279e6fcd2bcc15b6d").unwrap()],
@@ -545,6 +557,7 @@ mod tests {
disk_consistent_lsn: "0/16960E8".parse::<Lsn>().unwrap(),
metadata: TimelineMetadata::from_bytes(&[113,11,159,210,0,54,0,4,0,0,0,0,1,105,96,232,1,0,0,0,0,1,105,96,112,0,0,0,0,0,0,0,0,0,0,0,0,0,1,105,96,112,0,0,0,0,1,105,96,112,0,0,0,14,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0]).unwrap(),
deleted_at: Some(parse_naive_datetime("2023-07-31T09:00:00.123000000")),
archived_at: None,
lineage: Lineage {
reparenting_history_truncated: false,
reparenting_history: vec![TimelineId::from_str("e1bfd8c633d713d279e6fcd2bcc15b6d").unwrap()],
@@ -603,6 +616,63 @@ mod tests {
14,
).with_recalculated_checksum().unwrap(),
deleted_at: Some(parse_naive_datetime("2023-07-31T09:00:00.123000000")),
archived_at: None,
lineage: Default::default(),
last_aux_file_policy: Default::default(),
};
let part = IndexPart::from_s3_bytes(example.as_bytes()).unwrap();
assert_eq!(part, expected);
}
#[test]
fn v8_indexpart_is_parsed() {
let example = r#"{
"version": 8,
"layer_metadata":{
"000000000000000000000000000000000000-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF__0000000001696070-00000000016960E9": { "file_size": 25600000 },
"000000000000000000000000000000000000-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF__00000000016B59D8-00000000016B5A51": { "file_size": 9007199254741001 }
},
"disk_consistent_lsn":"0/16960E8",
"metadata": {
"disk_consistent_lsn": "0/16960E8",
"prev_record_lsn": "0/1696070",
"ancestor_timeline": "e45a7f37d3ee2ff17dc14bf4f4e3f52e",
"ancestor_lsn": "0/0",
"latest_gc_cutoff_lsn": "0/1696070",
"initdb_lsn": "0/1696070",
"pg_version": 14
},
"deleted_at": "2023-07-31T09:00:00.123",
"archived_at": "2023-04-29T09:00:00.123"
}"#;
let expected = IndexPart {
version: 8,
layer_metadata: HashMap::from([
("000000000000000000000000000000000000-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF__0000000001696070-00000000016960E9".parse().unwrap(), LayerFileMetadata {
file_size: 25600000,
generation: Generation::none(),
shard: ShardIndex::unsharded()
}),
("000000000000000000000000000000000000-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF__00000000016B59D8-00000000016B5A51".parse().unwrap(), LayerFileMetadata {
file_size: 9007199254741001,
generation: Generation::none(),
shard: ShardIndex::unsharded()
})
]),
disk_consistent_lsn: "0/16960E8".parse::<Lsn>().unwrap(),
metadata: TimelineMetadata::new(
Lsn::from_str("0/16960E8").unwrap(),
Some(Lsn::from_str("0/1696070").unwrap()),
Some(TimelineId::from_str("e45a7f37d3ee2ff17dc14bf4f4e3f52e").unwrap()),
Lsn::INVALID,
Lsn::from_str("0/1696070").unwrap(),
Lsn::from_str("0/1696070").unwrap(),
14,
).with_recalculated_checksum().unwrap(),
deleted_at: Some(parse_naive_datetime("2023-07-31T09:00:00.123000000")),
archived_at: Some(parse_naive_datetime("2023-04-29T09:00:00.123000000")),
lineage: Default::default(),
last_aux_file_policy: Default::default(),
};

View File

@@ -8,6 +8,9 @@ mod layer_desc;
mod layer_name;
pub mod merge_iterator;
#[cfg(test)]
pub mod split_writer;
use crate::context::{AccessStatsBehavior, RequestContext};
use crate::repository::Value;
use crate::walrecord::NeonWalRecord;
@@ -26,6 +29,7 @@ use utils::lsn::Lsn;
pub use delta_layer::{DeltaLayer, DeltaLayerWriter, ValueRef};
pub use image_layer::{ImageLayer, ImageLayerWriter};
pub use inmemory_layer::InMemoryLayer;
pub use inmemory_layer::SerializedBatch;
pub use layer_desc::{PersistentLayerDesc, PersistentLayerKey};
pub use layer_name::{DeltaLayerName, ImageLayerName, LayerName};
@@ -451,20 +455,14 @@ pub enum ValueReconstructResult {
/// than an authoritative value, so that we do not have to update it synchronously when changing the visibility
/// of layers (for example when creating a branch that makes some previously covered layers visible). It should
/// be used for cache management but not for correctness-critical checks.
#[derive(Default, Debug, Clone, PartialEq, Eq)]
pub(crate) enum LayerVisibilityHint {
#[derive(Debug, Clone, PartialEq, Eq)]
pub enum LayerVisibilityHint {
/// A Visible layer might be read while serving a read, because there is not an image layer between it
/// and a readable LSN (the tip of the branch or a child's branch point)
Visible,
/// A Covered layer probably won't be read right now, but _can_ be read in future if someone creates
/// a branch or ephemeral endpoint at an LSN below the layer that covers this.
#[allow(unused)]
Covered,
/// Calculating layer visibilty requires I/O, so until this has happened layers are loaded
/// in this state. Note that newly written layers may be called Visible immediately, this uninitialized
/// state is for when existing layers are constructed while loading a timeline.
#[default]
Uninitialized,
}
pub(crate) struct LayerAccessStats(std::sync::atomic::AtomicU64);
@@ -626,23 +624,30 @@ impl LayerAccessStats {
}
}
pub(crate) fn set_visibility(&self, visibility: LayerVisibilityHint) {
let value = match visibility {
LayerVisibilityHint::Visible => 0x1 << Self::VISIBILITY_SHIFT,
LayerVisibilityHint::Covered | LayerVisibilityHint::Uninitialized => 0x0,
};
self.write_bits(0x1 << Self::VISIBILITY_SHIFT, value);
}
pub(crate) fn visibility(&self) -> LayerVisibilityHint {
let read = self.0.load(std::sync::atomic::Ordering::Relaxed);
match (read >> Self::VISIBILITY_SHIFT) & 0x1 {
/// Helper for extracting the visibility hint from the literal value of our inner u64
fn decode_visibility(&self, bits: u64) -> LayerVisibilityHint {
match (bits >> Self::VISIBILITY_SHIFT) & 0x1 {
1 => LayerVisibilityHint::Visible,
0 => LayerVisibilityHint::Covered,
_ => unreachable!(),
}
}
/// Returns the old value which has been replaced
pub(crate) fn set_visibility(&self, visibility: LayerVisibilityHint) -> LayerVisibilityHint {
let value = match visibility {
LayerVisibilityHint::Visible => 0x1 << Self::VISIBILITY_SHIFT,
LayerVisibilityHint::Covered => 0x0,
};
let old_bits = self.write_bits(0x1 << Self::VISIBILITY_SHIFT, value);
self.decode_visibility(old_bits)
}
pub(crate) fn visibility(&self) -> LayerVisibilityHint {
let read = self.0.load(std::sync::atomic::Ordering::Relaxed);
self.decode_visibility(read)
}
}
/// Get a layer descriptor from a layer.

View File

@@ -36,13 +36,13 @@ use crate::tenant::block_io::{BlockBuf, BlockCursor, BlockLease, BlockReader, Fi
use crate::tenant::disk_btree::{
DiskBtreeBuilder, DiskBtreeIterator, DiskBtreeReader, VisitDirection,
};
use crate::tenant::storage_layer::{Layer, ValueReconstructResult, ValueReconstructState};
use crate::tenant::storage_layer::{ValueReconstructResult, ValueReconstructState};
use crate::tenant::timeline::GetVectoredError;
use crate::tenant::vectored_blob_io::{
BlobFlag, MaxVectoredReadBytes, StreamingVectoredReadPlanner, VectoredBlobReader, VectoredRead,
VectoredReadPlanner,
};
use crate::tenant::{PageReconstructError, Timeline};
use crate::tenant::PageReconstructError;
use crate::virtual_file::{self, VirtualFile};
use crate::{walrecord, TEMP_FILE_SUFFIX};
use crate::{DELTA_FILE_MAGIC, STORAGE_FORMAT_VERSION};
@@ -73,8 +73,7 @@ use utils::{
};
use super::{
AsLayerDesc, LayerAccessStats, LayerName, PersistentLayerDesc, ResidentLayer,
ValuesReconstructState,
AsLayerDesc, LayerAccessStats, LayerName, PersistentLayerDesc, ValuesReconstructState,
};
///
@@ -307,12 +306,10 @@ impl DeltaLayer {
.with_context(|| format!("Failed to load delta layer {}", self.path()))
}
async fn load_inner(&self, ctx: &RequestContext) -> Result<Arc<DeltaLayerInner>> {
async fn load_inner(&self, ctx: &RequestContext) -> anyhow::Result<Arc<DeltaLayerInner>> {
let path = self.path();
let loaded = DeltaLayerInner::load(&path, None, None, ctx)
.await
.and_then(|res| res)?;
let loaded = DeltaLayerInner::load(&path, None, None, ctx).await?;
// not production code
let actual_layer_name = LayerName::from_str(path.file_name().unwrap()).unwrap();
@@ -375,7 +372,6 @@ impl DeltaLayer {
/// 3. Call `finish`.
///
struct DeltaLayerWriterInner {
conf: &'static PageServerConf,
pub path: Utf8PathBuf,
timeline_id: TimelineId,
tenant_shard_id: TenantShardId,
@@ -386,6 +382,9 @@ struct DeltaLayerWriterInner {
tree: DiskBtreeBuilder<BlockBuf, DELTA_KEY_SIZE>,
blob_writer: BlobWriter<true>,
// Number of key-lsns in the layer.
num_keys: usize,
}
impl DeltaLayerWriterInner {
@@ -419,7 +418,6 @@ impl DeltaLayerWriterInner {
let tree_builder = DiskBtreeBuilder::new(block_buf);
Ok(Self {
conf,
path,
timeline_id,
tenant_shard_id,
@@ -427,6 +425,7 @@ impl DeltaLayerWriterInner {
lsn_range,
tree: tree_builder,
blob_writer,
num_keys: 0,
})
}
@@ -469,7 +468,7 @@ impl DeltaLayerWriterInner {
.write_blob_maybe_compressed(val, ctx, compression)
.await;
let off = match res {
Ok(off) => off,
Ok((off, _)) => off,
Err(e) => return (val, Err(anyhow::anyhow!(e))),
};
@@ -477,6 +476,9 @@ impl DeltaLayerWriterInner {
let delta_key = DeltaKey::from_key_lsn(&key, lsn);
let res = self.tree.append(&delta_key.0, blob_ref.0);
self.num_keys += 1;
(val, res.map_err(|e| anyhow::anyhow!(e)))
}
@@ -490,11 +492,10 @@ impl DeltaLayerWriterInner {
async fn finish(
self,
key_end: Key,
timeline: &Arc<Timeline>,
ctx: &RequestContext,
) -> anyhow::Result<ResidentLayer> {
) -> anyhow::Result<(PersistentLayerDesc, Utf8PathBuf)> {
let temp_path = self.path.clone();
let result = self.finish0(key_end, timeline, ctx).await;
let result = self.finish0(key_end, ctx).await;
if result.is_err() {
tracing::info!(%temp_path, "cleaning up temporary file after error during writing");
if let Err(e) = std::fs::remove_file(&temp_path) {
@@ -507,9 +508,8 @@ impl DeltaLayerWriterInner {
async fn finish0(
self,
key_end: Key,
timeline: &Arc<Timeline>,
ctx: &RequestContext,
) -> anyhow::Result<ResidentLayer> {
) -> anyhow::Result<(PersistentLayerDesc, Utf8PathBuf)> {
let index_start_blk =
((self.blob_writer.size() + PAGE_SZ as u64 - 1) / PAGE_SZ as u64) as u32;
@@ -574,11 +574,9 @@ impl DeltaLayerWriterInner {
// fsync the file
file.sync_all().await?;
let layer = Layer::finish_creating(self.conf, timeline, desc, &self.path)?;
trace!("created delta layer {}", self.path);
trace!("created delta layer {}", layer.local_path());
Ok(layer)
Ok((desc, self.path))
}
}
@@ -679,14 +677,20 @@ impl DeltaLayerWriter {
pub(crate) async fn finish(
mut self,
key_end: Key,
timeline: &Arc<Timeline>,
ctx: &RequestContext,
) -> anyhow::Result<ResidentLayer> {
self.inner
.take()
.unwrap()
.finish(key_end, timeline, ctx)
.await
) -> anyhow::Result<(PersistentLayerDesc, Utf8PathBuf)> {
self.inner.take().unwrap().finish(key_end, ctx).await
}
#[cfg(test)]
pub(crate) fn num_keys(&self) -> usize {
self.inner.as_ref().unwrap().num_keys
}
#[cfg(test)]
pub(crate) fn estimated_size(&self) -> u64 {
let inner = self.inner.as_ref().unwrap();
inner.blob_writer.size() + inner.tree.borrow_writer().size() + PAGE_SZ as u64
}
}
@@ -760,27 +764,24 @@ impl DeltaLayerInner {
&self.layer_lsn_range
}
/// Returns nested result following Result<Result<_, OpErr>, Critical>:
/// - inner has the success or transient failure
/// - outer has the permanent failure
pub(super) async fn load(
path: &Utf8Path,
summary: Option<Summary>,
max_vectored_read_bytes: Option<MaxVectoredReadBytes>,
ctx: &RequestContext,
) -> Result<Result<Self, anyhow::Error>, anyhow::Error> {
let file = match VirtualFile::open(path, ctx).await {
Ok(file) => file,
Err(e) => return Ok(Err(anyhow::Error::new(e).context("open layer file"))),
};
) -> anyhow::Result<Self> {
let file = VirtualFile::open(path, ctx)
.await
.context("open layer file")?;
let file_id = page_cache::next_file_id();
let block_reader = FileBlockReader::new(&file, file_id);
let summary_blk = match block_reader.read_blk(0, ctx).await {
Ok(blk) => blk,
Err(e) => return Ok(Err(anyhow::Error::new(e).context("read first block"))),
};
let summary_blk = block_reader
.read_blk(0, ctx)
.await
.context("read first block")?;
// TODO: this should be an assertion instead; see ImageLayerInner::load
let actual_summary =
@@ -802,7 +803,7 @@ impl DeltaLayerInner {
}
}
Ok(Ok(DeltaLayerInner {
Ok(DeltaLayerInner {
file,
file_id,
index_start_blk: actual_summary.index_start_blk,
@@ -810,7 +811,7 @@ impl DeltaLayerInner {
max_vectored_read_bytes,
layer_key_range: actual_summary.key_range,
layer_lsn_range: actual_summary.lsn_range,
}))
})
}
pub(super) async fn get_value_reconstruct_data(
@@ -1674,8 +1675,9 @@ pub(crate) mod test {
use super::*;
use crate::repository::Value;
use crate::tenant::harness::TIMELINE_ID;
use crate::tenant::storage_layer::{Layer, ResidentLayer};
use crate::tenant::vectored_blob_io::StreamingVectoredReadPlanner;
use crate::tenant::Tenant;
use crate::tenant::{Tenant, Timeline};
use crate::{
context::DownloadBehavior,
task_mgr::TaskKind,
@@ -1969,9 +1971,8 @@ pub(crate) mod test {
res?;
}
let resident = writer
.finish(entries_meta.key_range.end, &timeline, &ctx)
.await?;
let (desc, path) = writer.finish(entries_meta.key_range.end, &ctx).await?;
let resident = Layer::finish_creating(harness.conf, &timeline, desc, &path)?;
let inner = resident.get_as_delta(&ctx).await?;
@@ -2160,7 +2161,8 @@ pub(crate) mod test {
.await
.unwrap();
let copied_layer = writer.finish(Key::MAX, &branch, ctx).await.unwrap();
let (desc, path) = writer.finish(Key::MAX, ctx).await.unwrap();
let copied_layer = Layer::finish_creating(tenant.conf, &branch, desc, &path).unwrap();
copied_layer.get_as_delta(ctx).await.unwrap();
@@ -2288,7 +2290,9 @@ pub(crate) mod test {
for (key, lsn, value) in deltas {
writer.put_value(key, lsn, value, ctx).await?;
}
let delta_layer = writer.finish(key_end, tline, ctx).await?;
let (desc, path) = writer.finish(key_end, ctx).await?;
let delta_layer = Layer::finish_creating(tenant.conf, tline, desc, &path)?;
Ok::<_, anyhow::Error>(delta_layer)
}

View File

@@ -265,9 +265,8 @@ impl ImageLayer {
async fn load_inner(&self, ctx: &RequestContext) -> Result<ImageLayerInner> {
let path = self.path();
let loaded = ImageLayerInner::load(&path, self.desc.image_layer_lsn(), None, None, ctx)
.await
.and_then(|res| res)?;
let loaded =
ImageLayerInner::load(&path, self.desc.image_layer_lsn(), None, None, ctx).await?;
// not production code
let actual_layer_name = LayerName::from_str(path.file_name().unwrap()).unwrap();
@@ -385,17 +384,16 @@ impl ImageLayerInner {
summary: Option<Summary>,
max_vectored_read_bytes: Option<MaxVectoredReadBytes>,
ctx: &RequestContext,
) -> Result<Result<Self, anyhow::Error>, anyhow::Error> {
let file = match VirtualFile::open(path, ctx).await {
Ok(file) => file,
Err(e) => return Ok(Err(anyhow::Error::new(e).context("open layer file"))),
};
) -> anyhow::Result<Self> {
let file = VirtualFile::open(path, ctx)
.await
.context("open layer file")?;
let file_id = page_cache::next_file_id();
let block_reader = FileBlockReader::new(&file, file_id);
let summary_blk = match block_reader.read_blk(0, ctx).await {
Ok(blk) => blk,
Err(e) => return Ok(Err(anyhow::Error::new(e).context("read first block"))),
};
let summary_blk = block_reader
.read_blk(0, ctx)
.await
.context("read first block")?;
// length is the only way how this could fail, so it's not actually likely at all unless
// read_blk returns wrong sized block.
@@ -420,7 +418,7 @@ impl ImageLayerInner {
}
}
Ok(Ok(ImageLayerInner {
Ok(ImageLayerInner {
index_start_blk: actual_summary.index_start_blk,
index_root_blk: actual_summary.index_root_blk,
lsn,
@@ -428,7 +426,7 @@ impl ImageLayerInner {
file_id,
max_vectored_read_bytes,
key_range: actual_summary.key_range,
}))
})
}
pub(super) async fn get_value_reconstruct_data(
@@ -736,8 +734,22 @@ struct ImageLayerWriterInner {
// Total uncompressed bytes passed into put_image
uncompressed_bytes: u64,
// Like `uncompressed_bytes`,
// but only of images we might consider for compression
uncompressed_bytes_eligible: u64,
// Like `uncompressed_bytes`, but only of images
// where we have chosen their compressed form
uncompressed_bytes_chosen: u64,
// Number of keys in the layer.
num_keys: usize,
blob_writer: BlobWriter<false>,
tree: DiskBtreeBuilder<BlockBuf, KEY_SIZE>,
#[cfg_attr(not(feature = "testing"), allow(dead_code))]
last_written_key: Key,
}
impl ImageLayerWriterInner {
@@ -792,6 +804,10 @@ impl ImageLayerWriterInner {
tree: tree_builder,
blob_writer,
uncompressed_bytes: 0,
uncompressed_bytes_eligible: 0,
uncompressed_bytes_chosen: 0,
num_keys: 0,
last_written_key: Key::MIN,
};
Ok(writer)
@@ -810,18 +826,33 @@ impl ImageLayerWriterInner {
) -> anyhow::Result<()> {
ensure!(self.key_range.contains(&key));
let compression = self.conf.image_compression;
self.uncompressed_bytes += img.len() as u64;
let uncompressed_len = img.len() as u64;
self.uncompressed_bytes += uncompressed_len;
self.num_keys += 1;
let (_img, res) = self
.blob_writer
.write_blob_maybe_compressed(img, ctx, compression)
.await;
// TODO: re-use the buffer for `img` further upstack
let off = res?;
let (off, compression_info) = res?;
if compression_info.compressed_size.is_some() {
// The image has been considered for compression at least
self.uncompressed_bytes_eligible += uncompressed_len;
}
if compression_info.written_compressed {
// The image has been compressed
self.uncompressed_bytes_chosen += uncompressed_len;
}
let mut keybuf: [u8; KEY_SIZE] = [0u8; KEY_SIZE];
key.write_to_byte_slice(&mut keybuf);
self.tree.append(&keybuf, off)?;
#[cfg(feature = "testing")]
{
self.last_written_key = key;
}
Ok(())
}
@@ -832,6 +863,7 @@ impl ImageLayerWriterInner {
self,
timeline: &Arc<Timeline>,
ctx: &RequestContext,
end_key: Option<Key>,
) -> anyhow::Result<ResidentLayer> {
let index_start_blk =
((self.blob_writer.size() + PAGE_SZ as u64 - 1) / PAGE_SZ as u64) as u32;
@@ -839,6 +871,9 @@ impl ImageLayerWriterInner {
// Calculate compression ratio
let compressed_size = self.blob_writer.size() - PAGE_SZ as u64; // Subtract PAGE_SZ for header
crate::metrics::COMPRESSION_IMAGE_INPUT_BYTES.inc_by(self.uncompressed_bytes);
crate::metrics::COMPRESSION_IMAGE_INPUT_BYTES_CONSIDERED
.inc_by(self.uncompressed_bytes_eligible);
crate::metrics::COMPRESSION_IMAGE_INPUT_BYTES_CHOSEN.inc_by(self.uncompressed_bytes_chosen);
crate::metrics::COMPRESSION_IMAGE_OUTPUT_BYTES.inc_by(compressed_size);
let mut file = self.blob_writer.into_inner();
@@ -879,11 +914,23 @@ impl ImageLayerWriterInner {
let desc = PersistentLayerDesc::new_img(
self.tenant_shard_id,
self.timeline_id,
self.key_range.clone(),
if let Some(end_key) = end_key {
self.key_range.start..end_key
} else {
self.key_range.clone()
},
self.lsn,
metadata.len(),
);
#[cfg(feature = "testing")]
if let Some(end_key) = end_key {
assert!(
self.last_written_key < end_key,
"written key violates end_key range"
);
}
// Note: Because we open the file in write-only mode, we cannot
// reuse the same VirtualFile for reading later. That's why we don't
// set inner.file here. The first read will have to re-open it.
@@ -960,6 +1007,18 @@ impl ImageLayerWriter {
self.inner.as_mut().unwrap().put_image(key, img, ctx).await
}
#[cfg(test)]
/// Estimated size of the image layer.
pub(crate) fn estimated_size(&self) -> u64 {
let inner = self.inner.as_ref().unwrap();
inner.blob_writer.size() + inner.tree.borrow_writer().size() + PAGE_SZ as u64
}
#[cfg(test)]
pub(crate) fn num_keys(&self) -> usize {
self.inner.as_ref().unwrap().num_keys
}
///
/// Finish writing the image layer.
///
@@ -968,7 +1027,22 @@ impl ImageLayerWriter {
timeline: &Arc<Timeline>,
ctx: &RequestContext,
) -> anyhow::Result<super::ResidentLayer> {
self.inner.take().unwrap().finish(timeline, ctx).await
self.inner.take().unwrap().finish(timeline, ctx, None).await
}
#[cfg(test)]
/// Finish writing the image layer with an end key, used in [`super::split_writer::SplitImageLayerWriter`]. The end key determines the end of the image layer's covered range and is exclusive.
pub(super) async fn finish_with_end_key(
mut self,
timeline: &Arc<Timeline>,
end_key: Key,
ctx: &RequestContext,
) -> anyhow::Result<super::ResidentLayer> {
self.inner
.take()
.unwrap()
.finish(timeline, ctx, Some(end_key))
.await
}
}

View File

@@ -12,9 +12,10 @@ use crate::tenant::block_io::{BlockCursor, BlockReader, BlockReaderRef};
use crate::tenant::ephemeral_file::EphemeralFile;
use crate::tenant::storage_layer::ValueReconstructResult;
use crate::tenant::timeline::GetVectoredError;
use crate::tenant::{PageReconstructError, Timeline};
use crate::tenant::PageReconstructError;
use crate::{l0_flush, page_cache, walrecord};
use anyhow::{anyhow, ensure, Result};
use camino::Utf8PathBuf;
use pageserver_api::keyspace::KeySpace;
use pageserver_api::models::InMemoryLayerInfo;
use pageserver_api::shard::TenantShardId;
@@ -31,16 +32,46 @@ use std::fmt::Write;
use std::ops::Range;
use std::sync::atomic::Ordering as AtomicOrdering;
use std::sync::atomic::{AtomicU64, AtomicUsize};
use tokio::sync::{RwLock, RwLockWriteGuard};
use tokio::sync::RwLock;
use super::{
DeltaLayerWriter, ResidentLayer, ValueReconstructSituation, ValueReconstructState,
DeltaLayerWriter, PersistentLayerDesc, ValueReconstructSituation, ValueReconstructState,
ValuesReconstructState,
};
#[derive(Debug, PartialEq, Eq, Clone, Copy, Hash)]
pub(crate) struct InMemoryLayerFileId(page_cache::FileId);
#[derive(Ord, PartialOrd, Eq, PartialEq)]
struct IndexPrefix {
field1: u8,
field2: u32,
field3: u32,
field4: u32,
field5: u8,
}
fn materialize_key(prefix: &IndexPrefix, blkno: u32) -> Key {
Key {
field1: prefix.field1,
field2: prefix.field2,
field3: prefix.field3,
field4: prefix.field4,
field5: prefix.field5,
field6: blkno,
}
}
fn key_to_prefix(key: &Key) -> IndexPrefix {
IndexPrefix {
field1: key.field1,
field2: key.field2,
field3: key.field3,
field4: key.field4,
field5: key.field5,
}
}
pub struct InMemoryLayer {
conf: &'static PageServerConf,
tenant_shard_id: TenantShardId,
@@ -82,7 +113,7 @@ pub struct InMemoryLayerInner {
/// All versions of all pages in the layer are kept here. Indexed
/// by block number and LSN. The value is an offset into the
/// ephemeral file where the page version is stored.
index: BTreeMap<Key, VecMap<Lsn, u64>>,
index: BTreeMap<IndexPrefix, BTreeMap<u32, VecMap<Lsn, u64>>>,
/// The values are stored in a serialized format in this file.
/// Each serialized Value is preceded by a 'u32' length field.
@@ -273,30 +304,34 @@ impl InMemoryLayer {
let cursor = inner.file.block_cursor();
let mut buf = Vec::new();
for (key, vec_map) in inner.index.iter() {
for (lsn, pos) in vec_map.as_slice() {
let mut desc = String::new();
cursor.read_blob_into_buf(*pos, &mut buf, ctx).await?;
let val = Value::des(&buf);
match val {
Ok(Value::Image(img)) => {
write!(&mut desc, " img {} bytes", img.len())?;
}
Ok(Value::WalRecord(rec)) => {
let wal_desc = walrecord::describe_wal_record(&rec).unwrap();
write!(
&mut desc,
" rec {} bytes will_init: {} {}",
buf.len(),
rec.will_init(),
wal_desc
)?;
}
Err(err) => {
write!(&mut desc, " DESERIALIZATION ERROR: {}", err)?;
for (key_prefix, inner) in inner.index.iter() {
for (blkno, vec_map) in inner {
let key = materialize_key(key_prefix, *blkno);
for (lsn, pos) in vec_map.as_slice() {
let mut desc = String::new();
cursor.read_blob_into_buf(*pos, &mut buf, ctx).await?;
let val = Value::des(&buf);
match val {
Ok(Value::Image(img)) => {
write!(&mut desc, " img {} bytes", img.len())?;
}
Ok(Value::WalRecord(rec)) => {
let wal_desc = walrecord::describe_wal_record(&rec).unwrap();
write!(
&mut desc,
" rec {} bytes will_init: {} {}",
buf.len(),
rec.will_init(),
wal_desc
)?;
}
Err(err) => {
write!(&mut desc, " DESERIALIZATION ERROR: {}", err)?;
}
}
println!(" key {} at {}: {}", key, lsn, desc);
}
println!(" key {} at {}: {}", key, lsn, desc);
}
}
@@ -323,23 +358,25 @@ impl InMemoryLayer {
let reader = inner.file.block_cursor();
// Scan the page versions backwards, starting from `lsn`.
if let Some(vec_map) = inner.index.get(&key) {
let slice = vec_map.slice_range(lsn_range);
for (entry_lsn, pos) in slice.iter().rev() {
let buf = reader.read_blob(*pos, &ctx).await?;
let value = Value::des(&buf)?;
match value {
Value::Image(img) => {
reconstruct_state.img = Some((*entry_lsn, img));
return Ok(ValueReconstructResult::Complete);
}
Value::WalRecord(rec) => {
let will_init = rec.will_init();
reconstruct_state.records.push((*entry_lsn, rec));
if will_init {
// This WAL record initializes the page, so no need to go further back
need_image = false;
break;
if let Some(inner) = inner.index.get(&key_to_prefix(&key)) {
if let Some(vec_map) = inner.get(&key.field6) {
let slice = vec_map.slice_range(lsn_range);
for (entry_lsn, pos) in slice.iter().rev() {
let buf = reader.read_blob(*pos, &ctx).await?;
let value = Value::des(&buf)?;
match value {
Value::Image(img) => {
reconstruct_state.img = Some((*entry_lsn, img));
return Ok(ValueReconstructResult::Complete);
}
Value::WalRecord(rec) => {
let will_init = rec.will_init();
reconstruct_state.records.push((*entry_lsn, rec));
if will_init {
// This WAL record initializes the page, so no need to go further back
need_image = false;
break;
}
}
}
}
@@ -376,34 +413,54 @@ impl InMemoryLayer {
let reader = inner.file.block_cursor();
for range in keyspace.ranges.iter() {
for (key, vec_map) in inner.index.range(range.start..range.end) {
let lsn_range = match reconstruct_state.get_cached_lsn(key) {
Some(cached_lsn) => (cached_lsn + 1)..end_lsn,
None => self.start_lsn..end_lsn,
let range_incl = range.start..=Key::from_i128(Key::to_i128(&range.end) - 1);
let prefix_start = key_to_prefix(&range.start);
let prefix_end = key_to_prefix(&range.end);
for (prefix, relation_idx) in inner.index.range(prefix_start..=prefix_end) {
let blkno_start = if prefix == &key_to_prefix(&range_incl.start()) {
range_incl.start().field6
} else {
0
};
let slice = vec_map.slice_range(lsn_range);
let blkno_end = if prefix == &key_to_prefix(&range_incl.end()) {
range_incl.end().field6
} else {
0xffffffff
};
for (entry_lsn, pos) in slice.iter().rev() {
// TODO: this uses the page cache => https://github.com/neondatabase/neon/issues/8183
let buf = reader.read_blob(*pos, &ctx).await;
if let Err(e) = buf {
reconstruct_state
.on_key_error(*key, PageReconstructError::from(anyhow!(e)));
break;
}
for (blkno, vec_map) in relation_idx.range(blkno_start..=blkno_end) {
let key = materialize_key(prefix, *blkno);
let lsn_range = match reconstruct_state.get_cached_lsn(&key) {
Some(cached_lsn) => (cached_lsn + 1)..end_lsn,
None => self.start_lsn..end_lsn,
};
let value = Value::des(&buf.unwrap());
if let Err(e) = value {
reconstruct_state
.on_key_error(*key, PageReconstructError::from(anyhow!(e)));
break;
}
let slice = vec_map.slice_range(lsn_range);
let key_situation =
reconstruct_state.update_key(key, *entry_lsn, value.unwrap());
if key_situation == ValueReconstructSituation::Complete {
break;
for (entry_lsn, pos) in slice.iter().rev() {
// TODO: this uses the page cache => https://github.com/neondatabase/neon/issues/8183
let buf = reader.read_blob(*pos, &ctx).await;
if let Err(e) = buf {
reconstruct_state
.on_key_error(key, PageReconstructError::from(anyhow!(e)));
break;
}
let value = Value::des(&buf.unwrap());
if let Err(e) = value {
reconstruct_state
.on_key_error(key, PageReconstructError::from(anyhow!(e)));
break;
}
let key_situation =
reconstruct_state.update_key(&key, *entry_lsn, value.unwrap());
if key_situation == ValueReconstructSituation::Complete {
break;
}
}
}
}
@@ -415,6 +472,74 @@ impl InMemoryLayer {
}
}
pub struct SerializedBatch {
/// Blobs serialized in EphemeralFile's native format, ready for passing to [`EphemeralFile::write_raw`].
pub(crate) raw: Vec<u8>,
/// Index of values in [`Self::raw`], using offsets relative to the start of the buffer.
pub(crate) offsets: Vec<(Key, Lsn, u64)>,
/// The highest LSN of any value in the batch
pub(crate) max_lsn: Lsn,
}
impl SerializedBatch {
/// Write a blob length in the internal format of the EphemeralFile
pub(crate) fn write_blob_length(len: usize, cursor: &mut std::io::Cursor<Vec<u8>>) {
use std::io::Write;
if len < 0x80 {
// short one-byte length header
let len_buf = [len as u8];
cursor
.write_all(&len_buf)
.expect("Writing to Vec is infallible");
} else {
let mut len_buf = u32::to_be_bytes(len as u32);
len_buf[0] |= 0x80;
cursor
.write_all(&len_buf)
.expect("Writing to Vec is infallible");
}
}
pub fn from_values(batch: Vec<(Key, Lsn, Value)>) -> Self {
use std::io::Write;
let mut offsets: Vec<(Key, Lsn, u64)> = Vec::new();
let mut cursor = std::io::Cursor::new(Vec::<u8>::with_capacity(batch.len() * 8192));
let mut max_lsn: Lsn = Lsn(0);
let mut value_buf = smallvec::SmallVec::<[u8; 256]>::new();
for (key, lsn, val) in batch {
let relative_off = cursor.position();
value_buf.clear();
val.ser_into(&mut value_buf)
.expect("Value serialization is infallible");
Self::write_blob_length(value_buf.len(), &mut cursor);
cursor
.write_all(&value_buf)
.expect("Writing to Vec is infallible");
// We can't write straight into the buffer, because the InMemoryLayer file format requires
// the size to come before the value. However... we could probably calculate the size before
// actually serializing the value
//val.ser_into(&mut cursor)?;
offsets.push((key, lsn, relative_off));
max_lsn = std::cmp::max(max_lsn, lsn);
}
Self {
raw: cursor.into_inner(),
offsets,
max_lsn,
}
}
}
fn inmem_layer_display(mut f: impl Write, start_lsn: Lsn, end_lsn: Lsn) -> std::fmt::Result {
write!(f, "inmem-{:016X}-{:016X}", start_lsn.0, end_lsn.0)
}
@@ -478,38 +603,20 @@ impl InMemoryLayer {
})
}
// Write operations
/// Common subroutine of the public put_wal_record() and put_page_image() functions.
/// Adds the page version to the in-memory tree
pub(crate) async fn put_value(
// Write path.
pub async fn put_batch(
&self,
key: Key,
lsn: Lsn,
buf: &[u8],
serialized_batch: &SerializedBatch,
ctx: &RequestContext,
) -> Result<()> {
let mut inner = self.inner.write().await;
self.assert_writable();
self.put_value_locked(&mut inner, key, lsn, buf, ctx).await
}
//self.assert_writable();
async fn put_value_locked(
&self,
locked_inner: &mut RwLockWriteGuard<'_, InMemoryLayerInner>,
key: Key,
lsn: Lsn,
buf: &[u8],
ctx: &RequestContext,
) -> Result<()> {
trace!("put_value key {} at {}/{}", key, self.timeline_id, lsn);
let off = {
locked_inner
let base_off = {
inner
.file
.write_blob(
buf,
.write_raw(
&serialized_batch.raw,
&RequestContextBuilder::extend(ctx)
.page_content_kind(PageContentKind::InMemoryLayer)
.build(),
@@ -517,15 +624,21 @@ impl InMemoryLayer {
.await?
};
let vec_map = locked_inner.index.entry(key).or_default();
let old = vec_map.append_or_update_last(lsn, off).unwrap().0;
if old.is_some() {
// We already had an entry for this LSN. That's odd..
warn!("Key {} at {} already exists", key, lsn);
for (key, lsn, relative_off) in &serialized_batch.offsets {
let prefix = key_to_prefix(&key);
let relation_idx = match inner.index.get_mut(&prefix) {
Some(i) => i,
None => inner.index.entry(prefix).or_default(),
};
let off = base_off + relative_off;
let vec_map = relation_idx.entry(key.field6).or_default();
vec_map.append_fast(*lsn, off);
}
let size = locked_inner.file.len();
locked_inner.resource_units.maybe_publish_size(size);
let size = inner.file.len();
inner.resource_units.maybe_publish_size(size);
Ok(())
}
@@ -548,8 +661,6 @@ impl InMemoryLayer {
/// Records the end_lsn for non-dropped layers.
/// `end_lsn` is exclusive
pub async fn freeze(&self, end_lsn: Lsn) {
let inner = self.inner.write().await;
assert!(
self.start_lsn < end_lsn,
"{} >= {}",
@@ -567,11 +678,15 @@ impl InMemoryLayer {
})
.expect("frozen_local_path_str set only once");
for vec_map in inner.index.values() {
for (lsn, _pos) in vec_map.as_slice() {
assert!(*lsn < end_lsn);
}
}
// #[cfg(debug_assertions)]
// {
// let inner = self.inner.write().await;
// for vec_map in inner.index.values() {
// for (lsn, _pos) in vec_map.as_slice() {
// assert!(*lsn < end_lsn);
// }
// }
// }
}
/// Write this frozen in-memory layer to disk. If `key_range` is set, the delta
@@ -579,12 +694,12 @@ impl InMemoryLayer {
/// if there are no matching keys.
///
/// Returns a new delta layer with all the same data as this in-memory layer
pub(crate) async fn write_to_disk(
pub async fn write_to_disk(
&self,
timeline: &Arc<Timeline>,
ctx: &RequestContext,
key_range: Option<Range<Key>>,
) -> Result<Option<ResidentLayer>> {
l0_flush_global_state: &l0_flush::Inner,
) -> Result<Option<(PersistentLayerDesc, Utf8PathBuf)>> {
// Grab the lock in read-mode. We hold it over the I/O, but because this
// layer is not writeable anymore, no one should be trying to acquire the
// write lock on it, so we shouldn't block anyone. There's one exception
@@ -596,9 +711,8 @@ impl InMemoryLayer {
// rare though, so we just accept the potential latency hit for now.
let inner = self.inner.read().await;
let l0_flush_global_state = timeline.l0_flush_global_state.inner().clone();
use l0_flush::Inner;
let _concurrency_permit = match &*l0_flush_global_state {
let _concurrency_permit = match l0_flush_global_state {
Inner::PageCached => None,
Inner::Direct { semaphore, .. } => Some(semaphore.acquire().await),
};
@@ -606,11 +720,12 @@ impl InMemoryLayer {
let end_lsn = *self.end_lsn.get().unwrap();
let key_count = if let Some(key_range) = key_range {
inner
.index
.iter()
.filter(|(k, _)| key_range.contains(k))
.count()
panic!("Update for IndexPrefix");
// inner
// .index
// .iter()
// .filter(|(k, _)| key_range.contains(k))
// .count()
} else {
inner.index.len()
};
@@ -628,7 +743,7 @@ impl InMemoryLayer {
)
.await?;
match &*l0_flush_global_state {
match l0_flush_global_state {
l0_flush::Inner::PageCached => {
let ctx = RequestContextBuilder::extend(ctx)
.page_content_kind(PageContentKind::InMemoryLayer)
@@ -638,16 +753,20 @@ impl InMemoryLayer {
let cursor = inner.file.block_cursor();
for (key, vec_map) in inner.index.iter() {
// Write all page versions
for (lsn, pos) in vec_map.as_slice() {
cursor.read_blob_into_buf(*pos, &mut buf, &ctx).await?;
let will_init = Value::des(&buf)?.will_init();
let res;
(buf, res) = delta_layer_writer
.put_value_bytes(*key, *lsn, buf, will_init, &ctx)
.await;
res?;
for (key_prefix, inner) in inner.index.iter() {
for (blkno, vec_map) in inner {
let key = materialize_key(key_prefix, *blkno);
// Write all page versions
for (lsn, pos) in vec_map.as_slice() {
cursor.read_blob_into_buf(*pos, &mut buf, &ctx).await?;
let will_init = Value::des(&buf)?.will_init();
let res;
(buf, res) = delta_layer_writer
.put_value_bytes(key, *lsn, buf, will_init, &ctx)
.await;
res?;
}
}
}
}
@@ -671,29 +790,32 @@ impl InMemoryLayer {
let mut buf = Vec::new();
for (key, vec_map) in inner.index.iter() {
// Write all page versions
for (lsn, pos) in vec_map.as_slice() {
// TODO: once we have blob lengths in the in-memory index, we can
// 1. get rid of the blob_io / BlockReaderRef::Slice business and
// 2. load the file contents into a Bytes and
// 3. the use `Bytes::slice` to get the `buf` that is our blob
// 4. pass that `buf` into `put_value_bytes`
// => https://github.com/neondatabase/neon/issues/8183
cursor.read_blob_into_buf(*pos, &mut buf, ctx).await?;
let will_init = Value::des(&buf)?.will_init();
let res;
(buf, res) = delta_layer_writer
.put_value_bytes(*key, *lsn, buf, will_init, ctx)
.await;
res?;
for (key_prefix, inner) in inner.index.iter() {
for (blkno, vec_map) in inner {
// Write all page versions
let key = materialize_key(key_prefix, *blkno);
for (lsn, pos) in vec_map.as_slice() {
// TODO: once we have blob lengths in the in-memory index, we can
// 1. get rid of the blob_io / BlockReaderRef::Slice business and
// 2. load the file contents into a Bytes and
// 3. the use `Bytes::slice` to get the `buf` that is our blob
// 4. pass that `buf` into `put_value_bytes`
// => https://github.com/neondatabase/neon/issues/8183
cursor.read_blob_into_buf(*pos, &mut buf, ctx).await?;
let will_init = Value::des(&buf)?.will_init();
let res;
(buf, res) = delta_layer_writer
.put_value_bytes(key, *lsn, buf, will_init, ctx)
.await;
res?;
}
}
}
}
}
// MAX is used here because we identify L0 layers by full key range
let delta_layer = delta_layer_writer.finish(Key::MAX, timeline, ctx).await?;
let (desc, path) = delta_layer_writer.finish(Key::MAX, ctx).await?;
// Hold the permit until all the IO is done, including the fsync in `delta_layer_writer.finish()``.
//
@@ -705,6 +827,6 @@ impl InMemoryLayer {
// we dirtied when writing to the filesystem have been flushed and marked !dirty.
drop(_concurrency_permit);
Ok(Some(delta_layer))
Ok(Some((desc, path)))
}
}

View File

@@ -17,14 +17,15 @@ use crate::context::{DownloadBehavior, RequestContext};
use crate::repository::Key;
use crate::span::debug_assert_current_span_has_tenant_and_timeline_id;
use crate::task_mgr::TaskKind;
use crate::tenant::timeline::GetVectoredError;
use crate::tenant::timeline::{CompactionError, GetVectoredError};
use crate::tenant::{remote_timeline_client::LayerFileMetadata, Timeline};
use super::delta_layer::{self, DeltaEntry};
use super::image_layer::{self};
use super::{
AsLayerDesc, ImageLayerWriter, LayerAccessStats, LayerAccessStatsReset, LayerName,
PersistentLayerDesc, ValueReconstructResult, ValueReconstructState, ValuesReconstructState,
LayerVisibilityHint, PersistentLayerDesc, ValueReconstructResult, ValueReconstructState,
ValuesReconstructState,
};
use utils::generation::Generation;
@@ -246,7 +247,7 @@ impl Layer {
&timeline.generation,
);
let layer = LayerInner::new(
LayerInner::new(
conf,
timeline,
local_path,
@@ -254,14 +255,7 @@ impl Layer {
Some(inner),
timeline.generation,
timeline.get_shard_index(),
);
// Newly created layers are marked visible by default: the usual case is that they were created to be read.
layer
.access_stats
.set_visibility(super::LayerVisibilityHint::Visible);
layer
)
}));
let downloaded = resident.expect("just initialized");
@@ -426,7 +420,7 @@ impl Layer {
}
/// Downloads if necessary and creates a guard, which will keep this layer from being evicted.
pub(crate) async fn download_and_keep_resident(&self) -> anyhow::Result<ResidentLayer> {
pub(crate) async fn download_and_keep_resident(&self) -> Result<ResidentLayer, DownloadError> {
let downloaded = self.0.get_or_maybe_download(true, None).await?;
Ok(ResidentLayer {
@@ -493,6 +487,32 @@ impl Layer {
}
}
}
pub(crate) fn set_visibility(&self, visibility: LayerVisibilityHint) {
let old_visibility = self.access_stats().set_visibility(visibility.clone());
use LayerVisibilityHint::*;
match (old_visibility, visibility) {
(Visible, Covered) => {
// Subtract this layer's contribution to the visible size metric
if let Some(tl) = self.0.timeline.upgrade() {
tl.metrics
.visible_physical_size_gauge
.sub(self.0.desc.file_size)
}
}
(Covered, Visible) => {
// Add this layer's contribution to the visible size metric
if let Some(tl) = self.0.timeline.upgrade() {
tl.metrics
.visible_physical_size_gauge
.add(self.0.desc.file_size)
}
}
(Covered, Covered) | (Visible, Visible) => {
// no change
}
}
}
}
/// The download-ness ([`DownloadedLayer`]) can be either resident or wanted evicted.
@@ -693,6 +713,13 @@ impl Drop for LayerInner {
timeline.metrics.layer_count_image.dec();
timeline.metrics.layer_size_image.sub(self.desc.file_size);
}
if matches!(self.access_stats.visibility(), LayerVisibilityHint::Visible) {
timeline
.metrics
.visible_physical_size_gauge
.sub(self.desc.file_size);
}
}
if !*self.wanted_deleted.get_mut() {
@@ -801,6 +828,12 @@ impl LayerInner {
timeline.metrics.layer_size_image.add(desc.file_size);
}
// New layers are visible by default. This metric is later updated on drop or in set_visibility
timeline
.metrics
.visible_physical_size_gauge
.add(desc.file_size);
LayerInner {
conf,
debug_str: {
@@ -1651,8 +1684,9 @@ impl Drop for DownloadedLayer {
}
impl DownloadedLayer {
/// Initializes the `DeltaLayerInner` or `ImageLayerInner` within [`LayerKind`], or fails to
/// initialize it permanently.
/// Initializes the `DeltaLayerInner` or `ImageLayerInner` within [`LayerKind`].
/// Failure to load the layer is sticky, i.e., future `get()` calls will return
/// the initial load failure immediately.
///
/// `owner` parameter is a strong reference at the same `LayerInner` as the
/// `DownloadedLayer::owner` would be when upgraded. Given how this method ends up called,
@@ -1683,7 +1717,7 @@ impl DownloadedLayer {
ctx,
)
.await
.map(|res| res.map(LayerKind::Delta))
.map(LayerKind::Delta)
} else {
let lsn = owner.desc.image_layer_lsn();
let summary = Some(image_layer::Summary::expected(
@@ -1700,32 +1734,29 @@ impl DownloadedLayer {
ctx,
)
.await
.map(|res| res.map(LayerKind::Image))
.map(LayerKind::Image)
};
match res {
Ok(Ok(layer)) => Ok(Ok(layer)),
Ok(Err(transient)) => Err(transient),
Err(permanent) => {
Ok(layer) => Ok(layer),
Err(err) => {
LAYER_IMPL_METRICS.inc_permanent_loading_failures();
// TODO(#5815): we are not logging all errors, so temporarily log them **once**
// here as well
let permanent = permanent.context("load layer");
tracing::error!("layer loading failed permanently: {permanent:#}");
Ok(Err(permanent))
// We log this message once over the lifetime of `Self`
// => Ok and good to log backtrace and path here.
tracing::error!(
"layer load failed, assuming permanent failure: {}: {err:?}",
owner.path
);
Err(err)
}
}
};
self.kind
.get_or_try_init(init)
// return transient errors using `?`
.await?
.get_or_init(init)
.await
.as_ref()
.map_err(|e| {
// errors are not clonabled, cannot but stringify
// test_broken_timeline matches this string
anyhow::anyhow!("layer loading failed: {e:#}")
})
// We already logged the full backtrace above, once. Don't repeat that here.
.map_err(|e| anyhow::anyhow!("layer load failed earlier: {e}"))
}
async fn get_value_reconstruct_data(
@@ -1760,7 +1791,11 @@ impl DownloadedLayer {
) -> Result<(), GetVectoredError> {
use LayerKind::*;
match self.get(owner, ctx).await.map_err(GetVectoredError::from)? {
match self
.get(owner, ctx)
.await
.map_err(GetVectoredError::Other)?
{
Delta(d) => {
d.get_values_reconstruct_data(keyspace, lsn_range, reconstruct_data, ctx)
.await
@@ -1862,12 +1897,24 @@ impl ResidentLayer {
shard_identity: &ShardIdentity,
writer: &mut ImageLayerWriter,
ctx: &RequestContext,
) -> anyhow::Result<usize> {
) -> Result<usize, CompactionError> {
use LayerKind::*;
match self.downloaded.get(&self.owner.0, ctx).await? {
Delta(_) => anyhow::bail!(format!("cannot filter() on a delta layer {self}")),
Image(i) => i.filter(shard_identity, writer, ctx).await,
match self
.downloaded
.get(&self.owner.0, ctx)
.await
.map_err(CompactionError::Other)?
{
Delta(_) => {
return Err(CompactionError::Other(anyhow::anyhow!(format!(
"cannot filter() on a delta layer {self}"
))));
}
Image(i) => i
.filter(shard_identity, writer, ctx)
.await
.map_err(CompactionError::Other),
}
}

View File

@@ -828,9 +828,9 @@ async fn eviction_cancellation_on_drop() {
#[test]
#[cfg(target_arch = "x86_64")]
fn layer_size() {
assert_eq!(std::mem::size_of::<LayerAccessStats>(), 8);
assert_eq!(std::mem::size_of::<PersistentLayerDesc>(), 104);
assert_eq!(std::mem::size_of::<LayerInner>(), 312);
assert_eq!(size_of::<LayerAccessStats>(), 8);
assert_eq!(size_of::<PersistentLayerDesc>(), 104);
assert_eq!(size_of::<LayerInner>(), 312);
// it also has the utf8 path
}

View File

@@ -41,6 +41,20 @@ pub struct PersistentLayerKey {
pub is_delta: bool,
}
impl std::fmt::Display for PersistentLayerKey {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(
f,
"{}..{} {}..{} is_delta={}",
self.key_range.start,
self.key_range.end,
self.lsn_range.start,
self.lsn_range.end,
self.is_delta
)
}
}
impl PersistentLayerDesc {
pub fn key(&self) -> PersistentLayerKey {
PersistentLayerKey {

View File

@@ -204,9 +204,11 @@ impl<'a> IteratorWrapper<'a> {
/// A merge iterator over delta/image layer iterators. When duplicated records are
/// found, the iterator will not perform any deduplication, and the caller should handle
/// these situation. By saying duplicated records, there are many possibilities:
///
/// * Two same delta at the same LSN.
/// * Two same image at the same LSN.
/// * Delta/image at the same LSN where the image has already applied the delta.
///
/// The iterator will always put the image before the delta.
pub struct MergeIterator<'a> {
heap: BinaryHeap<IteratorWrapper<'a>>,

View File

@@ -0,0 +1,454 @@
use std::{ops::Range, sync::Arc};
use bytes::Bytes;
use pageserver_api::key::{Key, KEY_SIZE};
use utils::{id::TimelineId, lsn::Lsn, shard::TenantShardId};
use crate::tenant::storage_layer::Layer;
use crate::{config::PageServerConf, context::RequestContext, repository::Value, tenant::Timeline};
use super::{DeltaLayerWriter, ImageLayerWriter, ResidentLayer};
/// An image writer that takes images and produces multiple image layers. The interface does not
/// guarantee atomicity (i.e., if the image layer generation fails, there might be leftover files
/// to be cleaned up)
#[must_use]
pub struct SplitImageLayerWriter {
inner: ImageLayerWriter,
target_layer_size: u64,
generated_layers: Vec<ResidentLayer>,
conf: &'static PageServerConf,
timeline_id: TimelineId,
tenant_shard_id: TenantShardId,
lsn: Lsn,
}
impl SplitImageLayerWriter {
pub async fn new(
conf: &'static PageServerConf,
timeline_id: TimelineId,
tenant_shard_id: TenantShardId,
start_key: Key,
lsn: Lsn,
target_layer_size: u64,
ctx: &RequestContext,
) -> anyhow::Result<Self> {
Ok(Self {
target_layer_size,
inner: ImageLayerWriter::new(
conf,
timeline_id,
tenant_shard_id,
&(start_key..Key::MAX),
lsn,
ctx,
)
.await?,
generated_layers: Vec::new(),
conf,
timeline_id,
tenant_shard_id,
lsn,
})
}
pub async fn put_image(
&mut self,
key: Key,
img: Bytes,
tline: &Arc<Timeline>,
ctx: &RequestContext,
) -> anyhow::Result<()> {
// The current estimation is an upper bound of the space that the key/image could take
// because we did not consider compression in this estimation. The resulting image layer
// could be smaller than the target size.
let addition_size_estimation = KEY_SIZE as u64 + img.len() as u64;
if self.inner.num_keys() >= 1
&& self.inner.estimated_size() + addition_size_estimation >= self.target_layer_size
{
let next_image_writer = ImageLayerWriter::new(
self.conf,
self.timeline_id,
self.tenant_shard_id,
&(key..Key::MAX),
self.lsn,
ctx,
)
.await?;
let prev_image_writer = std::mem::replace(&mut self.inner, next_image_writer);
self.generated_layers.push(
prev_image_writer
.finish_with_end_key(tline, key, ctx)
.await?,
);
}
self.inner.put_image(key, img, ctx).await
}
pub(crate) async fn finish(
self,
tline: &Arc<Timeline>,
ctx: &RequestContext,
end_key: Key,
) -> anyhow::Result<Vec<ResidentLayer>> {
let Self {
mut generated_layers,
inner,
..
} = self;
generated_layers.push(inner.finish_with_end_key(tline, end_key, ctx).await?);
Ok(generated_layers)
}
/// When split writer fails, the caller should call this function and handle partially generated layers.
#[allow(dead_code)]
pub(crate) async fn take(self) -> anyhow::Result<(Vec<ResidentLayer>, ImageLayerWriter)> {
Ok((self.generated_layers, self.inner))
}
}
/// A delta writer that takes key-lsn-values and produces multiple delta layers. The interface does not
/// guarantee atomicity (i.e., if the delta layer generation fails, there might be leftover files
/// to be cleaned up).
#[must_use]
pub struct SplitDeltaLayerWriter {
inner: DeltaLayerWriter,
target_layer_size: u64,
generated_layers: Vec<ResidentLayer>,
conf: &'static PageServerConf,
timeline_id: TimelineId,
tenant_shard_id: TenantShardId,
lsn_range: Range<Lsn>,
}
impl SplitDeltaLayerWriter {
pub async fn new(
conf: &'static PageServerConf,
timeline_id: TimelineId,
tenant_shard_id: TenantShardId,
start_key: Key,
lsn_range: Range<Lsn>,
target_layer_size: u64,
ctx: &RequestContext,
) -> anyhow::Result<Self> {
Ok(Self {
target_layer_size,
inner: DeltaLayerWriter::new(
conf,
timeline_id,
tenant_shard_id,
start_key,
lsn_range.clone(),
ctx,
)
.await?,
generated_layers: Vec::new(),
conf,
timeline_id,
tenant_shard_id,
lsn_range,
})
}
pub async fn put_value(
&mut self,
key: Key,
lsn: Lsn,
val: Value,
tline: &Arc<Timeline>,
ctx: &RequestContext,
) -> anyhow::Result<()> {
// The current estimation is key size plus LSN size plus value size estimation. This is not an accurate
// number, and therefore the final layer size could be a little bit larger or smaller than the target.
let addition_size_estimation = KEY_SIZE as u64 + 8 /* LSN u64 size */ + 80 /* value size estimation */;
if self.inner.num_keys() >= 1
&& self.inner.estimated_size() + addition_size_estimation >= self.target_layer_size
{
let next_delta_writer = DeltaLayerWriter::new(
self.conf,
self.timeline_id,
self.tenant_shard_id,
key,
self.lsn_range.clone(),
ctx,
)
.await?;
let prev_delta_writer = std::mem::replace(&mut self.inner, next_delta_writer);
let (desc, path) = prev_delta_writer.finish(key, ctx).await?;
let delta_layer = Layer::finish_creating(self.conf, tline, desc, &path)?;
self.generated_layers.push(delta_layer);
}
self.inner.put_value(key, lsn, val, ctx).await
}
pub(crate) async fn finish(
self,
tline: &Arc<Timeline>,
ctx: &RequestContext,
end_key: Key,
) -> anyhow::Result<Vec<ResidentLayer>> {
let Self {
mut generated_layers,
inner,
..
} = self;
let (desc, path) = inner.finish(end_key, ctx).await?;
let delta_layer = Layer::finish_creating(self.conf, tline, desc, &path)?;
generated_layers.push(delta_layer);
Ok(generated_layers)
}
/// When split writer fails, the caller should call this function and handle partially generated layers.
#[allow(dead_code)]
pub(crate) async fn take(self) -> anyhow::Result<(Vec<ResidentLayer>, DeltaLayerWriter)> {
Ok((self.generated_layers, self.inner))
}
}
#[cfg(test)]
mod tests {
use crate::{
tenant::{
harness::{TenantHarness, TIMELINE_ID},
storage_layer::AsLayerDesc,
},
DEFAULT_PG_VERSION,
};
use super::*;
fn get_key(id: u32) -> Key {
let mut key = Key::from_hex("000000000033333333444444445500000000").unwrap();
key.field6 = id;
key
}
fn get_img(id: u32) -> Bytes {
format!("{id:064}").into()
}
fn get_large_img() -> Bytes {
vec![0; 8192].into()
}
#[tokio::test]
async fn write_one_image() {
let harness = TenantHarness::create("split_writer_write_one_image")
.await
.unwrap();
let (tenant, ctx) = harness.load().await;
let tline = tenant
.create_test_timeline(TIMELINE_ID, Lsn(0x10), DEFAULT_PG_VERSION, &ctx)
.await
.unwrap();
let mut image_writer = SplitImageLayerWriter::new(
tenant.conf,
tline.timeline_id,
tenant.tenant_shard_id,
get_key(0),
Lsn(0x18),
4 * 1024 * 1024,
&ctx,
)
.await
.unwrap();
let mut delta_writer = SplitDeltaLayerWriter::new(
tenant.conf,
tline.timeline_id,
tenant.tenant_shard_id,
get_key(0),
Lsn(0x18)..Lsn(0x20),
4 * 1024 * 1024,
&ctx,
)
.await
.unwrap();
image_writer
.put_image(get_key(0), get_img(0), &tline, &ctx)
.await
.unwrap();
let layers = image_writer
.finish(&tline, &ctx, get_key(10))
.await
.unwrap();
assert_eq!(layers.len(), 1);
delta_writer
.put_value(
get_key(0),
Lsn(0x18),
Value::Image(get_img(0)),
&tline,
&ctx,
)
.await
.unwrap();
let layers = delta_writer
.finish(&tline, &ctx, get_key(10))
.await
.unwrap();
assert_eq!(layers.len(), 1);
}
#[tokio::test]
async fn write_split() {
let harness = TenantHarness::create("split_writer_write_split")
.await
.unwrap();
let (tenant, ctx) = harness.load().await;
let tline = tenant
.create_test_timeline(TIMELINE_ID, Lsn(0x10), DEFAULT_PG_VERSION, &ctx)
.await
.unwrap();
let mut image_writer = SplitImageLayerWriter::new(
tenant.conf,
tline.timeline_id,
tenant.tenant_shard_id,
get_key(0),
Lsn(0x18),
4 * 1024 * 1024,
&ctx,
)
.await
.unwrap();
let mut delta_writer = SplitDeltaLayerWriter::new(
tenant.conf,
tline.timeline_id,
tenant.tenant_shard_id,
get_key(0),
Lsn(0x18)..Lsn(0x20),
4 * 1024 * 1024,
&ctx,
)
.await
.unwrap();
const N: usize = 2000;
for i in 0..N {
let i = i as u32;
image_writer
.put_image(get_key(i), get_large_img(), &tline, &ctx)
.await
.unwrap();
delta_writer
.put_value(
get_key(i),
Lsn(0x20),
Value::Image(get_large_img()),
&tline,
&ctx,
)
.await
.unwrap();
}
let image_layers = image_writer
.finish(&tline, &ctx, get_key(N as u32))
.await
.unwrap();
let delta_layers = delta_writer
.finish(&tline, &ctx, get_key(N as u32))
.await
.unwrap();
assert_eq!(image_layers.len(), N / 512 + 1);
assert_eq!(delta_layers.len(), N / 512 + 1);
for idx in 0..image_layers.len() {
assert_ne!(image_layers[idx].layer_desc().key_range.start, Key::MIN);
assert_ne!(image_layers[idx].layer_desc().key_range.end, Key::MAX);
assert_ne!(delta_layers[idx].layer_desc().key_range.start, Key::MIN);
assert_ne!(delta_layers[idx].layer_desc().key_range.end, Key::MAX);
if idx > 0 {
assert_eq!(
image_layers[idx - 1].layer_desc().key_range.end,
image_layers[idx].layer_desc().key_range.start
);
assert_eq!(
delta_layers[idx - 1].layer_desc().key_range.end,
delta_layers[idx].layer_desc().key_range.start
);
}
}
}
#[tokio::test]
async fn write_large_img() {
let harness = TenantHarness::create("split_writer_write_large_img")
.await
.unwrap();
let (tenant, ctx) = harness.load().await;
let tline = tenant
.create_test_timeline(TIMELINE_ID, Lsn(0x10), DEFAULT_PG_VERSION, &ctx)
.await
.unwrap();
let mut image_writer = SplitImageLayerWriter::new(
tenant.conf,
tline.timeline_id,
tenant.tenant_shard_id,
get_key(0),
Lsn(0x18),
4 * 1024,
&ctx,
)
.await
.unwrap();
let mut delta_writer = SplitDeltaLayerWriter::new(
tenant.conf,
tline.timeline_id,
tenant.tenant_shard_id,
get_key(0),
Lsn(0x18)..Lsn(0x20),
4 * 1024,
&ctx,
)
.await
.unwrap();
image_writer
.put_image(get_key(0), get_img(0), &tline, &ctx)
.await
.unwrap();
image_writer
.put_image(get_key(1), get_large_img(), &tline, &ctx)
.await
.unwrap();
let layers = image_writer
.finish(&tline, &ctx, get_key(10))
.await
.unwrap();
assert_eq!(layers.len(), 2);
delta_writer
.put_value(
get_key(0),
Lsn(0x18),
Value::Image(get_img(0)),
&tline,
&ctx,
)
.await
.unwrap();
delta_writer
.put_value(
get_key(1),
Lsn(0x1A),
Value::Image(get_large_img()),
&tline,
&ctx,
)
.await
.unwrap();
let layers = delta_writer
.finish(&tline, &ctx, get_key(10))
.await
.unwrap();
assert_eq!(layers.len(), 2);
}
}

View File

@@ -210,24 +210,28 @@ async fn compaction_loop(tenant: Arc<Tenant>, cancel: CancellationToken) {
Duration::from_secs(10)
} else {
// Run compaction
if let Err(e) = tenant.compaction_iteration(&cancel, &ctx).await {
let wait_duration = backoff::exponential_backoff_duration_seconds(
error_run_count + 1,
1.0,
MAX_BACKOFF_SECS,
);
error_run_count += 1;
let wait_duration = Duration::from_secs_f64(wait_duration);
log_compaction_error(
&e,
error_run_count,
&wait_duration,
cancel.is_cancelled(),
);
wait_duration
} else {
error_run_count = 0;
period
match tenant.compaction_iteration(&cancel, &ctx).await {
Err(e) => {
let wait_duration = backoff::exponential_backoff_duration_seconds(
error_run_count + 1,
1.0,
MAX_BACKOFF_SECS,
);
error_run_count += 1;
let wait_duration = Duration::from_secs_f64(wait_duration);
log_compaction_error(
&e,
error_run_count,
&wait_duration,
cancel.is_cancelled(),
);
wait_duration
}
Ok(has_pending_task) => {
error_run_count = 0;
// schedule the next compaction immediately in case there is a pending compaction task
if has_pending_task { Duration::from_secs(0) } else { period }
}
}
};

View File

@@ -3,6 +3,7 @@ pub(crate) mod compaction;
pub mod delete;
pub(crate) mod detach_ancestor;
mod eviction_task;
pub(crate) mod handle;
mod init;
pub mod layer_manager;
pub(crate) mod logical_size;
@@ -17,6 +18,7 @@ use camino::Utf8Path;
use chrono::{DateTime, Utc};
use enumset::EnumSet;
use fail::fail_point;
use handle::ShardTimelineId;
use once_cell::sync::Lazy;
use pageserver_api::{
key::{
@@ -45,7 +47,6 @@ use utils::{
bin_ser::BeSer,
fs_ext, pausable_failpoint,
sync::gate::{Gate, GateGuard},
vec_map::VecMap,
};
use std::pin::pin;
@@ -58,7 +59,7 @@ use std::{
sync::atomic::AtomicU64,
};
use std::{
cmp::{max, min, Ordering},
cmp::{max, min},
ops::ControlFlow,
};
use std::{
@@ -74,6 +75,7 @@ use crate::{
metadata::TimelineMetadata,
storage_layer::PersistentLayerDesc,
},
walredo,
};
use crate::{
context::{DownloadBehavior, RequestContext},
@@ -137,10 +139,13 @@ use self::layer_manager::LayerManager;
use self::logical_size::LogicalSize;
use self::walreceiver::{WalReceiver, WalReceiverConf};
use super::config::TenantConf;
use super::{config::TenantConf, storage_layer::inmemory_layer, upload_queue::NotInitialized};
use super::{debug_assert_current_span_has_tenant_and_timeline_id, AttachedTenantConf};
use super::{remote_timeline_client::index::IndexPart, storage_layer::LayerFringe};
use super::{remote_timeline_client::RemoteTimelineClient, storage_layer::ReadableLayer};
use super::{
remote_timeline_client::RemoteTimelineClient, remote_timeline_client::WaitCompletionError,
storage_layer::ReadableLayer,
};
use super::{
secondary::heatmap::{HeatMapLayer, HeatMapTimeline},
GcError,
@@ -177,25 +182,6 @@ impl std::fmt::Display for ImageLayerCreationMode {
}
}
/// Wrapper for key range to provide reverse ordering by range length for BinaryHeap
#[derive(Debug, Clone, PartialEq, Eq)]
pub(crate) struct Hole {
key_range: Range<Key>,
coverage_size: usize,
}
impl Ord for Hole {
fn cmp(&self, other: &Self) -> Ordering {
other.coverage_size.cmp(&self.coverage_size) // inverse order
}
}
impl PartialOrd for Hole {
fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
Some(self.cmp(other))
}
}
/// Temporary function for immutable storage state refactor, ensures we are dropping mutex guard instead of other things.
/// Can be removed after all refactors are done.
fn drop_rlock<T>(rlock: tokio::sync::RwLockReadGuard<T>) {
@@ -443,6 +429,8 @@ pub struct Timeline {
pub(crate) extra_test_dense_keyspace: ArcSwap<KeySpace>,
pub(crate) l0_flush_global_state: L0FlushGlobalState,
pub(crate) handles: handle::PerTimelineState<crate::page_service::TenantManagerTypes>,
}
pub struct WalReceiverInfo {
@@ -548,7 +536,6 @@ impl GetVectoredError {
}
}
#[derive(Debug)]
pub struct MissingKeyError {
key: Key,
shard: ShardNumber,
@@ -559,6 +546,12 @@ pub struct MissingKeyError {
backtrace: Option<std::backtrace::Backtrace>,
}
impl std::fmt::Debug for MissingKeyError {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(f, "{}", self)
}
}
impl std::fmt::Display for MissingKeyError {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
write!(
@@ -642,7 +635,13 @@ impl FlushLayerError {
// When crossing from generic anyhow errors to this error type, we explicitly check
// for timeline cancellation to avoid logging inoffensive shutdown errors as warn/err.
fn from_anyhow(timeline: &Timeline, err: anyhow::Error) -> Self {
if timeline.cancel.is_cancelled() {
let cancelled = timeline.cancel.is_cancelled()
// The upload queue might have been shut down before the official cancellation of the timeline.
|| err
.downcast_ref::<NotInitialized>()
.map(NotInitialized::is_stopping)
.unwrap_or_default();
if cancelled {
Self::Cancelled
} else {
Self::Other(Arc::new(err))
@@ -1004,7 +1003,10 @@ impl Timeline {
.for_get_kind(GetKind::Singular)
.observe(elapsed.as_secs_f64());
if cfg!(feature = "testing") && res.is_err() {
if cfg!(feature = "testing")
&& res.is_err()
&& !matches!(res, Err(PageReconstructError::Cancelled))
{
// it can only be walredo issue
use std::fmt::Write;
@@ -1763,13 +1765,14 @@ impl Timeline {
}
}
/// Outermost timeline compaction operation; downloads needed layers.
/// Outermost timeline compaction operation; downloads needed layers. Returns whether we have pending
/// compaction tasks.
pub(crate) async fn compact(
self: &Arc<Self>,
cancel: &CancellationToken,
flags: EnumSet<CompactFlags>,
ctx: &RequestContext,
) -> Result<(), CompactionError> {
) -> Result<bool, CompactionError> {
// most likely the cancellation token is from background task, but in tests it could be the
// request task as well.
@@ -1789,8 +1792,8 @@ impl Timeline {
// compaction task goes over it's period (20s) which is quite often in production.
let (_guard, _permit) = tokio::select! {
tuple = prepare => { tuple },
_ = self.cancel.cancelled() => return Ok(()),
_ = cancel.cancelled() => return Ok(()),
_ = self.cancel.cancelled() => return Ok(false),
_ = cancel.cancelled() => return Ok(false),
};
let last_record_lsn = self.get_last_record_lsn();
@@ -1798,11 +1801,14 @@ impl Timeline {
// Last record Lsn could be zero in case the timeline was just created
if !last_record_lsn.is_valid() {
warn!("Skipping compaction for potentially just initialized timeline, it has invalid last record lsn: {last_record_lsn}");
return Ok(());
return Ok(false);
}
match self.get_compaction_algorithm_settings().kind {
CompactionAlgorithm::Tiered => self.compact_tiered(cancel, ctx).await,
CompactionAlgorithm::Tiered => {
self.compact_tiered(cancel, ctx).await?;
Ok(false)
}
CompactionAlgorithm::Legacy => self.compact_legacy(cancel, flags, ctx).await,
}
}
@@ -1919,6 +1925,9 @@ impl Timeline {
tracing::debug!("Cancelling CancellationToken");
self.cancel.cancel();
// Ensure Prevent new page service requests from starting.
self.handles.shutdown();
// Transition the remote_client into a state where it's only useful for timeline deletion.
// (The deletion use case is why we can't just hook up remote_client to Self::cancel).)
self.remote_client.stop();
@@ -1991,6 +2000,11 @@ impl Timeline {
self.current_state() == TimelineState::Active
}
#[allow(unused)]
pub(crate) fn is_archived(&self) -> Option<bool> {
self.remote_client.is_archived()
}
pub(crate) fn is_stopping(&self) -> bool {
self.current_state() == TimelineState::Stopping
}
@@ -2439,6 +2453,8 @@ impl Timeline {
extra_test_dense_keyspace: ArcSwap::new(Arc::new(KeySpace::default())),
l0_flush_global_state: resources.l0_flush_global_state,
handles: Default::default(),
};
result.repartition_threshold =
result.get_checkpoint_distance() / REPARTITION_FREQ_IN_CHECKPOINT_DISTANCE;
@@ -2722,6 +2738,10 @@ impl Timeline {
// Tenant::create_timeline will wait for these uploads to happen before returning, or
// on retry.
// Now that we have the full layer map, we may calculate the visibility of layers within it (a global scan)
drop(guard); // drop write lock, update_layer_visibility will take a read lock.
self.update_layer_visibility().await;
info!(
"loaded layer map with {} layers at {}, total physical size: {}",
num_layers, disk_consistent_lsn, total_physical_size
@@ -3415,7 +3435,6 @@ impl Timeline {
}
}
#[allow(unknown_lints)] // doc_lazy_continuation is still a new lint
#[allow(clippy::doc_lazy_continuation)]
/// Get the data needed to reconstruct all keys in the provided keyspace
///
@@ -3709,6 +3728,17 @@ impl Timeline {
&self.shard_identity
}
#[inline(always)]
pub(crate) fn shard_timeline_id(&self) -> ShardTimelineId {
ShardTimelineId {
shard_index: ShardIndex {
shard_number: self.shard_identity.number,
shard_count: self.shard_identity.count,
},
timeline_id: self.timeline_id,
}
}
///
/// Get a handle to the latest layer for appending.
///
@@ -4061,6 +4091,21 @@ impl Timeline {
// release lock on 'layers'
};
// Backpressure mechanism: wait with continuation of the flush loop until we have uploaded all layer files.
// This makes us refuse ingest until the new layers have been persisted to the remote.
self.remote_client
.wait_completion()
.await
.map_err(|e| match e {
WaitCompletionError::UploadQueueShutDownOrStopped
| WaitCompletionError::NotInitialized(
NotInitialized::ShuttingDown | NotInitialized::Stopped,
) => FlushLayerError::Cancelled,
WaitCompletionError::NotInitialized(NotInitialized::Uninitialized) => {
FlushLayerError::Other(anyhow!(e).into())
}
})?;
// FIXME: between create_delta_layer and the scheduling of the upload in `update_metadata_file`,
// a compaction can delete the file and then it won't be available for uploads any more.
// We still schedule the upload, resulting in an error, but ideally we'd somehow avoid this
@@ -4153,12 +4198,14 @@ impl Timeline {
let frozen_layer = Arc::clone(frozen_layer);
let ctx = ctx.attached_child();
let work = async move {
let Some(new_delta) = frozen_layer
.write_to_disk(&self_clone, &ctx, key_range)
let Some((desc, path)) = frozen_layer
.write_to_disk(&ctx, key_range, self_clone.l0_flush_global_state.inner())
.await?
else {
return Ok(None);
};
let new_delta = Layer::finish_creating(self_clone.conf, &self_clone, desc, &path)?;
// The write_to_disk() above calls writer.finish() which already did the fsync of the inodes.
// We just need to fsync the directory in which these inodes are linked,
// which we know to be the timeline directory.
@@ -4653,27 +4700,6 @@ impl Timeline {
}
}
// The writer.finish() above already did the fsync of the inodes.
// We just need to fsync the directory in which these inodes are linked,
// which we know to be the timeline directory.
if !image_layers.is_empty() {
// We use fatal_err() below because the after writer.finish() returns with success,
// the in-memory state of the filesystem already has the layer file in its final place,
// and subsequent pageserver code could think it's durable while it really isn't.
let timeline_dir = VirtualFile::open(
&self
.conf
.timeline_path(&self.tenant_shard_id, &self.timeline_id),
ctx,
)
.await
.fatal_err("VirtualFile::open for timeline dir fsync");
timeline_dir
.sync_all()
.await
.fatal_err("VirtualFile::sync_all timeline dir");
}
let mut guard = self.layers.write().await;
// FIXME: we could add the images to be uploaded *before* returning from here, but right
@@ -4682,6 +4708,9 @@ impl Timeline {
drop_wlock(guard);
timer.stop_and_record();
// Creating image layers may have caused some previously visible layers to be covered
self.update_layer_visibility().await;
Ok(image_layers)
}
@@ -4786,7 +4815,7 @@ pub(crate) enum CompactionError {
ShuttingDown,
/// Compaction cannot be done right now; page reconstruction and so on.
#[error(transparent)]
Other(#[from] anyhow::Error),
Other(anyhow::Error),
}
impl From<CollectKeySpaceError> for CompactionError {
@@ -4801,6 +4830,38 @@ impl From<CollectKeySpaceError> for CompactionError {
}
}
impl From<super::upload_queue::NotInitialized> for CompactionError {
fn from(value: super::upload_queue::NotInitialized) -> Self {
match value {
super::upload_queue::NotInitialized::Uninitialized
| super::upload_queue::NotInitialized::Stopped => {
CompactionError::Other(anyhow::anyhow!(value))
}
super::upload_queue::NotInitialized::ShuttingDown => CompactionError::ShuttingDown,
}
}
}
impl CompactionError {
/// We cannot do compaction because we could not download a layer that is input to the compaction.
pub(crate) fn input_layer_download_failed(
e: super::storage_layer::layer::DownloadError,
) -> Self {
match e {
super::storage_layer::layer::DownloadError::TimelineShutdown |
/* TODO DownloadCancelled correct here? */
super::storage_layer::layer::DownloadError::DownloadCancelled => CompactionError::ShuttingDown,
super::storage_layer::layer::DownloadError::ContextAndConfigReallyDeniesDownloads |
super::storage_layer::layer::DownloadError::DownloadRequired |
super::storage_layer::layer::DownloadError::NotFile(_) |
super::storage_layer::layer::DownloadError::DownloadFailed |
super::storage_layer::layer::DownloadError::PreStatFailed(_)=>CompactionError::Other(anyhow::anyhow!(e)),
#[cfg(test)]
super::storage_layer::layer::DownloadError::Failpoint(_) => CompactionError::Other(anyhow::anyhow!(e)),
}
}
}
#[serde_as]
#[derive(serde::Serialize)]
struct RecordedDuration(#[serde_as(as = "serde_with::DurationMicroSeconds")] Duration);
@@ -4874,7 +4935,7 @@ impl Timeline {
new_deltas: &[ResidentLayer],
new_images: &[ResidentLayer],
layers_to_remove: &[Layer],
) -> anyhow::Result<()> {
) -> Result<(), CompactionError> {
let mut guard = self.layers.write().await;
let mut duplicated_layers = HashSet::new();
@@ -4892,7 +4953,7 @@ impl Timeline {
// because we have not implemented L0 => L0 compaction.
duplicated_layers.insert(l.layer_desc().key());
} else if LayerMap::is_l0(&l.layer_desc().key_range) {
bail!("compaction generates a L0 layer file as output, which will cause infinite compaction.");
return Err(CompactionError::Other(anyhow::anyhow!("compaction generates a L0 layer file as output, which will cause infinite compaction.")));
} else {
insert_layers.push(l.clone());
}
@@ -4924,7 +4985,7 @@ impl Timeline {
self: &Arc<Self>,
mut replace_layers: Vec<(Layer, ResidentLayer)>,
mut drop_layers: Vec<Layer>,
) -> anyhow::Result<()> {
) -> Result<(), super::upload_queue::NotInitialized> {
let mut guard = self.layers.write().await;
// Trim our lists in case our caller (compaction) raced with someone else (GC) removing layers: we want
@@ -4946,7 +5007,7 @@ impl Timeline {
fn upload_new_image_layers(
self: &Arc<Self>,
new_images: impl IntoIterator<Item = ResidentLayer>,
) -> anyhow::Result<()> {
) -> Result<(), super::upload_queue::NotInitialized> {
for layer in new_images {
self.remote_client.schedule_layer_file_upload(layer)?;
}
@@ -5414,20 +5475,22 @@ impl Timeline {
} else {
trace!("found {} WAL records that will init the page for {} at {}, performing WAL redo", data.records.len(), key, request_lsn);
};
let img = match self
let res = self
.walredo_mgr
.as_ref()
.context("timeline has no walredo manager")
.map_err(PageReconstructError::WalRedo)?
.request_redo(key, request_lsn, data.img, data.records, self.pg_version)
.await
.context("reconstruct a page image")
{
.await;
let img = match res {
Ok(img) => img,
Err(e) => return Err(PageReconstructError::WalRedo(e)),
Err(walredo::Error::Cancelled) => return Err(PageReconstructError::Cancelled),
Err(walredo::Error::Other(e)) => {
return Err(PageReconstructError::WalRedo(
e.context("reconstruct a page image"),
))
}
};
Ok(img)
}
}
@@ -5753,9 +5816,8 @@ impl Timeline {
for (key, lsn, val) in deltas.data {
delta_layer_writer.put_value(key, lsn, val, ctx).await?;
}
let delta_layer = delta_layer_writer
.finish(deltas.key_range.end, self, ctx)
.await?;
let (desc, path) = delta_layer_writer.finish(deltas.key_range.end, ctx).await?;
let delta_layer = Layer::finish_creating(self.conf, self, desc, &path)?;
{
let mut guard = self.layers.write().await;
@@ -5869,44 +5931,6 @@ enum OpenLayerAction {
}
impl<'a> TimelineWriter<'a> {
/// Put a new page version that can be constructed from a WAL record
///
/// This will implicitly extend the relation, if the page is beyond the
/// current end-of-file.
pub(crate) async fn put(
&mut self,
key: Key,
lsn: Lsn,
value: &Value,
ctx: &RequestContext,
) -> anyhow::Result<()> {
// Avoid doing allocations for "small" values.
// In the regression test suite, the limit of 256 avoided allocations in 95% of cases:
// https://github.com/neondatabase/neon/pull/5056#discussion_r1301975061
let mut buf = smallvec::SmallVec::<[u8; 256]>::new();
value.ser_into(&mut buf)?;
let buf_size: u64 = buf.len().try_into().expect("oversized value buf");
let action = self.get_open_layer_action(lsn, buf_size);
let layer = self.handle_open_layer_action(lsn, action, ctx).await?;
let res = layer.put_value(key, lsn, &buf, ctx).await;
if res.is_ok() {
// Update the current size only when the entire write was ok.
// In case of failures, we may have had partial writes which
// render the size tracking out of sync. That's ok because
// the checkpoint distance should be significantly smaller
// than the S3 single shot upload limit of 5GiB.
let state = self.write_guard.as_mut().unwrap();
state.current_size += buf_size;
state.prev_lsn = Some(lsn);
state.max_lsn = std::cmp::max(state.max_lsn, Some(lsn));
}
res
}
async fn handle_open_layer_action(
&mut self,
at: Lsn,
@@ -6009,18 +6033,52 @@ impl<'a> TimelineWriter<'a> {
}
/// Put a batch of keys at the specified Lsns.
///
/// The batch is sorted by Lsn (enforced by usage of [`utils::vec_map::VecMap`].
pub(crate) async fn put_batch(
&mut self,
batch: VecMap<Lsn, (Key, Value)>,
batch: Vec<(Key, Lsn, Value)>,
ctx: &RequestContext,
) -> anyhow::Result<()> {
for (lsn, (key, val)) in batch {
self.put(key, lsn, &val, ctx).await?
if batch.is_empty() {
return Ok(());
}
Ok(())
let serialized_batch = inmemory_layer::SerializedBatch::from_values(batch);
let batch_max_lsn = serialized_batch.max_lsn;
let buf_size: u64 = serialized_batch.raw.len() as u64;
let action = self.get_open_layer_action(batch_max_lsn, buf_size);
let layer = self
.handle_open_layer_action(batch_max_lsn, action, ctx)
.await?;
let res = layer.put_batch(&serialized_batch, ctx).await;
if res.is_ok() {
// Update the current size only when the entire write was ok.
// In case of failures, we may have had partial writes which
// render the size tracking out of sync. That's ok because
// the checkpoint distance should be significantly smaller
// than the S3 single shot upload limit of 5GiB.
let state = self.write_guard.as_mut().unwrap();
state.current_size += buf_size;
state.prev_lsn = Some(batch_max_lsn);
state.max_lsn = std::cmp::max(state.max_lsn, Some(batch_max_lsn));
}
res
}
#[cfg(test)]
/// Test helper, for tests that would like to poke individual values without composing a batch
pub(crate) async fn put(
&mut self,
key: Key,
lsn: Lsn,
value: &Value,
ctx: &RequestContext,
) -> anyhow::Result<()> {
self.put_batch(vec![(key, lsn, value.clone())], ctx).await
}
pub(crate) async fn delete_batch(

File diff suppressed because it is too large Load Diff

View File

@@ -63,10 +63,19 @@ pub(super) async fn delete_local_timeline_directory(
tenant_shard_id: TenantShardId,
timeline: &Timeline,
) -> anyhow::Result<()> {
let guards = async { tokio::join!(timeline.gc_lock.lock(), timeline.compaction_lock.lock()) };
let guards = crate::timed(
guards,
"acquire gc and compaction locks",
// Always ensure the lock order is compaction -> gc.
let compaction_lock = timeline.compaction_lock.lock();
let compaction_lock = crate::timed(
compaction_lock,
"acquires compaction lock",
std::time::Duration::from_secs(5),
)
.await;
let gc_lock = timeline.gc_lock.lock();
let gc_lock = crate::timed(
gc_lock,
"acquires gc lock",
std::time::Duration::from_secs(5),
)
.await;
@@ -107,7 +116,8 @@ pub(super) async fn delete_local_timeline_directory(
.context("fsync_pre_mark_remove")?;
info!("finished deleting layer files, releasing locks");
drop(guards);
drop(gc_lock);
drop(compaction_lock);
fail::fail_point!("timeline-delete-after-rm", |_| {
Err(anyhow::anyhow!("failpoint: timeline-delete-after-rm"))?
@@ -206,11 +216,10 @@ impl DeleteTimelineFlow {
// NB: If this fails half-way through, and is retried, the retry will go through
// all the same steps again. Make sure the code here is idempotent, and don't
// error out if some of the shutdown tasks have already been completed!
#[instrument(skip_all, fields(%inplace))]
#[instrument(skip_all)]
pub async fn run(
tenant: &Arc<Tenant>,
timeline_id: TimelineId,
inplace: bool,
) -> Result<(), DeleteTimelineError> {
super::debug_assert_current_span_has_tenant_and_timeline_id();
@@ -235,11 +244,7 @@ impl DeleteTimelineFlow {
))?
});
if inplace {
Self::background(guard, tenant.conf, tenant, &timeline).await?
} else {
Self::schedule_background(guard, tenant.conf, Arc::clone(tenant), timeline);
}
Self::schedule_background(guard, tenant.conf, Arc::clone(tenant), timeline);
Ok(())
}

View File

@@ -26,7 +26,7 @@ pub(crate) enum Error {
#[error("flushing failed")]
FlushAncestor(#[source] FlushLayerError),
#[error("layer download failed")]
RewrittenDeltaDownloadFailed(#[source] anyhow::Error),
RewrittenDeltaDownloadFailed(#[source] crate::tenant::storage_layer::layer::DownloadError),
#[error("copying LSN prefix locally failed")]
CopyDeltaPrefix(#[source] anyhow::Error),
#[error("upload rewritten layer")]
@@ -488,10 +488,12 @@ async fn copy_lsn_prefix(
// reuse the key instead of adding more holes between layers by using the real
// highest key in the layer.
let reused_highest_key = layer.layer_desc().key_range.end;
let copied = writer
.finish(reused_highest_key, target_timeline, ctx)
let (desc, path) = writer
.finish(reused_highest_key, ctx)
.await
.map_err(CopyDeltaPrefix)?;
let copied = Layer::finish_creating(target_timeline.conf, target_timeline, desc, &path)
.map_err(CopyDeltaPrefix)?;
tracing::debug!(%layer, %copied, "new layer produced");

View File

@@ -0,0 +1,967 @@
//! An efficient way to keep the timeline gate open without preventing
//! timeline shutdown for longer than a single call to a timeline method.
//!
//! # Motivation
//!
//! On a single page service connection, we're typically serving a single TenantTimelineId.
//!
//! Without sharding, there is a single Timeline object to which we dispatch
//! all requests. For example, a getpage request gets dispatched to the
//! Timeline::get method of the Timeline object that represents the
//! (tenant,timeline) of that connection.
//!
//! With sharding, for each request that comes in on the connection,
//! we first have to perform shard routing based on the requested key (=~ page number).
//! The result of shard routing is a Timeline object.
//! We then dispatch the request to that Timeline object.
//!
//! Regardless of whether the tenant is sharded or not, we want to ensure that
//! we hold the Timeline gate open while we're invoking the method on the
//! Timeline object.
//!
//! However, we want to avoid the overhead of entering the gate for every
//! method invocation.
//!
//! Further, for shard routing, we want to avoid calling the tenant manager to
//! resolve the shard for every request. Instead, we want to cache the
//! routing result so we can bypass the tenant manager for all subsequent requests
//! that get routed to that shard.
//!
//! Regardless of how we accomplish the above, it should not
//! prevent the Timeline from shutting down promptly.
//!
//! # Design
//!
//! There are three user-facing data structures:
//! - `PerTimelineState`: a struct embedded into each Timeline struct. Lifetime == Timeline lifetime.
//! - `Cache`: a struct private to each connection handler; Lifetime == connection lifetime.
//! - `Handle`: a smart pointer that holds the Timeline gate open and derefs to `&Timeline`.
//! Lifetime: for a single request dispatch on the Timeline (i.e., one getpage request)
//!
//! The `Handle` is just a wrapper around an `Arc<HandleInner>`.
//!
//! There is one long-lived `Arc<HandleInner>`, which is stored in the `PerTimelineState`.
//! The `Cache` stores a `Weak<HandleInner>` for each cached Timeline.
//!
//! To dispatch a request, the page service connection calls `Cache::get`.
//!
//! A cache miss means we consult the tenant manager for shard routing,
//! resulting in an `Arc<Timeline>`. We enter its gate _once_ and construct an
//! `Arc<HandleInner>`. We store a `Weak<HandleInner>` in the cache
//! and the `Arc<HandleInner>` in the `PerTimelineState`.
//!
//! For subsequent requests, `Cache::get` will perform a "fast path" shard routing
//! and find the `Weak<HandleInner>` in the cache.
//! We upgrade the `Weak<HandleInner>` to an `Arc<HandleInner>` and wrap it in the user-facing `Handle` type.
//!
//! The request handler dispatches the request to the right `<Handle as Deref<Target = Timeline>>::$request_method`.
//! It then drops the `Handle`, which drops the `Arc<HandleInner>`.
//!
//! # Memory Management / How The Reference Cycle Is Broken
//!
//! The attentive reader may have noticed the strong reference cycle
//! from `Arc<HandleInner>` to `PerTimelineState` to `Arc<Timeline>`.
//!
//! This cycle is intentional: while it exists, the `Cache` can upgrade its
//! `Weak<HandleInner>` to an `Arc<HandleInner>` in a single atomic operation.
//!
//! The cycle is broken by either
//! - `PerTimelineState::shutdown` or
//! - dropping the `Cache`.
//!
//! Concurrently existing `Handle`s will extend the existence of the cycle.
//! However, since `Handle`s are short-lived and new `Handle`s are not
//! handed out after either `PerTimelineState::shutdown` or `Cache` drop,
//! that extension of the cycle is bounded.
//!
//! # Fast Path for Shard Routing
//!
//! The `Cache` has a fast path for shard routing to avoid calling into
//! the tenant manager for every request.
//!
//! The `Cache` maintains a hash map of `ShardTimelineId` to `Weak<HandleInner>`.
//!
//! The current implementation uses the first entry in the hash map
//! to determine the `ShardParameters` and derive the correct
//! `ShardIndex` for the requested key.
//!
//! It then looks up the hash map for that `ShardTimelineId := {ShardIndex,TimelineId}`.
//!
//! If the lookup is successful and the `Weak<HandleInner>` can be upgraded,
//! it's a hit.
//!
//! ## Cache invalidation
//!
//! The insight is that cache invalidation is sufficient and most efficiently done lazily.
//! The only reasons why an entry in the cache can become stale are:
//! 1. The `PerTimelineState` / Timeline is shutting down e.g. because the shard is
//! being detached, timeline or shard deleted, or pageserver is shutting down.
//! 2. We're doing a shard split and new traffic should be routed to the child shards.
//!
//! Regarding (1), we will eventually fail to upgrade the `Weak<HandleInner>` once the
//! timeline has shut down, and when that happens, we remove the entry from the cache.
//!
//! Regarding (2), the insight is that it is toally fine to keep dispatching requests
//! to the parent shard during a shard split. Eventually, the shard split task will
//! shut down the parent => case (1).
use std::collections::hash_map;
use std::collections::HashMap;
use std::sync::atomic::AtomicBool;
use std::sync::atomic::Ordering;
use std::sync::Arc;
use std::sync::Mutex;
use std::sync::Weak;
use pageserver_api::shard::ShardIdentity;
use tracing::instrument;
use tracing::trace;
use utils::id::TimelineId;
use utils::shard::ShardIndex;
use utils::shard::ShardNumber;
use crate::tenant::mgr::ShardSelector;
/// The requirement for Debug is so that #[derive(Debug)] works in some places.
pub(crate) trait Types: Sized + std::fmt::Debug {
type TenantManagerError: Sized + std::fmt::Debug;
type TenantManager: TenantManager<Self> + Sized;
type Timeline: ArcTimeline<Self> + Sized;
}
/// Uniquely identifies a [`Cache`] instance over the lifetime of the process.
/// Required so [`Cache::drop`] can take out the handles from the [`PerTimelineState`].
/// Alternative to this would be to allocate [`Cache`] in a `Box` and identify it by the pointer.
#[derive(Debug, Hash, PartialEq, Eq, Clone, Copy)]
struct CacheId(u64);
impl CacheId {
fn next() -> Self {
static NEXT_ID: std::sync::atomic::AtomicU64 = std::sync::atomic::AtomicU64::new(1);
let id = NEXT_ID.fetch_add(1, std::sync::atomic::Ordering::Relaxed);
if id == 0 {
panic!("CacheId::new() returned 0, overflow");
}
Self(id)
}
}
/// See module-level comment.
pub(crate) struct Cache<T: Types> {
id: CacheId,
map: Map<T>,
}
type Map<T> = HashMap<ShardTimelineId, Weak<HandleInner<T>>>;
impl<T: Types> Default for Cache<T> {
fn default() -> Self {
Self {
id: CacheId::next(),
map: Default::default(),
}
}
}
#[derive(PartialEq, Eq, Debug, Hash, Clone, Copy)]
pub(crate) struct ShardTimelineId {
pub(crate) shard_index: ShardIndex,
pub(crate) timeline_id: TimelineId,
}
/// See module-level comment.
pub(crate) struct Handle<T: Types>(Arc<HandleInner<T>>);
struct HandleInner<T: Types> {
shut_down: AtomicBool,
timeline: T::Timeline,
// The timeline's gate held open.
_gate_guard: utils::sync::gate::GateGuard,
}
/// Embedded in each [`Types::Timeline`] as the anchor for the only long-lived strong ref to `HandleInner`.
///
/// See module-level comment for details.
pub struct PerTimelineState<T: Types> {
// None = shutting down
handles: Mutex<Option<HashMap<CacheId, Arc<HandleInner<T>>>>>,
}
impl<T: Types> Default for PerTimelineState<T> {
fn default() -> Self {
Self {
handles: Mutex::new(Some(Default::default())),
}
}
}
/// Abstract view of [`crate::tenant::mgr`], for testability.
pub(crate) trait TenantManager<T: Types> {
/// Invoked by [`Cache::get`] to resolve a [`ShardTimelineId`] to a [`Types::Timeline`].
/// Errors are returned as [`GetError::TenantManager`].
async fn resolve(
&self,
timeline_id: TimelineId,
shard_selector: ShardSelector,
) -> Result<T::Timeline, T::TenantManagerError>;
}
/// Abstract view of an [`Arc<Timeline>`], for testability.
pub(crate) trait ArcTimeline<T: Types>: Clone {
fn gate(&self) -> &utils::sync::gate::Gate;
fn shard_timeline_id(&self) -> ShardTimelineId;
fn get_shard_identity(&self) -> &ShardIdentity;
fn per_timeline_state(&self) -> &PerTimelineState<T>;
}
/// Errors returned by [`Cache::get`].
#[derive(Debug)]
pub(crate) enum GetError<T: Types> {
TenantManager(T::TenantManagerError),
TimelineGateClosed,
PerTimelineStateShutDown,
}
/// Internal type used in [`Cache::get`].
enum RoutingResult<T: Types> {
FastPath(Handle<T>),
SlowPath(ShardTimelineId),
NeedConsultTenantManager,
}
impl<T: Types> Cache<T> {
/// See module-level comment for details.
///
/// Does NOT check for the shutdown state of [`Types::Timeline`].
/// Instead, the methods of [`Types::Timeline`] that are invoked through
/// the [`Handle`] are responsible for checking these conditions
/// and if so, return an error that causes the page service to
/// close the connection.
#[instrument(level = "trace", skip_all)]
pub(crate) async fn get(
&mut self,
timeline_id: TimelineId,
shard_selector: ShardSelector,
tenant_manager: &T::TenantManager,
) -> Result<Handle<T>, GetError<T>> {
// terminates because each iteration removes an element from the map
loop {
let handle = self
.get_impl(timeline_id, shard_selector, tenant_manager)
.await?;
if handle.0.shut_down.load(Ordering::Relaxed) {
let removed = self
.map
.remove(&handle.0.timeline.shard_timeline_id())
.expect("invariant of get_impl is that the returned handle is in the map");
assert!(
Weak::ptr_eq(&removed, &Arc::downgrade(&handle.0)),
"shard_timeline_id() incorrect?"
);
} else {
return Ok(handle);
}
}
}
#[instrument(level = "trace", skip_all)]
async fn get_impl(
&mut self,
timeline_id: TimelineId,
shard_selector: ShardSelector,
tenant_manager: &T::TenantManager,
) -> Result<Handle<T>, GetError<T>> {
let miss: ShardSelector = {
let routing_state = self.shard_routing(timeline_id, shard_selector);
match routing_state {
RoutingResult::FastPath(handle) => return Ok(handle),
RoutingResult::SlowPath(key) => match self.map.get(&key) {
Some(cached) => match cached.upgrade() {
Some(upgraded) => return Ok(Handle(upgraded)),
None => {
trace!("handle cache stale");
self.map.remove(&key).unwrap();
ShardSelector::Known(key.shard_index)
}
},
None => ShardSelector::Known(key.shard_index),
},
RoutingResult::NeedConsultTenantManager => shard_selector,
}
};
self.get_miss(timeline_id, miss, tenant_manager).await
}
#[inline(always)]
fn shard_routing(
&mut self,
timeline_id: TimelineId,
shard_selector: ShardSelector,
) -> RoutingResult<T> {
loop {
// terminates because when every iteration we remove an element from the map
let Some((first_key, first_handle)) = self.map.iter().next() else {
return RoutingResult::NeedConsultTenantManager;
};
let Some(first_handle) = first_handle.upgrade() else {
// TODO: dedup with get()
trace!("handle cache stale");
let first_key_owned = *first_key;
self.map.remove(&first_key_owned).unwrap();
continue;
};
let first_handle_shard_identity = first_handle.timeline.get_shard_identity();
let make_shard_index = |shard_num: ShardNumber| ShardIndex {
shard_number: shard_num,
shard_count: first_handle_shard_identity.count,
};
let need_idx = match shard_selector {
ShardSelector::Page(key) => {
make_shard_index(first_handle_shard_identity.get_shard_number(&key))
}
ShardSelector::Zero => make_shard_index(ShardNumber(0)),
ShardSelector::Known(shard_idx) => shard_idx,
};
let need_shard_timeline_id = ShardTimelineId {
shard_index: need_idx,
timeline_id,
};
let first_handle_shard_timeline_id = ShardTimelineId {
shard_index: first_handle_shard_identity.shard_index(),
timeline_id: first_handle.timeline.shard_timeline_id().timeline_id,
};
if need_shard_timeline_id == first_handle_shard_timeline_id {
return RoutingResult::FastPath(Handle(first_handle));
} else {
return RoutingResult::SlowPath(need_shard_timeline_id);
}
}
}
#[instrument(level = "trace", skip_all)]
#[inline(always)]
async fn get_miss(
&mut self,
timeline_id: TimelineId,
shard_selector: ShardSelector,
tenant_manager: &T::TenantManager,
) -> Result<Handle<T>, GetError<T>> {
match tenant_manager.resolve(timeline_id, shard_selector).await {
Ok(timeline) => {
let key = timeline.shard_timeline_id();
match &shard_selector {
ShardSelector::Zero => assert_eq!(key.shard_index.shard_number, ShardNumber(0)),
ShardSelector::Page(_) => (), // gotta trust tenant_manager
ShardSelector::Known(idx) => assert_eq!(idx, &key.shard_index),
}
let gate_guard = match timeline.gate().enter() {
Ok(guard) => guard,
Err(_) => {
return Err(GetError::TimelineGateClosed);
}
};
trace!("creating new HandleInner");
let handle = Arc::new(
// TODO: global metric that keeps track of the number of live HandlerTimeline instances
// so we can identify reference cycle bugs.
HandleInner {
shut_down: AtomicBool::new(false),
_gate_guard: gate_guard,
timeline: timeline.clone(),
},
);
let handle = {
let mut lock_guard = timeline
.per_timeline_state()
.handles
.lock()
.expect("mutex poisoned");
match &mut *lock_guard {
Some(per_timeline_state) => {
let replaced = per_timeline_state.insert(self.id, Arc::clone(&handle));
assert!(replaced.is_none(), "some earlier code left a stale handle");
match self.map.entry(key) {
hash_map::Entry::Occupied(_o) => {
// This cannot not happen because
// 1. we're the _miss_ handle, i.e., `self.map` didn't contain an entry and
// 2. we were holding &mut self during .resolve().await above, so, no other thread can have inserted a handle
// while we were waiting for the tenant manager.
unreachable!()
}
hash_map::Entry::Vacant(v) => {
v.insert(Arc::downgrade(&handle));
handle
}
}
}
None => {
return Err(GetError::PerTimelineStateShutDown);
}
}
};
Ok(Handle(handle))
}
Err(e) => Err(GetError::TenantManager(e)),
}
}
}
impl<T: Types> PerTimelineState<T> {
/// After this method returns, [`Cache::get`] will never again return a [`Handle`]
/// to the [`Types::Timeline`] that embeds this per-timeline state.
/// Even if [`TenantManager::resolve`] would still resolve to it.
///
/// Already-alive [`Handle`]s for will remain open, usable, and keeping the [`ArcTimeline`] alive.
/// That's ok because they're short-lived. See module-level comment for details.
#[instrument(level = "trace", skip_all)]
pub(super) fn shutdown(&self) {
let handles = self
.handles
.lock()
.expect("mutex poisoned")
// NB: this .take() sets locked to None.
// That's what makes future `Cache::get` misses fail.
// Cache hits are taken care of below.
.take();
let Some(handles) = handles else {
trace!("already shut down");
return;
};
for handle in handles.values() {
// Make hits fail.
handle.shut_down.store(true, Ordering::Relaxed);
}
drop(handles);
}
}
impl<T: Types> std::ops::Deref for Handle<T> {
type Target = T::Timeline;
fn deref(&self) -> &Self::Target {
&self.0.timeline
}
}
#[cfg(test)]
impl<T: Types> Drop for HandleInner<T> {
fn drop(&mut self) {
trace!("HandleInner dropped");
}
}
// When dropping a [`Cache`], prune its handles in the [`PerTimelineState`] to break the reference cycle.
impl<T: Types> Drop for Cache<T> {
fn drop(&mut self) {
for (_, weak) in self.map.drain() {
if let Some(strong) = weak.upgrade() {
// handle is still being kept alive in PerTimelineState
let timeline = strong.timeline.per_timeline_state();
let mut handles = timeline.handles.lock().expect("mutex poisoned");
if let Some(handles) = &mut *handles {
let Some(removed) = handles.remove(&self.id) else {
// There could have been a shutdown inbetween us upgrading the weak and locking the mutex.
continue;
};
assert!(Arc::ptr_eq(&removed, &strong));
}
}
}
}
}
#[cfg(test)]
mod tests {
use pageserver_api::{
key::{rel_block_to_key, Key, DBDIR_KEY},
models::ShardParameters,
reltag::RelTag,
shard::ShardStripeSize,
};
use utils::shard::ShardCount;
use super::*;
const FOREVER: std::time::Duration = std::time::Duration::from_secs(u64::MAX);
#[derive(Debug)]
struct TestTypes;
impl Types for TestTypes {
type TenantManagerError = anyhow::Error;
type TenantManager = StubManager;
type Timeline = Arc<StubTimeline>;
}
struct StubManager {
shards: Vec<Arc<StubTimeline>>,
}
struct StubTimeline {
gate: utils::sync::gate::Gate,
id: TimelineId,
shard: ShardIdentity,
per_timeline_state: PerTimelineState<TestTypes>,
myself: Weak<StubTimeline>,
}
impl StubTimeline {
fn getpage(&self) {
// do nothing
}
}
impl ArcTimeline<TestTypes> for Arc<StubTimeline> {
fn gate(&self) -> &utils::sync::gate::Gate {
&self.gate
}
fn shard_timeline_id(&self) -> ShardTimelineId {
ShardTimelineId {
shard_index: self.shard.shard_index(),
timeline_id: self.id,
}
}
fn get_shard_identity(&self) -> &ShardIdentity {
&self.shard
}
fn per_timeline_state(&self) -> &PerTimelineState<TestTypes> {
&self.per_timeline_state
}
}
impl TenantManager<TestTypes> for StubManager {
async fn resolve(
&self,
timeline_id: TimelineId,
shard_selector: ShardSelector,
) -> anyhow::Result<Arc<StubTimeline>> {
for timeline in &self.shards {
if timeline.id == timeline_id {
match &shard_selector {
ShardSelector::Zero if timeline.shard.is_shard_zero() => {
return Ok(Arc::clone(timeline));
}
ShardSelector::Zero => continue,
ShardSelector::Page(key) if timeline.shard.is_key_local(key) => {
return Ok(Arc::clone(timeline));
}
ShardSelector::Page(_) => continue,
ShardSelector::Known(idx) if idx == &timeline.shard.shard_index() => {
return Ok(Arc::clone(timeline));
}
ShardSelector::Known(_) => continue,
}
}
}
anyhow::bail!("not found")
}
}
#[tokio::test(start_paused = true)]
async fn test_timeline_shutdown() {
crate::tenant::harness::setup_logging();
let timeline_id = TimelineId::generate();
let shard0 = Arc::new_cyclic(|myself| StubTimeline {
gate: Default::default(),
id: timeline_id,
shard: ShardIdentity::unsharded(),
per_timeline_state: PerTimelineState::default(),
myself: myself.clone(),
});
let mgr = StubManager {
shards: vec![shard0.clone()],
};
let key = DBDIR_KEY;
let mut cache = Cache::<TestTypes>::default();
//
// fill the cache
//
assert_eq!(
(Arc::strong_count(&shard0), Arc::weak_count(&shard0)),
(2, 1),
"strong: shard0, mgr; weak: myself"
);
let handle: Handle<_> = cache
.get(timeline_id, ShardSelector::Page(key), &mgr)
.await
.expect("we have the timeline");
let handle_inner_weak = Arc::downgrade(&handle.0);
assert!(Weak::ptr_eq(&handle.myself, &shard0.myself));
assert_eq!(
(
Weak::strong_count(&handle_inner_weak),
Weak::weak_count(&handle_inner_weak)
),
(2, 2),
"strong: handle, per_timeline_state, weak: handle_inner_weak, cache"
);
assert_eq!(cache.map.len(), 1);
assert_eq!(
(Arc::strong_count(&shard0), Arc::weak_count(&shard0)),
(3, 1),
"strong: handleinner(per_timeline_state), shard0, mgr; weak: myself"
);
drop(handle);
assert_eq!(
(Arc::strong_count(&shard0), Arc::weak_count(&shard0)),
(3, 1),
"strong: handleinner(per_timeline_state), shard0, mgr; weak: myself"
);
//
// demonstrate that Handle holds up gate closure
// but shutdown prevents new handles from being handed out
//
tokio::select! {
_ = shard0.gate.close() => {
panic!("cache and per-timeline handler state keep cache open");
}
_ = tokio::time::sleep(FOREVER) => {
// NB: first poll of close() makes it enter closing state
}
}
let handle = cache
.get(timeline_id, ShardSelector::Page(key), &mgr)
.await
.expect("we have the timeline");
assert!(Weak::ptr_eq(&handle.myself, &shard0.myself));
// SHUTDOWN
shard0.per_timeline_state.shutdown(); // keeping handle alive across shutdown
assert_eq!(
1,
Weak::strong_count(&handle_inner_weak),
"through local var handle"
);
assert_eq!(
cache.map.len(),
1,
"this is an implementation detail but worth pointing out: we can't clear the cache from shutdown(), it's cleared on first access after"
);
assert_eq!(
(Arc::strong_count(&shard0), Arc::weak_count(&shard0)),
(3, 1),
"strong: handleinner(via handle), shard0, mgr; weak: myself"
);
// this handle is perfectly usable
handle.getpage();
cache
.get(timeline_id, ShardSelector::Page(key), &mgr)
.await
.err()
.expect("documented behavior: can't get new handle after shutdown, even if there is an alive Handle");
assert_eq!(
cache.map.len(),
0,
"first access after shutdown cleans up the Weak's from the cache"
);
tokio::select! {
_ = shard0.gate.close() => {
panic!("handle is keeping gate open");
}
_ = tokio::time::sleep(FOREVER) => { }
}
drop(handle);
assert_eq!(
0,
Weak::strong_count(&handle_inner_weak),
"the HandleInner destructor already ran"
);
assert_eq!(
(Arc::strong_count(&shard0), Arc::weak_count(&shard0)),
(2, 1),
"strong: shard0, mgr; weak: myself"
);
// closing gate succeeds after dropping handle
tokio::select! {
_ = shard0.gate.close() => { }
_ = tokio::time::sleep(FOREVER) => {
panic!("handle is dropped, no other gate holders exist")
}
}
// map gets cleaned on next lookup
cache
.get(timeline_id, ShardSelector::Page(key), &mgr)
.await
.err()
.expect("documented behavior: can't get new handle after shutdown");
assert_eq!(cache.map.len(), 0);
// ensure all refs to shard0 are gone and we're not leaking anything
let myself = Weak::clone(&shard0.myself);
drop(shard0);
drop(mgr);
assert_eq!(Weak::strong_count(&myself), 0);
}
#[tokio::test]
async fn test_multiple_timelines_and_deletion() {
crate::tenant::harness::setup_logging();
let timeline_a = TimelineId::generate();
let timeline_b = TimelineId::generate();
assert_ne!(timeline_a, timeline_b);
let timeline_a = Arc::new_cyclic(|myself| StubTimeline {
gate: Default::default(),
id: timeline_a,
shard: ShardIdentity::unsharded(),
per_timeline_state: PerTimelineState::default(),
myself: myself.clone(),
});
let timeline_b = Arc::new_cyclic(|myself| StubTimeline {
gate: Default::default(),
id: timeline_b,
shard: ShardIdentity::unsharded(),
per_timeline_state: PerTimelineState::default(),
myself: myself.clone(),
});
let mut mgr = StubManager {
shards: vec![timeline_a.clone(), timeline_b.clone()],
};
let key = DBDIR_KEY;
let mut cache = Cache::<TestTypes>::default();
cache
.get(timeline_a.id, ShardSelector::Page(key), &mgr)
.await
.expect("we have it");
cache
.get(timeline_b.id, ShardSelector::Page(key), &mgr)
.await
.expect("we have it");
assert_eq!(cache.map.len(), 2);
// delete timeline A
timeline_a.per_timeline_state.shutdown();
mgr.shards.retain(|t| t.id != timeline_a.id);
assert!(
mgr.resolve(timeline_a.id, ShardSelector::Page(key))
.await
.is_err(),
"broken StubManager implementation"
);
assert_eq!(
cache.map.len(),
2,
"cache still has a Weak handle to Timeline A"
);
cache
.get(timeline_a.id, ShardSelector::Page(key), &mgr)
.await
.err()
.expect("documented behavior: can't get new handle after shutdown");
assert_eq!(cache.map.len(), 1, "next access cleans up the cache");
cache
.get(timeline_b.id, ShardSelector::Page(key), &mgr)
.await
.expect("we still have it");
}
fn make_relation_key_for_shard(shard: ShardNumber, params: &ShardParameters) -> Key {
rel_block_to_key(
RelTag {
spcnode: 1663,
dbnode: 208101,
relnode: 2620,
forknum: 0,
},
shard.0 as u32 * params.stripe_size.0,
)
}
#[tokio::test(start_paused = true)]
async fn test_shard_split() {
crate::tenant::harness::setup_logging();
let timeline_id = TimelineId::generate();
let parent = Arc::new_cyclic(|myself| StubTimeline {
gate: Default::default(),
id: timeline_id,
shard: ShardIdentity::unsharded(),
per_timeline_state: PerTimelineState::default(),
myself: myself.clone(),
});
let child_params = ShardParameters {
count: ShardCount(2),
stripe_size: ShardStripeSize::default(),
};
let child0 = Arc::new_cyclic(|myself| StubTimeline {
gate: Default::default(),
id: timeline_id,
shard: ShardIdentity::from_params(ShardNumber(0), &child_params),
per_timeline_state: PerTimelineState::default(),
myself: myself.clone(),
});
let child1 = Arc::new_cyclic(|myself| StubTimeline {
gate: Default::default(),
id: timeline_id,
shard: ShardIdentity::from_params(ShardNumber(1), &child_params),
per_timeline_state: PerTimelineState::default(),
myself: myself.clone(),
});
let child_shards_by_shard_number = [child0.clone(), child1.clone()];
let mut cache = Cache::<TestTypes>::default();
// fill the cache with the parent
for i in 0..2 {
let handle = cache
.get(
timeline_id,
ShardSelector::Page(make_relation_key_for_shard(ShardNumber(i), &child_params)),
&StubManager {
shards: vec![parent.clone()],
},
)
.await
.expect("we have it");
assert!(
Weak::ptr_eq(&handle.myself, &parent.myself),
"mgr returns parent first"
);
drop(handle);
}
//
// SHARD SPLIT: tenant manager changes, but the cache isn't informed
//
// while we haven't shut down the parent, the cache will return the cached parent, even
// if the tenant manager returns the child
for i in 0..2 {
let handle = cache
.get(
timeline_id,
ShardSelector::Page(make_relation_key_for_shard(ShardNumber(i), &child_params)),
&StubManager {
shards: vec![], // doesn't matter what's in here, the cache is fully loaded
},
)
.await
.expect("we have it");
assert!(
Weak::ptr_eq(&handle.myself, &parent.myself),
"mgr returns parent"
);
drop(handle);
}
let parent_handle = cache
.get(
timeline_id,
ShardSelector::Page(make_relation_key_for_shard(ShardNumber(0), &child_params)),
&StubManager {
shards: vec![parent.clone()],
},
)
.await
.expect("we have it");
assert!(Weak::ptr_eq(&parent_handle.myself, &parent.myself));
// invalidate the cache
parent.per_timeline_state.shutdown();
// the cache will now return the child, even though the parent handle still exists
for i in 0..2 {
let handle = cache
.get(
timeline_id,
ShardSelector::Page(make_relation_key_for_shard(ShardNumber(i), &child_params)),
&StubManager {
shards: vec![child0.clone(), child1.clone()], // <====== this changed compared to previous loop
},
)
.await
.expect("we have it");
assert!(
Weak::ptr_eq(
&handle.myself,
&child_shards_by_shard_number[i as usize].myself
),
"mgr returns child"
);
drop(handle);
}
// all the while the parent handle kept the parent gate open
tokio::select! {
_ = parent_handle.gate.close() => {
panic!("parent handle is keeping gate open");
}
_ = tokio::time::sleep(FOREVER) => { }
}
drop(parent_handle);
tokio::select! {
_ = parent.gate.close() => { }
_ = tokio::time::sleep(FOREVER) => {
panic!("parent handle is dropped, no other gate holders exist")
}
}
}
#[tokio::test(start_paused = true)]
async fn test_connection_handler_exit() {
crate::tenant::harness::setup_logging();
let timeline_id = TimelineId::generate();
let shard0 = Arc::new_cyclic(|myself| StubTimeline {
gate: Default::default(),
id: timeline_id,
shard: ShardIdentity::unsharded(),
per_timeline_state: PerTimelineState::default(),
myself: myself.clone(),
});
let mgr = StubManager {
shards: vec![shard0.clone()],
};
let key = DBDIR_KEY;
// Simulate 10 connections that's opened, used, and closed
let mut used_handles = vec![];
for _ in 0..10 {
let mut cache = Cache::<TestTypes>::default();
let handle = {
let handle = cache
.get(timeline_id, ShardSelector::Page(key), &mgr)
.await
.expect("we have the timeline");
assert!(Weak::ptr_eq(&handle.myself, &shard0.myself));
handle
};
handle.getpage();
used_handles.push(Arc::downgrade(&handle.0));
}
// No handles exist, thus gates are closed and don't require shutdown
assert!(used_handles
.iter()
.all(|weak| Weak::strong_count(weak) == 0));
// ... thus the gate should close immediately, even without shutdown
tokio::select! {
_ = shard0.gate.close() => { }
_ = tokio::time::sleep(FOREVER) => {
panic!("handle is dropped, no other gate holders exist")
}
}
}
}

View File

@@ -35,6 +35,10 @@ impl LayerManager {
self.layer_fmgr.get_from_desc(desc)
}
pub(crate) fn get_from_key(&self, desc: &PersistentLayerKey) -> Layer {
self.layer_fmgr.get_from_key(desc)
}
/// Get an immutable reference to the layer map.
///
/// We expect users only to be able to get an immutable layer map. If users want to make modifications,
@@ -365,16 +369,20 @@ impl<T> Default for LayerFileManager<T> {
}
impl<T: AsLayerDesc + Clone> LayerFileManager<T> {
fn get_from_desc(&self, desc: &PersistentLayerDesc) -> T {
fn get_from_key(&self, key: &PersistentLayerKey) -> T {
// The assumption for the `expect()` is that all code maintains the following invariant:
// A layer's descriptor is present in the LayerMap => the LayerFileManager contains a layer for the descriptor.
self.0
.get(&desc.key())
.with_context(|| format!("get layer from desc: {}", desc.layer_name()))
.get(key)
.with_context(|| format!("get layer from key: {}", key))
.expect("not found")
.clone()
}
fn get_from_desc(&self, desc: &PersistentLayerDesc) -> T {
self.get_from_key(&desc.key())
}
fn contains_key(&self, key: &PersistentLayerKey) -> bool {
self.0.contains_key(key)
}

View File

@@ -27,8 +27,8 @@ use super::TaskStateUpdate;
use crate::{
context::RequestContext,
metrics::{LIVE_CONNECTIONS, WALRECEIVER_STARTED_CONNECTIONS, WAL_INGEST},
task_mgr::TaskKind,
task_mgr::WALRECEIVER_RUNTIME,
pgdatadir_mapping::DatadirModification,
task_mgr::{TaskKind, WALRECEIVER_RUNTIME},
tenant::{debug_assert_current_span_has_tenant_and_timeline_id, Timeline, WalReceiverInfo},
walingest::WalIngest,
walrecord::DecodedWALRecord,
@@ -342,7 +342,10 @@ pub(super) async fn handle_walreceiver_connection(
// Commit every ingest_batch_size records. Even if we filtered out
// all records, we still need to call commit to advance the LSN.
uncommitted_records += 1;
if uncommitted_records >= ingest_batch_size {
if uncommitted_records >= ingest_batch_size
|| modification.approx_pending_bytes()
> DatadirModification::MAX_PENDING_BYTES
{
WAL_INGEST
.records_committed
.inc_by(uncommitted_records - filtered_records);

View File

@@ -130,7 +130,7 @@ pub(super) enum UploadQueueStopped {
}
#[derive(thiserror::Error, Debug)]
pub(crate) enum NotInitialized {
pub enum NotInitialized {
#[error("queue is in state Uninitialized")]
Uninitialized,
#[error("queue is in state Stopped")]

View File

@@ -618,7 +618,7 @@ impl WalIngest {
// the offsets array is omitted if XLOG_HEAP_INIT_PAGE is set
0
} else {
std::mem::size_of::<u16>() * xlrec.ntuples as usize
size_of::<u16>() * xlrec.ntuples as usize
};
assert_eq!(offset_array_len, buf.remaining());
@@ -685,7 +685,7 @@ impl WalIngest {
// the offsets array is omitted if XLOG_HEAP_INIT_PAGE is set
0
} else {
std::mem::size_of::<u16>() * xlrec.ntuples as usize
size_of::<u16>() * xlrec.ntuples as usize
};
assert_eq!(offset_array_len, buf.remaining());
@@ -752,7 +752,7 @@ impl WalIngest {
// the offsets array is omitted if XLOG_HEAP_INIT_PAGE is set
0
} else {
std::mem::size_of::<u16>() * xlrec.ntuples as usize
size_of::<u16>() * xlrec.ntuples as usize
};
assert_eq!(offset_array_len, buf.remaining());
@@ -920,7 +920,7 @@ impl WalIngest {
// the offsets array is omitted if XLOG_HEAP_INIT_PAGE is set
0
} else {
std::mem::size_of::<u16>() * xlrec.ntuples as usize
size_of::<u16>() * xlrec.ntuples as usize
};
assert_eq!(offset_array_len, buf.remaining());

View File

@@ -241,6 +241,9 @@ impl PostgresRedoManager {
/// Shut down the WAL redo manager.
///
/// Returns `true` if this call was the one that initiated shutdown.
/// `true` may be observed by no caller if the first caller stops polling.
///
/// After this future completes
/// - no redo process is running
/// - no new redo process will be spawned
@@ -250,22 +253,32 @@ impl PostgresRedoManager {
/// # Cancel-Safety
///
/// This method is cancellation-safe.
pub async fn shutdown(&self) {
pub async fn shutdown(&self) -> bool {
// prevent new processes from being spawned
let permit = match self.redo_process.get_or_init_detached().await {
let maybe_permit = match self.redo_process.get_or_init_detached().await {
Ok(guard) => {
let (proc, permit) = guard.take_and_deinit();
drop(proc); // this just drops the Arc, its refcount may not be zero yet
permit
if matches!(&*guard, ProcessOnceCell::ManagerShutDown) {
None
} else {
let (proc, permit) = guard.take_and_deinit();
drop(proc); // this just drops the Arc, its refcount may not be zero yet
Some(permit)
}
}
Err(permit) => permit,
Err(permit) => Some(permit),
};
let it_was_us = if let Some(permit) = maybe_permit {
self.redo_process
.set(ProcessOnceCell::ManagerShutDown, permit);
true
} else {
false
};
self.redo_process
.set(ProcessOnceCell::ManagerShutDown, permit);
// wait for ongoing requests to drain and the refcounts of all Arc<WalRedoProcess> that
// we ever launched to drop to zero, which when it happens synchronously kill()s & wait()s
// for the underlying process.
self.launched_processes.close().await;
it_was_us
}
/// This type doesn't have its own background task to check for idleness: we

View File

@@ -0,0 +1,7 @@
# This was captured from one shard of a large tenant in staging.
# It has a mixture of deltas and image layers, >1000 layers in total.
# This is suitable for general smoke tests that want an index which is not
# trivially small, but doesn't contain weird/pathological cases.

File diff suppressed because one or more lines are too long

153
poetry.lock generated
View File

@@ -870,6 +870,96 @@ files = [
[package.dependencies]
colorama = {version = "*", markers = "platform_system == \"Windows\""}
[[package]]
name = "clickhouse-connect"
version = "0.7.17"
description = "ClickHouse Database Core Driver for Python, Pandas, and Superset"
optional = false
python-versions = "~=3.8"
files = [
{file = "clickhouse-connect-0.7.17.tar.gz", hash = "sha256:854f1f9f3e024e7f89ae5d57cd3289d7a4c3dc91a9f24c4d233014f0ea19cb2d"},
{file = "clickhouse_connect-0.7.17-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:aca36f5f28be1ada2981fce87724bbf451f267c918015baec59e527de3c9c882"},
{file = "clickhouse_connect-0.7.17-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:66209e4634f457604c263bea176336079d26c284e251e68a8435b0b80c1a25ff"},
{file = "clickhouse_connect-0.7.17-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e4d86c5a561a2a99321c8b4af22257461b8e67142f34cfea6e70f39b45b1f406"},
{file = "clickhouse_connect-0.7.17-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d200c9afa2725a96f9f3718221f641276b80c11bf504d8a2fbaafb5a05b2f0d3"},
{file = "clickhouse_connect-0.7.17-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:004d867b1005445a46e6742db1054bf2a717a451372663b46e09b5e9e90a31e3"},
{file = "clickhouse_connect-0.7.17-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:4ef94a4a8e008882259151833c3c47cfbb9c8f08de0f100aaf3b95c366dcfb24"},
{file = "clickhouse_connect-0.7.17-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:ee732c3df50c8b07d16b5836ff85e6b84569922455c03837c3add5cf1388fe1f"},
{file = "clickhouse_connect-0.7.17-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:d9dbe1235465bb946e24b90b0ca5b8800b5d645acb2d7d6ee819448c3e2fd959"},
{file = "clickhouse_connect-0.7.17-cp310-cp310-win32.whl", hash = "sha256:e5db0d68dfb63db0297d44dc91406bcfd7d333708d7cd55086c8550fbf870b78"},
{file = "clickhouse_connect-0.7.17-cp310-cp310-win_amd64.whl", hash = "sha256:800750f568c097ea312887785025006d6098bffd8ed2dd6a57048fb3ced6d778"},
{file = "clickhouse_connect-0.7.17-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:4eb390623b3d15dc9cda78f5c68f83ef9ad11743797e70af8fabc384b015a73c"},
{file = "clickhouse_connect-0.7.17-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:35f172ca950f218f63072024c81d5b4ff6e5399620c255506c321ccc7b17c9a5"},
{file = "clickhouse_connect-0.7.17-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ae7918f060f7576fc931c692e0122b1b07576fabd81444af22e1f8582300d200"},
{file = "clickhouse_connect-0.7.17-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ff2881b93c7a1afb9c99fb59ad5fd666850421325d0931e2b77f3f4ba872303d"},
{file = "clickhouse_connect-0.7.17-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:8a4d9b4f97271addf66aadbaf7f154f19a0ad6c22026d575a995c55ebd8576db"},
{file = "clickhouse_connect-0.7.17-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:e431469b1ff2d5c3e4c406d55c6afdf7102f5d2524c2ceb5481b94ac24412aa3"},
{file = "clickhouse_connect-0.7.17-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:2b6f80115176559f181a6b3ecad11aa3d70ef6014c3d2905b90fcef3f27d25c2"},
{file = "clickhouse_connect-0.7.17-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:d8ac694f40dfafc8a3cc877116b4bc73e8877ebf66d4d96ee092484ee4c0b481"},
{file = "clickhouse_connect-0.7.17-cp311-cp311-win32.whl", hash = "sha256:78b7a3f6b0fad4eaf8afb5f9a2e855bde53e82ea5804960e9cf779538f4606a1"},
{file = "clickhouse_connect-0.7.17-cp311-cp311-win_amd64.whl", hash = "sha256:efd390cc045334ecc3f2a9c18cc07c041d0288b145967805fdcab65abeefa75f"},
{file = "clickhouse_connect-0.7.17-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:9228334a17dc0a7842222f54ba5b89fc563532424aad4f66be799df70ab37e9f"},
{file = "clickhouse_connect-0.7.17-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:e432a42bb788bda77e88eda2774392a60fbbb5ee2a79cb2881d182d26c45fe49"},
{file = "clickhouse_connect-0.7.17-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c85152ed2879965ee1fa2bd5e31fb27d281fd5f50d6e86a401efd95cd85b29ef"},
{file = "clickhouse_connect-0.7.17-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:29a126104aa5e11df570cbd89fca4988784084602ba77d17b2396b334c54fd75"},
{file = "clickhouse_connect-0.7.17-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:882d8f9570549258e6eb6a97915fbf64ed29fe395d5e360866ea8d42c8283a35"},
{file = "clickhouse_connect-0.7.17-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:06ebf99111171442f462fb8b357364c3e276da3e8f8557b2e8fee9eb55ab37d1"},
{file = "clickhouse_connect-0.7.17-cp312-cp312-musllinux_1_1_i686.whl", hash = "sha256:e0cf6f99b2777b0d164bf8b65ec39104cdc0789a56bcb52d98289bbd6f5cc70e"},
{file = "clickhouse_connect-0.7.17-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:ee46c508fddfff3b7ac52326788e0c6dd8dfb416b6d7e02e5d30e8110749dac2"},
{file = "clickhouse_connect-0.7.17-cp312-cp312-win32.whl", hash = "sha256:eb708b590a37d56b069a6088254ffa55d73b8cb65527339df81ef03fe67ffdec"},
{file = "clickhouse_connect-0.7.17-cp312-cp312-win_amd64.whl", hash = "sha256:17f00dccddaeaf43733faa1fa21f7d24641454a73669fda862545ba7c88627f5"},
{file = "clickhouse_connect-0.7.17-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:ab5d4b37a6dcc39e94c63beac0f22d9dda914f5eb865d166c64cf04dfadb7d16"},
{file = "clickhouse_connect-0.7.17-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:32aa90387f45f34cbc5a984789ed4c12760a3c0056c190ab0123ceafc36b1002"},
{file = "clickhouse_connect-0.7.17-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:21277b6bdd6c8ff14170bfcd52125c5c39f442ec4bafbb643ad7d0ca915f0029"},
{file = "clickhouse_connect-0.7.17-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ca68d8b7dee3fb4e7229e06152f5b0faaccafb4c87d9c2d48fa5bd117a3cc1c0"},
{file = "clickhouse_connect-0.7.17-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:841c56282102b2fba1e0b332bb1c7a0c50992fbc321746af8d3e0e6ca2450e8b"},
{file = "clickhouse_connect-0.7.17-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:8d7ffde5a4b95d8fe9ed38e08e504e497310e3d7a17691bd40bf65734648fdfc"},
{file = "clickhouse_connect-0.7.17-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:055960086b6b92b6e44f5ba04c81c40c10b038588e4b3908b033c99f66125332"},
{file = "clickhouse_connect-0.7.17-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:36491fec63ceb8503b6344c23477647030139f346b749dc5ee672c505939dbbe"},
{file = "clickhouse_connect-0.7.17-cp38-cp38-win32.whl", hash = "sha256:8779a907e026db32e6bc0bc0c8d5de0e2e3afd166afc2d4adcc0603399af5539"},
{file = "clickhouse_connect-0.7.17-cp38-cp38-win_amd64.whl", hash = "sha256:309854fa197885c6278438ddd032ab52e6fec56f162074e343c3635ca7266078"},
{file = "clickhouse_connect-0.7.17-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:e8009f94550178dc971aeb4f8787ba7a5b473c22647490428b7229f540a51d2b"},
{file = "clickhouse_connect-0.7.17-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:70f8422f407b13a404b3670fd097855abd5adaf890c710d6678d2b46ab61ac48"},
{file = "clickhouse_connect-0.7.17-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:082783eb1e8baf7b3465dd045132dc5cb5a91432c899dc4e19891c5f782d8d23"},
{file = "clickhouse_connect-0.7.17-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c1c30aad2a9c7584c4ee19e646a087b3bbd2d4daab3d88a2afeeae1a7f6febf9"},
{file = "clickhouse_connect-0.7.17-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:fc8e245a9f4f0dce39f155e626405f60f1d3cf4d1e52dd2c793ea6b603ca111b"},
{file = "clickhouse_connect-0.7.17-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:802372cb8a69c9ffdf4260e9f01616c8601ba531825ed6f08834827e0b880cd1"},
{file = "clickhouse_connect-0.7.17-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:193a60271a3b105cdbde96fb20b40eab8a50fca3bb1f397546f7a18b53d9aa9c"},
{file = "clickhouse_connect-0.7.17-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:59d58932916792fdbd09cb961a245a0c2d87b07b8296f9138915b998f4522941"},
{file = "clickhouse_connect-0.7.17-cp39-cp39-win32.whl", hash = "sha256:3cfd0edabb589f640636a97ffc38d1b3d760faef208d44e50829cc1ad3f0d3e5"},
{file = "clickhouse_connect-0.7.17-cp39-cp39-win_amd64.whl", hash = "sha256:5661b4629aac228481219abf2e149119af1a71d897f191665e182d9d192d7033"},
{file = "clickhouse_connect-0.7.17-pp310-pypy310_pp73-macosx_10_9_x86_64.whl", hash = "sha256:7429d309109e7e4a70fd867d69fcfea9ddcb1a1e910caa6b0e2c3776b71f4613"},
{file = "clickhouse_connect-0.7.17-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e5ae619151006da84a0b1585a9bcc81be32459d8061aeb2e116bad5bbaa7d108"},
{file = "clickhouse_connect-0.7.17-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ec0c84a0880621cb2389656a89886ef3133f0b3f8dc016eee6f25bbb49ff6f70"},
{file = "clickhouse_connect-0.7.17-pp310-pypy310_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:705464c23f821666b76f8f619cf2870225156276562756b3933aaa24708e0ff8"},
{file = "clickhouse_connect-0.7.17-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:1822016f4b769e89264fe26cefe0bc5e50e4c3ca0747d89bb52d57dc4f1e5ffb"},
{file = "clickhouse_connect-0.7.17-pp38-pypy38_pp73-macosx_10_9_x86_64.whl", hash = "sha256:6c92b0c342c1fbfa666010e8175e05026dc570a7ef91d8fa81ce503180f318aa"},
{file = "clickhouse_connect-0.7.17-pp38-pypy38_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d2e106536540e906c3c866f8615fcf870a9a77c1bfab9ef4b042febfd2fdb953"},
{file = "clickhouse_connect-0.7.17-pp38-pypy38_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bac9a32e62384b4341ba51a451084eb3b00c6e59aaac1499145dd8b897cb585c"},
{file = "clickhouse_connect-0.7.17-pp38-pypy38_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0feed93b9912b7862a8c41be1febcd44b68a824a5c1059b19d5c567afdaa6273"},
{file = "clickhouse_connect-0.7.17-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:2e2dd6db52e799f065fd565143fde5a872cfe903de1bee7775bc3a349856a790"},
{file = "clickhouse_connect-0.7.17-pp39-pypy39_pp73-macosx_10_9_x86_64.whl", hash = "sha256:ed13add5d579a5960155f3000420544368501c9703d2fb94f103b4a6126081f6"},
{file = "clickhouse_connect-0.7.17-pp39-pypy39_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c257a23ed3bf1858593fb03927d9d073fbbdfa24dc2afee537c3314bd66b4e24"},
{file = "clickhouse_connect-0.7.17-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d47866f64cbdc2d5cc4f8a7a8c49e3ee90c9e487091b9eda7c3a3576418e1cbe"},
{file = "clickhouse_connect-0.7.17-pp39-pypy39_pp73-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:9b850e2f17e0a0b5a37d996d3fb728050227489d64d271d678d166abea94f26e"},
{file = "clickhouse_connect-0.7.17-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:349682288987dc84ac7695f7cd6b510be8d0ec0eee7c1b72dbf2146b4e9efdb8"},
]
[package.dependencies]
certifi = "*"
lz4 = "*"
pytz = "*"
urllib3 = ">=1.26"
zstandard = "*"
[package.extras]
arrow = ["pyarrow"]
numpy = ["numpy"]
orjson = ["orjson"]
pandas = ["pandas"]
sqlalchemy = ["sqlalchemy (>1.3.21,<2.0)"]
tzlocal = ["tzlocal (>=4.0)"]
[[package]]
name = "colorama"
version = "0.4.5"
@@ -1470,6 +1560,56 @@ files = [
{file = "lazy_object_proxy-1.10.0-pp310.pp311.pp312.pp38.pp39-none-any.whl", hash = "sha256:80fa48bd89c8f2f456fc0765c11c23bf5af827febacd2f523ca5bc1893fcc09d"},
]
[[package]]
name = "lz4"
version = "4.3.3"
description = "LZ4 Bindings for Python"
optional = false
python-versions = ">=3.8"
files = [
{file = "lz4-4.3.3-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b891880c187e96339474af2a3b2bfb11a8e4732ff5034be919aa9029484cd201"},
{file = "lz4-4.3.3-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:222a7e35137d7539c9c33bb53fcbb26510c5748779364014235afc62b0ec797f"},
{file = "lz4-4.3.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f76176492ff082657ada0d0f10c794b6da5800249ef1692b35cf49b1e93e8ef7"},
{file = "lz4-4.3.3-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f1d18718f9d78182c6b60f568c9a9cec8a7204d7cb6fad4e511a2ef279e4cb05"},
{file = "lz4-4.3.3-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6cdc60e21ec70266947a48839b437d46025076eb4b12c76bd47f8e5eb8a75dcc"},
{file = "lz4-4.3.3-cp310-cp310-win32.whl", hash = "sha256:c81703b12475da73a5d66618856d04b1307e43428a7e59d98cfe5a5d608a74c6"},
{file = "lz4-4.3.3-cp310-cp310-win_amd64.whl", hash = "sha256:43cf03059c0f941b772c8aeb42a0813d68d7081c009542301637e5782f8a33e2"},
{file = "lz4-4.3.3-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:30e8c20b8857adef7be045c65f47ab1e2c4fabba86a9fa9a997d7674a31ea6b6"},
{file = "lz4-4.3.3-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:2f7b1839f795315e480fb87d9bc60b186a98e3e5d17203c6e757611ef7dcef61"},
{file = "lz4-4.3.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:edfd858985c23523f4e5a7526ca6ee65ff930207a7ec8a8f57a01eae506aaee7"},
{file = "lz4-4.3.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0e9c410b11a31dbdc94c05ac3c480cb4b222460faf9231f12538d0074e56c563"},
{file = "lz4-4.3.3-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d2507ee9c99dbddd191c86f0e0c8b724c76d26b0602db9ea23232304382e1f21"},
{file = "lz4-4.3.3-cp311-cp311-win32.whl", hash = "sha256:f180904f33bdd1e92967923a43c22899e303906d19b2cf8bb547db6653ea6e7d"},
{file = "lz4-4.3.3-cp311-cp311-win_amd64.whl", hash = "sha256:b14d948e6dce389f9a7afc666d60dd1e35fa2138a8ec5306d30cd2e30d36b40c"},
{file = "lz4-4.3.3-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:e36cd7b9d4d920d3bfc2369840da506fa68258f7bb176b8743189793c055e43d"},
{file = "lz4-4.3.3-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:31ea4be9d0059c00b2572d700bf2c1bc82f241f2c3282034a759c9a4d6ca4dc2"},
{file = "lz4-4.3.3-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:33c9a6fd20767ccaf70649982f8f3eeb0884035c150c0b818ea660152cf3c809"},
{file = "lz4-4.3.3-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bca8fccc15e3add173da91be8f34121578dc777711ffd98d399be35487c934bf"},
{file = "lz4-4.3.3-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e7d84b479ddf39fe3ea05387f10b779155fc0990125f4fb35d636114e1c63a2e"},
{file = "lz4-4.3.3-cp312-cp312-win32.whl", hash = "sha256:337cb94488a1b060ef1685187d6ad4ba8bc61d26d631d7ba909ee984ea736be1"},
{file = "lz4-4.3.3-cp312-cp312-win_amd64.whl", hash = "sha256:5d35533bf2cee56f38ced91f766cd0038b6abf46f438a80d50c52750088be93f"},
{file = "lz4-4.3.3-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:363ab65bf31338eb364062a15f302fc0fab0a49426051429866d71c793c23394"},
{file = "lz4-4.3.3-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:0a136e44a16fc98b1abc404fbabf7f1fada2bdab6a7e970974fb81cf55b636d0"},
{file = "lz4-4.3.3-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:abc197e4aca8b63f5ae200af03eb95fb4b5055a8f990079b5bdf042f568469dd"},
{file = "lz4-4.3.3-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:56f4fe9c6327adb97406f27a66420b22ce02d71a5c365c48d6b656b4aaeb7775"},
{file = "lz4-4.3.3-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f0e822cd7644995d9ba248cb4b67859701748a93e2ab7fc9bc18c599a52e4604"},
{file = "lz4-4.3.3-cp38-cp38-win32.whl", hash = "sha256:24b3206de56b7a537eda3a8123c644a2b7bf111f0af53bc14bed90ce5562d1aa"},
{file = "lz4-4.3.3-cp38-cp38-win_amd64.whl", hash = "sha256:b47839b53956e2737229d70714f1d75f33e8ac26e52c267f0197b3189ca6de24"},
{file = "lz4-4.3.3-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:6756212507405f270b66b3ff7f564618de0606395c0fe10a7ae2ffcbbe0b1fba"},
{file = "lz4-4.3.3-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:ee9ff50557a942d187ec85462bb0960207e7ec5b19b3b48949263993771c6205"},
{file = "lz4-4.3.3-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2b901c7784caac9a1ded4555258207d9e9697e746cc8532129f150ffe1f6ba0d"},
{file = "lz4-4.3.3-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b6d9ec061b9eca86e4dcc003d93334b95d53909afd5a32c6e4f222157b50c071"},
{file = "lz4-4.3.3-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:f4c7bf687303ca47d69f9f0133274958fd672efaa33fb5bcde467862d6c621f0"},
{file = "lz4-4.3.3-cp39-cp39-win32.whl", hash = "sha256:054b4631a355606e99a42396f5db4d22046a3397ffc3269a348ec41eaebd69d2"},
{file = "lz4-4.3.3-cp39-cp39-win_amd64.whl", hash = "sha256:eac9af361e0d98335a02ff12fb56caeb7ea1196cf1a49dbf6f17828a131da807"},
{file = "lz4-4.3.3.tar.gz", hash = "sha256:01fe674ef2889dbb9899d8a67361e0c4a2c833af5aeb37dd505727cf5d2a131e"},
]
[package.extras]
docs = ["sphinx (>=1.6.0)", "sphinx-bootstrap-theme"]
flake8 = ["flake8"]
tests = ["psutil", "pytest (!=3.3.0)", "pytest-cov"]
[[package]]
name = "markupsafe"
version = "2.1.1"
@@ -2361,6 +2501,17 @@ files = [
[package.dependencies]
six = ">=1.5"
[[package]]
name = "pytz"
version = "2024.1"
description = "World timezone definitions, modern and historical"
optional = false
python-versions = "*"
files = [
{file = "pytz-2024.1-py2.py3-none-any.whl", hash = "sha256:328171f4e3623139da4983451950b28e95ac706e13f3f2630a879749e7a8b319"},
{file = "pytz-2024.1.tar.gz", hash = "sha256:2a29735ea9c18baf14b448846bde5a48030ed267578472d8955cd0e7443a9812"},
]
[[package]]
name = "pywin32"
version = "301"
@@ -3206,4 +3357,4 @@ cffi = ["cffi (>=1.11)"]
[metadata]
lock-version = "2.0"
python-versions = "^3.9"
content-hash = "16ebd6a46768be7f67dbdb4ee5903b167d94edc9965f29252f038c67e9e907b0"
content-hash = "7cee6a8c30bc7f4bfb0a87c6bad3952dfb4da127fad853d2710a93ac3eab8a00"

View File

@@ -376,7 +376,7 @@ async fn main() -> anyhow::Result<()> {
let cancel_map = CancelMap::default();
let redis_publisher = match &redis_notifications_client {
let redis_publisher = match &regional_redis_client {
Some(redis_publisher) => Some(Arc::new(Mutex::new(RedisPublisherClient::new(
redis_publisher.clone(),
args.region.clone(),

View File

@@ -103,12 +103,8 @@ impl ConnCfg {
/// Reuse password or auth keys from the other config.
pub fn reuse_password(&mut self, other: Self) {
if let Some(password) = other.get_password() {
self.password(password);
}
if let Some(keys) = other.get_auth_keys() {
self.auth_keys(keys);
if let Some(password) = other.get_auth() {
self.auth(password);
}
}
@@ -124,48 +120,64 @@ impl ConnCfg {
/// Apply startup message params to the connection config.
pub fn set_startup_params(&mut self, params: &StartupMessageParams) {
// Only set `user` if it's not present in the config.
// Link auth flow takes username from the console's response.
if let (None, Some(user)) = (self.get_user(), params.get("user")) {
self.user(user);
}
// Only set `dbname` if it's not present in the config.
// Link auth flow takes dbname from the console's response.
if let (None, Some(dbname)) = (self.get_dbname(), params.get("database")) {
self.dbname(dbname);
}
// Don't add `options` if they were only used for specifying a project.
// Connection pools don't support `options`, because they affect backend startup.
if let Some(options) = filtered_options(params) {
self.options(&options);
}
if let Some(app_name) = params.get("application_name") {
self.application_name(app_name);
}
// TODO: This is especially ugly...
if let Some(replication) = params.get("replication") {
use tokio_postgres::config::ReplicationMode;
match replication {
"true" | "on" | "yes" | "1" => {
self.replication_mode(ReplicationMode::Physical);
let mut client_encoding = false;
for (k, v) in params.iter() {
match k {
"user" => {
// Only set `user` if it's not present in the config.
// Link auth flow takes username from the console's response.
if self.get_user().is_none() {
self.user(v);
}
}
"database" => {
self.replication_mode(ReplicationMode::Logical);
// Only set `dbname` if it's not present in the config.
// Link auth flow takes dbname from the console's response.
if self.get_dbname().is_none() {
self.dbname(v);
}
}
"options" => {
// Don't add `options` if they were only used for specifying a project.
// Connection pools don't support `options`, because they affect backend startup.
if let Some(options) = filtered_options(v) {
self.options(&options);
}
}
// the special ones in tokio-postgres that we don't want being set by the user
"dbname" => {}
"password" => {}
"sslmode" => {}
"host" => {}
"port" => {}
"connect_timeout" => {}
"keepalives" => {}
"keepalives_idle" => {}
"keepalives_interval" => {}
"keepalives_retries" => {}
"target_session_attrs" => {}
"channel_binding" => {}
"max_backend_message_size" => {}
"client_encoding" => {
client_encoding = true;
// only error should be from bad null bytes,
// but we've already checked for those.
_ = self.param("client_encoding", v);
}
_ => {
// only error should be from bad null bytes,
// but we've already checked for those.
_ = self.param(k, v);
}
_other => {}
}
}
// TODO: extend the list of the forwarded startup parameters.
// Currently, tokio-postgres doesn't allow us to pass
// arbitrary parameters, but the ones above are a good start.
//
// This and the reverse params problem can be better addressed
// in a bespoke connection machinery (a new library for that sake).
if !client_encoding {
// for compatibility since we removed it from tokio-postgres
self.param("client_encoding", "UTF8").unwrap();
}
}
}
@@ -338,10 +350,9 @@ impl ConnCfg {
}
/// Retrieve `options` from a startup message, dropping all proxy-secific flags.
fn filtered_options(params: &StartupMessageParams) -> Option<String> {
fn filtered_options(options: &str) -> Option<String> {
#[allow(unstable_name_collisions)]
let options: String = params
.options_raw()?
let options: String = StartupMessageParams::parse_options_raw(options)
.filter(|opt| parse_endpoint_param(opt).is_none() && neon_option(opt).is_none())
.intersperse(" ") // TODO: use impl from std once it's stabilized
.collect();
@@ -413,27 +424,23 @@ mod tests {
#[test]
fn test_filtered_options() {
// Empty options is unlikely to be useful anyway.
let params = StartupMessageParams::new([("options", "")]);
assert_eq!(filtered_options(&params), None);
assert_eq!(filtered_options(""), None);
// It's likely that clients will only use options to specify endpoint/project.
let params = StartupMessageParams::new([("options", "project=foo")]);
assert_eq!(filtered_options(&params), None);
let params = "project=foo";
assert_eq!(filtered_options(params), None);
// Same, because unescaped whitespaces are no-op.
let params = StartupMessageParams::new([("options", " project=foo ")]);
assert_eq!(filtered_options(&params).as_deref(), None);
let params = " project=foo ";
assert_eq!(filtered_options(params), None);
let params = StartupMessageParams::new([("options", r"\ project=foo \ ")]);
assert_eq!(filtered_options(&params).as_deref(), Some(r"\ \ "));
let params = r"\ project=foo \ ";
assert_eq!(filtered_options(params).as_deref(), Some(r"\ \ "));
let params = StartupMessageParams::new([("options", "project = foo")]);
assert_eq!(filtered_options(&params).as_deref(), Some("project = foo"));
let params = "project = foo";
assert_eq!(filtered_options(params).as_deref(), Some("project = foo"));
let params = StartupMessageParams::new([(
"options",
"project = foo neon_endpoint_type:read_write neon_lsn:0/2",
)]);
assert_eq!(filtered_options(&params).as_deref(), Some("project = foo"));
let params = "project = foo neon_endpoint_type:read_write neon_lsn:0/2";
assert_eq!(filtered_options(params).as_deref(), Some("project = foo"));
}
}

View File

@@ -1,7 +1,7 @@
//! Man-in-the-middle tests
//!
//! Channel binding should prevent a proxy server
//! - that has access to create valid certificates -
//! *that has access to create valid certificates*
//! from controlling the TLS connection.
use std::fmt::Debug;

View File

@@ -158,7 +158,7 @@ mod tests {
let N = 1021 * 4096;
let sketch = CountMinSketch::with_params(p / N as f64, 1.0 - q);
let memory = std::mem::size_of::<u32>() * sketch.buckets.len();
let memory = size_of::<u32>() * sketch.buckets.len();
let time = sketch.depth;
(memory, time)
}

View File

@@ -236,6 +236,10 @@ impl ConnectMechanism for TokioMechanism {
.dbname(&self.conn_info.dbname)
.connect_timeout(timeout);
config
.param("client_encoding", "UTF8")
.expect("client encoding UTF8 is always valid");
let pause = ctx.latency_timer.pause(crate::metrics::Waiting::Compute);
let res = config.connect(tokio_postgres::NoTls).await;
drop(pause);

View File

@@ -203,6 +203,7 @@ fn get_conn_info(
options = Some(NeonOptions::parse_options_raw(&value));
}
}
ctx.set_db_options(params.freeze());
let user_info = ComputeUserInfo {
endpoint,

View File

@@ -41,6 +41,7 @@ zstandard = "^0.21.0"
httpx = {extras = ["http2"], version = "^0.26.0"}
pytest-repeat = "^0.9.3"
websockets = "^12.0"
clickhouse-connect = "^0.7.16"
[tool.poetry.group.dev.dependencies]
mypy = "==1.3.0"

View File

@@ -1,5 +1,5 @@
[toolchain]
channel = "1.79.0"
channel = "1.80.0"
profile = "default"
# The default profile includes rustc, rust-std, cargo, rust-docs, rustfmt and clippy.
# https://rust-lang.github.io/rustup/concepts/profiles.html

View File

@@ -170,11 +170,6 @@ struct Args {
/// still needed for existing replication connection.
#[arg(long)]
walsenders_keep_horizon: bool,
/// Enable partial backup. If disabled, safekeeper will not upload partial
/// segments to remote storage.
/// TODO: now partial backup is always enabled, remove this flag.
#[arg(long)]
partial_backup_enabled: bool,
/// Controls how long backup will wait until uploading the partial segment.
#[arg(long, value_parser = humantime::parse_duration, default_value = DEFAULT_PARTIAL_BACKUP_TIMEOUT, verbatim_doc_comment)]
partial_backup_timeout: Duration,
@@ -347,7 +342,6 @@ async fn main() -> anyhow::Result<()> {
sk_auth_token,
current_thread_runtime: args.current_thread_runtime,
walsenders_keep_horizon: args.walsenders_keep_horizon,
partial_backup_enabled: true,
partial_backup_timeout: args.partial_backup_timeout,
disable_periodic_broker_push: args.disable_periodic_broker_push,
enable_offload: args.enable_offload,

View File

@@ -27,7 +27,7 @@ pub const SK_FORMAT_VERSION: u32 = 9;
pub const CONTROL_FILE_NAME: &str = "safekeeper.control";
// needed to atomically update the state using `rename`
const CONTROL_FILE_NAME_PARTIAL: &str = "safekeeper.control.partial";
pub const CHECKSUM_SIZE: usize = std::mem::size_of::<u32>();
pub const CHECKSUM_SIZE: usize = size_of::<u32>();
/// Storage should keep actual state inside of it. It should implement Deref
/// trait to access state fields and have persist method for updating that state.

View File

@@ -143,7 +143,12 @@ impl<IO: AsyncRead + AsyncWrite + Unpin + Send> postgres_backend::Handler<IO>
self.tenant_id.unwrap_or(TenantId::from([0u8; 16])),
self.timeline_id.unwrap_or(TimelineId::from([0u8; 16])),
);
tracing::Span::current().record("ttid", tracing::field::display(ttid));
tracing::Span::current()
.record("ttid", tracing::field::display(ttid))
.record(
"application_name",
tracing::field::debug(self.appname.clone()),
);
Ok(())
} else {

View File

@@ -21,6 +21,7 @@ pub mod json_ctrl;
pub mod metrics;
pub mod patch_control_file;
pub mod pull_timeline;
pub mod rate_limit;
pub mod receive_wal;
pub mod recovery;
pub mod remove_wal;
@@ -53,6 +54,7 @@ pub mod defaults {
pub const DEFAULT_PARTIAL_BACKUP_TIMEOUT: &str = "15m";
pub const DEFAULT_CONTROL_FILE_SAVE_INTERVAL: &str = "300s";
pub const DEFAULT_PARTIAL_BACKUP_CONCURRENCY: &str = "5";
pub const DEFAULT_EVICTION_CONCURRENCY: usize = 2;
// By default, our required residency before eviction is the same as the period that passes
// before uploading a partial segment, so that in normal operation the eviction can happen
@@ -91,7 +93,6 @@ pub struct SafeKeeperConf {
pub sk_auth_token: Option<SecretString>,
pub current_thread_runtime: bool,
pub walsenders_keep_horizon: bool,
pub partial_backup_enabled: bool,
pub partial_backup_timeout: Duration,
pub disable_periodic_broker_push: bool,
pub enable_offload: bool,
@@ -135,7 +136,6 @@ impl SafeKeeperConf {
max_offloader_lag_bytes: defaults::DEFAULT_MAX_OFFLOADER_LAG_BYTES,
current_thread_runtime: false,
walsenders_keep_horizon: false,
partial_backup_enabled: false,
partial_backup_timeout: Duration::from_secs(0),
disable_periodic_broker_push: false,
enable_offload: false,

View File

@@ -0,0 +1,49 @@
use std::sync::Arc;
use rand::Rng;
use crate::metrics::MISC_OPERATION_SECONDS;
/// Global rate limiter for background tasks.
#[derive(Clone)]
pub struct RateLimiter {
partial_backup: Arc<tokio::sync::Semaphore>,
eviction: Arc<tokio::sync::Semaphore>,
}
impl RateLimiter {
/// Create a new rate limiter.
/// - `partial_backup_max`: maximum number of concurrent partial backups.
/// - `eviction_max`: maximum number of concurrent timeline evictions.
pub fn new(partial_backup_max: usize, eviction_max: usize) -> Self {
Self {
partial_backup: Arc::new(tokio::sync::Semaphore::new(partial_backup_max)),
eviction: Arc::new(tokio::sync::Semaphore::new(eviction_max)),
}
}
/// Get a permit for partial backup. This will block if the maximum number of concurrent
/// partial backups is reached.
pub async fn acquire_partial_backup(&self) -> tokio::sync::OwnedSemaphorePermit {
let _timer = MISC_OPERATION_SECONDS
.with_label_values(&["partial_permit_acquire"])
.start_timer();
self.partial_backup
.clone()
.acquire_owned()
.await
.expect("semaphore is closed")
}
/// Try to get a permit for timeline eviction. This will return None if the maximum number of
/// concurrent timeline evictions is reached.
pub fn try_acquire_eviction(&self) -> Option<tokio::sync::OwnedSemaphorePermit> {
self.eviction.clone().try_acquire_owned().ok()
}
}
/// Generate a random duration that is a fraction of the given duration.
pub fn rand_duration(duration: &std::time::Duration) -> std::time::Duration {
let randf64 = rand::thread_rng().gen_range(0.0..1.0);
duration.mul_f64(randf64)
}

View File

@@ -25,6 +25,7 @@ use utils::{
use storage_broker::proto::SafekeeperTimelineInfo;
use storage_broker::proto::TenantTimelineId as ProtoTenantTimelineId;
use crate::rate_limit::RateLimiter;
use crate::receive_wal::WalReceivers;
use crate::safekeeper::{
AcceptorProposerMessage, ProposerAcceptorMessage, SafeKeeper, ServerInfo, Term, TermLsn,
@@ -36,7 +37,7 @@ use crate::timeline_guard::ResidenceGuard;
use crate::timeline_manager::{AtomicStatus, ManagerCtl};
use crate::timelines_set::TimelinesSet;
use crate::wal_backup::{self};
use crate::wal_backup_partial::{PartialRemoteSegment, RateLimiter};
use crate::wal_backup_partial::PartialRemoteSegment;
use crate::{control_file, safekeeper::UNKNOWN_SERVER_VERSION};
use crate::metrics::{FullTimelineInfo, WalStorageMetrics, MISC_OPERATION_SECONDS};

View File

@@ -5,7 +5,6 @@
use anyhow::Context;
use camino::Utf8PathBuf;
use remote_storage::RemotePath;
use std::time::Instant;
use tokio::{
fs::File,
io::{AsyncRead, AsyncWriteExt},
@@ -15,6 +14,7 @@ use utils::crashsafe::durable_rename;
use crate::{
metrics::{EvictionEvent, EVICTION_EVENTS_COMPLETED, EVICTION_EVENTS_STARTED},
rate_limit::rand_duration,
timeline_manager::{Manager, StateSnapshot},
wal_backup,
wal_backup_partial::{self, PartialRemoteSegment},
@@ -50,7 +50,6 @@ impl Manager {
.flush_lsn
.segment_number(self.wal_seg_size)
== self.last_removed_segno + 1
&& self.resident_since.elapsed() >= self.conf.eviction_min_resident
}
/// Evict the timeline to remote storage.
@@ -112,7 +111,8 @@ impl Manager {
return;
}
self.resident_since = Instant::now();
self.evict_not_before =
tokio::time::Instant::now() + rand_duration(&self.conf.eviction_min_resident);
info!("successfully restored evicted timeline");
}

View File

@@ -23,6 +23,7 @@ use utils::lsn::Lsn;
use crate::{
control_file::{FileStorage, Storage},
metrics::{MANAGER_ACTIVE_CHANGES, MANAGER_ITERATIONS_TOTAL, MISC_OPERATION_SECONDS},
rate_limit::{rand_duration, RateLimiter},
recovery::recovery_main,
remove_wal::calc_horizon_lsn,
safekeeper::Term,
@@ -32,7 +33,7 @@ use crate::{
timeline_guard::{AccessService, GuardId, ResidenceGuard},
timelines_set::{TimelineSetGuard, TimelinesSet},
wal_backup::{self, WalBackupTaskHandle},
wal_backup_partial::{self, PartialRemoteSegment, RateLimiter},
wal_backup_partial::{self, PartialRemoteSegment},
SafeKeeperConf,
};
@@ -185,11 +186,11 @@ pub(crate) struct Manager {
// misc
pub(crate) access_service: AccessService,
pub(crate) partial_backup_rate_limiter: RateLimiter,
pub(crate) global_rate_limiter: RateLimiter,
// Anti-flapping state: we evict timelines eagerly if they are inactive, but should not
// evict them if they go inactive very soon after being restored.
pub(crate) resident_since: std::time::Instant,
pub(crate) evict_not_before: Instant,
}
/// This task gets spawned alongside each timeline and is responsible for managing the timeline's
@@ -202,7 +203,7 @@ pub async fn main_task(
broker_active_set: Arc<TimelinesSet>,
manager_tx: tokio::sync::mpsc::UnboundedSender<ManagerCtlMessage>,
mut manager_rx: tokio::sync::mpsc::UnboundedReceiver<ManagerCtlMessage>,
partial_backup_rate_limiter: RateLimiter,
global_rate_limiter: RateLimiter,
) {
tli.set_status(Status::Started);
@@ -220,7 +221,7 @@ pub async fn main_task(
conf,
broker_active_set,
manager_tx,
partial_backup_rate_limiter,
global_rate_limiter,
)
.await;
@@ -254,9 +255,29 @@ pub async fn main_task(
mgr.set_status(Status::UpdatePartialBackup);
mgr.update_partial_backup(&state_snapshot).await;
if mgr.conf.enable_offload && mgr.ready_for_eviction(&next_event, &state_snapshot) {
mgr.set_status(Status::EvictTimeline);
mgr.evict_timeline().await;
let now = Instant::now();
if mgr.evict_not_before > now {
// we should wait until evict_not_before
update_next_event(&mut next_event, mgr.evict_not_before);
}
if mgr.conf.enable_offload
&& mgr.evict_not_before <= now
&& mgr.ready_for_eviction(&next_event, &state_snapshot)
{
// check rate limiter and evict timeline if possible
match mgr.global_rate_limiter.try_acquire_eviction() {
Some(_permit) => {
mgr.set_status(Status::EvictTimeline);
mgr.evict_timeline().await;
}
None => {
// we can't evict timeline now, will try again later
mgr.evict_not_before =
Instant::now() + rand_duration(&mgr.conf.eviction_min_resident);
update_next_event(&mut next_event, mgr.evict_not_before);
}
}
}
}
@@ -334,11 +355,10 @@ impl Manager {
conf: SafeKeeperConf,
broker_active_set: Arc<TimelinesSet>,
manager_tx: tokio::sync::mpsc::UnboundedSender<ManagerCtlMessage>,
partial_backup_rate_limiter: RateLimiter,
global_rate_limiter: RateLimiter,
) -> Manager {
let (is_offloaded, partial_backup_uploaded) = tli.bootstrap_mgr().await;
Manager {
conf,
wal_seg_size: tli.get_wal_seg_size().await,
walsenders: tli.get_walsenders().clone(),
state_version_rx: tli.get_state_version_rx(),
@@ -353,8 +373,10 @@ impl Manager {
partial_backup_uploaded,
access_service: AccessService::new(manager_tx),
tli,
partial_backup_rate_limiter,
resident_since: std::time::Instant::now(),
global_rate_limiter,
// to smooth out evictions spike after restart
evict_not_before: Instant::now() + rand_duration(&conf.eviction_min_resident),
conf,
}
}
@@ -522,8 +544,8 @@ impl Manager {
/// Spawns partial WAL backup task if needed.
async fn update_partial_backup(&mut self, state: &StateSnapshot) {
// check if partial backup is enabled and should be started
if !self.conf.is_wal_backup_enabled() || !self.conf.partial_backup_enabled {
// check if WAL backup is enabled and should be started
if !self.conf.is_wal_backup_enabled() {
return;
}
@@ -541,7 +563,7 @@ impl Manager {
self.partial_backup_task = Some(tokio::spawn(wal_backup_partial::main_task(
self.wal_resident_timeline(),
self.conf.clone(),
self.partial_backup_rate_limiter.clone(),
self.global_rate_limiter.clone(),
)));
}

Some files were not shown because too many files have changed in this diff Show More