## Problem
The regression test on extensions relied on the admin API to set the
default endpoint settings, which is not stable and requires admin
privileges. Specifically:
- The workflow was using `default_endpoint_settings` to configure
necessary PostgreSQL settings like `DateStyle`, `TimeZone`, and
`neon.allow_unstable_extensions`
- This approach was failing because the API endpoint for setting
`default_endpoint_settings` was changed (referenced in a comment as
issue #27108)
- The admin API requires special privileges.
## Summary of changes
We get rid of the admin API dependency and use ALTER DATABASE statements
instead:
**Removed the default_endpoint_settings mechanism:**
- Removed the default_endpoint_settings input parameter from the
neon-project-create action
- Removed the API call that was attempting to set these settings at the
project level
- Completely removed the default_endpoint_settings configuration from
the cloud-extensions workflow
**Added database-level settings:**
- Created a new `alter_db.sh` script that applies the same settings
directly to each test database
- Modified all extension test scripts to call this script after database
creation
## Problem
Hopefully resolves `test_gc_feedback` flakiness.
## Summary of changes
`accumulated_values` should not exceed 512MB to avoid OOM. Previously we
only use number of items, which is not a good estimation.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
Lifetime of imported timelines (and implicitly the import background
task) has some shortcomings:
1. Timeline activation upon import completion is tricky. Previously, a
timeline that finished importing
after a tenant detach would not get activated and there's concerns about
the safety of activating
concurrently with shut-down.
2. Import jobs can prevent tenant shut down since they hold the tenant
gate
## Summary of Changes
Track the import tasks in memory and abort them explicitly on tenant
shutdown.
Integrate more closely with the storage controller:
1. When an import task has finished all of its jobs, it notifies the
storage controller, but **does not** mark the import as done in the
index_part. When all shards have finished importing, the storage
controller will call the `/activate_post_import` idempotent endpoint for
all of them. The handler, marks the import complete in index part,
resets the tenant if required and checks if the timeline is active yet.
2. Not directly related, but the import job now gets the starting state
from the storage controller instead of the import bucket. This paves the
way for progress checkpointing.
Related: https://github.com/neondatabase/neon/issues/11568
## Problem
It's difficult to understand where proxy spends most of cpu and memory.
## Summary of changes
Expose cpu and heap profiling handlers for continuous profiling.
neondatabase/cloud#22670
## Problem
Prefetched and LFC results are not checked in DEBUG_COMPARE_LOCAL mode
## Summary of changes
Add check for this results as well.
---------
Co-authored-by: Konstantin Knizhnik <knizhnik@neon.tech>
Bump all minor versions.
the only conflict was
src/backend/storage/smgr/smgr.c in v17
where our smgr changes conflicted with
ee578921b6
but it was trivial to resolve.
## Problem
1. Safekeeper selection on the pageserver side isn't very dynamic. Once
you connect to one safekeeper, you'll use that one for as long as the
safekeeper keeps the connection alive. In principle, we could be more
eager, since the wal receiver connection can be cancelled but we don't
do that. We wait until the "session" is done and then we pick a new SK.
2. Picking a new SK is quite conservative. We will switch if:
a. We haven't received anything from the SK within the last 10 seconds
(wal_connect_timeout) or
b. The candidate SK is 1GiB ahead or
c. The candidate SK is in the same AZ as the PS or d. There's a
candidate that is ahead and we've not had any WAL within the last 10
seconds (lagging_wal_timeout)
Hence, we can end up with pageservers that are requesting WAL which
their safekeeper hasn't seen yet.
## Summary of changes
Downgrade warning log to info.
## Problem
part of https://github.com/neondatabase/neon/issues/11813
## Summary of changes
Add a lite PostHog client that only uses the local flag evaluation
functionality. Added a test case that parses an example feature flag and
gets the evaluation result.
TODO: support boolean flag, remote config; implement all operators in
PostHog.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
We implemented the retry logic in AWS S3 but not in Azure. Therefore, if
there is an error during Azure listing, we will return an Err to the
caller, and the stream will end without fetching more tenants.
Part of https://github.com/neondatabase/neon/issues/11159
Without this fix, listing tenant will stop once we hit an error (could
be network errors -- that happens more frequent on Azure). If we happen
to stop at a point that we only listed part of the shards, we will hit
the "missed shards" error or even remove layers being used.
This bug (for Azure listing) was introduced as part of
https://github.com/neondatabase/neon/pull/9840
There is also a bug that stops the stream for AWS when there's a timeout
-- this is fixed along with this patch.
## Summary of changes
Retry the request on error. In the future, we should make such streams
return something like `Result<Result<T>>` where the outer result is the
error that ends the stream and the inner one is the error that should be
retried by the caller.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
For `StoreCancelKey`, we were inserting 2 commands, but we were not
inserting two replies. This mismatch leads to errors when decoding the
response.
## Summary of changes
Abstract the command + reply pipeline so that commands and replies are
registered at the same time.
The first line in /etc/ld.so.conf is:
/etc/ld.so.conf.d/*
We want to control library load order so that our compiled binaries are
picked up before others from system packages. The previous solution
allowed the system libraries to load before ours.
Part-of: https://github.com/neondatabase/neon/issues/11857
Signed-off-by: Tristan Partin <tristan@neon.tech>
## Problem
We realised that pg-sni-router doesn't need to be separate from proxy.
just a separate port.
## Summary of changes
Add pg-sni-router config to proxy and expose the service.
## Problem
Further investigation on
https://github.com/neondatabase/neon/issues/11159 reveals that the
list_tenant function can find all the shards of the tenant, but then the
shard gets missing during the gc timeline list blob. One reason could be
that in some ways the timeline gets recognized as a relic timeline.
## Summary of changes
Add logging to help identify the issue.
Signed-off-by: Alex Chi Z <chi@neon.tech>
Make `pull_timeline` check tombstones by default. Otherwise, we'd be
recreating timelines if the order between creation and deletion got
mixed up, as seen in #11838.
Fixes#11838.
This PR adds a runtime validation mode to check adherence to alignment
and size-multiple requirements at the VirtualFile level.
This can help prevent alignment bugs from slipping into production
because test systems may have more lax requirements than production.
(This is not the case today, but it could change in the future).
It also allows catching O_DIRECT bugs on systems that don't have
O_DIRECT (macOS).
Consequently, we can now accept
`virtual_file_io_mode={direct,direct-rw}` on macOS now.
This has the side benefit of removing some annoying conditional
compilation around `IoMode`.
A third benefit is that it helped weed out size-multiple requirement
violation bugs in how the VirtualFile unit tests exercise read and write
APIs.
I seized the opportunity to trim these tests down to what actually
matters, i.e., exercising of the `OpenFiles` file descriptor cache.
Lastly, this PR flips the binary-built-in default to `DirectRw` so that
when running Python regress tests and benchmarks without specifying
`PAGESERVER_VIRTUAL_FILE_IO_MODE`, one gets the production behavior.
Refs
- fixes https://github.com/neondatabase/neon/issues/11676
PR
- github.com/neondatabase/neon/pull/11864
committed yesterday rendered the `PAGESERVER_VIRTUAL_FILE_IO_MODE`
env-var-based parametrization ineffective.
As a consequence, the tests and benchmarks in `test_runner/` were using
the binary built-in-default, i.e., `buffered`.
With the 50ms timeouts of pumping state in connector.c, we need to
correctly handle these timeouts that also wake up pg_usleep.
This new approach makes the connection attempts re-start the wait
whenever it gets woken up early; and CHECK_FOR_INTERRUPTS() is called to
make sure we don't miss query cancellations.
## Problem
https://neondb.slack.com/archives/C04DGM6SMTM/p1746794528680269
## Summary of changes
Make sure we start sleeping again if pg_usleep got woken up ahead of
time.
## Problem
Currently there is a memory leak, in that finished safekeeper
reconciliations leave a cancellation token behind which is never cleaned
up.
## Summary of changes
The change adds cleanup after finishing of a reconciliation. In order to
ensure we remove the correct cancellation token, and we haven't raced
with another reconciliation, we introduce a `TokenId` counter to tell
tokens apart.
Part of https://github.com/neondatabase/neon/issues/11670
## Problem
We observe image compaction errors after gc-compaction finishes
compacting below the gc_cutoff. This is because `repartition` returns an
LSN below the gc horizon as we (likely) determined that `distance <=
self.repartition_threshold`.
I think it's better to keep the current behavior of when to trigger
compaction but we should skip image compaction if the returned LSN is
below the gc horizon.
## Summary of changes
If the repartition returns an invalid LSN, skip image compaction.
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
SK timeline creations were skipped for imported timelines since we
didn't know the correct start LSN
of the timeline at that point.
## Summary of changes
Created imported timelines on the SK as part of the import finalize
step.
We use the last record LSN of shard 0 as the start LSN for the
safekeeper timeline.
Closes https://github.com/neondatabase/neon/issues/11569
## Problem
The limitation we imposed last week
https://github.com/neondatabase/neon/pull/11709 is not enough to protect
excessive memory usage.
## Summary of changes
If a single key accumulated too much history, give up compaction. In the
future, we can make the `generate_key_retention` function take a stream
of keys instead of first accumulating them in memory, thus easily
support such long key history cases.
Signed-off-by: Alex Chi Z <chi@neon.tech>
## Problem
Read replicas cannot grant permissions for roles for Neon RLS. Usually
the permission is already granted, so we can optimistically check. See
INC-509
## Summary of changes
Perform a permission lookup prior to actually executing any grants.
# Problem
Before this PR, timeline shutdown would
- cancel the walreceiver cancellation token subtree (child token of
Timeline::cancel)
- call freeze_and_flush
- Timeline::cancel.cancel()
- ... bunch of waiting for things ...
- Timeline::gate.close()
As noted by the comment that is deleted by this PR, this left a window
where, after freeze_and_flush, walreceiver could still be running and
ingest data into a new InMemoryLayer.
This presents a potential source of log noise during Timeline shutdown
where the InMemoryLayer created after the freeze_and_flush observes
that Timeline::cancel is cancelled, failing the ingest with some
anyhow::Error wrapping (deeply) a `FlushTaskError::Cancelled` instance
(`flush task cancelled` error message).
# Solution
It turns out that it is quite easy to shut down, not just cancel,
walreceiver completely
because the only subtask spawned by walreceiver connection manager is
the `handle_walreceiver_connection` task, which is properly shut down
and waited upon when the manager task observes cancellation and exits
its retry loop.
The alternative is to replace all the usage of `anyhow` on the ingest
path
with differentiated error types. A lot of busywork for little gain to
fix
a potential logging noise nuisance, so, not doing that for now.
# Correctness / Risk
We do not risk leaking walreceiver child tasks because existing
discipline
is to hold a gate guard.
We will prolong `Timeline::shutdown` to the degree that we're no longer
making
progress with the rest of shutdown while the walreceiver task hasn't yet
observed cancellation. In practice, this should be negligible.
`Timeline::shutdown` could fail to complete if there is a hidden
dependency
of walreceiver shutdown on some subsystem. The code certainly suggests
there
isn't, and I'm not aware of any such dependency. Anyway, impact will be
low
because we only shut down Timeline instances that are obsolete, either
because
there is a newer attachment at a different location, or because the
timeline
got deleted by the user. We would learn about this through stuck cplane
operations or stuck storcon reconciliations. We would be able to
mitigate by
cancelling such stuck operations/reconciliations and/or by rolling back
pageserver.
# Refs
- identified this while investigating
https://github.com/neondatabase/neon/issues/11762
- PR that _does_ fix a bunch _real_ `flush task cancelled` noise on the
compaction path: https://github.com/neondatabase/neon/pull/11853
## Problem
We want to see how many users of the legacy serverless driver are still
using the old URL for SQL-over-HTTP traffic.
## Summary of changes
Adds a protocol field to the connections_by_sni metric. Ensures it's
incremented for sql-over-http.
Second PR with fixes extracted from #11712, relating to
`--timelines-onto-safekeepers`. Does the following:
* Moves safekeeper registration to `neon_local` instead of the test
fixtures
* Pass safekeeper JWT token if `--timelines-onto-safekeepers` is enabled
* Allow some warnings related to offline safekeepers (similarly to how
we allow them for offline pageservers)
* Enable generations on the compute's config if
`--timelines-onto-safekeepers` is enabled
* fix parallel `pull_timeline` race condition (the one that #11786 put
for later)
Fixes#11424
Part of #11670
## Problem
At the moment, remote_client and target are recreated in download
function. We could reuse it from SnapshotDownloader instance. This isn't
a problem per se, just a quality of life improvement but it caught my
attention when we were trying out snapshot downloading in one of the
older version and ran into a curious case of s3 clients behaving in two
different manners. One client that used `force_path_style` and other one
didn't.
**Logs from this run:**
```
2025-05-02T12:56:22.384626Z DEBUG /data/snappie/2739e7da34e625e3934ef0b76fa12483/timelines/d44b831adb0a6ba96792dc3a5cc30910/000000000000000000000000000000000000-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF__00000000014E8F20-00000000014E8F99-00000001 requires download...
2025-05-02T12:56:22.384689Z DEBUG invoke{service=s3 operation=ListObjectVersions sdk_invocation_id=7315885}:apply_configuration: timeout settings for this operation: TimeoutConfig { connect_timeout: Set(3.1s), read_timeout: Disabled, operation_timeout: Disabled, operation_attempt_timeout: Disabled }
2025-05-02T12:56:22.384730Z DEBUG invoke{service=s3 operation=ListObjectVersions sdk_invocation_id=7315885}:try_op: entering 'serialization' phase
2025-05-02T12:56:22.384784Z DEBUG invoke{service=s3 operation=ListObjectVersions sdk_invocation_id=7315885}:try_op: entering 'before transmit' phase
2025-05-02T12:56:22.384813Z DEBUG invoke{service=s3 operation=ListObjectVersions sdk_invocation_id=7315885}:try_op: retry strategy has OKed initial request
2025-05-02T12:56:22.384841Z DEBUG invoke{service=s3 operation=ListObjectVersions sdk_invocation_id=7315885}:try_op: beginning attempt #1
2025-05-02T12:56:22.384870Z DEBUG invoke{service=s3 operation=ListObjectVersions sdk_invocation_id=7315885}:try_op:try_attempt: resolving endpoint endpoint_params=EndpointResolverParams(TypeErasedBox[!Clone]:Params { bucket: Some("bucket"), region: Some("eu-north-1"), use_fips: false, use_dual_stack: false, endpoint: Some("https://s3.self-hosted.company.com"), force_path_style: false, accelerate: false, use_global_endpoint: false, use_object_lambda_endpoint: None, key: None, prefix: Some("/pageserver/tenants/2739e7da34e625e3934ef0b76fa12483/timelines/d44b831adb0a6ba96792dc3a5cc30910/000000000000000000000000000000000000-FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF__00000000014E8F20-00000000014E8F99-00000001"), copy_source: None, disable_access_points: None, disable_multi_region_access_points: false, use_arn_region: None, use_s3_express_control_endpoint: None, disable_s3_express_session_auth: None }) endpoint_prefix=None
2025-05-02T12:56:22.384979Z DEBUG invoke{service=s3 operation=ListObjectVersions sdk_invocation_id=7315885}:try_op:try_attempt: will use endpoint Endpoint { url: "https://neon.s3.self-hosted.company.com", headers: {}, properties: {"authSchemes": Array([Object({"signingRegion": String("eu-north-1"), "disableDoubleEncoding": Bool(true), "name": String("sigv4"), "signingName": String("s3")})])} }
2025-05-02T12:56:22.385042Z DEBUG invoke{service=s3 operation=ListObjectVersions sdk_invocation_id=7315885}:try_op:try_attempt:lazy_load_identity:provide_credentials{provider=default_chain}: loaded credentials provider=Environment
2025-05-02T12:56:22.385066Z DEBUG invoke{service=s3 operation=ListObjectVersions sdk_invocation_id=7315885}:try_op:try_attempt:lazy_load_identity: identity cache miss occurred; added new identity (took 35.958µs) new_expiration=2025-05-02T13:11:22.385028Z valid_for=899.999961437s partition=IdentityCachePartition(5)
2025-05-02T12:56:22.385090Z DEBUG invoke{service=s3 operation=ListObjectVersions sdk_invocation_id=7315885}:try_op:try_attempt: loaded identity
2025-05-02T12:56:22.385162Z DEBUG invoke{service=s3 operation=ListObjectVersions sdk_invocation_id=7315885}:try_op:try_attempt: entering 'transmit' phase
2025-05-02T12:56:22.385211Z DEBUG invoke{service=s3 operation=ListObjectVersions sdk_invocation_id=7315885}:try_op:try_attempt: new TCP connector created in 361ns
2025-05-02T12:56:22.385288Z DEBUG resolving host="neon.s3.self-hosted.company.com"
2025-05-02T12:56:22.390796Z DEBUG invoke{service=s3 operation=ListObjectVersions sdk_invocation_id=7315885}:try_op:try_attempt: encountered orchestrator error; halting
```
## Problem
During deployment drains/fills, we often see the storage controller
giving up on warmups after 20 seconds, when the warmup is nearly
complete (~90%). This can cause latency spikes for migrated tenants if
they block on layer downloads.
Touches https://github.com/neondatabase/cloud/issues/26193.
## Summary of changes
Increase the drain and fill secondary warmup timeout from 20 to 30
seconds.
## Problem
Compute may flush WAL on page boundaries, leaving some records partially
flushed for a long time.
It leads to `wait_for_last_flush_lsn` stuck waiting for this partial
LSN.
- Closes: https://github.com/neondatabase/cloud/issues/27876
## Summary of changes
- Flush WAL via CHECKPOINT after requesting current_wal_lsn to make sure
that the record we point to is flushed in full
- Use proper endpoint in
`test_timeline_detach_with_aux_files_with_detach_v1`
## Problem
Import code is one big block. Separating planning and execution will
help with reporting
progress of import to storcon (building block for resuming import).
## Summary of changes
Split up the import into planning and execution.
A concurrency limit driven by PS config is also added.
# Refs
- fixes https://github.com/neondatabase/neon/issues/11762
# Problem
PR #10993 introduced internal retries for BufferedWriter flushes.
PR #11052 added cancellation sensitivity to that retry loop.
That cancellation sensitivity is an error path that didn't exist before.
The result is that during timeline shutdown, after we
`Timeline::cancel`, compaction can now fail with error `flush task
cancelled`.
The problem with that:
1. We mis-classify this as an `error!`-worthy event.
2. This causes tests to become flaky because the error is not in global
`allowed_errors`.
Technically we also trip the `compaction_circuit_breaker` because the
resulting `CompactionError` is variant `::Other`.
But since this is Timeline shutdown, is doesn't matter practically
speaking.
# Solution / Changes
- Log the anyhow stack trace when classifying a compaction error as
`error!`.
This was helpful to identify sources of `flush task cancelled` errors.
We only log at `error!` level in exceptional circumstances, so, it's ok
to have bit verbose logs.
- Introduce typed errors along the `BufferedWriter::write_*`=>
`BlobWriter::write_blob`
=> `{Delta,Image}LayerWriter::put_*` =>
`Split{Delta,Image}LayerWriter::put_{value,image}` chain.
- Proper mapping to `CompactionError`/`CreateImageLayersError` via new
`From` impls.
I am usually opposed to any magic `From` impls, but, it's how most of
the compaction code
works today.
# Testing
The symptoms are most prevalent in
`test_runner/regress/test_branch_and_gc.py::test_branch_and_gc`.
Before this PR, I was able to reproduce locally 1 or 2 times per 400
runs using
`DEFAULT_PG_VERSION=15 BUILD_TYPE=release poetry run pytest --count 400
-n 8`.
After this PR, it doesn't reproduce anymore after 2000 runs.
# Future Work
Technically the ingest path is also exposed to this new source of errors
because `InMemoryLayer` is backed by `BufferedWriter`.
But we haven't seen it occur in flaky tests yet.
Details and a fix in
- https://github.com/neondatabase/neon/pull/11851
# Problem
Before this PR, `test_pageserver_catchup_while_compute_down` would
occasionally fail due to scary-looking WARN log line
```
WARN ephemeral_file_buffered_writer{...}:flush_attempt{attempt=1}: \
error flushing buffered writer buffer to disk, retrying after backoff err=Operation canceled (os error 125)
```
After lengthy investigation, the conclusion is that this is likely due
to a kernel bug related due to io_uring async workers (io-wq) and
signals.
The main indicator is that the error only ever happens in correlation
with pageserver shtudown when SIGTERM is received.
There is a fix that is merged in 6.14
kernels (`io-wq: backoff when retrying worker creation`).
However, even when I revert that patch, the issue is not reproducible
on 6.14, so, it remains a speculation.
It was ruled out that the ECANCELED is due to the executor thread
exiting before the async worker starts processing the operation.
# Solution
The workaround in this issue is to retry the operation on ECANCELED
once.
Retries are safe because the low-level io_engine operations are
idempotent.
(We don't use O_APPEND and I can't think of another flag that would make
the APIs covered by this patch not idempotent.)
# Testing
With this PR, the warn! log no longer happens on [my reproducer
setup](https://github.com/neondatabase/neon/issues/11446#issuecomment-2843015111).
And the new rate-limited `info!`-level log line informing about the
internal retry shows up instead, as expected.
# Refs
- fixes https://github.com/neondatabase/neon/issues/11446
## Problem
`switch_timeline_membership` is implemented on safekeeper's server side,
but the is missing in the client.
- Part of https://github.com/neondatabase/neon/issues/11823
## Summary of changes
- Add `switch_timeline_membership` method to `SafekeeperClient`
Corrects the postgres extension s3 gateway address to
be not just a domain name but a full base URL.
To make the code more readable, the option is renamed
to "remote_ext_base_url", while keeping the old name
also accessible by providing a clap argument alias.
Also provides a very simple and, perhaps, even redundant
unit test to confirm the logic behind parsing of the
corresponding CLI argument.
## Problem
As it is clearly stated in
https://github.com/neondatabase/cloud/issues/26005, using of the short
version of the domain name might work for now, but in the future, we
should get rid of using the `default` namespace and this is where it
will, most likely, break down.
## Summary of changes
The changes adjust the domain name of the extension s3 gateway to use
the proper base url format instead of the just domain name assuming the
"default" namespace and add a new CLI argument name for to reflect the
change and the expectance.
## Problem
Users can override some configuration parameters on the DB level with
`ALTER DATABASE ... SET ...`. Some of these overrides, like `role` or
`default_transaction_read_only`, affect `compute_ctl`'s ability to
configure the DB schema properly.
## Summary of changes
Enforce `role=cloud_admin`, `statement_timeout=0`, and move
`default_transaction_read_only=off` override from control plane [1] to
`compute_ctl`. Also, enforce `search_path=public` just in case, although
we do not call any functions in user databases.
[1]:
133dd8c4db/goapp/controlplane/internal/pkg/compute/provisioner/provisioner_common.go (L70)
Fixes https://github.com/neondatabase/cloud/issues/28532
## Problem
There's a few rough edges around PS tracing.
## Summary of changes
* include compute request id in pageserver trace
* use the get page specific context for GET_REL_SIZE and GET_BATCH
* fix assertion in download layer trace

## Problem
We use `head_object` to determine whether an object exists or not.
However, it does not always error due to a missing object.
## Summary of changes
Log the error so that we can have a better idea what's going on with the
scrubber errors in prod.
---------
Signed-off-by: Alex Chi Z <chi@neon.tech>
According to RFC 7519, `aud` is generally an array of StringOrURI, but
in special cases may be a single StringOrURI value. To accomodate future
control plane work where a single token may work for multiple services,
make the claim a vector.
Link: https://www.rfc-editor.org/rfc/rfc7519#section-4.1.3
Signed-off-by: Tristan Partin <tristan@neon.tech>
Add `/lfc/(prewarm|offload)` routes to `compute_ctl` which interact with
endpoint storage.
Add `prewarm_lfc_on_startup` spec option which, if enabled, downloads
LFC prewarm data on compute startup.
Resolves: https://github.com/neondatabase/cloud/issues/26343
## Problem
Currently the setup for `anon` v2 in the compute image downloads the
latest version of the extension. This can be problematic as on a compute
start/restart it can download a version that is newer than what we have
tested and potentially break things, hence not giving us the ability to
control when the extension is updated.
We were also using `v2.2.0`, which is not ready for production yet and
has been clarified by the maintainer.
Additional context:
https://gitlab.com/dalibo/postgresql_anonymizer/-/issues/530
## Summary of changes
Changed the URL from which we download the `anon` extension to point to
`v2.1.0` instead of `latest`.
Currently we only have an admin scope which allows a user to bypass the
compute_id check. When the admin scope is provided, validate the
audience of the JWT to be "compute".
Closes: https://github.com/neondatabase/cloud/issues/27614
Signed-off-by: Tristan Partin <tristan@neon.tech>