Adapt S3 API code to use new multipart upload models

- Create and PutPart
- completemultipartupload
- upload part copy
- list_parts
This commit is contained in:
Alex 2023-05-03 12:02:59 +02:00
parent 38d6ac4295
commit 82e75c0e29
9 changed files with 610 additions and 546 deletions

View File

@ -27,6 +27,7 @@ use crate::s3::cors::*;
use crate::s3::delete::*;
use crate::s3::get::*;
use crate::s3::list::*;
use crate::s3::multipart::*;
use crate::s3::post_object::handle_post_object;
use crate::s3::put::*;
use crate::s3::router::Endpoint;

View File

@ -2,7 +2,7 @@ use std::pin::Pin;
use std::sync::Arc;
use std::time::{Duration, SystemTime, UNIX_EPOCH};
use futures::{stream, stream::Stream, StreamExt, TryFutureExt};
use futures::{stream, stream::Stream, StreamExt};
use md5::{Digest as Md5Digest, Md5};
use bytes::Bytes;
@ -18,12 +18,14 @@ use garage_util::time::*;
use garage_model::garage::Garage;
use garage_model::key_table::Key;
use garage_model::s3::block_ref_table::*;
use garage_model::s3::mpu_table::*;
use garage_model::s3::object_table::*;
use garage_model::s3::version_table::*;
use crate::helpers::parse_bucket_key;
use crate::s3::error::*;
use crate::s3::put::{decode_upload_id, get_headers};
use crate::s3::multipart;
use crate::s3::put::get_headers;
use crate::s3::xml::{self as s3_xml, xmlns_tag};
pub async fn handle_copy(
@ -92,7 +94,10 @@ pub async fn handle_copy(
let tmp_dest_object_version = ObjectVersion {
uuid: new_uuid,
timestamp: new_timestamp,
state: ObjectVersionState::Uploading(new_meta.headers.clone()),
state: ObjectVersionState::Uploading {
headers: new_meta.headers.clone(),
multipart: false,
},
};
let tmp_dest_object = Object::new(
dest_bucket_id,
@ -105,8 +110,14 @@ pub async fn handle_copy(
// this means that the BlockRef entries linked to this version cannot be
// 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_bucket_id, dest_key.to_string(), false);
let mut dest_version = Version::new(
new_uuid,
VersionBacklink::Object {
bucket_id: dest_bucket_id,
key: dest_key.to_string(),
},
false,
);
garage.version_table.insert(&dest_version).await?;
// Fill in block list for version and insert block refs
@ -179,17 +190,13 @@ pub async fn handle_upload_part_copy(
) -> Result<Response<Body>, Error> {
let copy_precondition = CopyPreconditionHeaders::parse(req)?;
let dest_version_uuid = decode_upload_id(upload_id)?;
let dest_upload_id = multipart::decode_upload_id(upload_id)?;
let dest_key = dest_key.to_string();
let (source_object, dest_object) = futures::try_join!(
let (source_object, (_, _, mut dest_mpu)) = futures::try_join!(
get_copy_source(&garage, api_key, req),
garage
.object_table
.get(&dest_bucket_id, &dest_key)
.map_err(Error::from),
multipart::get_upload(&garage, &dest_bucket_id, &dest_key, &dest_upload_id)
)?;
let dest_object = dest_object.ok_or(Error::NoSuchKey)?;
let (source_object_version, source_version_data, source_version_meta) =
extract_source_info(&source_object)?;
@ -217,15 +224,6 @@ pub async fn handle_upload_part_copy(
},
};
// Check destination version is indeed in uploading state
if !dest_object
.versions()
.iter()
.any(|v| v.uuid == dest_version_uuid && v.is_uploading())
{
return Err(Error::NoSuchUpload);
}
// Check source version is not inlined
match source_version_data {
ObjectVersionData::DeleteMarker => unreachable!(),
@ -242,23 +240,11 @@ pub async fn handle_upload_part_copy(
// Fetch source versin with its block list,
// and destination version to check part hasn't yet been uploaded
let (source_version, dest_version) = futures::try_join!(
garage
.version_table
.get(&source_object_version.uuid, &EmptyKey),
garage.version_table.get(&dest_version_uuid, &EmptyKey),
)?;
let source_version = source_version.ok_or(Error::NoSuchKey)?;
// Check this part number hasn't yet been uploaded
if let Some(dv) = dest_version {
if dv.has_part_number(part_number) {
return Err(Error::bad_request(format!(
"Part number {} has already been uploaded",
part_number
)));
}
}
let source_version = garage
.version_table
.get(&source_object_version.uuid, &EmptyKey)
.await?
.ok_or(Error::NoSuchKey)?;
// We want to reuse blocks from the source version as much as possible.
// However, we still need to get the data from these blocks
@ -299,6 +285,33 @@ pub async fn handle_upload_part_copy(
current_offset = block_end;
}
// Calculate the identity of destination part: timestamp, version id
let dest_version_id = gen_uuid();
let dest_mpu_part_key = MpuPartKey {
part_number,
timestamp: dest_mpu.next_timestamp(part_number),
};
// Create the uploaded part
dest_mpu.parts.clear();
dest_mpu.parts.put(
dest_mpu_part_key,
MpuPart {
version: dest_version_id,
etag: None,
size: None,
},
);
garage.mpu_table.insert(&dest_mpu).await?;
let mut dest_version = Version::new(
dest_version_id,
VersionBacklink::MultipartUpload {
upload_id: dest_upload_id,
},
false,
);
// Now, actually copy the blocks
let mut md5hasher = Md5::new();
@ -348,8 +361,8 @@ pub async fn handle_upload_part_copy(
let must_upload = existing_block_hash.is_none();
let final_hash = existing_block_hash.unwrap_or_else(|| blake2sum(&data[..]));
let mut version = Version::new(dest_version_uuid, dest_bucket_id, dest_key.clone(), false);
version.blocks.put(
dest_version.blocks.clear();
dest_version.blocks.put(
VersionBlockKey {
part_number,
offset: current_offset,
@ -363,7 +376,7 @@ pub async fn handle_upload_part_copy(
let block_ref = BlockRef {
block: final_hash,
version: dest_version_uuid,
version: dest_version_id,
deleted: false.into(),
};
@ -378,23 +391,33 @@ pub async fn handle_upload_part_copy(
Ok(())
}
},
// Thing 2: we need to insert the block in the version
garage.version_table.insert(&version),
// Thing 3: we need to add a block reference
garage.block_ref_table.insert(&block_ref),
async {
// Thing 2: we need to insert the block in the version
garage.version_table.insert(&dest_version).await?;
// Thing 3: we need to add a block reference
garage.block_ref_table.insert(&block_ref).await
},
// Thing 4: we need to prefetch the next block
defragmenter.next(),
)?;
next_block = res.3;
next_block = res.2;
}
assert_eq!(current_offset, source_range.length);
let data_md5sum = md5hasher.finalize();
let etag = hex::encode(data_md5sum);
// Put the part's ETag in the Versiontable
let mut version = Version::new(dest_version_uuid, dest_bucket_id, dest_key.clone(), false);
version.parts_etags.put(part_number, etag.clone());
garage.version_table.insert(&version).await?;
dest_mpu.parts.put(
dest_mpu_part_key,
MpuPart {
version: dest_version_id,
etag: Some(etag.clone()),
size: Some(current_offset),
},
);
garage.mpu_table.insert(&dest_mpu).await?;
// LGTM
let resp_xml = s3_xml::to_xml_with_header(&CopyPartResult {

View File

@ -149,7 +149,6 @@ pub async fn handle_head(
let (part_offset, part_end) =
calculate_part_bounds(&version, pn).ok_or(Error::InvalidPart)?;
let n_parts = version.parts_etags.items().len();
Ok(object_headers(object_version, version_meta)
.header(CONTENT_LENGTH, format!("{}", part_end - part_offset))
@ -162,7 +161,7 @@ pub async fn handle_head(
version_meta.size
),
)
.header(X_AMZ_MP_PARTS_COUNT, format!("{}", n_parts))
.header(X_AMZ_MP_PARTS_COUNT, format!("{}", version.n_parts()?))
.status(StatusCode::PARTIAL_CONTENT)
.body(Body::empty())?)
}
@ -376,7 +375,6 @@ async fn handle_get_part(
let (begin, end) =
calculate_part_bounds(&version, part_number).ok_or(Error::InvalidPart)?;
let n_parts = version.parts_etags.items().len();
let body = body_from_blocks_range(garage, version.blocks.items(), begin, end);
@ -386,7 +384,7 @@ async fn handle_get_part(
CONTENT_RANGE,
format!("bytes {}-{}/{}", begin, end - 1, version_meta.size),
)
.header(X_AMZ_MP_PARTS_COUNT, format!("{}", n_parts))
.header(X_AMZ_MP_PARTS_COUNT, format!("{}", version.n_parts()?))
.body(body)?)
}
_ => unreachable!(),

View File

@ -1,4 +1,3 @@
use std::cmp::Ordering;
use std::collections::{BTreeMap, BTreeSet};
use std::iter::{Iterator, Peekable};
use std::sync::Arc;
@ -11,15 +10,15 @@ use garage_util::error::Error as GarageError;
use garage_util::time::*;
use garage_model::garage::Garage;
use garage_model::s3::mpu_table::*;
use garage_model::s3::object_table::*;
use garage_model::s3::version_table::Version;
use garage_table::{EmptyKey, EnumerationOrder};
use garage_table::EnumerationOrder;
use crate::encoding::*;
use crate::helpers::key_after_prefix;
use crate::s3::error::*;
use crate::s3::put as s3_put;
use crate::s3::multipart as s3_multipart;
use crate::s3::xml as s3_xml;
const DUMMY_NAME: &str = "Dummy Key";
@ -176,7 +175,9 @@ pub async fn handle_list_multipart_upload(
t.get_range(
&bucket,
key,
Some(ObjectFilter::IsUploading),
Some(ObjectFilter::IsUploading {
check_multipart: Some(true),
}),
count,
EnumerationOrder::Forward,
)
@ -272,23 +273,23 @@ pub async fn handle_list_parts(
) -> Result<Response<Body>, Error> {
debug!("ListParts {:?}", query);
let upload_id = s3_put::decode_upload_id(&query.upload_id)?;
let upload_id = s3_multipart::decode_upload_id(&query.upload_id)?;
let (object, version) = futures::try_join!(
garage.object_table.get(&query.bucket_id, &query.key),
garage.version_table.get(&upload_id, &EmptyKey),
)?;
let (_, _, mpu) =
s3_multipart::get_upload(&garage, &query.bucket_id, &query.key, &upload_id).await?;
let (info, next) = fetch_part_info(query, object, version, upload_id)?;
let (info, next) = fetch_part_info(query, &mpu)?;
let result = s3_xml::ListPartsResult {
xmlns: (),
// Query parameters
bucket: s3_xml::Value(query.bucket_name.to_string()),
key: s3_xml::Value(query.key.to_string()),
upload_id: s3_xml::Value(query.upload_id.to_string()),
part_number_marker: query.part_number_marker.map(|e| s3_xml::IntValue(e as i64)),
next_part_number_marker: next.map(|e| s3_xml::IntValue(e as i64)),
max_parts: s3_xml::IntValue(query.max_parts as i64),
// Result values
next_part_number_marker: next.map(|e| s3_xml::IntValue(e as i64)),
is_truncated: s3_xml::Value(next.map(|_| "true").unwrap_or("false").to_string()),
parts: info
.iter()
@ -299,6 +300,7 @@ pub async fn handle_list_parts(
size: s3_xml::IntValue(part.size as i64),
})
.collect(),
// Dummy result values (unsupported features)
initiator: s3_xml::Initiator {
display_name: s3_xml::Value(DUMMY_NAME.to_string()),
id: s3_xml::Value(DUMMY_KEY.to_string()),
@ -335,8 +337,8 @@ struct UploadInfo {
}
#[derive(Debug, PartialEq)]
struct PartInfo {
etag: String,
struct PartInfo<'a> {
etag: &'a str,
timestamp: u64,
part_number: u64,
size: u64,
@ -456,106 +458,52 @@ where
}
}
fn fetch_part_info(
fn fetch_part_info<'a>(
query: &ListPartsQuery,
object: Option<Object>,
version: Option<Version>,
upload_id: Uuid,
) -> Result<(Vec<PartInfo>, Option<u64>), Error> {
// Check results
let object = object.ok_or(Error::NoSuchKey)?;
let obj_version = object
.versions()
.iter()
.find(|v| v.uuid == upload_id && v.is_uploading())
.ok_or(Error::NoSuchUpload)?;
let version = version.ok_or(Error::NoSuchKey)?;
// Cut the beginning of our 2 vectors if required
let (etags, blocks) = match &query.part_number_marker {
Some(marker) => {
let next = marker + 1;
let part_idx = into_ok_or_err(
version
.parts_etags
.items()
.binary_search_by(|(part_num, _)| part_num.cmp(&next)),
);
let parts = &version.parts_etags.items()[part_idx..];
let block_idx = into_ok_or_err(
version
.blocks
.items()
.binary_search_by(|(vkey, _)| vkey.part_number.cmp(&next)),
);
let blocks = &version.blocks.items()[block_idx..];
(parts, blocks)
}
None => (version.parts_etags.items(), version.blocks.items()),
};
// Use the block vector to compute a (part_number, size) vector
let mut size = Vec::<(u64, u64)>::new();
blocks.iter().for_each(|(key, val)| {
let mut new_size = val.size;
match size.pop() {
Some((part_number, size)) if part_number == key.part_number => new_size += size,
Some(v) => size.push(v),
None => (),
}
size.push((key.part_number, new_size))
});
// Merge the etag vector and size vector to build a PartInfo vector
let max_parts = query.max_parts as usize;
let (mut etag_iter, mut size_iter) = (etags.iter().peekable(), size.iter().peekable());
let mut info = Vec::<PartInfo>::with_capacity(max_parts);
while info.len() < max_parts {
match (etag_iter.peek(), size_iter.peek()) {
(Some((ep, etag)), Some((sp, size))) => match ep.cmp(sp) {
Ordering::Less => {
debug!("ETag information ignored due to missing corresponding block information. Query: {:?}", query);
etag_iter.next();
mpu: &'a MultipartUpload,
) -> Result<(Vec<PartInfo<'a>>, Option<u64>), Error> {
// Parse multipart upload part list, removing parts not yet finished
// and failed part uploads that were overwritten
let mut parts: Vec<PartInfo<'a>> = Vec::with_capacity(mpu.parts.items().len());
for (pk, p) in mpu.parts.items().iter() {
if let (Some(etag), Some(size)) = (&p.etag, p.size) {
let part_info = PartInfo {
part_number: pk.part_number,
timestamp: pk.timestamp,
etag,
size,
};
match parts.last_mut() {
Some(lastpart) if lastpart.part_number == pk.part_number => {
*lastpart = part_info;
}
Ordering::Equal => {
info.push(PartInfo {
etag: etag.to_string(),
timestamp: obj_version.timestamp,
part_number: *ep,
size: *size,
});
etag_iter.next();
size_iter.next();
_ => {
parts.push(part_info);
}
Ordering::Greater => {
debug!("Block information ignored due to missing corresponding ETag information. Query: {:?}", query);
size_iter.next();
}
},
(None, None) => return Ok((info, None)),
_ => {
debug!(
"Additional block or ETag information ignored. Query: {:?}",
query
);
return Ok((info, None));
}
}
}
match info.last() {
// Cut the beginning and end
match &query.part_number_marker {
Some(marker) => {
let next = marker + 1;
let part_idx =
into_ok_or_err(parts.binary_search_by(|part| part.part_number.cmp(&next)));
parts.truncate(part_idx + query.max_parts as usize);
parts = parts.split_off(part_idx);
}
None => {
parts.truncate(query.max_parts as usize);
}
};
match parts.last() {
Some(part_info) => {
let pagination = Some(part_info.part_number);
Ok((info, pagination))
Ok((parts, pagination))
}
None => Ok((info, None)),
None => Ok((parts, None)),
}
}
@ -651,7 +599,7 @@ impl ListMultipartUploadsQuery {
}),
uuid => Ok(RangeBegin::AfterUpload {
key: key_marker.to_string(),
upload: s3_put::decode_upload_id(uuid)?,
upload: s3_multipart::decode_upload_id(uuid)?,
}),
},
@ -843,7 +791,7 @@ impl ExtractAccumulator for UploadAccumulator {
let mut uploads_for_key = object
.versions()
.iter()
.filter(|x| x.is_uploading())
.filter(|x| x.is_uploading(Some(true)))
.collect::<Vec<&ObjectVersion>>();
// S3 logic requires lexicographically sorted upload ids.
@ -991,10 +939,13 @@ mod tests {
ObjectVersion {
uuid: Uuid::from(uuid),
timestamp: TS,
state: ObjectVersionState::Uploading(ObjectVersionHeaders {
content_type: "text/plain".to_string(),
other: BTreeMap::<String, String>::new(),
}),
state: ObjectVersionState::Uploading {
multipart: true,
headers: ObjectVersionHeaders {
content_type: "text/plain".to_string(),
other: BTreeMap::<String, String>::new(),
},
},
}
}
@ -1233,11 +1184,13 @@ mod tests {
];
Version {
bucket_id: uuid,
key: "a".to_string(),
uuid,
deleted: false.into(),
blocks: crdt::Map::<VersionBlockKey, VersionBlock>::from_iter(blocks),
backlink: VersionBacklink::Object {
bucket_id: uuid,
key: "a".to_string(),
},
parts_etags: crdt::Map::<u64, String>::from_iter(etags),
}
}

View File

@ -7,6 +7,7 @@ pub mod cors;
mod delete;
pub mod get;
mod list;
mod multipart;
mod post_object;
mod put;
mod website;

422
src/api/s3/multipart.rs Normal file
View File

@ -0,0 +1,422 @@
use std::collections::HashMap;
use std::sync::Arc;
use futures::prelude::*;
use hyper::body::Body;
use hyper::{Request, Response};
use md5::{Digest as Md5Digest, Md5};
use garage_table::*;
use garage_util::async_hash::*;
use garage_util::data::*;
use garage_util::time::*;
use garage_model::bucket_table::Bucket;
use garage_model::garage::Garage;
use garage_model::s3::block_ref_table::*;
use garage_model::s3::mpu_table::*;
use garage_model::s3::object_table::*;
use garage_model::s3::version_table::*;
use crate::s3::error::*;
use crate::s3::put::*;
use crate::s3::xml as s3_xml;
use crate::signature::verify_signed_content;
// ----
pub async fn handle_create_multipart_upload(
garage: Arc<Garage>,
req: &Request<Body>,
bucket_name: &str,
bucket_id: Uuid,
key: &str,
) -> Result<Response<Body>, Error> {
let upload_id = gen_uuid();
let headers = get_headers(req.headers())?;
// Create object in object table
let object_version = ObjectVersion {
uuid: upload_id,
timestamp: now_msec(),
state: ObjectVersionState::Uploading {
multipart: true,
headers,
},
};
let object = Object::new(bucket_id, key.to_string(), vec![object_version]);
garage.object_table.insert(&object).await?;
// Create multipart upload in mpu table
// This multipart upload will hold references to uploaded parts
// (which are entries in the Version table)
let mpu = MultipartUpload::new(upload_id, bucket_id, key.into(), false);
garage.mpu_table.insert(&mpu).await?;
// Send success response
let result = s3_xml::InitiateMultipartUploadResult {
xmlns: (),
bucket: s3_xml::Value(bucket_name.to_string()),
key: s3_xml::Value(key.to_string()),
upload_id: s3_xml::Value(hex::encode(upload_id)),
};
let xml = s3_xml::to_xml_with_header(&result)?;
Ok(Response::new(Body::from(xml.into_bytes())))
}
pub async fn handle_put_part(
garage: Arc<Garage>,
req: Request<Body>,
bucket_id: Uuid,
key: &str,
part_number: u64,
upload_id: &str,
content_sha256: Option<Hash>,
) -> Result<Response<Body>, Error> {
let upload_id = decode_upload_id(upload_id)?;
let content_md5 = match req.headers().get("content-md5") {
Some(x) => Some(x.to_str()?.to_string()),
None => None,
};
// Read first chuck, and at the same time try to get object to see if it exists
let key = key.to_string();
let body = req.into_body().map_err(Error::from);
let mut chunker = StreamChunker::new(body, garage.config.block_size);
let ((_, _, mut mpu), first_block) = futures::try_join!(
get_upload(&garage, &bucket_id, &key, &upload_id),
chunker.next(),
)?;
// Check object is valid and part can be accepted
let first_block = first_block.ok_or_bad_request("Empty body")?;
// Calculate part identity: timestamp, version id
let version_id = gen_uuid();
let mpu_part_key = MpuPartKey {
part_number,
timestamp: mpu.next_timestamp(part_number),
};
// Create version and link version from MPU
mpu.parts.clear();
mpu.parts.put(
mpu_part_key,
MpuPart {
version: version_id,
etag: None,
size: None,
},
);
garage.mpu_table.insert(&mpu).await?;
let version = Version::new(
version_id,
VersionBacklink::MultipartUpload { upload_id },
false,
);
garage.version_table.insert(&version).await?;
// Copy data to version
let first_block_hash = async_blake2sum(first_block.clone()).await;
let (total_size, data_md5sum, data_sha256sum) = read_and_put_blocks(
&garage,
&version,
part_number,
first_block,
first_block_hash,
&mut chunker,
)
.await?;
// Verify that checksums map
ensure_checksum_matches(
data_md5sum.as_slice(),
data_sha256sum,
content_md5.as_deref(),
content_sha256,
)?;
// Store part etag in version
let data_md5sum_hex = hex::encode(data_md5sum);
mpu.parts.put(
mpu_part_key,
MpuPart {
version: version_id,
etag: Some(data_md5sum_hex.clone()),
size: Some(total_size),
},
);
garage.mpu_table.insert(&mpu).await?;
let response = Response::builder()
.header("ETag", format!("\"{}\"", data_md5sum_hex))
.body(Body::empty())
.unwrap();
Ok(response)
}
pub async fn handle_complete_multipart_upload(
garage: Arc<Garage>,
req: Request<Body>,
bucket_name: &str,
bucket: &Bucket,
key: &str,
upload_id: &str,
content_sha256: Option<Hash>,
) -> Result<Response<Body>, Error> {
let body = hyper::body::to_bytes(req.into_body()).await?;
if let Some(content_sha256) = content_sha256 {
verify_signed_content(content_sha256, &body[..])?;
}
let body_xml = roxmltree::Document::parse(std::str::from_utf8(&body)?)?;
let body_list_of_parts = parse_complete_multipart_upload_body(&body_xml)
.ok_or_bad_request("Invalid CompleteMultipartUpload XML")?;
debug!(
"CompleteMultipartUpload list of parts: {:?}",
body_list_of_parts
);
let upload_id = decode_upload_id(upload_id)?;
// Get object and multipart upload
let key = key.to_string();
let (_, mut object_version, mpu) = get_upload(&garage, &bucket.id, &key, &upload_id).await?;
if mpu.parts.is_empty() {
return Err(Error::bad_request("No data was uploaded"));
}
let headers = match object_version.state {
ObjectVersionState::Uploading { headers, .. } => headers,
_ => unreachable!(),
};
// Check that part numbers are an increasing sequence.
// (it doesn't need to start at 1 nor to be a continuous sequence,
// see discussion in #192)
if body_list_of_parts.is_empty() {
return Err(Error::EntityTooSmall);
}
if !body_list_of_parts
.iter()
.zip(body_list_of_parts.iter().skip(1))
.all(|(p1, p2)| p1.part_number < p2.part_number)
{
return Err(Error::InvalidPartOrder);
}
// Check that the list of parts they gave us corresponds to parts we have here
debug!("Parts stored in multipart upload: {:?}", mpu.parts.items());
let mut have_parts = HashMap::new();
for (pk, pv) in mpu.parts.items().iter() {
have_parts.insert(pk.part_number, pv);
}
let mut parts = vec![];
for req_part in body_list_of_parts.iter() {
match have_parts.get(&req_part.part_number) {
Some(part) if part.etag.as_ref() == Some(&req_part.etag) && part.size.is_some() => {
parts.push(*part)
}
_ => return Err(Error::InvalidPart),
}
}
let grg = &garage;
let parts_versions = futures::future::try_join_all(parts.iter().map(|p| async move {
grg.version_table
.get(&p.version, &EmptyKey)
.await?
.ok_or_internal_error("Part version missing from version table")
}))
.await?;
// Create final version and block refs
let mut final_version = Version::new(
upload_id,
VersionBacklink::Object {
bucket_id: bucket.id,
key: key.to_string(),
},
false,
);
for (part_number, part_version) in parts_versions.iter().enumerate() {
if part_version.deleted.get() {
return Err(Error::InvalidPart);
}
for (vbk, vb) in part_version.blocks.items().iter() {
final_version.blocks.put(
VersionBlockKey {
part_number: part_number as u64,
offset: vbk.offset,
},
*vb,
);
}
}
garage.version_table.insert(&final_version).await?;
let block_refs = final_version.blocks.items().iter().map(|(_, b)| BlockRef {
block: b.hash,
version: upload_id,
deleted: false.into(),
});
garage.block_ref_table.insert_many(block_refs).await?;
// Calculate etag of final object
// To understand how etags are calculated, read more here:
// https://teppen.io/2018/06/23/aws_s3_etags/
let mut etag_md5_hasher = Md5::new();
for part in parts.iter() {
etag_md5_hasher.update(part.etag.as_ref().unwrap().as_bytes());
}
let etag = format!(
"{}-{}",
hex::encode(etag_md5_hasher.finalize()),
parts.len()
);
// Calculate total size of final object
let total_size = parts.iter().map(|x| x.size.unwrap()).sum();
if let Err(e) = check_quotas(&garage, bucket, &key, total_size).await {
object_version.state = ObjectVersionState::Aborted;
let final_object = Object::new(bucket.id, key.clone(), vec![object_version]);
garage.object_table.insert(&final_object).await?;
return Err(e);
}
// Write final object version
object_version.state = ObjectVersionState::Complete(ObjectVersionData::FirstBlock(
ObjectVersionMeta {
headers,
size: total_size,
etag: etag.clone(),
},
final_version.blocks.items()[0].1.hash,
));
let final_object = Object::new(bucket.id, key.clone(), vec![object_version]);
garage.object_table.insert(&final_object).await?;
// Send response saying ok we're done
let result = s3_xml::CompleteMultipartUploadResult {
xmlns: (),
location: None,
bucket: s3_xml::Value(bucket_name.to_string()),
key: s3_xml::Value(key),
etag: s3_xml::Value(format!("\"{}\"", etag)),
};
let xml = s3_xml::to_xml_with_header(&result)?;
Ok(Response::new(Body::from(xml.into_bytes())))
}
pub async fn handle_abort_multipart_upload(
garage: Arc<Garage>,
bucket_id: Uuid,
key: &str,
upload_id: &str,
) -> Result<Response<Body>, Error> {
let upload_id = decode_upload_id(upload_id)?;
let (_, mut object_version, _) =
get_upload(&garage, &bucket_id, &key.to_string(), &upload_id).await?;
object_version.state = ObjectVersionState::Aborted;
let final_object = Object::new(bucket_id, key.to_string(), vec![object_version]);
garage.object_table.insert(&final_object).await?;
Ok(Response::new(Body::from(vec![])))
}
// ======== helpers ============
pub(crate) async fn get_upload(
garage: &Garage,
bucket_id: &Uuid,
key: &String,
upload_id: &Uuid,
) -> Result<(Object, ObjectVersion, MultipartUpload), Error> {
let (object, mpu) = futures::try_join!(
garage
.object_table
.get(&bucket_id, &key)
.map_err(Error::from),
garage
.mpu_table
.get(&upload_id, &EmptyKey)
.map_err(Error::from),
)?;
let object = object.ok_or(Error::NoSuchUpload)?;
let mpu = mpu.ok_or(Error::NoSuchUpload)?;
let object_version = object
.versions()
.iter()
.find(|v| v.uuid == *upload_id && v.is_uploading(Some(true)))
.ok_or(Error::NoSuchUpload)?
.clone();
Ok((object, object_version, mpu))
}
pub fn decode_upload_id(id: &str) -> Result<Uuid, Error> {
let id_bin = hex::decode(id).map_err(|_| Error::NoSuchUpload)?;
if id_bin.len() != 32 {
return Err(Error::NoSuchUpload);
}
let mut uuid = [0u8; 32];
uuid.copy_from_slice(&id_bin[..]);
Ok(Uuid::from(uuid))
}
#[derive(Debug)]
struct CompleteMultipartUploadPart {
etag: String,
part_number: u64,
}
fn parse_complete_multipart_upload_body(
xml: &roxmltree::Document,
) -> Option<Vec<CompleteMultipartUploadPart>> {
let mut parts = vec![];
let root = xml.root();
let cmu = root.first_child()?;
if !cmu.has_tag_name("CompleteMultipartUpload") {
return None;
}
for item in cmu.children() {
// Only parse <Part> nodes
if !item.is_element() {
continue;
}
if item.has_tag_name("Part") {
let etag = item.children().find(|e| e.has_tag_name("ETag"))?.text()?;
let part_number = item
.children()
.find(|e| e.has_tag_name("PartNumber"))?
.text()?;
parts.push(CompleteMultipartUploadPart {
etag: etag.trim_matches('"').to_string(),
part_number: part_number.parse().ok()?,
});
} else {
return None;
}
}
Some(parts)
}

View File

@ -1,4 +1,4 @@
use std::collections::{BTreeMap, BTreeSet, HashMap};
use std::collections::{BTreeMap, HashMap};
use std::sync::Arc;
use base64::prelude::*;
@ -30,8 +30,6 @@ use garage_model::s3::object_table::*;
use garage_model::s3::version_table::*;
use crate::s3::error::*;
use crate::s3::xml as s3_xml;
use crate::signature::verify_signed_content;
pub async fn handle_put(
garage: Arc<Garage>,
@ -136,7 +134,10 @@ pub(crate) async fn save_stream<S: Stream<Item = Result<Bytes, Error>> + Unpin>(
let mut object_version = ObjectVersion {
uuid: version_uuid,
timestamp: version_timestamp,
state: ObjectVersionState::Uploading(headers.clone()),
state: ObjectVersionState::Uploading {
headers: headers.clone(),
multipart: false,
},
};
let object = Object::new(bucket.id, key.into(), vec![object_version.clone()]);
garage.object_table.insert(&object).await?;
@ -145,7 +146,14 @@ pub(crate) async fn save_stream<S: Stream<Item = Result<Bytes, Error>> + Unpin>(
// Write this entry now, even with empty block list,
// to prevent block_ref entries from being deleted (they can be deleted
// if the reference a version that isn't found in the version table)
let version = Version::new(version_uuid, bucket.id, key.into(), false);
let version = Version::new(
version_uuid,
VersionBacklink::Object {
bucket_id: bucket.id,
key: key.into(),
},
false,
);
garage.version_table.insert(&version).await?;
// Transfer data and verify checksum
@ -192,7 +200,7 @@ pub(crate) async fn save_stream<S: Stream<Item = Result<Bytes, Error>> + Unpin>(
/// Validate MD5 sum against content-md5 header
/// and sha256sum against signed content-sha256
fn ensure_checksum_matches(
pub(crate) fn ensure_checksum_matches(
data_md5sum: &[u8],
data_sha256sum: garage_util::data::FixedBytes32,
content_md5: Option<&str>,
@ -218,7 +226,7 @@ fn ensure_checksum_matches(
}
/// Check that inserting this object with this size doesn't exceed bucket quotas
async fn check_quotas(
pub(crate) async fn check_quotas(
garage: &Arc<Garage>,
bucket: &Bucket,
key: &str,
@ -275,7 +283,7 @@ async fn check_quotas(
Ok(())
}
async fn read_and_put_blocks<S: Stream<Item = Result<Bytes, Error>> + Unpin>(
pub(crate) async fn read_and_put_blocks<S: Stream<Item = Result<Bytes, Error>> + Unpin>(
garage: &Garage,
version: &Version,
part_number: u64,
@ -381,7 +389,7 @@ async fn put_block_meta(
Ok(())
}
struct StreamChunker<S: Stream<Item = Result<Bytes, Error>>> {
pub(crate) struct StreamChunker<S: Stream<Item = Result<Bytes, Error>>> {
stream: S,
read_all: bool,
block_size: usize,
@ -389,7 +397,7 @@ struct StreamChunker<S: Stream<Item = Result<Bytes, Error>>> {
}
impl<S: Stream<Item = Result<Bytes, Error>> + Unpin> StreamChunker<S> {
fn new(stream: S, block_size: usize) -> Self {
pub(crate) fn new(stream: S, block_size: usize) -> Self {
Self {
stream,
read_all: false,
@ -398,7 +406,7 @@ impl<S: Stream<Item = Result<Bytes, Error>> + Unpin> StreamChunker<S> {
}
}
async fn next(&mut self) -> Result<Option<Bytes>, Error> {
pub(crate) async fn next(&mut self) -> Result<Option<Bytes>, Error> {
while !self.read_all && self.buf.len() < self.block_size {
if let Some(block) = self.stream.next().await {
let bytes = block?;
@ -450,326 +458,9 @@ impl Drop for InterruptedCleanup {
}
}
// ----
// ============ helpers ============
pub async fn handle_create_multipart_upload(
garage: Arc<Garage>,
req: &Request<Body>,
bucket_name: &str,
bucket_id: Uuid,
key: &str,
) -> Result<Response<Body>, Error> {
let version_uuid = gen_uuid();
let headers = get_headers(req.headers())?;
// Create object in object table
let object_version = ObjectVersion {
uuid: version_uuid,
timestamp: now_msec(),
state: ObjectVersionState::Uploading(headers),
};
let object = Object::new(bucket_id, key.to_string(), vec![object_version]);
garage.object_table.insert(&object).await?;
// Insert empty version so that block_ref entries refer to something
// (they are inserted concurrently with blocks in the version table, so
// there is the possibility that they are inserted before the version table
// is created, in which case it is allowed to delete them, e.g. in repair_*)
let version = Version::new(version_uuid, bucket_id, key.into(), false);
garage.version_table.insert(&version).await?;
// Send success response
let result = s3_xml::InitiateMultipartUploadResult {
xmlns: (),
bucket: s3_xml::Value(bucket_name.to_string()),
key: s3_xml::Value(key.to_string()),
upload_id: s3_xml::Value(hex::encode(version_uuid)),
};
let xml = s3_xml::to_xml_with_header(&result)?;
Ok(Response::new(Body::from(xml.into_bytes())))
}
pub async fn handle_put_part(
garage: Arc<Garage>,
req: Request<Body>,
bucket_id: Uuid,
key: &str,
part_number: u64,
upload_id: &str,
content_sha256: Option<Hash>,
) -> Result<Response<Body>, Error> {
let version_uuid = decode_upload_id(upload_id)?;
let content_md5 = match req.headers().get("content-md5") {
Some(x) => Some(x.to_str()?.to_string()),
None => None,
};
// Read first chuck, and at the same time try to get object to see if it exists
let key = key.to_string();
let body = req.into_body().map_err(Error::from);
let mut chunker = StreamChunker::new(body, garage.config.block_size);
let (object, version, first_block) = futures::try_join!(
garage
.object_table
.get(&bucket_id, &key)
.map_err(Error::from),
garage
.version_table
.get(&version_uuid, &EmptyKey)
.map_err(Error::from),
chunker.next(),
)?;
// Check object is valid and multipart block can be accepted
let first_block = first_block.ok_or_bad_request("Empty body")?;
let object = object.ok_or_bad_request("Object not found")?;
if !object
.versions()
.iter()
.any(|v| v.uuid == version_uuid && v.is_uploading())
{
return Err(Error::NoSuchUpload);
}
// Check part hasn't already been uploaded
if let Some(v) = version {
if v.has_part_number(part_number) {
return Err(Error::bad_request(format!(
"Part number {} has already been uploaded",
part_number
)));
}
}
// Copy block to store
let version = Version::new(version_uuid, bucket_id, key, false);
let first_block_hash = async_blake2sum(first_block.clone()).await;
let (_, data_md5sum, data_sha256sum) = read_and_put_blocks(
&garage,
&version,
part_number,
first_block,
first_block_hash,
&mut chunker,
)
.await?;
// Verify that checksums map
ensure_checksum_matches(
data_md5sum.as_slice(),
data_sha256sum,
content_md5.as_deref(),
content_sha256,
)?;
// Store part etag in version
let data_md5sum_hex = hex::encode(data_md5sum);
let mut version = version;
version
.parts_etags
.put(part_number, data_md5sum_hex.clone());
garage.version_table.insert(&version).await?;
let response = Response::builder()
.header("ETag", format!("\"{}\"", data_md5sum_hex))
.body(Body::empty())
.unwrap();
Ok(response)
}
pub async fn handle_complete_multipart_upload(
garage: Arc<Garage>,
req: Request<Body>,
bucket_name: &str,
bucket: &Bucket,
key: &str,
upload_id: &str,
content_sha256: Option<Hash>,
) -> Result<Response<Body>, Error> {
let body = hyper::body::to_bytes(req.into_body()).await?;
if let Some(content_sha256) = content_sha256 {
verify_signed_content(content_sha256, &body[..])?;
}
let body_xml = roxmltree::Document::parse(std::str::from_utf8(&body)?)?;
let body_list_of_parts = parse_complete_multipart_upload_body(&body_xml)
.ok_or_bad_request("Invalid CompleteMultipartUpload XML")?;
debug!(
"CompleteMultipartUpload list of parts: {:?}",
body_list_of_parts
);
let version_uuid = decode_upload_id(upload_id)?;
// Get object and version
let key = key.to_string();
let (object, version) = futures::try_join!(
garage.object_table.get(&bucket.id, &key),
garage.version_table.get(&version_uuid, &EmptyKey),
)?;
let object = object.ok_or(Error::NoSuchKey)?;
let mut object_version = object
.versions()
.iter()
.find(|v| v.uuid == version_uuid && v.is_uploading())
.cloned()
.ok_or(Error::NoSuchUpload)?;
let version = version.ok_or(Error::NoSuchKey)?;
if version.blocks.is_empty() {
return Err(Error::bad_request("No data was uploaded"));
}
let headers = match object_version.state {
ObjectVersionState::Uploading(headers) => headers,
_ => unreachable!(),
};
// Check that part numbers are an increasing sequence.
// (it doesn't need to start at 1 nor to be a continuous sequence,
// see discussion in #192)
if body_list_of_parts.is_empty() {
return Err(Error::EntityTooSmall);
}
if !body_list_of_parts
.iter()
.zip(body_list_of_parts.iter().skip(1))
.all(|(p1, p2)| p1.part_number < p2.part_number)
{
return Err(Error::InvalidPartOrder);
}
// Garage-specific restriction, see #204: part numbers must be
// consecutive starting at 1
if body_list_of_parts[0].part_number != 1
|| !body_list_of_parts
.iter()
.zip(body_list_of_parts.iter().skip(1))
.all(|(p1, p2)| p1.part_number + 1 == p2.part_number)
{
return Err(Error::NotImplemented("Garage does not support completing a Multipart upload with non-consecutive part numbers. This is a restriction of Garage's data model, which might be fixed in a future release. See issue #204 for more information on this topic.".into()));
}
// Check that the list of parts they gave us corresponds to the parts we have here
debug!("Expected parts from request: {:?}", body_list_of_parts);
debug!("Parts stored in version: {:?}", version.parts_etags.items());
let parts = version
.parts_etags
.items()
.iter()
.map(|pair| (&pair.0, &pair.1));
let same_parts = body_list_of_parts
.iter()
.map(|x| (&x.part_number, &x.etag))
.eq(parts);
if !same_parts {
return Err(Error::InvalidPart);
}
// Check that all blocks belong to one of the parts
let block_parts = version
.blocks
.items()
.iter()
.map(|(bk, _)| bk.part_number)
.collect::<BTreeSet<_>>();
let same_parts = body_list_of_parts
.iter()
.map(|x| x.part_number)
.eq(block_parts.into_iter());
if !same_parts {
return Err(Error::bad_request(
"Part numbers in block list and part list do not match. This can happen if a part was partially uploaded. Please abort the multipart upload and try again."
));
}
// Calculate etag of final object
// To understand how etags are calculated, read more here:
// https://teppen.io/2018/06/23/aws_s3_etags/
let num_parts = body_list_of_parts.len();
let mut etag_md5_hasher = Md5::new();
for (_, etag) in version.parts_etags.items().iter() {
etag_md5_hasher.update(etag.as_bytes());
}
let etag = format!("{}-{}", hex::encode(etag_md5_hasher.finalize()), num_parts);
// Calculate total size of final object
let total_size = version.blocks.items().iter().map(|x| x.1.size).sum();
if let Err(e) = check_quotas(&garage, bucket, &key, total_size).await {
object_version.state = ObjectVersionState::Aborted;
let final_object = Object::new(bucket.id, key.clone(), vec![object_version]);
garage.object_table.insert(&final_object).await?;
return Err(e);
}
// Write final object version
object_version.state = ObjectVersionState::Complete(ObjectVersionData::FirstBlock(
ObjectVersionMeta {
headers,
size: total_size,
etag: etag.clone(),
},
version.blocks.items()[0].1.hash,
));
let final_object = Object::new(bucket.id, key.clone(), vec![object_version]);
garage.object_table.insert(&final_object).await?;
// Send response saying ok we're done
let result = s3_xml::CompleteMultipartUploadResult {
xmlns: (),
location: None,
bucket: s3_xml::Value(bucket_name.to_string()),
key: s3_xml::Value(key),
etag: s3_xml::Value(format!("\"{}\"", etag)),
};
let xml = s3_xml::to_xml_with_header(&result)?;
Ok(Response::new(Body::from(xml.into_bytes())))
}
pub async fn handle_abort_multipart_upload(
garage: Arc<Garage>,
bucket_id: Uuid,
key: &str,
upload_id: &str,
) -> Result<Response<Body>, Error> {
let version_uuid = decode_upload_id(upload_id)?;
let object = garage
.object_table
.get(&bucket_id, &key.to_string())
.await?;
let object = object.ok_or(Error::NoSuchKey)?;
let object_version = object
.versions()
.iter()
.find(|v| v.uuid == version_uuid && v.is_uploading());
let mut object_version = match object_version {
None => return Err(Error::NoSuchUpload),
Some(x) => x.clone(),
};
object_version.state = ObjectVersionState::Aborted;
let final_object = Object::new(bucket_id, key.to_string(), vec![object_version]);
garage.object_table.insert(&final_object).await?;
Ok(Response::new(Body::from(vec![])))
}
fn get_mime_type(headers: &HeaderMap<HeaderValue>) -> Result<String, Error> {
pub(crate) fn get_mime_type(headers: &HeaderMap<HeaderValue>) -> Result<String, Error> {
Ok(headers
.get(hyper::header::CONTENT_TYPE)
.map(|x| x.to_str())
@ -821,54 +512,3 @@ pub(crate) fn get_headers(headers: &HeaderMap<HeaderValue>) -> Result<ObjectVers
other,
})
}
pub fn decode_upload_id(id: &str) -> Result<Uuid, Error> {
let id_bin = hex::decode(id).map_err(|_| Error::NoSuchUpload)?;
if id_bin.len() != 32 {
return Err(Error::NoSuchUpload);
}
let mut uuid = [0u8; 32];
uuid.copy_from_slice(&id_bin[..]);
Ok(Uuid::from(uuid))
}
#[derive(Debug)]
struct CompleteMultipartUploadPart {
etag: String,
part_number: u64,
}
fn parse_complete_multipart_upload_body(
xml: &roxmltree::Document,
) -> Option<Vec<CompleteMultipartUploadPart>> {
let mut parts = vec![];
let root = xml.root();
let cmu = root.first_child()?;
if !cmu.has_tag_name("CompleteMultipartUpload") {
return None;
}
for item in cmu.children() {
// Only parse <Part> nodes
if !item.is_element() {
continue;
}
if item.has_tag_name("Part") {
let etag = item.children().find(|e| e.has_tag_name("ETag"))?.text()?;
let part_number = item
.children()
.find(|e| e.has_tag_name("PartNumber"))?
.text()?;
parts.push(CompleteMultipartUploadPart {
etag: etag.trim_matches('"').to_string(),
part_number: part_number.parse().ok()?,
});
} else {
return None;
}
}
Some(parts)
}

View File

@ -3,6 +3,7 @@ use std::sync::Arc;
use garage_db as db;
use garage_util::data::*;
use garage_util::time::*;
use garage_table::crdt::*;
use garage_table::replication::TableShardedReplication;
@ -94,6 +95,20 @@ impl MultipartUpload {
key,
}
}
pub fn next_timestamp(&self, part_number: u64) -> u64 {
std::cmp::max(
now_msec(),
1 + self
.parts
.items()
.iter()
.filter(|(x, _)| x.part_number == part_number)
.map(|(x, _)| x.timestamp)
.max()
.unwrap_or(0),
)
}
}
impl Entry<Uuid, EmptyKey> for MultipartUpload {

View File

@ -3,6 +3,7 @@ use std::sync::Arc;
use garage_db as db;
use garage_util::data::*;
use garage_util::error::*;
use garage_table::crdt::*;
use garage_table::replication::TableShardedReplication;
@ -188,6 +189,16 @@ impl Version {
.binary_search_by(|(k, _)| k.part_number.cmp(&part_number))
.is_ok()
}
pub fn n_parts(&self) -> Result<u64, Error> {
Ok(self
.blocks
.items()
.last()
.ok_or_message("version has no parts")?
.0
.part_number)
}
}
impl Ord for VersionBlockKey {