Support website publishing #7

Merged
lx merged 61 commits from feature/website into master 2021-01-15 16:49:51 +00:00
17 changed files with 480 additions and 14 deletions

19
Cargo.lock generated
View file

@ -425,6 +425,7 @@ dependencies = [
"garage_rpc 0.1.0",
"garage_table 0.1.1",
"garage_util 0.1.0",
"garage_web",
"hex",
"log",
"pretty_env_logger",
@ -666,6 +667,24 @@ dependencies = [
"webpki",
]
[[package]]
name = "garage_web"
version = "0.1.0"
dependencies = [
"err-derive",
"futures",
"garage_api",
"garage_model 0.1.1",
"garage_table 0.1.1",
"garage_util 0.1.0",
"http",
"hyper",
"idna",
"log",
"percent-encoding",
"roxmltree",
]
[[package]]
name = "generator"
version = "0.6.21"

View file

@ -5,6 +5,7 @@ members = [
"src/table",
"src/model",
"src/api",
"src/web",
"src/garage",
]

View file

@ -92,7 +92,9 @@ api_bind_addr = "[::1]:3900" # the S3 API port, HTTP without TLS. Add a reverse
s3_region = "garage" # set this to anything. S3 API calls will fail if they are not made against the region set here.
[s3_web]
web_bind_addr = "[::1]:3902"
bind_addr = "[::1]:3902"
root_domain = ".garage.tld"
quentin marked this conversation as resolved
Review

Why break the convention that bucket name = domain name of the hosted website ? This gives more flexibility as we can host sites on diffferent tld/sld. If a website needs to move, we will copy files to the new bucket, it shouldn't be too costly (we don't expect to store TBs of data for a website...)

Why break the convention that bucket name = domain name of the hosted website ? This gives more flexibility as we can host sites on diffferent tld/sld. If a website needs to move, we will copy files to the new bucket, it shouldn't be too costly (we don't expect to store TBs of data for a website...)
Review

Please check my answer directly in the issue feed

Please check my answer directly in the issue feed
index = "index.html"
```
Build Garage using `cargo build --release`.

View file

@ -17,4 +17,6 @@ api_bind_addr = "[::1]:3900" # the S3 API port, HTTP without TLS. Add a reverse
s3_region = "garage" # set this to anything. S3 API calls will fail if they are not made against the region set here.
[s3_web]
web_bind_addr = "[::1]:3902"
bind_addr = "[::1]:3902"
root_domain = ".garage.tld"
index = "index.html"

View file

@ -41,6 +41,8 @@ s3_region = "garage" # set this to anything. S3 API calls will fail if they a
[s3_web]
bind_addr = "[::]:$((3920+$count))"
root_domain = ".garage.tld"
index = "index.html"
EOF
echo -en "$LABEL configuration written to $CONF_PATH\n"

View file

@ -6,6 +6,7 @@ shopt -s expand_aliases
SCRIPT_FOLDER="`dirname \"$0\"`"
REPO_FOLDER="${SCRIPT_FOLDER}/../"
echo "setup"
cargo build
${SCRIPT_FOLDER}/dev-clean.sh
${SCRIPT_FOLDER}/dev-cluster.sh > /tmp/garage.log 2>&1 &
@ -22,6 +23,7 @@ dd if=/dev/urandom of=/tmp/garage.1.rnd bs=1k count=2 # < INLINE_THRESHOLD = 307
dd if=/dev/urandom of=/tmp/garage.2.rnd bs=1M count=5
dd if=/dev/urandom of=/tmp/garage.3.rnd bs=1M count=10
echo "s3 api testing..."
for idx in $(seq 1 3); do
# AWS sends
awsgrg cp /tmp/garage.$idx.rnd s3://eprouvette/garage.$idx.aws
@ -55,6 +57,18 @@ for idx in $(seq 1 3); do
done
rm /tmp/garage.{1,2,3}.rnd
echo "website testing"
echo "<h1>hello world</h1>" > /tmp/garage-index.html
awsgrg cp /tmp/garage-index.html s3://eprouvette/index.html
[ `curl -s -o /dev/null -w "%{http_code}" --header "Host: eprouvette.garage.tld" http://127.0.0.1:3923/ ` == 404 ]
garage bucket website --allow eprouvette
[ `curl -s -o /dev/null -w "%{http_code}" --header "Host: eprouvette.garage.tld" http://127.0.0.1:3923/ ` == 200 ]
garage bucket website --deny eprouvette
[ `curl -s -o /dev/null -w "%{http_code}" --header "Host: eprouvette.garage.tld" http://127.0.0.1:3923/ ` == 404 ]
awsgrg rm s3://eprouvette/index.html
rm /tmp/garage-index.html
echo "teardown"
garage bucket deny --read --write eprouvette --key $AWS_ACCESS_KEY_ID
garage bucket delete --yes eprouvette
garage key delete --yes $AWS_ACCESS_KEY_ID

