Implement multipart uploads

This commit is contained in:
Alex 2020-04-26 20:39:32 +00:00
parent 1999c0ae51
commit 81ecc4999e
9 changed files with 296 additions and 59 deletions

View File

@ -1,3 +1,3 @@
all: all:
cargo fmt || true #cargo fmt || true
RUSTFLAGS="-C link-arg=-fuse-ld=lld" cargo build RUSTFLAGS="-C link-arg=-fuse-ld=lld" cargo build

3
TODO
View File

@ -8,7 +8,8 @@ We will have to introduce lots of dummy data and then add/remove nodes many time
Attaining S3 compatibility Attaining S3 compatibility
-------------------------- --------------------------
- multipart uploads - test & fix multipart uploads
- abort multipart upload
- fix sync not working in some cases ? (when starting from empty?) - fix sync not working in some cases ? (when starting from empty?)
- api_server following the S3 semantics for head/get/put/list/delete: verify more that it works as intended - api_server following the S3 semantics for head/get/put/list/delete: verify more that it works as intended

View File

@ -3,7 +3,6 @@ use std::net::SocketAddr;
use std::sync::Arc; use std::sync::Arc;
use futures::future::Future; use futures::future::Future;
use hyper::body::{Bytes, HttpBody};
use hyper::server::conn::AddrStream; use hyper::server::conn::AddrStream;
use hyper::service::{make_service_fn, service_fn}; use hyper::service::{make_service_fn, service_fn};
use hyper::{Body, Method, Request, Response, Server}; use hyper::{Body, Method, Request, Response, Server};
@ -15,11 +14,9 @@ use garage_core::garage::Garage;
use crate::http_util::*; use crate::http_util::*;
use crate::signature::check_signature; use crate::signature::check_signature;
use crate::s3_get::{handle_get, handle_head}; use crate::s3_get::*;
use crate::s3_list::handle_list; use crate::s3_list::*;
use crate::s3_put::{handle_delete, handle_put}; use crate::s3_put::*;
pub type BodyType = Box<dyn HttpBody<Data = Bytes, Error = Error> + Send + Unpin>;
pub async fn run_api_server( pub async fn run_api_server(
garage: Arc<Garage>, garage: Arc<Garage>,
@ -100,33 +97,62 @@ async fn handler_inner(
))); )));
} }
let mut params = HashMap::new();
if let Some(query) = req.uri().query() {
let query_pairs = url::form_urlencoded::parse(query.as_bytes());
for (key, val) in query_pairs {
params.insert(key.to_lowercase(), val.to_string());
}
}
if let Some(key) = key { if let Some(key) = key {
match req.method() { match req.method() {
&Method::HEAD => Ok(handle_head(garage, &bucket, &key).await?), &Method::HEAD => {
&Method::GET => Ok(handle_get(garage, &bucket, &key).await?), // HeadObject query
Ok(handle_head(garage, &bucket, &key).await?)
}
&Method::GET => {
// GetObject query
Ok(handle_get(garage, &bucket, &key).await?)
}
&Method::PUT => { &Method::PUT => {
let mime_type = req if ["partnumber", "uploadid"]
.headers() .iter()
.get(hyper::header::CONTENT_TYPE) .all(|x| params.contains_key(&x.to_string()))
.map(|x| x.to_str()) {
.unwrap_or(Ok("blob"))? let part_number = params.get("partnumber").unwrap();
.to_string(); let upload_id = params.get("uploadid").unwrap();
let version_uuid = Ok(handle_put_part(garage, req, &bucket, &key, part_number, upload_id).await?)
handle_put(garage, &mime_type, &bucket, &key, req.into_body()).await?; } else {
let response = format!("{}\n", hex::encode(version_uuid,)); // PutObject query
Ok(Response::new(Box::new(BytesBody::from(response)))) Ok(handle_put(garage, req, &bucket, &key).await?)
}
} }
&Method::DELETE => { &Method::DELETE => {
// DeleteObject query
let version_uuid = handle_delete(garage, &bucket, &key).await?; let version_uuid = handle_delete(garage, &bucket, &key).await?;
let response = format!("{}\n", hex::encode(version_uuid,)); let response = format!("{}\n", hex::encode(version_uuid));
Ok(Response::new(Box::new(BytesBody::from(response)))) Ok(Response::new(Box::new(BytesBody::from(response))))
} }
&Method::POST => {
if params.contains_key(&"uploads".to_string()) {
// CreateMultipartUpload call
Ok(handle_create_multipart_upload(garage, &req, &bucket, &key).await?)
} else if params.contains_key(&"uploadid".to_string()) {
let upload_id = params.get("uploadid").unwrap();
Ok(handle_complete_multipart_upload(garage, req, &bucket, &key, upload_id).await?)
} else {
Err(Error::BadRequest(format!(
"Not a CreateMultipartUpload call, what is it?"
)))
}
}
_ => Err(Error::BadRequest(format!("Invalid method"))), _ => Err(Error::BadRequest(format!("Invalid method"))),
} }
} else { } else {
match req.method() { match req.method() {
&Method::PUT | &Method::HEAD => { &Method::PUT | &Method::HEAD => {
// If PUT: corresponds to a bucket creation call // If PUT: CreateBucket, if HEAD: HeadBucket
// If we're here, the bucket already exists, so just answer ok // If we're here, the bucket already exists, so just answer ok
let empty_body: BodyType = Box::new(BytesBody::from(vec![])); let empty_body: BodyType = Box::new(BytesBody::from(vec![]));
let response = Response::builder() let response = Response::builder()
@ -135,21 +161,18 @@ async fn handler_inner(
.unwrap(); .unwrap();
Ok(response) Ok(response)
} }
&Method::DELETE => Err(Error::Forbidden( &Method::DELETE => {
"Cannot delete buckets using S3 api, please talk to Garage directly".into(), // DeleteBucket query
)), Err(Error::Forbidden(
"Cannot delete buckets using S3 api, please talk to Garage directly".into(),
))
}
&Method::GET => { &Method::GET => {
let mut params = HashMap::new();
if let Some(query) = req.uri().query() {
let query_pairs = url::form_urlencoded::parse(query.as_bytes());
for (key, val) in query_pairs {
params.insert(key.to_lowercase(), val.to_string());
}
}
if ["delimiter", "prefix"] if ["delimiter", "prefix"]
.iter() .iter()
.all(|x| params.contains_key(&x.to_string())) .all(|x| params.contains_key(&x.to_string()))
{ {
// ListObjects query
let delimiter = params.get("delimiter").unwrap(); let delimiter = params.get("delimiter").unwrap();
let max_keys = params let max_keys = params
.get("max-keys") .get("max-keys")

View File

@ -7,6 +7,8 @@ use hyper::body::{Bytes, HttpBody};
use garage_util::error::Error; use garage_util::error::Error;
pub type BodyType = Box<dyn HttpBody<Data = Bytes, Error = Error> + Send + Unpin>;
type StreamType = Pin<Box<dyn Stream<Item = Result<Bytes, Error>> + Send>>; type StreamType = Pin<Box<dyn Stream<Item = Result<Bytes, Error>> + Send>>;
pub struct StreamBody { pub struct StreamBody {
@ -80,3 +82,9 @@ impl From<Vec<u8>> for BytesBody {
Self::new(Bytes::from(x)) Self::new(Bytes::from(x))
} }
} }
pub fn xml_escape(s: &str) -> String {
s.replace("<", "&lt;")
.replace(">", "&gt;")
.replace("\"", "&quot;")
}

View File

@ -12,7 +12,6 @@ use garage_table::EmptyKey;
use garage_core::garage::Garage; use garage_core::garage::Garage;
use garage_core::object_table::*; use garage_core::object_table::*;
use crate::api_server::BodyType;
use crate::http_util::*; use crate::http_util::*;
fn object_headers(version: &ObjectVersion) -> http::response::Builder { fn object_headers(version: &ObjectVersion) -> http::response::Builder {
@ -86,6 +85,9 @@ pub async fn handle_get(
let resp_builder = object_headers(&last_v).status(StatusCode::OK); let resp_builder = object_headers(&last_v).status(StatusCode::OK);
match &last_v.data { match &last_v.data {
ObjectVersionData::Uploading => Err(Error::Message(format!(
"Version is_complete() but data is stil Uploading (internal error)"
))),
ObjectVersionData::DeleteMarker => Err(Error::NotFound), ObjectVersionData::DeleteMarker => Err(Error::NotFound),
ObjectVersionData::Inline(bytes) => { ObjectVersionData::Inline(bytes) => {
let body: BodyType = Box::new(BytesBody::from(bytes.to_vec())); let body: BodyType = Box::new(BytesBody::from(bytes.to_vec()));

View File

@ -9,7 +9,6 @@ use garage_util::error::Error;
use garage_core::garage::Garage; use garage_core::garage::Garage;
use crate::api_server::BodyType;
use crate::http_util::*; use crate::http_util::*;
#[derive(Debug)] #[derive(Debug)]
@ -115,8 +114,3 @@ pub async fn handle_list(
Ok(Response::new(Box::new(BytesBody::from(xml.into_bytes())))) Ok(Response::new(Box::new(BytesBody::from(xml.into_bytes()))))
} }
fn xml_escape(s: &str) -> String {
s.replace("<", "&lt;")
.replace(">", "&gt;")
.replace("\"", "&quot;")
}

View File

@ -1,11 +1,13 @@
use std::collections::VecDeque; use std::collections::VecDeque;
use std::fmt::Write;
use std::sync::Arc; use std::sync::Arc;
use futures::stream::*; use futures::stream::*;
use hyper::Body; use hyper::{Body, Request, Response};
use garage_util::data::*; use garage_util::data::*;
use garage_util::error::Error; use garage_util::error::Error;
use garage_table::*;
use garage_core::block::INLINE_THRESHOLD; use garage_core::block::INLINE_THRESHOLD;
use garage_core::block_ref_table::*; use garage_core::block_ref_table::*;
@ -13,14 +15,17 @@ use garage_core::garage::Garage;
use garage_core::object_table::*; use garage_core::object_table::*;
use garage_core::version_table::*; use garage_core::version_table::*;
use crate::http_util::*;
pub async fn handle_put( pub async fn handle_put(
garage: Arc<Garage>, garage: Arc<Garage>,
mime_type: &str, req: Request<Body>,
bucket: &str, bucket: &str,
key: &str, key: &str,
body: Body, ) -> Result<Response<BodyType>, Error> {
) -> Result<UUID, Error> {
let version_uuid = gen_uuid(); let version_uuid = gen_uuid();
let mime_type = get_mime_type(&req)?;
let body = req.into_body();
let mut chunker = BodyChunker::new(body, garage.config.block_size); let mut chunker = BodyChunker::new(body, garage.config.block_size);
let first_block = match chunker.next().await? { let first_block = match chunker.next().await? {
@ -31,10 +36,10 @@ pub async fn handle_put(
let mut object_version = ObjectVersion { let mut object_version = ObjectVersion {
uuid: version_uuid, uuid: version_uuid,
timestamp: now_msec(), timestamp: now_msec(),
mime_type: mime_type.to_string(), mime_type,
size: first_block.len() as u64, size: first_block.len() as u64,
state: ObjectVersionState::Uploading, state: ObjectVersionState::Uploading,
data: ObjectVersionData::DeleteMarker, data: ObjectVersionData::Uploading,
}; };
if first_block.len() < INLINE_THRESHOLD { if first_block.len() < INLINE_THRESHOLD {
@ -43,7 +48,7 @@ pub async fn handle_put(
let object = Object::new(bucket.into(), key.into(), vec![object_version]); let object = Object::new(bucket.into(), key.into(), vec![object_version]);
garage.object_table.insert(&object).await?; garage.object_table.insert(&object).await?;
return Ok(version_uuid); return Ok(put_response(version_uuid));
} }
let version = Version::new(version_uuid, bucket.into(), key.into(), false, vec![]); let version = Version::new(version_uuid, bucket.into(), key.into(), false, vec![]);
@ -53,9 +58,30 @@ pub async fn handle_put(
let object = Object::new(bucket.into(), key.into(), vec![object_version.clone()]); let object = Object::new(bucket.into(), key.into(), vec![object_version.clone()]);
garage.object_table.insert(&object).await?; garage.object_table.insert(&object).await?;
let total_size = read_and_put_blocks(&garage, version, 1, first_block, first_block_hash, &mut chunker).await?;
// TODO: if at any step we have an error, we should undo everything we did
object_version.state = ObjectVersionState::Complete;
object_version.size = total_size;
let object = Object::new(bucket.into(), key.into(), vec![object_version]);
garage.object_table.insert(&object).await?;
Ok(put_response(version_uuid))
}
async fn read_and_put_blocks(
garage: &Arc<Garage>,
version: Version,
part_number: u64,
first_block: Vec<u8>,
first_block_hash: Hash,
chunker: &mut BodyChunker,
) -> Result<u64, Error> {
let mut next_offset = first_block.len(); let mut next_offset = first_block.len();
let mut put_curr_version_block = let mut put_curr_version_block =
put_block_meta(garage.clone(), &version, 0, 0, first_block_hash); put_block_meta(garage.clone(), &version, part_number, 0, first_block_hash, first_block.len() as u64);
let mut put_curr_block = garage let mut put_curr_block = garage
.block_manager .block_manager
.rpc_put_block(first_block_hash, first_block); .rpc_put_block(first_block_hash, first_block);
@ -67,7 +93,7 @@ pub async fn handle_put(
let block_hash = hash(&block[..]); let block_hash = hash(&block[..]);
let block_len = block.len(); let block_len = block.len();
put_curr_version_block = put_curr_version_block =
put_block_meta(garage.clone(), &version, 0, next_offset as u64, block_hash); put_block_meta(garage.clone(), &version, part_number, next_offset as u64, block_hash, block_len as u64);
put_curr_block = garage.block_manager.rpc_put_block(block_hash, block); put_curr_block = garage.block_manager.rpc_put_block(block_hash, block);
next_offset += block_len; next_offset += block_len;
} else { } else {
@ -75,15 +101,7 @@ pub async fn handle_put(
} }
} }
// TODO: if at any step we have an error, we should undo everything we did Ok(next_offset as u64)
object_version.state = ObjectVersionState::Complete;
object_version.size = next_offset as u64;
let object = Object::new(bucket.into(), key.into(), vec![object_version]);
garage.object_table.insert(&object).await?;
Ok(version_uuid)
} }
async fn put_block_meta( async fn put_block_meta(
@ -92,6 +110,7 @@ async fn put_block_meta(
part_number: u64, part_number: u64,
offset: u64, offset: u64,
hash: Hash, hash: Hash,
size: u64,
) -> Result<(), Error> { ) -> Result<(), Error> {
// TODO: don't clone, restart from empty block list ?? // TODO: don't clone, restart from empty block list ??
let mut version = version.clone(); let mut version = version.clone();
@ -100,6 +119,7 @@ async fn put_block_meta(
part_number, part_number,
offset, offset,
hash, hash,
size,
}) })
.unwrap(); .unwrap();
@ -154,6 +174,184 @@ impl BodyChunker {
} }
} }
fn put_response(version_uuid: UUID) -> Response<BodyType> {
let resp_bytes = format!("{}\n", hex::encode(version_uuid));
Response::new(Box::new(BytesBody::from(resp_bytes)))
}
pub async fn handle_create_multipart_upload(
garage: Arc<Garage>,
req: &Request<Body>,
bucket: &str,
key: &str,
) -> Result<Response<BodyType>, Error> {
let version_uuid = gen_uuid();
let mime_type = get_mime_type(req)?;
let object_version = ObjectVersion {
uuid: version_uuid,
timestamp: now_msec(),
mime_type,
size: 0,
state: ObjectVersionState::Uploading,
data: ObjectVersionData::Uploading,
};
let object = Object::new(bucket.to_string(), key.to_string(), vec![object_version]);
garage.object_table.insert(&object).await?;
let mut xml = String::new();
writeln!(&mut xml, r#"<?xml version="1.0" encoding="UTF-8"?>"#).unwrap();
writeln!(
&mut xml,
r#"<InitiateMultipartUploadResult xmlns="http://s3.amazonaws.com/doc/2006-03-01/">"#
)
.unwrap();
writeln!(&mut xml, "\t<Bucket>{}</Bucket>", bucket).unwrap();
writeln!(&mut xml, "\t<Key>{}</Key>", xml_escape(key)).unwrap();
writeln!(
&mut xml,
"\t<UploadId>{}</UploadId>",
hex::encode(version_uuid)
)
.unwrap();
writeln!(&mut xml, "</InitiateMultipartUploadResult>").unwrap();
Ok(Response::new(Box::new(BytesBody::from(xml.into_bytes()))))
}
pub async fn handle_put_part(
garage: Arc<Garage>,
req: Request<Body>,
bucket: &str,
key: &str,
part_number_str: &str,
upload_id: &str,
) -> Result<Response<BodyType>, Error> {
// Check parameters
let part_number = part_number_str
.parse::<u64>()
.map_err(|e| Error::BadRequest(format!("Invalid part number: {}", e)))?;
let version_uuid = uuid_from_str(upload_id).map_err(|_| Error::BadRequest(format!("Invalid upload ID")))?;
// Read first chuck, and at the same time try to get object to see if it exists
let mut chunker = BodyChunker::new(req.into_body(), garage.config.block_size);
let bucket = bucket.to_string();
let key = key.to_string();
let get_object_fut = garage.object_table.get(&bucket, &key);
let get_first_block_fut = chunker.next();
let (object, first_block) = futures::try_join!(get_object_fut, get_first_block_fut)?;
// Check object is valid and multipart block can be accepted
let first_block = match first_block {
None => return Err(Error::BadRequest(format!("Empty body"))),
Some(x) => x,
};
let object = match object {
None => return Err(Error::BadRequest(format!("Object not found"))),
Some(x) => x,
};
if !object.versions().iter().any(|v| {
v.uuid == version_uuid
&& v.state == ObjectVersionState::Uploading
&& v.data == ObjectVersionData::Uploading
}) {
return Err(Error::BadRequest(format!(
"Multipart upload does not exist or is otherwise invalid"
)));
}
// Copy block to store
let version = Version::new(version_uuid, bucket.into(), key.into(), false, vec![]);
let first_block_hash = hash(&first_block[..]);
read_and_put_blocks(&garage, version, part_number, first_block, first_block_hash, &mut chunker).await?;
Ok(Response::new(Box::new(BytesBody::from(vec![]))))
}
pub async fn handle_complete_multipart_upload(
garage: Arc<Garage>,
_req: Request<Body>,
bucket: &str,
key: &str,
upload_id: &str,
) -> Result<Response<BodyType>, Error> {
let version_uuid = uuid_from_str(upload_id).map_err(|_| Error::BadRequest(format!("Invalid upload ID")))?;
let bucket = bucket.to_string();
let key = key.to_string();
let (object, version) = futures::try_join!(
garage.object_table.get(&bucket, &key),
garage.version_table.get(&version_uuid, &EmptyKey),
)?;
let object = match object {
None => return Err(Error::BadRequest(format!("Object not found"))),
Some(x) => x,
};
let object_version = object.versions().iter().find(|v| {
v.uuid == version_uuid
&& v.state == ObjectVersionState::Uploading
&& v.data == ObjectVersionData::Uploading
});
let mut object_version = match object_version {
None => return Err(Error::BadRequest(format!(
"Multipart upload does not exist or has already been completed"
))),
Some(x) => x.clone(),
};
let version = match version {
None => return Err(Error::BadRequest(format!("Version not found"))),
Some(x) => x,
};
if version.blocks().len() == 0 {
return Err(Error::BadRequest(format!("No data was uploaded")));
}
// TODO: check that all the parts that they pretend they gave us are indeed there
// TODO: check MD5 sum of all uploaded parts? but that would mean we have to store them somewhere...
let total_size = version.blocks().iter().map(|x| x.size).fold(0, |x, y| x+y);
object_version.size = total_size;
object_version.state = ObjectVersionState::Complete;
object_version.data = ObjectVersionData::FirstBlock(version.blocks()[0].hash);
let final_object = Object::new(bucket.clone(), key.clone(), vec![object_version]);
garage.object_table.insert(&final_object).await?;
let mut xml = String::new();
writeln!(&mut xml, r#"<?xml version="1.0" encoding="UTF-8"?>"#).unwrap();
writeln!(
&mut xml,
r#"<CompleteMultipartUploadResult xmlns="http://s3.amazonaws.com/doc/2006-03-01/">"#
)
.unwrap();
writeln!(&mut xml, "\t<Location>{}</Location>", garage.config.s3_api.s3_region).unwrap();
writeln!(&mut xml, "\t<Bucket>{}</Bucket>", bucket).unwrap();
writeln!(&mut xml, "\t<Key>{}</Key>", xml_escape(&key)).unwrap();
writeln!(&mut xml, "</CompleteMultipartUploadResult>").unwrap();
Ok(Response::new(Box::new(BytesBody::from(xml.into_bytes()))))
}
fn get_mime_type(req: &Request<Body>) -> Result<String, Error> {
Ok(req
.headers()
.get(hyper::header::CONTENT_TYPE)
.map(|x| x.to_str())
.unwrap_or(Ok("blob"))?
.to_string())
}
fn uuid_from_str(id: &str) -> Result<UUID, ()> {
let id_bin = hex::decode(id).map_err(|_| ())?;
if id_bin.len() != 32 {
return Err(());
}
let mut uuid = [0u8; 32];
uuid.copy_from_slice(&id_bin[..]);
Ok(UUID::from(uuid))
}
pub async fn handle_delete(garage: Arc<Garage>, bucket: &str, key: &str) -> Result<UUID, Error> { pub async fn handle_delete(garage: Arc<Garage>, bucket: &str, key: &str) -> Result<UUID, Error> {
let object = match garage let object = match garage
.object_table .object_table

View File

@ -88,6 +88,7 @@ impl ObjectVersionState {
#[derive(PartialEq, Clone, Debug, Serialize, Deserialize)] #[derive(PartialEq, Clone, Debug, Serialize, Deserialize)]
pub enum ObjectVersionData { pub enum ObjectVersionData {
Uploading,
DeleteMarker, DeleteMarker,
Inline(#[serde(with = "serde_bytes")] Vec<u8>), Inline(#[serde(with = "serde_bytes")] Vec<u8>),
FirstBlock(Hash), FirstBlock(Hash),
@ -125,6 +126,9 @@ impl Entry<String, String> for Object {
v.size = other_v.size; v.size = other_v.size;
} }
v.state = v.state.max(other_v.state); v.state = v.state.max(other_v.state);
if v.data == ObjectVersionData::Uploading {
v.data = other_v.data.clone();
}
} }
Err(i) => { Err(i) => {
self.versions.insert(i, other_v.clone()); self.versions.insert(i, other_v.clone());

View File

@ -49,7 +49,7 @@ impl Version {
} }
/// Adds a block if it wasn't already present /// Adds a block if it wasn't already present
pub fn add_block(&mut self, new: VersionBlock) -> Result<(), ()> { pub fn add_block(&mut self, new: VersionBlock) -> Result<(), ()> {
match self.blocks.binary_search_by(|b| b.offset.cmp(&new.offset)) { match self.blocks.binary_search_by(|b| b.cmp_key().cmp(&new.cmp_key())) {
Err(i) => { Err(i) => {
self.blocks.insert(i, new); self.blocks.insert(i, new);
Ok(()) Ok(())
@ -67,6 +67,13 @@ pub struct VersionBlock {
pub part_number: u64, pub part_number: u64,
pub offset: u64, pub offset: u64,
pub hash: Hash, pub hash: Hash,
pub size: u64,
}
impl VersionBlock {
fn cmp_key(&self) -> (u64, u64) {
(self.part_number, self.offset)
}
} }
impl Entry<Hash, EmptyKey> for Version { impl Entry<Hash, EmptyKey> for Version {
@ -83,7 +90,7 @@ impl Entry<Hash, EmptyKey> for Version {
self.blocks.clear(); self.blocks.clear();
} else if !self.deleted { } else if !self.deleted {
for bi in other.blocks.iter() { for bi in other.blocks.iter() {
match self.blocks.binary_search_by(|x| x.offset.cmp(&bi.offset)) { match self.blocks.binary_search_by(|x| x.cmp_key().cmp(&bi.cmp_key())) {
Ok(_) => (), Ok(_) => (),
Err(pos) => { Err(pos) => {
self.blocks.insert(pos, bi.clone()); self.blocks.insert(pos, bi.clone());