WIP: Garage v2 #948
113 changed files with 14595 additions and 4879 deletions
|
@ -35,7 +35,15 @@ steps:
|
|||
- matrix:
|
||||
ARCH: i386
|
||||
|
||||
- name: upgrade tests
|
||||
- name: upgrade tests from v1.0.0
|
||||
image: nixpkgs/nix:nixos-22.05
|
||||
commands:
|
||||
- nix-shell --attr ci --run "./script/test-upgrade.sh v1.0.0 x86_64-unknown-linux-musl" || (cat /tmp/garage.log; false)
|
||||
when:
|
||||
- matrix:
|
||||
ARCH: amd64
|
||||
|
||||
- name: upgrade tests from v0.8.4
|
||||
image: nixpkgs/nix:nixos-22.05
|
||||
commands:
|
||||
- nix-shell --attr ci --run "./script/test-upgrade.sh v0.8.4 x86_64-unknown-linux-musl" || (cat /tmp/garage.log; false)
|
||||
|
|
37
Cargo.lock
generated
37
Cargo.lock
generated
|
@ -1240,7 +1240,6 @@ dependencies = [
|
|||
"opentelemetry-otlp",
|
||||
"opentelemetry-prometheus",
|
||||
"parse_duration",
|
||||
"serde",
|
||||
"serde_json",
|
||||
"sha1",
|
||||
"sha2",
|
||||
|
@ -1251,6 +1250,7 @@ dependencies = [
|
|||
"tokio",
|
||||
"tracing",
|
||||
"tracing-subscriber",
|
||||
"utoipa",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
|
@ -1259,9 +1259,13 @@ version = "1.1.0"
|
|||
dependencies = [
|
||||
"argon2",
|
||||
"async-trait",
|
||||
"bytesize",
|
||||
"chrono",
|
||||
"err-derive",
|
||||
"format_table",
|
||||
"futures",
|
||||
"garage_api_common",
|
||||
"garage_block",
|
||||
"garage_model",
|
||||
"garage_rpc",
|
||||
"garage_table",
|
||||
|
@ -1271,12 +1275,14 @@ dependencies = [
|
|||
"hyper 1.6.0",
|
||||
"opentelemetry",
|
||||
"opentelemetry-prometheus",
|
||||
"paste",
|
||||
"prometheus",
|
||||
"serde",
|
||||
"serde_json",
|
||||
"tokio",
|
||||
"tracing",
|
||||
"url",
|
||||
"utoipa",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
|
@ -1288,6 +1294,7 @@ dependencies = [
|
|||
"chrono",
|
||||
"crc32c",
|
||||
"crc32fast",
|
||||
"crc64fast-nvme",
|
||||
"crypto-common",
|
||||
"err-derive",
|
||||
"futures",
|
||||
|
@ -1348,6 +1355,7 @@ dependencies = [
|
|||
"chrono",
|
||||
"crc32c",
|
||||
"crc32fast",
|
||||
"crc64fast-nvme",
|
||||
"err-derive",
|
||||
"form_urlencoded",
|
||||
"futures",
|
||||
|
@ -1359,6 +1367,7 @@ dependencies = [
|
|||
"garage_table",
|
||||
"garage_util",
|
||||
"hex",
|
||||
"hmac",
|
||||
"http 1.3.1",
|
||||
"http-body-util",
|
||||
"http-range",
|
||||
|
@ -1395,7 +1404,6 @@ dependencies = [
|
|||
"garage_db",
|
||||
"garage_net",
|
||||
"garage_rpc",
|
||||
"garage_table",
|
||||
"garage_util",
|
||||
"hex",
|
||||
"opentelemetry",
|
||||
|
@ -1424,6 +1432,7 @@ dependencies = [
|
|||
name = "garage_model"
|
||||
version = "1.1.0"
|
||||
dependencies = [
|
||||
"argon2",
|
||||
"async-trait",
|
||||
"base64 0.21.7",
|
||||
"blake2",
|
||||
|
@ -2219,6 +2228,7 @@ checksum = "3954d50fe15b02142bf25d3b8bdadb634ec3948f103d04ffe3031bc8fe9d7058"
|
|||
dependencies = [
|
||||
"equivalent",
|
||||
"hashbrown 0.15.2",
|
||||
"serde",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
|
@ -4677,6 +4687,29 @@ version = "0.2.2"
|
|||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821"
|
||||
|
||||
[[package]]
|
||||
name = "utoipa"
|
||||
version = "5.3.1"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "435c6f69ef38c9017b4b4eea965dfb91e71e53d869e896db40d1cf2441dd75c0"
|
||||
dependencies = [
|
||||
"indexmap 2.8.0",
|
||||
"serde",
|
||||
"serde_json",
|
||||
"utoipa-gen",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "utoipa-gen"
|
||||
version = "5.3.1"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "a77d306bc75294fd52f3e99b13ece67c02c1a2789190a6f31d32f736624326f7"
|
||||
dependencies = [
|
||||
"proc-macro2",
|
||||
"quote",
|
||||
"syn 2.0.100",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "uuid"
|
||||
version = "1.4.1"
|
||||
|
|
|
@ -48,9 +48,10 @@ blake2 = "0.10"
|
|||
bytes = "1.0"
|
||||
bytesize = "1.1"
|
||||
cfg-if = "1.0"
|
||||
chrono = "0.4"
|
||||
chrono = { version = "0.4", features = ["serde"] }
|
||||
crc32fast = "1.4"
|
||||
crc32c = "0.6"
|
||||
crc64fast-nvme = "1.2"
|
||||
crypto-common = "0.1"
|
||||
err-derive = "0.3"
|
||||
gethostname = "0.4"
|
||||
|
@ -67,6 +68,7 @@ mktemp = "0.5"
|
|||
nix = { version = "0.29", default-features = false, features = ["fs"] }
|
||||
nom = "7.1"
|
||||
parse_duration = "2.1"
|
||||
paste = "1.0"
|
||||
pin-project = "1.0.12"
|
||||
pnet_datalink = "0.34"
|
||||
rand = "0.8"
|
||||
|
@ -100,6 +102,7 @@ serde = { version = "1.0", default-features = false, features = ["derive", "rc"]
|
|||
serde_bytes = "0.11"
|
||||
serde_json = "1.0"
|
||||
toml = { version = "0.8", default-features = false, features = ["parse"] }
|
||||
utoipa = { version = "5.3.1", features = ["chrono"] }
|
||||
|
||||
# newer version requires rust edition 2021
|
||||
k8s-openapi = { version = "0.21", features = ["v1_24"] }
|
||||
|
|
|
@ -1,7 +1,7 @@
|
|||
<!DOCTYPE html>
|
||||
<html>
|
||||
<head>
|
||||
<title>Garage Adminstration API v0</title>
|
||||
<title>Garage adminstration API v0</title>
|
||||
|
||||
<!-- needed for adaptive design -->
|
||||
<meta charset="utf-8"/>
|
||||
<meta name="viewport" content="width=device-width, initial-scale=1">
|
||||
|
|
|
@ -1,7 +1,7 @@
|
|||
<!DOCTYPE html>
|
||||
<html>
|
||||
<head>
|
||||
<title>Garage Adminstration API v0</title>
|
||||
<title>Garage adminstration API v1</title>
|
||||
arnaudlevy
commented
administration administration
|
||||
<!-- needed for adaptive design -->
|
||||
<meta charset="utf-8"/>
|
||||
<meta name="viewport" content="width=device-width, initial-scale=1">
|
||||
|
|
24
doc/api/garage-admin-v2.html
Normal file
24
doc/api/garage-admin-v2.html
Normal file
|
@ -0,0 +1,24 @@
|
|||
<!DOCTYPE html>
|
||||
<html>
|
||||
<head>
|
||||
<title>Garage adminstration API v2</title>
|
||||
<!-- needed for adaptive design -->
|
||||
<meta charset="utf-8"/>
|
||||
<meta name="viewport" content="width=device-width, initial-scale=1">
|
||||
<link href="./css/redoc.css" rel="stylesheet">
|
||||
|
||||
<!--
|
||||
Redoc doesn't change outer page styles
|
||||
-->
|
||||
<style>
|
||||
body {
|
||||
margin: 0;
|
||||
padding: 0;
|
||||
}
|
||||
</style>
|
||||
</head>
|
||||
<body>
|
||||
<redoc spec-url='./garage-admin-v2.json'></redoc>
|
||||
<script src="./redoc.standalone.js"> </script>
|
||||
</body>
|
||||
</html>
|
4273
doc/api/garage-admin-v2.json
Normal file
4273
doc/api/garage-admin-v2.json
Normal file
File diff suppressed because it is too large
Load diff
|
@ -80,6 +80,7 @@ add_host_to_metrics = true
|
|||
[admin]
|
||||
api_bind_addr = "0.0.0.0:3903"
|
||||
metrics_token = "BCAdFjoa9G0KJR0WXnHHm7fs1ZAbfpI8iIZ+Z/a2NgI="
|
||||
metrics_require_token = true
|
||||
admin_token = "UkLeGWEvHnXBqnueR3ISEMWpOnm40jH2tM2HnnL/0F4="
|
||||
trace_sink = "http://localhost:4317"
|
||||
```
|
||||
|
@ -145,6 +146,7 @@ The `[s3_web]` section:
|
|||
|
||||
The `[admin]` section:
|
||||
[`api_bind_addr`](#admin_api_bind_addr),
|
||||
[`metrics_require_token`](#admin_metrics_require_token),
|
||||
[`metrics_token`/`metrics_token_file`](#admin_metrics_token),
|
||||
[`admin_token`/`admin_token_file`](#admin_token),
|
||||
[`trace_sink`](#admin_trace_sink),
|
||||
|
@ -767,10 +769,34 @@ See [administration API reference](@/documentation/reference-manual/admin-api.md
|
|||
Alternatively, since `v0.8.5`, a path can be used to create a unix socket. Note that for security reasons,
|
||||
the socket will have 0220 mode. Make sure to set user and group permissions accordingly.
|
||||
|
||||
#### `admin_token`, `admin_token_file` or `GARAGE_ADMIN_TOKEN`, `GARAGE_ADMIN_TOKEN_FILE` (env) {#admin_token}
|
||||
|
||||
The token for accessing all administration functions on the admin endpoint,
|
||||
with the exception of the metrics endpoint (see `metrics_token`).
|
||||
|
||||
You can use any random string for this value. We recommend generating a random
|
||||
token with `openssl rand -base64 32`.
|
||||
|
||||
For Garage version earlier than `v2.0`, if this token is not set,
|
||||
access to these endpoints is disabled entirely.
|
||||
|
||||
Since Garage `v2.0`, additional admin API tokens can be defined dynamically
|
||||
in your Garage cluster using administration commands. This new admin token system
|
||||
is more flexible since it allows admin tokens to have an expiration date,
|
||||
and to have a scope restricted to certain admin API functions. If `admin_token`
|
||||
is set, it behaves as an admin token without expiration and with full scope.
|
||||
Otherwise, only admin API tokens defined dynamically can be used.
|
||||
|
||||
`admin_token` was introduced in Garage `v0.7.2`.
|
||||
`admin_token_file` and the `GARAGE_ADMIN_TOKEN` environment variable are supported since Garage `v0.8.2`.
|
||||
|
||||
`GARAGE_ADMIN_TOKEN_FILE` is supported since `v0.8.5` / `v0.9.1`.
|
||||
|
||||
#### `metrics_token`, `metrics_token_file` or `GARAGE_METRICS_TOKEN`, `GARAGE_METRICS_TOKEN_FILE` (env) {#admin_metrics_token}
|
||||
|
||||
The token for accessing the Metrics endpoint. If this token is not set, the
|
||||
Metrics endpoint can be accessed without access control.
|
||||
The token for accessing the Prometheus metrics endpoint (`/metrics`).
|
||||
If this token is not set, and unless `metrics_require_token` is set to `true`,
|
||||
the metrics endpoint can be accessed without access control.
|
||||
|
||||
You can use any random string for this value. We recommend generating a random token with `openssl rand -base64 32`.
|
||||
|
||||
|
@ -779,17 +805,12 @@ You can use any random string for this value. We recommend generating a random t
|
|||
|
||||
`GARAGE_METRICS_TOKEN_FILE` is supported since `v0.8.5` / `v0.9.1`.
|
||||
|
||||
#### `admin_token`, `admin_token_file` or `GARAGE_ADMIN_TOKEN`, `GARAGE_ADMIN_TOKEN_FILE` (env) {#admin_token}
|
||||
#### `metrics_require_token` (since `v2.0.0`) {#admin_metrics_require_token}
|
||||
|
||||
The token for accessing all of the other administration endpoints. If this
|
||||
token is not set, access to these endpoints is disabled entirely.
|
||||
|
||||
You can use any random string for this value. We recommend generating a random token with `openssl rand -base64 32`.
|
||||
|
||||
`admin_token` was introduced in Garage `v0.7.2`.
|
||||
`admin_token_file` and the `GARAGE_ADMIN_TOKEN` environment variable are supported since Garage `v0.8.2`.
|
||||
|
||||
`GARAGE_ADMIN_TOKEN_FILE` is supported since `v0.8.5` / `v0.9.1`.
|
||||
If this is set to `true`, accessing the metrics endpoint will always require
|
||||
an access token. Valid tokens include the `metrics_token` if it is set,
|
||||
and admin API token defined dynamicaly in Garage which have
|
||||
the `Metrics` endpoint in their scope.
|
||||
|
||||
#### `trace_sink` {#admin_trace_sink}
|
||||
|
||||
|
|
|
@ -13,8 +13,9 @@ We will bump the version numbers prefixed to each API endpoint each time the syn
|
|||
or semantics change, meaning that code that relies on these endpoints will break
|
||||
when changes are introduced.
|
||||
|
||||
The Garage administration API was introduced in version 0.7.2, this document
|
||||
does not apply to older versions of Garage.
|
||||
The Garage administration API was introduced in version 0.7.2, and was
|
||||
changed several times.
|
||||
This document applies only to the Garage v2 API (starting with Garage v2.0.0).
|
||||
|
||||
|
||||
## Access control
|
||||
|
@ -52,34 +53,28 @@ Returns an HTTP status 200 if the node is ready to answer user's requests,
|
|||
and an HTTP status 503 (Service Unavailable) if there are some partitions
|
||||
for which a quorum of nodes is not available.
|
||||
A simple textual message is also returned in a body with content-type `text/plain`.
|
||||
See `/v1/health` for an API that also returns JSON output.
|
||||
See `/v2/GetClusterHealth` for an API that also returns JSON output.
|
||||
|
||||
### Other special endpoints
|
||||
|
||||
#### CheckDomain `GET /check?domain=<domain>`
|
||||
|
||||
Checks whether this Garage cluster serves a website for domain `<domain>`.
|
||||
Returns HTTP 200 Ok if yes, or HTTP 4xx if no website is available for this domain.
|
||||
|
||||
### Cluster operations
|
||||
|
||||
#### GetClusterStatus `GET /v1/status`
|
||||
#### GetClusterStatus `GET /v2/GetClusterStatus`
|
||||
|
||||
Returns the cluster's current status in JSON, including:
|
||||
|
||||
- ID of the node being queried and its version of the Garage daemon
|
||||
- Live nodes
|
||||
- Currently configured cluster layout
|
||||
- Staged changes to the cluster layout
|
||||
|
||||
Example response body:
|
||||
|
||||
```json
|
||||
{
|
||||
"node": "b10c110e4e854e5aa3f4637681befac755154b20059ec163254ddbfae86b09df",
|
||||
"garageVersion": "v1.1.0",
|
||||
"garageFeatures": [
|
||||
"k2v",
|
||||
"lmdb",
|
||||
"sqlite",
|
||||
"metrics",
|
||||
"bundled-libs"
|
||||
],
|
||||
"rustVersion": "1.68.0",
|
||||
"dbEngine": "LMDB (using Heed crate)",
|
||||
"layoutVersion": 5,
|
||||
"nodes": [
|
||||
{
|
||||
|
@ -169,7 +164,7 @@ Example response body:
|
|||
}
|
||||
```
|
||||
|
||||
#### GetClusterHealth `GET /v1/health`
|
||||
#### GetClusterHealth `GET /v2/GetClusterHealth`
|
||||
|
||||
Returns the cluster's current health in JSON format, with the following variables:
|
||||
|
||||
|
@ -202,7 +197,7 @@ Example response body:
|
|||
}
|
||||
```
|
||||
|
||||
#### ConnectClusterNodes `POST /v1/connect`
|
||||
#### ConnectClusterNodes `POST /v2/ConnectClusterNodes`
|
||||
|
||||
Instructs this Garage node to connect to other Garage nodes at specified addresses.
|
||||
|
||||
|
@ -232,7 +227,7 @@ Example response:
|
|||
]
|
||||
```
|
||||
|
||||
#### GetClusterLayout `GET /v1/layout`
|
||||
#### GetClusterLayout `GET /v2/GetClusterLayout`
|
||||
|
||||
Returns the cluster's current layout in JSON, including:
|
||||
|
||||
|
@ -293,7 +288,7 @@ Example response body:
|
|||
}
|
||||
```
|
||||
|
||||
#### UpdateClusterLayout `POST /v1/layout`
|
||||
#### UpdateClusterLayout `POST /v2/UpdateClusterLayout`
|
||||
|
||||
Send modifications to the cluster layout. These modifications will
|
||||
be included in the staged role changes, visible in subsequent calls
|
||||
|
@ -330,7 +325,7 @@ This returns the new cluster layout with the proposed staged changes,
|
|||
as returned by GetClusterLayout.
|
||||
|
||||
|
||||
#### ApplyClusterLayout `POST /v1/layout/apply`
|
||||
#### ApplyClusterLayout `POST /v2/ApplyClusterLayout`
|
||||
|
||||
Applies to the cluster the layout changes currently registered as
|
||||
staged layout changes.
|
||||
|
@ -350,23 +345,11 @@ existing layout in the cluster.
|
|||
This returns the message describing all the calculations done to compute the new
|
||||
layout, as well as the description of the layout as returned by GetClusterLayout.
|
||||
|
||||
#### RevertClusterLayout `POST /v1/layout/revert`
|
||||
#### RevertClusterLayout `POST /v2/RevertClusterLayout`
|
||||
|
||||
Clears all of the staged layout changes.
|
||||
|
||||
Request body format:
|
||||
|
||||
```json
|
||||
{
|
||||
"version": 13
|
||||
}
|
||||
```
|
||||
|
||||
Reverting the staged changes is done by incrementing the version number
|
||||
and clearing the contents of the staged change list.
|
||||
Similarly to the CLI, the body must include the incremented
|
||||
version number, which MUST be 1 + the value of the currently
|
||||
existing layout in the cluster.
|
||||
This requests contains an empty body.
|
||||
|
||||
This returns the new cluster layout with all changes reverted,
|
||||
as returned by GetClusterLayout.
|
||||
|
@ -374,7 +357,7 @@ as returned by GetClusterLayout.
|
|||
|
||||
### Access key operations
|
||||
|
||||
#### ListKeys `GET /v1/key`
|
||||
#### ListKeys `GET /v2/ListKeys`
|
||||
|
||||
Returns all API access keys in the cluster.
|
||||
|
||||
|
@ -393,8 +376,8 @@ Example response:
|
|||
]
|
||||
```
|
||||
|
||||
#### GetKeyInfo `GET /v1/key?id=<acces key id>`
|
||||
#### GetKeyInfo `GET /v1/key?search=<pattern>`
|
||||
#### GetKeyInfo `GET /v2/GetKeyInfo?id=<acces key id>`
|
||||
#### GetKeyInfo `GET /v2/GetKeyInfo?search=<pattern>`
|
||||
|
||||
Returns information about the requested API access key.
|
||||
|
||||
|
@ -468,7 +451,7 @@ Example response:
|
|||
}
|
||||
```
|
||||
|
||||
#### CreateKey `POST /v1/key`
|
||||
#### CreateKey `POST /v2/CreateKey`
|
||||
|
||||
Creates a new API access key.
|
||||
|
||||
|
@ -483,7 +466,7 @@ Request body format:
|
|||
This returns the key info, including the created secret key,
|
||||
in the same format as the result of GetKeyInfo.
|
||||
|
||||
#### ImportKey `POST /v1/key/import`
|
||||
#### ImportKey `POST /v2/ImportKey`
|
||||
|
||||
Imports an existing API key.
|
||||
This will check that the imported key is in the valid format, i.e.
|
||||
|
@ -501,7 +484,7 @@ Request body format:
|
|||
|
||||
This returns the key info in the same format as the result of GetKeyInfo.
|
||||
|
||||
#### UpdateKey `POST /v1/key?id=<acces key id>`
|
||||
#### UpdateKey `POST /v2/UpdateKey?id=<acces key id>`
|
||||
|
||||
Updates information about the specified API access key.
|
||||
|
||||
|
@ -523,14 +506,14 @@ The possible flags in `allow` and `deny` are: `createBucket`.
|
|||
|
||||
This returns the key info in the same format as the result of GetKeyInfo.
|
||||
|
||||
#### DeleteKey `DELETE /v1/key?id=<acces key id>`
|
||||
#### DeleteKey `POST /v2/DeleteKey?id=<acces key id>`
|
||||
|
||||
Deletes an API access key.
|
||||
|
||||
|
||||
### Bucket operations
|
||||
|
||||
#### ListBuckets `GET /v1/bucket`
|
||||
#### ListBuckets `GET /v2/ListBuckets`
|
||||
|
||||
Returns all storage buckets in the cluster.
|
||||
|
||||
|
@ -572,8 +555,8 @@ Example response:
|
|||
]
|
||||
```
|
||||
|
||||
#### GetBucketInfo `GET /v1/bucket?id=<bucket id>`
|
||||
#### GetBucketInfo `GET /v1/bucket?globalAlias=<alias>`
|
||||
#### GetBucketInfo `GET /v2/GetBucketInfo?id=<bucket id>`
|
||||
#### GetBucketInfo `GET /v2/GetBucketInfo?globalAlias=<alias>`
|
||||
|
||||
Returns information about the requested storage bucket.
|
||||
|
||||
|
@ -616,7 +599,7 @@ Example response:
|
|||
}
|
||||
```
|
||||
|
||||
#### CreateBucket `POST /v1/bucket`
|
||||
#### CreateBucket `POST /v2/CreateBucket`
|
||||
|
||||
Creates a new storage bucket.
|
||||
|
||||
|
@ -656,7 +639,7 @@ or no alias at all.
|
|||
Technically, you can also specify both `globalAlias` and `localAlias` and that would create
|
||||
two aliases, but I don't see why you would want to do that.
|
||||
|
||||
#### UpdateBucket `PUT /v1/bucket?id=<bucket id>`
|
||||
#### UpdateBucket `POST /v2/UpdateBucket?id=<bucket id>`
|
||||
|
||||
Updates configuration of the given bucket.
|
||||
|
||||
|
@ -688,16 +671,38 @@ In `quotas`: new values of `maxSize` and `maxObjects` must both be specified, or
|
|||
to remove the quotas. An absent value will be considered the same as a `null`. It is not possible
|
||||
to change only one of the two quotas.
|
||||
|
||||
#### DeleteBucket `DELETE /v1/bucket?id=<bucket id>`
|
||||
#### DeleteBucket `POST /v2/DeleteBucket?id=<bucket id>`
|
||||
|
||||
Deletes a storage bucket. A bucket cannot be deleted if it is not empty.
|
||||
|
||||
Warning: this will delete all aliases associated with the bucket!
|
||||
|
||||
#### CleanupIncompleteUploads `POST /v2/CleanupIncompleteUploads`
|
||||
|
||||
Cleanup all incomplete uploads in a bucket that are older than a specified number
|
||||
of seconds.
|
||||
|
||||
Request body format:
|
||||
|
||||
```json
|
||||
{
|
||||
"bucketId": "e6a14cd6a27f48684579ec6b381c078ab11697e6bc8513b72b2f5307e25fff9b",
|
||||
"olderThanSecs": 3600
|
||||
}
|
||||
```
|
||||
|
||||
Response format
|
||||
|
||||
```json
|
||||
{
|
||||
"uploadsDeleted": 12
|
||||
}
|
||||
```
|
||||
|
||||
|
||||
### Operations on permissions for keys on buckets
|
||||
|
||||
#### BucketAllowKey `POST /v1/bucket/allow`
|
||||
#### AllowBucketKey `POST /v2/AllowBucketKey`
|
||||
|
||||
Allows a key to do read/write/owner operations on a bucket.
|
||||
|
||||
|
@ -718,7 +723,7 @@ Request body format:
|
|||
Flags in `permissions` which have the value `true` will be activated.
|
||||
Other flags will remain unchanged.
|
||||
|
||||
#### BucketDenyKey `POST /v1/bucket/deny`
|
||||
#### DenyBucketKey `POST /v2/DenyBucketKey`
|
||||
|
||||
Denies a key from doing read/write/owner operations on a bucket.
|
||||
|
||||
|
@ -742,19 +747,35 @@ Other flags will remain unchanged.
|
|||
|
||||
### Operations on bucket aliases
|
||||
|
||||
#### GlobalAliasBucket `PUT /v1/bucket/alias/global?id=<bucket id>&alias=<global alias>`
|
||||
#### AddBucketAlias `POST /v2/AddBucketAlias`
|
||||
|
||||
Empty body. Creates a global alias for a bucket.
|
||||
Creates an alias for a bucket in the namespace of a specific access key.
|
||||
To create a global alias, specify the `globalAlias` field.
|
||||
To create a local alias, specify the `localAlias` and `accessKeyId` fields.
|
||||
|
||||
#### GlobalUnaliasBucket `DELETE /v1/bucket/alias/global?id=<bucket id>&alias=<global alias>`
|
||||
Request body format:
|
||||
|
||||
Removes a global alias for a bucket.
|
||||
```json
|
||||
{
|
||||
"bucketId": "e6a14cd6a27f48684579ec6b381c078ab11697e6bc8513b72b2f5307e25fff9b",
|
||||
"globalAlias": "my-bucket"
|
||||
}
|
||||
```
|
||||
|
||||
#### LocalAliasBucket `PUT /v1/bucket/alias/local?id=<bucket id>&accessKeyId=<access key ID>&alias=<local alias>`
|
||||
or:
|
||||
|
||||
Empty body. Creates a local alias for a bucket in the namespace of a specific access key.
|
||||
```json
|
||||
{
|
||||
"bucketId": "e6a14cd6a27f48684579ec6b381c078ab11697e6bc8513b72b2f5307e25fff9b",
|
||||
"accessKeyId": "GK31c2f218a2e44f485b94239e",
|
||||
"localAlias": "my-bucket"
|
||||
}
|
||||
```
|
||||
|
||||
#### LocalUnaliasBucket `DELETE /v1/bucket/alias/local?id=<bucket id>&accessKeyId<access key ID>&alias=<local alias>`
|
||||
#### RemoveBucketAlias `POST /v2/RemoveBucketAlias`
|
||||
|
||||
Removes a local alias for a bucket in the namespace of a specific access key.
|
||||
Removes an alias for a bucket in the namespace of a specific access key.
|
||||
To remove a global alias, specify the `globalAlias` field.
|
||||
To remove a local alias, specify the `localAlias` and `accessKeyId` fields.
|
||||
|
||||
Request body format: same as AddBucketAlias.
|
||||
|
|
|
@ -17,13 +17,19 @@ else
|
|||
fi
|
||||
|
||||
$GARAGE_BIN -c /tmp/config.1.toml bucket create eprouvette
|
||||
if [ "$GARAGE_08" = "1" ]; then
|
||||
if [ "$GARAGE_OLDVER" = "v08" ]; then
|
||||
KEY_INFO=$($GARAGE_BIN -c /tmp/config.1.toml key new --name opérateur)
|
||||
else
|
||||
ACCESS_KEY=`echo $KEY_INFO|grep -Po 'GK[a-f0-9]+'`
|
||||
SECRET_KEY=`echo $KEY_INFO|grep -Po 'Secret key: [a-f0-9]+'|grep -Po '[a-f0-9]+$'`
|
||||
elif [ "$GARAGE_OLDVER" = "v1" ]; then
|
||||
KEY_INFO=$($GARAGE_BIN -c /tmp/config.1.toml key create opérateur)
|
||||
ACCESS_KEY=`echo $KEY_INFO|grep -Po 'GK[a-f0-9]+'`
|
||||
SECRET_KEY=`echo $KEY_INFO|grep -Po 'Secret key: [a-f0-9]+'|grep -Po '[a-f0-9]+$'`
|
||||
else
|
||||
KEY_INFO=$($GARAGE_BIN -c /tmp/config.1.toml json-api CreateKey '{"name":"opérateur"}')
|
||||
ACCESS_KEY=`echo $KEY_INFO|jq -r .accessKeyId`
|
||||
SECRET_KEY=`echo $KEY_INFO|jq -r .secretAccessKey`
|
||||
fi
|
||||
ACCESS_KEY=`echo $KEY_INFO|grep -Po 'GK[a-f0-9]+'`
|
||||
SECRET_KEY=`echo $KEY_INFO|grep -Po 'Secret key: [a-f0-9]+'|grep -Po '[a-f0-9]+$'`
|
||||
$GARAGE_BIN -c /tmp/config.1.toml bucket allow eprouvette --read --write --owner --key $ACCESS_KEY
|
||||
echo "$ACCESS_KEY $SECRET_KEY" > /tmp/garage.s3
|
||||
|
||||
|
|
|
@ -29,7 +29,7 @@ until $GARAGE_BIN -c /tmp/config.1.toml status 2>&1|grep -q HEALTHY ; do
|
|||
sleep 1
|
||||
done
|
||||
|
||||
if [ "$GARAGE_08" = "1" ]; then
|
||||
if [ "$GARAGE_OLDVER" = "v08" ]; then
|
||||
$GARAGE_BIN -c /tmp/config.1.toml status \
|
||||
| grep 'NO ROLE' \
|
||||
| grep -Po '^[0-9a-f]+' \
|
||||
|
|
|
@ -24,9 +24,17 @@ echo "============= insert data into old version cluster ================="
|
|||
export GARAGE_BIN=/tmp/old_garage
|
||||
if echo $OLD_VERSION | grep 'v0\.8\.'; then
|
||||
echo "Detected Garage v0.8.x"
|
||||
export GARAGE_08=1
|
||||
export GARAGE_OLDVER=v08
|
||||
elif (echo $OLD_VERSION | grep 'v0\.9\.') || (echo $OLD_VERSION | grep 'v1\.'); then
|
||||
echo "Detected Garage v0.9.x / v1.x"
|
||||
export GARAGE_OLDVER=v1
|
||||
fi
|
||||
|
||||
if echo $OLD_VERSION | grep 'v1\.'; then
|
||||
DO_SSEC_TEST=1
|
||||
fi
|
||||
SSEC_KEY="u8zCfnEyt5Imo/krN+sxA1DQXxLWtPJavU6T6gOVj1Y="
|
||||
|
||||
echo "⏳ Setup cluster using old version"
|
||||
$GARAGE_BIN --version
|
||||
${SCRIPT_FOLDER}/dev-clean.sh
|
||||
|
@ -37,7 +45,23 @@ ${SCRIPT_FOLDER}/dev-bucket.sh
|
|||
|
||||
echo "🛠️ Inserting data in old cluster"
|
||||
source ${SCRIPT_FOLDER}/dev-env-rclone.sh
|
||||
rclone copy "${SCRIPT_FOLDER}/../.git/" garage:eprouvette/test_dotgit --stats=1s --stats-log-level=NOTICE --stats-one-line
|
||||
rclone copy "${SCRIPT_FOLDER}/../.git/" garage:eprouvette/test_dotgit \
|
||||
--stats=1s --stats-log-level=NOTICE --stats-one-line
|
||||
|
||||
if [ "$DO_SSEC_TEST" = "1" ]; then
|
||||
# upload small file (should be single part)
|
||||
rclone copy "${SCRIPT_FOLDER}/test-upgrade.sh" garage:eprouvette/test-ssec \
|
||||
--s3-sse-customer-algorithm AES256 \
|
||||
--s3-sse-customer-key-base64 "$SSEC_KEY" \
|
||||
--stats=1s --stats-log-level=NOTICE --stats-one-line
|
||||
# do a multipart upload
|
||||
dd if=/dev/urandom of=/tmp/randfile-for-upgrade bs=5M count=5
|
||||
rclone copy "/tmp/randfile-for-upgrade" garage:eprouvette/test-ssec \
|
||||
--s3-chunk-size 5M \
|
||||
--s3-sse-customer-algorithm AES256 \
|
||||
--s3-sse-customer-key-base64 "$SSEC_KEY" \
|
||||
--stats=1s --stats-log-level=NOTICE --stats-one-line
|
||||
fi
|
||||
|
||||
echo "🏁 Stopping old cluster"
|
||||
killall -INT old_garage
|
||||
|
@ -47,7 +71,7 @@ killall -9 old_garage || true
|
|||
echo "🏁 Removing old garage version"
|
||||
rm -rv $GARAGE_BIN
|
||||
export -n GARAGE_BIN
|
||||
export -n GARAGE_08
|
||||
export -n GARAGE_OLDVER
|
||||
|
||||
echo "================ read data from new cluster ==================="
|
||||
|
||||
|
@ -60,7 +84,8 @@ ${SCRIPT_FOLDER}/dev-cluster.sh >> /tmp/garage.log 2>&1 &
|
|||
sleep 3
|
||||
|
||||
echo "🛠️ Retrieving data from old cluster"
|
||||
rclone copy garage:eprouvette/test_dotgit /tmp/test_dotgit --stats=1s --stats-log-level=NOTICE --stats-one-line --fast-list
|
||||
rclone copy garage:eprouvette/test_dotgit /tmp/test_dotgit \
|
||||
--stats=1s --stats-log-level=NOTICE --stats-one-line --fast-list
|
||||
|
||||
if ! diff <(find "${SCRIPT_FOLDER}/../.git" -type f | xargs md5sum | cut -d ' ' -f 1 | sort) <(find /tmp/test_dotgit -type f | xargs md5sum | cut -d ' ' -f 1 | sort); then
|
||||
echo "TEST FAILURE: directories are different"
|
||||
|
@ -68,6 +93,23 @@ if ! diff <(find "${SCRIPT_FOLDER}/../.git" -type f | xargs md5sum | cut -d ' '
|
|||
fi
|
||||
rm -r /tmp/test_dotgit
|
||||
|
||||
if [ "$DO_SSEC_TEST" = "1" ]; then
|
||||
rclone copy garage:eprouvette/test-ssec /tmp/test_ssec_out \
|
||||
--s3-sse-customer-algorithm AES256 \
|
||||
--s3-sse-customer-key-base64 "$SSEC_KEY" \
|
||||
--stats=1s --stats-log-level=NOTICE --stats-one-line
|
||||
if ! diff "/tmp/test_ssec_out/test-upgrade.sh" "${SCRIPT_FOLDER}/test-upgrade.sh"; then
|
||||
echo "SSEC-FAILURE (small file)"
|
||||
exit 1
|
||||
fi
|
||||
if ! diff "/tmp/test_ssec_out/randfile-for-upgrade" "/tmp/randfile-for-upgrade"; then
|
||||
echo "SSEC-FAILURE (big file)"
|
||||
exit 1
|
||||
fi
|
||||
rm -r /tmp/test_ssec_out
|
||||
rm /tmp/randfile-for-upgrade
|
||||
fi
|
||||
|
||||
echo "🏁 Teardown"
|
||||
rm -rf /tmp/garage-{data,meta}-*
|
||||
rm -rf /tmp/config.*.toml
|
||||
|
|
|
@ -14,7 +14,9 @@ path = "lib.rs"
|
|||
# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html
|
||||
|
||||
[dependencies]
|
||||
format_table.workspace = true
|
||||
garage_model.workspace = true
|
||||
garage_block.workspace = true
|
||||
garage_table.workspace = true
|
||||
garage_util.workspace = true
|
||||
garage_rpc.workspace = true
|
||||
|
@ -22,8 +24,11 @@ garage_api_common.workspace = true
|
|||
|
||||
argon2.workspace = true
|
||||
async-trait.workspace = true
|
||||
bytesize.workspace = true
|
||||
chrono.workspace = true
|
||||
err-derive.workspace = true
|
||||
hex.workspace = true
|
||||
paste.workspace = true
|
||||
tracing.workspace = true
|
||||
|
||||
futures.workspace = true
|
||||
|
@ -34,6 +39,7 @@ url.workspace = true
|
|||
|
||||
serde.workspace = true
|
||||
serde_json.workspace = true
|
||||
utoipa.workspace = true
|
||||
|
||||
opentelemetry.workspace = true
|
||||
opentelemetry-prometheus = { workspace = true, optional = true }
|
||||
|
@ -41,3 +47,4 @@ prometheus = { workspace = true, optional = true }
|
|||
|
||||
[features]
|
||||
metrics = [ "opentelemetry-prometheus", "prometheus" ]
|
||||
k2v = [ "garage_model/k2v" ]
|
||||
|
|
225
src/api/admin/admin_token.rs
Normal file
225
src/api/admin/admin_token.rs
Normal file
|
@ -0,0 +1,225 @@
|
|||
use std::sync::Arc;
|
||||
|
||||
use chrono::{DateTime, Utc};
|
||||
|
||||
use garage_table::*;
|
||||
use garage_util::time::now_msec;
|
||||
|
||||
use garage_model::admin_token_table::*;
|
||||
use garage_model::garage::Garage;
|
||||
|
||||
use crate::api::*;
|
||||
use crate::error::*;
|
||||
use crate::{Admin, RequestHandler};
|
||||
|
||||
impl RequestHandler for ListAdminTokensRequest {
|
||||
type Response = ListAdminTokensResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<ListAdminTokensResponse, Error> {
|
||||
let now = now_msec();
|
||||
|
||||
let mut res = garage
|
||||
.admin_token_table
|
||||
.get_range(
|
||||
&EmptyKey,
|
||||
None,
|
||||
Some(KeyFilter::Deleted(DeletedFilter::NotDeleted)),
|
||||
10000,
|
||||
EnumerationOrder::Forward,
|
||||
)
|
||||
.await?
|
||||
.iter()
|
||||
.map(|t| admin_token_info_results(t, now))
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
if garage.config.admin.admin_token.is_some() {
|
||||
res.insert(
|
||||
0,
|
||||
GetAdminTokenInfoResponse {
|
||||
id: None,
|
||||
created: None,
|
||||
name: "admin_token (from daemon configuration)".into(),
|
||||
expiration: None,
|
||||
expired: false,
|
||||
scope: vec!["*".into()],
|
||||
},
|
||||
);
|
||||
}
|
||||
|
||||
if garage.config.admin.metrics_token.is_some() {
|
||||
res.insert(
|
||||
1,
|
||||
GetAdminTokenInfoResponse {
|
||||
id: None,
|
||||
created: None,
|
||||
name: "metrics_token (from daemon configuration)".into(),
|
||||
expiration: None,
|
||||
expired: false,
|
||||
scope: vec!["Metrics".into()],
|
||||
},
|
||||
);
|
||||
}
|
||||
|
||||
Ok(ListAdminTokensResponse(res))
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for GetAdminTokenInfoRequest {
|
||||
type Response = GetAdminTokenInfoResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<GetAdminTokenInfoResponse, Error> {
|
||||
let token = match (self.id, self.search) {
|
||||
(Some(id), None) => get_existing_admin_token(garage, &id).await?,
|
||||
(None, Some(search)) => {
|
||||
let candidates = garage
|
||||
.admin_token_table
|
||||
.get_range(
|
||||
&EmptyKey,
|
||||
None,
|
||||
Some(KeyFilter::MatchesAndNotDeleted(search.to_string())),
|
||||
10,
|
||||
EnumerationOrder::Forward,
|
||||
)
|
||||
.await?
|
||||
.into_iter()
|
||||
.collect::<Vec<_>>();
|
||||
if candidates.len() != 1 {
|
||||
return Err(Error::bad_request(format!(
|
||||
"{} matching admin tokens",
|
||||
candidates.len()
|
||||
)));
|
||||
}
|
||||
candidates.into_iter().next().unwrap()
|
||||
}
|
||||
_ => {
|
||||
return Err(Error::bad_request(
|
||||
"Either id or search must be provided (but not both)",
|
||||
));
|
||||
}
|
||||
};
|
||||
|
||||
Ok(admin_token_info_results(&token, now_msec()))
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for CreateAdminTokenRequest {
|
||||
type Response = CreateAdminTokenResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<CreateAdminTokenResponse, Error> {
|
||||
let (mut token, secret) = if self.0.name.is_some() {
|
||||
AdminApiToken::new("")
|
||||
} else {
|
||||
AdminApiToken::new(&format!("token_{}", Utc::now().format("%Y%m%d_%H%M")))
|
||||
};
|
||||
|
||||
apply_token_updates(&mut token, self.0);
|
||||
|
||||
garage.admin_token_table.insert(&token).await?;
|
||||
|
||||
Ok(CreateAdminTokenResponse {
|
||||
secret_token: secret,
|
||||
info: admin_token_info_results(&token, now_msec()),
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for UpdateAdminTokenRequest {
|
||||
type Response = UpdateAdminTokenResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<UpdateAdminTokenResponse, Error> {
|
||||
let mut token = get_existing_admin_token(&garage, &self.id).await?;
|
||||
|
||||
apply_token_updates(&mut token, self.body);
|
||||
|
||||
garage.admin_token_table.insert(&token).await?;
|
||||
|
||||
Ok(UpdateAdminTokenResponse(admin_token_info_results(
|
||||
&token,
|
||||
now_msec(),
|
||||
)))
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for DeleteAdminTokenRequest {
|
||||
type Response = DeleteAdminTokenResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<DeleteAdminTokenResponse, Error> {
|
||||
let token = get_existing_admin_token(&garage, &self.id).await?;
|
||||
|
||||
garage
|
||||
.admin_token_table
|
||||
.insert(&AdminApiToken::delete(token.prefix))
|
||||
.await?;
|
||||
|
||||
Ok(DeleteAdminTokenResponse)
|
||||
}
|
||||
}
|
||||
|
||||
// ---- helpers ----
|
||||
|
||||
fn admin_token_info_results(token: &AdminApiToken, now: u64) -> GetAdminTokenInfoResponse {
|
||||
let params = token.params().unwrap();
|
||||
|
||||
GetAdminTokenInfoResponse {
|
||||
id: Some(token.prefix.clone()),
|
||||
created: Some(
|
||||
DateTime::from_timestamp_millis(params.created as i64)
|
||||
.expect("invalid timestamp stored in db"),
|
||||
),
|
||||
name: params.name.get().to_string(),
|
||||
expiration: params.expiration.get().map(|x| {
|
||||
DateTime::from_timestamp_millis(x as i64).expect("invalid timestamp stored in db")
|
||||
}),
|
||||
expired: params
|
||||
.expiration
|
||||
.get()
|
||||
.map(|exp| now > exp)
|
||||
.unwrap_or(false),
|
||||
scope: params.scope.get().0.clone(),
|
||||
}
|
||||
}
|
||||
|
||||
async fn get_existing_admin_token(garage: &Garage, id: &String) -> Result<AdminApiToken, Error> {
|
||||
garage
|
||||
.admin_token_table
|
||||
.get(&EmptyKey, id)
|
||||
.await?
|
||||
.filter(|k| !k.state.is_deleted())
|
||||
.ok_or_else(|| Error::NoSuchAdminToken(id.to_string()))
|
||||
}
|
||||
|
||||
fn apply_token_updates(token: &mut AdminApiToken, updates: UpdateAdminTokenRequestBody) {
|
||||
let params = token.params_mut().unwrap();
|
||||
|
||||
if let Some(name) = updates.name {
|
||||
params.name.update(name);
|
||||
}
|
||||
if let Some(expiration) = updates.expiration {
|
||||
params
|
||||
.expiration
|
||||
.update(Some(expiration.timestamp_millis() as u64));
|
||||
}
|
||||
if let Some(scope) = updates.scope {
|
||||
params.scope.update(AdminApiTokenScope(scope));
|
||||
}
|
||||
}
|
1292
src/api/admin/api.rs
Normal file
1292
src/api/admin/api.rs
Normal file
File diff suppressed because it is too large
Load diff
|
@ -1,333 +1,230 @@
|
|||
use std::collections::HashMap;
|
||||
use std::borrow::Cow;
|
||||
use std::sync::Arc;
|
||||
|
||||
use argon2::password_hash::PasswordHash;
|
||||
|
||||
use http::header::{ACCESS_CONTROL_ALLOW_METHODS, ACCESS_CONTROL_ALLOW_ORIGIN, ALLOW};
|
||||
use hyper::{body::Incoming as IncomingBody, Request, Response, StatusCode};
|
||||
use http::header::{HeaderValue, ACCESS_CONTROL_ALLOW_ORIGIN, AUTHORIZATION};
|
||||
use hyper::{body::Incoming as IncomingBody, Request, Response};
|
||||
use serde::{Deserialize, Serialize};
|
||||
use tokio::sync::watch;
|
||||
|
||||
use opentelemetry::trace::SpanRef;
|
||||
|
||||
#[cfg(feature = "metrics")]
|
||||
use opentelemetry_prometheus::PrometheusExporter;
|
||||
#[cfg(feature = "metrics")]
|
||||
use prometheus::{Encoder, TextEncoder};
|
||||
|
||||
use garage_model::garage::Garage;
|
||||
use garage_rpc::system::ClusterHealthStatus;
|
||||
use garage_rpc::{Endpoint as RpcEndpoint, *};
|
||||
use garage_table::EmptyKey;
|
||||
use garage_util::background::BackgroundRunner;
|
||||
use garage_util::data::Uuid;
|
||||
use garage_util::error::Error as GarageError;
|
||||
use garage_util::socket_address::UnixOrTCPSocketAddress;
|
||||
use garage_util::time::now_msec;
|
||||
|
||||
use garage_api_common::generic_server::*;
|
||||
use garage_api_common::helpers::*;
|
||||
|
||||
use crate::bucket::*;
|
||||
use crate::cluster::*;
|
||||
use crate::api::*;
|
||||
use crate::error::*;
|
||||
use crate::key::*;
|
||||
use crate::router_v0;
|
||||
use crate::router_v1::{Authorization, Endpoint};
|
||||
use crate::router_v1;
|
||||
use crate::Authorization;
|
||||
use crate::RequestHandler;
|
||||
|
||||
// ---- FOR RPC ----
|
||||
|
||||
pub const ADMIN_RPC_PATH: &str = "garage_api/admin/rpc.rs/Rpc";
|
||||
|
||||
#[derive(Debug, Serialize, Deserialize)]
|
||||
pub enum AdminRpc {
|
||||
Proxy(AdminApiRequest),
|
||||
Internal(LocalAdminApiRequest),
|
||||
}
|
||||
|
||||
#[derive(Debug, Serialize, Deserialize)]
|
||||
pub enum AdminRpcResponse {
|
||||
ProxyApiOkResponse(TaggedAdminApiResponse),
|
||||
InternalApiOkResponse(LocalAdminApiResponse),
|
||||
ApiErrorResponse {
|
||||
http_code: u16,
|
||||
error_code: String,
|
||||
message: String,
|
||||
},
|
||||
}
|
||||
|
||||
impl Rpc for AdminRpc {
|
||||
type Response = Result<AdminRpcResponse, GarageError>;
|
||||
}
|
||||
|
||||
impl EndpointHandler<AdminRpc> for AdminApiServer {
|
||||
async fn handle(
|
||||
self: &Arc<Self>,
|
||||
message: &AdminRpc,
|
||||
_from: NodeID,
|
||||
) -> Result<AdminRpcResponse, GarageError> {
|
||||
match message {
|
||||
AdminRpc::Proxy(req) => {
|
||||
info!("Proxied admin API request: {}", req.name());
|
||||
let res = req.clone().handle(&self.garage, &self).await;
|
||||
match res {
|
||||
Ok(res) => Ok(AdminRpcResponse::ProxyApiOkResponse(res.tagged())),
|
||||
Err(e) => Ok(AdminRpcResponse::ApiErrorResponse {
|
||||
http_code: e.http_status_code().as_u16(),
|
||||
error_code: e.code().to_string(),
|
||||
message: e.to_string(),
|
||||
}),
|
||||
}
|
||||
}
|
||||
AdminRpc::Internal(req) => {
|
||||
info!("Internal admin API request: {}", req.name());
|
||||
let res = req.clone().handle(&self.garage, &self).await;
|
||||
match res {
|
||||
Ok(res) => Ok(AdminRpcResponse::InternalApiOkResponse(res)),
|
||||
Err(e) => Ok(AdminRpcResponse::ApiErrorResponse {
|
||||
http_code: e.http_status_code().as_u16(),
|
||||
error_code: e.code().to_string(),
|
||||
message: e.to_string(),
|
||||
}),
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// ---- FOR HTTP ----
|
||||
|
||||
pub type ResBody = BoxBody<Error>;
|
||||
|
||||
pub struct AdminApiServer {
|
||||
garage: Arc<Garage>,
|
||||
#[cfg(feature = "metrics")]
|
||||
exporter: PrometheusExporter,
|
||||
pub(crate) exporter: PrometheusExporter,
|
||||
metrics_token: Option<String>,
|
||||
metrics_require_token: bool,
|
||||
admin_token: Option<String>,
|
||||
pub(crate) background: Arc<BackgroundRunner>,
|
||||
pub(crate) endpoint: Arc<RpcEndpoint<AdminRpc, Self>>,
|
||||
}
|
||||
|
||||
pub enum HttpEndpoint {
|
||||
Old(router_v1::Endpoint),
|
||||
New(String),
|
||||
}
|
||||
|
||||
impl AdminApiServer {
|
||||
pub fn new(
|
||||
garage: Arc<Garage>,
|
||||
background: Arc<BackgroundRunner>,
|
||||
#[cfg(feature = "metrics")] exporter: PrometheusExporter,
|
||||
) -> Self {
|
||||
) -> Arc<Self> {
|
||||
let cfg = &garage.config.admin;
|
||||
let metrics_token = cfg.metrics_token.as_deref().map(hash_bearer_token);
|
||||
let admin_token = cfg.admin_token.as_deref().map(hash_bearer_token);
|
||||
Self {
|
||||
let metrics_require_token = cfg.metrics_require_token;
|
||||
|
||||
let endpoint = garage.system.netapp.endpoint(ADMIN_RPC_PATH.into());
|
||||
let admin = Arc::new(Self {
|
||||
garage,
|
||||
#[cfg(feature = "metrics")]
|
||||
exporter,
|
||||
metrics_token,
|
||||
metrics_require_token,
|
||||
admin_token,
|
||||
}
|
||||
background,
|
||||
endpoint,
|
||||
});
|
||||
admin.endpoint.set_handler(admin.clone());
|
||||
admin
|
||||
}
|
||||
|
||||
pub async fn run(
|
||||
self,
|
||||
self: Arc<Self>,
|
||||
bind_addr: UnixOrTCPSocketAddress,
|
||||
must_exit: watch::Receiver<bool>,
|
||||
) -> Result<(), GarageError> {
|
||||
let region = self.garage.config.s3_api.s3_region.clone();
|
||||
ApiServer::new(region, self)
|
||||
ApiServer::new(region, ArcAdminApiServer(self))
|
||||
.run_server(bind_addr, Some(0o220), must_exit)
|
||||
.await
|
||||
}
|
||||
|
||||
fn handle_options(&self, _req: &Request<IncomingBody>) -> Result<Response<ResBody>, Error> {
|
||||
Ok(Response::builder()
|
||||
.status(StatusCode::NO_CONTENT)
|
||||
.header(ALLOW, "OPTIONS, GET, POST")
|
||||
.header(ACCESS_CONTROL_ALLOW_METHODS, "OPTIONS, GET, POST")
|
||||
.header(ACCESS_CONTROL_ALLOW_ORIGIN, "*")
|
||||
.body(empty_body())?)
|
||||
}
|
||||
|
||||
async fn handle_check_domain(
|
||||
async fn handle_http_api(
|
||||
&self,
|
||||
req: Request<IncomingBody>,
|
||||
endpoint: HttpEndpoint,
|
||||
) -> Result<Response<ResBody>, Error> {
|
||||
let query_params: HashMap<String, String> = req
|
||||
.uri()
|
||||
.query()
|
||||
.map(|v| {
|
||||
url::form_urlencoded::parse(v.as_bytes())
|
||||
.into_owned()
|
||||
.collect()
|
||||
})
|
||||
.unwrap_or_else(HashMap::new);
|
||||
let auth_header = req.headers().get(AUTHORIZATION).cloned();
|
||||
|
||||
let has_domain_key = query_params.contains_key("domain");
|
||||
|
||||
if !has_domain_key {
|
||||
return Err(Error::bad_request("No domain query string found"));
|
||||
}
|
||||
|
||||
let domain = query_params
|
||||
.get("domain")
|
||||
.ok_or_internal_error("Could not parse domain query string")?;
|
||||
|
||||
if self.check_domain(domain).await? {
|
||||
Ok(Response::builder()
|
||||
.status(StatusCode::OK)
|
||||
.body(string_body(format!(
|
||||
"Domain '{domain}' is managed by Garage"
|
||||
)))?)
|
||||
} else {
|
||||
Err(Error::bad_request(format!(
|
||||
"Domain '{domain}' is not managed by Garage"
|
||||
)))
|
||||
}
|
||||
}
|
||||
|
||||
async fn check_domain(&self, domain: &str) -> Result<bool, Error> {
|
||||
// Resolve bucket from domain name, inferring if the website must be activated for the
|
||||
// domain to be valid.
|
||||
let (bucket_name, must_check_website) = if let Some(bname) = self
|
||||
.garage
|
||||
.config
|
||||
.s3_api
|
||||
.root_domain
|
||||
.as_ref()
|
||||
.and_then(|rd| host_to_bucket(domain, rd))
|
||||
{
|
||||
(bname.to_string(), false)
|
||||
} else if let Some(bname) = self
|
||||
.garage
|
||||
.config
|
||||
.s3_web
|
||||
.as_ref()
|
||||
.and_then(|sw| host_to_bucket(domain, sw.root_domain.as_str()))
|
||||
{
|
||||
(bname.to_string(), true)
|
||||
} else {
|
||||
(domain.to_string(), true)
|
||||
let request = match endpoint {
|
||||
HttpEndpoint::Old(endpoint_v1) => AdminApiRequest::from_v1(endpoint_v1, req).await?,
|
||||
HttpEndpoint::New(_) => AdminApiRequest::from_request(req).await?,
|
||||
};
|
||||
|
||||
let bucket_id = match self
|
||||
.garage
|
||||
.bucket_helper()
|
||||
.resolve_global_bucket_name(&bucket_name)
|
||||
.await?
|
||||
{
|
||||
Some(bucket_id) => bucket_id,
|
||||
None => return Ok(false),
|
||||
};
|
||||
|
||||
if !must_check_website {
|
||||
return Ok(true);
|
||||
}
|
||||
|
||||
let bucket = self
|
||||
.garage
|
||||
.bucket_helper()
|
||||
.get_existing_bucket(bucket_id)
|
||||
.await?;
|
||||
|
||||
let bucket_state = bucket.state.as_option().unwrap();
|
||||
let bucket_website_config = bucket_state.website_config.get();
|
||||
|
||||
match bucket_website_config {
|
||||
Some(_v) => Ok(true),
|
||||
None => Ok(false),
|
||||
}
|
||||
}
|
||||
|
||||
fn handle_health(&self) -> Result<Response<ResBody>, Error> {
|
||||
let health = self.garage.system.health();
|
||||
|
||||
let (status, status_str) = match health.status {
|
||||
ClusterHealthStatus::Healthy => (StatusCode::OK, "Garage is fully operational"),
|
||||
ClusterHealthStatus::Degraded => (
|
||||
StatusCode::OK,
|
||||
"Garage is operational but some storage nodes are unavailable",
|
||||
),
|
||||
ClusterHealthStatus::Unavailable => (
|
||||
StatusCode::SERVICE_UNAVAILABLE,
|
||||
"Quorum is not available for some/all partitions, reads and writes will fail",
|
||||
let (global_token_hash, token_required) = match request.authorization_type() {
|
||||
Authorization::None => (None, false),
|
||||
Authorization::MetricsToken => (
|
||||
self.metrics_token.as_deref(),
|
||||
self.metrics_token.is_some() || self.metrics_require_token,
|
||||
),
|
||||
Authorization::AdminToken => (self.admin_token.as_deref(), true),
|
||||
};
|
||||
let status_str = format!(
|
||||
"{}\nConsult the full health check API endpoint at /v1/health for more details\n",
|
||||
status_str
|
||||
);
|
||||
|
||||
Ok(Response::builder()
|
||||
.status(status)
|
||||
.header(http::header::CONTENT_TYPE, "text/plain")
|
||||
.body(string_body(status_str))?)
|
||||
}
|
||||
|
||||
fn handle_metrics(&self) -> Result<Response<ResBody>, Error> {
|
||||
#[cfg(feature = "metrics")]
|
||||
{
|
||||
use opentelemetry::trace::Tracer;
|
||||
|
||||
let mut buffer = vec![];
|
||||
let encoder = TextEncoder::new();
|
||||
|
||||
let tracer = opentelemetry::global::tracer("garage");
|
||||
let metric_families = tracer.in_span("admin/gather_metrics", |_| {
|
||||
self.exporter.registry().gather()
|
||||
});
|
||||
|
||||
encoder
|
||||
.encode(&metric_families, &mut buffer)
|
||||
.ok_or_internal_error("Could not serialize metrics")?;
|
||||
|
||||
Ok(Response::builder()
|
||||
.status(StatusCode::OK)
|
||||
.header(http::header::CONTENT_TYPE, encoder.format_type())
|
||||
.body(bytes_body(buffer.into()))?)
|
||||
if token_required {
|
||||
verify_authorization(&self.garage, global_token_hash, auth_header, request.name())?;
|
||||
}
|
||||
|
||||
match request {
|
||||
AdminApiRequest::Options(req) => req.handle(&self.garage, &self).await,
|
||||
AdminApiRequest::CheckDomain(req) => req.handle(&self.garage, &self).await,
|
||||
AdminApiRequest::Health(req) => req.handle(&self.garage, &self).await,
|
||||
AdminApiRequest::Metrics(req) => req.handle(&self.garage, &self).await,
|
||||
req => {
|
||||
let res = req.handle(&self.garage, &self).await?;
|
||||
let mut res = json_ok_response(&res)?;
|
||||
res.headers_mut()
|
||||
.insert(ACCESS_CONTROL_ALLOW_ORIGIN, HeaderValue::from_static("*"));
|
||||
Ok(res)
|
||||
}
|
||||
}
|
||||
#[cfg(not(feature = "metrics"))]
|
||||
Err(Error::bad_request(
|
||||
"Garage was built without the metrics feature".to_string(),
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
impl ApiHandler for AdminApiServer {
|
||||
struct ArcAdminApiServer(Arc<AdminApiServer>);
|
||||
|
||||
impl ApiHandler for ArcAdminApiServer {
|
||||
const API_NAME: &'static str = "admin";
|
||||
const API_NAME_DISPLAY: &'static str = "Admin";
|
||||
|
||||
type Endpoint = Endpoint;
|
||||
type Endpoint = HttpEndpoint;
|
||||
type Error = Error;
|
||||
|
||||
fn parse_endpoint(&self, req: &Request<IncomingBody>) -> Result<Endpoint, Error> {
|
||||
fn parse_endpoint(&self, req: &Request<IncomingBody>) -> Result<HttpEndpoint, Error> {
|
||||
if req.uri().path().starts_with("/v0/") {
|
||||
let endpoint_v0 = router_v0::Endpoint::from_request(req)?;
|
||||
Endpoint::from_v0(endpoint_v0)
|
||||
let endpoint_v1 = router_v1::Endpoint::from_v0(endpoint_v0)?;
|
||||
Ok(HttpEndpoint::Old(endpoint_v1))
|
||||
} else if req.uri().path().starts_with("/v1/") {
|
||||
let endpoint_v1 = router_v1::Endpoint::from_request(req)?;
|
||||
Ok(HttpEndpoint::Old(endpoint_v1))
|
||||
} else {
|
||||
Endpoint::from_request(req)
|
||||
Ok(HttpEndpoint::New(req.uri().path().to_string()))
|
||||
}
|
||||
}
|
||||
|
||||
async fn handle(
|
||||
&self,
|
||||
req: Request<IncomingBody>,
|
||||
endpoint: Endpoint,
|
||||
endpoint: HttpEndpoint,
|
||||
) -> Result<Response<ResBody>, Error> {
|
||||
let required_auth_hash =
|
||||
match endpoint.authorization_type() {
|
||||
Authorization::None => None,
|
||||
Authorization::MetricsToken => self.metrics_token.as_deref(),
|
||||
Authorization::AdminToken => match self.admin_token.as_deref() {
|
||||
None => return Err(Error::forbidden(
|
||||
"Admin token isn't configured, admin API access is disabled for security.",
|
||||
)),
|
||||
Some(t) => Some(t),
|
||||
},
|
||||
};
|
||||
|
||||
if let Some(password_hash) = required_auth_hash {
|
||||
match req.headers().get("Authorization") {
|
||||
None => return Err(Error::forbidden("Authorization token must be provided")),
|
||||
Some(authorization) => {
|
||||
verify_bearer_token(&authorization, password_hash)?;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
match endpoint {
|
||||
Endpoint::Options => self.handle_options(&req),
|
||||
Endpoint::CheckDomain => self.handle_check_domain(req).await,
|
||||
Endpoint::Health => self.handle_health(),
|
||||
Endpoint::Metrics => self.handle_metrics(),
|
||||
Endpoint::GetClusterStatus => handle_get_cluster_status(&self.garage).await,
|
||||
Endpoint::GetClusterHealth => handle_get_cluster_health(&self.garage).await,
|
||||
Endpoint::ConnectClusterNodes => handle_connect_cluster_nodes(&self.garage, req).await,
|
||||
// Layout
|
||||
Endpoint::GetClusterLayout => handle_get_cluster_layout(&self.garage).await,
|
||||
Endpoint::UpdateClusterLayout => handle_update_cluster_layout(&self.garage, req).await,
|
||||
Endpoint::ApplyClusterLayout => handle_apply_cluster_layout(&self.garage, req).await,
|
||||
Endpoint::RevertClusterLayout => handle_revert_cluster_layout(&self.garage).await,
|
||||
// Keys
|
||||
Endpoint::ListKeys => handle_list_keys(&self.garage).await,
|
||||
Endpoint::GetKeyInfo {
|
||||
id,
|
||||
search,
|
||||
show_secret_key,
|
||||
} => {
|
||||
let show_secret_key = show_secret_key.map(|x| x == "true").unwrap_or(false);
|
||||
handle_get_key_info(&self.garage, id, search, show_secret_key).await
|
||||
}
|
||||
Endpoint::CreateKey => handle_create_key(&self.garage, req).await,
|
||||
Endpoint::ImportKey => handle_import_key(&self.garage, req).await,
|
||||
Endpoint::UpdateKey { id } => handle_update_key(&self.garage, id, req).await,
|
||||
Endpoint::DeleteKey { id } => handle_delete_key(&self.garage, id).await,
|
||||
// Buckets
|
||||
Endpoint::ListBuckets => handle_list_buckets(&self.garage).await,
|
||||
Endpoint::GetBucketInfo { id, global_alias } => {
|
||||
handle_get_bucket_info(&self.garage, id, global_alias).await
|
||||
}
|
||||
Endpoint::CreateBucket => handle_create_bucket(&self.garage, req).await,
|
||||
Endpoint::DeleteBucket { id } => handle_delete_bucket(&self.garage, id).await,
|
||||
Endpoint::UpdateBucket { id } => handle_update_bucket(&self.garage, id, req).await,
|
||||
// Bucket-key permissions
|
||||
Endpoint::BucketAllowKey => {
|
||||
handle_bucket_change_key_perm(&self.garage, req, true).await
|
||||
}
|
||||
Endpoint::BucketDenyKey => {
|
||||
handle_bucket_change_key_perm(&self.garage, req, false).await
|
||||
}
|
||||
// Bucket aliasing
|
||||
Endpoint::GlobalAliasBucket { id, alias } => {
|
||||
handle_global_alias_bucket(&self.garage, id, alias).await
|
||||
}
|
||||
Endpoint::GlobalUnaliasBucket { id, alias } => {
|
||||
handle_global_unalias_bucket(&self.garage, id, alias).await
|
||||
}
|
||||
Endpoint::LocalAliasBucket {
|
||||
id,
|
||||
access_key_id,
|
||||
alias,
|
||||
} => handle_local_alias_bucket(&self.garage, id, access_key_id, alias).await,
|
||||
Endpoint::LocalUnaliasBucket {
|
||||
id,
|
||||
access_key_id,
|
||||
alias,
|
||||
} => handle_local_unalias_bucket(&self.garage, id, access_key_id, alias).await,
|
||||
}
|
||||
self.0.handle_http_api(req, endpoint).await
|
||||
}
|
||||
}
|
||||
|
||||
impl ApiEndpoint for Endpoint {
|
||||
fn name(&self) -> &'static str {
|
||||
Endpoint::name(self)
|
||||
impl ApiEndpoint for HttpEndpoint {
|
||||
fn name(&self) -> Cow<'static, str> {
|
||||
match self {
|
||||
Self::Old(endpoint_v1) => Cow::Borrowed(endpoint_v1.name()),
|
||||
Self::New(path) => Cow::Owned(path.clone()),
|
||||
}
|
||||
}
|
||||
|
||||
fn add_span_attributes(&self, _span: SpanRef<'_>) {}
|
||||
|
@ -347,20 +244,98 @@ fn hash_bearer_token(token: &str) -> String {
|
|||
.to_string()
|
||||
}
|
||||
|
||||
fn verify_bearer_token(token: &hyper::http::HeaderValue, password_hash: &str) -> Result<(), Error> {
|
||||
use argon2::{password_hash::PasswordVerifier, Argon2};
|
||||
fn verify_authorization(
|
||||
garage: &Garage,
|
||||
global_token_hash: Option<&str>,
|
||||
auth_header: Option<hyper::http::HeaderValue>,
|
||||
endpoint_name: &str,
|
||||
) -> Result<(), Error> {
|
||||
use argon2::{password_hash::PasswordHash, password_hash::PasswordVerifier, Argon2};
|
||||
|
||||
let parsed_hash = PasswordHash::new(&password_hash).unwrap();
|
||||
let invalid_msg = "Invalid bearer token";
|
||||
|
||||
token
|
||||
.to_str()?
|
||||
.strip_prefix("Bearer ")
|
||||
.and_then(|token| {
|
||||
Argon2::default()
|
||||
.verify_password(token.trim().as_bytes(), &parsed_hash)
|
||||
.ok()
|
||||
})
|
||||
.ok_or_else(|| Error::forbidden("Invalid authorization token"))?;
|
||||
let token = match &auth_header {
|
||||
None => {
|
||||
return Err(Error::forbidden(
|
||||
"Bearer token must be provided in Authorization header",
|
||||
))
|
||||
}
|
||||
Some(authorization) => authorization
|
||||
.to_str()?
|
||||
.strip_prefix("Bearer ")
|
||||
.ok_or_else(|| Error::forbidden("Invalid Authorization header"))?
|
||||
.trim(),
|
||||
};
|
||||
|
||||
let token_hash_string = if let Some((prefix, _)) = token.split_once('.') {
|
||||
garage
|
||||
.admin_token_table
|
||||
.get_local(&EmptyKey, &prefix.to_string())?
|
||||
.and_then(|k| k.state.into_option())
|
||||
.filter(|p| {
|
||||
p.expiration
|
||||
.get()
|
||||
.map(|exp| now_msec() < exp)
|
||||
.unwrap_or(true)
|
||||
})
|
||||
.filter(|p| {
|
||||
p.scope
|
||||
.get()
|
||||
.0
|
||||
.iter()
|
||||
.any(|x| x == "*" || x == endpoint_name)
|
||||
})
|
||||
.ok_or_else(|| Error::forbidden(invalid_msg))?
|
||||
.token_hash
|
||||
} else {
|
||||
global_token_hash
|
||||
.ok_or_else(|| Error::forbidden(invalid_msg))?
|
||||
.to_string()
|
||||
};
|
||||
|
||||
let token_hash =
|
||||
PasswordHash::new(&token_hash_string).ok_or_internal_error("Could not parse token hash")?;
|
||||
|
||||
Argon2::default()
|
||||
.verify_password(token.as_bytes(), &token_hash)
|
||||
.map_err(|_| Error::forbidden(invalid_msg))?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub(crate) fn find_matching_nodes(garage: &Garage, spec: &str) -> Result<Vec<Uuid>, Error> {
|
||||
let mut res = vec![];
|
||||
if spec == "*" {
|
||||
res = garage.system.cluster_layout().all_nodes().to_vec();
|
||||
for node in garage.system.get_known_nodes() {
|
||||
if node.is_up && !res.contains(&node.id) {
|
||||
res.push(node.id);
|
||||
}
|
||||
}
|
||||
} else if spec == "self" {
|
||||
res.push(garage.system.id);
|
||||
} else {
|
||||
let layout = garage.system.cluster_layout();
|
||||
let known_nodes = garage.system.get_known_nodes();
|
||||
let all_nodes = layout
|
||||
.all_nodes()
|
||||
.iter()
|
||||
.copied()
|
||||
.chain(known_nodes.iter().filter(|x| x.is_up).map(|x| x.id));
|
||||
for node in all_nodes {
|
||||
if !res.contains(&node) && hex::encode(node).starts_with(spec) {
|
||||
res.push(node);
|
||||
}
|
||||
}
|
||||
if res.is_empty() {
|
||||
return Err(Error::bad_request(format!("No nodes matching {}", spec)));
|
||||
}
|
||||
if res.len() > 1 {
|
||||
return Err(Error::bad_request(format!(
|
||||
"Multiple nodes matching {}: {:?}",
|
||||
spec, res
|
||||
)));
|
||||
}
|
||||
}
|
||||
Ok(res)
|
||||
}
|
||||
|
|
276
src/api/admin/block.rs
Normal file
276
src/api/admin/block.rs
Normal file
|
@ -0,0 +1,276 @@
|
|||
use std::sync::Arc;
|
||||
|
||||
use garage_util::data::*;
|
||||
use garage_util::error::Error as GarageError;
|
||||
use garage_util::time::now_msec;
|
||||
|
||||
use garage_table::EmptyKey;
|
||||
|
||||
use garage_model::garage::Garage;
|
||||
use garage_model::s3::object_table::*;
|
||||
use garage_model::s3::version_table::*;
|
||||
|
||||
use garage_api_common::common_error::CommonErrorDerivative;
|
||||
|
||||
use crate::api::*;
|
||||
use crate::error::*;
|
||||
use crate::{Admin, RequestHandler};
|
||||
|
||||
impl RequestHandler for LocalListBlockErrorsRequest {
|
||||
type Response = LocalListBlockErrorsResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<LocalListBlockErrorsResponse, Error> {
|
||||
let errors = garage.block_manager.list_resync_errors()?;
|
||||
let now = now_msec();
|
||||
let errors = errors
|
||||
.into_iter()
|
||||
.map(|e| BlockError {
|
||||
block_hash: hex::encode(&e.hash),
|
||||
refcount: e.refcount,
|
||||
error_count: e.error_count,
|
||||
last_try_secs_ago: now.saturating_sub(e.last_try) / 1000,
|
||||
next_try_in_secs: e.next_try.saturating_sub(now) / 1000,
|
||||
})
|
||||
.collect();
|
||||
Ok(LocalListBlockErrorsResponse(errors))
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for LocalGetBlockInfoRequest {
|
||||
type Response = LocalGetBlockInfoResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<LocalGetBlockInfoResponse, Error> {
|
||||
let hash = find_block_hash_by_prefix(garage, &self.block_hash)?;
|
||||
let refcount = garage.block_manager.get_block_rc(&hash)?;
|
||||
let block_refs = garage
|
||||
.block_ref_table
|
||||
.get_range(&hash, None, None, 10000, Default::default())
|
||||
.await?;
|
||||
let mut versions = vec![];
|
||||
for br in block_refs {
|
||||
if let Some(v) = garage.version_table.get(&br.version, &EmptyKey).await? {
|
||||
let bl = match &v.backlink {
|
||||
VersionBacklink::MultipartUpload { upload_id } => {
|
||||
if let Some(u) = garage.mpu_table.get(upload_id, &EmptyKey).await? {
|
||||
BlockVersionBacklink::Upload {
|
||||
upload_id: hex::encode(&upload_id),
|
||||
upload_deleted: u.deleted.get(),
|
||||
upload_garbage_collected: false,
|
||||
bucket_id: Some(hex::encode(&u.bucket_id)),
|
||||
key: Some(u.key.to_string()),
|
||||
}
|
||||
} else {
|
||||
BlockVersionBacklink::Upload {
|
||||
upload_id: hex::encode(&upload_id),
|
||||
upload_deleted: true,
|
||||
upload_garbage_collected: true,
|
||||
bucket_id: None,
|
||||
key: None,
|
||||
}
|
||||
}
|
||||
}
|
||||
VersionBacklink::Object { bucket_id, key } => BlockVersionBacklink::Object {
|
||||
bucket_id: hex::encode(&bucket_id),
|
||||
key: key.to_string(),
|
||||
},
|
||||
};
|
||||
versions.push(BlockVersion {
|
||||
version_id: hex::encode(&br.version),
|
||||
ref_deleted: br.deleted.get(),
|
||||
version_deleted: v.deleted.get(),
|
||||
garbage_collected: false,
|
||||
backlink: Some(bl),
|
||||
});
|
||||
} else {
|
||||
versions.push(BlockVersion {
|
||||
version_id: hex::encode(&br.version),
|
||||
ref_deleted: br.deleted.get(),
|
||||
version_deleted: true,
|
||||
garbage_collected: true,
|
||||
backlink: None,
|
||||
});
|
||||
}
|
||||
}
|
||||
Ok(LocalGetBlockInfoResponse {
|
||||
block_hash: hex::encode(&hash),
|
||||
refcount,
|
||||
versions,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for LocalRetryBlockResyncRequest {
|
||||
type Response = LocalRetryBlockResyncResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<LocalRetryBlockResyncResponse, Error> {
|
||||
match self {
|
||||
Self::All { all: true } => {
|
||||
let blocks = garage.block_manager.list_resync_errors()?;
|
||||
for b in blocks.iter() {
|
||||
garage.block_manager.resync.clear_backoff(&b.hash)?;
|
||||
}
|
||||
Ok(LocalRetryBlockResyncResponse {
|
||||
count: blocks.len() as u64,
|
||||
})
|
||||
}
|
||||
Self::All { all: false } => Err(Error::bad_request("nonsense")),
|
||||
Self::Blocks { block_hashes } => {
|
||||
for hash in block_hashes.iter() {
|
||||
let hash = hex::decode(hash).ok_or_bad_request("invalid hash")?;
|
||||
let hash = Hash::try_from(&hash).ok_or_bad_request("invalid hash")?;
|
||||
garage.block_manager.resync.clear_backoff(&hash)?;
|
||||
}
|
||||
Ok(LocalRetryBlockResyncResponse {
|
||||
count: block_hashes.len() as u64,
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for LocalPurgeBlocksRequest {
|
||||
type Response = LocalPurgeBlocksResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<LocalPurgeBlocksResponse, Error> {
|
||||
let mut obj_dels = 0;
|
||||
let mut mpu_dels = 0;
|
||||
let mut ver_dels = 0;
|
||||
|
||||
for hash in self.0.iter() {
|
||||
let hash = hex::decode(hash).ok_or_bad_request("invalid hash")?;
|
||||
let hash = Hash::try_from(&hash).ok_or_bad_request("invalid hash")?;
|
||||
let block_refs = garage
|
||||
.block_ref_table
|
||||
.get_range(&hash, None, None, 10000, Default::default())
|
||||
.await?;
|
||||
|
||||
for br in block_refs {
|
||||
if let Some(version) = garage.version_table.get(&br.version, &EmptyKey).await? {
|
||||
handle_block_purge_version_backlink(
|
||||
garage,
|
||||
&version,
|
||||
&mut obj_dels,
|
||||
&mut mpu_dels,
|
||||
)
|
||||
.await?;
|
||||
|
||||
if !version.deleted.get() {
|
||||
let deleted_version = Version::new(version.uuid, version.backlink, true);
|
||||
garage.version_table.insert(&deleted_version).await?;
|
||||
ver_dels += 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(LocalPurgeBlocksResponse {
|
||||
blocks_purged: self.0.len() as u64,
|
||||
versions_deleted: ver_dels,
|
||||
objects_deleted: obj_dels,
|
||||
uploads_deleted: mpu_dels,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
fn find_block_hash_by_prefix(garage: &Arc<Garage>, prefix: &str) -> Result<Hash, Error> {
|
||||
if prefix.len() < 4 {
|
||||
return Err(Error::bad_request(
|
||||
"Please specify at least 4 characters of the block hash",
|
||||
));
|
||||
}
|
||||
|
||||
let prefix_bin = hex::decode(&prefix[..prefix.len() & !1]).ok_or_bad_request("invalid hash")?;
|
||||
|
||||
let iter = garage
|
||||
.block_ref_table
|
||||
.data
|
||||
.store
|
||||
.range(&prefix_bin[..]..)
|
||||
.map_err(GarageError::from)?;
|
||||
let mut found = None;
|
||||
for item in iter {
|
||||
let (k, _v) = item.map_err(GarageError::from)?;
|
||||
let hash = Hash::try_from(&k[..32]).unwrap();
|
||||
if &hash.as_slice()[..prefix_bin.len()] != prefix_bin {
|
||||
break;
|
||||
}
|
||||
if hex::encode(hash.as_slice()).starts_with(prefix) {
|
||||
match &found {
|
||||
Some(x) if *x == hash => (),
|
||||
Some(_) => {
|
||||
return Err(Error::bad_request(format!(
|
||||
"Several blocks match prefix `{}`",
|
||||
prefix
|
||||
)));
|
||||
}
|
||||
None => {
|
||||
found = Some(hash);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
found.ok_or_else(|| Error::NoSuchBlock(prefix.to_string()))
|
||||
}
|
||||
|
||||
async fn handle_block_purge_version_backlink(
|
||||
garage: &Arc<Garage>,
|
||||
version: &Version,
|
||||
obj_dels: &mut u64,
|
||||
mpu_dels: &mut u64,
|
||||
) -> Result<(), Error> {
|
||||
let (bucket_id, key, ov_id) = match &version.backlink {
|
||||
VersionBacklink::Object { bucket_id, key } => (*bucket_id, key.clone(), version.uuid),
|
||||
VersionBacklink::MultipartUpload { upload_id } => {
|
||||
if let Some(mut mpu) = garage.mpu_table.get(upload_id, &EmptyKey).await? {
|
||||
if !mpu.deleted.get() {
|
||||
mpu.parts.clear();
|
||||
mpu.deleted.set();
|
||||
garage.mpu_table.insert(&mpu).await?;
|
||||
*mpu_dels += 1;
|
||||
}
|
||||
(mpu.bucket_id, mpu.key.clone(), *upload_id)
|
||||
} else {
|
||||
return Ok(());
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
if let Some(object) = garage.object_table.get(&bucket_id, &key).await? {
|
||||
let ov = object.versions().iter().rev().find(|v| v.is_complete());
|
||||
if let Some(ov) = ov {
|
||||
if ov.uuid == ov_id {
|
||||
let del_uuid = gen_uuid();
|
||||
let deleted_object = Object::new(
|
||||
bucket_id,
|
||||
key,
|
||||
vec![ObjectVersion {
|
||||
uuid: del_uuid,
|
||||
timestamp: ov.timestamp + 1,
|
||||
state: ObjectVersionState::Complete(ObjectVersionData::DeleteMarker),
|
||||
}],
|
||||
);
|
||||
garage.object_table.insert(&deleted_object).await?;
|
||||
*obj_dels += 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
File diff suppressed because it is too large
Load diff
|
@ -1,411 +1,276 @@
|
|||
use std::collections::HashMap;
|
||||
use std::net::SocketAddr;
|
||||
use std::fmt::Write;
|
||||
use std::sync::Arc;
|
||||
|
||||
use hyper::{body::Incoming as IncomingBody, Request, Response};
|
||||
use serde::{Deserialize, Serialize};
|
||||
use format_table::format_table_to_string;
|
||||
|
||||
use garage_util::crdt::*;
|
||||
use garage_util::data::*;
|
||||
|
||||
use garage_rpc::layout;
|
||||
use garage_rpc::layout::PARTITION_BITS;
|
||||
|
||||
use garage_model::garage::Garage;
|
||||
|
||||
use garage_api_common::helpers::{json_ok_response, parse_json_body};
|
||||
|
||||
use crate::api_server::ResBody;
|
||||
use crate::api::*;
|
||||
use crate::error::*;
|
||||
use crate::{Admin, RequestHandler};
|
||||
|
||||
pub async fn handle_get_cluster_status(garage: &Arc<Garage>) -> Result<Response<ResBody>, Error> {
|
||||
let layout = garage.system.cluster_layout();
|
||||
let mut nodes = garage
|
||||
.system
|
||||
.get_known_nodes()
|
||||
.into_iter()
|
||||
.map(|i| {
|
||||
(
|
||||
i.id,
|
||||
NodeResp {
|
||||
id: hex::encode(i.id),
|
||||
addr: i.addr,
|
||||
hostname: i.status.hostname,
|
||||
is_up: i.is_up,
|
||||
last_seen_secs_ago: i.last_seen_secs_ago,
|
||||
data_partition: i
|
||||
.status
|
||||
.data_disk_avail
|
||||
.map(|(avail, total)| FreeSpaceResp {
|
||||
available: avail,
|
||||
total,
|
||||
impl RequestHandler for GetClusterStatusRequest {
|
||||
type Response = GetClusterStatusResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<GetClusterStatusResponse, Error> {
|
||||
let layout = garage.system.cluster_layout();
|
||||
let mut nodes = garage
|
||||
.system
|
||||
.get_known_nodes()
|
||||
.into_iter()
|
||||
.map(|i| {
|
||||
(
|
||||
i.id,
|
||||
NodeResp {
|
||||
id: hex::encode(i.id),
|
||||
garage_version: i.status.garage_version,
|
||||
addr: i.addr,
|
||||
hostname: i.status.hostname,
|
||||
is_up: i.is_up,
|
||||
last_seen_secs_ago: i.last_seen_secs_ago,
|
||||
data_partition: i.status.data_disk_avail.map(|(avail, total)| {
|
||||
FreeSpaceResp {
|
||||
available: avail,
|
||||
total,
|
||||
}
|
||||
}),
|
||||
metadata_partition: i.status.meta_disk_avail.map(|(avail, total)| {
|
||||
FreeSpaceResp {
|
||||
available: avail,
|
||||
total,
|
||||
}
|
||||
}),
|
||||
..Default::default()
|
||||
},
|
||||
)
|
||||
})
|
||||
.collect::<HashMap<_, _>>();
|
||||
metadata_partition: i.status.meta_disk_avail.map(|(avail, total)| {
|
||||
FreeSpaceResp {
|
||||
available: avail,
|
||||
total,
|
||||
}
|
||||
}),
|
||||
..Default::default()
|
||||
},
|
||||
)
|
||||
})
|
||||
.collect::<HashMap<_, _>>();
|
||||
|
||||
for (id, _, role) in layout.current().roles.items().iter() {
|
||||
if let layout::NodeRoleV(Some(r)) = role {
|
||||
let role = NodeRoleResp {
|
||||
id: hex::encode(id),
|
||||
zone: r.zone.to_string(),
|
||||
capacity: r.capacity,
|
||||
tags: r.tags.clone(),
|
||||
};
|
||||
match nodes.get_mut(id) {
|
||||
None => {
|
||||
nodes.insert(
|
||||
*id,
|
||||
NodeResp {
|
||||
id: hex::encode(id),
|
||||
role: Some(role),
|
||||
..Default::default()
|
||||
},
|
||||
);
|
||||
}
|
||||
Some(n) => {
|
||||
n.role = Some(role);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for ver in layout.versions().iter().rev().skip(1) {
|
||||
for (id, _, role) in ver.roles.items().iter() {
|
||||
for (id, _, role) in layout.current().roles.items().iter() {
|
||||
if let layout::NodeRoleV(Some(r)) = role {
|
||||
if r.capacity.is_some() {
|
||||
if let Some(n) = nodes.get_mut(id) {
|
||||
if n.role.is_none() {
|
||||
n.draining = true;
|
||||
}
|
||||
} else {
|
||||
let role = NodeAssignedRole {
|
||||
zone: r.zone.to_string(),
|
||||
capacity: r.capacity,
|
||||
tags: r.tags.clone(),
|
||||
};
|
||||
match nodes.get_mut(id) {
|
||||
None => {
|
||||
nodes.insert(
|
||||
*id,
|
||||
NodeResp {
|
||||
id: hex::encode(id),
|
||||
draining: true,
|
||||
role: Some(role),
|
||||
..Default::default()
|
||||
},
|
||||
);
|
||||
}
|
||||
Some(n) => {
|
||||
n.role = Some(role);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
let mut nodes = nodes.into_values().collect::<Vec<_>>();
|
||||
nodes.sort_by(|x, y| x.id.cmp(&y.id));
|
||||
for ver in layout.versions().iter().rev().skip(1) {
|
||||
for (id, _, role) in ver.roles.items().iter() {
|
||||
if let layout::NodeRoleV(Some(r)) = role {
|
||||
if r.capacity.is_some() {
|
||||
if let Some(n) = nodes.get_mut(id) {
|
||||
if n.role.is_none() {
|
||||
n.draining = true;
|
||||
}
|
||||
} else {
|
||||
nodes.insert(
|
||||
*id,
|
||||
NodeResp {
|
||||
id: hex::encode(id),
|
||||
draining: true,
|
||||
..Default::default()
|
||||
},
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
let res = GetClusterStatusResponse {
|
||||
node: hex::encode(garage.system.id),
|
||||
garage_version: garage_util::version::garage_version(),
|
||||
garage_features: garage_util::version::garage_features(),
|
||||
rust_version: garage_util::version::rust_version(),
|
||||
db_engine: garage.db.engine(),
|
||||
layout_version: layout.current().version,
|
||||
nodes,
|
||||
};
|
||||
let mut nodes = nodes.into_values().collect::<Vec<_>>();
|
||||
nodes.sort_by(|x, y| x.id.cmp(&y.id));
|
||||
|
||||
Ok(json_ok_response(&res)?)
|
||||
}
|
||||
|
||||
pub async fn handle_get_cluster_health(garage: &Arc<Garage>) -> Result<Response<ResBody>, Error> {
|
||||
use garage_rpc::system::ClusterHealthStatus;
|
||||
let health = garage.system.health();
|
||||
let health = ClusterHealth {
|
||||
status: match health.status {
|
||||
ClusterHealthStatus::Healthy => "healthy",
|
||||
ClusterHealthStatus::Degraded => "degraded",
|
||||
ClusterHealthStatus::Unavailable => "unavailable",
|
||||
},
|
||||
known_nodes: health.known_nodes,
|
||||
connected_nodes: health.connected_nodes,
|
||||
storage_nodes: health.storage_nodes,
|
||||
storage_nodes_ok: health.storage_nodes_ok,
|
||||
partitions: health.partitions,
|
||||
partitions_quorum: health.partitions_quorum,
|
||||
partitions_all_ok: health.partitions_all_ok,
|
||||
};
|
||||
Ok(json_ok_response(&health)?)
|
||||
}
|
||||
|
||||
pub async fn handle_connect_cluster_nodes(
|
||||
garage: &Arc<Garage>,
|
||||
req: Request<IncomingBody>,
|
||||
) -> Result<Response<ResBody>, Error> {
|
||||
let req = parse_json_body::<Vec<String>, _, Error>(req).await?;
|
||||
|
||||
let res = futures::future::join_all(req.iter().map(|node| garage.system.connect(node)))
|
||||
.await
|
||||
.into_iter()
|
||||
.map(|r| match r {
|
||||
Ok(()) => ConnectClusterNodesResponse {
|
||||
success: true,
|
||||
error: None,
|
||||
},
|
||||
Err(e) => ConnectClusterNodesResponse {
|
||||
success: false,
|
||||
error: Some(format!("{}", e)),
|
||||
},
|
||||
Ok(GetClusterStatusResponse {
|
||||
layout_version: layout.current().version,
|
||||
nodes,
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
Ok(json_ok_response(&res)?)
|
||||
}
|
||||
|
||||
pub async fn handle_get_cluster_layout(garage: &Arc<Garage>) -> Result<Response<ResBody>, Error> {
|
||||
let res = format_cluster_layout(garage.system.cluster_layout().inner());
|
||||
|
||||
Ok(json_ok_response(&res)?)
|
||||
}
|
||||
|
||||
fn format_cluster_layout(layout: &layout::LayoutHistory) -> GetClusterLayoutResponse {
|
||||
let roles = layout
|
||||
.current()
|
||||
.roles
|
||||
.items()
|
||||
.iter()
|
||||
.filter_map(|(k, _, v)| v.0.clone().map(|x| (k, x)))
|
||||
.map(|(k, v)| NodeRoleResp {
|
||||
id: hex::encode(k),
|
||||
zone: v.zone.clone(),
|
||||
capacity: v.capacity,
|
||||
tags: v.tags.clone(),
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
let staged_role_changes = layout
|
||||
.staging
|
||||
.get()
|
||||
.roles
|
||||
.items()
|
||||
.iter()
|
||||
.filter(|(k, _, v)| layout.current().roles.get(k) != Some(v))
|
||||
.map(|(k, _, v)| match &v.0 {
|
||||
None => NodeRoleChange {
|
||||
id: hex::encode(k),
|
||||
action: NodeRoleChangeEnum::Remove { remove: true },
|
||||
},
|
||||
Some(r) => NodeRoleChange {
|
||||
id: hex::encode(k),
|
||||
action: NodeRoleChangeEnum::Update {
|
||||
zone: r.zone.clone(),
|
||||
capacity: r.capacity,
|
||||
tags: r.tags.clone(),
|
||||
},
|
||||
},
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
GetClusterLayoutResponse {
|
||||
version: layout.current().version,
|
||||
roles,
|
||||
staged_role_changes,
|
||||
}
|
||||
}
|
||||
|
||||
// ----
|
||||
impl RequestHandler for GetClusterHealthRequest {
|
||||
type Response = GetClusterHealthResponse;
|
||||
|
||||
#[derive(Debug, Clone, Copy, Serialize)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
pub struct ClusterHealth {
|
||||
status: &'static str,
|
||||
known_nodes: usize,
|
||||
connected_nodes: usize,
|
||||
storage_nodes: usize,
|
||||
storage_nodes_ok: usize,
|
||||
partitions: usize,
|
||||
partitions_quorum: usize,
|
||||
partitions_all_ok: usize,
|
||||
}
|
||||
|
||||
#[derive(Serialize)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
struct GetClusterStatusResponse {
|
||||
node: String,
|
||||
garage_version: &'static str,
|
||||
garage_features: Option<&'static [&'static str]>,
|
||||
rust_version: &'static str,
|
||||
db_engine: String,
|
||||
layout_version: u64,
|
||||
nodes: Vec<NodeResp>,
|
||||
}
|
||||
|
||||
#[derive(Serialize)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
struct ApplyClusterLayoutResponse {
|
||||
message: Vec<String>,
|
||||
layout: GetClusterLayoutResponse,
|
||||
}
|
||||
|
||||
#[derive(Serialize)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
struct ConnectClusterNodesResponse {
|
||||
success: bool,
|
||||
error: Option<String>,
|
||||
}
|
||||
|
||||
#[derive(Serialize)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
struct GetClusterLayoutResponse {
|
||||
version: u64,
|
||||
roles: Vec<NodeRoleResp>,
|
||||
staged_role_changes: Vec<NodeRoleChange>,
|
||||
}
|
||||
|
||||
#[derive(Serialize)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
struct NodeRoleResp {
|
||||
id: String,
|
||||
zone: String,
|
||||
capacity: Option<u64>,
|
||||
tags: Vec<String>,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Default)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
struct FreeSpaceResp {
|
||||
available: u64,
|
||||
total: u64,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Default)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
struct NodeResp {
|
||||
id: String,
|
||||
role: Option<NodeRoleResp>,
|
||||
addr: Option<SocketAddr>,
|
||||
hostname: Option<String>,
|
||||
is_up: bool,
|
||||
last_seen_secs_ago: Option<u64>,
|
||||
draining: bool,
|
||||
#[serde(skip_serializing_if = "Option::is_none")]
|
||||
data_partition: Option<FreeSpaceResp>,
|
||||
#[serde(skip_serializing_if = "Option::is_none")]
|
||||
metadata_partition: Option<FreeSpaceResp>,
|
||||
}
|
||||
|
||||
// ---- update functions ----
|
||||
|
||||
pub async fn handle_update_cluster_layout(
|
||||
garage: &Arc<Garage>,
|
||||
req: Request<IncomingBody>,
|
||||
) -> Result<Response<ResBody>, Error> {
|
||||
let updates = parse_json_body::<UpdateClusterLayoutRequest, _, Error>(req).await?;
|
||||
|
||||
let mut layout = garage.system.cluster_layout().inner().clone();
|
||||
|
||||
let mut roles = layout.current().roles.clone();
|
||||
roles.merge(&layout.staging.get().roles);
|
||||
|
||||
for change in updates {
|
||||
let node = hex::decode(&change.id).ok_or_bad_request("Invalid node identifier")?;
|
||||
let node = Uuid::try_from(&node).ok_or_bad_request("Invalid node identifier")?;
|
||||
|
||||
let new_role = match change.action {
|
||||
NodeRoleChangeEnum::Remove { remove: true } => None,
|
||||
NodeRoleChangeEnum::Update {
|
||||
zone,
|
||||
capacity,
|
||||
tags,
|
||||
} => Some(layout::NodeRole {
|
||||
zone,
|
||||
capacity,
|
||||
tags,
|
||||
}),
|
||||
_ => return Err(Error::bad_request("Invalid layout change")),
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<GetClusterHealthResponse, Error> {
|
||||
use garage_rpc::system::ClusterHealthStatus;
|
||||
let health = garage.system.health();
|
||||
let health = GetClusterHealthResponse {
|
||||
status: match health.status {
|
||||
ClusterHealthStatus::Healthy => "healthy",
|
||||
ClusterHealthStatus::Degraded => "degraded",
|
||||
ClusterHealthStatus::Unavailable => "unavailable",
|
||||
}
|
||||
.to_string(),
|
||||
known_nodes: health.known_nodes,
|
||||
connected_nodes: health.connected_nodes,
|
||||
storage_nodes: health.storage_nodes,
|
||||
storage_nodes_ok: health.storage_nodes_ok,
|
||||
partitions: health.partitions,
|
||||
partitions_quorum: health.partitions_quorum,
|
||||
partitions_all_ok: health.partitions_all_ok,
|
||||
};
|
||||
|
||||
layout
|
||||
.staging
|
||||
.get_mut()
|
||||
.roles
|
||||
.merge(&roles.update_mutator(node, layout::NodeRoleV(new_role)));
|
||||
Ok(health)
|
||||
}
|
||||
|
||||
garage
|
||||
.system
|
||||
.layout_manager
|
||||
.update_cluster_layout(&layout)
|
||||
.await?;
|
||||
|
||||
let res = format_cluster_layout(&layout);
|
||||
Ok(json_ok_response(&res)?)
|
||||
}
|
||||
|
||||
pub async fn handle_apply_cluster_layout(
|
||||
garage: &Arc<Garage>,
|
||||
req: Request<IncomingBody>,
|
||||
) -> Result<Response<ResBody>, Error> {
|
||||
let param = parse_json_body::<ApplyLayoutRequest, _, Error>(req).await?;
|
||||
impl RequestHandler for GetClusterStatisticsRequest {
|
||||
type Response = GetClusterStatisticsResponse;
|
||||
|
||||
let layout = garage.system.cluster_layout().inner().clone();
|
||||
let (layout, msg) = layout.apply_staged_changes(Some(param.version))?;
|
||||
// FIXME: return this as a JSON struct instead of text
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<GetClusterStatisticsResponse, Error> {
|
||||
let mut ret = String::new();
|
||||
|
||||
garage
|
||||
.system
|
||||
.layout_manager
|
||||
.update_cluster_layout(&layout)
|
||||
.await?;
|
||||
// Gather storage node and free space statistics for current nodes
|
||||
let layout = &garage.system.cluster_layout();
|
||||
let mut node_partition_count = HashMap::<Uuid, u64>::new();
|
||||
for short_id in layout.current().ring_assignment_data.iter() {
|
||||
let id = layout.current().node_id_vec[*short_id as usize];
|
||||
*node_partition_count.entry(id).or_default() += 1;
|
||||
}
|
||||
let node_info = garage
|
||||
.system
|
||||
.get_known_nodes()
|
||||
.into_iter()
|
||||
.map(|n| (n.id, n))
|
||||
.collect::<HashMap<_, _>>();
|
||||
|
||||
let res = ApplyClusterLayoutResponse {
|
||||
message: msg,
|
||||
layout: format_cluster_layout(&layout),
|
||||
};
|
||||
Ok(json_ok_response(&res)?)
|
||||
let mut table = vec![" ID\tHostname\tZone\tCapacity\tPart.\tDataAvail\tMetaAvail".into()];
|
||||
for (id, parts) in node_partition_count.iter() {
|
||||
let info = node_info.get(id);
|
||||
let status = info.map(|x| &x.status);
|
||||
let role = layout.current().roles.get(id).and_then(|x| x.0.as_ref());
|
||||
let hostname = status.and_then(|x| x.hostname.as_deref()).unwrap_or("?");
|
||||
let zone = role.map(|x| x.zone.as_str()).unwrap_or("?");
|
||||
let capacity = role
|
||||
.map(|x| x.capacity_string())
|
||||
.unwrap_or_else(|| "?".into());
|
||||
let avail_str = |x| match x {
|
||||
Some((avail, total)) => {
|
||||
let pct = (avail as f64) / (total as f64) * 100.;
|
||||
let avail = bytesize::ByteSize::b(avail);
|
||||
let total = bytesize::ByteSize::b(total);
|
||||
format!("{}/{} ({:.1}%)", avail, total, pct)
|
||||
}
|
||||
None => "?".into(),
|
||||
};
|
||||
let data_avail = avail_str(status.and_then(|x| x.data_disk_avail));
|
||||
let meta_avail = avail_str(status.and_then(|x| x.meta_disk_avail));
|
||||
table.push(format!(
|
||||
" {:?}\t{}\t{}\t{}\t{}\t{}\t{}",
|
||||
id, hostname, zone, capacity, parts, data_avail, meta_avail
|
||||
));
|
||||
}
|
||||
write!(
|
||||
&mut ret,
|
||||
"Storage nodes:\n{}",
|
||||
format_table_to_string(table)
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
let meta_part_avail = node_partition_count
|
||||
.iter()
|
||||
.filter_map(|(id, parts)| {
|
||||
node_info
|
||||
.get(id)
|
||||
.and_then(|x| x.status.meta_disk_avail)
|
||||
.map(|c| c.0 / *parts)
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
let data_part_avail = node_partition_count
|
||||
.iter()
|
||||
.filter_map(|(id, parts)| {
|
||||
node_info
|
||||
.get(id)
|
||||
.and_then(|x| x.status.data_disk_avail)
|
||||
.map(|c| c.0 / *parts)
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
if !meta_part_avail.is_empty() && !data_part_avail.is_empty() {
|
||||
let meta_avail =
|
||||
bytesize::ByteSize(meta_part_avail.iter().min().unwrap() * (1 << PARTITION_BITS));
|
||||
let data_avail =
|
||||
bytesize::ByteSize(data_part_avail.iter().min().unwrap() * (1 << PARTITION_BITS));
|
||||
writeln!(
|
||||
&mut ret,
|
||||
"\nEstimated available storage space cluster-wide (might be lower in practice):"
|
||||
)
|
||||
.unwrap();
|
||||
if meta_part_avail.len() < node_partition_count.len()
|
||||
|| data_part_avail.len() < node_partition_count.len()
|
||||
{
|
||||
ret += &format_table_to_string(vec![
|
||||
format!(" data: < {}", data_avail),
|
||||
format!(" metadata: < {}", meta_avail),
|
||||
]);
|
||||
writeln!(&mut ret, "A precise estimate could not be given as information is missing for some storage nodes.").unwrap();
|
||||
} else {
|
||||
ret += &format_table_to_string(vec![
|
||||
format!(" data: {}", data_avail),
|
||||
format!(" metadata: {}", meta_avail),
|
||||
]);
|
||||
}
|
||||
}
|
||||
|
||||
Ok(GetClusterStatisticsResponse { freeform: ret })
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn handle_revert_cluster_layout(
|
||||
garage: &Arc<Garage>,
|
||||
) -> Result<Response<ResBody>, Error> {
|
||||
let layout = garage.system.cluster_layout().inner().clone();
|
||||
let layout = layout.revert_staged_changes()?;
|
||||
garage
|
||||
.system
|
||||
.layout_manager
|
||||
.update_cluster_layout(&layout)
|
||||
.await?;
|
||||
impl RequestHandler for ConnectClusterNodesRequest {
|
||||
type Response = ConnectClusterNodesResponse;
|
||||
|
||||
let res = format_cluster_layout(&layout);
|
||||
Ok(json_ok_response(&res)?)
|
||||
}
|
||||
|
||||
// ----
|
||||
|
||||
type UpdateClusterLayoutRequest = Vec<NodeRoleChange>;
|
||||
|
||||
#[derive(Deserialize)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
struct ApplyLayoutRequest {
|
||||
version: u64,
|
||||
}
|
||||
|
||||
// ----
|
||||
|
||||
#[derive(Serialize, Deserialize)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
struct NodeRoleChange {
|
||||
id: String,
|
||||
#[serde(flatten)]
|
||||
action: NodeRoleChangeEnum,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize)]
|
||||
#[serde(untagged)]
|
||||
enum NodeRoleChangeEnum {
|
||||
#[serde(rename_all = "camelCase")]
|
||||
Remove { remove: bool },
|
||||
#[serde(rename_all = "camelCase")]
|
||||
Update {
|
||||
zone: String,
|
||||
capacity: Option<u64>,
|
||||
tags: Vec<String>,
|
||||
},
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<ConnectClusterNodesResponse, Error> {
|
||||
let res = futures::future::join_all(self.0.iter().map(|node| garage.system.connect(node)))
|
||||
.await
|
||||
.into_iter()
|
||||
.map(|r| match r {
|
||||
Ok(()) => ConnectNodeResponse {
|
||||
success: true,
|
||||
error: None,
|
||||
},
|
||||
Err(e) => ConnectNodeResponse {
|
||||
success: false,
|
||||
error: Some(format!("{}", e)),
|
||||
},
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
Ok(ConnectClusterNodesResponse(res))
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,10 +21,26 @@ pub enum Error {
|
|||
Common(#[error(source)] CommonError),
|
||||
|
||||
// Category: cannot process
|
||||
/// The admin API token does not exist
|
||||
#[error(display = "Admin token not found: {}", _0)]
|
||||
NoSuchAdminToken(String),
|
||||
|
||||
/// The API access key does not exist
|
||||
#[error(display = "Access key not found: {}", _0)]
|
||||
NoSuchAccessKey(String),
|
||||
|
||||
/// The requested block does not exist
|
||||
#[error(display = "Block not found: {}", _0)]
|
||||
NoSuchBlock(String),
|
||||
|
||||
/// The requested worker does not exist
|
||||
#[error(display = "Worker not found: {}", _0)]
|
||||
NoSuchWorker(u64),
|
||||
|
||||
/// The object requested don't exists
|
||||
#[error(display = "Key not found")]
|
||||
NoSuchKey,
|
||||
|
||||
/// In Import key, the key already exists
|
||||
#[error(
|
||||
display = "Key {} already exists in data store. Even if it is deleted, we can't let you create a new key with the same ID. Sorry.",
|
||||
|
@ -49,11 +65,15 @@ impl From<HelperError> for Error {
|
|||
}
|
||||
|
||||
impl Error {
|
||||
fn code(&self) -> &'static str {
|
||||
pub fn code(&self) -> &'static str {
|
||||
match self {
|
||||
Error::Common(c) => c.aws_code(),
|
||||
Error::NoSuchAdminToken(_) => "NoSuchAdminToken",
|
||||
Error::NoSuchAccessKey(_) => "NoSuchAccessKey",
|
||||
Error::NoSuchWorker(_) => "NoSuchWorker",
|
||||
Error::NoSuchBlock(_) => "NoSuchBlock",
|
||||
Error::KeyAlreadyExists(_) => "KeyAlreadyExists",
|
||||
Error::NoSuchKey => "NoSuchKey",
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -63,7 +83,11 @@ impl ApiError for Error {
|
|||
fn http_status_code(&self) -> StatusCode {
|
||||
match self {
|
||||
Error::Common(c) => c.http_status_code(),
|
||||
Error::NoSuchAccessKey(_) => StatusCode::NOT_FOUND,
|
||||
Error::NoSuchAdminToken(_)
|
||||
| Error::NoSuchAccessKey(_)
|
||||
| Error::NoSuchWorker(_)
|
||||
| Error::NoSuchBlock(_)
|
||||
| Error::NoSuchKey => StatusCode::NOT_FOUND,
|
||||
Error::KeyAlreadyExists(_) => StatusCode::CONFLICT,
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,173 +1,183 @@
|
|||
use std::collections::HashMap;
|
||||
use std::sync::Arc;
|
||||
|
||||
use hyper::{body::Incoming as IncomingBody, Request, Response, StatusCode};
|
||||
use serde::{Deserialize, Serialize};
|
||||
|
||||
use garage_table::*;
|
||||
|
||||
use garage_model::garage::Garage;
|
||||
use garage_model::key_table::*;
|
||||
|
||||
use garage_api_common::helpers::*;
|
||||
|
||||
use crate::api_server::ResBody;
|
||||
use crate::api::*;
|
||||
use crate::error::*;
|
||||
use crate::{Admin, RequestHandler};
|
||||
|
||||
pub async fn handle_list_keys(garage: &Arc<Garage>) -> Result<Response<ResBody>, Error> {
|
||||
let res = garage
|
||||
.key_table
|
||||
.get_range(
|
||||
&EmptyKey,
|
||||
None,
|
||||
Some(KeyFilter::Deleted(DeletedFilter::NotDeleted)),
|
||||
10000,
|
||||
EnumerationOrder::Forward,
|
||||
)
|
||||
.await?
|
||||
.iter()
|
||||
.map(|k| ListKeyResultItem {
|
||||
id: k.key_id.to_string(),
|
||||
name: k.params().unwrap().name.get().clone(),
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
impl RequestHandler for ListKeysRequest {
|
||||
type Response = ListKeysResponse;
|
||||
|
||||
Ok(json_ok_response(&res)?)
|
||||
}
|
||||
|
||||
#[derive(Serialize)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
struct ListKeyResultItem {
|
||||
id: String,
|
||||
name: String,
|
||||
}
|
||||
|
||||
pub async fn handle_get_key_info(
|
||||
garage: &Arc<Garage>,
|
||||
id: Option<String>,
|
||||
search: Option<String>,
|
||||
show_secret_key: bool,
|
||||
) -> Result<Response<ResBody>, Error> {
|
||||
let key = if let Some(id) = id {
|
||||
garage.key_helper().get_existing_key(&id).await?
|
||||
} else if let Some(search) = search {
|
||||
garage
|
||||
.key_helper()
|
||||
.get_existing_matching_key(&search)
|
||||
async fn handle(self, garage: &Arc<Garage>, _admin: &Admin) -> Result<ListKeysResponse, Error> {
|
||||
let res = garage
|
||||
.key_table
|
||||
.get_range(
|
||||
&EmptyKey,
|
||||
None,
|
||||
Some(KeyFilter::Deleted(DeletedFilter::NotDeleted)),
|
||||
10000,
|
||||
EnumerationOrder::Forward,
|
||||
)
|
||||
.await?
|
||||
} else {
|
||||
unreachable!();
|
||||
};
|
||||
.iter()
|
||||
.map(|k| ListKeysResponseItem {
|
||||
id: k.key_id.to_string(),
|
||||
name: k.params().unwrap().name.get().clone(),
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
key_info_results(garage, key, show_secret_key).await
|
||||
}
|
||||
|
||||
pub async fn handle_create_key(
|
||||
garage: &Arc<Garage>,
|
||||
req: Request<IncomingBody>,
|
||||
) -> Result<Response<ResBody>, Error> {
|
||||
let req = parse_json_body::<CreateKeyRequest, _, Error>(req).await?;
|
||||
|
||||
let key = Key::new(req.name.as_deref().unwrap_or("Unnamed key"));
|
||||
garage.key_table.insert(&key).await?;
|
||||
|
||||
key_info_results(garage, key, true).await
|
||||
}
|
||||
|
||||
#[derive(Deserialize)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
struct CreateKeyRequest {
|
||||
name: Option<String>,
|
||||
}
|
||||
|
||||
pub async fn handle_import_key(
|
||||
garage: &Arc<Garage>,
|
||||
req: Request<IncomingBody>,
|
||||
) -> Result<Response<ResBody>, Error> {
|
||||
let req = parse_json_body::<ImportKeyRequest, _, Error>(req).await?;
|
||||
|
||||
let prev_key = garage.key_table.get(&EmptyKey, &req.access_key_id).await?;
|
||||
if prev_key.is_some() {
|
||||
return Err(Error::KeyAlreadyExists(req.access_key_id.to_string()));
|
||||
Ok(ListKeysResponse(res))
|
||||
}
|
||||
|
||||
let imported_key = Key::import(
|
||||
&req.access_key_id,
|
||||
&req.secret_access_key,
|
||||
req.name.as_deref().unwrap_or("Imported key"),
|
||||
)
|
||||
.ok_or_bad_request("Invalid key format")?;
|
||||
garage.key_table.insert(&imported_key).await?;
|
||||
|
||||
key_info_results(garage, imported_key, false).await
|
||||
}
|
||||
|
||||
#[derive(Deserialize)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
struct ImportKeyRequest {
|
||||
access_key_id: String,
|
||||
secret_access_key: String,
|
||||
name: Option<String>,
|
||||
}
|
||||
impl RequestHandler for GetKeyInfoRequest {
|
||||
type Response = GetKeyInfoResponse;
|
||||
|
||||
pub async fn handle_update_key(
|
||||
garage: &Arc<Garage>,
|
||||
id: String,
|
||||
req: Request<IncomingBody>,
|
||||
) -> Result<Response<ResBody>, Error> {
|
||||
let req = parse_json_body::<UpdateKeyRequest, _, Error>(req).await?;
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<GetKeyInfoResponse, Error> {
|
||||
let key = match (self.id, self.search) {
|
||||
(Some(id), None) => garage.key_helper().get_existing_key(&id).await?,
|
||||
(None, Some(search)) => {
|
||||
let candidates = garage
|
||||
.key_table
|
||||
.get_range(
|
||||
&EmptyKey,
|
||||
None,
|
||||
Some(KeyFilter::MatchesAndNotDeleted(search.to_string())),
|
||||
10,
|
||||
EnumerationOrder::Forward,
|
||||
)
|
||||
.await?
|
||||
.into_iter()
|
||||
.collect::<Vec<_>>();
|
||||
if candidates.len() != 1 {
|
||||
return Err(Error::bad_request(format!(
|
||||
"{} matching keys",
|
||||
candidates.len()
|
||||
)));
|
||||
}
|
||||
candidates.into_iter().next().unwrap()
|
||||
}
|
||||
_ => {
|
||||
return Err(Error::bad_request(
|
||||
"Either id or search must be provided (but not both)",
|
||||
));
|
||||
}
|
||||
};
|
||||
|
||||
let mut key = garage.key_helper().get_existing_key(&id).await?;
|
||||
|
||||
let key_state = key.state.as_option_mut().unwrap();
|
||||
|
||||
if let Some(new_name) = req.name {
|
||||
key_state.name.update(new_name);
|
||||
Ok(key_info_results(garage, key, self.show_secret_key).await?)
|
||||
}
|
||||
if let Some(allow) = req.allow {
|
||||
if allow.create_bucket {
|
||||
key_state.allow_create_bucket.update(true);
|
||||
}
|
||||
|
||||
impl RequestHandler for CreateKeyRequest {
|
||||
type Response = CreateKeyResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<CreateKeyResponse, Error> {
|
||||
let key = Key::new(self.name.as_deref().unwrap_or("Unnamed key"));
|
||||
garage.key_table.insert(&key).await?;
|
||||
|
||||
Ok(CreateKeyResponse(
|
||||
key_info_results(garage, key, true).await?,
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for ImportKeyRequest {
|
||||
type Response = ImportKeyResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<ImportKeyResponse, Error> {
|
||||
let prev_key = garage.key_table.get(&EmptyKey, &self.access_key_id).await?;
|
||||
if prev_key.is_some() {
|
||||
return Err(Error::KeyAlreadyExists(self.access_key_id.to_string()));
|
||||
}
|
||||
|
||||
let imported_key = Key::import(
|
||||
&self.access_key_id,
|
||||
&self.secret_access_key,
|
||||
self.name.as_deref().unwrap_or("Imported key"),
|
||||
)
|
||||
.ok_or_bad_request("Invalid key format")?;
|
||||
garage.key_table.insert(&imported_key).await?;
|
||||
|
||||
Ok(ImportKeyResponse(
|
||||
key_info_results(garage, imported_key, false).await?,
|
||||
))
|
||||
}
|
||||
if let Some(deny) = req.deny {
|
||||
if deny.create_bucket {
|
||||
key_state.allow_create_bucket.update(false);
|
||||
}
|
||||
|
||||
impl RequestHandler for UpdateKeyRequest {
|
||||
type Response = UpdateKeyResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<UpdateKeyResponse, Error> {
|
||||
let mut key = garage.key_helper().get_existing_key(&self.id).await?;
|
||||
|
||||
let key_state = key.state.as_option_mut().unwrap();
|
||||
|
||||
if let Some(new_name) = self.body.name {
|
||||
key_state.name.update(new_name);
|
||||
}
|
||||
if let Some(allow) = self.body.allow {
|
||||
if allow.create_bucket {
|
||||
key_state.allow_create_bucket.update(true);
|
||||
}
|
||||
}
|
||||
if let Some(deny) = self.body.deny {
|
||||
if deny.create_bucket {
|
||||
key_state.allow_create_bucket.update(false);
|
||||
}
|
||||
}
|
||||
|
||||
garage.key_table.insert(&key).await?;
|
||||
|
||||
Ok(UpdateKeyResponse(
|
||||
key_info_results(garage, key, false).await?,
|
||||
))
|
||||
}
|
||||
|
||||
garage.key_table.insert(&key).await?;
|
||||
|
||||
key_info_results(garage, key, false).await
|
||||
}
|
||||
|
||||
#[derive(Deserialize)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
struct UpdateKeyRequest {
|
||||
name: Option<String>,
|
||||
allow: Option<KeyPerm>,
|
||||
deny: Option<KeyPerm>,
|
||||
}
|
||||
impl RequestHandler for DeleteKeyRequest {
|
||||
type Response = DeleteKeyResponse;
|
||||
|
||||
pub async fn handle_delete_key(
|
||||
garage: &Arc<Garage>,
|
||||
id: String,
|
||||
) -> Result<Response<ResBody>, Error> {
|
||||
let helper = garage.locked_helper().await;
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<DeleteKeyResponse, Error> {
|
||||
let helper = garage.locked_helper().await;
|
||||
|
||||
let mut key = helper.key().get_existing_key(&id).await?;
|
||||
let mut key = helper.key().get_existing_key(&self.id).await?;
|
||||
|
||||
helper.delete_key(&mut key).await?;
|
||||
helper.delete_key(&mut key).await?;
|
||||
|
||||
Ok(Response::builder()
|
||||
.status(StatusCode::NO_CONTENT)
|
||||
.body(empty_body())?)
|
||||
Ok(DeleteKeyResponse)
|
||||
}
|
||||
}
|
||||
|
||||
async fn key_info_results(
|
||||
garage: &Arc<Garage>,
|
||||
key: Key,
|
||||
show_secret: bool,
|
||||
) -> Result<Response<ResBody>, Error> {
|
||||
) -> Result<GetKeyInfoResponse, Error> {
|
||||
let mut relevant_buckets = HashMap::new();
|
||||
|
||||
let key_state = key.state.as_option().unwrap();
|
||||
|
@ -193,7 +203,7 @@ async fn key_info_results(
|
|||
}
|
||||
}
|
||||
|
||||
let res = GetKeyInfoResult {
|
||||
let res = GetKeyInfoResponse {
|
||||
name: key_state.name.get().clone(),
|
||||
access_key_id: key.key_id.clone(),
|
||||
secret_access_key: if show_secret {
|
||||
|
@ -208,7 +218,7 @@ async fn key_info_results(
|
|||
.into_values()
|
||||
.map(|bucket| {
|
||||
let state = bucket.state.as_option().unwrap();
|
||||
KeyInfoBucketResult {
|
||||
KeyInfoBucketResponse {
|
||||
id: hex::encode(bucket.id),
|
||||
global_aliases: state
|
||||
.aliases
|
||||
|
@ -238,43 +248,5 @@ async fn key_info_results(
|
|||
.collect::<Vec<_>>(),
|
||||
};
|
||||
|
||||
Ok(json_ok_response(&res)?)
|
||||
}
|
||||
|
||||
#[derive(Serialize)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
struct GetKeyInfoResult {
|
||||
name: String,
|
||||
access_key_id: String,
|
||||
#[serde(skip_serializing_if = "is_default")]
|
||||
secret_access_key: Option<String>,
|
||||
permissions: KeyPerm,
|
||||
buckets: Vec<KeyInfoBucketResult>,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
struct KeyPerm {
|
||||
#[serde(default)]
|
||||
create_bucket: bool,
|
||||
}
|
||||
|
||||
#[derive(Serialize)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
struct KeyInfoBucketResult {
|
||||
id: String,
|
||||
global_aliases: Vec<String>,
|
||||
local_aliases: Vec<String>,
|
||||
permissions: ApiBucketKeyPerm,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, Default)]
|
||||
#[serde(rename_all = "camelCase")]
|
||||
pub(crate) struct ApiBucketKeyPerm {
|
||||
#[serde(default)]
|
||||
pub(crate) read: bool,
|
||||
#[serde(default)]
|
||||
pub(crate) write: bool,
|
||||
#[serde(default)]
|
||||
pub(crate) owner: bool,
|
||||
Ok(res)
|
||||
}
|
||||
|
|
406
src/api/admin/layout.rs
Normal file
406
src/api/admin/layout.rs
Normal file
|
@ -0,0 +1,406 @@
|
|||
use std::sync::Arc;
|
||||
|
||||
use garage_util::crdt::*;
|
||||
use garage_util::data::*;
|
||||
use garage_util::error::Error as GarageError;
|
||||
|
||||
use garage_rpc::layout;
|
||||
|
||||
use garage_model::garage::Garage;
|
||||
|
||||
use crate::api::*;
|
||||
use crate::error::*;
|
||||
use crate::{Admin, RequestHandler};
|
||||
|
||||
impl RequestHandler for GetClusterLayoutRequest {
|
||||
type Response = GetClusterLayoutResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<GetClusterLayoutResponse, Error> {
|
||||
Ok(format_cluster_layout(
|
||||
garage.system.cluster_layout().inner(),
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
fn format_cluster_layout(layout: &layout::LayoutHistory) -> GetClusterLayoutResponse {
|
||||
let current = layout.current();
|
||||
|
||||
let roles = current
|
||||
.roles
|
||||
.items()
|
||||
.iter()
|
||||
.filter_map(|(k, _, v)| v.0.clone().map(|x| (k, x)))
|
||||
.map(|(k, v)| {
|
||||
let stored_partitions = current.get_node_usage(k).ok().map(|x| x as u64);
|
||||
LayoutNodeRole {
|
||||
id: hex::encode(k),
|
||||
zone: v.zone.clone(),
|
||||
capacity: v.capacity,
|
||||
stored_partitions,
|
||||
usable_capacity: stored_partitions.map(|x| x * current.partition_size),
|
||||
tags: v.tags.clone(),
|
||||
}
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
let staged_role_changes = layout
|
||||
.staging
|
||||
.get()
|
||||
.roles
|
||||
.items()
|
||||
.iter()
|
||||
.filter(|(k, _, v)| current.roles.get(k) != Some(v))
|
||||
.map(|(k, _, v)| match &v.0 {
|
||||
None => NodeRoleChange {
|
||||
id: hex::encode(k),
|
||||
action: NodeRoleChangeEnum::Remove { remove: true },
|
||||
},
|
||||
Some(r) => NodeRoleChange {
|
||||
id: hex::encode(k),
|
||||
action: NodeRoleChangeEnum::Update(NodeAssignedRole {
|
||||
zone: r.zone.clone(),
|
||||
capacity: r.capacity,
|
||||
tags: r.tags.clone(),
|
||||
}),
|
||||
},
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
let staged_parameters = if *layout.staging.get().parameters.get() != current.parameters {
|
||||
Some((*layout.staging.get().parameters.get()).into())
|
||||
} else {
|
||||
None
|
||||
};
|
||||
|
||||
GetClusterLayoutResponse {
|
||||
version: current.version,
|
||||
roles,
|
||||
partition_size: current.partition_size,
|
||||
parameters: current.parameters.into(),
|
||||
staged_role_changes,
|
||||
staged_parameters,
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for GetClusterLayoutHistoryRequest {
|
||||
type Response = GetClusterLayoutHistoryResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<GetClusterLayoutHistoryResponse, Error> {
|
||||
let layout_helper = garage.system.cluster_layout();
|
||||
let layout = layout_helper.inner();
|
||||
let min_stored = layout.min_stored();
|
||||
|
||||
let versions = layout
|
||||
.versions
|
||||
.iter()
|
||||
.rev()
|
||||
.chain(layout.old_versions.iter().rev())
|
||||
.map(|ver| {
|
||||
let status = if ver.version == layout.current().version {
|
||||
ClusterLayoutVersionStatus::Current
|
||||
} else if ver.version >= min_stored {
|
||||
ClusterLayoutVersionStatus::Draining
|
||||
} else {
|
||||
ClusterLayoutVersionStatus::Historical
|
||||
};
|
||||
ClusterLayoutVersion {
|
||||
version: ver.version,
|
||||
status,
|
||||
storage_nodes: ver
|
||||
.roles
|
||||
.items()
|
||||
.iter()
|
||||
.filter(
|
||||
|(_, _, x)| matches!(x, layout::NodeRoleV(Some(c)) if c.capacity.is_some()),
|
||||
)
|
||||
.count() as u64,
|
||||
gateway_nodes: ver
|
||||
.roles
|
||||
.items()
|
||||
.iter()
|
||||
.filter(
|
||||
|(_, _, x)| matches!(x, layout::NodeRoleV(Some(c)) if c.capacity.is_none()),
|
||||
)
|
||||
.count() as u64,
|
||||
}
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
|
||||
let all_nodes = layout.get_all_nodes();
|
||||
let min_ack = layout_helper.ack_map_min();
|
||||
|
||||
let update_trackers = if layout.versions.len() > 1 {
|
||||
Some(
|
||||
all_nodes
|
||||
.iter()
|
||||
.map(|node| {
|
||||
(
|
||||
hex::encode(&node),
|
||||
NodeUpdateTrackers {
|
||||
ack: layout.update_trackers.ack_map.get(node, min_stored),
|
||||
sync: layout.update_trackers.sync_map.get(node, min_stored),
|
||||
sync_ack: layout.update_trackers.sync_ack_map.get(node, min_stored),
|
||||
},
|
||||
)
|
||||
})
|
||||
.collect(),
|
||||
)
|
||||
} else {
|
||||
None
|
||||
};
|
||||
|
||||
Ok(GetClusterLayoutHistoryResponse {
|
||||
current_version: layout.current().version,
|
||||
min_ack,
|
||||
versions,
|
||||
update_trackers,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
// ----
|
||||
|
||||
// ---- update functions ----
|
||||
|
||||
impl RequestHandler for UpdateClusterLayoutRequest {
|
||||
type Response = UpdateClusterLayoutResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<UpdateClusterLayoutResponse, Error> {
|
||||
let mut layout = garage.system.cluster_layout().inner().clone();
|
||||
|
||||
let mut roles = layout.current().roles.clone();
|
||||
roles.merge(&layout.staging.get().roles);
|
||||
|
||||
for change in self.roles {
|
||||
let node = hex::decode(&change.id).ok_or_bad_request("Invalid node identifier")?;
|
||||
let node = Uuid::try_from(&node).ok_or_bad_request("Invalid node identifier")?;
|
||||
|
||||
let new_role = match change.action {
|
||||
NodeRoleChangeEnum::Remove { remove: true } => None,
|
||||
NodeRoleChangeEnum::Update(NodeAssignedRole {
|
||||
zone,
|
||||
capacity,
|
||||
tags,
|
||||
}) => {
|
||||
if matches!(capacity, Some(cap) if cap < 1024) {
|
||||
return Err(Error::bad_request("Capacity should be at least 1K (1024)"));
|
||||
}
|
||||
Some(layout::NodeRole {
|
||||
zone,
|
||||
capacity,
|
||||
tags,
|
||||
})
|
||||
}
|
||||
_ => return Err(Error::bad_request("Invalid layout change")),
|
||||
};
|
||||
|
||||
layout
|
||||
.staging
|
||||
.get_mut()
|
||||
.roles
|
||||
.merge(&roles.update_mutator(node, layout::NodeRoleV(new_role)));
|
||||
}
|
||||
|
||||
if let Some(param) = self.parameters {
|
||||
if let ZoneRedundancy::AtLeast(r_int) = param.zone_redundancy {
|
||||
if r_int > layout.current().replication_factor {
|
||||
return Err(Error::bad_request(format!(
|
||||
"The zone redundancy must be smaller or equal to the replication factor ({}).",
|
||||
layout.current().replication_factor
|
||||
)));
|
||||
} else if r_int < 1 {
|
||||
return Err(Error::bad_request(
|
||||
"The zone redundancy must be at least 1.",
|
||||
));
|
||||
}
|
||||
}
|
||||
layout.staging.get_mut().parameters.update(param.into());
|
||||
}
|
||||
|
||||
garage
|
||||
.system
|
||||
.layout_manager
|
||||
.update_cluster_layout(&layout)
|
||||
.await?;
|
||||
|
||||
let res = format_cluster_layout(&layout);
|
||||
Ok(UpdateClusterLayoutResponse(res))
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for PreviewClusterLayoutChangesRequest {
|
||||
type Response = PreviewClusterLayoutChangesResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<PreviewClusterLayoutChangesResponse, Error> {
|
||||
let layout = garage.system.cluster_layout().inner().clone();
|
||||
let new_ver = layout.current().version + 1;
|
||||
match layout.apply_staged_changes(new_ver) {
|
||||
Err(GarageError::Message(error)) => {
|
||||
Ok(PreviewClusterLayoutChangesResponse::Error { error })
|
||||
}
|
||||
Err(e) => Err(e.into()),
|
||||
Ok((new_layout, msg)) => Ok(PreviewClusterLayoutChangesResponse::Success {
|
||||
message: msg,
|
||||
new_layout: format_cluster_layout(&new_layout),
|
||||
}),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for ApplyClusterLayoutRequest {
|
||||
type Response = ApplyClusterLayoutResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<ApplyClusterLayoutResponse, Error> {
|
||||
let layout = garage.system.cluster_layout().inner().clone();
|
||||
let (layout, msg) = layout.apply_staged_changes(self.version)?;
|
||||
|
||||
garage
|
||||
.system
|
||||
.layout_manager
|
||||
.update_cluster_layout(&layout)
|
||||
.await?;
|
||||
|
||||
Ok(ApplyClusterLayoutResponse {
|
||||
message: msg,
|
||||
layout: format_cluster_layout(&layout),
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for RevertClusterLayoutRequest {
|
||||
type Response = RevertClusterLayoutResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<RevertClusterLayoutResponse, Error> {
|
||||
let layout = garage.system.cluster_layout().inner().clone();
|
||||
let layout = layout.revert_staged_changes()?;
|
||||
garage
|
||||
.system
|
||||
.layout_manager
|
||||
.update_cluster_layout(&layout)
|
||||
.await?;
|
||||
|
||||
let res = format_cluster_layout(&layout);
|
||||
Ok(RevertClusterLayoutResponse(res))
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for ClusterLayoutSkipDeadNodesRequest {
|
||||
type Response = ClusterLayoutSkipDeadNodesResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<ClusterLayoutSkipDeadNodesResponse, Error> {
|
||||
let status = garage.system.get_known_nodes();
|
||||
|
||||
let mut layout = garage.system.cluster_layout().inner().clone();
|
||||
let mut ack_updated = vec![];
|
||||
let mut sync_updated = vec![];
|
||||
|
||||
if layout.versions.len() == 1 {
|
||||
return Err(Error::bad_request(
|
||||
"This command cannot be called when there is only one live cluster layout version",
|
||||
));
|
||||
}
|
||||
|
||||
let min_v = layout.min_stored();
|
||||
if self.version <= min_v || self.version > layout.current().version {
|
||||
return Err(Error::bad_request(format!(
|
||||
"Invalid version, you may use the following version numbers: {}",
|
||||
(min_v + 1..=layout.current().version)
|
||||
.map(|x| x.to_string())
|
||||
.collect::<Vec<_>>()
|
||||
.join(" ")
|
||||
)));
|
||||
}
|
||||
|
||||
let all_nodes = layout.get_all_nodes();
|
||||
for node in all_nodes.iter() {
|
||||
// Update ACK tracker for dead nodes or for all nodes if --allow-missing-data
|
||||
if self.allow_missing_data || !status.iter().any(|x| x.id == *node && x.is_up) {
|
||||
if layout.update_trackers.ack_map.set_max(*node, self.version) {
|
||||
ack_updated.push(hex::encode(node));
|
||||
}
|
||||
}
|
||||
|
||||
// If --allow-missing-data, update SYNC tracker for all nodes.
|
||||
if self.allow_missing_data {
|
||||
if layout.update_trackers.sync_map.set_max(*node, self.version) {
|
||||
sync_updated.push(hex::encode(node));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
garage
|
||||
.system
|
||||
.layout_manager
|
||||
.update_cluster_layout(&layout)
|
||||
.await?;
|
||||
|
||||
Ok(ClusterLayoutSkipDeadNodesResponse {
|
||||
ack_updated,
|
||||
sync_updated,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
// ----
|
||||
|
||||
impl From<layout::ZoneRedundancy> for ZoneRedundancy {
|
||||
fn from(x: layout::ZoneRedundancy) -> Self {
|
||||
match x {
|
||||
layout::ZoneRedundancy::Maximum => ZoneRedundancy::Maximum,
|
||||
layout::ZoneRedundancy::AtLeast(x) => ZoneRedundancy::AtLeast(x),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Into<layout::ZoneRedundancy> for ZoneRedundancy {
|
||||
fn into(self) -> layout::ZoneRedundancy {
|
||||
match self {
|
||||
ZoneRedundancy::Maximum => layout::ZoneRedundancy::Maximum,
|
||||
ZoneRedundancy::AtLeast(x) => layout::ZoneRedundancy::AtLeast(x),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl From<layout::LayoutParameters> for LayoutParameters {
|
||||
fn from(x: layout::LayoutParameters) -> Self {
|
||||
LayoutParameters {
|
||||
zone_redundancy: x.zone_redundancy.into(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Into<layout::LayoutParameters> for LayoutParameters {
|
||||
fn into(self) -> layout::LayoutParameters {
|
||||
layout::LayoutParameters {
|
||||
zone_redundancy: self.zone_redundancy.into(),
|
||||
}
|
||||
}
|
||||
}
|
|
@ -3,9 +3,44 @@ extern crate tracing;
|
|||
|
||||
pub mod api_server;
|
||||
mod error;
|
||||
mod macros;
|
||||
|
||||
pub mod api;
|
||||
pub mod openapi;
|
||||
mod router_v0;
|
||||
mod router_v1;
|
||||
mod router_v2;
|
||||
|
||||
mod admin_token;
|
||||
mod bucket;
|
||||
mod cluster;
|
||||
mod key;
|
||||
mod layout;
|
||||
mod special;
|
||||
|
||||
mod block;
|
||||
mod node;
|
||||
mod repair;
|
||||
mod worker;
|
||||
|
||||
use std::sync::Arc;
|
||||
|
||||
use garage_model::garage::Garage;
|
||||
|
||||
pub use api_server::AdminApiServer as Admin;
|
||||
|
||||
pub enum Authorization {
|
||||
None,
|
||||
MetricsToken,
|
||||
AdminToken,
|
||||
}
|
||||
|
||||
pub trait RequestHandler {
|
||||
type Response;
|
||||
|
||||
fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
admin: &Admin,
|
||||
) -> impl std::future::Future<Output = Result<Self::Response, error::Error>> + Send;
|
||||
}
|
||||
|
|
206
src/api/admin/macros.rs
Normal file
206
src/api/admin/macros.rs
Normal file
|
@ -0,0 +1,206 @@
|
|||
macro_rules! admin_endpoints {
|
||||
[
|
||||
$(@special $special_endpoint:ident,)*
|
||||
$($endpoint:ident,)*
|
||||
] => {
|
||||
paste! {
|
||||
#[derive(Debug, Clone, Serialize, Deserialize)]
|
||||
pub enum AdminApiRequest {
|
||||
$(
|
||||
$special_endpoint( [<$special_endpoint Request>] ),
|
||||
)*
|
||||
$(
|
||||
$endpoint( [<$endpoint Request>] ),
|
||||
)*
|
||||
}
|
||||
|
||||
#[derive(Debug, Clone, Serialize)]
|
||||
#[serde(untagged)]
|
||||
pub enum AdminApiResponse {
|
||||
$(
|
||||
$endpoint( [<$endpoint Response>] ),
|
||||
)*
|
||||
}
|
||||
|
||||
#[derive(Debug, Clone, Serialize, Deserialize)]
|
||||
pub enum TaggedAdminApiResponse {
|
||||
$(
|
||||
$endpoint( [<$endpoint Response>] ),
|
||||
)*
|
||||
}
|
||||
|
||||
impl AdminApiRequest {
|
||||
pub fn name(&self) -> &'static str {
|
||||
match self {
|
||||
$(
|
||||
Self::$special_endpoint(_) => stringify!($special_endpoint),
|
||||
)*
|
||||
$(
|
||||
Self::$endpoint(_) => stringify!($endpoint),
|
||||
)*
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl AdminApiResponse {
|
||||
pub fn tagged(self) -> TaggedAdminApiResponse {
|
||||
match self {
|
||||
$(
|
||||
Self::$endpoint(res) => TaggedAdminApiResponse::$endpoint(res),
|
||||
)*
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
$(
|
||||
impl From< [< $endpoint Request >] > for AdminApiRequest {
|
||||
fn from(req: [< $endpoint Request >]) -> AdminApiRequest {
|
||||
AdminApiRequest::$endpoint(req)
|
||||
}
|
||||
}
|
||||
|
||||
impl TryFrom<TaggedAdminApiResponse> for [< $endpoint Response >] {
|
||||
type Error = TaggedAdminApiResponse;
|
||||
fn try_from(resp: TaggedAdminApiResponse) -> Result< [< $endpoint Response >], TaggedAdminApiResponse> {
|
||||
match resp {
|
||||
TaggedAdminApiResponse::$endpoint(v) => Ok(v),
|
||||
x => Err(x),
|
||||
}
|
||||
}
|
||||
}
|
||||
)*
|
||||
|
||||
impl RequestHandler for AdminApiRequest {
|
||||
type Response = AdminApiResponse;
|
||||
|
||||
async fn handle(self, garage: &Arc<Garage>, admin: &Admin) -> Result<AdminApiResponse, Error> {
|
||||
Ok(match self {
|
||||
$(
|
||||
AdminApiRequest::$special_endpoint(_) => panic!(
|
||||
concat!(stringify!($special_endpoint), " needs to go through a special handler")
|
||||
),
|
||||
)*
|
||||
$(
|
||||
AdminApiRequest::$endpoint(req) => AdminApiResponse::$endpoint(req.handle(garage, admin).await?),
|
||||
)*
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
macro_rules! local_admin_endpoints {
|
||||
[
|
||||
$($endpoint:ident,)*
|
||||
] => {
|
||||
paste! {
|
||||
#[derive(Debug, Clone, Serialize, Deserialize)]
|
||||
pub enum LocalAdminApiRequest {
|
||||
$(
|
||||
$endpoint( [<Local $endpoint Request>] ),
|
||||
)*
|
||||
}
|
||||
|
||||
#[derive(Debug, Clone, Serialize, Deserialize)]
|
||||
pub enum LocalAdminApiResponse {
|
||||
$(
|
||||
$endpoint( [<Local $endpoint Response>] ),
|
||||
)*
|
||||
}
|
||||
|
||||
$(
|
||||
pub type [< $endpoint Request >] = MultiRequest< [< Local $endpoint Request >] >;
|
||||
|
||||
pub type [< $endpoint RequestBody >] = [< Local $endpoint Request >];
|
||||
|
||||
pub type [< $endpoint Response >] = MultiResponse< [< Local $endpoint Response >] >;
|
||||
|
||||
impl From< [< Local $endpoint Request >] > for LocalAdminApiRequest {
|
||||
fn from(req: [< Local $endpoint Request >]) -> LocalAdminApiRequest {
|
||||
LocalAdminApiRequest::$endpoint(req)
|
||||
}
|
||||
}
|
||||
|
||||
impl TryFrom<LocalAdminApiResponse> for [< Local $endpoint Response >] {
|
||||
type Error = LocalAdminApiResponse;
|
||||
fn try_from(resp: LocalAdminApiResponse) -> Result< [< Local $endpoint Response >], LocalAdminApiResponse> {
|
||||
match resp {
|
||||
LocalAdminApiResponse::$endpoint(v) => Ok(v),
|
||||
x => Err(x),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for [< $endpoint Request >] {
|
||||
type Response = [< $endpoint Response >];
|
||||
|
||||
async fn handle(self, garage: &Arc<Garage>, admin: &Admin) -> Result<Self::Response, Error> {
|
||||
let to = find_matching_nodes(garage, self.node.as_str())?;
|
||||
|
||||
let resps = garage.system.rpc_helper().call_many(&admin.endpoint,
|
||||
&to,
|
||||
AdminRpc::Internal(self.body.into()),
|
||||
RequestStrategy::with_priority(PRIO_NORMAL),
|
||||
).await?;
|
||||
|
||||
let mut ret = [< $endpoint Response >] {
|
||||
success: HashMap::new(),
|
||||
error: HashMap::new(),
|
||||
};
|
||||
for (node, resp) in resps {
|
||||
match resp {
|
||||
Ok(AdminRpcResponse::InternalApiOkResponse(r)) => {
|
||||
match [< Local $endpoint Response >]::try_from(r) {
|
||||
Ok(r) => {
|
||||
ret.success.insert(hex::encode(node), r);
|
||||
}
|
||||
Err(_) => {
|
||||
ret.error.insert(hex::encode(node), "returned invalid value".to_string());
|
||||
}
|
||||
}
|
||||
}
|
||||
Ok(AdminRpcResponse::ApiErrorResponse{error_code, http_code, message}) => {
|
||||
ret.error.insert(hex::encode(node), format!("{} ({}): {}", error_code, http_code, message));
|
||||
}
|
||||
Ok(_) => {
|
||||
ret.error.insert(hex::encode(node), "returned invalid value".to_string());
|
||||
}
|
||||
Err(e) => {
|
||||
ret.error.insert(hex::encode(node), e.to_string());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(ret)
|
||||
}
|
||||
}
|
||||
)*
|
||||
|
||||
impl LocalAdminApiRequest {
|
||||
pub fn name(&self) -> &'static str {
|
||||
match self {
|
||||
$(
|
||||
Self::$endpoint(_) => stringify!($endpoint),
|
||||
)*
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for LocalAdminApiRequest {
|
||||
type Response = LocalAdminApiResponse;
|
||||
|
||||
async fn handle(self, garage: &Arc<Garage>, admin: &Admin) -> Result<LocalAdminApiResponse, Error> {
|
||||
Ok(match self {
|
||||
$(
|
||||
LocalAdminApiRequest::$endpoint(req) => LocalAdminApiResponse::$endpoint(req.handle(garage, admin).await?),
|
||||
)*
|
||||
})
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
pub(crate) use admin_endpoints;
|
||||
pub(crate) use local_admin_endpoints;
|
144
src/api/admin/node.rs
Normal file
144
src/api/admin/node.rs
Normal file
|
@ -0,0 +1,144 @@
|
|||
use std::fmt::Write;
|
||||
use std::sync::Arc;
|
||||
|
||||
use format_table::format_table_to_string;
|
||||
|
||||
use garage_util::error::Error as GarageError;
|
||||
|
||||
use garage_table::replication::*;
|
||||
use garage_table::*;
|
||||
|
||||
use garage_model::garage::Garage;
|
||||
|
||||
use crate::api::*;
|
||||
use crate::error::Error;
|
||||
use crate::{Admin, RequestHandler};
|
||||
|
||||
impl RequestHandler for LocalGetNodeInfoRequest {
|
||||
type Response = LocalGetNodeInfoResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<LocalGetNodeInfoResponse, Error> {
|
||||
Ok(LocalGetNodeInfoResponse {
|
||||
node_id: hex::encode(garage.system.id),
|
||||
garage_version: garage_util::version::garage_version().to_string(),
|
||||
garage_features: garage_util::version::garage_features()
|
||||
.map(|features| features.iter().map(ToString::to_string).collect()),
|
||||
rust_version: garage_util::version::rust_version().to_string(),
|
||||
db_engine: garage.db.engine(),
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for LocalCreateMetadataSnapshotRequest {
|
||||
type Response = LocalCreateMetadataSnapshotResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<LocalCreateMetadataSnapshotResponse, Error> {
|
||||
garage_model::snapshot::async_snapshot_metadata(garage).await?;
|
||||
Ok(LocalCreateMetadataSnapshotResponse)
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for LocalGetNodeStatisticsRequest {
|
||||
type Response = LocalGetNodeStatisticsResponse;
|
||||
|
||||
// FIXME: return this as a JSON struct instead of text
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<LocalGetNodeStatisticsResponse, Error> {
|
||||
let sys_status = garage.system.local_status();
|
||||
|
||||
let mut ret = format_table_to_string(vec![
|
||||
format!("Node ID:\t{:?}", garage.system.id),
|
||||
format!("Hostname:\t{}", sys_status.hostname.unwrap_or_default(),),
|
||||
format!(
|
||||
"Garage version:\t{}",
|
||||
garage_util::version::garage_version(),
|
||||
),
|
||||
format!(
|
||||
"Garage features:\t{}",
|
||||
garage_util::version::garage_features()
|
||||
.map(|list| list.join(", "))
|
||||
.unwrap_or_else(|| "(unknown)".into()),
|
||||
),
|
||||
format!(
|
||||
"Rust compiler version:\t{}",
|
||||
garage_util::version::rust_version(),
|
||||
),
|
||||
format!("Database engine:\t{}", garage.db.engine()),
|
||||
]);
|
||||
|
||||
// Gather table statistics
|
||||
let mut table = vec![" Table\tItems\tMklItems\tMklTodo\tInsQueue\tGcTodo".into()];
|
||||
table.push(gather_table_stats(&garage.admin_token_table)?);
|
||||
table.push(gather_table_stats(&garage.bucket_table)?);
|
||||
table.push(gather_table_stats(&garage.bucket_alias_table)?);
|
||||
table.push(gather_table_stats(&garage.key_table)?);
|
||||
|
||||
table.push(gather_table_stats(&garage.object_table)?);
|
||||
table.push(gather_table_stats(&garage.object_counter_table.table)?);
|
||||
table.push(gather_table_stats(&garage.mpu_table)?);
|
||||
table.push(gather_table_stats(&garage.mpu_counter_table.table)?);
|
||||
table.push(gather_table_stats(&garage.version_table)?);
|
||||
table.push(gather_table_stats(&garage.block_ref_table)?);
|
||||
|
||||
#[cfg(feature = "k2v")]
|
||||
{
|
||||
table.push(gather_table_stats(&garage.k2v.item_table)?);
|
||||
table.push(gather_table_stats(&garage.k2v.counter_table.table)?);
|
||||
}
|
||||
|
||||
write!(
|
||||
&mut ret,
|
||||
"\nTable stats:\n{}",
|
||||
format_table_to_string(table)
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
// Gather block manager statistics
|
||||
writeln!(&mut ret, "\nBlock manager stats:").unwrap();
|
||||
let rc_len = garage.block_manager.rc_len()?.to_string();
|
||||
|
||||
ret += &format_table_to_string(vec![
|
||||
format!(" number of RC entries:\t{} (~= number of blocks)", rc_len),
|
||||
format!(
|
||||
" resync queue length:\t{}",
|
||||
garage.block_manager.resync.queue_len()?
|
||||
),
|
||||
format!(
|
||||
" blocks with resync errors:\t{}",
|
||||
garage.block_manager.resync.errors_len()?
|
||||
),
|
||||
]);
|
||||
|
||||
Ok(LocalGetNodeStatisticsResponse { freeform: ret })
|
||||
}
|
||||
}
|
||||
|
||||
fn gather_table_stats<F, R>(t: &Arc<Table<F, R>>) -> Result<String, Error>
|
||||
where
|
||||
F: TableSchema + 'static,
|
||||
R: TableReplication + 'static,
|
||||
{
|
||||
let data_len = t.data.store.len().map_err(GarageError::from)?.to_string();
|
||||
let mkl_len = t.merkle_updater.merkle_tree_len()?.to_string();
|
||||
|
||||
Ok(format!(
|
||||
" {}\t{}\t{}\t{}\t{}\t{}",
|
||||
F::TABLE_NAME,
|
||||
data_len,
|
||||
mkl_len,
|
||||
t.merkle_updater.todo_len()?,
|
||||
t.data.insert_queue_len()?,
|
||||
t.data.gc_todo_len()?
|
||||
))
|
||||
}
|
924
src/api/admin/openapi.rs
Normal file
924
src/api/admin/openapi.rs
Normal file
|
@ -0,0 +1,924 @@
|
|||
#![allow(dead_code)]
|
||||
#![allow(non_snake_case)]
|
||||
|
||||
use utoipa::{Modify, OpenApi};
|
||||
|
||||
use crate::api::*;
|
||||
|
||||
// **********************************************
|
||||
// Special endpoints
|
||||
// **********************************************
|
||||
|
||||
#[utoipa::path(get,
|
||||
path = "/metrics",
|
||||
tag = "Special endpoints",
|
||||
description = "Prometheus metrics endpoint",
|
||||
security((), ("bearerAuth" = [])),
|
||||
responses(
|
||||
(status = 200, description = "Garage daemon metrics exported in Prometheus format"),
|
||||
),
|
||||
)]
|
||||
fn Metrics() -> () {}
|
||||
|
||||
#[utoipa::path(get,
|
||||
path = "/health",
|
||||
tag = "Special endpoints",
|
||||
description = "
|
||||
Check cluster health. The status code returned by this function indicates
|
||||
whether this Garage daemon can answer API requests.
|
||||
Garage will return `200 OK` even if some storage nodes are disconnected,
|
||||
as long as it is able to have a quorum of nodes for read and write operations.
|
||||
",
|
||||
security(()),
|
||||
responses(
|
||||
(status = 200, description = "Garage is able to answer requests"),
|
||||
(status = 503, description = "This Garage daemon is not able to handle requests")
|
||||
),
|
||||
)]
|
||||
fn Health() -> () {}
|
||||
|
||||
#[utoipa::path(get,
|
||||
path = "/check",
|
||||
tag = "Special endpoints",
|
||||
description = "
|
||||
Static website domain name check. Checks whether a bucket is configured to serve
|
||||
a static website for the requested domain. This is used by reverse proxies such
|
||||
as Caddy or Tricot, to avoid requesting TLS certificates for domain names that
|
||||
do not correspond to an actual website.
|
||||
",
|
||||
params(
|
||||
("domain", description = "The domain name to check for"),
|
||||
),
|
||||
security(()),
|
||||
responses(
|
||||
(status = 200, description = "The domain name redirects to a static website bucket"),
|
||||
(status = 400, description = "No static website bucket exists for this domain")
|
||||
),
|
||||
)]
|
||||
fn CheckDomain() -> () {}
|
||||
|
||||
// **********************************************
|
||||
// Cluster operations
|
||||
// **********************************************
|
||||
|
||||
#[utoipa::path(get,
|
||||
path = "/v2/GetClusterStatus",
|
||||
tag = "Cluster",
|
||||
description = "
|
||||
Returns the cluster's current status, including:
|
||||
|
||||
- ID of the node being queried and its version of the Garage daemon
|
||||
- Live nodes
|
||||
- Currently configured cluster layout
|
||||
- Staged changes to the cluster layout
|
||||
|
||||
*Capacity is given in bytes*
|
||||
",
|
||||
responses(
|
||||
(status = 200, description = "Cluster status report", body = GetClusterStatusResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn GetClusterStatus() -> () {}
|
||||
|
||||
#[utoipa::path(get,
|
||||
path = "/v2/GetClusterHealth",
|
||||
tag = "Cluster",
|
||||
description = "Returns the global status of the cluster, the number of connected nodes (over the number of known ones), the number of healthy storage nodes (over the declared ones), and the number of healthy partitions (over the total).",
|
||||
responses(
|
||||
(status = 200, description = "Cluster health report", body = GetClusterHealthResponse),
|
||||
),
|
||||
)]
|
||||
fn GetClusterHealth() -> () {}
|
||||
|
||||
#[utoipa::path(get,
|
||||
path = "/v2/GetClusterStatistics",
|
||||
tag = "Cluster",
|
||||
description = "
|
||||
Fetch global cluster statistics.
|
||||
|
||||
*Note: do not try to parse the `freeform` field of the response, it is given as a string specifically because its format is not stable.*
|
||||
",
|
||||
responses(
|
||||
(status = 200, description = "Global cluster statistics", body = GetClusterStatisticsResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn GetClusterStatistics() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/ConnectClusterNodes",
|
||||
tag = "Cluster",
|
||||
description = "Instructs this Garage node to connect to other Garage nodes at specified `<node_id>@<net_address>`. `node_id` is generated automatically on node start.",
|
||||
request_body=ConnectClusterNodesRequest,
|
||||
responses(
|
||||
(status = 200, description = "The request has been handled correctly but it does not mean that all connection requests succeeded; some might have fail, you need to check the body!", body = ConnectClusterNodesResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn ConnectClusterNodes() -> () {}
|
||||
|
||||
// **********************************************
|
||||
// Admin API token operations
|
||||
// **********************************************
|
||||
|
||||
#[utoipa::path(get,
|
||||
path = "/v2/ListAdminTokens",
|
||||
tag = "Admin API token",
|
||||
description = "Returns all admin API tokens in the cluster.",
|
||||
responses(
|
||||
(status = 200, description = "Returns info about all admin API tokens", body = ListAdminTokensResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn ListAdminTokens() -> () {}
|
||||
|
||||
#[utoipa::path(get,
|
||||
path = "/v2/GetAdminTokenInfo",
|
||||
tag = "Admin API token",
|
||||
description = "
|
||||
Return information about a specific admin API token.
|
||||
You can search by specifying the exact token identifier (`id`) or by specifying a pattern (`search`).
|
||||
",
|
||||
params(GetAdminTokenInfoRequest),
|
||||
responses(
|
||||
(status = 200, description = "Information about the admin token", body = GetAdminTokenInfoResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn GetAdminTokenInfo() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/CreateAdminToken",
|
||||
tag = "Admin API token",
|
||||
description = "Creates a new admin API token",
|
||||
request_body = UpdateAdminTokenRequestBody,
|
||||
responses(
|
||||
(status = 200, description = "Admin token has been created", body = CreateAdminTokenResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn CreateAdminToken() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/UpdateAdminToken",
|
||||
tag = "Admin API token",
|
||||
description = "
|
||||
Updates information about the specified admin API token.
|
||||
",
|
||||
request_body = UpdateAdminTokenRequestBody,
|
||||
params(
|
||||
("id", description = "Admin API token ID"),
|
||||
),
|
||||
responses(
|
||||
(status = 200, description = "Admin token has been updated", body = UpdateAdminTokenResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn UpdateAdminToken() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/DeleteAdminToken",
|
||||
tag = "Admin API token",
|
||||
description = "Delete an admin API token from the cluster, revoking all its permissions.",
|
||||
params(
|
||||
("id", description = "Admin API token ID"),
|
||||
),
|
||||
responses(
|
||||
(status = 200, description = "Admin token has been deleted"),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn DeleteAdminToken() -> () {}
|
||||
|
||||
// **********************************************
|
||||
// Layout operations
|
||||
// **********************************************
|
||||
|
||||
#[utoipa::path(get,
|
||||
path = "/v2/GetClusterLayout",
|
||||
tag = "Cluster layout",
|
||||
description = "
|
||||
Returns the cluster's current layout, including:
|
||||
|
||||
- Currently configured cluster layout
|
||||
- Staged changes to the cluster layout
|
||||
|
||||
*Capacity is given in bytes*
|
||||
",
|
||||
responses(
|
||||
(status = 200, description = "Current cluster layout", body = GetClusterLayoutResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn GetClusterLayout() -> () {}
|
||||
|
||||
#[utoipa::path(get,
|
||||
path = "/v2/GetClusterLayoutHistory",
|
||||
tag = "Cluster layout",
|
||||
description = "
|
||||
Returns the history of layouts in the cluster
|
||||
",
|
||||
responses(
|
||||
(status = 200, description = "Cluster layout history", body = GetClusterLayoutHistoryResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn GetClusterLayoutHistory() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/UpdateClusterLayout",
|
||||
tag = "Cluster layout",
|
||||
description = "
|
||||
Send modifications to the cluster layout. These modifications will be included in the staged role changes, visible in subsequent calls of `GET /GetClusterHealth`. Once the set of staged changes is satisfactory, the user may call `POST /ApplyClusterLayout` to apply the changed changes, or `POST /RevertClusterLayout` to clear all of the staged changes in the layout.
|
||||
|
||||
Setting the capacity to `null` will configure the node as a gateway.
|
||||
Otherwise, capacity must be now set in bytes (before Garage 0.9 it was arbitrary weights).
|
||||
For example to declare 100GB, you must set `capacity: 100000000000`.
|
||||
|
||||
Garage uses internally the International System of Units (SI), it assumes that 1kB = 1000 bytes, and displays storage as kB, MB, GB (and not KiB, MiB, GiB that assume 1KiB = 1024 bytes).
|
||||
",
|
||||
request_body(
|
||||
content=UpdateClusterLayoutRequest,
|
||||
description="
|
||||
To add a new node to the layout or to change the configuration of an existing node, simply set the values you want (`zone`, `capacity`, and `tags`).
|
||||
To remove a node, simply pass the `remove: true` field.
|
||||
This logic is represented in OpenAPI with a 'One Of' object.
|
||||
|
||||
Contrary to the CLI that may update only a subset of the fields capacity, zone and tags, when calling this API all of these values must be specified.
|
||||
"
|
||||
),
|
||||
responses(
|
||||
(status = 200, description = "Proposed changes have been added to the list of pending changes", body = UpdateClusterLayoutResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn UpdateClusterLayout() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/PreviewClusterLayoutChanges",
|
||||
tag = "Cluster layout",
|
||||
description = "
|
||||
Computes a new layout taking into account the staged parameters, and returns it with detailed statistics. The new layout is not applied in the cluster.
|
||||
|
||||
*Note: do not try to parse the `message` field of the response, it is given as an array of string specifically because its format is not stable.*
|
||||
",
|
||||
responses(
|
||||
(status = 200, description = "Information about the new layout", body = PreviewClusterLayoutChangesResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn PreviewClusterLayoutChanges() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/ApplyClusterLayout",
|
||||
tag = "Cluster layout",
|
||||
description = "
|
||||
Applies to the cluster the layout changes currently registered as staged layout changes.
|
||||
|
||||
*Note: do not try to parse the `message` field of the response, it is given as an array of string specifically because its format is not stable.*
|
||||
",
|
||||
request_body=ApplyClusterLayoutRequest,
|
||||
responses(
|
||||
(status = 200, description = "The updated cluster layout has been applied in the cluster", body = ApplyClusterLayoutResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn ApplyClusterLayout() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/RevertClusterLayout",
|
||||
tag = "Cluster layout",
|
||||
description = "Clear staged layout changes",
|
||||
responses(
|
||||
(status = 200, description = "All pending changes to the cluster layout have been erased", body = RevertClusterLayoutResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn RevertClusterLayout() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/ClusterLayoutSkipDeadNodes",
|
||||
tag = "Cluster layout",
|
||||
description = "Force progress in layout update trackers",
|
||||
request_body = ClusterLayoutSkipDeadNodesRequest,
|
||||
responses(
|
||||
(status = 200, description = "Request has been taken into account", body = ClusterLayoutSkipDeadNodesResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn ClusterLayoutSkipDeadNodes() -> () {}
|
||||
|
||||
// **********************************************
|
||||
// Access key operations
|
||||
// **********************************************
|
||||
|
||||
#[utoipa::path(get,
|
||||
path = "/v2/ListKeys",
|
||||
tag = "Access key",
|
||||
description = "Returns all API access keys in the cluster.",
|
||||
responses(
|
||||
(status = 200, description = "Returns the key identifier (aka `AWS_ACCESS_KEY_ID`) and its associated, human friendly, name if any (otherwise return an empty string)", body = ListKeysResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn ListKeys() -> () {}
|
||||
|
||||
#[utoipa::path(get,
|
||||
path = "/v2/GetKeyInfo",
|
||||
tag = "Access key",
|
||||
description = "
|
||||
Return information about a specific key like its identifiers, its permissions and buckets on which it has permissions.
|
||||
You can search by specifying the exact key identifier (`id`) or by specifying a pattern (`search`).
|
||||
|
||||
For confidentiality reasons, the secret key is not returned by default: you must pass the `showSecretKey` query parameter to get it.
|
||||
",
|
||||
params(GetKeyInfoRequest),
|
||||
responses(
|
||||
(status = 200, description = "Information about the access key", body = GetKeyInfoResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn GetKeyInfo() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/CreateKey",
|
||||
tag = "Access key",
|
||||
description = "Creates a new API access key.",
|
||||
request_body = CreateKeyRequest,
|
||||
responses(
|
||||
(status = 200, description = "Access key has been created", body = CreateKeyResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn CreateKey() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/ImportKey",
|
||||
tag = "Access key",
|
||||
description = "
|
||||
Imports an existing API key. This feature must only be used for migrations and backup restore.
|
||||
|
||||
**Do not use it to generate custom key identifiers or you will break your Garage cluster.**
|
||||
",
|
||||
request_body = ImportKeyRequest,
|
||||
responses(
|
||||
(status = 200, description = "Access key has been imported", body = ImportKeyResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn ImportKey() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/UpdateKey",
|
||||
tag = "Access key",
|
||||
description = "
|
||||
Updates information about the specified API access key.
|
||||
|
||||
*Note: the secret key is not returned in the response, `null` is sent instead.*
|
||||
",
|
||||
request_body = UpdateKeyRequestBody,
|
||||
params(
|
||||
("id", description = "Access key ID"),
|
||||
),
|
||||
responses(
|
||||
(status = 200, description = "Access key has been updated", body = UpdateKeyResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn UpdateKey() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/DeleteKey",
|
||||
tag = "Access key",
|
||||
description = "Delete a key from the cluster. Its access will be removed from all the buckets. Buckets are not automatically deleted and can be dangling. You should manually delete them before. ",
|
||||
params(
|
||||
("id", description = "Access key ID"),
|
||||
),
|
||||
responses(
|
||||
(status = 200, description = "Access key has been deleted"),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn DeleteKey() -> () {}
|
||||
|
||||
// **********************************************
|
||||
// Bucket operations
|
||||
// **********************************************
|
||||
|
||||
#[utoipa::path(get,
|
||||
path = "/v2/ListBuckets",
|
||||
tag = "Bucket",
|
||||
description = "List all the buckets on the cluster with their UUID and their global and local aliases.",
|
||||
responses(
|
||||
(status = 200, description = "Returns the UUID of all the buckets and all their aliases", body = ListBucketsResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn ListBuckets() -> () {}
|
||||
|
||||
#[utoipa::path(get,
|
||||
path = "/v2/GetBucketInfo",
|
||||
tag = "Bucket",
|
||||
description = "
|
||||
Given a bucket identifier (`id`) or a global alias (`alias`), get its information.
|
||||
It includes its aliases, its web configuration, keys that have some permissions
|
||||
on it, some statistics (number of objects, size), number of dangling multipart uploads,
|
||||
and its quotas (if any).
|
||||
",
|
||||
params(GetBucketInfoRequest),
|
||||
responses(
|
||||
(status = 200, description = "Returns exhaustive information about the bucket", body = GetBucketInfoResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn GetBucketInfo() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/CreateBucket",
|
||||
tag = "Bucket",
|
||||
description = "
|
||||
Creates a new bucket, either with a global alias, a local one, or no alias at all.
|
||||
Technically, you can also specify both `globalAlias` and `localAlias` and that would create two aliases.
|
||||
",
|
||||
request_body = CreateBucketRequest,
|
||||
responses(
|
||||
(status = 200, description = "Returns exhaustive information about the bucket", body = CreateBucketResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn CreateBucket() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/UpdateBucket",
|
||||
tag = "Bucket",
|
||||
description = "
|
||||
All fields (`websiteAccess` and `quotas`) are optional.
|
||||
If they are present, the corresponding modifications are applied to the bucket, otherwise nothing is changed.
|
||||
|
||||
In `websiteAccess`: if `enabled` is `true`, `indexDocument` must be specified.
|
||||
The field `errorDocument` is optional, if no error document is set a generic
|
||||
error message is displayed when errors happen. Conversely, if `enabled` is
|
||||
`false`, neither `indexDocument` nor `errorDocument` must be specified.
|
||||
|
||||
In `quotas`: new values of `maxSize` and `maxObjects` must both be specified, or set to `null`
|
||||
to remove the quotas. An absent value will be considered the same as a `null`. It is not possible
|
||||
to change only one of the two quotas.
|
||||
",
|
||||
params(
|
||||
("id", description = "ID of the bucket to update"),
|
||||
),
|
||||
request_body = UpdateBucketRequestBody,
|
||||
responses(
|
||||
(status = 200, description = "Bucket has been updated", body = UpdateBucketResponse),
|
||||
(status = 404, description = "Bucket not found"),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn UpdateBucket() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/DeleteBucket",
|
||||
tag = "Bucket",
|
||||
description = "
|
||||
Deletes a storage bucket. A bucket cannot be deleted if it is not empty.
|
||||
|
||||
**Warning:** this will delete all aliases associated with the bucket!
|
||||
",
|
||||
params(
|
||||
("id", description = "ID of the bucket to delete"),
|
||||
),
|
||||
responses(
|
||||
(status = 200, description = "Bucket has been deleted"),
|
||||
(status = 400, description = "Bucket is not empty"),
|
||||
(status = 404, description = "Bucket not found"),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn DeleteBucket() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/CleanupIncompleteUploads",
|
||||
tag = "Bucket",
|
||||
description = "Removes all incomplete multipart uploads that are older than the specified number of seconds.",
|
||||
request_body = CleanupIncompleteUploadsRequest,
|
||||
responses(
|
||||
(status = 200, description = "The bucket was cleaned up successfully", body = CleanupIncompleteUploadsResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn CleanupIncompleteUploads() -> () {}
|
||||
|
||||
#[utoipa::path(get,
|
||||
path = "/v2/InspectObject",
|
||||
tag = "Bucket",
|
||||
description = "
|
||||
Returns detailed information about an object in a bucket, including its internal state in Garage.
|
||||
|
||||
This API call can be used to list the data blocks referenced by an object,
|
||||
as well as to view metadata associated to the object.
|
||||
|
||||
This call may return a list of more than one version for the object, for instance in the
|
||||
case where there is a currently stored version of the object, and a newer version whose
|
||||
upload is in progress and not yet finished.
|
||||
",
|
||||
params(InspectObjectRequest),
|
||||
responses(
|
||||
(status = 200, description = "Returns exhaustive information about the object", body = InspectObjectResponse),
|
||||
(status = 404, description = "Object not found"),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn InspectObject() -> () {}
|
||||
|
||||
// **********************************************
|
||||
// Operations on permissions for keys on buckets
|
||||
// **********************************************
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/AllowBucketKey",
|
||||
tag = "Permission",
|
||||
description = "
|
||||
⚠️ **DISCLAIMER**: Garage's developers are aware that this endpoint has an unconventional semantic. Be extra careful when implementing it, its behavior is not obvious.
|
||||
|
||||
Allows a key to do read/write/owner operations on a bucket.
|
||||
|
||||
Flags in permissions which have the value true will be activated. Other flags will remain unchanged (ie. they will keep their internal value).
|
||||
|
||||
For example, if you set read to true, the key will be allowed to read the bucket.
|
||||
If you set it to false, the key will keeps its previous read permission.
|
||||
If you want to disallow read for the key, check the DenyBucketKey operation.
|
||||
",
|
||||
request_body = AllowBucketKeyRequest,
|
||||
responses(
|
||||
(status = 200, description = "Returns exhaustive information about the bucket", body = AllowBucketKeyResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn AllowBucketKey() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/DenyBucketKey",
|
||||
tag = "Permission",
|
||||
description = "
|
||||
⚠️ **DISCLAIMER**: Garage's developers are aware that this endpoint has an unconventional semantic. Be extra careful when implementing it, its behavior is not obvious.
|
||||
|
||||
Denies a key from doing read/write/owner operations on a bucket.
|
||||
|
||||
Flags in permissions which have the value true will be deactivated. Other flags will remain unchanged.
|
||||
|
||||
For example, if you set read to true, the key will be denied from reading.
|
||||
If you set read to false, the key will keep its previous permissions.
|
||||
If you want the key to have the reading permission, check the AllowBucketKey operation.
|
||||
",
|
||||
request_body = DenyBucketKeyRequest,
|
||||
responses(
|
||||
(status = 200, description = "Returns exhaustive information about the bucket", body = DenyBucketKeyResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn DenyBucketKey() -> () {}
|
||||
|
||||
// **********************************************
|
||||
// Operations on bucket aliases
|
||||
// **********************************************
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/AddBucketAlias",
|
||||
tag = "Bucket alias",
|
||||
description = "Add an alias for the target bucket. This can be either a global or a local alias, depending on which fields are specified.",
|
||||
request_body = AddBucketAliasRequest,
|
||||
responses(
|
||||
(status = 200, description = "Returns exhaustive information about the bucket", body = AddBucketAliasResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn AddBucketAlias() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/RemoveBucketAlias",
|
||||
tag = "Bucket alias",
|
||||
description = "Remove an alias for the target bucket. This can be either a global or a local alias, depending on which fields are specified.",
|
||||
request_body = RemoveBucketAliasRequest,
|
||||
responses(
|
||||
(status = 200, description = "Returns exhaustive information about the bucket", body = RemoveBucketAliasResponse),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn RemoveBucketAlias() -> () {}
|
||||
|
||||
// **********************************************
|
||||
// Node operations
|
||||
// **********************************************
|
||||
|
||||
#[utoipa::path(get,
|
||||
path = "/v2/GetNodeInfo",
|
||||
tag = "Node",
|
||||
description = "
|
||||
Return information about the Garage daemon running on one or several nodes.
|
||||
",
|
||||
params(
|
||||
("node", description = "Node ID to query, or `*` for all nodes, or `self` for the node responding to the request"),
|
||||
),
|
||||
responses(
|
||||
(status = 200, description = "Responses from individual cluster nodes", body = MultiResponse<LocalGetNodeInfoResponse>),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn GetNodeInfo() -> () {}
|
||||
|
||||
#[utoipa::path(get,
|
||||
path = "/v2/GetNodeStatistics",
|
||||
tag = "Node",
|
||||
description = "
|
||||
Fetch statistics for one or several Garage nodes.
|
||||
|
||||
*Note: do not try to parse the `freeform` field of the response, it is given as a string specifically because its format is not stable.*
|
||||
",
|
||||
params(
|
||||
("node", description = "Node ID to query, or `*` for all nodes, or `self` for the node responding to the request"),
|
||||
),
|
||||
responses(
|
||||
(status = 200, description = "Responses from individual cluster nodes", body = MultiResponse<LocalGetNodeStatisticsResponse>),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn GetNodeStatistics() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/CreateMetadataSnapshot",
|
||||
tag = "Node",
|
||||
description = "
|
||||
Instruct one or several nodes to take a snapshot of their metadata databases.
|
||||
",
|
||||
params(
|
||||
("node", description = "Node ID to query, or `*` for all nodes, or `self` for the node responding to the request"),
|
||||
),
|
||||
responses(
|
||||
(status = 200, description = "Responses from individual cluster nodes", body = MultiResponse<LocalCreateMetadataSnapshotResponse>),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn CreateMetadataSnapshot() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/LaunchRepairOperation",
|
||||
tag = "Node",
|
||||
description = "
|
||||
Launch a repair operation on one or several cluster nodes.
|
||||
",
|
||||
params(
|
||||
("node", description = "Node ID to query, or `*` for all nodes, or `self` for the node responding to the request"),
|
||||
),
|
||||
request_body = LocalLaunchRepairOperationRequest,
|
||||
responses(
|
||||
(status = 200, description = "Responses from individual cluster nodes", body = MultiResponse<LocalLaunchRepairOperationResponse>),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn LaunchRepairOperation() -> () {}
|
||||
|
||||
// **********************************************
|
||||
// Worker operations
|
||||
// **********************************************
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/ListWorkers",
|
||||
tag = "Worker",
|
||||
description = "
|
||||
List background workers currently running on one or several cluster nodes.
|
||||
",
|
||||
params(
|
||||
("node", description = "Node ID to query, or `*` for all nodes, or `self` for the node responding to the request"),
|
||||
),
|
||||
request_body = LocalListWorkersRequest,
|
||||
responses(
|
||||
(status = 200, description = "Responses from individual cluster nodes", body = MultiResponse<LocalListWorkersResponse>),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn ListWorkers() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/GetWorkerInfo",
|
||||
tag = "Worker",
|
||||
description = "
|
||||
Get information about the specified background worker on one or several cluster nodes.
|
||||
",
|
||||
params(
|
||||
("node", description = "Node ID to query, or `*` for all nodes, or `self` for the node responding to the request"),
|
||||
),
|
||||
request_body = LocalGetWorkerInfoRequest,
|
||||
responses(
|
||||
(status = 200, description = "Responses from individual cluster nodes", body = MultiResponse<LocalGetWorkerInfoResponse>),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn GetWorkerInfo() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/GetWorkerVariable",
|
||||
tag = "Worker",
|
||||
description = "
|
||||
Fetch values of one or several worker variables, from one or several cluster nodes.
|
||||
",
|
||||
params(
|
||||
("node", description = "Node ID to query, or `*` for all nodes, or `self` for the node responding to the request"),
|
||||
),
|
||||
request_body = LocalGetWorkerVariableRequest,
|
||||
responses(
|
||||
(status = 200, description = "Responses from individual cluster nodes", body = MultiResponse<LocalGetWorkerVariableResponse>),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn GetWorkerVariable() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/SetWorkerVariable",
|
||||
tag = "Worker",
|
||||
description = "
|
||||
Set the value for a worker variable, on one or several cluster nodes.
|
||||
",
|
||||
params(
|
||||
("node", description = "Node ID to query, or `*` for all nodes, or `self` for the node responding to the request"),
|
||||
),
|
||||
request_body = LocalSetWorkerVariableRequest,
|
||||
responses(
|
||||
(status = 200, description = "Responses from individual cluster nodes", body = MultiResponse<LocalSetWorkerVariableResponse>),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn SetWorkerVariable() -> () {}
|
||||
|
||||
// **********************************************
|
||||
// Block operations
|
||||
// **********************************************
|
||||
|
||||
#[utoipa::path(get,
|
||||
path = "/v2/ListBlockErrors",
|
||||
tag = "Block",
|
||||
description = "
|
||||
List data blocks that are currently in an errored state on one or several Garage nodes.
|
||||
",
|
||||
params(
|
||||
("node", description = "Node ID to query, or `*` for all nodes, or `self` for the node responding to the request"),
|
||||
),
|
||||
responses(
|
||||
(status = 200, description = "Responses from individual cluster nodes", body = MultiResponse<LocalListBlockErrorsResponse>),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn ListBlockErrors() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/GetBlockInfo",
|
||||
tag = "Block",
|
||||
description = "
|
||||
Get detailed information about a data block stored on a Garage node, including all object versions and in-progress multipart uploads that contain a reference to this block.
|
||||
",
|
||||
params(
|
||||
("node", description = "Node ID to query, or `*` for all nodes, or `self` for the node responding to the request"),
|
||||
),
|
||||
request_body = LocalGetBlockInfoRequest,
|
||||
responses(
|
||||
(status = 200, description = "Detailed block information", body = MultiResponse<LocalGetBlockInfoResponse>),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn GetBlockInfo() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/RetryBlockResync",
|
||||
tag = "Block",
|
||||
description = "
|
||||
Instruct Garage node(s) to retry the resynchronization of one or several missing data block(s).
|
||||
",
|
||||
params(
|
||||
("node", description = "Node ID to query, or `*` for all nodes, or `self` for the node responding to the request"),
|
||||
),
|
||||
request_body = LocalRetryBlockResyncRequest,
|
||||
responses(
|
||||
(status = 200, description = "Responses from individual cluster nodes", body = MultiResponse<LocalRetryBlockResyncResponse>),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn RetryBlockResync() -> () {}
|
||||
|
||||
#[utoipa::path(post,
|
||||
path = "/v2/PurgeBlocks",
|
||||
tag = "Block",
|
||||
description = "
|
||||
Purge references to one or several missing data blocks.
|
||||
|
||||
This will remove all objects and in-progress multipart uploads that contain the specified data block(s). The objects will be permanently deleted from the buckets in which they appear. Use with caution.
|
||||
",
|
||||
params(
|
||||
("node", description = "Node ID to query, or `*` for all nodes, or `self` for the node responding to the request"),
|
||||
),
|
||||
request_body = LocalPurgeBlocksRequest,
|
||||
responses(
|
||||
(status = 200, description = "Responses from individual cluster nodes", body = MultiResponse<LocalPurgeBlocksResponse>),
|
||||
(status = 500, description = "Internal server error")
|
||||
),
|
||||
)]
|
||||
fn PurgeBlocks() -> () {}
|
||||
|
||||
// **********************************************
|
||||
// **********************************************
|
||||
// **********************************************
|
||||
|
||||
struct SecurityAddon;
|
||||
|
||||
impl Modify for SecurityAddon {
|
||||
fn modify(&self, openapi: &mut utoipa::openapi::OpenApi) {
|
||||
use utoipa::openapi::security::*;
|
||||
let components = openapi.components.as_mut().unwrap(); // we can unwrap safely since there already is components registered.
|
||||
components.add_security_scheme(
|
||||
"bearerAuth",
|
||||
SecurityScheme::Http(Http::builder().scheme(HttpAuthScheme::Bearer).build()),
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(OpenApi)]
|
||||
#[openapi(
|
||||
info(
|
||||
version = "v2.0.0",
|
||||
title = "Garage administration API",
|
||||
description = "Administrate your Garage cluster programatically, including status, layout, keys, buckets, and maintainance tasks.
|
||||
|
||||
*Disclaimer: This API may change in future Garage versions. Read the changelog and upgrade your scripts before upgrading. Additionnaly, this specification is early stage and can contain bugs, so be careful and please report any issues on our issue tracker.*",
|
||||
contact(
|
||||
name = "The Garage team",
|
||||
email = "garagehq@deuxfleurs.fr",
|
||||
url = "https://garagehq.deuxfleurs.fr/",
|
||||
),
|
||||
),
|
||||
modifiers(&SecurityAddon),
|
||||
security(("bearerAuth" = [])),
|
||||
paths(
|
||||
// Special ops
|
||||
Metrics,
|
||||
Health,
|
||||
CheckDomain,
|
||||
// Cluster operations
|
||||
GetClusterHealth,
|
||||
GetClusterStatus,
|
||||
GetClusterStatistics,
|
||||
ConnectClusterNodes,
|
||||
// Admin token operations
|
||||
ListAdminTokens,
|
||||
GetAdminTokenInfo,
|
||||
CreateAdminToken,
|
||||
UpdateAdminToken,
|
||||
DeleteAdminToken,
|
||||
// Layout operations
|
||||
GetClusterLayout,
|
||||
GetClusterLayoutHistory,
|
||||
UpdateClusterLayout,
|
||||
PreviewClusterLayoutChanges,
|
||||
ApplyClusterLayout,
|
||||
RevertClusterLayout,
|
||||
ClusterLayoutSkipDeadNodes,
|
||||
// Key operations
|
||||
ListKeys,
|
||||
GetKeyInfo,
|
||||
CreateKey,
|
||||
ImportKey,
|
||||
UpdateKey,
|
||||
DeleteKey,
|
||||
// Bucket operations
|
||||
ListBuckets,
|
||||
GetBucketInfo,
|
||||
CreateBucket,
|
||||
UpdateBucket,
|
||||
DeleteBucket,
|
||||
CleanupIncompleteUploads,
|
||||
InspectObject,
|
||||
// Operations on permissions
|
||||
AllowBucketKey,
|
||||
DenyBucketKey,
|
||||
// Operations on aliases
|
||||
AddBucketAlias,
|
||||
RemoveBucketAlias,
|
||||
// Node operations
|
||||
GetNodeInfo,
|
||||
GetNodeStatistics,
|
||||
CreateMetadataSnapshot,
|
||||
LaunchRepairOperation,
|
||||
// Worker operations
|
||||
ListWorkers,
|
||||
GetWorkerInfo,
|
||||
GetWorkerVariable,
|
||||
SetWorkerVariable,
|
||||
// Block operations
|
||||
ListBlockErrors,
|
||||
GetBlockInfo,
|
||||
RetryBlockResync,
|
||||
PurgeBlocks,
|
||||
),
|
||||
servers(
|
||||
(url = "http://localhost:3903/", description = "A local server")
|
||||
),
|
||||
)]
|
||||
pub struct ApiDoc;
|
|
@ -5,6 +5,14 @@ use std::time::Duration;
|
|||
use async_trait::async_trait;
|
||||
use tokio::sync::watch;
|
||||
|
||||
use garage_util::background::*;
|
||||
use garage_util::data::*;
|
||||
use garage_util::error::{Error as GarageError, OkOrMessage};
|
||||
use garage_util::migrate::Migrate;
|
||||
|
||||
use garage_table::replication::*;
|
||||
use garage_table::*;
|
||||
|
||||
use garage_block::manager::BlockManager;
|
||||
use garage_block::repair::ScrubWorkerCommand;
|
||||
|
||||
|
@ -14,82 +22,76 @@ use garage_model::s3::mpu_table::*;
|
|||
use garage_model::s3::object_table::*;
|
||||
use garage_model::s3::version_table::*;
|
||||
|
||||
use garage_table::replication::*;
|
||||
use garage_table::*;
|
||||
|
||||
use garage_util::background::*;
|
||||
use garage_util::data::*;
|
||||
use garage_util::error::Error;
|
||||
use garage_util::migrate::Migrate;
|
||||
|
||||
use crate::*;
|
||||
use crate::api::*;
|
||||
use crate::error::Error;
|
||||
use crate::{Admin, RequestHandler};
|
||||
|
||||
const RC_REPAIR_ITER_COUNT: usize = 64;
|
||||
|
||||
pub async fn launch_online_repair(
|
||||
garage: &Arc<Garage>,
|
||||
bg: &BackgroundRunner,
|
||||
opt: RepairOpt,
|
||||
) -> Result<(), Error> {
|
||||
match opt.what {
|
||||
RepairWhat::Tables => {
|
||||
info!("Launching a full sync of tables");
|
||||
garage.bucket_table.syncer.add_full_sync()?;
|
||||
garage.object_table.syncer.add_full_sync()?;
|
||||
garage.version_table.syncer.add_full_sync()?;
|
||||
garage.block_ref_table.syncer.add_full_sync()?;
|
||||
garage.key_table.syncer.add_full_sync()?;
|
||||
}
|
||||
RepairWhat::Versions => {
|
||||
info!("Repairing the versions table");
|
||||
bg.spawn_worker(TableRepairWorker::new(garage.clone(), RepairVersions));
|
||||
}
|
||||
RepairWhat::MultipartUploads => {
|
||||
info!("Repairing the multipart uploads table");
|
||||
bg.spawn_worker(TableRepairWorker::new(garage.clone(), RepairMpu));
|
||||
}
|
||||
RepairWhat::BlockRefs => {
|
||||
info!("Repairing the block refs table");
|
||||
bg.spawn_worker(TableRepairWorker::new(garage.clone(), RepairBlockRefs));
|
||||
}
|
||||
RepairWhat::BlockRc => {
|
||||
info!("Repairing the block reference counters");
|
||||
bg.spawn_worker(BlockRcRepair::new(
|
||||
garage.block_manager.clone(),
|
||||
garage.block_ref_table.clone(),
|
||||
));
|
||||
}
|
||||
RepairWhat::Blocks => {
|
||||
info!("Repairing the stored blocks");
|
||||
bg.spawn_worker(garage_block::repair::RepairWorker::new(
|
||||
garage.block_manager.clone(),
|
||||
));
|
||||
}
|
||||
RepairWhat::Scrub { cmd } => {
|
||||
let cmd = match cmd {
|
||||
ScrubCmd::Start => ScrubWorkerCommand::Start,
|
||||
ScrubCmd::Pause => ScrubWorkerCommand::Pause(Duration::from_secs(3600 * 24)),
|
||||
ScrubCmd::Resume => ScrubWorkerCommand::Resume,
|
||||
ScrubCmd::Cancel => ScrubWorkerCommand::Cancel,
|
||||
ScrubCmd::SetTranquility { tranquility } => {
|
||||
garage
|
||||
.block_manager
|
||||
.scrub_persister
|
||||
.set_with(|x| x.tranquility = tranquility)?;
|
||||
return Ok(());
|
||||
}
|
||||
};
|
||||
info!("Sending command to scrub worker: {:?}", cmd);
|
||||
garage.block_manager.send_scrub_command(cmd).await?;
|
||||
}
|
||||
RepairWhat::Rebalance => {
|
||||
info!("Rebalancing the stored blocks among storage locations");
|
||||
bg.spawn_worker(garage_block::repair::RebalanceWorker::new(
|
||||
garage.block_manager.clone(),
|
||||
));
|
||||
impl RequestHandler for LocalLaunchRepairOperationRequest {
|
||||
type Response = LocalLaunchRepairOperationResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
admin: &Admin,
|
||||
) -> Result<LocalLaunchRepairOperationResponse, Error> {
|
||||
let bg = &admin.background;
|
||||
match self.repair_type {
|
||||
RepairType::Tables => {
|
||||
info!("Launching a full sync of tables");
|
||||
garage.bucket_table.syncer.add_full_sync()?;
|
||||
garage.object_table.syncer.add_full_sync()?;
|
||||
garage.version_table.syncer.add_full_sync()?;
|
||||
garage.block_ref_table.syncer.add_full_sync()?;
|
||||
garage.key_table.syncer.add_full_sync()?;
|
||||
}
|
||||
RepairType::Versions => {
|
||||
info!("Repairing the versions table");
|
||||
bg.spawn_worker(TableRepairWorker::new(garage.clone(), RepairVersions));
|
||||
}
|
||||
RepairType::MultipartUploads => {
|
||||
info!("Repairing the multipart uploads table");
|
||||
bg.spawn_worker(TableRepairWorker::new(garage.clone(), RepairMpu));
|
||||
}
|
||||
RepairType::BlockRefs => {
|
||||
info!("Repairing the block refs table");
|
||||
bg.spawn_worker(TableRepairWorker::new(garage.clone(), RepairBlockRefs));
|
||||
}
|
||||
RepairType::BlockRc => {
|
||||
info!("Repairing the block reference counters");
|
||||
bg.spawn_worker(BlockRcRepair::new(
|
||||
garage.block_manager.clone(),
|
||||
garage.block_ref_table.clone(),
|
||||
));
|
||||
}
|
||||
RepairType::Blocks => {
|
||||
info!("Repairing the stored blocks");
|
||||
bg.spawn_worker(garage_block::repair::RepairWorker::new(
|
||||
garage.block_manager.clone(),
|
||||
));
|
||||
}
|
||||
RepairType::Scrub(cmd) => {
|
||||
let cmd = match cmd {
|
||||
ScrubCommand::Start => ScrubWorkerCommand::Start,
|
||||
ScrubCommand::Pause => {
|
||||
ScrubWorkerCommand::Pause(Duration::from_secs(3600 * 24))
|
||||
}
|
||||
ScrubCommand::Resume => ScrubWorkerCommand::Resume,
|
||||
ScrubCommand::Cancel => ScrubWorkerCommand::Cancel,
|
||||
};
|
||||
info!("Sending command to scrub worker: {:?}", cmd);
|
||||
garage.block_manager.send_scrub_command(cmd).await?;
|
||||
}
|
||||
RepairType::Rebalance => {
|
||||
info!("Rebalancing the stored blocks among storage locations");
|
||||
bg.spawn_worker(garage_block::repair::RebalanceWorker::new(
|
||||
garage.block_manager.clone(),
|
||||
));
|
||||
}
|
||||
}
|
||||
Ok(LocalLaunchRepairOperationResponse)
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
// ----
|
||||
|
@ -103,7 +105,7 @@ trait TableRepair: Send + Sync + 'static {
|
|||
&mut self,
|
||||
garage: &Garage,
|
||||
entry: <<Self as TableRepair>::T as TableSchema>::E,
|
||||
) -> impl Future<Output = Result<bool, Error>> + Send;
|
||||
) -> impl Future<Output = Result<bool, GarageError>> + Send;
|
||||
}
|
||||
|
||||
struct TableRepairWorker<T: TableRepair> {
|
||||
|
@ -139,7 +141,10 @@ impl<R: TableRepair> Worker for TableRepairWorker<R> {
|
|||
}
|
||||
}
|
||||
|
||||
async fn work(&mut self, _must_exit: &mut watch::Receiver<bool>) -> Result<WorkerState, Error> {
|
||||
async fn work(
|
||||
&mut self,
|
||||
_must_exit: &mut watch::Receiver<bool>,
|
||||
) -> Result<WorkerState, GarageError> {
|
||||
let (item_bytes, next_pos) = match R::table(&self.garage).data.store.get_gt(&self.pos)? {
|
||||
Some((k, v)) => (v, k),
|
||||
None => {
|
||||
|
@ -181,7 +186,7 @@ impl TableRepair for RepairVersions {
|
|||
&garage.version_table
|
||||
}
|
||||
|
||||
async fn process(&mut self, garage: &Garage, version: Version) -> Result<bool, Error> {
|
||||
async fn process(&mut self, garage: &Garage, version: Version) -> Result<bool, GarageError> {
|
||||
if !version.deleted.get() {
|
||||
let ref_exists = match &version.backlink {
|
||||
VersionBacklink::Object { bucket_id, key } => garage
|
||||
|
@ -227,7 +232,11 @@ impl TableRepair for RepairBlockRefs {
|
|||
&garage.block_ref_table
|
||||
}
|
||||
|
||||
async fn process(&mut self, garage: &Garage, mut block_ref: BlockRef) -> Result<bool, Error> {
|
||||
async fn process(
|
||||
&mut self,
|
||||
garage: &Garage,
|
||||
mut block_ref: BlockRef,
|
||||
) -> Result<bool, GarageError> {
|
||||
if !block_ref.deleted.get() {
|
||||
let ref_exists = garage
|
||||
.version_table
|
||||
|
@ -262,7 +271,11 @@ impl TableRepair for RepairMpu {
|
|||
&garage.mpu_table
|
||||
}
|
||||
|
||||
async fn process(&mut self, garage: &Garage, mut mpu: MultipartUpload) -> Result<bool, Error> {
|
||||
async fn process(
|
||||
&mut self,
|
||||
garage: &Garage,
|
||||
mut mpu: MultipartUpload,
|
||||
) -> Result<bool, GarageError> {
|
||||
if !mpu.deleted.get() {
|
||||
let ref_exists = garage
|
||||
.object_table
|
||||
|
@ -329,7 +342,10 @@ impl Worker for BlockRcRepair {
|
|||
}
|
||||
}
|
||||
|
||||
async fn work(&mut self, _must_exit: &mut watch::Receiver<bool>) -> Result<WorkerState, Error> {
|
||||
async fn work(
|
||||
&mut self,
|
||||
_must_exit: &mut watch::Receiver<bool>,
|
||||
) -> Result<WorkerState, GarageError> {
|
||||
for _i in 0..RC_REPAIR_ITER_COUNT {
|
||||
let next1 = self
|
||||
.block_manager
|
|
@ -7,12 +7,6 @@ use garage_api_common::router_macros::*;
|
|||
use crate::error::*;
|
||||
use crate::router_v0;
|
||||
|
||||
pub enum Authorization {
|
||||
None,
|
||||
MetricsToken,
|
||||
AdminToken,
|
||||
}
|
||||
|
||||
router_match! {@func
|
||||
|
||||
/// List of all Admin API endpoints.
|
||||
|
@ -211,15 +205,6 @@ impl Endpoint {
|
|||
))),
|
||||
}
|
||||
}
|
||||
/// Get the kind of authorization which is required to perform the operation.
|
||||
pub fn authorization_type(&self) -> Authorization {
|
||||
match self {
|
||||
Self::Health => Authorization::None,
|
||||
Self::CheckDomain => Authorization::None,
|
||||
Self::Metrics => Authorization::MetricsToken,
|
||||
_ => Authorization::AdminToken,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
generateQueryParameters! {
|
||||
|
|
275
src/api/admin/router_v2.rs
Normal file
275
src/api/admin/router_v2.rs
Normal file
|
@ -0,0 +1,275 @@
|
|||
use std::borrow::Cow;
|
||||
|
||||
use hyper::body::Incoming as IncomingBody;
|
||||
use hyper::{Method, Request};
|
||||
use paste::paste;
|
||||
|
||||
use garage_api_common::helpers::*;
|
||||
use garage_api_common::router_macros::*;
|
||||
|
||||
use crate::api::*;
|
||||
use crate::error::*;
|
||||
use crate::router_v1;
|
||||
use crate::Authorization;
|
||||
|
||||
impl AdminApiRequest {
|
||||
/// Determine which S3 endpoint a request is for using the request, and a bucket which was
|
||||
/// possibly extracted from the Host header.
|
||||
/// Returns Self plus bucket name, if endpoint is not Endpoint::ListBuckets
|
||||
pub async fn from_request(req: Request<IncomingBody>) -> Result<Self, Error> {
|
||||
let uri = req.uri().clone();
|
||||
let path = uri.path();
|
||||
let query = uri.query();
|
||||
|
||||
let method = req.method().clone();
|
||||
|
||||
let mut query = QueryParameters::from_query(query.unwrap_or_default())?;
|
||||
|
||||
let res = router_match!(@gen_path_parser_v2 (&method, path, "/v2/", query, req) [
|
||||
@special OPTIONS _ => Options (),
|
||||
@special GET "/check" => CheckDomain (query::domain),
|
||||
@special GET "/health" => Health (),
|
||||
@special GET "/metrics" => Metrics (),
|
||||
// Cluster endpoints
|
||||
GET GetClusterStatus (),
|
||||
GET GetClusterHealth (),
|
||||
POST ConnectClusterNodes (body),
|
||||
// Admin token endpoints
|
||||
GET ListAdminTokens (),
|
||||
GET GetAdminTokenInfo (query_opt::id, query_opt::search),
|
||||
POST CreateAdminToken (body),
|
||||
POST UpdateAdminToken (body_field, query::id),
|
||||
POST DeleteAdminToken (query::id),
|
||||
// Layout endpoints
|
||||
GET GetClusterLayout (),
|
||||
GET GetClusterLayoutHistory (),
|
||||
POST UpdateClusterLayout (body),
|
||||
POST PreviewClusterLayoutChanges (),
|
||||
POST ApplyClusterLayout (body),
|
||||
POST RevertClusterLayout (),
|
||||
POST ClusterLayoutSkipDeadNodes (body),
|
||||
// API key endpoints
|
||||
GET GetKeyInfo (query_opt::id, query_opt::search, parse_default(false)::show_secret_key),
|
||||
POST UpdateKey (body_field, query::id),
|
||||
POST CreateKey (body),
|
||||
POST ImportKey (body),
|
||||
POST DeleteKey (query::id),
|
||||
GET ListKeys (),
|
||||
// Bucket endpoints
|
||||
GET GetBucketInfo (query_opt::id, query_opt::global_alias, query_opt::search),
|
||||
GET ListBuckets (),
|
||||
POST CreateBucket (body),
|
||||
POST DeleteBucket (query::id),
|
||||
POST UpdateBucket (body_field, query::id),
|
||||
POST CleanupIncompleteUploads (body),
|
||||
GET InspectObject (query::bucket_id, query::key),
|
||||
// Bucket-key permissions
|
||||
POST AllowBucketKey (body),
|
||||
POST DenyBucketKey (body),
|
||||
// Bucket aliases
|
||||
POST AddBucketAlias (body),
|
||||
POST RemoveBucketAlias (body),
|
||||
// Node APIs
|
||||
GET GetNodeInfo (default::body, query::node),
|
||||
POST CreateMetadataSnapshot (default::body, query::node),
|
||||
GET GetNodeStatistics (default::body, query::node),
|
||||
GET GetClusterStatistics (),
|
||||
POST LaunchRepairOperation (body_field, query::node),
|
||||
// Worker APIs
|
||||
POST ListWorkers (body_field, query::node),
|
||||
POST GetWorkerInfo (body_field, query::node),
|
||||
POST GetWorkerVariable (body_field, query::node),
|
||||
POST SetWorkerVariable (body_field, query::node),
|
||||
// Block APIs
|
||||
GET ListBlockErrors (default::body, query::node),
|
||||
POST GetBlockInfo (body_field, query::node),
|
||||
POST RetryBlockResync (body_field, query::node),
|
||||
POST PurgeBlocks (body_field, query::node),
|
||||
]);
|
||||
|
||||
if let Some(message) = query.nonempty_message() {
|
||||
debug!("Unused query parameter: {}", message)
|
||||
}
|
||||
|
||||
Ok(res)
|
||||
}
|
||||
|
||||
/// Some endpoints work exactly the same in their v2/ version as they did in their v1/ version.
|
||||
/// For these endpoints, we can convert a v1/ call to its equivalent as if it was made using
|
||||
/// its v2/ URL.
|
||||
pub async fn from_v1(
|
||||
v1_endpoint: router_v1::Endpoint,
|
||||
req: Request<IncomingBody>,
|
||||
) -> Result<Self, Error> {
|
||||
use router_v1::Endpoint;
|
||||
|
||||
match v1_endpoint {
|
||||
// GetClusterStatus semantics changed:
|
||||
// info about local node is no longer returned
|
||||
Endpoint::GetClusterHealth => {
|
||||
Ok(AdminApiRequest::GetClusterHealth(GetClusterHealthRequest))
|
||||
}
|
||||
Endpoint::ConnectClusterNodes => {
|
||||
let req = parse_json_body::<ConnectClusterNodesRequest, _, Error>(req).await?;
|
||||
Ok(AdminApiRequest::ConnectClusterNodes(req))
|
||||
}
|
||||
|
||||
// Layout
|
||||
Endpoint::GetClusterLayout => {
|
||||
Ok(AdminApiRequest::GetClusterLayout(GetClusterLayoutRequest))
|
||||
}
|
||||
// UpdateClusterLayout semantics changed
|
||||
Endpoint::ApplyClusterLayout => {
|
||||
let param = parse_json_body::<ApplyClusterLayoutRequest, _, Error>(req).await?;
|
||||
Ok(AdminApiRequest::ApplyClusterLayout(param))
|
||||
}
|
||||
Endpoint::RevertClusterLayout => Ok(AdminApiRequest::RevertClusterLayout(
|
||||
RevertClusterLayoutRequest,
|
||||
)),
|
||||
|
||||
// Keys
|
||||
Endpoint::ListKeys => Ok(AdminApiRequest::ListKeys(ListKeysRequest)),
|
||||
Endpoint::GetKeyInfo {
|
||||
id,
|
||||
search,
|
||||
show_secret_key,
|
||||
} => {
|
||||
let show_secret_key = show_secret_key.map(|x| x == "true").unwrap_or(false);
|
||||
Ok(AdminApiRequest::GetKeyInfo(GetKeyInfoRequest {
|
||||
id,
|
||||
search,
|
||||
show_secret_key,
|
||||
}))
|
||||
}
|
||||
Endpoint::CreateKey => {
|
||||
let req = parse_json_body::<CreateKeyRequest, _, Error>(req).await?;
|
||||
Ok(AdminApiRequest::CreateKey(req))
|
||||
}
|
||||
Endpoint::ImportKey => {
|
||||
let req = parse_json_body::<ImportKeyRequest, _, Error>(req).await?;
|
||||
Ok(AdminApiRequest::ImportKey(req))
|
||||
}
|
||||
Endpoint::UpdateKey { id } => {
|
||||
let body = parse_json_body::<UpdateKeyRequestBody, _, Error>(req).await?;
|
||||
Ok(AdminApiRequest::UpdateKey(UpdateKeyRequest { id, body }))
|
||||
}
|
||||
|
||||
// DeleteKey semantics changed:
|
||||
// - in v1/ : HTTP DELETE => HTTP 204 No Content
|
||||
// - in v2/ : HTTP POST => HTTP 200 Ok
|
||||
// Endpoint::DeleteKey { id } => Ok(AdminApiRequest::DeleteKey(DeleteKeyRequest { id })),
|
||||
|
||||
// Buckets
|
||||
Endpoint::ListBuckets => Ok(AdminApiRequest::ListBuckets(ListBucketsRequest)),
|
||||
Endpoint::GetBucketInfo { id, global_alias } => {
|
||||
Ok(AdminApiRequest::GetBucketInfo(GetBucketInfoRequest {
|
||||
id,
|
||||
global_alias,
|
||||
search: None,
|
||||
}))
|
||||
}
|
||||
Endpoint::CreateBucket => {
|
||||
let req = parse_json_body::<CreateBucketRequest, _, Error>(req).await?;
|
||||
Ok(AdminApiRequest::CreateBucket(req))
|
||||
}
|
||||
|
||||
// DeleteBucket semantics changed::
|
||||
// - in v1/ : HTTP DELETE => HTTP 204 No Content
|
||||
// - in v2/ : HTTP POST => HTTP 200 Ok
|
||||
// Endpoint::DeleteBucket { id } => {
|
||||
// Ok(AdminApiRequest::DeleteBucket(DeleteBucketRequest { id }))
|
||||
// }
|
||||
Endpoint::UpdateBucket { id } => {
|
||||
let body = parse_json_body::<UpdateBucketRequestBody, _, Error>(req).await?;
|
||||
Ok(AdminApiRequest::UpdateBucket(UpdateBucketRequest {
|
||||
id,
|
||||
body,
|
||||
}))
|
||||
}
|
||||
|
||||
// Bucket-key permissions
|
||||
Endpoint::BucketAllowKey => {
|
||||
let req = parse_json_body::<BucketKeyPermChangeRequest, _, Error>(req).await?;
|
||||
Ok(AdminApiRequest::AllowBucketKey(AllowBucketKeyRequest(req)))
|
||||
}
|
||||
Endpoint::BucketDenyKey => {
|
||||
let req = parse_json_body::<BucketKeyPermChangeRequest, _, Error>(req).await?;
|
||||
Ok(AdminApiRequest::DenyBucketKey(DenyBucketKeyRequest(req)))
|
||||
}
|
||||
// Bucket aliasing
|
||||
Endpoint::GlobalAliasBucket { id, alias } => {
|
||||
Ok(AdminApiRequest::AddBucketAlias(AddBucketAliasRequest {
|
||||
bucket_id: id,
|
||||
alias: BucketAliasEnum::Global {
|
||||
global_alias: alias,
|
||||
},
|
||||
}))
|
||||
}
|
||||
Endpoint::GlobalUnaliasBucket { id, alias } => Ok(AdminApiRequest::RemoveBucketAlias(
|
||||
RemoveBucketAliasRequest {
|
||||
bucket_id: id,
|
||||
alias: BucketAliasEnum::Global {
|
||||
global_alias: alias,
|
||||
},
|
||||
},
|
||||
)),
|
||||
Endpoint::LocalAliasBucket {
|
||||
id,
|
||||
access_key_id,
|
||||
alias,
|
||||
} => Ok(AdminApiRequest::AddBucketAlias(AddBucketAliasRequest {
|
||||
bucket_id: id,
|
||||
alias: BucketAliasEnum::Local {
|
||||
local_alias: alias,
|
||||
access_key_id,
|
||||
},
|
||||
})),
|
||||
Endpoint::LocalUnaliasBucket {
|
||||
id,
|
||||
access_key_id,
|
||||
alias,
|
||||
} => Ok(AdminApiRequest::RemoveBucketAlias(
|
||||
RemoveBucketAliasRequest {
|
||||
bucket_id: id,
|
||||
alias: BucketAliasEnum::Local {
|
||||
local_alias: alias,
|
||||
access_key_id,
|
||||
},
|
||||
},
|
||||
)),
|
||||
|
||||
// For endpoints that have different body content syntax, issue
|
||||
// deprecation warning
|
||||
_ => Err(Error::bad_request(format!(
|
||||
"v1/ endpoint is no longer supported: {}",
|
||||
v1_endpoint.name()
|
||||
))),
|
||||
}
|
||||
}
|
||||
|
||||
/// Get the kind of authorization which is required to perform the operation.
|
||||
pub fn authorization_type(&self) -> Authorization {
|
||||
match self {
|
||||
Self::Options(_) | Self::Health(_) | Self::CheckDomain(_) => Authorization::None,
|
||||
Self::Metrics(_) => Authorization::MetricsToken,
|
||||
_ => Authorization::AdminToken,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
generateQueryParameters! {
|
||||
keywords: [],
|
||||
fields: [
|
||||
"node" => node,
|
||||
"domain" => domain,
|
||||
"format" => format,
|
||||
"id" => id,
|
||||
"search" => search,
|
||||
"globalAlias" => global_alias,
|
||||
"alias" => alias,
|
||||
"accessKeyId" => access_key_id,
|
||||
"showSecretKey" => show_secret_key,
|
||||
"bucketId" => bucket_id,
|
||||
"key" => key
|
||||
]
|
||||
}
|
173
src/api/admin/special.rs
Normal file
173
src/api/admin/special.rs
Normal file
|
@ -0,0 +1,173 @@
|
|||
use std::sync::Arc;
|
||||
|
||||
use http::header::{
|
||||
ACCESS_CONTROL_ALLOW_HEADERS, ACCESS_CONTROL_ALLOW_METHODS, ACCESS_CONTROL_ALLOW_ORIGIN, ALLOW,
|
||||
};
|
||||
use hyper::{Response, StatusCode};
|
||||
|
||||
#[cfg(feature = "metrics")]
|
||||
use prometheus::{Encoder, TextEncoder};
|
||||
|
||||
use garage_model::garage::Garage;
|
||||
use garage_rpc::system::ClusterHealthStatus;
|
||||
|
||||
use garage_api_common::helpers::*;
|
||||
|
||||
use crate::api::{CheckDomainRequest, HealthRequest, MetricsRequest, OptionsRequest};
|
||||
use crate::api_server::ResBody;
|
||||
use crate::error::*;
|
||||
use crate::{Admin, RequestHandler};
|
||||
|
||||
impl RequestHandler for OptionsRequest {
|
||||
type Response = Response<ResBody>;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
_garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<Response<ResBody>, Error> {
|
||||
Ok(Response::builder()
|
||||
.status(StatusCode::OK)
|
||||
.header(ALLOW, "OPTIONS,GET,POST")
|
||||
.header(ACCESS_CONTROL_ALLOW_METHODS, "OPTIONS,GET,POST")
|
||||
.header(ACCESS_CONTROL_ALLOW_HEADERS, "authorization,content-type")
|
||||
.header(ACCESS_CONTROL_ALLOW_ORIGIN, "*")
|
||||
.body(empty_body())?)
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for MetricsRequest {
|
||||
type Response = Response<ResBody>;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
_garage: &Arc<Garage>,
|
||||
admin: &Admin,
|
||||
) -> Result<Response<ResBody>, Error> {
|
||||
#[cfg(feature = "metrics")]
|
||||
{
|
||||
use opentelemetry::trace::Tracer;
|
||||
|
||||
let mut buffer = vec![];
|
||||
let encoder = TextEncoder::new();
|
||||
|
||||
let tracer = opentelemetry::global::tracer("garage");
|
||||
let metric_families = tracer.in_span("admin/gather_metrics", |_| {
|
||||
admin.exporter.registry().gather()
|
||||
});
|
||||
|
||||
encoder
|
||||
.encode(&metric_families, &mut buffer)
|
||||
.ok_or_internal_error("Could not serialize metrics")?;
|
||||
|
||||
Ok(Response::builder()
|
||||
.status(StatusCode::OK)
|
||||
.header(http::header::CONTENT_TYPE, encoder.format_type())
|
||||
.body(bytes_body(buffer.into()))?)
|
||||
}
|
||||
#[cfg(not(feature = "metrics"))]
|
||||
Err(Error::bad_request(
|
||||
"Garage was built without the metrics feature".to_string(),
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for HealthRequest {
|
||||
type Response = Response<ResBody>;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<Response<ResBody>, Error> {
|
||||
let health = garage.system.health();
|
||||
|
||||
let (status, status_str) = match health.status {
|
||||
ClusterHealthStatus::Healthy => (StatusCode::OK, "Garage is fully operational"),
|
||||
ClusterHealthStatus::Degraded => (
|
||||
StatusCode::OK,
|
||||
"Garage is operational but some storage nodes are unavailable",
|
||||
),
|
||||
ClusterHealthStatus::Unavailable => (
|
||||
StatusCode::SERVICE_UNAVAILABLE,
|
||||
"Quorum is not available for some/all partitions, reads and writes will fail",
|
||||
),
|
||||
};
|
||||
let status_str = format!(
|
||||
"{}\nConsult the full health check API endpoint at /v2/GetClusterHealth for more details\n",
|
||||
status_str
|
||||
);
|
||||
|
||||
Ok(Response::builder()
|
||||
.status(status)
|
||||
.header(http::header::CONTENT_TYPE, "text/plain")
|
||||
.body(string_body(status_str))?)
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for CheckDomainRequest {
|
||||
type Response = Response<ResBody>;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<Response<ResBody>, Error> {
|
||||
if check_domain(garage, &self.domain).await? {
|
||||
Ok(Response::builder()
|
||||
.status(StatusCode::OK)
|
||||
.body(string_body(format!(
|
||||
"Domain '{}' is managed by Garage",
|
||||
self.domain
|
||||
)))?)
|
||||
} else {
|
||||
Err(Error::bad_request(format!(
|
||||
"Domain '{}' is not managed by Garage",
|
||||
self.domain
|
||||
)))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
async fn check_domain(garage: &Arc<Garage>, domain: &str) -> Result<bool, Error> {
|
||||
// Resolve bucket from domain name, inferring if the website must be activated for the
|
||||
// domain to be valid.
|
||||
let (bucket_name, must_check_website) = if let Some(bname) = garage
|
||||
.config
|
||||
.s3_api
|
||||
.root_domain
|
||||
.as_ref()
|
||||
.and_then(|rd| host_to_bucket(domain, rd))
|
||||
{
|
||||
(bname.to_string(), false)
|
||||
} else if let Some(bname) = garage
|
||||
.config
|
||||
.s3_web
|
||||
.as_ref()
|
||||
.and_then(|sw| host_to_bucket(domain, sw.root_domain.as_str()))
|
||||
{
|
||||
(bname.to_string(), true)
|
||||
} else {
|
||||
(domain.to_string(), true)
|
||||
};
|
||||
|
||||
let bucket = match garage
|
||||
.bucket_helper()
|
||||
.resolve_global_bucket_fast(&bucket_name)?
|
||||
{
|
||||
Some(b) => b,
|
||||
None => return Ok(false),
|
||||
};
|
||||
|
||||
if !must_check_website {
|
||||
return Ok(true);
|
||||
}
|
||||
|
||||
let bucket_state = bucket.state.as_option().unwrap();
|
||||
let bucket_website_config = bucket_state.website_config.get();
|
||||
|
||||
match bucket_website_config {
|
||||
Some(_v) => Ok(true),
|
||||
None => Ok(false),
|
||||
}
|
||||
}
|
118
src/api/admin/worker.rs
Normal file
118
src/api/admin/worker.rs
Normal file
|
@ -0,0 +1,118 @@
|
|||
use std::collections::HashMap;
|
||||
use std::sync::Arc;
|
||||
|
||||
use garage_util::background::*;
|
||||
use garage_util::time::now_msec;
|
||||
|
||||
use garage_model::garage::Garage;
|
||||
|
||||
use crate::api::*;
|
||||
use crate::error::Error;
|
||||
use crate::{Admin, RequestHandler};
|
||||
|
||||
impl RequestHandler for LocalListWorkersRequest {
|
||||
type Response = LocalListWorkersResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
_garage: &Arc<Garage>,
|
||||
admin: &Admin,
|
||||
) -> Result<LocalListWorkersResponse, Error> {
|
||||
let workers = admin.background.get_worker_info();
|
||||
let info = workers
|
||||
.into_iter()
|
||||
.filter(|(_, w)| {
|
||||
(!self.busy_only
|
||||
|| matches!(w.state, WorkerState::Busy | WorkerState::Throttled(_)))
|
||||
&& (!self.error_only || w.errors > 0)
|
||||
})
|
||||
.map(|(id, w)| worker_info_to_api(id as u64, w))
|
||||
.collect::<Vec<_>>();
|
||||
Ok(LocalListWorkersResponse(info))
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for LocalGetWorkerInfoRequest {
|
||||
type Response = LocalGetWorkerInfoResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
_garage: &Arc<Garage>,
|
||||
admin: &Admin,
|
||||
) -> Result<LocalGetWorkerInfoResponse, Error> {
|
||||
let info = admin
|
||||
.background
|
||||
.get_worker_info()
|
||||
.get(&(self.id as usize))
|
||||
.ok_or(Error::NoSuchWorker(self.id))?
|
||||
.clone();
|
||||
Ok(LocalGetWorkerInfoResponse(worker_info_to_api(
|
||||
self.id, info,
|
||||
)))
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for LocalGetWorkerVariableRequest {
|
||||
type Response = LocalGetWorkerVariableResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<LocalGetWorkerVariableResponse, Error> {
|
||||
let mut res = HashMap::new();
|
||||
if let Some(k) = self.variable {
|
||||
res.insert(k.clone(), garage.bg_vars.get(&k)?);
|
||||
} else {
|
||||
let vars = garage.bg_vars.get_all();
|
||||
for (k, v) in vars.iter() {
|
||||
res.insert(k.to_string(), v.to_string());
|
||||
}
|
||||
}
|
||||
Ok(LocalGetWorkerVariableResponse(res))
|
||||
}
|
||||
}
|
||||
|
||||
impl RequestHandler for LocalSetWorkerVariableRequest {
|
||||
type Response = LocalSetWorkerVariableResponse;
|
||||
|
||||
async fn handle(
|
||||
self,
|
||||
garage: &Arc<Garage>,
|
||||
_admin: &Admin,
|
||||
) -> Result<LocalSetWorkerVariableResponse, Error> {
|
||||
garage.bg_vars.set(&self.variable, &self.value)?;
|
||||
|
||||
Ok(LocalSetWorkerVariableResponse {
|
||||
variable: self.variable,
|
||||
value: self.value,
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
// ---- helper functions ----
|
||||
|
||||
fn worker_info_to_api(id: u64, info: WorkerInfo) -> WorkerInfoResp {
|
||||
WorkerInfoResp {
|
||||
id,
|
||||
name: info.name,
|
||||
state: match info.state {
|
||||
WorkerState::Busy => WorkerStateResp::Busy,
|
||||
WorkerState::Throttled(t) => WorkerStateResp::Throttled { duration_secs: t },
|
||||
WorkerState::Idle => WorkerStateResp::Idle,
|
||||
WorkerState::Done => WorkerStateResp::Done,
|
||||
},
|
||||
errors: info.errors as u64,
|
||||
consecutive_errors: info.consecutive_errors as u64,
|
||||
last_error: info.last_error.map(|(message, t)| WorkerLastError {
|
||||
message,
|
||||
secs_ago: now_msec().saturating_sub(t) / 1000,
|
||||
}),
|
||||
|
||||
tranquility: info.status.tranquility,
|
||||
progress: info.status.progress,
|
||||
queue_length: info.status.queue_length,
|
||||
persistent_errors: info.status.persistent_errors,
|
||||
freeform: info.status.freeform,
|
||||
}
|
||||
}
|
|
@ -23,6 +23,7 @@ bytes.workspace = true
|
|||
chrono.workspace = true
|
||||
crc32fast.workspace = true
|
||||
crc32c.workspace = true
|
||||
crc64fast-nvme.workspace = true
|
||||
crypto-common.workspace = true
|
||||
err-derive.workspace = true
|
||||
hex.workspace = true
|
||||
|
|
|
@ -9,9 +9,7 @@ use hyper::{body::Body, body::Incoming as IncomingBody, Request, Response, Statu
|
|||
use garage_model::bucket_table::{BucketParams, CorsRule as GarageCorsRule};
|
||||
use garage_model::garage::Garage;
|
||||
|
||||
use crate::common_error::{
|
||||
helper_error_as_internal, CommonError, OkOrBadRequest, OkOrInternalError,
|
||||
};
|
||||
use crate::common_error::{CommonError, OkOrBadRequest, OkOrInternalError};
|
||||
use crate::helpers::*;
|
||||
|
||||
pub fn find_matching_cors_rule<'a, B>(
|
||||
|
@ -76,7 +74,7 @@ pub fn add_cors_headers(
|
|||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn handle_options_api(
|
||||
pub fn handle_options_api(
|
||||
garage: Arc<Garage>,
|
||||
req: &Request<IncomingBody>,
|
||||
bucket_name: Option<String>,
|
||||
|
@ -93,16 +91,8 @@ pub async fn handle_options_api(
|
|||
// OPTIONS calls are not auhtenticated).
|
||||
if let Some(bn) = bucket_name {
|
||||
let helper = garage.bucket_helper();
|
||||
let bucket_id = helper
|
||||
.resolve_global_bucket_name(&bn)
|
||||
.await
|
||||
.map_err(helper_error_as_internal)?;
|
||||
if let Some(id) = bucket_id {
|
||||
let bucket = garage
|
||||
.bucket_helper()
|
||||
.get_existing_bucket(id)
|
||||
.await
|
||||
.map_err(helper_error_as_internal)?;
|
||||
let bucket_opt = helper.resolve_global_bucket_fast(&bn)?;
|
||||
if let Some(bucket) = bucket_opt {
|
||||
let bucket_params = bucket.state.into_option().unwrap();
|
||||
handle_options_for_bucket(req, &bucket_params)
|
||||
} else {
|
||||
|
|
|
@ -1,3 +1,4 @@
|
|||
use std::borrow::Cow;
|
||||
use std::convert::Infallible;
|
||||
use std::fs::{self, Permissions};
|
||||
use std::os::unix::fs::PermissionsExt;
|
||||
|
@ -35,7 +36,7 @@ use garage_util::socket_address::UnixOrTCPSocketAddress;
|
|||
use crate::helpers::{BoxBody, ErrorBody};
|
||||
|
||||
pub trait ApiEndpoint: Send + Sync + 'static {
|
||||
fn name(&self) -> &'static str;
|
||||
fn name(&self) -> Cow<'static, str>;
|
||||
fn add_span_attributes(&self, span: SpanRef<'_>);
|
||||
}
|
||||
|
||||
|
|
|
@ -45,6 +45,68 @@ macro_rules! router_match {
|
|||
}
|
||||
}
|
||||
}};
|
||||
(@gen_path_parser_v2 ($method:expr, $reqpath:expr, $pathprefix:literal, $query:expr, $req:expr)
|
||||
[
|
||||
$(@special $spec_meth:ident $spec_path:pat => $spec_api:ident $spec_params:tt,)*
|
||||
$($meth:ident $api:ident $params:tt,)*
|
||||
]) => {{
|
||||
{
|
||||
#[allow(unused_parens)]
|
||||
match ($method, $reqpath) {
|
||||
$(
|
||||
(&Method::$spec_meth, $spec_path) => AdminApiRequest::$spec_api (
|
||||
router_match!(@@gen_parse_request $spec_api, $spec_params, $query, $req)
|
||||
),
|
||||
)*
|
||||
$(
|
||||
(&Method::$meth, concat!($pathprefix, stringify!($api)))
|
||||
=> AdminApiRequest::$api (
|
||||
router_match!(@@gen_parse_request $api, $params, $query, $req)
|
||||
),
|
||||
)*
|
||||
(m, p) => {
|
||||
return Err(Error::bad_request(format!(
|
||||
"Unknown API endpoint: {} {}",
|
||||
m, p
|
||||
)))
|
||||
}
|
||||
}
|
||||
}
|
||||
}};
|
||||
(@@gen_parse_request $api:ident, (), $query: expr, $req:expr) => {{
|
||||
paste!(
|
||||
[< $api Request >]
|
||||
)
|
||||
}};
|
||||
(@@gen_parse_request $api:ident, (body), $query: expr, $req:expr) => {{
|
||||
paste!({
|
||||
parse_json_body::< [<$api Request>], _, Error>($req).await?
|
||||
})
|
||||
}};
|
||||
(@@gen_parse_request $api:ident, (body_field, $($conv:ident $(($conv_arg:expr))? :: $param:ident),*), $query: expr, $req:expr)
|
||||
=>
|
||||
{{
|
||||
paste!({
|
||||
let body = parse_json_body::< [<$api RequestBody>], _, Error>($req).await?;
|
||||
[< $api Request >] {
|
||||
body,
|
||||
$(
|
||||
$param: router_match!(@@parse_param $query, $conv $(($conv_arg))?, $param),
|
||||
)+
|
||||
}
|
||||
})
|
||||
}};
|
||||
(@@gen_parse_request $api:ident, ($($conv:ident $(($conv_arg:expr))? :: $param:ident),*), $query: expr, $req:expr)
|
||||
=>
|
||||
{{
|
||||
paste!({
|
||||
[< $api Request >] {
|
||||
$(
|
||||
$param: router_match!(@@parse_param $query, $conv $(($conv_arg))?, $param),
|
||||
)+
|
||||
}
|
||||
})
|
||||
}};
|
||||
(@gen_parser ($keyword:expr, $key:ident, $query:expr, $header:expr),
|
||||
key: [$($kw_k:ident $(if $required_k:ident)? $(header $header_k:expr)? => $api_k:ident $(($($conv_k:ident :: $param_k:ident),*))?,)*],
|
||||
no_key: [$($kw_nk:ident $(if $required_nk:ident)? $(if_header $header_nk:expr)? => $api_nk:ident $(($($conv_nk:ident :: $param_nk:ident),*))?,)*]) => {{
|
||||
|
@ -79,13 +141,19 @@ macro_rules! router_match {
|
|||
}
|
||||
}};
|
||||
|
||||
(@@parse_param $query:expr, default, $param:ident) => {{
|
||||
Default::default()
|
||||
}};
|
||||
(@@parse_param $query:expr, query_opt, $param:ident) => {{
|
||||
// extract optional query parameter
|
||||
$query.$param.take().map(|param| param.into_owned())
|
||||
}};
|
||||
(@@parse_param $query:expr, query, $param:ident) => {{
|
||||
// extract mendatory query parameter
|
||||
$query.$param.take().ok_or_bad_request("Missing argument for endpoint")?.into_owned()
|
||||
$query.$param.take()
|
||||
.ok_or_bad_request(
|
||||
format!("Missing argument `{}` for endpoint", stringify!($param))
|
||||
)?.into_owned()
|
||||
}};
|
||||
(@@parse_param $query:expr, opt_parse, $param:ident) => {{
|
||||
// extract and parse optional query parameter
|
||||
|
@ -99,10 +167,22 @@ macro_rules! router_match {
|
|||
(@@parse_param $query:expr, parse, $param:ident) => {{
|
||||
// extract and parse mandatory query parameter
|
||||
// both missing and un-parseable parameters are reported as errors
|
||||
$query.$param.take().ok_or_bad_request("Missing argument for endpoint")?
|
||||
$query.$param.take()
|
||||
.ok_or_bad_request(
|
||||
format!("Missing argument `{}` for endpoint", stringify!($param))
|
||||
)?
|
||||
.parse()
|
||||
.map_err(|_| Error::bad_request("Failed to parse query parameter"))?
|
||||
}};
|
||||
(@@parse_param $query:expr, parse_default($default:expr), $param:ident) => {{
|
||||
// extract and parse optional query parameter
|
||||
// using provided value as default if paramter is missing
|
||||
$query.$param.take().map(|x| x
|
||||
.parse()
|
||||
.map_err(|_| Error::bad_request("Failed to parse query parameter")))
|
||||
.transpose()?
|
||||
.unwrap_or($default)
|
||||
}};
|
||||
(@func
|
||||
$(#[$doc:meta])*
|
||||
pub enum Endpoint {
|
||||
|
@ -187,6 +267,7 @@ macro_rules! generateQueryParameters {
|
|||
},
|
||||
)*
|
||||
$(
|
||||
// FIXME: remove if !v.is_empty() ?
|
||||
$f_param => if !v.is_empty() {
|
||||
if res.$f_name.replace(v).is_some() {
|
||||
return Err(Error::bad_request(format!(
|
||||
|
|
|
@ -4,6 +4,7 @@ use std::hash::Hasher;
|
|||
use base64::prelude::*;
|
||||
use crc32c::Crc32cHasher as Crc32c;
|
||||
use crc32fast::Hasher as Crc32;
|
||||
use crc64fast_nvme::Digest as Crc64Nvme;
|
||||
use md5::{Digest, Md5};
|
||||
use sha1::Sha1;
|
||||
use sha2::Sha256;
|
||||
|
@ -23,11 +24,14 @@ pub const X_AMZ_CHECKSUM_ALGORITHM: HeaderName =
|
|||
pub const X_AMZ_CHECKSUM_MODE: HeaderName = HeaderName::from_static("x-amz-checksum-mode");
|
||||
pub const X_AMZ_CHECKSUM_CRC32: HeaderName = HeaderName::from_static("x-amz-checksum-crc32");
|
||||
pub const X_AMZ_CHECKSUM_CRC32C: HeaderName = HeaderName::from_static("x-amz-checksum-crc32c");
|
||||
pub const X_AMZ_CHECKSUM_CRC64NVME: HeaderName =
|
||||
HeaderName::from_static("x-amz-checksum-crc64nvme");
|
||||
pub const X_AMZ_CHECKSUM_SHA1: HeaderName = HeaderName::from_static("x-amz-checksum-sha1");
|
||||
pub const X_AMZ_CHECKSUM_SHA256: HeaderName = HeaderName::from_static("x-amz-checksum-sha256");
|
||||
|
||||
pub type Crc32Checksum = [u8; 4];
|
||||
pub type Crc32cChecksum = [u8; 4];
|
||||
pub type Crc64NvmeChecksum = [u8; 8];
|
||||
pub type Md5Checksum = [u8; 16];
|
||||
pub type Sha1Checksum = [u8; 20];
|
||||
pub type Sha256Checksum = [u8; 32];
|
||||
|
@ -45,6 +49,7 @@ pub struct ExpectedChecksums {
|
|||
pub struct Checksummer {
|
||||
pub crc32: Option<Crc32>,
|
||||
pub crc32c: Option<Crc32c>,
|
||||
pub crc64nvme: Option<Crc64Nvme>,
|
||||
pub md5: Option<Md5>,
|
||||
pub sha1: Option<Sha1>,
|
||||
pub sha256: Option<Sha256>,
|
||||
|
@ -54,6 +59,7 @@ pub struct Checksummer {
|
|||
pub struct Checksums {
|
||||
pub crc32: Option<Crc32Checksum>,
|
||||
pub crc32c: Option<Crc32cChecksum>,
|
||||
pub crc64nvme: Option<Crc64NvmeChecksum>,
|
||||
pub md5: Option<Md5Checksum>,
|
||||
pub sha1: Option<Sha1Checksum>,
|
||||
pub sha256: Option<Sha256Checksum>,
|
||||
|
@ -64,6 +70,7 @@ impl Checksummer {
|
|||
Self {
|
||||
crc32: None,
|
||||
crc32c: None,
|
||||
crc64nvme: None,
|
||||
md5: None,
|
||||
sha1: None,
|
||||
sha256: None,
|
||||
|
@ -96,6 +103,9 @@ impl Checksummer {
|
|||
if matches!(&expected.extra, Some(ChecksumValue::Crc32c(_))) {
|
||||
self.crc32c = Some(Crc32c::default());
|
||||
}
|
||||
if matches!(&expected.extra, Some(ChecksumValue::Crc64Nvme(_))) {
|
||||
self.crc64nvme = Some(Crc64Nvme::default());
|
||||
}
|
||||
if matches!(&expected.extra, Some(ChecksumValue::Sha1(_))) {
|
||||
self.sha1 = Some(Sha1::new());
|
||||
}
|
||||
|
@ -109,6 +119,9 @@ impl Checksummer {
|
|||
Some(ChecksumAlgorithm::Crc32c) => {
|
||||
self.crc32c = Some(Crc32c::default());
|
||||
}
|
||||
Some(ChecksumAlgorithm::Crc64Nvme) => {
|
||||
self.crc64nvme = Some(Crc64Nvme::default());
|
||||
}
|
||||
Some(ChecksumAlgorithm::Sha1) => {
|
||||
self.sha1 = Some(Sha1::new());
|
||||
}
|
||||
|
@ -127,6 +140,9 @@ impl Checksummer {
|
|||
if let Some(crc32c) = &mut self.crc32c {
|
||||
crc32c.write(bytes);
|
||||
}
|
||||
if let Some(crc64nvme) = &mut self.crc64nvme {
|
||||
crc64nvme.write(bytes);
|
||||
}
|
||||
if let Some(md5) = &mut self.md5 {
|
||||
md5.update(bytes);
|
||||
}
|
||||
|
@ -144,6 +160,7 @@ impl Checksummer {
|
|||
crc32c: self
|
||||
.crc32c
|
||||
.map(|x| u32::to_be_bytes(u32::try_from(x.finish()).unwrap())),
|
||||
crc64nvme: self.crc64nvme.map(|x| u64::to_be_bytes(x.sum64())),
|
||||
md5: self.md5.map(|x| x.finalize()[..].try_into().unwrap()),
|
||||
sha1: self.sha1.map(|x| x.finalize()[..].try_into().unwrap()),
|
||||
sha256: self.sha256.map(|x| x.finalize()[..].try_into().unwrap()),
|
||||
|
@ -190,6 +207,9 @@ impl Checksums {
|
|||
None => None,
|
||||
Some(ChecksumAlgorithm::Crc32) => Some(ChecksumValue::Crc32(self.crc32.unwrap())),
|
||||
Some(ChecksumAlgorithm::Crc32c) => Some(ChecksumValue::Crc32c(self.crc32c.unwrap())),
|
||||
Some(ChecksumAlgorithm::Crc64Nvme) => {
|
||||
Some(ChecksumValue::Crc64Nvme(self.crc64nvme.unwrap()))
|
||||
}
|
||||
Some(ChecksumAlgorithm::Sha1) => Some(ChecksumValue::Sha1(self.sha1.unwrap())),
|
||||
Some(ChecksumAlgorithm::Sha256) => Some(ChecksumValue::Sha256(self.sha256.unwrap())),
|
||||
}
|
||||
|
@ -202,6 +222,7 @@ pub fn parse_checksum_algorithm(algo: &str) -> Result<ChecksumAlgorithm, Error>
|
|||
match algo {
|
||||
"CRC32" => Ok(ChecksumAlgorithm::Crc32),
|
||||
"CRC32C" => Ok(ChecksumAlgorithm::Crc32c),
|
||||
"CRC64NVME" => Ok(ChecksumAlgorithm::Crc64Nvme),
|
||||
"SHA1" => Ok(ChecksumAlgorithm::Sha1),
|
||||
"SHA256" => Ok(ChecksumAlgorithm::Sha256),
|
||||
_ => Err(Error::bad_request("invalid checksum algorithm")),
|
||||
|
@ -225,6 +246,7 @@ pub fn request_trailer_checksum_algorithm(
|
|||
None => Ok(None),
|
||||
Some(x) if x == X_AMZ_CHECKSUM_CRC32 => Ok(Some(ChecksumAlgorithm::Crc32)),
|
||||
Some(x) if x == X_AMZ_CHECKSUM_CRC32C => Ok(Some(ChecksumAlgorithm::Crc32c)),
|
||||
Some(x) if x == X_AMZ_CHECKSUM_CRC64NVME => Ok(Some(ChecksumAlgorithm::Crc64Nvme)),
|
||||
Some(x) if x == X_AMZ_CHECKSUM_SHA1 => Ok(Some(ChecksumAlgorithm::Sha1)),
|
||||
Some(x) if x == X_AMZ_CHECKSUM_SHA256 => Ok(Some(ChecksumAlgorithm::Sha256)),
|
||||
_ => Err(Error::bad_request("invalid checksum algorithm")),
|
||||
|
@ -243,6 +265,12 @@ pub fn request_checksum_value(
|
|||
if headers.contains_key(X_AMZ_CHECKSUM_CRC32C) {
|
||||
ret.push(extract_checksum_value(headers, ChecksumAlgorithm::Crc32c)?);
|
||||
}
|
||||
if headers.contains_key(X_AMZ_CHECKSUM_CRC64NVME) {
|
||||
ret.push(extract_checksum_value(
|
||||
headers,
|
||||
ChecksumAlgorithm::Crc64Nvme,
|
||||
)?);
|
||||
}
|
||||
if headers.contains_key(X_AMZ_CHECKSUM_SHA1) {
|
||||
ret.push(extract_checksum_value(headers, ChecksumAlgorithm::Sha1)?);
|
||||
}
|
||||
|
@ -281,6 +309,14 @@ pub fn extract_checksum_value(
|
|||
.ok_or_bad_request("invalid x-amz-checksum-crc32c header")?;
|
||||
Ok(ChecksumValue::Crc32c(crc32c))
|
||||
}
|
||||
ChecksumAlgorithm::Crc64Nvme => {
|
||||
let crc64nvme = headers
|
||||
.get(X_AMZ_CHECKSUM_CRC64NVME)
|
||||
.and_then(|x| BASE64_STANDARD.decode(&x).ok())
|
||||
.and_then(|x| x.try_into().ok())
|
||||
.ok_or_bad_request("invalid x-amz-checksum-crc64nvme header")?;
|
||||
Ok(ChecksumValue::Crc64Nvme(crc64nvme))
|
||||
}
|
||||
ChecksumAlgorithm::Sha1 => {
|
||||
let sha1 = headers
|
||||
.get(X_AMZ_CHECKSUM_SHA1)
|
||||
|
@ -311,6 +347,9 @@ pub fn add_checksum_response_headers(
|
|||
Some(ChecksumValue::Crc32c(crc32c)) => {
|
||||
resp = resp.header(X_AMZ_CHECKSUM_CRC32C, BASE64_STANDARD.encode(&crc32c));
|
||||
}
|
||||
Some(ChecksumValue::Crc64Nvme(crc64nvme)) => {
|
||||
resp = resp.header(X_AMZ_CHECKSUM_CRC64NVME, BASE64_STANDARD.encode(&crc64nvme));
|
||||
}
|
||||
Some(ChecksumValue::Sha1(sha1)) => {
|
||||
resp = resp.header(X_AMZ_CHECKSUM_SHA1, BASE64_STANDARD.encode(&sha1));
|
||||
}
|
||||
|
|
|
@ -64,12 +64,12 @@ pub struct VerifiedRequest {
|
|||
pub content_sha256_header: ContentSha256Header,
|
||||
}
|
||||
|
||||
pub async fn verify_request(
|
||||
pub fn verify_request(
|
||||
garage: &Garage,
|
||||
mut req: Request<IncomingBody>,
|
||||
service: &'static str,
|
||||
) -> Result<VerifiedRequest, Error> {
|
||||
let checked_signature = payload::check_payload_signature(&garage, &mut req, service).await?;
|
||||
let checked_signature = payload::check_payload_signature(&garage, &mut req, service)?;
|
||||
|
||||
let request = streaming::parse_streaming_body(
|
||||
req,
|
||||
|
|
|
@ -32,7 +32,7 @@ pub struct CheckedSignature {
|
|||
pub signature_header: Option<String>,
|
||||
}
|
||||
|
||||
pub async fn check_payload_signature(
|
||||
pub fn check_payload_signature(
|
||||
garage: &Garage,
|
||||
request: &mut Request<IncomingBody>,
|
||||
service: &'static str,
|
||||
|
@ -43,9 +43,9 @@ pub async fn check_payload_signature(
|
|||
// We check for presigned-URL-style authentication first, because
|
||||
// the browser or something else could inject an Authorization header
|
||||
// that is totally unrelated to AWS signatures.
|
||||
check_presigned_signature(garage, service, request, query).await
|
||||
check_presigned_signature(garage, service, request, query)
|
||||
} else if request.headers().contains_key(AUTHORIZATION) {
|
||||
check_standard_signature(garage, service, request, query).await
|
||||
check_standard_signature(garage, service, request, query)
|
||||
} else {
|
||||
// Unsigned (anonymous) request
|
||||
let content_sha256 = request
|
||||
|
@ -93,7 +93,7 @@ fn parse_x_amz_content_sha256(header: Option<&str>) -> Result<ContentSha256Heade
|
|||
}
|
||||
}
|
||||
|
||||
async fn check_standard_signature(
|
||||
fn check_standard_signature(
|
||||
garage: &Garage,
|
||||
service: &'static str,
|
||||
request: &Request<IncomingBody>,
|
||||
|
@ -128,7 +128,7 @@ async fn check_standard_signature(
|
|||
trace!("canonical request:\n{}", canonical_request);
|
||||
trace!("string to sign:\n{}", string_to_sign);
|
||||
|
||||
let key = verify_v4(garage, service, &authorization, string_to_sign.as_bytes()).await?;
|
||||
let key = verify_v4(garage, service, &authorization, string_to_sign.as_bytes())?;
|
||||
|
||||
let content_sha256_header = parse_x_amz_content_sha256(Some(&authorization.content_sha256))?;
|
||||
|
||||
|
@ -139,7 +139,7 @@ async fn check_standard_signature(
|
|||
})
|
||||
}
|
||||
|
||||
async fn check_presigned_signature(
|
||||
fn check_presigned_signature(
|
||||
garage: &Garage,
|
||||
service: &'static str,
|
||||
request: &mut Request<IncomingBody>,
|
||||
|
@ -178,7 +178,7 @@ async fn check_presigned_signature(
|
|||
trace!("canonical request (presigned url):\n{}", canonical_request);
|
||||
trace!("string to sign (presigned url):\n{}", string_to_sign);
|
||||
|
||||
let key = verify_v4(garage, service, &authorization, string_to_sign.as_bytes()).await?;
|
||||
let key = verify_v4(garage, service, &authorization, string_to_sign.as_bytes())?;
|
||||
|
||||
// In the page on presigned URLs, AWS specifies that if a signed query
|
||||
// parameter and a signed header of the same name have different values,
|
||||
|
@ -378,7 +378,7 @@ pub fn parse_date(date: &str) -> Result<DateTime<Utc>, Error> {
|
|||
Ok(Utc.from_utc_datetime(&date))
|
||||
}
|
||||
|
||||
pub async fn verify_v4(
|
||||
pub fn verify_v4(
|
||||
garage: &Garage,
|
||||
service: &str,
|
||||
auth: &Authorization,
|
||||
|
@ -391,8 +391,7 @@ pub async fn verify_v4(
|
|||
|
||||
let key = garage
|
||||
.key_table
|
||||
.get(&EmptyKey, &auth.key_id)
|
||||
.await?
|
||||
.get_local(&EmptyKey, &auth.key_id)?
|
||||
.filter(|k| !k.state.is_deleted())
|
||||
.ok_or_else(|| Error::forbidden(format!("No such key: {}", &auth.key_id)))?;
|
||||
let key_p = key.params().unwrap();
|
||||
|
|
|
@ -1,3 +1,4 @@
|
|||
use std::borrow::Cow;
|
||||
use std::sync::Arc;
|
||||
|
||||
use hyper::{body::Incoming as IncomingBody, Method, Request, Response};
|
||||
|
@ -76,25 +77,19 @@ impl ApiHandler for K2VApiServer {
|
|||
// The OPTIONS method is processed early, before we even check for an API key
|
||||
if let Endpoint::Options = endpoint {
|
||||
let options_res = handle_options_api(garage, &req, Some(bucket_name))
|
||||
.await
|
||||
.ok_or_bad_request("Error handling OPTIONS")?;
|
||||
return Ok(options_res.map(|_empty_body: EmptyBody| empty_body()));
|
||||
}
|
||||
|
||||
let verified_request = verify_request(&garage, req, "k2v").await?;
|
||||
let verified_request = verify_request(&garage, req, "k2v")?;
|
||||
let req = verified_request.request;
|
||||
let api_key = verified_request.access_key;
|
||||
|
||||
let bucket_id = garage
|
||||
.bucket_helper()
|
||||
.resolve_bucket(&bucket_name, &api_key)
|
||||
.await
|
||||
.map_err(pass_helper_error)?;
|
||||
let bucket = garage
|
||||
.bucket_helper()
|
||||
.get_existing_bucket(bucket_id)
|
||||
.await
|
||||
.map_err(helper_error_as_internal)?;
|
||||
.resolve_bucket_fast(&bucket_name, &api_key)
|
||||
.map_err(pass_helper_error)?;
|
||||
let bucket_id = bucket.id;
|
||||
let bucket_params = bucket.state.into_option().unwrap();
|
||||
|
||||
let allowed = match endpoint.authorization_type() {
|
||||
|
@ -179,8 +174,8 @@ impl ApiHandler for K2VApiServer {
|
|||
}
|
||||
|
||||
impl ApiEndpoint for K2VApiEndpoint {
|
||||
fn name(&self) -> &'static str {
|
||||
self.endpoint.name()
|
||||
fn name(&self) -> Cow<'static, str> {
|
||||
Cow::Borrowed(self.endpoint.name())
|
||||
}
|
||||
|
||||
fn add_span_attributes(&self, span: SpanRef<'_>) {
|
||||
|
|
|
@ -2,8 +2,8 @@ use err_derive::Error;
|
|||
use hyper::header::HeaderValue;
|
||||
use hyper::{HeaderMap, StatusCode};
|
||||
|
||||
pub(crate) use garage_api_common::common_error::pass_helper_error;
|
||||
use garage_api_common::common_error::{commonErrorDerivative, CommonError};
|
||||
pub(crate) use garage_api_common::common_error::{helper_error_as_internal, pass_helper_error};
|
||||
pub use garage_api_common::common_error::{
|
||||
CommonErrorDerivative, OkOrBadRequest, OkOrInternalError,
|
||||
};
|
||||
|
|
|
@ -29,8 +29,10 @@ bytes.workspace = true
|
|||
chrono.workspace = true
|
||||
crc32fast.workspace = true
|
||||
crc32c.workspace = true
|
||||
crc64fast-nvme.workspace = true
|
||||
err-derive.workspace = true
|
||||
hex.workspace = true
|
||||
hmac.workspace = true
|
||||
tracing.workspace = true
|
||||
md-5.workspace = true
|
||||
pin-project.workspace = true
|
||||
|
|
|
@ -1,3 +1,4 @@
|
|||
use std::borrow::Cow;
|
||||
use std::sync::Arc;
|
||||
|
||||
use hyper::header;
|
||||
|
@ -117,11 +118,11 @@ impl ApiHandler for S3ApiServer {
|
|||
return handle_post_object(garage, req, bucket_name.unwrap()).await;
|
||||
}
|
||||
if let Endpoint::Options = endpoint {
|
||||
let options_res = handle_options_api(garage, &req, bucket_name).await?;
|
||||
let options_res = handle_options_api(garage, &req, bucket_name)?;
|
||||
return Ok(options_res.map(|_empty_body: EmptyBody| empty_body()));
|
||||
}
|
||||
|
||||
let verified_request = verify_request(&garage, req, "s3").await?;
|
||||
let verified_request = verify_request(&garage, req, "s3")?;
|
||||
let req = verified_request.request;
|
||||
let api_key = verified_request.access_key;
|
||||
|
||||
|
@ -139,15 +140,11 @@ impl ApiHandler for S3ApiServer {
|
|||
return handle_create_bucket(&garage, req, &api_key.key_id, bucket_name).await;
|
||||
}
|
||||
|
||||
let bucket_id = garage
|
||||
.bucket_helper()
|
||||
.resolve_bucket(&bucket_name, &api_key)
|
||||
.await
|
||||
.map_err(pass_helper_error)?;
|
||||
let bucket = garage
|
||||
.bucket_helper()
|
||||
.get_existing_bucket(bucket_id)
|
||||
.await?;
|
||||
.resolve_bucket_fast(&bucket_name, &api_key)
|
||||
.map_err(pass_helper_error)?;
|
||||
let bucket_id = bucket.id;
|
||||
let bucket_params = bucket.state.into_option().unwrap();
|
||||
|
||||
let allowed = match endpoint.authorization_type() {
|
||||
|
@ -345,8 +342,8 @@ impl ApiHandler for S3ApiServer {
|
|||
}
|
||||
|
||||
impl ApiEndpoint for S3ApiEndpoint {
|
||||
fn name(&self) -> &'static str {
|
||||
self.endpoint.name()
|
||||
fn name(&self) -> Cow<'static, str> {
|
||||
Cow::Borrowed(self.endpoint.name())
|
||||
}
|
||||
|
||||
fn add_span_attributes(&self, span: SpanRef<'_>) {
|
||||
|
|
|
@ -143,21 +143,16 @@ pub async fn handle_create_bucket(
|
|||
let api_key = helper.key().get_existing_key(api_key_id).await?;
|
||||
let key_params = api_key.params().unwrap();
|
||||
|
||||
let existing_bucket = if let Some(Some(bucket_id)) = key_params.local_aliases.get(&bucket_name)
|
||||
{
|
||||
Some(*bucket_id)
|
||||
} else {
|
||||
helper
|
||||
.bucket()
|
||||
.resolve_global_bucket_name(&bucket_name)
|
||||
.await?
|
||||
};
|
||||
let existing_bucket = helper
|
||||
.bucket()
|
||||
.resolve_bucket(&bucket_name, &api_key.key_id)
|
||||
.await?;
|
||||
|
||||
if let Some(bucket_id) = existing_bucket {
|
||||
if let Some(bucket) = existing_bucket {
|
||||
// Check we have write or owner permission on the bucket,
|
||||
// in that case it's fine, return 200 OK, bucket exists;
|
||||
// otherwise return a forbidden error.
|
||||
let kp = api_key.bucket_permissions(&bucket_id);
|
||||
let kp = api_key.bucket_permissions(&bucket.id);
|
||||
if !(kp.allow_write || kp.allow_owner) {
|
||||
return Err(CommonError::BucketAlreadyExists.into());
|
||||
}
|
||||
|
|
|
@ -24,7 +24,7 @@ use garage_api_common::helpers::*;
|
|||
use garage_api_common::signature::checksum::*;
|
||||
|
||||
use crate::api_server::{ReqBody, ResBody};
|
||||
use crate::encryption::EncryptionParams;
|
||||
use crate::encryption::{EncryptionParams, OekDerivationInfo};
|
||||
use crate::error::*;
|
||||
use crate::get::{full_object_byte_stream, PreconditionHeaders};
|
||||
use crate::multipart;
|
||||
|
@ -65,8 +65,18 @@ pub async fn handle_copy(
|
|||
&ctx.garage,
|
||||
req.headers(),
|
||||
&source_version_meta.encryption,
|
||||
OekDerivationInfo::for_object(&source_object, source_version),
|
||||
)?;
|
||||
let dest_encryption = EncryptionParams::new_from_headers(&ctx.garage, req.headers())?;
|
||||
let dest_uuid = gen_uuid();
|
||||
let dest_encryption = EncryptionParams::new_from_headers(
|
||||
&ctx.garage,
|
||||
req.headers(),
|
||||
OekDerivationInfo {
|
||||
bucket_id: ctx.bucket_id,
|
||||
version_id: dest_uuid,
|
||||
object_key: dest_key,
|
||||
},
|
||||
)?;
|
||||
|
||||
// Extract source checksum info before source_object_meta_inner is consumed
|
||||
let source_checksum = source_object_meta_inner.checksum;
|
||||
|
@ -115,6 +125,7 @@ pub async fn handle_copy(
|
|||
handle_copy_metaonly(
|
||||
ctx,
|
||||
dest_key,
|
||||
dest_uuid,
|
||||
dest_object_meta,
|
||||
dest_encryption,
|
||||
source_version,
|
||||
|
@ -138,6 +149,7 @@ pub async fn handle_copy(
|
|||
handle_copy_reencrypt(
|
||||
ctx,
|
||||
dest_key,
|
||||
dest_uuid,
|
||||
dest_object_meta,
|
||||
dest_encryption,
|
||||
source_version,
|
||||
|
@ -169,6 +181,7 @@ pub async fn handle_copy(
|
|||
async fn handle_copy_metaonly(
|
||||
ctx: ReqCtx,
|
||||
dest_key: &str,
|
||||
dest_uuid: Uuid,
|
||||
dest_object_meta: ObjectVersionMetaInner,
|
||||
dest_encryption: EncryptionParams,
|
||||
source_version: &ObjectVersion,
|
||||
|
@ -182,7 +195,6 @@ async fn handle_copy_metaonly(
|
|||
} = ctx;
|
||||
|
||||
// Generate parameters for copied object
|
||||
let new_uuid = gen_uuid();
|
||||
let new_timestamp = now_msec();
|
||||
|
||||
let new_meta = ObjectVersionMeta {
|
||||
|
@ -192,7 +204,7 @@ async fn handle_copy_metaonly(
|
|||
};
|
||||
|
||||
let res = SaveStreamResult {
|
||||
version_uuid: new_uuid,
|
||||
version_uuid: dest_uuid,
|
||||
version_timestamp: new_timestamp,
|
||||
etag: new_meta.etag.clone(),
|
||||
};
|
||||
|
@ -204,7 +216,7 @@ async fn handle_copy_metaonly(
|
|||
// bytes is either plaintext before&after or encrypted with the
|
||||
// same keys, so it's ok to just copy it as is
|
||||
let dest_object_version = ObjectVersion {
|
||||
uuid: new_uuid,
|
||||
uuid: dest_uuid,
|
||||
timestamp: new_timestamp,
|
||||
state: ObjectVersionState::Complete(ObjectVersionData::Inline(
|
||||
new_meta,
|
||||
|
@ -230,7 +242,7 @@ async fn handle_copy_metaonly(
|
|||
// This holds a reference to the object in the Version table
|
||||
// so that it won't be deleted, e.g. by repair_versions.
|
||||
let tmp_dest_object_version = ObjectVersion {
|
||||
uuid: new_uuid,
|
||||
uuid: dest_uuid,
|
||||
timestamp: new_timestamp,
|
||||
state: ObjectVersionState::Uploading {
|
||||
encryption: new_meta.encryption.clone(),
|
||||
|
@ -250,7 +262,7 @@ async fn handle_copy_metaonly(
|
|||
// marked as deleted (they are marked as deleted only if the Version
|
||||
// doesn't exist or is marked as deleted).
|
||||
let mut dest_version = Version::new(
|
||||
new_uuid,
|
||||
dest_uuid,
|
||||
VersionBacklink::Object {
|
||||
bucket_id: dest_bucket_id,
|
||||
key: dest_key.to_string(),
|
||||
|
@ -269,7 +281,7 @@ async fn handle_copy_metaonly(
|
|||
.iter()
|
||||
.map(|b| BlockRef {
|
||||
block: b.1.hash,
|
||||
version: new_uuid,
|
||||
version: dest_uuid,
|
||||
deleted: false.into(),
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
|
@ -285,7 +297,7 @@ async fn handle_copy_metaonly(
|
|||
// with the stuff before, the block's reference counts could be decremented before
|
||||
// they are incremented again for the new version, leading to data being deleted.
|
||||
let dest_object_version = ObjectVersion {
|
||||
uuid: new_uuid,
|
||||
uuid: dest_uuid,
|
||||
timestamp: new_timestamp,
|
||||
state: ObjectVersionState::Complete(ObjectVersionData::FirstBlock(
|
||||
new_meta,
|
||||
|
@ -307,6 +319,7 @@ async fn handle_copy_metaonly(
|
|||
async fn handle_copy_reencrypt(
|
||||
ctx: ReqCtx,
|
||||
dest_key: &str,
|
||||
dest_uuid: Uuid,
|
||||
dest_object_meta: ObjectVersionMetaInner,
|
||||
dest_encryption: EncryptionParams,
|
||||
source_version: &ObjectVersion,
|
||||
|
@ -326,6 +339,7 @@ async fn handle_copy_reencrypt(
|
|||
|
||||
save_stream(
|
||||
&ctx,
|
||||
dest_uuid,
|
||||
dest_object_meta,
|
||||
dest_encryption,
|
||||
source_stream.map_err(|e| Error::from(GarageError::from(e))),
|
||||
|
@ -349,7 +363,7 @@ pub async fn handle_upload_part_copy(
|
|||
let dest_upload_id = multipart::decode_upload_id(upload_id)?;
|
||||
|
||||
let dest_key = dest_key.to_string();
|
||||
let (source_object, (_, dest_version, mut dest_mpu)) = futures::try_join!(
|
||||
let (source_object, (dest_object, dest_version, mut dest_mpu)) = futures::try_join!(
|
||||
get_copy_source(&ctx, req),
|
||||
multipart::get_upload(&ctx, &dest_key, &dest_upload_id)
|
||||
)?;
|
||||
|
@ -367,7 +381,10 @@ pub async fn handle_upload_part_copy(
|
|||
&garage,
|
||||
req.headers(),
|
||||
&source_version_meta.encryption,
|
||||
OekDerivationInfo::for_object(&source_object, source_object_version),
|
||||
)?;
|
||||
|
||||
let dest_oek_params = OekDerivationInfo::for_object(&dest_object, &dest_version);
|
||||
let (dest_object_encryption, dest_object_checksum_algorithm) = match dest_version.state {
|
||||
ObjectVersionState::Uploading {
|
||||
encryption,
|
||||
|
@ -376,8 +393,12 @@ pub async fn handle_upload_part_copy(
|
|||
} => (encryption, checksum_algorithm),
|
||||
_ => unreachable!(),
|
||||
};
|
||||
let (dest_encryption, _) =
|
||||
EncryptionParams::check_decrypt(&garage, req.headers(), &dest_object_encryption)?;
|
||||
let (dest_encryption, _) = EncryptionParams::check_decrypt(
|
||||
&garage,
|
||||
req.headers(),
|
||||
&dest_object_encryption,
|
||||
dest_oek_params,
|
||||
)?;
|
||||
let same_encryption = EncryptionParams::is_same(&source_encryption, &dest_encryption);
|
||||
|
||||
// Check source range is valid
|
||||
|
@ -662,16 +683,15 @@ async fn get_copy_source(ctx: &ReqCtx, req: &Request<ReqBody>) -> Result<Object,
|
|||
let copy_source = percent_encoding::percent_decode_str(copy_source).decode_utf8()?;
|
||||
|
||||
let (source_bucket, source_key) = parse_bucket_key(©_source, None)?;
|
||||
let source_bucket_id = garage
|
||||
let source_bucket = garage
|
||||
.bucket_helper()
|
||||
.resolve_bucket(&source_bucket.to_string(), api_key)
|
||||
.await
|
||||
.resolve_bucket_fast(&source_bucket.to_string(), api_key)
|
||||
.map_err(pass_helper_error)?;
|
||||
|
||||
if !api_key.allow_read(&source_bucket_id) {
|
||||
if !api_key.allow_read(&source_bucket.id) {
|
||||
return Err(Error::forbidden(format!(
|
||||
"Reading from bucket {} not allowed for this key",
|
||||
source_bucket
|
||||
"Reading from bucket {:?} not allowed for this key",
|
||||
source_bucket.id
|
||||
)));
|
||||
}
|
||||
|
||||
|
@ -679,7 +699,7 @@ async fn get_copy_source(ctx: &ReqCtx, req: &Request<ReqBody>) -> Result<Object,
|
|||
|
||||
let source_object = garage
|
||||
.object_table
|
||||
.get(&source_bucket_id, &source_key.to_string())
|
||||
.get(&source_bucket.id, &source_key.to_string())
|
||||
.await?
|
||||
.ok_or(Error::NoSuchKey)?;
|
||||
|
||||
|
|
|
@ -11,6 +11,7 @@ use aes_gcm::{
|
|||
};
|
||||
use base64::prelude::*;
|
||||
use bytes::Bytes;
|
||||
use sha2::Sha256;
|
||||
|
||||
use futures::stream::Stream;
|
||||
use futures::task;
|
||||
|
@ -21,12 +22,12 @@ use http::header::{HeaderMap, HeaderName, HeaderValue};
|
|||
use garage_net::bytes_buf::BytesBuf;
|
||||
use garage_net::stream::{stream_asyncread, ByteStream};
|
||||
use garage_rpc::rpc_helper::OrderTag;
|
||||
use garage_util::data::Hash;
|
||||
use garage_util::data::{Hash, Uuid};
|
||||
use garage_util::error::Error as GarageError;
|
||||
use garage_util::migrate::Migrate;
|
||||
|
||||
use garage_model::garage::Garage;
|
||||
use garage_model::s3::object_table::{ObjectVersionEncryption, ObjectVersionMetaInner};
|
||||
use garage_model::s3::object_table::*;
|
||||
|
||||
use garage_api_common::common_error::*;
|
||||
use garage_api_common::signature::checksum::Md5Checksum;
|
||||
|
@ -64,32 +65,45 @@ const STREAM_ENC_CYPER_CHUNK_SIZE: usize = STREAM_ENC_PLAIN_CHUNK_SIZE + 16;
|
|||
pub enum EncryptionParams {
|
||||
Plaintext,
|
||||
SseC {
|
||||
/// the value of x-amz-server-side-encryption-customer-key
|
||||
client_key: Key<Aes256Gcm>,
|
||||
/// the value of x-amz-server-side-encryption-customer-key-md5
|
||||
client_key_md5: Md5Output,
|
||||
/// the object encryption key, for uploads created in garage v2+
|
||||
object_key: Option<Key<Aes256Gcm>>,
|
||||
/// the compression level used for compressing data blocks
|
||||
compression_level: Option<i32>,
|
||||
},
|
||||
}
|
||||
|
||||
#[derive(Clone, Copy)]
|
||||
pub struct OekDerivationInfo<'a> {
|
||||
pub bucket_id: Uuid,
|
||||
pub version_id: Uuid,
|
||||
pub object_key: &'a str,
|
||||
}
|
||||
|
||||
impl EncryptionParams {
|
||||
pub fn is_encrypted(&self) -> bool {
|
||||
!matches!(self, Self::Plaintext)
|
||||
}
|
||||
|
||||
pub fn is_same(a: &Self, b: &Self) -> bool {
|
||||
let relevant_info = |x: &Self| match x {
|
||||
Self::Plaintext => None,
|
||||
Self::SseC {
|
||||
client_key,
|
||||
compression_level,
|
||||
..
|
||||
} => Some((*client_key, compression_level.is_some())),
|
||||
};
|
||||
relevant_info(a) == relevant_info(b)
|
||||
// This function is used in CopyObject and UploadPartCopy to determine
|
||||
// whether the object must be re-encrypted. If this returns true,
|
||||
// data blocks are reused as-is. Since Garage v2, we are using
|
||||
// object-specific encryption keys, so we know that if both source
|
||||
// and destination are encrypted, it can't be with the same key.
|
||||
match (a, b) {
|
||||
(Self::Plaintext, Self::Plaintext) => true,
|
||||
_ => false,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn new_from_headers(
|
||||
garage: &Garage,
|
||||
headers: &HeaderMap,
|
||||
oek_info: OekDerivationInfo<'_>,
|
||||
) -> Result<EncryptionParams, Error> {
|
||||
let key = parse_request_headers(
|
||||
headers,
|
||||
|
@ -101,6 +115,7 @@ impl EncryptionParams {
|
|||
Some((client_key, client_key_md5)) => Ok(EncryptionParams::SseC {
|
||||
client_key,
|
||||
client_key_md5,
|
||||
object_key: Some(oek_info.derive_oek(&client_key)),
|
||||
compression_level: garage.config.compression_level,
|
||||
}),
|
||||
None => Ok(EncryptionParams::Plaintext),
|
||||
|
@ -126,6 +141,7 @@ impl EncryptionParams {
|
|||
garage: &Garage,
|
||||
headers: &HeaderMap,
|
||||
obj_enc: &'a ObjectVersionEncryption,
|
||||
oek_info: OekDerivationInfo<'_>,
|
||||
) -> Result<(Self, Cow<'a, ObjectVersionMetaInner>), Error> {
|
||||
let key = parse_request_headers(
|
||||
headers,
|
||||
|
@ -133,13 +149,14 @@ impl EncryptionParams {
|
|||
&X_AMZ_SERVER_SIDE_ENCRYPTION_CUSTOMER_KEY,
|
||||
&X_AMZ_SERVER_SIDE_ENCRYPTION_CUSTOMER_KEY_MD5,
|
||||
)?;
|
||||
Self::check_decrypt_common(garage, key, obj_enc)
|
||||
Self::check_decrypt_common(garage, key, obj_enc, oek_info)
|
||||
}
|
||||
|
||||
pub fn check_decrypt_for_copy_source<'a>(
|
||||
garage: &Garage,
|
||||
headers: &HeaderMap,
|
||||
obj_enc: &'a ObjectVersionEncryption,
|
||||
oek_info: OekDerivationInfo<'_>,
|
||||
) -> Result<(Self, Cow<'a, ObjectVersionMetaInner>), Error> {
|
||||
let key = parse_request_headers(
|
||||
headers,
|
||||
|
@ -147,22 +164,32 @@ impl EncryptionParams {
|
|||
&X_AMZ_COPY_SOURCE_SERVER_SIDE_ENCRYPTION_CUSTOMER_KEY,
|
||||
&X_AMZ_COPY_SOURCE_SERVER_SIDE_ENCRYPTION_CUSTOMER_KEY_MD5,
|
||||
)?;
|
||||
Self::check_decrypt_common(garage, key, obj_enc)
|
||||
Self::check_decrypt_common(garage, key, obj_enc, oek_info)
|
||||
}
|
||||
|
||||
fn check_decrypt_common<'a>(
|
||||
garage: &Garage,
|
||||
key: Option<(Key<Aes256Gcm>, Md5Output)>,
|
||||
obj_enc: &'a ObjectVersionEncryption,
|
||||
oek_info: OekDerivationInfo<'_>,
|
||||
) -> Result<(Self, Cow<'a, ObjectVersionMetaInner>), Error> {
|
||||
match (key, &obj_enc) {
|
||||
(
|
||||
Some((client_key, client_key_md5)),
|
||||
ObjectVersionEncryption::SseC { inner, compressed },
|
||||
ObjectVersionEncryption::SseC {
|
||||
inner,
|
||||
compressed,
|
||||
use_oek,
|
||||
},
|
||||
) => {
|
||||
let enc = Self::SseC {
|
||||
client_key,
|
||||
client_key_md5,
|
||||
object_key: if *use_oek {
|
||||
Some(oek_info.derive_oek(&client_key))
|
||||
} else {
|
||||
None
|
||||
},
|
||||
compression_level: if *compressed {
|
||||
Some(garage.config.compression_level.unwrap_or(1))
|
||||
} else {
|
||||
|
@ -193,13 +220,16 @@ impl EncryptionParams {
|
|||
) -> Result<ObjectVersionEncryption, Error> {
|
||||
match self {
|
||||
Self::SseC {
|
||||
compression_level, ..
|
||||
compression_level,
|
||||
object_key,
|
||||
..
|
||||
} => {
|
||||
let plaintext = meta.encode().map_err(GarageError::from)?;
|
||||
let ciphertext = self.encrypt_blob(&plaintext)?;
|
||||
Ok(ObjectVersionEncryption::SseC {
|
||||
inner: ciphertext.into_owned(),
|
||||
compressed: compression_level.is_some(),
|
||||
use_oek: object_key.is_some(),
|
||||
})
|
||||
}
|
||||
Self::Plaintext => Ok(ObjectVersionEncryption::Plaintext { inner: meta }),
|
||||
|
@ -228,24 +258,37 @@ impl EncryptionParams {
|
|||
// This is used for encrypting object metadata and inlined data for small objects.
|
||||
// This does not compress anything.
|
||||
|
||||
pub fn encrypt_blob<'a>(&self, blob: &'a [u8]) -> Result<Cow<'a, [u8]>, Error> {
|
||||
fn cipher(&self) -> Option<Aes256Gcm> {
|
||||
match self {
|
||||
Self::SseC { client_key, .. } => {
|
||||
let cipher = Aes256Gcm::new(&client_key);
|
||||
Self::SseC {
|
||||
object_key: Some(oek),
|
||||
..
|
||||
} => Some(Aes256Gcm::new(&oek)),
|
||||
Self::SseC {
|
||||
client_key,
|
||||
object_key: None,
|
||||
..
|
||||
} => Some(Aes256Gcm::new(&client_key)),
|
||||
Self::Plaintext => None,
|
||||
}
|
||||
}
|
||||
|
||||
pub fn encrypt_blob<'a>(&self, blob: &'a [u8]) -> Result<Cow<'a, [u8]>, Error> {
|
||||
match self.cipher() {
|
||||
Some(cipher) => {
|
||||
let nonce = Aes256Gcm::generate_nonce(&mut OsRng);
|
||||
let ciphertext = cipher
|
||||
.encrypt(&nonce, blob)
|
||||
.ok_or_internal_error("Encryption failed")?;
|
||||
Ok(Cow::Owned([nonce.to_vec(), ciphertext].concat()))
|
||||
}
|
||||
Self::Plaintext => Ok(Cow::Borrowed(blob)),
|
||||
None => Ok(Cow::Borrowed(blob)),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn decrypt_blob<'a>(&self, blob: &'a [u8]) -> Result<Cow<'a, [u8]>, Error> {
|
||||
match self {
|
||||
Self::SseC { client_key, .. } => {
|
||||
let cipher = Aes256Gcm::new(&client_key);
|
||||
match self.cipher() {
|
||||
Some(cipher) => {
|
||||
let nonce_size = <Aes256Gcm as AeadCore>::NonceSize::to_usize();
|
||||
let nonce = Nonce::from_slice(
|
||||
blob.get(..nonce_size)
|
||||
|
@ -258,7 +301,7 @@ impl EncryptionParams {
|
|||
)?;
|
||||
Ok(Cow::Owned(plaintext))
|
||||
}
|
||||
Self::Plaintext => Ok(Cow::Borrowed(blob)),
|
||||
None => Ok(Cow::Borrowed(blob)),
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -284,10 +327,12 @@ impl EncryptionParams {
|
|||
Self::Plaintext => stream,
|
||||
Self::SseC {
|
||||
client_key,
|
||||
object_key,
|
||||
compression_level,
|
||||
..
|
||||
} => {
|
||||
let plaintext = DecryptStream::new(stream, *client_key);
|
||||
let key = object_key.as_ref().unwrap_or(client_key);
|
||||
let plaintext = DecryptStream::new(stream, *key);
|
||||
if compression_level.is_some() {
|
||||
let reader = stream_asyncread(Box::pin(plaintext));
|
||||
let reader = BufReader::new(reader);
|
||||
|
@ -307,9 +352,12 @@ impl EncryptionParams {
|
|||
Self::Plaintext => Ok(block),
|
||||
Self::SseC {
|
||||
client_key,
|
||||
object_key,
|
||||
compression_level,
|
||||
..
|
||||
} => {
|
||||
let key = object_key.as_ref().unwrap_or(client_key);
|
||||
|
||||
let block = if let Some(level) = compression_level {
|
||||
Cow::Owned(
|
||||
garage_block::zstd_encode(block.as_ref(), *level)
|
||||
|
@ -325,7 +373,7 @@ impl EncryptionParams {
|
|||
OsRng.fill_bytes(&mut nonce);
|
||||
ret.extend_from_slice(nonce.as_slice());
|
||||
|
||||
let mut cipher = EncryptorLE31::<Aes256Gcm>::new(&client_key, &nonce);
|
||||
let mut cipher = EncryptorLE31::<Aes256Gcm>::new(key, &nonce);
|
||||
let mut iter = block.chunks(STREAM_ENC_PLAIN_CHUNK_SIZE).peekable();
|
||||
|
||||
if iter.peek().is_none() {
|
||||
|
@ -361,6 +409,13 @@ impl EncryptionParams {
|
|||
}
|
||||
}
|
||||
|
||||
pub fn has_encryption_header(headers: &HeaderMap) -> bool {
|
||||
match headers.get(X_AMZ_SERVER_SIDE_ENCRYPTION_CUSTOMER_ALGORITHM) {
|
||||
Some(h) => h.as_bytes() == CUSTOMER_ALGORITHM_AES256,
|
||||
None => false,
|
||||
}
|
||||
}
|
||||
|
||||
fn parse_request_headers(
|
||||
headers: &HeaderMap,
|
||||
alg_header: &HeaderName,
|
||||
|
@ -420,6 +475,30 @@ fn parse_request_headers(
|
|||
}
|
||||
}
|
||||
|
||||
impl<'a> OekDerivationInfo<'a> {
|
||||
pub fn for_object<'b>(object: &'a Object, version: &'b ObjectVersion) -> Self {
|
||||
Self {
|
||||
bucket_id: object.bucket_id,
|
||||
version_id: version.uuid,
|
||||
object_key: &object.key,
|
||||
}
|
||||
}
|
||||
|
||||
fn derive_oek(&self, client_key: &Key<Aes256Gcm>) -> Key<Aes256Gcm> {
|
||||
use hmac::{Hmac, Mac};
|
||||
|
||||
// info = bucket_id + object_name + version_uuid + "garage-object-encryption-key"
|
||||
// oek = hmac_sha256(ssec_key, info)
|
||||
let mut hmac = <Hmac<Sha256> as Mac>::new_from_slice(client_key.as_slice())
|
||||
.expect("create hmac-sha256");
|
||||
hmac.update(b"garage-object-encryption-key");
|
||||
hmac.update(self.bucket_id.as_slice());
|
||||
hmac.update(self.version_id.as_slice());
|
||||
hmac.update(self.object_key.as_bytes());
|
||||
hmac.finalize().into_bytes()
|
||||
}
|
||||
}
|
||||
|
||||
// ---- encrypt & decrypt streams ----
|
||||
|
||||
#[pin_project::pin_project]
|
||||
|
@ -569,6 +648,7 @@ mod tests {
|
|||
let enc = EncryptionParams::SseC {
|
||||
client_key: Aes256Gcm::generate_key(&mut OsRng),
|
||||
client_key_md5: Default::default(), // not needed
|
||||
object_key: Some(Aes256Gcm::generate_key(&mut OsRng)),
|
||||
compression_level,
|
||||
};
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@ use garage_api_common::signature::checksum::{add_checksum_response_headers, X_AM
|
|||
|
||||
use crate::api_server::ResBody;
|
||||
use crate::copy::*;
|
||||
use crate::encryption::EncryptionParams;
|
||||
use crate::encryption::{EncryptionParams, OekDerivationInfo};
|
||||
use crate::error::*;
|
||||
|
||||
const X_AMZ_MP_PARTS_COUNT: HeaderName = HeaderName::from_static("x-amz-mp-parts-count");
|
||||
|
@ -181,8 +181,12 @@ pub async fn handle_head_without_ctx(
|
|||
return Ok(res);
|
||||
}
|
||||
|
||||
let (encryption, headers) =
|
||||
EncryptionParams::check_decrypt(&garage, req.headers(), &version_meta.encryption)?;
|
||||
let (encryption, headers) = EncryptionParams::check_decrypt(
|
||||
&garage,
|
||||
req.headers(),
|
||||
&version_meta.encryption,
|
||||
OekDerivationInfo::for_object(&object, object_version),
|
||||
)?;
|
||||
|
||||
let checksum_mode = checksum_mode(&req);
|
||||
|
||||
|
@ -303,8 +307,12 @@ pub async fn handle_get_without_ctx(
|
|||
return Ok(res);
|
||||
}
|
||||
|
||||
let (enc, headers) =
|
||||
EncryptionParams::check_decrypt(&garage, req.headers(), &last_v_meta.encryption)?;
|
||||
let (enc, headers) = EncryptionParams::check_decrypt(
|
||||
&garage,
|
||||
req.headers(),
|
||||
&last_v_meta.encryption,
|
||||
OekDerivationInfo::for_object(&object, last_v),
|
||||
)?;
|
||||
|
||||
let checksum_mode = checksum_mode(&req);
|
||||
|
||||
|
|
|
@ -17,7 +17,7 @@ use garage_api_common::encoding::*;
|
|||
use garage_api_common::helpers::*;
|
||||
|
||||
use crate::api_server::{ReqBody, ResBody};
|
||||
use crate::encryption::EncryptionParams;
|
||||
use crate::encryption::{EncryptionParams, OekDerivationInfo};
|
||||
use crate::error::*;
|
||||
use crate::multipart as s3_multipart;
|
||||
use crate::xml as s3_xml;
|
||||
|
@ -285,8 +285,16 @@ pub async fn handle_list_parts(
|
|||
ObjectVersionState::Uploading { encryption, .. } => encryption,
|
||||
_ => unreachable!(),
|
||||
};
|
||||
let encryption_res =
|
||||
EncryptionParams::check_decrypt(&ctx.garage, req.headers(), &object_encryption);
|
||||
let encryption_res = EncryptionParams::check_decrypt(
|
||||
&ctx.garage,
|
||||
req.headers(),
|
||||
&object_encryption,
|
||||
OekDerivationInfo {
|
||||
bucket_id: ctx.bucket_id,
|
||||
version_id: upload_id,
|
||||
object_key: &query.key,
|
||||
},
|
||||
);
|
||||
|
||||
let (info, next) = fetch_part_info(query, &mpu)?;
|
||||
|
||||
|
@ -326,6 +334,12 @@ pub async fn handle_list_parts(
|
|||
}
|
||||
_ => None,
|
||||
},
|
||||
checksum_crc64nvme: match &checksum {
|
||||
Some(ChecksumValue::Crc64Nvme(x)) => {
|
||||
Some(s3_xml::Value(BASE64_STANDARD.encode(&x)))
|
||||
}
|
||||
_ => None,
|
||||
},
|
||||
checksum_sha1: match &checksum {
|
||||
Some(ChecksumValue::Sha1(x)) => {
|
||||
Some(s3_xml::Value(BASE64_STANDARD.encode(&x)))
|
||||
|
|
|
@ -6,6 +6,7 @@ use std::sync::Arc;
|
|||
use base64::prelude::*;
|
||||
use crc32c::Crc32cHasher as Crc32c;
|
||||
use crc32fast::Hasher as Crc32;
|
||||
use crc64fast_nvme::Digest as Crc64Nvme;
|
||||
use futures::prelude::*;
|
||||
use hyper::{Request, Response};
|
||||
use md5::{Digest, Md5};
|
||||
|
@ -26,7 +27,7 @@ use garage_api_common::helpers::*;
|
|||
use garage_api_common::signature::checksum::*;
|
||||
|
||||
use crate::api_server::{ReqBody, ResBody};
|
||||
use crate::encryption::EncryptionParams;
|
||||
use crate::encryption::{has_encryption_header, EncryptionParams, OekDerivationInfo};
|
||||
use crate::error::*;
|
||||
use crate::put::*;
|
||||
use crate::xml as s3_xml;
|
||||
|
@ -56,7 +57,15 @@ pub async fn handle_create_multipart_upload(
|
|||
};
|
||||
|
||||
// Determine whether object should be encrypted, and if so the key
|
||||
let encryption = EncryptionParams::new_from_headers(&garage, req.headers())?;
|
||||
let encryption = EncryptionParams::new_from_headers(
|
||||
&garage,
|
||||
req.headers(),
|
||||
OekDerivationInfo {
|
||||
bucket_id: *bucket_id,
|
||||
version_id: upload_id,
|
||||
object_key: &key,
|
||||
},
|
||||
)?;
|
||||
let object_encryption = encryption.encrypt_meta(meta)?;
|
||||
|
||||
let checksum_algorithm = request_checksum_algorithm(req.headers())?;
|
||||
|
@ -120,8 +129,7 @@ pub async fn handle_put_part(
|
|||
|
||||
// Before we stream the body, configure the needed checksums.
|
||||
req_body.add_expected_checksums(expected_checksums.clone());
|
||||
// TODO: avoid parsing encryption headers twice...
|
||||
if !EncryptionParams::new_from_headers(&garage, &req_head.headers)?.is_encrypted() {
|
||||
if !has_encryption_header(&req_head.headers) {
|
||||
// For non-encrypted objects, we need to compute the md5sum in all cases
|
||||
// (even if content-md5 is not set), because it is used as an etag of the
|
||||
// part, which is in turn used in the etag computation of the whole object
|
||||
|
@ -134,10 +142,11 @@ pub async fn handle_put_part(
|
|||
let mut chunker = StreamChunker::new(stream, garage.config.block_size);
|
||||
|
||||
// Read first chuck, and at the same time try to get object to see if it exists
|
||||
let ((_, object_version, mut mpu), first_block) =
|
||||
let ((object, object_version, mut mpu), first_block) =
|
||||
futures::try_join!(get_upload(&ctx, &key, &upload_id), chunker.next(),)?;
|
||||
|
||||
// Check encryption params
|
||||
let oek_params = OekDerivationInfo::for_object(&object, &object_version);
|
||||
let (object_encryption, checksum_algorithm) = match object_version.state {
|
||||
ObjectVersionState::Uploading {
|
||||
encryption,
|
||||
|
@ -146,8 +155,12 @@ pub async fn handle_put_part(
|
|||
} => (encryption, checksum_algorithm),
|
||||
_ => unreachable!(),
|
||||
};
|
||||
let (encryption, _) =
|
||||
EncryptionParams::check_decrypt(&garage, &req_head.headers, &object_encryption)?;
|
||||
let (encryption, _) = EncryptionParams::check_decrypt(
|
||||
&garage,
|
||||
&req_head.headers,
|
||||
&object_encryption,
|
||||
oek_params,
|
||||
)?;
|
||||
|
||||
// Check object is valid and part can be accepted
|
||||
let first_block = first_block.ok_or_bad_request("Empty body")?;
|
||||
|
@ -297,6 +310,7 @@ pub async fn handle_complete_multipart_upload(
|
|||
return Err(Error::bad_request("No data was uploaded"));
|
||||
}
|
||||
|
||||
let oek_params = OekDerivationInfo::for_object(&object, &object_version);
|
||||
let (object_encryption, checksum_algorithm) = match object_version.state {
|
||||
ObjectVersionState::Uploading {
|
||||
encryption,
|
||||
|
@ -417,8 +431,12 @@ pub async fn handle_complete_multipart_upload(
|
|||
let object_encryption = match checksum_algorithm {
|
||||
None => object_encryption,
|
||||
Some(_) => {
|
||||
let (encryption, meta) =
|
||||
EncryptionParams::check_decrypt(&garage, &req_head.headers, &object_encryption)?;
|
||||
let (encryption, meta) = EncryptionParams::check_decrypt(
|
||||
&garage,
|
||||
&req_head.headers,
|
||||
&object_encryption,
|
||||
oek_params,
|
||||
)?;
|
||||
let new_meta = ObjectVersionMetaInner {
|
||||
headers: meta.into_owned().headers,
|
||||
checksum: checksum_extra,
|
||||
|
@ -464,6 +482,10 @@ pub async fn handle_complete_multipart_upload(
|
|||
Some(ChecksumValue::Crc32c(x)) => Some(s3_xml::Value(BASE64_STANDARD.encode(&x))),
|
||||
_ => None,
|
||||
},
|
||||
checksum_crc64nvme: match &checksum_extra {
|
||||
Some(ChecksumValue::Crc64Nvme(x)) => Some(s3_xml::Value(BASE64_STANDARD.encode(&x))),
|
||||
_ => None,
|
||||
},
|
||||
checksum_sha1: match &checksum_extra {
|
||||
Some(ChecksumValue::Sha1(x)) => Some(s3_xml::Value(BASE64_STANDARD.encode(&x))),
|
||||
_ => None,
|
||||
|
@ -587,6 +609,15 @@ fn parse_complete_multipart_upload_body(
|
|||
.try_into()
|
||||
.ok()?,
|
||||
))
|
||||
} else if let Some(crc64nvme) = item
|
||||
.children()
|
||||
.find(|e| e.has_tag_name("ChecksumCRC64NVME"))
|
||||
{
|
||||
Some(ChecksumValue::Crc64Nvme(
|
||||
BASE64_STANDARD.decode(crc64nvme.text()?).ok()?[..]
|
||||
.try_into()
|
||||
.ok()?,
|
||||
))
|
||||
} else if let Some(sha1) = item.children().find(|e| e.has_tag_name("ChecksumSHA1")) {
|
||||
Some(ChecksumValue::Sha1(
|
||||
BASE64_STANDARD.decode(sha1.text()?).ok()?[..]
|
||||
|
@ -627,6 +658,7 @@ pub(crate) struct MultipartChecksummer {
|
|||
pub(crate) enum MultipartExtraChecksummer {
|
||||
Crc32(Crc32),
|
||||
Crc32c(Crc32c),
|
||||
Crc64Nvme(Crc64Nvme),
|
||||
Sha1(Sha1),
|
||||
Sha256(Sha256),
|
||||
}
|
||||
|
@ -643,6 +675,9 @@ impl MultipartChecksummer {
|
|||
Some(ChecksumAlgorithm::Crc32c) => {
|
||||
Some(MultipartExtraChecksummer::Crc32c(Crc32c::default()))
|
||||
}
|
||||
Some(ChecksumAlgorithm::Crc64Nvme) => {
|
||||
Some(MultipartExtraChecksummer::Crc64Nvme(Crc64Nvme::default()))
|
||||
}
|
||||
Some(ChecksumAlgorithm::Sha1) => Some(MultipartExtraChecksummer::Sha1(Sha1::new())),
|
||||
Some(ChecksumAlgorithm::Sha256) => {
|
||||
Some(MultipartExtraChecksummer::Sha256(Sha256::new()))
|
||||
|
@ -672,6 +707,12 @@ impl MultipartChecksummer {
|
|||
) => {
|
||||
crc32c.write(&x);
|
||||
}
|
||||
(
|
||||
Some(MultipartExtraChecksummer::Crc64Nvme(ref mut crc64nvme)),
|
||||
Some(ChecksumValue::Crc64Nvme(x)),
|
||||
) => {
|
||||
crc64nvme.write(&x);
|
||||
}
|
||||
(Some(MultipartExtraChecksummer::Sha1(ref mut sha1)), Some(ChecksumValue::Sha1(x))) => {
|
||||
sha1.update(&x);
|
||||
}
|
||||
|
@ -701,6 +742,9 @@ impl MultipartChecksummer {
|
|||
Some(MultipartExtraChecksummer::Crc32c(crc32c)) => Some(ChecksumValue::Crc32c(
|
||||
u32::to_be_bytes(u32::try_from(crc32c.finish()).unwrap()),
|
||||
)),
|
||||
Some(MultipartExtraChecksummer::Crc64Nvme(crc64nvme)) => Some(
|
||||
ChecksumValue::Crc64Nvme(u64::to_be_bytes(crc64nvme.sum64())),
|
||||
),
|
||||
Some(MultipartExtraChecksummer::Sha1(sha1)) => {
|
||||
Some(ChecksumValue::Sha1(sha1.finalize()[..].try_into().unwrap()))
|
||||
}
|
||||
|
|
|
@ -15,6 +15,7 @@ use serde::Deserialize;
|
|||
|
||||
use garage_model::garage::Garage;
|
||||
use garage_model::s3::object_table::*;
|
||||
use garage_util::data::gen_uuid;
|
||||
|
||||
use garage_api_common::cors::*;
|
||||
use garage_api_common::helpers::*;
|
||||
|
@ -22,7 +23,7 @@ use garage_api_common::signature::checksum::*;
|
|||
use garage_api_common::signature::payload::{verify_v4, Authorization};
|
||||
|
||||
use crate::api_server::ResBody;
|
||||
use crate::encryption::EncryptionParams;
|
||||
use crate::encryption::{EncryptionParams, OekDerivationInfo};
|
||||
use crate::error::*;
|
||||
use crate::put::{extract_metadata_headers, save_stream, ChecksumMode};
|
||||
use crate::xml as s3_xml;
|
||||
|
@ -103,22 +104,18 @@ pub async fn handle_post_object(
|
|||
key.to_owned()
|
||||
};
|
||||
|
||||
let api_key = verify_v4(&garage, "s3", &authorization, policy.as_bytes()).await?;
|
||||
let api_key = verify_v4(&garage, "s3", &authorization, policy.as_bytes())?;
|
||||
|
||||
let bucket_id = garage
|
||||
let bucket = garage
|
||||
.bucket_helper()
|
||||
.resolve_bucket(&bucket_name, &api_key)
|
||||
.await
|
||||
.resolve_bucket_fast(&bucket_name, &api_key)
|
||||
.map_err(pass_helper_error)?;
|
||||
let bucket_id = bucket.id;
|
||||
|
||||
if !api_key.allow_write(&bucket_id) {
|
||||
return Err(Error::forbidden("Operation is not allowed for this key."));
|
||||
}
|
||||
|
||||
let bucket = garage
|
||||
.bucket_helper()
|
||||
.get_existing_bucket(bucket_id)
|
||||
.await?;
|
||||
let bucket_params = bucket.state.into_option().unwrap();
|
||||
let matching_cors_rule = find_matching_cors_rule(
|
||||
&bucket_params,
|
||||
|
@ -231,12 +228,22 @@ pub async fn handle_post_object(
|
|||
.transpose()?,
|
||||
};
|
||||
|
||||
let version_uuid = gen_uuid();
|
||||
|
||||
let meta = ObjectVersionMetaInner {
|
||||
headers,
|
||||
checksum: expected_checksums.extra,
|
||||
};
|
||||
|
||||
let encryption = EncryptionParams::new_from_headers(&garage, ¶ms)?;
|
||||
let encryption = EncryptionParams::new_from_headers(
|
||||
&garage,
|
||||
¶ms,
|
||||
OekDerivationInfo {
|
||||
bucket_id,
|
||||
version_id: version_uuid,
|
||||
object_key: &key,
|
||||
},
|
||||
)?;
|
||||
|
||||
let stream = file_field.map(|r| r.map_err(Into::into));
|
||||
let ctx = ReqCtx {
|
||||
|
@ -249,6 +256,7 @@ pub async fn handle_post_object(
|
|||
|
||||
let res = save_stream(
|
||||
&ctx,
|
||||
version_uuid,
|
||||
meta,
|
||||
encryption,
|
||||
StreamLimiter::new(stream, conditions.content_length),
|
||||
|
|
|
@ -35,7 +35,7 @@ use garage_api_common::signature::body::StreamingChecksumReceiver;
|
|||
use garage_api_common::signature::checksum::*;
|
||||
|
||||
use crate::api_server::{ReqBody, ResBody};
|
||||
use crate::encryption::EncryptionParams;
|
||||
use crate::encryption::{EncryptionParams, OekDerivationInfo};
|
||||
use crate::error::*;
|
||||
use crate::website::X_AMZ_WEBSITE_REDIRECT_LOCATION;
|
||||
|
||||
|
@ -62,6 +62,10 @@ pub async fn handle_put(
|
|||
req: Request<ReqBody>,
|
||||
key: &String,
|
||||
) -> Result<Response<ResBody>, Error> {
|
||||
// Generate version uuid now, because it is necessary to compute SSE-C
|
||||
// encryption parameters
|
||||
let version_uuid = gen_uuid();
|
||||
|
||||
// Retrieve interesting headers from request
|
||||
let headers = extract_metadata_headers(req.headers())?;
|
||||
debug!("Object headers: {:?}", headers);
|
||||
|
@ -82,7 +86,15 @@ pub async fn handle_put(
|
|||
};
|
||||
|
||||
// Determine whether object should be encrypted, and if so the key
|
||||
let encryption = EncryptionParams::new_from_headers(&ctx.garage, req.headers())?;
|
||||
let encryption = EncryptionParams::new_from_headers(
|
||||
&ctx.garage,
|
||||
req.headers(),
|
||||
OekDerivationInfo {
|
||||
bucket_id: ctx.bucket_id,
|
||||
version_id: version_uuid,
|
||||
object_key: &key,
|
||||
},
|
||||
)?;
|
||||
|
||||
// The request body is a special ReqBody object (see garage_api_common::signature::body)
|
||||
// which supports calculating checksums while streaming the data.
|
||||
|
@ -100,6 +112,7 @@ pub async fn handle_put(
|
|||
|
||||
let res = save_stream(
|
||||
&ctx,
|
||||
version_uuid,
|
||||
meta,
|
||||
encryption,
|
||||
stream,
|
||||
|
@ -121,6 +134,7 @@ pub async fn handle_put(
|
|||
|
||||
pub(crate) async fn save_stream<S: Stream<Item = Result<Bytes, Error>> + Unpin>(
|
||||
ctx: &ReqCtx,
|
||||
version_uuid: Uuid,
|
||||
mut meta: ObjectVersionMetaInner,
|
||||
encryption: EncryptionParams,
|
||||
body: S,
|
||||
|
@ -140,7 +154,6 @@ pub(crate) async fn save_stream<S: Stream<Item = Result<Bytes, Error>> + Unpin>(
|
|||
let first_block = first_block_opt.unwrap_or_default();
|
||||
|
||||
// Generate identity of new version
|
||||
let version_uuid = gen_uuid();
|
||||
let version_timestamp = next_timestamp(existing_object.as_ref());
|
||||
|
||||
let mut checksummer = match &checksum_mode {
|
||||
|
|
|
@ -3,7 +3,7 @@ use quick_xml::de::from_reader;
|
|||
use hyper::{header::HeaderName, Request, Response, StatusCode};
|
||||
use serde::{Deserialize, Serialize};
|
||||
|
||||
use garage_model::bucket_table::*;
|
||||
use garage_model::bucket_table::{self, *};
|
||||
|
||||
use garage_api_common::helpers::*;
|
||||
|
||||
|
@ -26,7 +26,28 @@ pub async fn handle_get_website(ctx: ReqCtx) -> Result<Response<ResBody>, Error>
|
|||
suffix: Value(website.index_document.to_string()),
|
||||
}),
|
||||
redirect_all_requests_to: None,
|
||||
routing_rules: None,
|
||||
routing_rules: RoutingRules {
|
||||
rules: website
|
||||
.routing_rules
|
||||
.clone()
|
||||
.into_iter()
|
||||
.map(|rule| RoutingRule {
|
||||
condition: rule.condition.map(|cond| Condition {
|
||||
http_error_code: cond.http_error_code.map(|c| IntValue(c as i64)),
|
||||
prefix: cond.prefix.map(Value),
|
||||
}),
|
||||
redirect: Redirect {
|
||||
hostname: rule.redirect.hostname.map(Value),
|
||||
http_redirect_code: Some(IntValue(
|
||||
rule.redirect.http_redirect_code as i64,
|
||||
)),
|
||||
protocol: rule.redirect.protocol.map(Value),
|
||||
replace_full: rule.redirect.replace_key.map(Value),
|
||||
replace_prefix: rule.redirect.replace_key_prefix.map(Value),
|
||||
},
|
||||
})
|
||||
.collect(),
|
||||
},
|
||||
};
|
||||
let xml = to_xml_with_header(&wc)?;
|
||||
Ok(Response::builder()
|
||||
|
@ -97,18 +118,28 @@ pub struct WebsiteConfiguration {
|
|||
pub index_document: Option<Suffix>,
|
||||
#[serde(rename = "RedirectAllRequestsTo")]
|
||||
pub redirect_all_requests_to: Option<Target>,
|
||||
#[serde(rename = "RoutingRules")]
|
||||
pub routing_rules: Option<Vec<RoutingRule>>,
|
||||
#[serde(
|
||||
rename = "RoutingRules",
|
||||
default,
|
||||
skip_serializing_if = "RoutingRules::is_empty"
|
||||
)]
|
||||
pub routing_rules: RoutingRules,
|
||||
}
|
||||
|
||||
#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, PartialOrd, Ord, Default)]
|
||||
pub struct RoutingRules {
|
||||
#[serde(rename = "RoutingRule")]
|
||||
pub rules: Vec<RoutingRule>,
|
||||
}
|
||||
|
||||
impl RoutingRules {
|
||||
fn is_empty(&self) -> bool {
|
||||
self.rules.is_empty()
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, PartialOrd, Ord)]
|
||||
pub struct RoutingRule {
|
||||
#[serde(rename = "RoutingRule")]
|
||||
pub inner: RoutingRuleInner,
|
||||
}
|
||||
|
||||
#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, PartialOrd, Ord)]
|
||||
pub struct RoutingRuleInner {
|
||||
#[serde(rename = "Condition")]
|
||||
pub condition: Option<Condition>,
|
||||
#[serde(rename = "Redirect")]
|
||||
|
@ -162,7 +193,7 @@ impl WebsiteConfiguration {
|
|||
if self.redirect_all_requests_to.is_some()
|
||||
&& (self.error_document.is_some()
|
||||
|| self.index_document.is_some()
|
||||
|| self.routing_rules.is_some())
|
||||
|| !self.routing_rules.is_empty())
|
||||
{
|
||||
return Err(Error::bad_request(
|
||||
"Bad XML: can't have RedirectAllRequestsTo and other fields",
|
||||
|
@ -177,10 +208,15 @@ impl WebsiteConfiguration {
|
|||
if let Some(ref rart) = self.redirect_all_requests_to {
|
||||
rart.validate()?;
|
||||
}
|
||||
if let Some(ref rrs) = self.routing_rules {
|
||||
for rr in rrs {
|
||||
rr.inner.validate()?;
|
||||
}
|
||||
for rr in &self.routing_rules.rules {
|
||||
rr.validate()?;
|
||||
}
|
||||
if self.routing_rules.rules.len() > 1000 {
|
||||
// we will do linear scans, best to avoid overly long configuration. The
|
||||
// limit was choosen arbitrarily
|
||||
return Err(Error::bad_request(
|
||||
"Bad XML: RoutingRules can't have more than 1000 child elements",
|
||||
));
|
||||
}
|
||||
|
||||
Ok(())
|
||||
|
@ -189,11 +225,7 @@ impl WebsiteConfiguration {
|
|||
pub fn into_garage_website_config(self) -> Result<WebsiteConfig, Error> {
|
||||
if self.redirect_all_requests_to.is_some() {
|
||||
Err(Error::NotImplemented(
|
||||
"S3 website redirects are not currently implemented in Garage.".into(),
|
||||
))
|
||||
} else if self.routing_rules.map(|x| !x.is_empty()).unwrap_or(false) {
|
||||
Err(Error::NotImplemented(
|
||||
"S3 routing rules are not currently implemented in Garage.".into(),
|
||||
"RedirectAllRequestsTo is not currently implemented in Garage, however its effect can be emulated using a single inconditional RoutingRule.".into(),
|
||||
))
|
||||
} else {
|
||||
Ok(WebsiteConfig {
|
||||
|
@ -202,6 +234,36 @@ impl WebsiteConfiguration {
|
|||
.map(|x| x.suffix.0)
|
||||
.unwrap_or_else(|| "index.html".to_string()),
|
||||
error_document: self.error_document.map(|x| x.key.0),
|
||||
redirect_all: None,
|
||||
routing_rules: self
|
||||
.routing_rules
|
||||
.rules
|
||||
.into_iter()
|
||||
.map(|rule| {
|
||||
bucket_table::RoutingRule {
|
||||
condition: rule.condition.map(|condition| {
|
||||
bucket_table::RedirectCondition {
|
||||
http_error_code: condition.http_error_code.map(|c| c.0 as u16),
|
||||
prefix: condition.prefix.map(|p| p.0),
|
||||
}
|
||||
}),
|
||||
redirect: bucket_table::Redirect {
|
||||
hostname: rule.redirect.hostname.map(|h| h.0),
|
||||
protocol: rule.redirect.protocol.map(|p| p.0),
|
||||
// aws default to 301, which i find punitive in case of
|
||||
// missconfiguration (can be permanently cached on the
|
||||
// user agent)
|
||||
http_redirect_code: rule
|
||||
.redirect
|
||||
.http_redirect_code
|
||||
.map(|c| c.0 as u16)
|
||||
.unwrap_or(302),
|
||||
replace_key_prefix: rule.redirect.replace_prefix.map(|k| k.0),
|
||||
replace_key: rule.redirect.replace_full.map(|k| k.0),
|
||||
},
|
||||
}
|
||||
})
|
||||
.collect(),
|
||||
})
|
||||
}
|
||||
}
|
||||
|
@ -242,37 +304,69 @@ impl Target {
|
|||
}
|
||||
}
|
||||
|
||||
impl RoutingRuleInner {
|
||||
impl RoutingRule {
|
||||
pub fn validate(&self) -> Result<(), Error> {
|
||||
let has_prefix = self
|
||||
.condition
|
||||
.as_ref()
|
||||
.and_then(|c| c.prefix.as_ref())
|
||||
.is_some();
|
||||
self.redirect.validate(has_prefix)
|
||||
if let Some(condition) = &self.condition {
|
||||
condition.validate()?;
|
||||
}
|
||||
self.redirect.validate()
|
||||
}
|
||||
}
|
||||
|
||||
impl Condition {
|
||||
pub fn validate(&self) -> Result<bool, Error> {
|
||||
if let Some(ref error_code) = self.http_error_code {
|
||||
// TODO do other error codes make sense? Aws only allows 4xx and 5xx
|
||||
if error_code.0 != 404 {
|
||||
return Err(Error::bad_request(
|
||||
"Bad XML: HttpErrorCodeReturnedEquals must be 404 or absent",
|
||||
));
|
||||
}
|
||||
}
|
||||
Ok(self.prefix.is_some())
|
||||
}
|
||||
}
|
||||
|
||||
impl Redirect {
|
||||
pub fn validate(&self, has_prefix: bool) -> Result<(), Error> {
|
||||
if self.replace_prefix.is_some() {
|
||||
if self.replace_full.is_some() {
|
||||
return Err(Error::bad_request(
|
||||
"Bad XML: both ReplaceKeyPrefixWith and ReplaceKeyWith are set",
|
||||
));
|
||||
}
|
||||
if !has_prefix {
|
||||
return Err(Error::bad_request(
|
||||
"Bad XML: ReplaceKeyPrefixWith is set, but KeyPrefixEquals isn't",
|
||||
));
|
||||
}
|
||||
pub fn validate(&self) -> Result<(), Error> {
|
||||
if self.replace_prefix.is_some() && self.replace_full.is_some() {
|
||||
return Err(Error::bad_request(
|
||||
"Bad XML: both ReplaceKeyPrefixWith and ReplaceKeyWith are set",
|
||||
));
|
||||
}
|
||||
if let Some(ref protocol) = self.protocol {
|
||||
if protocol.0 != "http" && protocol.0 != "https" {
|
||||
return Err(Error::bad_request("Bad XML: invalid protocol"));
|
||||
}
|
||||
}
|
||||
// TODO there are probably more invalid cases, but which ones?
|
||||
if let Some(ref http_redirect_code) = self.http_redirect_code {
|
||||
match http_redirect_code.0 {
|
||||
// aws allows all 3xx except 300, but some are non-sensical (not modified,
|
||||
// use proxy...)
|
||||
301 | 302 | 303 | 307 | 308 => {
|
||||
if self.hostname.is_none() && self.protocol.is_some() {
|
||||
return Err(Error::bad_request(
|
||||
"Bad XML: HostName must be set if Protocol is set",
|
||||
));
|
||||
}
|
||||
}
|
||||
// aws doesn't allow these codes, but netlify does, and it seems like a
|
||||
// cool feature (change the page seen without changing the url shown by the
|
||||
// user agent)
|
||||
200 | 404 => {
|
||||
if self.hostname.is_some() || self.protocol.is_some() {
|
||||
// hostname would mean different bucket, protocol doesn't make
|
||||
// sense
|
||||
return Err(Error::bad_request(
|
||||
"Bad XML: an HttpRedirectCode of 200 is not acceptable alongside HostName or Protocol",
|
||||
));
|
||||
}
|
||||
}
|
||||
_ => {
|
||||
return Err(Error::bad_request("Bad XML: invalid HttpRedirectCode"));
|
||||
}
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
@ -311,6 +405,15 @@ mod tests {
|
|||
<ReplaceKeyWith>fullkey</ReplaceKeyWith>
|
||||
</Redirect>
|
||||
</RoutingRule>
|
||||
<RoutingRule>
|
||||
<Condition>
|
||||
<KeyPrefixEquals></KeyPrefixEquals>
|
||||
</Condition>
|
||||
<Redirect>
|
||||
<HttpRedirectCode>404</HttpRedirectCode>
|
||||
<ReplaceKeyWith>missing</ReplaceKeyWith>
|
||||
</Redirect>
|
||||
</RoutingRule>
|
||||
</RoutingRules>
|
||||
</WebsiteConfiguration>"#;
|
||||
let conf: WebsiteConfiguration = from_str(message).unwrap();
|
||||
|
@ -326,21 +429,36 @@ mod tests {
|
|||
hostname: Value("garage.tld".to_owned()),
|
||||
protocol: Some(Value("https".to_owned())),
|
||||
}),
|
||||
routing_rules: Some(vec![RoutingRule {
|
||||
inner: RoutingRuleInner {
|
||||
condition: Some(Condition {
|
||||
http_error_code: Some(IntValue(404)),
|
||||
prefix: Some(Value("prefix1".to_owned())),
|
||||
}),
|
||||
redirect: Redirect {
|
||||
hostname: Some(Value("gara.ge".to_owned())),
|
||||
protocol: Some(Value("http".to_owned())),
|
||||
http_redirect_code: Some(IntValue(303)),
|
||||
replace_prefix: Some(Value("prefix2".to_owned())),
|
||||
replace_full: Some(Value("fullkey".to_owned())),
|
||||
routing_rules: RoutingRules {
|
||||
rules: vec![
|
||||
RoutingRule {
|
||||
condition: Some(Condition {
|
||||
http_error_code: Some(IntValue(404)),
|
||||
prefix: Some(Value("prefix1".to_owned())),
|
||||
}),
|
||||
redirect: Redirect {
|
||||
hostname: Some(Value("gara.ge".to_owned())),
|
||||
protocol: Some(Value("http".to_owned())),
|
||||
http_redirect_code: Some(IntValue(303)),
|
||||
replace_prefix: Some(Value("prefix2".to_owned())),
|
||||
replace_full: Some(Value("fullkey".to_owned())),
|
||||
},
|
||||
},
|
||||
},
|
||||
}]),
|
||||
RoutingRule {
|
||||
condition: Some(Condition {
|
||||
http_error_code: None,
|
||||
prefix: Some(Value("".to_owned())),
|
||||
}),
|
||||
redirect: Redirect {
|
||||
hostname: None,
|
||||
protocol: None,
|
||||
http_redirect_code: Some(IntValue(404)),
|
||||
replace_prefix: None,
|
||||
replace_full: Some(Value("missing".to_owned())),
|
||||
},
|
||||
},
|
||||
],
|
||||
},
|
||||
};
|
||||
assert_eq! {
|
||||
ref_value,
|
||||
|
|
|
@ -135,6 +135,8 @@ pub struct CompleteMultipartUploadResult {
|
|||
pub checksum_crc32: Option<Value>,
|
||||
#[serde(rename = "ChecksumCRC32C")]
|
||||
pub checksum_crc32c: Option<Value>,
|
||||
#[serde(rename = "ChecksumCR64NVME")]
|
||||
pub checksum_crc64nvme: Option<Value>,
|
||||
#[serde(rename = "ChecksumSHA1")]
|
||||
pub checksum_sha1: Option<Value>,
|
||||
#[serde(rename = "ChecksumSHA256")]
|
||||
|
@ -209,6 +211,8 @@ pub struct PartItem {
|
|||
pub checksum_crc32: Option<Value>,
|
||||
#[serde(rename = "ChecksumCRC32C")]
|
||||
pub checksum_crc32c: Option<Value>,
|
||||
#[serde(rename = "ChecksumCRC64NVME")]
|
||||
pub checksum_crc64nvme: Option<Value>,
|
||||
#[serde(rename = "ChecksumSHA1")]
|
||||
pub checksum_sha1: Option<Value>,
|
||||
#[serde(rename = "ChecksumSHA256")]
|
||||
|
@ -518,6 +522,7 @@ mod tests {
|
|||
etag: Value("\"3858f62230ac3c915f300c664312c11f-9\"".to_string()),
|
||||
checksum_crc32: None,
|
||||
checksum_crc32c: None,
|
||||
checksum_crc64nvme: None,
|
||||
checksum_sha1: Some(Value("ZJAnHyG8PeKz9tI8UTcHrJos39A=".into())),
|
||||
checksum_sha256: None,
|
||||
};
|
||||
|
@ -803,6 +808,7 @@ mod tests {
|
|||
size: IntValue(10485760),
|
||||
checksum_crc32: None,
|
||||
checksum_crc32c: None,
|
||||
checksum_crc64nvme: None,
|
||||
checksum_sha256: Some(Value(
|
||||
"5RQ3A5uk0w7ojNjvegohch4JRBBGN/cLhsNrPzfv/hA=".into(),
|
||||
)),
|
||||
|
@ -816,6 +822,7 @@ mod tests {
|
|||
checksum_sha256: None,
|
||||
checksum_crc32c: None,
|
||||
checksum_crc32: Some(Value("ZJAnHyG8=".into())),
|
||||
checksum_crc64nvme: None,
|
||||
checksum_sha1: None,
|
||||
},
|
||||
],
|
||||
|
|
|
@ -18,7 +18,6 @@ garage_db.workspace = true
|
|||
garage_net.workspace = true
|
||||
garage_rpc.workspace = true
|
||||
garage_util.workspace = true
|
||||
garage_table.workspace = true
|
||||
|
||||
opentelemetry.workspace = true
|
||||
|
||||
|
|
|
@ -33,8 +33,6 @@ use garage_rpc::rpc_helper::OrderTag;
|
|||
use garage_rpc::system::System;
|
||||
use garage_rpc::*;
|
||||
|
||||
use garage_table::replication::{TableReplication, TableShardedReplication};
|
||||
|
||||
use crate::block::*;
|
||||
use crate::layout::*;
|
||||
use crate::metrics::*;
|
||||
|
@ -74,8 +72,8 @@ impl Rpc for BlockRpc {
|
|||
|
||||
/// The block manager, handling block exchange between nodes, and block storage on local node
|
||||
pub struct BlockManager {
|
||||
/// Replication strategy, allowing to find on which node blocks should be located
|
||||
pub replication: TableShardedReplication,
|
||||
/// Quorum of nodes for write operations
|
||||
pub write_quorum: usize,
|
||||
|
||||
/// Data layout
|
||||
pub(crate) data_layout: ArcSwap<DataLayout>,
|
||||
|
@ -122,7 +120,7 @@ impl BlockManager {
|
|||
pub fn new(
|
||||
db: &db::Db,
|
||||
config: &Config,
|
||||
replication: TableShardedReplication,
|
||||
write_quorum: usize,
|
||||
system: Arc<System>,
|
||||
) -> Result<Arc<Self>, Error> {
|
||||
// Load or compute layout, i.e. assignment of data blocks to the different data directories
|
||||
|
@ -166,7 +164,7 @@ impl BlockManager {
|
|||
let scrub_persister = PersisterShared::new(&system.metadata_dir, "scrub_info");
|
||||
|
||||
let block_manager = Arc::new(Self {
|
||||
replication,
|
||||
write_quorum,
|
||||
data_layout: ArcSwap::new(Arc::new(data_layout)),
|
||||
data_layout_persister,
|
||||
data_fsync: config.data_fsync,
|
||||
|
@ -338,6 +336,19 @@ impl BlockManager {
|
|||
Err(err)
|
||||
}
|
||||
|
||||
/// Returns the set of nodes that should store a copy of a given block.
|
||||
/// These are the nodes assigned to the block's hash in the current
|
||||
/// layout version only: since blocks are immutable, we don't need to
|
||||
/// do complex logic when several layour versions are active at once,
|
||||
/// just move them directly to the new nodes.
|
||||
pub(crate) fn storage_nodes_of(&self, hash: &Hash) -> Vec<Uuid> {
|
||||
self.system
|
||||
.cluster_layout()
|
||||
.current()
|
||||
.nodes_of(hash)
|
||||
.collect()
|
||||
}
|
||||
|
||||
// ---- Public interface ----
|
||||
|
||||
/// Ask nodes that might have a block for it, return it as a stream
|
||||
|
@ -370,7 +381,7 @@ impl BlockManager {
|
|||
prevent_compression: bool,
|
||||
order_tag: Option<OrderTag>,
|
||||
) -> Result<(), Error> {
|
||||
let who = self.system.cluster_layout().current_storage_nodes_of(&hash);
|
||||
let who = self.storage_nodes_of(&hash);
|
||||
|
||||
let compression_level = self.compression_level.filter(|_| !prevent_compression);
|
||||
let (header, bytes) = DataBlock::from_buffer(data, compression_level)
|
||||
|
@ -400,7 +411,7 @@ impl BlockManager {
|
|||
put_block_rpc,
|
||||
RequestStrategy::with_priority(PRIO_NORMAL | PRIO_SECONDARY)
|
||||
.with_drop_on_completion(permit)
|
||||
.with_quorum(self.replication.write_quorum()),
|
||||
.with_quorum(self.write_quorum),
|
||||
)
|
||||
.await?;
|
||||
|
||||
|
|
|
@ -27,8 +27,6 @@ use garage_util::tranquilizer::Tranquilizer;
|
|||
use garage_rpc::system::System;
|
||||
use garage_rpc::*;
|
||||
|
||||
use garage_table::replication::TableReplication;
|
||||
|
||||
use crate::manager::*;
|
||||
|
||||
// The delay between the time where a resync operation fails
|
||||
|
@ -377,11 +375,8 @@ impl BlockResyncManager {
|
|||
info!("Resync block {:?}: offloading and deleting", hash);
|
||||
let existing_path = existing_path.unwrap();
|
||||
|
||||
let mut who = manager
|
||||
.system
|
||||
.cluster_layout()
|
||||
.current_storage_nodes_of(hash);
|
||||
if who.len() < manager.replication.write_quorum() {
|
||||
let mut who = manager.storage_nodes_of(hash);
|
||||
if who.len() < manager.write_quorum {
|
||||
return Err(Error::Message("Not trying to offload block because we don't have a quorum of nodes to write to".to_string()));
|
||||
}
|
||||
who.retain(|id| *id != manager.system.id);
|
||||
|
@ -463,10 +458,7 @@ impl BlockResyncManager {
|
|||
|
||||
// First, check whether we are still supposed to store that
|
||||
// block in the latest cluster layout version.
|
||||
let storage_nodes = manager
|
||||
.system
|
||||
.cluster_layout()
|
||||
.current_storage_nodes_of(&hash);
|
||||
let storage_nodes = manager.storage_nodes_of(&hash);
|
||||
|
||||
if !storage_nodes.contains(&manager.system.id) {
|
||||
info!(
|
||||
|
|
|
@ -26,6 +26,7 @@ garage_db.workspace = true
|
|||
garage_api_admin.workspace = true
|
||||
garage_api_s3.workspace = true
|
||||
garage_api_k2v = { workspace = true, optional = true }
|
||||
garage_api_common.workspace = true
|
||||
garage_block.workspace = true
|
||||
garage_model.workspace = true
|
||||
garage_net.workspace = true
|
||||
|
@ -37,6 +38,7 @@ garage_web.workspace = true
|
|||
backtrace.workspace = true
|
||||
bytes.workspace = true
|
||||
bytesize.workspace = true
|
||||
chrono.workspace = true
|
||||
timeago.workspace = true
|
||||
parse_duration.workspace = true
|
||||
hex.workspace = true
|
||||
|
@ -47,8 +49,8 @@ sha1.workspace = true
|
|||
sodiumoxide.workspace = true
|
||||
structopt.workspace = true
|
||||
git-version.workspace = true
|
||||
|
||||
serde.workspace = true
|
||||
utoipa.workspace = true
|
||||
serde_json.workspace = true
|
||||
|
||||
futures.workspace = true
|
||||
tokio.workspace = true
|
||||
|
@ -85,7 +87,7 @@ k2v-client.workspace = true
|
|||
[features]
|
||||
default = [ "bundled-libs", "metrics", "lmdb", "sqlite", "k2v" ]
|
||||
|
||||
k2v = [ "garage_util/k2v", "garage_api_k2v" ]
|
||||
k2v = [ "garage_util/k2v", "garage_api_k2v", "garage_api_admin/k2v" ]
|
||||
|
||||
# Database engines
|
||||
lmdb = [ "garage_model/lmdb" ]
|
||||
|
|
|
@ -1,235 +0,0 @@
|
|||
use garage_util::data::*;
|
||||
|
||||
use garage_table::*;
|
||||
|
||||
use garage_model::helper::error::{Error, OkOrBadRequest};
|
||||
use garage_model::s3::object_table::*;
|
||||
use garage_model::s3::version_table::*;
|
||||
|
||||
use crate::cli::*;
|
||||
|
||||
use super::*;
|
||||
|
||||
impl AdminRpcHandler {
|
||||
pub(super) async fn handle_block_cmd(&self, cmd: &BlockOperation) -> Result<AdminRpc, Error> {
|
||||
match cmd {
|
||||
BlockOperation::ListErrors => Ok(AdminRpc::BlockErrorList(
|
||||
self.garage.block_manager.list_resync_errors()?,
|
||||
)),
|
||||
BlockOperation::Info { hash } => self.handle_block_info(hash).await,
|
||||
BlockOperation::RetryNow { all, blocks } => {
|
||||
self.handle_block_retry_now(*all, blocks).await
|
||||
}
|
||||
BlockOperation::Purge { yes, blocks } => self.handle_block_purge(*yes, blocks).await,
|
||||
}
|
||||
}
|
||||
|
||||
async fn handle_block_info(&self, hash: &String) -> Result<AdminRpc, Error> {
|
||||
let hash = self.find_block_hash_by_prefix(hash)?;
|
||||
let refcount = self.garage.block_manager.get_block_rc(&hash)?;
|
||||
let block_refs = self
|
||||
.garage
|
||||
.block_ref_table
|
||||
.get_range(&hash, None, None, 10000, Default::default())
|
||||
.await?;
|
||||
let mut versions = vec![];
|
||||
let mut uploads = vec![];
|
||||
for br in block_refs {
|
||||
if let Some(v) = self
|
||||
.garage
|
||||
.version_table
|
||||
.get(&br.version, &EmptyKey)
|
||||
.await?
|
||||
{
|
||||
if let VersionBacklink::MultipartUpload { upload_id } = &v.backlink {
|
||||
if let Some(u) = self.garage.mpu_table.get(upload_id, &EmptyKey).await? {
|
||||
uploads.push(u);
|
||||
}
|
||||
}
|
||||
versions.push(Ok(v));
|
||||
} else {
|
||||
versions.push(Err(br.version));
|
||||
}
|
||||
}
|
||||
Ok(AdminRpc::BlockInfo {
|
||||
hash,
|
||||
refcount,
|
||||
versions,
|
||||
uploads,
|
||||
})
|
||||
}
|
||||
|
||||
async fn handle_block_retry_now(
|
||||
&self,
|
||||
all: bool,
|
||||
blocks: &[String],
|
||||
) -> Result<AdminRpc, Error> {
|
||||
if all {
|
||||
if !blocks.is_empty() {
|
||||
return Err(Error::BadRequest(
|
||||
"--all was specified, cannot also specify blocks".into(),
|
||||
));
|
||||
}
|
||||
let blocks = self.garage.block_manager.list_resync_errors()?;
|
||||
for b in blocks.iter() {
|
||||
self.garage.block_manager.resync.clear_backoff(&b.hash)?;
|
||||
}
|
||||
Ok(AdminRpc::Ok(format!(
|
||||
"{} blocks returned in queue for a retry now (check logs to see results)",
|
||||
blocks.len()
|
||||
)))
|
||||
} else {
|
||||
for hash in blocks {
|
||||
let hash = hex::decode(hash).ok_or_bad_request("invalid hash")?;
|
||||
let hash = Hash::try_from(&hash).ok_or_bad_request("invalid hash")?;
|
||||
self.garage.block_manager.resync.clear_backoff(&hash)?;
|
||||
}
|
||||
Ok(AdminRpc::Ok(format!(
|
||||
"{} blocks returned in queue for a retry now (check logs to see results)",
|
||||
blocks.len()
|
||||
)))
|
||||
}
|
||||
}
|
||||
|
||||
async fn handle_block_purge(&self, yes: bool, blocks: &[String]) -> Result<AdminRpc, Error> {
|
||||
if !yes {
|
||||
return Err(Error::BadRequest(
|
||||
"Pass the --yes flag to confirm block purge operation.".into(),
|
||||
));
|
||||
}
|
||||
|
||||
let mut obj_dels = 0;
|
||||
let mut mpu_dels = 0;
|
||||
let mut ver_dels = 0;
|
||||
|
||||
for hash in blocks {
|
||||
let hash = hex::decode(hash).ok_or_bad_request("invalid hash")?;
|
||||
let hash = Hash::try_from(&hash).ok_or_bad_request("invalid hash")?;
|
||||
let block_refs = self
|
||||
.garage
|
||||
.block_ref_table
|
||||
.get_range(&hash, None, None, 10000, Default::default())
|
||||
.await?;
|
||||
|
||||
for br in block_refs {
|
||||
if let Some(version) = self
|
||||
.garage
|
||||
.version_table
|
||||
.get(&br.version, &EmptyKey)
|
||||
.await?
|
||||
{
|
||||
self.handle_block_purge_version_backlink(
|
||||
&version,
|
||||
&mut obj_dels,
|
||||
&mut mpu_dels,
|
||||
)
|
||||
.await?;
|
||||
|
||||
if !version.deleted.get() {
|
||||
let deleted_version = Version::new(version.uuid, version.backlink, true);
|
||||
self.garage.version_table.insert(&deleted_version).await?;
|
||||
ver_dels += 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(AdminRpc::Ok(format!(
|
||||
"Purged {} blocks, {} versions, {} objects, {} multipart uploads",
|
||||
blocks.len(),
|
||||
ver_dels,
|
||||
obj_dels,
|
||||
mpu_dels,
|
||||
)))
|
||||
}
|
||||
|
||||
async fn handle_block_purge_version_backlink(
|
||||
&self,
|
||||
version: &Version,
|
||||
obj_dels: &mut usize,
|
||||
mpu_dels: &mut usize,
|
||||
) -> Result<(), Error> {
|
||||
let (bucket_id, key, ov_id) = match &version.backlink {
|
||||
VersionBacklink::Object { bucket_id, key } => (*bucket_id, key.clone(), version.uuid),
|
||||
VersionBacklink::MultipartUpload { upload_id } => {
|
||||
if let Some(mut mpu) = self.garage.mpu_table.get(upload_id, &EmptyKey).await? {
|
||||
if !mpu.deleted.get() {
|
||||
mpu.parts.clear();
|
||||
mpu.deleted.set();
|
||||
self.garage.mpu_table.insert(&mpu).await?;
|
||||
*mpu_dels += 1;
|
||||
}
|
||||
(mpu.bucket_id, mpu.key.clone(), *upload_id)
|
||||
} else {
|
||||
return Ok(());
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
if let Some(object) = self.garage.object_table.get(&bucket_id, &key).await? {
|
||||
let ov = object.versions().iter().rev().find(|v| v.is_complete());
|
||||
if let Some(ov) = ov {
|
||||
if ov.uuid == ov_id {
|
||||
let del_uuid = gen_uuid();
|
||||
let deleted_object = Object::new(
|
||||
bucket_id,
|
||||
key,
|
||||
vec![ObjectVersion {
|
||||
uuid: del_uuid,
|
||||
timestamp: ov.timestamp + 1,
|
||||
state: ObjectVersionState::Complete(ObjectVersionData::DeleteMarker),
|
||||
}],
|
||||
);
|
||||
self.garage.object_table.insert(&deleted_object).await?;
|
||||
*obj_dels += 1;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
// ---- helper function ----
|
||||
fn find_block_hash_by_prefix(&self, prefix: &str) -> Result<Hash, Error> {
|
||||
if prefix.len() < 4 {
|
||||
return Err(Error::BadRequest(
|
||||
"Please specify at least 4 characters of the block hash".into(),
|
||||
));
|
||||
}
|
||||
|
||||
let prefix_bin =
|
||||
hex::decode(&prefix[..prefix.len() & !1]).ok_or_bad_request("invalid hash")?;
|
||||
|
||||
let iter = self
|
||||
.garage
|
||||
.block_ref_table
|
||||
.data
|
||||
.store
|
||||
.range(&prefix_bin[..]..)
|
||||
.map_err(GarageError::from)?;
|
||||
let mut found = None;
|
||||
for item in iter {
|
||||
let (k, _v) = item.map_err(GarageError::from)?;
|
||||
let hash = Hash::try_from(&k[..32]).unwrap();
|
||||
if &hash.as_slice()[..prefix_bin.len()] != prefix_bin {
|
||||
break;
|
||||
}
|
||||
if hex::encode(hash.as_slice()).starts_with(prefix) {
|
||||
match &found {
|
||||
Some(x) if *x == hash => (),
|
||||
Some(_) => {
|
||||
return Err(Error::BadRequest(format!(
|
||||
"Several blocks match prefix `{}`",
|
||||
prefix
|
||||
)));
|
||||
}
|
||||
None => {
|
||||
found = Some(hash);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
found.ok_or_else(|| Error::BadRequest("No matching block found".into()))
|
||||
}
|
||||
}
|
|
@ -1,500 +0,0 @@
|
|||
use std::collections::HashMap;
|
||||
use std::fmt::Write;
|
||||
|
||||
use garage_util::crdt::*;
|
||||
use garage_util::time::*;
|
||||
|
||||
use garage_table::*;
|
||||
|
||||
use garage_model::bucket_alias_table::*;
|
||||
use garage_model::bucket_table::*;
|
||||
use garage_model::helper::error::{Error, OkOrBadRequest};
|
||||
use garage_model::permission::*;
|
||||
|
||||
use crate::cli::*;
|
||||
|
||||
use super::*;
|
||||
|
||||
impl AdminRpcHandler {
|
||||
pub(super) async fn handle_bucket_cmd(&self, cmd: &BucketOperation) -> Result<AdminRpc, Error> {
|
||||
match cmd {
|
||||
BucketOperation::List => self.handle_list_buckets().await,
|
||||
BucketOperation::Info(query) => self.handle_bucket_info(query).await,
|
||||
BucketOperation::Create(query) => self.handle_create_bucket(&query.name).await,
|
||||
BucketOperation::Delete(query) => self.handle_delete_bucket(query).await,
|
||||
BucketOperation::Alias(query) => self.handle_alias_bucket(query).await,
|
||||
BucketOperation::Unalias(query) => self.handle_unalias_bucket(query).await,
|
||||
BucketOperation::Allow(query) => self.handle_bucket_allow(query).await,
|
||||
BucketOperation::Deny(query) => self.handle_bucket_deny(query).await,
|
||||
BucketOperation::Website(query) => self.handle_bucket_website(query).await,
|
||||
BucketOperation::SetQuotas(query) => self.handle_bucket_set_quotas(query).await,
|
||||
BucketOperation::CleanupIncompleteUploads(query) => {
|
||||
self.handle_bucket_cleanup_incomplete_uploads(query).await
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
async fn handle_list_buckets(&self) -> Result<AdminRpc, Error> {
|
||||
let buckets = self
|
||||
.garage
|
||||
.bucket_table
|
||||
.get_range(
|
||||
&EmptyKey,
|
||||
None,
|
||||
Some(DeletedFilter::NotDeleted),
|
||||
10000,
|
||||
EnumerationOrder::Forward,
|
||||
)
|
||||
.await?;
|
||||
|
||||
Ok(AdminRpc::BucketList(buckets))
|
||||
}
|
||||
|
||||
async fn handle_bucket_info(&self, query: &BucketOpt) -> Result<AdminRpc, Error> {
|
||||
let bucket_id = self
|
||||
.garage
|
||||
.bucket_helper()
|
||||
.admin_get_existing_matching_bucket(&query.name)
|
||||
.await?;
|
||||
|
||||
let bucket = self
|
||||
.garage
|
||||
.bucket_helper()
|
||||
.get_existing_bucket(bucket_id)
|
||||
.await?;
|
||||
|
||||
let counters = self
|
||||
.garage
|
||||
.object_counter_table
|
||||
.table
|
||||
.get(&bucket_id, &EmptyKey)
|
||||
.await?
|
||||
.map(|x| x.filtered_values(&self.garage.system.cluster_layout()))
|
||||
.unwrap_or_default();
|
||||
|
||||
let mpu_counters = self
|
||||
.garage
|
||||
.mpu_counter_table
|
||||
.table
|
||||
.get(&bucket_id, &EmptyKey)
|
||||
.await?
|
||||
.map(|x| x.filtered_values(&self.garage.system.cluster_layout()))
|
||||
.unwrap_or_default();
|
||||
|
||||
let mut relevant_keys = HashMap::new();
|
||||
for (k, _) in bucket
|
||||
.state
|
||||
.as_option()
|
||||
.unwrap()
|
||||
.authorized_keys
|
||||
.items()
|
||||
.iter()
|
||||
{
|
||||
if let Some(key) = self
|
||||
.garage
|
||||
.key_table
|
||||
.get(&EmptyKey, k)
|
||||
.await?
|
||||
.filter(|k| !k.is_deleted())
|
||||
{
|
||||
relevant_keys.insert(k.clone(), key);
|
||||
}
|
||||
}
|
||||
for ((k, _), _, _) in bucket
|
||||
.state
|
||||
.as_option()
|
||||
.unwrap()
|
||||
.local_aliases
|
||||
.items()
|
||||
.iter()
|
||||
{
|
||||
if relevant_keys.contains_key(k) {
|
||||
continue;
|
||||
}
|
||||
if let Some(key) = self.garage.key_table.get(&EmptyKey, k).await? {
|
||||
relevant_keys.insert(k.clone(), key);
|
||||
}
|
||||
}
|
||||
|
||||
Ok(AdminRpc::BucketInfo {
|
||||
bucket,
|
||||
relevant_keys,
|
||||
counters,
|
||||
mpu_counters,
|
||||
})
|
||||
}
|
||||
|
||||
#[allow(clippy::ptr_arg)]
|
||||
async fn handle_create_bucket(&self, name: &String) -> Result<AdminRpc, Error> {
|
||||
if !is_valid_bucket_name(name) {
|
||||
return Err(Error::BadRequest(format!(
|
||||
"{}: {}",
|
||||
name, INVALID_BUCKET_NAME_MESSAGE
|
||||
)));
|
||||
}
|
||||
|
||||
let helper = self.garage.locked_helper().await;
|
||||
|
||||
if let Some(alias) = self.garage.bucket_alias_table.get(&EmptyKey, name).await? {
|
||||
if alias.state.get().is_some() {
|
||||
return Err(Error::BadRequest(format!("Bucket {} already exists", name)));
|
||||
}
|
||||
}
|
||||
|
||||
// ---- done checking, now commit ----
|
||||
|
||||
let bucket = Bucket::new();
|
||||
self.garage.bucket_table.insert(&bucket).await?;
|
||||
|
||||
helper.set_global_bucket_alias(bucket.id, name).await?;
|
||||
|
||||
Ok(AdminRpc::Ok(format!("Bucket {} was created.", name)))
|
||||
}
|
||||
|
||||
async fn handle_delete_bucket(&self, query: &DeleteBucketOpt) -> Result<AdminRpc, Error> {
|
||||
let helper = self.garage.locked_helper().await;
|
||||
|
||||
let bucket_id = helper
|
||||
.bucket()
|
||||
.admin_get_existing_matching_bucket(&query.name)
|
||||
.await?;
|
||||
|
||||
// Get the alias, but keep in minde here the bucket name
|
||||
// given in parameter can also be directly the bucket's ID.
|
||||
// In that case bucket_alias will be None, and
|
||||
// we can still delete the bucket if it has zero aliases
|
||||
// (a condition which we try to prevent but that could still happen somehow).
|
||||
// We just won't try to delete an alias entry because there isn't one.
|
||||
let bucket_alias = self
|
||||
.garage
|
||||
.bucket_alias_table
|
||||
.get(&EmptyKey, &query.name)
|
||||
.await?;
|
||||
|
||||
// Check bucket doesn't have other aliases
|
||||
let mut bucket = helper.bucket().get_existing_bucket(bucket_id).await?;
|
||||
let bucket_state = bucket.state.as_option().unwrap();
|
||||
if bucket_state
|
||||
.aliases
|
||||
.items()
|
||||
.iter()
|
||||
.filter(|(_, _, active)| *active)
|
||||
.any(|(name, _, _)| name != &query.name)
|
||||
{
|
||||
return Err(Error::BadRequest(format!("Bucket {} still has other global aliases. Use `bucket unalias` to delete them one by one.", query.name)));
|
||||
}
|
||||
if bucket_state
|
||||
.local_aliases
|
||||
.items()
|
||||
.iter()
|
||||
.any(|(_, _, active)| *active)
|
||||
{
|
||||
return Err(Error::BadRequest(format!("Bucket {} still has other local aliases. Use `bucket unalias` to delete them one by one.", query.name)));
|
||||
}
|
||||
|
||||
// Check bucket is empty
|
||||
if !helper.bucket().is_bucket_empty(bucket_id).await? {
|
||||
return Err(Error::BadRequest(format!(
|
||||
"Bucket {} is not empty",
|
||||
query.name
|
||||
)));
|
||||
}
|
||||
|
||||
if !query.yes {
|
||||
return Err(Error::BadRequest(
|
||||
"Add --yes flag to really perform this operation".to_string(),
|
||||
));
|
||||
}
|
||||
|
||||
// --- done checking, now commit ---
|
||||
// 1. delete authorization from keys that had access
|
||||
for (key_id, _) in bucket.authorized_keys() {
|
||||
helper
|
||||
.set_bucket_key_permissions(bucket.id, key_id, BucketKeyPerm::NO_PERMISSIONS)
|
||||
.await?;
|
||||
}
|
||||
|
||||
// 2. delete bucket alias
|
||||
if bucket_alias.is_some() {
|
||||
helper
|
||||
.purge_global_bucket_alias(bucket_id, &query.name)
|
||||
.await?;
|
||||
}
|
||||
|
||||
// 3. delete bucket
|
||||
bucket.state = Deletable::delete();
|
||||
self.garage.bucket_table.insert(&bucket).await?;
|
||||
|
||||
Ok(AdminRpc::Ok(format!("Bucket {} was deleted.", query.name)))
|
||||
}
|
||||
|
||||
async fn handle_alias_bucket(&self, query: &AliasBucketOpt) -> Result<AdminRpc, Error> {
|
||||
let helper = self.garage.locked_helper().await;
|
||||
|
||||
let bucket_id = helper
|
||||
.bucket()
|
||||
.admin_get_existing_matching_bucket(&query.existing_bucket)
|
||||
.await?;
|
||||
|
||||
if let Some(key_pattern) = &query.local {
|
||||
let key = helper.key().get_existing_matching_key(key_pattern).await?;
|
||||
|
||||
helper
|
||||
.set_local_bucket_alias(bucket_id, &key.key_id, &query.new_name)
|
||||
.await?;
|
||||
Ok(AdminRpc::Ok(format!(
|
||||
"Alias {} now points to bucket {:?} in namespace of key {}",
|
||||
query.new_name, bucket_id, key.key_id
|
||||
)))
|
||||
} else {
|
||||
helper
|
||||
.set_global_bucket_alias(bucket_id, &query.new_name)
|
||||
.await?;
|
||||
Ok(AdminRpc::Ok(format!(
|
||||
"Alias {} now points to bucket {:?}",
|
||||
query.new_name, bucket_id
|
||||
)))
|
||||
}
|
||||
}
|
||||
|
||||
async fn handle_unalias_bucket(&self, query: &UnaliasBucketOpt) -> Result<AdminRpc, Error> {
|
||||
let helper = self.garage.locked_helper().await;
|
||||
|
||||
if let Some(key_pattern) = &query.local {
|
||||
let key = helper.key().get_existing_matching_key(key_pattern).await?;
|
||||
|
||||
let bucket_id = key
|
||||
.state
|
||||
.as_option()
|
||||
.unwrap()
|
||||
.local_aliases
|
||||
.get(&query.name)
|
||||
.cloned()
|
||||
.flatten()
|
||||
.ok_or_bad_request("Bucket not found")?;
|
||||
|
||||
helper
|
||||
.unset_local_bucket_alias(bucket_id, &key.key_id, &query.name)
|
||||
.await?;
|
||||
|
||||
Ok(AdminRpc::Ok(format!(
|
||||
"Alias {} no longer points to bucket {:?} in namespace of key {}",
|
||||
&query.name, bucket_id, key.key_id
|
||||
)))
|
||||
} else {
|
||||
let bucket_id = helper
|
||||
.bucket()
|
||||
.resolve_global_bucket_name(&query.name)
|
||||
.await?
|
||||
.ok_or_bad_request("Bucket not found")?;
|
||||
|
||||
helper
|
||||
.unset_global_bucket_alias(bucket_id, &query.name)
|
||||
.await?;
|
||||
|
||||
Ok(AdminRpc::Ok(format!(
|
||||
"Alias {} no longer points to bucket {:?}",
|
||||
&query.name, bucket_id
|
||||
)))
|
||||
}
|
||||
}
|
||||
|
||||
async fn handle_bucket_allow(&self, query: &PermBucketOpt) -> Result<AdminRpc, Error> {
|
||||
let helper = self.garage.locked_helper().await;
|
||||
|
||||
let bucket_id = helper
|
||||
.bucket()
|
||||
.admin_get_existing_matching_bucket(&query.bucket)
|
||||
.await?;
|
||||
let key = helper
|
||||
.key()
|
||||
.get_existing_matching_key(&query.key_pattern)
|
||||
.await?;
|
||||
|
||||
let allow_read = query.read || key.allow_read(&bucket_id);
|
||||
let allow_write = query.write || key.allow_write(&bucket_id);
|
||||
let allow_owner = query.owner || key.allow_owner(&bucket_id);
|
||||
|
||||
helper
|
||||
.set_bucket_key_permissions(
|
||||
bucket_id,
|
||||
&key.key_id,
|
||||
BucketKeyPerm {
|
||||
timestamp: now_msec(),
|
||||
allow_read,
|
||||
allow_write,
|
||||
allow_owner,
|
||||
},
|
||||
)
|
||||
.await?;
|
||||
|
||||
Ok(AdminRpc::Ok(format!(
|
||||
"New permissions for {} on {}: read {}, write {}, owner {}.",
|
||||
&key.key_id, &query.bucket, allow_read, allow_write, allow_owner
|
||||
)))
|
||||
}
|
||||
|
||||
async fn handle_bucket_deny(&self, query: &PermBucketOpt) -> Result<AdminRpc, Error> {
|
||||
let helper = self.garage.locked_helper().await;
|
||||
|
||||
let bucket_id = helper
|
||||
.bucket()
|
||||
.admin_get_existing_matching_bucket(&query.bucket)
|
||||
.await?;
|
||||
let key = helper
|
||||
.key()
|
||||
.get_existing_matching_key(&query.key_pattern)
|
||||
.await?;
|
||||
|
||||
let allow_read = !query.read && key.allow_read(&bucket_id);
|
||||
let allow_write = !query.write && key.allow_write(&bucket_id);
|
||||
let allow_owner = !query.owner && key.allow_owner(&bucket_id);
|
||||
|
||||
helper
|
||||
.set_bucket_key_permissions(
|
||||
bucket_id,
|
||||
&key.key_id,
|
||||
BucketKeyPerm {
|
||||
timestamp: now_msec(),
|
||||
allow_read,
|
||||
allow_write,
|
||||
allow_owner,
|
||||
},
|
||||
)
|
||||
.await?;
|
||||
|
||||
Ok(AdminRpc::Ok(format!(
|
||||
"New permissions for {} on {}: read {}, write {}, owner {}.",
|
||||
&key.key_id, &query.bucket, allow_read, allow_write, allow_owner
|
||||
)))
|
||||
}
|
||||
|
||||
async fn handle_bucket_website(&self, query: &WebsiteOpt) -> Result<AdminRpc, Error> {
|
||||
let bucket_id = self
|
||||
.garage
|
||||
.bucket_helper()
|
||||
.admin_get_existing_matching_bucket(&query.bucket)
|
||||
.await?;
|
||||
|
||||
let mut bucket = self
|
||||
.garage
|
||||
.bucket_helper()
|
||||
.get_existing_bucket(bucket_id)
|
||||
.await?;
|
||||
let bucket_state = bucket.state.as_option_mut().unwrap();
|
||||
|
||||
if !(query.allow ^ query.deny) {
|
||||
return Err(Error::BadRequest(
|
||||
"You must specify exactly one flag, either --allow or --deny".to_string(),
|
||||
));
|
||||
}
|
||||
|
||||
let website = if query.allow {
|
||||
Some(WebsiteConfig {
|
||||
index_document: query.index_document.clone(),
|
||||
error_document: query.error_document.clone(),
|
||||
})
|
||||
} else {
|
||||
None
|
||||
};
|
||||
|
||||
bucket_state.website_config.update(website);
|
||||
self.garage.bucket_table.insert(&bucket).await?;
|
||||
|
||||
let msg = if query.allow {
|
||||
format!("Website access allowed for {}", &query.bucket)
|
||||
} else {
|
||||
format!("Website access denied for {}", &query.bucket)
|
||||
};
|
||||
|
||||
Ok(AdminRpc::Ok(msg))
|
||||
}
|
||||
|
||||
async fn handle_bucket_set_quotas(&self, query: &SetQuotasOpt) -> Result<AdminRpc, Error> {
|
||||
let bucket_id = self
|
||||
.garage
|
||||
.bucket_helper()
|
||||
.admin_get_existing_matching_bucket(&query.bucket)
|
||||
.await?;
|
||||
|
||||
let mut bucket = self
|
||||
.garage
|
||||
.bucket_helper()
|
||||
.get_existing_bucket(bucket_id)
|
||||
.await?;
|
||||
let bucket_state = bucket.state.as_option_mut().unwrap();
|
||||
|
||||
if query.max_size.is_none() && query.max_objects.is_none() {
|
||||
return Err(Error::BadRequest(
|
||||
"You must specify either --max-size or --max-objects (or both) for this command to do something.".to_string(),
|
||||
));
|
||||
}
|
||||
|
||||
let mut quotas = bucket_state.quotas.get().clone();
|
||||
|
||||
match query.max_size.as_ref().map(String::as_ref) {
|
||||
Some("none") => quotas.max_size = None,
|
||||
Some(v) => {
|
||||
let bs = v
|
||||
.parse::<bytesize::ByteSize>()
|
||||
.ok_or_bad_request(format!("Invalid size specified: {}", v))?;
|
||||
quotas.max_size = Some(bs.as_u64());
|
||||
}
|
||||
_ => (),
|
||||
}
|
||||
|
||||
match query.max_objects.as_ref().map(String::as_ref) {
|
||||
Some("none") => quotas.max_objects = None,
|
||||
Some(v) => {
|
||||
let mo = v
|
||||
.parse::<u64>()
|
||||
.ok_or_bad_request(format!("Invalid number specified: {}", v))?;
|
||||
quotas.max_objects = Some(mo);
|
||||
}
|
||||
_ => (),
|
||||
}
|
||||
|
||||
bucket_state.quotas.update(quotas);
|
||||
self.garage.bucket_table.insert(&bucket).await?;
|
||||
|
||||
Ok(AdminRpc::Ok(format!(
|
||||
"Quotas updated for {}",
|
||||
&query.bucket
|
||||
)))
|
||||
}
|
||||
|
||||
async fn handle_bucket_cleanup_incomplete_uploads(
|
||||
&self,
|
||||
query: &CleanupIncompleteUploadsOpt,
|
||||
) -> Result<AdminRpc, Error> {
|
||||
let mut bucket_ids = vec![];
|
||||
for b in query.buckets.iter() {
|
||||
bucket_ids.push(
|
||||
self.garage
|
||||
.bucket_helper()
|
||||
.admin_get_existing_matching_bucket(b)
|
||||
.await?,
|
||||
);
|
||||
}
|
||||
|
||||
let duration = parse_duration::parse::parse(&query.older_than)
|
||||
.ok_or_bad_request("Invalid duration passed for --older-than parameter")?;
|
||||
|
||||
let mut ret = String::new();
|
||||
for bucket in bucket_ids {
|
||||
let count = self
|
||||
.garage
|
||||
.bucket_helper()
|
||||
.cleanup_incomplete_uploads(&bucket, duration)
|
||||
.await?;
|
||||
writeln!(
|
||||
&mut ret,
|
||||
"Bucket {:?}: {} incomplete uploads aborted",
|
||||
bucket, count
|
||||
)
|
||||
.unwrap();
|
||||
}
|
||||
|
||||
Ok(AdminRpc::Ok(ret))
|
||||
}
|
||||
}
|
|
@ -1,161 +0,0 @@
|
|||
use std::collections::HashMap;
|
||||
|
||||
use garage_table::*;
|
||||
|
||||
use garage_model::helper::error::*;
|
||||
use garage_model::key_table::*;
|
||||
|
||||
use crate::cli::*;
|
||||
|
||||
use super::*;
|
||||
|
||||
impl AdminRpcHandler {
|
||||
pub(super) async fn handle_key_cmd(&self, cmd: &KeyOperation) -> Result<AdminRpc, Error> {
|
||||
match cmd {
|
||||
KeyOperation::List => self.handle_list_keys().await,
|
||||
KeyOperation::Info(query) => self.handle_key_info(query).await,
|
||||
KeyOperation::Create(query) => self.handle_create_key(query).await,
|
||||
KeyOperation::Rename(query) => self.handle_rename_key(query).await,
|
||||
KeyOperation::Delete(query) => self.handle_delete_key(query).await,
|
||||
KeyOperation::Allow(query) => self.handle_allow_key(query).await,
|
||||
KeyOperation::Deny(query) => self.handle_deny_key(query).await,
|
||||
KeyOperation::Import(query) => self.handle_import_key(query).await,
|
||||
}
|
||||
}
|
||||
|
||||
async fn handle_list_keys(&self) -> Result<AdminRpc, Error> {
|
||||
let key_ids = self
|
||||
.garage
|
||||
.key_table
|
||||
.get_range(
|
||||
&EmptyKey,
|
||||
None,
|
||||
Some(KeyFilter::Deleted(DeletedFilter::NotDeleted)),
|
||||
10000,
|
||||
EnumerationOrder::Forward,
|
||||
)
|
||||
.await?
|
||||
.iter()
|
||||
.map(|k| (k.key_id.to_string(), k.params().unwrap().name.get().clone()))
|
||||
.collect::<Vec<_>>();
|
||||
Ok(AdminRpc::KeyList(key_ids))
|
||||
}
|
||||
|
||||
async fn handle_key_info(&self, query: &KeyInfoOpt) -> Result<AdminRpc, Error> {
|
||||
let mut key = self
|
||||
.garage
|
||||
.key_helper()
|
||||
.get_existing_matching_key(&query.key_pattern)
|
||||
.await?;
|
||||
|
||||
if !query.show_secret {
|
||||
key.state.as_option_mut().unwrap().secret_key = "(redacted)".into();
|
||||
}
|
||||
|
||||
self.key_info_result(key).await
|
||||
}
|
||||
|
||||
async fn handle_create_key(&self, query: &KeyNewOpt) -> Result<AdminRpc, Error> {
|
||||
let key = Key::new(&query.name);
|
||||
self.garage.key_table.insert(&key).await?;
|
||||
self.key_info_result(key).await
|
||||
}
|
||||
|
||||
async fn handle_rename_key(&self, query: &KeyRenameOpt) -> Result<AdminRpc, Error> {
|
||||
let mut key = self
|
||||
.garage
|
||||
.key_helper()
|
||||
.get_existing_matching_key(&query.key_pattern)
|
||||
.await?;
|
||||
key.params_mut()
|
||||
.unwrap()
|
||||
.name
|
||||
.update(query.new_name.clone());
|
||||
self.garage.key_table.insert(&key).await?;
|
||||
self.key_info_result(key).await
|
||||
}
|
||||
|
||||
async fn handle_delete_key(&self, query: &KeyDeleteOpt) -> Result<AdminRpc, Error> {
|
||||
let helper = self.garage.locked_helper().await;
|
||||
|
||||
let mut key = helper
|
||||
.key()
|
||||
.get_existing_matching_key(&query.key_pattern)
|
||||
.await?;
|
||||
|
||||
if !query.yes {
|
||||
return Err(Error::BadRequest(
|
||||
"Add --yes flag to really perform this operation".to_string(),
|
||||
));
|
||||
}
|
||||
|
||||
helper.delete_key(&mut key).await?;
|
||||
|
||||
Ok(AdminRpc::Ok(format!(
|
||||
"Key {} was deleted successfully.",
|
||||
key.key_id
|
||||
)))
|
||||
}
|
||||
|
||||
async fn handle_allow_key(&self, query: &KeyPermOpt) -> Result<AdminRpc, Error> {
|
||||
let mut key = self
|
||||
.garage
|
||||
.key_helper()
|
||||
.get_existing_matching_key(&query.key_pattern)
|
||||
.await?;
|
||||
if query.create_bucket {
|
||||
key.params_mut().unwrap().allow_create_bucket.update(true);
|
||||
}
|
||||
self.garage.key_table.insert(&key).await?;
|
||||
self.key_info_result(key).await
|
||||
}
|
||||
|
||||
async fn handle_deny_key(&self, query: &KeyPermOpt) -> Result<AdminRpc, Error> {
|
||||
let mut key = self
|
||||
.garage
|
||||
.key_helper()
|
||||
.get_existing_matching_key(&query.key_pattern)
|
||||
.await?;
|
||||
if query.create_bucket {
|
||||
key.params_mut().unwrap().allow_create_bucket.update(false);
|
||||
}
|
||||
self.garage.key_table.insert(&key).await?;
|
||||
self.key_info_result(key).await
|
||||
}
|
||||
|
||||
async fn handle_import_key(&self, query: &KeyImportOpt) -> Result<AdminRpc, Error> {
|
||||
if !query.yes {
|
||||
return Err(Error::BadRequest("This command is intended to re-import keys that were previously generated by Garage. If you want to create a new key, use `garage key new` instead. Add the --yes flag if you really want to re-import a key.".to_string()));
|
||||
}
|
||||
|
||||
let prev_key = self.garage.key_table.get(&EmptyKey, &query.key_id).await?;
|
||||
if prev_key.is_some() {
|
||||
return Err(Error::BadRequest(format!("Key {} already exists in data store. Even if it is deleted, we can't let you create a new key with the same ID. Sorry.", query.key_id)));
|
||||
}
|
||||
|
||||
let imported_key = Key::import(&query.key_id, &query.secret_key, &query.name)
|
||||
.ok_or_bad_request("Invalid key format")?;
|
||||
self.garage.key_table.insert(&imported_key).await?;
|
||||
|
||||
self.key_info_result(imported_key).await
|
||||
}
|
||||
|
||||
async fn key_info_result(&self, key: Key) -> Result<AdminRpc, Error> {
|
||||
let mut relevant_buckets = HashMap::new();
|
||||
|
||||
for (id, _) in key
|
||||
.state
|
||||
.as_option()
|
||||
.unwrap()
|
||||
.authorized_buckets
|
||||
.items()
|
||||
.iter()
|
||||
{
|
||||
if let Some(b) = self.garage.bucket_table.get(&EmptyKey, id).await? {
|
||||
relevant_buckets.insert(*id, b);
|
||||
}
|
||||
}
|
||||
|
||||
Ok(AdminRpc::KeyInfo(key, relevant_buckets))
|
||||
}
|
||||
}
|
|
@ -1,540 +0,0 @@
|
|||
mod block;
|
||||
mod bucket;
|
||||
mod key;
|
||||
|
||||
use std::collections::HashMap;
|
||||
use std::fmt::Write;
|
||||
use std::future::Future;
|
||||
use std::sync::Arc;
|
||||
|
||||
use futures::future::FutureExt;
|
||||
|
||||
use serde::{Deserialize, Serialize};
|
||||
|
||||
use format_table::format_table_to_string;
|
||||
|
||||
use garage_util::background::BackgroundRunner;
|
||||
use garage_util::data::*;
|
||||
use garage_util::error::Error as GarageError;
|
||||
|
||||
use garage_table::replication::*;
|
||||
use garage_table::*;
|
||||
|
||||
use garage_rpc::layout::PARTITION_BITS;
|
||||
use garage_rpc::*;
|
||||
|
||||
use garage_block::manager::BlockResyncErrorInfo;
|
||||
|
||||
use garage_model::bucket_table::*;
|
||||
use garage_model::garage::Garage;
|
||||
use garage_model::helper::error::{Error, OkOrBadRequest};
|
||||
use garage_model::key_table::*;
|
||||
use garage_model::s3::mpu_table::MultipartUpload;
|
||||
use garage_model::s3::version_table::Version;
|
||||
|
||||
use crate::cli::*;
|
||||
use crate::repair::online::launch_online_repair;
|
||||
|
||||
pub const ADMIN_RPC_PATH: &str = "garage/admin_rpc.rs/Rpc";
|
||||
|
||||
#[derive(Debug, Serialize, Deserialize)]
|
||||
#[allow(clippy::large_enum_variant)]
|
||||
pub enum AdminRpc {
|
||||
BucketOperation(BucketOperation),
|
||||
KeyOperation(KeyOperation),
|
||||
LaunchRepair(RepairOpt),
|
||||
Stats(StatsOpt),
|
||||
Worker(WorkerOperation),
|
||||
BlockOperation(BlockOperation),
|
||||
MetaOperation(MetaOperation),
|
||||
|
||||
// Replies
|
||||
Ok(String),
|
||||
BucketList(Vec<Bucket>),
|
||||
BucketInfo {
|
||||
bucket: Bucket,
|
||||
relevant_keys: HashMap<String, Key>,
|
||||
counters: HashMap<String, i64>,
|
||||
mpu_counters: HashMap<String, i64>,
|
||||
},
|
||||
KeyList(Vec<(String, String)>),
|
||||
KeyInfo(Key, HashMap<Uuid, Bucket>),
|
||||
WorkerList(
|
||||
HashMap<usize, garage_util::background::WorkerInfo>,
|
||||
WorkerListOpt,
|
||||
),
|
||||
WorkerVars(Vec<(Uuid, String, String)>),
|
||||
WorkerInfo(usize, garage_util::background::WorkerInfo),
|
||||
BlockErrorList(Vec<BlockResyncErrorInfo>),
|
||||
BlockInfo {
|
||||
hash: Hash,
|
||||
refcount: u64,
|
||||
versions: Vec<Result<Version, Uuid>>,
|
||||
uploads: Vec<MultipartUpload>,
|
||||
},
|
||||
}
|
||||
|
||||
impl Rpc for AdminRpc {
|
||||
type Response = Result<AdminRpc, Error>;
|
||||
}
|
||||
|
||||
pub struct AdminRpcHandler {
|
||||
garage: Arc<Garage>,
|
||||
background: Arc<BackgroundRunner>,
|
||||
endpoint: Arc<Endpoint<AdminRpc, Self>>,
|
||||
}
|
||||
|
||||
impl AdminRpcHandler {
|
||||
pub fn new(garage: Arc<Garage>, background: Arc<BackgroundRunner>) -> Arc<Self> {
|
||||
let endpoint = garage.system.netapp.endpoint(ADMIN_RPC_PATH.into());
|
||||
let admin = Arc::new(Self {
|
||||
garage,
|
||||
background,
|
||||
endpoint,
|
||||
});
|
||||
admin.endpoint.set_handler(admin.clone());
|
||||
admin
|
||||
}
|
||||
|
||||
// ================ REPAIR COMMANDS ====================
|
||||
|
||||
async fn handle_launch_repair(self: &Arc<Self>, opt: RepairOpt) -> Result<AdminRpc, Error> {
|
||||
if !opt.yes {
|
||||
return Err(Error::BadRequest(
|
||||
"Please provide the --yes flag to initiate repair operations.".to_string(),
|
||||
));
|
||||
}
|
||||
if opt.all_nodes {
|
||||
let mut opt_to_send = opt.clone();
|
||||
opt_to_send.all_nodes = false;
|
||||
|
||||
let mut failures = vec![];
|
||||
let all_nodes = self.garage.system.cluster_layout().all_nodes().to_vec();
|
||||
for node in all_nodes.iter() {
|
||||
let node = (*node).into();
|
||||
let resp = self
|
||||
.endpoint
|
||||
.call(
|
||||
&node,
|
||||
AdminRpc::LaunchRepair(opt_to_send.clone()),
|
||||
PRIO_NORMAL,
|
||||
)
|
||||
.await;
|
||||
if !matches!(resp, Ok(Ok(_))) {
|
||||
failures.push(node);
|
||||
}
|
||||
}
|
||||
if failures.is_empty() {
|
||||
Ok(AdminRpc::Ok("Repair launched on all nodes".to_string()))
|
||||
} else {
|
||||
Err(Error::BadRequest(format!(
|
||||
"Could not launch repair on nodes: {:?} (launched successfully on other nodes)",
|
||||
failures
|
||||
)))
|
||||
}
|
||||
} else {
|
||||
launch_online_repair(&self.garage, &self.background, opt).await?;
|
||||
Ok(AdminRpc::Ok(format!(
|
||||
"Repair launched on {:?}",
|
||||
self.garage.system.id
|
||||
)))
|
||||
}
|
||||
}
|
||||
|
||||
// ================ STATS COMMANDS ====================
|
||||
|
||||
async fn handle_stats(&self, opt: StatsOpt) -> Result<AdminRpc, Error> {
|
||||
if opt.all_nodes {
|
||||
let mut ret = String::new();
|
||||
let mut all_nodes = self.garage.system.cluster_layout().all_nodes().to_vec();
|
||||
for node in self.garage.system.get_known_nodes().iter() {
|
||||
if node.is_up && !all_nodes.contains(&node.id) {
|
||||
all_nodes.push(node.id);
|
||||
}
|
||||
}
|
||||
|
||||
for node in all_nodes.iter() {
|
||||
let mut opt = opt.clone();
|
||||
opt.all_nodes = false;
|
||||
opt.skip_global = true;
|
||||
|
||||
writeln!(&mut ret, "\n======================").unwrap();
|
||||
writeln!(&mut ret, "Stats for node {:?}:", node).unwrap();
|
||||
|
||||
let node_id = (*node).into();
|
||||
match self
|
||||
.endpoint
|
||||
.call(&node_id, AdminRpc::Stats(opt), PRIO_NORMAL)
|
||||
.await
|
||||
{
|
||||
Ok(Ok(AdminRpc::Ok(s))) => writeln!(&mut ret, "{}", s).unwrap(),
|
||||
Ok(Ok(x)) => writeln!(&mut ret, "Bad answer: {:?}", x).unwrap(),
|
||||
Ok(Err(e)) => writeln!(&mut ret, "Remote error: {}", e).unwrap(),
|
||||
Err(e) => writeln!(&mut ret, "Network error: {}", e).unwrap(),
|
||||
}
|
||||
}
|
||||
|
||||
writeln!(&mut ret, "\n======================").unwrap();
|
||||
write!(
|
||||
&mut ret,
|
||||
"Cluster statistics:\n\n{}",
|
||||
self.gather_cluster_stats()
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
Ok(AdminRpc::Ok(ret))
|
||||
} else {
|
||||
Ok(AdminRpc::Ok(self.gather_stats_local(opt)?))
|
||||
}
|
||||
}
|
||||
|
||||
fn gather_stats_local(&self, opt: StatsOpt) -> Result<String, Error> {
|
||||
let mut ret = String::new();
|
||||
writeln!(
|
||||
&mut ret,
|
||||
"\nGarage version: {} [features: {}]\nRust compiler version: {}",
|
||||
garage_util::version::garage_version(),
|
||||
garage_util::version::garage_features()
|
||||
.map(|list| list.join(", "))
|
||||
.unwrap_or_else(|| "(unknown)".into()),
|
||||
garage_util::version::rust_version(),
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
writeln!(&mut ret, "\nDatabase engine: {}", self.garage.db.engine()).unwrap();
|
||||
|
||||
// Gather table statistics
|
||||
let mut table = vec![" Table\tItems\tMklItems\tMklTodo\tGcTodo".into()];
|
||||
table.push(self.gather_table_stats(&self.garage.bucket_table)?);
|
||||
table.push(self.gather_table_stats(&self.garage.key_table)?);
|
||||
table.push(self.gather_table_stats(&self.garage.object_table)?);
|
||||
table.push(self.gather_table_stats(&self.garage.version_table)?);
|
||||
table.push(self.gather_table_stats(&self.garage.block_ref_table)?);
|
||||
write!(
|
||||
&mut ret,
|
||||
"\nTable stats:\n{}",
|
||||
format_table_to_string(table)
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
// Gather block manager statistics
|
||||
writeln!(&mut ret, "\nBlock manager stats:").unwrap();
|
||||
let rc_len = self.garage.block_manager.rc_len()?.to_string();
|
||||
|
||||
writeln!(
|
||||
&mut ret,
|
||||
" number of RC entries (~= number of blocks): {}",
|
||||
rc_len
|
||||
)
|
||||
.unwrap();
|
||||
writeln!(
|
||||
&mut ret,
|
||||
" resync queue length: {}",
|
||||
self.garage.block_manager.resync.queue_len()?
|
||||
)
|
||||
.unwrap();
|
||||
writeln!(
|
||||
&mut ret,
|
||||
" blocks with resync errors: {}",
|
||||
self.garage.block_manager.resync.errors_len()?
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
if !opt.skip_global {
|
||||
write!(&mut ret, "\n{}", self.gather_cluster_stats()).unwrap();
|
||||
}
|
||||
|
||||
Ok(ret)
|
||||
}
|
||||
|
||||
fn gather_cluster_stats(&self) -> String {
|
||||
let mut ret = String::new();
|
||||
|
||||
// Gather storage node and free space statistics for current nodes
|
||||
let layout = &self.garage.system.cluster_layout();
|
||||
let mut node_partition_count = HashMap::<Uuid, u64>::new();
|
||||
for short_id in layout.current().ring_assignment_data.iter() {
|
||||
let id = layout.current().node_id_vec[*short_id as usize];
|
||||
*node_partition_count.entry(id).or_default() += 1;
|
||||
}
|
||||
let node_info = self
|
||||
.garage
|
||||
.system
|
||||
.get_known_nodes()
|
||||
.into_iter()
|
||||
.map(|n| (n.id, n))
|
||||
.collect::<HashMap<_, _>>();
|
||||
|
||||
let mut table = vec![" ID\tHostname\tZone\tCapacity\tPart.\tDataAvail\tMetaAvail".into()];
|
||||
for (id, parts) in node_partition_count.iter() {
|
||||
let info = node_info.get(id);
|
||||
let status = info.map(|x| &x.status);
|
||||
let role = layout.current().roles.get(id).and_then(|x| x.0.as_ref());
|
||||
let hostname = status.and_then(|x| x.hostname.as_deref()).unwrap_or("?");
|
||||
let zone = role.map(|x| x.zone.as_str()).unwrap_or("?");
|
||||
let capacity = role
|
||||
.map(|x| x.capacity_string())
|
||||
.unwrap_or_else(|| "?".into());
|
||||
let avail_str = |x| match x {
|
||||
Some((avail, total)) => {
|
||||
let pct = (avail as f64) / (total as f64) * 100.;
|
||||
let avail = bytesize::ByteSize::b(avail);
|
||||
let total = bytesize::ByteSize::b(total);
|
||||
format!("{}/{} ({:.1}%)", avail, total, pct)
|
||||
}
|
||||
None => "?".into(),
|
||||
};
|
||||
let data_avail = avail_str(status.and_then(|x| x.data_disk_avail));
|
||||
let meta_avail = avail_str(status.and_then(|x| x.meta_disk_avail));
|
||||
table.push(format!(
|
||||
" {:?}\t{}\t{}\t{}\t{}\t{}\t{}",
|
||||
id, hostname, zone, capacity, parts, data_avail, meta_avail
|
||||
));
|
||||
}
|
||||
write!(
|
||||
&mut ret,
|
||||
"Storage nodes:\n{}",
|
||||
format_table_to_string(table)
|
||||
)
|
||||
.unwrap();
|
||||
|
||||
let meta_part_avail = node_partition_count
|
||||
.iter()
|
||||
.filter_map(|(id, parts)| {
|
||||
node_info
|
||||
.get(id)
|
||||
.and_then(|x| x.status.meta_disk_avail)
|
||||
.map(|c| c.0 / *parts)
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
let data_part_avail = node_partition_count
|
||||
.iter()
|
||||
.filter_map(|(id, parts)| {
|
||||
node_info
|
||||
.get(id)
|
||||
.and_then(|x| x.status.data_disk_avail)
|
||||
.map(|c| c.0 / *parts)
|
||||
})
|
||||
.collect::<Vec<_>>();
|
||||
if !meta_part_avail.is_empty() && !data_part_avail.is_empty() {
|
||||
let meta_avail =
|
||||
bytesize::ByteSize(meta_part_avail.iter().min().unwrap() * (1 << PARTITION_BITS));
|
||||
let data_avail =
|
||||
bytesize::ByteSize(data_part_avail.iter().min().unwrap() * (1 << PARTITION_BITS));
|
||||
writeln!(
|
||||
&mut ret,
|
||||
"\nEstimated available storage space cluster-wide (might be lower in practice):"
|
||||
)
|
||||
.unwrap();
|
||||
if meta_part_avail.len() < node_partition_count.len()
|
||||
|| data_part_avail.len() < node_partition_count.len()
|
||||
{
|
||||
writeln!(&mut ret, " data: < {}", data_avail).unwrap();
|
||||
writeln!(&mut ret, " metadata: < {}", meta_avail).unwrap();
|
||||
writeln!(&mut ret, "A precise estimate could not be given as information is missing for some storage nodes.").unwrap();
|
||||
} else {
|
||||
writeln!(&mut ret, " data: {}", data_avail).unwrap();
|
||||
writeln!(&mut ret, " metadata: {}", meta_avail).unwrap();
|
||||
}
|
||||
}
|
||||
|
||||
ret
|
||||
}
|
||||
|
||||
fn gather_table_stats<F, R>(&self, t: &Arc<Table<F, R>>) -> Result<String, Error>
|
||||
where
|
||||
F: TableSchema + 'static,
|
||||
R: TableReplication + 'static,
|
||||
{
|
||||
let data_len = t.data.store.len().map_err(GarageError::from)?.to_string();
|
||||
let mkl_len = t.merkle_updater.merkle_tree_len()?.to_string();
|
||||
|
||||
Ok(format!(
|
||||
" {}\t{}\t{}\t{}\t{}",
|
||||
F::TABLE_NAME,
|
||||
data_len,
|
||||
mkl_len,
|
||||
t.merkle_updater.todo_len()?,
|
||||
t.data.gc_todo_len()?
|
||||
))
|
||||
}
|
||||
|
||||
// ================ WORKER COMMANDS ====================
|
||||
|
||||
async fn handle_worker_cmd(&self, cmd: &WorkerOperation) -> Result<AdminRpc, Error> {
|
||||
match cmd {
|
||||
WorkerOperation::List { opt } => {
|
||||
let workers = self.background.get_worker_info();
|
||||
Ok(AdminRpc::WorkerList(workers, *opt))
|
||||
}
|
||||
WorkerOperation::Info { tid } => {
|
||||
let info = self
|
||||
.background
|
||||
.get_worker_info()
|
||||
.get(tid)
|
||||
.ok_or_bad_request(format!("No worker with TID {}", tid))?
|
||||
.clone();
|
||||
Ok(AdminRpc::WorkerInfo(*tid, info))
|
||||
}
|
||||
WorkerOperation::Get {
|
||||
all_nodes,
|
||||
variable,
|
||||
} => self.handle_get_var(*all_nodes, variable).await,
|
||||
WorkerOperation::Set {
|
||||
all_nodes,
|
||||
variable,
|
||||
value,
|
||||
} => self.handle_set_var(*all_nodes, variable, value).await,
|
||||
}
|
||||
}
|
||||
|
||||
async fn handle_get_var(
|
||||
&self,
|
||||
all_nodes: bool,
|
||||
variable: &Option<String>,
|
||||
) -> Result<AdminRpc, Error> {
|
||||
if all_nodes {
|
||||
let mut ret = vec![];
|
||||
let all_nodes = self.garage.system.cluster_layout().all_nodes().to_vec();
|
||||
for node in all_nodes.iter() {
|
||||
let node = (*node).into();
|
||||
match self
|
||||
.endpoint
|
||||
.call(
|
||||
&node,
|
||||
AdminRpc::Worker(WorkerOperation::Get {
|
||||
all_nodes: false,
|
||||
variable: variable.clone(),
|
||||
}),
|
||||
PRIO_NORMAL,
|
||||
)
|
||||
.await??
|
||||
{
|
||||
AdminRpc::WorkerVars(v) => ret.extend(v),
|
||||
m => return Err(GarageError::unexpected_rpc_message(m).into()),
|
||||
}
|
||||
}
|
||||
Ok(AdminRpc::WorkerVars(ret))
|
||||
} else {
|
||||
#[allow(clippy::collapsible_else_if)]
|
||||
if let Some(v) = variable {
|
||||
Ok(AdminRpc::WorkerVars(vec![(
|
||||
self.garage.system.id,
|
||||
v.clone(),
|
||||
self.garage.bg_vars.get(v)?,
|
||||
)]))
|
||||
} else {
|
||||
let mut vars = self.garage.bg_vars.get_all();
|
||||
vars.sort();
|
||||
Ok(AdminRpc::WorkerVars(
|
||||
vars.into_iter()
|
||||
.map(|(k, v)| (self.garage.system.id, k.to_string(), v))
|
||||
.collect(),
|
||||
))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
async fn handle_set_var(
|
||||
&self,
|
||||
all_nodes: bool,
|
||||
variable: &str,
|
||||
value: &str,
|
||||
) -> Result<AdminRpc, Error> {
|
||||
if all_nodes {
|
||||
let mut ret = vec![];
|
||||
let all_nodes = self.garage.system.cluster_layout().all_nodes().to_vec();
|
||||
for node in all_nodes.iter() {
|
||||
let node = (*node).into();
|
||||
match self
|
||||
.endpoint
|
||||
.call(
|
||||
&node,
|
||||
AdminRpc::Worker(WorkerOperation::Set {
|
||||
all_nodes: false,
|
||||
variable: variable.to_string(),
|
||||
value: value.to_string(),
|
||||
}),
|
||||
PRIO_NORMAL,
|
||||
)
|
||||
.await??
|
||||
{
|
||||
AdminRpc::WorkerVars(v) => ret.extend(v),
|
||||
m => return Err(GarageError::unexpected_rpc_message(m).into()),
|
||||
}
|
||||
}
|
||||
Ok(AdminRpc::WorkerVars(ret))
|
||||
} else {
|
||||
self.garage.bg_vars.set(variable, value)?;
|
||||
Ok(AdminRpc::WorkerVars(vec![(
|
||||
self.garage.system.id,
|
||||
variable.to_string(),
|
||||
value.to_string(),
|
||||
)]))
|
||||
}
|
||||
}
|
||||
|
||||
// ================ META DB COMMANDS ====================
|
||||
|
||||
async fn handle_meta_cmd(self: &Arc<Self>, mo: &MetaOperation) -> Result<AdminRpc, Error> {
|
||||
match mo {
|
||||
MetaOperation::Snapshot { all: true } => {
|
||||
let to = self.garage.system.cluster_layout().all_nodes().to_vec();
|
||||
|
||||
let resps = futures::future::join_all(to.iter().map(|to| async move {
|
||||
let to = (*to).into();
|
||||
self.endpoint
|
||||
.call(
|
||||
&to,
|
||||
AdminRpc::MetaOperation(MetaOperation::Snapshot { all: false }),
|
||||
PRIO_NORMAL,
|
||||
)
|
||||
.await?
|
||||
}))
|
||||
.await;
|
||||
|
||||
let mut ret = vec![];
|
||||
for (to, resp) in to.iter().zip(resps.iter()) {
|
||||
let res_str = match resp {
|
||||
Ok(_) => "ok".to_string(),
|
||||
Err(e) => format!("error: {}", e),
|
||||
};
|
||||
ret.push(format!("{:?}\t{}", to, res_str));
|
||||
}
|
||||
|
||||
if resps.iter().any(Result::is_err) {
|
||||
Err(GarageError::Message(format_table_to_string(ret)).into())
|
||||
} else {
|
||||
Ok(AdminRpc::Ok(format_table_to_string(ret)))
|
||||
}
|
||||
}
|
||||
MetaOperation::Snapshot { all: false } => {
|
||||
garage_model::snapshot::async_snapshot_metadata(&self.garage).await?;
|
||||
Ok(AdminRpc::Ok("Snapshot has been saved.".into()))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl EndpointHandler<AdminRpc> for AdminRpcHandler {
|
||||
fn handle(
|
||||
self: &Arc<Self>,
|
||||
message: &AdminRpc,
|
||||
_from: NodeID,
|
||||
) -> impl Future<Output = Result<AdminRpc, Error>> + Send {
|
||||
let self2 = self.clone();
|
||||
async move {
|
||||
match message {
|
||||
AdminRpc::BucketOperation(bo) => self2.handle_bucket_cmd(bo).await,
|
||||
AdminRpc::KeyOperation(ko) => self2.handle_key_cmd(ko).await,
|
||||
AdminRpc::LaunchRepair(opt) => self2.handle_launch_repair(opt.clone()).await,
|
||||
AdminRpc::Stats(opt) => self2.handle_stats(opt.clone()).await,
|
||||
AdminRpc::Worker(wo) => self2.handle_worker_cmd(wo).await,
|
||||
AdminRpc::BlockOperation(bo) => self2.handle_block_cmd(bo).await,
|
||||
AdminRpc::MetaOperation(mo) => self2.handle_meta_cmd(mo).await,
|
||||
m => Err(GarageError::unexpected_rpc_message(m).into()),
|
||||
}
|
||||
}
|
||||
.boxed()
|
||||
}
|
||||
}
|
|
@ -1,280 +0,0 @@
|
|||
use std::collections::{HashMap, HashSet};
|
||||
use std::time::Duration;
|
||||
|
||||
use format_table::format_table;
|
||||
use garage_util::error::*;
|
||||
|
||||
use garage_rpc::layout::*;
|
||||
use garage_rpc::system::*;
|
||||
use garage_rpc::*;
|
||||
|
||||
use garage_model::helper::error::Error as HelperError;
|
||||
|
||||
use crate::admin::*;
|
||||
use crate::cli::*;
|
||||
|
||||
pub async fn cli_command_dispatch(
|
||||
cmd: Command,
|
||||
system_rpc_endpoint: &Endpoint<SystemRpc, ()>,
|
||||
admin_rpc_endpoint: &Endpoint<AdminRpc, ()>,
|
||||
rpc_host: NodeID,
|
||||
) -> Result<(), HelperError> {
|
||||
match cmd {
|
||||
Command::Status => Ok(cmd_status(system_rpc_endpoint, rpc_host).await?),
|
||||
Command::Node(NodeOperation::Connect(connect_opt)) => {
|
||||
Ok(cmd_connect(system_rpc_endpoint, rpc_host, connect_opt).await?)
|
||||
}
|
||||
Command::Layout(layout_opt) => {
|
||||
Ok(cli_layout_command_dispatch(layout_opt, system_rpc_endpoint, rpc_host).await?)
|
||||
}
|
||||
Command::Bucket(bo) => {
|
||||
cmd_admin(admin_rpc_endpoint, rpc_host, AdminRpc::BucketOperation(bo)).await
|
||||
}
|
||||
Command::Key(ko) => {
|
||||
cmd_admin(admin_rpc_endpoint, rpc_host, AdminRpc::KeyOperation(ko)).await
|
||||
}
|
||||
Command::Repair(ro) => {
|
||||
cmd_admin(admin_rpc_endpoint, rpc_host, AdminRpc::LaunchRepair(ro)).await
|
||||
}
|
||||
Command::Stats(so) => cmd_admin(admin_rpc_endpoint, rpc_host, AdminRpc::Stats(so)).await,
|
||||
Command::Worker(wo) => cmd_admin(admin_rpc_endpoint, rpc_host, AdminRpc::Worker(wo)).await,
|
||||
Command::Block(bo) => {
|
||||
cmd_admin(admin_rpc_endpoint, rpc_host, AdminRpc::BlockOperation(bo)).await
|
||||
}
|
||||
Command::Meta(mo) => {
|
||||
cmd_admin(admin_rpc_endpoint, rpc_host, AdminRpc::MetaOperation(mo)).await
|
||||
}
|
||||
_ => unreachable!(),
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn cmd_status(rpc_cli: &Endpoint<SystemRpc, ()>, rpc_host: NodeID) -> Result<(), Error> {
|
||||
let status = fetch_status(rpc_cli, rpc_host).await?;
|
||||
let layout = fetch_layout(rpc_cli, rpc_host).await?;
|
||||
|
||||
println!("==== HEALTHY NODES ====");
|
||||
let mut healthy_nodes =
|
||||
vec!["ID\tHostname\tAddress\tTags\tZone\tCapacity\tDataAvail".to_string()];
|
||||
for adv in status.iter().filter(|adv| adv.is_up) {
|
||||
let host = adv.status.hostname.as_deref().unwrap_or("?");
|
||||
let addr = match adv.addr {
|
||||
Some(addr) => addr.to_string(),
|
||||
None => "N/A".to_string(),
|
||||
};
|
||||
if let Some(NodeRoleV(Some(cfg))) = layout.current().roles.get(&adv.id) {
|
||||
let data_avail = match &adv.status.data_disk_avail {
|
||||
_ if cfg.capacity.is_none() => "N/A".into(),
|
||||
Some((avail, total)) => {
|
||||
let pct = (*avail as f64) / (*total as f64) * 100.;
|
||||
let avail = bytesize::ByteSize::b(*avail);
|
||||
format!("{} ({:.1}%)", avail, pct)
|
||||
}
|
||||
None => "?".into(),
|
||||
};
|
||||
healthy_nodes.push(format!(
|
||||
"{id:?}\t{host}\t{addr}\t[{tags}]\t{zone}\t{capacity}\t{data_avail}",
|
||||
id = adv.id,
|
||||
host = host,
|
||||
addr = addr,
|
||||
tags = cfg.tags.join(","),
|
||||
zone = cfg.zone,
|
||||
capacity = cfg.capacity_string(),
|
||||
data_avail = data_avail,
|
||||
));
|
||||
} else {
|
||||
let prev_role = layout
|
||||
.versions
|
||||
.iter()
|
||||
.rev()
|
||||
.find_map(|x| match x.roles.get(&adv.id) {
|
||||
Some(NodeRoleV(Some(cfg))) => Some(cfg),
|
||||
_ => None,
|
||||
});
|
||||
if let Some(cfg) = prev_role {
|
||||
healthy_nodes.push(format!(
|
||||
"{id:?}\t{host}\t{addr}\t[{tags}]\t{zone}\tdraining metadata...",
|
||||
id = adv.id,
|
||||
host = host,
|
||||
addr = addr,
|
||||
tags = cfg.tags.join(","),
|
||||
zone = cfg.zone,
|
||||
));
|
||||
} else {
|
||||
let new_role = match layout.staging.get().roles.get(&adv.id) {
|
||||
Some(NodeRoleV(Some(_))) => "pending...",
|
||||
_ => "NO ROLE ASSIGNED",
|
||||
};
|
||||
healthy_nodes.push(format!(
|
||||
"{id:?}\t{h}\t{addr}\t\t\t{new_role}",
|
||||
id = adv.id,
|
||||
h = host,
|
||||
addr = addr,
|
||||
new_role = new_role,
|
||||
));
|
||||
}
|
||||
}
|
||||
}
|
||||
format_table(healthy_nodes);
|
||||
|
||||
// Determine which nodes are unhealthy and print that to stdout
|
||||
let status_map = status
|
||||
.iter()
|
||||
.map(|adv| (adv.id, adv))
|
||||
.collect::<HashMap<_, _>>();
|
||||
|
||||
let tf = timeago::Formatter::new();
|
||||
let mut drain_msg = false;
|
||||
let mut failed_nodes = vec!["ID\tHostname\tTags\tZone\tCapacity\tLast seen".to_string()];
|
||||
let mut listed = HashSet::new();
|
||||
for ver in layout.versions.iter().rev() {
|
||||
for (node, _, role) in ver.roles.items().iter() {
|
||||
let cfg = match role {
|
||||
NodeRoleV(Some(role)) if role.capacity.is_some() => role,
|
||||
_ => continue,
|
||||
};
|
||||
|
||||
if listed.contains(node) {
|
||||
continue;
|
||||
}
|
||||
listed.insert(*node);
|
||||
|
||||
let adv = status_map.get(node);
|
||||
if adv.map(|x| x.is_up).unwrap_or(false) {
|
||||
continue;
|
||||
}
|
||||
|
||||
// Node is in a layout version, is not a gateway node, and is not up:
|
||||
// it is in a failed state, add proper line to the output
|
||||
let (host, last_seen) = match adv {
|
||||
Some(adv) => (
|
||||
adv.status.hostname.as_deref().unwrap_or("?"),
|
||||
adv.last_seen_secs_ago
|
||||
.map(|s| tf.convert(Duration::from_secs(s)))
|
||||
.unwrap_or_else(|| "never seen".into()),
|
||||
),
|
||||
None => ("??", "never seen".into()),
|
||||
};
|
||||
let capacity = if ver.version == layout.current().version {
|
||||
cfg.capacity_string()
|
||||
} else {
|
||||
drain_msg = true;
|
||||
"draining metadata...".to_string()
|
||||
};
|
||||
failed_nodes.push(format!(
|
||||
"{id:?}\t{host}\t[{tags}]\t{zone}\t{capacity}\t{last_seen}",
|
||||
id = node,
|
||||
host = host,
|
||||
tags = cfg.tags.join(","),
|
||||
zone = cfg.zone,
|
||||
capacity = capacity,
|
||||
last_seen = last_seen,
|
||||
));
|
||||
}
|
||||
}
|
||||
|
||||
if failed_nodes.len() > 1 {
|
||||
println!("\n==== FAILED NODES ====");
|
||||
format_table(failed_nodes);
|
||||
if drain_msg {
|
||||
println!();
|
||||
println!("Your cluster is expecting to drain data from nodes that are currently unavailable.");
|
||||
println!("If these nodes are definitely dead, please review the layout history with");
|
||||
println!(
|
||||
"`garage layout history` and use `garage layout skip-dead-nodes` to force progress."
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
if print_staging_role_changes(&layout) {
|
||||
println!();
|
||||
println!("Please use `garage layout show` to check the proposed new layout and apply it.");
|
||||
println!();
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_connect(
|
||||
rpc_cli: &Endpoint<SystemRpc, ()>,
|
||||
rpc_host: NodeID,
|
||||
args: ConnectNodeOpt,
|
||||
) -> Result<(), Error> {
|
||||
match rpc_cli
|
||||
.call(&rpc_host, SystemRpc::Connect(args.node), PRIO_NORMAL)
|
||||
.await??
|
||||
{
|
||||
SystemRpc::Ok => {
|
||||
println!("Success.");
|
||||
Ok(())
|
||||
}
|
||||
m => Err(Error::unexpected_rpc_message(m)),
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn cmd_admin(
|
||||
rpc_cli: &Endpoint<AdminRpc, ()>,
|
||||
rpc_host: NodeID,
|
||||
args: AdminRpc,
|
||||
) -> Result<(), HelperError> {
|
||||
match rpc_cli.call(&rpc_host, args, PRIO_NORMAL).await?? {
|
||||
AdminRpc::Ok(msg) => {
|
||||
println!("{}", msg);
|
||||
}
|
||||
AdminRpc::BucketList(bl) => {
|
||||
print_bucket_list(bl);
|
||||
}
|
||||
AdminRpc::BucketInfo {
|
||||
bucket,
|
||||
relevant_keys,
|
||||
counters,
|
||||
mpu_counters,
|
||||
} => {
|
||||
print_bucket_info(&bucket, &relevant_keys, &counters, &mpu_counters);
|
||||
}
|
||||
AdminRpc::KeyList(kl) => {
|
||||
print_key_list(kl);
|
||||
}
|
||||
AdminRpc::KeyInfo(key, rb) => {
|
||||
print_key_info(&key, &rb);
|
||||
}
|
||||
AdminRpc::WorkerList(wi, wlo) => {
|
||||
print_worker_list(wi, wlo);
|
||||
}
|
||||
AdminRpc::WorkerVars(wv) => {
|
||||
print_worker_vars(wv);
|
||||
}
|
||||
AdminRpc::WorkerInfo(tid, wi) => {
|
||||
print_worker_info(tid, wi);
|
||||
}
|
||||
AdminRpc::BlockErrorList(el) => {
|
||||
print_block_error_list(el);
|
||||
}
|
||||
AdminRpc::BlockInfo {
|
||||
hash,
|
||||
refcount,
|
||||
versions,
|
||||
uploads,
|
||||
} => {
|
||||
print_block_info(hash, refcount, versions, uploads);
|
||||
}
|
||||
r => {
|
||||
error!("Unexpected response: {:?}", r);
|
||||
}
|
||||
}
|
||||
Ok(())
|
||||
}
|
||||
|
||||
// ---- utility ----
|
||||
|
||||
pub async fn fetch_status(
|
||||
rpc_cli: &Endpoint<SystemRpc, ()>,
|
||||
rpc_host: NodeID,
|
||||
) -> Result<Vec<KnownNodeInfo>, Error> {
|
||||
match rpc_cli
|
||||
.call(&rpc_host, SystemRpc::GetKnownNodes, PRIO_NORMAL)
|
||||
.await??
|
||||
{
|
||||
SystemRpc::ReturnKnownNodes(nodes) => Ok(nodes),
|
||||
resp => Err(Error::unexpected_rpc_message(resp)),
|
||||
}
|
||||
}
|
|
@ -1,584 +0,0 @@
|
|||
use bytesize::ByteSize;
|
||||
|
||||
use format_table::format_table;
|
||||
use garage_util::crdt::Crdt;
|
||||
use garage_util::error::*;
|
||||
|
||||
use garage_rpc::layout::*;
|
||||
use garage_rpc::system::*;
|
||||
use garage_rpc::*;
|
||||
|
||||
use crate::cli::*;
|
||||
|
||||
pub async fn cli_layout_command_dispatch(
|
||||
cmd: LayoutOperation,
|
||||
system_rpc_endpoint: &Endpoint<SystemRpc, ()>,
|
||||
rpc_host: NodeID,
|
||||
) -> Result<(), Error> {
|
||||
match cmd {
|
||||
LayoutOperation::Assign(assign_opt) => {
|
||||
cmd_assign_role(system_rpc_endpoint, rpc_host, assign_opt).await
|
||||
}
|
||||
LayoutOperation::Remove(remove_opt) => {
|
||||
cmd_remove_role(system_rpc_endpoint, rpc_host, remove_opt).await
|
||||
}
|
||||
LayoutOperation::Show => cmd_show_layout(system_rpc_endpoint, rpc_host).await,
|
||||
LayoutOperation::Apply(apply_opt) => {
|
||||
cmd_apply_layout(system_rpc_endpoint, rpc_host, apply_opt).await
|
||||
}
|
||||
LayoutOperation::Revert(revert_opt) => {
|
||||
cmd_revert_layout(system_rpc_endpoint, rpc_host, revert_opt).await
|
||||
}
|
||||
LayoutOperation::Config(config_opt) => {
|
||||
cmd_config_layout(system_rpc_endpoint, rpc_host, config_opt).await
|
||||
}
|
||||
LayoutOperation::History => cmd_layout_history(system_rpc_endpoint, rpc_host).await,
|
||||
LayoutOperation::SkipDeadNodes(assume_sync_opt) => {
|
||||
cmd_layout_skip_dead_nodes(system_rpc_endpoint, rpc_host, assume_sync_opt).await
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn cmd_assign_role(
|
||||
rpc_cli: &Endpoint<SystemRpc, ()>,
|
||||
rpc_host: NodeID,
|
||||
args: AssignRoleOpt,
|
||||
) -> Result<(), Error> {
|
||||
let status = match rpc_cli
|
||||
.call(&rpc_host, SystemRpc::GetKnownNodes, PRIO_NORMAL)
|
||||
.await??
|
||||
{
|
||||
SystemRpc::ReturnKnownNodes(nodes) => nodes,
|
||||
resp => return Err(Error::Message(format!("Invalid RPC response: {:?}", resp))),
|
||||
};
|
||||
|
||||
let mut layout = fetch_layout(rpc_cli, rpc_host).await?;
|
||||
let all_nodes = layout.get_all_nodes();
|
||||
|
||||
let added_nodes = args
|
||||
.node_ids
|
||||
.iter()
|
||||
.map(|node_id| {
|
||||
find_matching_node(
|
||||
status
|
||||
.iter()
|
||||
.map(|adv| adv.id)
|
||||
.chain(all_nodes.iter().cloned()),
|
||||
node_id,
|
||||
)
|
||||
})
|
||||
.collect::<Result<Vec<_>, _>>()?;
|
||||
|
||||
let mut roles = layout.current().roles.clone();
|
||||
roles.merge(&layout.staging.get().roles);
|
||||
|
||||
for replaced in args.replace.iter() {
|
||||
let replaced_node = find_matching_node(all_nodes.iter().cloned(), replaced)?;
|
||||
match roles.get(&replaced_node) {
|
||||
Some(NodeRoleV(Some(_))) => {
|
||||
layout
|
||||
.staging
|
||||
.get_mut()
|
||||
.roles
|
||||
.merge(&roles.update_mutator(replaced_node, NodeRoleV(None)));
|
||||
}
|
||||
_ => {
|
||||
return Err(Error::Message(format!(
|
||||
"Cannot replace node {:?} as it is not currently in planned layout",
|
||||
replaced_node
|
||||
)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if args.capacity.is_some() && args.gateway {
|
||||
return Err(Error::Message(
|
||||
"-c and -g are mutually exclusive, please configure node either with c>0 to act as a storage node or with -g to act as a gateway node".into()));
|
||||
}
|
||||
if args.capacity == Some(ByteSize::b(0)) {
|
||||
return Err(Error::Message("Invalid capacity value: 0".into()));
|
||||
}
|
||||
|
||||
for added_node in added_nodes {
|
||||
let new_entry = match roles.get(&added_node) {
|
||||
Some(NodeRoleV(Some(old))) => {
|
||||
let capacity = match args.capacity {
|
||||
Some(c) => Some(c.as_u64()),
|
||||
None if args.gateway => None,
|
||||
None => old.capacity,
|
||||
};
|
||||
let tags = if args.tags.is_empty() {
|
||||
old.tags.clone()
|
||||
} else {
|
||||
args.tags.clone()
|
||||
};
|
||||
NodeRole {
|
||||
zone: args.zone.clone().unwrap_or_else(|| old.zone.to_string()),
|
||||
capacity,
|
||||
tags,
|
||||
}
|
||||
}
|
||||
_ => {
|
||||
let capacity = match args.capacity {
|
||||
Some(c) => Some(c.as_u64()),
|
||||
None if args.gateway => None,
|
||||
None => return Err(Error::Message(
|
||||
"Please specify a capacity with the -c flag, or set node explicitly as gateway with -g".into())),
|
||||
};
|
||||
NodeRole {
|
||||
zone: args
|
||||
.zone
|
||||
.clone()
|
||||
.ok_or("Please specify a zone with the -z flag")?,
|
||||
capacity,
|
||||
tags: args.tags.clone(),
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
layout
|
||||
.staging
|
||||
.get_mut()
|
||||
.roles
|
||||
.merge(&roles.update_mutator(added_node, NodeRoleV(Some(new_entry))));
|
||||
}
|
||||
|
||||
send_layout(rpc_cli, rpc_host, layout).await?;
|
||||
|
||||
println!("Role changes are staged but not yet committed.");
|
||||
println!("Use `garage layout show` to view staged role changes,");
|
||||
println!("and `garage layout apply` to enact staged changes.");
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_remove_role(
|
||||
rpc_cli: &Endpoint<SystemRpc, ()>,
|
||||
rpc_host: NodeID,
|
||||
args: RemoveRoleOpt,
|
||||
) -> Result<(), Error> {
|
||||
let mut layout = fetch_layout(rpc_cli, rpc_host).await?;
|
||||
|
||||
let mut roles = layout.current().roles.clone();
|
||||
roles.merge(&layout.staging.get().roles);
|
||||
|
||||
let deleted_node =
|
||||
find_matching_node(roles.items().iter().map(|(id, _, _)| *id), &args.node_id)?;
|
||||
|
||||
layout
|
||||
.staging
|
||||
.get_mut()
|
||||
.roles
|
||||
.merge(&roles.update_mutator(deleted_node, NodeRoleV(None)));
|
||||
|
||||
send_layout(rpc_cli, rpc_host, layout).await?;
|
||||
|
||||
println!("Role removal is staged but not yet committed.");
|
||||
println!("Use `garage layout show` to view staged role changes,");
|
||||
println!("and `garage layout apply` to enact staged changes.");
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_show_layout(
|
||||
rpc_cli: &Endpoint<SystemRpc, ()>,
|
||||
rpc_host: NodeID,
|
||||
) -> Result<(), Error> {
|
||||
let layout = fetch_layout(rpc_cli, rpc_host).await?;
|
||||
|
||||
println!("==== CURRENT CLUSTER LAYOUT ====");
|
||||
print_cluster_layout(layout.current(), "No nodes currently have a role in the cluster.\nSee `garage status` to view available nodes.");
|
||||
println!();
|
||||
println!(
|
||||
"Current cluster layout version: {}",
|
||||
layout.current().version
|
||||
);
|
||||
|
||||
let has_role_changes = print_staging_role_changes(&layout);
|
||||
if has_role_changes {
|
||||
let v = layout.current().version;
|
||||
let res_apply = layout.apply_staged_changes(Some(v + 1));
|
||||
|
||||
// this will print the stats of what partitions
|
||||
// will move around when we apply
|
||||
match res_apply {
|
||||
Ok((layout, msg)) => {
|
||||
println!();
|
||||
println!("==== NEW CLUSTER LAYOUT AFTER APPLYING CHANGES ====");
|
||||
print_cluster_layout(layout.current(), "No nodes have a role in the new layout.");
|
||||
println!();
|
||||
|
||||
for line in msg.iter() {
|
||||
println!("{}", line);
|
||||
}
|
||||
println!("To enact the staged role changes, type:");
|
||||
println!();
|
||||
println!(" garage layout apply --version {}", v + 1);
|
||||
println!();
|
||||
println!("You can also revert all proposed changes with: garage layout revert");
|
||||
}
|
||||
Err(e) => {
|
||||
println!("Error while trying to compute the assignment: {}", e);
|
||||
println!("This new layout cannot yet be applied.");
|
||||
println!("You can also revert all proposed changes with: garage layout revert");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_apply_layout(
|
||||
rpc_cli: &Endpoint<SystemRpc, ()>,
|
||||
rpc_host: NodeID,
|
||||
apply_opt: ApplyLayoutOpt,
|
||||
) -> Result<(), Error> {
|
||||
let layout = fetch_layout(rpc_cli, rpc_host).await?;
|
||||
|
||||
let (layout, msg) = layout.apply_staged_changes(apply_opt.version)?;
|
||||
for line in msg.iter() {
|
||||
println!("{}", line);
|
||||
}
|
||||
|
||||
send_layout(rpc_cli, rpc_host, layout).await?;
|
||||
|
||||
println!("New cluster layout with updated role assignment has been applied in cluster.");
|
||||
println!("Data will now be moved around between nodes accordingly.");
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_revert_layout(
|
||||
rpc_cli: &Endpoint<SystemRpc, ()>,
|
||||
rpc_host: NodeID,
|
||||
revert_opt: RevertLayoutOpt,
|
||||
) -> Result<(), Error> {
|
||||
if !revert_opt.yes {
|
||||
return Err(Error::Message(
|
||||
"Please add the --yes flag to run the layout revert operation".into(),
|
||||
));
|
||||
}
|
||||
|
||||
let layout = fetch_layout(rpc_cli, rpc_host).await?;
|
||||
|
||||
let layout = layout.revert_staged_changes()?;
|
||||
|
||||
send_layout(rpc_cli, rpc_host, layout).await?;
|
||||
|
||||
println!("All proposed role changes in cluster layout have been canceled.");
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_config_layout(
|
||||
rpc_cli: &Endpoint<SystemRpc, ()>,
|
||||
rpc_host: NodeID,
|
||||
config_opt: ConfigLayoutOpt,
|
||||
) -> Result<(), Error> {
|
||||
let mut layout = fetch_layout(rpc_cli, rpc_host).await?;
|
||||
|
||||
let mut did_something = false;
|
||||
match config_opt.redundancy {
|
||||
None => (),
|
||||
Some(r_str) => {
|
||||
let r = r_str
|
||||
.parse::<ZoneRedundancy>()
|
||||
.ok_or_message("invalid zone redundancy value")?;
|
||||
if let ZoneRedundancy::AtLeast(r_int) = r {
|
||||
if r_int > layout.current().replication_factor {
|
||||
return Err(Error::Message(format!(
|
||||
"The zone redundancy must be smaller or equal to the \
|
||||
replication factor ({}).",
|
||||
layout.current().replication_factor
|
||||
)));
|
||||
} else if r_int < 1 {
|
||||
return Err(Error::Message(
|
||||
"The zone redundancy must be at least 1.".into(),
|
||||
));
|
||||
}
|
||||
}
|
||||
|
||||
layout
|
||||
.staging
|
||||
.get_mut()
|
||||
.parameters
|
||||
.update(LayoutParameters { zone_redundancy: r });
|
||||
println!("The zone redundancy parameter has been set to '{}'.", r);
|
||||
did_something = true;
|
||||
}
|
||||
}
|
||||
|
||||
if !did_something {
|
||||
return Err(Error::Message(
|
||||
"Please specify an action for `garage layout config`".into(),
|
||||
));
|
||||
}
|
||||
|
||||
send_layout(rpc_cli, rpc_host, layout).await?;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_layout_history(
|
||||
rpc_cli: &Endpoint<SystemRpc, ()>,
|
||||
rpc_host: NodeID,
|
||||
) -> Result<(), Error> {
|
||||
let layout = fetch_layout(rpc_cli, rpc_host).await?;
|
||||
let min_stored = layout.min_stored();
|
||||
|
||||
println!("==== LAYOUT HISTORY ====");
|
||||
let mut table = vec!["Version\tStatus\tStorage nodes\tGateway nodes".to_string()];
|
||||
for ver in layout
|
||||
.versions
|
||||
.iter()
|
||||
.rev()
|
||||
.chain(layout.old_versions.iter().rev())
|
||||
{
|
||||
let status = if ver.version == layout.current().version {
|
||||
"current"
|
||||
} else if ver.version >= min_stored {
|
||||
"draining"
|
||||
} else {
|
||||
"historical"
|
||||
};
|
||||
table.push(format!(
|
||||
"#{}\t{}\t{}\t{}",
|
||||
ver.version,
|
||||
status,
|
||||
ver.roles
|
||||
.items()
|
||||
.iter()
|
||||
.filter(|(_, _, x)| matches!(x, NodeRoleV(Some(c)) if c.capacity.is_some()))
|
||||
.count(),
|
||||
ver.roles
|
||||
.items()
|
||||
.iter()
|
||||
.filter(|(_, _, x)| matches!(x, NodeRoleV(Some(c)) if c.capacity.is_none()))
|
||||
.count(),
|
||||
));
|
||||
}
|
||||
format_table(table);
|
||||
println!();
|
||||
|
||||
if layout.versions.len() > 1 {
|
||||
println!("==== UPDATE TRACKERS ====");
|
||||
println!("Several layout versions are currently live in the cluster, and data is being migrated.");
|
||||
println!(
|
||||
"This is the internal data that Garage stores to know which nodes have what data."
|
||||
);
|
||||
println!();
|
||||
let mut table = vec!["Node\tAck\tSync\tSync_ack".to_string()];
|
||||
let all_nodes = layout.get_all_nodes();
|
||||
for node in all_nodes.iter() {
|
||||
table.push(format!(
|
||||
"{:?}\t#{}\t#{}\t#{}",
|
||||
node,
|
||||
layout.update_trackers.ack_map.get(node, min_stored),
|
||||
layout.update_trackers.sync_map.get(node, min_stored),
|
||||
layout.update_trackers.sync_ack_map.get(node, min_stored),
|
||||
));
|
||||
}
|
||||
table[1..].sort();
|
||||
format_table(table);
|
||||
|
||||
let min_ack = layout
|
||||
.update_trackers
|
||||
.ack_map
|
||||
.min_among(&all_nodes, layout.min_stored());
|
||||
|
||||
println!();
|
||||
println!(
|
||||
"If some nodes are not catching up to the latest layout version in the update trackers,"
|
||||
);
|
||||
println!("it might be because they are offline or unable to complete a sync successfully.");
|
||||
if min_ack < layout.current().version {
|
||||
println!(
|
||||
"You may force progress using `garage layout skip-dead-nodes --version {}`",
|
||||
layout.current().version
|
||||
);
|
||||
} else {
|
||||
println!(
|
||||
"You may force progress using `garage layout skip-dead-nodes --version {} --allow-missing-data`.",
|
||||
layout.current().version
|
||||
);
|
||||
}
|
||||
} else {
|
||||
println!("Your cluster is currently in a stable state with a single live layout version.");
|
||||
println!("No metadata migration is in progress. Note that the migration of data blocks is not tracked,");
|
||||
println!(
|
||||
"so you might want to keep old nodes online until their data directories become empty."
|
||||
);
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_layout_skip_dead_nodes(
|
||||
rpc_cli: &Endpoint<SystemRpc, ()>,
|
||||
rpc_host: NodeID,
|
||||
opt: SkipDeadNodesOpt,
|
||||
) -> Result<(), Error> {
|
||||
let status = fetch_status(rpc_cli, rpc_host).await?;
|
||||
let mut layout = fetch_layout(rpc_cli, rpc_host).await?;
|
||||
|
||||
if layout.versions.len() == 1 {
|
||||
return Err(Error::Message(
|
||||
"This command cannot be called when there is only one live cluster layout version"
|
||||
.into(),
|
||||
));
|
||||
}
|
||||
|
||||
let min_v = layout.min_stored();
|
||||
if opt.version <= min_v || opt.version > layout.current().version {
|
||||
return Err(Error::Message(format!(
|
||||
"Invalid version, you may use the following version numbers: {}",
|
||||
(min_v + 1..=layout.current().version)
|
||||
.map(|x| x.to_string())
|
||||
.collect::<Vec<_>>()
|
||||
.join(" ")
|
||||
)));
|
||||
}
|
||||
|
||||
let all_nodes = layout.get_all_nodes();
|
||||
let mut did_something = false;
|
||||
for node in all_nodes.iter() {
|
||||
// Update ACK tracker for dead nodes or for all nodes if --allow-missing-data
|
||||
if opt.allow_missing_data || !status.iter().any(|x| x.id == *node && x.is_up) {
|
||||
if layout.update_trackers.ack_map.set_max(*node, opt.version) {
|
||||
println!("Increased the ACK tracker for node {:?}", node);
|
||||
did_something = true;
|
||||
}
|
||||
}
|
||||
|
||||
// If --allow-missing-data, update SYNC tracker for all nodes.
|
||||
if opt.allow_missing_data {
|
||||
if layout.update_trackers.sync_map.set_max(*node, opt.version) {
|
||||
println!("Increased the SYNC tracker for node {:?}", node);
|
||||
did_something = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if did_something {
|
||||
send_layout(rpc_cli, rpc_host, layout).await?;
|
||||
println!("Success.");
|
||||
Ok(())
|
||||
} else if !opt.allow_missing_data {
|
||||
Err(Error::Message("Nothing was done, try passing the `--allow-missing-data` flag to force progress even when not enough nodes can complete a metadata sync.".into()))
|
||||
} else {
|
||||
Err(Error::Message(
|
||||
"Sorry, there is nothing I can do for you. Please wait patiently. If you ask for help, please send the output of the `garage layout history` command.".into(),
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
// --- utility ---
|
||||
|
||||
pub async fn fetch_layout(
|
||||
rpc_cli: &Endpoint<SystemRpc, ()>,
|
||||
rpc_host: NodeID,
|
||||
) -> Result<LayoutHistory, Error> {
|
||||
match rpc_cli
|
||||
.call(&rpc_host, SystemRpc::PullClusterLayout, PRIO_NORMAL)
|
||||
.await??
|
||||
{
|
||||
SystemRpc::AdvertiseClusterLayout(t) => Ok(t),
|
||||
resp => Err(Error::unexpected_rpc_message(resp)),
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn send_layout(
|
||||
rpc_cli: &Endpoint<SystemRpc, ()>,
|
||||
rpc_host: NodeID,
|
||||
layout: LayoutHistory,
|
||||
) -> Result<(), Error> {
|
||||
rpc_cli
|
||||
.call(
|
||||
&rpc_host,
|
||||
SystemRpc::AdvertiseClusterLayout(layout),
|
||||
PRIO_NORMAL,
|
||||
)
|
||||
.await??;
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub fn print_cluster_layout(layout: &LayoutVersion, empty_msg: &str) {
|
||||
let mut table = vec!["ID\tTags\tZone\tCapacity\tUsable capacity".to_string()];
|
||||
for (id, _, role) in layout.roles.items().iter() {
|
||||
let role = match &role.0 {
|
||||
Some(r) => r,
|
||||
_ => continue,
|
||||
};
|
||||
let tags = role.tags.join(",");
|
||||
let usage = layout.get_node_usage(id).unwrap_or(0);
|
||||
let capacity = layout.get_node_capacity(id).unwrap_or(0);
|
||||
if capacity > 0 {
|
||||
table.push(format!(
|
||||
"{:?}\t{}\t{}\t{}\t{} ({:.1}%)",
|
||||
id,
|
||||
tags,
|
||||
role.zone,
|
||||
role.capacity_string(),
|
||||
ByteSize::b(usage as u64 * layout.partition_size).to_string_as(false),
|
||||
(100.0 * usage as f32 * layout.partition_size as f32) / (capacity as f32)
|
||||
));
|
||||
} else {
|
||||
table.push(format!(
|
||||
"{:?}\t{}\t{}\t{}",
|
||||
id,
|
||||
tags,
|
||||
role.zone,
|
||||
role.capacity_string()
|
||||
));
|
||||
};
|
||||
}
|
||||
if table.len() > 1 {
|
||||
format_table(table);
|
||||
println!();
|
||||
println!("Zone redundancy: {}", layout.parameters.zone_redundancy);
|
||||
} else {
|
||||
println!("{}", empty_msg);
|
||||
}
|
||||
}
|
||||
|
||||
pub fn print_staging_role_changes(layout: &LayoutHistory) -> bool {
|
||||
let staging = layout.staging.get();
|
||||
let has_role_changes = staging
|
||||
.roles
|
||||
.items()
|
||||
.iter()
|
||||
.any(|(k, _, v)| layout.current().roles.get(k) != Some(v));
|
||||
let has_layout_changes = *staging.parameters.get() != layout.current().parameters;
|
||||
|
||||
if has_role_changes || has_layout_changes {
|
||||
println!();
|
||||
println!("==== STAGED ROLE CHANGES ====");
|
||||
if has_role_changes {
|
||||
let mut table = vec!["ID\tTags\tZone\tCapacity".to_string()];
|
||||
for (id, _, role) in staging.roles.items().iter() {
|
||||
if layout.current().roles.get(id) == Some(role) {
|
||||
continue;
|
||||
}
|
||||
if let Some(role) = &role.0 {
|
||||
let tags = role.tags.join(",");
|
||||
table.push(format!(
|
||||
"{:?}\t{}\t{}\t{}",
|
||||
id,
|
||||
tags,
|
||||
role.zone,
|
||||
role.capacity_string()
|
||||
));
|
||||
} else {
|
||||
table.push(format!("{:?}\tREMOVED", id));
|
||||
}
|
||||
}
|
||||
format_table(table);
|
||||
println!();
|
||||
}
|
||||
if has_layout_changes {
|
||||
println!(
|
||||
"Zone redundancy: {}",
|
||||
staging.parameters.get().zone_redundancy
|
||||
);
|
||||
}
|
||||
true
|
||||
} else {
|
||||
false
|
||||
}
|
||||
}
|
|
@ -36,16 +36,6 @@ pub fn node_id_command(config_file: PathBuf, quiet: bool) -> Result<(), Error> {
|
|||
);
|
||||
eprintln!(" garage [-c <config file path>] node connect {}", idstr);
|
||||
eprintln!();
|
||||
eprintln!("Or instruct them to connect from here by running:");
|
||||
eprintln!(
|
||||
" garage -c {} -h <remote node> node connect {}",
|
||||
config_file.to_string_lossy(),
|
||||
idstr
|
||||
);
|
||||
eprintln!(
|
||||
"where <remote_node> is their own node identifier in the format: <full-node-id>@<ip>:<port>"
|
||||
);
|
||||
eprintln!();
|
||||
eprintln!("This node identifier can also be added as a bootstrap node in other node's garage.toml files:");
|
||||
eprintln!(" bootstrap_peers = [");
|
||||
eprintln!(" \"{}\",", idstr);
|
3
src/garage/cli/local/mod.rs
Normal file
3
src/garage/cli/local/mod.rs
Normal file
|
@ -0,0 +1,3 @@
|
|||
pub(crate) mod convert_db;
|
||||
pub(crate) mod init;
|
||||
pub(crate) mod repair;
|
|
@ -1,13 +1,4 @@
|
|||
pub(crate) mod cmd;
|
||||
pub(crate) mod init;
|
||||
pub(crate) mod layout;
|
||||
pub(crate) mod structs;
|
||||
pub(crate) mod util;
|
||||
pub mod structs;
|
||||
|
||||
pub(crate) mod convert_db;
|
||||
|
||||
pub(crate) use cmd::*;
|
||||
pub(crate) use init::*;
|
||||
pub(crate) use layout::*;
|
||||
pub(crate) use structs::*;
|
||||
pub(crate) use util::*;
|
||||
pub mod local;
|
||||
pub mod remote;
|
||||
|
|
261
src/garage/cli/remote/admin_token.rs
Normal file
261
src/garage/cli/remote/admin_token.rs
Normal file
|
@ -0,0 +1,261 @@
|
|||
use format_table::format_table;
|
||||
|
||||
use chrono::{Local, Utc};
|
||||
|
||||
use garage_util::error::*;
|
||||
|
||||
use garage_api_admin::api::*;
|
||||
|
||||
use crate::cli::remote::*;
|
||||
use crate::cli::structs::*;
|
||||
|
||||
impl Cli {
|
||||
pub async fn cmd_admin_token(&self, cmd: AdminTokenOperation) -> Result<(), Error> {
|
||||
match cmd {
|
||||
AdminTokenOperation::List => self.cmd_list_admin_tokens().await,
|
||||
AdminTokenOperation::Info { api_token } => self.cmd_admin_token_info(api_token).await,
|
||||
AdminTokenOperation::Create(opt) => self.cmd_create_admin_token(opt).await,
|
||||
AdminTokenOperation::Rename {
|
||||
api_token,
|
||||
new_name,
|
||||
} => self.cmd_rename_admin_token(api_token, new_name).await,
|
||||
AdminTokenOperation::Set(opt) => self.cmd_update_admin_token(opt).await,
|
||||
AdminTokenOperation::Delete { api_token, yes } => {
|
||||
self.cmd_delete_admin_token(api_token, yes).await
|
||||
}
|
||||
AdminTokenOperation::DeleteExpired { yes } => {
|
||||
self.cmd_delete_expired_admin_tokens(yes).await
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn cmd_list_admin_tokens(&self) -> Result<(), Error> {
|
||||
let mut list = self.api_request(ListAdminTokensRequest).await?;
|
||||
|
||||
list.0.sort_by_key(|x| x.created);
|
||||
|
||||
let mut table = vec!["ID\tCreated\tName\tExpiration\tScope".to_string()];
|
||||
for tok in list.0.iter() {
|
||||
let scope = if tok.expired {
|
||||
String::new()
|
||||
} else {
|
||||
table_list_abbr(&tok.scope)
|
||||
};
|
||||
let exp = if tok.expired {
|
||||
"expired".to_string()
|
||||
} else {
|
||||
tok.expiration
|
||||
.map(|x| x.with_timezone(&Local).to_string())
|
||||
.unwrap_or("never".into())
|
||||
};
|
||||
table.push(format!(
|
||||
"{}\t{}\t{}\t{}\t{}",
|
||||
tok.id.as_deref().unwrap_or("-"),
|
||||
tok.created
|
||||
.map(|x| x.with_timezone(&Local).date_naive().to_string())
|
||||
.unwrap_or("-".into()),
|
||||
tok.name,
|
||||
exp,
|
||||
scope,
|
||||
));
|
||||
}
|
||||
format_table(table);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_admin_token_info(&self, search: String) -> Result<(), Error> {
|
||||
let info = self
|
||||
.api_request(GetAdminTokenInfoRequest {
|
||||
id: None,
|
||||
search: Some(search),
|
||||
})
|
||||
.await?;
|
||||
|
||||
print_token_info(&info);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_create_admin_token(&self, opt: AdminTokenCreateOp) -> Result<(), Error> {
|
||||
// TODO
|
||||
let res = self
|
||||
.api_request(CreateAdminTokenRequest(UpdateAdminTokenRequestBody {
|
||||
name: opt.name,
|
||||
expiration: opt
|
||||
.expires_in
|
||||
.map(|x| parse_duration::parse::parse(&x))
|
||||
.transpose()
|
||||
.ok_or_message("Invalid duration passed for --expires-in parameter")?
|
||||
.map(|dur| Utc::now() + dur),
|
||||
scope: opt.scope.map(|s| {
|
||||
s.split(",")
|
||||
.map(|x| x.trim().to_string())
|
||||
.collect::<Vec<_>>()
|
||||
}),
|
||||
}))
|
||||
.await?;
|
||||
|
||||
if opt.quiet {
|
||||
println!("{}", res.secret_token);
|
||||
} else {
|
||||
println!("This is your secret bearer token, it will not be shown again by Garage:");
|
||||
println!("\n {}\n", res.secret_token);
|
||||
print_token_info(&res.info);
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_rename_admin_token(&self, old: String, new: String) -> Result<(), Error> {
|
||||
let token = self
|
||||
.api_request(GetAdminTokenInfoRequest {
|
||||
id: None,
|
||||
search: Some(old),
|
||||
})
|
||||
.await?;
|
||||
|
||||
let info = self
|
||||
.api_request(UpdateAdminTokenRequest {
|
||||
id: token.id.unwrap(),
|
||||
body: UpdateAdminTokenRequestBody {
|
||||
name: Some(new),
|
||||
expiration: None,
|
||||
scope: None,
|
||||
},
|
||||
})
|
||||
.await?;
|
||||
|
||||
print_token_info(&info.0);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_update_admin_token(&self, opt: AdminTokenSetOp) -> Result<(), Error> {
|
||||
let token = self
|
||||
.api_request(GetAdminTokenInfoRequest {
|
||||
id: None,
|
||||
search: Some(opt.api_token),
|
||||
})
|
||||
.await?;
|
||||
|
||||
let info = self
|
||||
.api_request(UpdateAdminTokenRequest {
|
||||
id: token.id.unwrap(),
|
||||
body: UpdateAdminTokenRequestBody {
|
||||
name: None,
|
||||
expiration: opt
|
||||
.expires_in
|
||||
.map(|x| parse_duration::parse::parse(&x))
|
||||
.transpose()
|
||||
.ok_or_message("Invalid duration passed for --expires-in parameter")?
|
||||
.map(|dur| Utc::now() + dur),
|
||||
scope: opt.scope.map({
|
||||
let mut new_scope = token.scope;
|
||||
|scope_str| {
|
||||
if let Some(add) = scope_str.strip_prefix("+") {
|
||||
for a in add.split(",").map(|x| x.trim().to_string()) {
|
||||
if !new_scope.contains(&a) {
|
||||
new_scope.push(a);
|
||||
}
|
||||
}
|
||||
new_scope
|
||||
} else if let Some(sub) = scope_str.strip_prefix("-") {
|
||||
for r in sub.split(",").map(|x| x.trim()) {
|
||||
new_scope.retain(|x| x != r);
|
||||
}
|
||||
new_scope
|
||||
} else {
|
||||
scope_str
|
||||
.split(",")
|
||||
.map(|x| x.trim().to_string())
|
||||
.collect::<Vec<_>>()
|
||||
}
|
||||
}
|
||||
}),
|
||||
},
|
||||
})
|
||||
.await?;
|
||||
|
||||
print_token_info(&info.0);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_delete_admin_token(&self, token: String, yes: bool) -> Result<(), Error> {
|
||||
let token = self
|
||||
.api_request(GetAdminTokenInfoRequest {
|
||||
id: None,
|
||||
search: Some(token),
|
||||
})
|
||||
.await?;
|
||||
|
||||
let id = token.id.unwrap();
|
||||
|
||||
if !yes {
|
||||
return Err(Error::Message(format!(
|
||||
"Add the --yes flag to delete API token `{}` ({})",
|
||||
token.name, id
|
||||
)));
|
||||
}
|
||||
|
||||
self.api_request(DeleteAdminTokenRequest { id }).await?;
|
||||
|
||||
println!("Admin API token has been deleted.");
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_delete_expired_admin_tokens(&self, yes: bool) -> Result<(), Error> {
|
||||
let mut list = self.api_request(ListAdminTokensRequest).await?.0;
|
||||
|
||||
list.retain(|tok| tok.expired);
|
||||
|
||||
if !yes {
|
||||
return Err(Error::Message(format!(
|
||||
"This would delete {} admin API tokens, add the --yes flag to proceed.",
|
||||
list.len(),
|
||||
)));
|
||||
}
|
||||
|
||||
for token in list.iter() {
|
||||
let id = token.id.clone().unwrap();
|
||||
println!("Deleting token `{}` ({})", token.name, id);
|
||||
self.api_request(DeleteAdminTokenRequest { id }).await?;
|
||||
}
|
||||
|
||||
println!("{} admin API tokens have been deleted.", list.len());
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
fn print_token_info(token: &GetAdminTokenInfoResponse) {
|
||||
let mut table = vec![
|
||||
format!("ID:\t{}", token.id.as_ref().unwrap()),
|
||||
format!("Name:\t{}", token.name),
|
||||
format!("Created:\t{}", token.created.unwrap().with_timezone(&Local)),
|
||||
format!(
|
||||
"Validity:\t{}",
|
||||
token.expired.then_some("EXPIRED").unwrap_or("valid")
|
||||
),
|
||||
format!(
|
||||
"Expiration:\t{}",
|
||||
token
|
||||
.expiration
|
||||
.map(|x| x.with_timezone(&Local).to_string())
|
||||
.unwrap_or("never".into())
|
||||
),
|
||||
String::new(),
|
||||
];
|
||||
|
||||
for (i, scope) in token.scope.iter().enumerate() {
|
||||
if i == 0 {
|
||||
table.push(format!("Scope:\t{}", scope));
|
||||
} else {
|
||||
table.push(format!("\t{}", scope));
|
||||
}
|
||||
}
|
||||
|
||||
format_table(table);
|
||||
}
|
169
src/garage/cli/remote/block.rs
Normal file
169
src/garage/cli/remote/block.rs
Normal file
|
@ -0,0 +1,169 @@
|
|||
//use bytesize::ByteSize;
|
||||
use format_table::format_table;
|
||||
|
||||
use garage_util::error::*;
|
||||
|
||||
use garage_api_admin::api::*;
|
||||
|
||||
use crate::cli::remote::*;
|
||||
use crate::cli::structs::*;
|
||||
|
||||
impl Cli {
|
||||
pub async fn cmd_block(&self, cmd: BlockOperation) -> Result<(), Error> {
|
||||
match cmd {
|
||||
BlockOperation::ListErrors => self.cmd_list_block_errors().await,
|
||||
BlockOperation::Info { hash } => self.cmd_get_block_info(hash).await,
|
||||
BlockOperation::RetryNow { all, blocks } => self.cmd_block_retry_now(all, blocks).await,
|
||||
BlockOperation::Purge { yes, blocks } => self.cmd_block_purge(yes, blocks).await,
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn cmd_list_block_errors(&self) -> Result<(), Error> {
|
||||
let errors = self.local_api_request(LocalListBlockErrorsRequest).await?.0;
|
||||
|
||||
let tf = timeago::Formatter::new();
|
||||
let mut tf2 = timeago::Formatter::new();
|
||||
tf2.ago("");
|
||||
|
||||
let mut table = vec!["Hash\tRC\tErrors\tLast error\tNext try".into()];
|
||||
for e in errors {
|
||||
let next_try = if e.next_try_in_secs > 0 {
|
||||
tf2.convert(Duration::from_secs(e.next_try_in_secs))
|
||||
} else {
|
||||
"asap".to_string()
|
||||
};
|
||||
table.push(format!(
|
||||
"{}\t{}\t{}\t{}\tin {}",
|
||||
e.block_hash,
|
||||
e.refcount,
|
||||
e.error_count,
|
||||
tf.convert(Duration::from_secs(e.last_try_secs_ago)),
|
||||
next_try
|
||||
));
|
||||
}
|
||||
format_table(table);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_get_block_info(&self, hash: String) -> Result<(), Error> {
|
||||
let info = self
|
||||
.local_api_request(LocalGetBlockInfoRequest { block_hash: hash })
|
||||
.await?;
|
||||
|
||||
println!("==== BLOCK INFORMATION ====");
|
||||
format_table(vec![
|
||||
format!("Block hash:\t{}", info.block_hash),
|
||||
format!("Refcount:\t{}", info.refcount),
|
||||
]);
|
||||
println!();
|
||||
|
||||
println!("==== REFERENCES TO THIS BLOCK ====");
|
||||
let mut table = vec!["Status\tVersion\tBucket\tKey\tMPU".into()];
|
||||
let mut nondeleted_count = 0;
|
||||
let mut inconsistent_refs = false;
|
||||
for ver in info.versions.iter() {
|
||||
match &ver.backlink {
|
||||
Some(BlockVersionBacklink::Object { bucket_id, key }) => {
|
||||
table.push(format!(
|
||||
"{}\t{:.16}{}\t{:.16}\t{}",
|
||||
ver.ref_deleted.then_some("deleted").unwrap_or("active"),
|
||||
ver.version_id,
|
||||
ver.version_deleted
|
||||
.then_some(" (deleted)")
|
||||
.unwrap_or_default(),
|
||||
bucket_id,
|
||||
key
|
||||
));
|
||||
}
|
||||
Some(BlockVersionBacklink::Upload {
|
||||
upload_id,
|
||||
upload_deleted,
|
||||
upload_garbage_collected: _,
|
||||
bucket_id,
|
||||
key,
|
||||
}) => {
|
||||
table.push(format!(
|
||||
"{}\t{:.16}{}\t{:.16}\t{}\t{:.16}{}",
|
||||
ver.ref_deleted.then_some("deleted").unwrap_or("active"),
|
||||
ver.version_id,
|
||||
ver.version_deleted
|
||||
.then_some(" (deleted)")
|
||||
.unwrap_or_default(),
|
||||
bucket_id.as_deref().unwrap_or(""),
|
||||
key.as_deref().unwrap_or(""),
|
||||
upload_id,
|
||||
upload_deleted.then_some(" (deleted)").unwrap_or_default(),
|
||||
));
|
||||
}
|
||||
None => {
|
||||
table.push(format!("{:.16}\t\t\tyes", ver.version_id));
|
||||
}
|
||||
}
|
||||
if ver.ref_deleted != ver.version_deleted {
|
||||
inconsistent_refs = true;
|
||||
}
|
||||
if !ver.ref_deleted {
|
||||
nondeleted_count += 1;
|
||||
}
|
||||
}
|
||||
format_table(table);
|
||||
|
||||
if inconsistent_refs {
|
||||
println!();
|
||||
println!("There are inconsistencies between the block_ref and the version tables.");
|
||||
println!("Fix them by running `garage repair block-refs`");
|
||||
}
|
||||
|
||||
if info.refcount != nondeleted_count {
|
||||
println!();
|
||||
println!(
|
||||
"Warning: refcount does not match number of non-deleted versions, you should try `garage repair block-rc`."
|
||||
);
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_block_retry_now(&self, all: bool, blocks: Vec<String>) -> Result<(), Error> {
|
||||
let req = match (all, blocks.len()) {
|
||||
(true, 0) => LocalRetryBlockResyncRequest::All { all: true },
|
||||
(false, n) if n > 0 => LocalRetryBlockResyncRequest::Blocks {
|
||||
block_hashes: blocks,
|
||||
},
|
||||
_ => {
|
||||
return Err(Error::Message(
|
||||
"Please specify block hashes or --all (not both)".into(),
|
||||
))
|
||||
}
|
||||
};
|
||||
|
||||
let res = self.local_api_request(req).await?;
|
||||
|
||||
println!(
|
||||
"{} blocks returned in queue for a retry now (check logs to see results)",
|
||||
res.count
|
||||
);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_block_purge(&self, yes: bool, blocks: Vec<String>) -> Result<(), Error> {
|
||||
if !yes {
|
||||
return Err(Error::Message(
|
||||
"Pass the --yes flag to confirm block purge operation.".into(),
|
||||
));
|
||||
}
|
||||
|
||||
let res = self
|
||||
.local_api_request(LocalPurgeBlocksRequest(blocks))
|
||||
.await?;
|
||||
|
||||
println!(
|
||||
"Purged {} blocks: deleted {} versions, {} objects, {} multipart uploads",
|
||||
res.blocks_purged, res.versions_deleted, res.objects_deleted, res.uploads_deleted,
|
||||
);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
582
src/garage/cli/remote/bucket.rs
Normal file
582
src/garage/cli/remote/bucket.rs
Normal file
|
@ -0,0 +1,582 @@
|
|||
//use bytesize::ByteSize;
|
||||
use format_table::format_table;
|
||||
|
||||
use garage_util::error::*;
|
||||
|
||||
use garage_api_admin::api::*;
|
||||
|
||||
use crate::cli::remote::*;
|
||||
use crate::cli::structs::*;
|
||||
|
||||
impl Cli {
|
||||
pub async fn cmd_bucket(&self, cmd: BucketOperation) -> Result<(), Error> {
|
||||
match cmd {
|
||||
BucketOperation::List => self.cmd_list_buckets().await,
|
||||
BucketOperation::Info(query) => self.cmd_bucket_info(query).await,
|
||||
BucketOperation::Create(query) => self.cmd_create_bucket(query).await,
|
||||
BucketOperation::Delete(query) => self.cmd_delete_bucket(query).await,
|
||||
BucketOperation::Alias(query) => self.cmd_alias_bucket(query).await,
|
||||
BucketOperation::Unalias(query) => self.cmd_unalias_bucket(query).await,
|
||||
BucketOperation::Allow(query) => self.cmd_bucket_allow(query).await,
|
||||
BucketOperation::Deny(query) => self.cmd_bucket_deny(query).await,
|
||||
BucketOperation::Website(query) => self.cmd_bucket_website(query).await,
|
||||
BucketOperation::SetQuotas(query) => self.cmd_bucket_set_quotas(query).await,
|
||||
BucketOperation::CleanupIncompleteUploads(query) => {
|
||||
self.cmd_cleanup_incomplete_uploads(query).await
|
||||
}
|
||||
BucketOperation::InspectObject(query) => self.cmd_inspect_object(query).await,
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn cmd_list_buckets(&self) -> Result<(), Error> {
|
||||
let buckets = self.api_request(ListBucketsRequest).await?;
|
||||
|
||||
let mut table = vec!["ID\tGlobal aliases\tLocal aliases".to_string()];
|
||||
for bucket in buckets.0.iter() {
|
||||
table.push(format!(
|
||||
"{:.16}\t{}\t{}",
|
||||
bucket.id,
|
||||
table_list_abbr(&bucket.global_aliases),
|
||||
table_list_abbr(
|
||||
bucket
|
||||
.local_aliases
|
||||
.iter()
|
||||
.map(|x| format!("{}:{}", x.access_key_id, x.alias))
|
||||
),
|
||||
));
|
||||
}
|
||||
format_table(table);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_bucket_info(&self, opt: BucketOpt) -> Result<(), Error> {
|
||||
let bucket = self
|
||||
.api_request(GetBucketInfoRequest {
|
||||
id: None,
|
||||
global_alias: None,
|
||||
search: Some(opt.name),
|
||||
})
|
||||
.await?;
|
||||
|
||||
print_bucket_info(&bucket);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_create_bucket(&self, opt: BucketOpt) -> Result<(), Error> {
|
||||
let bucket = self
|
||||
.api_request(CreateBucketRequest {
|
||||
global_alias: Some(opt.name.clone()),
|
||||
local_alias: None,
|
||||
})
|
||||
.await?;
|
||||
|
||||
print_bucket_info(&bucket.0);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_delete_bucket(&self, opt: DeleteBucketOpt) -> Result<(), Error> {
|
||||
let bucket = self
|
||||
.api_request(GetBucketInfoRequest {
|
||||
id: None,
|
||||
global_alias: None,
|
||||
search: Some(opt.name.clone()),
|
||||
})
|
||||
.await?;
|
||||
|
||||
// CLI-only checks: the bucket must not have other aliases
|
||||
if bucket
|
||||
.global_aliases
|
||||
.iter()
|
||||
.find(|a| **a != opt.name)
|
||||
.is_some()
|
||||
{
|
||||
return Err(Error::Message(format!("Bucket {} still has other global aliases. Use `bucket unalias` to delete them one by one.", opt.name)));
|
||||
}
|
||||
|
||||
if bucket
|
||||
.keys
|
||||
.iter()
|
||||
.any(|k| !k.bucket_local_aliases.is_empty())
|
||||
{
|
||||
return Err(Error::Message(format!("Bucket {} still has other local aliases. Use `bucket unalias` to delete them one by one.", opt.name)));
|
||||
}
|
||||
|
||||
if !opt.yes {
|
||||
println!("About to delete bucket {}.", bucket.id);
|
||||
return Err(Error::Message(
|
||||
"Add --yes flag to really perform this operation".to_string(),
|
||||
));
|
||||
}
|
||||
|
||||
self.api_request(DeleteBucketRequest {
|
||||
id: bucket.id.clone(),
|
||||
})
|
||||
.await?;
|
||||
|
||||
println!("Bucket {} has been deleted.", bucket.id);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_alias_bucket(&self, opt: AliasBucketOpt) -> Result<(), Error> {
|
||||
let bucket = self
|
||||
.api_request(GetBucketInfoRequest {
|
||||
id: None,
|
||||
global_alias: None,
|
||||
search: Some(opt.existing_bucket.clone()),
|
||||
})
|
||||
.await?;
|
||||
|
||||
let res = if let Some(key_pat) = &opt.local {
|
||||
let key = self
|
||||
.api_request(GetKeyInfoRequest {
|
||||
search: Some(key_pat.clone()),
|
||||
id: None,
|
||||
show_secret_key: false,
|
||||
})
|
||||
.await?;
|
||||
|
||||
self.api_request(AddBucketAliasRequest {
|
||||
bucket_id: bucket.id.clone(),
|
||||
alias: BucketAliasEnum::Local {
|
||||
local_alias: opt.new_name.clone(),
|
||||
access_key_id: key.access_key_id.clone(),
|
||||
},
|
||||
})
|
||||
.await?
|
||||
} else {
|
||||
self.api_request(AddBucketAliasRequest {
|
||||
bucket_id: bucket.id.clone(),
|
||||
alias: BucketAliasEnum::Global {
|
||||
global_alias: opt.new_name.clone(),
|
||||
},
|
||||
})
|
||||
.await?
|
||||
};
|
||||
|
||||
print_bucket_info(&res.0);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_unalias_bucket(&self, opt: UnaliasBucketOpt) -> Result<(), Error> {
|
||||
let res = if let Some(key_pat) = &opt.local {
|
||||
let key = self
|
||||
.api_request(GetKeyInfoRequest {
|
||||
search: Some(key_pat.clone()),
|
||||
id: None,
|
||||
show_secret_key: false,
|
||||
})
|
||||
.await?;
|
||||
|
||||
let bucket = key
|
||||
.buckets
|
||||
.iter()
|
||||
.find(|x| x.local_aliases.contains(&opt.name))
|
||||
.ok_or_message(format!(
|
||||
"No bucket called {} in namespace of key {}",
|
||||
opt.name, key.access_key_id
|
||||
))?;
|
||||
|
||||
self.api_request(RemoveBucketAliasRequest {
|
||||
bucket_id: bucket.id.clone(),
|
||||
alias: BucketAliasEnum::Local {
|
||||
access_key_id: key.access_key_id.clone(),
|
||||
local_alias: opt.name.clone(),
|
||||
},
|
||||
})
|
||||
.await?
|
||||
} else {
|
||||
let bucket = self
|
||||
.api_request(GetBucketInfoRequest {
|
||||
id: None,
|
||||
global_alias: Some(opt.name.clone()),
|
||||
search: None,
|
||||
})
|
||||
.await?;
|
||||
|
||||
self.api_request(RemoveBucketAliasRequest {
|
||||
bucket_id: bucket.id.clone(),
|
||||
alias: BucketAliasEnum::Global {
|
||||
global_alias: opt.name.clone(),
|
||||
},
|
||||
})
|
||||
.await?
|
||||
};
|
||||
|
||||
print_bucket_info(&res.0);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_bucket_allow(&self, opt: PermBucketOpt) -> Result<(), Error> {
|
||||
let bucket = self
|
||||
.api_request(GetBucketInfoRequest {
|
||||
id: None,
|
||||
global_alias: None,
|
||||
search: Some(opt.bucket.clone()),
|
||||
})
|
||||
.await?;
|
||||
|
||||
let key = self
|
||||
.api_request(GetKeyInfoRequest {
|
||||
id: None,
|
||||
search: Some(opt.key_pattern.clone()),
|
||||
show_secret_key: false,
|
||||
})
|
||||
.await?;
|
||||
|
||||
let res = self
|
||||
.api_request(AllowBucketKeyRequest(BucketKeyPermChangeRequest {
|
||||
bucket_id: bucket.id.clone(),
|
||||
access_key_id: key.access_key_id.clone(),
|
||||
permissions: ApiBucketKeyPerm {
|
||||
read: opt.read,
|
||||
write: opt.write,
|
||||
owner: opt.owner,
|
||||
},
|
||||
}))
|
||||
.await?;
|
||||
|
||||
print_bucket_info(&res.0);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_bucket_deny(&self, opt: PermBucketOpt) -> Result<(), Error> {
|
||||
let bucket = self
|
||||
.api_request(GetBucketInfoRequest {
|
||||
id: None,
|
||||
global_alias: None,
|
||||
search: Some(opt.bucket.clone()),
|
||||
})
|
||||
.await?;
|
||||
|
||||
let key = self
|
||||
.api_request(GetKeyInfoRequest {
|
||||
id: None,
|
||||
search: Some(opt.key_pattern.clone()),
|
||||
show_secret_key: false,
|
||||
})
|
||||
.await?;
|
||||
|
||||
let res = self
|
||||
.api_request(DenyBucketKeyRequest(BucketKeyPermChangeRequest {
|
||||
bucket_id: bucket.id.clone(),
|
||||
access_key_id: key.access_key_id.clone(),
|
||||
permissions: ApiBucketKeyPerm {
|
||||
read: opt.read,
|
||||
write: opt.write,
|
||||
owner: opt.owner,
|
||||
},
|
||||
}))
|
||||
.await?;
|
||||
|
||||
print_bucket_info(&res.0);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_bucket_website(&self, opt: WebsiteOpt) -> Result<(), Error> {
|
||||
let bucket = self
|
||||
.api_request(GetBucketInfoRequest {
|
||||
id: None,
|
||||
global_alias: None,
|
||||
search: Some(opt.bucket.clone()),
|
||||
})
|
||||
.await?;
|
||||
|
||||
if !(opt.allow ^ opt.deny) {
|
||||
return Err(Error::Message(
|
||||
"You must specify exactly one flag, either --allow or --deny".to_string(),
|
||||
));
|
||||
}
|
||||
|
||||
let wa = if opt.allow {
|
||||
UpdateBucketWebsiteAccess {
|
||||
enabled: true,
|
||||
index_document: Some(opt.index_document.clone()),
|
||||
error_document: opt
|
||||
.error_document
|
||||
.or(bucket.website_config.and_then(|x| x.error_document.clone())),
|
||||
}
|
||||
} else {
|
||||
UpdateBucketWebsiteAccess {
|
||||
enabled: false,
|
||||
index_document: None,
|
||||
error_document: None,
|
||||
}
|
||||
};
|
||||
|
||||
let res = self
|
||||
.api_request(UpdateBucketRequest {
|
||||
id: bucket.id,
|
||||
body: UpdateBucketRequestBody {
|
||||
website_access: Some(wa),
|
||||
quotas: None,
|
||||
},
|
||||
})
|
||||
.await?;
|
||||
|
||||
print_bucket_info(&res.0);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_bucket_set_quotas(&self, opt: SetQuotasOpt) -> Result<(), Error> {
|
||||
let bucket = self
|
||||
.api_request(GetBucketInfoRequest {
|
||||
id: None,
|
||||
global_alias: None,
|
||||
search: Some(opt.bucket.clone()),
|
||||
})
|
||||
.await?;
|
||||
|
||||
if opt.max_size.is_none() && opt.max_objects.is_none() {
|
||||
return Err(Error::Message(
|
||||
"You must specify either --max-size or --max-objects (or both) for this command to do something.".to_string(),
|
||||
));
|
||||
}
|
||||
|
||||
let new_quotas = ApiBucketQuotas {
|
||||
max_size: match opt.max_size.as_deref() {
|
||||
Some("none") => None,
|
||||
Some(v) => Some(
|
||||
v.parse::<bytesize::ByteSize>()
|
||||
.ok_or_message(format!("Invalid size specified: {}", v))?
|
||||
.as_u64(),
|
||||
),
|
||||
None => bucket.quotas.max_size,
|
||||
},
|
||||
max_objects: match opt.max_objects.as_deref() {
|
||||
Some("none") => None,
|
||||
Some(v) => Some(
|
||||
v.parse::<u64>()
|
||||
.ok_or_message(format!("Invalid number: {}", v))?,
|
||||
),
|
||||
None => bucket.quotas.max_objects,
|
||||
},
|
||||
};
|
||||
|
||||
let res = self
|
||||
.api_request(UpdateBucketRequest {
|
||||
id: bucket.id.clone(),
|
||||
body: UpdateBucketRequestBody {
|
||||
website_access: None,
|
||||
quotas: Some(new_quotas),
|
||||
},
|
||||
})
|
||||
.await?;
|
||||
|
||||
print_bucket_info(&res.0);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_cleanup_incomplete_uploads(
|
||||
&self,
|
||||
opt: CleanupIncompleteUploadsOpt,
|
||||
) -> Result<(), Error> {
|
||||
let older_than = parse_duration::parse::parse(&opt.older_than)
|
||||
.ok_or_message("Invalid duration passed for --older-than parameter")?;
|
||||
|
||||
for b in opt.buckets.iter() {
|
||||
let bucket = self
|
||||
.api_request(GetBucketInfoRequest {
|
||||
id: None,
|
||||
global_alias: None,
|
||||
search: Some(b.clone()),
|
||||
})
|
||||
.await?;
|
||||
|
||||
let res = self
|
||||
.api_request(CleanupIncompleteUploadsRequest {
|
||||
bucket_id: bucket.id.clone(),
|
||||
older_than_secs: older_than.as_secs(),
|
||||
})
|
||||
.await?;
|
||||
|
||||
if res.uploads_deleted > 0 {
|
||||
println!("{:.16}: {} uploads deleted", bucket.id, res.uploads_deleted);
|
||||
} else {
|
||||
println!("{:.16}: no uploads deleted", bucket.id);
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_inspect_object(&self, opt: InspectObjectOpt) -> Result<(), Error> {
|
||||
let bucket = self
|
||||
.api_request(GetBucketInfoRequest {
|
||||
id: None,
|
||||
global_alias: None,
|
||||
search: Some(opt.bucket),
|
||||
})
|
||||
.await?;
|
||||
|
||||
let info = self
|
||||
.api_request(InspectObjectRequest {
|
||||
bucket_id: bucket.id,
|
||||
key: opt.key,
|
||||
})
|
||||
.await?;
|
||||
|
||||
for ver in info.versions {
|
||||
println!("==== OBJECT VERSION ====");
|
||||
let mut tab = vec![
|
||||
format!("Bucket ID:\t{}", info.bucket_id),
|
||||
format!("Key:\t{}", info.key),
|
||||
format!("Version ID:\t{}", ver.uuid),
|
||||
format!("Timestamp:\t{}", ver.timestamp),
|
||||
];
|
||||
if let Some(size) = ver.size {
|
||||
let bs = bytesize::ByteSize::b(size);
|
||||
tab.push(format!(
|
||||
"Size:\t{} ({})",
|
||||
bs.to_string_as(true),
|
||||
bs.to_string_as(false)
|
||||
));
|
||||
tab.push(format!("Size (exact):\t{}", size));
|
||||
if !ver.blocks.is_empty() {
|
||||
tab.push(format!("Number of blocks:\t{:?}", ver.blocks.len()));
|
||||
}
|
||||
}
|
||||
if let Some(etag) = ver.etag {
|
||||
tab.push(format!("Etag:\t{}", etag));
|
||||
}
|
||||
tab.extend([
|
||||
format!("Encrypted:\t{}", ver.encrypted),
|
||||
format!("Uploading:\t{}", ver.uploading),
|
||||
format!("Aborted:\t{}", ver.aborted),
|
||||
format!("Delete marker:\t{}", ver.delete_marker),
|
||||
format!("Inline data:\t{}", ver.inline),
|
||||
]);
|
||||
if !ver.headers.is_empty() {
|
||||
tab.push(String::new());
|
||||
tab.extend(ver.headers.iter().map(|(k, v)| format!("{}\t{}", k, v)));
|
||||
}
|
||||
format_table(tab);
|
||||
|
||||
if !ver.blocks.is_empty() {
|
||||
let mut tab = vec!["Part#\tOffset\tBlock hash\tSize".to_string()];
|
||||
tab.extend(ver.blocks.iter().map(|b| {
|
||||
format!(
|
||||
"{:4}\t{:9}\t{}\t{:9}",
|
||||
b.part_number, b.offset, b.hash, b.size
|
||||
)
|
||||
}));
|
||||
println!();
|
||||
format_table(tab);
|
||||
}
|
||||
println!();
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
fn print_bucket_info(bucket: &GetBucketInfoResponse) {
|
||||
println!("==== BUCKET INFORMATION ====");
|
||||
|
||||
let mut info = vec![
|
||||
format!("Bucket:\t{}", bucket.id),
|
||||
String::new(),
|
||||
{
|
||||
let size = bytesize::ByteSize::b(bucket.bytes as u64);
|
||||
format!(
|
||||
"Size:\t{} ({})",
|
||||
size.to_string_as(true),
|
||||
size.to_string_as(false)
|
||||
)
|
||||
},
|
||||
format!("Objects:\t{}", bucket.objects),
|
||||
];
|
||||
|
||||
if bucket.unfinished_uploads > 0 {
|
||||
info.extend([
|
||||
format!(
|
||||
"Unfinished uploads:\t{} multipart uploads",
|
||||
bucket.unfinished_multipart_uploads
|
||||
),
|
||||
format!("\t{} including regular uploads", bucket.unfinished_uploads),
|
||||
{
|
||||
let mpu_size =
|
||||
bytesize::ByteSize::b(bucket.unfinished_multipart_upload_bytes as u64);
|
||||
format!(
|
||||
"Size of unfinished multipart uploads:\t{} ({})",
|
||||
mpu_size.to_string_as(true),
|
||||
mpu_size.to_string_as(false),
|
||||
)
|
||||
},
|
||||
]);
|
||||
}
|
||||
|
||||
info.extend([
|
||||
String::new(),
|
||||
format!("Website access:\t{}", bucket.website_access),
|
||||
]);
|
||||
|
||||
if let Some(wc) = &bucket.website_config {
|
||||
info.extend([
|
||||
format!(" index document:\t{}", wc.index_document),
|
||||
format!(
|
||||
" error document:\t{}",
|
||||
wc.error_document.as_deref().unwrap_or("(not defined)")
|
||||
),
|
||||
]);
|
||||
}
|
||||
|
||||
if bucket.quotas.max_size.is_some() || bucket.quotas.max_objects.is_some() {
|
||||
info.push(String::new());
|
||||
info.push("Quotas:\tenabled".into());
|
||||
if let Some(ms) = bucket.quotas.max_size {
|
||||
let ms = bytesize::ByteSize::b(ms);
|
||||
info.push(format!(
|
||||
" maximum size:\t{} ({})",
|
||||
ms.to_string_as(true),
|
||||
ms.to_string_as(false)
|
||||
));
|
||||
}
|
||||
if let Some(mo) = bucket.quotas.max_objects {
|
||||
info.push(format!(" maximum number of objects:\t{}", mo));
|
||||
}
|
||||
}
|
||||
|
||||
if !bucket.global_aliases.is_empty() {
|
||||
info.push(String::new());
|
||||
for (i, alias) in bucket.global_aliases.iter().enumerate() {
|
||||
if i == 0 && bucket.global_aliases.len() > 1 {
|
||||
info.push(format!("Global aliases:\t{}", alias));
|
||||
} else if i == 0 {
|
||||
info.push(format!("Global alias:\t{}", alias));
|
||||
} else {
|
||||
info.push(format!("\t{}", alias));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
format_table(info);
|
||||
|
||||
println!("");
|
||||
println!("==== KEYS FOR THIS BUCKET ====");
|
||||
let mut key_info = vec!["Permissions\tAccess key\t\tLocal aliases".to_string()];
|
||||
key_info.extend(bucket.keys.iter().map(|key| {
|
||||
let rflag = if key.permissions.read { "R" } else { " " };
|
||||
let wflag = if key.permissions.write { "W" } else { " " };
|
||||
let oflag = if key.permissions.owner { "O" } else { " " };
|
||||
format!(
|
||||
"{}{}{}\t{}\t{}\t{}",
|
||||
rflag,
|
||||
wflag,
|
||||
oflag,
|
||||
key.access_key_id,
|
||||
key.name,
|
||||
key.bucket_local_aliases.to_vec().join(","),
|
||||
)
|
||||
}));
|
||||
format_table(key_info);
|
||||
}
|
160
src/garage/cli/remote/cluster.rs
Normal file
160
src/garage/cli/remote/cluster.rs
Normal file
|
@ -0,0 +1,160 @@
|
|||
use format_table::format_table;
|
||||
|
||||
use garage_util::error::*;
|
||||
|
||||
use garage_api_admin::api::*;
|
||||
|
||||
use crate::cli::remote::layout::*;
|
||||
use crate::cli::remote::*;
|
||||
use crate::cli::structs::*;
|
||||
|
||||
impl Cli {
|
||||
pub async fn cmd_status(&self) -> Result<(), Error> {
|
||||
let status = self.api_request(GetClusterStatusRequest).await?;
|
||||
let layout = self.api_request(GetClusterLayoutRequest).await?;
|
||||
|
||||
println!("==== HEALTHY NODES ====");
|
||||
|
||||
let mut healthy_nodes =
|
||||
vec!["ID\tHostname\tAddress\tTags\tZone\tCapacity\tDataAvail\tVersion".to_string()];
|
||||
|
||||
for adv in status.nodes.iter().filter(|adv| adv.is_up) {
|
||||
let host = adv.hostname.as_deref().unwrap_or("?");
|
||||
let addr = match adv.addr {
|
||||
Some(addr) => addr.to_string(),
|
||||
None => "N/A".to_string(),
|
||||
};
|
||||
if let Some(cfg) = &adv.role {
|
||||
let data_avail = match &adv.data_partition {
|
||||
_ if cfg.capacity.is_none() => "N/A".into(),
|
||||
Some(FreeSpaceResp { available, total }) => {
|
||||
let pct = (*available as f64) / (*total as f64) * 100.;
|
||||
let avail_str = bytesize::ByteSize::b(*available);
|
||||
format!("{} ({:.1}%)", avail_str, pct)
|
||||
}
|
||||
None => "?".into(),
|
||||
};
|
||||
healthy_nodes.push(format!(
|
||||
"{id:.16}\t{host}\t{addr}\t[{tags}]\t{zone}\t{capacity}\t{data_avail}\t{version}",
|
||||
id = adv.id,
|
||||
host = host,
|
||||
addr = addr,
|
||||
tags = cfg.tags.join(","),
|
||||
zone = cfg.zone,
|
||||
capacity = capacity_string(cfg.capacity),
|
||||
data_avail = data_avail,
|
||||
version = adv.garage_version.as_deref().unwrap_or_default(),
|
||||
));
|
||||
} else {
|
||||
let status = match layout.staged_role_changes.iter().find(|x| x.id == adv.id) {
|
||||
Some(NodeRoleChange {
|
||||
action: NodeRoleChangeEnum::Update { .. },
|
||||
..
|
||||
}) => "pending...",
|
||||
_ if adv.draining => "draining metadata..",
|
||||
_ => "NO ROLE ASSIGNED",
|
||||
};
|
||||
healthy_nodes.push(format!(
|
||||
"{id:.16}\t{h}\t{addr}\t\t\t{status}\t\t{version}",
|
||||
id = adv.id,
|
||||
h = host,
|
||||
addr = addr,
|
||||
status = status,
|
||||
version = adv.garage_version.as_deref().unwrap_or_default(),
|
||||
));
|
||||
}
|
||||
}
|
||||
format_table(healthy_nodes);
|
||||
|
||||
let tf = timeago::Formatter::new();
|
||||
let mut drain_msg = false;
|
||||
let mut failed_nodes = vec!["ID\tHostname\tTags\tZone\tCapacity\tLast seen".to_string()];
|
||||
for adv in status.nodes.iter().filter(|x| !x.is_up) {
|
||||
let node = &adv.id;
|
||||
|
||||
let host = adv.hostname.as_deref().unwrap_or("?");
|
||||
let last_seen = adv
|
||||
.last_seen_secs_ago
|
||||
.map(|s| tf.convert(Duration::from_secs(s)))
|
||||
.unwrap_or_else(|| "never seen".into());
|
||||
|
||||
if let Some(cfg) = &adv.role {
|
||||
let capacity = capacity_string(cfg.capacity);
|
||||
|
||||
failed_nodes.push(format!(
|
||||
"{id:.16}\t{host}\t[{tags}]\t{zone}\t{capacity}\t{last_seen}",
|
||||
id = node,
|
||||
host = host,
|
||||
tags = cfg.tags.join(","),
|
||||
zone = cfg.zone,
|
||||
capacity = capacity,
|
||||
last_seen = last_seen,
|
||||
));
|
||||
} else {
|
||||
let status = match layout.staged_role_changes.iter().find(|x| x.id == adv.id) {
|
||||
Some(NodeRoleChange {
|
||||
action: NodeRoleChangeEnum::Update { .. },
|
||||
..
|
||||
}) => "pending...",
|
||||
_ if adv.draining => {
|
||||
drain_msg = true;
|
||||
"draining metadata.."
|
||||
}
|
||||
_ => unreachable!(),
|
||||
};
|
||||
|
||||
failed_nodes.push(format!(
|
||||
"{id:.16}\t{host}\t\t\t{status}\t{last_seen}",
|
||||
id = node,
|
||||
host = host,
|
||||
status = status,
|
||||
last_seen = last_seen,
|
||||
));
|
||||
}
|
||||
}
|
||||
|
||||
if failed_nodes.len() > 1 {
|
||||
println!("\n==== FAILED NODES ====");
|
||||
format_table(failed_nodes);
|
||||
if drain_msg {
|
||||
println!();
|
||||
println!("Your cluster is expecting to drain data from nodes that are currently unavailable.");
|
||||
println!(
|
||||
"If these nodes are definitely dead, please review the layout history with"
|
||||
);
|
||||
println!(
|
||||
"`garage layout history` and use `garage layout skip-dead-nodes` to force progress."
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
if print_staging_role_changes(&layout) {
|
||||
println!();
|
||||
println!(
|
||||
"Please use `garage layout show` to check the proposed new layout and apply it."
|
||||
);
|
||||
println!();
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_connect(&self, opt: ConnectNodeOpt) -> Result<(), Error> {
|
||||
let res = self
|
||||
.api_request(ConnectClusterNodesRequest(vec![opt.node]))
|
||||
.await?;
|
||||
if res.0.len() != 1 {
|
||||
return Err(Error::Message(format!("unexpected response: {:?}", res)));
|
||||
}
|
||||
let res = res.0.into_iter().next().unwrap();
|
||||
if res.success {
|
||||
println!("Success.");
|
||||
Ok(())
|
||||
} else {
|
||||
Err(Error::Message(format!(
|
||||
"Failure: {}",
|
||||
res.error.unwrap_or_default()
|
||||
)))
|
||||
}
|
||||
}
|
||||
}
|
218
src/garage/cli/remote/key.rs
Normal file
218
src/garage/cli/remote/key.rs
Normal file
|
@ -0,0 +1,218 @@
|
|||
use format_table::format_table;
|
||||
|
||||
use garage_util::error::*;
|
||||
|
||||
use garage_api_admin::api::*;
|
||||
|
||||
use crate::cli::remote::*;
|
||||
use crate::cli::structs::*;
|
||||
|
||||
impl Cli {
|
||||
pub async fn cmd_key(&self, cmd: KeyOperation) -> Result<(), Error> {
|
||||
match cmd {
|
||||
KeyOperation::List => self.cmd_list_keys().await,
|
||||
KeyOperation::Info(query) => self.cmd_key_info(query).await,
|
||||
KeyOperation::Create(query) => self.cmd_create_key(query).await,
|
||||
KeyOperation::Rename(query) => self.cmd_rename_key(query).await,
|
||||
KeyOperation::Delete(query) => self.cmd_delete_key(query).await,
|
||||
KeyOperation::Allow(query) => self.cmd_allow_key(query).await,
|
||||
KeyOperation::Deny(query) => self.cmd_deny_key(query).await,
|
||||
KeyOperation::Import(query) => self.cmd_import_key(query).await,
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn cmd_list_keys(&self) -> Result<(), Error> {
|
||||
let keys = self.api_request(ListKeysRequest).await?;
|
||||
|
||||
let mut table = vec!["ID\tName".to_string()];
|
||||
for key in keys.0.iter() {
|
||||
table.push(format!("{}\t{}", key.id, key.name));
|
||||
}
|
||||
format_table(table);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_key_info(&self, opt: KeyInfoOpt) -> Result<(), Error> {
|
||||
let key = self
|
||||
.api_request(GetKeyInfoRequest {
|
||||
id: None,
|
||||
search: Some(opt.key_pattern),
|
||||
show_secret_key: opt.show_secret,
|
||||
})
|
||||
.await?;
|
||||
|
||||
print_key_info(&key);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_create_key(&self, opt: KeyNewOpt) -> Result<(), Error> {
|
||||
let key = self
|
||||
.api_request(CreateKeyRequest {
|
||||
name: Some(opt.name),
|
||||
})
|
||||
.await?;
|
||||
|
||||
print_key_info(&key.0);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_rename_key(&self, opt: KeyRenameOpt) -> Result<(), Error> {
|
||||
let key = self
|
||||
.api_request(GetKeyInfoRequest {
|
||||
id: None,
|
||||
search: Some(opt.key_pattern),
|
||||
show_secret_key: false,
|
||||
})
|
||||
.await?;
|
||||
|
||||
let new_key = self
|
||||
.api_request(UpdateKeyRequest {
|
||||
id: key.access_key_id,
|
||||
body: UpdateKeyRequestBody {
|
||||
name: Some(opt.new_name),
|
||||
allow: None,
|
||||
deny: None,
|
||||
},
|
||||
})
|
||||
.await?;
|
||||
|
||||
print_key_info(&new_key.0);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_delete_key(&self, opt: KeyDeleteOpt) -> Result<(), Error> {
|
||||
let key = self
|
||||
.api_request(GetKeyInfoRequest {
|
||||
id: None,
|
||||
search: Some(opt.key_pattern),
|
||||
show_secret_key: false,
|
||||
})
|
||||
.await?;
|
||||
|
||||
if !opt.yes {
|
||||
println!("About to delete key {}...", key.access_key_id);
|
||||
return Err(Error::Message(
|
||||
"Add --yes flag to really perform this operation".to_string(),
|
||||
));
|
||||
}
|
||||
|
||||
self.api_request(DeleteKeyRequest {
|
||||
id: key.access_key_id.clone(),
|
||||
})
|
||||
.await?;
|
||||
|
||||
println!("Access key {} has been deleted.", key.access_key_id);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_allow_key(&self, opt: KeyPermOpt) -> Result<(), Error> {
|
||||
let key = self
|
||||
.api_request(GetKeyInfoRequest {
|
||||
id: None,
|
||||
search: Some(opt.key_pattern),
|
||||
show_secret_key: false,
|
||||
})
|
||||
.await?;
|
||||
|
||||
let new_key = self
|
||||
.api_request(UpdateKeyRequest {
|
||||
id: key.access_key_id,
|
||||
body: UpdateKeyRequestBody {
|
||||
name: None,
|
||||
allow: Some(KeyPerm {
|
||||
create_bucket: opt.create_bucket,
|
||||
}),
|
||||
deny: None,
|
||||
},
|
||||
})
|
||||
.await?;
|
||||
|
||||
print_key_info(&new_key.0);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_deny_key(&self, opt: KeyPermOpt) -> Result<(), Error> {
|
||||
let key = self
|
||||
.api_request(GetKeyInfoRequest {
|
||||
id: None,
|
||||
search: Some(opt.key_pattern),
|
||||
show_secret_key: false,
|
||||
})
|
||||
.await?;
|
||||
|
||||
let new_key = self
|
||||
.api_request(UpdateKeyRequest {
|
||||
id: key.access_key_id,
|
||||
body: UpdateKeyRequestBody {
|
||||
name: None,
|
||||
allow: None,
|
||||
deny: Some(KeyPerm {
|
||||
create_bucket: opt.create_bucket,
|
||||
}),
|
||||
},
|
||||
})
|
||||
.await?;
|
||||
|
||||
print_key_info(&new_key.0);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_import_key(&self, opt: KeyImportOpt) -> Result<(), Error> {
|
||||
if !opt.yes {
|
||||
return Err(Error::Message("This command is intended to re-import keys that were previously generated by Garage. If you want to create a new key, use `garage key new` instead. Add the --yes flag if you really want to re-import a key.".to_string()));
|
||||
}
|
||||
|
||||
let new_key = self
|
||||
.api_request(ImportKeyRequest {
|
||||
name: Some(opt.name),
|
||||
access_key_id: opt.key_id,
|
||||
secret_access_key: opt.secret_key,
|
||||
})
|
||||
.await?;
|
||||
|
||||
print_key_info(&new_key.0);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
fn print_key_info(key: &GetKeyInfoResponse) {
|
||||
println!("==== ACCESS KEY INFORMATION ====");
|
||||
|
||||
format_table(vec![
|
||||
format!("Key name:\t{}", key.name),
|
||||
format!("Key ID:\t{}", key.access_key_id),
|
||||
format!(
|
||||
"Secret key:\t{}",
|
||||
key.secret_access_key.as_deref().unwrap_or("(redacted)")
|
||||
),
|
||||
format!("Can create buckets:\t{}", key.permissions.create_bucket),
|
||||
]);
|
||||
|
||||
println!("");
|
||||
println!("==== BUCKETS FOR THIS KEY ====");
|
||||
let mut bucket_info = vec!["Permissions\tID\tGlobal aliases\tLocal aliases".to_string()];
|
||||
bucket_info.extend(key.buckets.iter().map(|bucket| {
|
||||
let rflag = if bucket.permissions.read { "R" } else { " " };
|
||||
let wflag = if bucket.permissions.write { "W" } else { " " };
|
||||
let oflag = if bucket.permissions.owner { "O" } else { " " };
|
||||
format!(
|
||||
"{}{}{}\t{:.16}\t{}\t{}",
|
||||
rflag,
|
||||
wflag,
|
||||
oflag,
|
||||
bucket.id,
|
||||
table_list_abbr(&bucket.global_aliases),
|
||||
bucket.local_aliases.join(","),
|
||||
)
|
||||
}));
|
||||
|
||||
format_table(bucket_info);
|
||||
}
|
474
src/garage/cli/remote/layout.rs
Normal file
474
src/garage/cli/remote/layout.rs
Normal file
|
@ -0,0 +1,474 @@
|
|||
use bytesize::ByteSize;
|
||||
use format_table::format_table;
|
||||
|
||||
use garage_util::error::*;
|
||||
|
||||
use garage_api_admin::api::*;
|
||||
|
||||
use crate::cli::remote::*;
|
||||
use crate::cli::structs::*;
|
||||
|
||||
impl Cli {
|
||||
pub async fn layout_command_dispatch(&self, cmd: LayoutOperation) -> Result<(), Error> {
|
||||
match cmd {
|
||||
LayoutOperation::Show => self.cmd_show_layout().await,
|
||||
LayoutOperation::Assign(assign_opt) => self.cmd_assign_role(assign_opt).await,
|
||||
LayoutOperation::Remove(remove_opt) => self.cmd_remove_role(remove_opt).await,
|
||||
LayoutOperation::Config(config_opt) => self.cmd_config_layout(config_opt).await,
|
||||
LayoutOperation::Apply(apply_opt) => self.cmd_apply_layout(apply_opt).await,
|
||||
LayoutOperation::Revert(revert_opt) => self.cmd_revert_layout(revert_opt).await,
|
||||
LayoutOperation::History => self.cmd_layout_history().await,
|
||||
LayoutOperation::SkipDeadNodes(opt) => self.cmd_skip_dead_nodes(opt).await,
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn cmd_show_layout(&self) -> Result<(), Error> {
|
||||
let layout = self.api_request(GetClusterLayoutRequest).await?;
|
||||
|
||||
println!("==== CURRENT CLUSTER LAYOUT ====");
|
||||
print_cluster_layout(&layout, "No nodes currently have a role in the cluster.\nSee `garage status` to view available nodes.");
|
||||
println!();
|
||||
println!("Current cluster layout version: {}", layout.version);
|
||||
|
||||
let has_role_changes = print_staging_role_changes(&layout);
|
||||
if has_role_changes {
|
||||
let res_apply = self.api_request(PreviewClusterLayoutChangesRequest).await?;
|
||||
|
||||
// this will print the stats of what partitions
|
||||
// will move around when we apply
|
||||
match res_apply {
|
||||
PreviewClusterLayoutChangesResponse::Success {
|
||||
message,
|
||||
new_layout,
|
||||
} => {
|
||||
println!();
|
||||
println!("==== NEW CLUSTER LAYOUT AFTER APPLYING CHANGES ====");
|
||||
print_cluster_layout(&new_layout, "No nodes have a role in the new layout.");
|
||||
println!();
|
||||
|
||||
for line in message.iter() {
|
||||
println!("{}", line);
|
||||
}
|
||||
println!("To enact the staged role changes, type:");
|
||||
println!();
|
||||
println!(" garage layout apply --version {}", new_layout.version);
|
||||
println!();
|
||||
println!("You can also revert all proposed changes with: garage layout revert");
|
||||
}
|
||||
PreviewClusterLayoutChangesResponse::Error { error } => {
|
||||
println!("Error while trying to compute the assignment: {}", error);
|
||||
println!("This new layout cannot yet be applied.");
|
||||
println!("You can also revert all proposed changes with: garage layout revert");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_assign_role(&self, opt: AssignRoleOpt) -> Result<(), Error> {
|
||||
let status = self.api_request(GetClusterStatusRequest).await?;
|
||||
let layout = self.api_request(GetClusterLayoutRequest).await?;
|
||||
|
||||
let mut actions = vec![];
|
||||
|
||||
for node in opt.replace.iter() {
|
||||
let id = find_matching_node(&status, &layout, &node)?;
|
||||
|
||||
actions.push(NodeRoleChange {
|
||||
id,
|
||||
action: NodeRoleChangeEnum::Remove { remove: true },
|
||||
});
|
||||
}
|
||||
|
||||
for node in opt.node_ids.iter() {
|
||||
let id = find_matching_node(&status, &layout, &node)?;
|
||||
|
||||
let current = get_staged_or_current_role(&id, &layout);
|
||||
|
||||
let zone = opt
|
||||
.zone
|
||||
.clone()
|
||||
.or_else(|| current.as_ref().map(|c| c.zone.clone()))
|
||||
.ok_or_message("Please specify a zone with the -z flag")?;
|
||||
|
||||
let capacity = if opt.gateway {
|
||||
if opt.capacity.is_some() {
|
||||
return Err(Error::Message("Please specify only -c or -g".into()));
|
||||
}
|
||||
None
|
||||
} else if let Some(cap) = opt.capacity {
|
||||
Some(cap.as_u64())
|
||||
} else {
|
||||
current.as_ref().ok_or_message("Please specify a capacity with the -c flag, or set node explicitly as gateway with -g")?.capacity
|
||||
};
|
||||
|
||||
let tags = if !opt.tags.is_empty() {
|
||||
opt.tags.clone()
|
||||
} else if let Some(cur) = current.as_ref() {
|
||||
cur.tags.clone()
|
||||
} else {
|
||||
vec![]
|
||||
};
|
||||
|
||||
actions.push(NodeRoleChange {
|
||||
id,
|
||||
action: NodeRoleChangeEnum::Update(NodeAssignedRole {
|
||||
zone,
|
||||
capacity,
|
||||
tags,
|
||||
}),
|
||||
});
|
||||
}
|
||||
|
||||
self.api_request(UpdateClusterLayoutRequest {
|
||||
roles: actions,
|
||||
parameters: None,
|
||||
})
|
||||
.await?;
|
||||
|
||||
println!("Role changes are staged but not yet committed.");
|
||||
println!("Use `garage layout show` to view staged role changes,");
|
||||
println!("and `garage layout apply` to enact staged changes.");
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_remove_role(&self, opt: RemoveRoleOpt) -> Result<(), Error> {
|
||||
let status = self.api_request(GetClusterStatusRequest).await?;
|
||||
let layout = self.api_request(GetClusterLayoutRequest).await?;
|
||||
|
||||
let id = find_matching_node(&status, &layout, &opt.node_id)?;
|
||||
|
||||
let actions = vec![NodeRoleChange {
|
||||
id,
|
||||
action: NodeRoleChangeEnum::Remove { remove: true },
|
||||
}];
|
||||
|
||||
self.api_request(UpdateClusterLayoutRequest {
|
||||
roles: actions,
|
||||
parameters: None,
|
||||
})
|
||||
.await?;
|
||||
|
||||
println!("Role removal is staged but not yet committed.");
|
||||
println!("Use `garage layout show` to view staged role changes,");
|
||||
println!("and `garage layout apply` to enact staged changes.");
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_config_layout(&self, config_opt: ConfigLayoutOpt) -> Result<(), Error> {
|
||||
let mut did_something = false;
|
||||
match config_opt.redundancy {
|
||||
None => (),
|
||||
Some(r_str) => {
|
||||
let r = parse_zone_redundancy(&r_str)?;
|
||||
|
||||
self.api_request(UpdateClusterLayoutRequest {
|
||||
roles: vec![],
|
||||
parameters: Some(LayoutParameters { zone_redundancy: r }),
|
||||
})
|
||||
.await?;
|
||||
println!(
|
||||
"The zone redundancy parameter has been set to '{}'.",
|
||||
display_zone_redundancy(r)
|
||||
);
|
||||
did_something = true;
|
||||
}
|
||||
}
|
||||
|
||||
if !did_something {
|
||||
return Err(Error::Message(
|
||||
"Please specify an action for `garage layout config`".into(),
|
||||
));
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_apply_layout(&self, apply_opt: ApplyLayoutOpt) -> Result<(), Error> {
|
||||
let missing_version_error = r#"
|
||||
Please pass the new layout version number to ensure that you are writing the correct version of the cluster layout.
|
||||
To know the correct value of the new layout version, invoke `garage layout show` and review the proposed changes.
|
||||
"#;
|
||||
|
||||
let req = ApplyClusterLayoutRequest {
|
||||
version: apply_opt.version.ok_or_message(missing_version_error)?,
|
||||
};
|
||||
let res = self.api_request(req).await?;
|
||||
|
||||
for line in res.message.iter() {
|
||||
println!("{}", line);
|
||||
}
|
||||
|
||||
println!("New cluster layout with updated role assignment has been applied in cluster.");
|
||||
println!("Data will now be moved around between nodes accordingly.");
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_revert_layout(&self, revert_opt: RevertLayoutOpt) -> Result<(), Error> {
|
||||
if !revert_opt.yes {
|
||||
return Err(Error::Message(
|
||||
"Please add the --yes flag to run the layout revert operation".into(),
|
||||
));
|
||||
}
|
||||
|
||||
self.api_request(RevertClusterLayoutRequest).await?;
|
||||
|
||||
println!("All proposed role changes in cluster layout have been canceled.");
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_layout_history(&self) -> Result<(), Error> {
|
||||
let history = self.api_request(GetClusterLayoutHistoryRequest).await?;
|
||||
|
||||
println!("==== LAYOUT HISTORY ====");
|
||||
let mut table = vec!["Version\tStatus\tStorage nodes\tGateway nodes".to_string()];
|
||||
for ver in history.versions.iter() {
|
||||
table.push(format!(
|
||||
"#{}\t{:?}\t{}\t{}",
|
||||
ver.version, ver.status, ver.storage_nodes, ver.gateway_nodes,
|
||||
));
|
||||
}
|
||||
format_table(table);
|
||||
println!();
|
||||
|
||||
if let Some(update_trackers) = history.update_trackers {
|
||||
println!("==== UPDATE TRACKERS ====");
|
||||
println!("Several layout versions are currently live in the cluster, and data is being migrated.");
|
||||
println!(
|
||||
"This is the internal data that Garage stores to know which nodes have what data."
|
||||
);
|
||||
println!();
|
||||
let mut table = vec!["Node\tAck\tSync\tSync_ack".to_string()];
|
||||
for (node, trackers) in update_trackers.iter() {
|
||||
table.push(format!(
|
||||
"{:.16}\t#{}\t#{}\t#{}",
|
||||
node, trackers.ack, trackers.sync, trackers.sync_ack,
|
||||
));
|
||||
}
|
||||
table[1..].sort();
|
||||
format_table(table);
|
||||
|
||||
println!();
|
||||
println!(
|
||||
"If some nodes are not catching up to the latest layout version in the update trackers,"
|
||||
);
|
||||
println!(
|
||||
"it might be because they are offline or unable to complete a sync successfully."
|
||||
);
|
||||
if history.min_ack < history.current_version {
|
||||
println!(
|
||||
"You may force progress using `garage layout skip-dead-nodes --version {}`",
|
||||
history.current_version
|
||||
);
|
||||
} else {
|
||||
println!(
|
||||
"You may force progress using `garage layout skip-dead-nodes --version {} --allow-missing-data`.",
|
||||
history.current_version
|
||||
);
|
||||
}
|
||||
} else {
|
||||
println!(
|
||||
"Your cluster is currently in a stable state with a single live layout version."
|
||||
);
|
||||
println!("No metadata migration is in progress. Note that the migration of data blocks is not tracked,");
|
||||
println!(
|
||||
"so you might want to keep old nodes online until their data directories become empty."
|
||||
);
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_skip_dead_nodes(&self, opt: SkipDeadNodesOpt) -> Result<(), Error> {
|
||||
let res = self
|
||||
.api_request(ClusterLayoutSkipDeadNodesRequest {
|
||||
version: opt.version,
|
||||
allow_missing_data: opt.allow_missing_data,
|
||||
})
|
||||
.await?;
|
||||
|
||||
if !res.sync_updated.is_empty() || !res.ack_updated.is_empty() {
|
||||
for node in res.ack_updated.iter() {
|
||||
println!("Increased the ACK tracker for node {:.16}", node);
|
||||
}
|
||||
for node in res.sync_updated.iter() {
|
||||
println!("Increased the SYNC tracker for node {:.16}", node);
|
||||
}
|
||||
Ok(())
|
||||
} else if !opt.allow_missing_data {
|
||||
Err(Error::Message("Nothing was done, try passing the `--allow-missing-data` flag to force progress even when not enough nodes can complete a metadata sync.".into()))
|
||||
} else {
|
||||
Err(Error::Message(
|
||||
"Sorry, there is nothing I can do for you. Please wait patiently. If you ask for help, please send the output of the `garage layout history` command.".into(),
|
||||
))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// --------------------------
|
||||
// ---- helper functions ----
|
||||
// --------------------------
|
||||
|
||||
pub fn capacity_string(v: Option<u64>) -> String {
|
||||
match v {
|
||||
Some(c) => ByteSize::b(c).to_string_as(false),
|
||||
None => "gateway".to_string(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn get_staged_or_current_role(
|
||||
id: &str,
|
||||
layout: &GetClusterLayoutResponse,
|
||||
) -> Option<NodeAssignedRole> {
|
||||
for node in layout.staged_role_changes.iter() {
|
||||
if node.id == id {
|
||||
return match &node.action {
|
||||
NodeRoleChangeEnum::Remove { .. } => None,
|
||||
NodeRoleChangeEnum::Update(role) => Some(role.clone()),
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
for node in layout.roles.iter() {
|
||||
if node.id == id {
|
||||
return Some(NodeAssignedRole {
|
||||
zone: node.zone.clone(),
|
||||
capacity: node.capacity,
|
||||
tags: node.tags.clone(),
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
None
|
||||
}
|
||||
|
||||
pub fn find_matching_node<'a>(
|
||||
status: &GetClusterStatusResponse,
|
||||
layout: &GetClusterLayoutResponse,
|
||||
pattern: &'a str,
|
||||
) -> Result<String, Error> {
|
||||
let all_node_ids_iter = status
|
||||
.nodes
|
||||
.iter()
|
||||
.map(|x| x.id.as_str())
|
||||
.chain(layout.roles.iter().map(|x| x.id.as_str()));
|
||||
|
||||
let mut candidates = vec![];
|
||||
for c in all_node_ids_iter {
|
||||
if c.starts_with(pattern) && !candidates.contains(&c) {
|
||||
candidates.push(c);
|
||||
}
|
||||
}
|
||||
if candidates.len() != 1 {
|
||||
Err(Error::Message(format!(
|
||||
"{} nodes match '{}'",
|
||||
candidates.len(),
|
||||
pattern,
|
||||
)))
|
||||
} else {
|
||||
Ok(candidates[0].to_string())
|
||||
}
|
||||
}
|
||||
|
||||
pub fn print_cluster_layout(layout: &GetClusterLayoutResponse, empty_msg: &str) {
|
||||
let mut table = vec!["ID\tTags\tZone\tCapacity\tUsable capacity".to_string()];
|
||||
for role in layout.roles.iter() {
|
||||
let tags = role.tags.join(",");
|
||||
if let (Some(capacity), Some(usable_capacity)) = (role.capacity, role.usable_capacity) {
|
||||
table.push(format!(
|
||||
"{:.16}\t[{}]\t{}\t{}\t{} ({:.1}%)",
|
||||
role.id,
|
||||
tags,
|
||||
role.zone,
|
||||
capacity_string(role.capacity),
|
||||
ByteSize::b(usable_capacity).to_string_as(false),
|
||||
(100.0 * usable_capacity as f32) / (capacity as f32)
|
||||
));
|
||||
} else {
|
||||
table.push(format!(
|
||||
"{:.16}\t[{}]\t{}\t{}",
|
||||
role.id,
|
||||
tags,
|
||||
role.zone,
|
||||
capacity_string(role.capacity),
|
||||
));
|
||||
};
|
||||
}
|
||||
if table.len() > 1 {
|
||||
format_table(table);
|
||||
println!();
|
||||
println!(
|
||||
"Zone redundancy: {}",
|
||||
display_zone_redundancy(layout.parameters.zone_redundancy),
|
||||
);
|
||||
} else {
|
||||
println!("{}", empty_msg);
|
||||
}
|
||||
}
|
||||
|
||||
pub fn print_staging_role_changes(layout: &GetClusterLayoutResponse) -> bool {
|
||||
let has_role_changes = !layout.staged_role_changes.is_empty();
|
||||
|
||||
let has_layout_changes = layout.staged_parameters.is_some();
|
||||
|
||||
if has_role_changes || has_layout_changes {
|
||||
println!();
|
||||
println!("==== STAGED ROLE CHANGES ====");
|
||||
if has_role_changes {
|
||||
let mut table = vec!["ID\tTags\tZone\tCapacity".to_string()];
|
||||
for change in layout.staged_role_changes.iter() {
|
||||
match &change.action {
|
||||
NodeRoleChangeEnum::Update(NodeAssignedRole {
|
||||
tags,
|
||||
zone,
|
||||
capacity,
|
||||
}) => {
|
||||
let tags = tags.join(",");
|
||||
table.push(format!(
|
||||
"{:.16}\t[{}]\t{}\t{}",
|
||||
change.id,
|
||||
tags,
|
||||
zone,
|
||||
capacity_string(*capacity),
|
||||
));
|
||||
}
|
||||
NodeRoleChangeEnum::Remove { .. } => {
|
||||
table.push(format!("{:.16}\tREMOVED", change.id));
|
||||
}
|
||||
}
|
||||
}
|
||||
format_table(table);
|
||||
println!();
|
||||
}
|
||||
if let Some(p) = layout.staged_parameters.as_ref() {
|
||||
println!(
|
||||
"Zone redundancy: {}",
|
||||
display_zone_redundancy(p.zone_redundancy)
|
||||
);
|
||||
}
|
||||
true
|
||||
} else {
|
||||
false
|
||||
}
|
||||
}
|
||||
|
||||
pub fn display_zone_redundancy(z: ZoneRedundancy) -> String {
|
||||
match z {
|
||||
ZoneRedundancy::Maximum => "maximum".into(),
|
||||
ZoneRedundancy::AtLeast(x) => x.to_string(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn parse_zone_redundancy(s: &str) -> Result<ZoneRedundancy, Error> {
|
||||
match s {
|
||||
"none" | "max" | "maximum" => Ok(ZoneRedundancy::Maximum),
|
||||
x => {
|
||||
let v = x.parse::<usize>().map_err(|_| {
|
||||
Error::Message("zone redundancy must be 'none'/'max' or an integer".into())
|
||||
})?;
|
||||
Ok(ZoneRedundancy::AtLeast(v))
|
||||
}
|
||||
}
|
||||
}
|
164
src/garage/cli/remote/mod.rs
Normal file
164
src/garage/cli/remote/mod.rs
Normal file
|
@ -0,0 +1,164 @@
|
|||
pub mod admin_token;
|
||||
pub mod bucket;
|
||||
pub mod cluster;
|
||||
pub mod key;
|
||||
pub mod layout;
|
||||
|
||||
pub mod block;
|
||||
pub mod node;
|
||||
pub mod worker;
|
||||
|
||||
use std::convert::TryFrom;
|
||||
use std::sync::Arc;
|
||||
use std::time::Duration;
|
||||
|
||||
use garage_util::error::*;
|
||||
|
||||
use garage_rpc::*;
|
||||
|
||||
use garage_api_admin::api::*;
|
||||
use garage_api_admin::api_server::{AdminRpc as ProxyRpc, AdminRpcResponse as ProxyRpcResponse};
|
||||
use garage_api_admin::RequestHandler;
|
||||
|
||||
use crate::cli::structs::*;
|
||||
|
||||
pub struct Cli {
|
||||
pub proxy_rpc_endpoint: Arc<Endpoint<ProxyRpc, ()>>,
|
||||
pub rpc_host: NodeID,
|
||||
}
|
||||
|
||||
impl Cli {
|
||||
pub async fn handle(&self, cmd: Command) -> Result<(), Error> {
|
||||
match cmd {
|
||||
Command::Status => self.cmd_status().await,
|
||||
Command::Node(NodeOperation::Connect(connect_opt)) => {
|
||||
self.cmd_connect(connect_opt).await
|
||||
}
|
||||
Command::Layout(layout_opt) => self.layout_command_dispatch(layout_opt).await,
|
||||
Command::Bucket(bo) => self.cmd_bucket(bo).await,
|
||||
Command::AdminToken(to) => self.cmd_admin_token(to).await,
|
||||
Command::Key(ko) => self.cmd_key(ko).await,
|
||||
Command::Worker(wo) => self.cmd_worker(wo).await,
|
||||
Command::Block(bo) => self.cmd_block(bo).await,
|
||||
Command::Meta(mo) => self.cmd_meta(mo).await,
|
||||
Command::Stats(so) => self.cmd_stats(so).await,
|
||||
Command::Repair(ro) => self.cmd_repair(ro).await,
|
||||
Command::JsonApi { endpoint, payload } => self.cmd_json_api(endpoint, payload).await,
|
||||
|
||||
_ => unreachable!(),
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn api_request<T>(&self, req: T) -> Result<<T as RequestHandler>::Response, Error>
|
||||
where
|
||||
T: RequestHandler,
|
||||
AdminApiRequest: From<T>,
|
||||
<T as RequestHandler>::Response: TryFrom<TaggedAdminApiResponse>,
|
||||
{
|
||||
let req = AdminApiRequest::from(req);
|
||||
let req_name = req.name();
|
||||
match self
|
||||
.proxy_rpc_endpoint
|
||||
.call(&self.rpc_host, ProxyRpc::Proxy(req), PRIO_NORMAL)
|
||||
.await??
|
||||
{
|
||||
ProxyRpcResponse::ProxyApiOkResponse(resp) => {
|
||||
<T as RequestHandler>::Response::try_from(resp).map_err(|_| {
|
||||
Error::Message(format!("{} returned unexpected response", req_name))
|
||||
})
|
||||
}
|
||||
ProxyRpcResponse::ApiErrorResponse {
|
||||
http_code,
|
||||
error_code,
|
||||
message,
|
||||
} => Err(Error::Message(format!(
|
||||
"{} returned {} ({}): {}",
|
||||
req_name, error_code, http_code, message
|
||||
))),
|
||||
m => Err(Error::unexpected_rpc_message(m)),
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn local_api_request<T>(
|
||||
&self,
|
||||
req: T,
|
||||
) -> Result<<T as RequestHandler>::Response, Error>
|
||||
where
|
||||
T: RequestHandler,
|
||||
MultiRequest<T>: RequestHandler<Response = MultiResponse<<T as RequestHandler>::Response>>,
|
||||
AdminApiRequest: From<MultiRequest<T>>,
|
||||
<MultiRequest<T> as RequestHandler>::Response: TryFrom<TaggedAdminApiResponse>,
|
||||
{
|
||||
let req = MultiRequest {
|
||||
node: hex::encode(self.rpc_host),
|
||||
body: req,
|
||||
};
|
||||
let resp = self.api_request(req).await?;
|
||||
|
||||
if let Some((_, e)) = resp.error.into_iter().next() {
|
||||
return Err(Error::Message(e));
|
||||
}
|
||||
if resp.success.len() != 1 {
|
||||
return Err(Error::Message(format!(
|
||||
"{} responses returned, expected 1",
|
||||
resp.success.len()
|
||||
)));
|
||||
}
|
||||
Ok(resp.success.into_iter().next().unwrap().1)
|
||||
}
|
||||
|
||||
pub async fn cmd_json_api(&self, endpoint: String, payload: String) -> Result<(), Error> {
|
||||
let payload: serde_json::Value = if payload == "-" {
|
||||
serde_json::from_reader(&std::io::stdin())?
|
||||
} else {
|
||||
serde_json::from_str(&payload)?
|
||||
};
|
||||
|
||||
let request: AdminApiRequest = serde_json::from_value(serde_json::json!({
|
||||
endpoint.clone(): payload,
|
||||
}))?;
|
||||
|
||||
let resp = match self
|
||||
.proxy_rpc_endpoint
|
||||
.call(&self.rpc_host, ProxyRpc::Proxy(request), PRIO_NORMAL)
|
||||
.await??
|
||||
{
|
||||
ProxyRpcResponse::ProxyApiOkResponse(resp) => resp,
|
||||
ProxyRpcResponse::ApiErrorResponse {
|
||||
http_code,
|
||||
error_code,
|
||||
message,
|
||||
} => {
|
||||
return Err(Error::Message(format!(
|
||||
"{} ({}): {}",
|
||||
error_code, http_code, message
|
||||
)))
|
||||
}
|
||||
m => return Err(Error::unexpected_rpc_message(m)),
|
||||
};
|
||||
|
||||
if let serde_json::Value::Object(map) = serde_json::to_value(&resp)? {
|
||||
if let Some(inner) = map.get(&endpoint) {
|
||||
serde_json::to_writer_pretty(std::io::stdout(), &inner)?;
|
||||
return Ok(());
|
||||
}
|
||||
}
|
||||
|
||||
Err(Error::Message(format!(
|
||||
"Invalid response: {}",
|
||||
serde_json::to_string(&resp)?
|
||||
)))
|
||||
}
|
||||
}
|
||||
|
||||
pub fn table_list_abbr<T: IntoIterator<Item = S>, S: AsRef<str>>(values: T) -> String {
|
||||
let mut iter = values.into_iter();
|
||||
|
||||
match iter.next() {
|
||||
Some(first) => match iter.count() {
|
||||
0 => first.as_ref().to_string(),
|
||||
n => format!("{}, ... ({})", first.as_ref(), n + 1),
|
||||
},
|
||||
None => String::new(),
|
||||
}
|
||||
}
|
118
src/garage/cli/remote/node.rs
Normal file
118
src/garage/cli/remote/node.rs
Normal file
|
@ -0,0 +1,118 @@
|
|||
use format_table::format_table;
|
||||
|
||||
use garage_util::error::*;
|
||||
|
||||
use garage_api_admin::api::*;
|
||||
|
||||
use crate::cli::remote::*;
|
||||
use crate::cli::structs::*;
|
||||
|
||||
impl Cli {
|
||||
pub async fn cmd_meta(&self, cmd: MetaOperation) -> Result<(), Error> {
|
||||
let MetaOperation::Snapshot { all } = cmd;
|
||||
|
||||
let res = self
|
||||
.api_request(CreateMetadataSnapshotRequest {
|
||||
node: if all {
|
||||
"*".to_string()
|
||||
} else {
|
||||
hex::encode(self.rpc_host)
|
||||
},
|
||||
body: LocalCreateMetadataSnapshotRequest,
|
||||
})
|
||||
.await?;
|
||||
|
||||
let mut table = vec!["Node\tResult".to_string()];
|
||||
for (node, _) in res.success.iter() {
|
||||
table.push(format!("{:.16}\tSnapshot created", node));
|
||||
}
|
||||
for (node, err) in res.error.iter() {
|
||||
table.push(format!("{:.16}\tError: {}", node, err));
|
||||
}
|
||||
format_table(table);
|
||||
|
||||
if !res.error.is_empty() {
|
||||
return Err(Error::Message(format!(
|
||||
"{} nodes returned an error",
|
||||
res.error.len()
|
||||
)));
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_stats(&self, cmd: StatsOpt) -> Result<(), Error> {
|
||||
let res = self
|
||||
.api_request(GetNodeStatisticsRequest {
|
||||
node: if cmd.all_nodes {
|
||||
"*".to_string()
|
||||
} else {
|
||||
hex::encode(self.rpc_host)
|
||||
},
|
||||
body: LocalGetNodeStatisticsRequest,
|
||||
})
|
||||
.await?;
|
||||
|
||||
for (node, res) in res.success.iter() {
|
||||
println!("==== NODE [{:.16}] ====", node);
|
||||
println!("{}\n", res.freeform);
|
||||
}
|
||||
|
||||
for (node, err) in res.error.iter() {
|
||||
println!("==== NODE [{:.16}] ====", node);
|
||||
println!("Error: {}\n", err);
|
||||
}
|
||||
|
||||
let res = self.api_request(GetClusterStatisticsRequest).await?;
|
||||
println!("==== CLUSTER STATISTICS ====");
|
||||
println!("{}\n", res.freeform);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_repair(&self, cmd: RepairOpt) -> Result<(), Error> {
|
||||
if !cmd.yes {
|
||||
return Err(Error::Message(
|
||||
"Please add --yes to start the repair operation".into(),
|
||||
));
|
||||
}
|
||||
|
||||
let repair_type = match cmd.what {
|
||||
RepairWhat::Tables => RepairType::Tables,
|
||||
RepairWhat::Blocks => RepairType::Blocks,
|
||||
RepairWhat::Versions => RepairType::Versions,
|
||||
RepairWhat::MultipartUploads => RepairType::MultipartUploads,
|
||||
RepairWhat::BlockRefs => RepairType::BlockRefs,
|
||||
RepairWhat::BlockRc => RepairType::BlockRc,
|
||||
RepairWhat::Rebalance => RepairType::Rebalance,
|
||||
RepairWhat::Scrub { cmd } => RepairType::Scrub(match cmd {
|
||||
ScrubCmd::Start => ScrubCommand::Start,
|
||||
ScrubCmd::Cancel => ScrubCommand::Cancel,
|
||||
ScrubCmd::Pause => ScrubCommand::Pause,
|
||||
ScrubCmd::Resume => ScrubCommand::Resume,
|
||||
}),
|
||||
};
|
||||
|
||||
let res = self
|
||||
.api_request(LaunchRepairOperationRequest {
|
||||
node: if cmd.all_nodes {
|
||||
"*".to_string()
|
||||
} else {
|
||||
hex::encode(self.rpc_host)
|
||||
},
|
||||
body: LocalLaunchRepairOperationRequest { repair_type },
|
||||
})
|
||||
.await?;
|
||||
|
||||
let mut table = vec![];
|
||||
for (node, err) in res.error.iter() {
|
||||
table.push(format!("{:.16}\tError: {}", node, err));
|
||||
}
|
||||
for (node, _) in res.success.iter() {
|
||||
table.push(format!("{:.16}\tRepair launched", node));
|
||||
}
|
||||
format_table(table);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
213
src/garage/cli/remote/worker.rs
Normal file
213
src/garage/cli/remote/worker.rs
Normal file
|
@ -0,0 +1,213 @@
|
|||
use format_table::format_table;
|
||||
|
||||
use garage_util::error::*;
|
||||
|
||||
use garage_api_admin::api::*;
|
||||
|
||||
use crate::cli::remote::*;
|
||||
use crate::cli::structs::*;
|
||||
|
||||
impl Cli {
|
||||
pub async fn cmd_worker(&self, cmd: WorkerOperation) -> Result<(), Error> {
|
||||
match cmd {
|
||||
WorkerOperation::List { opt } => self.cmd_list_workers(opt).await,
|
||||
WorkerOperation::Info { tid } => self.cmd_worker_info(tid).await,
|
||||
WorkerOperation::Get {
|
||||
all_nodes,
|
||||
variable,
|
||||
} => self.cmd_get_var(all_nodes, variable).await,
|
||||
WorkerOperation::Set {
|
||||
all_nodes,
|
||||
variable,
|
||||
value,
|
||||
} => self.cmd_set_var(all_nodes, variable, value).await,
|
||||
}
|
||||
}
|
||||
|
||||
pub async fn cmd_list_workers(&self, opt: WorkerListOpt) -> Result<(), Error> {
|
||||
let mut list = self
|
||||
.local_api_request(LocalListWorkersRequest {
|
||||
busy_only: opt.busy,
|
||||
error_only: opt.errors,
|
||||
})
|
||||
.await?
|
||||
.0;
|
||||
|
||||
list.sort_by_key(|info| {
|
||||
(
|
||||
match info.state {
|
||||
WorkerStateResp::Busy | WorkerStateResp::Throttled { .. } => 0,
|
||||
WorkerStateResp::Idle => 1,
|
||||
WorkerStateResp::Done => 2,
|
||||
},
|
||||
info.id,
|
||||
)
|
||||
});
|
||||
|
||||
let mut table =
|
||||
vec!["TID\tState\tName\tTranq\tDone\tQueue\tErrors\tConsec\tLast".to_string()];
|
||||
let tf = timeago::Formatter::new();
|
||||
for info in list.iter() {
|
||||
let err_ago = info
|
||||
.last_error
|
||||
.as_ref()
|
||||
.map(|x| tf.convert(Duration::from_secs(x.secs_ago)))
|
||||
.unwrap_or_default();
|
||||
let (total_err, consec_err) = if info.errors > 0 {
|
||||
(info.errors.to_string(), info.consecutive_errors.to_string())
|
||||
} else {
|
||||
("-".into(), "-".into())
|
||||
};
|
||||
|
||||
table.push(format!(
|
||||
"{}\t{}\t{}\t{}\t{}\t{}\t{}\t{}\t{}",
|
||||
info.id,
|
||||
format_worker_state(&info.state),
|
||||
info.name,
|
||||
info.tranquility
|
||||
.as_ref()
|
||||
.map(ToString::to_string)
|
||||
.unwrap_or_else(|| "-".into()),
|
||||
info.progress.as_deref().unwrap_or("-"),
|
||||
info.queue_length
|
||||
.as_ref()
|
||||
.map(ToString::to_string)
|
||||
.unwrap_or_else(|| "-".into()),
|
||||
total_err,
|
||||
consec_err,
|
||||
err_ago,
|
||||
));
|
||||
}
|
||||
format_table(table);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_worker_info(&self, tid: usize) -> Result<(), Error> {
|
||||
let info = self
|
||||
.local_api_request(LocalGetWorkerInfoRequest { id: tid as u64 })
|
||||
.await?
|
||||
.0;
|
||||
|
||||
let mut table = vec![];
|
||||
table.push(format!("Task id:\t{}", info.id));
|
||||
table.push(format!("Worker name:\t{}", info.name));
|
||||
match &info.state {
|
||||
WorkerStateResp::Throttled { duration_secs } => {
|
||||
table.push(format!(
|
||||
"Worker state:\tBusy (throttled, paused for {:.3}s)",
|
||||
duration_secs
|
||||
));
|
||||
}
|
||||
s => {
|
||||
table.push(format!("Worker state:\t{}", format_worker_state(s)));
|
||||
}
|
||||
};
|
||||
if let Some(tql) = info.tranquility {
|
||||
table.push(format!("Tranquility:\t{}", tql));
|
||||
}
|
||||
|
||||
table.push("".into());
|
||||
table.push(format!("Total errors:\t{}", info.errors));
|
||||
table.push(format!("Consecutive errs:\t{}", info.consecutive_errors));
|
||||
if let Some(err) = info.last_error {
|
||||
table.push(format!("Last error:\t{}", err.message));
|
||||
let tf = timeago::Formatter::new();
|
||||
table.push(format!(
|
||||
"Last error time:\t{}",
|
||||
tf.convert(Duration::from_secs(err.secs_ago))
|
||||
));
|
||||
}
|
||||
|
||||
table.push("".into());
|
||||
if let Some(p) = info.progress {
|
||||
table.push(format!("Progress:\t{}", p));
|
||||
}
|
||||
if let Some(ql) = info.queue_length {
|
||||
table.push(format!("Queue length:\t{}", ql));
|
||||
}
|
||||
if let Some(pe) = info.persistent_errors {
|
||||
table.push(format!("Persistent errors:\t{}", pe));
|
||||
}
|
||||
|
||||
for (i, s) in info.freeform.iter().enumerate() {
|
||||
if i == 0 {
|
||||
if table.last() != Some(&"".into()) {
|
||||
table.push("".into());
|
||||
}
|
||||
table.push(format!("Message:\t{}", s));
|
||||
} else {
|
||||
table.push(format!("\t{}", s));
|
||||
}
|
||||
}
|
||||
format_table(table);
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_get_var(&self, all: bool, var: Option<String>) -> Result<(), Error> {
|
||||
let res = self
|
||||
.api_request(GetWorkerVariableRequest {
|
||||
node: if all {
|
||||
"*".to_string()
|
||||
} else {
|
||||
hex::encode(self.rpc_host)
|
||||
},
|
||||
body: LocalGetWorkerVariableRequest { variable: var },
|
||||
})
|
||||
.await?;
|
||||
|
||||
let mut table = vec![];
|
||||
for (node, vars) in res.success.iter() {
|
||||
for (key, val) in vars.0.iter() {
|
||||
table.push(format!("{:.16}\t{}\t{}", node, key, val));
|
||||
}
|
||||
}
|
||||
format_table(table);
|
||||
|
||||
for (node, err) in res.error.iter() {
|
||||
eprintln!("{:.16}: error: {}", node, err);
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
|
||||
pub async fn cmd_set_var(
|
||||
&self,
|
||||
all: bool,
|
||||
variable: String,
|
||||
value: String,
|
||||
) -> Result<(), Error> {
|
||||
let res = self
|
||||
.api_request(SetWorkerVariableRequest {
|
||||
node: if all {
|
||||
"*".to_string()
|
||||
} else {
|
||||
hex::encode(self.rpc_host)
|
||||
},
|
||||
body: LocalSetWorkerVariableRequest { variable, value },
|
||||
})
|
||||
.await?;
|
||||
|
||||
let mut table = vec![];
|
||||
for (node, kv) in res.success.iter() {
|
||||
table.push(format!("{:.16}\t{}\t{}", node, kv.variable, kv.value));
|
||||
}
|
||||
format_table(table);
|
||||
|
||||
for (node, err) in res.error.iter() {
|
||||
eprintln!("{:.16}: error: {}", node, err);
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
fn format_worker_state(s: &WorkerStateResp) -> &'static str {
|
||||
match s {
|
||||
WorkerStateResp::Busy => "Busy",
|
||||
WorkerStateResp::Throttled { .. } => "Busy*",
|
||||
WorkerStateResp::Idle => "Idle",
|
||||
WorkerStateResp::Done => "Done",
|
||||
}
|
||||
}
|
|
@ -1,9 +1,8 @@
|
|||
use serde::{Deserialize, Serialize};
|
||||
use structopt::StructOpt;
|
||||
|
||||
use garage_util::version::garage_version;
|
||||
|
||||
use crate::cli::convert_db;
|
||||
use crate::cli::local::convert_db;
|
||||
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub enum Command {
|
||||
|
@ -31,6 +30,10 @@ pub enum Command {
|
|||
#[structopt(name = "key", version = garage_version())]
|
||||
Key(KeyOperation),
|
||||
|
||||
/// Operations on admin API tokens
|
||||
#[structopt(name = "admin-token", version = garage_version())]
|
||||
AdminToken(AdminTokenOperation),
|
||||
|
||||
/// Start repair of node data on remote node
|
||||
#[structopt(name = "repair", version = garage_version())]
|
||||
Repair(RepairOpt),
|
||||
|
@ -59,8 +62,27 @@ pub enum Command {
|
|||
/// Convert metadata db between database engine formats
|
||||
#[structopt(name = "convert-db", version = garage_version())]
|
||||
ConvertDb(convert_db::ConvertDbOpt),
|
||||
|
||||
/// Output openapi JSON schema for admin api
|
||||
#[structopt(name = "admin-api-schema", version = garage_version(), setting(structopt::clap::AppSettings::Hidden))]
|
||||
AdminApiSchema,
|
||||
|
||||
/// Directly invoke the admin API using a JSON payload.
|
||||
/// The result is printed to `stdout` in JSON format.
|
||||
#[structopt(name = "json-api", version = garage_version())]
|
||||
JsonApi {
|
||||
/// The admin API endpoint to invoke, e.g. GetClusterStatus
|
||||
endpoint: String,
|
||||
/// The JSON payload, or `-` to read from `stdin`
|
||||
#[structopt(default_value = "null")]
|
||||
payload: String,
|
||||
},
|
||||
}
|
||||
|
||||
// -------------------------
|
||||
// ---- garage node ... ----
|
||||
// -------------------------
|
||||
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub enum NodeOperation {
|
||||
/// Print the full node ID (public key) of this Garage node, and its publicly reachable IP
|
||||
|
@ -88,6 +110,10 @@ pub struct ConnectNodeOpt {
|
|||
pub(crate) node: String,
|
||||
}
|
||||
|
||||
// ---------------------------
|
||||
// ---- garage layout ... ----
|
||||
// ---------------------------
|
||||
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub enum LayoutOperation {
|
||||
/// Assign role to Garage node
|
||||
|
@ -190,7 +216,11 @@ pub struct SkipDeadNodesOpt {
|
|||
pub(crate) allow_missing_data: bool,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug)]
|
||||
// ---------------------------
|
||||
// ---- garage bucket ... ----
|
||||
// ---------------------------
|
||||
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub enum BucketOperation {
|
||||
/// List buckets
|
||||
#[structopt(name = "list", version = garage_version())]
|
||||
|
@ -235,9 +265,13 @@ pub enum BucketOperation {
|
|||
/// Clean up (abort) old incomplete multipart uploads
|
||||
#[structopt(name = "cleanup-incomplete-uploads", version = garage_version())]
|
||||
CleanupIncompleteUploads(CleanupIncompleteUploadsOpt),
|
||||
|
||||
/// Inspect an object in a bucket
|
||||
#[structopt(name = "inspect-object", version = garage_version())]
|
||||
InspectObject(InspectObjectOpt),
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug)]
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub struct WebsiteOpt {
|
||||
/// Create
|
||||
#[structopt(long = "allow")]
|
||||
|
@ -259,13 +293,13 @@ pub struct WebsiteOpt {
|
|||
pub error_document: Option<String>,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug)]
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub struct BucketOpt {
|
||||
/// Bucket name
|
||||
pub name: String,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug)]
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub struct DeleteBucketOpt {
|
||||
/// Bucket name
|
||||
pub name: String,
|
||||
|
@ -275,7 +309,7 @@ pub struct DeleteBucketOpt {
|
|||
pub yes: bool,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug)]
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub struct AliasBucketOpt {
|
||||
/// Existing bucket name (its alias in global namespace or its full hex uuid)
|
||||
pub existing_bucket: String,
|
||||
|
@ -288,7 +322,7 @@ pub struct AliasBucketOpt {
|
|||
pub local: Option<String>,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug)]
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub struct UnaliasBucketOpt {
|
||||
/// Bucket name
|
||||
pub name: String,
|
||||
|
@ -298,7 +332,7 @@ pub struct UnaliasBucketOpt {
|
|||
pub local: Option<String>,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug)]
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub struct PermBucketOpt {
|
||||
/// Access key name or ID
|
||||
#[structopt(long = "key")]
|
||||
|
@ -321,7 +355,7 @@ pub struct PermBucketOpt {
|
|||
pub bucket: String,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug)]
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub struct SetQuotasOpt {
|
||||
/// Bucket name
|
||||
pub bucket: String,
|
||||
|
@ -336,7 +370,7 @@ pub struct SetQuotasOpt {
|
|||
pub max_objects: Option<String>,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug)]
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub struct CleanupIncompleteUploadsOpt {
|
||||
/// Abort multipart uploads older than this value
|
||||
#[structopt(long = "older-than", default_value = "1d")]
|
||||
|
@ -347,7 +381,19 @@ pub struct CleanupIncompleteUploadsOpt {
|
|||
pub buckets: Vec<String>,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug)]
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub struct InspectObjectOpt {
|
||||
/// Name or ID of bucket
|
||||
pub bucket: String,
|
||||
/// Key of object to inspect
|
||||
pub key: String,
|
||||
}
|
||||
|
||||
// ------------------------
|
||||
// ---- garage key ... ----
|
||||
// ------------------------
|
||||
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub enum KeyOperation {
|
||||
/// List keys
|
||||
#[structopt(name = "list", version = garage_version())]
|
||||
|
@ -382,7 +428,7 @@ pub enum KeyOperation {
|
|||
Import(KeyImportOpt),
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug)]
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub struct KeyInfoOpt {
|
||||
/// ID or name of the key
|
||||
pub key_pattern: String,
|
||||
|
@ -391,14 +437,14 @@ pub struct KeyInfoOpt {
|
|||
pub show_secret: bool,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug)]
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub struct KeyNewOpt {
|
||||
/// Name of the key
|
||||
#[structopt(default_value = "Unnamed key")]
|
||||
pub name: String,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug)]
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub struct KeyRenameOpt {
|
||||
/// ID or name of the key
|
||||
pub key_pattern: String,
|
||||
|
@ -407,7 +453,7 @@ pub struct KeyRenameOpt {
|
|||
pub new_name: String,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug)]
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub struct KeyDeleteOpt {
|
||||
/// ID or name of the key
|
||||
pub key_pattern: String,
|
||||
|
@ -417,7 +463,7 @@ pub struct KeyDeleteOpt {
|
|||
pub yes: bool,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug)]
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub struct KeyPermOpt {
|
||||
/// ID or name of the key
|
||||
pub key_pattern: String,
|
||||
|
@ -427,7 +473,7 @@ pub struct KeyPermOpt {
|
|||
pub create_bucket: bool,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug)]
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub struct KeyImportOpt {
|
||||
/// Access key ID
|
||||
pub key_id: String,
|
||||
|
@ -444,7 +490,105 @@ pub struct KeyImportOpt {
|
|||
pub yes: bool,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug, Clone)]
|
||||
// --------------------------------
|
||||
// ---- garage admin-token ... ----
|
||||
// --------------------------------
|
||||
|
||||
#[derive(StructOpt, Debug)]
|
||||
pub enum AdminTokenOperation {
|
||||
/// List all admin API tokens
|
||||
#[structopt(name = "list", version = garage_version())]
|
||||
List,
|
||||
|
||||
/// Fetch info about a specific admin API token
|
||||
#[structopt(name = "info", version = garage_version())]
|
||||
Info {
|
||||
/// Name or prefix of the ID of the token to look up
|
||||
api_token: String,
|
||||
},
|
||||
|
||||
/// Create new admin API token
|
||||
#[structopt(name = "create", version = garage_version())]
|
||||
Create(AdminTokenCreateOp),
|
||||
|
||||
/// Rename an admin API token
|
||||
#[structopt(name = "rename", version = garage_version())]
|
||||
Rename {
|
||||
/// Name or prefix of the ID of the token to rename
|
||||
api_token: String,
|
||||
/// New name of the admintoken
|
||||
new_name: String,
|
||||
},
|
||||
|
||||
/// Set parameters for an admin API token
|
||||
#[structopt(name = "set", version = garage_version())]
|
||||
Set(AdminTokenSetOp),
|
||||
|
||||
/// Delete an admin API token
|
||||
#[structopt(name = "delete", version = garage_version())]
|
||||
Delete {
|
||||
/// Name or prefix of the ID of the token to delete
|
||||
api_token: String,
|
||||
/// Confirm deletion
|
||||
#[structopt(long = "yes")]
|
||||
yes: bool,
|
||||
},
|
||||
|
||||
/// Delete all expired admin API tokens
|
||||
#[structopt(name = "delete-expired", version = garage_version())]
|
||||
DeleteExpired {
|
||||
/// Confirm deletion
|
||||
#[structopt(long = "yes")]
|
||||
yes: bool,
|
||||
},
|
||||
}
|
||||
|
||||
#[derive(StructOpt, Debug, Clone)]
|
||||
pub struct AdminTokenCreateOp {
|
||||
/// Set a name for the token
|
||||
pub name: Option<String>,
|
||||
/// Set an expiration time for the token (see docs.rs/parse_duration for date
|
||||
/// format)
|
||||
#[structopt(long = "expires-in")]
|
||||
pub expires_in: Option<String>,
|
||||
/// Set a limited scope for the token, as a comma-separated list of
|
||||
/// admin API functions (e.g. GetClusterStatus, etc.). The default scope
|
||||
/// is `*`, which allows access to all admin API functions.
|
||||
/// Note that granting a scope that allows `CreateAdminToken` or
|
||||
/// `UpdateAdminToken` allows for privilege escalation, and is therefore
|
||||
/// equivalent to `*`.
|
||||
#[structopt(long = "scope")]
|
||||
pub scope: Option<String>,
|
||||
/// Print only the newly generated API token to stdout
|
||||
#[structopt(short = "q", long = "quiet")]
|
||||
pub quiet: bool,
|
||||
}
|
||||
|
||||
#[derive(StructOpt, Debug, Clone)]
|
||||
pub struct AdminTokenSetOp {
|
||||
/// Name or prefix of the ID of the token to modify
|
||||
pub api_token: String,
|
||||
/// Set an expiration time for the token (see docs.rs/parse_duration for date
|
||||
/// format)
|
||||
#[structopt(long = "expires-in")]
|
||||
pub expires_in: Option<String>,
|
||||
/// Set a limited scope for the token, as a comma-separated list of
|
||||
/// admin API functions (e.g. GetClusterStatus, etc.), or `*` to allow
|
||||
/// all admin API functions.
|
||||
/// Use `--scope=+Scope1,Scope2` to add scopes to the existing list,
|
||||
/// and `--scope=-Scope1,Scope2` to remove scopes from the existing list.
|
||||
/// Note that granting a scope that allows `CreateAdminToken` or
|
||||
/// `UpdateAdminToken` allows for privilege escalation, and is therefore
|
||||
/// equivalent to `*`.
|
||||
#[structopt(long = "scope")]
|
||||
pub scope: Option<String>,
|
||||
}
|
||||
|
||||
// ---------------------------
|
||||
// ---- garage repair ... ----
|
||||
// ---------------------------
|
||||
|
||||
#[derive(StructOpt, Debug, Clone)]
|
||||
pub struct RepairOpt {
|
||||
/// Launch repair operation on all nodes
|
||||
#[structopt(short = "a", long = "all-nodes")]
|
||||
|
@ -458,7 +602,7 @@ pub struct RepairOpt {
|
|||
pub what: RepairWhat,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug, Eq, PartialEq, Clone)]
|
||||
#[derive(StructOpt, Debug, Eq, PartialEq, Clone)]
|
||||
pub enum RepairWhat {
|
||||
/// Do a full sync of metadata tables
|
||||
#[structopt(name = "tables", version = garage_version())]
|
||||
|
@ -489,7 +633,7 @@ pub enum RepairWhat {
|
|||
Rebalance,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug, Eq, PartialEq, Clone)]
|
||||
#[derive(StructOpt, Debug, Eq, PartialEq, Clone)]
|
||||
pub enum ScrubCmd {
|
||||
/// Start scrub
|
||||
#[structopt(name = "start", version = garage_version())]
|
||||
|
@ -503,15 +647,13 @@ pub enum ScrubCmd {
|
|||
/// Cancel scrub in progress
|
||||
#[structopt(name = "cancel", version = garage_version())]
|
||||
Cancel,
|
||||
/// Set tranquility level for in-progress and future scrubs
|
||||
#[structopt(name = "set-tranquility", version = garage_version())]
|
||||
SetTranquility {
|
||||
#[structopt()]
|
||||
tranquility: u32,
|
||||
},
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug, Clone)]
|
||||
// -----------------------------------
|
||||
// ---- garage offline-repair ... ----
|
||||
// -----------------------------------
|
||||
|
||||
#[derive(StructOpt, Debug, Clone)]
|
||||
pub struct OfflineRepairOpt {
|
||||
/// Confirm the launch of the repair operation
|
||||
#[structopt(long = "yes")]
|
||||
|
@ -521,7 +663,7 @@ pub struct OfflineRepairOpt {
|
|||
pub what: OfflineRepairWhat,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug, Eq, PartialEq, Clone)]
|
||||
#[derive(StructOpt, Debug, Eq, PartialEq, Clone)]
|
||||
pub enum OfflineRepairWhat {
|
||||
/// Repair K2V item counters
|
||||
#[cfg(feature = "k2v")]
|
||||
|
@ -532,19 +674,22 @@ pub enum OfflineRepairWhat {
|
|||
ObjectCounters,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug, Clone)]
|
||||
// --------------------------
|
||||
// ---- garage stats ... ----
|
||||
// --------------------------
|
||||
|
||||
#[derive(StructOpt, Debug, Clone)]
|
||||
pub struct StatsOpt {
|
||||
/// Gather statistics from all nodes
|
||||
#[structopt(short = "a", long = "all-nodes")]
|
||||
pub all_nodes: bool,
|
||||
|
||||
/// Don't show global cluster stats (internal use in RPC)
|
||||
#[structopt(skip)]
|
||||
#[serde(default)]
|
||||
pub skip_global: bool,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug, Eq, PartialEq, Clone)]
|
||||
// ---------------------------
|
||||
// ---- garage worker ... ----
|
||||
// ---------------------------
|
||||
|
||||
#[derive(StructOpt, Debug, Eq, PartialEq, Clone)]
|
||||
pub enum WorkerOperation {
|
||||
/// List all workers on Garage node
|
||||
#[structopt(name = "list", version = garage_version())]
|
||||
|
@ -577,7 +722,7 @@ pub enum WorkerOperation {
|
|||
},
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug, Eq, PartialEq, Clone, Copy)]
|
||||
#[derive(StructOpt, Debug, Eq, PartialEq, Clone, Copy)]
|
||||
pub struct WorkerListOpt {
|
||||
/// Show only busy workers
|
||||
#[structopt(short = "b", long = "busy")]
|
||||
|
@ -587,7 +732,11 @@ pub struct WorkerListOpt {
|
|||
pub errors: bool,
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug, Eq, PartialEq, Clone)]
|
||||
// --------------------------
|
||||
// ---- garage block ... ----
|
||||
// --------------------------
|
||||
|
||||
#[derive(StructOpt, Debug, Eq, PartialEq, Clone)]
|
||||
pub enum BlockOperation {
|
||||
/// List all blocks that currently have a resync error
|
||||
#[structopt(name = "list-errors", version = garage_version())]
|
||||
|
@ -619,7 +768,11 @@ pub enum BlockOperation {
|
|||
},
|
||||
}
|
||||
|
||||
#[derive(Serialize, Deserialize, StructOpt, Debug, Eq, PartialEq, Clone, Copy)]
|
||||
// -------------------------
|
||||
// ---- garage meta ... ----
|
||||
// -------------------------
|
||||
|
||||
#[derive(StructOpt, Debug, Eq, PartialEq, Clone, Copy)]
|
||||
pub enum MetaOperation {
|
||||
/// Save a snapshot of the metadata db file
|
||||
#[structopt(name = "snapshot", version = garage_version())]
|
||||
|
|
|
@ -1,457 +0,0 @@
|
|||
use std::collections::HashMap;
|
||||
use std::time::Duration;
|
||||
|
||||
use format_table::format_table;
|
||||
use garage_util::background::*;
|
||||
use garage_util::crdt::*;
|
||||
use garage_util::data::*;
|
||||
use garage_util::error::*;
|
||||
use garage_util::time::*;
|
||||
|
||||
use garage_block::manager::BlockResyncErrorInfo;
|
||||
|
||||
use garage_model::bucket_table::*;
|
||||
use garage_model::key_table::*;
|
||||
use garage_model::s3::mpu_table::{self, MultipartUpload};
|
||||
use garage_model::s3::object_table;
|
||||
use garage_model::s3::version_table::*;
|
||||
|
||||
use crate::cli::structs::WorkerListOpt;
|
||||
|
||||
pub fn print_bucket_list(bl: Vec<Bucket>) {
|
||||
println!("List of buckets:");
|
||||
|
||||
let mut table = vec![];
|
||||
for bucket in bl {
|
||||
let aliases = bucket
|
||||
.aliases()
|
||||
.iter()
|
||||
.filter(|(_, _, active)| *active)
|
||||
.map(|(name, _, _)| name.to_string())
|
||||
.collect::<Vec<_>>();
|
||||
let local_aliases_n = match &bucket
|
||||
.local_aliases()
|
||||
.iter()
|
||||
.filter(|(_, _, active)| *active)
|
||||
.collect::<Vec<_>>()[..]
|
||||
{
|
||||
[] => "".into(),
|
||||
[((k, n), _, _)] => format!("{}:{}", k, n),
|
||||
s => format!("[{} local aliases]", s.len()),
|
||||
};
|
||||
|
||||
table.push(format!(
|
||||
"\t{}\t{}\t{}",
|
||||
aliases.join(","),
|
||||
local_aliases_n,
|
||||
hex::encode(bucket.id),
|
||||
));
|
||||
}
|
||||
format_table(table);
|
||||
}
|
||||
|
||||
pub fn print_key_list(kl: Vec<(String, String)>) {
|
||||
println!("List of keys:");
|
||||
let mut table = vec![];
|
||||
for key in kl {
|
||||
table.push(format!("\t{}\t{}", key.0, key.1));
|
||||
}
|
||||
format_table(table);
|
||||
}
|
||||
|
||||
pub fn print_key_info(key: &Key, relevant_buckets: &HashMap<Uuid, Bucket>) {
|
||||
let bucket_global_aliases = |b: &Uuid| {
|
||||
if let Some(bucket) = relevant_buckets.get(b) {
|
||||
if let Some(p) = bucket.state.as_option() {
|
||||
return p
|
||||
.aliases
|
||||
.items()
|
||||
.iter()
|
||||
.filter(|(_, _, active)| *active)
|
||||
.map(|(a, _, _)| a.clone())
|
||||
.collect::<Vec<_>>()
|
||||
.join(", ");
|
||||
}
|
||||
}
|
||||
|
||||
"".to_string()
|
||||
};
|
||||
|
||||
match &key.state {
|
||||
Deletable::Present(p) => {
|
||||
println!("Key name: {}", p.name.get());
|
||||
println!("Key ID: {}", key.key_id);
|
||||
println!("Secret key: {}", p.secret_key);
|
||||
println!("Can create buckets: {}", p.allow_create_bucket.get());
|
||||
println!("\nKey-specific bucket aliases:");
|
||||
let mut table = vec![];
|
||||
for (alias_name, _, alias) in p.local_aliases.items().iter() {
|
||||
if let Some(bucket_id) = alias {
|
||||
table.push(format!(
|
||||
"\t{}\t{}\t{}",
|
||||
alias_name,
|
||||
bucket_global_aliases(bucket_id),
|
||||
hex::encode(bucket_id)
|
||||
));
|
||||
}
|
||||
}
|
||||
format_table(table);
|
||||
|
||||
println!("\nAuthorized buckets:");
|
||||
let mut table = vec![];
|
||||
for (bucket_id, perm) in p.authorized_buckets.items().iter() {
|
||||
if !perm.is_any() {
|
||||
continue;
|
||||
}
|
||||
let rflag = if perm.allow_read { "R" } else { " " };
|
||||
let wflag = if perm.allow_write { "W" } else { " " };
|
||||
let oflag = if perm.allow_owner { "O" } else { " " };
|
||||
let local_aliases = p
|
||||
.local_aliases
|
||||
.items()
|
||||
.iter()
|
||||
.filter(|(_, _, a)| *a == Some(*bucket_id))
|
||||
.map(|(a, _, _)| a.clone())
|
||||
.collect::<Vec<_>>()
|
||||
.join(", ");
|
||||
table.push(format!(
|
||||
"\t{}{}{}\t{}\t{}\t{:?}",
|
||||
rflag,
|
||||
wflag,
|
||||
oflag,
|
||||
bucket_global_aliases(bucket_id),
|
||||
local_aliases,
|
||||
bucket_id
|
||||
));
|
||||
}
|
||||
format_table(table);
|
||||
}
|
||||
Deletable::Deleted => {
|
||||
println!("Key {} is deleted.", key.key_id);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub fn print_bucket_info(
|
||||
bucket: &Bucket,
|
||||
relevant_keys: &HashMap<String, Key>,
|
||||
counters: &HashMap<String, i64>,
|
||||
mpu_counters: &HashMap<String, i64>,
|
||||
) {
|
||||
let key_name = |k| {
|
||||
relevant_keys
|
||||
.get(k)
|
||||
.map(|k| k.params().unwrap().name.get().as_str())
|
||||
.unwrap_or("<deleted>")
|
||||
};
|
||||
|
||||
println!("Bucket: {}", hex::encode(bucket.id));
|
||||
match &bucket.state {
|
||||
Deletable::Deleted => println!("Bucket is deleted."),
|
||||
Deletable::Present(p) => {
|
||||
let size =
|
||||
bytesize::ByteSize::b(*counters.get(object_table::BYTES).unwrap_or(&0) as u64);
|
||||
println!(
|
||||
"\nSize: {} ({})",
|
||||
size.to_string_as(true),
|
||||
size.to_string_as(false)
|
||||
);
|
||||
println!(
|
||||
"Objects: {}",
|
||||
*counters.get(object_table::OBJECTS).unwrap_or(&0)
|
||||
);
|
||||
println!(
|
||||
"Unfinished uploads (multipart and non-multipart): {}",
|
||||
*counters.get(object_table::UNFINISHED_UPLOADS).unwrap_or(&0)
|
||||
);
|
||||
println!(
|
||||
"Unfinished multipart uploads: {}",
|
||||
*mpu_counters.get(mpu_table::UPLOADS).unwrap_or(&0)
|
||||
);
|
||||
let mpu_size =
|
||||
bytesize::ByteSize::b(*mpu_counters.get(mpu_table::BYTES).unwrap_or(&0) as u64);
|
||||
println!(
|
||||
"Size of unfinished multipart uploads: {} ({})",
|
||||
mpu_size.to_string_as(true),
|
||||
mpu_size.to_string_as(false),
|
||||
);
|
||||
|
||||
println!("\nWebsite access: {}", p.website_config.get().is_some());
|
||||
|
||||
let quotas = p.quotas.get();
|
||||
if quotas.max_size.is_some() || quotas.max_objects.is_some() {
|
||||
println!("\nQuotas:");
|
||||
if let Some(ms) = quotas.max_size {
|
||||
let ms = bytesize::ByteSize::b(ms);
|
||||
println!(
|
||||
" maximum size: {} ({})",
|
||||
ms.to_string_as(true),
|
||||
ms.to_string_as(false)
|
||||
);
|
||||
}
|
||||
if let Some(mo) = quotas.max_objects {
|
||||
println!(" maximum number of objects: {}", mo);
|
||||
}
|
||||
}
|
||||
|
||||
println!("\nGlobal aliases:");
|
||||
for (alias, _, active) in p.aliases.items().iter() {
|
||||
if *active {
|
||||
println!(" {}", alias);
|
||||
}
|
||||
}
|
||||
|
||||
println!("\nKey-specific aliases:");
|
||||
let mut table = vec![];
|
||||
for ((key_id, alias), _, active) in p.local_aliases.items().iter() {
|
||||
if *active {
|
||||
table.push(format!("\t{} ({})\t{}", key_id, key_name(key_id), alias));
|
||||
}
|
||||
}
|
||||
format_table(table);
|
||||
|
||||
println!("\nAuthorized keys:");
|
||||
let mut table = vec![];
|
||||
for (k, perm) in p.authorized_keys.items().iter() {
|
||||
if !perm.is_any() {
|
||||
continue;
|
||||
}
|
||||
let rflag = if perm.allow_read { "R" } else { " " };
|
||||
let wflag = if perm.allow_write { "W" } else { " " };
|
||||
let oflag = if perm.allow_owner { "O" } else { " " };
|
||||
table.push(format!(
|
||||
"\t{}{}{}\t{}\t{}",
|
||||
rflag,
|
||||
wflag,
|
||||
oflag,
|
||||
k,
|
||||
key_name(k)
|
||||
));
|
||||
}
|
||||
format_table(table);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
pub fn find_matching_node(
|
||||
cand: impl std::iter::Iterator<Item = Uuid>,
|
||||
pattern: &str,
|
||||
) -> Result<Uuid, Error> {
|
||||
let mut candidates = vec![];
|
||||
for c in cand {
|
||||
if hex::encode(c).starts_with(pattern) && !candidates.contains(&c) {
|
||||
candidates.push(c);
|
||||
}
|
||||
}
|
||||
if candidates.len() != 1 {
|
||||
Err(Error::Message(format!(
|
||||
"{} nodes match '{}'",
|
||||
candidates.len(),
|
||||
pattern,
|
||||
)))
|
||||
} else {
|
||||
Ok(candidates[0])
|
||||
}
|
||||
}
|
||||
|
||||
pub fn print_worker_list(wi: HashMap<usize, WorkerInfo>, wlo: WorkerListOpt) {
|
||||
let mut wi = wi.into_iter().collect::<Vec<_>>();
|
||||
wi.sort_by_key(|(tid, info)| {
|
||||
(
|
||||
match info.state {
|
||||
WorkerState::Busy | WorkerState::Throttled(_) => 0,
|
||||
WorkerState::Idle => 1,
|
||||
WorkerState::Done => 2,
|
||||
},
|
||||
*tid,
|
||||
)
|
||||
});
|
||||
|
||||
let mut table = vec!["TID\tState\tName\tTranq\tDone\tQueue\tErrors\tConsec\tLast".to_string()];
|
||||
for (tid, info) in wi.iter() {
|
||||
if wlo.busy && !matches!(info.state, WorkerState::Busy | WorkerState::Throttled(_)) {
|
||||
continue;
|
||||
}
|
||||
if wlo.errors && info.errors == 0 {
|
||||
continue;
|
||||
}
|
||||
|
||||
let tf = timeago::Formatter::new();
|
||||
let err_ago = info
|
||||
.last_error
|
||||
.as_ref()
|
||||
.map(|(_, t)| tf.convert(Duration::from_millis(now_msec() - t)))
|
||||
.unwrap_or_default();
|
||||
let (total_err, consec_err) = if info.errors > 0 {
|
||||
(info.errors.to_string(), info.consecutive_errors.to_string())
|
||||
} else {
|
||||
("-".into(), "-".into())
|
||||
};
|
||||
|
||||
table.push(format!(
|
||||
"{}\t{}\t{}\t{}\t{}\t{}\t{}\t{}\t{}",
|
||||
tid,
|
||||
info.state,
|
||||
info.name,
|
||||
info.status
|
||||
.tranquility
|
||||
.as_ref()
|
||||
.map(ToString::to_string)
|
||||
.unwrap_or_else(|| "-".into()),
|
||||
info.status.progress.as_deref().unwrap_or("-"),
|
||||
info.status
|
||||
.queue_length
|
||||
.as_ref()
|
||||
.map(ToString::to_string)
|
||||
.unwrap_or_else(|| "-".into()),
|
||||
total_err,
|
||||
consec_err,
|
||||
err_ago,
|
||||
));
|
||||
}
|
||||
format_table(table);
|
||||
}
|
||||
|
||||
pub fn print_worker_info(tid: usize, info: WorkerInfo) {
|
||||
let mut table = vec![];
|
||||
table.push(format!("Task id:\t{}", tid));
|
||||
table.push(format!("Worker name:\t{}", info.name));
|
||||
match info.state {
|
||||
WorkerState::Throttled(t) => {
|
||||
table.push(format!(
|
||||
"Worker state:\tBusy (throttled, paused for {:.3}s)",
|
||||
t
|
||||
));
|
||||
}
|
||||
s => {
|
||||
table.push(format!("Worker state:\t{}", s));
|
||||
}
|
||||
};
|
||||
if let Some(tql) = info.status.tranquility {
|
||||
table.push(format!("Tranquility:\t{}", tql));
|
||||
}
|
||||
|
||||
table.push("".into());
|
||||
table.push(format!("Total errors:\t{}", info.errors));
|
||||
table.push(format!("Consecutive errs:\t{}", info.consecutive_errors));
|
||||
if let Some((s, t)) = info.last_error {
|
||||
table.push(format!("Last error:\t{}", s));
|
||||
let tf = timeago::Formatter::new();
|
||||
table.push(format!(
|
||||
"Last error time:\t{}",
|
||||
tf.convert(Duration::from_millis(now_msec() - t))
|
||||
));
|
||||
}
|
||||
|
||||
table.push("".into());
|
||||
if let Some(p) = info.status.progress {
|
||||
table.push(format!("Progress:\t{}", p));
|
||||
}
|
||||
if let Some(ql) = info.status.queue_length {
|
||||
table.push(format!("Queue length:\t{}", ql));
|
||||
}
|
||||
if let Some(pe) = info.status.persistent_errors {
|
||||
table.push(format!("Persistent errors:\t{}", pe));
|
||||
}
|
||||
|
||||
for (i, s) in info.status.freeform.iter().enumerate() {
|
||||
if i == 0 {
|
||||
if table.last() != Some(&"".into()) {
|
||||
table.push("".into());
|
||||
}
|
||||
table.push(format!("Message:\t{}", s));
|
||||
} else {
|
||||
table.push(format!("\t{}", s));
|
||||
}
|
||||
}
|
||||
format_table(table);
|
||||
}
|
||||
|
||||
pub fn print_worker_vars(wv: Vec<(Uuid, String, String)>) {
|
||||
let table = wv
|
||||
.into_iter()
|
||||
.map(|(n, k, v)| format!("{:?}\t{}\t{}", n, k, v))
|
||||
.collect::<Vec<_>>();
|
||||
format_table(table);
|
||||
}
|
||||
|
||||
pub fn print_block_error_list(el: Vec<BlockResyncErrorInfo>) {
|
||||
let now = now_msec();
|
||||
let tf = timeago::Formatter::new();
|
||||
let mut tf2 = timeago::Formatter::new();
|
||||
tf2.ago("");
|
||||
|
||||
let mut table = vec!["Hash\tRC\tErrors\tLast error\tNext try".into()];
|
||||
for e in el {
|
||||
let next_try = if e.next_try > now {
|
||||
tf2.convert(Duration::from_millis(e.next_try - now))
|
||||
} else {
|
||||
"asap".to_string()
|
||||
};
|
||||
table.push(format!(
|
||||
"{}\t{}\t{}\t{}\tin {}",
|
||||
hex::encode(e.hash.as_slice()),
|
||||
e.refcount,
|
||||
e.error_count,
|
||||
tf.convert(Duration::from_millis(now - e.last_try)),
|
||||
next_try
|
||||
));
|
||||
}
|
||||
format_table(table);
|
||||
}
|
||||
|
||||
pub fn print_block_info(
|
||||
hash: Hash,
|
||||
refcount: u64,
|
||||
versions: Vec<Result<Version, Uuid>>,
|
||||
uploads: Vec<MultipartUpload>,
|
||||
) {
|
||||
println!("Block hash: {}", hex::encode(hash.as_slice()));
|
||||
println!("Refcount: {}", refcount);
|
||||
println!();
|
||||
|
||||
let mut table = vec!["Version\tBucket\tKey\tMPU\tDeleted".into()];
|
||||
let mut nondeleted_count = 0;
|
||||
for v in versions.iter() {
|
||||
match v {
|
||||
Ok(ver) => {
|
||||
match &ver.backlink {
|
||||
VersionBacklink::Object { bucket_id, key } => {
|
||||
table.push(format!(
|
||||
"{:?}\t{:?}\t{}\t\t{:?}",
|
||||
ver.uuid,
|
||||
bucket_id,
|
||||
key,
|
||||
ver.deleted.get()
|
||||
));
|
||||
}
|
||||
VersionBacklink::MultipartUpload { upload_id } => {
|
||||
let upload = uploads.iter().find(|x| x.upload_id == *upload_id);
|
||||
table.push(format!(
|
||||
"{:?}\t{:?}\t{}\t{:?}\t{:?}",
|
||||
ver.uuid,
|
||||
upload.map(|u| u.bucket_id).unwrap_or_default(),
|
||||
upload.map(|u| u.key.as_str()).unwrap_or_default(),
|
||||
upload_id,
|
||||
ver.deleted.get()
|
||||
));
|
||||
}
|
||||
}
|
||||
if !ver.deleted.get() {
|
||||
nondeleted_count += 1;
|
||||
}
|
||||
}
|
||||
Err(vh) => {
|
||||
table.push(format!("{:?}\t\t\t\tyes", vh));
|
||||
}
|
||||
}
|
||||
}
|
||||
format_table(table);
|
||||
|
||||
if refcount != nondeleted_count {
|
||||
println!();
|
||||
println!(
|
||||
"Warning: refcount does not match number of non-deleted versions, you should try `garage repair block-rc`."
|
||||
);
|
||||
}
|
||||
}
|
|
@ -4,9 +4,7 @@
|
|||
#[macro_use]
|
||||
extern crate tracing;
|
||||
|
||||
mod admin;
|
||||
mod cli;
|
||||
mod repair;
|
||||
mod secrets;
|
||||
mod server;
|
||||
#[cfg(feature = "telemetry-otlp")]
|
||||
|
@ -25,6 +23,7 @@ use std::net::SocketAddr;
|
|||
use std::path::PathBuf;
|
||||
|
||||
use structopt::StructOpt;
|
||||
use utoipa::OpenApi;
|
||||
|
||||
use garage_net::util::parse_and_resolve_peer_addr;
|
||||
use garage_net::NetworkKey;
|
||||
|
@ -34,10 +33,9 @@ use garage_util::error::*;
|
|||
use garage_rpc::system::*;
|
||||
use garage_rpc::*;
|
||||
|
||||
use garage_model::helper::error::Error as HelperError;
|
||||
use garage_api_admin::api_server::{AdminRpc as ProxyRpc, ADMIN_RPC_PATH as PROXY_RPC_PATH};
|
||||
|
||||
use admin::*;
|
||||
use cli::*;
|
||||
use cli::structs::*;
|
||||
use secrets::Secrets;
|
||||
|
||||
#[derive(StructOpt, Debug)]
|
||||
|
@ -145,13 +143,22 @@ async fn main() {
|
|||
let res = match opt.cmd {
|
||||
Command::Server => server::run_server(opt.config_file, opt.secrets).await,
|
||||
Command::OfflineRepair(repair_opt) => {
|
||||
repair::offline::offline_repair(opt.config_file, opt.secrets, repair_opt).await
|
||||
cli::local::repair::offline_repair(opt.config_file, opt.secrets, repair_opt).await
|
||||
}
|
||||
Command::ConvertDb(conv_opt) => {
|
||||
cli::convert_db::do_conversion(conv_opt).map_err(From::from)
|
||||
cli::local::convert_db::do_conversion(conv_opt).map_err(From::from)
|
||||
}
|
||||
Command::Node(NodeOperation::NodeId(node_id_opt)) => {
|
||||
node_id_command(opt.config_file, node_id_opt.quiet)
|
||||
cli::local::init::node_id_command(opt.config_file, node_id_opt.quiet)
|
||||
}
|
||||
Command::AdminApiSchema => {
|
||||
println!(
|
||||
"{}",
|
||||
garage_api_admin::openapi::ApiDoc::openapi()
|
||||
.to_pretty_json()
|
||||
.unwrap()
|
||||
);
|
||||
Ok(())
|
||||
}
|
||||
_ => cli_command(opt).await,
|
||||
};
|
||||
|
@ -252,7 +259,7 @@ async fn cli_command(opt: Opt) -> Result<(), Error> {
|
|||
(id, addrs[0], false)
|
||||
} else {
|
||||
let node_id = garage_rpc::system::read_node_id(&config.as_ref().unwrap().metadata_dir)
|
||||
.err_context(READ_KEY_ERROR)?;
|
||||
.err_context(cli::local::init::READ_KEY_ERROR)?;
|
||||
if let Some(a) = config.as_ref().and_then(|c| c.rpc_public_addr.as_ref()) {
|
||||
use std::net::ToSocketAddrs;
|
||||
let a = a
|
||||
|
@ -281,13 +288,12 @@ async fn cli_command(opt: Opt) -> Result<(), Error> {
|
|||
Err(e).err_context("Unable to connect to destination RPC host. Check that you are using the same value of rpc_secret as them, and that you have their correct full-length node ID (public key).")?;
|
||||
}
|
||||
|
||||
let system_rpc_endpoint = netapp.endpoint::<SystemRpc, ()>(SYSTEM_RPC_PATH.into());
|
||||
let admin_rpc_endpoint = netapp.endpoint::<AdminRpc, ()>(ADMIN_RPC_PATH.into());
|
||||
let proxy_rpc_endpoint = netapp.endpoint::<ProxyRpc, ()>(PROXY_RPC_PATH.into());
|
||||
|
||||
match cli_command_dispatch(opt.cmd, &system_rpc_endpoint, &admin_rpc_endpoint, id).await {
|
||||
Err(HelperError::Internal(i)) => Err(Error::Message(format!("Internal error: {}", i))),
|
||||
Err(HelperError::BadRequest(b)) => Err(Error::Message(b)),
|
||||
Err(e) => Err(Error::Message(format!("{}", e))),
|
||||
Ok(x) => Ok(x),
|
||||
}
|
||||
let cli = cli::remote::Cli {
|
||||
proxy_rpc_endpoint,
|
||||
rpc_host: id,
|
||||
};
|
||||
|
||||
cli.handle(opt.cmd).await
|
||||
}
|
||||
|
|
|
@ -1,2 +0,0 @@
|
|||
pub mod offline;
|
||||
pub mod online;
|
|
@ -14,7 +14,6 @@ use garage_web::WebServer;
|
|||
#[cfg(feature = "k2v")]
|
||||
use garage_api_k2v::api_server::K2VApiServer;
|
||||
|
||||
use crate::admin::*;
|
||||
use crate::secrets::{fill_secrets, Secrets};
|
||||
#[cfg(feature = "telemetry-otlp")]
|
||||
use crate::tracing_setup::*;
|
||||
|
@ -66,6 +65,7 @@ pub async fn run_server(config_file: PathBuf, secrets: Secrets) -> Result<(), Er
|
|||
info!("Initialize Admin API server and metrics collector...");
|
||||
let admin_server = AdminApiServer::new(
|
||||
garage.clone(),
|
||||
background.clone(),
|
||||
#[cfg(feature = "metrics")]
|
||||
metrics_exporter,
|
||||
);
|
||||
|
@ -73,9 +73,6 @@ pub async fn run_server(config_file: PathBuf, secrets: Secrets) -> Result<(), Er
|
|||
info!("Launching internal Garage cluster communications...");
|
||||
let run_system = tokio::spawn(garage.system.clone().run(watch_cancel.clone()));
|
||||
|
||||
info!("Create admin RPC handler...");
|
||||
AdminRpcHandler::new(garage.clone(), background.clone());
|
||||
|
||||
// ---- Launch public-facing API servers ----
|
||||
|
||||
let mut servers = vec![];
|
||||
|
|
|
@ -3,6 +3,8 @@ use std::path::{Path, PathBuf};
|
|||
use std::process;
|
||||
use std::sync::Once;
|
||||
|
||||
use serde_json::json;
|
||||
|
||||
use super::ext::*;
|
||||
|
||||
// https://xkcd.com/221/
|
||||
|
@ -193,27 +195,17 @@ api_bind_addr = "127.0.0.1:{admin_port}"
|
|||
let mut key = Key::default();
|
||||
|
||||
let mut cmd = self.command();
|
||||
let base = cmd.args(["key", "create"]);
|
||||
let base = cmd.args(["json-api", "CreateKey"]);
|
||||
let with_name = match maybe_name {
|
||||
Some(name) => base.args([name]),
|
||||
None => base,
|
||||
Some(name) => base.args([serde_json::to_string(&json!({"name": name})).unwrap()]),
|
||||
None => base.args(["{}"]),
|
||||
};
|
||||
|
||||
let output = with_name.expect_success_output("Could not create key");
|
||||
let stdout = String::from_utf8(output.stdout).unwrap();
|
||||
let stdout: serde_json::Value = serde_json::from_slice(&output.stdout).unwrap();
|
||||
|
||||
for line in stdout.lines() {
|
||||
if let Some(key_id) = line.strip_prefix("Key ID: ") {
|
||||
key.id = key_id.to_owned();
|
||||
continue;
|
||||
}
|
||||
if let Some(key_secret) = line.strip_prefix("Secret key: ") {
|
||||
key.secret = key_secret.to_owned();
|
||||
continue;
|
||||
}
|
||||
}
|
||||
assert!(!key.id.is_empty(), "Invalid key: Key ID is empty");
|
||||
assert!(!key.secret.is_empty(), "Invalid key: Key secret is empty");
|
||||
key.id = stdout["accessKeyId"].as_str().unwrap().to_string();
|
||||
key.secret = stdout["secretAccessKey"].as_str().unwrap().to_string();
|
||||
|
||||
key
|
||||
}
|
||||
|
|
|
@ -5,7 +5,10 @@ use crate::json_body;
|
|||
use assert_json_diff::assert_json_eq;
|
||||
use aws_sdk_s3::{
|
||||
primitives::ByteStream,
|
||||
types::{CorsConfiguration, CorsRule, ErrorDocument, IndexDocument, WebsiteConfiguration},
|
||||
types::{
|
||||
Condition, CorsConfiguration, CorsRule, ErrorDocument, IndexDocument, Protocol, Redirect,
|
||||
RoutingRule, WebsiteConfiguration,
|
||||
},
|
||||
};
|
||||
use http::{Request, StatusCode};
|
||||
use http_body_util::BodyExt;
|
||||
|
@ -455,12 +458,18 @@ async fn test_website_check_domain() {
|
|||
res_body,
|
||||
json!({
|
||||
"code": "InvalidRequest",
|
||||
"message": "Bad request: No domain query string found",
|
||||
"message": "Bad request: Missing argument `domain` for endpoint",
|
||||
"region": "garage-integ-test",
|
||||
"path": "/check",
|
||||
})
|
||||
);
|
||||
|
||||
// FIXME: Edge case with empty domain
|
||||
// Currently, empty domain is interpreted as an absent parameter
|
||||
// due to logic in router_macros.rs, so this test fails.
|
||||
// Maybe we want empty parameters to be acceptable? But that might
|
||||
// break a lot of S3 stuff.
|
||||
/*
|
||||
let admin_req = || {
|
||||
Request::builder()
|
||||
.method("GET")
|
||||
|
@ -484,6 +493,7 @@ async fn test_website_check_domain() {
|
|||
"path": "/check",
|
||||
})
|
||||
);
|
||||
*/
|
||||
|
||||
let admin_req = || {
|
||||
Request::builder()
|
||||
|
@ -533,3 +543,444 @@ async fn test_website_check_domain() {
|
|||
})
|
||||
);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_website_redirect_full_bucket() {
|
||||
const BCKT_NAME: &str = "my-redirect-full";
|
||||
let ctx = common::context();
|
||||
let bucket = ctx.create_bucket(BCKT_NAME);
|
||||
|
||||
let conf = WebsiteConfiguration::builder()
|
||||
.routing_rules(
|
||||
RoutingRule::builder()
|
||||
.condition(Condition::builder().key_prefix_equals("").build())
|
||||
.redirect(
|
||||
Redirect::builder()
|
||||
.protocol(Protocol::Https)
|
||||
.host_name("other.tld")
|
||||
.replace_key_prefix_with("")
|
||||
.build(),
|
||||
)
|
||||
.build(),
|
||||
)
|
||||
.build();
|
||||
|
||||
ctx.client
|
||||
.put_bucket_website()
|
||||
.bucket(&bucket)
|
||||
.website_configuration(conf)
|
||||
.send()
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let req = Request::builder()
|
||||
.method("GET")
|
||||
.uri(format!("http://127.0.0.1:{}/my-path", ctx.garage.web_port))
|
||||
.header("Host", format!("{}.web.garage", BCKT_NAME))
|
||||
.body(Body::new(Bytes::new()))
|
||||
.unwrap();
|
||||
|
||||
let client = Client::builder(TokioExecutor::new()).build_http();
|
||||
let resp = client.request(req).await.unwrap();
|
||||
assert_eq!(resp.status(), StatusCode::FOUND);
|
||||
assert_eq!(
|
||||
resp.headers()
|
||||
.get(hyper::header::LOCATION)
|
||||
.unwrap()
|
||||
.to_str()
|
||||
.unwrap(),
|
||||
"https://other.tld/my-path"
|
||||
);
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_website_redirect() {
|
||||
const BCKT_NAME: &str = "my-redirect";
|
||||
let ctx = common::context();
|
||||
let bucket = ctx.create_bucket(BCKT_NAME);
|
||||
|
||||
ctx.client
|
||||
.put_object()
|
||||
.bucket(&bucket)
|
||||
.key("index.html")
|
||||
.body(ByteStream::from_static(b"index"))
|
||||
.send()
|
||||
.await
|
||||
.unwrap();
|
||||
ctx.client
|
||||
.put_object()
|
||||
.bucket(&bucket)
|
||||
.key("404.html")
|
||||
.body(ByteStream::from_static(b"main 404"))
|
||||
.send()
|
||||
.await
|
||||
.unwrap();
|
||||
ctx.client
|
||||
.put_object()
|
||||
.bucket(&bucket)
|
||||
.key("static-file")
|
||||
.body(ByteStream::from_static(b"static file"))
|
||||
.send()
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let mut conf = WebsiteConfiguration::builder()
|
||||
.index_document(
|
||||
IndexDocument::builder()
|
||||
.suffix("home.html")
|
||||
.build()
|
||||
.unwrap(),
|
||||
)
|
||||
.error_document(ErrorDocument::builder().key("404.html").build().unwrap());
|
||||
|
||||
for (prefix, condition) in [("unconditional", false), ("conditional", true)] {
|
||||
let code = condition.then(|| "404".to_string());
|
||||
conf = conf
|
||||
// simple redirect
|
||||
.routing_rules(
|
||||
RoutingRule::builder()
|
||||
.condition(
|
||||
Condition::builder()
|
||||
.set_http_error_code_returned_equals(code.clone())
|
||||
.key_prefix_equals(format!("{prefix}/redirect-prefix/"))
|
||||
.build(),
|
||||
)
|
||||
.redirect(
|
||||
Redirect::builder()
|
||||
.http_redirect_code("302")
|
||||
.replace_key_prefix_with("other-prefix/")
|
||||
.build(),
|
||||
)
|
||||
.build(),
|
||||
)
|
||||
.routing_rules(
|
||||
RoutingRule::builder()
|
||||
.condition(
|
||||
Condition::builder()
|
||||
.set_http_error_code_returned_equals(code.clone())
|
||||
.key_prefix_equals(format!("{prefix}/redirect-prefix-307/"))
|
||||
.build(),
|
||||
)
|
||||
.redirect(
|
||||
Redirect::builder()
|
||||
.http_redirect_code("307")
|
||||
.replace_key_prefix_with("other-prefix/")
|
||||
.build(),
|
||||
)
|
||||
.build(),
|
||||
)
|
||||
// simple redirect
|
||||
.routing_rules(
|
||||
RoutingRule::builder()
|
||||
.condition(
|
||||
Condition::builder()
|
||||
.set_http_error_code_returned_equals(code.clone())
|
||||
.key_prefix_equals(format!("{prefix}/redirect-fixed/"))
|
||||
.build(),
|
||||
)
|
||||
.redirect(
|
||||
Redirect::builder()
|
||||
.http_redirect_code("302")
|
||||
.replace_key_with("fixed_key")
|
||||
.build(),
|
||||
)
|
||||
.build(),
|
||||
)
|
||||
// stream other file
|
||||
.routing_rules(
|
||||
RoutingRule::builder()
|
||||
.condition(
|
||||
Condition::builder()
|
||||
.set_http_error_code_returned_equals(code.clone())
|
||||
.key_prefix_equals(format!("{prefix}/stream-fixed/"))
|
||||
.build(),
|
||||
)
|
||||
.redirect(
|
||||
Redirect::builder()
|
||||
.http_redirect_code("200")
|
||||
.replace_key_with("static-file")
|
||||
.build(),
|
||||
)
|
||||
.build(),
|
||||
)
|
||||
// stream other file as error
|
||||
.routing_rules(
|
||||
RoutingRule::builder()
|
||||
.condition(
|
||||
Condition::builder()
|
||||
.set_http_error_code_returned_equals(code.clone())
|
||||
.key_prefix_equals(format!("{prefix}/stream-404/"))
|
||||
.build(),
|
||||
)
|
||||
.redirect(
|
||||
Redirect::builder()
|
||||
.http_redirect_code("404")
|
||||
.replace_key_with("static-file")
|
||||
.build(),
|
||||
)
|
||||
.build(),
|
||||
)
|
||||
// fail to stream other file
|
||||
.routing_rules(
|
||||
RoutingRule::builder()
|
||||
.condition(
|
||||
Condition::builder()
|
||||
.set_http_error_code_returned_equals(code.clone())
|
||||
.key_prefix_equals(format!("{prefix}/stream-missing/"))
|
||||
.build(),
|
||||
)
|
||||
.redirect(
|
||||
Redirect::builder()
|
||||
.http_redirect_code("200")
|
||||
.replace_key_with("missing-file")
|
||||
.build(),
|
||||
)
|
||||
.build(),
|
||||
);
|
||||
}
|
||||
let conf = conf.build();
|
||||
|
||||
ctx.client
|
||||
.put_bucket_website()
|
||||
.bucket(&bucket)
|
||||
.website_configuration(conf.clone())
|
||||
.send()
|
||||
.await
|
||||
.unwrap();
|
||||
|
||||
let stored_cfg = ctx
|
||||
.client
|
||||
.get_bucket_website()
|
||||
.bucket(&bucket)
|
||||
.send()
|
||||
.await
|
||||
.unwrap();
|
||||
assert_eq!(stored_cfg.index_document, conf.index_document);
|
||||
assert_eq!(stored_cfg.error_document, conf.error_document);
|
||||
assert_eq!(stored_cfg.routing_rules, conf.routing_rules);
|
||||
|
||||
let req = |path| {
|
||||
Request::builder()
|
||||
.method("GET")
|
||||
.uri(format!(
|
||||
"http://127.0.0.1:{}/{}/path",
|
||||
ctx.garage.web_port, path
|
||||
))
|
||||
.header("Host", format!("{}.web.garage", BCKT_NAME))
|
||||
.body(Body::new(Bytes::new()))
|
||||
.unwrap()
|
||||
};
|
||||
|
||||
test_redirect_helper("unconditional", true, &req).await;
|
||||
test_redirect_helper("conditional", true, &req).await;
|
||||
for prefix in ["unconditional", "conditional"] {
|
||||
for rule_path in [
|
||||
"redirect-prefix",
|
||||
"redirect-prefix-307",
|
||||
"redirect-fixed",
|
||||
"stream-fixed",
|
||||
"stream-404",
|
||||
"stream-missing",
|
||||
] {
|
||||
ctx.client
|
||||
.put_object()
|
||||
.bucket(&bucket)
|
||||
.key(format!("{prefix}/{rule_path}/path"))
|
||||
.body(ByteStream::from_static(b"i exist"))
|
||||
.send()
|
||||
.await
|
||||
.unwrap();
|
||||
}
|
||||
}
|
||||
test_redirect_helper("unconditional", true, &req).await;
|
||||
test_redirect_helper("conditional", false, &req).await;
|
||||
}
|
||||
|
||||
async fn test_redirect_helper(
|
||||
prefix: &str,
|
||||
should_see_redirect: bool,
|
||||
req: impl Fn(String) -> Request<http_body_util::Full<Bytes>>,
|
||||
) {
|
||||
use http::header;
|
||||
let client = Client::builder(TokioExecutor::new()).build_http();
|
||||
let expected_body = b"i exist".as_ref();
|
||||
|
||||
let resp = client
|
||||
.request(req(format!("{prefix}/redirect-prefix")))
|
||||
.await
|
||||
.unwrap();
|
||||
if should_see_redirect {
|
||||
assert_eq!(resp.status(), StatusCode::FOUND);
|
||||
assert_eq!(
|
||||
resp.headers()
|
||||
.get(header::LOCATION)
|
||||
.unwrap()
|
||||
.to_str()
|
||||
.unwrap(),
|
||||
"/other-prefix/path"
|
||||
);
|
||||
assert!(resp
|
||||
.into_body()
|
||||
.collect()
|
||||
.await
|
||||
.unwrap()
|
||||
.to_bytes()
|
||||
.is_empty());
|
||||
} else {
|
||||
assert_eq!(resp.status(), StatusCode::OK);
|
||||
assert!(resp.headers().get(header::LOCATION).is_none());
|
||||
assert_eq!(
|
||||
resp.into_body().collect().await.unwrap().to_bytes(),
|
||||
expected_body,
|
||||
);
|
||||
}
|
||||
|
||||
let resp = client
|
||||
.request(req(format!("{prefix}/redirect-prefix-307")))
|
||||
.await
|
||||
.unwrap();
|
||||
if should_see_redirect {
|
||||
assert_eq!(resp.status(), StatusCode::TEMPORARY_REDIRECT);
|
||||
assert_eq!(
|
||||
resp.headers()
|
||||
.get(header::LOCATION)
|
||||
.unwrap()
|
||||
.to_str()
|
||||
.unwrap(),
|
||||
"/other-prefix/path"
|
||||
);
|
||||
assert!(resp
|
||||
.into_body()
|
||||
.collect()
|
||||
.await
|
||||
.unwrap()
|
||||
.to_bytes()
|
||||
.is_empty());
|
||||
} else {
|
||||
assert_eq!(resp.status(), StatusCode::OK);
|
||||
assert!(resp.headers().get(header::LOCATION).is_none());
|
||||
assert_eq!(
|
||||
resp.into_body().collect().await.unwrap().to_bytes(),
|
||||
expected_body,
|
||||
);
|
||||
}
|
||||
|
||||
let resp = client
|
||||
.request(req(format!("{prefix}/redirect-fixed")))
|
||||
.await
|
||||
.unwrap();
|
||||
if should_see_redirect {
|
||||
assert_eq!(resp.status(), StatusCode::FOUND);
|
||||
assert_eq!(
|
||||
resp.headers()
|
||||
.get(header::LOCATION)
|
||||
.unwrap()
|
||||
.to_str()
|
||||
.unwrap(),
|
||||
"/fixed_key"
|
||||
);
|
||||
assert!(resp
|
||||
.into_body()
|
||||
.collect()
|
||||
.await
|
||||
.unwrap()
|
||||
.to_bytes()
|
||||
.is_empty());
|
||||
} else {
|
||||
assert_eq!(resp.status(), StatusCode::OK);
|
||||
assert!(resp.headers().get(header::LOCATION).is_none());
|
||||
assert_eq!(
|
||||
resp.into_body().collect().await.unwrap().to_bytes(),
|
||||
expected_body,
|
||||
);
|
||||
}
|
||||
let resp = client
|
||||
.request(req(format!("{prefix}/stream-fixed")))
|
||||
.await
|
||||
.unwrap();
|
||||
if should_see_redirect {
|
||||
assert_eq!(resp.status(), StatusCode::OK);
|
||||
assert!(resp.headers().get(header::LOCATION).is_none());
|
||||
assert_eq!(
|
||||
resp.into_body().collect().await.unwrap().to_bytes(),
|
||||
b"static file".as_ref(),
|
||||
);
|
||||
} else {
|
||||
assert_eq!(resp.status(), StatusCode::OK);
|
||||
assert!(resp.headers().get(header::LOCATION).is_none());
|
||||
assert_eq!(
|
||||
resp.into_body().collect().await.unwrap().to_bytes(),
|
||||
expected_body,
|
||||
);
|
||||
}
|
||||
let resp = client
|
||||
.request(req(format!("{prefix}/stream-404")))
|
||||
.await
|
||||
.unwrap();
|
||||
if should_see_redirect {
|
||||
assert_eq!(resp.status(), StatusCode::NOT_FOUND);
|
||||
assert!(resp.headers().get(header::LOCATION).is_none());
|
||||
assert_eq!(
|
||||
resp.into_body().collect().await.unwrap().to_bytes(),
|
||||
b"static file".as_ref(),
|
||||
);
|
||||
} else {
|
||||
assert_eq!(resp.status(), StatusCode::OK);
|
||||
assert!(resp.headers().get(header::LOCATION).is_none());
|
||||
assert_eq!(
|
||||
resp.into_body().collect().await.unwrap().to_bytes(),
|
||||
expected_body,
|
||||
);
|
||||
}
|
||||
let resp = client
|
||||
.request(req(format!("{prefix}/stream-404")))
|
||||
.await
|
||||
.unwrap();
|
||||
if should_see_redirect {
|
||||
assert_eq!(resp.status(), StatusCode::NOT_FOUND);
|
||||
assert!(resp.headers().get(header::LOCATION).is_none());
|
||||
assert_eq!(
|
||||
resp.into_body().collect().await.unwrap().to_bytes(),
|
||||
b"static file".as_ref(),
|
||||
);
|
||||
} else {
|
||||
assert_eq!(resp.status(), StatusCode::OK);
|
||||
assert!(resp.headers().get(header::LOCATION).is_none());
|
||||
assert_eq!(
|
||||
resp.into_body().collect().await.unwrap().to_bytes(),
|
||||
expected_body,
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
#[tokio::test]
|
||||
async fn test_website_invalid_redirect() {
|
||||
const BCKT_NAME: &str = "my-invalid-redirect";
|
||||
let ctx = common::context();
|
||||
let bucket = ctx.create_bucket(BCKT_NAME);
|
||||
|
||||
let conf = WebsiteConfiguration::builder()
|
||||
.routing_rules(
|
||||
RoutingRule::builder()
|
||||
.condition(Condition::builder().key_prefix_equals("").build())
|
||||
.redirect(
|
||||
Redirect::builder()
|
||||
.protocol(Protocol::Https)
|
||||
.host_name("other.tld")
|
||||
.replace_key_prefix_with("")
|
||||
// we don't allow 200 with hostname
|
||||
.http_redirect_code("200")
|
||||
.build(),
|
||||
)
|
||||
.build(),
|
||||
)
|
||||
.build();
|
||||
|
||||
ctx.client
|
||||
.put_bucket_website()
|
||||
.bucket(&bucket)
|
||||
.website_configuration(conf)
|
||||
.send()
|
||||
.await
|
||||
.unwrap_err();
|
||||
}
|
||||
|
|
|
@ -21,6 +21,7 @@ garage_block.workspace = true
|
|||
garage_util.workspace = true
|
||||
garage_net.workspace = true
|
||||
|
||||
argon2.workspace = true
|
||||
async-trait.workspace = true
|
||||
blake2.workspace = true
|
||||
chrono.workspace = true
|
||||
|
|
177
src/model/admin_token_table.rs
Normal file
177
src/model/admin_token_table.rs
Normal file
|
@ -0,0 +1,177 @@
|
|||
use base64::prelude::*;
|
||||
|
||||
use garage_util::crdt::{self, Crdt};
|
||||
use garage_util::time::now_msec;
|
||||
|
||||
use garage_table::{EmptyKey, Entry, TableSchema};
|
||||
|
||||
pub use crate::key_table::KeyFilter;
|
||||
|
||||
mod v2 {
|
||||
use garage_util::crdt;
|
||||
use serde::{Deserialize, Serialize};
|
||||
|
||||
#[derive(PartialEq, Eq, Clone, Debug, Serialize, Deserialize)]
|
||||
pub struct AdminApiToken {
|
||||
/// An admin API token is a bearer token of the following form:
|
||||
/// `<prefix>.<suffix>`
|
||||
/// Only the prefix is saved here, it is used as an identifier.
|
||||
/// The entire API token is hashed and saved in `token_hash` in `state`.
|
||||
pub prefix: String,
|
||||
|
||||
/// If the token is not deleted, its parameters
|
||||
pub state: crdt::Deletable<AdminApiTokenParams>,
|
||||
}
|
||||
|
||||
#[derive(PartialEq, Eq, Clone, Debug, Serialize, Deserialize)]
|
||||
pub struct AdminApiTokenParams {
|
||||
/// Creation date
|
||||
pub created: u64,
|
||||
|
||||
/// The entire API token hashed as a password
|
||||
pub token_hash: String,
|
||||
|
||||
/// User-defined name
|
||||
pub name: crdt::Lww<String>,
|
||||
|
||||
/// The optional time of expiration of the token
|
||||
pub expiration: crdt::Lww<Option<u64>>,
|
||||
|
||||
/// The scope of the token, i.e. list of authorized admin API calls
|
||||
pub scope: crdt::Lww<AdminApiTokenScope>,
|
||||
}
|
||||
|
||||
#[derive(PartialEq, Eq, Clone, Debug, Serialize, Deserialize)]
|
||||
pub struct AdminApiTokenScope(pub Vec<String>);
|
||||
|
||||
impl garage_util::migrate::InitialFormat for AdminApiToken {
|
||||
const VERSION_MARKER: &'static [u8] = b"G2admtok";
|
||||
}
|
||||
}
|
||||
|
||||
pub use v2::*;
|
||||
|
||||
impl Crdt for AdminApiTokenParams {
|
||||
fn merge(&mut self, o: &Self) {
|
||||
self.name.merge(&o.name);
|
||||
self.expiration.merge(&o.expiration);
|
||||
self.scope.merge(&o.scope);
|
||||
}
|
||||
}
|
||||
|
||||
impl Crdt for AdminApiToken {
|
||||
fn merge(&mut self, other: &Self) {
|
||||
self.state.merge(&other.state);
|
||||
}
|
||||
}
|
||||
|
||||
impl Crdt for AdminApiTokenScope {
|
||||
fn merge(&mut self, other: &Self) {
|
||||
self.0.retain(|x| other.0.contains(x));
|
||||
}
|
||||
}
|
||||
|
||||
impl AdminApiToken {
|
||||
/// Create a new admin API token.
|
||||
/// Returns the AdminApiToken object, which contains the hashed bearer token,
|
||||
/// as well as the plaintext bearer token.
|
||||
pub fn new(name: &str) -> (Self, String) {
|
||||
use argon2::{
|
||||
password_hash::{rand_core::OsRng, PasswordHasher, SaltString},
|
||||
Argon2,
|
||||
};
|
||||
|
||||
let prefix = hex::encode(&rand::random::<[u8; 12]>()[..]);
|
||||
let secret = BASE64_URL_SAFE_NO_PAD.encode(&rand::random::<[u8; 32]>()[..]);
|
||||
let token = format!("{}.{}", prefix, secret);
|
||||
|
||||
let salt = SaltString::generate(&mut OsRng);
|
||||
let argon2 = Argon2::default();
|
||||
let hashed_token = argon2
|
||||
.hash_password(token.as_bytes(), &salt)
|
||||
.expect("could not hash admin API token")
|
||||
.to_string();
|
||||
|
||||
let ret = AdminApiToken {
|
||||
prefix,
|
||||
state: crdt::Deletable::present(AdminApiTokenParams {
|
||||
created: now_msec(),
|
||||
token_hash: hashed_token,
|
||||
name: crdt::Lww::new(name.to_string()),
|
||||
expiration: crdt::Lww::new(None),
|
||||
scope: crdt::Lww::new(AdminApiTokenScope(vec!["*".to_string()])),
|
||||
}),
|
||||
};
|
||||
|
||||
(ret, token)
|
||||
}
|
||||
|
||||
pub fn delete(prefix: String) -> Self {
|
||||
Self {
|
||||
prefix,
|
||||
state: crdt::Deletable::Deleted,
|
||||
}
|
||||
}
|
||||
|
||||
/// Returns true if this represents a deleted bucket
|
||||
pub fn is_deleted(&self) -> bool {
|
||||
self.state.is_deleted()
|
||||
}
|
||||
|
||||
/// Returns an option representing the params (None if in deleted state)
|
||||
pub fn params(&self) -> Option<&AdminApiTokenParams> {
|
||||
self.state.as_option()
|
||||
}
|
||||
|
||||
/// Mutable version of `.state()`
|
||||
pub fn params_mut(&mut self) -> Option<&mut AdminApiTokenParams> {
|
||||
self.state.as_option_mut()
|
||||
}
|
||||
|
||||
/// Scope, if not deleted, or empty slice
|
||||
pub fn scope(&self) -> &[String] {
|
||||
self.state
|
||||
.as_option()
|
||||
.map(|x| &x.scope.get().0[..])
|
||||
.unwrap_or_default()
|
||||
}
|
||||
}
|
||||
|
||||
impl Entry<EmptyKey, String> for AdminApiToken {
|
||||
fn partition_key(&self) -> &EmptyKey {
|
||||
&EmptyKey
|
||||
}
|
||||
fn sort_key(&self) -> &String {
|
||||
&self.prefix
|
||||
}
|
||||
fn is_tombstone(&self) -> bool {
|
||||
self.is_deleted()
|
||||
}
|
||||
}
|
||||
|
||||
pub struct AdminApiTokenTable;
|
||||
|
||||
impl TableSchema for AdminApiTokenTable {
|
||||
const TABLE_NAME: &'static str = "admin_token";
|
||||
|
||||
type P = EmptyKey;
|
||||
type S = String;
|
||||
type E = AdminApiToken;
|
||||
type Filter = KeyFilter;
|
||||
|
||||
fn matches_filter(entry: &Self::E, filter: &Self::Filter) -> bool {
|
||||
match filter {
|
||||
KeyFilter::Deleted(df) => df.apply(entry.state.is_deleted()),
|
||||
KeyFilter::MatchesAndNotDeleted(pat) => {
|
||||
let pat = pat.to_lowercase();
|
||||
entry
|
||||
.params()
|
||||
.map(|p| {
|
||||
entry.prefix.to_lowercase().starts_with(&pat)
|
||||
|| p.name.get().to_lowercase() == pat
|
||||
})
|
||||
.unwrap_or(false)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -119,7 +119,122 @@ mod v08 {
|
|||
impl garage_util::migrate::InitialFormat for Bucket {}
|
||||
}
|
||||
|
||||
pub use v08::*;
|
||||
mod v2 {
|
||||
use crate::permission::BucketKeyPerm;
|
||||
use garage_util::crdt;
|
||||
use garage_util::data::Uuid;
|
||||
use serde::{Deserialize, Serialize};
|
||||
|
||||
use super::v08;
|
||||
|
||||
pub use v08::{BucketQuotas, CorsRule, LifecycleExpiration, LifecycleFilter, LifecycleRule};
|
||||
|
||||
#[derive(PartialEq, Eq, Clone, Debug, Serialize, Deserialize)]
|
||||
pub struct Bucket {
|
||||
/// ID of the bucket
|
||||
pub id: Uuid,
|
||||
/// State, and configuration if not deleted, of the bucket
|
||||
pub state: crdt::Deletable<BucketParams>,
|
||||
}
|
||||
|
||||
/// Configuration for a bucket
|
||||
#[derive(PartialEq, Eq, Clone, Debug, Serialize, Deserialize)]
|
||||
pub struct BucketParams {
|
||||
/// Bucket's creation date
|
||||
pub creation_date: u64,
|
||||
/// Map of key with access to the bucket, and what kind of access they give
|
||||
pub authorized_keys: crdt::Map<String, BucketKeyPerm>,
|
||||
|
||||
/// Map of aliases that are or have been given to this bucket
|
||||
/// in the global namespace
|
||||
/// (not authoritative: this is just used as an indication to
|
||||
/// map back to aliases when doing ListBuckets)
|
||||
pub aliases: crdt::LwwMap<String, bool>,
|
||||
/// Map of aliases that are or have been given to this bucket
|
||||
/// in namespaces local to keys
|
||||
/// key = (access key id, alias name)
|
||||
pub local_aliases: crdt::LwwMap<(String, String), bool>,
|
||||
|
||||
/// Whether this bucket is allowed for website access
|
||||
/// (under all of its global alias names),
|
||||
/// and if so, the website configuration XML document
|
||||
pub website_config: crdt::Lww<Option<WebsiteConfig>>,
|
||||
/// CORS rules
|
||||
pub cors_config: crdt::Lww<Option<Vec<CorsRule>>>,
|
||||
/// Lifecycle configuration
|
||||
pub lifecycle_config: crdt::Lww<Option<Vec<LifecycleRule>>>,
|
||||
/// Bucket quotas
|
||||
pub quotas: crdt::Lww<BucketQuotas>,
|
||||
}
|
||||
|
||||
#[derive(PartialEq, Eq, Clone, Debug, Serialize, Deserialize)]
|
||||
pub struct WebsiteConfig {
|
||||
pub index_document: String,
|
||||
pub error_document: Option<String>,
|
||||
// this field is currently unused, but present so adding it in the future doesn't
|
||||
// need a new migration
|
||||
pub redirect_all: Option<RedirectAll>,
|
||||
pub routing_rules: Vec<RoutingRule>,
|
||||
}
|
||||
|
||||
#[derive(PartialEq, Eq, Clone, Debug, Serialize, Deserialize)]
|
||||
pub struct RedirectAll {
|
||||
pub hostname: String,
|
||||
pub protocol: String,
|
||||
}
|
||||
|
||||
#[derive(PartialEq, Eq, Clone, Debug, Serialize, Deserialize)]
|
||||
pub struct RoutingRule {
|
||||
pub condition: Option<RedirectCondition>,
|
||||
pub redirect: Redirect,
|
||||
}
|
||||
|
||||
#[derive(PartialEq, Eq, Clone, Debug, Serialize, Deserialize)]
|
||||
pub struct RedirectCondition {
|
||||
pub http_error_code: Option<u16>,
|
||||
pub prefix: Option<String>,
|
||||
}
|
||||
|
||||
#[derive(PartialEq, Eq, Clone, Debug, Serialize, Deserialize)]
|
||||
pub struct Redirect {
|
||||
pub hostname: Option<String>,
|
||||
pub http_redirect_code: u16,
|
||||
pub protocol: Option<String>,
|
||||
pub replace_key_prefix: Option<String>,
|
||||
pub replace_key: Option<String>,
|
||||
}
|
||||
|
||||
impl garage_util::migrate::Migrate for Bucket {
|
||||
const VERSION_MARKER: &'static [u8] = b"G2bkt";
|
||||
|
||||
type Previous = v08::Bucket;
|
||||
|
||||
fn migrate(old: v08::Bucket) -> Bucket {
|
||||
Bucket {
|
||||
id: old.id,
|
||||
state: old.state.map(|x| BucketParams {
|
||||
creation_date: x.creation_date,
|
||||
authorized_keys: x.authorized_keys,
|
||||
aliases: x.aliases,
|
||||
local_aliases: x.local_aliases,
|
||||
website_config: x.website_config.map(|wc_opt| {
|
||||
wc_opt.map(|wc| WebsiteConfig {
|
||||
index_document: wc.index_document,
|
||||
error_document: wc.error_document,
|
||||
redirect_all: None,
|
||||
routing_rules: vec![],
|
||||
})
|
||||
}),
|
||||
cors_config: x.cors_config,
|
||||
lifecycle_config: x.lifecycle_config,
|
||||
quotas: x.quotas,
|
||||
}),
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub use v2::*;
|
||||
|
||||
impl AutoCrdt for BucketQuotas {
|
||||
const WARN_IF_DIFFERENT: bool = true;
|
||||
|
|
|
@ -24,6 +24,7 @@ use crate::s3::mpu_table::*;
|
|||
use crate::s3::object_table::*;
|
||||
use crate::s3::version_table::*;
|
||||
|
||||
use crate::admin_token_table::*;
|
||||
use crate::bucket_alias_table::*;
|
||||
use crate::bucket_table::*;
|
||||
use crate::helper;
|
||||
|
@ -50,6 +51,8 @@ pub struct Garage {
|
|||
/// The block manager
|
||||
pub block_manager: Arc<BlockManager>,
|
||||
|
||||
/// Table containing admin API keys
|
||||
pub admin_token_table: Arc<Table<AdminApiTokenTable, TableFullReplication>>,
|
||||
/// Table containing buckets
|
||||
pub bucket_table: Arc<Table<BucketTable, TableFullReplication>>,
|
||||
/// Table containing bucket aliases
|
||||
|
@ -151,13 +154,6 @@ impl Garage {
|
|||
info!("Initialize membership management system...");
|
||||
let system = System::new(network_key, replication_factor, consistency_mode, &config)?;
|
||||
|
||||
let data_rep_param = TableShardedReplication {
|
||||
system: system.clone(),
|
||||
replication_factor: replication_factor.into(),
|
||||
write_quorum: replication_factor.write_quorum(consistency_mode),
|
||||
read_quorum: 1,
|
||||
};
|
||||
|
||||
let meta_rep_param = TableShardedReplication {
|
||||
system: system.clone(),
|
||||
replication_factor: replication_factor.into(),
|
||||
|
@ -170,10 +166,19 @@ impl Garage {
|
|||
};
|
||||
|
||||
info!("Initialize block manager...");
|
||||
let block_manager = BlockManager::new(&db, &config, data_rep_param, system.clone())?;
|
||||
let block_write_quorum = replication_factor.write_quorum(consistency_mode);
|
||||
let block_manager = BlockManager::new(&db, &config, block_write_quorum, system.clone())?;
|
||||
block_manager.register_bg_vars(&mut bg_vars);
|
||||
|
||||
// ---- admin tables ----
|
||||
info!("Initialize admin_token_table...");
|
||||
let admin_token_table = Table::new(
|
||||
AdminApiTokenTable,
|
||||
control_rep_param.clone(),
|
||||
system.clone(),
|
||||
&db,
|
||||
);
|
||||
|
||||
info!("Initialize bucket_table...");
|
||||
let bucket_table = Table::new(BucketTable, control_rep_param.clone(), system.clone(), &db);
|
||||
|
||||
|
@ -263,6 +268,7 @@ impl Garage {
|
|||
db,
|
||||
system,
|
||||
block_manager,
|
||||
admin_token_table,
|
||||
bucket_table,
|
||||
bucket_alias_table,
|
||||
key_table,
|
||||
|
@ -282,6 +288,7 @@ impl Garage {
|
|||
pub fn spawn_workers(self: &Arc<Self>, bg: &BackgroundRunner) -> Result<(), Error> {
|
||||
self.block_manager.spawn_workers(bg);
|
||||
|
||||
self.admin_token_table.spawn_workers(bg);
|
||||
self.bucket_table.spawn_workers(bg);
|
||||
self.bucket_alias_table.spawn_workers(bg);
|
||||
self.key_table.spawn_workers(bg);
|
||||
|
|
|
@ -1,7 +1,7 @@
|
|||
use std::time::Duration;
|
||||
|
||||
use garage_util::data::*;
|
||||
use garage_util::error::OkOrMessage;
|
||||
use garage_util::error::{Error as GarageError, OkOrMessage};
|
||||
use garage_util::time::*;
|
||||
|
||||
use garage_table::util::*;
|
||||
|
@ -16,104 +16,172 @@ pub struct BucketHelper<'a>(pub(crate) &'a Garage);
|
|||
|
||||
#[allow(clippy::ptr_arg)]
|
||||
impl<'a> BucketHelper<'a> {
|
||||
pub async fn resolve_global_bucket_name(
|
||||
// ================
|
||||
// Local functions to find buckets FAST.
|
||||
// This is only for the fast path in API requests.
|
||||
// They do not provide the read-after-write guarantee
|
||||
// when used in conjunction with other operations that
|
||||
// modify buckets and bucket aliases.
|
||||
// ================
|
||||
|
||||
/// Return bucket corresponding to global bucket name, if it exists
|
||||
/// (and is not a tombstone entry).
|
||||
///
|
||||
/// The name can be of two forms:
|
||||
/// 1. A global bucket alias
|
||||
/// 2. The full ID of a bucket encoded in hex
|
||||
///
|
||||
/// Note that there is no possible ambiguity between the two forms,
|
||||
/// as the maximum length of a bucket name is 63 characters, and the full
|
||||
/// hex id is 64 chars long.
|
||||
///
|
||||
/// This will not do any network interaction to check the alias and
|
||||
/// bucket tables, it will only check the local copy of the table.
|
||||
/// As a consequence, it does not provide read-after-write guarantees.
|
||||
pub fn resolve_global_bucket_fast(
|
||||
&self,
|
||||
bucket_name: &String,
|
||||
) -> Result<Option<Uuid>, Error> {
|
||||
// Bucket names in Garage are aliases, true bucket identifiers
|
||||
// are 32-byte UUIDs. This function resolves bucket names into
|
||||
// their full identifier by looking up in the bucket_alias_table.
|
||||
// This function also allows buckets to be identified by their
|
||||
// full UUID (hex-encoded). Here, if the name to be resolved is a
|
||||
// hex string of the correct length, it is directly parsed as a bucket
|
||||
// identifier which is returned. There is no risk of this conflicting
|
||||
// with an actual bucket name: bucket names are max 63 chars long by
|
||||
// the AWS spec, and hex-encoded UUIDs are 64 chars long.
|
||||
) -> Result<Option<Bucket>, GarageError> {
|
||||
let hexbucket = hex::decode(bucket_name.as_str())
|
||||
.ok()
|
||||
.and_then(|by| Uuid::try_from(&by));
|
||||
if let Some(bucket_id) = hexbucket {
|
||||
Ok(self
|
||||
.0
|
||||
.bucket_table
|
||||
.get(&EmptyKey, &bucket_id)
|
||||
.await?
|
||||
.filter(|x| !x.state.is_deleted())
|
||||
.map(|_| bucket_id))
|
||||
} else {
|
||||
Ok(self
|
||||
.0
|
||||
.bucket_alias_table
|
||||
.get(&EmptyKey, bucket_name)
|
||||
.await?
|
||||
.and_then(|x| *x.state.get()))
|
||||
}
|
||||
let bucket_id = match hexbucket {
|
||||
Some(id) => id,
|
||||
None => {
|
||||
let alias = self
|
||||
.0
|
||||
.bucket_alias_table
|
||||
.get_local(&EmptyKey, bucket_name)?
|
||||
.and_then(|x| *x.state.get());
|
||||
match alias {
|
||||
Some(id) => id,
|
||||
None => return Ok(None),
|
||||
}
|
||||
}
|
||||
};
|
||||
Ok(self
|
||||
.0
|
||||
.bucket_table
|
||||
.get_local(&EmptyKey, &bucket_id)?
|
||||
.filter(|x| !x.state.is_deleted()))
|
||||
}
|
||||
|
||||
/// Return bucket corresponding to a bucket name from the perspective of
|
||||
/// a given access key, if it exists (and is not a tombstone entry).
|
||||
///
|
||||
/// The name can be of three forms:
|
||||
/// 1. A global bucket alias
|
||||
/// 2. A local bucket alias
|
||||
/// 3. The full ID of a bucket encoded in hex
|
||||
///
|
||||
/// This will not do any network interaction, it will only check the local
|
||||
/// copy of the bucket and global alias table. It will also resolve local
|
||||
/// aliases directly using the data provided in the `api_key` parameter.
|
||||
/// As a consequence, it does not provide read-after-write guarantees.
|
||||
///
|
||||
/// In case no such bucket is found, this function returns a NoSuchBucket error.
|
||||
#[allow(clippy::ptr_arg)]
|
||||
pub async fn resolve_bucket(&self, bucket_name: &String, api_key: &Key) -> Result<Uuid, Error> {
|
||||
pub fn resolve_bucket_fast(
|
||||
&self,
|
||||
bucket_name: &String,
|
||||
api_key: &Key,
|
||||
) -> Result<Bucket, Error> {
|
||||
let api_key_params = api_key
|
||||
.state
|
||||
.as_option()
|
||||
.ok_or_message("Key should not be deleted at this point")?;
|
||||
|
||||
if let Some(Some(bucket_id)) = api_key_params.local_aliases.get(bucket_name) {
|
||||
Ok(*bucket_id)
|
||||
} else {
|
||||
Ok(self
|
||||
.resolve_global_bucket_name(bucket_name)
|
||||
.await?
|
||||
.ok_or_else(|| Error::NoSuchBucket(bucket_name.to_string()))?)
|
||||
}
|
||||
let bucket_opt =
|
||||
if let Some(Some(bucket_id)) = api_key_params.local_aliases.get(bucket_name) {
|
||||
self.0
|
||||
.bucket_table
|
||||
.get_local(&EmptyKey, &bucket_id)?
|
||||
.filter(|x| !x.state.is_deleted())
|
||||
} else {
|
||||
self.resolve_global_bucket_fast(bucket_name)?
|
||||
};
|
||||
bucket_opt.ok_or_else(|| Error::NoSuchBucket(bucket_name.to_string()))
|
||||
}
|
||||
|
||||
/// Find a bucket by its global alias or a prefix of its uuid
|
||||
pub async fn admin_get_existing_matching_bucket(
|
||||
// ================
|
||||
// Global functions that do quorum reads/writes,
|
||||
// for admin operations.
|
||||
// ================
|
||||
|
||||
/// This is the same as `resolve_global_bucket_fast`,
|
||||
/// except that it does quorum reads to ensure consistency.
|
||||
pub async fn resolve_global_bucket(
|
||||
&self,
|
||||
pattern: &String,
|
||||
) -> Result<Uuid, Error> {
|
||||
if let Some(uuid) = self.resolve_global_bucket_name(pattern).await? {
|
||||
return Ok(uuid);
|
||||
} else if pattern.len() >= 2 {
|
||||
let hexdec = pattern
|
||||
.get(..pattern.len() & !1)
|
||||
.and_then(|x| hex::decode(x).ok());
|
||||
if let Some(hex) = hexdec {
|
||||
let mut start = [0u8; 32];
|
||||
start
|
||||
.as_mut_slice()
|
||||
.get_mut(..hex.len())
|
||||
.ok_or_bad_request("invalid length")?
|
||||
.copy_from_slice(&hex);
|
||||
let mut candidates = self
|
||||
bucket_name: &String,
|
||||
) -> Result<Option<Bucket>, GarageError> {
|
||||
let hexbucket = hex::decode(bucket_name.as_str())
|
||||
.ok()
|
||||
.and_then(|by| Uuid::try_from(&by));
|
||||
let bucket_id = match hexbucket {
|
||||
Some(id) => id,
|
||||
None => {
|
||||
let alias = self
|
||||
.0
|
||||
.bucket_table
|
||||
.get_range(
|
||||
&EmptyKey,
|
||||
Some(start.into()),
|
||||
Some(DeletedFilter::NotDeleted),
|
||||
10,
|
||||
EnumerationOrder::Forward,
|
||||
)
|
||||
.bucket_alias_table
|
||||
.get(&EmptyKey, bucket_name)
|
||||
.await?
|
||||
.into_iter()
|
||||
.collect::<Vec<_>>();
|
||||
candidates.retain(|x| hex::encode(x.id).starts_with(pattern));
|
||||
if candidates.len() == 1 {
|
||||
return Ok(candidates.into_iter().next().unwrap().id);
|
||||
.and_then(|x| *x.state.get());
|
||||
match alias {
|
||||
Some(id) => id,
|
||||
None => return Ok(None),
|
||||
}
|
||||
}
|
||||
};
|
||||
Ok(self
|
||||
.0
|
||||
.bucket_table
|
||||
.get(&EmptyKey, &bucket_id)
|
||||
.await?
|
||||
.filter(|x| !x.state.is_deleted()))
|
||||
}
|
||||
|
||||
/// Return bucket corresponding to a bucket name from the perspective of
|
||||
/// a given access key, if it exists (and is not a tombstone entry).
|
||||
///
|
||||
/// This is the same as `resolve_bucket_fast`, with the following differences:
|
||||
///
|
||||
/// - this function does quorum reads to ensure consistency.
|
||||
/// - this function fetches the Key entry from the key table to ensure up-to-date data
|
||||
/// - this function returns None if the bucket is not found, instead of HelperError::NoSuchBucket
|
||||
#[allow(clippy::ptr_arg)]
|
||||
pub async fn resolve_bucket(
|
||||
&self,
|
||||
bucket_name: &String,
|
||||
key_id: &String,
|
||||
) -> Result<Option<Bucket>, GarageError> {
|
||||
let local_alias = self
|
||||
.0
|
||||
.key_table
|
||||
.get(&EmptyKey, &key_id)
|
||||
.await?
|
||||
.and_then(|k| k.state.into_option())
|
||||
.ok_or_else(|| GarageError::Message(format!("access key {} has been deleted", key_id)))?
|
||||
.local_aliases
|
||||
.get(bucket_name)
|
||||
.copied()
|
||||
.flatten();
|
||||
|
||||
if let Some(bucket_id) = local_alias {
|
||||
Ok(self
|
||||
.0
|
||||
.bucket_table
|
||||
.get(&EmptyKey, &bucket_id)
|
||||
.await?
|
||||
.filter(|x| !x.state.is_deleted()))
|
||||
} else {
|
||||
Ok(self.resolve_global_bucket(bucket_name).await?)
|
||||
}
|
||||
Err(Error::BadRequest(format!(
|
||||
"Bucket not found / several matching buckets: {}",
|
||||
pattern
|
||||
)))
|
||||
}
|
||||
|
||||
/// Returns a Bucket if it is present in bucket table,
|
||||
/// even if it is in deleted state. Querying a non-existing
|
||||
/// bucket ID returns an internal error.
|
||||
pub async fn get_internal_bucket(&self, bucket_id: Uuid) -> Result<Bucket, Error> {
|
||||
pub(crate) async fn get_internal_bucket(&self, bucket_id: Uuid) -> Result<Bucket, Error> {
|
||||
Ok(self
|
||||
.0
|
||||
.bucket_table
|
||||
|
|
|
@ -3,7 +3,7 @@ use garage_util::error::OkOrMessage;
|
|||
|
||||
use crate::garage::Garage;
|
||||
use crate::helper::error::*;
|
||||
use crate::key_table::{Key, KeyFilter};
|
||||
use crate::key_table::Key;
|
||||
|
||||
pub struct KeyHelper<'a>(pub(crate) &'a Garage);
|
||||
|
||||
|
@ -33,33 +33,4 @@ impl<'a> KeyHelper<'a> {
|
|||
.filter(|b| !b.state.is_deleted())
|
||||
.ok_or_else(|| Error::NoSuchAccessKey(key_id.to_string()))
|
||||
}
|
||||
|
||||
/// Returns a Key if it is present in key table,
|
||||
/// looking it up by key ID or by a match on its name,
|
||||
/// only if it is in non-deleted state.
|
||||
/// Querying a non-existing key ID or a deleted key
|
||||
/// returns a bad request error.
|
||||
pub async fn get_existing_matching_key(&self, pattern: &str) -> Result<Key, Error> {
|
||||
let candidates = self
|
||||
.0
|
||||
.key_table
|
||||
.get_range(
|
||||
&EmptyKey,
|
||||
None,
|
||||
Some(KeyFilter::MatchesAndNotDeleted(pattern.to_string())),
|
||||
10,
|
||||
EnumerationOrder::Forward,
|
||||
)
|
||||
.await?
|
||||
.into_iter()
|
||||
.collect::<Vec<_>>();
|
||||
if candidates.len() != 1 {
|
||||
Err(Error::BadRequest(format!(
|
||||
"{} matching keys",
|
||||
candidates.len()
|
||||
)))
|
||||
} else {
|
||||
Ok(candidates.into_iter().next().unwrap())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -451,10 +451,7 @@ impl K2VRpcHandler {
|
|||
|
||||
let mut value = self
|
||||
.item_table
|
||||
.data
|
||||
.read_entry(&key.partition, &key.sort_key)?
|
||||
.map(|bytes| self.item_table.data.decode_entry(&bytes[..]))
|
||||
.transpose()?
|
||||
.get_local(&key.partition, &key.sort_key)?
|
||||
.unwrap_or_else(|| {
|
||||
K2VItem::new(
|
||||
key.partition.bucket_id,
|
||||
|
|
|
@ -5,6 +5,7 @@ pub mod permission;
|
|||
|
||||
pub mod index_counter;
|
||||
|
||||
pub mod admin_token_table;
|
||||
pub mod bucket_alias_table;
|
||||
pub mod bucket_table;
|
||||
pub mod key_table;
|
||||
|
|
|
@ -257,6 +257,11 @@ mod v010 {
|
|||
/// (compression happens before encryption, whereas for non-encrypted
|
||||
/// objects, compression is handled at the level of the block manager)
|
||||
compressed: bool,
|
||||
/// Whether the encryption uses an Object Encryption Key derived
|
||||
/// from the master SSE-C key, instead of the master SSE-C key itself.
|
||||
/// This is the case of objects created in Garage v2+
|
||||
#[serde(default)]
|
||||
use_oek: bool,
|
||||
},
|
||||
Plaintext {
|
||||
/// Plain-text headers
|
||||
|
@ -277,6 +282,7 @@ mod v010 {
|
|||
pub enum ChecksumAlgorithm {
|
||||
Crc32,
|
||||
Crc32c,
|
||||
Crc64Nvme,
|
||||
Sha1,
|
||||
Sha256,
|
||||
}
|
||||
|
@ -286,6 +292,7 @@ mod v010 {
|
|||
pub enum ChecksumValue {
|
||||
Crc32(#[serde(with = "serde_bytes")] [u8; 4]),
|
||||
Crc32c(#[serde(with = "serde_bytes")] [u8; 4]),
|
||||
Crc64Nvme(#[serde(with = "serde_bytes")] [u8; 8]),
|
||||
Sha1(#[serde(with = "serde_bytes")] [u8; 20]),
|
||||
Sha256(#[serde(with = "serde_bytes")] [u8; 32]),
|
||||
}
|
||||
|
@ -487,6 +494,7 @@ impl ChecksumValue {
|
|||
match self {
|
||||
ChecksumValue::Crc32(_) => ChecksumAlgorithm::Crc32,
|
||||
ChecksumValue::Crc32c(_) => ChecksumAlgorithm::Crc32c,
|
||||
ChecksumValue::Crc64Nvme(_) => ChecksumAlgorithm::Crc64Nvme,
|
||||
ChecksumValue::Sha1(_) => ChecksumAlgorithm::Sha1,
|
||||
ChecksumValue::Sha256(_) => ChecksumAlgorithm::Sha256,
|
||||
}
|
||||
|
|
|
@ -149,14 +149,27 @@ impl LayoutHelper {
|
|||
self.layout.as_ref().unwrap()
|
||||
}
|
||||
|
||||
/// Returns the current layout version
|
||||
pub fn current(&self) -> &LayoutVersion {
|
||||
self.inner().current()
|
||||
}
|
||||
|
||||
/// Returns all layout versions currently active in the cluster
|
||||
pub fn versions(&self) -> &[LayoutVersion] {
|
||||
&self.inner().versions
|
||||
}
|
||||
|
||||
/// Returns the latest layout version for which it is safe to read data from,
|
||||
/// i.e. the version whose version number is sync_map_min
|
||||
pub fn read_version(&self) -> &LayoutVersion {
|
||||
let sync_min = self.sync_map_min;
|
||||
self.versions()
|
||||
.iter()
|
||||
.find(|x| x.version == sync_min)
|
||||
.or(self.versions().last())
|
||||
.unwrap()
|
||||
}
|
||||
|
||||
pub fn is_check_ok(&self) -> bool {
|
||||
self.is_check_ok
|
||||
}
|
||||
|
@ -181,6 +194,8 @@ impl LayoutHelper {
|
|||
self.sync_map_min
|
||||
}
|
||||
|
||||
// ---- helpers for layout synchronization ----
|
||||
|
||||
pub fn sync_digest(&self) -> SyncLayoutDigest {
|
||||
SyncLayoutDigest {
|
||||
current: self.current().version,
|
||||
|
@ -189,50 +204,7 @@ impl LayoutHelper {
|
|||
}
|
||||
}
|
||||
|
||||
pub fn read_nodes_of(&self, position: &Hash) -> Vec<Uuid> {
|
||||
let sync_min = self.sync_map_min;
|
||||
let version = self
|
||||
.versions()
|
||||
.iter()
|
||||
.find(|x| x.version == sync_min)
|
||||
.or(self.versions().last())
|
||||
.unwrap();
|
||||
version
|
||||
.nodes_of(position, version.replication_factor)
|
||||
.collect()
|
||||
}
|
||||
|
||||
pub fn storage_sets_of(&self, position: &Hash) -> Vec<Vec<Uuid>> {
|
||||
self.versions()
|
||||
.iter()
|
||||
.map(|x| x.nodes_of(position, x.replication_factor).collect())
|
||||
.collect()
|
||||
}
|
||||
|
||||
pub fn storage_nodes_of(&self, position: &Hash) -> Vec<Uuid> {
|
||||
let mut ret = vec![];
|
||||
for version in self.versions().iter() {
|
||||
ret.extend(version.nodes_of(position, version.replication_factor));
|
||||
}
|
||||
ret.sort();
|
||||
ret.dedup();
|
||||
ret
|
||||
}
|
||||
|
||||
pub fn current_storage_nodes_of(&self, position: &Hash) -> Vec<Uuid> {
|
||||
let ver = self.current();
|
||||
ver.nodes_of(position, ver.replication_factor).collect()
|
||||
}
|
||||
|
||||
pub fn trackers_hash(&self) -> Hash {
|
||||
self.trackers_hash
|
||||
}
|
||||
|
||||
pub fn staging_hash(&self) -> Hash {
|
||||
self.staging_hash
|
||||
}
|
||||
|
||||
pub fn digest(&self) -> RpcLayoutDigest {
|
||||
pub(crate) fn digest(&self) -> RpcLayoutDigest {
|
||||
RpcLayoutDigest {
|
||||
current_version: self.current().version,
|
||||
active_versions: self.versions().len(),
|
||||
|
|
|
@ -180,9 +180,7 @@ impl LayoutHistory {
|
|||
|
||||
// Determine set of nodes for partition p in layout version v.
|
||||
// Sort the node set to avoid duplicate computations.
|
||||
let mut set = v
|
||||
.nodes_of(&p_hash, v.replication_factor)
|
||||
.collect::<Vec<Uuid>>();
|
||||
let mut set = v.nodes_of(&p_hash).collect::<Vec<Uuid>>();
|
||||
set.sort();
|
||||
|
||||
// If this set was already processed, skip it.
|
||||
|
@ -267,20 +265,9 @@ impl LayoutHistory {
|
|||
changed
|
||||
}
|
||||
|
||||
pub fn apply_staged_changes(mut self, version: Option<u64>) -> Result<(Self, Message), Error> {
|
||||
match version {
|
||||
None => {
|
||||
let error = r#"
|
||||
Please pass the new layout version number to ensure that you are writing the correct version of the cluster layout.
|
||||
To know the correct value of the new layout version, invoke `garage layout show` and review the proposed changes.
|
||||
"#;
|
||||
return Err(Error::Message(error.into()));
|
||||
}
|
||||
Some(v) => {
|
||||
if v != self.current().version + 1 {
|
||||
return Err(Error::Message("Invalid new layout version".into()));
|
||||
}
|
||||
}
|
||||
pub fn apply_staged_changes(mut self, version: u64) -> Result<(Self, Message), Error> {
|
||||
if version != self.current().version + 1 {
|
||||
return Err(Error::Message("Invalid new layout version".into()));
|
||||
}
|
||||
|
||||
// Compute new version and add it to history
|
||||
|
|
|
@ -143,16 +143,19 @@ impl LayoutManager {
|
|||
|
||||
// ---- ACK LOCKING ----
|
||||
|
||||
pub fn write_sets_of(self: &Arc<Self>, position: &Hash) -> WriteLock<Vec<Vec<Uuid>>> {
|
||||
pub fn write_lock_with<T, F>(self: &Arc<Self>, f: F) -> WriteLock<T>
|
||||
where
|
||||
F: FnOnce(&LayoutHelper) -> T,
|
||||
{
|
||||
let layout = self.layout();
|
||||
let version = layout.current().version;
|
||||
let nodes = layout.storage_sets_of(position);
|
||||
let value = f(&layout);
|
||||
layout
|
||||
.ack_lock
|
||||
.get(&version)
|
||||
.unwrap()
|
||||
.fetch_add(1, Ordering::Relaxed);
|
||||
WriteLock::new(version, self, nodes)
|
||||
WriteLock::new(version, self, value)
|
||||
}
|
||||
|
||||
// ---- INTERNALS ---
|
||||
|
|
|
@ -1,5 +1,3 @@
|
|||
use std::fmt;
|
||||
|
||||
use bytesize::ByteSize;
|
||||
|
||||
use garage_util::crdt::{AutoCrdt, Crdt};
|
||||
|
@ -397,30 +395,6 @@ impl NodeRole {
|
|||
}
|
||||
}
|
||||
|
||||
impl fmt::Display for ZoneRedundancy {
|
||||
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
||||
match self {
|
||||
ZoneRedundancy::Maximum => write!(f, "maximum"),
|
||||
ZoneRedundancy::AtLeast(x) => write!(f, "{}", x),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl core::str::FromStr for ZoneRedundancy {
|
||||
type Err = &'static str;
|
||||
fn from_str(s: &str) -> Result<Self, Self::Err> {
|
||||
match s {
|
||||
"none" | "max" | "maximum" => Ok(ZoneRedundancy::Maximum),
|
||||
x => {
|
||||
let v = x
|
||||
.parse::<usize>()
|
||||
.map_err(|_| "zone redundancy must be 'none'/'max' or an integer")?;
|
||||
Ok(ZoneRedundancy::AtLeast(v))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl UpdateTracker {
|
||||
fn merge(&mut self, other: &UpdateTracker) -> bool {
|
||||
let mut changed = false;
|
||||
|
@ -455,7 +429,7 @@ impl UpdateTracker {
|
|||
}
|
||||
}
|
||||
|
||||
pub fn min_among(&self, storage_nodes: &[Uuid], min_version: u64) -> u64 {
|
||||
fn min_among(&self, storage_nodes: &[Uuid], min_version: u64) -> u64 {
|
||||
storage_nodes
|
||||
.iter()
|
||||
.map(|x| self.get(x, min_version))
|
||||
|
|
|
@ -124,7 +124,7 @@ fn test_assignment() {
|
|||
let mut cl = LayoutHistory::new(ReplicationFactor::new(3).unwrap());
|
||||
update_layout(&mut cl, &node_capacity_vec, &node_zone_vec, 3);
|
||||
let v = cl.current().version;
|
||||
let (mut cl, msg) = cl.apply_staged_changes(Some(v + 1)).unwrap();
|
||||
let (mut cl, msg) = cl.apply_staged_changes(v + 1).unwrap();
|
||||
show_msg(&msg);
|
||||
assert_eq!(cl.check(), Ok(()));
|
||||
assert!(check_against_naive(cl.current()).unwrap());
|
||||
|
@ -133,7 +133,7 @@ fn test_assignment() {
|
|||
node_zone_vec = vec!["A", "B", "C", "C", "C", "B", "G", "H", "I"];
|
||||
update_layout(&mut cl, &node_capacity_vec, &node_zone_vec, 2);
|
||||
let v = cl.current().version;
|
||||
let (mut cl, msg) = cl.apply_staged_changes(Some(v + 1)).unwrap();
|
||||
let (mut cl, msg) = cl.apply_staged_changes(v + 1).unwrap();
|
||||
show_msg(&msg);
|
||||
assert_eq!(cl.check(), Ok(()));
|
||||
assert!(check_against_naive(cl.current()).unwrap());
|
||||
|
@ -141,7 +141,7 @@ fn test_assignment() {
|
|||
node_capacity_vec = vec![4000, 1000, 2000, 7000, 1000, 1000, 2000, 10000, 2000];
|
||||
update_layout(&mut cl, &node_capacity_vec, &node_zone_vec, 3);
|
||||
let v = cl.current().version;
|
||||
let (mut cl, msg) = cl.apply_staged_changes(Some(v + 1)).unwrap();
|
||||
let (mut cl, msg) = cl.apply_staged_changes(v + 1).unwrap();
|
||||
show_msg(&msg);
|
||||
assert_eq!(cl.check(), Ok(()));
|
||||
assert!(check_against_naive(cl.current()).unwrap());
|
||||
|
@ -151,7 +151,7 @@ fn test_assignment() {
|
|||
];
|
||||
update_layout(&mut cl, &node_capacity_vec, &node_zone_vec, 1);
|
||||
let v = cl.current().version;
|
||||
let (cl, msg) = cl.apply_staged_changes(Some(v + 1)).unwrap();
|
||||
let (cl, msg) = cl.apply_staged_changes(v + 1).unwrap();
|
||||
show_msg(&msg);
|
||||
assert_eq!(cl.check(), Ok(()));
|
||||
assert!(check_against_naive(cl.current()).unwrap());
|
||||
|
|
|
@ -114,9 +114,7 @@ impl LayoutVersion {
|
|||
}
|
||||
|
||||
/// Return the n servers in which data for this hash should be replicated
|
||||
pub fn nodes_of(&self, position: &Hash, n: usize) -> impl Iterator<Item = Uuid> + '_ {
|
||||
assert_eq!(n, self.replication_factor);
|
||||
|
||||
pub fn nodes_of(&self, position: &Hash) -> impl Iterator<Item = Uuid> + '_ {
|
||||
let data = &self.ring_assignment_data;
|
||||
|
||||
let partition_nodes = if data.len() == self.replication_factor * (1 << PARTITION_BITS) {
|
||||
|
@ -823,7 +821,7 @@ impl LayoutVersion {
|
|||
let total_cap_n = self.expect_get_node_capacity(&self.node_id_vec[*n]);
|
||||
let tags_n = (self.node_role(&self.node_id_vec[*n]).ok_or("<??>"))?.tags_string();
|
||||
table.push(format!(
|
||||
" {:?}\t{}\t{} ({} new)\t{}\t{} ({:.1}%)",
|
||||
" {:?}\t[{}]\t{} ({} new)\t{}\t{} ({:.1}%)",
|
||||
self.node_id_vec[*n],
|
||||
tags_n,
|
||||
stored_partitions[*n],
|
||||
|
|
|
@ -573,7 +573,7 @@ impl RpcHelper {
|
|||
// Compute, for each layout version, the set of nodes that might store
|
||||
// the block, and put them in their preferred order as of `request_order`.
|
||||
let mut vernodes = layout.versions().iter().map(|ver| {
|
||||
let nodes = ver.nodes_of(position, ver.replication_factor);
|
||||
let nodes = ver.nodes_of(position);
|
||||
rpc_helper.request_order(layout.current(), nodes)
|
||||
});
|
||||
|
||||
|
@ -607,7 +607,7 @@ impl RpcHelper {
|
|||
// Second step: add nodes of older layout versions
|
||||
let old_ver_iter = layout.inner().old_versions.iter().rev();
|
||||
for ver in old_ver_iter {
|
||||
let nodes = ver.nodes_of(position, ver.replication_factor);
|
||||
let nodes = ver.nodes_of(position);
|
||||
for node in rpc_helper.request_order(layout.current(), nodes) {
|
||||
if !ret.contains(&node) {
|
||||
ret.push(node);
|
||||
|
|
|
@ -124,6 +124,9 @@ pub struct NodeStatus {
|
|||
/// Hostname of the node
|
||||
pub hostname: Option<String>,
|
||||
|
||||
/// Garage version of the node
|
||||
pub garage_version: Option<String>,
|
||||
|
||||
/// Replication factor configured on the node
|
||||
pub replication_factor: usize,
|
||||
|
||||
|
@ -369,6 +372,10 @@ impl System {
|
|||
&self.layout_manager.rpc_helper
|
||||
}
|
||||
|
||||
pub fn local_status(&self) -> NodeStatus {
|
||||
self.local_status.read().unwrap().clone()
|
||||
}
|
||||
|
||||
// ---- Administrative operations (directly available and
|
||||
// also available through RPC) ----
|
||||
|
||||
|
@ -468,10 +475,7 @@ impl System {
|
|||
let mut partitions_quorum = 0;
|
||||
let mut partitions_all_ok = 0;
|
||||
for (_, hash) in partitions.iter() {
|
||||
let mut write_sets = layout
|
||||
.versions()
|
||||
.iter()
|
||||
.map(|x| x.nodes_of(hash, x.replication_factor));
|
||||
let mut write_sets = layout.versions().iter().map(|x| x.nodes_of(hash));
|
||||
let has_quorum = write_sets
|
||||
.clone()
|
||||
.all(|set| set.filter(|x| node_up(x)).count() >= quorum);
|
||||
|
@ -786,6 +790,7 @@ impl NodeStatus {
|
|||
.into_string()
|
||||
.unwrap_or_else(|_| "<invalid utf-8>".to_string()),
|
||||
),
|
||||
garage_version: Some(garage_util::version::garage_version().to_string()),
|
||||
replication_factor: replication_factor.into(),
|
||||
layout_digest: layout_manager.layout().digest(),
|
||||
meta_disk_avail: None,
|
||||
|
@ -796,6 +801,7 @@ impl NodeStatus {
|
|||
fn unknown() -> Self {
|
||||
NodeStatus {
|
||||
hostname: None,
|
||||
garage_version: None,
|
||||
replication_factor: 0,
|
||||
layout_digest: Default::default(),
|
||||
meta_disk_avail: None,
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show more
Loading…
Add table
Reference in a new issue
administration