View file

@ -19,6 +19,7 @@ garage_rpc = { version = "0.1", path = "../rpc" }
garage_table = { version = "0.1.1", path = "../table" }
garage_model = { version = "0.1.1", path = "../model" }
garage_api = { version = "0.1.1", path = "../api" }
garage_web = { version = "0.1", path = "../web" }
bytes = "0.4"
rand = "0.7"

View file

@ -89,7 +89,7 @@ impl AdminRpcHandler {
}
bucket
.state
.update(BucketState::Present(crdt::LWWMap::new()));
.update(BucketState::Present(BucketParams::new()));
bucket
}
None => Bucket::new(query.name.clone()),
@ -155,6 +155,29 @@ impl AdminRpcHandler {
&query.key_id, &query.bucket, allow_read, allow_write
)))
}
BucketOperation::Website(query) => {
let mut bucket = self.get_existing_bucket(&query.bucket).await?;
if !(query.allow ^ query.deny) {
return Err(Error::Message(format!(
"You must specify exactly one flag, either --allow or --deny"
)));
}
if let BucketState::Present(state) = bucket.state.get_mut() {
state.website.update(query.allow);
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.to_string()))
} else {
quentin marked this conversation as resolved
Review

Probably put unreachable!() in else branch ? Because get_existing_bucket does not return a bucket that is deleted. (I think so, this should be checked)

Probably put `unreachable!()` in else branch ? Because get_existing_bucket does not return a bucket that is deleted. (I think so, this should be checked)
Review

Checked.
Indeed, it does not return deleted buckets.

        async fn get_existing_bucket(&self, bucket: &String) -> Result<Bucket, Error> {
                self.garage
                        .bucket_table
                        .get(&EmptyKey, bucket)
                        .await?
                        .filter(|b| !b.is_deleted())
                        .map(Ok)
                        .unwrap_or(Err(Error::BadRPC(format!(
                                "Bucket {} does not exist",
                                bucket
                        ))))
        }
Checked. Indeed, it does not return deleted buckets. ```rust async fn get_existing_bucket(&self, bucket: &String) -> Result<Bucket, Error> { self.garage .bucket_table .get(&EmptyKey, bucket) .await? .filter(|b| !b.is_deleted()) .map(Ok) .unwrap_or(Err(Error::BadRPC(format!( "Bucket {} does not exist", bucket )))) } ```
unreachable!();
}
quentin marked this conversation as resolved
Review

update_bucket_key ?

`update_bucket_key` ?
}
}
}
@ -237,6 +260,7 @@ impl AdminRpcHandler {
.unwrap_or(Err(Error::BadRPC(format!("Key {} does not exist", id))))
}
/// Update **bucket table** to inform of the new linked key
async fn update_bucket_key(
&self,
mut bucket: Bucket,
@ -244,7 +268,8 @@ impl AdminRpcHandler {
allow_read: bool,
allow_write: bool,
) -> Result<(), Error> {
if let BucketState::Present(ak) = bucket.state.get_mut() {
if let BucketState::Present(params) = bucket.state.get_mut() {
let ak = &mut params.authorized_keys;
let old_ak = ak.take_and_clear();
ak.merge(&old_ak.update_mutator(
key_id.to_string(),
@ -262,6 +287,7 @@ impl AdminRpcHandler {
Ok(())
}
/// Update **key table** to inform of the new linked bucket
async fn update_key_bucket(
&self,
mut key: Key,

View file

@ -141,6 +141,24 @@ pub enum BucketOperation {
/// Allow key to read or write to bucket
#[structopt(name = "deny")]
Deny(PermBucketOpt),
/// Expose as website or not
#[structopt(name = "website")]
Website(WebsiteOpt),
}
#[derive(Serialize, Deserialize, StructOpt, Debug)]
pub struct WebsiteOpt {
/// Create
#[structopt(long = "allow")]
pub allow: bool,
/// Delete
#[structopt(long = "deny")]
pub deny: bool,
/// Bucket name
pub bucket: String,
}
#[derive(Serialize, Deserialize, StructOpt, Debug)]

View file

@ -11,6 +11,7 @@ use garage_util::error::Error;
use garage_api::api_server;
use garage_model::garage::Garage;
use garage_rpc::rpc_server::RpcServer;
use garage_web::web_server;
use crate::admin_rpc::*;
@ -56,6 +57,7 @@ pub async fn run_server(config_file: PathBuf) -> Result<(), Error> {
info!("Initializing RPC and API servers...");
let run_rpc_server = Arc::new(rpc_server).run(wait_from(watch_cancel.clone()));
let api_server = api_server::run_api_server(garage.clone(), wait_from(watch_cancel.clone()));
let web_server = web_server::run_web_server(garage.clone(), wait_from(watch_cancel.clone()));
futures::try_join!(
garage
@ -78,6 +80,10 @@ pub async fn run_server(config_file: PathBuf) -> Result<(), Error> {
info!("API server exited");
rv
}),
web_server.map(|rv| {
info!("Web server exited");
rv
}),
background.run().map(|rv| {
info!("Background runner exited");
Ok(rv)

View file

@ -13,6 +13,11 @@ use crate::key_table::PermissionSet;
// We use them to perform migrations.
use model010::bucket_table as prev;
/// A bucket is a collection of objects
///
/// Its parameters are not directly accessible as:
/// - It must be possible to merge paramaters, hence the use of a LWW CRDT.
/// - A bucket has 2 states, Present or Deleted and parameters make sense only if present.
#[derive(PartialEq, Clone, Debug, Serialize, Deserialize)]
pub struct Bucket {
// Primary key
@ -24,27 +29,49 @@ pub struct Bucket {
#[derive(PartialEq, Clone, Debug, Serialize, Deserialize)]
pub enum BucketState {
Deleted,
Present(crdt::LWWMap<String, PermissionSet>),
Present(BucketParams),
}
impl CRDT for BucketState {
fn merge(&mut self, o: &Self) {
match o {
BucketState::Deleted => *self = BucketState::Deleted,
BucketState::Present(other_ak) => {
if let BucketState::Present(ak) = self {
ak.merge(other_ak);
BucketState::Present(other_params) => {
if let BucketState::Present(params) = self {
params.merge(other_params);
}
}
}
}
}
#[derive(PartialEq, Clone, Debug, Serialize, Deserialize)]
pub struct BucketParams {
pub authorized_keys: crdt::LWWMap<String, PermissionSet>,
pub website: crdt::LWW<bool>,
}
impl CRDT for BucketParams {
fn merge(&mut self, o: &Self) {
self.authorized_keys.merge(&o.authorized_keys);
self.website.merge(&o.website);
}
}
impl BucketParams {
pub fn new() -> Self {
BucketParams {
authorized_keys: crdt::LWWMap::new(),
website: crdt::LWW::new(false),
}
}
}
impl Bucket {
pub fn new(name: String) -> Self {
Bucket {
name,
state: crdt::LWW::new(BucketState::Present(crdt::LWWMap::new())),
state: crdt::LWW::new(BucketState::Present(BucketParams::new())),
}
}
pub fn is_deleted(&self) -> bool {
@ -53,7 +80,7 @@ impl Bucket {
pub fn authorized_keys(&self) -> &[(String, u64, PermissionSet)] {
match self.state.get() {
BucketState::Deleted => &[],
BucketState::Present(ak) => ak.items(),
BucketState::Present(state) => state.authorized_keys.items(),
}
}
}
@ -110,9 +137,15 @@ impl TableSchema for BucketTable {
},
));
}
let params = BucketParams {
authorized_keys: keys,
website: crdt::LWW::new(false),
};
Some(Bucket {
name: old.name,
state: crdt::LWW::migrate_from_raw(old.timestamp, BucketState::Present(keys)),
state: crdt::LWW::migrate_from_raw(old.timestamp, BucketState::Present(params)),
})
}
}

View file

@ -239,7 +239,7 @@ where
///
/// Typically, to update the value associated to a key in the map, you would do the following:
///
/// ```
/// ```ignore
/// let my_update = my_crdt.update_mutator(key_to_modify, new_value);
/// my_crdt.merge(&my_update);
/// ```
@ -261,7 +261,7 @@ where
/// empty map. This is very usefull to produce in-place a new map that contains only a delta
/// that modifies a certain value:
///
/// ```
/// ```ignore
/// let mut a = get_my_crdt_value();
/// let old_a = a.take_and_clear();
/// a.merge(&old_a.update_mutator(key_to_modify, new_value));
@ -273,7 +273,7 @@ where
/// but in the case where the map is a field in a struct for instance (as is always the case),
/// this becomes very handy:
///
/// ```
/// ```ignore
/// let mut a = get_my_crdt_value();
/// let old_a_map = a.map_field.take_and_clear();
/// a.map_field.merge(&old_a_map.update_mutator(key_to_modify, new_value));

View file

@ -35,6 +35,8 @@ pub struct Config {
pub rpc_tls: Option<TlsConfig>,
pub s3_api: ApiConfig,
pub s3_web: WebConfig,
}
#[derive(Deserialize, Debug, Clone)]
@ -50,6 +52,13 @@ pub struct ApiConfig {
pub s3_region: String,
}
#[derive(Deserialize, Debug, Clone)]
pub struct WebConfig {
pub bind_addr: SocketAddr,
pub root_domain: String,
pub index: String,
}
fn default_max_concurrent_rpc_requests() -> usize {
12
}

28
src/web/Cargo.toml Normal file
View file

@ -0,0 +1,28 @@
[package]
name = "garage_web"
version = "0.1.0"
authors = ["Alex Auvolat <alex@adnab.me>", "Quentin Dufour <quentin@dufour.io>"]
edition = "2018"
license = "GPL-3.0"
description = "S3-like website endpoint crate for the Garage object store"
repository = "https://git.deuxfleurs.fr/Deuxfleurs/garage"
[lib]
path = "lib.rs"
# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html
[dependencies]
garage_util = { version = "0.1", path = "../util" }
garage_table = { version = "0.1.1", path = "../table" }
garage_model = { version = "0.1.1", path = "../model" }
garage_api = { version = "0.1.1", path = "../api" }
err-derive = "0.2.3"
log = "0.4"
futures = "0.3"
http = "0.2"
hyper = "0.13"
percent-encoding = "2.1.0"
roxmltree = "0.11"
idna = "0.2"

39
src/web/error.rs Normal file
View file

@ -0,0 +1,39 @@
use err_derive::Error;
use hyper::StatusCode;
use garage_util::error::Error as GarageError;
#[derive(Debug, Error)]
pub enum Error {
quentin marked this conversation as resolved
Review

are all variants used? (try commenting them to find out)

are all variants used? (try commenting them to find out)
Review

Now fixed, I commented all error types and decommented one by one according to the compiler errors. And once it compiled, I removed the remaining commented entries.

Now fixed, I commented all error types and decommented one by one according to the compiler errors. And once it compiled, I removed the remaining commented entries.
#[error(display = "API error: {}", _0)]
ApiError(#[error(source)] garage_api::error::Error),
// Category: internal error
#[error(display = "Internal error: {}", _0)]
InternalError(#[error(source)] GarageError),
#[error(display = "Not found")]
NotFound,
// Category: bad request
#[error(display = "Invalid UTF-8: {}", _0)]
InvalidUTF8(#[error(source)] std::str::Utf8Error),
#[error(display = "Invalid header value: {}", _0)]
InvalidHeader(#[error(source)] hyper::header::ToStrError),
#[error(display = "Bad request: {}", _0)]
BadRequest(String),
}
impl Error {
pub fn http_status_code(&self) -> StatusCode {
match self {
Error::NotFound => StatusCode::NOT_FOUND,
Error::ApiError(e) => e.http_status_code(),
Error::InternalError(GarageError::RPC(_)) => StatusCode::SERVICE_UNAVAILABLE,
Error::InternalError(_) => StatusCode::INTERNAL_SERVER_ERROR,
_ => StatusCode::BAD_REQUEST,
}
}
}

6
src/web/lib.rs Normal file
View file

@ -0,0 +1,6 @@
#[macro_use]
extern crate log;
pub mod error;
pub mod web_server;

260
src/web/web_server.rs Normal file
View file

@ -0,0 +1,260 @@
use std::{borrow::Cow, convert::Infallible, net::SocketAddr, sync::Arc};
use futures::future::Future;
use hyper::{
header::HOST,
server::conn::AddrStream,
service::{make_service_fn, service_fn},
Body, Method, Request, Response, Server,
};
use idna::domain_to_unicode;
use crate::error::*;
use garage_api::s3_get::{handle_get, handle_head};
use garage_model::bucket_table::*;
use garage_model::garage::Garage;
use garage_table::*;
use garage_util::error::Error as GarageError;
pub async fn run_web_server(
garage: Arc<Garage>,
shutdown_signal: impl Future<Output = ()>,
) -> Result<(), GarageError> {
let addr = &garage.config.s3_web.bind_addr;
let service = make_service_fn(|conn: &AddrStream| {
let garage = garage.clone();
let client_addr = conn.remote_addr();
async move {
Ok::<_, Error>(service_fn(move |req: Request<Body>| {
let garage = garage.clone();
handle_request(garage, req, client_addr)
}))
}
});
let server = Server::bind(&addr).serve(service);
let graceful = server.with_graceful_shutdown(shutdown_signal);
info!("Web server listening on http://{}", addr);
graceful.await?;
Ok(())
}
async fn handle_request(
garage: Arc<Garage>,
req: Request<Body>,
addr: SocketAddr,
) -> Result<Response<Body>, Infallible> {
info!("{} {} {}", addr, req.method(), req.uri());
let res = serve_file(garage, req).await;
match &res {
Ok(r) => debug!("{} {:?}", r.status(), r.headers()),
Err(e) => warn!("Response: error {}, {}", e.http_status_code(), e),
}
Ok(res.unwrap_or_else(error_to_res))
}
fn error_to_res(e: Error) -> Response<Body> {
let body: Body = Body::from(format!("{}\n", e));
let mut http_error = Response::new(body);
*http_error.status_mut() = e.http_status_code();
http_error
}
async fn serve_file(garage: Arc<Garage>, req: Request<Body>) -> Result<Response<Body>, Error> {
// Get http authority string (eg. [::1]:3902 or garage.tld:80)
let authority = req
.headers()
.get(HOST)
.ok_or(Error::BadRequest(format!("HOST header required")))?
.to_str()?;
// Get bucket
let (host, _) = domain_to_unicode(authority_to_host(authority)?);
let root = &garage.config.s3_web.root_domain;
let bucket = host_to_bucket(&host, root);
// Check bucket is exposed as a website
let bucket_desc = garage
.bucket_table
.get(&EmptyKey, &bucket.to_string())
.await?
.filter(|b| !b.is_deleted())
.ok_or(Error::NotFound)?;
match bucket_desc.state.get() {
BucketState::Present(params) if *params.website.get() => Ok(()),
_ => Err(Error::NotFound),
}?;
quentin marked this conversation as resolved
Review

Rewrite as only two cases:

BucketState::Present(params) if params.website.get() => Ok(()),
_ => Err(Error::NotFound),
Rewrite as only two cases: ``` BucketState::Present(params) if params.website.get() => Ok(()), _ => Err(Error::NotFound), ```
// Get path
let path = req.uri().path().to_string();
let index = &garage.config.s3_web.index;
let key = path_to_key(&path, &index)?;
info!("Selected bucket: \"{}\", selected key: \"{}\"", bucket, key);
let res = match req.method() {
&Method::HEAD => handle_head(garage, &bucket, &key).await?,
&Method::GET => handle_get(garage, &req, bucket, &key).await?,
_ => return Err(Error::BadRequest(format!("HTTP method not supported"))),
};
Ok(res)
}
/// Extract host from the authority section given by the HTTP host header
///
/// The HTTP host contains both a host and a port.
/// Extracting the port is more complex than just finding the colon (:) symbol due to IPv6
/// We do not use the collect pattern as there is no way in std rust to collect over a stack allocated value
/// check here: https://docs.rs/collect_slice/1.2.0/collect_slice/
fn authority_to_host(authority: &str) -> Result<&str, Error> {
let mut iter = authority.chars().enumerate();
let (_, first_char) = iter
.next()
.ok_or(Error::BadRequest(format!("Authority is empty")))?;
let split = match first_char {
'[' => {
let mut iter = iter.skip_while(|(_, c)| c != &']');
match iter.next() {
Some((_, ']')) => iter.next(),
quentin marked this conversation as resolved Outdated
Outdated
Review

NO!! expect makes us crash (panic) if the condition is not verified, this is not acceptable (we don't want to assume that authority is well formed, maybe hyper checks it for us but maybe not and we don't want to take risks)

NO!! `expect` makes us crash (panic) if the condition is not verified, this is not acceptable (we don't want to assume that authority is well formed, maybe hyper checks it for us but maybe not and we don't want to take risks)
_ => {
return Err(Error::BadRequest(format!(
"Authority {} has an illegal format",
authority
)))
}
}
}
_ => iter.skip_while(|(_, c)| c != &':').next(),
};
match split {
Some((i, ':')) => Ok(&authority[..i]),
None => Ok(authority),
Some((_, _)) => Err(Error::BadRequest(format!(
"Authority {} has an illegal format",
authority
))),
}
}
/// Host to bucket
///
/// Convert a host, like "bucket.garage-site.tld" or "john.doe.com"
/// to the corresponding bucket, resp. "bucket" and "john.doe.com"
/// considering that ".garage-site.tld" is the "root domain".
/// This behavior has been chosen to follow AWS S3 semantic.
fn host_to_bucket<'a>(host: &'a str, root: &str) -> &'a str {
if root.len() >= host.len() || !host.ends_with(root) {
return host;
}
let len_diff = host.len() - root.len();
let missing_starting_dot = root.chars().next() != Some('.');
let cursor = if missing_starting_dot {
len_diff - 1
} else {
len_diff
};
&host[..cursor]
}
/// Path to key
///
/// Convert the provided path to the internal key
/// When a path ends with "/", we append the index name to match traditional web server behavior
/// which is also AWS S3 behavior.
fn path_to_key<'a>(path: &'a str, index: &str) -> Result<Cow<'a, str>, Error> {
let path_utf8 = percent_encoding::percent_decode_str(&path).decode_utf8()?;
if path_utf8.chars().next() != Some('/') {
return Err(Error::BadRequest(format!(
quentin marked this conversation as resolved Outdated
Outdated
Review

this case is unreachable!() thanks to the previous if

this case is `unreachable!()` thanks to the previous if
"Path must start with a / (slash)"
)));
}
match path_utf8.chars().last() {
None => unreachable!(),
Some('/') => {
let mut key = String::with_capacity(path_utf8.len() + index.len());
key.push_str(&path_utf8[1..]);
key.push_str(index);
Ok(key.into())
}
Some(_) => match path_utf8 {
Cow::Borrowed(pu8) => Ok((&pu8[1..]).into()),
Cow::Owned(pu8) => Ok((&pu8[1..]).to_string().into()),
},
}
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn authority_to_host_with_port() -> Result<(), Error> {
let domain = authority_to_host("[::1]:3902")?;
assert_eq!(domain, "[::1]");
let domain2 = authority_to_host("garage.tld:65200")?;
assert_eq!(domain2, "garage.tld");
let domain3 = authority_to_host("127.0.0.1:80")?;
assert_eq!(domain3, "127.0.0.1");
Ok(())
}
#[test]
fn authority_to_host_without_port() -> Result<(), Error> {
let domain = authority_to_host("[::1]")?;
assert_eq!(domain, "[::1]");
let domain2 = authority_to_host("garage.tld")?;
assert_eq!(domain2, "garage.tld");
let domain3 = authority_to_host("127.0.0.1")?;
assert_eq!(domain3, "127.0.0.1");
assert!(authority_to_host("[").is_err());
assert!(authority_to_host("[hello").is_err());
Ok(())
}
#[test]
fn host_to_bucket_test() {
assert_eq!(
host_to_bucket("john.doe.garage.tld", ".garage.tld"),
"john.doe"
);
assert_eq!(
host_to_bucket("john.doe.garage.tld", "garage.tld"),
"john.doe"
);
assert_eq!(host_to_bucket("john.doe.com", "garage.tld"), "john.doe.com");
assert_eq!(
host_to_bucket("john.doe.com", ".garage.tld"),
"john.doe.com"
);
assert_eq!(host_to_bucket("garage.tld", "garage.tld"), "garage.tld");
assert_eq!(host_to_bucket("garage.tld", ".garage.tld"), "garage.tld");
}
#[test]
fn path_to_key_test() -> Result<(), Error> {
assert_eq!(path_to_key("/file%20.jpg", "index.html")?, "file .jpg");
assert_eq!(path_to_key("/%20t/", "index.html")?, " t/index.html");
assert_eq!(path_to_key("/", "index.html")?, "index.html");
assert_eq!(path_to_key("/hello", "index.html")?, "hello");
assert!(path_to_key("", "index.html").is_err());
assert!(path_to_key("i/am/relative", "index.html").is_err());
Ok(())
}
}