From f319a7d3740ba8b83c9c0eae27edfda1c1d14c03 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 10 Mar 2021 16:21:56 +0100 Subject: Refactor model stuff, including cleaner CRDTs --- src/api/s3_copy.rs | 13 +- src/api/s3_get.rs | 12 +- src/api/s3_put.rs | 37 ++--- src/garage/repair.rs | 9 +- src/model/block.rs | 2 +- src/model/block_ref_table.rs | 17 ++- src/model/bucket_table.rs | 2 + src/model/garage.rs | 1 - src/model/key_table.rs | 13 +- src/model/object_table.rs | 39 +++--- src/model/version_table.rs | 96 ++++++------- src/table/crdt.rs | 327 ------------------------------------------- src/table/crdt/bool.rs | 34 +++++ src/table/crdt/crdt.rs | 73 ++++++++++ src/table/crdt/lww.rs | 114 +++++++++++++++ src/table/crdt/lww_map.rs | 145 +++++++++++++++++++ src/table/crdt/map.rs | 83 +++++++++++ src/table/crdt/mod.rs | 22 +++ src/table/schema.rs | 6 +- src/table/table.rs | 1 + 20 files changed, 590 insertions(+), 456 deletions(-) delete mode 100644 src/table/crdt.rs create mode 100644 src/table/crdt/bool.rs create mode 100644 src/table/crdt/crdt.rs create mode 100644 src/table/crdt/lww.rs create mode 100644 src/table/crdt/lww_map.rs create mode 100644 src/table/crdt/map.rs create mode 100644 src/table/crdt/mod.rs diff --git a/src/api/s3_copy.rs b/src/api/s3_copy.rs index b6ec48b0..c6c30095 100644 --- a/src/api/s3_copy.rs +++ b/src/api/s3_copy.rs @@ -66,25 +66,28 @@ pub async fn handle_copy( .await?; let source_version = source_version.ok_or(Error::NotFound)?; - let dest_version = Version::new( + let mut dest_version = Version::new( new_uuid, dest_bucket.to_string(), dest_key.to_string(), false, - source_version.blocks().to_vec(), ); + for (bk, bv) in source_version.blocks.items().iter() { + dest_version.blocks.put(*bk, *bv); + } let dest_object = Object::new( dest_bucket.to_string(), dest_key.to_string(), vec![dest_object_version], ); let dest_block_refs = dest_version - .blocks() + .blocks + .items() .iter() .map(|b| BlockRef { - block: b.hash, + block: b.1.hash, version: new_uuid, - deleted: false, + deleted: false.into(), }) .collect::>(); futures::try_join!( diff --git a/src/api/s3_get.rs b/src/api/s3_get.rs index 68e7c66a..22a55b55 100644 --- a/src/api/s3_get.rs +++ b/src/api/s3_get.rs @@ -146,9 +146,10 @@ pub async fn handle_get( let version = version.ok_or(Error::NotFound)?; let mut blocks = version - .blocks() + .blocks + .items() .iter() - .map(|vb| (vb.hash, None)) + .map(|(_, vb)| (vb.hash, None)) .collect::>(); blocks[0].1 = Some(first_block); @@ -219,11 +220,12 @@ pub async fn handle_get_range( // file (whereas block.offset designates the offset of the block WITHIN THE PART // block.part_number, which is not the same in the case of a multipart upload) let mut blocks = Vec::with_capacity(std::cmp::min( - version.blocks().len(), - 4 + ((end - begin) / std::cmp::max(version.blocks()[0].size as u64, 1024)) as usize, + version.blocks.len(), + 4 + ((end - begin) / std::cmp::max(version.blocks.items()[0].1.size as u64, 1024)) + as usize, )); let mut true_offset = 0; - for b in version.blocks().iter() { + for (_, b) in version.blocks.items().iter() { if true_offset >= end { break; } diff --git a/src/api/s3_put.rs b/src/api/s3_put.rs index ec599a05..37a1ece2 100644 --- a/src/api/s3_put.rs +++ b/src/api/s3_put.rs @@ -94,7 +94,7 @@ pub async fn handle_put( garage.object_table.insert(&object).await?; // Initialize corresponding entry in version table - let version = Version::new(version_uuid, bucket.into(), key.into(), false, vec![]); + let version = Version::new(version_uuid, bucket.into(), key.into(), false); let first_block_hash = sha256sum(&first_block[..]); // Transfer data and verify checksum @@ -242,19 +242,18 @@ async fn put_block_meta( ) -> Result<(), GarageError> { // TODO: don't clone, restart from empty block list ?? let mut version = version.clone(); - version - .add_block(VersionBlock { + version.blocks.put( + VersionBlockKey { part_number, offset, - hash, - size, - }) - .unwrap(); + }, + VersionBlock { hash, size }, + ); let block_ref = BlockRef { block: hash, version: version.uuid, - deleted: false, + deleted: false.into(), }; futures::try_join!( @@ -389,7 +388,7 @@ pub async fn handle_put_part( } // Copy block to store - let version = Version::new(version_uuid, bucket, key, false, vec![]); + let version = Version::new(version_uuid, bucket, key, false); let first_block_hash = sha256sum(&first_block[..]); let (_, md5sum_arr, sha256sum) = read_and_put_blocks( &garage, @@ -454,7 +453,7 @@ pub async fn handle_complete_multipart_upload( }; let version = version.ok_or(Error::BadRequest(format!("Version not found")))?; - if version.blocks().len() == 0 { + if version.blocks.len() == 0 { return Err(Error::BadRequest(format!("No data was uploaded"))); } @@ -466,9 +465,10 @@ pub async fn handle_complete_multipart_upload( // Check that the list of parts they gave us corresponds to the parts we have here // TODO: check MD5 sum of all uploaded parts? but that would mean we have to store them somewhere... let mut parts = version - .blocks() + .blocks + .items() .iter() - .map(|x| x.part_number) + .map(|x| x.0.part_number) .collect::>(); parts.dedup(); let same_parts = body_list_of_parts @@ -485,8 +485,8 @@ pub async fn handle_complete_multipart_upload( // shouldn't impact compatibility as the S3 docs specify that // the ETag is an opaque value in case of a multipart upload. // See also: https://teppen.io/2018/06/23/aws_s3_etags/ - let num_parts = version.blocks().last().unwrap().part_number - - version.blocks().first().unwrap().part_number + let num_parts = version.blocks.items().last().unwrap().0.part_number + - version.blocks.items().first().unwrap().0.part_number + 1; let etag = format!( "{}-{}", @@ -495,17 +495,18 @@ pub async fn handle_complete_multipart_upload( ); let total_size = version - .blocks() + .blocks + .items() .iter() - .map(|x| x.size) + .map(|x| x.1.size) .fold(0, |x, y| x + y); object_version.state = ObjectVersionState::Complete(ObjectVersionData::FirstBlock( ObjectVersionMeta { headers, size: total_size, - etag: etag, + etag, }, - version.blocks()[0].hash, + version.blocks.items()[0].1.hash, )); let final_object = Object::new(bucket.clone(), key.clone(), vec![object_version]); diff --git a/src/garage/repair.rs b/src/garage/repair.rs index 297ae9cd..e330f7bb 100644 --- a/src/garage/repair.rs +++ b/src/garage/repair.rs @@ -97,7 +97,7 @@ impl Repair { pos = item_key.to_vec(); let version = rmp_serde::decode::from_read_ref::<_, Version>(item_bytes.as_ref())?; - if version.deleted { + if version.deleted.get() { continue; } let object = self @@ -127,7 +127,6 @@ impl Repair { version.bucket, version.key, true, - vec![], )) .await?; } @@ -146,7 +145,7 @@ impl Repair { pos = item_key.to_vec(); let block_ref = rmp_serde::decode::from_read_ref::<_, BlockRef>(item_bytes.as_ref())?; - if block_ref.deleted { + if block_ref.deleted.get() { continue; } let version = self @@ -155,7 +154,7 @@ impl Repair { .get(&block_ref.version, &EmptyKey) .await?; let ref_exists = match version { - Some(v) => !v.deleted, + Some(v) => !v.deleted.get(), None => { warn!( "Block ref repair: version for block ref {:?} not found, skipping.", @@ -174,7 +173,7 @@ impl Repair { .insert(&BlockRef { block: block_ref.block, version: block_ref.version, - deleted: true, + deleted: true.into(), }) .await?; } diff --git a/src/model/block.rs b/src/model/block.rs index 56c85c6a..d3957403 100644 --- a/src/model/block.rs +++ b/src/model/block.rs @@ -420,7 +420,7 @@ impl BlockManager { if Some(&block_ref.block) == last_hash.as_ref() { continue; } - if !block_ref.deleted { + if !block_ref.deleted.get() { last_hash = Some(block_ref.block); self.put_to_resync(&block_ref.block, 0)?; } diff --git a/src/model/block_ref_table.rs b/src/model/block_ref_table.rs index 9ab67737..07fa5144 100644 --- a/src/model/block_ref_table.rs +++ b/src/model/block_ref_table.rs @@ -1,9 +1,9 @@ use serde::{Deserialize, Serialize}; use std::sync::Arc; -use garage_util::background::*; use garage_util::data::*; +use garage_table::crdt::CRDT; use garage_table::*; use crate::block::*; @@ -17,7 +17,7 @@ pub struct BlockRef { pub version: UUID, // Keep track of deleted status - pub deleted: bool, + pub deleted: crdt::Bool, } impl Entry for BlockRef { @@ -27,16 +27,15 @@ impl Entry for BlockRef { fn sort_key(&self) -> &UUID { &self.version } +} +impl CRDT for BlockRef { fn merge(&mut self, other: &Self) { - if other.deleted { - self.deleted = true; - } + self.deleted.merge(&other.deleted); } } pub struct BlockRefTable { - pub background: Arc, pub block_manager: Arc, } @@ -48,8 +47,8 @@ impl TableSchema for BlockRefTable { fn updated(&self, old: Option, new: Option) { let block = &old.as_ref().or(new.as_ref()).unwrap().block; - let was_before = old.as_ref().map(|x| !x.deleted).unwrap_or(false); - let is_after = new.as_ref().map(|x| !x.deleted).unwrap_or(false); + let was_before = old.as_ref().map(|x| !x.deleted.get()).unwrap_or(false); + let is_after = new.as_ref().map(|x| !x.deleted.get()).unwrap_or(false); if is_after && !was_before { if let Err(e) = self.block_manager.block_incref(block) { warn!("block_incref failed for block {:?}: {}", block, e); @@ -63,6 +62,6 @@ impl TableSchema for BlockRefTable { } fn matches_filter(entry: &Self::E, filter: &Self::Filter) -> bool { - filter.apply(entry.deleted) + filter.apply(entry.deleted.get()) } } diff --git a/src/model/bucket_table.rs b/src/model/bucket_table.rs index 2878aa38..5bc8b7f9 100644 --- a/src/model/bucket_table.rs +++ b/src/model/bucket_table.rs @@ -89,7 +89,9 @@ impl Entry for Bucket { fn sort_key(&self) -> &String { &self.name } +} +impl CRDT for Bucket { fn merge(&mut self, other: &Self) { self.state.merge(&other.state); } diff --git a/src/model/garage.rs b/src/model/garage.rs index 467d0aec..d109fdaa 100644 --- a/src/model/garage.rs +++ b/src/model/garage.rs @@ -79,7 +79,6 @@ impl Garage { info!("Initialize block_ref_table..."); let block_ref_table = Table::new( BlockRefTable { - background: background.clone(), block_manager: block_manager.clone(), }, data_rep_param.clone(), diff --git a/src/model/key_table.rs b/src/model/key_table.rs index 5942df75..6d8cc6c0 100644 --- a/src/model/key_table.rs +++ b/src/model/key_table.rs @@ -1,6 +1,6 @@ use serde::{Deserialize, Serialize}; -use garage_table::crdt::CRDT; +use garage_table::crdt::*; use garage_table::*; use model010::key_table as prev; @@ -66,6 +66,10 @@ pub struct PermissionSet { pub allow_write: bool, } +impl AutoCRDT for PermissionSet { + const WARN_IF_DIFFERENT: bool = true; +} + impl Entry for Key { fn partition_key(&self) -> &EmptyKey { &EmptyKey @@ -73,17 +77,18 @@ impl Entry for Key { fn sort_key(&self) -> &String { &self.key_id } +} +impl CRDT for Key { fn merge(&mut self, other: &Self) { self.name.merge(&other.name); self.deleted.merge(&other.deleted); if self.deleted.get() { self.authorized_buckets.clear(); - return; + } else { + self.authorized_buckets.merge(&other.authorized_buckets); } - - self.authorized_buckets.merge(&other.authorized_buckets); } } diff --git a/src/model/object_table.rs b/src/model/object_table.rs index 16cce72c..75c37f6d 100644 --- a/src/model/object_table.rs +++ b/src/model/object_table.rs @@ -5,6 +5,7 @@ use std::sync::Arc; use garage_util::background::BackgroundRunner; use garage_util::data::*; +use garage_table::crdt::*; use garage_table::table_sharded::*; use garage_table::*; @@ -70,7 +71,7 @@ pub enum ObjectVersionState { Aborted, } -impl ObjectVersionState { +impl CRDT for ObjectVersionState { fn merge(&mut self, other: &Self) { use ObjectVersionState::*; match other { @@ -91,37 +92,30 @@ impl ObjectVersionState { } } -#[derive(PartialEq, Clone, Debug, Serialize, Deserialize)] +#[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Debug, Serialize, Deserialize)] pub enum ObjectVersionData { DeleteMarker, Inline(ObjectVersionMeta, #[serde(with = "serde_bytes")] Vec), FirstBlock(ObjectVersionMeta, Hash), } -#[derive(PartialEq, Clone, Debug, Serialize, Deserialize)] +impl AutoCRDT for ObjectVersionData { + const WARN_IF_DIFFERENT: bool = true; +} + +#[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Debug, Serialize, Deserialize)] pub struct ObjectVersionMeta { pub headers: ObjectVersionHeaders, pub size: u64, pub etag: String, } -#[derive(PartialEq, Clone, Debug, Serialize, Deserialize)] +#[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Debug, Serialize, Deserialize)] pub struct ObjectVersionHeaders { pub content_type: String, pub other: BTreeMap, } -impl ObjectVersionData { - fn merge(&mut self, b: &Self) { - if *self != *b { - warn!( - "Inconsistent object version data: {:?} (local) vs {:?} (remote)", - self, b - ); - } - } -} - impl ObjectVersion { fn cmp_key(&self) -> (u64, UUID) { (self.timestamp, self.uuid) @@ -154,8 +148,11 @@ impl Entry for Object { fn sort_key(&self) -> &String { &self.key } +} +impl CRDT for Object { fn merge(&mut self, other: &Self) { + // Merge versions from other into here for other_v in other.versions.iter() { match self .versions @@ -169,6 +166,9 @@ impl Entry for Object { } } } + + // Remove versions which are obsolete, i.e. those that come + // before the last version which .is_complete(). let last_complete = self .versions .iter() @@ -212,13 +212,8 @@ impl TableSchema for ObjectTable { } }; if newly_deleted { - let deleted_version = Version::new( - v.uuid, - old_v.bucket.clone(), - old_v.key.clone(), - true, - vec![], - ); + let deleted_version = + Version::new(v.uuid, old_v.bucket.clone(), old_v.key.clone(), true); version_table.insert(&deleted_version).await?; } } diff --git a/src/model/version_table.rs b/src/model/version_table.rs index cf9fbe98..26abb64e 100644 --- a/src/model/version_table.rs +++ b/src/model/version_table.rs @@ -4,6 +4,7 @@ use std::sync::Arc; use garage_util::background::BackgroundRunner; use garage_util::data::*; +use garage_table::crdt::*; use garage_table::table_sharded::*; use garage_table::*; @@ -15,8 +16,8 @@ pub struct Version { pub uuid: UUID, // Actual data: the blocks for this version - pub deleted: bool, - blocks: Vec, + pub deleted: crdt::Bool, + pub blocks: crdt::Map, // Back link to bucket+key so that we can figure if // this was deleted later on @@ -25,56 +26,45 @@ pub struct Version { } impl Version { - pub fn new( - uuid: UUID, - bucket: String, - key: String, - deleted: bool, - blocks: Vec, - ) -> Self { - let mut ret = Self { + pub fn new(uuid: UUID, bucket: String, key: String, deleted: bool) -> Self { + Self { uuid, - deleted, - blocks: vec![], + deleted: deleted.into(), + blocks: crdt::Map::new(), bucket, key, - }; - for b in blocks { - ret.add_block(b) - .expect("Twice the same VersionBlock in Version constructor"); } - ret } - /// Adds a block if it wasn't already present - pub fn add_block(&mut self, new: VersionBlock) -> Result<(), ()> { - match self - .blocks - .binary_search_by(|b| b.cmp_key().cmp(&new.cmp_key())) - { - Err(i) => { - self.blocks.insert(i, new); - Ok(()) - } - Ok(_) => Err(()), - } +} + +#[derive(PartialEq, Eq, Clone, Copy, Debug, Serialize, Deserialize)] +pub struct VersionBlockKey { + pub part_number: u64, + pub offset: u64, +} + +impl Ord for VersionBlockKey { + fn cmp(&self, other: &Self) -> std::cmp::Ordering { + self.part_number + .cmp(&other.part_number) + .then(self.offset.cmp(&other.offset)) } - pub fn blocks(&self) -> &[VersionBlock] { - &self.blocks[..] +} + +impl PartialOrd for VersionBlockKey { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) } } -#[derive(PartialEq, Clone, Debug, Serialize, Deserialize)] +#[derive(PartialEq, Eq, Ord, PartialOrd, Clone, Copy, Debug, Serialize, Deserialize)] pub struct VersionBlock { - pub part_number: u64, - pub offset: u64, pub hash: Hash, pub size: u64, } -impl VersionBlock { - fn cmp_key(&self) -> (u64, u64) { - (self.part_number, self.offset) - } +impl AutoCRDT for VersionBlock { + const WARN_IF_DIFFERENT: bool = true; } impl Entry for Version { @@ -84,23 +74,16 @@ impl Entry for Version { fn sort_key(&self) -> &EmptyKey { &EmptyKey } +} +impl CRDT for Version { fn merge(&mut self, other: &Self) { - if other.deleted { - self.deleted = true; + self.deleted.merge(&other.deleted); + + if self.deleted.get() { self.blocks.clear(); - } else if !self.deleted { - for bi in other.blocks.iter() { - match self - .blocks - .binary_search_by(|x| x.cmp_key().cmp(&bi.cmp_key())) - { - Ok(_) => (), - Err(pos) => { - self.blocks.insert(pos, bi.clone()); - } - } - } + } else { + self.blocks.merge(&other.blocks); } } } @@ -121,14 +104,15 @@ impl TableSchema for VersionTable { self.background.spawn(async move { if let (Some(old_v), Some(new_v)) = (old, new) { // Propagate deletion of version blocks - if new_v.deleted && !old_v.deleted { + if new_v.deleted.get() && !old_v.deleted.get() { let deleted_block_refs = old_v .blocks + .items() .iter() - .map(|vb| BlockRef { + .map(|(_k, vb)| BlockRef { block: vb.hash, version: old_v.uuid, - deleted: true, + deleted: true.into(), }) .collect::>(); block_ref_table.insert_many(&deleted_block_refs[..]).await?; @@ -139,6 +123,6 @@ impl TableSchema for VersionTable { } fn matches_filter(entry: &Self::E, filter: &Self::Filter) -> bool { - filter.apply(entry.deleted) + filter.apply(entry.deleted.get()) } } diff --git a/src/table/crdt.rs b/src/table/crdt.rs deleted file mode 100644 index 4cba10ce..00000000 --- a/src/table/crdt.rs +++ /dev/null @@ -1,327 +0,0 @@ -//! This package provides a simple implementation of conflict-free replicated data types (CRDTs) -//! -//! CRDTs are a type of data structures that do not require coordination. In other words, we can -//! edit them in parallel, we will always find a way to merge it. -//! -//! A general example is a counter. Its initial value is 0. Alice and Bob get a copy of the -//! counter. Alice does +1 on her copy, she reads 1. Bob does +3 on his copy, he reads 3. Now, -//! it is easy to merge their counters, order does not count: we always get 4. -//! -//! Learn more about CRDT [on Wikipedia](https://en.wikipedia.org/wiki/Conflict-free_replicated_data_type) - -use serde::{Deserialize, Serialize}; - -use garage_util::data::*; - -/// Definition of a CRDT - all CRDT Rust types implement this. -/// -/// A CRDT is defined as a merge operator that respects a certain set of axioms. -/// -/// In particular, the merge operator must be commutative, associative, -/// idempotent, and monotonic. -/// In other words, if `a`, `b` and `c` are CRDTs, and `⊔` denotes the merge operator, -/// the following axioms must apply: -/// -/// ```text -/// a ⊔ b = b ⊔ a (commutativity) -/// (a ⊔ b) ⊔ c = a ⊔ (b ⊔ c) (associativity) -/// (a ⊔ b) ⊔ b = a ⊔ b (idempotence) -/// ``` -/// -/// Moreover, the relationship `≥` defined by `a ≥ b ⇔ ∃c. a = b ⊔ c` must be a partial order. -/// This implies a few properties such as: if `a ⊔ b ≠ a`, then there is no `c` such that `(a ⊔ b) ⊔ c = a`, -/// as this would imply a cycle in the partial order. -pub trait CRDT { - /// Merge the two datastructures according to the CRDT rules. - /// `self` is modified to contain the merged CRDT value. `other` is not modified. - /// - /// # Arguments - /// - /// * `other` - the other CRDT we wish to merge with - fn merge(&mut self, other: &Self); -} - -/// All types that implement `Ord` (a total order) also implement a trivial CRDT -/// defined by the merge rule: `a ⊔ b = max(a, b)`. -impl CRDT for T -where - T: Ord + Clone, -{ - fn merge(&mut self, other: &Self) { - if other > self { - *self = other.clone(); - } - } -} - -// ---- LWW Register ---- - -/// Last Write Win (LWW) -/// -/// An LWW CRDT associates a timestamp with a value, in order to implement a -/// time-based reconciliation rule: the most recent write wins. -/// For completeness, the LWW reconciliation rule must also be defined for two LWW CRDTs -/// with the same timestamp but different values. -/// -/// In our case, we add the constraint that the value that is wrapped inside the LWW CRDT must -/// itself be a CRDT: in the case when the timestamp does not allow us to decide on which value to -/// keep, the merge rule of the inner CRDT is applied on the wrapped values. (Note that all types -/// that implement the `Ord` trait get a default CRDT implemetnation that keeps the maximum value. -/// This enables us to use LWW directly with primitive data types such as numbers or strings. It is -/// generally desirable in this case to never explicitly produce LWW values with the same timestamp -/// but different inner values, as the rule to keep the maximum value isn't generally the desired -/// semantics.) -/// -/// As multiple computers clocks are always desynchronized, -/// when operations are close enough, it is equivalent to -/// take one copy and drop the other one. -/// -/// Given that clocks are not too desynchronized, this assumption -/// is enough for most cases, as there is few chance that two humans -/// coordonate themself faster than the time difference between two NTP servers. -/// -/// As a more concret example, let's suppose you want to upload a file -/// with the same key (path) in the same bucket at the very same time. -/// For each request, the file will be timestamped by the receiving server -/// and may differ from what you observed with your atomic clock! -/// -/// This scheme is used by AWS S3 or Soundcloud and often without knowing -/// in entreprise when reconciliating databases with ad-hoc scripts. -#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] -pub struct LWW { - ts: u64, - v: T, -} - -impl LWW -where - T: CRDT, -{ - /// Creates a new CRDT - /// - /// CRDT's internal timestamp is set with current node's clock. - pub fn new(value: T) -> Self { - Self { - ts: now_msec(), - v: value, - } - } - - /// Build a new CRDT from a previous non-compatible one - /// - /// Compared to new, the CRDT's timestamp is not set to now - /// but must be set to the previous, non-compatible, CRDT's timestamp. - pub fn migrate_from_raw(ts: u64, value: T) -> Self { - Self { ts, v: value } - } - - /// Update the LWW CRDT while keeping some causal ordering. - /// - /// The timestamp of the LWW CRDT is updated to be the current node's clock - /// at time of update, or the previous timestamp + 1 if that's bigger, - /// so that the new timestamp is always strictly larger than the previous one. - /// This ensures that merging the update with the old value will result in keeping - /// the updated value. - pub fn update(&mut self, new_value: T) { - self.ts = std::cmp::max(self.ts + 1, now_msec()); - self.v = new_value; - } - - /// Get the CRDT value - pub fn get(&self) -> &T { - &self.v - } - - /// Get a mutable reference to the CRDT's value - /// - /// This is usefull to mutate the inside value without changing the LWW timestamp. - /// When such mutation is done, the merge between two LWW values is done using the inner - /// CRDT's merge operation. This is usefull in the case where the inner CRDT is a large - /// data type, such as a map, and we only want to change a single item in the map. - /// To do this, we can produce a "CRDT delta", i.e. a LWW that contains only the modification. - /// This delta consists in a LWW with the same timestamp, and the map - /// inside only contains the updated value. - /// The advantage of such a delta is that it is much smaller than the whole map. - /// - /// Avoid using this if the inner data type is a primitive type such as a number or a string, - /// as you will then rely on the merge function defined on `Ord` types by keeping the maximum - /// of both values. - pub fn get_mut(&mut self) -> &mut T { - &mut self.v - } -} - -impl CRDT for LWW -where - T: Clone + CRDT, -{ - fn merge(&mut self, other: &Self) { - if other.ts > self.ts { - self.ts = other.ts; - self.v = other.v.clone(); - } else if other.ts == self.ts { - self.v.merge(&other.v); - } - } -} - -/// Boolean, where `true` is an absorbing state -#[derive(Clone, Copy, Debug, Serialize, Deserialize, PartialEq)] -pub struct Bool(bool); - -impl Bool { - /// Create a new boolean with the specified value - pub fn new(b: bool) -> Self { - Self(b) - } - /// Set the boolean to true - pub fn set(&mut self) { - self.0 = true; - } - /// Get the boolean value - pub fn get(&self) -> bool { - self.0 - } -} - -impl CRDT for Bool { - fn merge(&mut self, other: &Self) { - self.0 = self.0 || other.0; - } -} - -/// Last Write Win Map -/// -/// This types defines a CRDT for a map from keys to values. -/// The values have an associated timestamp, such that the last written value -/// takes precedence over previous ones. As for the simpler `LWW` type, the value -/// type `V` is also required to implement the CRDT trait. -/// We do not encourage mutating the values associated with a given key -/// without updating the timestamp, in fact at the moment we do not provide a `.get_mut()` -/// method that would allow that. -/// -/// Internally, the map is stored as a vector of keys and values, sorted by ascending key order. -/// This is why the key type `K` must implement `Ord` (and also to ensure a unique serialization, -/// such that two values can be compared for equality based on their hashes). As a consequence, -/// insertions take `O(n)` time. This means that LWWMap should be used for reasonably small maps. -/// However, note that even if we were using a more efficient data structure such as a `BTreeMap`, -/// the serialization cost `O(n)` would still have to be paid at each modification, so we are -/// actually not losing anything here. -#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] -pub struct LWWMap { - vals: Vec<(K, u64, V)>, -} - -impl LWWMap -where - K: Ord, - V: CRDT, -{ - /// Create a new empty map CRDT - pub fn new() -> Self { - Self { vals: vec![] } - } - /// Used to migrate from a map defined in an incompatible format. This produces - /// a map that contains a single item with the specified timestamp (copied from - /// the incompatible format). Do this as many times as you have items to migrate, - /// and put them all together using the CRDT merge operator. - pub fn migrate_from_raw_item(k: K, ts: u64, v: V) -> Self { - Self { - vals: vec![(k, ts, v)], - } - } - /// Returns a map that contains a single mapping from the specified key to the specified value. - /// This map is a mutator, or a delta-CRDT, such that when it is merged with the original map, - /// the previous value will be replaced with the one specified here. - /// The timestamp in the provided mutator is set to the maximum of the current system's clock - /// and 1 + the previous value's timestamp (if there is one), so that the new value will always - /// take precedence (LWW rule). - /// - /// 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); - /// ``` - /// - /// However extracting the mutator on its own and only sending that on the network is very - /// interesting as it is much smaller than the whole map. - pub fn update_mutator(&self, k: K, new_v: V) -> Self { - let new_vals = match self.vals.binary_search_by(|(k2, _, _)| k2.cmp(&k)) { - Ok(i) => { - let (_, old_ts, _) = self.vals[i]; - let new_ts = std::cmp::max(old_ts + 1, now_msec()); - vec![(k, new_ts, new_v)] - } - Err(_) => vec![(k, now_msec(), new_v)], - }; - Self { vals: new_vals } - } - /// Takes all of the values of the map and returns them. The current map is reset to the - /// 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)); - /// put_my_crdt_value(a); - /// ``` - /// - /// Of course in this simple example we could have written simply - /// `pyt_my_crdt_value(a.update_mutator(key_to_modify, new_value))`, - /// 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)); - /// put_my_crdt_value(a); - /// ``` - pub fn take_and_clear(&mut self) -> Self { - let vals = std::mem::replace(&mut self.vals, vec![]); - Self { vals } - } - /// Removes all values from the map - pub fn clear(&mut self) { - self.vals.clear(); - } - /// Get a reference to the value assigned to a key - pub fn get(&self, k: &K) -> Option<&V> { - match self.vals.binary_search_by(|(k2, _, _)| k2.cmp(&k)) { - Ok(i) => Some(&self.vals[i].2), - Err(_) => None, - } - } - /// Gets a reference to all of the items, as a slice. Usefull to iterate on all map values. - /// In most case you will want to ignore the timestamp (second item of the tuple). - pub fn items(&self) -> &[(K, u64, V)] { - &self.vals[..] - } -} - -impl CRDT for LWWMap -where - K: Clone + Ord, - V: Clone + CRDT, -{ - fn merge(&mut self, other: &Self) { - for (k, ts2, v2) in other.vals.iter() { - match self.vals.binary_search_by(|(k2, _, _)| k2.cmp(&k)) { - Ok(i) => { - let (_, ts1, _v1) = &self.vals[i]; - if ts2 > ts1 { - self.vals[i].1 = *ts2; - self.vals[i].2 = v2.clone(); - } else if ts1 == ts2 { - self.vals[i].2.merge(&v2); - } - } - Err(i) => { - self.vals.insert(i, (k.clone(), *ts2, v2.clone())); - } - } - } - } -} diff --git a/src/table/crdt/bool.rs b/src/table/crdt/bool.rs new file mode 100644 index 00000000..1989c92e --- /dev/null +++ b/src/table/crdt/bool.rs @@ -0,0 +1,34 @@ +use serde::{Deserialize, Serialize}; + +use crate::crdt::crdt::*; + +/// Boolean, where `true` is an absorbing state +#[derive(Clone, Copy, Debug, Serialize, Deserialize, PartialEq)] +pub struct Bool(bool); + +impl Bool { + /// Create a new boolean with the specified value + pub fn new(b: bool) -> Self { + Self(b) + } + /// Set the boolean to true + pub fn set(&mut self) { + self.0 = true; + } + /// Get the boolean value + pub fn get(&self) -> bool { + self.0 + } +} + +impl From for Bool { + fn from(b: bool) -> Bool { + Bool::new(b) + } +} + +impl CRDT for Bool { + fn merge(&mut self, other: &Self) { + self.0 = self.0 || other.0; + } +} diff --git a/src/table/crdt/crdt.rs b/src/table/crdt/crdt.rs new file mode 100644 index 00000000..636b6df6 --- /dev/null +++ b/src/table/crdt/crdt.rs @@ -0,0 +1,73 @@ +use garage_util::data::*; + +/// Definition of a CRDT - all CRDT Rust types implement this. +/// +/// A CRDT is defined as a merge operator that respects a certain set of axioms. +/// +/// In particular, the merge operator must be commutative, associative, +/// idempotent, and monotonic. +/// In other words, if `a`, `b` and `c` are CRDTs, and `⊔` denotes the merge operator, +/// the following axioms must apply: +/// +/// ```text +/// a ⊔ b = b ⊔ a (commutativity) +/// (a ⊔ b) ⊔ c = a ⊔ (b ⊔ c) (associativity) +/// (a ⊔ b) ⊔ b = a ⊔ b (idempotence) +/// ``` +/// +/// Moreover, the relationship `≥` defined by `a ≥ b ⇔ ∃c. a = b ⊔ c` must be a partial order. +/// This implies a few properties such as: if `a ⊔ b ≠ a`, then there is no `c` such that `(a ⊔ b) ⊔ c = a`, +/// as this would imply a cycle in the partial order. +pub trait CRDT { + /// Merge the two datastructures according to the CRDT rules. + /// `self` is modified to contain the merged CRDT value. `other` is not modified. + /// + /// # Arguments + /// + /// * `other` - the other CRDT we wish to merge with + fn merge(&mut self, other: &Self); +} + +/// All types that implement `Ord` (a total order) can also implement a trivial CRDT +/// defined by the merge rule: `a ⊔ b = max(a, b)`. Implement this trait for your type +/// to enable this behavior. +pub trait AutoCRDT: Ord + Clone + std::fmt::Debug { + /// WARN_IF_DIFFERENT: emit a warning when values differ. Set this to true if + /// different values in your application should never happen. Set this to false + /// if you are actually relying on the semantics of `a ⊔ b = max(a, b)`. + const WARN_IF_DIFFERENT: bool; +} + +impl CRDT for T +where + T: AutoCRDT, +{ + fn merge(&mut self, other: &Self) { + if Self::WARN_IF_DIFFERENT && self != other { + warn!( + "Different CRDT values should be the same (logic error!): {:?} vs {:?}", + self, other + ); + if other > self { + *self = other.clone(); + } + warn!("Making an arbitrary choice: {:?}", self); + } else { + if other > self { + *self = other.clone(); + } + } + } +} + +impl AutoCRDT for String { + const WARN_IF_DIFFERENT: bool = true; +} + +impl AutoCRDT for bool { + const WARN_IF_DIFFERENT: bool = true; +} + +impl AutoCRDT for FixedBytes32 { + const WARN_IF_DIFFERENT: bool = true; +} diff --git a/src/table/crdt/lww.rs b/src/table/crdt/lww.rs new file mode 100644 index 00000000..9a3ab671 --- /dev/null +++ b/src/table/crdt/lww.rs @@ -0,0 +1,114 @@ +use serde::{Deserialize, Serialize}; + +use garage_util::data::now_msec; + +use crate::crdt::crdt::*; + +/// Last Write Win (LWW) +/// +/// An LWW CRDT associates a timestamp with a value, in order to implement a +/// time-based reconciliation rule: the most recent write wins. +/// For completeness, the LWW reconciliation rule must also be defined for two LWW CRDTs +/// with the same timestamp but different values. +/// +/// In our case, we add the constraint that the value that is wrapped inside the LWW CRDT must +/// itself be a CRDT: in the case when the timestamp does not allow us to decide on which value to +/// keep, the merge rule of the inner CRDT is applied on the wrapped values. (Note that all types +/// that implement the `Ord` trait get a default CRDT implemetnation that keeps the maximum value. +/// This enables us to use LWW directly with primitive data types such as numbers or strings. It is +/// generally desirable in this case to never explicitly produce LWW values with the same timestamp +/// but different inner values, as the rule to keep the maximum value isn't generally the desired +/// semantics.) +/// +/// As multiple computers clocks are always desynchronized, +/// when operations are close enough, it is equivalent to +/// take one copy and drop the other one. +/// +/// Given that clocks are not too desynchronized, this assumption +/// is enough for most cases, as there is few chance that two humans +/// coordonate themself faster than the time difference between two NTP servers. +/// +/// As a more concret example, let's suppose you want to upload a file +/// with the same key (path) in the same bucket at the very same time. +/// For each request, the file will be timestamped by the receiving server +/// and may differ from what you observed with your atomic clock! +/// +/// This scheme is used by AWS S3 or Soundcloud and often without knowing +/// in entreprise when reconciliating databases with ad-hoc scripts. +#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] +pub struct LWW { + ts: u64, + v: T, +} + +impl LWW +where + T: CRDT, +{ + /// Creates a new CRDT + /// + /// CRDT's internal timestamp is set with current node's clock. + pub fn new(value: T) -> Self { + Self { + ts: now_msec(), + v: value, + } + } + + /// Build a new CRDT from a previous non-compatible one + /// + /// Compared to new, the CRDT's timestamp is not set to now + /// but must be set to the previous, non-compatible, CRDT's timestamp. + pub fn migrate_from_raw(ts: u64, value: T) -> Self { + Self { ts, v: value } + } + + /// Update the LWW CRDT while keeping some causal ordering. + /// + /// The timestamp of the LWW CRDT is updated to be the current node's clock + /// at time of update, or the previous timestamp + 1 if that's bigger, + /// so that the new timestamp is always strictly larger than the previous one. + /// This ensures that merging the update with the old value will result in keeping + /// the updated value. + pub fn update(&mut self, new_value: T) { + self.ts = std::cmp::max(self.ts + 1, now_msec()); + self.v = new_value; + } + + /// Get the CRDT value + pub fn get(&self) -> &T { + &self.v + } + + /// Get a mutable reference to the CRDT's value + /// + /// This is usefull to mutate the inside value without changing the LWW timestamp. + /// When such mutation is done, the merge between two LWW values is done using the inner + /// CRDT's merge operation. This is usefull in the case where the inner CRDT is a large + /// data type, such as a map, and we only want to change a single item in the map. + /// To do this, we can produce a "CRDT delta", i.e. a LWW that contains only the modification. + /// This delta consists in a LWW with the same timestamp, and the map + /// inside only contains the updated value. + /// The advantage of such a delta is that it is much smaller than the whole map. + /// + /// Avoid using this if the inner data type is a primitive type such as a number or a string, + /// as you will then rely on the merge function defined on `Ord` types by keeping the maximum + /// of both values. + pub fn get_mut(&mut self) -> &mut T { + &mut self.v + } +} + +impl CRDT for LWW +where + T: Clone + CRDT, +{ + fn merge(&mut self, other: &Self) { + if other.ts > self.ts { + self.ts = other.ts; + self.v = other.v.clone(); + } else if other.ts == self.ts { + self.v.merge(&other.v); + } + } +} diff --git a/src/table/crdt/lww_map.rs b/src/table/crdt/lww_map.rs new file mode 100644 index 00000000..bd40f368 --- /dev/null +++ b/src/table/crdt/lww_map.rs @@ -0,0 +1,145 @@ +use serde::{Deserialize, Serialize}; + +use garage_util::data::now_msec; + +use crate::crdt::crdt::*; + +/// Last Write Win Map +/// +/// This types defines a CRDT for a map from keys to values. +/// The values have an associated timestamp, such that the last written value +/// takes precedence over previous ones. As for the simpler `LWW` type, the value +/// type `V` is also required to implement the CRDT trait. +/// We do not encourage mutating the values associated with a given key +/// without updating the timestamp, in fact at the moment we do not provide a `.get_mut()` +/// method that would allow that. +/// +/// Internally, the map is stored as a vector of keys and values, sorted by ascending key order. +/// This is why the key type `K` must implement `Ord` (and also to ensure a unique serialization, +/// such that two values can be compared for equality based on their hashes). As a consequence, +/// insertions take `O(n)` time. This means that LWWMap should be used for reasonably small maps. +/// However, note that even if we were using a more efficient data structure such as a `BTreeMap`, +/// the serialization cost `O(n)` would still have to be paid at each modification, so we are +/// actually not losing anything here. +#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] +pub struct LWWMap { + vals: Vec<(K, u64, V)>, +} + +impl LWWMap +where + K: Ord, + V: CRDT, +{ + /// Create a new empty map CRDT + pub fn new() -> Self { + Self { vals: vec![] } + } + /// Used to migrate from a map defined in an incompatible format. This produces + /// a map that contains a single item with the specified timestamp (copied from + /// the incompatible format). Do this as many times as you have items to migrate, + /// and put them all together using the CRDT merge operator. + pub fn migrate_from_raw_item(k: K, ts: u64, v: V) -> Self { + Self { + vals: vec![(k, ts, v)], + } + } + /// Returns a map that contains a single mapping from the specified key to the specified value. + /// This map is a mutator, or a delta-CRDT, such that when it is merged with the original map, + /// the previous value will be replaced with the one specified here. + /// The timestamp in the provided mutator is set to the maximum of the current system's clock + /// and 1 + the previous value's timestamp (if there is one), so that the new value will always + /// take precedence (LWW rule). + /// + /// 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); + /// ``` + /// + /// However extracting the mutator on its own and only sending that on the network is very + /// interesting as it is much smaller than the whole map. + pub fn update_mutator(&self, k: K, new_v: V) -> Self { + let new_vals = match self.vals.binary_search_by(|(k2, _, _)| k2.cmp(&k)) { + Ok(i) => { + let (_, old_ts, _) = self.vals[i]; + let new_ts = std::cmp::max(old_ts + 1, now_msec()); + vec![(k, new_ts, new_v)] + } + Err(_) => vec![(k, now_msec(), new_v)], + }; + Self { vals: new_vals } + } + /// Takes all of the values of the map and returns them. The current map is reset to the + /// 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)); + /// put_my_crdt_value(a); + /// ``` + /// + /// Of course in this simple example we could have written simply + /// `pyt_my_crdt_value(a.update_mutator(key_to_modify, new_value))`, + /// 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)); + /// put_my_crdt_value(a); + /// ``` + pub fn take_and_clear(&mut self) -> Self { + let vals = std::mem::replace(&mut self.vals, vec![]); + Self { vals } + } + /// Removes all values from the map + pub fn clear(&mut self) { + self.vals.clear(); + } + /// Get a reference to the value assigned to a key + pub fn get(&self, k: &K) -> Option<&V> { + match self.vals.binary_search_by(|(k2, _, _)| k2.cmp(&k)) { + Ok(i) => Some(&self.vals[i].2), + Err(_) => None, + } + } + /// Gets a reference to all of the items, as a slice. Usefull to iterate on all map values. + /// In most case you will want to ignore the timestamp (second item of the tuple). + pub fn items(&self) -> &[(K, u64, V)] { + &self.vals[..] + } + /// Returns the number of items in the map + pub fn len(&self) -> usize { + self.vals.len() + } +} + +impl CRDT for LWWMap +where + K: Clone + Ord, + V: Clone + CRDT, +{ + fn merge(&mut self, other: &Self) { + for (k, ts2, v2) in other.vals.iter() { + match self.vals.binary_search_by(|(k2, _, _)| k2.cmp(&k)) { + Ok(i) => { + let (_, ts1, _v1) = &self.vals[i]; + if ts2 > ts1 { + self.vals[i].1 = *ts2; + self.vals[i].2 = v2.clone(); + } else if ts1 == ts2 { + self.vals[i].2.merge(&v2); + } + } + Err(i) => { + self.vals.insert(i, (k.clone(), *ts2, v2.clone())); + } + } + } + } +} diff --git a/src/table/crdt/map.rs b/src/table/crdt/map.rs new file mode 100644 index 00000000..1193e6db --- /dev/null +++ b/src/table/crdt/map.rs @@ -0,0 +1,83 @@ +use serde::{Deserialize, Serialize}; + +use crate::crdt::crdt::*; + +/// Simple CRDT Map +/// +/// This types defines a CRDT for a map from keys to values. Values are CRDT types which +/// can have their own updating logic. +/// +/// Internally, the map is stored as a vector of keys and values, sorted by ascending key order. +/// This is why the key type `K` must implement `Ord` (and also to ensure a unique serialization, +/// such that two values can be compared for equality based on their hashes). As a consequence, +/// insertions take `O(n)` time. This means that Map should be used for reasonably small maps. +/// However, note that even if we were using a more efficient data structure such as a `BTreeMap`, +/// the serialization cost `O(n)` would still have to be paid at each modification, so we are +/// actually not losing anything here. +#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] +pub struct Map { + vals: Vec<(K, V)>, +} + +impl Map +where + K: Clone + Ord, + V: Clone + CRDT, +{ + /// Create a new empty map CRDT + pub fn new() -> Self { + Self { vals: vec![] } + } + + /// Returns a map that contains a single mapping from the specified key to the specified value. + /// This can be used to build a delta-mutator: + /// when merged with another map, the value will be added or CRDT-merged if a previous + /// value already exists. + pub fn put_mutator(k: K, v: V) -> Self { + Self { vals: vec![(k, v)] } + } + + pub fn put(&mut self, k: K, v: V) { + self.merge(&Self::put_mutator(k, v)); + } + + /// Removes all values from the map + pub fn clear(&mut self) { + self.vals.clear(); + } + + /// Get a reference to the value assigned to a key + pub fn get(&self, k: &K) -> Option<&V> { + match self.vals.binary_search_by(|(k2, _)| k2.cmp(&k)) { + Ok(i) => Some(&self.vals[i].1), + Err(_) => None, + } + } + /// Gets a reference to all of the items, as a slice. Usefull to iterate on all map values. + pub fn items(&self) -> &[(K, V)] { + &self.vals[..] + } + /// Returns the number of items in the map + pub fn len(&self) -> usize { + self.vals.len() + } +} + +impl CRDT for Map +where + K: Clone + Ord, + V: Clone + CRDT, +{ + fn merge(&mut self, other: &Self) { + for (k, v2) in other.vals.iter() { + match self.vals.binary_search_by(|(k2, _)| k2.cmp(&k)) { + Ok(i) => { + self.vals[i].1.merge(&v2); + } + Err(i) => { + self.vals.insert(i, (k.clone(), v2.clone())); + } + } + } + } +} diff --git a/src/table/crdt/mod.rs b/src/table/crdt/mod.rs new file mode 100644 index 00000000..eb75d061 --- /dev/null +++ b/src/table/crdt/mod.rs @@ -0,0 +1,22 @@ +//! This package provides a simple implementation of conflict-free replicated data types (CRDTs) +//! +//! CRDTs are a type of data structures that do not require coordination. In other words, we can +//! edit them in parallel, we will always find a way to merge it. +//! +//! A general example is a counter. Its initial value is 0. Alice and Bob get a copy of the +//! counter. Alice does +1 on her copy, she reads 1. Bob does +3 on his copy, he reads 3. Now, +//! it is easy to merge their counters, order does not count: we always get 4. +//! +//! Learn more about CRDT [on Wikipedia](https://en.wikipedia.org/wiki/Conflict-free_replicated_data_type) + +mod bool; +mod crdt; +mod lww; +mod lww_map; +mod map; + +pub use self::bool::*; +pub use crdt::*; +pub use lww::*; +pub use lww_map::*; +pub use map::*; diff --git a/src/table/schema.rs b/src/table/schema.rs index edd04000..5b789a02 100644 --- a/src/table/schema.rs +++ b/src/table/schema.rs @@ -2,6 +2,8 @@ use serde::{Deserialize, Serialize}; use garage_util::data::*; +use crate::crdt::CRDT; + pub trait PartitionKey { fn hash(&self) -> Hash; } @@ -35,12 +37,10 @@ impl SortKey for Hash { } pub trait Entry: - PartialEq + Clone + Serialize + for<'de> Deserialize<'de> + Send + Sync + CRDT + PartialEq + Clone + Serialize + for<'de> Deserialize<'de> + Send + Sync { fn partition_key(&self) -> &P; fn sort_key(&self) -> &S; - - fn merge(&mut self, other: &Self); } pub trait TableSchema: Send + Sync { diff --git a/src/table/table.rs b/src/table/table.rs index 1f6b7d25..366ce925 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -17,6 +17,7 @@ use garage_rpc::ring::Ring; use garage_rpc::rpc_client::*; use garage_rpc::rpc_server::*; +use crate::crdt::CRDT; use crate::schema::*; use crate::table_sync::*; -- cgit v1.2.3 From 2afd2c81baacf66f00333de3f0fb18df0b497b41 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 10 Mar 2021 16:23:57 +0100 Subject: Change hash function to blake2 for partition keys based on strings --- src/table/schema.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/table/schema.rs b/src/table/schema.rs index 5b789a02..7fbb7b25 100644 --- a/src/table/schema.rs +++ b/src/table/schema.rs @@ -10,7 +10,7 @@ pub trait PartitionKey { impl PartitionKey for String { fn hash(&self) -> Hash { - sha256sum(self.as_bytes()) + blake2sum(self.as_bytes()) } } -- cgit v1.2.3 From 0fd7df8fa0741caa622b2bae979aac867f443154 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 10 Mar 2021 16:33:31 +0100 Subject: Switch to blake2 sum for identifying blocks by their data --- src/api/s3_put.rs | 54 +++++++++++++++++++++++++++--------------------------- src/model/block.rs | 2 +- 2 files changed, 28 insertions(+), 28 deletions(-) diff --git a/src/api/s3_put.rs b/src/api/s3_put.rs index 37a1ece2..1939f2cf 100644 --- a/src/api/s3_put.rs +++ b/src/api/s3_put.rs @@ -52,14 +52,14 @@ pub async fn handle_put( if first_block.len() < INLINE_THRESHOLD { let mut md5sum = Md5::new(); md5sum.update(&first_block[..]); - let md5sum_arr = md5sum.finalize(); - let md5sum_hex = hex::encode(md5sum_arr); + let data_md5sum = md5sum.finalize(); + let data_md5sum_hex = hex::encode(data_md5sum); - let sha256sum_hash = sha256sum(&first_block[..]); + let data_sha256sum = sha256sum(&first_block[..]); ensure_checksum_matches( - md5sum_arr.as_slice(), - sha256sum_hash, + data_md5sum.as_slice(), + data_sha256sum, content_md5.as_deref(), content_sha256, )?; @@ -71,7 +71,7 @@ pub async fn handle_put( ObjectVersionMeta { headers, size: first_block.len() as u64, - etag: md5sum_hex.clone(), + etag: data_md5sum_hex.clone(), }, first_block, )), @@ -80,7 +80,7 @@ pub async fn handle_put( let object = Object::new(bucket.into(), key.into(), vec![object_version]); garage.object_table.insert(&object).await?; - return Ok(put_response(version_uuid, md5sum_hex)); + return Ok(put_response(version_uuid, data_md5sum_hex)); } // Write version identifier in object table so that we have a trace @@ -95,7 +95,7 @@ pub async fn handle_put( // Initialize corresponding entry in version table let version = Version::new(version_uuid, bucket.into(), key.into(), false); - let first_block_hash = sha256sum(&first_block[..]); + let first_block_hash = blake2sum(&first_block[..]); // Transfer data and verify checksum let tx_result = read_and_put_blocks( @@ -107,14 +107,14 @@ pub async fn handle_put( &mut chunker, ) .await - .and_then(|(total_size, md5sum_arr, sha256sum)| { + .and_then(|(total_size, data_md5sum, data_sha256sum)| { ensure_checksum_matches( - md5sum_arr.as_slice(), - sha256sum, + data_md5sum.as_slice(), + data_sha256sum, content_md5.as_deref(), content_sha256, ) - .map(|()| (total_size, md5sum_arr)) + .map(|()| (total_size, data_md5sum)) }); // If something went wrong, clean up @@ -148,13 +148,13 @@ pub async fn handle_put( /// Validate MD5 sum against content-md5 header /// and sha256sum against signed content-sha256 fn ensure_checksum_matches( - md5sum: &[u8], - sha256sum: garage_util::data::FixedBytes32, + data_md5sum: &[u8], + data_sha256sum: garage_util::data::FixedBytes32, content_md5: Option<&str>, content_sha256: Option, ) -> Result<(), Error> { if let Some(expected_sha256) = content_sha256 { - if expected_sha256 != sha256sum { + if expected_sha256 != data_sha256sum { return Err(Error::BadRequest(format!( "Unable to validate x-amz-content-sha256" ))); @@ -163,7 +163,7 @@ fn ensure_checksum_matches( } } if let Some(expected_md5) = content_md5 { - if expected_md5.trim_matches('"') != base64::encode(md5sum) { + if expected_md5.trim_matches('"') != base64::encode(data_md5sum) { return Err(Error::BadRequest(format!("Unable to validate content-md5"))); } else { trace!("Successfully validated content-md5"); @@ -204,7 +204,7 @@ async fn read_and_put_blocks( if let Some(block) = next_block { md5hasher.update(&block[..]); sha256hasher.input(&block[..]); - let block_hash = sha256sum(&block[..]); + let block_hash = blake2sum(&block[..]); let block_len = block.len(); put_curr_version_block = put_block_meta( garage.clone(), @@ -222,14 +222,14 @@ async fn read_and_put_blocks( } let total_size = next_offset as u64; - let md5sum_arr = md5hasher.finalize(); + let data_md5sum = md5hasher.finalize(); - let sha256sum_arr = sha256hasher.result(); + let data_sha256sum = sha256hasher.result(); let mut hash = [0u8; 32]; - hash.copy_from_slice(&sha256sum_arr[..]); - let sha256sum_arr = Hash::from(hash); + hash.copy_from_slice(&data_sha256sum[..]); + let data_sha256sum = Hash::from(hash); - Ok((total_size, md5sum_arr, sha256sum_arr)) + Ok((total_size, data_md5sum, data_sha256sum)) } async fn put_block_meta( @@ -389,8 +389,8 @@ pub async fn handle_put_part( // Copy block to store let version = Version::new(version_uuid, bucket, key, false); - let first_block_hash = sha256sum(&first_block[..]); - let (_, md5sum_arr, sha256sum) = read_and_put_blocks( + let first_block_hash = blake2sum(&first_block[..]); + let (_, data_md5sum, data_sha256sum) = read_and_put_blocks( &garage, version, part_number, @@ -401,14 +401,14 @@ pub async fn handle_put_part( .await?; ensure_checksum_matches( - md5sum_arr.as_slice(), - sha256sum, + data_md5sum.as_slice(), + data_sha256sum, content_md5.as_deref(), content_sha256, )?; let response = Response::builder() - .header("ETag", format!("\"{}\"", hex::encode(md5sum_arr))) + .header("ETag", format!("\"{}\"", hex::encode(data_md5sum))) .body(Body::from(vec![])) .unwrap(); Ok(response) diff --git a/src/model/block.rs b/src/model/block.rs index d3957403..57f4c077 100644 --- a/src/model/block.rs +++ b/src/model/block.rs @@ -176,7 +176,7 @@ impl BlockManager { f.read_to_end(&mut data).await?; drop(f); - if data::sha256sum(&data[..]) != *hash { + if data::blake2sum(&data[..]) != *hash { let _lock = self.data_dir_lock.lock().await; warn!("Block {:?} is corrupted. Deleting and resyncing.", hash); fs::remove_file(path).await?; -- cgit v1.2.3 From 445912dc6a3b65d0726b9378b3542b4061272cf4 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 10 Mar 2021 16:38:31 +0100 Subject: Remove migration paths from 0.1 branch --- Cargo.lock | 260 +++++----------------------------------------- src/garage/Cargo.toml | 1 - src/garage/server.rs | 23 +--- src/model/Cargo.toml | 1 - src/model/bucket_table.rs | 40 ------- src/model/key_table.rs | 28 ----- src/model/object_table.rs | 52 ---------- 7 files changed, 26 insertions(+), 379 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index a34dea28..6d4cc6a3 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -194,15 +194,6 @@ dependencies = [ "unicode-width", ] -[[package]] -name = "cloudabi" -version = "0.0.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ddfc5b9aa5d4507acaf872de71051dfd0e309860e88966e1051e462a077aac4f" -dependencies = [ - "bitflags", -] - [[package]] name = "crc32fast" version = "1.2.1" @@ -212,21 +203,6 @@ dependencies = [ "cfg-if 1.0.0", ] -[[package]] -name = "crossbeam-epoch" -version = "0.8.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "058ed274caafc1f60c4997b5fc07bf7dc7cca454af7c6e81edffe5f33f70dace" -dependencies = [ - "autocfg", - "cfg-if 0.1.10", - "crossbeam-utils 0.7.2", - "lazy_static", - "maybe-uninit", - "memoffset 0.5.6", - "scopeguard", -] - [[package]] name = "crossbeam-epoch" version = "0.9.2" @@ -234,24 +210,13 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d60ab4a8dba064f2fbb5aa270c28da5cf4bbd0e72dae1140a6b0353a779dbe00" dependencies = [ "cfg-if 1.0.0", - "crossbeam-utils 0.8.2", + "crossbeam-utils", "lazy_static", "loom", - "memoffset 0.6.1", + "memoffset", "scopeguard", ] -[[package]] -name = "crossbeam-utils" -version = "0.7.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3c7c73a2d1e9fc0886a08b93e98eb643461230d5f1925e4036204d5f2e261a8" -dependencies = [ - "autocfg", - "cfg-if 0.1.10", - "lazy_static", -] - [[package]] name = "crossbeam-utils" version = "0.8.2" @@ -518,9 +483,9 @@ dependencies = [ "futures-util", "garage_api", "garage_model", - "garage_rpc 0.1.1", - "garage_table 0.1.1", - "garage_util 0.1.1", + "garage_rpc", + "garage_table", + "garage_util", "garage_web", "hex", "log", @@ -529,8 +494,7 @@ dependencies = [ "rmp-serde", "serde", "sha2", - "sled 0.31.0", - "sled 0.34.6", + "sled", "structopt", "tokio", "toml", @@ -548,8 +512,8 @@ dependencies = [ "futures", "futures-util", "garage_model", - "garage_table 0.1.1", - "garage_util 0.1.1", + "garage_table", + "garage_util", "hex", "hmac", "http", @@ -575,35 +539,9 @@ dependencies = [ "bytes 0.4.12", "futures", "futures-util", - "garage_model_010b", - "garage_rpc 0.1.1", - "garage_table 0.1.1", - "garage_util 0.1.1", - "hex", - "log", - "rand 0.7.3", - "rmp-serde", - "serde", - "serde_bytes", - "sha2", - "sled 0.34.6", - "tokio", -] - -[[package]] -name = "garage_model_010b" -version = "0.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aa7bd48aaf0f423a4d18e762c8c8c19a230f9fc237c63c1356ded0463a810428" -dependencies = [ - "arc-swap", - "async-trait", - "bytes 0.4.12", - "futures", - "futures-util", - "garage_rpc 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)", - "garage_table 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)", - "garage_util 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)", + "garage_rpc", + "garage_table", + "garage_util", "hex", "log", "rand 0.7.3", @@ -611,7 +549,7 @@ dependencies = [ "serde", "serde_bytes", "sha2", - "sled 0.31.0", + "sled", "tokio", ] @@ -623,7 +561,7 @@ dependencies = [ "bytes 0.4.12", "futures", "futures-util", - "garage_util 0.1.1", + "garage_util", "gethostname", "hex", "http", @@ -641,69 +579,17 @@ dependencies = [ "webpki", ] -[[package]] -name = "garage_rpc" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d374863eb8eff7a9593dbbb73e4c0ea33cd8e88ffc6ac77076ae36721bda805c" -dependencies = [ - "arc-swap", - "bytes 0.4.12", - "futures", - "futures-util", - "garage_util 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)", - "gethostname", - "hex", - "http", - "hyper", - "hyper-rustls", - "log", - "rand 0.7.3", - "rmp-serde", - "rustls", - "serde", - "serde_json", - "sha2", - "tokio", - "tokio-rustls", - "webpki", -] - -[[package]] -name = "garage_table" -version = "0.1.1" -dependencies = [ - "arc-swap", - "async-trait", - "bytes 0.4.12", - "futures", - "futures-util", - "garage_rpc 0.1.1", - "garage_util 0.1.1", - "hex", - "hexdump", - "log", - "rand 0.7.3", - "rmp-serde", - "serde", - "serde_bytes", - "sled 0.34.6", - "tokio", -] - [[package]] name = "garage_table" version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "65d14bc1a342b4806a824ece64d37cb849ea5b1c015be9946695b5c163807154" dependencies = [ "arc-swap", "async-trait", "bytes 0.4.12", "futures", "futures-util", - "garage_rpc 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)", - "garage_util 0.1.1 (registry+https://github.com/rust-lang/crates.io-index)", + "garage_rpc", + "garage_util", "hex", "hexdump", "log", @@ -711,7 +597,7 @@ dependencies = [ "rmp-serde", "serde", "serde_bytes", - "sled 0.31.0", + "sled", "tokio", ] @@ -735,33 +621,7 @@ dependencies = [ "serde", "serde_json", "sha2", - "sled 0.34.6", - "tokio", - "toml", - "webpki", -] - -[[package]] -name = "garage_util" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "393cc63d44006449fc72c15760aa2962a0e47d95ed8d9d82713aac7decb7f988" -dependencies = [ - "err-derive", - "futures", - "futures-util", - "hex", - "http", - "hyper", - "log", - "rand 0.7.3", - "rmp-serde", - "roxmltree", - "rustls", - "serde", - "serde_json", - "sha2", - "sled 0.31.0", + "sled", "tokio", "toml", "webpki", @@ -775,8 +635,8 @@ dependencies = [ "futures", "garage_api", "garage_model", - "garage_table 0.1.1", - "garage_util 0.1.1", + "garage_table", + "garage_util", "http", "hyper", "idna", @@ -1084,15 +944,6 @@ version = "0.2.82" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "89203f3fba0a3795506acaad8ebce3c80c0af93f994d5a1d7a0b1eeb23271929" -[[package]] -name = "lock_api" -version = "0.3.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4da24a77a3d8a6d4862d95f72e6fdb9c09a643ecdb402d754004a557f2bec75" -dependencies = [ - "scopeguard", -] - [[package]] name = "lock_api" version = "0.4.2" @@ -1128,12 +979,6 @@ version = "0.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7ffc5c5338469d4d3ea17d269fa8ea3512ad247247c30bd2df69e68309ed0a08" -[[package]] -name = "maybe-uninit" -version = "2.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "60302e4db3a61da70c0cb7991976248362f30319e88850c487b9b95bbf059e00" - [[package]] name = "md-5" version = "0.9.1" @@ -1151,15 +996,6 @@ version = "2.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0ee1c47aaa256ecabcaea351eae4a9b01ef39ed810004e298d2511ed284b1525" -[[package]] -name = "memoffset" -version = "0.5.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "043175f069eda7b85febe4a74abbaeff828d9f8b448515d3151a14a3542811aa" -dependencies = [ - "autocfg", -] - [[package]] name = "memoffset" version = "0.6.1" @@ -1281,16 +1117,6 @@ version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "624a8340c38c1b80fd549087862da4ba43e08858af025b236e509b6649fc13d5" -[[package]] -name = "parking_lot" -version = "0.10.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d3a704eb390aafdc107b0e392f56a82b668e3a71366993b5340f5833fd62505e" -dependencies = [ - "lock_api 0.3.4", - "parking_lot_core 0.7.2", -] - [[package]] name = "parking_lot" version = "0.11.1" @@ -1298,22 +1124,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6d7744ac029df22dca6284efe4e898991d28e3085c706c972bcd7da4a27a15eb" dependencies = [ "instant", - "lock_api 0.4.2", - "parking_lot_core 0.8.3", -] - -[[package]] -name = "parking_lot_core" -version = "0.7.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d58c7c768d4ba344e3e8d72518ac13e259d7c7ade24167003b8488e10b6740a3" -dependencies = [ - "cfg-if 0.1.10", - "cloudabi", - "libc", - "redox_syscall 0.1.57", - "smallvec", - "winapi 0.3.9", + "lock_api", + "parking_lot_core", ] [[package]] @@ -1325,7 +1137,7 @@ dependencies = [ "cfg-if 1.0.0", "instant", "libc", - "redox_syscall 0.2.5", + "redox_syscall", "smallvec", "winapi 0.3.9", ] @@ -1548,12 +1360,6 @@ dependencies = [ "rand_core 0.3.1", ] -[[package]] -name = "redox_syscall" -version = "0.1.57" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "41cc0f7e4d5d4544e8861606a285bb08d3e70712ccc7d2b84d7c0ccfaf4b05ce" - [[package]] name = "redox_syscall" version = "0.2.5" @@ -1746,22 +1552,6 @@ version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c111b5bd5695e56cffe5129854aa230b39c93a305372fdbb2668ca2394eea9f8" -[[package]] -name = "sled" -version = "0.31.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8fb6824dde66ad33bf20c6e8476f5b82b871bc8bc3c129a10ea2f7dae5060fa3" -dependencies = [ - "crc32fast", - "crossbeam-epoch 0.8.2", - "crossbeam-utils 0.7.2", - "fs2", - "fxhash", - "libc", - "log", - "parking_lot 0.10.2", -] - [[package]] name = "sled" version = "0.34.6" @@ -1769,13 +1559,13 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1d0132f3e393bcb7390c60bb45769498cf4550bcb7a21d7f95c02b69f6362cdc" dependencies = [ "crc32fast", - "crossbeam-epoch 0.9.2", - "crossbeam-utils 0.8.2", + "crossbeam-epoch", + "crossbeam-utils", "fs2", "fxhash", "libc", "log", - "parking_lot 0.11.1", + "parking_lot", ] [[package]] diff --git a/src/garage/Cargo.toml b/src/garage/Cargo.toml index 03bc472d..115e2d0e 100644 --- a/src/garage/Cargo.toml +++ b/src/garage/Cargo.toml @@ -29,7 +29,6 @@ log = "0.4" pretty_env_logger = "0.4" sled = "0.34" -old_sled = { package = "sled", version = "0.31" } structopt = { version = "0.3", default-features = false } toml = "0.5" diff --git a/src/garage/server.rs b/src/garage/server.rs index 2e109f8b..a0ab17c4 100644 --- a/src/garage/server.rs +++ b/src/garage/server.rs @@ -40,28 +40,7 @@ pub async fn run_server(config_file: PathBuf) -> Result<(), Error> { info!("Opening database..."); let mut db_path = config.metadata_dir.clone(); db_path.push("db"); - let db = match sled::open(&db_path) { - Ok(db) => db, - Err(e) => { - warn!("Old DB could not be openned ({}), attempting migration.", e); - let old = old_sled::open(&db_path).expect("Unable to open old DB for migration"); - let mut new_path = config.metadata_dir.clone(); - new_path.push("db2"); - let new = sled::open(&new_path).expect("Unable to open new DB for migration"); - new.import(old.export()); - if old.checksum().expect("unable to compute old db checksum") - != new.checksum().expect("unable to compute new db checksum") - { - panic!("db checksums don't match after migration"); - } - drop(new); - drop(old); - std::fs::remove_dir_all(&db_path).expect("Cannot remove old DB folder"); - std::fs::rename(new_path, &db_path) - .expect("Cannot move new DB folder to correct place"); - sled::open(db_path).expect("Unable to open new DB after migration") - } - }; + let db = sled::open(&db_path).expect("Unable to open sled DB"); info!("Initialize RPC server..."); let mut rpc_server = RpcServer::new(config.rpc_bind_addr.clone(), config.rpc_tls.clone()); diff --git a/src/model/Cargo.toml b/src/model/Cargo.toml index 48b75d24..caeed66c 100644 --- a/src/model/Cargo.toml +++ b/src/model/Cargo.toml @@ -16,7 +16,6 @@ path = "lib.rs" garage_util = { version = "0.1.1", path = "../util" } garage_rpc = { version = "0.1.1", path = "../rpc" } garage_table = { version = "0.1.1", path = "../table" } -model010 = { package = "garage_model_010b", version = "0.0.1" } bytes = "0.4" rand = "0.7" diff --git a/src/model/bucket_table.rs b/src/model/bucket_table.rs index 5bc8b7f9..6330dced 100644 --- a/src/model/bucket_table.rs +++ b/src/model/bucket_table.rs @@ -5,11 +5,6 @@ use garage_table::*; use crate::key_table::PermissionSet; -// We import the same file but in its version 0.1.0. -// We can then access v0.1.0 data structures. -// 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: @@ -108,39 +103,4 @@ impl TableSchema for BucketTable { fn matches_filter(entry: &Self::E, filter: &Self::Filter) -> bool { filter.apply(entry.is_deleted()) } - - fn try_migrate(bytes: &[u8]) -> Option { - let old = match rmp_serde::decode::from_read_ref::<_, prev::Bucket>(bytes) { - Ok(x) => x, - Err(_) => return None, - }; - if old.deleted { - Some(Bucket { - name: old.name, - state: crdt::LWW::migrate_from_raw(old.timestamp, BucketState::Deleted), - }) - } else { - let mut keys = crdt::LWWMap::new(); - for ak in old.authorized_keys() { - keys.merge(&crdt::LWWMap::migrate_from_raw_item( - ak.key_id.clone(), - ak.timestamp, - PermissionSet { - allow_read: ak.allow_read, - allow_write: ak.allow_write, - }, - )); - } - - 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(params)), - }) - } - } } diff --git a/src/model/key_table.rs b/src/model/key_table.rs index 6d8cc6c0..b4ab65b6 100644 --- a/src/model/key_table.rs +++ b/src/model/key_table.rs @@ -3,8 +3,6 @@ use serde::{Deserialize, Serialize}; use garage_table::crdt::*; use garage_table::*; -use model010::key_table as prev; - #[derive(PartialEq, Clone, Debug, Serialize, Deserialize)] pub struct Key { // Primary key @@ -103,30 +101,4 @@ impl TableSchema for KeyTable { fn matches_filter(entry: &Self::E, filter: &Self::Filter) -> bool { filter.apply(entry.deleted.get()) } - - fn try_migrate(bytes: &[u8]) -> Option { - let old = match rmp_serde::decode::from_read_ref::<_, prev::Key>(bytes) { - Ok(x) => x, - Err(_) => return None, - }; - let mut new = Self::E { - key_id: old.key_id.clone(), - secret_key: old.secret_key.clone(), - name: crdt::LWW::migrate_from_raw(old.name_timestamp, old.name.clone()), - deleted: crdt::Bool::new(old.deleted), - authorized_buckets: crdt::LWWMap::new(), - }; - for ab in old.authorized_buckets() { - let it = crdt::LWWMap::migrate_from_raw_item( - ab.bucket.clone(), - ab.timestamp, - PermissionSet { - allow_read: ab.allow_read, - allow_write: ab.allow_write, - }, - ); - new.authorized_buckets.merge(&it); - } - Some(new) - } } diff --git a/src/model/object_table.rs b/src/model/object_table.rs index 75c37f6d..3280e7b5 100644 --- a/src/model/object_table.rs +++ b/src/model/object_table.rs @@ -11,8 +11,6 @@ use garage_table::*; use crate::version_table::*; -use model010::object_table as prev; - #[derive(PartialEq, Clone, Debug, Serialize, Deserialize)] pub struct Object { // Primary key @@ -226,55 +224,5 @@ impl TableSchema for ObjectTable { let deleted = !entry.versions.iter().any(|v| v.is_data()); filter.apply(deleted) } - - fn try_migrate(bytes: &[u8]) -> Option { - let old = match rmp_serde::decode::from_read_ref::<_, prev::Object>(bytes) { - Ok(x) => x, - Err(_) => return None, - }; - let new_v = old - .versions() - .iter() - .map(migrate_version) - .collect::>(); - let new = Object::new(old.bucket.clone(), old.key.clone(), new_v); - Some(new) - } } -fn migrate_version(old: &prev::ObjectVersion) -> ObjectVersion { - let headers = ObjectVersionHeaders { - content_type: old.mime_type.clone(), - other: BTreeMap::new(), - }; - let meta = ObjectVersionMeta { - headers: headers.clone(), - size: old.size, - etag: "".to_string(), - }; - let state = match old.state { - prev::ObjectVersionState::Uploading => ObjectVersionState::Uploading(headers), - prev::ObjectVersionState::Aborted => ObjectVersionState::Aborted, - prev::ObjectVersionState::Complete => match &old.data { - prev::ObjectVersionData::Uploading => ObjectVersionState::Uploading(headers), - prev::ObjectVersionData::DeleteMarker => { - ObjectVersionState::Complete(ObjectVersionData::DeleteMarker) - } - prev::ObjectVersionData::Inline(x) => { - ObjectVersionState::Complete(ObjectVersionData::Inline(meta, x.clone())) - } - prev::ObjectVersionData::FirstBlock(h) => { - let mut hash = [0u8; 32]; - hash.copy_from_slice(h.as_ref()); - ObjectVersionState::Complete(ObjectVersionData::FirstBlock(meta, Hash::from(hash))) - } - }, - }; - let mut uuid = [0u8; 32]; - uuid.copy_from_slice(old.uuid.as_ref()); - ObjectVersion { - uuid: UUID::from(uuid), - timestamp: old.timestamp, - state, - } -} -- cgit v1.2.3 From af7600f989d79d07253405647973828435f9d16c Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 10 Mar 2021 17:01:05 +0100 Subject: Correctly implement CompleteMultipartUpload with etag check of parts --- src/api/s3_put.rs | 49 ++++++++++++++++++++++++++++------------------ src/model/version_table.rs | 6 ++++++ 2 files changed, 36 insertions(+), 19 deletions(-) diff --git a/src/api/s3_put.rs b/src/api/s3_put.rs index 1939f2cf..7aec12ed 100644 --- a/src/api/s3_put.rs +++ b/src/api/s3_put.rs @@ -100,7 +100,7 @@ pub async fn handle_put( // Transfer data and verify checksum let tx_result = read_and_put_blocks( &garage, - version, + &version, 1, first_block, first_block_hash, @@ -174,7 +174,7 @@ fn ensure_checksum_matches( async fn read_and_put_blocks( garage: &Arc, - version: Version, + version: &Version, part_number: u64, first_block: Vec, first_block_hash: Hash, @@ -392,7 +392,7 @@ pub async fn handle_put_part( let first_block_hash = blake2sum(&first_block[..]); let (_, data_md5sum, data_sha256sum) = read_and_put_blocks( &garage, - version, + &version, part_number, first_block, first_block_hash, @@ -400,6 +400,7 @@ pub async fn handle_put_part( ) .await?; + // Verify that checksums map ensure_checksum_matches( data_md5sum.as_slice(), data_sha256sum, @@ -407,8 +408,14 @@ pub async fn handle_put_part( 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!("\"{}\"", hex::encode(data_md5sum))) + .header("ETag", format!("\"{}\"", data_md5sum_hex)) .body(Body::from(vec![])) .unwrap(); Ok(response) @@ -463,43 +470,46 @@ pub async fn handle_complete_multipart_upload( }; // Check that the list of parts they gave us corresponds to the parts we have here - // TODO: check MD5 sum of all uploaded parts? but that would mean we have to store them somewhere... - let mut parts = version - .blocks + println!("Expected parts from request: {:?}", body_list_of_parts); + println!("Parts stored in version: {:?}", version.parts_etags.items()); + let parts = version + .parts_etags .items() .iter() - .map(|x| x.0.part_number) - .collect::>(); - parts.dedup(); + .map(|pair| (&pair.0, &pair.1)); let same_parts = body_list_of_parts .iter() - .map(|x| &x.part_number) - .eq(parts.iter()); + .map(|x| (&x.part_number, &x.etag)) + .eq(parts); if !same_parts { return Err(Error::BadRequest(format!("We don't have the same parts"))); } - // ETag calculation: we produce ETags that have the same form as - // those of S3 multipart uploads, but we don't use their actual - // calculation for the first part (we use random bytes). This - // shouldn't impact compatibility as the S3 docs specify that - // the ETag is an opaque value in case of a multipart upload. - // See also: https://teppen.io/2018/06/23/aws_s3_etags/ + // 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 = version.blocks.items().last().unwrap().0.part_number - version.blocks.items().first().unwrap().0.part_number + 1; + 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(&rand::random::<[u8; 16]>()[..]), + 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) .fold(0, |x, y| x + y); + + // Write final object version object_version.state = ObjectVersionState::Complete(ObjectVersionData::FirstBlock( ObjectVersionMeta { headers, @@ -512,6 +522,7 @@ pub async fn handle_complete_multipart_upload( let final_object = Object::new(bucket.clone(), key.clone(), vec![object_version]); garage.object_table.insert(&final_object).await?; + // Send response saying ok we're done let mut xml = String::new(); writeln!(&mut xml, r#""#).unwrap(); writeln!( diff --git a/src/model/version_table.rs b/src/model/version_table.rs index 26abb64e..7ccc6a33 100644 --- a/src/model/version_table.rs +++ b/src/model/version_table.rs @@ -16,8 +16,11 @@ pub struct Version { pub uuid: UUID, // Actual data: the blocks for this version + // In the case of a multipart upload, also store the etags + // of individual parts and check them when doing CompleteMultipartUpload pub deleted: crdt::Bool, pub blocks: crdt::Map, + pub parts_etags: crdt::Map, // Back link to bucket+key so that we can figure if // this was deleted later on @@ -31,6 +34,7 @@ impl Version { uuid, deleted: deleted.into(), blocks: crdt::Map::new(), + parts_etags: crdt::Map::new(), bucket, key, } @@ -82,8 +86,10 @@ impl CRDT for Version { if self.deleted.get() { self.blocks.clear(); + self.parts_etags.clear(); } else { self.blocks.merge(&other.blocks); + self.parts_etags.merge(&other.parts_etags); } } } -- cgit v1.2.3 From 3214dd52dd144c99353830d7340ea158e262b06f Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 10 Mar 2021 21:50:09 +0100 Subject: Very minor changes --- src/api/s3_put.rs | 10 ++++------ src/model/object_table.rs | 1 - src/rpc/ring.rs | 2 +- src/table/table_sharded.rs | 1 + src/table/table_sync.rs | 12 ++++++++++-- 5 files changed, 16 insertions(+), 10 deletions(-) diff --git a/src/api/s3_put.rs b/src/api/s3_put.rs index 7aec12ed..2c5e364f 100644 --- a/src/api/s3_put.rs +++ b/src/api/s3_put.rs @@ -411,7 +411,9 @@ pub async fn handle_put_part( // 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()); + version + .parts_etags + .put(part_number, data_md5sum_hex.clone()); garage.version_table.insert(&version).await?; let response = Response::builder() @@ -495,11 +497,7 @@ pub async fn handle_complete_multipart_upload( 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 - ); + let etag = format!("{}-{}", hex::encode(etag_md5_hasher.finalize()), num_parts); // Calculate total size of final object let total_size = version diff --git a/src/model/object_table.rs b/src/model/object_table.rs index 3280e7b5..cd09f678 100644 --- a/src/model/object_table.rs +++ b/src/model/object_table.rs @@ -225,4 +225,3 @@ impl TableSchema for ObjectTable { filter.apply(deleted) } } - diff --git a/src/rpc/ring.rs b/src/rpc/ring.rs index 85caafeb..0f86bbb2 100644 --- a/src/rpc/ring.rs +++ b/src/rpc/ring.rs @@ -172,7 +172,7 @@ impl Ring { pub fn walk_ring(&self, from: &Hash, n: usize) -> Vec { if self.ring.len() != 1 << PARTITION_BITS { - warn!("Ring not yet ready, read/writes will be lost"); + warn!("Ring not yet ready, read/writes will be lost!"); return vec![]; } diff --git a/src/table/table_sharded.rs b/src/table/table_sharded.rs index 098637dd..47bdfeaf 100644 --- a/src/table/table_sharded.rs +++ b/src/table/table_sharded.rs @@ -44,6 +44,7 @@ impl TableReplication for TableShardedReplication { fn split_points(&self, ring: &Ring) -> Vec { let mut ret = vec![]; + ret.push([0u8; 32].into()); for entry in ring.ring.iter() { ret.push(entry.location); } diff --git a/src/table/table_sync.rs b/src/table/table_sync.rs index 5fa6793b..c38b6bd5 100644 --- a/src/table/table_sync.rs +++ b/src/table/table_sync.rs @@ -18,10 +18,14 @@ use garage_util::error::Error; use crate::*; const MAX_DEPTH: usize = 16; -const SCAN_INTERVAL: Duration = Duration::from_secs(3600); -const CHECKSUM_CACHE_TIMEOUT: Duration = Duration::from_secs(1800); const TABLE_SYNC_RPC_TIMEOUT: Duration = Duration::from_secs(30); +// Scan & sync every 12 hours +const SCAN_INTERVAL: Duration = Duration::from_secs(12 * 60 * 60); + +// Expire cache after 30 minutes +const CHECKSUM_CACHE_TIMEOUT: Duration = Duration::from_secs(30 * 60); + pub struct TableSyncer { table: Arc>, todo: Mutex, @@ -797,6 +801,10 @@ impl SyncTodo { for i in 0..split_points.len() - 1 { let begin = split_points[i]; let end = split_points[i + 1]; + if begin == end { + continue; + } + let nodes = table.replication.replication_nodes(&begin, &ring); let retain = nodes.contains(&my_id); -- cgit v1.2.3 From 8d63738cb062e816fc01c6aa2b32936ad31ff65b Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 11 Mar 2021 13:47:21 +0100 Subject: Checkpoint: add merkle tree in data table --- .drone.yml | 4 +- src/garage/admin_rpc.rs | 3 +- src/garage/server.rs | 2 +- src/model/block.rs | 14 +- src/model/garage.rs | 19 +-- src/rpc/membership.rs | 6 +- src/table/lib.rs | 1 + src/table/merkle.rs | 352 ++++++++++++++++++++++++++++++++++++++++++++++++ src/table/table.rs | 91 +++++++++---- src/table/table_sync.rs | 28 ++-- src/util/background.rs | 16 +-- 11 files changed, 452 insertions(+), 84 deletions(-) create mode 100644 src/table/merkle.rs diff --git a/.drone.yml b/.drone.yml index 15582cc3..76379e40 100644 --- a/.drone.yml +++ b/.drone.yml @@ -12,7 +12,7 @@ steps: image: alpine/git commands: - mkdir -p cargo - - git clone https://git.deuxfleurs.fr/Deuxfleurs/garage.git + - git clone $DRONE_GIT_HTTP_URL - cd garage - git checkout $DRONE_COMMIT @@ -34,8 +34,8 @@ steps: - 'garage/target' - 'cargo/registry/index' - 'cargo/registry/cache' - - 'cargo/git/db' - 'cargo/bin' + - 'cargo/git/db' path_style: true endpoint: https://garage.deuxfleurs.fr diff --git a/src/garage/admin_rpc.rs b/src/garage/admin_rpc.rs index e1981e3a..b4a65cad 100644 --- a/src/garage/admin_rpc.rs +++ b/src/garage/admin_rpc.rs @@ -350,8 +350,7 @@ impl AdminRpcHandler { .background .spawn_worker("Repair worker".into(), move |must_exit| async move { repair.repair_worker(opt, must_exit).await - }) - .await; + }); Ok(AdminRPC::Ok(format!( "Repair launched on {:?}", self.garage.system.id diff --git a/src/garage/server.rs b/src/garage/server.rs index a0ab17c4..8dddd7bb 100644 --- a/src/garage/server.rs +++ b/src/garage/server.rs @@ -49,7 +49,7 @@ pub async fn run_server(config_file: PathBuf) -> Result<(), Error> { let (send_cancel, watch_cancel) = watch::channel(false); let background = BackgroundRunner::new(16, watch_cancel.clone()); - let garage = Garage::new(config, db, background.clone(), &mut rpc_server).await; + let garage = Garage::new(config, db, background.clone(), &mut rpc_server); info!("Crate admin RPC handler..."); AdminRpcHandler::new(garage.clone()).register_handler(&mut rpc_server); diff --git a/src/model/block.rs b/src/model/block.rs index 57f4c077..ba5544a3 100644 --- a/src/model/block.rs +++ b/src/model/block.rs @@ -127,18 +127,16 @@ impl BlockManager { } } - pub async fn spawn_background_worker(self: Arc) { + pub fn spawn_background_worker(self: Arc) { // Launch 2 simultaneous workers for background resync loop preprocessing - for i in 0..2usize { + for i in 0..2u64 { let bm2 = self.clone(); let background = self.system.background.clone(); tokio::spawn(async move { - tokio::time::delay_for(Duration::from_secs(10)).await; - background - .spawn_worker(format!("block resync worker {}", i), move |must_exit| { - bm2.resync_loop(must_exit) - }) - .await; + tokio::time::delay_for(Duration::from_secs(10 * (i + 1))).await; + background.spawn_worker(format!("block resync worker {}", i), move |must_exit| { + bm2.resync_loop(must_exit) + }); }); } } diff --git a/src/model/garage.rs b/src/model/garage.rs index d109fdaa..aac79a85 100644 --- a/src/model/garage.rs +++ b/src/model/garage.rs @@ -35,7 +35,7 @@ pub struct Garage { } impl Garage { - pub async fn new( + pub fn new( config: Config, db: sled::Db, background: Arc, @@ -86,8 +86,7 @@ impl Garage { &db, "block_ref".to_string(), rpc_server, - ) - .await; + ); info!("Initialize version_table..."); let version_table = Table::new( @@ -100,8 +99,7 @@ impl Garage { &db, "version".to_string(), rpc_server, - ) - .await; + ); info!("Initialize object_table..."); let object_table = Table::new( @@ -114,8 +112,7 @@ impl Garage { &db, "object".to_string(), rpc_server, - ) - .await; + ); info!("Initialize bucket_table..."); let bucket_table = Table::new( @@ -125,8 +122,7 @@ impl Garage { &db, "bucket".to_string(), rpc_server, - ) - .await; + ); info!("Initialize key_table_table..."); let key_table = Table::new( @@ -136,8 +132,7 @@ impl Garage { &db, "key".to_string(), rpc_server, - ) - .await; + ); info!("Initialize Garage..."); let garage = Arc::new(Self { @@ -155,7 +150,7 @@ impl Garage { info!("Start block manager background thread..."); garage.block_manager.garage.swap(Some(garage.clone())); - garage.block_manager.clone().spawn_background_worker().await; + garage.block_manager.clone().spawn_background_worker(); garage } diff --git a/src/rpc/membership.rs b/src/rpc/membership.rs index 44d7122a..e1dc297e 100644 --- a/src/rpc/membership.rs +++ b/src/rpc/membership.rs @@ -319,8 +319,7 @@ impl System { .background .spawn_worker(format!("ping loop"), |stop_signal| { self2.ping_loop(stop_signal).map(Ok) - }) - .await; + }); if let (Some(consul_host), Some(consul_service_name)) = (consul_host, consul_service_name) { let self2 = self.clone(); @@ -330,8 +329,7 @@ impl System { self2 .consul_loop(stop_signal, consul_host, consul_service_name) .map(Ok) - }) - .await; + }); } } diff --git a/src/table/lib.rs b/src/table/lib.rs index 704f8f1e..62fd30c5 100644 --- a/src/table/lib.rs +++ b/src/table/lib.rs @@ -7,6 +7,7 @@ pub mod crdt; pub mod schema; pub mod util; +pub mod merkle; pub mod table; pub mod table_fullcopy; pub mod table_sharded; diff --git a/src/table/merkle.rs b/src/table/merkle.rs new file mode 100644 index 00000000..50cb90d5 --- /dev/null +++ b/src/table/merkle.rs @@ -0,0 +1,352 @@ +use std::convert::TryInto; +use std::sync::Arc; +use std::time::Duration; + +use futures::select; +use futures_util::future::*; +use log::{info, warn}; +use serde::{Deserialize, Serialize}; +use sled::transaction::{ + ConflictableTransactionError, ConflictableTransactionResult, TransactionalTree, +}; +use tokio::sync::{watch, Notify}; + +use garage_util::background::BackgroundRunner; +use garage_util::data::*; +use garage_util::error::Error; + +// This modules partitions the data in 2**16 partitions, based on the top +// 16 bits (two bytes) of item's partition keys' hashes. +// It builds one Merkle tree for each of these 2**16 partitions. + +pub(crate) struct MerkleUpdater { + table_name: String, + background: Arc, + + // Content of the todo tree: items where + // - key = the key of an item in the main table, ie hash(partition_key)+sort_key + // - value = the hash of the full serialized item, if present, + // or an empty vec if item is absent (deleted) + pub(crate) todo: sled::Tree, + pub(crate) todo_notify: Notify, + + // Content of the merkle tree: items where + // - key = .bytes() for MerkleNodeKey + // - value = serialization of a MerkleNode, assumed to be MerkleNode::empty if not found + pub(crate) merkle_tree: sled::Tree, + empty_node_hash: Hash, +} + +#[derive(Clone)] +pub struct MerkleNodeKey { + // partition: first 16 bits (two bytes) of the partition_key's hash + pub partition: [u8; 2], + + // prefix: a prefix for the hash of full keys, i.e. hash(hash(partition_key)+sort_key) + pub prefix: Vec, +} + +#[derive(PartialEq, Eq, Debug, Serialize, Deserialize)] +pub enum MerkleNode { + // The empty Merkle node + Empty, + + // An intermediate Merkle tree node for a prefix + // Contains the hashes of the 256 possible next prefixes + Intermediate(Vec<(u8, Hash)>), + + // A final node for an item + // Contains the full key of the item and the hash of the value + Leaf(Vec, Hash), +} + +impl MerkleUpdater { + pub(crate) fn new( + table_name: String, + background: Arc, + todo: sled::Tree, + merkle_tree: sled::Tree, + ) -> Arc { + let empty_node_hash = blake2sum(&rmp_to_vec_all_named(&MerkleNode::Empty).unwrap()[..]); + + Arc::new(Self { + table_name, + background, + todo, + todo_notify: Notify::new(), + merkle_tree, + empty_node_hash, + }) + } + + pub(crate) fn launch(self: &Arc) { + let self2 = self.clone(); + self.background.spawn_worker( + format!("Merkle tree updater for {}", self.table_name), + |must_exit: watch::Receiver| self2.updater_loop(must_exit), + ); + } + + async fn updater_loop( + self: Arc, + mut must_exit: watch::Receiver, + ) -> Result<(), Error> { + while !*must_exit.borrow() { + if let Some(x) = self.todo.iter().next() { + match x { + Ok((key, valhash)) => { + if let Err(e) = self.update_item(&key[..], &valhash[..]) { + warn!("Error while updating Merkle tree item: {}", e); + } + } + Err(e) => { + warn!("Error while iterating on Merkle todo tree: {}", e); + tokio::time::delay_for(Duration::from_secs(10)).await; + } + } + } else { + select! { + _ = self.todo_notify.notified().fuse() => (), + _ = must_exit.recv().fuse() => (), + } + } + } + Ok(()) + } + + fn update_item(&self, k: &[u8], vhash_by: &[u8]) -> Result<(), Error> { + let khash = blake2sum(k); + + let new_vhash = if vhash_by.len() == 0 { + None + } else { + let vhash_by: [u8; 32] = vhash_by + .try_into() + .map_err(|_| Error::Message(format!("Invalid value in Merkle todo table")))?; + Some(Hash::from(vhash_by)) + }; + + let key = MerkleNodeKey { + partition: k[0..2].try_into().unwrap(), + prefix: vec![], + }; + self.merkle_tree + .transaction(|tx| self.update_item_rec(tx, k, khash, &key, new_vhash))?; + + let deleted = self + .todo + .compare_and_swap::<_, _, Vec>(k, Some(vhash_by), None)? + .is_ok(); + + if !deleted { + info!( + "Item not deleted from Merkle todo because it changed: {:?}", + k + ); + } + Ok(()) + } + + fn update_item_rec( + &self, + tx: &TransactionalTree, + k: &[u8], + khash: Hash, + key: &MerkleNodeKey, + new_vhash: Option, + ) -> ConflictableTransactionResult, Error> { + let i = key.prefix.len(); + let mutate = match self.read_node_txn(tx, &key)? { + MerkleNode::Empty => { + if let Some(vhv) = new_vhash { + Some(MerkleNode::Leaf(k.to_vec(), vhv)) + } else { + None + } + } + MerkleNode::Intermediate(mut children) => { + let key2 = key.next_key(khash); + if let Some(subhash) = self.update_item_rec(tx, k, khash, &key2, new_vhash)? { + if subhash == self.empty_node_hash { + intermediate_rm_child(&mut children, key2.prefix[i]); + } else { + intermediate_set_child(&mut children, key2.prefix[i], subhash); + } + if children.len() == 0 { + // should not happen + warn!("Replacing intermediate node with empty node, should not happen."); + Some(MerkleNode::Empty) + } else if children.len() == 1 { + // move node down to this level + let key_sub = key.add_byte(children[0].0); + let subnode = self.read_node_txn(tx, &key_sub)?; + tx.remove(key_sub.encode())?; + Some(subnode) + } else { + Some(MerkleNode::Intermediate(children)) + } + } else { + None + } + } + MerkleNode::Leaf(exlf_key, exlf_hash) => { + if exlf_key == k { + match new_vhash { + Some(vhv) if vhv == exlf_hash => None, + Some(vhv) => Some(MerkleNode::Leaf(k.to_vec(), vhv)), + None => Some(MerkleNode::Empty), + } + } else { + if let Some(vhv) = new_vhash { + // Create two sub-nodes and replace by intermediary node + let (pos1, h1) = { + let key2 = key.next_key(blake2sum(&exlf_key[..])); + let subhash = + self.put_node_txn(tx, &key2, &MerkleNode::Leaf(exlf_key, exlf_hash))?; + (key2.prefix[i], subhash) + }; + let (pos2, h2) = { + let key2 = key.next_key(khash); + let subhash = + self.put_node_txn(tx, &key2, &MerkleNode::Leaf(k.to_vec(), vhv))?; + (key2.prefix[i], subhash) + }; + let mut int = vec![]; + intermediate_set_child(&mut int, pos1, h1); + intermediate_set_child(&mut int, pos2, h2); + Some(MerkleNode::Intermediate(int)) + } else { + None + } + } + } + }; + + if let Some(new_node) = mutate { + let hash = self.put_node_txn(tx, &key, &new_node)?; + Ok(Some(hash)) + } else { + Ok(None) + } + } + + // Merkle tree node manipulation + + fn read_node_txn( + &self, + tx: &TransactionalTree, + k: &MerkleNodeKey, + ) -> ConflictableTransactionResult { + let ent = tx.get(k.encode())?; + match ent { + None => Ok(MerkleNode::Empty), + Some(v) => Ok(rmp_serde::decode::from_read_ref::<_, MerkleNode>(&v[..]) + .map_err(|e| ConflictableTransactionError::Abort(e.into()))?), + } + } + + fn put_node_txn( + &self, + tx: &TransactionalTree, + k: &MerkleNodeKey, + v: &MerkleNode, + ) -> ConflictableTransactionResult { + if *v == MerkleNode::Empty { + tx.remove(k.encode())?; + Ok(self.empty_node_hash) + } else { + let vby = rmp_to_vec_all_named(v) + .map_err(|e| ConflictableTransactionError::Abort(e.into()))?; + let rethash = blake2sum(&vby[..]); + tx.insert(k.encode(), vby)?; + Ok(rethash) + } + } + + pub(crate) fn read_node( + &self, + k: &MerkleNodeKey, + ) -> Result { + let ent = self.merkle_tree.get(k.encode())?; + match ent { + None => Ok(MerkleNode::Empty), + Some(v) => Ok(rmp_serde::decode::from_read_ref::<_, MerkleNode>(&v[..])?) + } + } +} + +impl MerkleNodeKey { + fn encode(&self) -> Vec { + let mut ret = Vec::with_capacity(2 + self.prefix.len()); + ret.extend(&self.partition[..]); + ret.extend(&self.prefix[..]); + ret + } + + pub fn next_key(&self, h: Hash) -> Self { + assert!(&h.as_slice()[0..self.prefix.len()] == &self.prefix[..]); + let mut s2 = self.clone(); + s2.prefix.push(h.as_slice()[self.prefix.len()]); + s2 + } + + pub fn add_byte(&self, b: u8) -> Self { + let mut s2 = self.clone(); + s2.prefix.push(b); + s2 + } +} + +fn intermediate_set_child(ch: &mut Vec<(u8, Hash)>, pos: u8, v: Hash) { + for i in 0..ch.len() { + if ch[i].0 == pos { + ch[i].1 = v; + return; + } else if ch[i].0 > pos { + ch.insert(i, (pos, v)); + return; + } + } + ch.insert(ch.len(), (pos, v)); +} + +fn intermediate_rm_child(ch: &mut Vec<(u8, Hash)>, pos: u8) { + for i in 0..ch.len() { + if ch[i].0 == pos { + ch.remove(i); + return; + } + } +} + +#[test] +fn test_intermediate_aux() { + let mut v = vec![]; + + intermediate_set_child(&mut v, 12u8, [12u8; 32].into()); + assert!(v == vec![(12u8, [12u8; 32].into())]); + + intermediate_set_child(&mut v, 42u8, [42u8; 32].into()); + assert!(v == vec![(12u8, [12u8; 32].into()), (42u8, [42u8; 32].into())]); + + intermediate_set_child(&mut v, 4u8, [4u8; 32].into()); + assert!(v == vec![(4u8, [4u8; 32].into()), (12u8, [12u8; 32].into()), (42u8, [42u8; 32].into())]); + + intermediate_set_child(&mut v, 12u8, [8u8; 32].into()); + assert!(v == vec![(4u8, [4u8; 32].into()), (12u8, [8u8; 32].into()), (42u8, [42u8; 32].into())]); + + intermediate_set_child(&mut v, 6u8, [6u8; 32].into()); + assert!(v == vec![(4u8, [4u8; 32].into()), (6u8, [6u8; 32].into()), (12u8, [8u8; 32].into()), (42u8, [42u8; 32].into())]); + + intermediate_rm_child(&mut v, 42u8); + assert!(v == vec![(4u8, [4u8; 32].into()), (6u8, [6u8; 32].into()), (12u8, [8u8; 32].into())]); + + intermediate_rm_child(&mut v, 11u8); + assert!(v == vec![(4u8, [4u8; 32].into()), (6u8, [6u8; 32].into()), (12u8, [8u8; 32].into())]); + + intermediate_rm_child(&mut v, 6u8); + assert!(v == vec![(4u8, [4u8; 32].into()), (12u8, [8u8; 32].into())]); + + intermediate_set_child(&mut v, 6u8, [7u8; 32].into()); + assert!(v == vec![(4u8, [4u8; 32].into()), (6u8, [7u8; 32].into()), (12u8, [8u8; 32].into())]); +} diff --git a/src/table/table.rs b/src/table/table.rs index 366ce925..0e75754c 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -8,6 +8,7 @@ use arc_swap::ArcSwapOption; use futures::stream::*; use serde::{Deserialize, Serialize}; use serde_bytes::ByteBuf; +use sled::Transactional; use garage_util::data::*; use garage_util::error::Error; @@ -18,6 +19,7 @@ use garage_rpc::rpc_client::*; use garage_rpc::rpc_server::*; use crate::crdt::CRDT; +use crate::merkle::*; use crate::schema::*; use crate::table_sync::*; @@ -33,6 +35,7 @@ pub struct Table { pub system: Arc, pub store: sled::Tree, pub syncer: ArcSwapOption>, + merkle_updater: Arc, } #[derive(Serialize, Deserialize)] @@ -77,7 +80,7 @@ where { // =============== PUBLIC INTERFACE FUNCTIONS (new, insert, get, etc) =============== - pub async fn new( + pub fn new( instance: F, replication: R, system: Arc, @@ -85,11 +88,27 @@ where name: String, rpc_server: &mut RpcServer, ) -> Arc { - let store = db.open_tree(&name).expect("Unable to open DB tree"); + let store = db + .open_tree(&format!("{}:table", name)) + .expect("Unable to open DB tree"); + + let merkle_todo_store = db + .open_tree(&format!("{}:merkle_todo", name)) + .expect("Unable to open DB Merkle TODO tree"); + let merkle_tree_store = db + .open_tree(&format!("{}:merkle_tree", name)) + .expect("Unable to open DB Merkle tree tree"); let rpc_path = format!("table_{}", name); let rpc_client = system.rpc_client::>(&rpc_path); + let merkle_updater = MerkleUpdater::new( + name.clone(), + system.background.clone(), + merkle_todo_store, + merkle_tree_store, + ); + let table = Arc::new(Self { instance, replication, @@ -98,12 +117,15 @@ where system, store, syncer: ArcSwapOption::from(None), + merkle_updater, }); table.clone().register_handler(rpc_server, rpc_path); - let syncer = TableSyncer::launch(table.clone()).await; + let syncer = TableSyncer::launch(table.clone()); table.syncer.swap(Some(syncer)); + table.merkle_updater.launch(); + table } @@ -322,7 +344,7 @@ where Ok(TableRPC::Update(values)) } TableRPC::Update(pairs) => { - self.handle_update(pairs).await?; + self.handle_update(pairs)?; Ok(TableRPC::Ok) } TableRPC::SyncRPC(rpc) => { @@ -380,53 +402,64 @@ where Ok(ret) } - pub async fn handle_update(self: &Arc, entries: &[Arc]) -> Result<(), Error> { - let syncer = self.syncer.load_full().unwrap(); + // ========== CODE THAT ACTUALLY MODIFIES THE TREE ================ + pub fn handle_update(self: &Arc, entries: &[Arc]) -> Result<(), Error> { for update_bytes in entries.iter() { - let update = self.decode_entry(update_bytes.as_slice())?; - - let tree_key = self.tree_key(update.partition_key(), update.sort_key()); - - let (old_entry, new_entry) = self.store.transaction(|db| { - let (old_entry, new_entry) = match db.get(&tree_key)? { - Some(prev_bytes) => { - let old_entry = self - .decode_entry(&prev_bytes) - .map_err(sled::transaction::ConflictableTransactionError::Abort)?; - let mut new_entry = old_entry.clone(); - new_entry.merge(&update); - (Some(old_entry), new_entry) - } - None => (None, update.clone()), - }; + self.update_entry(update_bytes.as_slice())?; + } + Ok(()) + } + pub(crate) fn update_entry(self: &Arc, update_bytes: &[u8]) -> Result<(), Error> { + let update = self.decode_entry(update_bytes)?; + let tree_key = self.tree_key(update.partition_key(), update.sort_key()); + + let changed = (&self.store, &self.merkle_updater.todo).transaction(|(db, mkl_todo)| { + let (old_entry, new_entry) = match db.get(&tree_key)? { + Some(prev_bytes) => { + let old_entry = self + .decode_entry(&prev_bytes) + .map_err(sled::transaction::ConflictableTransactionError::Abort)?; + let mut new_entry = old_entry.clone(); + new_entry.merge(&update); + (Some(old_entry), new_entry) + } + None => (None, update.clone()), + }; + + if Some(&new_entry) != old_entry.as_ref() { let new_bytes = rmp_to_vec_all_named(&new_entry) .map_err(Error::RMPEncode) .map_err(sled::transaction::ConflictableTransactionError::Abort)?; + mkl_todo.insert(tree_key.clone(), blake2sum(&new_bytes[..]).to_vec())?; db.insert(tree_key.clone(), new_bytes)?; - Ok((old_entry, new_entry)) - })?; - - if old_entry.as_ref() != Some(&new_entry) { - self.instance.updated(old_entry, Some(new_entry)); - syncer.invalidate(&tree_key[..]); + Ok(Some((old_entry, new_entry))) + } else { + Ok(None) } + })?; + + if let Some((old_entry, new_entry)) = changed { + self.instance.updated(old_entry, Some(new_entry)); + self.syncer.load_full().unwrap().invalidate(&tree_key[..]); } Ok(()) } pub(crate) fn delete_if_equal(self: &Arc, k: &[u8], v: &[u8]) -> Result { - let removed = self.store.transaction(|txn| { + let removed = (&self.store, &self.merkle_updater.todo).transaction(|(txn, mkl_todo)| { if let Some(cur_v) = txn.get(k)? { if cur_v == v { txn.remove(k)?; + mkl_todo.insert(k, vec![])?; return Ok(true); } } Ok(false) })?; + if removed { let old_entry = self.decode_entry(v)?; self.instance.updated(Some(old_entry), None); diff --git a/src/table/table_sync.rs b/src/table/table_sync.rs index c38b6bd5..51f8cd6f 100644 --- a/src/table/table_sync.rs +++ b/src/table/table_sync.rs @@ -106,7 +106,7 @@ where F: TableSchema + 'static, R: TableReplication + 'static, { - pub(crate) async fn launch(table: Arc>) -> Arc { + pub(crate) fn launch(table: Arc>) -> Arc { let todo = SyncTodo { todo: Vec::new() }; let syncer = Arc::new(TableSyncer { table: table.clone(), @@ -119,24 +119,16 @@ where let (busy_tx, busy_rx) = mpsc::unbounded_channel(); let s1 = syncer.clone(); - table - .system - .background - .spawn_worker( - format!("table sync watcher for {}", table.name), - move |must_exit: watch::Receiver| s1.watcher_task(must_exit, busy_rx), - ) - .await; + table.system.background.spawn_worker( + format!("table sync watcher for {}", table.name), + move |must_exit: watch::Receiver| s1.watcher_task(must_exit, busy_rx), + ); let s2 = syncer.clone(); - table - .system - .background - .spawn_worker( - format!("table syncer for {}", table.name), - move |must_exit: watch::Receiver| s2.syncer_task(must_exit, busy_tx), - ) - .await; + table.system.background.spawn_worker( + format!("table syncer for {}", table.name), + move |must_exit: watch::Receiver| s2.syncer_task(must_exit, busy_tx), + ); let s3 = syncer.clone(); tokio::spawn(async move { @@ -630,7 +622,7 @@ where } } if diff_items.len() > 0 { - self.table.handle_update(&diff_items[..]).await?; + self.table.handle_update(&diff_items[..])?; } if items_to_send.len() > 0 { self.send_items(who, items_to_send).await?; diff --git a/src/util/background.rs b/src/util/background.rs index 937062dd..8081f157 100644 --- a/src/util/background.rs +++ b/src/util/background.rs @@ -1,11 +1,11 @@ use core::future::Future; use std::pin::Pin; +use std::sync::Mutex; use futures::future::join_all; use futures::select; use futures_util::future::*; use std::sync::Arc; -use tokio::sync::Mutex; use tokio::sync::{mpsc, watch, Notify}; use crate::error::Error; @@ -38,7 +38,7 @@ impl BackgroundRunner { } pub async fn run(self: Arc) { - let mut workers = self.workers.lock().await; + let mut workers = self.workers.lock().unwrap(); for i in 0..self.n_runners { workers.push(tokio::spawn(self.clone().runner(i))); } @@ -47,7 +47,7 @@ impl BackgroundRunner { let mut stop_signal = self.stop_signal.clone(); while let Some(exit_now) = stop_signal.recv().await { if exit_now { - let mut workers = self.workers.lock().await; + let mut workers = self.workers.lock().unwrap(); let workers_vec = workers.drain(..).collect::>(); join_all(workers_vec).await; return; @@ -73,12 +73,12 @@ impl BackgroundRunner { self.job_notify.notify(); } - pub async fn spawn_worker(&self, name: String, worker: F) + pub fn spawn_worker(&self, name: String, worker: F) where F: FnOnce(watch::Receiver) -> T + Send + 'static, T: Future + Send + 'static, { - let mut workers = self.workers.lock().await; + let mut workers = self.workers.lock().unwrap(); let stop_signal = self.stop_signal.clone(); workers.push(tokio::spawn(async move { if let Err(e) = worker(stop_signal).await { @@ -93,7 +93,7 @@ impl BackgroundRunner { let mut stop_signal = self.stop_signal.clone(); loop { let must_exit: bool = *stop_signal.borrow(); - if let Some(job) = self.dequeue_job(must_exit).await { + if let Some(job) = self.dequeue_job(must_exit) { if let Err(e) = job.await { error!("Job failed: {}", e) } @@ -110,8 +110,8 @@ impl BackgroundRunner { } } - async fn dequeue_job(&self, must_exit: bool) -> Option { - let mut queue = self.queue_out.lock().await; + fn dequeue_job(&self, must_exit: bool) -> Option { + let mut queue = self.queue_out.lock().unwrap(); while let Ok((job, cancellable)) = queue.try_recv() { if cancellable && must_exit { continue; -- cgit v1.2.3 From 94f3d287742ff90f179f528421c690b00b71a912 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 11 Mar 2021 16:54:15 +0100 Subject: WIP big refactoring --- Cargo.lock | 3 +- src/garage/repair.rs | 29 +--- src/model/block.rs | 5 +- src/model/garage.rs | 4 +- src/model/object_table.rs | 2 +- src/model/version_table.rs | 2 +- src/table/Cargo.toml | 1 - src/table/data.rs | 189 +++++++++++++++++++++++++ src/table/lib.rs | 4 +- src/table/merkle.rs | 39 ++++-- src/table/replication/fullcopy.rs | 59 ++++++++ src/table/replication/mod.rs | 6 + src/table/replication/parameters.rs | 22 +++ src/table/replication/sharded.rs | 54 ++++++++ src/table/table.rs | 265 +++++++----------------------------- src/table/table_fullcopy.rs | 59 -------- src/table/table_sharded.rs | 54 -------- src/table/table_sync.rs | 129 +++++++++--------- 18 files changed, 494 insertions(+), 432 deletions(-) create mode 100644 src/table/data.rs create mode 100644 src/table/replication/fullcopy.rs create mode 100644 src/table/replication/mod.rs create mode 100644 src/table/replication/parameters.rs create mode 100644 src/table/replication/sharded.rs delete mode 100644 src/table/table_fullcopy.rs delete mode 100644 src/table/table_sharded.rs diff --git a/Cargo.lock b/Cargo.lock index 6d4cc6a3..45244b8b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1,5 +1,7 @@ # This file is automatically @generated by Cargo. # It is not intended for manual editing. +version = 3 + [[package]] name = "aho-corasick" version = "0.7.15" @@ -583,7 +585,6 @@ dependencies = [ name = "garage_table" version = "0.1.1" dependencies = [ - "arc-swap", "async-trait", "bytes 0.4.12", "futures", diff --git a/src/garage/repair.rs b/src/garage/repair.rs index e330f7bb..f9cd5884 100644 --- a/src/garage/repair.rs +++ b/src/garage/repair.rs @@ -28,38 +28,23 @@ impl Repair { self.garage .bucket_table .syncer - .load_full() - .unwrap() - .add_full_scan() - .await; + .add_full_scan(); self.garage .object_table .syncer - .load_full() - .unwrap() - .add_full_scan() - .await; + .add_full_scan(); self.garage .version_table .syncer - .load_full() - .unwrap() - .add_full_scan() - .await; + .add_full_scan(); self.garage .block_ref_table .syncer - .load_full() - .unwrap() - .add_full_scan() - .await; + .add_full_scan(); self.garage .key_table .syncer - .load_full() - .unwrap() - .add_full_scan() - .await; + .add_full_scan(); } // TODO: wait for full sync to finish before proceeding to the rest? @@ -93,7 +78,7 @@ impl Repair { async fn repair_versions(&self, must_exit: &watch::Receiver) -> Result<(), Error> { let mut pos = vec![]; - while let Some((item_key, item_bytes)) = self.garage.version_table.store.get_gt(&pos)? { + while let Some((item_key, item_bytes)) = self.garage.version_table.data.store.get_gt(&pos)? { pos = item_key.to_vec(); let version = rmp_serde::decode::from_read_ref::<_, Version>(item_bytes.as_ref())?; @@ -141,7 +126,7 @@ impl Repair { async fn repair_block_ref(&self, must_exit: &watch::Receiver) -> Result<(), Error> { let mut pos = vec![]; - while let Some((item_key, item_bytes)) = self.garage.block_ref_table.store.get_gt(&pos)? { + while let Some((item_key, item_bytes)) = self.garage.block_ref_table.data.store.get_gt(&pos)? { pos = item_key.to_vec(); let block_ref = rmp_serde::decode::from_read_ref::<_, BlockRef>(item_bytes.as_ref())?; diff --git a/src/model/block.rs b/src/model/block.rs index ba5544a3..987ec9e4 100644 --- a/src/model/block.rs +++ b/src/model/block.rs @@ -19,8 +19,7 @@ use garage_rpc::membership::System; use garage_rpc::rpc_client::*; use garage_rpc::rpc_server::*; -use garage_table::table_sharded::TableShardedReplication; -use garage_table::TableReplication; +use garage_table::replication::{sharded::TableShardedReplication, TableReplication}; use crate::block_ref_table::*; @@ -412,7 +411,7 @@ impl BlockManager { let garage = self.garage.load_full().unwrap(); let mut last_hash = None; let mut i = 0usize; - for entry in garage.block_ref_table.store.iter() { + for entry in garage.block_ref_table.data.store.iter() { let (_k, v_bytes) = entry?; let block_ref = rmp_serde::decode::from_read_ref::<_, BlockRef>(v_bytes.as_ref())?; if Some(&block_ref.block) == last_hash.as_ref() { diff --git a/src/model/garage.rs b/src/model/garage.rs index aac79a85..193c71d2 100644 --- a/src/model/garage.rs +++ b/src/model/garage.rs @@ -7,8 +7,8 @@ use garage_rpc::membership::System; use garage_rpc::rpc_client::RpcHttpClient; use garage_rpc::rpc_server::RpcServer; -use garage_table::table_fullcopy::*; -use garage_table::table_sharded::*; +use garage_table::replication::sharded::*; +use garage_table::replication::fullcopy::*; use garage_table::*; use crate::block::*; diff --git a/src/model/object_table.rs b/src/model/object_table.rs index cd09f678..99fad3ce 100644 --- a/src/model/object_table.rs +++ b/src/model/object_table.rs @@ -6,7 +6,7 @@ use garage_util::background::BackgroundRunner; use garage_util::data::*; use garage_table::crdt::*; -use garage_table::table_sharded::*; +use garage_table::replication::sharded::*; use garage_table::*; use crate::version_table::*; diff --git a/src/model/version_table.rs b/src/model/version_table.rs index 7ccc6a33..cdc73a85 100644 --- a/src/model/version_table.rs +++ b/src/model/version_table.rs @@ -5,7 +5,7 @@ use garage_util::background::BackgroundRunner; use garage_util::data::*; use garage_table::crdt::*; -use garage_table::table_sharded::*; +use garage_table::replication::sharded::*; use garage_table::*; use crate::block_ref_table::*; diff --git a/src/table/Cargo.toml b/src/table/Cargo.toml index 6485a542..6b3aaceb 100644 --- a/src/table/Cargo.toml +++ b/src/table/Cargo.toml @@ -19,7 +19,6 @@ garage_rpc = { version = "0.1.1", path = "../rpc" } bytes = "0.4" rand = "0.7" hex = "0.3" -arc-swap = "0.4" log = "0.4" hexdump = "0.1" diff --git a/src/table/data.rs b/src/table/data.rs new file mode 100644 index 00000000..fa89fc27 --- /dev/null +++ b/src/table/data.rs @@ -0,0 +1,189 @@ +use std::sync::Arc; + +use log::warn; +use sled::Transactional; +use serde_bytes::ByteBuf; + +use garage_util::data::*; +use garage_util::error::*; +use garage_util::background::BackgroundRunner; + +use crate::schema::*; +use crate::merkle::*; +use crate::crdt::CRDT; + +pub struct TableData { + pub name: String, + pub instance: F, + + pub store: sled::Tree, + pub(crate) merkle_updater: Arc, +} + +impl TableData where F: TableSchema { + pub fn new( + name: String, + instance: F, + db: &sled::Db, + background: Arc, + ) -> Arc { + let store = db + .open_tree(&format!("{}:table", name)) + .expect("Unable to open DB tree"); + + let merkle_todo_store = db + .open_tree(&format!("{}:merkle_todo", name)) + .expect("Unable to open DB Merkle TODO tree"); + let merkle_tree_store = db + .open_tree(&format!("{}:merkle_tree", name)) + .expect("Unable to open DB Merkle tree tree"); + + let merkle_updater = MerkleUpdater::launch( + name.clone(), + background, + merkle_todo_store, + merkle_tree_store, + ); + + Arc::new(Self{ + name, + instance, + store, + merkle_updater, + }) + } + + // Read functions + + pub fn read_entry(&self, p: &F::P, s: &F::S) -> Result, Error> { + let tree_key = self.tree_key(p, s); + if let Some(bytes) = self.store.get(&tree_key)? { + Ok(Some(ByteBuf::from(bytes.to_vec()))) + } else { + Ok(None) + } + } + + pub fn read_range( + &self, + p: &F::P, + s: &Option, + filter: &Option, + limit: usize, + ) -> Result>, Error> { + let partition_hash = p.hash(); + let first_key = match s { + None => partition_hash.to_vec(), + Some(sk) => self.tree_key(p, sk), + }; + let mut ret = vec![]; + for item in self.store.range(first_key..) { + let (key, value) = item?; + if &key[..32] != partition_hash.as_slice() { + break; + } + let keep = match filter { + None => true, + Some(f) => { + let entry = self.decode_entry(value.as_ref())?; + F::matches_filter(&entry, f) + } + }; + if keep { + ret.push(Arc::new(ByteBuf::from(value.as_ref()))); + } + if ret.len() >= limit { + break; + } + } + Ok(ret) + } + + // Mutation functions + + pub(crate) fn update_many(&self, entries: &[Arc]) -> Result<(), Error> { + for update_bytes in entries.iter() { + self.update_entry(update_bytes.as_slice())?; + } + Ok(()) + } + + pub(crate) fn update_entry(&self, update_bytes: &[u8]) -> Result<(), Error> { + let update = self.decode_entry(update_bytes)?; + let tree_key = self.tree_key(update.partition_key(), update.sort_key()); + + let changed = (&self.store, &self.merkle_updater.todo).transaction(|(db, mkl_todo)| { + let (old_entry, new_entry) = match db.get(&tree_key)? { + Some(prev_bytes) => { + let old_entry = self + .decode_entry(&prev_bytes) + .map_err(sled::transaction::ConflictableTransactionError::Abort)?; + let mut new_entry = old_entry.clone(); + new_entry.merge(&update); + (Some(old_entry), new_entry) + } + None => (None, update.clone()), + }; + + if Some(&new_entry) != old_entry.as_ref() { + let new_bytes = rmp_to_vec_all_named(&new_entry) + .map_err(Error::RMPEncode) + .map_err(sled::transaction::ConflictableTransactionError::Abort)?; + mkl_todo.insert(tree_key.clone(), blake2sum(&new_bytes[..]).to_vec())?; + db.insert(tree_key.clone(), new_bytes)?; + Ok(Some((old_entry, new_entry))) + } else { + Ok(None) + } + })?; + + if let Some((old_entry, new_entry)) = changed { + self.instance.updated(old_entry, Some(new_entry)); + //self.syncer.load_full().unwrap().invalidate(&tree_key[..]); + } + + Ok(()) + } + + pub(crate) fn delete_if_equal(self: &Arc, k: &[u8], v: &[u8]) -> Result { + let removed = (&self.store, &self.merkle_updater.todo).transaction(|(txn, mkl_todo)| { + if let Some(cur_v) = txn.get(k)? { + if cur_v == v { + txn.remove(k)?; + mkl_todo.insert(k, vec![])?; + return Ok(true); + } + } + Ok(false) + })?; + + if removed { + let old_entry = self.decode_entry(v)?; + self.instance.updated(Some(old_entry), None); + //self.syncer.load_full().unwrap().invalidate(k); + } + Ok(removed) + } + + pub(crate) fn tree_key(&self, p: &F::P, s: &F::S) -> Vec { + let mut ret = p.hash().to_vec(); + ret.extend(s.sort_key()); + ret + } + + pub(crate) fn decode_entry(&self, bytes: &[u8]) -> Result { + match rmp_serde::decode::from_read_ref::<_, F::E>(bytes) { + Ok(x) => Ok(x), + Err(e) => match F::try_migrate(bytes) { + Some(x) => Ok(x), + None => { + warn!("Unable to decode entry of {}: {}", self.name, e); + for line in hexdump::hexdump_iter(bytes) { + debug!("{}", line); + } + Err(e.into()) + } + }, + } + } +} diff --git a/src/table/lib.rs b/src/table/lib.rs index 62fd30c5..bb249a56 100644 --- a/src/table/lib.rs +++ b/src/table/lib.rs @@ -8,9 +8,9 @@ pub mod schema; pub mod util; pub mod merkle; +pub mod replication; +pub mod data; pub mod table; -pub mod table_fullcopy; -pub mod table_sharded; pub mod table_sync; pub use schema::*; diff --git a/src/table/merkle.rs b/src/table/merkle.rs index 50cb90d5..ef197dc8 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -61,7 +61,7 @@ pub enum MerkleNode { } impl MerkleUpdater { - pub(crate) fn new( + pub(crate) fn launch( table_name: String, background: Arc, todo: sled::Tree, @@ -69,22 +69,22 @@ impl MerkleUpdater { ) -> Arc { let empty_node_hash = blake2sum(&rmp_to_vec_all_named(&MerkleNode::Empty).unwrap()[..]); - Arc::new(Self { + let ret = Arc::new(Self { table_name, background, todo, todo_notify: Notify::new(), merkle_tree, empty_node_hash, - }) - } + }); - pub(crate) fn launch(self: &Arc) { - let self2 = self.clone(); - self.background.spawn_worker( - format!("Merkle tree updater for {}", self.table_name), - |must_exit: watch::Receiver| self2.updater_loop(must_exit), + let ret2 = ret.clone(); + ret.background.spawn_worker( + format!("Merkle tree updater for {}", ret.table_name), + |must_exit: watch::Receiver| ret2.updater_loop(must_exit), ); + + ret } async fn updater_loop( @@ -156,28 +156,37 @@ impl MerkleUpdater { new_vhash: Option, ) -> ConflictableTransactionResult, Error> { let i = key.prefix.len(); + + // Read node at current position (defined by the prefix stored in key) + // Calculate an update to apply to this node + // This update is an Option<_>, so that it is None if the update is a no-op + // and we can thus skip recalculating and re-storing everything let mutate = match self.read_node_txn(tx, &key)? { MerkleNode::Empty => { if let Some(vhv) = new_vhash { Some(MerkleNode::Leaf(k.to_vec(), vhv)) } else { + // Nothing to do, keep empty node None } } MerkleNode::Intermediate(mut children) => { let key2 = key.next_key(khash); if let Some(subhash) = self.update_item_rec(tx, k, khash, &key2, new_vhash)? { + // Subtree changed, update this node as well if subhash == self.empty_node_hash { intermediate_rm_child(&mut children, key2.prefix[i]); } else { intermediate_set_child(&mut children, key2.prefix[i], subhash); } + if children.len() == 0 { // should not happen warn!("Replacing intermediate node with empty node, should not happen."); Some(MerkleNode::Empty) } else if children.len() == 1 { - // move node down to this level + // We now have a single node (case when the update deleted one of only two + // children). Move that single child to this level of the tree. let key_sub = key.add_byte(children[0].0); let subnode = self.read_node_txn(tx, &key_sub)?; tx.remove(key_sub.encode())?; @@ -186,19 +195,23 @@ impl MerkleUpdater { Some(MerkleNode::Intermediate(children)) } } else { + // Subtree not changed, nothing to do None } } MerkleNode::Leaf(exlf_key, exlf_hash) => { if exlf_key == k { + // This leaf is for the same key that the one we are updating match new_vhash { Some(vhv) if vhv == exlf_hash => None, Some(vhv) => Some(MerkleNode::Leaf(k.to_vec(), vhv)), None => Some(MerkleNode::Empty), } } else { + // This is an only leaf for another key if let Some(vhv) = new_vhash { - // Create two sub-nodes and replace by intermediary node + // Move that other key to a subnode, create another subnode for our + // insertion and replace current node by an intermediary node let (pos1, h1) = { let key2 = key.next_key(blake2sum(&exlf_key[..])); let subhash = @@ -216,6 +229,9 @@ impl MerkleUpdater { intermediate_set_child(&mut int, pos2, h2); Some(MerkleNode::Intermediate(int)) } else { + // Nothing to do, we don't want to insert this value because it is None, + // and we don't want to change the other value because it's for something + // else None } } @@ -263,6 +279,7 @@ impl MerkleUpdater { } } + // Access a node in the Merkle tree, used by the sync protocol pub(crate) fn read_node( &self, k: &MerkleNodeKey, diff --git a/src/table/replication/fullcopy.rs b/src/table/replication/fullcopy.rs new file mode 100644 index 00000000..a62a6c3c --- /dev/null +++ b/src/table/replication/fullcopy.rs @@ -0,0 +1,59 @@ +use std::sync::Arc; + +use garage_rpc::membership::System; +use garage_rpc::ring::Ring; +use garage_util::data::*; + +use crate::replication::*; + +#[derive(Clone)] +pub struct TableFullReplication { + pub max_faults: usize, +} + +#[derive(Clone)] +struct Neighbors { + ring: Arc, + neighbors: Vec, +} + +impl TableFullReplication { + pub fn new(max_faults: usize) -> Self { + TableFullReplication { max_faults } + } +} + +impl TableReplication for TableFullReplication { + // Full replication schema: all nodes store everything + // Writes are disseminated in an epidemic manner in the network + + // Advantage: do all reads locally, extremely fast + // Inconvenient: only suitable to reasonably small tables + + fn read_nodes(&self, _hash: &Hash, system: &System) -> Vec { + vec![system.id] + } + fn read_quorum(&self) -> usize { + 1 + } + + fn write_nodes(&self, hash: &Hash, system: &System) -> Vec { + self.replication_nodes(hash, system.ring.borrow().as_ref()) + } + fn write_quorum(&self, system: &System) -> usize { + system.ring.borrow().config.members.len() - self.max_faults + } + fn max_write_errors(&self) -> usize { + self.max_faults + } + + fn replication_nodes(&self, _hash: &Hash, ring: &Ring) -> Vec { + ring.config.members.keys().cloned().collect::>() + } + fn split_points(&self, _ring: &Ring) -> Vec { + let mut ret = vec![]; + ret.push([0u8; 32].into()); + ret.push([0xFFu8; 32].into()); + ret + } +} diff --git a/src/table/replication/mod.rs b/src/table/replication/mod.rs new file mode 100644 index 00000000..d43d7f19 --- /dev/null +++ b/src/table/replication/mod.rs @@ -0,0 +1,6 @@ +mod parameters; + +pub mod fullcopy; +pub mod sharded; + +pub use parameters::*; diff --git a/src/table/replication/parameters.rs b/src/table/replication/parameters.rs new file mode 100644 index 00000000..4607b050 --- /dev/null +++ b/src/table/replication/parameters.rs @@ -0,0 +1,22 @@ +use garage_rpc::membership::System; +use garage_rpc::ring::Ring; + +use garage_util::data::*; + +pub trait TableReplication: Send + Sync { + // See examples in table_sharded.rs and table_fullcopy.rs + // To understand various replication methods + + // Which nodes to send reads from + fn read_nodes(&self, hash: &Hash, system: &System) -> Vec; + fn read_quorum(&self) -> usize; + + // Which nodes to send writes to + fn write_nodes(&self, hash: &Hash, system: &System) -> Vec; + fn write_quorum(&self, system: &System) -> usize; + fn max_write_errors(&self) -> usize; + + // Which are the nodes that do actually replicate the data + fn replication_nodes(&self, hash: &Hash, ring: &Ring) -> Vec; + fn split_points(&self, ring: &Ring) -> Vec; +} diff --git a/src/table/replication/sharded.rs b/src/table/replication/sharded.rs new file mode 100644 index 00000000..42a742cd --- /dev/null +++ b/src/table/replication/sharded.rs @@ -0,0 +1,54 @@ +use garage_rpc::membership::System; +use garage_rpc::ring::Ring; +use garage_util::data::*; + +use crate::replication::*; + +#[derive(Clone)] +pub struct TableShardedReplication { + pub replication_factor: usize, + pub read_quorum: usize, + pub write_quorum: usize, +} + +impl TableReplication for TableShardedReplication { + // Sharded replication schema: + // - based on the ring of nodes, a certain set of neighbors + // store entries, given as a function of the position of the + // entry's hash in the ring + // - reads are done on all of the nodes that replicate the data + // - writes as well + + fn read_nodes(&self, hash: &Hash, system: &System) -> Vec { + let ring = system.ring.borrow().clone(); + ring.walk_ring(&hash, self.replication_factor) + } + fn read_quorum(&self) -> usize { + self.read_quorum + } + + fn write_nodes(&self, hash: &Hash, system: &System) -> Vec { + let ring = system.ring.borrow().clone(); + ring.walk_ring(&hash, self.replication_factor) + } + fn write_quorum(&self, _system: &System) -> usize { + self.write_quorum + } + fn max_write_errors(&self) -> usize { + self.replication_factor - self.write_quorum + } + + fn replication_nodes(&self, hash: &Hash, ring: &Ring) -> Vec { + ring.walk_ring(&hash, self.replication_factor) + } + fn split_points(&self, ring: &Ring) -> Vec { + let mut ret = vec![]; + + ret.push([0u8; 32].into()); + for entry in ring.ring.iter() { + ret.push(entry.location); + } + ret.push([0xFFu8; 32].into()); + ret + } +} diff --git a/src/table/table.rs b/src/table/table.rs index 0e75754c..a4cb4b24 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -2,40 +2,35 @@ use std::collections::{BTreeMap, HashMap}; use std::sync::Arc; use std::time::Duration; -use log::warn; - -use arc_swap::ArcSwapOption; use futures::stream::*; use serde::{Deserialize, Serialize}; use serde_bytes::ByteBuf; -use sled::Transactional; use garage_util::data::*; use garage_util::error::Error; use garage_rpc::membership::System; -use garage_rpc::ring::Ring; use garage_rpc::rpc_client::*; use garage_rpc::rpc_server::*; use crate::crdt::CRDT; -use crate::merkle::*; +use crate::data::*; use crate::schema::*; use crate::table_sync::*; +use crate::replication::*; const TABLE_RPC_TIMEOUT: Duration = Duration::from_secs(10); -pub struct Table { - pub instance: F, +pub struct TableAux { + pub system: Arc, pub replication: R, - - pub name: String, pub(crate) rpc_client: Arc>>, +} - pub system: Arc, - pub store: sled::Tree, - pub syncer: ArcSwapOption>, - merkle_updater: Arc, +pub struct Table { + pub data: Arc>, + pub aux: Arc>, + pub syncer: Arc>, } #[derive(Serialize, Deserialize)] @@ -55,23 +50,6 @@ pub(crate) enum TableRPC { impl RpcMessage for TableRPC {} -pub trait TableReplication: Send + Sync { - // See examples in table_sharded.rs and table_fullcopy.rs - // To understand various replication methods - - // Which nodes to send reads from - fn read_nodes(&self, hash: &Hash, system: &System) -> Vec; - fn read_quorum(&self) -> usize; - - // Which nodes to send writes to - fn write_nodes(&self, hash: &Hash, system: &System) -> Vec; - fn write_quorum(&self, system: &System) -> usize; - fn max_write_errors(&self) -> usize; - - // Which are the nodes that do actually replicate the data - fn replication_nodes(&self, hash: &Hash, ring: &Ring) -> Vec; - fn split_points(&self, ring: &Ring) -> Vec; -} impl Table where @@ -88,60 +66,51 @@ where name: String, rpc_server: &mut RpcServer, ) -> Arc { - let store = db - .open_tree(&format!("{}:table", name)) - .expect("Unable to open DB tree"); - - let merkle_todo_store = db - .open_tree(&format!("{}:merkle_todo", name)) - .expect("Unable to open DB Merkle TODO tree"); - let merkle_tree_store = db - .open_tree(&format!("{}:merkle_tree", name)) - .expect("Unable to open DB Merkle tree tree"); - let rpc_path = format!("table_{}", name); let rpc_client = system.rpc_client::>(&rpc_path); - let merkle_updater = MerkleUpdater::new( - name.clone(), + let data = TableData::new( + name, + instance, + db, system.background.clone(), - merkle_todo_store, - merkle_tree_store, ); - let table = Arc::new(Self { - instance, + let aux = Arc::new(TableAux{ + system, replication, - name, rpc_client, - system, - store, - syncer: ArcSwapOption::from(None), - merkle_updater, }); - table.clone().register_handler(rpc_server, rpc_path); - let syncer = TableSyncer::launch(table.clone()); - table.syncer.swap(Some(syncer)); + let syncer = TableSyncer::launch( + data.clone(), + aux.clone(), + ); - table.merkle_updater.launch(); + let table = Arc::new(Self { + data, + aux, + syncer, + }); + + table.clone().register_handler(rpc_server, rpc_path); table } pub async fn insert(&self, e: &F::E) -> Result<(), Error> { let hash = e.partition_key().hash(); - let who = self.replication.write_nodes(&hash, &self.system); + let who = self.aux.replication.write_nodes(&hash, &self.aux.system); //eprintln!("insert who: {:?}", who); let e_enc = Arc::new(ByteBuf::from(rmp_to_vec_all_named(e)?)); let rpc = TableRPC::::Update(vec![e_enc]); - self.rpc_client + self.aux.rpc_client .try_call_many( &who[..], rpc, - RequestStrategy::with_quorum(self.replication.write_quorum(&self.system)) + RequestStrategy::with_quorum(self.aux.replication.write_quorum(&self.aux.system)) .with_timeout(TABLE_RPC_TIMEOUT), ) .await?; @@ -153,7 +122,7 @@ where for entry in entries.iter() { let hash = entry.partition_key().hash(); - let who = self.replication.write_nodes(&hash, &self.system); + let who = self.aux.replication.write_nodes(&hash, &self.aux.system); let e_enc = Arc::new(ByteBuf::from(rmp_to_vec_all_named(entry)?)); for node in who { if !call_list.contains_key(&node) { @@ -166,7 +135,7 @@ where let call_futures = call_list.drain().map(|(node, entries)| async move { let rpc = TableRPC::::Update(entries); - let resp = self.rpc_client.call(node, rpc, TABLE_RPC_TIMEOUT).await?; + let resp = self.aux.rpc_client.call(node, rpc, TABLE_RPC_TIMEOUT).await?; Ok::<_, Error>((node, resp)) }); let mut resps = call_futures.collect::>(); @@ -177,7 +146,7 @@ where errors.push(e); } } - if errors.len() > self.replication.max_write_errors() { + if errors.len() > self.aux.replication.max_write_errors() { Err(Error::Message("Too many errors".into())) } else { Ok(()) @@ -190,16 +159,17 @@ where sort_key: &F::S, ) -> Result, Error> { let hash = partition_key.hash(); - let who = self.replication.read_nodes(&hash, &self.system); + let who = self.aux.replication.read_nodes(&hash, &self.aux.system); //eprintln!("get who: {:?}", who); let rpc = TableRPC::::ReadEntry(partition_key.clone(), sort_key.clone()); let resps = self + .aux .rpc_client .try_call_many( &who[..], rpc, - RequestStrategy::with_quorum(self.replication.read_quorum()) + RequestStrategy::with_quorum(self.aux.replication.read_quorum()) .with_timeout(TABLE_RPC_TIMEOUT) .interrupt_after_quorum(true), ) @@ -210,7 +180,7 @@ where for resp in resps { if let TableRPC::ReadEntryResponse(value) = resp { if let Some(v_bytes) = value { - let v = self.decode_entry(v_bytes.as_slice())?; + let v = self.data.decode_entry(v_bytes.as_slice())?; ret = match ret { None => Some(v), Some(mut x) => { @@ -230,7 +200,7 @@ where if not_all_same { let self2 = self.clone(); let ent2 = ret_entry.clone(); - self.system + self.aux.system .background .spawn_cancellable(async move { self2.repair_on_read(&who[..], ent2).await }); } @@ -246,16 +216,16 @@ where limit: usize, ) -> Result, Error> { let hash = partition_key.hash(); - let who = self.replication.read_nodes(&hash, &self.system); + let who = self.aux.replication.read_nodes(&hash, &self.aux.system); let rpc = TableRPC::::ReadRange(partition_key.clone(), begin_sort_key, filter, limit); let resps = self - .rpc_client + .aux.rpc_client .try_call_many( &who[..], rpc, - RequestStrategy::with_quorum(self.replication.read_quorum()) + RequestStrategy::with_quorum(self.aux.replication.read_quorum()) .with_timeout(TABLE_RPC_TIMEOUT) .interrupt_after_quorum(true), ) @@ -266,8 +236,8 @@ where for resp in resps { if let TableRPC::Update(entries) = resp { for entry_bytes in entries.iter() { - let entry = self.decode_entry(entry_bytes.as_slice())?; - let entry_key = self.tree_key(entry.partition_key(), entry.sort_key()); + let entry = self.data.decode_entry(entry_bytes.as_slice())?; + let entry_key = self.data.tree_key(entry.partition_key(), entry.sort_key()); match ret.remove(&entry_key) { None => { ret.insert(entry_key, Some(entry)); @@ -287,7 +257,7 @@ where } if !to_repair.is_empty() { let self2 = self.clone(); - self.system.background.spawn_cancellable(async move { + self.aux.system.background.spawn_cancellable(async move { for (_, v) in to_repair.iter_mut() { self2.repair_on_read(&who[..], v.take().unwrap()).await?; } @@ -306,7 +276,7 @@ where async fn repair_on_read(&self, who: &[UUID], what: F::E) -> Result<(), Error> { let what_enc = Arc::new(ByteBuf::from(rmp_to_vec_all_named(&what)?)); - self.rpc_client + self.aux.rpc_client .try_call_many( &who[..], TableRPC::::Update(vec![what_enc]), @@ -326,8 +296,8 @@ where }); let self2 = self.clone(); - self.rpc_client - .set_local_handler(self.system.id, move |msg| { + self.aux.rpc_client + .set_local_handler(self.aux.system.id, move |msg| { let self2 = self2.clone(); async move { self2.handle(&msg).await } }); @@ -336,157 +306,24 @@ where async fn handle(self: &Arc, msg: &TableRPC) -> Result, Error> { match msg { TableRPC::ReadEntry(key, sort_key) => { - let value = self.handle_read_entry(key, sort_key)?; + let value = self.data.read_entry(key, sort_key)?; Ok(TableRPC::ReadEntryResponse(value)) } TableRPC::ReadRange(key, begin_sort_key, filter, limit) => { - let values = self.handle_read_range(key, begin_sort_key, filter, *limit)?; + let values = self.data.read_range(key, begin_sort_key, filter, *limit)?; Ok(TableRPC::Update(values)) } TableRPC::Update(pairs) => { - self.handle_update(pairs)?; + self.data.update_many(pairs)?; Ok(TableRPC::Ok) } TableRPC::SyncRPC(rpc) => { - let syncer = self.syncer.load_full().unwrap(); - let response = syncer - .handle_rpc(rpc, self.system.background.stop_signal.clone()) + let response = self.syncer + .handle_rpc(rpc, self.aux.system.background.stop_signal.clone()) .await?; Ok(TableRPC::SyncRPC(response)) } _ => Err(Error::BadRPC(format!("Unexpected table RPC"))), } } - - fn handle_read_entry(&self, p: &F::P, s: &F::S) -> Result, Error> { - let tree_key = self.tree_key(p, s); - if let Some(bytes) = self.store.get(&tree_key)? { - Ok(Some(ByteBuf::from(bytes.to_vec()))) - } else { - Ok(None) - } - } - - fn handle_read_range( - &self, - p: &F::P, - s: &Option, - filter: &Option, - limit: usize, - ) -> Result>, Error> { - let partition_hash = p.hash(); - let first_key = match s { - None => partition_hash.to_vec(), - Some(sk) => self.tree_key(p, sk), - }; - let mut ret = vec![]; - for item in self.store.range(first_key..) { - let (key, value) = item?; - if &key[..32] != partition_hash.as_slice() { - break; - } - let keep = match filter { - None => true, - Some(f) => { - let entry = self.decode_entry(value.as_ref())?; - F::matches_filter(&entry, f) - } - }; - if keep { - ret.push(Arc::new(ByteBuf::from(value.as_ref()))); - } - if ret.len() >= limit { - break; - } - } - Ok(ret) - } - - // ========== CODE THAT ACTUALLY MODIFIES THE TREE ================ - - pub fn handle_update(self: &Arc, entries: &[Arc]) -> Result<(), Error> { - for update_bytes in entries.iter() { - self.update_entry(update_bytes.as_slice())?; - } - Ok(()) - } - - pub(crate) fn update_entry(self: &Arc, update_bytes: &[u8]) -> Result<(), Error> { - let update = self.decode_entry(update_bytes)?; - let tree_key = self.tree_key(update.partition_key(), update.sort_key()); - - let changed = (&self.store, &self.merkle_updater.todo).transaction(|(db, mkl_todo)| { - let (old_entry, new_entry) = match db.get(&tree_key)? { - Some(prev_bytes) => { - let old_entry = self - .decode_entry(&prev_bytes) - .map_err(sled::transaction::ConflictableTransactionError::Abort)?; - let mut new_entry = old_entry.clone(); - new_entry.merge(&update); - (Some(old_entry), new_entry) - } - None => (None, update.clone()), - }; - - if Some(&new_entry) != old_entry.as_ref() { - let new_bytes = rmp_to_vec_all_named(&new_entry) - .map_err(Error::RMPEncode) - .map_err(sled::transaction::ConflictableTransactionError::Abort)?; - mkl_todo.insert(tree_key.clone(), blake2sum(&new_bytes[..]).to_vec())?; - db.insert(tree_key.clone(), new_bytes)?; - Ok(Some((old_entry, new_entry))) - } else { - Ok(None) - } - })?; - - if let Some((old_entry, new_entry)) = changed { - self.instance.updated(old_entry, Some(new_entry)); - self.syncer.load_full().unwrap().invalidate(&tree_key[..]); - } - - Ok(()) - } - - pub(crate) fn delete_if_equal(self: &Arc, k: &[u8], v: &[u8]) -> Result { - let removed = (&self.store, &self.merkle_updater.todo).transaction(|(txn, mkl_todo)| { - if let Some(cur_v) = txn.get(k)? { - if cur_v == v { - txn.remove(k)?; - mkl_todo.insert(k, vec![])?; - return Ok(true); - } - } - Ok(false) - })?; - - if removed { - let old_entry = self.decode_entry(v)?; - self.instance.updated(Some(old_entry), None); - self.syncer.load_full().unwrap().invalidate(k); - } - Ok(removed) - } - - fn tree_key(&self, p: &F::P, s: &F::S) -> Vec { - let mut ret = p.hash().to_vec(); - ret.extend(s.sort_key()); - ret - } - - fn decode_entry(&self, bytes: &[u8]) -> Result { - match rmp_serde::decode::from_read_ref::<_, F::E>(bytes) { - Ok(x) => Ok(x), - Err(e) => match F::try_migrate(bytes) { - Some(x) => Ok(x), - None => { - warn!("Unable to decode entry of {}: {}", self.name, e); - for line in hexdump::hexdump_iter(bytes) { - debug!("{}", line); - } - Err(e.into()) - } - }, - } - } } diff --git a/src/table/table_fullcopy.rs b/src/table/table_fullcopy.rs deleted file mode 100644 index c55879d8..00000000 --- a/src/table/table_fullcopy.rs +++ /dev/null @@ -1,59 +0,0 @@ -use std::sync::Arc; - -use garage_rpc::membership::System; -use garage_rpc::ring::Ring; -use garage_util::data::*; - -use crate::*; - -#[derive(Clone)] -pub struct TableFullReplication { - pub max_faults: usize, -} - -#[derive(Clone)] -struct Neighbors { - ring: Arc, - neighbors: Vec, -} - -impl TableFullReplication { - pub fn new(max_faults: usize) -> Self { - TableFullReplication { max_faults } - } -} - -impl TableReplication for TableFullReplication { - // Full replication schema: all nodes store everything - // Writes are disseminated in an epidemic manner in the network - - // Advantage: do all reads locally, extremely fast - // Inconvenient: only suitable to reasonably small tables - - fn read_nodes(&self, _hash: &Hash, system: &System) -> Vec { - vec![system.id] - } - fn read_quorum(&self) -> usize { - 1 - } - - fn write_nodes(&self, hash: &Hash, system: &System) -> Vec { - self.replication_nodes(hash, system.ring.borrow().as_ref()) - } - fn write_quorum(&self, system: &System) -> usize { - system.ring.borrow().config.members.len() - self.max_faults - } - fn max_write_errors(&self) -> usize { - self.max_faults - } - - fn replication_nodes(&self, _hash: &Hash, ring: &Ring) -> Vec { - ring.config.members.keys().cloned().collect::>() - } - fn split_points(&self, _ring: &Ring) -> Vec { - let mut ret = vec![]; - ret.push([0u8; 32].into()); - ret.push([0xFFu8; 32].into()); - ret - } -} diff --git a/src/table/table_sharded.rs b/src/table/table_sharded.rs deleted file mode 100644 index 47bdfeaf..00000000 --- a/src/table/table_sharded.rs +++ /dev/null @@ -1,54 +0,0 @@ -use garage_rpc::membership::System; -use garage_rpc::ring::Ring; -use garage_util::data::*; - -use crate::*; - -#[derive(Clone)] -pub struct TableShardedReplication { - pub replication_factor: usize, - pub read_quorum: usize, - pub write_quorum: usize, -} - -impl TableReplication for TableShardedReplication { - // Sharded replication schema: - // - based on the ring of nodes, a certain set of neighbors - // store entries, given as a function of the position of the - // entry's hash in the ring - // - reads are done on all of the nodes that replicate the data - // - writes as well - - fn read_nodes(&self, hash: &Hash, system: &System) -> Vec { - let ring = system.ring.borrow().clone(); - ring.walk_ring(&hash, self.replication_factor) - } - fn read_quorum(&self) -> usize { - self.read_quorum - } - - fn write_nodes(&self, hash: &Hash, system: &System) -> Vec { - let ring = system.ring.borrow().clone(); - ring.walk_ring(&hash, self.replication_factor) - } - fn write_quorum(&self, _system: &System) -> usize { - self.write_quorum - } - fn max_write_errors(&self) -> usize { - self.replication_factor - self.write_quorum - } - - fn replication_nodes(&self, hash: &Hash, ring: &Ring) -> Vec { - ring.walk_ring(&hash, self.replication_factor) - } - fn split_points(&self, ring: &Ring) -> Vec { - let mut ret = vec![]; - - ret.push([0u8; 32].into()); - for entry in ring.ring.iter() { - ret.push(entry.location); - } - ret.push([0xFFu8; 32].into()); - ret - } -} diff --git a/src/table/table_sync.rs b/src/table/table_sync.rs index 51f8cd6f..7394be1b 100644 --- a/src/table/table_sync.rs +++ b/src/table/table_sync.rs @@ -16,18 +16,22 @@ use garage_util::data::*; use garage_util::error::Error; use crate::*; +use crate::data::*; +use crate::replication::*; const MAX_DEPTH: usize = 16; + const TABLE_SYNC_RPC_TIMEOUT: Duration = Duration::from_secs(30); -// Scan & sync every 12 hours -const SCAN_INTERVAL: Duration = Duration::from_secs(12 * 60 * 60); +// Do anti-entropy every 10 minutes +const SCAN_INTERVAL: Duration = Duration::from_secs(10 * 60); -// Expire cache after 30 minutes -const CHECKSUM_CACHE_TIMEOUT: Duration = Duration::from_secs(30 * 60); +const CHECKSUM_CACHE_TIMEOUT: Duration = Duration::from_secs(10 * 60); pub struct TableSyncer { - table: Arc>, + data: Arc>, + aux: Arc>, + todo: Mutex, cache: Vec>>, } @@ -106,10 +110,13 @@ where F: TableSchema + 'static, R: TableReplication + 'static, { - pub(crate) fn launch(table: Arc>) -> Arc { - let todo = SyncTodo { todo: Vec::new() }; - let syncer = Arc::new(TableSyncer { - table: table.clone(), + pub(crate) fn launch(data: Arc>, + aux: Arc>) -> Arc { + let todo = SyncTodo{ todo: vec![] }; + + let syncer = Arc::new(Self { + data: data.clone(), + aux: aux.clone(), todo: Mutex::new(todo), cache: (0..MAX_DEPTH) .map(|_| Mutex::new(BTreeMap::new())) @@ -119,21 +126,21 @@ where let (busy_tx, busy_rx) = mpsc::unbounded_channel(); let s1 = syncer.clone(); - table.system.background.spawn_worker( - format!("table sync watcher for {}", table.name), + aux.system.background.spawn_worker( + format!("table sync watcher for {}", data.name), move |must_exit: watch::Receiver| s1.watcher_task(must_exit, busy_rx), ); let s2 = syncer.clone(); - table.system.background.spawn_worker( - format!("table syncer for {}", table.name), + aux.system.background.spawn_worker( + format!("table syncer for {}", data.name), move |must_exit: watch::Receiver| s2.syncer_task(must_exit, busy_tx), ); let s3 = syncer.clone(); tokio::spawn(async move { tokio::time::delay_for(Duration::from_secs(20)).await; - s3.add_full_scan().await; + s3.add_full_scan(); }); syncer @@ -144,8 +151,8 @@ where mut must_exit: watch::Receiver, mut busy_rx: mpsc::UnboundedReceiver, ) -> Result<(), Error> { - let mut prev_ring: Arc = self.table.system.ring.borrow().clone(); - let mut ring_recv: watch::Receiver> = self.table.system.ring.clone(); + let mut prev_ring: Arc = self.aux.system.ring.borrow().clone(); + let mut ring_recv: watch::Receiver> = self.aux.system.ring.clone(); let mut nothing_to_do_since = Some(Instant::now()); while !*must_exit.borrow() { @@ -158,8 +165,8 @@ where select! { new_ring_r = s_ring_recv => { if let Some(new_ring) = new_ring_r { - debug!("({}) Adding ring difference to syncer todo list", self.table.name); - self.todo.lock().unwrap().add_ring_difference(&self.table, &prev_ring, &new_ring); + debug!("({}) Adding ring difference to syncer todo list", self.data.name); + self.todo.lock().unwrap().add_ring_difference(&prev_ring, &new_ring, &self.data, &self.aux); prev_ring = new_ring; } } @@ -182,8 +189,8 @@ where _ = s_timeout => { if nothing_to_do_since.map(|t| Instant::now() - t >= SCAN_INTERVAL).unwrap_or(false) { nothing_to_do_since = None; - debug!("({}) Adding full scan to syncer todo list", self.table.name); - self.add_full_scan().await; + debug!("({}) Adding full scan to syncer todo list", self.data.name); + self.add_full_scan(); } } } @@ -191,8 +198,8 @@ where Ok(()) } - pub async fn add_full_scan(&self) { - self.todo.lock().unwrap().add_full_scan(&self.table); + pub fn add_full_scan(&self) { + self.todo.lock().unwrap().add_full_scan(&self.data, &self.aux); } async fn syncer_task( @@ -211,7 +218,7 @@ where if let Err(e) = res { warn!( "({}) Error while syncing {:?}: {}", - self.table.name, partition, e + self.data.name, partition, e ); } } else { @@ -228,18 +235,18 @@ where must_exit: &mut watch::Receiver, ) -> Result<(), Error> { if partition.retain { - let my_id = self.table.system.id; + let my_id = self.aux.system.id; let nodes = self - .table + .aux .replication - .write_nodes(&partition.begin, &self.table.system) + .write_nodes(&partition.begin, &self.aux.system) .into_iter() .filter(|node| *node != my_id) .collect::>(); debug!( "({}) Preparing to sync {:?} with {:?}...", - self.table.name, partition, nodes + self.data.name, partition, nodes ); let root_cks = self.root_checksum(&partition.begin, &partition.end, must_exit)?; @@ -259,10 +266,10 @@ where while let Some(r) = sync_futures.next().await { if let Err(e) = r { n_errors += 1; - warn!("({}) Sync error: {}", self.table.name, e); + warn!("({}) Sync error: {}", self.data.name, e); } } - if n_errors > self.table.replication.max_write_errors() { + if n_errors > self.aux.replication.max_write_errors() { return Err(Error::Message(format!( "Sync failed with too many nodes (should have been: {:?}).", nodes @@ -293,7 +300,7 @@ where while !*must_exit.borrow() { let mut items = Vec::new(); - for item in self.table.store.range(begin.to_vec()..end.to_vec()) { + for item in self.data.store.range(begin.to_vec()..end.to_vec()) { let (key, value) = item?; items.push((key.to_vec(), Arc::new(ByteBuf::from(value.as_ref())))); @@ -304,12 +311,12 @@ where if items.len() > 0 { let nodes = self - .table + .aux .replication - .write_nodes(&begin, &self.table.system) + .write_nodes(&begin, &self.aux.system) .into_iter() .collect::>(); - if nodes.contains(&self.table.system.id) { + if nodes.contains(&self.aux.system.id) { warn!("Interrupting offload as partitions seem to have changed"); break; } @@ -340,7 +347,7 @@ where let update_msg = Arc::new(TableRPC::::Update(values)); for res in join_all(nodes.iter().map(|to| { - self.table + self.aux .rpc_client .call_arc(*to, update_msg.clone(), TABLE_SYNC_RPC_TIMEOUT) })) @@ -352,7 +359,7 @@ where // All remote nodes have written those items, now we can delete them locally let mut not_removed = 0; for (k, v) in items.iter() { - if !self.table.delete_if_equal(&k[..], &v[..])? { + if !self.data.delete_if_equal(&k[..], &v[..])? { not_removed += 1; } } @@ -399,7 +406,7 @@ where if range.level == 1 { let mut children = vec![]; for item in self - .table + .data .store .range(range.begin.clone()..range.end.clone()) { @@ -516,7 +523,7 @@ where let v = self.range_checksum(&range, must_exit)?; trace!( "({}) New checksum calculated for {}-{}/{}, {} children", - self.table.name, + self.data.name, hex::encode(&range.begin) .chars() .take(16) @@ -553,7 +560,7 @@ where // If their root checksum has level > than us, use that as a reference let root_cks_resp = self - .table + .aux .rpc_client .call( who, @@ -582,7 +589,7 @@ where let total_children = todo.iter().map(|x| x.children.len()).fold(0, |x, y| x + y); trace!( "({}) Sync with {:?}: {} ({}) remaining", - self.table.name, + self.data.name, who, todo.len(), total_children @@ -592,7 +599,7 @@ where let step = todo.drain(..step_size).collect::>(); let rpc_resp = self - .table + .aux .rpc_client .call( who, @@ -606,7 +613,7 @@ where if diff_ranges.len() > 0 || diff_items.len() > 0 { info!( "({}) Sync with {:?}: difference {} ranges, {} items", - self.table.name, + self.data.name, who, diff_ranges.len(), diff_items.len() @@ -622,7 +629,7 @@ where } } if diff_items.len() > 0 { - self.table.handle_update(&diff_items[..])?; + self.data.update_many(&diff_items[..])?; } if items_to_send.len() > 0 { self.send_items(who, items_to_send).await?; @@ -640,19 +647,19 @@ where async fn send_items(&self, who: UUID, item_list: Vec>) -> Result<(), Error> { info!( "({}) Sending {} items to {:?}", - self.table.name, + self.data.name, item_list.len(), who ); let mut values = vec![]; for item in item_list.iter() { - if let Some(v) = self.table.store.get(&item[..])? { + if let Some(v) = self.data.store.get(&item[..])? { values.push(Arc::new(ByteBuf::from(v.as_ref()))); } } let rpc_resp = self - .table + .aux .rpc_client .call(who, TableRPC::::Update(values), TABLE_SYNC_RPC_TIMEOUT) .await?; @@ -714,7 +721,7 @@ where ret_ranges.push(their_range.clone()); if their_range.level == 0 { if let Some(item_bytes) = - self.table.store.get(their_range.begin.as_slice())? + self.data.store.get(their_range.begin.as_slice())? { ret_items.push(Arc::new(ByteBuf::from(item_bytes.to_vec()))); } @@ -738,7 +745,7 @@ where } if our_range.level == 0 { if let Some(item_bytes) = - self.table.store.get(our_range.begin.as_slice())? + self.data.store.get(our_range.begin.as_slice())? { ret_items.push(Arc::new(ByteBuf::from(item_bytes.to_vec()))); } @@ -753,7 +760,7 @@ where if ret_ranges.len() > 0 || ret_items.len() > 0 { trace!( "({}) Checksum comparison RPC: {} different + {} items for {} received", - self.table.name, + self.data.name, ret_ranges.len(), ret_items.len(), n_checksums @@ -782,13 +789,13 @@ where } impl SyncTodo { - fn add_full_scan(&mut self, table: &Table) { - let my_id = table.system.id; + fn add_full_scan(&mut self, data: &TableData, aux: &TableAux) { + let my_id = aux.system.id; self.todo.clear(); - let ring = table.system.ring.borrow().clone(); - let split_points = table.replication.split_points(&ring); + let ring = aux.system.ring.borrow().clone(); + let split_points = aux.replication.split_points(&ring); for i in 0..split_points.len() - 1 { let begin = split_points[i]; @@ -797,12 +804,12 @@ impl SyncTodo { continue; } - let nodes = table.replication.replication_nodes(&begin, &ring); + let nodes = aux.replication.replication_nodes(&begin, &ring); let retain = nodes.contains(&my_id); if !retain { // Check if we have some data to send, otherwise skip - if table.store.range(begin..end).next().is_none() { + if data.store.range(begin..end).next().is_none() { continue; } } @@ -813,25 +820,25 @@ impl SyncTodo { fn add_ring_difference( &mut self, - table: &Table, old_ring: &Ring, new_ring: &Ring, + data: &TableData, aux: &TableAux, ) { - let my_id = table.system.id; + let my_id = aux.system.id; // If it is us who are entering or leaving the system, // initiate a full sync instead of incremental sync if old_ring.config.members.contains_key(&my_id) != new_ring.config.members.contains_key(&my_id) { - self.add_full_scan(table); + self.add_full_scan(data, aux); return; } let mut all_points = None .into_iter() - .chain(table.replication.split_points(old_ring).drain(..)) - .chain(table.replication.split_points(new_ring).drain(..)) + .chain(aux.replication.split_points(old_ring).drain(..)) + .chain(aux.replication.split_points(new_ring).drain(..)) .chain(self.todo.iter().map(|x| x.begin)) .chain(self.todo.iter().map(|x| x.end)) .collect::>(); @@ -845,11 +852,11 @@ impl SyncTodo { for i in 0..all_points.len() - 1 { let begin = all_points[i]; let end = all_points[i + 1]; - let was_ours = table + let was_ours = aux .replication .replication_nodes(&begin, &old_ring) .contains(&my_id); - let is_ours = table + let is_ours = aux .replication .replication_nodes(&begin, &new_ring) .contains(&my_id); -- cgit v1.2.3 From 046b649bcc3b147140fc2b0af0e071d3dd1b2c8d Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 11 Mar 2021 18:28:03 +0100 Subject: (not well tested) use merkle tree for sync --- src/garage/repair.rs | 33 +- src/model/garage.rs | 2 +- src/rpc/ring.rs | 2 +- src/table/data.rs | 15 +- src/table/lib.rs | 4 +- src/table/merkle.rs | 107 ++++- src/table/replication/fullcopy.rs | 1 - src/table/replication/sharded.rs | 6 +- src/table/sync.rs | 632 +++++++++++++++++++++++++++ src/table/table.rs | 50 +-- src/table/table_sync.rs | 898 -------------------------------------- 11 files changed, 765 insertions(+), 985 deletions(-) create mode 100644 src/table/sync.rs delete mode 100644 src/table/table_sync.rs diff --git a/src/garage/repair.rs b/src/garage/repair.rs index f9cd5884..47fc1ae1 100644 --- a/src/garage/repair.rs +++ b/src/garage/repair.rs @@ -25,26 +25,11 @@ impl Repair { if todo(RepairWhat::Tables) { info!("Launching a full sync of tables"); - self.garage - .bucket_table - .syncer - .add_full_scan(); - self.garage - .object_table - .syncer - .add_full_scan(); - self.garage - .version_table - .syncer - .add_full_scan(); - self.garage - .block_ref_table - .syncer - .add_full_scan(); - self.garage - .key_table - .syncer - .add_full_scan(); + self.garage.bucket_table.syncer.add_full_sync(); + self.garage.object_table.syncer.add_full_sync(); + self.garage.version_table.syncer.add_full_sync(); + self.garage.block_ref_table.syncer.add_full_sync(); + self.garage.key_table.syncer.add_full_sync(); } // TODO: wait for full sync to finish before proceeding to the rest? @@ -78,7 +63,9 @@ impl Repair { async fn repair_versions(&self, must_exit: &watch::Receiver) -> Result<(), Error> { let mut pos = vec![]; - while let Some((item_key, item_bytes)) = self.garage.version_table.data.store.get_gt(&pos)? { + while let Some((item_key, item_bytes)) = + self.garage.version_table.data.store.get_gt(&pos)? + { pos = item_key.to_vec(); let version = rmp_serde::decode::from_read_ref::<_, Version>(item_bytes.as_ref())?; @@ -126,7 +113,9 @@ impl Repair { async fn repair_block_ref(&self, must_exit: &watch::Receiver) -> Result<(), Error> { let mut pos = vec![]; - while let Some((item_key, item_bytes)) = self.garage.block_ref_table.data.store.get_gt(&pos)? { + while let Some((item_key, item_bytes)) = + self.garage.block_ref_table.data.store.get_gt(&pos)? + { pos = item_key.to_vec(); let block_ref = rmp_serde::decode::from_read_ref::<_, BlockRef>(item_bytes.as_ref())?; diff --git a/src/model/garage.rs b/src/model/garage.rs index 193c71d2..ced3c29e 100644 --- a/src/model/garage.rs +++ b/src/model/garage.rs @@ -7,8 +7,8 @@ use garage_rpc::membership::System; use garage_rpc::rpc_client::RpcHttpClient; use garage_rpc::rpc_server::RpcServer; -use garage_table::replication::sharded::*; use garage_table::replication::fullcopy::*; +use garage_table::replication::sharded::*; use garage_table::*; use crate::block::*; diff --git a/src/rpc/ring.rs b/src/rpc/ring.rs index 0f86bbb2..215ab031 100644 --- a/src/rpc/ring.rs +++ b/src/rpc/ring.rs @@ -183,7 +183,7 @@ impl Ring { let partition_top = u16::from_be_bytes(partition.location.as_slice()[0..2].try_into().unwrap()); - assert!(partition_top & PARTITION_MASK_U16 == top & PARTITION_MASK_U16); + assert_eq!(partition_top & PARTITION_MASK_U16, top & PARTITION_MASK_U16); assert!(n <= partition.nodes.len()); partition.nodes[..n].iter().cloned().collect::>() diff --git a/src/table/data.rs b/src/table/data.rs index fa89fc27..6217bf6d 100644 --- a/src/table/data.rs +++ b/src/table/data.rs @@ -1,16 +1,16 @@ use std::sync::Arc; use log::warn; -use sled::Transactional; use serde_bytes::ByteBuf; +use sled::Transactional; +use garage_util::background::BackgroundRunner; use garage_util::data::*; use garage_util::error::*; -use garage_util::background::BackgroundRunner; -use crate::schema::*; -use crate::merkle::*; use crate::crdt::CRDT; +use crate::merkle::*; +use crate::schema::*; pub struct TableData { pub name: String, @@ -20,7 +20,10 @@ pub struct TableData { pub(crate) merkle_updater: Arc, } -impl TableData where F: TableSchema { +impl TableData +where + F: TableSchema, +{ pub fn new( name: String, instance: F, @@ -45,7 +48,7 @@ impl TableData where F: TableSchema { merkle_tree_store, ); - Arc::new(Self{ + Arc::new(Self { name, instance, store, diff --git a/src/table/lib.rs b/src/table/lib.rs index bb249a56..18c29c35 100644 --- a/src/table/lib.rs +++ b/src/table/lib.rs @@ -7,11 +7,11 @@ pub mod crdt; pub mod schema; pub mod util; +pub mod data; pub mod merkle; pub mod replication; -pub mod data; +pub mod sync; pub mod table; -pub mod table_sync; pub use schema::*; pub use table::*; diff --git a/src/table/merkle.rs b/src/table/merkle.rs index ef197dc8..92c18e09 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -15,6 +15,19 @@ use garage_util::background::BackgroundRunner; use garage_util::data::*; use garage_util::error::Error; +pub type MerklePartition = [u8; 2]; + +pub fn hash_of_merkle_partition(p: MerklePartition) -> Hash { + let mut partition_pos = [0u8; 32]; + partition_pos[0..2].copy_from_slice(&p[..]); + partition_pos.into() +} + +pub fn hash_of_merkle_partition_opt(p: Option) -> Hash { + p.map(hash_of_merkle_partition) + .unwrap_or([0xFFu8; 32].into()) +} + // This modules partitions the data in 2**16 partitions, based on the top // 16 bits (two bytes) of item's partition keys' hashes. // It builds one Merkle tree for each of these 2**16 partitions. @@ -37,10 +50,10 @@ pub(crate) struct MerkleUpdater { empty_node_hash: Hash, } -#[derive(Clone)] +#[derive(Clone, Serialize, Deserialize)] pub struct MerkleNodeKey { // partition: first 16 bits (two bytes) of the partition_key's hash - pub partition: [u8; 2], + pub partition: MerklePartition, // prefix: a prefix for the hash of full keys, i.e. hash(hash(partition_key)+sort_key) pub prefix: Vec, @@ -214,8 +227,11 @@ impl MerkleUpdater { // insertion and replace current node by an intermediary node let (pos1, h1) = { let key2 = key.next_key(blake2sum(&exlf_key[..])); - let subhash = - self.put_node_txn(tx, &key2, &MerkleNode::Leaf(exlf_key, exlf_hash))?; + let subhash = self.put_node_txn( + tx, + &key2, + &MerkleNode::Leaf(exlf_key, exlf_hash), + )?; (key2.prefix[i], subhash) }; let (pos2, h2) = { @@ -280,14 +296,11 @@ impl MerkleUpdater { } // Access a node in the Merkle tree, used by the sync protocol - pub(crate) fn read_node( - &self, - k: &MerkleNodeKey, - ) -> Result { + pub(crate) fn read_node(&self, k: &MerkleNodeKey) -> Result { let ent = self.merkle_tree.get(k.encode())?; match ent { None => Ok(MerkleNode::Empty), - Some(v) => Ok(rmp_serde::decode::from_read_ref::<_, MerkleNode>(&v[..])?) + Some(v) => Ok(rmp_serde::decode::from_read_ref::<_, MerkleNode>(&v[..])?), } } } @@ -339,31 +352,77 @@ fn intermediate_rm_child(ch: &mut Vec<(u8, Hash)>, pos: u8) { #[test] fn test_intermediate_aux() { let mut v = vec![]; - + intermediate_set_child(&mut v, 12u8, [12u8; 32].into()); - assert!(v == vec![(12u8, [12u8; 32].into())]); - + assert_eq!(v, vec![(12u8, [12u8; 32].into())]); + intermediate_set_child(&mut v, 42u8, [42u8; 32].into()); - assert!(v == vec![(12u8, [12u8; 32].into()), (42u8, [42u8; 32].into())]); - + assert_eq!( + v, + vec![(12u8, [12u8; 32].into()), (42u8, [42u8; 32].into())] + ); + intermediate_set_child(&mut v, 4u8, [4u8; 32].into()); - assert!(v == vec![(4u8, [4u8; 32].into()), (12u8, [12u8; 32].into()), (42u8, [42u8; 32].into())]); - + assert_eq!( + v, + vec![ + (4u8, [4u8; 32].into()), + (12u8, [12u8; 32].into()), + (42u8, [42u8; 32].into()) + ] + ); + intermediate_set_child(&mut v, 12u8, [8u8; 32].into()); - assert!(v == vec![(4u8, [4u8; 32].into()), (12u8, [8u8; 32].into()), (42u8, [42u8; 32].into())]); - + assert_eq!( + v, + vec![ + (4u8, [4u8; 32].into()), + (12u8, [8u8; 32].into()), + (42u8, [42u8; 32].into()) + ] + ); + intermediate_set_child(&mut v, 6u8, [6u8; 32].into()); - assert!(v == vec![(4u8, [4u8; 32].into()), (6u8, [6u8; 32].into()), (12u8, [8u8; 32].into()), (42u8, [42u8; 32].into())]); + assert_eq!( + v, + vec![ + (4u8, [4u8; 32].into()), + (6u8, [6u8; 32].into()), + (12u8, [8u8; 32].into()), + (42u8, [42u8; 32].into()) + ] + ); intermediate_rm_child(&mut v, 42u8); - assert!(v == vec![(4u8, [4u8; 32].into()), (6u8, [6u8; 32].into()), (12u8, [8u8; 32].into())]); + assert_eq!( + v, + vec![ + (4u8, [4u8; 32].into()), + (6u8, [6u8; 32].into()), + (12u8, [8u8; 32].into()) + ] + ); intermediate_rm_child(&mut v, 11u8); - assert!(v == vec![(4u8, [4u8; 32].into()), (6u8, [6u8; 32].into()), (12u8, [8u8; 32].into())]); + assert_eq!( + v, + vec![ + (4u8, [4u8; 32].into()), + (6u8, [6u8; 32].into()), + (12u8, [8u8; 32].into()) + ] + ); intermediate_rm_child(&mut v, 6u8); - assert!(v == vec![(4u8, [4u8; 32].into()), (12u8, [8u8; 32].into())]); - + assert_eq!(v, vec![(4u8, [4u8; 32].into()), (12u8, [8u8; 32].into())]); + intermediate_set_child(&mut v, 6u8, [7u8; 32].into()); - assert!(v == vec![(4u8, [4u8; 32].into()), (6u8, [7u8; 32].into()), (12u8, [8u8; 32].into())]); + assert_eq!( + v, + vec![ + (4u8, [4u8; 32].into()), + (6u8, [7u8; 32].into()), + (12u8, [8u8; 32].into()) + ] + ); } diff --git a/src/table/replication/fullcopy.rs b/src/table/replication/fullcopy.rs index a62a6c3c..a20f20b7 100644 --- a/src/table/replication/fullcopy.rs +++ b/src/table/replication/fullcopy.rs @@ -53,7 +53,6 @@ impl TableReplication for TableFullReplication { fn split_points(&self, _ring: &Ring) -> Vec { let mut ret = vec![]; ret.push([0u8; 32].into()); - ret.push([0xFFu8; 32].into()); ret } } diff --git a/src/table/replication/sharded.rs b/src/table/replication/sharded.rs index 42a742cd..886c7c08 100644 --- a/src/table/replication/sharded.rs +++ b/src/table/replication/sharded.rs @@ -44,11 +44,13 @@ impl TableReplication for TableShardedReplication { fn split_points(&self, ring: &Ring) -> Vec { let mut ret = vec![]; - ret.push([0u8; 32].into()); for entry in ring.ring.iter() { ret.push(entry.location); } - ret.push([0xFFu8; 32].into()); + if ret.len() > 0 { + assert_eq!(ret[0], [0u8; 32].into()); + } + ret } } diff --git a/src/table/sync.rs b/src/table/sync.rs new file mode 100644 index 00000000..9c37c286 --- /dev/null +++ b/src/table/sync.rs @@ -0,0 +1,632 @@ +use std::collections::VecDeque; +use std::convert::TryInto; +use std::sync::{Arc, Mutex}; +use std::time::{Duration, Instant}; + +use futures::future::join_all; +use futures::{pin_mut, select}; +use futures_util::future::*; +use futures_util::stream::*; +use rand::Rng; +use serde::{Deserialize, Serialize}; +use serde_bytes::ByteBuf; +use tokio::sync::{mpsc, watch}; + +use garage_rpc::ring::Ring; +use garage_util::data::*; +use garage_util::error::Error; + +use crate::data::*; +use crate::merkle::*; +use crate::replication::*; +use crate::*; + +const TABLE_SYNC_RPC_TIMEOUT: Duration = Duration::from_secs(30); + +// Do anti-entropy every 10 minutes +const ANTI_ENTROPY_INTERVAL: Duration = Duration::from_secs(10 * 60); + +pub struct TableSyncer { + data: Arc>, + aux: Arc>, + + todo: Mutex, +} + +type RootCk = Vec<(MerklePartition, Hash)>; + +#[derive(Debug, Clone, Copy, Serialize, Deserialize)] +pub struct PartitionRange { + begin: MerklePartition, + // if end is None, go all the way to partition 0xFFFF included + end: Option, +} + +#[derive(Serialize, Deserialize)] +pub(crate) enum SyncRPC { + RootCkHash(PartitionRange, Hash), + RootCkList(PartitionRange, RootCk), + CkNoDifference, + GetNode(MerkleNodeKey), + Node(MerkleNodeKey, MerkleNode), + Items(Vec>), +} + +struct SyncTodo { + todo: Vec, +} + +#[derive(Debug, Clone)] +struct TodoPartition { + range: PartitionRange, + + // Are we a node that stores this partition or not? + retain: bool, +} + +impl TableSyncer +where + F: TableSchema + 'static, + R: TableReplication + 'static, +{ + pub(crate) fn launch(data: Arc>, aux: Arc>) -> Arc { + let todo = SyncTodo { todo: vec![] }; + + let syncer = Arc::new(Self { + data: data.clone(), + aux: aux.clone(), + todo: Mutex::new(todo), + }); + + let (busy_tx, busy_rx) = mpsc::unbounded_channel(); + + let s1 = syncer.clone(); + aux.system.background.spawn_worker( + format!("table sync watcher for {}", data.name), + move |must_exit: watch::Receiver| s1.watcher_task(must_exit, busy_rx), + ); + + let s2 = syncer.clone(); + aux.system.background.spawn_worker( + format!("table syncer for {}", data.name), + move |must_exit: watch::Receiver| s2.syncer_task(must_exit, busy_tx), + ); + + let s3 = syncer.clone(); + tokio::spawn(async move { + tokio::time::delay_for(Duration::from_secs(20)).await; + s3.add_full_sync(); + }); + + syncer + } + + async fn watcher_task( + self: Arc, + mut must_exit: watch::Receiver, + mut busy_rx: mpsc::UnboundedReceiver, + ) -> Result<(), Error> { + let mut ring_recv: watch::Receiver> = self.aux.system.ring.clone(); + let mut nothing_to_do_since = Some(Instant::now()); + + while !*must_exit.borrow() { + let s_ring_recv = ring_recv.recv().fuse(); + let s_busy = busy_rx.recv().fuse(); + let s_must_exit = must_exit.recv().fuse(); + let s_timeout = tokio::time::delay_for(Duration::from_secs(1)).fuse(); + pin_mut!(s_ring_recv, s_busy, s_must_exit, s_timeout); + + select! { + new_ring_r = s_ring_recv => { + if new_ring_r.is_some() { + debug!("({}) Adding ring difference to syncer todo list", self.data.name); + self.add_full_sync(); + } + } + busy_opt = s_busy => { + if let Some(busy) = busy_opt { + if busy { + nothing_to_do_since = None; + } else { + if nothing_to_do_since.is_none() { + nothing_to_do_since = Some(Instant::now()); + } + } + } + } + must_exit_v = s_must_exit => { + if must_exit_v.unwrap_or(false) { + break; + } + } + _ = s_timeout => { + if nothing_to_do_since.map(|t| Instant::now() - t >= ANTI_ENTROPY_INTERVAL).unwrap_or(false) { + nothing_to_do_since = None; + debug!("({}) Adding full sync to syncer todo list", self.data.name); + self.add_full_sync(); + } + } + } + } + Ok(()) + } + + pub fn add_full_sync(&self) { + self.todo + .lock() + .unwrap() + .add_full_sync(&self.data, &self.aux); + } + + async fn syncer_task( + self: Arc, + mut must_exit: watch::Receiver, + busy_tx: mpsc::UnboundedSender, + ) -> Result<(), Error> { + while !*must_exit.borrow() { + let task = self.todo.lock().unwrap().pop_task(); + if let Some(partition) = task { + busy_tx.send(true)?; + let res = self + .clone() + .sync_partition(&partition, &mut must_exit) + .await; + if let Err(e) = res { + warn!( + "({}) Error while syncing {:?}: {}", + self.data.name, partition, e + ); + } + } else { + busy_tx.send(false)?; + tokio::time::delay_for(Duration::from_secs(1)).await; + } + } + Ok(()) + } + + async fn sync_partition( + self: Arc, + partition: &TodoPartition, + must_exit: &mut watch::Receiver, + ) -> Result<(), Error> { + if partition.retain { + let my_id = self.aux.system.id; + + let nodes = self + .aux + .replication + .write_nodes( + &hash_of_merkle_partition(partition.range.begin), + &self.aux.system, + ) + .into_iter() + .filter(|node| *node != my_id) + .collect::>(); + + debug!( + "({}) Syncing {:?} with {:?}...", + self.data.name, partition, nodes + ); + let mut sync_futures = nodes + .iter() + .map(|node| { + self.clone() + .do_sync_with(partition.clone(), *node, must_exit.clone()) + }) + .collect::>(); + + let mut n_errors = 0; + while let Some(r) = sync_futures.next().await { + if let Err(e) = r { + n_errors += 1; + warn!("({}) Sync error: {}", self.data.name, e); + } + } + if n_errors > self.aux.replication.max_write_errors() { + return Err(Error::Message(format!( + "Sync failed with too many nodes (should have been: {:?}).", + nodes + ))); + } + } else { + self.offload_partition( + &hash_of_merkle_partition(partition.range.begin), + &hash_of_merkle_partition_opt(partition.range.end), + must_exit, + ) + .await?; + } + + Ok(()) + } + + // Offload partition: this partition is not something we are storing, + // so send it out to all other nodes that store it and delete items locally. + // We don't bother checking if the remote nodes already have the items, + // we just batch-send everything. Offloading isn't supposed to happen very often. + // If any of the nodes that are supposed to store the items is unable to + // save them, we interrupt the process. + async fn offload_partition( + self: &Arc, + begin: &Hash, + end: &Hash, + must_exit: &mut watch::Receiver, + ) -> Result<(), Error> { + let mut counter: usize = 0; + + while !*must_exit.borrow() { + let mut items = Vec::new(); + + for item in self.data.store.range(begin.to_vec()..end.to_vec()) { + let (key, value) = item?; + items.push((key.to_vec(), Arc::new(ByteBuf::from(value.as_ref())))); + + if items.len() >= 1024 { + break; + } + } + + if items.len() > 0 { + let nodes = self + .aux + .replication + .write_nodes(&begin, &self.aux.system) + .into_iter() + .collect::>(); + if nodes.contains(&self.aux.system.id) { + warn!("Interrupting offload as partitions seem to have changed"); + break; + } + + counter += 1; + debug!( + "Offloading {} items from {:?}..{:?} ({})", + items.len(), + begin, + end, + counter + ); + self.offload_items(&items, &nodes[..]).await?; + } else { + break; + } + } + + Ok(()) + } + + async fn offload_items( + self: &Arc, + items: &Vec<(Vec, Arc)>, + nodes: &[UUID], + ) -> Result<(), Error> { + let values = items.iter().map(|(_k, v)| v.clone()).collect::>(); + let update_msg = Arc::new(TableRPC::::Update(values)); + + for res in join_all(nodes.iter().map(|to| { + self.aux + .rpc_client + .call_arc(*to, update_msg.clone(), TABLE_SYNC_RPC_TIMEOUT) + })) + .await + { + res?; + } + + // All remote nodes have written those items, now we can delete them locally + let mut not_removed = 0; + for (k, v) in items.iter() { + if !self.data.delete_if_equal(&k[..], &v[..])? { + not_removed += 1; + } + } + + if not_removed > 0 { + debug!("{} items not removed during offload because they changed in between (trying again...)", not_removed); + } + + Ok(()) + } + + // ======= SYNCHRONIZATION PROCEDURE -- DRIVER SIDE ====== + + fn get_root_ck(&self, range: PartitionRange) -> Result { + let begin = u16::from_be_bytes(range.begin); + let range_iter = match range.end { + Some(end) => { + let end = u16::from_be_bytes(end); + begin..=(end - 1) + } + None => begin..=0xFFFF, + }; + + let mut ret = vec![]; + for i in range_iter { + let key = MerkleNodeKey { + partition: u16::to_be_bytes(i), + prefix: vec![], + }; + match self.data.merkle_updater.read_node(&key)? { + MerkleNode::Empty => (), + x => { + ret.push((key.partition, hash_of(&x)?)); + } + } + } + Ok(ret) + } + + async fn do_sync_with( + self: Arc, + partition: TodoPartition, + who: UUID, + must_exit: watch::Receiver, + ) -> Result<(), Error> { + let root_ck = self.get_root_ck(partition.range)?; + let root_ck_hash = hash_of(&root_ck)?; + + // If their root checksum has level > than us, use that as a reference + let root_resp = self + .aux + .rpc_client + .call( + who, + TableRPC::::SyncRPC(SyncRPC::RootCkHash(partition.range, root_ck_hash)), + TABLE_SYNC_RPC_TIMEOUT, + ) + .await?; + + let mut todo = match root_resp { + TableRPC::::SyncRPC(SyncRPC::CkNoDifference) => { + debug!( + "({}) Sync {:?} with {:?}: no difference", + self.data.name, partition, who + ); + return Ok(()); + } + TableRPC::::SyncRPC(SyncRPC::RootCkList(_, their_root_ck)) => { + let join = join_ordered(&root_ck[..], &their_root_ck[..]); + let mut todo = VecDeque::new(); + for (p, v1, v2) in join.iter() { + let diff = match (v1, v2) { + (Some(_), None) | (None, Some(_)) => true, + (Some(a), Some(b)) => a != b, + _ => false, + }; + if diff { + todo.push_back(MerkleNodeKey { + partition: **p, + prefix: vec![], + }); + } + } + debug!( + "({}) Sync {:?} with {:?}: todo.len() = {}", + self.data.name, + partition, + who, + todo.len() + ); + todo + } + x => { + return Err(Error::Message(format!( + "Invalid respone to RootCkHash RPC: {}", + debug_serialize(x) + ))); + } + }; + + let mut todo_items = vec![]; + + while !todo.is_empty() && !*must_exit.borrow() { + let key = todo.pop_front().unwrap(); + let node = self.data.merkle_updater.read_node(&key)?; + + match node { + MerkleNode::Empty => { + // They have items we don't have. + // We don't request those items from them, they will send them. + // We only bother with pushing items that differ + } + MerkleNode::Leaf(ik, _) => { + // Just send that item directly + if let Some(val) = self.data.store.get(ik)? { + todo_items.push(val.to_vec()); + } + } + MerkleNode::Intermediate(l) => { + let remote_node = match self + .aux + .rpc_client + .call( + who, + TableRPC::::SyncRPC(SyncRPC::GetNode(key.clone())), + TABLE_SYNC_RPC_TIMEOUT, + ) + .await? + { + TableRPC::::SyncRPC(SyncRPC::Node(_, node)) => node, + x => { + return Err(Error::Message(format!( + "Invalid respone to GetNode RPC: {}", + debug_serialize(x) + ))); + } + }; + let int_l2 = match remote_node { + MerkleNode::Intermediate(l2) => l2, + _ => vec![], + }; + + let join = join_ordered(&l[..], &int_l2[..]); + for (p, v1, v2) in join.into_iter() { + let diff = match (v1, v2) { + (Some(_), None) | (None, Some(_)) => true, + (Some(a), Some(b)) => a != b, + _ => false, + }; + if diff { + todo.push_back(key.add_byte(*p)); + } + } + } + } + + if todo_items.len() >= 256 { + self.send_items(who, std::mem::replace(&mut todo_items, vec![])) + .await?; + } + } + + if !todo_items.is_empty() { + self.send_items(who, todo_items).await?; + } + + Ok(()) + } + + async fn send_items(&self, who: UUID, item_list: Vec>) -> Result<(), Error> { + info!( + "({}) Sending {} items to {:?}", + self.data.name, + item_list.len(), + who + ); + + let mut values = vec![]; + for item in item_list.iter() { + if let Some(v) = self.data.store.get(&item[..])? { + values.push(Arc::new(ByteBuf::from(v.as_ref()))); + } + } + let rpc_resp = self + .aux + .rpc_client + .call(who, TableRPC::::Update(values), TABLE_SYNC_RPC_TIMEOUT) + .await?; + if let TableRPC::::Ok = rpc_resp { + Ok(()) + } else { + Err(Error::Message(format!( + "Unexpected response to RPC Update: {}", + debug_serialize(&rpc_resp) + ))) + } + } + + // ======= SYNCHRONIZATION PROCEDURE -- RECEIVER SIDE ====== + + pub(crate) async fn handle_rpc(self: &Arc, message: &SyncRPC) -> Result { + match message { + SyncRPC::RootCkHash(range, h) => { + let root_ck = self.get_root_ck(*range)?; + let hash = hash_of(&root_ck)?; + if hash == *h { + Ok(SyncRPC::CkNoDifference) + } else { + Ok(SyncRPC::RootCkList(*range, root_ck)) + } + } + SyncRPC::GetNode(k) => { + let node = self.data.merkle_updater.read_node(&k)?; + Ok(SyncRPC::Node(k.clone(), node)) + } + _ => Err(Error::Message(format!("Unexpected sync RPC"))), + } + } +} + +impl SyncTodo { + fn add_full_sync( + &mut self, + data: &TableData, + aux: &TableAux, + ) { + let my_id = aux.system.id; + + self.todo.clear(); + + let ring = aux.system.ring.borrow().clone(); + let split_points = aux.replication.split_points(&ring); + + for i in 0..split_points.len() { + let begin: MerklePartition = { + let b = split_points[i]; + assert_eq!(b.as_slice()[2..], [0u8; 30][..]); + b.as_slice()[..2].try_into().unwrap() + }; + + let end: Option = if i + 1 < split_points.len() { + let e = split_points[i + 1]; + assert_eq!(e.as_slice()[2..], [0u8; 30][..]); + Some(e.as_slice()[..2].try_into().unwrap()) + } else { + None + }; + + let begin_hash = hash_of_merkle_partition(begin); + let end_hash = hash_of_merkle_partition_opt(end); + + let nodes = aux.replication.replication_nodes(&begin_hash, &ring); + + let retain = nodes.contains(&my_id); + if !retain { + // Check if we have some data to send, otherwise skip + if data.store.range(begin_hash..end_hash).next().is_none() { + continue; + } + } + + self.todo.push(TodoPartition { + range: PartitionRange { begin, end }, + retain, + }); + } + } + + fn pop_task(&mut self) -> Option { + if self.todo.is_empty() { + return None; + } + + let i = rand::thread_rng().gen_range::(0, self.todo.len()); + if i == self.todo.len() - 1 { + self.todo.pop() + } else { + let replacement = self.todo.pop().unwrap(); + let ret = std::mem::replace(&mut self.todo[i], replacement); + Some(ret) + } + } +} + +fn hash_of(x: &T) -> Result { + Ok(blake2sum(&rmp_to_vec_all_named(x)?[..])) +} + +fn join_ordered<'a, K: Ord + Eq, V1, V2>( + x: &'a [(K, V1)], + y: &'a [(K, V2)], +) -> Vec<(&'a K, Option<&'a V1>, Option<&'a V2>)> { + let mut ret = vec![]; + let mut i = 0; + let mut j = 0; + while i < x.len() || j < y.len() { + if i < x.len() && j < y.len() && x[i].0 == y[j].0 { + ret.push((&x[i].0, Some(&x[i].1), Some(&y[j].1))); + i += 1; + j += 1; + } else if i < x.len() && (j == y.len() || x[i].0 < y[j].0) { + ret.push((&x[i].0, Some(&x[i].1), None)); + i += 1; + } else if j < y.len() && (i == x.len() || x[i].0 > y[j].0) { + ret.push((&x[i].0, None, Some(&y[j].1))); + j += 1; + } else { + unreachable!(); + } + } + ret +} diff --git a/src/table/table.rs b/src/table/table.rs index a4cb4b24..516c9358 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -15,9 +15,9 @@ use garage_rpc::rpc_server::*; use crate::crdt::CRDT; use crate::data::*; -use crate::schema::*; -use crate::table_sync::*; use crate::replication::*; +use crate::schema::*; +use crate::sync::*; const TABLE_RPC_TIMEOUT: Duration = Duration::from_secs(10); @@ -50,7 +50,6 @@ pub(crate) enum TableRPC { impl RpcMessage for TableRPC {} - impl Table where F: TableSchema + 'static, @@ -69,29 +68,17 @@ where let rpc_path = format!("table_{}", name); let rpc_client = system.rpc_client::>(&rpc_path); - let data = TableData::new( - name, - instance, - db, - system.background.clone(), - ); + let data = TableData::new(name, instance, db, system.background.clone()); - let aux = Arc::new(TableAux{ + let aux = Arc::new(TableAux { system, replication, rpc_client, }); - let syncer = TableSyncer::launch( - data.clone(), - aux.clone(), - ); + let syncer = TableSyncer::launch(data.clone(), aux.clone()); - let table = Arc::new(Self { - data, - aux, - syncer, - }); + let table = Arc::new(Self { data, aux, syncer }); table.clone().register_handler(rpc_server, rpc_path); @@ -106,7 +93,8 @@ where let e_enc = Arc::new(ByteBuf::from(rmp_to_vec_all_named(e)?)); let rpc = TableRPC::::Update(vec![e_enc]); - self.aux.rpc_client + self.aux + .rpc_client .try_call_many( &who[..], rpc, @@ -135,7 +123,11 @@ where let call_futures = call_list.drain().map(|(node, entries)| async move { let rpc = TableRPC::::Update(entries); - let resp = self.aux.rpc_client.call(node, rpc, TABLE_RPC_TIMEOUT).await?; + let resp = self + .aux + .rpc_client + .call(node, rpc, TABLE_RPC_TIMEOUT) + .await?; Ok::<_, Error>((node, resp)) }); let mut resps = call_futures.collect::>(); @@ -200,7 +192,8 @@ where if not_all_same { let self2 = self.clone(); let ent2 = ret_entry.clone(); - self.aux.system + self.aux + .system .background .spawn_cancellable(async move { self2.repair_on_read(&who[..], ent2).await }); } @@ -221,7 +214,8 @@ where let rpc = TableRPC::::ReadRange(partition_key.clone(), begin_sort_key, filter, limit); let resps = self - .aux.rpc_client + .aux + .rpc_client .try_call_many( &who[..], rpc, @@ -276,7 +270,8 @@ where async fn repair_on_read(&self, who: &[UUID], what: F::E) -> Result<(), Error> { let what_enc = Arc::new(ByteBuf::from(rmp_to_vec_all_named(&what)?)); - self.aux.rpc_client + self.aux + .rpc_client .try_call_many( &who[..], TableRPC::::Update(vec![what_enc]), @@ -296,7 +291,8 @@ where }); let self2 = self.clone(); - self.aux.rpc_client + self.aux + .rpc_client .set_local_handler(self.aux.system.id, move |msg| { let self2 = self2.clone(); async move { self2.handle(&msg).await } @@ -318,9 +314,7 @@ where Ok(TableRPC::Ok) } TableRPC::SyncRPC(rpc) => { - let response = self.syncer - .handle_rpc(rpc, self.aux.system.background.stop_signal.clone()) - .await?; + let response = self.syncer.handle_rpc(rpc).await?; Ok(TableRPC::SyncRPC(response)) } _ => Err(Error::BadRPC(format!("Unexpected table RPC"))), diff --git a/src/table/table_sync.rs b/src/table/table_sync.rs deleted file mode 100644 index 7394be1b..00000000 --- a/src/table/table_sync.rs +++ /dev/null @@ -1,898 +0,0 @@ -use rand::Rng; -use std::collections::{BTreeMap, VecDeque}; -use std::sync::{Arc, Mutex}; -use std::time::{Duration, Instant}; - -use futures::future::join_all; -use futures::{pin_mut, select}; -use futures_util::future::*; -use futures_util::stream::*; -use serde::{Deserialize, Serialize}; -use serde_bytes::ByteBuf; -use tokio::sync::{mpsc, watch}; - -use garage_rpc::ring::Ring; -use garage_util::data::*; -use garage_util::error::Error; - -use crate::*; -use crate::data::*; -use crate::replication::*; - -const MAX_DEPTH: usize = 16; - -const TABLE_SYNC_RPC_TIMEOUT: Duration = Duration::from_secs(30); - -// Do anti-entropy every 10 minutes -const SCAN_INTERVAL: Duration = Duration::from_secs(10 * 60); - -const CHECKSUM_CACHE_TIMEOUT: Duration = Duration::from_secs(10 * 60); - -pub struct TableSyncer { - data: Arc>, - aux: Arc>, - - todo: Mutex, - cache: Vec>>, -} - -#[derive(Serialize, Deserialize)] -pub(crate) enum SyncRPC { - GetRootChecksumRange(Hash, Hash), - RootChecksumRange(SyncRange), - Checksums(Vec), - Difference(Vec, Vec>), -} - -struct SyncTodo { - todo: Vec, -} - -#[derive(Debug, Clone)] -struct TodoPartition { - // Partition consists in hashes between begin included and end excluded - begin: Hash, - end: Hash, - - // Are we a node that stores this partition or not? - retain: bool, -} - -// A SyncRange defines a query on the dataset stored by a node, in the following way: -// - all items whose key are >= `begin` -// - stopping at the first item whose key hash has at least `level` leading zero bytes (excluded) -// - except if the first item of the range has such many leading zero bytes -// - and stopping at `end` (excluded) if such an item is not found -// The checksum itself does not store all of the items in the database, only the hashes of the "sub-ranges" -// i.e. of ranges of level `level-1` that cover the same range -// (ranges of level 0 do not exist and their hash is simply the hash of the first item >= begin) -// See RangeChecksum for the struct that stores this information. -#[derive(Hash, PartialEq, Eq, Debug, Clone, Serialize, Deserialize)] -pub(crate) struct SyncRange { - begin: Vec, - end: Vec, - level: usize, -} - -impl std::cmp::PartialOrd for SyncRange { - fn partial_cmp(&self, other: &Self) -> Option { - Some(self.cmp(other)) - } -} -impl std::cmp::Ord for SyncRange { - fn cmp(&self, other: &Self) -> std::cmp::Ordering { - self.begin - .cmp(&other.begin) - .then(self.level.cmp(&other.level)) - .then(self.end.cmp(&other.end)) - } -} - -#[derive(Debug, Clone, Serialize, Deserialize)] -pub(crate) struct RangeChecksum { - bounds: SyncRange, - children: Vec<(SyncRange, Hash)>, - found_limit: Option>, - - #[serde(skip, default = "std::time::Instant::now")] - time: Instant, -} - -#[derive(Debug, Clone)] -struct RangeChecksumCache { - hash: Option, // None if no children - found_limit: Option>, - time: Instant, -} - -impl TableSyncer -where - F: TableSchema + 'static, - R: TableReplication + 'static, -{ - pub(crate) fn launch(data: Arc>, - aux: Arc>) -> Arc { - let todo = SyncTodo{ todo: vec![] }; - - let syncer = Arc::new(Self { - data: data.clone(), - aux: aux.clone(), - todo: Mutex::new(todo), - cache: (0..MAX_DEPTH) - .map(|_| Mutex::new(BTreeMap::new())) - .collect::>(), - }); - - let (busy_tx, busy_rx) = mpsc::unbounded_channel(); - - let s1 = syncer.clone(); - aux.system.background.spawn_worker( - format!("table sync watcher for {}", data.name), - move |must_exit: watch::Receiver| s1.watcher_task(must_exit, busy_rx), - ); - - let s2 = syncer.clone(); - aux.system.background.spawn_worker( - format!("table syncer for {}", data.name), - move |must_exit: watch::Receiver| s2.syncer_task(must_exit, busy_tx), - ); - - let s3 = syncer.clone(); - tokio::spawn(async move { - tokio::time::delay_for(Duration::from_secs(20)).await; - s3.add_full_scan(); - }); - - syncer - } - - async fn watcher_task( - self: Arc, - mut must_exit: watch::Receiver, - mut busy_rx: mpsc::UnboundedReceiver, - ) -> Result<(), Error> { - let mut prev_ring: Arc = self.aux.system.ring.borrow().clone(); - let mut ring_recv: watch::Receiver> = self.aux.system.ring.clone(); - let mut nothing_to_do_since = Some(Instant::now()); - - while !*must_exit.borrow() { - let s_ring_recv = ring_recv.recv().fuse(); - let s_busy = busy_rx.recv().fuse(); - let s_must_exit = must_exit.recv().fuse(); - let s_timeout = tokio::time::delay_for(Duration::from_secs(1)).fuse(); - pin_mut!(s_ring_recv, s_busy, s_must_exit, s_timeout); - - select! { - new_ring_r = s_ring_recv => { - if let Some(new_ring) = new_ring_r { - debug!("({}) Adding ring difference to syncer todo list", self.data.name); - self.todo.lock().unwrap().add_ring_difference(&prev_ring, &new_ring, &self.data, &self.aux); - prev_ring = new_ring; - } - } - busy_opt = s_busy => { - if let Some(busy) = busy_opt { - if busy { - nothing_to_do_since = None; - } else { - if nothing_to_do_since.is_none() { - nothing_to_do_since = Some(Instant::now()); - } - } - } - } - must_exit_v = s_must_exit => { - if must_exit_v.unwrap_or(false) { - break; - } - } - _ = s_timeout => { - if nothing_to_do_since.map(|t| Instant::now() - t >= SCAN_INTERVAL).unwrap_or(false) { - nothing_to_do_since = None; - debug!("({}) Adding full scan to syncer todo list", self.data.name); - self.add_full_scan(); - } - } - } - } - Ok(()) - } - - pub fn add_full_scan(&self) { - self.todo.lock().unwrap().add_full_scan(&self.data, &self.aux); - } - - async fn syncer_task( - self: Arc, - mut must_exit: watch::Receiver, - busy_tx: mpsc::UnboundedSender, - ) -> Result<(), Error> { - while !*must_exit.borrow() { - let task = self.todo.lock().unwrap().pop_task(); - if let Some(partition) = task { - busy_tx.send(true)?; - let res = self - .clone() - .sync_partition(&partition, &mut must_exit) - .await; - if let Err(e) = res { - warn!( - "({}) Error while syncing {:?}: {}", - self.data.name, partition, e - ); - } - } else { - busy_tx.send(false)?; - tokio::time::delay_for(Duration::from_secs(1)).await; - } - } - Ok(()) - } - - async fn sync_partition( - self: Arc, - partition: &TodoPartition, - must_exit: &mut watch::Receiver, - ) -> Result<(), Error> { - if partition.retain { - let my_id = self.aux.system.id; - let nodes = self - .aux - .replication - .write_nodes(&partition.begin, &self.aux.system) - .into_iter() - .filter(|node| *node != my_id) - .collect::>(); - - debug!( - "({}) Preparing to sync {:?} with {:?}...", - self.data.name, partition, nodes - ); - let root_cks = self.root_checksum(&partition.begin, &partition.end, must_exit)?; - - let mut sync_futures = nodes - .iter() - .map(|node| { - self.clone().do_sync_with( - partition.clone(), - root_cks.clone(), - *node, - must_exit.clone(), - ) - }) - .collect::>(); - - let mut n_errors = 0; - while let Some(r) = sync_futures.next().await { - if let Err(e) = r { - n_errors += 1; - warn!("({}) Sync error: {}", self.data.name, e); - } - } - if n_errors > self.aux.replication.max_write_errors() { - return Err(Error::Message(format!( - "Sync failed with too many nodes (should have been: {:?}).", - nodes - ))); - } - } else { - self.offload_partition(&partition.begin, &partition.end, must_exit) - .await?; - } - - Ok(()) - } - - // Offload partition: this partition is not something we are storing, - // so send it out to all other nodes that store it and delete items locally. - // We don't bother checking if the remote nodes already have the items, - // we just batch-send everything. Offloading isn't supposed to happen very often. - // If any of the nodes that are supposed to store the items is unable to - // save them, we interrupt the process. - async fn offload_partition( - self: &Arc, - begin: &Hash, - end: &Hash, - must_exit: &mut watch::Receiver, - ) -> Result<(), Error> { - let mut counter: usize = 0; - - while !*must_exit.borrow() { - let mut items = Vec::new(); - - for item in self.data.store.range(begin.to_vec()..end.to_vec()) { - let (key, value) = item?; - items.push((key.to_vec(), Arc::new(ByteBuf::from(value.as_ref())))); - - if items.len() >= 1024 { - break; - } - } - - if items.len() > 0 { - let nodes = self - .aux - .replication - .write_nodes(&begin, &self.aux.system) - .into_iter() - .collect::>(); - if nodes.contains(&self.aux.system.id) { - warn!("Interrupting offload as partitions seem to have changed"); - break; - } - - counter += 1; - debug!( - "Offloading {} items from {:?}..{:?} ({})", - items.len(), - begin, - end, - counter - ); - self.offload_items(&items, &nodes[..]).await?; - } else { - break; - } - } - - Ok(()) - } - - async fn offload_items( - self: &Arc, - items: &Vec<(Vec, Arc)>, - nodes: &[UUID], - ) -> Result<(), Error> { - let values = items.iter().map(|(_k, v)| v.clone()).collect::>(); - let update_msg = Arc::new(TableRPC::::Update(values)); - - for res in join_all(nodes.iter().map(|to| { - self.aux - .rpc_client - .call_arc(*to, update_msg.clone(), TABLE_SYNC_RPC_TIMEOUT) - })) - .await - { - res?; - } - - // All remote nodes have written those items, now we can delete them locally - let mut not_removed = 0; - for (k, v) in items.iter() { - if !self.data.delete_if_equal(&k[..], &v[..])? { - not_removed += 1; - } - } - - if not_removed > 0 { - debug!("{} items not removed during offload because they changed in between (trying again...)", not_removed); - } - - Ok(()) - } - - fn root_checksum( - self: &Arc, - begin: &Hash, - end: &Hash, - must_exit: &mut watch::Receiver, - ) -> Result { - for i in 1..MAX_DEPTH { - let rc = self.range_checksum( - &SyncRange { - begin: begin.to_vec(), - end: end.to_vec(), - level: i, - }, - must_exit, - )?; - if rc.found_limit.is_none() { - return Ok(rc); - } - } - Err(Error::Message(format!( - "Unable to compute root checksum (this should never happen)" - ))) - } - - fn range_checksum( - self: &Arc, - range: &SyncRange, - must_exit: &mut watch::Receiver, - ) -> Result { - assert!(range.level != 0); - trace!("Call range_checksum {:?}", range); - - if range.level == 1 { - let mut children = vec![]; - for item in self - .data - .store - .range(range.begin.clone()..range.end.clone()) - { - let (key, value) = item?; - let key_hash = blake2sum(&key[..]); - if children.len() > 0 - && key_hash.as_slice()[0..range.level] - .iter() - .all(|x| *x == 0u8) - { - trace!( - "range_checksum {:?} returning {} items", - range, - children.len() - ); - return Ok(RangeChecksum { - bounds: range.clone(), - children, - found_limit: Some(key.to_vec()), - time: Instant::now(), - }); - } - let item_range = SyncRange { - begin: key.to_vec(), - end: vec![], - level: 0, - }; - children.push((item_range, blake2sum(&value[..]))); - } - trace!( - "range_checksum {:?} returning {} items", - range, - children.len() - ); - Ok(RangeChecksum { - bounds: range.clone(), - children, - found_limit: None, - time: Instant::now(), - }) - } else { - let mut children = vec![]; - let mut sub_range = SyncRange { - begin: range.begin.clone(), - end: range.end.clone(), - level: range.level - 1, - }; - let mut time = Instant::now(); - while !*must_exit.borrow() { - let sub_ck = self.range_checksum_cached_hash(&sub_range, must_exit)?; - - if let Some(hash) = sub_ck.hash { - children.push((sub_range.clone(), hash)); - if sub_ck.time < time { - time = sub_ck.time; - } - } - - if sub_ck.found_limit.is_none() || sub_ck.hash.is_none() { - trace!( - "range_checksum {:?} returning {} items", - range, - children.len() - ); - return Ok(RangeChecksum { - bounds: range.clone(), - children, - found_limit: None, - time, - }); - } - let found_limit = sub_ck.found_limit.unwrap(); - - let actual_limit_hash = blake2sum(&found_limit[..]); - if actual_limit_hash.as_slice()[0..range.level] - .iter() - .all(|x| *x == 0u8) - { - trace!( - "range_checksum {:?} returning {} items", - range, - children.len() - ); - return Ok(RangeChecksum { - bounds: range.clone(), - children, - found_limit: Some(found_limit.clone()), - time, - }); - } - - sub_range.begin = found_limit; - } - trace!("range_checksum {:?} exiting due to must_exit", range); - Err(Error::Message(format!("Exiting."))) - } - } - - fn range_checksum_cached_hash( - self: &Arc, - range: &SyncRange, - must_exit: &mut watch::Receiver, - ) -> Result { - { - let mut cache = self.cache[range.level].lock().unwrap(); - if let Some(v) = cache.get(&range) { - if Instant::now() - v.time < CHECKSUM_CACHE_TIMEOUT { - return Ok(v.clone()); - } - } - cache.remove(&range); - } - - let v = self.range_checksum(&range, must_exit)?; - trace!( - "({}) New checksum calculated for {}-{}/{}, {} children", - self.data.name, - hex::encode(&range.begin) - .chars() - .take(16) - .collect::(), - hex::encode(&range.end).chars().take(16).collect::(), - range.level, - v.children.len() - ); - - let hash = if v.children.len() > 0 { - Some(blake2sum(&rmp_to_vec_all_named(&v)?[..])) - } else { - None - }; - let cache_entry = RangeChecksumCache { - hash, - found_limit: v.found_limit, - time: v.time, - }; - - let mut cache = self.cache[range.level].lock().unwrap(); - cache.insert(range.clone(), cache_entry.clone()); - Ok(cache_entry) - } - - async fn do_sync_with( - self: Arc, - partition: TodoPartition, - root_ck: RangeChecksum, - who: UUID, - mut must_exit: watch::Receiver, - ) -> Result<(), Error> { - let mut todo = VecDeque::new(); - - // If their root checksum has level > than us, use that as a reference - let root_cks_resp = self - .aux - .rpc_client - .call( - who, - TableRPC::::SyncRPC(SyncRPC::GetRootChecksumRange( - partition.begin.clone(), - partition.end.clone(), - )), - TABLE_SYNC_RPC_TIMEOUT, - ) - .await?; - if let TableRPC::::SyncRPC(SyncRPC::RootChecksumRange(range)) = root_cks_resp { - if range.level > root_ck.bounds.level { - let their_root_range_ck = self.range_checksum(&range, &mut must_exit)?; - todo.push_back(their_root_range_ck); - } else { - todo.push_back(root_ck); - } - } else { - return Err(Error::Message(format!( - "Invalid respone to GetRootChecksumRange RPC: {}", - debug_serialize(root_cks_resp) - ))); - } - - while !todo.is_empty() && !*must_exit.borrow() { - let total_children = todo.iter().map(|x| x.children.len()).fold(0, |x, y| x + y); - trace!( - "({}) Sync with {:?}: {} ({}) remaining", - self.data.name, - who, - todo.len(), - total_children - ); - - let step_size = std::cmp::min(16, todo.len()); - let step = todo.drain(..step_size).collect::>(); - - let rpc_resp = self - .aux - .rpc_client - .call( - who, - TableRPC::::SyncRPC(SyncRPC::Checksums(step)), - TABLE_SYNC_RPC_TIMEOUT, - ) - .await?; - if let TableRPC::::SyncRPC(SyncRPC::Difference(mut diff_ranges, diff_items)) = - rpc_resp - { - if diff_ranges.len() > 0 || diff_items.len() > 0 { - info!( - "({}) Sync with {:?}: difference {} ranges, {} items", - self.data.name, - who, - diff_ranges.len(), - diff_items.len() - ); - } - let mut items_to_send = vec![]; - for differing in diff_ranges.drain(..) { - if differing.level == 0 { - items_to_send.push(differing.begin); - } else { - let checksum = self.range_checksum(&differing, &mut must_exit)?; - todo.push_back(checksum); - } - } - if diff_items.len() > 0 { - self.data.update_many(&diff_items[..])?; - } - if items_to_send.len() > 0 { - self.send_items(who, items_to_send).await?; - } - } else { - return Err(Error::Message(format!( - "Unexpected response to sync RPC checksums: {}", - debug_serialize(&rpc_resp) - ))); - } - } - Ok(()) - } - - async fn send_items(&self, who: UUID, item_list: Vec>) -> Result<(), Error> { - info!( - "({}) Sending {} items to {:?}", - self.data.name, - item_list.len(), - who - ); - - let mut values = vec![]; - for item in item_list.iter() { - if let Some(v) = self.data.store.get(&item[..])? { - values.push(Arc::new(ByteBuf::from(v.as_ref()))); - } - } - let rpc_resp = self - .aux - .rpc_client - .call(who, TableRPC::::Update(values), TABLE_SYNC_RPC_TIMEOUT) - .await?; - if let TableRPC::::Ok = rpc_resp { - Ok(()) - } else { - Err(Error::Message(format!( - "Unexpected response to RPC Update: {}", - debug_serialize(&rpc_resp) - ))) - } - } - - pub(crate) async fn handle_rpc( - self: &Arc, - message: &SyncRPC, - mut must_exit: watch::Receiver, - ) -> Result { - match message { - SyncRPC::GetRootChecksumRange(begin, end) => { - let root_cks = self.root_checksum(&begin, &end, &mut must_exit)?; - Ok(SyncRPC::RootChecksumRange(root_cks.bounds)) - } - SyncRPC::Checksums(checksums) => { - self.handle_checksums_rpc(&checksums[..], &mut must_exit) - .await - } - _ => Err(Error::Message(format!("Unexpected sync RPC"))), - } - } - - async fn handle_checksums_rpc( - self: &Arc, - checksums: &[RangeChecksum], - must_exit: &mut watch::Receiver, - ) -> Result { - let mut ret_ranges = vec![]; - let mut ret_items = vec![]; - - for their_ckr in checksums.iter() { - let our_ckr = self.range_checksum(&their_ckr.bounds, must_exit)?; - for (their_range, their_hash) in their_ckr.children.iter() { - let differs = match our_ckr - .children - .binary_search_by(|(our_range, _)| our_range.cmp(&their_range)) - { - Err(_) => { - if their_range.level >= 1 { - let cached_hash = - self.range_checksum_cached_hash(&their_range, must_exit)?; - cached_hash.hash.map(|h| h != *their_hash).unwrap_or(true) - } else { - true - } - } - Ok(i) => our_ckr.children[i].1 != *their_hash, - }; - if differs { - ret_ranges.push(their_range.clone()); - if their_range.level == 0 { - if let Some(item_bytes) = - self.data.store.get(their_range.begin.as_slice())? - { - ret_items.push(Arc::new(ByteBuf::from(item_bytes.to_vec()))); - } - } - } - } - for (our_range, _hash) in our_ckr.children.iter() { - if let Some(their_found_limit) = &their_ckr.found_limit { - if our_range.begin.as_slice() > their_found_limit.as_slice() { - break; - } - } - - let not_present = our_ckr - .children - .binary_search_by(|(their_range, _)| their_range.cmp(&our_range)) - .is_err(); - if not_present { - if our_range.level > 0 { - ret_ranges.push(our_range.clone()); - } - if our_range.level == 0 { - if let Some(item_bytes) = - self.data.store.get(our_range.begin.as_slice())? - { - ret_items.push(Arc::new(ByteBuf::from(item_bytes.to_vec()))); - } - } - } - } - } - let n_checksums = checksums - .iter() - .map(|x| x.children.len()) - .fold(0, |x, y| x + y); - if ret_ranges.len() > 0 || ret_items.len() > 0 { - trace!( - "({}) Checksum comparison RPC: {} different + {} items for {} received", - self.data.name, - ret_ranges.len(), - ret_items.len(), - n_checksums - ); - } - Ok(SyncRPC::Difference(ret_ranges, ret_items)) - } - - pub(crate) fn invalidate(self: &Arc, item_key: &[u8]) { - for i in 1..MAX_DEPTH { - let needle = SyncRange { - begin: item_key.to_vec(), - end: vec![], - level: i, - }; - let mut cache = self.cache[i].lock().unwrap(); - if let Some(cache_entry) = cache.range(..=needle).rev().next() { - if cache_entry.0.begin[..] <= *item_key && cache_entry.0.end[..] > *item_key { - let index = cache_entry.0.clone(); - drop(cache_entry); - cache.remove(&index); - } - } - } - } -} - -impl SyncTodo { - fn add_full_scan(&mut self, data: &TableData, aux: &TableAux) { - let my_id = aux.system.id; - - self.todo.clear(); - - let ring = aux.system.ring.borrow().clone(); - let split_points = aux.replication.split_points(&ring); - - for i in 0..split_points.len() - 1 { - let begin = split_points[i]; - let end = split_points[i + 1]; - if begin == end { - continue; - } - - let nodes = aux.replication.replication_nodes(&begin, &ring); - - let retain = nodes.contains(&my_id); - if !retain { - // Check if we have some data to send, otherwise skip - if data.store.range(begin..end).next().is_none() { - continue; - } - } - - self.todo.push(TodoPartition { begin, end, retain }); - } - } - - fn add_ring_difference( - &mut self, - old_ring: &Ring, - new_ring: &Ring, - data: &TableData, aux: &TableAux, - ) { - let my_id = aux.system.id; - - // If it is us who are entering or leaving the system, - // initiate a full sync instead of incremental sync - if old_ring.config.members.contains_key(&my_id) - != new_ring.config.members.contains_key(&my_id) - { - self.add_full_scan(data, aux); - return; - } - - let mut all_points = None - .into_iter() - .chain(aux.replication.split_points(old_ring).drain(..)) - .chain(aux.replication.split_points(new_ring).drain(..)) - .chain(self.todo.iter().map(|x| x.begin)) - .chain(self.todo.iter().map(|x| x.end)) - .collect::>(); - all_points.sort(); - all_points.dedup(); - - let mut old_todo = std::mem::replace(&mut self.todo, vec![]); - old_todo.sort_by(|x, y| x.begin.cmp(&y.begin)); - let mut new_todo = vec![]; - - for i in 0..all_points.len() - 1 { - let begin = all_points[i]; - let end = all_points[i + 1]; - let was_ours = aux - .replication - .replication_nodes(&begin, &old_ring) - .contains(&my_id); - let is_ours = aux - .replication - .replication_nodes(&begin, &new_ring) - .contains(&my_id); - - let was_todo = match old_todo.binary_search_by(|x| x.begin.cmp(&begin)) { - Ok(_) => true, - Err(j) => { - (j > 0 && old_todo[j - 1].begin < end && begin < old_todo[j - 1].end) - || (j < old_todo.len() - && old_todo[j].begin < end && begin < old_todo[j].end) - } - }; - if was_todo || (is_ours && !was_ours) || (was_ours && !is_ours) { - new_todo.push(TodoPartition { - begin, - end, - retain: is_ours, - }); - } - } - - self.todo = new_todo; - } - - fn pop_task(&mut self) -> Option { - if self.todo.is_empty() { - return None; - } - - let i = rand::thread_rng().gen_range::(0, self.todo.len()); - if i == self.todo.len() - 1 { - self.todo.pop() - } else { - let replacement = self.todo.pop().unwrap(); - let ret = std::mem::replace(&mut self.todo[i], replacement); - Some(ret) - } - } -} -- cgit v1.2.3 From db7a9d4948d41e4b641f9c50f6ff8921a436431d Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 11 Mar 2021 18:45:26 +0100 Subject: Tiny changes --- src/table/sync.rs | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/table/sync.rs b/src/table/sync.rs index 9c37c286..42321ac6 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -49,7 +49,6 @@ pub(crate) enum SyncRPC { CkNoDifference, GetNode(MerkleNodeKey), Node(MerkleNodeKey, MerkleNode), - Items(Vec>), } struct SyncTodo { @@ -119,7 +118,7 @@ where select! { new_ring_r = s_ring_recv => { if new_ring_r.is_some() { - debug!("({}) Adding ring difference to syncer todo list", self.data.name); + debug!("({}) Ring changed, adding full sync to syncer todo list", self.data.name); self.add_full_sync(); } } @@ -142,7 +141,7 @@ where _ = s_timeout => { if nothing_to_do_since.map(|t| Instant::now() - t >= ANTI_ENTROPY_INTERVAL).unwrap_or(false) { nothing_to_do_since = None; - debug!("({}) Adding full sync to syncer todo list", self.data.name); + debug!("({}) Interval passed, adding full sync to syncer todo list", self.data.name); self.add_full_sync(); } } @@ -330,6 +329,10 @@ where } // ======= SYNCHRONIZATION PROCEDURE -- DRIVER SIDE ====== + // The driver side is only concerned with sending out the item it has + // and the other side might not have. Receiving items that differ from one + // side to the other will happen when the other side syncs with us, + // which they also do regularly. fn get_root_ck(&self, range: PartitionRange) -> Result { let begin = u16::from_be_bytes(range.begin); -- cgit v1.2.3 From fae5104a2cf91206f995b183c5f217ea6729a551 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 11 Mar 2021 18:50:32 +0100 Subject: Add a nice warning --- src/table/sync.rs | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/table/sync.rs b/src/table/sync.rs index 42321ac6..68fc9fcb 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -433,9 +433,12 @@ where // We don't request those items from them, they will send them. // We only bother with pushing items that differ } - MerkleNode::Leaf(ik, _) => { + MerkleNode::Leaf(ik, ivhash) => { // Just send that item directly - if let Some(val) = self.data.store.get(ik)? { + if let Some(val) = self.data.store.get(&ik[..])? { + if blake2sum(&val[..]) != ivhash { + warn!("Hashes differ between stored value and Merkle tree, key: {:?} (if your server is very busy, don't worry, this happens when the Merkle tree can't be updated fast enough)", ik); + } todo_items.push(val.to_vec()); } } -- cgit v1.2.3 From f7c2cd1cd7ee15b9c97b9fbdef25c0644b3523bb Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 11 Mar 2021 18:55:17 +0100 Subject: Add comment, and also whoops, this wasn't doing what we expected --- src/table/sync.rs | 22 ++++++++++++++-------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/src/table/sync.rs b/src/table/sync.rs index 68fc9fcb..b5044a4e 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -440,9 +440,13 @@ where warn!("Hashes differ between stored value and Merkle tree, key: {:?} (if your server is very busy, don't worry, this happens when the Merkle tree can't be updated fast enough)", ik); } todo_items.push(val.to_vec()); + } else { + warn!("Item from Merkle tree not found in store: {:?} (if your server is very busy, don't worry, this happens when the Merkle tree can't be updated fast enough)", ik); } } MerkleNode::Intermediate(l) => { + // Get Merkle node for this tree position at remote node + // and compare it with local node let remote_node = match self .aux .rpc_client @@ -462,7 +466,11 @@ where } }; let int_l2 = match remote_node { + // If they have an intermediate node at this tree position, + // we can compare them to find differences MerkleNode::Intermediate(l2) => l2, + // Otherwise, treat it as if they have nothing for this subtree, + // which will have the consequence of sending them everything _ => vec![], }; @@ -493,20 +501,18 @@ where Ok(()) } - async fn send_items(&self, who: UUID, item_list: Vec>) -> Result<(), Error> { + async fn send_items(&self, who: UUID, item_value_list: Vec>) -> Result<(), Error> { info!( "({}) Sending {} items to {:?}", self.data.name, - item_list.len(), + item_value_list.len(), who ); - let mut values = vec![]; - for item in item_list.iter() { - if let Some(v) = self.data.store.get(&item[..])? { - values.push(Arc::new(ByteBuf::from(v.as_ref()))); - } - } + let values = item_value_list.into_iter() + .map(|x| Arc::new(ByteBuf::from(x))) + .collect::>(); + let rpc_resp = self .aux .rpc_client -- cgit v1.2.3 From 3f7a496355bdbeeeee859912fa6fa7a95cb47f3b Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 11 Mar 2021 19:06:27 +0100 Subject: More security: don't delete stuff too easily --- src/model/block.rs | 3 +++ src/table/replication/fullcopy.rs | 7 ++++++- src/table/sync.rs | 3 +++ 3 files changed, 12 insertions(+), 1 deletion(-) diff --git a/src/model/block.rs b/src/model/block.rs index 987ec9e4..a3bbe43d 100644 --- a/src/model/block.rs +++ b/src/model/block.rs @@ -305,6 +305,9 @@ impl BlockManager { let ring = self.system.ring.borrow().clone(); let mut who = self.replication.replication_nodes(&hash, &ring); + if who.len() < self.replication.write_quorum(&self.system) { + return Err(Error::Message(format!("Not trying to offload block because we don't have a quorum of nodes to write to"))); + } who.retain(|id| *id != self.system.id); let msg = Arc::new(Message::NeedBlockQuery(*hash)); diff --git a/src/table/replication/fullcopy.rs b/src/table/replication/fullcopy.rs index a20f20b7..a5faece9 100644 --- a/src/table/replication/fullcopy.rs +++ b/src/table/replication/fullcopy.rs @@ -41,7 +41,12 @@ impl TableReplication for TableFullReplication { self.replication_nodes(hash, system.ring.borrow().as_ref()) } fn write_quorum(&self, system: &System) -> usize { - system.ring.borrow().config.members.len() - self.max_faults + let nmembers = system.ring.borrow().config.members.len(); + if nmembers > self.max_faults { + nmembers - self.max_faults + } else { + 1 + } } fn max_write_errors(&self) -> usize { self.max_faults diff --git a/src/table/sync.rs b/src/table/sync.rs index b5044a4e..f8ebb2f0 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -277,6 +277,9 @@ where warn!("Interrupting offload as partitions seem to have changed"); break; } + if nodes.len() < self.aux.replication.write_quorum(&self.aux.system) { + return Err(Error::Message(format!("Not offloading as we don't have a quorum of nodes to write to."))); + } counter += 1; debug!( -- cgit v1.2.3 From 1ec49980ec876ef9395a9ae088f82d86a1a0d9f6 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 11 Mar 2021 19:30:24 +0100 Subject: whoops --- src/table/merkle.rs | 3 ++- src/table/sync.rs | 10 +++++++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/table/merkle.rs b/src/table/merkle.rs index 92c18e09..467ce615 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -50,7 +50,7 @@ pub(crate) struct MerkleUpdater { empty_node_hash: Hash, } -#[derive(Clone, Serialize, Deserialize)] +#[derive(Clone, Debug, Serialize, Deserialize)] pub struct MerkleNodeKey { // partition: first 16 bits (two bytes) of the partition_key's hash pub partition: MerklePartition, @@ -283,6 +283,7 @@ impl MerkleUpdater { k: &MerkleNodeKey, v: &MerkleNode, ) -> ConflictableTransactionResult { + trace!("Put Merkle node: {:?} => {:?}", k, v); if *v == MerkleNode::Empty { tx.remove(k.encode())?; Ok(self.empty_node_hash) diff --git a/src/table/sync.rs b/src/table/sync.rs index f8ebb2f0..07d48155 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -370,6 +370,14 @@ where must_exit: watch::Receiver, ) -> Result<(), Error> { let root_ck = self.get_root_ck(partition.range)?; + if root_ck.is_empty() { + debug!( + "({}) Sync {:?} with {:?}: partition is empty.", + self.data.name, partition, who + ); + return Ok(()) + } + let root_ck_hash = hash_of(&root_ck)?; // If their root checksum has level > than us, use that as a reference @@ -637,7 +645,7 @@ fn join_ordered<'a, K: Ord + Eq, V1, V2>( ret.push((&x[i].0, Some(&x[i].1), None)); i += 1; } else if j < y.len() && (i == x.len() || x[i].0 > y[j].0) { - ret.push((&x[i].0, None, Some(&y[j].1))); + ret.push((&y[j].0, None, Some(&y[j].1))); j += 1; } else { unreachable!(); -- cgit v1.2.3 From 7fdaf7aef0c2aa8b38dbc7dac630f6f9baf8f0a4 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 12 Mar 2021 14:37:46 +0100 Subject: Fix merkle updater not being notified; improved logging --- src/api/s3_put.rs | 4 ++-- src/table/data.rs | 4 ++-- src/table/merkle.rs | 13 +++++++------ src/table/sync.rs | 13 +++++++------ 4 files changed, 18 insertions(+), 16 deletions(-) diff --git a/src/api/s3_put.rs b/src/api/s3_put.rs index 2c5e364f..17732ced 100644 --- a/src/api/s3_put.rs +++ b/src/api/s3_put.rs @@ -472,8 +472,8 @@ pub async fn handle_complete_multipart_upload( }; // Check that the list of parts they gave us corresponds to the parts we have here - println!("Expected parts from request: {:?}", body_list_of_parts); - println!("Parts stored in version: {:?}", version.parts_etags.items()); + debug!("Expected parts from request: {:?}", body_list_of_parts); + debug!("Parts stored in version: {:?}", version.parts_etags.items()); let parts = version .parts_etags .items() diff --git a/src/table/data.rs b/src/table/data.rs index 6217bf6d..2817a849 100644 --- a/src/table/data.rs +++ b/src/table/data.rs @@ -142,7 +142,7 @@ where if let Some((old_entry, new_entry)) = changed { self.instance.updated(old_entry, Some(new_entry)); - //self.syncer.load_full().unwrap().invalidate(&tree_key[..]); + self.merkle_updater.todo_notify.notify(); } Ok(()) @@ -163,7 +163,7 @@ where if removed { let old_entry = self.decode_entry(v)?; self.instance.updated(Some(old_entry), None); - //self.syncer.load_full().unwrap().invalidate(k); + self.merkle_updater.todo_notify.notify(); } Ok(removed) } diff --git a/src/table/merkle.rs b/src/table/merkle.rs index 467ce615..a694c9e9 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -4,7 +4,7 @@ use std::time::Duration; use futures::select; use futures_util::future::*; -use log::{info, warn}; +use log::{debug, warn}; use serde::{Deserialize, Serialize}; use sled::transaction::{ ConflictableTransactionError, ConflictableTransactionResult, TransactionalTree, @@ -109,11 +109,11 @@ impl MerkleUpdater { match x { Ok((key, valhash)) => { if let Err(e) = self.update_item(&key[..], &valhash[..]) { - warn!("Error while updating Merkle tree item: {}", e); + warn!("({}) Error while updating Merkle tree item: {}", self.table_name, e); } } Err(e) => { - warn!("Error while iterating on Merkle todo tree: {}", e); + warn!("({}) Error while iterating on Merkle todo tree: {}", self.table_name, e); tokio::time::delay_for(Duration::from_secs(10)).await; } } @@ -152,8 +152,9 @@ impl MerkleUpdater { .is_ok(); if !deleted { - info!( - "Item not deleted from Merkle todo because it changed: {:?}", + debug!( + "({}) Item not deleted from Merkle todo because it changed: {:?}", + self.table_name, k ); } @@ -195,7 +196,7 @@ impl MerkleUpdater { if children.len() == 0 { // should not happen - warn!("Replacing intermediate node with empty node, should not happen."); + warn!("({}) Replacing intermediate node with empty node, should not happen.", self.table_name); Some(MerkleNode::Empty) } else if children.len() == 1 { // We now have a single node (case when the update deleted one of only two diff --git a/src/table/sync.rs b/src/table/sync.rs index 07d48155..dbfa0a9f 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -274,7 +274,7 @@ where .into_iter() .collect::>(); if nodes.contains(&self.aux.system.id) { - warn!("Interrupting offload as partitions seem to have changed"); + warn!("({}) Interrupting offload as partitions seem to have changed", self.data.name); break; } if nodes.len() < self.aux.replication.write_quorum(&self.aux.system) { @@ -282,8 +282,9 @@ where } counter += 1; - debug!( - "Offloading {} items from {:?}..{:?} ({})", + info!( + "({}) Offloading {} items from {:?}..{:?} ({})", + self.data.name, items.len(), begin, end, @@ -325,7 +326,7 @@ where } if not_removed > 0 { - debug!("{} items not removed during offload because they changed in between (trying again...)", not_removed); + debug!("({}) {} items not removed during offload because they changed in between (trying again...)", self.data.name, not_removed); } Ok(()) @@ -448,11 +449,11 @@ where // Just send that item directly if let Some(val) = self.data.store.get(&ik[..])? { if blake2sum(&val[..]) != ivhash { - warn!("Hashes differ between stored value and Merkle tree, key: {:?} (if your server is very busy, don't worry, this happens when the Merkle tree can't be updated fast enough)", ik); + warn!("({}) Hashes differ between stored value and Merkle tree, key: {:?} (if your server is very busy, don't worry, this happens when the Merkle tree can't be updated fast enough)", self.data.name, ik); } todo_items.push(val.to_vec()); } else { - warn!("Item from Merkle tree not found in store: {:?} (if your server is very busy, don't worry, this happens when the Merkle tree can't be updated fast enough)", ik); + warn!("({}) Item from Merkle tree not found in store: {:?} (if your server is very busy, don't worry, this happens when the Merkle tree can't be updated fast enough)", self.data.name, ik); } } MerkleNode::Intermediate(l) => { -- cgit v1.2.3 From 1fea257291bdbf447f9918274ebf73848afb3a0c Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 12 Mar 2021 14:51:17 +0100 Subject: Don't sync at beginning --- src/table/sync.rs | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/table/sync.rs b/src/table/sync.rs index dbfa0a9f..049a16ae 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -105,6 +105,7 @@ where mut must_exit: watch::Receiver, mut busy_rx: mpsc::UnboundedReceiver, ) -> Result<(), Error> { + let mut prev_ring: Arc = self.aux.system.ring.borrow().clone(); let mut ring_recv: watch::Receiver> = self.aux.system.ring.clone(); let mut nothing_to_do_since = Some(Instant::now()); @@ -117,9 +118,12 @@ where select! { new_ring_r = s_ring_recv => { - if new_ring_r.is_some() { - debug!("({}) Ring changed, adding full sync to syncer todo list", self.data.name); - self.add_full_sync(); + if let Some(new_ring) = new_ring_r { + if !Arc::ptr_eq(&new_ring, &prev_ring) { + debug!("({}) Ring changed, adding full sync to syncer todo list", self.data.name); + self.add_full_sync(); + prev_ring = new_ring; + } } } busy_opt = s_busy => { -- cgit v1.2.3 From 8860aa19b867183b83ee48efd9990cd34e567f53 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 12 Mar 2021 15:05:26 +0100 Subject: Make syncer have its own rpc client/server --- src/table/merkle.rs | 18 ++++++++---- src/table/sync.rs | 81 ++++++++++++++++++++++++++++++++++++++--------------- src/table/table.rs | 4 +-- 3 files changed, 73 insertions(+), 30 deletions(-) diff --git a/src/table/merkle.rs b/src/table/merkle.rs index a694c9e9..a164df04 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -109,11 +109,17 @@ impl MerkleUpdater { match x { Ok((key, valhash)) => { if let Err(e) = self.update_item(&key[..], &valhash[..]) { - warn!("({}) Error while updating Merkle tree item: {}", self.table_name, e); + warn!( + "({}) Error while updating Merkle tree item: {}", + self.table_name, e + ); } } Err(e) => { - warn!("({}) Error while iterating on Merkle todo tree: {}", self.table_name, e); + warn!( + "({}) Error while iterating on Merkle todo tree: {}", + self.table_name, e + ); tokio::time::delay_for(Duration::from_secs(10)).await; } } @@ -154,8 +160,7 @@ impl MerkleUpdater { if !deleted { debug!( "({}) Item not deleted from Merkle todo because it changed: {:?}", - self.table_name, - k + self.table_name, k ); } Ok(()) @@ -196,7 +201,10 @@ impl MerkleUpdater { if children.len() == 0 { // should not happen - warn!("({}) Replacing intermediate node with empty node, should not happen.", self.table_name); + warn!( + "({}) Replacing intermediate node with empty node, should not happen.", + self.table_name + ); Some(MerkleNode::Empty) } else if children.len() == 1 { // We now have a single node (case when the update deleted one of only two diff --git a/src/table/sync.rs b/src/table/sync.rs index 049a16ae..23161d15 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -12,10 +12,13 @@ use serde::{Deserialize, Serialize}; use serde_bytes::ByteBuf; use tokio::sync::{mpsc, watch}; -use garage_rpc::ring::Ring; use garage_util::data::*; use garage_util::error::Error; +use garage_rpc::ring::Ring; +use garage_rpc::rpc_client::*; +use garage_rpc::rpc_server::*; + use crate::data::*; use crate::merkle::*; use crate::replication::*; @@ -31,6 +34,7 @@ pub struct TableSyncer { aux: Arc>, todo: Mutex, + rpc_client: Arc>, } type RootCk = Vec<(MerklePartition, Hash)>; @@ -49,8 +53,12 @@ pub(crate) enum SyncRPC { CkNoDifference, GetNode(MerkleNodeKey), Node(MerkleNodeKey, MerkleNode), + Items(Vec>), + Ok, } +impl RpcMessage for SyncRPC {} + struct SyncTodo { todo: Vec, } @@ -68,15 +76,25 @@ where F: TableSchema + 'static, R: TableReplication + 'static, { - pub(crate) fn launch(data: Arc>, aux: Arc>) -> Arc { + pub(crate) fn launch( + data: Arc>, + aux: Arc>, + rpc_server: &mut RpcServer, + ) -> Arc { + let rpc_path = format!("table_{}/sync", data.name); + let rpc_client = aux.system.rpc_client::(&rpc_path); + let todo = SyncTodo { todo: vec![] }; let syncer = Arc::new(Self { data: data.clone(), aux: aux.clone(), todo: Mutex::new(todo), + rpc_client, }); + syncer.register_handler(rpc_server, rpc_path); + let (busy_tx, busy_rx) = mpsc::unbounded_channel(); let s1 = syncer.clone(); @@ -100,6 +118,21 @@ where syncer } + fn register_handler(self: &Arc, rpc_server: &mut RpcServer, path: String) { + let self2 = self.clone(); + rpc_server.add_handler::(path, move |msg, _addr| { + let self2 = self2.clone(); + async move { self2.handle_rpc(&msg).await } + }); + + let self2 = self.clone(); + self.rpc_client + .set_local_handler(self.aux.system.id, move |msg| { + let self2 = self2.clone(); + async move { self2.handle_rpc(&msg).await } + }); + } + async fn watcher_task( self: Arc, mut must_exit: watch::Receiver, @@ -278,11 +311,16 @@ where .into_iter() .collect::>(); if nodes.contains(&self.aux.system.id) { - warn!("({}) Interrupting offload as partitions seem to have changed", self.data.name); + warn!( + "({}) Interrupting offload as partitions seem to have changed", + self.data.name + ); break; } if nodes.len() < self.aux.replication.write_quorum(&self.aux.system) { - return Err(Error::Message(format!("Not offloading as we don't have a quorum of nodes to write to."))); + return Err(Error::Message(format!( + "Not offloading as we don't have a quorum of nodes to write to." + ))); } counter += 1; @@ -309,11 +347,10 @@ where nodes: &[UUID], ) -> Result<(), Error> { let values = items.iter().map(|(_k, v)| v.clone()).collect::>(); - let update_msg = Arc::new(TableRPC::::Update(values)); + let update_msg = Arc::new(SyncRPC::Items(values)); for res in join_all(nodes.iter().map(|to| { - self.aux - .rpc_client + self.rpc_client .call_arc(*to, update_msg.clone(), TABLE_SYNC_RPC_TIMEOUT) })) .await @@ -380,31 +417,30 @@ where "({}) Sync {:?} with {:?}: partition is empty.", self.data.name, partition, who ); - return Ok(()) + return Ok(()); } let root_ck_hash = hash_of(&root_ck)?; // If their root checksum has level > than us, use that as a reference let root_resp = self - .aux .rpc_client .call( who, - TableRPC::::SyncRPC(SyncRPC::RootCkHash(partition.range, root_ck_hash)), + SyncRPC::RootCkHash(partition.range, root_ck_hash), TABLE_SYNC_RPC_TIMEOUT, ) .await?; let mut todo = match root_resp { - TableRPC::::SyncRPC(SyncRPC::CkNoDifference) => { + SyncRPC::CkNoDifference => { debug!( "({}) Sync {:?} with {:?}: no difference", self.data.name, partition, who ); return Ok(()); } - TableRPC::::SyncRPC(SyncRPC::RootCkList(_, their_root_ck)) => { + SyncRPC::RootCkList(_, their_root_ck) => { let join = join_ordered(&root_ck[..], &their_root_ck[..]); let mut todo = VecDeque::new(); for (p, v1, v2) in join.iter() { @@ -464,16 +500,11 @@ where // Get Merkle node for this tree position at remote node // and compare it with local node let remote_node = match self - .aux .rpc_client - .call( - who, - TableRPC::::SyncRPC(SyncRPC::GetNode(key.clone())), - TABLE_SYNC_RPC_TIMEOUT, - ) + .call(who, SyncRPC::GetNode(key.clone()), TABLE_SYNC_RPC_TIMEOUT) .await? { - TableRPC::::SyncRPC(SyncRPC::Node(_, node)) => node, + SyncRPC::Node(_, node) => node, x => { return Err(Error::Message(format!( "Invalid respone to GetNode RPC: {}", @@ -525,16 +556,16 @@ where who ); - let values = item_value_list.into_iter() + let values = item_value_list + .into_iter() .map(|x| Arc::new(ByteBuf::from(x))) .collect::>(); let rpc_resp = self - .aux .rpc_client - .call(who, TableRPC::::Update(values), TABLE_SYNC_RPC_TIMEOUT) + .call(who, SyncRPC::Items(values), TABLE_SYNC_RPC_TIMEOUT) .await?; - if let TableRPC::::Ok = rpc_resp { + if let SyncRPC::Ok = rpc_resp { Ok(()) } else { Err(Error::Message(format!( @@ -561,6 +592,10 @@ where let node = self.data.merkle_updater.read_node(&k)?; Ok(SyncRPC::Node(k.clone(), node)) } + SyncRPC::Items(items) => { + self.data.update_many(items)?; + Ok(SyncRPC::Ok) + } _ => Err(Error::Message(format!("Unexpected sync RPC"))), } } diff --git a/src/table/table.rs b/src/table/table.rs index 516c9358..edb1be3f 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -24,7 +24,7 @@ const TABLE_RPC_TIMEOUT: Duration = Duration::from_secs(10); pub struct TableAux { pub system: Arc, pub replication: R, - pub(crate) rpc_client: Arc>>, + rpc_client: Arc>>, } pub struct Table { @@ -76,7 +76,7 @@ where rpc_client, }); - let syncer = TableSyncer::launch(data.clone(), aux.clone()); + let syncer = TableSyncer::launch(data.clone(), aux.clone(), rpc_server); let table = Arc::new(Self { data, aux, syncer }); -- cgit v1.2.3 From cbe7e1a66a9dceaaeae0467b4eefe51afd5b297c Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 12 Mar 2021 15:07:23 +0100 Subject: Move table rpc client out of tableaux --- src/table/sync.rs | 6 +++--- src/table/table.rs | 31 +++++++++++++------------------ 2 files changed, 16 insertions(+), 21 deletions(-) diff --git a/src/table/sync.rs b/src/table/sync.rs index 23161d15..4be8cd10 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -31,7 +31,7 @@ const ANTI_ENTROPY_INTERVAL: Duration = Duration::from_secs(10 * 60); pub struct TableSyncer { data: Arc>, - aux: Arc>, + aux: Arc>, todo: Mutex, rpc_client: Arc>, @@ -78,7 +78,7 @@ where { pub(crate) fn launch( data: Arc>, - aux: Arc>, + aux: Arc>, rpc_server: &mut RpcServer, ) -> Arc { let rpc_path = format!("table_{}/sync", data.name); @@ -605,7 +605,7 @@ impl SyncTodo { fn add_full_sync( &mut self, data: &TableData, - aux: &TableAux, + aux: &TableAux, ) { let my_id = aux.system.id; diff --git a/src/table/table.rs b/src/table/table.rs index edb1be3f..dd3394bd 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -21,16 +21,16 @@ use crate::sync::*; const TABLE_RPC_TIMEOUT: Duration = Duration::from_secs(10); -pub struct TableAux { +pub struct TableAux { pub system: Arc, pub replication: R, - rpc_client: Arc>>, } pub struct Table { pub data: Arc>, - pub aux: Arc>, + pub aux: Arc>, pub syncer: Arc>, + rpc_client: Arc>>, } #[derive(Serialize, Deserialize)] @@ -73,12 +73,16 @@ where let aux = Arc::new(TableAux { system, replication, - rpc_client, }); let syncer = TableSyncer::launch(data.clone(), aux.clone(), rpc_server); - let table = Arc::new(Self { data, aux, syncer }); + let table = Arc::new(Self { + data, + aux, + syncer, + rpc_client, + }); table.clone().register_handler(rpc_server, rpc_path); @@ -93,8 +97,7 @@ where let e_enc = Arc::new(ByteBuf::from(rmp_to_vec_all_named(e)?)); let rpc = TableRPC::::Update(vec![e_enc]); - self.aux - .rpc_client + self.rpc_client .try_call_many( &who[..], rpc, @@ -123,11 +126,7 @@ where let call_futures = call_list.drain().map(|(node, entries)| async move { let rpc = TableRPC::::Update(entries); - let resp = self - .aux - .rpc_client - .call(node, rpc, TABLE_RPC_TIMEOUT) - .await?; + let resp = self.rpc_client.call(node, rpc, TABLE_RPC_TIMEOUT).await?; Ok::<_, Error>((node, resp)) }); let mut resps = call_futures.collect::>(); @@ -156,7 +155,6 @@ where let rpc = TableRPC::::ReadEntry(partition_key.clone(), sort_key.clone()); let resps = self - .aux .rpc_client .try_call_many( &who[..], @@ -214,7 +212,6 @@ where let rpc = TableRPC::::ReadRange(partition_key.clone(), begin_sort_key, filter, limit); let resps = self - .aux .rpc_client .try_call_many( &who[..], @@ -270,8 +267,7 @@ where async fn repair_on_read(&self, who: &[UUID], what: F::E) -> Result<(), Error> { let what_enc = Arc::new(ByteBuf::from(rmp_to_vec_all_named(&what)?)); - self.aux - .rpc_client + self.rpc_client .try_call_many( &who[..], TableRPC::::Update(vec![what_enc]), @@ -291,8 +287,7 @@ where }); let self2 = self.clone(); - self.aux - .rpc_client + self.rpc_client .set_local_handler(self.aux.system.id, move |msg| { let self2 = self2.clone(); async move { self2.handle(&msg).await } -- cgit v1.2.3 From a1442f072ad9427851f49103083582637ddcdbd4 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 12 Mar 2021 15:40:54 +0100 Subject: Implement garage stats to get info on node contents --- Cargo.lock | 23 ++++++++++++++ Makefile | 2 +- src/garage/Cargo.toml | 1 + src/garage/admin_rpc.rs | 80 +++++++++++++++++++++++++++++++++++++++++++++++++ src/garage/main.rs | 18 +++++++++++ src/table/data.rs | 2 +- src/table/merkle.rs | 4 +-- 7 files changed, 126 insertions(+), 4 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 45244b8b..b0633e21 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -489,6 +489,7 @@ dependencies = [ "garage_table", "garage_util", "garage_web", + "git-version", "hex", "log", "pretty_env_logger", @@ -705,6 +706,28 @@ dependencies = [ "wasi 0.9.0+wasi-snapshot-preview1", ] +[[package]] +name = "git-version" +version = "0.3.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94918e83f1e01dedc2e361d00ce9487b14c58c7f40bab148026fa39d42cb41e2" +dependencies = [ + "git-version-macro", + "proc-macro-hack", +] + +[[package]] +name = "git-version-macro" +version = "0.3.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34a97a52fdee1870a34fa6e4b77570cba531b27d1838874fef4429a791a3d657" +dependencies = [ + "proc-macro-hack", + "proc-macro2", + "quote", + "syn", +] + [[package]] name = "h2" version = "0.2.7" diff --git a/Makefile b/Makefile index 61105856..cbbcee29 100644 --- a/Makefile +++ b/Makefile @@ -4,7 +4,7 @@ DOCKER=lxpz/garage_amd64 all: #cargo fmt || true #RUSTFLAGS="-C link-arg=-fuse-ld=lld" cargo build - cargo build + clear; cargo build $(BIN): #RUSTFLAGS="-C link-arg=-fuse-ld=lld" cargo build --release diff --git a/src/garage/Cargo.toml b/src/garage/Cargo.toml index 115e2d0e..8c28394b 100644 --- a/src/garage/Cargo.toml +++ b/src/garage/Cargo.toml @@ -27,6 +27,7 @@ hex = "0.3" sha2 = "0.8" log = "0.4" pretty_env_logger = "0.4" +git-version = "0.3.4" sled = "0.34" diff --git a/src/garage/admin_rpc.rs b/src/garage/admin_rpc.rs index b4a65cad..42540d15 100644 --- a/src/garage/admin_rpc.rs +++ b/src/garage/admin_rpc.rs @@ -1,4 +1,6 @@ use std::sync::Arc; +use std::fmt::Write; +use std::collections::HashMap; use serde::{Deserialize, Serialize}; @@ -6,6 +8,7 @@ use garage_util::error::Error; use garage_table::crdt::CRDT; use garage_table::*; +use garage_table::replication::*; use garage_rpc::rpc_client::*; use garage_rpc::rpc_server::*; @@ -25,6 +28,7 @@ pub enum AdminRPC { BucketOperation(BucketOperation), KeyOperation(KeyOperation), LaunchRepair(RepairOpt), + Stats(StatsOpt), // Replies Ok(String), @@ -55,6 +59,7 @@ impl AdminRpcHandler { 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).await, + AdminRPC::Stats(opt) => self2.handle_stats(opt).await, _ => Err(Error::BadRPC(format!("Invalid RPC"))), } } @@ -357,4 +362,79 @@ impl AdminRpcHandler { ))) } } + + async fn handle_stats(&self, opt: StatsOpt) -> Result { + if opt.all_nodes { + + let mut ret = String::new(); + let ring = self.garage.system.ring.borrow().clone(); + + for node in ring.config.members.keys() { + let mut opt = opt.clone(); + opt.all_nodes = false; + + writeln!(&mut ret, "\n======================").unwrap(); + writeln!(&mut ret, "Stats for node {:?}:", node).unwrap(); + match self + .rpc_client + .call( + *node, + AdminRPC::Stats(opt), + ADMIN_RPC_TIMEOUT, + ) + .await + { + Ok(AdminRPC::Ok(s)) => writeln!(&mut ret, "{}", s).unwrap(), + Ok(x) => writeln!(&mut ret, "Bad answer: {:?}", x).unwrap(), + Err(e) => writeln!(&mut ret, "Error: {}", e).unwrap(), + } + } + Ok(AdminRPC::Ok(ret)) + } else { + Ok(AdminRPC::Ok(self.gather_stats_local(opt)?)) + } + } + + fn gather_stats_local(&self, opt: StatsOpt) -> Result { + let mut ret = String::new(); + writeln!(&mut ret, "\nGarage version: {}", git_version::git_version!()).unwrap(); + + // Gather ring statistics + let ring = self.garage.system.ring.borrow().clone(); + let mut ring_nodes = HashMap::new(); + for r in ring.ring.iter() { + for n in r.nodes.iter() { + if !ring_nodes.contains_key(n) { + ring_nodes.insert(*n, 0usize); + } + *ring_nodes.get_mut(n).unwrap() += 1; + } + } + writeln!(&mut ret, "\nRing nodes & partition count:").unwrap(); + for (n, c) in ring_nodes.iter() { + writeln!(&mut ret, " {:?} {}", n, c).unwrap(); + } + + self.gather_table_stats(&mut ret, &self.garage.bucket_table, &opt)?; + self.gather_table_stats(&mut ret, &self.garage.key_table, &opt)?; + self.gather_table_stats(&mut ret, &self.garage.object_table, &opt)?; + self.gather_table_stats(&mut ret, &self.garage.version_table, &opt)?; + self.gather_table_stats(&mut ret, &self.garage.block_ref_table, &opt)?; + + writeln!(&mut ret, "\nBlock manager stats:").unwrap(); + writeln!(&mut ret, " resync queue length: {}", self.garage.block_manager.resync_queue.len()).unwrap(); + + if opt.detailed { + writeln!(&mut ret, "\nDetailed stats not implemented yet.").unwrap(); + } + + Ok(ret) + } + + fn gather_table_stats(&self, to: &mut String, t: &Arc>, _opt: &StatsOpt) -> Result<(), Error> { + writeln!(to, "\nTable stats for {}", t.data.name).unwrap(); + writeln!(to, " number of items: {}", t.data.store.len()).unwrap(); + writeln!(to, " Merkle updater todo queue length: {}", t.data.merkle_updater.todo.len()).unwrap(); + Ok(()) + } } diff --git a/src/garage/main.rs b/src/garage/main.rs index 8757a1bb..0bd70c79 100644 --- a/src/garage/main.rs +++ b/src/garage/main.rs @@ -69,6 +69,10 @@ pub enum Command { /// Start repair of node data #[structopt(name = "repair")] Repair(RepairOpt), + + /// Gather node statistics + #[structopt(name = "stats")] + Stats(StatsOpt), } #[derive(StructOpt, Debug)] @@ -281,6 +285,17 @@ pub enum RepairWhat { BlockRefs, } +#[derive(Serialize, Deserialize, StructOpt, Debug, Clone)] +pub struct StatsOpt { + /// Gather statistics from all nodes + #[structopt(short = "a", long = "all-nodes")] + pub all_nodes: bool, + + /// Gather detailed statistics (this can be long) + #[structopt(short = "d", long = "detailed")] + pub detailed: bool, +} + #[tokio::main] async fn main() { pretty_env_logger::init(); @@ -332,6 +347,9 @@ async fn main() { Command::Repair(ro) => { cmd_admin(admin_rpc_cli, opt.rpc_host, AdminRPC::LaunchRepair(ro)).await } + Command::Stats(so) => { + cmd_admin(admin_rpc_cli, opt.rpc_host, AdminRPC::Stats(so)).await + } }; if let Err(e) = resp { diff --git a/src/table/data.rs b/src/table/data.rs index 2817a849..5e7314d2 100644 --- a/src/table/data.rs +++ b/src/table/data.rs @@ -17,7 +17,7 @@ pub struct TableData { pub instance: F, pub store: sled::Tree, - pub(crate) merkle_updater: Arc, + pub merkle_updater: Arc, } impl TableData diff --git a/src/table/merkle.rs b/src/table/merkle.rs index a164df04..b04a2a88 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -32,7 +32,7 @@ pub fn hash_of_merkle_partition_opt(p: Option) -> Hash { // 16 bits (two bytes) of item's partition keys' hashes. // It builds one Merkle tree for each of these 2**16 partitions. -pub(crate) struct MerkleUpdater { +pub struct MerkleUpdater { table_name: String, background: Arc, @@ -40,7 +40,7 @@ pub(crate) struct MerkleUpdater { // - key = the key of an item in the main table, ie hash(partition_key)+sort_key // - value = the hash of the full serialized item, if present, // or an empty vec if item is absent (deleted) - pub(crate) todo: sled::Tree, + pub todo: sled::Tree, pub(crate) todo_notify: Notify, // Content of the merkle tree: items where -- cgit v1.2.3 From 5ab33fddacc1c40eaf09e0c0cedb2224eb7def80 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 12 Mar 2021 18:12:31 +0100 Subject: Refactor CLI and prettify CLI outpu --- script/dev-bucket.sh | 2 +- src/garage/admin_rpc.rs | 1 + src/garage/cli.rs | 540 ++++++++++++++++++++++++++++++++++++++++++++++++ src/garage/main.rs | 536 +++-------------------------------------------- src/garage/server.rs | 1 + 5 files changed, 571 insertions(+), 509 deletions(-) create mode 100644 src/garage/cli.rs diff --git a/script/dev-bucket.sh b/script/dev-bucket.sh index 973d6d30..8a3587be 100755 --- a/script/dev-bucket.sh +++ b/script/dev-bucket.sh @@ -11,7 +11,7 @@ PATH="${GARAGE_DEBUG}:${GARAGE_RELEASE}:$PATH" garage bucket create eprouvette KEY_INFO=`garage key new --name 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]+$'` +SECRET_KEY=`echo $KEY_INFO|grep -Po 'Secret key: [a-f0-9]+'|grep -Po '[a-f0-9]+$'` garage bucket allow eprouvette --read --write --key $ACCESS_KEY echo "$ACCESS_KEY $SECRET_KEY" > /tmp/garage.s3 diff --git a/src/garage/admin_rpc.rs b/src/garage/admin_rpc.rs index 42540d15..fccd0e93 100644 --- a/src/garage/admin_rpc.rs +++ b/src/garage/admin_rpc.rs @@ -18,6 +18,7 @@ use garage_model::garage::Garage; use garage_model::key_table::*; use crate::repair::Repair; +use crate::cli::*; use crate::*; pub const ADMIN_RPC_TIMEOUT: Duration = Duration::from_secs(30); diff --git a/src/garage/cli.rs b/src/garage/cli.rs new file mode 100644 index 00000000..f6fdb09e --- /dev/null +++ b/src/garage/cli.rs @@ -0,0 +1,540 @@ +use std::collections::HashSet; +use std::net::SocketAddr; +use std::path::PathBuf; + +use serde::{Deserialize, Serialize}; +use structopt::StructOpt; + +use garage_util::data::*; +use garage_util::error::Error; + +use garage_rpc::membership::*; +use garage_rpc::ring::*; +use garage_rpc::rpc_client::*; + +use garage_model::bucket_table::*; +use garage_model::key_table::*; + +use crate::admin_rpc::*; + + +#[derive(StructOpt, Debug)] +pub enum Command { + /// Run Garage server + #[structopt(name = "server")] + Server(ServerOpt), + + /// Get network status + #[structopt(name = "status")] + Status, + + /// Garage node operations + #[structopt(name = "node")] + Node(NodeOperation), + + /// Bucket operations + #[structopt(name = "bucket")] + Bucket(BucketOperation), + + /// Key operations + #[structopt(name = "key")] + Key(KeyOperation), + + /// Start repair of node data + #[structopt(name = "repair")] + Repair(RepairOpt), + + /// Gather node statistics + #[structopt(name = "stats")] + Stats(StatsOpt), +} + +#[derive(StructOpt, Debug)] +pub struct ServerOpt { + /// Configuration file + #[structopt(short = "c", long = "config", default_value = "./config.toml")] + pub config_file: PathBuf, +} + +#[derive(StructOpt, Debug)] +pub enum NodeOperation { + /// Configure Garage node + #[structopt(name = "configure")] + Configure(ConfigureNodeOpt), + + /// Remove Garage node from cluster + #[structopt(name = "remove")] + Remove(RemoveNodeOpt), +} + +#[derive(StructOpt, Debug)] +pub struct ConfigureNodeOpt { + /// Node to configure (prefix of hexadecimal node id) + node_id: String, + + /// Location (datacenter) of the node + #[structopt(short = "d", long = "datacenter")] + datacenter: Option, + + /// Capacity (in relative terms, use 1 to represent your smallest server) + #[structopt(short = "c", long = "capacity")] + capacity: Option, + + /// Optionnal node tag + #[structopt(short = "t", long = "tag")] + tag: Option, +} + +#[derive(StructOpt, Debug)] +pub struct RemoveNodeOpt { + /// Node to configure (prefix of hexadecimal node id) + node_id: String, + + /// If this flag is not given, the node won't be removed + #[structopt(long = "yes")] + yes: bool, +} + +#[derive(Serialize, Deserialize, StructOpt, Debug)] +pub enum BucketOperation { + /// List buckets + #[structopt(name = "list")] + List, + + /// Get bucket info + #[structopt(name = "info")] + Info(BucketOpt), + + /// Create bucket + #[structopt(name = "create")] + Create(BucketOpt), + + /// Delete bucket + #[structopt(name = "delete")] + Delete(DeleteBucketOpt), + + /// Allow key to read or write to bucket + #[structopt(name = "allow")] + Allow(PermBucketOpt), + + /// 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)] +pub struct BucketOpt { + /// Bucket name + pub name: String, +} + +#[derive(Serialize, Deserialize, StructOpt, Debug)] +pub struct DeleteBucketOpt { + /// Bucket name + pub name: String, + + /// If this flag is not given, the bucket won't be deleted + #[structopt(long = "yes")] + pub yes: bool, +} + +#[derive(Serialize, Deserialize, StructOpt, Debug)] +pub struct PermBucketOpt { + /// Access key ID + #[structopt(long = "key")] + pub key_id: String, + + /// Allow/deny read operations + #[structopt(long = "read")] + pub read: bool, + + /// Allow/deny write operations + #[structopt(long = "write")] + pub write: bool, + + /// Bucket name + pub bucket: String, +} + +#[derive(Serialize, Deserialize, StructOpt, Debug)] +pub enum KeyOperation { + /// List keys + #[structopt(name = "list")] + List, + + /// Get key info + #[structopt(name = "info")] + Info(KeyOpt), + + /// Create new key + #[structopt(name = "new")] + New(KeyNewOpt), + + /// Rename key + #[structopt(name = "rename")] + Rename(KeyRenameOpt), + + /// Delete key + #[structopt(name = "delete")] + Delete(KeyDeleteOpt), +} + +#[derive(Serialize, Deserialize, StructOpt, Debug)] +pub struct KeyOpt { + /// ID of the key + pub key_id: String, +} + +#[derive(Serialize, Deserialize, StructOpt, Debug)] +pub struct KeyNewOpt { + /// Name of the key + #[structopt(long = "name", default_value = "Unnamed key")] + pub name: String, +} + +#[derive(Serialize, Deserialize, StructOpt, Debug)] +pub struct KeyRenameOpt { + /// ID of the key + pub key_id: String, + + /// New name of the key + pub new_name: String, +} + +#[derive(Serialize, Deserialize, StructOpt, Debug)] +pub struct KeyDeleteOpt { + /// ID of the key + pub key_id: String, + + /// Confirm deletion + #[structopt(long = "yes")] + pub yes: bool, +} + +#[derive(Serialize, Deserialize, StructOpt, Debug, Clone)] +pub struct RepairOpt { + /// Launch repair operation on all nodes + #[structopt(short = "a", long = "all-nodes")] + pub all_nodes: bool, + + /// Confirm the launch of the repair operation + #[structopt(long = "yes")] + pub yes: bool, + + #[structopt(subcommand)] + pub what: Option, +} + +#[derive(Serialize, Deserialize, StructOpt, Debug, Eq, PartialEq, Clone)] +pub enum RepairWhat { + /// Only do a full sync of metadata tables + #[structopt(name = "tables")] + Tables, + /// Only repair (resync/rebalance) the set of stored blocks + #[structopt(name = "blocks")] + Blocks, + /// Only redo the propagation of object deletions to the version table (slow) + #[structopt(name = "versions")] + Versions, + /// Only redo the propagation of version deletions to the block ref table (extremely slow) + #[structopt(name = "block_refs")] + BlockRefs, +} + +#[derive(Serialize, Deserialize, StructOpt, Debug, Clone)] +pub struct StatsOpt { + /// Gather statistics from all nodes + #[structopt(short = "a", long = "all-nodes")] + pub all_nodes: bool, + + /// Gather detailed statistics (this can be long) + #[structopt(short = "d", long = "detailed")] + pub detailed: bool, +} + + +pub async fn cli_cmd( + cmd: Command, + membership_rpc_cli: RpcAddrClient, + admin_rpc_cli: RpcAddrClient, + rpc_host: SocketAddr) -> Result<(), Error> { + match cmd { + Command::Status => cmd_status(membership_rpc_cli, rpc_host).await, + Command::Node(NodeOperation::Configure(configure_opt)) => { + cmd_configure(membership_rpc_cli, rpc_host, configure_opt).await + } + Command::Node(NodeOperation::Remove(remove_opt)) => { + cmd_remove(membership_rpc_cli, rpc_host, remove_opt).await + } + Command::Bucket(bo) => { + cmd_admin(admin_rpc_cli, rpc_host, AdminRPC::BucketOperation(bo)).await + } + Command::Key(ko) => { + cmd_admin(admin_rpc_cli, rpc_host, AdminRPC::KeyOperation(ko)).await + } + Command::Repair(ro) => { + cmd_admin(admin_rpc_cli, rpc_host, AdminRPC::LaunchRepair(ro)).await + } + Command::Stats(so) => { + cmd_admin(admin_rpc_cli, rpc_host, AdminRPC::Stats(so)).await + } + _ => unreachable!(), + } +} + + +pub async fn cmd_status(rpc_cli: RpcAddrClient, rpc_host: SocketAddr) -> Result<(), Error> { + let status = match rpc_cli + .call(&rpc_host, &Message::PullStatus, ADMIN_RPC_TIMEOUT) + .await?? + { + Message::AdvertiseNodesUp(nodes) => nodes, + resp => return Err(Error::Message(format!("Invalid RPC response: {:?}", resp))), + }; + let config = match rpc_cli + .call(&rpc_host, &Message::PullConfig, ADMIN_RPC_TIMEOUT) + .await?? + { + Message::AdvertiseConfig(cfg) => cfg, + resp => return Err(Error::Message(format!("Invalid RPC response: {:?}", resp))), + }; + + println!("Healthy nodes:"); + for adv in status.iter().filter(|x| x.is_up) { + if let Some(cfg) = config.members.get(&adv.id) { + println!( + "{:?}\t{}\t{}\t[{}]\t{}\t{}", + adv.id, adv.state_info.hostname, adv.addr, cfg.tag, cfg.datacenter, cfg.capacity + ); + } else { + println!( + "{:?}\t{}\t{}\tUNCONFIGURED/REMOVED", + adv.id, adv.state_info.hostname, adv.addr + ); + } + } + + let status_keys = status.iter().map(|x| x.id).collect::>(); + let failure_case_1 = status.iter().any(|x| !x.is_up); + let failure_case_2 = config + .members + .iter() + .any(|(id, _)| !status_keys.contains(id)); + if failure_case_1 || failure_case_2 { + println!("\nFailed nodes:"); + for adv in status.iter().filter(|x| !x.is_up) { + if let Some(cfg) = config.members.get(&adv.id) { + println!( + "{:?}\t{}\t{}\t[{}]\t{}\t{}\tlast seen: {}s ago", + adv.id, + adv.state_info.hostname, + adv.addr, + cfg.tag, + cfg.datacenter, + cfg.capacity, + (now_msec() - adv.last_seen) / 1000, + ); + } + } + for (id, cfg) in config.members.iter() { + if !status.iter().any(|x| x.id == *id) { + println!( + "{:?}\t{}\t{}\t{}\tnever seen", + id, cfg.tag, cfg.datacenter, cfg.capacity + ); + } + } + } + + Ok(()) +} + +pub async fn cmd_configure( + rpc_cli: RpcAddrClient, + rpc_host: SocketAddr, + args: ConfigureNodeOpt, +) -> Result<(), Error> { + let status = match rpc_cli + .call(&rpc_host, &Message::PullStatus, ADMIN_RPC_TIMEOUT) + .await?? + { + Message::AdvertiseNodesUp(nodes) => nodes, + resp => return Err(Error::Message(format!("Invalid RPC response: {:?}", resp))), + }; + + let mut candidates = vec![]; + for adv in status.iter() { + if hex::encode(&adv.id).starts_with(&args.node_id) { + candidates.push(adv.id); + } + } + if candidates.len() != 1 { + return Err(Error::Message(format!( + "{} matching nodes", + candidates.len() + ))); + } + + let mut config = match rpc_cli + .call(&rpc_host, &Message::PullConfig, ADMIN_RPC_TIMEOUT) + .await?? + { + Message::AdvertiseConfig(cfg) => cfg, + resp => return Err(Error::Message(format!("Invalid RPC response: {:?}", resp))), + }; + + let new_entry = match config.members.get(&candidates[0]) { + None => NetworkConfigEntry { + datacenter: args + .datacenter + .expect("Please specifiy a datacenter with the -d flag"), + capacity: args + .capacity + .expect("Please specifiy a capacity with the -c flag"), + tag: args.tag.unwrap_or("".to_string()), + }, + Some(old) => NetworkConfigEntry { + datacenter: args.datacenter.unwrap_or(old.datacenter.to_string()), + capacity: args.capacity.unwrap_or(old.capacity), + tag: args.tag.unwrap_or(old.tag.to_string()), + }, + }; + + config.members.insert(candidates[0].clone(), new_entry); + config.version += 1; + + rpc_cli + .call( + &rpc_host, + &Message::AdvertiseConfig(config), + ADMIN_RPC_TIMEOUT, + ) + .await??; + Ok(()) +} + +pub async fn cmd_remove( + rpc_cli: RpcAddrClient, + rpc_host: SocketAddr, + args: RemoveNodeOpt, +) -> Result<(), Error> { + let mut config = match rpc_cli + .call(&rpc_host, &Message::PullConfig, ADMIN_RPC_TIMEOUT) + .await?? + { + Message::AdvertiseConfig(cfg) => cfg, + resp => return Err(Error::Message(format!("Invalid RPC response: {:?}", resp))), + }; + + let mut candidates = vec![]; + for (key, _) in config.members.iter() { + if hex::encode(key).starts_with(&args.node_id) { + candidates.push(*key); + } + } + if candidates.len() != 1 { + return Err(Error::Message(format!( + "{} matching nodes", + candidates.len() + ))); + } + + if !args.yes { + return Err(Error::Message(format!( + "Add the flag --yes to really remove {:?} from the cluster", + candidates[0] + ))); + } + + config.members.remove(&candidates[0]); + config.version += 1; + + rpc_cli + .call( + &rpc_host, + &Message::AdvertiseConfig(config), + ADMIN_RPC_TIMEOUT, + ) + .await??; + Ok(()) +} + +pub async fn cmd_admin( + rpc_cli: RpcAddrClient, + rpc_host: SocketAddr, + args: AdminRPC, +) -> Result<(), Error> { + match rpc_cli.call(&rpc_host, args, ADMIN_RPC_TIMEOUT).await?? { + AdminRPC::Ok(msg) => { + println!("{}", msg); + } + AdminRPC::BucketList(bl) => { + println!("List of buckets:"); + for bucket in bl { + println!("{}", bucket); + } + } + AdminRPC::BucketInfo(bucket) => { + print_bucket_info(&bucket); + } + AdminRPC::KeyList(kl) => { + println!("List of keys:"); + for key in kl { + println!("{}\t{}", key.0, key.1); + } + } + AdminRPC::KeyInfo(key) => { + print_key_info(&key); + } + r => { + error!("Unexpected response: {:?}", r); + } + } + Ok(()) +} + +fn print_key_info(key: &Key) { + println!("Key name: {}", key.name.get()); + println!("Key ID: {}", key.key_id); + println!("Secret key: {}", key.secret_key); + if key.deleted.get() { + println!("Key is deleted."); + } else { + println!("Authorized buckets:"); + for (b, _, perm) in key.authorized_buckets.items().iter() { + println!("- {} R:{} W:{}", b, perm.allow_read, perm.allow_write); + } + } +} + +fn print_bucket_info(bucket: &Bucket) { + println!("Bucket name: {}", bucket.name); + match bucket.state.get() { + BucketState::Deleted => println!("Bucket is deleted."), + BucketState::Present(p) => { + println!("Authorized keys:"); + for (k, _, perm) in p.authorized_keys.items().iter() { + println!("- {} R:{} W:{}", k, perm.allow_read, perm.allow_write); + } + println!("Website access: {}", p.website.get()); + } + }; +} diff --git a/src/garage/main.rs b/src/garage/main.rs index 0bd70c79..76fc46ed 100644 --- a/src/garage/main.rs +++ b/src/garage/main.rs @@ -6,24 +6,21 @@ extern crate log; mod admin_rpc; mod repair; mod server; +mod cli; -use std::collections::HashSet; use std::net::SocketAddr; -use std::path::PathBuf; -use std::sync::Arc; use std::time::Duration; +use std::sync::Arc; -use serde::{Deserialize, Serialize}; use structopt::StructOpt; -use garage_util::config::TlsConfig; -use garage_util::data::*; use garage_util::error::Error; +use garage_util::config::TlsConfig; -use garage_rpc::membership::*; -use garage_rpc::ring::*; use garage_rpc::rpc_client::*; +use garage_rpc::membership::*; +use cli::*; use admin_rpc::*; #[derive(StructOpt, Debug)] @@ -31,277 +28,43 @@ use admin_rpc::*; pub struct Opt { /// RPC connect to this host to execute client operations #[structopt(short = "h", long = "rpc-host", default_value = "127.0.0.1:3901")] - rpc_host: SocketAddr, + pub rpc_host: SocketAddr, #[structopt(long = "ca-cert")] - ca_cert: Option, + pub ca_cert: Option, #[structopt(long = "client-cert")] - client_cert: Option, + pub client_cert: Option, #[structopt(long = "client-key")] - client_key: Option, + pub client_key: Option, #[structopt(subcommand)] cmd: Command, } -#[derive(StructOpt, Debug)] -pub enum Command { - /// Run Garage server - #[structopt(name = "server")] - Server(ServerOpt), - - /// Get network status - #[structopt(name = "status")] - Status, - - /// Garage node operations - #[structopt(name = "node")] - Node(NodeOperation), - - /// Bucket operations - #[structopt(name = "bucket")] - Bucket(BucketOperation), - - /// Key operations - #[structopt(name = "key")] - Key(KeyOperation), - - /// Start repair of node data - #[structopt(name = "repair")] - Repair(RepairOpt), - - /// Gather node statistics - #[structopt(name = "stats")] - Stats(StatsOpt), -} - -#[derive(StructOpt, Debug)] -pub struct ServerOpt { - /// Configuration file - #[structopt(short = "c", long = "config", default_value = "./config.toml")] - config_file: PathBuf, -} - -#[derive(StructOpt, Debug)] -pub enum NodeOperation { - /// Configure Garage node - #[structopt(name = "configure")] - Configure(ConfigureNodeOpt), - - /// Remove Garage node from cluster - #[structopt(name = "remove")] - Remove(RemoveNodeOpt), -} - -#[derive(StructOpt, Debug)] -pub struct ConfigureNodeOpt { - /// Node to configure (prefix of hexadecimal node id) - node_id: String, - - /// Location (datacenter) of the node - #[structopt(short = "d", long = "datacenter")] - datacenter: Option, - - /// Capacity (in relative terms, use 1 to represent your smallest server) - #[structopt(short = "c", long = "capacity")] - capacity: Option, - - /// Optionnal node tag - #[structopt(short = "t", long = "tag")] - tag: Option, -} - -#[derive(StructOpt, Debug)] -pub struct RemoveNodeOpt { - /// Node to configure (prefix of hexadecimal node id) - node_id: String, - - /// If this flag is not given, the node won't be removed - #[structopt(long = "yes")] - yes: bool, -} - -#[derive(Serialize, Deserialize, StructOpt, Debug)] -pub enum BucketOperation { - /// List buckets - #[structopt(name = "list")] - List, - - /// Get bucket info - #[structopt(name = "info")] - Info(BucketOpt), - - /// Create bucket - #[structopt(name = "create")] - Create(BucketOpt), - - /// Delete bucket - #[structopt(name = "delete")] - Delete(DeleteBucketOpt), - - /// Allow key to read or write to bucket - #[structopt(name = "allow")] - Allow(PermBucketOpt), - - /// 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)] -pub struct BucketOpt { - /// Bucket name - pub name: String, -} - -#[derive(Serialize, Deserialize, StructOpt, Debug)] -pub struct DeleteBucketOpt { - /// Bucket name - pub name: String, - - /// If this flag is not given, the bucket won't be deleted - #[structopt(long = "yes")] - pub yes: bool, -} - -#[derive(Serialize, Deserialize, StructOpt, Debug)] -pub struct PermBucketOpt { - /// Access key ID - #[structopt(long = "key")] - pub key_id: String, - - /// Allow/deny read operations - #[structopt(long = "read")] - pub read: bool, - - /// Allow/deny write operations - #[structopt(long = "write")] - pub write: bool, - - /// Bucket name - pub bucket: String, -} - -#[derive(Serialize, Deserialize, StructOpt, Debug)] -pub enum KeyOperation { - /// List keys - #[structopt(name = "list")] - List, - - /// Get key info - #[structopt(name = "info")] - Info(KeyOpt), - - /// Create new key - #[structopt(name = "new")] - New(KeyNewOpt), - - /// Rename key - #[structopt(name = "rename")] - Rename(KeyRenameOpt), - - /// Delete key - #[structopt(name = "delete")] - Delete(KeyDeleteOpt), -} - -#[derive(Serialize, Deserialize, StructOpt, Debug)] -pub struct KeyOpt { - /// ID of the key - key_id: String, -} - -#[derive(Serialize, Deserialize, StructOpt, Debug)] -pub struct KeyNewOpt { - /// Name of the key - #[structopt(long = "name", default_value = "Unnamed key")] - name: String, -} - -#[derive(Serialize, Deserialize, StructOpt, Debug)] -pub struct KeyRenameOpt { - /// ID of the key - key_id: String, - - /// New name of the key - new_name: String, -} - -#[derive(Serialize, Deserialize, StructOpt, Debug)] -pub struct KeyDeleteOpt { - /// ID of the key - key_id: String, - - /// Confirm deletion - #[structopt(long = "yes")] - yes: bool, -} - -#[derive(Serialize, Deserialize, StructOpt, Debug, Clone)] -pub struct RepairOpt { - /// Launch repair operation on all nodes - #[structopt(short = "a", long = "all-nodes")] - pub all_nodes: bool, - - /// Confirm the launch of the repair operation - #[structopt(long = "yes")] - pub yes: bool, - - #[structopt(subcommand)] - pub what: Option, -} - -#[derive(Serialize, Deserialize, StructOpt, Debug, Eq, PartialEq, Clone)] -pub enum RepairWhat { - /// Only do a full sync of metadata tables - #[structopt(name = "tables")] - Tables, - /// Only repair (resync/rebalance) the set of stored blocks - #[structopt(name = "blocks")] - Blocks, - /// Only redo the propagation of object deletions to the version table (slow) - #[structopt(name = "versions")] - Versions, - /// Only redo the propagation of version deletions to the block ref table (extremely slow) - #[structopt(name = "block_refs")] - BlockRefs, -} - -#[derive(Serialize, Deserialize, StructOpt, Debug, Clone)] -pub struct StatsOpt { - /// Gather statistics from all nodes - #[structopt(short = "a", long = "all-nodes")] - pub all_nodes: bool, - - /// Gather detailed statistics (this can be long) - #[structopt(short = "d", long = "detailed")] - pub detailed: bool, -} - #[tokio::main] async fn main() { pretty_env_logger::init(); let opt = Opt::from_args(); + let res = if let Command::Server(server_opt) = opt.cmd { + // Abort on panic (same behavior as in Go) + std::panic::set_hook(Box::new(|panic_info| { + error!("{}", panic_info.to_string()); + std::process::abort(); + })); + + server::run_server(server_opt.config_file).await + } else { + cli_command(opt).await + }; + + if let Err(e) = res { + error!("{}", e); + } +} + +async fn cli_command(opt: Opt) -> Result<(), Error> { let tls_config = match (opt.ca_cert, opt.client_cert, opt.client_key) { (Some(ca_cert), Some(client_cert), Some(client_key)) => Some(TlsConfig { ca_cert, @@ -321,248 +84,5 @@ async fn main() { RpcAddrClient::new(rpc_http_cli.clone(), MEMBERSHIP_RPC_PATH.to_string()); let admin_rpc_cli = RpcAddrClient::new(rpc_http_cli.clone(), ADMIN_RPC_PATH.to_string()); - let resp = match opt.cmd { - Command::Server(server_opt) => { - // Abort on panic (same behavior as in Go) - std::panic::set_hook(Box::new(|panic_info| { - error!("{}", panic_info.to_string()); - std::process::abort(); - })); - - server::run_server(server_opt.config_file).await - } - Command::Status => cmd_status(membership_rpc_cli, opt.rpc_host).await, - Command::Node(NodeOperation::Configure(configure_opt)) => { - cmd_configure(membership_rpc_cli, opt.rpc_host, configure_opt).await - } - Command::Node(NodeOperation::Remove(remove_opt)) => { - cmd_remove(membership_rpc_cli, opt.rpc_host, remove_opt).await - } - Command::Bucket(bo) => { - cmd_admin(admin_rpc_cli, opt.rpc_host, AdminRPC::BucketOperation(bo)).await - } - Command::Key(bo) => { - cmd_admin(admin_rpc_cli, opt.rpc_host, AdminRPC::KeyOperation(bo)).await - } - Command::Repair(ro) => { - cmd_admin(admin_rpc_cli, opt.rpc_host, AdminRPC::LaunchRepair(ro)).await - } - Command::Stats(so) => { - cmd_admin(admin_rpc_cli, opt.rpc_host, AdminRPC::Stats(so)).await - } - }; - - if let Err(e) = resp { - error!("Error: {}", e); - } -} - -async fn cmd_status(rpc_cli: RpcAddrClient, rpc_host: SocketAddr) -> Result<(), Error> { - let status = match rpc_cli - .call(&rpc_host, &Message::PullStatus, ADMIN_RPC_TIMEOUT) - .await?? - { - Message::AdvertiseNodesUp(nodes) => nodes, - resp => return Err(Error::Message(format!("Invalid RPC response: {:?}", resp))), - }; - let config = match rpc_cli - .call(&rpc_host, &Message::PullConfig, ADMIN_RPC_TIMEOUT) - .await?? - { - Message::AdvertiseConfig(cfg) => cfg, - resp => return Err(Error::Message(format!("Invalid RPC response: {:?}", resp))), - }; - - println!("Healthy nodes:"); - for adv in status.iter().filter(|x| x.is_up) { - if let Some(cfg) = config.members.get(&adv.id) { - println!( - "{:?}\t{}\t{}\t[{}]\t{}\t{}", - adv.id, adv.state_info.hostname, adv.addr, cfg.tag, cfg.datacenter, cfg.capacity - ); - } else { - println!( - "{:?}\t{}\t{}\tUNCONFIGURED/REMOVED", - adv.id, adv.state_info.hostname, adv.addr - ); - } - } - - let status_keys = status.iter().map(|x| x.id).collect::>(); - let failure_case_1 = status.iter().any(|x| !x.is_up); - let failure_case_2 = config - .members - .iter() - .any(|(id, _)| !status_keys.contains(id)); - if failure_case_1 || failure_case_2 { - println!("\nFailed nodes:"); - for adv in status.iter().filter(|x| !x.is_up) { - if let Some(cfg) = config.members.get(&adv.id) { - println!( - "{:?}\t{}\t{}\t[{}]\t{}\t{}\tlast seen: {}s ago", - adv.id, - adv.state_info.hostname, - adv.addr, - cfg.tag, - cfg.datacenter, - cfg.capacity, - (now_msec() - adv.last_seen) / 1000, - ); - } - } - for (id, cfg) in config.members.iter() { - if !status.iter().any(|x| x.id == *id) { - println!( - "{:?}\t{}\t{}\t{}\tnever seen", - id, cfg.tag, cfg.datacenter, cfg.capacity - ); - } - } - } - - Ok(()) -} - -async fn cmd_configure( - rpc_cli: RpcAddrClient, - rpc_host: SocketAddr, - args: ConfigureNodeOpt, -) -> Result<(), Error> { - let status = match rpc_cli - .call(&rpc_host, &Message::PullStatus, ADMIN_RPC_TIMEOUT) - .await?? - { - Message::AdvertiseNodesUp(nodes) => nodes, - resp => return Err(Error::Message(format!("Invalid RPC response: {:?}", resp))), - }; - - let mut candidates = vec![]; - for adv in status.iter() { - if hex::encode(&adv.id).starts_with(&args.node_id) { - candidates.push(adv.id); - } - } - if candidates.len() != 1 { - return Err(Error::Message(format!( - "{} matching nodes", - candidates.len() - ))); - } - - let mut config = match rpc_cli - .call(&rpc_host, &Message::PullConfig, ADMIN_RPC_TIMEOUT) - .await?? - { - Message::AdvertiseConfig(cfg) => cfg, - resp => return Err(Error::Message(format!("Invalid RPC response: {:?}", resp))), - }; - - let new_entry = match config.members.get(&candidates[0]) { - None => NetworkConfigEntry { - datacenter: args - .datacenter - .expect("Please specifiy a datacenter with the -d flag"), - capacity: args - .capacity - .expect("Please specifiy a capacity with the -c flag"), - tag: args.tag.unwrap_or("".to_string()), - }, - Some(old) => NetworkConfigEntry { - datacenter: args.datacenter.unwrap_or(old.datacenter.to_string()), - capacity: args.capacity.unwrap_or(old.capacity), - tag: args.tag.unwrap_or(old.tag.to_string()), - }, - }; - - config.members.insert(candidates[0].clone(), new_entry); - config.version += 1; - - rpc_cli - .call( - &rpc_host, - &Message::AdvertiseConfig(config), - ADMIN_RPC_TIMEOUT, - ) - .await??; - Ok(()) -} - -async fn cmd_remove( - rpc_cli: RpcAddrClient, - rpc_host: SocketAddr, - args: RemoveNodeOpt, -) -> Result<(), Error> { - let mut config = match rpc_cli - .call(&rpc_host, &Message::PullConfig, ADMIN_RPC_TIMEOUT) - .await?? - { - Message::AdvertiseConfig(cfg) => cfg, - resp => return Err(Error::Message(format!("Invalid RPC response: {:?}", resp))), - }; - - let mut candidates = vec![]; - for (key, _) in config.members.iter() { - if hex::encode(key).starts_with(&args.node_id) { - candidates.push(*key); - } - } - if candidates.len() != 1 { - return Err(Error::Message(format!( - "{} matching nodes", - candidates.len() - ))); - } - - if !args.yes { - return Err(Error::Message(format!( - "Add the flag --yes to really remove {:?} from the cluster", - candidates[0] - ))); - } - - config.members.remove(&candidates[0]); - config.version += 1; - - rpc_cli - .call( - &rpc_host, - &Message::AdvertiseConfig(config), - ADMIN_RPC_TIMEOUT, - ) - .await??; - Ok(()) -} - -async fn cmd_admin( - rpc_cli: RpcAddrClient, - rpc_host: SocketAddr, - args: AdminRPC, -) -> Result<(), Error> { - match rpc_cli.call(&rpc_host, args, ADMIN_RPC_TIMEOUT).await?? { - AdminRPC::Ok(msg) => { - println!("{}", msg); - } - AdminRPC::BucketList(bl) => { - println!("List of buckets:"); - for bucket in bl { - println!("{}", bucket); - } - } - AdminRPC::BucketInfo(bucket) => { - println!("{:?}", bucket); - } - AdminRPC::KeyList(kl) => { - println!("List of keys:"); - for key in kl { - println!("{}\t{}", key.0, key.1); - } - } - AdminRPC::KeyInfo(key) => { - println!("{:?}", key); - } - r => { - error!("Unexpected response: {:?}", r); - } - } - Ok(()) + cli_cmd(opt.cmd, membership_rpc_cli, admin_rpc_cli, opt.rpc_host).await } diff --git a/src/garage/server.rs b/src/garage/server.rs index 8dddd7bb..29740feb 100644 --- a/src/garage/server.rs +++ b/src/garage/server.rs @@ -49,6 +49,7 @@ pub async fn run_server(config_file: PathBuf) -> Result<(), Error> { let (send_cancel, watch_cancel) = watch::channel(false); let background = BackgroundRunner::new(16, watch_cancel.clone()); + info!("Initializing Garage main data store..."); let garage = Garage::new(config, db, background.clone(), &mut rpc_server); info!("Crate admin RPC handler..."); -- cgit v1.2.3 From f4aad8fe6e36fe05e05c88c322b99fc87d896578 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 12 Mar 2021 18:16:03 +0100 Subject: cargo fmt --- src/garage/admin_rpc.rs | 43 +++++++++++++++++++++++++++++-------------- src/garage/cli.rs | 23 +++++++++-------------- src/garage/main.rs | 10 +++++----- 3 files changed, 43 insertions(+), 33 deletions(-) diff --git a/src/garage/admin_rpc.rs b/src/garage/admin_rpc.rs index fccd0e93..a75c9013 100644 --- a/src/garage/admin_rpc.rs +++ b/src/garage/admin_rpc.rs @@ -1,14 +1,14 @@ -use std::sync::Arc; -use std::fmt::Write; use std::collections::HashMap; +use std::fmt::Write; +use std::sync::Arc; use serde::{Deserialize, Serialize}; use garage_util::error::Error; use garage_table::crdt::CRDT; -use garage_table::*; use garage_table::replication::*; +use garage_table::*; use garage_rpc::rpc_client::*; use garage_rpc::rpc_server::*; @@ -17,8 +17,8 @@ use garage_model::bucket_table::*; use garage_model::garage::Garage; use garage_model::key_table::*; -use crate::repair::Repair; use crate::cli::*; +use crate::repair::Repair; use crate::*; pub const ADMIN_RPC_TIMEOUT: Duration = Duration::from_secs(30); @@ -366,7 +366,6 @@ impl AdminRpcHandler { async fn handle_stats(&self, opt: StatsOpt) -> Result { if opt.all_nodes { - let mut ret = String::new(); let ring = self.garage.system.ring.borrow().clone(); @@ -378,11 +377,7 @@ impl AdminRpcHandler { writeln!(&mut ret, "Stats for node {:?}:", node).unwrap(); match self .rpc_client - .call( - *node, - AdminRPC::Stats(opt), - ADMIN_RPC_TIMEOUT, - ) + .call(*node, AdminRPC::Stats(opt), ADMIN_RPC_TIMEOUT) .await { Ok(AdminRPC::Ok(s)) => writeln!(&mut ret, "{}", s).unwrap(), @@ -398,7 +393,12 @@ impl AdminRpcHandler { fn gather_stats_local(&self, opt: StatsOpt) -> Result { let mut ret = String::new(); - writeln!(&mut ret, "\nGarage version: {}", git_version::git_version!()).unwrap(); + writeln!( + &mut ret, + "\nGarage version: {}", + git_version::git_version!() + ) + .unwrap(); // Gather ring statistics let ring = self.garage.system.ring.borrow().clone(); @@ -423,7 +423,12 @@ impl AdminRpcHandler { self.gather_table_stats(&mut ret, &self.garage.block_ref_table, &opt)?; writeln!(&mut ret, "\nBlock manager stats:").unwrap(); - writeln!(&mut ret, " resync queue length: {}", self.garage.block_manager.resync_queue.len()).unwrap(); + writeln!( + &mut ret, + " resync queue length: {}", + self.garage.block_manager.resync_queue.len() + ) + .unwrap(); if opt.detailed { writeln!(&mut ret, "\nDetailed stats not implemented yet.").unwrap(); @@ -432,10 +437,20 @@ impl AdminRpcHandler { Ok(ret) } - fn gather_table_stats(&self, to: &mut String, t: &Arc>, _opt: &StatsOpt) -> Result<(), Error> { + fn gather_table_stats( + &self, + to: &mut String, + t: &Arc>, + _opt: &StatsOpt, + ) -> Result<(), Error> { writeln!(to, "\nTable stats for {}", t.data.name).unwrap(); writeln!(to, " number of items: {}", t.data.store.len()).unwrap(); - writeln!(to, " Merkle updater todo queue length: {}", t.data.merkle_updater.todo.len()).unwrap(); + writeln!( + to, + " Merkle updater todo queue length: {}", + t.data.merkle_updater.todo.len() + ) + .unwrap(); Ok(()) } } diff --git a/src/garage/cli.rs b/src/garage/cli.rs index f6fdb09e..56f03c8b 100644 --- a/src/garage/cli.rs +++ b/src/garage/cli.rs @@ -17,7 +17,6 @@ use garage_model::key_table::*; use crate::admin_rpc::*; - #[derive(StructOpt, Debug)] pub enum Command { /// Run Garage server @@ -270,12 +269,12 @@ pub struct StatsOpt { pub detailed: bool, } - pub async fn cli_cmd( cmd: Command, membership_rpc_cli: RpcAddrClient, admin_rpc_cli: RpcAddrClient, - rpc_host: SocketAddr) -> Result<(), Error> { + rpc_host: SocketAddr, +) -> Result<(), Error> { match cmd { Command::Status => cmd_status(membership_rpc_cli, rpc_host).await, Command::Node(NodeOperation::Configure(configure_opt)) => { @@ -287,21 +286,17 @@ pub async fn cli_cmd( Command::Bucket(bo) => { cmd_admin(admin_rpc_cli, rpc_host, AdminRPC::BucketOperation(bo)).await } - Command::Key(ko) => { - cmd_admin(admin_rpc_cli, rpc_host, AdminRPC::KeyOperation(ko)).await - } - Command::Repair(ro) => { - cmd_admin(admin_rpc_cli, rpc_host, AdminRPC::LaunchRepair(ro)).await - } - Command::Stats(so) => { - cmd_admin(admin_rpc_cli, rpc_host, AdminRPC::Stats(so)).await - } + Command::Key(ko) => cmd_admin(admin_rpc_cli, rpc_host, AdminRPC::KeyOperation(ko)).await, + Command::Repair(ro) => cmd_admin(admin_rpc_cli, rpc_host, AdminRPC::LaunchRepair(ro)).await, + Command::Stats(so) => cmd_admin(admin_rpc_cli, rpc_host, AdminRPC::Stats(so)).await, _ => unreachable!(), } } - -pub async fn cmd_status(rpc_cli: RpcAddrClient, rpc_host: SocketAddr) -> Result<(), Error> { +pub async fn cmd_status( + rpc_cli: RpcAddrClient, + rpc_host: SocketAddr, +) -> Result<(), Error> { let status = match rpc_cli .call(&rpc_host, &Message::PullStatus, ADMIN_RPC_TIMEOUT) .await?? diff --git a/src/garage/main.rs b/src/garage/main.rs index 76fc46ed..6c86d0fb 100644 --- a/src/garage/main.rs +++ b/src/garage/main.rs @@ -4,24 +4,24 @@ extern crate log; mod admin_rpc; +mod cli; mod repair; mod server; -mod cli; use std::net::SocketAddr; -use std::time::Duration; use std::sync::Arc; +use std::time::Duration; use structopt::StructOpt; -use garage_util::error::Error; use garage_util::config::TlsConfig; +use garage_util::error::Error; -use garage_rpc::rpc_client::*; use garage_rpc::membership::*; +use garage_rpc::rpc_client::*; -use cli::*; use admin_rpc::*; +use cli::*; #[derive(StructOpt, Debug)] #[structopt(name = "garage")] -- cgit v1.2.3 From c475471e7a8e7544f2be490898f4249cf27a17e9 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 12 Mar 2021 19:57:37 +0100 Subject: Implement table gc, currently for block_ref and version only --- src/api/s3_put.rs | 4 +- src/api/signature.rs | 8 +- src/model/block.rs | 18 ++-- src/model/block_ref_table.rs | 3 + src/model/version_table.rs | 3 + src/rpc/rpc_client.rs | 5 +- src/table/data.rs | 67 +++++++++++--- src/table/gc.rs | 212 +++++++++++++++++++++++++++++++++++++++++++ src/table/lib.rs | 1 + src/table/merkle.rs | 5 +- src/table/schema.rs | 2 + src/table/sync.rs | 18 ++-- src/table/table.rs | 8 +- src/util/data.rs | 8 ++ 14 files changed, 303 insertions(+), 59 deletions(-) create mode 100644 src/table/gc.rs diff --git a/src/api/s3_put.rs b/src/api/s3_put.rs index 17732ced..c1774d6b 100644 --- a/src/api/s3_put.rs +++ b/src/api/s3_put.rs @@ -225,9 +225,7 @@ async fn read_and_put_blocks( let data_md5sum = md5hasher.finalize(); let data_sha256sum = sha256hasher.result(); - let mut hash = [0u8; 32]; - hash.copy_from_slice(&data_sha256sum[..]); - let data_sha256sum = Hash::from(hash); + let data_sha256sum = Hash::try_from(&data_sha256sum[..]).unwrap(); Ok((total_size, data_md5sum, data_sha256sum)) } diff --git a/src/api/signature.rs b/src/api/signature.rs index d7fbd3f7..04d6666c 100644 --- a/src/api/signature.rs +++ b/src/api/signature.rs @@ -106,12 +106,8 @@ pub async fn check_signature( } else { let bytes = hex::decode(authorization.content_sha256) .ok_or_bad_request("Invalid content sha256 hash")?; - let mut hash = [0u8; 32]; - if bytes.len() != 32 { - return Err(Error::BadRequest(format!("Invalid content sha256 hash"))); - } - hash.copy_from_slice(&bytes[..]); - Some(Hash::from(hash)) + Some(Hash::try_from(&bytes[..]) + .ok_or(Error::BadRequest(format!("Invalid content sha256 hash")))?) }; Ok((key, content_sha256)) diff --git a/src/model/block.rs b/src/model/block.rs index a3bbe43d..8b065c04 100644 --- a/src/model/block.rs +++ b/src/model/block.rs @@ -248,9 +248,7 @@ impl BlockManager { let time_msec = u64_from_bytes(&time_bytes[0..8]); let now = now_msec(); if now >= time_msec { - let mut hash = [0u8; 32]; - hash.copy_from_slice(hash_bytes.as_ref()); - let hash = Hash::from(hash); + let hash = Hash::try_from(&hash_bytes[..]).unwrap(); if let Err(e) = self.resync_iter(&hash).await { warn!("Failed to resync block {:?}, retrying later: {}", hash, e); @@ -340,15 +338,11 @@ impl BlockManager { need_nodes.len() ); - let put_block_message = Arc::new(self.read_block(hash).await?); - let put_resps = join_all(need_nodes.iter().map(|to| { - self.rpc_client - .call_arc(*to, put_block_message.clone(), BLOCK_RW_TIMEOUT) - })) - .await; - for resp in put_resps { - resp?; - } + let put_block_message = self.read_block(hash).await?; + self.rpc_client.try_call_many( + &need_nodes[..], + put_block_message, + RequestStrategy::with_quorum(need_nodes.len()).with_timeout(BLOCK_RW_TIMEOUT)).await?; } trace!( "Deleting block {:?}, offload finished ({} / {})", diff --git a/src/model/block_ref_table.rs b/src/model/block_ref_table.rs index 07fa5144..e4372717 100644 --- a/src/model/block_ref_table.rs +++ b/src/model/block_ref_table.rs @@ -27,6 +27,9 @@ impl Entry for BlockRef { fn sort_key(&self) -> &UUID { &self.version } + fn is_tombstone(&self) -> bool { + self.deleted.get() + } } impl CRDT for BlockRef { diff --git a/src/model/version_table.rs b/src/model/version_table.rs index cdc73a85..841fbfea 100644 --- a/src/model/version_table.rs +++ b/src/model/version_table.rs @@ -78,6 +78,9 @@ impl Entry for Version { fn sort_key(&self) -> &EmptyKey { &EmptyKey } + fn is_tombstone(&self) -> bool { + self.deleted.get() + } } impl CRDT for Version { diff --git a/src/rpc/rpc_client.rs b/src/rpc/rpc_client.rs index 70384391..cffcf106 100644 --- a/src/rpc/rpc_client.rs +++ b/src/rpc/rpc_client.rs @@ -197,11 +197,8 @@ impl RpcClient { if !strategy.rs_interrupt_after_quorum { let wait_finished_fut = tokio::spawn(async move { resp_stream.collect::>().await; - Ok(()) }); - self.background.spawn(wait_finished_fut.map(|x| { - x.unwrap_or_else(|e| Err(Error::Message(format!("Await failed: {}", e)))) - })); + self.background.spawn(wait_finished_fut.map(|_| Ok(()))); } Ok(results) diff --git a/src/table/data.rs b/src/table/data.rs index 5e7314d2..91607f7a 100644 --- a/src/table/data.rs +++ b/src/table/data.rs @@ -1,3 +1,4 @@ +use core::borrow::Borrow; use std::sync::Arc; use log::warn; @@ -17,6 +18,7 @@ pub struct TableData { pub instance: F, pub store: sled::Tree, + pub gc_todo: sled::Tree, pub merkle_updater: Arc, } @@ -41,6 +43,10 @@ where .open_tree(&format!("{}:merkle_tree", name)) .expect("Unable to open DB Merkle tree tree"); + let gc_todo = db + .open_tree(&format!("{}:gc_todo", name)) + .expect("Unable to open DB tree"); + let merkle_updater = MerkleUpdater::launch( name.clone(), background, @@ -52,6 +58,7 @@ where name, instance, store, + gc_todo, merkle_updater, }) } @@ -103,10 +110,17 @@ where } // Mutation functions - - pub(crate) fn update_many(&self, entries: &[Arc]) -> Result<(), Error> { + // When changing this code, take care of propagating modifications correctly: + // - When an entry is modified or deleted, call the updated() function + // on the table instance + // - When an entry is modified or deleted, add it to the merkle updater's todo list. + // This has to be done atomically with the modification for the merkle updater + // to maintain consistency. The merkle updater must then be notified with todo_notify. + // - When an entry is updated to be a tombstone, add it to the gc_todo tree + + pub(crate) fn update_many>(&self, entries: &[T]) -> Result<(), Error> { for update_bytes in entries.iter() { - self.update_entry(update_bytes.as_slice())?; + self.update_entry(update_bytes.borrow().as_slice())?; } Ok(()) } @@ -115,8 +129,8 @@ where let update = self.decode_entry(update_bytes)?; let tree_key = self.tree_key(update.partition_key(), update.sort_key()); - let changed = (&self.store, &self.merkle_updater.todo).transaction(|(db, mkl_todo)| { - let (old_entry, new_entry) = match db.get(&tree_key)? { + let changed = (&self.store, &self.merkle_updater.todo).transaction(|(store, mkl_todo)| { + let (old_entry, new_entry) = match store.get(&tree_key)? { Some(prev_bytes) => { let old_entry = self .decode_entry(&prev_bytes) @@ -132,27 +146,32 @@ where let new_bytes = rmp_to_vec_all_named(&new_entry) .map_err(Error::RMPEncode) .map_err(sled::transaction::ConflictableTransactionError::Abort)?; - mkl_todo.insert(tree_key.clone(), blake2sum(&new_bytes[..]).to_vec())?; - db.insert(tree_key.clone(), new_bytes)?; - Ok(Some((old_entry, new_entry))) + let new_bytes_hash = blake2sum(&new_bytes[..]); + mkl_todo.insert(tree_key.clone(), new_bytes_hash.as_slice())?; + store.insert(tree_key.clone(), new_bytes)?; + Ok(Some((old_entry, new_entry, new_bytes_hash))) } else { Ok(None) } })?; - if let Some((old_entry, new_entry)) = changed { + if let Some((old_entry, new_entry, new_bytes_hash)) = changed { + let is_tombstone = new_entry.is_tombstone(); self.instance.updated(old_entry, Some(new_entry)); self.merkle_updater.todo_notify.notify(); + if is_tombstone { + self.gc_todo.insert(&tree_key, new_bytes_hash.as_slice())?; + } } Ok(()) } pub(crate) fn delete_if_equal(self: &Arc, k: &[u8], v: &[u8]) -> Result { - let removed = (&self.store, &self.merkle_updater.todo).transaction(|(txn, mkl_todo)| { - if let Some(cur_v) = txn.get(k)? { + let removed = (&self.store, &self.merkle_updater.todo).transaction(|(store, mkl_todo)| { + if let Some(cur_v) = store.get(k)? { if cur_v == v { - txn.remove(k)?; + store.remove(k)?; mkl_todo.insert(k, vec![])?; return Ok(true); } @@ -168,6 +187,30 @@ where Ok(removed) } + pub(crate) fn delete_if_equal_hash(self: &Arc, k: &[u8], vhash: Hash) -> Result { + let removed = (&self.store, &self.merkle_updater.todo).transaction(|(store, mkl_todo)| { + if let Some(cur_v) = store.get(k)? { + if blake2sum(&cur_v[..]) == vhash { + store.remove(k)?; + mkl_todo.insert(k, vec![])?; + return Ok(Some(cur_v)); + } + } + Ok(None) + })?; + + if let Some(old_v) = removed { + let old_entry = self.decode_entry(&old_v[..])?; + self.instance.updated(Some(old_entry), None); + self.merkle_updater.todo_notify.notify(); + Ok(true) + } else { + Ok(false) + } + } + + // ---- Utility functions ---- + pub(crate) fn tree_key(&self, p: &F::P, s: &F::S) -> Vec { let mut ret = p.hash().to_vec(); ret.extend(s.sort_key()); diff --git a/src/table/gc.rs b/src/table/gc.rs new file mode 100644 index 00000000..afc8a473 --- /dev/null +++ b/src/table/gc.rs @@ -0,0 +1,212 @@ +use std::sync::Arc; +use std::time::Duration; +use std::collections::HashMap; + +use serde::{Deserialize, Serialize}; +use serde_bytes::ByteBuf; + +use futures::future::join_all; +use futures::select; +use futures_util::future::*; +use tokio::sync::watch; + +use garage_util::data::*; +use garage_util::error::Error; + +use garage_rpc::rpc_client::*; +use garage_rpc::rpc_server::*; + +use crate::data::*; +use crate::table::*; +use crate::schema::*; +use crate::replication::*; + +const TABLE_GC_BATCH_SIZE: usize = 1024; +const TABLE_GC_RPC_TIMEOUT: Duration = Duration::from_secs(30); + +pub struct TableGC { + data: Arc>, + aux: Arc>, + + rpc_client: Arc>, +} + +#[derive(Serialize, Deserialize)] +enum GcRPC { + Update(Vec), + DeleteIfEqualHash(Vec<(ByteBuf, Hash)>), + Ok, +} + +impl RpcMessage for GcRPC {} + +impl TableGC +where + F: TableSchema + 'static, + R: TableReplication + 'static, +{ + pub(crate) fn launch( + data: Arc>, + aux: Arc>, + rpc_server: &mut RpcServer, + ) -> Arc { + let rpc_path = format!("table_{}/gc", data.name); + let rpc_client = aux.system.rpc_client::(&rpc_path); + + let gc = Arc::new(Self { + data: data.clone(), + aux: aux.clone(), + rpc_client, + }); + + gc.register_handler(rpc_server, rpc_path); + + let gc1 = gc.clone(); + aux.system.background.spawn_worker( + format!("GC loop for {}", data.name), + move |must_exit: watch::Receiver| gc1.gc_loop(must_exit), + ); + + gc + } + + async fn gc_loop(self: Arc, mut must_exit: watch::Receiver) -> Result<(), Error> { + while !*must_exit.borrow() { + match self.gc_loop_iter().await { + Ok(true) => { + // Stuff was done, loop imediately + } + Ok(false) => { + select! { + _ = tokio::time::delay_for(Duration::from_secs(10)).fuse() => (), + _ = must_exit.recv().fuse() => (), + } + } + Err(e) => { + warn!("({}) Error doing GC: {}", self.data.name, e); + } + } + } + Ok(()) + } + + async fn gc_loop_iter(&self) -> Result { + let mut entries = vec![]; + let mut excluded = vec![]; + + for item in self.data.gc_todo.iter() { + let (k, vhash) = item?; + + let vhash = Hash::try_from(&vhash[..]).unwrap(); + + let v_opt = self.data.store.get(&k[..])? + .filter(|v| blake2sum(&v[..]) == vhash); + + if let Some(v) = v_opt { + entries.push((ByteBuf::from(k.to_vec()), vhash, ByteBuf::from(v.to_vec()))); + if entries.len() >= TABLE_GC_BATCH_SIZE { + break; + } + } else { + excluded.push((k, vhash)); + } + } + + for (k, vhash) in excluded { + let _ = self.data.gc_todo.compare_and_swap::<_, _, Vec>(k, Some(vhash), None)?; + } + + if entries.len() == 0 { + // Nothing to do in this iteration + return Ok(false); + } + + debug!("({}) GC: doing {} items", self.data.name, entries.len()); + + let mut partitions = HashMap::new(); + for (k, vhash, v) in entries { + let pkh = Hash::try_from(&k[..32]).unwrap(); + let mut nodes = self.aux.replication.write_nodes(&pkh, &self.aux.system); + nodes.retain(|x| *x != self.aux.system.id); + nodes.sort(); + + if !partitions.contains_key(&nodes) { + partitions.insert(nodes.clone(), vec![]); + } + partitions.get_mut(&nodes).unwrap().push((k, vhash, v)); + } + + let resps = join_all(partitions.into_iter() + .map(|(nodes, items)| self.try_send_and_delete(nodes, items))).await; + for resp in resps { + if let Err(e) = resp { + warn!("({}) Unable to send and delete for GC: {}", self.data.name, e); + } + } + + Ok(true) + } + + async fn try_send_and_delete(&self, nodes: Vec, items: Vec<(ByteBuf, Hash, ByteBuf)>) -> Result<(), Error> { + let n_items = items.len(); + + let mut updates = vec![]; + let mut deletes = vec![]; + for (k, vhash, v) in items { + updates.push(v); + deletes.push((k, vhash)); + } + + self.rpc_client.try_call_many( + &nodes[..], + GcRPC::Update(updates), + RequestStrategy::with_quorum(nodes.len()).with_timeout(TABLE_GC_RPC_TIMEOUT)).await?; + + info!("({}) GC: {} items successfully pushed, will try to delete.", self.data.name, n_items); + + self.rpc_client.try_call_many( + &nodes[..], + GcRPC::DeleteIfEqualHash(deletes.clone()), + RequestStrategy::with_quorum(nodes.len()).with_timeout(TABLE_GC_RPC_TIMEOUT)).await?; + + for (k, vhash) in deletes { + self.data.delete_if_equal_hash(&k[..], vhash)?; + let _ = self.data.gc_todo.compare_and_swap::<_, _, Vec>(k, Some(vhash), None)?; + } + + Ok(()) + } + + // ---- RPC HANDLER ---- + + fn register_handler(self: &Arc, rpc_server: &mut RpcServer, path: String) { + let self2 = self.clone(); + rpc_server.add_handler::(path, move |msg, _addr| { + let self2 = self2.clone(); + async move { self2.handle_rpc(&msg).await } + }); + + let self2 = self.clone(); + self.rpc_client + .set_local_handler(self.aux.system.id, move |msg| { + let self2 = self2.clone(); + async move { self2.handle_rpc(&msg).await } + }); + } + + async fn handle_rpc(self: &Arc, message: &GcRPC) -> Result { + match message { + GcRPC::Update(items) => { + self.data.update_many(items)?; + Ok(GcRPC::Ok) + } + GcRPC::DeleteIfEqualHash(items) => { + for (key, vhash) in items.iter() { + self.data.delete_if_equal_hash(&key[..], *vhash)?; + } + Ok(GcRPC::Ok) + } + _ => Err(Error::Message(format!("Unexpected GC RPC"))), + } + } +} diff --git a/src/table/lib.rs b/src/table/lib.rs index 18c29c35..8a64ff0b 100644 --- a/src/table/lib.rs +++ b/src/table/lib.rs @@ -11,6 +11,7 @@ pub mod data; pub mod merkle; pub mod replication; pub mod sync; +pub mod gc; pub mod table; pub use schema::*; diff --git a/src/table/merkle.rs b/src/table/merkle.rs index b04a2a88..7a0adba1 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -139,10 +139,7 @@ impl MerkleUpdater { let new_vhash = if vhash_by.len() == 0 { None } else { - let vhash_by: [u8; 32] = vhash_by - .try_into() - .map_err(|_| Error::Message(format!("Invalid value in Merkle todo table")))?; - Some(Hash::from(vhash_by)) + Some(Hash::try_from(&vhash_by[..]).unwrap()) }; let key = MerkleNodeKey { diff --git a/src/table/schema.rs b/src/table/schema.rs index 7fbb7b25..5112ea15 100644 --- a/src/table/schema.rs +++ b/src/table/schema.rs @@ -41,6 +41,8 @@ pub trait Entry: { fn partition_key(&self) -> &P; fn sort_key(&self) -> &S; + + fn is_tombstone(&self) -> bool { false } } pub trait TableSchema: Send + Sync { diff --git a/src/table/sync.rs b/src/table/sync.rs index 4be8cd10..aae65852 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -3,7 +3,6 @@ use std::convert::TryInto; use std::sync::{Arc, Mutex}; use std::time::{Duration, Instant}; -use futures::future::join_all; use futures::{pin_mut, select}; use futures_util::future::*; use futures_util::stream::*; @@ -347,16 +346,11 @@ where nodes: &[UUID], ) -> Result<(), Error> { let values = items.iter().map(|(_k, v)| v.clone()).collect::>(); - let update_msg = Arc::new(SyncRPC::Items(values)); - - for res in join_all(nodes.iter().map(|to| { - self.rpc_client - .call_arc(*to, update_msg.clone(), TABLE_SYNC_RPC_TIMEOUT) - })) - .await - { - res?; - } + + self.rpc_client.try_call_many( + &nodes[..], + SyncRPC::Items(values), + RequestStrategy::with_quorum(nodes.len()).with_timeout(TABLE_SYNC_RPC_TIMEOUT)).await?; // All remote nodes have written those items, now we can delete them locally let mut not_removed = 0; @@ -577,7 +571,7 @@ where // ======= SYNCHRONIZATION PROCEDURE -- RECEIVER SIDE ====== - pub(crate) async fn handle_rpc(self: &Arc, message: &SyncRPC) -> Result { + async fn handle_rpc(self: &Arc, message: &SyncRPC) -> Result { match message { SyncRPC::RootCkHash(range, h) => { let root_ck = self.get_root_ck(*range)?; diff --git a/src/table/table.rs b/src/table/table.rs index dd3394bd..7b0d9a24 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -18,6 +18,7 @@ use crate::data::*; use crate::replication::*; use crate::schema::*; use crate::sync::*; +use crate::gc::*; const TABLE_RPC_TIMEOUT: Duration = Duration::from_secs(10); @@ -44,8 +45,6 @@ pub(crate) enum TableRPC { ReadRange(F::P, Option, Option, usize), Update(Vec>), - - SyncRPC(SyncRPC), } impl RpcMessage for TableRPC {} @@ -76,6 +75,7 @@ where }); let syncer = TableSyncer::launch(data.clone(), aux.clone(), rpc_server); + TableGC::launch(data.clone(), aux.clone(), rpc_server); let table = Arc::new(Self { data, @@ -308,10 +308,6 @@ where self.data.update_many(pairs)?; Ok(TableRPC::Ok) } - TableRPC::SyncRPC(rpc) => { - let response = self.syncer.handle_rpc(rpc).await?; - Ok(TableRPC::SyncRPC(response)) - } _ => Err(Error::BadRPC(format!("Unexpected table RPC"))), } } diff --git a/src/util/data.rs b/src/util/data.rs index f46454be..0dbd6df4 100644 --- a/src/util/data.rs +++ b/src/util/data.rs @@ -71,6 +71,14 @@ impl FixedBytes32 { pub fn to_vec(&self) -> Vec { self.0.to_vec() } + pub fn try_from(by: &[u8]) -> Option { + if by.len() != 32 { + return None; + } + let mut ret = [0u8; 32]; + ret.copy_from_slice(by); + Some(Self(ret)) + } } pub type UUID = FixedBytes32; -- cgit v1.2.3 From 831eb35763fdaeecb7b6d6aa13ebd78da14db04e Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 12 Mar 2021 21:52:19 +0100 Subject: cargo fmt --- src/api/signature.rs | 6 ++-- src/model/block.rs | 8 +++-- src/table/data.rs | 91 ++++++++++++++++++++++++++++------------------------ src/table/gc.rs | 69 +++++++++++++++++++++++++++------------ src/table/lib.rs | 2 +- src/table/schema.rs | 4 ++- src/table/sync.rs | 11 ++++--- src/table/table.rs | 2 +- 8 files changed, 120 insertions(+), 73 deletions(-) diff --git a/src/api/signature.rs b/src/api/signature.rs index 04d6666c..b3d61ff4 100644 --- a/src/api/signature.rs +++ b/src/api/signature.rs @@ -106,8 +106,10 @@ pub async fn check_signature( } else { let bytes = hex::decode(authorization.content_sha256) .ok_or_bad_request("Invalid content sha256 hash")?; - Some(Hash::try_from(&bytes[..]) - .ok_or(Error::BadRequest(format!("Invalid content sha256 hash")))?) + Some( + Hash::try_from(&bytes[..]) + .ok_or(Error::BadRequest(format!("Invalid content sha256 hash")))?, + ) }; Ok((key, content_sha256)) diff --git a/src/model/block.rs b/src/model/block.rs index 8b065c04..eccc2cbd 100644 --- a/src/model/block.rs +++ b/src/model/block.rs @@ -339,10 +339,14 @@ impl BlockManager { ); let put_block_message = self.read_block(hash).await?; - self.rpc_client.try_call_many( + self.rpc_client + .try_call_many( &need_nodes[..], put_block_message, - RequestStrategy::with_quorum(need_nodes.len()).with_timeout(BLOCK_RW_TIMEOUT)).await?; + RequestStrategy::with_quorum(need_nodes.len()) + .with_timeout(BLOCK_RW_TIMEOUT), + ) + .await?; } trace!( "Deleting block {:?}, offload finished ({} / {})", diff --git a/src/table/data.rs b/src/table/data.rs index 91607f7a..a491f877 100644 --- a/src/table/data.rs +++ b/src/table/data.rs @@ -129,31 +129,32 @@ where let update = self.decode_entry(update_bytes)?; let tree_key = self.tree_key(update.partition_key(), update.sort_key()); - let changed = (&self.store, &self.merkle_updater.todo).transaction(|(store, mkl_todo)| { - let (old_entry, new_entry) = match store.get(&tree_key)? { - Some(prev_bytes) => { - let old_entry = self - .decode_entry(&prev_bytes) + let changed = + (&self.store, &self.merkle_updater.todo).transaction(|(store, mkl_todo)| { + let (old_entry, new_entry) = match store.get(&tree_key)? { + Some(prev_bytes) => { + let old_entry = self + .decode_entry(&prev_bytes) + .map_err(sled::transaction::ConflictableTransactionError::Abort)?; + let mut new_entry = old_entry.clone(); + new_entry.merge(&update); + (Some(old_entry), new_entry) + } + None => (None, update.clone()), + }; + + if Some(&new_entry) != old_entry.as_ref() { + let new_bytes = rmp_to_vec_all_named(&new_entry) + .map_err(Error::RMPEncode) .map_err(sled::transaction::ConflictableTransactionError::Abort)?; - let mut new_entry = old_entry.clone(); - new_entry.merge(&update); - (Some(old_entry), new_entry) + let new_bytes_hash = blake2sum(&new_bytes[..]); + mkl_todo.insert(tree_key.clone(), new_bytes_hash.as_slice())?; + store.insert(tree_key.clone(), new_bytes)?; + Ok(Some((old_entry, new_entry, new_bytes_hash))) + } else { + Ok(None) } - None => (None, update.clone()), - }; - - if Some(&new_entry) != old_entry.as_ref() { - let new_bytes = rmp_to_vec_all_named(&new_entry) - .map_err(Error::RMPEncode) - .map_err(sled::transaction::ConflictableTransactionError::Abort)?; - let new_bytes_hash = blake2sum(&new_bytes[..]); - mkl_todo.insert(tree_key.clone(), new_bytes_hash.as_slice())?; - store.insert(tree_key.clone(), new_bytes)?; - Ok(Some((old_entry, new_entry, new_bytes_hash))) - } else { - Ok(None) - } - })?; + })?; if let Some((old_entry, new_entry, new_bytes_hash)) = changed { let is_tombstone = new_entry.is_tombstone(); @@ -168,16 +169,17 @@ where } pub(crate) fn delete_if_equal(self: &Arc, k: &[u8], v: &[u8]) -> Result { - let removed = (&self.store, &self.merkle_updater.todo).transaction(|(store, mkl_todo)| { - if let Some(cur_v) = store.get(k)? { - if cur_v == v { - store.remove(k)?; - mkl_todo.insert(k, vec![])?; - return Ok(true); + let removed = + (&self.store, &self.merkle_updater.todo).transaction(|(store, mkl_todo)| { + if let Some(cur_v) = store.get(k)? { + if cur_v == v { + store.remove(k)?; + mkl_todo.insert(k, vec![])?; + return Ok(true); + } } - } - Ok(false) - })?; + Ok(false) + })?; if removed { let old_entry = self.decode_entry(v)?; @@ -187,17 +189,22 @@ where Ok(removed) } - pub(crate) fn delete_if_equal_hash(self: &Arc, k: &[u8], vhash: Hash) -> Result { - let removed = (&self.store, &self.merkle_updater.todo).transaction(|(store, mkl_todo)| { - if let Some(cur_v) = store.get(k)? { - if blake2sum(&cur_v[..]) == vhash { - store.remove(k)?; - mkl_todo.insert(k, vec![])?; - return Ok(Some(cur_v)); + pub(crate) fn delete_if_equal_hash( + self: &Arc, + k: &[u8], + vhash: Hash, + ) -> Result { + let removed = + (&self.store, &self.merkle_updater.todo).transaction(|(store, mkl_todo)| { + if let Some(cur_v) = store.get(k)? { + if blake2sum(&cur_v[..]) == vhash { + store.remove(k)?; + mkl_todo.insert(k, vec![])?; + return Ok(Some(cur_v)); + } } - } - Ok(None) - })?; + Ok(None) + })?; if let Some(old_v) = removed { let old_entry = self.decode_entry(&old_v[..])?; diff --git a/src/table/gc.rs b/src/table/gc.rs index afc8a473..594044b8 100644 --- a/src/table/gc.rs +++ b/src/table/gc.rs @@ -1,6 +1,6 @@ +use std::collections::HashMap; use std::sync::Arc; use std::time::Duration; -use std::collections::HashMap; use serde::{Deserialize, Serialize}; use serde_bytes::ByteBuf; @@ -17,9 +17,9 @@ use garage_rpc::rpc_client::*; use garage_rpc::rpc_server::*; use crate::data::*; -use crate::table::*; -use crate::schema::*; use crate::replication::*; +use crate::schema::*; +use crate::table::*; const TABLE_GC_BATCH_SIZE: usize = 1024; const TABLE_GC_RPC_TIMEOUT: Duration = Duration::from_secs(30); @@ -99,7 +99,10 @@ where let vhash = Hash::try_from(&vhash[..]).unwrap(); - let v_opt = self.data.store.get(&k[..])? + let v_opt = self + .data + .store + .get(&k[..])? .filter(|v| blake2sum(&v[..]) == vhash); if let Some(v) = v_opt { @@ -113,7 +116,10 @@ where } for (k, vhash) in excluded { - let _ = self.data.gc_todo.compare_and_swap::<_, _, Vec>(k, Some(vhash), None)?; + let _ = self + .data + .gc_todo + .compare_and_swap::<_, _, Vec>(k, Some(vhash), None)?; } if entries.len() == 0 { @@ -136,18 +142,29 @@ where partitions.get_mut(&nodes).unwrap().push((k, vhash, v)); } - let resps = join_all(partitions.into_iter() - .map(|(nodes, items)| self.try_send_and_delete(nodes, items))).await; + let resps = join_all( + partitions + .into_iter() + .map(|(nodes, items)| self.try_send_and_delete(nodes, items)), + ) + .await; for resp in resps { if let Err(e) = resp { - warn!("({}) Unable to send and delete for GC: {}", self.data.name, e); + warn!( + "({}) Unable to send and delete for GC: {}", + self.data.name, e + ); } } Ok(true) } - async fn try_send_and_delete(&self, nodes: Vec, items: Vec<(ByteBuf, Hash, ByteBuf)>) -> Result<(), Error> { + async fn try_send_and_delete( + &self, + nodes: Vec, + items: Vec<(ByteBuf, Hash, ByteBuf)>, + ) -> Result<(), Error> { let n_items = items.len(); let mut updates = vec![]; @@ -157,21 +174,33 @@ where deletes.push((k, vhash)); } - self.rpc_client.try_call_many( - &nodes[..], - GcRPC::Update(updates), - RequestStrategy::with_quorum(nodes.len()).with_timeout(TABLE_GC_RPC_TIMEOUT)).await?; + self.rpc_client + .try_call_many( + &nodes[..], + GcRPC::Update(updates), + RequestStrategy::with_quorum(nodes.len()).with_timeout(TABLE_GC_RPC_TIMEOUT), + ) + .await?; + + info!( + "({}) GC: {} items successfully pushed, will try to delete.", + self.data.name, n_items + ); - info!("({}) GC: {} items successfully pushed, will try to delete.", self.data.name, n_items); + self.rpc_client + .try_call_many( + &nodes[..], + GcRPC::DeleteIfEqualHash(deletes.clone()), + RequestStrategy::with_quorum(nodes.len()).with_timeout(TABLE_GC_RPC_TIMEOUT), + ) + .await?; - self.rpc_client.try_call_many( - &nodes[..], - GcRPC::DeleteIfEqualHash(deletes.clone()), - RequestStrategy::with_quorum(nodes.len()).with_timeout(TABLE_GC_RPC_TIMEOUT)).await?; - for (k, vhash) in deletes { self.data.delete_if_equal_hash(&k[..], vhash)?; - let _ = self.data.gc_todo.compare_and_swap::<_, _, Vec>(k, Some(vhash), None)?; + let _ = self + .data + .gc_todo + .compare_and_swap::<_, _, Vec>(k, Some(vhash), None)?; } Ok(()) diff --git a/src/table/lib.rs b/src/table/lib.rs index 8a64ff0b..3b73163b 100644 --- a/src/table/lib.rs +++ b/src/table/lib.rs @@ -8,10 +8,10 @@ pub mod schema; pub mod util; pub mod data; +pub mod gc; pub mod merkle; pub mod replication; pub mod sync; -pub mod gc; pub mod table; pub use schema::*; diff --git a/src/table/schema.rs b/src/table/schema.rs index 5112ea15..4d754664 100644 --- a/src/table/schema.rs +++ b/src/table/schema.rs @@ -42,7 +42,9 @@ pub trait Entry: fn partition_key(&self) -> &P; fn sort_key(&self) -> &S; - fn is_tombstone(&self) -> bool { false } + fn is_tombstone(&self) -> bool { + false + } } pub trait TableSchema: Send + Sync { diff --git a/src/table/sync.rs b/src/table/sync.rs index aae65852..6c8792d2 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -347,10 +347,13 @@ where ) -> Result<(), Error> { let values = items.iter().map(|(_k, v)| v.clone()).collect::>(); - self.rpc_client.try_call_many( - &nodes[..], - SyncRPC::Items(values), - RequestStrategy::with_quorum(nodes.len()).with_timeout(TABLE_SYNC_RPC_TIMEOUT)).await?; + self.rpc_client + .try_call_many( + &nodes[..], + SyncRPC::Items(values), + RequestStrategy::with_quorum(nodes.len()).with_timeout(TABLE_SYNC_RPC_TIMEOUT), + ) + .await?; // All remote nodes have written those items, now we can delete them locally let mut not_removed = 0; diff --git a/src/table/table.rs b/src/table/table.rs index 7b0d9a24..2d3c5fe9 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -15,10 +15,10 @@ use garage_rpc::rpc_server::*; use crate::crdt::CRDT; use crate::data::*; +use crate::gc::*; use crate::replication::*; use crate::schema::*; use crate::sync::*; -use crate::gc::*; const TABLE_RPC_TIMEOUT: Duration = Duration::from_secs(10); -- cgit v1.2.3 From 9b118160a8b668b376d25ac16b097bce050f8b67 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 12 Mar 2021 22:06:56 +0100 Subject: Optim & refactor --- src/table/gc.rs | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/src/table/gc.rs b/src/table/gc.rs index 594044b8..5b7f1ee7 100644 --- a/src/table/gc.rs +++ b/src/table/gc.rs @@ -116,10 +116,7 @@ where } for (k, vhash) in excluded { - let _ = self - .data - .gc_todo - .compare_and_swap::<_, _, Vec>(k, Some(vhash), None)?; + self.todo_remove_if_equal(&k[..], vhash)?; } if entries.len() == 0 { @@ -197,15 +194,20 @@ where for (k, vhash) in deletes { self.data.delete_if_equal_hash(&k[..], vhash)?; - let _ = self - .data - .gc_todo - .compare_and_swap::<_, _, Vec>(k, Some(vhash), None)?; + self.todo_remove_if_equal(&k[..], vhash)?; } Ok(()) } + fn todo_remove_if_equal(&self, key: &[u8], vhash: Hash) -> Result<(), Error> { + let _ = self + .data + .gc_todo + .compare_and_swap::<_, _, Vec>(key, Some(vhash), None)?; + Ok(()) + } + // ---- RPC HANDLER ---- fn register_handler(self: &Arc, rpc_server: &mut RpcServer, path: String) { @@ -232,6 +234,7 @@ where GcRPC::DeleteIfEqualHash(items) => { for (key, vhash) in items.iter() { self.data.delete_if_equal_hash(&key[..], *vhash)?; + self.todo_remove_if_equal(&key[..], *vhash)?; } Ok(GcRPC::Ok) } -- cgit v1.2.3 From bdcbdd1cd854bd8125458af0ac20b8682f810967 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 15 Mar 2021 14:46:37 +0100 Subject: Fix list API bug --- src/api/s3_list.rs | 2 +- src/model/block.rs | 28 +++++++++++++++++----------- 2 files changed, 18 insertions(+), 12 deletions(-) diff --git a/src/api/s3_list.rs b/src/api/s3_list.rs index 98d774db..16d96a49 100644 --- a/src/api/s3_list.rs +++ b/src/api/s3_list.rs @@ -42,7 +42,7 @@ pub fn parse_list_objects_query( Ok(ListObjectsQuery { is_v2: params.get("list-type").map(|x| x == "2").unwrap_or(false), bucket: bucket.to_string(), - delimiter: params.get("delimiter").cloned(), + delimiter: params.get("delimiter").filter(|x| !x.is_empty()).cloned(), max_keys: params .get("max-keys") .map(|x| { diff --git a/src/model/block.rs b/src/model/block.rs index eccc2cbd..9426f683 100644 --- a/src/model/block.rs +++ b/src/model/block.rs @@ -165,7 +165,7 @@ impl BlockManager { Ok(f) => f, Err(e) => { // Not found but maybe we should have had it ?? - self.put_to_resync(hash, 0)?; + self.put_to_resync(hash, Duration::from_millis(0))?; return Err(Into::into(e)); } }; @@ -175,9 +175,11 @@ impl BlockManager { if data::blake2sum(&data[..]) != *hash { let _lock = self.data_dir_lock.lock().await; - warn!("Block {:?} is corrupted. Deleting and resyncing.", hash); - fs::remove_file(path).await?; - self.put_to_resync(&hash, 0)?; + warn!("Block {:?} is corrupted. Renaming to .corrupted and resyncing.", hash); + let mut path2 = path.clone(); + path2.set_extension(".corrupted"); + fs::rename(path, path2).await?; + self.put_to_resync(&hash, Duration::from_millis(0))?; return Err(Error::CorruptData(*hash)); } @@ -215,7 +217,7 @@ impl BlockManager { let old_rc = self.rc.get(&hash)?; self.rc.merge(&hash, vec![1])?; if old_rc.map(|x| u64_from_bytes(&x[..]) == 0).unwrap_or(true) { - self.put_to_resync(&hash, BLOCK_RW_TIMEOUT.as_millis() as u64)?; + self.put_to_resync(&hash, BLOCK_RW_TIMEOUT)?; } Ok(()) } @@ -223,13 +225,13 @@ impl BlockManager { pub fn block_decref(&self, hash: &Hash) -> Result<(), Error> { let new_rc = self.rc.merge(&hash, vec![0])?; if new_rc.map(|x| u64_from_bytes(&x[..]) == 0).unwrap_or(true) { - self.put_to_resync(&hash, 0)?; + self.put_to_resync(&hash, Duration::from_secs(0))?; } Ok(()) } - fn put_to_resync(&self, hash: &Hash, delay_millis: u64) -> Result<(), Error> { - let when = now_msec() + delay_millis; + fn put_to_resync(&self, hash: &Hash, delay: Duration) -> Result<(), Error> { + let when = now_msec() + delay.as_millis() as u64; trace!("Put resync_queue: {} {:?}", when, hash); let mut key = u64::to_be_bytes(when).to_vec(); key.extend(hash.as_ref()); @@ -252,7 +254,7 @@ impl BlockManager { if let Err(e) = self.resync_iter(&hash).await { warn!("Failed to resync block {:?}, retrying later: {}", hash, e); - self.put_to_resync(&hash, RESYNC_RETRY_TIMEOUT.as_millis() as u64)?; + self.put_to_resync(&hash, RESYNC_RETRY_TIMEOUT)?; n_failures += 1; if n_failures >= 10 { warn!("Too many resync failures, throttling."); @@ -281,6 +283,8 @@ impl BlockManager { } async fn resync_iter(&self, hash: &Hash) -> Result<(), Error> { + let lock = self.data_dir_lock.lock().await; + let path = self.block_path(hash); let exists = fs::metadata(&path).await.is_ok(); @@ -360,6 +364,8 @@ impl BlockManager { } if needed && !exists { + drop(lock); + // TODO find a way to not do this if they are sending it to us // Let's suppose this isn't an issue for now with the BLOCK_RW_TIMEOUT delay // between the RC being incremented and this part being called. @@ -420,7 +426,7 @@ impl BlockManager { } if !block_ref.deleted.get() { last_hash = Some(block_ref.block); - self.put_to_resync(&block_ref.block, 0)?; + self.put_to_resync(&block_ref.block, Duration::from_secs(0))?; } i += 1; if i & 0xFF == 0 && *must_exit.borrow() { @@ -464,7 +470,7 @@ impl BlockManager { }; let mut hash = [0u8; 32]; hash.copy_from_slice(&hash_bytes[..]); - self.put_to_resync(&hash.into(), 0)?; + self.put_to_resync(&hash.into(),Duration::from_secs(0))?; } if *must_exit.borrow() { -- cgit v1.2.3 From 097c339d981dba0420af17d30d1221181d8bf1d7 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 15 Mar 2021 15:26:29 +0100 Subject: Fix race condition --- src/api/s3_copy.rs | 56 ++++++++++++++++++++++++++++++++++++++-------------- src/api/s3_delete.rs | 8 ++++---- src/api/s3_put.rs | 43 ++++++++++++++++++++++------------------ src/garage/repair.rs | 20 +++---------------- src/model/block.rs | 10 +++++++--- 5 files changed, 79 insertions(+), 58 deletions(-) diff --git a/src/api/s3_copy.rs b/src/api/s3_copy.rs index c6c30095..8407faee 100644 --- a/src/api/s3_copy.rs +++ b/src/api/s3_copy.rs @@ -41,45 +41,64 @@ pub async fn handle_copy( }; let new_uuid = gen_uuid(); + let new_timestamp = now_msec(); let dest_object_version = ObjectVersion { uuid: new_uuid, - timestamp: now_msec(), + timestamp: new_timestamp, state: ObjectVersionState::Complete(source_last_state.clone()), }; + let dest_object = Object::new( + dest_bucket.to_string(), + dest_key.to_string(), + vec![dest_object_version], + ); - match &source_last_state { + match source_last_state { ObjectVersionData::DeleteMarker => { return Err(Error::NotFound); } ObjectVersionData::Inline(_meta, _bytes) => { - let dest_object = Object::new( - dest_bucket.to_string(), - dest_key.to_string(), - vec![dest_object_version], - ); garage.object_table.insert(&dest_object).await?; } - ObjectVersionData::FirstBlock(_meta, _first_block_hash) => { + ObjectVersionData::FirstBlock(meta, _first_block_hash) => { + // Get block list from source version let source_version = garage .version_table .get(&source_last_v.uuid, &EmptyKey) .await?; let source_version = source_version.ok_or(Error::NotFound)?; + // Write an "uploading" marker in Object table + // 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, + timestamp: new_timestamp, + state: ObjectVersionState::Uploading(meta.headers.clone()), + }; + let tmp_dest_object = Object::new( + dest_bucket.to_string(), + dest_key.to_string(), + vec![tmp_dest_object_version], + ); + garage.object_table.insert(&tmp_dest_object).await?; + + // Write version in the version table. Even with empty block list, + // 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.to_string(), dest_key.to_string(), false, ); + garage.version_table.insert(&dest_version).await?; + + // Fill in block list for version and insert block refs for (bk, bv) in source_version.blocks.items().iter() { dest_version.blocks.put(*bk, *bv); } - let dest_object = Object::new( - dest_bucket.to_string(), - dest_key.to_string(), - vec![dest_object_version], - ); let dest_block_refs = dest_version .blocks .items() @@ -91,14 +110,21 @@ pub async fn handle_copy( }) .collect::>(); futures::try_join!( - garage.object_table.insert(&dest_object), garage.version_table.insert(&dest_version), garage.block_ref_table.insert_many(&dest_block_refs[..]), )?; + + // Insert final object + // We do this last because otherwise there is a race condition in the case where + // the copy call has the same source and destination (this happens, rclone does + // it to update the modification timestamp for instance). If we did this concurrently + // 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. + garage.object_table.insert(&dest_object).await?; } } - let now = Utc::now(); + let now = Utc::now(); // FIXME use the unix timestamp from above let last_modified = now.to_rfc3339_opts(SecondsFormat::Secs, true); let mut xml = String::new(); writeln!(&mut xml, r#""#).unwrap(); diff --git a/src/api/s3_delete.rs b/src/api/s3_delete.rs index 4b6a2b18..7f752566 100644 --- a/src/api/s3_delete.rs +++ b/src/api/s3_delete.rs @@ -29,16 +29,16 @@ async fn handle_delete_internal( _ => true, }); - let mut must_delete = None; + let mut version_to_delete = None; let mut timestamp = now_msec(); for v in interesting_versions { - if v.timestamp + 1 > timestamp || must_delete.is_none() { - must_delete = Some(v.uuid); + if v.timestamp + 1 > timestamp || version_to_delete.is_none() { + version_to_delete = Some(v.uuid); } timestamp = std::cmp::max(timestamp, v.timestamp + 1); } - let deleted_version = must_delete.ok_or(Error::NotFound)?; + let deleted_version = version_to_delete.ok_or(Error::NotFound)?; let version_uuid = gen_uuid(); diff --git a/src/api/s3_put.rs b/src/api/s3_put.rs index c1774d6b..6f675e37 100644 --- a/src/api/s3_put.rs +++ b/src/api/s3_put.rs @@ -87,17 +87,21 @@ pub async fn handle_put( // that we are uploading something let mut object_version = ObjectVersion { uuid: version_uuid, - timestamp: now_msec(), + timestamp: version_timestamp, state: ObjectVersionState::Uploading(headers.clone()), }; let object = Object::new(bucket.into(), key.into(), vec![object_version.clone()]); garage.object_table.insert(&object).await?; // Initialize corresponding entry in version table + // 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.into(), key.into(), false); - let first_block_hash = blake2sum(&first_block[..]); + garage.version_table.insert(&version).await?; // Transfer data and verify checksum + let first_block_hash = blake2sum(&first_block[..]); let tx_result = read_and_put_blocks( &garage, &version, @@ -173,7 +177,7 @@ fn ensure_checksum_matches( } async fn read_and_put_blocks( - garage: &Arc, + garage: &Garage, version: &Version, part_number: u64, first_block: Vec, @@ -187,7 +191,7 @@ async fn read_and_put_blocks( let mut next_offset = first_block.len(); let mut put_curr_version_block = put_block_meta( - garage.clone(), + &garage, &version, part_number, 0, @@ -207,7 +211,7 @@ async fn read_and_put_blocks( let block_hash = blake2sum(&block[..]); let block_len = block.len(); put_curr_version_block = put_block_meta( - garage.clone(), + &garage, &version, part_number, next_offset as u64, @@ -231,14 +235,13 @@ async fn read_and_put_blocks( } async fn put_block_meta( - garage: Arc, + garage: &Garage, version: &Version, part_number: u64, offset: u64, hash: Hash, size: u64, ) -> Result<(), GarageError> { - // TODO: don't clone, restart from empty block list ?? let mut version = version.clone(); version.blocks.put( VersionBlockKey { @@ -316,6 +319,7 @@ pub async fn handle_create_multipart_upload( let version_uuid = gen_uuid(); let headers = get_headers(req)?; + // Create object in object table let object_version = ObjectVersion { uuid: version_uuid, timestamp: now_msec(), @@ -324,6 +328,14 @@ pub async fn handle_create_multipart_upload( let object = Object::new(bucket.to_string(), 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.into(), key.into(), false); + garage.version_table.insert(&version).await?; + + // Send success response let mut xml = String::new(); writeln!(&mut xml, r#""#).unwrap(); writeln!( @@ -450,14 +462,12 @@ pub async fn handle_complete_multipart_upload( )?; let object = object.ok_or(Error::BadRequest(format!("Object not found")))?; - let object_version = object + let mut 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::NotFound), - Some(x) => x.clone(), - }; + .find(|v| v.uuid == version_uuid && v.is_uploading()) + .cloned() + .ok_or(Error::BadRequest(format!("Version not found")))?; let version = version.ok_or(Error::BadRequest(format!("Version not found")))?; if version.blocks.len() == 0 { @@ -498,12 +508,7 @@ pub async fn handle_complete_multipart_upload( 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) - .fold(0, |x, y| x + y); + let total_size = version.blocks.items().iter().map(|x| x.1.size).sum(); // Write final object version object_version.state = ObjectVersionState::Complete(ObjectVersionData::FirstBlock( diff --git a/src/garage/repair.rs b/src/garage/repair.rs index 47fc1ae1..599c1965 100644 --- a/src/garage/repair.rs +++ b/src/garage/repair.rs @@ -82,13 +82,7 @@ impl Repair { .versions() .iter() .any(|x| x.uuid == version.uuid && x.state != ObjectVersionState::Aborted), - None => { - warn!( - "Repair versions: object for version {:?} not found, skipping.", - version - ); - continue; - } + None => false, }; if !version_exists { info!("Repair versions: marking version as deleted: {:?}", version); @@ -127,16 +121,8 @@ impl Repair { .version_table .get(&block_ref.version, &EmptyKey) .await?; - let ref_exists = match version { - Some(v) => !v.deleted.get(), - None => { - warn!( - "Block ref repair: version for block ref {:?} not found, skipping.", - block_ref - ); - continue; - } - }; + // The version might not exist if it has been GC'ed + let ref_exists = version.map(|v| !v.deleted.get()).unwrap_or(false); if !ref_exists { info!( "Repair block ref: marking block_ref as deleted: {:?}", diff --git a/src/model/block.rs b/src/model/block.rs index 9426f683..5934f20c 100644 --- a/src/model/block.rs +++ b/src/model/block.rs @@ -28,6 +28,7 @@ use crate::garage::Garage; pub const INLINE_THRESHOLD: usize = 3072; const BLOCK_RW_TIMEOUT: Duration = Duration::from_secs(42); +const BLOCK_GC_TIMEOUT: Duration = Duration::from_secs(60); const NEED_BLOCK_QUERY_TIMEOUT: Duration = Duration::from_secs(5); const RESYNC_RETRY_TIMEOUT: Duration = Duration::from_secs(10); @@ -175,7 +176,10 @@ impl BlockManager { if data::blake2sum(&data[..]) != *hash { let _lock = self.data_dir_lock.lock().await; - warn!("Block {:?} is corrupted. Renaming to .corrupted and resyncing.", hash); + warn!( + "Block {:?} is corrupted. Renaming to .corrupted and resyncing.", + hash + ); let mut path2 = path.clone(); path2.set_extension(".corrupted"); fs::rename(path, path2).await?; @@ -225,7 +229,7 @@ impl BlockManager { pub fn block_decref(&self, hash: &Hash) -> Result<(), Error> { let new_rc = self.rc.merge(&hash, vec![0])?; if new_rc.map(|x| u64_from_bytes(&x[..]) == 0).unwrap_or(true) { - self.put_to_resync(&hash, Duration::from_secs(0))?; + self.put_to_resync(&hash, BLOCK_GC_TIMEOUT)?; } Ok(()) } @@ -470,7 +474,7 @@ impl BlockManager { }; let mut hash = [0u8; 32]; hash.copy_from_slice(&hash_bytes[..]); - self.put_to_resync(&hash.into(),Duration::from_secs(0))?; + self.put_to_resync(&hash.into(), Duration::from_secs(0))?; } if *must_exit.borrow() { -- cgit v1.2.3 From 3bf2df622a070fe8f233bec6d60bd5cca995fbfc Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 15 Mar 2021 16:21:41 +0100 Subject: Time and metadata improvements --- Cargo.lock | 1 + src/api/api_server.rs | 5 +++- src/api/s3_copy.rs | 69 ++++++++++++++++++++++++++++++++++------------- src/api/s3_delete.rs | 1 + src/api/s3_list.rs | 6 ++--- src/api/s3_put.rs | 26 +++++++++++++++--- src/garage/cli.rs | 2 +- src/model/block.rs | 4 +-- src/rpc/membership.rs | 1 + src/table/crdt/lww.rs | 2 +- src/table/crdt/lww_map.rs | 2 +- src/table/gc.rs | 10 ++++--- src/util/Cargo.toml | 1 + src/util/data.rs | 8 ------ src/util/lib.rs | 1 + src/util/time.rs | 16 +++++++++++ 16 files changed, 111 insertions(+), 44 deletions(-) create mode 100644 src/util/time.rs diff --git a/Cargo.lock b/Cargo.lock index b0633e21..9fb368e5 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -608,6 +608,7 @@ name = "garage_util" version = "0.1.1" dependencies = [ "blake2", + "chrono", "err-derive", "fasthash", "futures", diff --git a/src/api/api_server.rs b/src/api/api_server.rs index c6b1d483..bc98686d 100644 --- a/src/api/api_server.rs +++ b/src/api/api_server.rs @@ -137,7 +137,10 @@ async fn handler_inner(garage: Arc, req: Request) -> Result, + req: &Request, dest_bucket: &str, dest_key: &str, source_bucket: &str, @@ -42,25 +44,44 @@ pub async fn handle_copy( let new_uuid = gen_uuid(); let new_timestamp = now_msec(); - let dest_object_version = ObjectVersion { - uuid: new_uuid, - timestamp: new_timestamp, - state: ObjectVersionState::Complete(source_last_state.clone()), - }; - let dest_object = Object::new( - dest_bucket.to_string(), - dest_key.to_string(), - vec![dest_object_version], - ); - match source_last_state { + // Implement x-amz-metadata-directive: REPLACE + let old_meta = match source_last_state { ObjectVersionData::DeleteMarker => { return Err(Error::NotFound); } - ObjectVersionData::Inline(_meta, _bytes) => { + ObjectVersionData::Inline(meta, _bytes) => meta, + ObjectVersionData::FirstBlock(meta, _fbh) => meta, + }; + let new_meta = match req.headers().get("x-amz-metadata-directive") { + Some(v) if v == hyper::header::HeaderValue::from_static("REPLACE") => ObjectVersionMeta { + headers: get_headers(req)?, + size: old_meta.size, + etag: old_meta.etag.clone(), + }, + _ => old_meta.clone(), + }; + + // Save object copy + match source_last_state { + ObjectVersionData::DeleteMarker => unreachable!(), + ObjectVersionData::Inline(_meta, bytes) => { + let dest_object_version = ObjectVersion { + uuid: new_uuid, + timestamp: new_timestamp, + state: ObjectVersionState::Complete(ObjectVersionData::Inline( + new_meta, + bytes.clone(), + )), + }; + let dest_object = Object::new( + dest_bucket.to_string(), + dest_key.to_string(), + vec![dest_object_version], + ); garage.object_table.insert(&dest_object).await?; } - ObjectVersionData::FirstBlock(meta, _first_block_hash) => { + ObjectVersionData::FirstBlock(_meta, first_block_hash) => { // Get block list from source version let source_version = garage .version_table @@ -74,7 +95,7 @@ pub async fn handle_copy( let tmp_dest_object_version = ObjectVersion { uuid: new_uuid, timestamp: new_timestamp, - state: ObjectVersionState::Uploading(meta.headers.clone()), + state: ObjectVersionState::Uploading(new_meta.headers.clone()), }; let tmp_dest_object = Object::new( dest_bucket.to_string(), @@ -120,12 +141,24 @@ pub async fn handle_copy( // it to update the modification timestamp for instance). If we did this concurrently // 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, + timestamp: new_timestamp, + state: ObjectVersionState::Complete(ObjectVersionData::FirstBlock( + new_meta, + *first_block_hash, + )), + }; + let dest_object = Object::new( + dest_bucket.to_string(), + dest_key.to_string(), + vec![dest_object_version], + ); garage.object_table.insert(&dest_object).await?; } } - let now = Utc::now(); // FIXME use the unix timestamp from above - let last_modified = now.to_rfc3339_opts(SecondsFormat::Secs, true); + let last_modified = msec_to_rfc3339(new_timestamp); let mut xml = String::new(); writeln!(&mut xml, r#""#).unwrap(); writeln!(&mut xml, r#""#).unwrap(); diff --git a/src/api/s3_delete.rs b/src/api/s3_delete.rs index 7f752566..bb42d90c 100644 --- a/src/api/s3_delete.rs +++ b/src/api/s3_delete.rs @@ -4,6 +4,7 @@ use std::sync::Arc; use hyper::{Body, Request, Response}; use garage_util::data::*; +use garage_util::time::*; use garage_model::garage::Garage; use garage_model::object_table::*; diff --git a/src/api/s3_list.rs b/src/api/s3_list.rs index 16d96a49..4d6c32bc 100644 --- a/src/api/s3_list.rs +++ b/src/api/s3_list.rs @@ -2,10 +2,10 @@ use std::collections::{BTreeMap, BTreeSet, HashMap}; use std::fmt::Write; use std::sync::Arc; -use chrono::{DateTime, NaiveDateTime, SecondsFormat, Utc}; use hyper::{Body, Response}; use garage_util::error::Error as GarageError; +use garage_util::time::*; use garage_model::garage::Garage; use garage_model::object_table::*; @@ -247,9 +247,7 @@ pub async fn handle_list( } for (key, info) in result_keys.iter() { - let last_modif = NaiveDateTime::from_timestamp(info.last_modified as i64 / 1000, 0); - let last_modif = DateTime::::from_utc(last_modif, Utc); - let last_modif = last_modif.to_rfc3339_opts(SecondsFormat::Millis, true); + let last_modif = msec_to_rfc3339(info.last_modified); writeln!(&mut xml, "\t").unwrap(); writeln!( &mut xml, diff --git a/src/api/s3_put.rs b/src/api/s3_put.rs index 6f675e37..ea3664bd 100644 --- a/src/api/s3_put.rs +++ b/src/api/s3_put.rs @@ -10,6 +10,7 @@ use sha2::{Digest as Sha256Digest, Sha256}; use garage_table::*; use garage_util::data::*; use garage_util::error::Error as GarageError; +use garage_util::time::*; use garage_model::block::INLINE_THRESHOLD; use garage_model::block_ref_table::*; @@ -583,17 +584,19 @@ fn get_mime_type(req: &Request) -> Result { .to_string()) } -fn get_headers(req: &Request) -> Result { +pub(crate) fn get_headers(req: &Request) -> Result { let content_type = get_mime_type(req)?; - let other_headers = vec![ + let mut other = BTreeMap::new(); + + // Preserve standard headers + let standard_header = vec![ hyper::header::CACHE_CONTROL, hyper::header::CONTENT_DISPOSITION, hyper::header::CONTENT_ENCODING, hyper::header::CONTENT_LANGUAGE, hyper::header::EXPIRES, ]; - let mut other = BTreeMap::new(); - for h in other_headers.iter() { + for h in standard_header.iter() { if let Some(v) = req.headers().get(h) { match v.to_str() { Ok(v_str) => { @@ -605,6 +608,21 @@ fn get_headers(req: &Request) -> Result { } } } + + // Preserve x-amz-meta- headers + for (k, v) in req.headers().iter() { + if k.as_str().starts_with("x-amz-meta-") { + match v.to_str() { + Ok(v_str) => { + other.insert(k.to_string(), v_str.to_string()); + } + Err(e) => { + warn!("Discarding header {}, error in .to_str(): {}", k, e); + } + } + } + } + Ok(ObjectVersionHeaders { content_type, other, diff --git a/src/garage/cli.rs b/src/garage/cli.rs index 56f03c8b..b5c91ffc 100644 --- a/src/garage/cli.rs +++ b/src/garage/cli.rs @@ -5,8 +5,8 @@ use std::path::PathBuf; use serde::{Deserialize, Serialize}; use structopt::StructOpt; -use garage_util::data::*; use garage_util::error::Error; +use garage_util::time::*; use garage_rpc::membership::*; use garage_rpc::ring::*; diff --git a/src/model/block.rs b/src/model/block.rs index 5934f20c..36ad867a 100644 --- a/src/model/block.rs +++ b/src/model/block.rs @@ -11,9 +11,9 @@ use tokio::fs; use tokio::prelude::*; use tokio::sync::{watch, Mutex, Notify}; -use garage_util::data; use garage_util::data::*; use garage_util::error::Error; +use garage_util::time::*; use garage_rpc::membership::System; use garage_rpc::rpc_client::*; @@ -174,7 +174,7 @@ impl BlockManager { f.read_to_end(&mut data).await?; drop(f); - if data::blake2sum(&data[..]) != *hash { + if blake2sum(&data[..]) != *hash { let _lock = self.data_dir_lock.lock().await; warn!( "Block {:?} is corrupted. Renaming to .corrupted and resyncing.", diff --git a/src/rpc/membership.rs b/src/rpc/membership.rs index e1dc297e..6636e50b 100644 --- a/src/rpc/membership.rs +++ b/src/rpc/membership.rs @@ -18,6 +18,7 @@ use tokio::sync::Mutex; use garage_util::background::BackgroundRunner; use garage_util::data::*; use garage_util::error::Error; +use garage_util::time::*; use crate::consul::get_consul_nodes; use crate::ring::*; diff --git a/src/table/crdt/lww.rs b/src/table/crdt/lww.rs index 9a3ab671..25ecdb07 100644 --- a/src/table/crdt/lww.rs +++ b/src/table/crdt/lww.rs @@ -1,6 +1,6 @@ use serde::{Deserialize, Serialize}; -use garage_util::data::now_msec; +use garage_util::time::now_msec; use crate::crdt::crdt::*; diff --git a/src/table/crdt/lww_map.rs b/src/table/crdt/lww_map.rs index bd40f368..7b372191 100644 --- a/src/table/crdt/lww_map.rs +++ b/src/table/crdt/lww_map.rs @@ -1,6 +1,6 @@ use serde::{Deserialize, Serialize}; -use garage_util::data::now_msec; +use garage_util::time::now_msec; use crate::crdt::crdt::*; diff --git a/src/table/gc.rs b/src/table/gc.rs index 5b7f1ee7..c13c8234 100644 --- a/src/table/gc.rs +++ b/src/table/gc.rs @@ -75,17 +75,19 @@ where match self.gc_loop_iter().await { Ok(true) => { // Stuff was done, loop imediately + continue; } Ok(false) => { - select! { - _ = tokio::time::delay_for(Duration::from_secs(10)).fuse() => (), - _ = must_exit.recv().fuse() => (), - } + // Nothing was done, sleep for some time (below) } Err(e) => { warn!("({}) Error doing GC: {}", self.data.name, e); } } + select! { + _ = tokio::time::delay_for(Duration::from_secs(10)).fuse() => (), + _ = must_exit.recv().fuse() => (), + } } Ok(()) } diff --git a/src/util/Cargo.toml b/src/util/Cargo.toml index 35130c96..7bb7cb31 100644 --- a/src/util/Cargo.toml +++ b/src/util/Cargo.toml @@ -27,6 +27,7 @@ toml = "0.5" rmp-serde = "0.14.3" serde = { version = "1.0", default-features = false, features = ["derive", "rc"] } serde_json = "1.0" +chrono = "0.4" futures = "0.3" futures-util = "0.3" diff --git a/src/util/data.rs b/src/util/data.rs index 0dbd6df4..591b7605 100644 --- a/src/util/data.rs +++ b/src/util/data.rs @@ -2,7 +2,6 @@ use rand::Rng; use serde::de::{self, Visitor}; use serde::{Deserialize, Deserializer, Serialize, Serializer}; use std::fmt; -use std::time::{SystemTime, UNIX_EPOCH}; #[derive(Default, PartialOrd, Ord, Clone, Hash, PartialEq, Copy)] pub struct FixedBytes32([u8; 32]); @@ -119,13 +118,6 @@ pub fn gen_uuid() -> UUID { rand::thread_rng().gen::<[u8; 32]>().into() } -pub fn now_msec() -> u64 { - SystemTime::now() - .duration_since(UNIX_EPOCH) - .expect("Fix your clock :o") - .as_millis() as u64 -} - // RMP serialization with names of fields and variants pub fn rmp_to_vec_all_named(val: &T) -> Result, rmp_serde::encode::Error> diff --git a/src/util/lib.rs b/src/util/lib.rs index 0bf09bf6..e544a872 100644 --- a/src/util/lib.rs +++ b/src/util/lib.rs @@ -5,3 +5,4 @@ pub mod background; pub mod config; pub mod data; pub mod error; +pub mod time; diff --git a/src/util/time.rs b/src/util/time.rs new file mode 100644 index 00000000..148860e0 --- /dev/null +++ b/src/util/time.rs @@ -0,0 +1,16 @@ +use chrono::{SecondsFormat, TimeZone, Utc}; +use std::time::{SystemTime, UNIX_EPOCH}; + +pub fn now_msec() -> u64 { + SystemTime::now() + .duration_since(UNIX_EPOCH) + .expect("Fix your clock :o") + .as_millis() as u64 +} + +pub fn msec_to_rfc3339(msecs: u64) -> String { + let secs = msecs as i64 / 1000; + let nanos = (msecs as i64 % 1000) as u32 * 1_000_000; + let timestamp = Utc.timestamp(secs, nanos); + timestamp.to_rfc3339_opts(SecondsFormat::Secs, true) +} -- cgit v1.2.3 From 0290afe1f8eafabf71695d677807e07658d078ab Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 15 Mar 2021 18:27:26 +0100 Subject: Make block rc code more understandable --- src/garage/admin_rpc.rs | 18 +++++++++++++++++ src/model/block.rs | 54 +++++++++++++++++++------------------------------ src/table/merkle.rs | 2 +- 3 files changed, 40 insertions(+), 34 deletions(-) diff --git a/src/garage/admin_rpc.rs b/src/garage/admin_rpc.rs index a75c9013..512e00dd 100644 --- a/src/garage/admin_rpc.rs +++ b/src/garage/admin_rpc.rs @@ -423,6 +423,12 @@ impl AdminRpcHandler { self.gather_table_stats(&mut ret, &self.garage.block_ref_table, &opt)?; writeln!(&mut ret, "\nBlock manager stats:").unwrap(); + writeln!( + &mut ret, + " number of blocks: {}", + self.garage.block_manager.rc.len() + ) + .unwrap(); writeln!( &mut ret, " resync queue length: {}", @@ -451,6 +457,18 @@ impl AdminRpcHandler { t.data.merkle_updater.todo.len() ) .unwrap(); + writeln!( + to, + " Merkle tree size: {}", + t.data.merkle_updater.merkle_tree.len() + ) + .unwrap(); + writeln!( + to, + " GC todo queue length: {}", + t.data.gc_todo.len() + ) + .unwrap(); Ok(()) } } diff --git a/src/model/block.rs b/src/model/block.rs index 36ad867a..8523474a 100644 --- a/src/model/block.rs +++ b/src/model/block.rs @@ -77,7 +77,6 @@ impl BlockManager { let rc = db .open_tree("block_local_rc") .expect("Unable to open block_local_rc tree"); - rc.set_merge_operator(rc_merge); let resync_queue = db .open_tree("block_local_resync_queue") @@ -194,7 +193,7 @@ impl BlockManager { let needed = self .rc .get(hash.as_ref())? - .map(|x| u64_from_bytes(x.as_ref()) > 0) + .map(|x| u64_from_be_bytes(x) > 0) .unwrap_or(false); if needed { let path = self.block_path(hash); @@ -218,17 +217,27 @@ impl BlockManager { } pub fn block_incref(&self, hash: &Hash) -> Result<(), Error> { - let old_rc = self.rc.get(&hash)?; - self.rc.merge(&hash, vec![1])?; - if old_rc.map(|x| u64_from_bytes(&x[..]) == 0).unwrap_or(true) { + let old_rc = self.rc.fetch_and_update(&hash, |old| { + let old_v = old.map(u64_from_be_bytes).unwrap_or(0); + Some(u64::to_be_bytes(old_v + 1).to_vec()) + })?; + let old_rc = old_rc.map(u64_from_be_bytes).unwrap_or(0); + if old_rc == 0 { self.put_to_resync(&hash, BLOCK_RW_TIMEOUT)?; } Ok(()) } pub fn block_decref(&self, hash: &Hash) -> Result<(), Error> { - let new_rc = self.rc.merge(&hash, vec![0])?; - if new_rc.map(|x| u64_from_bytes(&x[..]) == 0).unwrap_or(true) { + let new_rc = self.rc.update_and_fetch(&hash, |old| { + let old_v = old.map(u64_from_be_bytes).unwrap_or(0); + if old_v > 1 { + Some(u64::to_be_bytes(old_v - 1).to_vec()) + } else { + None + } + })?; + if new_rc.is_none() { self.put_to_resync(&hash, BLOCK_GC_TIMEOUT)?; } Ok(()) @@ -251,7 +260,7 @@ impl BlockManager { let mut n_failures = 0usize; while !*must_exit.borrow() { if let Some((time_bytes, hash_bytes)) = self.resync_queue.pop_min()? { - let time_msec = u64_from_bytes(&time_bytes[0..8]); + let time_msec = u64_from_be_bytes(&time_bytes[0..8]); let now = now_msec(); if now >= time_msec { let hash = Hash::try_from(&hash_bytes[..]).unwrap(); @@ -295,7 +304,7 @@ impl BlockManager { let needed = self .rc .get(hash.as_ref())? - .map(|x| u64_from_bytes(x.as_ref()) > 0) + .map(|x| u64_from_be_bytes(x) > 0) .unwrap_or(false); if exists != needed { @@ -487,30 +496,9 @@ impl BlockManager { } } -fn u64_from_bytes(bytes: &[u8]) -> u64 { - assert!(bytes.len() == 8); +fn u64_from_be_bytes>(bytes: T) -> u64 { + assert!(bytes.as_ref().len() == 8); let mut x8 = [0u8; 8]; - x8.copy_from_slice(bytes); + x8.copy_from_slice(bytes.as_ref()); u64::from_be_bytes(x8) } - -fn rc_merge(_key: &[u8], old: Option<&[u8]>, new: &[u8]) -> Option> { - let old = old.map(u64_from_bytes).unwrap_or(0); - assert!(new.len() == 1); - let new = match new[0] { - 0 => { - if old > 0 { - old - 1 - } else { - 0 - } - } - 1 => old + 1, - _ => unreachable!(), - }; - if new == 0 { - None - } else { - Some(u64::to_be_bytes(new).to_vec()) - } -} diff --git a/src/table/merkle.rs b/src/table/merkle.rs index 7a0adba1..a917a028 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -46,7 +46,7 @@ pub struct MerkleUpdater { // Content of the merkle tree: items where // - key = .bytes() for MerkleNodeKey // - value = serialization of a MerkleNode, assumed to be MerkleNode::empty if not found - pub(crate) merkle_tree: sled::Tree, + pub merkle_tree: sled::Tree, empty_node_hash: Hash, } -- cgit v1.2.3 From 537f652fec479c7c5676bba14c23ea6634613122 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 15 Mar 2021 18:40:27 +0100 Subject: Tiny things --- src/rpc/rpc_server.rs | 2 +- src/table/merkle.rs | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/rpc/rpc_server.rs b/src/rpc/rpc_server.rs index 1c6bc8d2..4d14b790 100644 --- a/src/rpc/rpc_server.rs +++ b/src/rpc/rpc_server.rs @@ -51,7 +51,7 @@ where trace!( "Request message: {}", - serde_json::to_string(&msg).unwrap_or("".into()) + serde_json::to_string(&msg).unwrap_or("".into()).chars().take(100).collect::() ); match handler(msg, sockaddr).await { diff --git a/src/table/merkle.rs b/src/table/merkle.rs index a917a028..f60be8a8 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -56,6 +56,7 @@ pub struct MerkleNodeKey { pub partition: MerklePartition, // prefix: a prefix for the hash of full keys, i.e. hash(hash(partition_key)+sort_key) + #[serde(with = "serde_bytes")] pub prefix: Vec, } -- cgit v1.2.3 From 5ee1d956b6c7dd847a304ef524253b2e067e1245 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 15 Mar 2021 19:14:26 +0100 Subject: Allow manipulation of keys by their shorthand in the CLI --- src/garage/admin_rpc.rs | 48 +++++++++++++++++++++++++++--------------------- src/garage/cli.rs | 16 ++++++++-------- src/model/key_table.rs | 15 +++++++++++++-- 3 files changed, 48 insertions(+), 31 deletions(-) diff --git a/src/garage/admin_rpc.rs b/src/garage/admin_rpc.rs index 512e00dd..07c1b582 100644 --- a/src/garage/admin_rpc.rs +++ b/src/garage/admin_rpc.rs @@ -122,7 +122,7 @@ impl AdminRpcHandler { for (key_id, _, _) in bucket.authorized_keys() { if let Some(key) = self.garage.key_table.get(&EmptyKey, key_id).await? { if !key.deleted.get() { - self.update_key_bucket(key, &bucket.name, false, false) + self.update_key_bucket(&key, &bucket.name, false, false) .await?; } } else { @@ -134,31 +134,31 @@ impl AdminRpcHandler { Ok(AdminRPC::Ok(format!("Bucket {} was deleted.", query.name))) } BucketOperation::Allow(query) => { - let key = self.get_existing_key(&query.key_id).await?; + let key = self.get_existing_key(&query.key_pattern).await?; let bucket = self.get_existing_bucket(&query.bucket).await?; let allow_read = query.read || key.allow_read(&query.bucket); let allow_write = query.write || key.allow_write(&query.bucket); - self.update_key_bucket(key, &query.bucket, allow_read, allow_write) + self.update_key_bucket(&key, &query.bucket, allow_read, allow_write) .await?; - self.update_bucket_key(bucket, &query.key_id, allow_read, allow_write) + self.update_bucket_key(bucket, &key.key_id, allow_read, allow_write) .await?; Ok(AdminRPC::Ok(format!( "New permissions for {} on {}: read {}, write {}.", - &query.key_id, &query.bucket, allow_read, allow_write + &key.key_id, &query.bucket, allow_read, allow_write ))) } BucketOperation::Deny(query) => { - let key = self.get_existing_key(&query.key_id).await?; + let key = self.get_existing_key(&query.key_pattern).await?; let bucket = self.get_existing_bucket(&query.bucket).await?; let allow_read = !query.read && key.allow_read(&query.bucket); let allow_write = !query.write && key.allow_write(&query.bucket); - self.update_key_bucket(key, &query.bucket, allow_read, allow_write) + self.update_key_bucket(&key, &query.bucket, allow_read, allow_write) .await?; - self.update_bucket_key(bucket, &query.key_id, allow_read, allow_write) + self.update_bucket_key(bucket, &key.key_id, allow_read, allow_write) .await?; Ok(AdminRPC::Ok(format!( "New permissions for {} on {}: read {}, write {}.", - &query.key_id, &query.bucket, allow_read, allow_write + &key.key_id, &query.bucket, allow_read, allow_write ))) } BucketOperation::Website(query) => { @@ -193,7 +193,7 @@ impl AdminRpcHandler { let key_ids = self .garage .key_table - .get_range(&EmptyKey, None, Some(DeletedFilter::NotDeleted), 10000) + .get_range(&EmptyKey, None, Some(KeyFilter::Deleted(DeletedFilter::NotDeleted)), 10000) .await? .iter() .map(|k| (k.key_id.to_string(), k.name.get().clone())) @@ -201,7 +201,7 @@ impl AdminRpcHandler { Ok(AdminRPC::KeyList(key_ids)) } KeyOperation::Info(query) => { - let key = self.get_existing_key(&query.key_id).await?; + let key = self.get_existing_key(&query.key_pattern).await?; Ok(AdminRPC::KeyInfo(key)) } KeyOperation::New(query) => { @@ -210,13 +210,13 @@ impl AdminRpcHandler { Ok(AdminRPC::KeyInfo(key)) } KeyOperation::Rename(query) => { - let mut key = self.get_existing_key(&query.key_id).await?; + let mut key = self.get_existing_key(&query.key_pattern).await?; key.name.update(query.new_name); self.garage.key_table.insert(&key).await?; Ok(AdminRPC::KeyInfo(key)) } KeyOperation::Delete(query) => { - let key = self.get_existing_key(&query.key_id).await?; + let key = self.get_existing_key(&query.key_pattern).await?; if !query.yes { return Err(Error::BadRPC(format!( "Add --yes flag to really perform this operation" @@ -233,11 +233,11 @@ impl AdminRpcHandler { return Err(Error::Message(format!("Bucket not found: {}", ab_name))); } } - let del_key = Key::delete(key.key_id); + let del_key = Key::delete(key.key_id.to_string()); self.garage.key_table.insert(&del_key).await?; Ok(AdminRPC::Ok(format!( "Key {} was deleted successfully.", - query.key_id + key.key_id ))) } } @@ -256,14 +256,19 @@ impl AdminRpcHandler { )))) } - async fn get_existing_key(&self, id: &String) -> Result { - self.garage + async fn get_existing_key(&self, pattern: &str) -> Result { + let candidates = self.garage .key_table - .get(&EmptyKey, id) + .get_range(&EmptyKey, None, Some(KeyFilter::Matches(pattern.to_string())), 10) .await? + .into_iter() .filter(|k| !k.deleted.get()) - .map(Ok) - .unwrap_or(Err(Error::BadRPC(format!("Key {} does not exist", id)))) + .collect::>(); + if candidates.len() != 1 { + Err(Error::Message(format!("{} matching keys", candidates.len()))) + } else { + Ok(candidates.into_iter().next().unwrap()) + } } /// Update **bucket table** to inform of the new linked key @@ -296,11 +301,12 @@ impl AdminRpcHandler { /// Update **key table** to inform of the new linked bucket async fn update_key_bucket( &self, - mut key: Key, + key: &Key, bucket: &String, allow_read: bool, allow_write: bool, ) -> Result<(), Error> { + let mut key = key.clone(); let old_map = key.authorized_buckets.take_and_clear(); key.authorized_buckets.merge(&old_map.update_mutator( bucket.clone(), diff --git a/src/garage/cli.rs b/src/garage/cli.rs index b5c91ffc..e74f59a2 100644 --- a/src/garage/cli.rs +++ b/src/garage/cli.rs @@ -157,9 +157,9 @@ pub struct DeleteBucketOpt { #[derive(Serialize, Deserialize, StructOpt, Debug)] pub struct PermBucketOpt { - /// Access key ID + /// Access key name or ID #[structopt(long = "key")] - pub key_id: String, + pub key_pattern: String, /// Allow/deny read operations #[structopt(long = "read")] @@ -198,8 +198,8 @@ pub enum KeyOperation { #[derive(Serialize, Deserialize, StructOpt, Debug)] pub struct KeyOpt { - /// ID of the key - pub key_id: String, + /// ID or name of the key + pub key_pattern: String, } #[derive(Serialize, Deserialize, StructOpt, Debug)] @@ -211,8 +211,8 @@ pub struct KeyNewOpt { #[derive(Serialize, Deserialize, StructOpt, Debug)] pub struct KeyRenameOpt { - /// ID of the key - pub key_id: String, + /// ID or name of the key + pub key_pattern: String, /// New name of the key pub new_name: String, @@ -220,8 +220,8 @@ pub struct KeyRenameOpt { #[derive(Serialize, Deserialize, StructOpt, Debug)] pub struct KeyDeleteOpt { - /// ID of the key - pub key_id: String, + /// ID or name of the key + pub key_pattern: String, /// Confirm deletion #[structopt(long = "yes")] diff --git a/src/model/key_table.rs b/src/model/key_table.rs index b4ab65b6..ce5888ce 100644 --- a/src/model/key_table.rs +++ b/src/model/key_table.rs @@ -92,13 +92,24 @@ impl CRDT for Key { pub struct KeyTable; +#[derive(Clone, Debug, Serialize, Deserialize)] +pub enum KeyFilter { + Deleted(DeletedFilter), + Matches(String), +} + impl TableSchema for KeyTable { type P = EmptyKey; type S = String; type E = Key; - type Filter = DeletedFilter; + type Filter = KeyFilter; fn matches_filter(entry: &Self::E, filter: &Self::Filter) -> bool { - filter.apply(entry.deleted.get()) + match filter { + KeyFilter::Deleted(df) => df.apply(entry.deleted.get()), + KeyFilter::Matches(pat) => { + entry.key_id.starts_with(pat) || entry.name.get().to_lowercase() == pat.to_lowercase() + } + } } } -- cgit v1.2.3 From 642bed601f9b5f8a0cf3e99223235e6b3eb33c0a Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 15 Mar 2021 19:16:42 +0100 Subject: Make it case-insensitive --- src/model/key_table.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/model/key_table.rs b/src/model/key_table.rs index ce5888ce..88d7b4ff 100644 --- a/src/model/key_table.rs +++ b/src/model/key_table.rs @@ -108,7 +108,8 @@ impl TableSchema for KeyTable { match filter { KeyFilter::Deleted(df) => df.apply(entry.deleted.get()), KeyFilter::Matches(pat) => { - entry.key_id.starts_with(pat) || entry.name.get().to_lowercase() == pat.to_lowercase() + let pat = pat.to_lowercase(); + entry.key_id.to_lowercase().starts_with(&pat) || entry.name.get().to_lowercase() == pat } } } -- cgit v1.2.3 From 667e4e72a8e64a094d57ceeb6442cef08f1ef0e1 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 15 Mar 2021 19:51:16 +0100 Subject: Small fixes --- src/api/s3_delete.rs | 2 +- src/garage/admin_rpc.rs | 10 +++++----- src/model/block.rs | 23 ++++++++++++++++------- src/table/data.rs | 6 +++++- src/table/merkle.rs | 12 ++++++++++-- 5 files changed, 37 insertions(+), 16 deletions(-) diff --git a/src/api/s3_delete.rs b/src/api/s3_delete.rs index bb42d90c..6abbfc48 100644 --- a/src/api/s3_delete.rs +++ b/src/api/s3_delete.rs @@ -48,7 +48,7 @@ async fn handle_delete_internal( key.into(), vec![ObjectVersion { uuid: version_uuid, - timestamp: now_msec(), + timestamp, state: ObjectVersionState::Complete(ObjectVersionData::DeleteMarker), }], ); diff --git a/src/garage/admin_rpc.rs b/src/garage/admin_rpc.rs index 07c1b582..40674e75 100644 --- a/src/garage/admin_rpc.rs +++ b/src/garage/admin_rpc.rs @@ -432,13 +432,13 @@ impl AdminRpcHandler { writeln!( &mut ret, " number of blocks: {}", - self.garage.block_manager.rc.len() + self.garage.block_manager.rc_len() ) .unwrap(); writeln!( &mut ret, " resync queue length: {}", - self.garage.block_manager.resync_queue.len() + self.garage.block_manager.resync_queue_len() ) .unwrap(); @@ -460,19 +460,19 @@ impl AdminRpcHandler { writeln!( to, " Merkle updater todo queue length: {}", - t.data.merkle_updater.todo.len() + t.data.merkle_updater.todo_len() ) .unwrap(); writeln!( to, " Merkle tree size: {}", - t.data.merkle_updater.merkle_tree.len() + t.data.merkle_updater.merkle_tree_len() ) .unwrap(); writeln!( to, " GC todo queue length: {}", - t.data.gc_todo.len() + t.data.gc_todo_len() ) .unwrap(); Ok(()) diff --git a/src/model/block.rs b/src/model/block.rs index 8523474a..9fe6c76b 100644 --- a/src/model/block.rs +++ b/src/model/block.rs @@ -27,6 +27,8 @@ use crate::garage::Garage; pub const INLINE_THRESHOLD: usize = 3072; +pub const BACKGROUND_WORKERS: u64 = 1; + const BLOCK_RW_TIMEOUT: Duration = Duration::from_secs(42); const BLOCK_GC_TIMEOUT: Duration = Duration::from_secs(60); const NEED_BLOCK_QUERY_TIMEOUT: Duration = Duration::from_secs(5); @@ -56,14 +58,14 @@ pub struct BlockManager { pub data_dir: PathBuf, pub data_dir_lock: Mutex<()>, - pub rc: sled::Tree, + rc: sled::Tree, - pub resync_queue: sled::Tree, - pub resync_notify: Notify, + resync_queue: sled::Tree, + resync_notify: Notify, - pub system: Arc, + system: Arc, rpc_client: Arc>, - pub garage: ArcSwapOption, + pub(crate) garage: ArcSwapOption, } impl BlockManager { @@ -128,7 +130,7 @@ impl BlockManager { pub fn spawn_background_worker(self: Arc) { // Launch 2 simultaneous workers for background resync loop preprocessing - for i in 0..2u64 { + for i in 0..BACKGROUND_WORKERS { let bm2 = self.clone(); let background = self.system.background.clone(); tokio::spawn(async move { @@ -373,7 +375,6 @@ impl BlockManager { ); fs::remove_file(path).await?; - self.resync_queue.remove(&hash)?; } if needed && !exists { @@ -494,6 +495,14 @@ impl BlockManager { } .boxed() } + + pub fn resync_queue_len(&self) -> usize { + self.resync_queue.len() + } + + pub fn rc_len(&self) -> usize { + self.rc.len() + } } fn u64_from_be_bytes>(bytes: T) -> u64 { diff --git a/src/table/data.rs b/src/table/data.rs index a491f877..0a7b2cec 100644 --- a/src/table/data.rs +++ b/src/table/data.rs @@ -18,7 +18,7 @@ pub struct TableData { pub instance: F, pub store: sled::Tree, - pub gc_todo: sled::Tree, + pub(crate) gc_todo: sled::Tree, pub merkle_updater: Arc, } @@ -239,4 +239,8 @@ where }, } } + + pub fn gc_todo_len(&self) -> usize { + self.gc_todo.len() + } } diff --git a/src/table/merkle.rs b/src/table/merkle.rs index f60be8a8..aefb5169 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -40,13 +40,13 @@ pub struct MerkleUpdater { // - key = the key of an item in the main table, ie hash(partition_key)+sort_key // - value = the hash of the full serialized item, if present, // or an empty vec if item is absent (deleted) - pub todo: sled::Tree, + pub(crate) todo: sled::Tree, pub(crate) todo_notify: Notify, // Content of the merkle tree: items where // - key = .bytes() for MerkleNodeKey // - value = serialization of a MerkleNode, assumed to be MerkleNode::empty if not found - pub merkle_tree: sled::Tree, + pub(crate) merkle_tree: sled::Tree, empty_node_hash: Hash, } @@ -311,6 +311,14 @@ impl MerkleUpdater { Some(v) => Ok(rmp_serde::decode::from_read_ref::<_, MerkleNode>(&v[..])?), } } + + pub fn merkle_tree_len(&self) -> usize { + self.merkle_tree.len() + } + + pub fn todo_len(&self) -> usize { + self.todo.len() + } } impl MerkleNodeKey { -- cgit v1.2.3 From 4d4117f2b4eb69b63e2329f6e0b8929e6a8b5b31 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 15 Mar 2021 20:09:44 +0100 Subject: Refactor block resync loop; make workers infaillible --- src/garage/repair.rs | 10 +++++++++ src/model/block.rs | 58 ++++++++++++++++++++++++++------------------------ src/rpc/membership.rs | 5 +---- src/table/gc.rs | 3 +-- src/table/merkle.rs | 3 +-- src/table/sync.rs | 10 ++++----- src/util/background.rs | 9 +++----- 7 files changed, 50 insertions(+), 48 deletions(-) diff --git a/src/garage/repair.rs b/src/garage/repair.rs index 599c1965..4ee66452 100644 --- a/src/garage/repair.rs +++ b/src/garage/repair.rs @@ -20,6 +20,16 @@ impl Repair { &self, opt: RepairOpt, must_exit: watch::Receiver, + ) { + if let Err(e) = self.repair_worker_aux(opt, must_exit).await { + warn!("Repair worker failed with error: {}", e); + } + } + + async fn repair_worker_aux( + &self, + opt: RepairOpt, + must_exit: watch::Receiver, ) -> Result<(), Error> { let todo = |x| opt.what.as_ref().map(|y| *y == x).unwrap_or(true); diff --git a/src/model/block.rs b/src/model/block.rs index 9fe6c76b..023ed3ab 100644 --- a/src/model/block.rs +++ b/src/model/block.rs @@ -258,46 +258,48 @@ impl BlockManager { async fn resync_loop( self: Arc, mut must_exit: watch::Receiver, - ) -> Result<(), Error> { - let mut n_failures = 0usize; + ) { while !*must_exit.borrow() { - if let Some((time_bytes, hash_bytes)) = self.resync_queue.pop_min()? { - let time_msec = u64_from_be_bytes(&time_bytes[0..8]); - let now = now_msec(); - if now >= time_msec { - let hash = Hash::try_from(&hash_bytes[..]).unwrap(); - - if let Err(e) = self.resync_iter(&hash).await { - warn!("Failed to resync block {:?}, retrying later: {}", hash, e); - self.put_to_resync(&hash, RESYNC_RETRY_TIMEOUT)?; - n_failures += 1; - if n_failures >= 10 { - warn!("Too many resync failures, throttling."); - tokio::time::delay_for(Duration::from_secs(1)).await; - } - } else { - n_failures = 0; - } - } else { - self.resync_queue.insert(time_bytes, hash_bytes)?; - let delay = tokio::time::delay_for(Duration::from_millis(time_msec - now)); - select! { - _ = delay.fuse() => (), - _ = self.resync_notify.notified().fuse() => (), - _ = must_exit.recv().fuse() => (), - } + if let Err(e) = self.resync_iter(&mut must_exit).await { + warn!("Error in block resync loop: {}", e); + tokio::time::delay_for(Duration::from_secs(10)).await; + } + } + } + + + async fn resync_iter(&self, must_exit: &mut watch::Receiver) -> Result<(), Error> { + if let Some(first_item) = self.resync_queue.iter().next() { + let (time_bytes, hash_bytes) = first_item?; + let time_msec = u64_from_be_bytes(&time_bytes[0..8]); + let now = now_msec(); + if now >= time_msec { + let hash = Hash::try_from(&hash_bytes[..]).unwrap(); + let res = self.resync_block(&hash).await; + if let Err(e) = &res { + warn!("Error when resyncing {:?}: {}", hash, e); + self.put_to_resync(&hash, RESYNC_RETRY_TIMEOUT)?; } + self.resync_queue.remove(&time_bytes)?; + res?; // propagate error to delay main loop } else { + let delay = tokio::time::delay_for(Duration::from_millis(time_msec - now)); select! { + _ = delay.fuse() => (), _ = self.resync_notify.notified().fuse() => (), _ = must_exit.recv().fuse() => (), } } + } else { + select! { + _ = self.resync_notify.notified().fuse() => (), + _ = must_exit.recv().fuse() => (), + } } Ok(()) } - async fn resync_iter(&self, hash: &Hash) -> Result<(), Error> { + async fn resync_block(&self, hash: &Hash) -> Result<(), Error> { let lock = self.data_dir_lock.lock().await; let path = self.block_path(hash); diff --git a/src/rpc/membership.rs b/src/rpc/membership.rs index 6636e50b..6749478a 100644 --- a/src/rpc/membership.rs +++ b/src/rpc/membership.rs @@ -318,9 +318,7 @@ impl System { let self2 = self.clone(); self.clone() .background - .spawn_worker(format!("ping loop"), |stop_signal| { - self2.ping_loop(stop_signal).map(Ok) - }); + .spawn_worker(format!("ping loop"), |stop_signal| self2.ping_loop(stop_signal)); if let (Some(consul_host), Some(consul_service_name)) = (consul_host, consul_service_name) { let self2 = self.clone(); @@ -329,7 +327,6 @@ impl System { .spawn_worker(format!("Consul loop"), |stop_signal| { self2 .consul_loop(stop_signal, consul_host, consul_service_name) - .map(Ok) }); } } diff --git a/src/table/gc.rs b/src/table/gc.rs index c13c8234..fd9a26d1 100644 --- a/src/table/gc.rs +++ b/src/table/gc.rs @@ -70,7 +70,7 @@ where gc } - async fn gc_loop(self: Arc, mut must_exit: watch::Receiver) -> Result<(), Error> { + async fn gc_loop(self: Arc, mut must_exit: watch::Receiver) { while !*must_exit.borrow() { match self.gc_loop_iter().await { Ok(true) => { @@ -89,7 +89,6 @@ where _ = must_exit.recv().fuse() => (), } } - Ok(()) } async fn gc_loop_iter(&self) -> Result { diff --git a/src/table/merkle.rs b/src/table/merkle.rs index aefb5169..5ce9cee3 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -104,7 +104,7 @@ impl MerkleUpdater { async fn updater_loop( self: Arc, mut must_exit: watch::Receiver, - ) -> Result<(), Error> { + ) { while !*must_exit.borrow() { if let Some(x) = self.todo.iter().next() { match x { @@ -131,7 +131,6 @@ impl MerkleUpdater { } } } - Ok(()) } fn update_item(&self, k: &[u8], vhash_by: &[u8]) -> Result<(), Error> { diff --git a/src/table/sync.rs b/src/table/sync.rs index 6c8792d2..b344eb88 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -136,7 +136,7 @@ where self: Arc, mut must_exit: watch::Receiver, mut busy_rx: mpsc::UnboundedReceiver, - ) -> Result<(), Error> { + ) { let mut prev_ring: Arc = self.aux.system.ring.borrow().clone(); let mut ring_recv: watch::Receiver> = self.aux.system.ring.clone(); let mut nothing_to_do_since = Some(Instant::now()); @@ -183,7 +183,6 @@ where } } } - Ok(()) } pub fn add_full_sync(&self) { @@ -197,11 +196,11 @@ where self: Arc, mut must_exit: watch::Receiver, busy_tx: mpsc::UnboundedSender, - ) -> Result<(), Error> { + ) { while !*must_exit.borrow() { let task = self.todo.lock().unwrap().pop_task(); if let Some(partition) = task { - busy_tx.send(true)?; + busy_tx.send(true).unwrap(); let res = self .clone() .sync_partition(&partition, &mut must_exit) @@ -213,11 +212,10 @@ where ); } } else { - busy_tx.send(false)?; + busy_tx.send(false).unwrap(); tokio::time::delay_for(Duration::from_secs(1)).await; } } - Ok(()) } async fn sync_partition( diff --git a/src/util/background.rs b/src/util/background.rs index 8081f157..3e600fdf 100644 --- a/src/util/background.rs +++ b/src/util/background.rs @@ -76,16 +76,13 @@ impl BackgroundRunner { pub fn spawn_worker(&self, name: String, worker: F) where F: FnOnce(watch::Receiver) -> T + Send + 'static, - T: Future + Send + 'static, + T: Future + Send + 'static, { let mut workers = self.workers.lock().unwrap(); let stop_signal = self.stop_signal.clone(); workers.push(tokio::spawn(async move { - if let Err(e) = worker(stop_signal).await { - error!("Worker stopped with error: {}, error: {}", name, e); - } else { - info!("Worker exited successfully: {}", name); - } + worker(stop_signal).await; + info!("Worker exited: {}", name); })); } -- cgit v1.2.3 From 0cd5b2ae19965b8c1f3176afeb8f678c4d8366dd Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 15 Mar 2021 22:36:41 +0100 Subject: WIP migrate to tokio 1 --- Cargo.lock | 274 ++++++++++++++------------------------------- src/api/Cargo.toml | 4 +- src/garage/Cargo.toml | 2 +- src/garage/server.rs | 6 +- src/model/Cargo.toml | 3 +- src/model/block.rs | 23 ++-- src/model/object_table.rs | 3 +- src/model/version_table.rs | 3 +- src/rpc/Cargo.toml | 11 +- src/rpc/membership.rs | 30 +++-- src/rpc/rpc_client.rs | 2 +- src/rpc/rpc_server.rs | 5 +- src/table/Cargo.toml | 2 +- src/table/data.rs | 6 +- src/table/gc.rs | 4 +- src/table/merkle.rs | 4 +- src/table/sync.rs | 35 ++---- src/util/Cargo.toml | 7 +- src/util/background.rs | 131 +++++++++++----------- src/util/error.rs | 8 +- src/web/Cargo.toml | 2 +- 21 files changed, 231 insertions(+), 334 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 9fb368e5..8e8641ef 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -17,6 +17,12 @@ version = "0.4.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dabe5a181f83789739c194cbe5a897dde195078fac08568d09221fd6137a7ba8" +[[package]] +name = "arc-swap" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d4d7d63395147b81a9e570bcc6243aaf71c017bd666d4909cfef0085bdda8d73" + [[package]] name = "arrayvec" version = "0.3.25" @@ -46,7 +52,7 @@ checksum = "d9b39be18770d11421cdb1b9947a45dd3f37e93092cbf377614828a319d5fee8" dependencies = [ "hermit-abi", "libc", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -55,12 +61,6 @@ version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cdb031dd78e28731d87d56cc8ffef4a8f36ca26c38fe2de700543e627f8a464a" -[[package]] -name = "base64" -version = "0.11.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b41b7ea54a0c9d92199de89e20e58d49f02f8e699814ef3fdf266f6f748d15c7" - [[package]] name = "base64" version = "0.13.0" @@ -142,12 +142,6 @@ dependencies = [ "iovec", ] -[[package]] -name = "bytes" -version = "0.5.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e4cec68f03f32e44924783795810fa50a7035d8c8ebe78580ad7e6c703fba38" - [[package]] name = "bytes" version = "1.0.1" @@ -182,7 +176,7 @@ dependencies = [ "num-integer", "num-traits", "time", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -347,7 +341,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9564fc758e15025b46aa6643b1b77d047d1a56a1aea6e01002ac0c7026876213" dependencies = [ "libc", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -356,22 +350,6 @@ version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a06f77d526c1a601b7c4cdd98f54b5eaabffc14d5f2f0296febdc7f357c6d3ba" -[[package]] -name = "fuchsia-zircon" -version = "0.3.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e9763c69ebaae630ba35f74888db465e49e259ba1bc0eda7d06f4a067615d82" -dependencies = [ - "bitflags", - "fuchsia-zircon-sys", -] - -[[package]] -name = "fuchsia-zircon-sys" -version = "0.3.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3dcaa9ae7725d12cdb85b3ad99a434db70b468c09ded17e012d86b5c1010f7a7" - [[package]] name = "futures" version = "0.3.12" @@ -460,7 +438,7 @@ dependencies = [ "futures-sink", "futures-task", "memchr", - "pin-project-lite 0.2.4", + "pin-project-lite", "pin-utils", "proc-macro-hack", "proc-macro-nested", @@ -507,7 +485,7 @@ dependencies = [ name = "garage_api" version = "0.1.1" dependencies = [ - "base64 0.13.0", + "base64", "bytes 0.4.12", "chrono", "crypto-mac 0.7.0", @@ -537,7 +515,7 @@ dependencies = [ name = "garage_model" version = "0.1.1" dependencies = [ - "arc-swap", + "arc-swap 0.4.8", "async-trait", "bytes 0.4.12", "futures", @@ -560,7 +538,7 @@ dependencies = [ name = "garage_rpc" version = "0.1.1" dependencies = [ - "arc-swap", + "arc-swap 0.4.8", "bytes 0.4.12", "futures", "futures-util", @@ -579,6 +557,7 @@ dependencies = [ "sha2", "tokio", "tokio-rustls", + "tokio-stream", "webpki", ] @@ -607,6 +586,7 @@ dependencies = [ name = "garage_util" version = "0.1.1" dependencies = [ + "arc-swap 1.2.0", "blake2", "chrono", "err-derive", @@ -664,7 +644,7 @@ dependencies = [ "libc", "log", "rustversion", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -693,7 +673,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e692e296bfac1d2533ef168d0b60ff5897b8b70a4009276834014dd8924cc028" dependencies = [ "libc", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -731,11 +711,11 @@ dependencies = [ [[package]] name = "h2" -version = "0.2.7" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e4728fd124914ad25e99e3d15a9361a879f6620f63cb56bbb08f95abb97a535" +checksum = "d832b01df74254fe364568d6ddc294443f61cbec82816b60904303af87efae78" dependencies = [ - "bytes 0.5.6", + "bytes 1.0.1", "fnv", "futures-core", "futures-sink", @@ -746,7 +726,6 @@ dependencies = [ "tokio", "tokio-util", "tracing", - "tracing-futures", ] [[package]] @@ -812,11 +791,11 @@ dependencies = [ [[package]] name = "http-body" -version = "0.3.1" +version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13d5ff830006f7646652e057693569bfe0d51760c0085a071769d142a205111b" +checksum = "2861bd27ee074e5ee891e8b539837a9430012e249d7f0ca2d795650f579c1994" dependencies = [ - "bytes 0.5.6", + "bytes 1.0.1", "http", ] @@ -849,11 +828,11 @@ dependencies = [ [[package]] name = "hyper" -version = "0.13.9" +version = "0.14.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f6ad767baac13b44d4529fcf58ba2cd0995e36e7b435bc5b039de6f47e880dbf" +checksum = "e8e946c2b1349055e0b72ae281b238baf1a3ea7307c7e9f9d64673bdd9c26ac7" dependencies = [ - "bytes 0.5.6", + "bytes 1.0.1", "futures-channel", "futures-core", "futures-util", @@ -863,7 +842,7 @@ dependencies = [ "httparse", "httpdate", "itoa", - "pin-project 1.0.4", + "pin-project", "socket2", "tokio", "tower-service", @@ -873,11 +852,10 @@ dependencies = [ [[package]] name = "hyper-rustls" -version = "0.20.0" +version = "0.22.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac965ea399ec3a25ac7d13b8affd4b8f39325cca00858ddf5eb29b79e6b14b08" +checksum = "5f9f7a97316d44c0af9b0301e65010573a853a9fc97046d7331d7f6bc0fd5a64" dependencies = [ - "bytes 0.5.6", "futures-util", "hyper", "log", @@ -947,16 +925,6 @@ dependencies = [ "wasm-bindgen", ] -[[package]] -name = "kernel32-sys" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7507624b29483431c0ba2d82aece8ca6cdba9382bff4ddd0f7490560c056098d" -dependencies = [ - "winapi 0.2.8", - "winapi-build", -] - [[package]] name = "lazy_static" version = "1.4.0" @@ -965,9 +933,9 @@ checksum = "e2abad23fbc42b3700f2f279844dc832adb2b2eb069b2df918f455c4e18cc646" [[package]] name = "libc" -version = "0.2.82" +version = "0.2.89" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89203f3fba0a3795506acaad8ebce3c80c0af93f994d5a1d7a0b1eeb23271929" +checksum = "538c092e5586f4cdd7dd8078c4a79220e3e168880218124dcbce860f0ea938c6" [[package]] name = "lock_api" @@ -1032,63 +1000,42 @@ dependencies = [ [[package]] name = "mio" -version = "0.6.23" +version = "0.7.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4afd66f5b91bf2a3bc13fad0e21caedac168ca4c707504e75585648ae80e4cc4" +checksum = "2182a122f3b7f3f5329cb1972cee089ba2459a0a80a56935e6e674f096f8d839" dependencies = [ - "cfg-if 0.1.10", - "fuchsia-zircon", - "fuchsia-zircon-sys", - "iovec", - "kernel32-sys", "libc", "log", "miow", - "net2", - "slab", - "winapi 0.2.8", + "ntapi", + "winapi", ] [[package]] -name = "mio-uds" -version = "0.6.8" +name = "miow" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "afcb699eb26d4332647cc848492bbc15eafb26f08d0304550d5aa1f612e066f0" +checksum = "5a33c1b55807fbed163481b5ba66db4b2fa6cde694a5027be10fb724206c5897" dependencies = [ - "iovec", - "libc", - "mio", + "socket2", + "winapi", ] [[package]] -name = "miow" -version = "0.2.2" +name = "nodrop" +version = "0.1.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ebd808424166322d4a38da87083bfddd3ac4c131334ed55856112eb06d46944d" -dependencies = [ - "kernel32-sys", - "net2", - "winapi 0.2.8", - "ws2_32-sys", -] +checksum = "72ef4a56884ca558e5ddb05a1d1e7e1bfd9a68d9ed024c21704cc98872dae1bb" [[package]] -name = "net2" -version = "0.2.37" +name = "ntapi" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "391630d12b68002ae1e25e8f974306474966550ad82dac6886fb8910c19568ae" +checksum = "3f6bb902e437b6d86e03cce10a7e2af662292c5dfef23b65899ea3ac9354ad44" dependencies = [ - "cfg-if 0.1.10", - "libc", - "winapi 0.3.9", + "winapi", ] -[[package]] -name = "nodrop" -version = "0.1.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72ef4a56884ca558e5ddb05a1d1e7e1bfd9a68d9ed024c21704cc98872dae1bb" - [[package]] name = "num-integer" version = "0.1.44" @@ -1164,7 +1111,7 @@ dependencies = [ "libc", "redox_syscall", "smallvec", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -1173,33 +1120,13 @@ version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d4fd5641d01c8f18a23da7b6fe29298ff4b55afcccdf78973b24cf3175fee32e" -[[package]] -name = "pin-project" -version = "0.4.27" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2ffbc8e94b38ea3d2d8ba92aea2983b503cd75d0888d75b86bb37970b5698e15" -dependencies = [ - "pin-project-internal 0.4.27", -] - [[package]] name = "pin-project" version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "95b70b68509f17aa2857863b6fa00bf21fc93674c7a8893de2f469f6aa7ca2f2" dependencies = [ - "pin-project-internal 1.0.4", -] - -[[package]] -name = "pin-project-internal" -version = "0.4.27" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "65ad2ae56b6abe3a1ee25f15ee605bacadb9a764edaba9c2bf4103800d4a1895" -dependencies = [ - "proc-macro2", - "quote", - "syn", + "pin-project-internal", ] [[package]] @@ -1213,12 +1140,6 @@ dependencies = [ "syn", ] -[[package]] -name = "pin-project-lite" -version = "0.1.11" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c917123afa01924fc84bb20c4c03f004d9c38e5127e3c039bbf7f4b9c76a2f6b" - [[package]] name = "pin-project-lite" version = "0.2.4" @@ -1317,7 +1238,7 @@ dependencies = [ "libc", "rand_core 0.3.1", "rdrand", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -1424,7 +1345,7 @@ dependencies = [ "spin", "untrusted", "web-sys", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -1459,11 +1380,11 @@ dependencies = [ [[package]] name = "rustls" -version = "0.17.0" +version = "0.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0d4a31f5d68413404705d6982529b0e11a9aacd4839d1d6222ee3b8cb4015e1" +checksum = "064fd21ff87c6e87ed4506e68beb42459caa4a0e2eb144932e6776768556980b" dependencies = [ - "base64 0.11.0", + "base64", "log", "ring", "sct", @@ -1607,7 +1528,7 @@ checksum = "122e570113d28d773067fab24266b66753f6ea915758651696b6e35e49f88d6e" dependencies = [ "cfg-if 1.0.0", "libc", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -1710,7 +1631,7 @@ checksum = "6db9e6914ab8b1ae1c260a4ae7a49b6c5611b40328a735b21862567685e73255" dependencies = [ "libc", "wasi 0.10.0+wasi-snapshot-preview1", - "winapi 0.3.9", + "winapi", ] [[package]] @@ -1730,32 +1651,28 @@ checksum = "cda74da7e1a664f795bb1f8a87ec406fb89a02522cf6e50620d016add6dbbf5c" [[package]] name = "tokio" -version = "0.2.24" +version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "099837d3464c16a808060bb3f02263b412f6fafcb5d01c533d309985fbeebe48" +checksum = "8d56477f6ed99e10225f38f9f75f872f29b8b8bd8c0b946f63345bb144e9eeda" dependencies = [ - "bytes 0.5.6", - "fnv", - "futures-core", - "iovec", - "lazy_static", + "autocfg", + "bytes 1.0.1", "libc", "memchr", "mio", - "mio-uds", "num_cpus", - "pin-project-lite 0.1.11", + "once_cell", + "pin-project-lite", "signal-hook-registry", - "slab", "tokio-macros", - "winapi 0.3.9", + "winapi", ] [[package]] name = "tokio-macros" -version = "0.2.6" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e44da00bfc73a25f814cd8d7e57a68a5c31b74b3152a0a1d1f590c97ed06265a" +checksum = "caf7b11a536f46a809a8a9f0bb4237020f70ecbf115b842360afb127ea2fda57" dependencies = [ "proc-macro2", "quote", @@ -1764,27 +1681,37 @@ dependencies = [ [[package]] name = "tokio-rustls" -version = "0.13.1" +version = "0.22.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15cb62a0d2770787abc96e99c1cd98fcf17f94959f3af63ca85bdfb203f051b4" +checksum = "bc6844de72e57df1980054b38be3a9f4702aba4858be64dd700181a8a6d0e1b6" dependencies = [ - "futures-core", "rustls", "tokio", "webpki", ] +[[package]] +name = "tokio-stream" +version = "0.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c535f53c0cfa1acace62995a8994fc9cc1f12d202420da96ff306ee24d576469" +dependencies = [ + "futures-core", + "pin-project-lite", + "tokio", +] + [[package]] name = "tokio-util" -version = "0.3.1" +version = "0.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be8242891f2b6cbef26a2d7e8605133c2c554cd35b3e4948ea892d6d68436499" +checksum = "ec31e5cc6b46e653cf57762f36f71d5e6386391d88a72fd6db4508f8f676fb29" dependencies = [ - "bytes 0.5.6", + "bytes 1.0.1", "futures-core", "futures-sink", "log", - "pin-project-lite 0.1.11", + "pin-project-lite", "tokio", ] @@ -1810,8 +1737,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9f47026cdc4080c07e49b37087de021820269d996f581aac150ef9e5583eefe3" dependencies = [ "cfg-if 1.0.0", - "log", - "pin-project-lite 0.2.4", + "pin-project-lite", "tracing-core", ] @@ -1824,16 +1750,6 @@ dependencies = [ "lazy_static", ] -[[package]] -name = "tracing-futures" -version = "0.2.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab7bb6f14721aa00656086e9335d363c5c8747bae02ebe32ea2c7dece5689b4c" -dependencies = [ - "pin-project 0.4.27", - "tracing", -] - [[package]] name = "try-lock" version = "0.2.3" @@ -2002,12 +1918,6 @@ dependencies = [ "untrusted", ] -[[package]] -name = "winapi" -version = "0.2.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "167dc9d6949a9b857f3451275e911c3f44255842c1f7a76f33c55103a909087a" - [[package]] name = "winapi" version = "0.3.9" @@ -2018,12 +1928,6 @@ dependencies = [ "winapi-x86_64-pc-windows-gnu", ] -[[package]] -name = "winapi-build" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2d315eee3b34aca4797b2da6b13ed88266e6d612562a0c46390af8299fc699bc" - [[package]] name = "winapi-i686-pc-windows-gnu" version = "0.4.0" @@ -2036,7 +1940,7 @@ version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "70ec6ce85bb158151cae5e5c87f95a8e97d2c0c4b001223f33a334e3ce5de178" dependencies = [ - "winapi 0.3.9", + "winapi", ] [[package]] @@ -2045,16 +1949,6 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" -[[package]] -name = "ws2_32-sys" -version = "0.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d59cefebd0c892fa2dd6de581e937301d8552cb44489cdff035c6187cb63fa5e" -dependencies = [ - "winapi 0.2.8", - "winapi-build", -] - [[package]] name = "xmlparser" version = "0.13.3" diff --git a/src/api/Cargo.toml b/src/api/Cargo.toml index 45388eff..c3208b66 100644 --- a/src/api/Cargo.toml +++ b/src/api/Cargo.toml @@ -31,10 +31,10 @@ rand = "0.7" futures = "0.3" futures-util = "0.3" -tokio = { version = "0.2", default-features = false, features = ["rt-core", "rt-threaded", "io-driver", "net", "tcp", "time", "macros", "sync", "signal", "fs"] } +tokio = { version = "1.0", default-features = false, features = ["rt", "rt-multi-thread", "io-util", "net", "time", "macros", "sync", "signal", "fs"] } http = "0.2" -hyper = "^0.13.6" +hyper = "0.14" url = "2.1" httpdate = "0.3" percent-encoding = "2.1.0" diff --git a/src/garage/Cargo.toml b/src/garage/Cargo.toml index 8c28394b..36bbcd50 100644 --- a/src/garage/Cargo.toml +++ b/src/garage/Cargo.toml @@ -38,4 +38,4 @@ serde = { version = "1.0", default-features = false, features = ["derive", "rc"] futures = "0.3" futures-util = "0.3" -tokio = { version = "0.2", default-features = false, features = ["rt-core", "rt-threaded", "io-driver", "net", "tcp", "time", "macros", "sync", "signal", "fs"] } +tokio = { version = "1.0", default-features = false, features = ["rt", "rt-multi-thread", "io-util", "net", "time", "macros", "sync", "signal", "fs"] } diff --git a/src/garage/server.rs b/src/garage/server.rs index 29740feb..ce90ecab 100644 --- a/src/garage/server.rs +++ b/src/garage/server.rs @@ -21,13 +21,13 @@ async fn shutdown_signal(send_cancel: watch::Sender) -> Result<(), Error> .await .expect("failed to install CTRL+C signal handler"); info!("Received CTRL+C, shutting down."); - send_cancel.broadcast(true)?; + send_cancel.send(true)?; Ok(()) } async fn wait_from(mut chan: watch::Receiver) -> () { - while let Some(exit_now) = chan.recv().await { - if exit_now { + while !*chan.borrow() { + if chan.changed().await.is_err() { return; } } diff --git a/src/model/Cargo.toml b/src/model/Cargo.toml index caeed66c..8f36cf2e 100644 --- a/src/model/Cargo.toml +++ b/src/model/Cargo.toml @@ -33,5 +33,4 @@ serde_bytes = "0.11" async-trait = "0.1.30" futures = "0.3" futures-util = "0.3" -tokio = { version = "0.2", default-features = false, features = ["rt-core", "rt-threaded", "io-driver", "net", "tcp", "time", "macros", "sync", "signal", "fs"] } - +tokio = { version = "1.0", default-features = false, features = ["rt", "rt-multi-thread", "io-util", "net", "time", "macros", "sync", "signal", "fs"] } diff --git a/src/model/block.rs b/src/model/block.rs index 023ed3ab..7185372c 100644 --- a/src/model/block.rs +++ b/src/model/block.rs @@ -5,10 +5,9 @@ use std::time::Duration; use arc_swap::ArcSwapOption; use futures::future::*; use futures::select; -use futures::stream::*; use serde::{Deserialize, Serialize}; use tokio::fs; -use tokio::prelude::*; +use tokio::io::{AsyncReadExt, AsyncWriteExt}; use tokio::sync::{watch, Mutex, Notify}; use garage_util::data::*; @@ -134,7 +133,7 @@ impl BlockManager { let bm2 = self.clone(); let background = self.system.background.clone(); tokio::spawn(async move { - tokio::time::delay_for(Duration::from_secs(10 * (i + 1))).await; + tokio::time::sleep(Duration::from_secs(10 * (i + 1))).await; background.spawn_worker(format!("block resync worker {}", i), move |must_exit| { bm2.resync_loop(must_exit) }); @@ -251,7 +250,7 @@ impl BlockManager { let mut key = u64::to_be_bytes(when).to_vec(); key.extend(hash.as_ref()); self.resync_queue.insert(key, hash.as_ref())?; - self.resync_notify.notify(); + self.resync_notify.notify_waiters(); Ok(()) } @@ -262,7 +261,7 @@ impl BlockManager { while !*must_exit.borrow() { if let Err(e) = self.resync_iter(&mut must_exit).await { warn!("Error in block resync loop: {}", e); - tokio::time::delay_for(Duration::from_secs(10)).await; + tokio::time::sleep(Duration::from_secs(10)).await; } } } @@ -283,17 +282,17 @@ impl BlockManager { self.resync_queue.remove(&time_bytes)?; res?; // propagate error to delay main loop } else { - let delay = tokio::time::delay_for(Duration::from_millis(time_msec - now)); + let delay = tokio::time::sleep(Duration::from_millis(time_msec - now)); select! { _ = delay.fuse() => (), _ = self.resync_notify.notified().fuse() => (), - _ = must_exit.recv().fuse() => (), + _ = must_exit.changed().fuse() => (), } } } else { select! { _ = self.resync_notify.notified().fuse() => (), - _ = must_exit.recv().fuse() => (), + _ = must_exit.changed().fuse() => (), } } Ok(()) @@ -467,8 +466,12 @@ impl BlockManager { // so that we can offload them if necessary and then delete them locally. async move { let mut ls_data_dir = fs::read_dir(path).await?; - while let Some(data_dir_ent) = ls_data_dir.next().await { - let data_dir_ent = data_dir_ent?; + loop { + let data_dir_ent = ls_data_dir.next_entry().await?; + let data_dir_ent = match data_dir_ent { + Some(x) => x, + None => break, + }; let name = data_dir_ent.file_name(); let name = match name.into_string() { Ok(x) => x, diff --git a/src/model/object_table.rs b/src/model/object_table.rs index 99fad3ce..d08bba70 100644 --- a/src/model/object_table.rs +++ b/src/model/object_table.rs @@ -195,7 +195,8 @@ impl TableSchema for ObjectTable { fn updated(&self, old: Option, new: Option) { let version_table = self.version_table.clone(); - self.background.spawn(async move { + // TODO not cancellable + self.background.spawn_cancellable(async move { if let (Some(old_v), Some(new_v)) = (old, new) { // Propagate deletion of old versions for v in old_v.versions.iter() { diff --git a/src/model/version_table.rs b/src/model/version_table.rs index 841fbfea..19343890 100644 --- a/src/model/version_table.rs +++ b/src/model/version_table.rs @@ -110,7 +110,8 @@ impl TableSchema for VersionTable { fn updated(&self, old: Option, new: Option) { let block_ref_table = self.block_ref_table.clone(); - self.background.spawn(async move { + // TODO not cancellable + self.background.spawn_cancellable(async move { if let (Some(old_v), Some(new_v)) = (old, new) { // Propagate deletion of version blocks if new_v.deleted.get() && !old_v.deleted.get() { diff --git a/src/rpc/Cargo.toml b/src/rpc/Cargo.toml index 48f05755..fc066bef 100644 --- a/src/rpc/Cargo.toml +++ b/src/rpc/Cargo.toml @@ -29,13 +29,14 @@ serde_json = "1.0" futures = "0.3" futures-util = "0.3" -tokio = { version = "0.2", default-features = false, features = ["rt-core", "rt-threaded", "io-driver", "net", "tcp", "time", "macros", "sync", "signal", "fs"] } +tokio = { version = "1.0", default-features = false, features = ["rt", "rt-multi-thread", "io-util", "net", "time", "macros", "sync", "signal", "fs"] } +tokio-stream = {version = "0.1", features = ["net"] } http = "0.2" -hyper = "0.13" -rustls = "0.17" -tokio-rustls = "0.13" -hyper-rustls = { version = "0.20", default-features = false } +hyper = { version = "0.14", features = ["full"] } +rustls = "0.19" +tokio-rustls = "0.22" +hyper-rustls = { version = "0.22", default-features = false } webpki = "0.21" diff --git a/src/rpc/membership.rs b/src/rpc/membership.rs index 6749478a..6cc3ed2e 100644 --- a/src/rpc/membership.rs +++ b/src/rpc/membership.rs @@ -11,9 +11,9 @@ use futures::future::join_all; use futures::select; use futures_util::future::*; use serde::{Deserialize, Serialize}; -use tokio::prelude::*; use tokio::sync::watch; use tokio::sync::Mutex; +use tokio::io::AsyncWriteExt; use garage_util::background::BackgroundRunner; use garage_util::data::*; @@ -395,7 +395,7 @@ impl System { if has_changes { status.recalculate_hash(); } - if let Err(e) = update_locked.0.broadcast(Arc::new(status)) { + if let Err(e) = update_locked.0.send(Arc::new(status)) { error!("In ping_nodes: could not save status update ({})", e); } drop(update_locked); @@ -421,7 +421,7 @@ impl System { let status_hash = status.hash; let config_version = self.ring.borrow().config.version; - update_locked.0.broadcast(Arc::new(status))?; + update_locked.0.send(Arc::new(status))?; drop(update_locked); if is_new || status_hash != ping.status_hash { @@ -503,7 +503,7 @@ impl System { if has_changed { status.recalculate_hash(); } - update_lock.0.broadcast(Arc::new(status))?; + update_lock.0.send(Arc::new(status))?; drop(update_lock); if to_ping.len() > 0 { @@ -523,7 +523,7 @@ impl System { if adv.version > ring.config.version { let ring = Ring::new(adv.clone()); - update_lock.1.broadcast(Arc::new(ring))?; + update_lock.1.send(Arc::new(ring))?; drop(update_lock); self.background.spawn_cancellable( @@ -531,7 +531,7 @@ impl System { .broadcast(Message::AdvertiseConfig(adv.clone()), PING_TIMEOUT) .map(Ok), ); - self.background.spawn(self.clone().save_network_config()); + self.background.spawn(self.clone().save_network_config()).await; } Ok(Message::Ok) @@ -539,7 +539,7 @@ impl System { async fn ping_loop(self: Arc, mut stop_signal: watch::Receiver) { loop { - let restart_at = tokio::time::delay_for(PING_INTERVAL); + let restart_at = tokio::time::sleep(PING_INTERVAL); let status = self.status.borrow().clone(); let ping_addrs = status @@ -553,10 +553,9 @@ impl System { select! { _ = restart_at.fuse() => (), - must_exit = stop_signal.recv().fuse() => { - match must_exit { - None | Some(true) => return, - _ => (), + _ = stop_signal.changed().fuse() => { + if *stop_signal.borrow() { + return; } } } @@ -570,7 +569,7 @@ impl System { consul_service_name: String, ) { loop { - let restart_at = tokio::time::delay_for(CONSUL_INTERVAL); + let restart_at = tokio::time::sleep(CONSUL_INTERVAL); match get_consul_nodes(&consul_host, &consul_service_name).await { Ok(mut node_list) => { @@ -584,10 +583,9 @@ impl System { select! { _ = restart_at.fuse() => (), - must_exit = stop_signal.recv().fuse() => { - match must_exit { - None | Some(true) => return, - _ => (), + _ = stop_signal.changed().fuse() => { + if *stop_signal.borrow() { + return; } } } diff --git a/src/rpc/rpc_client.rs b/src/rpc/rpc_client.rs index cffcf106..60286256 100644 --- a/src/rpc/rpc_client.rs +++ b/src/rpc/rpc_client.rs @@ -198,7 +198,7 @@ impl RpcClient { let wait_finished_fut = tokio::spawn(async move { resp_stream.collect::>().await; }); - self.background.spawn(wait_finished_fut.map(|_| Ok(()))); + self.background.spawn(wait_finished_fut.map(|_| Ok(()))).await; } Ok(results) diff --git a/src/rpc/rpc_server.rs b/src/rpc/rpc_server.rs index 4d14b790..3c5014c4 100644 --- a/src/rpc/rpc_server.rs +++ b/src/rpc/rpc_server.rs @@ -13,6 +13,7 @@ use hyper::service::{make_service_fn, service_fn}; use hyper::{Body, Method, Request, Response, Server, StatusCode}; use serde::{Deserialize, Serialize}; use tokio::net::{TcpListener, TcpStream}; +use tokio_stream::wrappers::TcpListenerStream; use tokio_rustls::server::TlsStream; use tokio_rustls::TlsAcceptor; @@ -171,8 +172,8 @@ impl RpcServer { config.set_single_cert([&node_certs[..], &ca_certs[..]].concat(), node_key)?; let tls_acceptor = Arc::new(TlsAcceptor::from(Arc::new(config))); - let mut listener = TcpListener::bind(&self.bind_addr).await?; - let incoming = listener.incoming().filter_map(|socket| async { + let listener = TcpListener::bind(&self.bind_addr).await?; + let incoming = TcpListenerStream::new(listener).filter_map(|socket| async { match socket { Ok(stream) => match tls_acceptor.clone().accept(stream).await { Ok(x) => Some(Ok::<_, hyper::Error>(x)), diff --git a/src/table/Cargo.toml b/src/table/Cargo.toml index 6b3aaceb..8f73470e 100644 --- a/src/table/Cargo.toml +++ b/src/table/Cargo.toml @@ -31,5 +31,5 @@ serde_bytes = "0.11" async-trait = "0.1.30" futures = "0.3" futures-util = "0.3" -tokio = { version = "0.2", default-features = false, features = ["rt-core", "rt-threaded", "io-driver", "net", "tcp", "time", "macros", "sync", "signal", "fs"] } +tokio = { version = "1.0", default-features = false, features = ["rt", "rt-multi-thread", "io-util", "net", "time", "macros", "sync", "signal", "fs"] } diff --git a/src/table/data.rs b/src/table/data.rs index 0a7b2cec..0029b936 100644 --- a/src/table/data.rs +++ b/src/table/data.rs @@ -159,7 +159,7 @@ where if let Some((old_entry, new_entry, new_bytes_hash)) = changed { let is_tombstone = new_entry.is_tombstone(); self.instance.updated(old_entry, Some(new_entry)); - self.merkle_updater.todo_notify.notify(); + self.merkle_updater.todo_notify.notify_one(); if is_tombstone { self.gc_todo.insert(&tree_key, new_bytes_hash.as_slice())?; } @@ -184,7 +184,7 @@ where if removed { let old_entry = self.decode_entry(v)?; self.instance.updated(Some(old_entry), None); - self.merkle_updater.todo_notify.notify(); + self.merkle_updater.todo_notify.notify_one(); } Ok(removed) } @@ -209,7 +209,7 @@ where if let Some(old_v) = removed { let old_entry = self.decode_entry(&old_v[..])?; self.instance.updated(Some(old_entry), None); - self.merkle_updater.todo_notify.notify(); + self.merkle_updater.todo_notify.notify_one(); Ok(true) } else { Ok(false) diff --git a/src/table/gc.rs b/src/table/gc.rs index fd9a26d1..d37fdf35 100644 --- a/src/table/gc.rs +++ b/src/table/gc.rs @@ -85,8 +85,8 @@ where } } select! { - _ = tokio::time::delay_for(Duration::from_secs(10)).fuse() => (), - _ = must_exit.recv().fuse() => (), + _ = tokio::time::sleep(Duration::from_secs(10)).fuse() => (), + _ = must_exit.changed().fuse() => (), } } } diff --git a/src/table/merkle.rs b/src/table/merkle.rs index 5ce9cee3..86289bf1 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -121,13 +121,13 @@ impl MerkleUpdater { "({}) Error while iterating on Merkle todo tree: {}", self.table_name, e ); - tokio::time::delay_for(Duration::from_secs(10)).await; + tokio::time::sleep(Duration::from_secs(10)).await; } } } else { select! { _ = self.todo_notify.notified().fuse() => (), - _ = must_exit.recv().fuse() => (), + _ = must_exit.changed().fuse() => (), } } } diff --git a/src/table/sync.rs b/src/table/sync.rs index b344eb88..65231cd5 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -3,7 +3,7 @@ use std::convert::TryInto; use std::sync::{Arc, Mutex}; use std::time::{Duration, Instant}; -use futures::{pin_mut, select}; +use futures::{select}; use futures_util::future::*; use futures_util::stream::*; use rand::Rng; @@ -110,7 +110,7 @@ where let s3 = syncer.clone(); tokio::spawn(async move { - tokio::time::delay_for(Duration::from_secs(20)).await; + tokio::time::sleep(Duration::from_secs(20)).await; s3.add_full_sync(); }); @@ -142,23 +142,16 @@ where let mut nothing_to_do_since = Some(Instant::now()); while !*must_exit.borrow() { - let s_ring_recv = ring_recv.recv().fuse(); - let s_busy = busy_rx.recv().fuse(); - let s_must_exit = must_exit.recv().fuse(); - let s_timeout = tokio::time::delay_for(Duration::from_secs(1)).fuse(); - pin_mut!(s_ring_recv, s_busy, s_must_exit, s_timeout); - select! { - new_ring_r = s_ring_recv => { - if let Some(new_ring) = new_ring_r { - if !Arc::ptr_eq(&new_ring, &prev_ring) { - debug!("({}) Ring changed, adding full sync to syncer todo list", self.data.name); - self.add_full_sync(); - prev_ring = new_ring; - } + _ = ring_recv.changed().fuse() => { + let new_ring = ring_recv.borrow(); + if !Arc::ptr_eq(&new_ring, &prev_ring) { + debug!("({}) Ring changed, adding full sync to syncer todo list", self.data.name); + self.add_full_sync(); + prev_ring = new_ring.clone(); } } - busy_opt = s_busy => { + busy_opt = busy_rx.recv().fuse() => { if let Some(busy) = busy_opt { if busy { nothing_to_do_since = None; @@ -169,12 +162,8 @@ where } } } - must_exit_v = s_must_exit => { - if must_exit_v.unwrap_or(false) { - break; - } - } - _ = s_timeout => { + _ = must_exit.changed().fuse() => (), + _ = tokio::time::sleep(Duration::from_secs(1)).fuse() => { if nothing_to_do_since.map(|t| Instant::now() - t >= ANTI_ENTROPY_INTERVAL).unwrap_or(false) { nothing_to_do_since = None; debug!("({}) Interval passed, adding full sync to syncer todo list", self.data.name); @@ -213,7 +202,7 @@ where } } else { busy_tx.send(false).unwrap(); - tokio::time::delay_for(Duration::from_secs(1)).await; + tokio::time::sleep(Duration::from_secs(1)).await; } } } diff --git a/src/util/Cargo.toml b/src/util/Cargo.toml index 7bb7cb31..2ae4796c 100644 --- a/src/util/Cargo.toml +++ b/src/util/Cargo.toml @@ -28,14 +28,15 @@ rmp-serde = "0.14.3" serde = { version = "1.0", default-features = false, features = ["derive", "rc"] } serde_json = "1.0" chrono = "0.4" +arc-swap = "1.2" futures = "0.3" futures-util = "0.3" -tokio = { version = "0.2", default-features = false, features = ["rt-core", "rt-threaded", "io-driver", "net", "tcp", "time", "macros", "sync", "signal", "fs"] } +tokio = { version = "1.0", default-features = false, features = ["rt", "rt-multi-thread", "io-util", "net", "time", "macros", "sync", "signal", "fs"] } http = "0.2" -hyper = "0.13" -rustls = "0.17" +hyper = "0.14" +rustls = "0.19" webpki = "0.21" roxmltree = "0.11" diff --git a/src/util/background.rs b/src/util/background.rs index 3e600fdf..0ec9779a 100644 --- a/src/util/background.rs +++ b/src/util/background.rs @@ -2,11 +2,9 @@ use core::future::Future; use std::pin::Pin; use std::sync::Mutex; -use futures::future::join_all; -use futures::select; -use futures_util::future::*; +use arc_swap::ArcSwapOption; use std::sync::Arc; -use tokio::sync::{mpsc, watch, Notify}; +use tokio::sync::{mpsc, watch}; use crate::error::Error; @@ -14,12 +12,9 @@ type JobOutput = Result<(), Error>; type Job = Pin + Send>>; pub struct BackgroundRunner { - n_runners: usize, pub stop_signal: watch::Receiver, - queue_in: mpsc::UnboundedSender<(Job, bool)>, - queue_out: Mutex>, - job_notify: Notify, + queue_in: ArcSwapOption>, workers: Mutex>>, } @@ -27,50 +22,91 @@ pub struct BackgroundRunner { impl BackgroundRunner { pub fn new(n_runners: usize, stop_signal: watch::Receiver) -> Arc { let (queue_in, queue_out) = mpsc::unbounded_channel(); + + let mut workers = vec![]; + let queue_out = Arc::new(tokio::sync::Mutex::new(queue_out)); + + for i in 0..n_runners { + let queue_out = queue_out.clone(); + let stop_signal = stop_signal.clone(); + + workers.push(tokio::spawn(async move { + while let Some((job, cancellable)) = queue_out.lock().await.recv().await { + if cancellable && *stop_signal.borrow() { + continue; + } + if let Err(e) = job.await { + error!("Job failed: {}", e) + } + } + info!("Worker {} exiting", i); + })); + } + Arc::new(Self { - n_runners, stop_signal, - queue_in, - queue_out: Mutex::new(queue_out), - job_notify: Notify::new(), - workers: Mutex::new(Vec::new()), + queue_in: ArcSwapOption::new(Some(Arc::new(queue_in))), + workers: Mutex::new(workers), }) } pub async fn run(self: Arc) { - let mut workers = self.workers.lock().unwrap(); - for i in 0..self.n_runners { - workers.push(tokio::spawn(self.clone().runner(i))); - } - drop(workers); - let mut stop_signal = self.stop_signal.clone(); - while let Some(exit_now) = stop_signal.recv().await { + + loop { + let exit_now = match stop_signal.changed().await { + Ok(()) => *stop_signal.borrow(), + Err(e) => { + error!("Watch .changed() error: {}", e); + true + } + }; if exit_now { - let mut workers = self.workers.lock().unwrap(); - let workers_vec = workers.drain(..).collect::>(); - join_all(workers_vec).await; - return; + break; + } + } + + info!("Closing background job queue_in..."); + drop(self.queue_in.swap(None)); + + info!("Waiting for all workers to terminate..."); + while let Some(task) = self.workers.lock().unwrap().pop() { + if let Err(e) = task.await { + warn!("Error awaiting task: {}", e); } } } - pub fn spawn(&self, job: T) + // Spawn a task to be run in background + pub async fn spawn(&self, job: T) where T: Future + Send + 'static, { - let boxed: Job = Box::pin(job); - let _: Result<_, _> = self.queue_in.clone().send((boxed, false)); - self.job_notify.notify(); + match self.queue_in.load().as_ref() { + Some(chan) => { + let boxed: Job = Box::pin(job); + chan.send((boxed, false)).map_err(|_| "send error").unwrap(); + } + None => { + warn!("Doing background job now because we are exiting..."); + if let Err(e) = job.await { + warn!("Task failed: {}", e); + } + } + } } pub fn spawn_cancellable(&self, job: T) where T: Future + Send + 'static, { - let boxed: Job = Box::pin(job); - let _: Result<_, _> = self.queue_in.clone().send((boxed, true)); - self.job_notify.notify(); + match self.queue_in.load().as_ref() { + Some(chan) => { + let boxed: Job = Box::pin(job); + chan.send((boxed, false)).map_err(|_| "send error").unwrap(); + } + None => (), // drop job if we are exiting + } } pub fn spawn_worker(&self, name: String, worker: F) @@ -85,37 +121,4 @@ impl BackgroundRunner { info!("Worker exited: {}", name); })); } - - async fn runner(self: Arc, i: usize) { - let mut stop_signal = self.stop_signal.clone(); - loop { - let must_exit: bool = *stop_signal.borrow(); - if let Some(job) = self.dequeue_job(must_exit) { - if let Err(e) = job.await { - error!("Job failed: {}", e) - } - } else { - if must_exit { - info!("Background runner {} exiting", i); - return; - } - select! { - _ = self.job_notify.notified().fuse() => (), - _ = stop_signal.recv().fuse() => (), - } - } - } - } - - fn dequeue_job(&self, must_exit: bool) -> Option { - let mut queue = self.queue_out.lock().unwrap(); - while let Ok((job, cancellable)) = queue.try_recv() { - if cancellable && must_exit { - continue; - } else { - return Some(job); - } - } - None - } } diff --git a/src/util/error.rs b/src/util/error.rs index dbf71ac1..a9bf0824 100644 --- a/src/util/error.rs +++ b/src/util/error.rs @@ -8,16 +8,22 @@ use crate::data::*; pub enum RPCError { #[error(display = "Node is down: {:?}.", _0)] NodeDown(UUID), + #[error(display = "Timeout: {}", _0)] - Timeout(#[error(source)] tokio::time::Elapsed), + Timeout(#[error(source)] tokio::time::error::Elapsed), + #[error(display = "HTTP error: {}", _0)] HTTP(#[error(source)] http::Error), + #[error(display = "Hyper error: {}", _0)] Hyper(#[error(source)] hyper::Error), + #[error(display = "Messagepack encode error: {}", _0)] RMPEncode(#[error(source)] rmp_serde::encode::Error), + #[error(display = "Messagepack decode error: {}", _0)] RMPDecode(#[error(source)] rmp_serde::decode::Error), + #[error(display = "Too many errors: {:?}", _0)] TooManyErrors(Vec), } diff --git a/src/web/Cargo.toml b/src/web/Cargo.toml index 5cc8683c..8c340f6b 100644 --- a/src/web/Cargo.toml +++ b/src/web/Cargo.toml @@ -22,7 +22,7 @@ err-derive = "0.2.3" log = "0.4" futures = "0.3" http = "0.2" -hyper = "0.13" +hyper = "0.14" percent-encoding = "2.1.0" roxmltree = "0.11" idna = "0.2" -- cgit v1.2.3 From 6a8439fd1345ecae7414386f76dda7a03eb14df2 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 15 Mar 2021 23:14:12 +0100 Subject: Some improvements in background worker but we terminate late --- src/garage/admin_rpc.rs | 29 +++++--- src/garage/repair.rs | 6 +- src/garage/server.rs | 33 ++++----- src/model/block.rs | 13 ++-- src/model/key_table.rs | 3 +- src/model/object_table.rs | 3 +- src/model/version_table.rs | 3 +- src/rpc/membership.rs | 25 +++---- src/rpc/ring.rs | 10 +-- src/rpc/rpc_client.rs | 2 +- src/rpc/rpc_server.rs | 8 ++- src/table/merkle.rs | 5 +- src/table/sync.rs | 2 +- src/util/background.rs | 175 ++++++++++++++++++++++++++------------------- 14 files changed, 169 insertions(+), 148 deletions(-) diff --git a/src/garage/admin_rpc.rs b/src/garage/admin_rpc.rs index 40674e75..b2145ca5 100644 --- a/src/garage/admin_rpc.rs +++ b/src/garage/admin_rpc.rs @@ -193,7 +193,12 @@ impl AdminRpcHandler { let key_ids = self .garage .key_table - .get_range(&EmptyKey, None, Some(KeyFilter::Deleted(DeletedFilter::NotDeleted)), 10000) + .get_range( + &EmptyKey, + None, + Some(KeyFilter::Deleted(DeletedFilter::NotDeleted)), + 10000, + ) .await? .iter() .map(|k| (k.key_id.to_string(), k.name.get().clone())) @@ -257,15 +262,24 @@ impl AdminRpcHandler { } async fn get_existing_key(&self, pattern: &str) -> Result { - let candidates = self.garage + let candidates = self + .garage .key_table - .get_range(&EmptyKey, None, Some(KeyFilter::Matches(pattern.to_string())), 10) + .get_range( + &EmptyKey, + None, + Some(KeyFilter::Matches(pattern.to_string())), + 10, + ) .await? .into_iter() .filter(|k| !k.deleted.get()) .collect::>(); if candidates.len() != 1 { - Err(Error::Message(format!("{} matching keys", candidates.len()))) + Err(Error::Message(format!( + "{} matching keys", + candidates.len() + ))) } else { Ok(candidates.into_iter().next().unwrap()) } @@ -469,12 +483,7 @@ impl AdminRpcHandler { t.data.merkle_updater.merkle_tree_len() ) .unwrap(); - writeln!( - to, - " GC todo queue length: {}", - t.data.gc_todo_len() - ) - .unwrap(); + writeln!(to, " GC todo queue length: {}", t.data.gc_todo_len()).unwrap(); Ok(()) } } diff --git a/src/garage/repair.rs b/src/garage/repair.rs index 4ee66452..8200f1f0 100644 --- a/src/garage/repair.rs +++ b/src/garage/repair.rs @@ -16,11 +16,7 @@ pub struct Repair { } impl Repair { - pub async fn repair_worker( - &self, - opt: RepairOpt, - must_exit: watch::Receiver, - ) { + pub async fn repair_worker(&self, opt: RepairOpt, must_exit: watch::Receiver) { if let Err(e) = self.repair_worker_aux(opt, must_exit).await { warn!("Repair worker failed with error: {}", e); } diff --git a/src/garage/server.rs b/src/garage/server.rs index ce90ecab..c45a69b8 100644 --- a/src/garage/server.rs +++ b/src/garage/server.rs @@ -47,10 +47,15 @@ pub async fn run_server(config_file: PathBuf) -> Result<(), Error> { info!("Initializing background runner..."); let (send_cancel, watch_cancel) = watch::channel(false); - let background = BackgroundRunner::new(16, watch_cancel.clone()); + let (background, await_background_done) = BackgroundRunner::new(16, watch_cancel.clone()); info!("Initializing Garage main data store..."); - let garage = Garage::new(config, db, background.clone(), &mut rpc_server); + let garage = Garage::new(config.clone(), db, background, &mut rpc_server); + let bootstrap = garage.system.clone().bootstrap( + &config.bootstrap_peers[..], + config.consul_host, + config.consul_service_name, + ); info!("Crate admin RPC handler..."); AdminRpcHandler::new(garage.clone()).register_handler(&mut rpc_server); @@ -58,21 +63,13 @@ 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())); + let web_server = web_server::run_web_server(garage, wait_from(watch_cancel.clone())); futures::try_join!( - garage - .system - .clone() - .bootstrap( - &garage.config.bootstrap_peers[..], - garage.config.consul_host.clone(), - garage.config.consul_service_name.clone() - ) - .map(|rv| { - info!("Bootstrap done"); - Ok(rv) - }), + bootstrap.map(|rv| { + info!("Bootstrap done"); + Ok(rv) + }), run_rpc_server.map(|rv| { info!("RPC server exited"); rv @@ -85,9 +82,9 @@ pub async fn run_server(config_file: PathBuf) -> Result<(), Error> { info!("Web server exited"); rv }), - background.run().map(|rv| { - info!("Background runner exited"); - Ok(rv) + await_background_done.map(|rv| { + info!("Background runner exited: {:?}", rv); + Ok(()) }), shutdown_signal(send_cancel), )?; diff --git a/src/model/block.rs b/src/model/block.rs index 7185372c..a3958866 100644 --- a/src/model/block.rs +++ b/src/model/block.rs @@ -254,19 +254,18 @@ impl BlockManager { Ok(()) } - async fn resync_loop( - self: Arc, - mut must_exit: watch::Receiver, - ) { + async fn resync_loop(self: Arc, mut must_exit: watch::Receiver) { while !*must_exit.borrow() { if let Err(e) = self.resync_iter(&mut must_exit).await { warn!("Error in block resync loop: {}", e); - tokio::time::sleep(Duration::from_secs(10)).await; + select! { + _ = tokio::time::sleep(Duration::from_secs(10)).fuse() => (), + _ = must_exit.changed().fuse() => (), + } } } } - async fn resync_iter(&self, must_exit: &mut watch::Receiver) -> Result<(), Error> { if let Some(first_item) = self.resync_queue.iter().next() { let (time_bytes, hash_bytes) = first_item?; @@ -280,7 +279,7 @@ impl BlockManager { self.put_to_resync(&hash, RESYNC_RETRY_TIMEOUT)?; } self.resync_queue.remove(&time_bytes)?; - res?; // propagate error to delay main loop + res?; // propagate error to delay main loop } else { let delay = tokio::time::sleep(Duration::from_millis(time_msec - now)); select! { diff --git a/src/model/key_table.rs b/src/model/key_table.rs index 88d7b4ff..02dcf68c 100644 --- a/src/model/key_table.rs +++ b/src/model/key_table.rs @@ -109,7 +109,8 @@ impl TableSchema for KeyTable { KeyFilter::Deleted(df) => df.apply(entry.deleted.get()), KeyFilter::Matches(pat) => { let pat = pat.to_lowercase(); - entry.key_id.to_lowercase().starts_with(&pat) || entry.name.get().to_lowercase() == pat + entry.key_id.to_lowercase().starts_with(&pat) + || entry.name.get().to_lowercase() == pat } } } diff --git a/src/model/object_table.rs b/src/model/object_table.rs index d08bba70..99fad3ce 100644 --- a/src/model/object_table.rs +++ b/src/model/object_table.rs @@ -195,8 +195,7 @@ impl TableSchema for ObjectTable { fn updated(&self, old: Option, new: Option) { let version_table = self.version_table.clone(); - // TODO not cancellable - self.background.spawn_cancellable(async move { + self.background.spawn(async move { if let (Some(old_v), Some(new_v)) = (old, new) { // Propagate deletion of old versions for v in old_v.versions.iter() { diff --git a/src/model/version_table.rs b/src/model/version_table.rs index 19343890..841fbfea 100644 --- a/src/model/version_table.rs +++ b/src/model/version_table.rs @@ -110,8 +110,7 @@ impl TableSchema for VersionTable { fn updated(&self, old: Option, new: Option) { let block_ref_table = self.block_ref_table.clone(); - // TODO not cancellable - self.background.spawn_cancellable(async move { + self.background.spawn(async move { if let (Some(old_v), Some(new_v)) = (old, new) { // Propagate deletion of version blocks if new_v.deleted.get() && !old_v.deleted.get() { diff --git a/src/rpc/membership.rs b/src/rpc/membership.rs index 6cc3ed2e..4e9822fa 100644 --- a/src/rpc/membership.rs +++ b/src/rpc/membership.rs @@ -11,9 +11,9 @@ use futures::future::join_all; use futures::select; use futures_util::future::*; use serde::{Deserialize, Serialize}; +use tokio::io::AsyncWriteExt; use tokio::sync::watch; use tokio::sync::Mutex; -use tokio::io::AsyncWriteExt; use garage_util::background::BackgroundRunner; use garage_util::data::*; @@ -316,17 +316,16 @@ impl System { self.clone().ping_nodes(bootstrap_peers).await; let self2 = self.clone(); - self.clone() - .background - .spawn_worker(format!("ping loop"), |stop_signal| self2.ping_loop(stop_signal)); + self.background + .spawn_worker(format!("ping loop"), |stop_signal| { + self2.ping_loop(stop_signal) + }); if let (Some(consul_host), Some(consul_service_name)) = (consul_host, consul_service_name) { let self2 = self.clone(); - self.clone() - .background + self.background .spawn_worker(format!("Consul loop"), |stop_signal| { - self2 - .consul_loop(stop_signal, consul_host, consul_service_name) + self2.consul_loop(stop_signal, consul_host, consul_service_name) }); } } @@ -531,7 +530,7 @@ impl System { .broadcast(Message::AdvertiseConfig(adv.clone()), PING_TIMEOUT) .map(Ok), ); - self.background.spawn(self.clone().save_network_config()).await; + self.background.spawn(self.clone().save_network_config()); } Ok(Message::Ok) @@ -568,7 +567,7 @@ impl System { consul_host: String, consul_service_name: String, ) { - loop { + while !*stop_signal.borrow() { let restart_at = tokio::time::sleep(CONSUL_INTERVAL); match get_consul_nodes(&consul_host, &consul_service_name).await { @@ -583,11 +582,7 @@ impl System { select! { _ = restart_at.fuse() => (), - _ = stop_signal.changed().fuse() => { - if *stop_signal.borrow() { - return; - } - } + _ = stop_signal.changed().fuse() => (), } } } diff --git a/src/rpc/ring.rs b/src/rpc/ring.rs index 215ab031..a89b730c 100644 --- a/src/rpc/ring.rs +++ b/src/rpc/ring.rs @@ -161,11 +161,11 @@ impl Ring { }) .collect::>(); - eprintln!("RING: --"); - for e in ring.iter() { - eprintln!("{:?}", e); - } - eprintln!("END --"); + // eprintln!("RING: --"); + // for e in ring.iter() { + // eprintln!("{:?}", e); + // } + // eprintln!("END --"); Self { config, ring } } diff --git a/src/rpc/rpc_client.rs b/src/rpc/rpc_client.rs index 60286256..cffcf106 100644 --- a/src/rpc/rpc_client.rs +++ b/src/rpc/rpc_client.rs @@ -198,7 +198,7 @@ impl RpcClient { let wait_finished_fut = tokio::spawn(async move { resp_stream.collect::>().await; }); - self.background.spawn(wait_finished_fut.map(|_| Ok(()))).await; + self.background.spawn(wait_finished_fut.map(|_| Ok(()))); } Ok(results) diff --git a/src/rpc/rpc_server.rs b/src/rpc/rpc_server.rs index 3c5014c4..0c5bf6f9 100644 --- a/src/rpc/rpc_server.rs +++ b/src/rpc/rpc_server.rs @@ -13,9 +13,9 @@ use hyper::service::{make_service_fn, service_fn}; use hyper::{Body, Method, Request, Response, Server, StatusCode}; use serde::{Deserialize, Serialize}; use tokio::net::{TcpListener, TcpStream}; -use tokio_stream::wrappers::TcpListenerStream; use tokio_rustls::server::TlsStream; use tokio_rustls::TlsAcceptor; +use tokio_stream::wrappers::TcpListenerStream; use garage_util::config::TlsConfig; use garage_util::data::*; @@ -52,7 +52,11 @@ where trace!( "Request message: {}", - serde_json::to_string(&msg).unwrap_or("".into()).chars().take(100).collect::() + serde_json::to_string(&msg) + .unwrap_or("".into()) + .chars() + .take(100) + .collect::() ); match handler(msg, sockaddr).await { diff --git a/src/table/merkle.rs b/src/table/merkle.rs index 86289bf1..60b7833f 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -101,10 +101,7 @@ impl MerkleUpdater { ret } - async fn updater_loop( - self: Arc, - mut must_exit: watch::Receiver, - ) { + async fn updater_loop(self: Arc, mut must_exit: watch::Receiver) { while !*must_exit.borrow() { if let Some(x) = self.todo.iter().next() { match x { diff --git a/src/table/sync.rs b/src/table/sync.rs index 65231cd5..f8fef53c 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -3,7 +3,7 @@ use std::convert::TryInto; use std::sync::{Arc, Mutex}; use std::time::{Duration, Instant}; -use futures::{select}; +use futures::select; use futures_util::future::*; use futures_util::stream::*; use rand::Rng; diff --git a/src/util/background.rs b/src/util/background.rs index 0ec9779a..35d41d9f 100644 --- a/src/util/background.rs +++ b/src/util/background.rs @@ -1,10 +1,11 @@ use core::future::Future; use std::pin::Pin; -use std::sync::Mutex; - -use arc_swap::ArcSwapOption; use std::sync::Arc; -use tokio::sync::{mpsc, watch}; +use std::time::Duration; + +use futures::future::*; +use futures::select; +use tokio::sync::{mpsc, watch, Mutex}; use crate::error::Error; @@ -14,99 +15,115 @@ type Job = Pin + Send>>; pub struct BackgroundRunner { pub stop_signal: watch::Receiver, - queue_in: ArcSwapOption>, - - workers: Mutex>>, + queue_in: mpsc::UnboundedSender<(Job, bool)>, + worker_in: mpsc::UnboundedSender>, } impl BackgroundRunner { - pub fn new(n_runners: usize, stop_signal: watch::Receiver) -> Arc { - let (queue_in, queue_out) = mpsc::unbounded_channel(); + pub fn new( + n_runners: usize, + stop_signal: watch::Receiver, + ) -> (Arc, tokio::task::JoinHandle<()>) { + let (worker_in, mut worker_out) = mpsc::unbounded_channel(); + + let stop_signal_2 = stop_signal.clone(); + let await_all_done = tokio::spawn(async move { + loop { + let wkr = { + select! { + item = worker_out.recv().fuse() => { + match item { + Some(x) => x, + None => break, + } + } + _ = tokio::time::sleep(Duration::from_secs(10)).fuse() => { + if *stop_signal_2.borrow() { + break; + } else { + continue; + } + } + } + }; + if let Err(e) = wkr.await { + error!("Error while awaiting for worker: {}", e); + } + } + }); - let mut workers = vec![]; - let queue_out = Arc::new(tokio::sync::Mutex::new(queue_out)); + let (queue_in, queue_out) = mpsc::unbounded_channel(); + let queue_out = Arc::new(Mutex::new(queue_out)); for i in 0..n_runners { let queue_out = queue_out.clone(); let stop_signal = stop_signal.clone(); - workers.push(tokio::spawn(async move { - while let Some((job, cancellable)) = queue_out.lock().await.recv().await { - if cancellable && *stop_signal.borrow() { - continue; - } - if let Err(e) = job.await { - error!("Job failed: {}", e) + worker_in + .send(tokio::spawn(async move { + loop { + let (job, cancellable) = { + select! { + item = wait_job(&queue_out).fuse() => match item { + // We received a task, process it + Some(x) => x, + // We received a signal that no more tasks will ever be sent + // because the sending side was dropped. Exit now. + None => break, + }, + _ = tokio::time::sleep(Duration::from_secs(10)).fuse() => { + if *stop_signal.borrow() { + // Nothing has been going on for 10 secs, and we are shutting + // down. Exit now. + break; + } else { + // Nothing is going on but we don't want to exit. + continue; + } + } + } + }; + if cancellable && *stop_signal.borrow() { + continue; + } + if let Err(e) = job.await { + error!("Job failed: {}", e) + } } - } - info!("Worker {} exiting", i); - })); + info!("Background worker {} exiting", i); + })) + .unwrap(); } - Arc::new(Self { + let bgrunner = Arc::new(Self { stop_signal, - queue_in: ArcSwapOption::new(Some(Arc::new(queue_in))), - workers: Mutex::new(workers), - }) - } - - pub async fn run(self: Arc) { - let mut stop_signal = self.stop_signal.clone(); - - loop { - let exit_now = match stop_signal.changed().await { - Ok(()) => *stop_signal.borrow(), - Err(e) => { - error!("Watch .changed() error: {}", e); - true - } - }; - if exit_now { - break; - } - } - - info!("Closing background job queue_in..."); - drop(self.queue_in.swap(None)); - - info!("Waiting for all workers to terminate..."); - while let Some(task) = self.workers.lock().unwrap().pop() { - if let Err(e) = task.await { - warn!("Error awaiting task: {}", e); - } - } + queue_in, + worker_in, + }); + (bgrunner, await_all_done) } // Spawn a task to be run in background - pub async fn spawn(&self, job: T) + pub fn spawn(&self, job: T) where T: Future + Send + 'static, { - match self.queue_in.load().as_ref() { - Some(chan) => { - let boxed: Job = Box::pin(job); - chan.send((boxed, false)).map_err(|_| "send error").unwrap(); - } - None => { - warn!("Doing background job now because we are exiting..."); - if let Err(e) = job.await { - warn!("Task failed: {}", e); - } - } - } + let boxed: Job = Box::pin(job); + self.queue_in + .send((boxed, false)) + .map_err(|_| "could not put job in queue") + .unwrap(); } pub fn spawn_cancellable(&self, job: T) where T: Future + Send + 'static, { - match self.queue_in.load().as_ref() { - Some(chan) => { - let boxed: Job = Box::pin(job); - chan.send((boxed, false)).map_err(|_| "send error").unwrap(); - } - None => (), // drop job if we are exiting - } + let boxed: Job = Box::pin(job); + self.queue_in + .send((boxed, true)) + .map_err(|_| "could not put job in queue") + .unwrap(); } pub fn spawn_worker(&self, name: String, worker: F) @@ -114,11 +131,19 @@ impl BackgroundRunner { F: FnOnce(watch::Receiver) -> T + Send + 'static, T: Future + Send + 'static, { - let mut workers = self.workers.lock().unwrap(); let stop_signal = self.stop_signal.clone(); - workers.push(tokio::spawn(async move { + let task = tokio::spawn(async move { + info!("Worker started: {}", name); worker(stop_signal).await; info!("Worker exited: {}", name); - })); + }); + self.worker_in + .send(task) + .map_err(|_| "could not put job in queue") + .unwrap(); } } + +async fn wait_job(q: &Mutex>) -> Option<(Job, bool)> { + q.lock().await.recv().await +} -- cgit v1.2.3 From 1d9961e4118af0e26068e1d6c5c6c009a1292a88 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 16 Mar 2021 11:14:27 +0100 Subject: Simplify replication logic --- src/model/block.rs | 14 ++++++-------- src/model/garage.rs | 7 ++++++- src/rpc/ring.rs | 8 +++++++- src/table/gc.rs | 2 +- src/table/replication/fullcopy.rs | 33 ++++++++++++--------------------- src/table/replication/parameters.rs | 13 +++++++------ src/table/replication/sharded.rs | 20 ++++++++++++-------- src/table/sync.rs | 11 ++++------- src/table/table.rs | 10 +++++----- 9 files changed, 60 insertions(+), 58 deletions(-) diff --git a/src/model/block.rs b/src/model/block.rs index a3958866..41729685 100644 --- a/src/model/block.rs +++ b/src/model/block.rs @@ -319,10 +319,8 @@ impl BlockManager { if exists && !needed { trace!("Offloading block {:?}", hash); - let ring = self.system.ring.borrow().clone(); - - let mut who = self.replication.replication_nodes(&hash, &ring); - if who.len() < self.replication.write_quorum(&self.system) { + let mut who = self.replication.write_nodes(&hash); + if who.len() < self.replication.write_quorum() { return Err(Error::Message(format!("Not trying to offload block because we don't have a quorum of nodes to write to"))); } who.retain(|id| *id != self.system.id); @@ -367,7 +365,7 @@ impl BlockManager { ) .await?; } - trace!( + info!( "Deleting block {:?}, offload finished ({} / {})", hash, need_nodes.len(), @@ -391,7 +389,7 @@ impl BlockManager { } pub async fn rpc_get_block(&self, hash: &Hash) -> Result, Error> { - let who = self.replication.read_nodes(&hash, &self.system); + let who = self.replication.read_nodes(&hash); let resps = self .rpc_client .try_call_many( @@ -415,12 +413,12 @@ impl BlockManager { } pub async fn rpc_put_block(&self, hash: Hash, data: Vec) -> Result<(), Error> { - let who = self.replication.write_nodes(&hash, &self.system); + let who = self.replication.write_nodes(&hash); self.rpc_client .try_call_many( &who[..], Message::PutBlock(PutBlockMessage { hash, data }), - RequestStrategy::with_quorum(self.replication.write_quorum(&self.system)) + RequestStrategy::with_quorum(self.replication.write_quorum()) .with_timeout(BLOCK_RW_TIMEOUT), ) .await?; diff --git a/src/model/garage.rs b/src/model/garage.rs index ced3c29e..5f7a67c9 100644 --- a/src/model/garage.rs +++ b/src/model/garage.rs @@ -54,18 +54,23 @@ impl Garage { ); let data_rep_param = TableShardedReplication { + system: system.clone(), replication_factor: config.data_replication_factor, write_quorum: (config.data_replication_factor + 1) / 2, read_quorum: 1, }; let meta_rep_param = TableShardedReplication { + system: system.clone(), replication_factor: config.meta_replication_factor, write_quorum: (config.meta_replication_factor + 1) / 2, read_quorum: (config.meta_replication_factor + 1) / 2, }; - let control_rep_param = TableFullReplication::new(config.control_write_max_faults); + let control_rep_param = TableFullReplication { + system: system.clone(), + max_faults: config.control_write_max_faults, + }; info!("Initialize block manager..."); let block_manager = BlockManager::new( diff --git a/src/rpc/ring.rs b/src/rpc/ring.rs index a89b730c..a0fdcf84 100644 --- a/src/rpc/ring.rs +++ b/src/rpc/ring.rs @@ -170,6 +170,11 @@ impl Ring { Self { config, ring } } + pub fn partition_of(&self, from: &Hash) -> u16 { + let top = u16::from_be_bytes(from.as_slice()[0..2].try_into().unwrap()); + top >> (16 - PARTITION_BITS) + } + pub fn walk_ring(&self, from: &Hash, n: usize) -> Vec { if self.ring.len() != 1 << PARTITION_BITS { warn!("Ring not yet ready, read/writes will be lost!"); @@ -177,8 +182,9 @@ impl Ring { } let top = u16::from_be_bytes(from.as_slice()[0..2].try_into().unwrap()); - let partition_idx = (top >> (16 - PARTITION_BITS)) as usize; + assert_eq!(partition_idx, self.partition_of(from) as usize); + let partition = &self.ring[partition_idx]; let partition_top = diff --git a/src/table/gc.rs b/src/table/gc.rs index d37fdf35..061c5045 100644 --- a/src/table/gc.rs +++ b/src/table/gc.rs @@ -130,7 +130,7 @@ where let mut partitions = HashMap::new(); for (k, vhash, v) in entries { let pkh = Hash::try_from(&k[..32]).unwrap(); - let mut nodes = self.aux.replication.write_nodes(&pkh, &self.aux.system); + let mut nodes = self.aux.replication.write_nodes(&pkh); nodes.retain(|x| *x != self.aux.system.id); nodes.sort(); diff --git a/src/table/replication/fullcopy.rs b/src/table/replication/fullcopy.rs index a5faece9..aea8c1f3 100644 --- a/src/table/replication/fullcopy.rs +++ b/src/table/replication/fullcopy.rs @@ -8,21 +8,10 @@ use crate::replication::*; #[derive(Clone)] pub struct TableFullReplication { + pub system: Arc, pub max_faults: usize, } -#[derive(Clone)] -struct Neighbors { - ring: Arc, - neighbors: Vec, -} - -impl TableFullReplication { - pub fn new(max_faults: usize) -> Self { - TableFullReplication { max_faults } - } -} - impl TableReplication for TableFullReplication { // Full replication schema: all nodes store everything // Writes are disseminated in an epidemic manner in the network @@ -30,18 +19,23 @@ impl TableReplication for TableFullReplication { // Advantage: do all reads locally, extremely fast // Inconvenient: only suitable to reasonably small tables - fn read_nodes(&self, _hash: &Hash, system: &System) -> Vec { - vec![system.id] + fn partition_of(&self, _hash: &Hash) -> u16 { + 0u16 + } + + fn read_nodes(&self, _hash: &Hash) -> Vec { + vec![self.system.id] } fn read_quorum(&self) -> usize { 1 } - fn write_nodes(&self, hash: &Hash, system: &System) -> Vec { - self.replication_nodes(hash, system.ring.borrow().as_ref()) + fn write_nodes(&self, _hash: &Hash) -> Vec { + let ring = self.system.ring.borrow(); + ring.config.members.keys().cloned().collect::>() } - fn write_quorum(&self, system: &System) -> usize { - let nmembers = system.ring.borrow().config.members.len(); + fn write_quorum(&self) -> usize { + let nmembers = self.system.ring.borrow().config.members.len(); if nmembers > self.max_faults { nmembers - self.max_faults } else { @@ -52,9 +46,6 @@ impl TableReplication for TableFullReplication { self.max_faults } - fn replication_nodes(&self, _hash: &Hash, ring: &Ring) -> Vec { - ring.config.members.keys().cloned().collect::>() - } fn split_points(&self, _ring: &Ring) -> Vec { let mut ret = vec![]; ret.push([0u8; 32].into()); diff --git a/src/table/replication/parameters.rs b/src/table/replication/parameters.rs index 4607b050..ace82bd9 100644 --- a/src/table/replication/parameters.rs +++ b/src/table/replication/parameters.rs @@ -1,4 +1,3 @@ -use garage_rpc::membership::System; use garage_rpc::ring::Ring; use garage_util::data::*; @@ -7,16 +6,18 @@ pub trait TableReplication: Send + Sync { // See examples in table_sharded.rs and table_fullcopy.rs // To understand various replication methods + // Partition number of data item (for Merkle tree) + fn partition_of(&self, hash: &Hash) -> u16; + // Which nodes to send reads from - fn read_nodes(&self, hash: &Hash, system: &System) -> Vec; + fn read_nodes(&self, hash: &Hash) -> Vec; fn read_quorum(&self) -> usize; // Which nodes to send writes to - fn write_nodes(&self, hash: &Hash, system: &System) -> Vec; - fn write_quorum(&self, system: &System) -> usize; + fn write_nodes(&self, hash: &Hash) -> Vec; + fn write_quorum(&self) -> usize; fn max_write_errors(&self) -> usize; - // Which are the nodes that do actually replicate the data - fn replication_nodes(&self, hash: &Hash, ring: &Ring) -> Vec; + // Get partition boundaries fn split_points(&self, ring: &Ring) -> Vec; } diff --git a/src/table/replication/sharded.rs b/src/table/replication/sharded.rs index 886c7c08..966be31a 100644 --- a/src/table/replication/sharded.rs +++ b/src/table/replication/sharded.rs @@ -1,3 +1,5 @@ +use std::sync::Arc; + use garage_rpc::membership::System; use garage_rpc::ring::Ring; use garage_util::data::*; @@ -6,6 +8,7 @@ use crate::replication::*; #[derive(Clone)] pub struct TableShardedReplication { + pub system: Arc, pub replication_factor: usize, pub read_quorum: usize, pub write_quorum: usize, @@ -19,28 +22,29 @@ impl TableReplication for TableShardedReplication { // - reads are done on all of the nodes that replicate the data // - writes as well - fn read_nodes(&self, hash: &Hash, system: &System) -> Vec { - let ring = system.ring.borrow().clone(); + fn partition_of(&self, hash: &Hash) -> u16 { + self.system.ring.borrow().partition_of(hash) + } + + fn read_nodes(&self, hash: &Hash) -> Vec { + let ring = self.system.ring.borrow().clone(); ring.walk_ring(&hash, self.replication_factor) } fn read_quorum(&self) -> usize { self.read_quorum } - fn write_nodes(&self, hash: &Hash, system: &System) -> Vec { - let ring = system.ring.borrow().clone(); + fn write_nodes(&self, hash: &Hash) -> Vec { + let ring = self.system.ring.borrow(); ring.walk_ring(&hash, self.replication_factor) } - fn write_quorum(&self, _system: &System) -> usize { + fn write_quorum(&self) -> usize { self.write_quorum } fn max_write_errors(&self) -> usize { self.replication_factor - self.write_quorum } - fn replication_nodes(&self, hash: &Hash, ring: &Ring) -> Vec { - ring.walk_ring(&hash, self.replication_factor) - } fn split_points(&self, ring: &Ring) -> Vec { let mut ret = vec![]; diff --git a/src/table/sync.rs b/src/table/sync.rs index f8fef53c..ac0305e2 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -218,10 +218,7 @@ where let nodes = self .aux .replication - .write_nodes( - &hash_of_merkle_partition(partition.range.begin), - &self.aux.system, - ) + .write_nodes(&hash_of_merkle_partition(partition.range.begin)) .into_iter() .filter(|node| *node != my_id) .collect::>(); @@ -293,7 +290,7 @@ where let nodes = self .aux .replication - .write_nodes(&begin, &self.aux.system) + .write_nodes(&begin) .into_iter() .collect::>(); if nodes.contains(&self.aux.system.id) { @@ -303,7 +300,7 @@ where ); break; } - if nodes.len() < self.aux.replication.write_quorum(&self.aux.system) { + if nodes.len() < self.aux.replication.write_quorum() { return Err(Error::Message(format!( "Not offloading as we don't have a quorum of nodes to write to." ))); @@ -616,7 +613,7 @@ impl SyncTodo { let begin_hash = hash_of_merkle_partition(begin); let end_hash = hash_of_merkle_partition_opt(end); - let nodes = aux.replication.replication_nodes(&begin_hash, &ring); + let nodes = aux.replication.write_nodes(&begin_hash); let retain = nodes.contains(&my_id); if !retain { diff --git a/src/table/table.rs b/src/table/table.rs index 2d3c5fe9..2ce5868f 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -91,7 +91,7 @@ where pub async fn insert(&self, e: &F::E) -> Result<(), Error> { let hash = e.partition_key().hash(); - let who = self.aux.replication.write_nodes(&hash, &self.aux.system); + let who = self.aux.replication.write_nodes(&hash); //eprintln!("insert who: {:?}", who); let e_enc = Arc::new(ByteBuf::from(rmp_to_vec_all_named(e)?)); @@ -101,7 +101,7 @@ where .try_call_many( &who[..], rpc, - RequestStrategy::with_quorum(self.aux.replication.write_quorum(&self.aux.system)) + RequestStrategy::with_quorum(self.aux.replication.write_quorum()) .with_timeout(TABLE_RPC_TIMEOUT), ) .await?; @@ -113,7 +113,7 @@ where for entry in entries.iter() { let hash = entry.partition_key().hash(); - let who = self.aux.replication.write_nodes(&hash, &self.aux.system); + let who = self.aux.replication.write_nodes(&hash); let e_enc = Arc::new(ByteBuf::from(rmp_to_vec_all_named(entry)?)); for node in who { if !call_list.contains_key(&node) { @@ -150,7 +150,7 @@ where sort_key: &F::S, ) -> Result, Error> { let hash = partition_key.hash(); - let who = self.aux.replication.read_nodes(&hash, &self.aux.system); + let who = self.aux.replication.read_nodes(&hash); //eprintln!("get who: {:?}", who); let rpc = TableRPC::::ReadEntry(partition_key.clone(), sort_key.clone()); @@ -207,7 +207,7 @@ where limit: usize, ) -> Result, Error> { let hash = partition_key.hash(); - let who = self.aux.replication.read_nodes(&hash, &self.aux.system); + let who = self.aux.replication.read_nodes(&hash); let rpc = TableRPC::::ReadRange(partition_key.clone(), begin_sort_key, filter, limit); -- cgit v1.2.3 From 515029d026937d29395379c76188f509984b8ace Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 16 Mar 2021 11:43:58 +0100 Subject: Refactor code --- src/garage/admin_rpc.rs | 12 +++-- src/table/data.rs | 136 +++++++++++++++++++++++------------------------- src/table/gc.rs | 22 ++++---- src/table/merkle.rs | 61 ++++++++++++---------- src/table/sync.rs | 58 +++++++++++---------- src/table/table.rs | 53 +++++++++---------- 6 files changed, 175 insertions(+), 167 deletions(-) diff --git a/src/garage/admin_rpc.rs b/src/garage/admin_rpc.rs index b2145ca5..aec791a4 100644 --- a/src/garage/admin_rpc.rs +++ b/src/garage/admin_rpc.rs @@ -463,24 +463,28 @@ impl AdminRpcHandler { Ok(ret) } - fn gather_table_stats( + fn gather_table_stats( &self, to: &mut String, t: &Arc>, _opt: &StatsOpt, - ) -> Result<(), Error> { + ) -> Result<(), Error> + where + F: TableSchema + 'static, + R: TableReplication + 'static, + { writeln!(to, "\nTable stats for {}", t.data.name).unwrap(); writeln!(to, " number of items: {}", t.data.store.len()).unwrap(); writeln!( to, " Merkle updater todo queue length: {}", - t.data.merkle_updater.todo_len() + t.merkle_updater.todo_len() ) .unwrap(); writeln!( to, " Merkle tree size: {}", - t.data.merkle_updater.merkle_tree_len() + t.merkle_updater.merkle_tree_len() ) .unwrap(); writeln!(to, " GC todo queue length: {}", t.data.gc_todo_len()).unwrap(); diff --git a/src/table/data.rs b/src/table/data.rs index 0029b936..9aa2a3bc 100644 --- a/src/table/data.rs +++ b/src/table/data.rs @@ -4,62 +4,59 @@ use std::sync::Arc; use log::warn; use serde_bytes::ByteBuf; use sled::Transactional; +use tokio::sync::Notify; -use garage_util::background::BackgroundRunner; use garage_util::data::*; use garage_util::error::*; use crate::crdt::CRDT; -use crate::merkle::*; +use crate::replication::*; use crate::schema::*; -pub struct TableData { +pub struct TableData { pub name: String, - pub instance: F, + + pub(crate) instance: F, + pub(crate) replication: R, pub store: sled::Tree, + + pub(crate) merkle_tree: sled::Tree, + pub(crate) merkle_todo: sled::Tree, + pub(crate) merkle_todo_notify: Notify, pub(crate) gc_todo: sled::Tree, - pub merkle_updater: Arc, } -impl TableData +impl TableData where F: TableSchema, + R: TableReplication, { - pub fn new( - name: String, - instance: F, - db: &sled::Db, - background: Arc, - ) -> Arc { + pub fn new(name: String, instance: F, replication: R, db: &sled::Db) -> Arc { let store = db .open_tree(&format!("{}:table", name)) .expect("Unable to open DB tree"); - let merkle_todo_store = db - .open_tree(&format!("{}:merkle_todo", name)) - .expect("Unable to open DB Merkle TODO tree"); - let merkle_tree_store = db + let merkle_tree = db .open_tree(&format!("{}:merkle_tree", name)) .expect("Unable to open DB Merkle tree tree"); + let merkle_todo = db + .open_tree(&format!("{}:merkle_todo", name)) + .expect("Unable to open DB Merkle TODO tree"); let gc_todo = db .open_tree(&format!("{}:gc_todo", name)) .expect("Unable to open DB tree"); - let merkle_updater = MerkleUpdater::launch( - name.clone(), - background, - merkle_todo_store, - merkle_tree_store, - ); - Arc::new(Self { name, instance, + replication, store, + merkle_tree, + merkle_todo, + merkle_todo_notify: Notify::new(), gc_todo, - merkle_updater, }) } @@ -129,37 +126,36 @@ where let update = self.decode_entry(update_bytes)?; let tree_key = self.tree_key(update.partition_key(), update.sort_key()); - let changed = - (&self.store, &self.merkle_updater.todo).transaction(|(store, mkl_todo)| { - let (old_entry, new_entry) = match store.get(&tree_key)? { - Some(prev_bytes) => { - let old_entry = self - .decode_entry(&prev_bytes) - .map_err(sled::transaction::ConflictableTransactionError::Abort)?; - let mut new_entry = old_entry.clone(); - new_entry.merge(&update); - (Some(old_entry), new_entry) - } - None => (None, update.clone()), - }; - - if Some(&new_entry) != old_entry.as_ref() { - let new_bytes = rmp_to_vec_all_named(&new_entry) - .map_err(Error::RMPEncode) + let changed = (&self.store, &self.merkle_todo).transaction(|(store, mkl_todo)| { + let (old_entry, new_entry) = match store.get(&tree_key)? { + Some(prev_bytes) => { + let old_entry = self + .decode_entry(&prev_bytes) .map_err(sled::transaction::ConflictableTransactionError::Abort)?; - let new_bytes_hash = blake2sum(&new_bytes[..]); - mkl_todo.insert(tree_key.clone(), new_bytes_hash.as_slice())?; - store.insert(tree_key.clone(), new_bytes)?; - Ok(Some((old_entry, new_entry, new_bytes_hash))) - } else { - Ok(None) + let mut new_entry = old_entry.clone(); + new_entry.merge(&update); + (Some(old_entry), new_entry) } - })?; + None => (None, update.clone()), + }; + + if Some(&new_entry) != old_entry.as_ref() { + let new_bytes = rmp_to_vec_all_named(&new_entry) + .map_err(Error::RMPEncode) + .map_err(sled::transaction::ConflictableTransactionError::Abort)?; + let new_bytes_hash = blake2sum(&new_bytes[..]); + mkl_todo.insert(tree_key.clone(), new_bytes_hash.as_slice())?; + store.insert(tree_key.clone(), new_bytes)?; + Ok(Some((old_entry, new_entry, new_bytes_hash))) + } else { + Ok(None) + } + })?; if let Some((old_entry, new_entry, new_bytes_hash)) = changed { let is_tombstone = new_entry.is_tombstone(); self.instance.updated(old_entry, Some(new_entry)); - self.merkle_updater.todo_notify.notify_one(); + self.merkle_todo_notify.notify_one(); if is_tombstone { self.gc_todo.insert(&tree_key, new_bytes_hash.as_slice())?; } @@ -169,22 +165,21 @@ where } pub(crate) fn delete_if_equal(self: &Arc, k: &[u8], v: &[u8]) -> Result { - let removed = - (&self.store, &self.merkle_updater.todo).transaction(|(store, mkl_todo)| { - if let Some(cur_v) = store.get(k)? { - if cur_v == v { - store.remove(k)?; - mkl_todo.insert(k, vec![])?; - return Ok(true); - } + let removed = (&self.store, &self.merkle_todo).transaction(|(store, mkl_todo)| { + if let Some(cur_v) = store.get(k)? { + if cur_v == v { + store.remove(k)?; + mkl_todo.insert(k, vec![])?; + return Ok(true); } - Ok(false) - })?; + } + Ok(false) + })?; if removed { let old_entry = self.decode_entry(v)?; self.instance.updated(Some(old_entry), None); - self.merkle_updater.todo_notify.notify_one(); + self.merkle_todo_notify.notify_one(); } Ok(removed) } @@ -194,22 +189,21 @@ where k: &[u8], vhash: Hash, ) -> Result { - let removed = - (&self.store, &self.merkle_updater.todo).transaction(|(store, mkl_todo)| { - if let Some(cur_v) = store.get(k)? { - if blake2sum(&cur_v[..]) == vhash { - store.remove(k)?; - mkl_todo.insert(k, vec![])?; - return Ok(Some(cur_v)); - } + let removed = (&self.store, &self.merkle_todo).transaction(|(store, mkl_todo)| { + if let Some(cur_v) = store.get(k)? { + if blake2sum(&cur_v[..]) == vhash { + store.remove(k)?; + mkl_todo.insert(k, vec![])?; + return Ok(Some(cur_v)); } - Ok(None) - })?; + } + Ok(None) + })?; if let Some(old_v) = removed { let old_entry = self.decode_entry(&old_v[..])?; self.instance.updated(Some(old_entry), None); - self.merkle_updater.todo_notify.notify_one(); + self.merkle_todo_notify.notify_one(); Ok(true) } else { Ok(false) diff --git a/src/table/gc.rs b/src/table/gc.rs index 061c5045..d99e3e40 100644 --- a/src/table/gc.rs +++ b/src/table/gc.rs @@ -13,20 +13,20 @@ use tokio::sync::watch; use garage_util::data::*; use garage_util::error::Error; +use garage_rpc::membership::System; use garage_rpc::rpc_client::*; use garage_rpc::rpc_server::*; use crate::data::*; use crate::replication::*; use crate::schema::*; -use crate::table::*; const TABLE_GC_BATCH_SIZE: usize = 1024; const TABLE_GC_RPC_TIMEOUT: Duration = Duration::from_secs(30); pub struct TableGC { - data: Arc>, - aux: Arc>, + data: Arc>, + system: Arc, rpc_client: Arc>, } @@ -46,23 +46,23 @@ where R: TableReplication + 'static, { pub(crate) fn launch( - data: Arc>, - aux: Arc>, + data: Arc>, + system: Arc, rpc_server: &mut RpcServer, ) -> Arc { let rpc_path = format!("table_{}/gc", data.name); - let rpc_client = aux.system.rpc_client::(&rpc_path); + let rpc_client = system.rpc_client::(&rpc_path); let gc = Arc::new(Self { data: data.clone(), - aux: aux.clone(), + system: system.clone(), rpc_client, }); gc.register_handler(rpc_server, rpc_path); let gc1 = gc.clone(); - aux.system.background.spawn_worker( + system.background.spawn_worker( format!("GC loop for {}", data.name), move |must_exit: watch::Receiver| gc1.gc_loop(must_exit), ); @@ -130,8 +130,8 @@ where let mut partitions = HashMap::new(); for (k, vhash, v) in entries { let pkh = Hash::try_from(&k[..32]).unwrap(); - let mut nodes = self.aux.replication.write_nodes(&pkh); - nodes.retain(|x| *x != self.aux.system.id); + let mut nodes = self.data.replication.write_nodes(&pkh); + nodes.retain(|x| *x != self.system.id); nodes.sort(); if !partitions.contains_key(&nodes) { @@ -220,7 +220,7 @@ where let self2 = self.clone(); self.rpc_client - .set_local_handler(self.aux.system.id, move |msg| { + .set_local_handler(self.system.id, move |msg| { let self2 = self2.clone(); async move { self2.handle_rpc(&msg).await } }); diff --git a/src/table/merkle.rs b/src/table/merkle.rs index 60b7833f..8c3dcad9 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -9,12 +9,16 @@ use serde::{Deserialize, Serialize}; use sled::transaction::{ ConflictableTransactionError, ConflictableTransactionResult, TransactionalTree, }; -use tokio::sync::{watch, Notify}; +use tokio::sync::watch; use garage_util::background::BackgroundRunner; use garage_util::data::*; use garage_util::error::Error; +use crate::data::*; +use crate::replication::*; +use crate::schema::*; + pub type MerklePartition = [u8; 2]; pub fn hash_of_merkle_partition(p: MerklePartition) -> Hash { @@ -32,28 +36,30 @@ pub fn hash_of_merkle_partition_opt(p: Option) -> Hash { // 16 bits (two bytes) of item's partition keys' hashes. // It builds one Merkle tree for each of these 2**16 partitions. -pub struct MerkleUpdater { - table_name: String, +pub struct MerkleUpdater { + data: Arc>, background: Arc, // Content of the todo tree: items where // - key = the key of an item in the main table, ie hash(partition_key)+sort_key // - value = the hash of the full serialized item, if present, // or an empty vec if item is absent (deleted) - pub(crate) todo: sled::Tree, - pub(crate) todo_notify: Notify, + // Fields in data: + // pub(crate) merkle_todo: sled::Tree, + // pub(crate) merkle_todo_notify: Notify, // Content of the merkle tree: items where // - key = .bytes() for MerkleNodeKey // - value = serialization of a MerkleNode, assumed to be MerkleNode::empty if not found - pub(crate) merkle_tree: sled::Tree, + // Field in data: + // pub(crate) merkle_tree: sled::Tree, empty_node_hash: Hash, } #[derive(Clone, Debug, Serialize, Deserialize)] pub struct MerkleNodeKey { // partition: first 16 bits (two bytes) of the partition_key's hash - pub partition: MerklePartition, + pub partition: [u8; 2], // prefix: a prefix for the hash of full keys, i.e. hash(hash(partition_key)+sort_key) #[serde(with = "serde_bytes")] @@ -74,27 +80,26 @@ pub enum MerkleNode { Leaf(Vec, Hash), } -impl MerkleUpdater { +impl MerkleUpdater +where + F: TableSchema + 'static, + R: TableReplication + 'static, +{ pub(crate) fn launch( - table_name: String, + data: Arc>, background: Arc, - todo: sled::Tree, - merkle_tree: sled::Tree, ) -> Arc { let empty_node_hash = blake2sum(&rmp_to_vec_all_named(&MerkleNode::Empty).unwrap()[..]); let ret = Arc::new(Self { - table_name, + data, background, - todo, - todo_notify: Notify::new(), - merkle_tree, empty_node_hash, }); let ret2 = ret.clone(); ret.background.spawn_worker( - format!("Merkle tree updater for {}", ret.table_name), + format!("Merkle tree updater for {}", ret.data.name), |must_exit: watch::Receiver| ret2.updater_loop(must_exit), ); @@ -103,27 +108,27 @@ impl MerkleUpdater { async fn updater_loop(self: Arc, mut must_exit: watch::Receiver) { while !*must_exit.borrow() { - if let Some(x) = self.todo.iter().next() { + if let Some(x) = self.data.merkle_todo.iter().next() { match x { Ok((key, valhash)) => { if let Err(e) = self.update_item(&key[..], &valhash[..]) { warn!( "({}) Error while updating Merkle tree item: {}", - self.table_name, e + self.data.name, e ); } } Err(e) => { warn!( "({}) Error while iterating on Merkle todo tree: {}", - self.table_name, e + self.data.name, e ); tokio::time::sleep(Duration::from_secs(10)).await; } } } else { select! { - _ = self.todo_notify.notified().fuse() => (), + _ = self.data.merkle_todo_notify.notified().fuse() => (), _ = must_exit.changed().fuse() => (), } } @@ -143,18 +148,20 @@ impl MerkleUpdater { partition: k[0..2].try_into().unwrap(), prefix: vec![], }; - self.merkle_tree + self.data + .merkle_tree .transaction(|tx| self.update_item_rec(tx, k, khash, &key, new_vhash))?; let deleted = self - .todo + .data + .merkle_todo .compare_and_swap::<_, _, Vec>(k, Some(vhash_by), None)? .is_ok(); if !deleted { debug!( "({}) Item not deleted from Merkle todo because it changed: {:?}", - self.table_name, k + self.data.name, k ); } Ok(()) @@ -197,7 +204,7 @@ impl MerkleUpdater { // should not happen warn!( "({}) Replacing intermediate node with empty node, should not happen.", - self.table_name + self.data.name ); Some(MerkleNode::Empty) } else if children.len() == 1 { @@ -301,7 +308,7 @@ impl MerkleUpdater { // Access a node in the Merkle tree, used by the sync protocol pub(crate) fn read_node(&self, k: &MerkleNodeKey) -> Result { - let ent = self.merkle_tree.get(k.encode())?; + let ent = self.data.merkle_tree.get(k.encode())?; match ent { None => Ok(MerkleNode::Empty), Some(v) => Ok(rmp_serde::decode::from_read_ref::<_, MerkleNode>(&v[..])?), @@ -309,11 +316,11 @@ impl MerkleUpdater { } pub fn merkle_tree_len(&self) -> usize { - self.merkle_tree.len() + self.data.merkle_tree.len() } pub fn todo_len(&self) -> usize { - self.todo.len() + self.data.merkle_todo.len() } } diff --git a/src/table/sync.rs b/src/table/sync.rs index ac0305e2..9c148393 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -14,6 +14,7 @@ use tokio::sync::{mpsc, watch}; use garage_util::data::*; use garage_util::error::Error; +use garage_rpc::membership::System; use garage_rpc::ring::Ring; use garage_rpc::rpc_client::*; use garage_rpc::rpc_server::*; @@ -29,8 +30,9 @@ const TABLE_SYNC_RPC_TIMEOUT: Duration = Duration::from_secs(30); const ANTI_ENTROPY_INTERVAL: Duration = Duration::from_secs(10 * 60); pub struct TableSyncer { - data: Arc>, - aux: Arc>, + system: Arc, + data: Arc>, + merkle: Arc>, todo: Mutex, rpc_client: Arc>, @@ -76,18 +78,20 @@ where R: TableReplication + 'static, { pub(crate) fn launch( - data: Arc>, - aux: Arc>, + system: Arc, + data: Arc>, + merkle: Arc>, rpc_server: &mut RpcServer, ) -> Arc { let rpc_path = format!("table_{}/sync", data.name); - let rpc_client = aux.system.rpc_client::(&rpc_path); + let rpc_client = system.rpc_client::(&rpc_path); let todo = SyncTodo { todo: vec![] }; let syncer = Arc::new(Self { + system: system.clone(), data: data.clone(), - aux: aux.clone(), + merkle, todo: Mutex::new(todo), rpc_client, }); @@ -97,13 +101,13 @@ where let (busy_tx, busy_rx) = mpsc::unbounded_channel(); let s1 = syncer.clone(); - aux.system.background.spawn_worker( + system.background.spawn_worker( format!("table sync watcher for {}", data.name), move |must_exit: watch::Receiver| s1.watcher_task(must_exit, busy_rx), ); let s2 = syncer.clone(); - aux.system.background.spawn_worker( + system.background.spawn_worker( format!("table syncer for {}", data.name), move |must_exit: watch::Receiver| s2.syncer_task(must_exit, busy_tx), ); @@ -126,7 +130,7 @@ where let self2 = self.clone(); self.rpc_client - .set_local_handler(self.aux.system.id, move |msg| { + .set_local_handler(self.system.id, move |msg| { let self2 = self2.clone(); async move { self2.handle_rpc(&msg).await } }); @@ -137,8 +141,8 @@ where mut must_exit: watch::Receiver, mut busy_rx: mpsc::UnboundedReceiver, ) { - let mut prev_ring: Arc = self.aux.system.ring.borrow().clone(); - let mut ring_recv: watch::Receiver> = self.aux.system.ring.clone(); + let mut prev_ring: Arc = self.system.ring.borrow().clone(); + let mut ring_recv: watch::Receiver> = self.system.ring.clone(); let mut nothing_to_do_since = Some(Instant::now()); while !*must_exit.borrow() { @@ -178,7 +182,7 @@ where self.todo .lock() .unwrap() - .add_full_sync(&self.data, &self.aux); + .add_full_sync(&self.data, &self.system); } async fn syncer_task( @@ -213,10 +217,10 @@ where must_exit: &mut watch::Receiver, ) -> Result<(), Error> { if partition.retain { - let my_id = self.aux.system.id; + let my_id = self.system.id; let nodes = self - .aux + .data .replication .write_nodes(&hash_of_merkle_partition(partition.range.begin)) .into_iter() @@ -242,7 +246,7 @@ where warn!("({}) Sync error: {}", self.data.name, e); } } - if n_errors > self.aux.replication.max_write_errors() { + if n_errors > self.data.replication.max_write_errors() { return Err(Error::Message(format!( "Sync failed with too many nodes (should have been: {:?}).", nodes @@ -288,19 +292,19 @@ where if items.len() > 0 { let nodes = self - .aux + .data .replication .write_nodes(&begin) .into_iter() .collect::>(); - if nodes.contains(&self.aux.system.id) { + if nodes.contains(&self.system.id) { warn!( "({}) Interrupting offload as partitions seem to have changed", self.data.name ); break; } - if nodes.len() < self.aux.replication.write_quorum() { + if nodes.len() < self.data.replication.write_quorum() { return Err(Error::Message(format!( "Not offloading as we don't have a quorum of nodes to write to." ))); @@ -376,7 +380,7 @@ where partition: u16::to_be_bytes(i), prefix: vec![], }; - match self.data.merkle_updater.read_node(&key)? { + match self.merkle.read_node(&key)? { MerkleNode::Empty => (), x => { ret.push((key.partition, hash_of(&x)?)); @@ -458,7 +462,7 @@ where while !todo.is_empty() && !*must_exit.borrow() { let key = todo.pop_front().unwrap(); - let node = self.data.merkle_updater.read_node(&key)?; + let node = self.merkle.read_node(&key)?; match node { MerkleNode::Empty => { @@ -570,7 +574,7 @@ where } } SyncRPC::GetNode(k) => { - let node = self.data.merkle_updater.read_node(&k)?; + let node = self.merkle.read_node(&k)?; Ok(SyncRPC::Node(k.clone(), node)) } SyncRPC::Items(items) => { @@ -585,15 +589,15 @@ where impl SyncTodo { fn add_full_sync( &mut self, - data: &TableData, - aux: &TableAux, + data: &TableData, + system: &System, ) { - let my_id = aux.system.id; + let my_id = system.id; self.todo.clear(); - let ring = aux.system.ring.borrow().clone(); - let split_points = aux.replication.split_points(&ring); + let ring = system.ring.borrow().clone(); + let split_points = data.replication.split_points(&ring); for i in 0..split_points.len() { let begin: MerklePartition = { @@ -613,7 +617,7 @@ impl SyncTodo { let begin_hash = hash_of_merkle_partition(begin); let end_hash = hash_of_merkle_partition_opt(end); - let nodes = aux.replication.write_nodes(&begin_hash); + let nodes = data.replication.write_nodes(&begin_hash); let retain = nodes.contains(&my_id); if !retain { diff --git a/src/table/table.rs b/src/table/table.rs index 2ce5868f..f00b4239 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -16,20 +16,17 @@ use garage_rpc::rpc_server::*; use crate::crdt::CRDT; use crate::data::*; use crate::gc::*; +use crate::merkle::*; use crate::replication::*; use crate::schema::*; use crate::sync::*; const TABLE_RPC_TIMEOUT: Duration = Duration::from_secs(10); -pub struct TableAux { - pub system: Arc, - pub replication: R, -} - pub struct Table { - pub data: Arc>, - pub aux: Arc>, + pub system: Arc, + pub data: Arc>, + pub merkle_updater: Arc>, pub syncer: Arc>, rpc_client: Arc>>, } @@ -67,19 +64,22 @@ where let rpc_path = format!("table_{}", name); let rpc_client = system.rpc_client::>(&rpc_path); - let data = TableData::new(name, instance, db, system.background.clone()); + let data = TableData::new(name, instance, replication, db); - let aux = Arc::new(TableAux { - system, - replication, - }); + let merkle_updater = MerkleUpdater::launch(data.clone(), system.background.clone()); - let syncer = TableSyncer::launch(data.clone(), aux.clone(), rpc_server); - TableGC::launch(data.clone(), aux.clone(), rpc_server); + let syncer = TableSyncer::launch( + system.clone(), + data.clone(), + merkle_updater.clone(), + rpc_server, + ); + TableGC::launch(data.clone(), system.clone(), rpc_server); let table = Arc::new(Self { + system, data, - aux, + merkle_updater, syncer, rpc_client, }); @@ -91,7 +91,7 @@ where pub async fn insert(&self, e: &F::E) -> Result<(), Error> { let hash = e.partition_key().hash(); - let who = self.aux.replication.write_nodes(&hash); + let who = self.data.replication.write_nodes(&hash); //eprintln!("insert who: {:?}", who); let e_enc = Arc::new(ByteBuf::from(rmp_to_vec_all_named(e)?)); @@ -101,7 +101,7 @@ where .try_call_many( &who[..], rpc, - RequestStrategy::with_quorum(self.aux.replication.write_quorum()) + RequestStrategy::with_quorum(self.data.replication.write_quorum()) .with_timeout(TABLE_RPC_TIMEOUT), ) .await?; @@ -113,7 +113,7 @@ where for entry in entries.iter() { let hash = entry.partition_key().hash(); - let who = self.aux.replication.write_nodes(&hash); + let who = self.data.replication.write_nodes(&hash); let e_enc = Arc::new(ByteBuf::from(rmp_to_vec_all_named(entry)?)); for node in who { if !call_list.contains_key(&node) { @@ -137,7 +137,7 @@ where errors.push(e); } } - if errors.len() > self.aux.replication.max_write_errors() { + if errors.len() > self.data.replication.max_write_errors() { Err(Error::Message("Too many errors".into())) } else { Ok(()) @@ -150,7 +150,7 @@ where sort_key: &F::S, ) -> Result, Error> { let hash = partition_key.hash(); - let who = self.aux.replication.read_nodes(&hash); + let who = self.data.replication.read_nodes(&hash); //eprintln!("get who: {:?}", who); let rpc = TableRPC::::ReadEntry(partition_key.clone(), sort_key.clone()); @@ -159,7 +159,7 @@ where .try_call_many( &who[..], rpc, - RequestStrategy::with_quorum(self.aux.replication.read_quorum()) + RequestStrategy::with_quorum(self.data.replication.read_quorum()) .with_timeout(TABLE_RPC_TIMEOUT) .interrupt_after_quorum(true), ) @@ -190,8 +190,7 @@ where if not_all_same { let self2 = self.clone(); let ent2 = ret_entry.clone(); - self.aux - .system + self.system .background .spawn_cancellable(async move { self2.repair_on_read(&who[..], ent2).await }); } @@ -207,7 +206,7 @@ where limit: usize, ) -> Result, Error> { let hash = partition_key.hash(); - let who = self.aux.replication.read_nodes(&hash); + let who = self.data.replication.read_nodes(&hash); let rpc = TableRPC::::ReadRange(partition_key.clone(), begin_sort_key, filter, limit); @@ -216,7 +215,7 @@ where .try_call_many( &who[..], rpc, - RequestStrategy::with_quorum(self.aux.replication.read_quorum()) + RequestStrategy::with_quorum(self.data.replication.read_quorum()) .with_timeout(TABLE_RPC_TIMEOUT) .interrupt_after_quorum(true), ) @@ -248,7 +247,7 @@ where } if !to_repair.is_empty() { let self2 = self.clone(); - self.aux.system.background.spawn_cancellable(async move { + self.system.background.spawn_cancellable(async move { for (_, v) in to_repair.iter_mut() { self2.repair_on_read(&who[..], v.take().unwrap()).await?; } @@ -288,7 +287,7 @@ where let self2 = self.clone(); self.rpc_client - .set_local_handler(self.aux.system.id, move |msg| { + .set_local_handler(self.system.id, move |msg| { let self2 = self2.clone(); async move { self2.handle(&msg).await } }); -- cgit v1.2.3 From 0aad2f2e066b5914ac94bb319e7679e2e7761b2b Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 16 Mar 2021 11:47:39 +0100 Subject: some reordering --- src/table/gc.rs | 6 +++--- src/table/merkle.rs | 6 ++---- src/table/table.rs | 4 ++-- 3 files changed, 7 insertions(+), 9 deletions(-) diff --git a/src/table/gc.rs b/src/table/gc.rs index d99e3e40..20ebe3c7 100644 --- a/src/table/gc.rs +++ b/src/table/gc.rs @@ -25,8 +25,8 @@ const TABLE_GC_BATCH_SIZE: usize = 1024; const TABLE_GC_RPC_TIMEOUT: Duration = Duration::from_secs(30); pub struct TableGC { - data: Arc>, system: Arc, + data: Arc>, rpc_client: Arc>, } @@ -46,16 +46,16 @@ where R: TableReplication + 'static, { pub(crate) fn launch( - data: Arc>, system: Arc, + data: Arc>, rpc_server: &mut RpcServer, ) -> Arc { let rpc_path = format!("table_{}/gc", data.name); let rpc_client = system.rpc_client::(&rpc_path); let gc = Arc::new(Self { - data: data.clone(), system: system.clone(), + data: data.clone(), rpc_client, }); diff --git a/src/table/merkle.rs b/src/table/merkle.rs index 8c3dcad9..86fef4c5 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -38,7 +38,6 @@ pub fn hash_of_merkle_partition_opt(p: Option) -> Hash { pub struct MerkleUpdater { data: Arc>, - background: Arc, // Content of the todo tree: items where // - key = the key of an item in the main table, ie hash(partition_key)+sort_key @@ -86,19 +85,18 @@ where R: TableReplication + 'static, { pub(crate) fn launch( + background: &BackgroundRunner, data: Arc>, - background: Arc, ) -> Arc { let empty_node_hash = blake2sum(&rmp_to_vec_all_named(&MerkleNode::Empty).unwrap()[..]); let ret = Arc::new(Self { data, - background, empty_node_hash, }); let ret2 = ret.clone(); - ret.background.spawn_worker( + background.spawn_worker( format!("Merkle tree updater for {}", ret.data.name), |must_exit: watch::Receiver| ret2.updater_loop(must_exit), ); diff --git a/src/table/table.rs b/src/table/table.rs index f00b4239..421c8bf5 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -66,7 +66,7 @@ where let data = TableData::new(name, instance, replication, db); - let merkle_updater = MerkleUpdater::launch(data.clone(), system.background.clone()); + let merkle_updater = MerkleUpdater::launch(&system.background, data.clone()); let syncer = TableSyncer::launch( system.clone(), @@ -74,7 +74,7 @@ where merkle_updater.clone(), rpc_server, ); - TableGC::launch(data.clone(), system.clone(), rpc_server); + TableGC::launch(system.clone(), data.clone(), rpc_server); let table = Arc::new(Self { system, -- cgit v1.2.3 From 2a41b8238496dfeac5ee0f273445299cbd112ff6 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 16 Mar 2021 12:18:03 +0100 Subject: Simpler Merkle & sync --- src/rpc/ring.rs | 21 +++++- src/table/merkle.rs | 29 ++++---- src/table/replication/fullcopy.rs | 15 ++-- src/table/replication/parameters.rs | 10 ++- src/table/replication/sharded.rs | 22 ++---- src/table/sync.rs | 134 ++++++++++-------------------------- 6 files changed, 85 insertions(+), 146 deletions(-) diff --git a/src/rpc/ring.rs b/src/rpc/ring.rs index a0fdcf84..490fb1de 100644 --- a/src/rpc/ring.rs +++ b/src/rpc/ring.rs @@ -5,6 +5,11 @@ use serde::{Deserialize, Serialize}; use garage_util::data::*; +// A partition number is encoded on 16 bits, +// i.e. we have up to 2**16 partitions. +// (in practice we have exactly 2**PARTITION_BITS partitions) +pub type Partition = u16; + // TODO: make this constant parametrizable in the config file // For deployments with many nodes it might make sense to bump // it up to 10. @@ -17,6 +22,7 @@ const PARTITION_MASK_U16: u16 = ((1 << PARTITION_BITS) - 1) << (16 - PARTITION_B // (most deployments use a replication factor of 3, so...) pub const MAX_REPLICATION: usize = 3; + #[derive(Clone, Debug, Serialize, Deserialize)] pub struct NetworkConfig { pub members: HashMap, @@ -170,11 +176,24 @@ impl Ring { Self { config, ring } } - pub fn partition_of(&self, from: &Hash) -> u16 { + pub fn partition_of(&self, from: &Hash) -> Partition { let top = u16::from_be_bytes(from.as_slice()[0..2].try_into().unwrap()); top >> (16 - PARTITION_BITS) } + pub fn partitions(&self) -> Vec<(Partition, Hash)> { + let mut ret = vec![]; + + for (i, entry) in self.ring.iter().enumerate() { + ret.push((i as u16, entry.location)); + } + if ret.len() > 0 { + assert_eq!(ret[0].1, [0u8; 32].into()); + } + + ret + } + pub fn walk_ring(&self, from: &Hash, n: usize) -> Vec { if self.ring.len() != 1 << PARTITION_BITS { warn!("Ring not yet ready, read/writes will be lost!"); diff --git a/src/table/merkle.rs b/src/table/merkle.rs index 86fef4c5..db05cca4 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -1,4 +1,3 @@ -use std::convert::TryInto; use std::sync::Arc; use std::time::Duration; @@ -15,22 +14,12 @@ use garage_util::background::BackgroundRunner; use garage_util::data::*; use garage_util::error::Error; +use garage_rpc::ring::*; + use crate::data::*; use crate::replication::*; use crate::schema::*; -pub type MerklePartition = [u8; 2]; - -pub fn hash_of_merkle_partition(p: MerklePartition) -> Hash { - let mut partition_pos = [0u8; 32]; - partition_pos[0..2].copy_from_slice(&p[..]); - partition_pos.into() -} - -pub fn hash_of_merkle_partition_opt(p: Option) -> Hash { - p.map(hash_of_merkle_partition) - .unwrap_or([0xFFu8; 32].into()) -} // This modules partitions the data in 2**16 partitions, based on the top // 16 bits (two bytes) of item's partition keys' hashes. @@ -57,8 +46,8 @@ pub struct MerkleUpdater { #[derive(Clone, Debug, Serialize, Deserialize)] pub struct MerkleNodeKey { - // partition: first 16 bits (two bytes) of the partition_key's hash - pub partition: [u8; 2], + // partition number + pub partition: Partition, // prefix: a prefix for the hash of full keys, i.e. hash(hash(partition_key)+sort_key) #[serde(with = "serde_bytes")] @@ -143,7 +132,7 @@ where }; let key = MerkleNodeKey { - partition: k[0..2].try_into().unwrap(), + partition: self.data.replication.partition_of(&Hash::try_from(&k[0..32]).unwrap()), prefix: vec![], }; self.data @@ -325,7 +314,7 @@ where impl MerkleNodeKey { fn encode(&self) -> Vec { let mut ret = Vec::with_capacity(2 + self.prefix.len()); - ret.extend(&self.partition[..]); + ret.extend(&u16::to_be_bytes(self.partition)[..]); ret.extend(&self.prefix[..]); ret } @@ -443,3 +432,9 @@ fn test_intermediate_aux() { ] ); } + +impl MerkleNode { + pub fn is_empty(&self) -> bool { + *self == MerkleNode::Empty + } +} diff --git a/src/table/replication/fullcopy.rs b/src/table/replication/fullcopy.rs index aea8c1f3..bd658f63 100644 --- a/src/table/replication/fullcopy.rs +++ b/src/table/replication/fullcopy.rs @@ -1,7 +1,7 @@ use std::sync::Arc; use garage_rpc::membership::System; -use garage_rpc::ring::Ring; +use garage_rpc::ring::*; use garage_util::data::*; use crate::replication::*; @@ -19,10 +19,6 @@ impl TableReplication for TableFullReplication { // Advantage: do all reads locally, extremely fast // Inconvenient: only suitable to reasonably small tables - fn partition_of(&self, _hash: &Hash) -> u16 { - 0u16 - } - fn read_nodes(&self, _hash: &Hash) -> Vec { vec![self.system.id] } @@ -46,9 +42,10 @@ impl TableReplication for TableFullReplication { self.max_faults } - fn split_points(&self, _ring: &Ring) -> Vec { - let mut ret = vec![]; - ret.push([0u8; 32].into()); - ret + fn partition_of(&self, _hash: &Hash) -> Partition { + 0u16 + } + fn partitions(&self) -> Vec<(Partition, Hash)> { + vec![(0u16, [0u8; 32].into())] } } diff --git a/src/table/replication/parameters.rs b/src/table/replication/parameters.rs index ace82bd9..e46bd172 100644 --- a/src/table/replication/parameters.rs +++ b/src/table/replication/parameters.rs @@ -1,4 +1,4 @@ -use garage_rpc::ring::Ring; +use garage_rpc::ring::*; use garage_util::data::*; @@ -6,9 +6,6 @@ pub trait TableReplication: Send + Sync { // See examples in table_sharded.rs and table_fullcopy.rs // To understand various replication methods - // Partition number of data item (for Merkle tree) - fn partition_of(&self, hash: &Hash) -> u16; - // Which nodes to send reads from fn read_nodes(&self, hash: &Hash) -> Vec; fn read_quorum(&self) -> usize; @@ -18,6 +15,7 @@ pub trait TableReplication: Send + Sync { fn write_quorum(&self) -> usize; fn max_write_errors(&self) -> usize; - // Get partition boundaries - fn split_points(&self, ring: &Ring) -> Vec; + // Accessing partitions, for Merkle tree & sync + fn partition_of(&self, hash: &Hash) -> Partition; + fn partitions(&self) -> Vec<(Partition, Hash)>; } diff --git a/src/table/replication/sharded.rs b/src/table/replication/sharded.rs index 966be31a..dce74b03 100644 --- a/src/table/replication/sharded.rs +++ b/src/table/replication/sharded.rs @@ -1,7 +1,7 @@ use std::sync::Arc; use garage_rpc::membership::System; -use garage_rpc::ring::Ring; +use garage_rpc::ring::*; use garage_util::data::*; use crate::replication::*; @@ -22,10 +22,6 @@ impl TableReplication for TableShardedReplication { // - reads are done on all of the nodes that replicate the data // - writes as well - fn partition_of(&self, hash: &Hash) -> u16 { - self.system.ring.borrow().partition_of(hash) - } - fn read_nodes(&self, hash: &Hash) -> Vec { let ring = self.system.ring.borrow().clone(); ring.walk_ring(&hash, self.replication_factor) @@ -45,16 +41,10 @@ impl TableReplication for TableShardedReplication { self.replication_factor - self.write_quorum } - fn split_points(&self, ring: &Ring) -> Vec { - let mut ret = vec![]; - - for entry in ring.ring.iter() { - ret.push(entry.location); - } - if ret.len() > 0 { - assert_eq!(ret[0], [0u8; 32].into()); - } - - ret + fn partition_of(&self, hash: &Hash) -> Partition { + self.system.ring.borrow().partition_of(hash) + } + fn partitions(&self) -> Vec<(Partition, Hash)> { + self.system.ring.borrow().partitions() } } diff --git a/src/table/sync.rs b/src/table/sync.rs index 9c148393..f5c2ef33 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -1,5 +1,4 @@ use std::collections::VecDeque; -use std::convert::TryInto; use std::sync::{Arc, Mutex}; use std::time::{Duration, Instant}; @@ -15,7 +14,7 @@ use garage_util::data::*; use garage_util::error::Error; use garage_rpc::membership::System; -use garage_rpc::ring::Ring; +use garage_rpc::ring::*; use garage_rpc::rpc_client::*; use garage_rpc::rpc_server::*; @@ -38,20 +37,10 @@ pub struct TableSyncer { rpc_client: Arc>, } -type RootCk = Vec<(MerklePartition, Hash)>; - -#[derive(Debug, Clone, Copy, Serialize, Deserialize)] -pub struct PartitionRange { - begin: MerklePartition, - // if end is None, go all the way to partition 0xFFFF included - end: Option, -} - #[derive(Serialize, Deserialize)] pub(crate) enum SyncRPC { - RootCkHash(PartitionRange, Hash), - RootCkList(PartitionRange, RootCk), - CkNoDifference, + RootCkHash(Partition, Hash), + RootCkDifferent(bool), GetNode(MerkleNodeKey), Node(MerkleNodeKey, MerkleNode), Items(Vec>), @@ -66,7 +55,9 @@ struct SyncTodo { #[derive(Debug, Clone)] struct TodoPartition { - range: PartitionRange, + partition: Partition, + begin: Hash, + end: Hash, // Are we a node that stores this partition or not? retain: bool, @@ -222,7 +213,7 @@ where let nodes = self .data .replication - .write_nodes(&hash_of_merkle_partition(partition.range.begin)) + .write_nodes(&partition.begin) .into_iter() .filter(|node| *node != my_id) .collect::>(); @@ -254,8 +245,8 @@ where } } else { self.offload_partition( - &hash_of_merkle_partition(partition.range.begin), - &hash_of_merkle_partition_opt(partition.range.end), + &partition.begin, + &partition.end, must_exit, ) .await?; @@ -364,30 +355,13 @@ where // side to the other will happen when the other side syncs with us, // which they also do regularly. - fn get_root_ck(&self, range: PartitionRange) -> Result { - let begin = u16::from_be_bytes(range.begin); - let range_iter = match range.end { - Some(end) => { - let end = u16::from_be_bytes(end); - begin..=(end - 1) - } - None => begin..=0xFFFF, + fn get_root_ck(&self, partition: Partition) -> Result<(MerkleNodeKey, MerkleNode), Error> { + let key = MerkleNodeKey { + partition, + prefix: vec![], }; - - let mut ret = vec![]; - for i in range_iter { - let key = MerkleNodeKey { - partition: u16::to_be_bytes(i), - prefix: vec![], - }; - match self.merkle.read_node(&key)? { - MerkleNode::Empty => (), - x => { - ret.push((key.partition, hash_of(&x)?)); - } - } - } - Ok(ret) + let node = self.merkle.read_node(&key)?; + Ok((key, node)) } async fn do_sync_with( @@ -396,7 +370,7 @@ where who: UUID, must_exit: watch::Receiver, ) -> Result<(), Error> { - let root_ck = self.get_root_ck(partition.range)?; + let (root_ck_key, root_ck) = self.get_root_ck(partition.partition)?; if root_ck.is_empty() { debug!( "({}) Sync {:?} with {:?}: partition is empty.", @@ -404,51 +378,29 @@ where ); return Ok(()); } + let root_ck_hash = hash_of::(&root_ck)?; - let root_ck_hash = hash_of(&root_ck)?; - - // If their root checksum has level > than us, use that as a reference + // Check if they have the same root checksum + // If so, do nothing. let root_resp = self .rpc_client .call( who, - SyncRPC::RootCkHash(partition.range, root_ck_hash), + SyncRPC::RootCkHash(partition.partition, root_ck_hash), TABLE_SYNC_RPC_TIMEOUT, ) .await?; let mut todo = match root_resp { - SyncRPC::CkNoDifference => { + SyncRPC::RootCkDifferent(false) => { debug!( "({}) Sync {:?} with {:?}: no difference", self.data.name, partition, who ); return Ok(()); } - SyncRPC::RootCkList(_, their_root_ck) => { - let join = join_ordered(&root_ck[..], &their_root_ck[..]); - let mut todo = VecDeque::new(); - for (p, v1, v2) in join.iter() { - let diff = match (v1, v2) { - (Some(_), None) | (None, Some(_)) => true, - (Some(a), Some(b)) => a != b, - _ => false, - }; - if diff { - todo.push_back(MerkleNodeKey { - partition: **p, - prefix: vec![], - }); - } - } - debug!( - "({}) Sync {:?} with {:?}: todo.len() = {}", - self.data.name, - partition, - who, - todo.len() - ); - todo + SyncRPC::RootCkDifferent(true) => { + VecDeque::from(vec![root_ck_key]) } x => { return Err(Error::Message(format!( @@ -565,13 +517,9 @@ where async fn handle_rpc(self: &Arc, message: &SyncRPC) -> Result { match message { SyncRPC::RootCkHash(range, h) => { - let root_ck = self.get_root_ck(*range)?; - let hash = hash_of(&root_ck)?; - if hash == *h { - Ok(SyncRPC::CkNoDifference) - } else { - Ok(SyncRPC::RootCkList(*range, root_ck)) - } + let (_root_ck_key, root_ck) = self.get_root_ck(*range)?; + let hash = hash_of::(&root_ck)?; + Ok(SyncRPC::RootCkDifferent(hash != *h)) } SyncRPC::GetNode(k) => { let node = self.merkle.read_node(&k)?; @@ -596,39 +544,31 @@ impl SyncTodo { self.todo.clear(); - let ring = system.ring.borrow().clone(); - let split_points = data.replication.split_points(&ring); + let partitions = data.replication.partitions(); - for i in 0..split_points.len() { - let begin: MerklePartition = { - let b = split_points[i]; - assert_eq!(b.as_slice()[2..], [0u8; 30][..]); - b.as_slice()[..2].try_into().unwrap() - }; + for i in 0..partitions.len() { + let begin = partitions[i].1; - let end: Option = if i + 1 < split_points.len() { - let e = split_points[i + 1]; - assert_eq!(e.as_slice()[2..], [0u8; 30][..]); - Some(e.as_slice()[..2].try_into().unwrap()) + let end = if i + 1 < partitions.len() { + partitions[i+1].1 } else { - None + [0xFFu8; 32].into() }; - let begin_hash = hash_of_merkle_partition(begin); - let end_hash = hash_of_merkle_partition_opt(end); - - let nodes = data.replication.write_nodes(&begin_hash); + let nodes = data.replication.write_nodes(&begin); let retain = nodes.contains(&my_id); if !retain { // Check if we have some data to send, otherwise skip - if data.store.range(begin_hash..end_hash).next().is_none() { + if data.store.range(begin..end).next().is_none() { continue; } } self.todo.push(TodoPartition { - range: PartitionRange { begin, end }, + partition: partitions[i].0, + begin, + end, retain, }); } -- cgit v1.2.3 From f4346cc5f45839ace93d2d11ce6beea632fd8f2c Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 16 Mar 2021 15:58:40 +0100 Subject: Update dependencies --- Cargo.lock | 254 ++++++++++++++----------------------------------- src/api/Cargo.toml | 17 ++-- src/api/error.rs | 8 +- src/api/s3_put.rs | 8 +- src/api/signature.rs | 26 ++--- src/garage/Cargo.toml | 9 +- src/model/Cargo.toml | 11 +-- src/rpc/Cargo.toml | 12 +-- src/rpc/ring.rs | 1 - src/rpc/rpc_client.rs | 3 +- src/rpc/rpc_server.rs | 3 +- src/table/Cargo.toml | 8 +- src/table/merkle.rs | 11 +-- src/table/sync.rs | 16 +--- src/util/Cargo.toml | 15 +-- src/util/background.rs | 6 +- src/util/data.rs | 4 +- src/web/Cargo.toml | 3 +- 18 files changed, 142 insertions(+), 273 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8e8641ef..39d54685 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -11,12 +11,6 @@ dependencies = [ "memchr", ] -[[package]] -name = "arc-swap" -version = "0.4.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dabe5a181f83789739c194cbe5a897dde195078fac08568d09221fd6137a7ba8" - [[package]] name = "arc-swap" version = "1.2.0" @@ -33,17 +27,6 @@ dependencies = [ "odds", ] -[[package]] -name = "async-trait" -version = "0.1.42" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d3a45e77e34375a7923b1e8febb049bb011f064714a8e17a1a616fef01da13d" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] - [[package]] name = "atty" version = "0.2.14" @@ -80,20 +63,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "10a5720225ef5daecf08657f23791354e1685a8c91a4c60c7f3d3b2892f978f4" dependencies = [ "crypto-mac 0.8.0", - "digest 0.9.0", - "opaque-debug 0.3.0", -] - -[[package]] -name = "block-buffer" -version = "0.7.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0940dc441f31689269e10ac70eb1002a3a1d3ad1390e030043662eb7fe4688b" -dependencies = [ - "block-padding", - "byte-tools", - "byteorder", - "generic-array 0.12.3", + "digest", + "opaque-debug", ] [[package]] @@ -102,16 +73,7 @@ version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4152116fd6e9dadb291ae18fc1ec3575ed6d84c29642d97890f4b4a3417297e4" dependencies = [ - "generic-array 0.14.4", -] - -[[package]] -name = "block-padding" -version = "0.1.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa79dedbb091f449f1f39e53edf88d5dbe95f895dae6135a8d7b881fb5af73f5" -dependencies = [ - "byte-tools", + "generic-array", ] [[package]] @@ -120,28 +82,12 @@ version = "3.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2e8c087f005730276d1096a652e92a8bacee2e2472bcc9715a74d2bec38b5820" -[[package]] -name = "byte-tools" -version = "0.3.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3b5ca7a04898ad4bcd41c90c5285445ff5b791899bb1b0abdd2a2aa791211d7" - [[package]] name = "byteorder" version = "1.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ae44d1a3d5a19df61dd0c8beb138458ac2a53a7ac09eba97d55592540004306b" -[[package]] -name = "bytes" -version = "0.4.12" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "206fdffcfa2df7cbe15601ef46c813fce0965eb3286db6b56c583b814b51c81c" -dependencies = [ - "byteorder", - "iovec", -] - [[package]] name = "bytes" version = "1.0.1" @@ -190,6 +136,12 @@ dependencies = [ "unicode-width", ] +[[package]] +name = "cpuid-bool" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8aebca1129a03dc6dc2b127edd729435bbc4a37e1d5f4d7513165089ceb02634" + [[package]] name = "crc32fast" version = "1.2.1" @@ -225,33 +177,24 @@ dependencies = [ "loom", ] -[[package]] -name = "crypto-mac" -version = "0.7.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4434400df11d95d556bac068ddfedd482915eb18fe8bea89bc80b6e4b1c179e5" -dependencies = [ - "generic-array 0.12.3", - "subtle 1.0.0", -] - [[package]] name = "crypto-mac" version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b584a330336237c1eecd3e94266efb216c56ed91225d634cb2991c5f3fd1aeab" dependencies = [ - "generic-array 0.14.4", - "subtle 2.4.0", + "generic-array", + "subtle", ] [[package]] -name = "digest" -version = "0.8.1" +name = "crypto-mac" +version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3d0c8c8752312f9713efd397ff63acb9f85585afbf179282e720e7704954dd5" +checksum = "4857fd85a0c34b3c3297875b747c1e02e06b6a0ea32dd892d8192b9ce0813ea6" dependencies = [ - "generic-array 0.12.3", + "generic-array", + "subtle", ] [[package]] @@ -260,7 +203,7 @@ version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d3dd60d1080a57a05ab032377049e0591415d2b31afd7028356dbf3cc6dcb066" dependencies = [ - "generic-array 0.14.4", + "generic-array", ] [[package]] @@ -278,9 +221,9 @@ dependencies = [ [[package]] name = "err-derive" -version = "0.2.4" +version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22deed3a8124cff5fa835713fa105621e43bbdc46690c3a6b68328a012d350d4" +checksum = "dcc7f65832b62ed38939f98966824eb6294911c3629b0e9a262bfb80836d9686" dependencies = [ "proc-macro-error", "proc-macro2", @@ -290,12 +233,6 @@ dependencies = [ "synstructure", ] -[[package]] -name = "fake-simd" -version = "0.1.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e88a8acf291dafb59c2d96e8f59828f3838bb1a70398823ade51a84de6a6deed" - [[package]] name = "fasthash" version = "0.4.0" @@ -458,7 +395,7 @@ dependencies = [ name = "garage" version = "0.1.1" dependencies = [ - "bytes 0.4.12", + "bytes", "futures", "futures-util", "garage_api", @@ -471,10 +408,9 @@ dependencies = [ "hex", "log", "pretty_env_logger", - "rand 0.7.3", + "rand 0.8.3", "rmp-serde", "serde", - "sha2", "sled", "structopt", "tokio", @@ -486,9 +422,9 @@ name = "garage_api" version = "0.1.1" dependencies = [ "base64", - "bytes 0.4.12", + "bytes", "chrono", - "crypto-mac 0.7.0", + "crypto-mac 0.10.0", "err-derive", "futures", "futures-util", @@ -504,7 +440,6 @@ dependencies = [ "log", "md-5", "percent-encoding", - "rand 0.7.3", "roxmltree", "sha2", "tokio", @@ -515,9 +450,7 @@ dependencies = [ name = "garage_model" version = "0.1.1" dependencies = [ - "arc-swap 0.4.8", - "async-trait", - "bytes 0.4.12", + "arc-swap", "futures", "futures-util", "garage_rpc", @@ -525,11 +458,10 @@ dependencies = [ "garage_util", "hex", "log", - "rand 0.7.3", + "rand 0.8.3", "rmp-serde", "serde", "serde_bytes", - "sha2", "sled", "tokio", ] @@ -538,8 +470,8 @@ dependencies = [ name = "garage_rpc" version = "0.1.1" dependencies = [ - "arc-swap 0.4.8", - "bytes 0.4.12", + "arc-swap", + "bytes", "futures", "futures-util", "garage_util", @@ -549,12 +481,10 @@ dependencies = [ "hyper", "hyper-rustls", "log", - "rand 0.7.3", "rmp-serde", "rustls", "serde", "serde_json", - "sha2", "tokio", "tokio-rustls", "tokio-stream", @@ -565,16 +495,14 @@ dependencies = [ name = "garage_table" version = "0.1.1" dependencies = [ - "async-trait", - "bytes 0.4.12", + "bytes", "futures", "futures-util", "garage_rpc", "garage_util", - "hex", "hexdump", "log", - "rand 0.7.3", + "rand 0.8.3", "rmp-serde", "serde", "serde_bytes", @@ -586,20 +514,17 @@ dependencies = [ name = "garage_util" version = "0.1.1" dependencies = [ - "arc-swap 1.2.0", "blake2", "chrono", "err-derive", "fasthash", "futures", - "futures-util", "hex", "http", "hyper", "log", - "rand 0.7.3", + "rand 0.8.3", "rmp-serde", - "roxmltree", "rustls", "serde", "serde_json", @@ -625,7 +550,6 @@ dependencies = [ "idna", "log", "percent-encoding", - "roxmltree", ] [[package]] @@ -647,15 +571,6 @@ dependencies = [ "winapi", ] -[[package]] -name = "generic-array" -version = "0.12.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c68f0274ae0e023facc3c97b2e00f076be70e254bc851d972503b328db79b2ec" -dependencies = [ - "typenum", -] - [[package]] name = "generic-array" version = "0.14.4" @@ -678,13 +593,13 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.1.16" +version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8fc3cb4d91f53b50155bdcfd23f6a4c39ae1969c2ae85982b135750cccaf5fce" +checksum = "c9495705279e7140bf035dde1f6e750c162df8b625267cd52cc44e0b156732c8" dependencies = [ "cfg-if 1.0.0", "libc", - "wasi 0.9.0+wasi-snapshot-preview1", + "wasi", ] [[package]] @@ -715,7 +630,7 @@ version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d832b01df74254fe364568d6ddc294443f61cbec82816b60904303af87efae78" dependencies = [ - "bytes 1.0.1", + "bytes", "fnv", "futures-core", "futures-sink", @@ -754,9 +669,9 @@ dependencies = [ [[package]] name = "hex" -version = "0.3.2" +version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "805026a5d0141ffc30abb3be3173848ad46a1b1664fe632428479619a3644d77" +checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" [[package]] name = "hexdump" @@ -770,12 +685,12 @@ dependencies = [ [[package]] name = "hmac" -version = "0.7.1" +version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5dcb5e64cda4c23119ab41ba960d1e170a774c8e4b9d9e6a9bc18aabf5e59695" +checksum = "c1441c6b1e930e2817404b5046f1f989899143a12bf92de603b69f4e0aee1e15" dependencies = [ - "crypto-mac 0.7.0", - "digest 0.8.1", + "crypto-mac 0.10.0", + "digest", ] [[package]] @@ -784,7 +699,7 @@ version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7245cd7449cc792608c3c8a9eaf69bd4eabbabf802713748fd739c98b82f0747" dependencies = [ - "bytes 1.0.1", + "bytes", "fnv", "itoa", ] @@ -795,7 +710,7 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2861bd27ee074e5ee891e8b539837a9430012e249d7f0ca2d795650f579c1994" dependencies = [ - "bytes 1.0.1", + "bytes", "http", ] @@ -832,7 +747,7 @@ version = "0.14.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e8e946c2b1349055e0b72ae281b238baf1a3ea7307c7e9f9d64673bdd9c26ac7" dependencies = [ - "bytes 1.0.1", + "bytes", "futures-channel", "futures-core", "futures-util", @@ -895,15 +810,6 @@ dependencies = [ "cfg-if 1.0.0", ] -[[package]] -name = "iovec" -version = "0.1.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b2b3ea6ff95e175473f8ffe6a7eb7c00d054240321b84c57051175fe3c1e075e" -dependencies = [ - "libc", -] - [[package]] name = "itertools" version = "0.4.19" @@ -978,9 +884,9 @@ version = "0.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7b5a279bb9607f9f53c22d496eade00d138d1bdcccd07d74650387cf94942a15" dependencies = [ - "block-buffer 0.9.0", - "digest 0.9.0", - "opaque-debug 0.3.0", + "block-buffer", + "digest", + "opaque-debug", ] [[package]] @@ -1077,12 +983,6 @@ version = "1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "13bd41f508810a131401606d54ac32a467c97172d74ba7662562ebba5ad07fa0" -[[package]] -name = "opaque-debug" -version = "0.2.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2839e79665f131bdb5782e51f2c6c9599c133c6098982a54c794358bf432529c" - [[package]] name = "opaque-debug" version = "0.3.0" @@ -1243,25 +1143,24 @@ dependencies = [ [[package]] name = "rand" -version = "0.7.3" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a6b1679d49b24bbfe0c803429aa1874472f50d9b363131f0e89fc356b544d03" +checksum = "0ef9e7e66b4468674bfcb0c81af8b7fa0bb154fa9f28eb840da5c447baeb8d7e" dependencies = [ - "getrandom", "libc", "rand_chacha", - "rand_core 0.5.1", + "rand_core 0.6.2", "rand_hc", ] [[package]] name = "rand_chacha" -version = "0.2.2" +version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f4c8ed856279c9737206bf725bf36935d8666ead7aa69b52be55af369d193402" +checksum = "e12735cf05c9e10bf21534da50a147b924d555dc7a547c42e6bb2d5b6017ae0d" dependencies = [ "ppv-lite86", - "rand_core 0.5.1", + "rand_core 0.6.2", ] [[package]] @@ -1281,20 +1180,20 @@ checksum = "9c33a3c44ca05fa6f1807d8e6743f3824e8509beca625669633be0acbdf509dc" [[package]] name = "rand_core" -version = "0.5.1" +version = "0.6.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90bde5296fc891b0cef12a6d03ddccc162ce7b2aff54160af9338f8d40df6d19" +checksum = "34cf66eb183df1c5876e2dcf6b13d57340741e8dc255b48e40a26de954d06ae7" dependencies = [ "getrandom", ] [[package]] name = "rand_hc" -version = "0.2.0" +version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca3129af7b92a17112d59ad498c6f81eaf463253766b90396d39ea7a39d6613c" +checksum = "3190ef7066a446f2e7f42e239d161e905420ccab01eb967c9eb27d21b2322a73" dependencies = [ - "rand_core 0.5.1", + "rand_core 0.6.2", ] [[package]] @@ -1360,9 +1259,9 @@ dependencies = [ [[package]] name = "rmp-serde" -version = "0.14.4" +version = "0.15.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ce7d70c926fe472aed493b902010bccc17fa9f7284145cb8772fd22fdb052d8" +checksum = "839395ef53057db96b84c9238ab29e1a13f2e5c8ec9f66bef853ab4197303924" dependencies = [ "byteorder", "rmp", @@ -1371,9 +1270,9 @@ dependencies = [ [[package]] name = "roxmltree" -version = "0.11.0" +version = "0.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d5001f134077069d87f77c8b9452b690df2445f7a43f1c7ca4a1af8dd505789d" +checksum = "bf58a7d05b28e14b1e8902fa04c4d5d6109f5450ef71a5e6597f66e53f541504" dependencies = [ "xmlparser", ] @@ -1473,14 +1372,15 @@ dependencies = [ [[package]] name = "sha2" -version = "0.8.2" +version = "0.9.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a256f46ea78a0c0d9ff00077504903ac881a1dafdc20da66545699e7776b3e69" +checksum = "fa827a14b29ab7f44778d14a88d3cb76e949c45083f7dbfa507d0cb699dc12de" dependencies = [ - "block-buffer 0.7.3", - "digest 0.8.1", - "fake-simd", - "opaque-debug 0.2.3", + "block-buffer", + "cfg-if 1.0.0", + "cpuid-bool", + "digest", + "opaque-debug", ] [[package]] @@ -1561,12 +1461,6 @@ dependencies = [ "syn", ] -[[package]] -name = "subtle" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2d67a5a62ba6e01cb2192ff309324cb4875d0c451d55fe2319433abe7a05a8ee" - [[package]] name = "subtle" version = "2.4.0" @@ -1630,7 +1524,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6db9e6914ab8b1ae1c260a4ae7a49b6c5611b40328a735b21862567685e73255" dependencies = [ "libc", - "wasi 0.10.0+wasi-snapshot-preview1", + "wasi", "winapi", ] @@ -1656,7 +1550,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8d56477f6ed99e10225f38f9f75f872f29b8b8bd8c0b946f63345bb144e9eeda" dependencies = [ "autocfg", - "bytes 1.0.1", + "bytes", "libc", "memchr", "mio", @@ -1707,7 +1601,7 @@ version = "0.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ec31e5cc6b46e653cf57762f36f71d5e6386391d88a72fd6db4508f8f676fb29" dependencies = [ - "bytes 1.0.1", + "bytes", "futures-core", "futures-sink", "log", @@ -1832,12 +1726,6 @@ dependencies = [ "try-lock", ] -[[package]] -name = "wasi" -version = "0.9.0+wasi-snapshot-preview1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cccddf32554fecc6acb585f82a32a72e28b48f8c4c1883ddfeeeaa96f7d8e519" - [[package]] name = "wasi" version = "0.10.0+wasi-snapshot-preview1" diff --git a/src/api/Cargo.toml b/src/api/Cargo.toml index c3208b66..bce9946e 100644 --- a/src/api/Cargo.toml +++ b/src/api/Cargo.toml @@ -17,17 +17,16 @@ garage_util = { version = "0.1.1", path = "../util" } garage_table = { version = "0.1.1", path = "../table" } garage_model = { version = "0.1.1", path = "../model" } -err-derive = "0.2.3" -bytes = "0.4" -hex = "0.3" +err-derive = "0.3" +bytes = "1.0" +hex = "0.4" base64 = "0.13" log = "0.4" chrono = "0.4" -md-5 = "0.9.1" -sha2 = "0.8" -hmac = "0.7" -crypto-mac = "0.7" -rand = "0.7" +md-5 = "0.9" +sha2 = "0.9" +hmac = "0.10" +crypto-mac = "0.10" futures = "0.3" futures-util = "0.3" @@ -38,5 +37,5 @@ hyper = "0.14" url = "2.1" httpdate = "0.3" percent-encoding = "2.1.0" -roxmltree = "0.11" +roxmltree = "0.14" http-range = "0.1" diff --git a/src/api/error.rs b/src/api/error.rs index a1681fc3..42a7ab10 100644 --- a/src/api/error.rs +++ b/src/api/error.rs @@ -33,7 +33,7 @@ pub enum Error { InvalidBase64(#[error(source)] base64::DecodeError), #[error(display = "Invalid XML: {}", _0)] - InvalidXML(#[error(source)] roxmltree::Error), + InvalidXML(String), #[error(display = "Invalid header value: {}", _0)] InvalidHeader(#[error(source)] hyper::header::ToStrError), @@ -45,6 +45,12 @@ pub enum Error { BadRequest(String), } +impl From for Error { + fn from(err: roxmltree::Error) -> Self { + Self::InvalidXML(format!("{}", err)) + } +} + impl Error { pub fn http_status_code(&self) -> StatusCode { match self { diff --git a/src/api/s3_put.rs b/src/api/s3_put.rs index ea3664bd..c4e3b818 100644 --- a/src/api/s3_put.rs +++ b/src/api/s3_put.rs @@ -5,7 +5,7 @@ use std::sync::Arc; use futures::stream::*; use hyper::{Body, Request, Response}; use md5::{digest::generic_array::*, Digest as Md5Digest, Md5}; -use sha2::{Digest as Sha256Digest, Sha256}; +use sha2::Sha256; use garage_table::*; use garage_util::data::*; @@ -188,7 +188,7 @@ async fn read_and_put_blocks( let mut md5hasher = Md5::new(); let mut sha256hasher = Sha256::new(); md5hasher.update(&first_block[..]); - sha256hasher.input(&first_block[..]); + sha256hasher.update(&first_block[..]); let mut next_offset = first_block.len(); let mut put_curr_version_block = put_block_meta( @@ -208,7 +208,7 @@ async fn read_and_put_blocks( futures::try_join!(put_curr_block, put_curr_version_block, chunker.next())?; if let Some(block) = next_block { md5hasher.update(&block[..]); - sha256hasher.input(&block[..]); + sha256hasher.update(&block[..]); let block_hash = blake2sum(&block[..]); let block_len = block.len(); put_curr_version_block = put_block_meta( @@ -229,7 +229,7 @@ async fn read_and_put_blocks( let total_size = next_offset as u64; let data_md5sum = md5hasher.finalize(); - let data_sha256sum = sha256hasher.result(); + let data_sha256sum = sha256hasher.finalize(); let data_sha256sum = Hash::try_from(&data_sha256sum[..]).unwrap(); Ok((total_size, data_md5sum, data_sha256sum)) diff --git a/src/api/signature.rs b/src/api/signature.rs index b3d61ff4..6dc69afa 100644 --- a/src/api/signature.rs +++ b/src/api/signature.rs @@ -1,7 +1,7 @@ use std::collections::HashMap; use chrono::{DateTime, Duration, NaiveDateTime, Utc}; -use hmac::{Hmac, Mac}; +use hmac::{Hmac, Mac, NewMac}; use hyper::{Body, Method, Request}; use sha2::{Digest, Sha256}; @@ -91,8 +91,8 @@ pub async fn check_signature( "s3", ) .ok_or_internal_error("Unable to build signing HMAC")?; - hmac.input(string_to_sign.as_bytes()); - let signature = hex::encode(hmac.result().code()); + hmac.update(string_to_sign.as_bytes()); + let signature = hex::encode(hmac.finalize().into_bytes()); if authorization.signature != signature { trace!("Canonical request: ``{}``", canonical_request); @@ -218,12 +218,12 @@ fn parse_credential(cred: &str) -> Result<(String, String), Error> { fn string_to_sign(datetime: &DateTime, scope_string: &str, canonical_req: &str) -> String { let mut hasher = Sha256::default(); - hasher.input(canonical_req.as_bytes()); + hasher.update(canonical_req.as_bytes()); [ "AWS4-HMAC-SHA256", &datetime.format(LONG_DATETIME).to_string(), scope_string, - &hex::encode(hasher.result().as_slice()), + &hex::encode(hasher.finalize().as_slice()), ] .join("\n") } @@ -236,14 +236,14 @@ fn signing_hmac( ) -> Result { let secret = String::from("AWS4") + secret_key; let mut date_hmac = HmacSha256::new_varkey(secret.as_bytes())?; - date_hmac.input(datetime.format(SHORT_DATE).to_string().as_bytes()); - let mut region_hmac = HmacSha256::new_varkey(&date_hmac.result().code())?; - region_hmac.input(region.as_bytes()); - let mut service_hmac = HmacSha256::new_varkey(®ion_hmac.result().code())?; - service_hmac.input(service.as_bytes()); - let mut signing_hmac = HmacSha256::new_varkey(&service_hmac.result().code())?; - signing_hmac.input(b"aws4_request"); - let hmac = HmacSha256::new_varkey(&signing_hmac.result().code())?; + date_hmac.update(datetime.format(SHORT_DATE).to_string().as_bytes()); + let mut region_hmac = HmacSha256::new_varkey(&date_hmac.finalize().into_bytes())?; + region_hmac.update(region.as_bytes()); + let mut service_hmac = HmacSha256::new_varkey(®ion_hmac.finalize().into_bytes())?; + service_hmac.update(service.as_bytes()); + let mut signing_hmac = HmacSha256::new_varkey(&service_hmac.finalize().into_bytes())?; + signing_hmac.update(b"aws4_request"); + let hmac = HmacSha256::new_varkey(&signing_hmac.finalize().into_bytes())?; Ok(hmac) } diff --git a/src/garage/Cargo.toml b/src/garage/Cargo.toml index 36bbcd50..c1817bf2 100644 --- a/src/garage/Cargo.toml +++ b/src/garage/Cargo.toml @@ -21,10 +21,9 @@ garage_model = { version = "0.1.1", path = "../model" } garage_api = { version = "0.1.1", path = "../api" } garage_web = { version = "0.1.1", path = "../web" } -bytes = "0.4" -rand = "0.7" -hex = "0.3" -sha2 = "0.8" +bytes = "1.0" +rand = "0.8" +hex = "0.4" log = "0.4" pretty_env_logger = "0.4" git-version = "0.3.4" @@ -33,7 +32,7 @@ sled = "0.34" structopt = { version = "0.3", default-features = false } toml = "0.5" -rmp-serde = "0.14.3" +rmp-serde = "0.15" serde = { version = "1.0", default-features = false, features = ["derive", "rc"] } futures = "0.3" diff --git a/src/model/Cargo.toml b/src/model/Cargo.toml index 8f36cf2e..98656ea9 100644 --- a/src/model/Cargo.toml +++ b/src/model/Cargo.toml @@ -17,20 +17,17 @@ garage_util = { version = "0.1.1", path = "../util" } garage_rpc = { version = "0.1.1", path = "../rpc" } garage_table = { version = "0.1.1", path = "../table" } -bytes = "0.4" -rand = "0.7" -hex = "0.3" -sha2 = "0.8" -arc-swap = "0.4" +rand = "0.8" +hex = "0.4" +arc-swap = "1.0" log = "0.4" sled = "0.34" -rmp-serde = "0.14.3" +rmp-serde = "0.15" serde = { version = "1.0", default-features = false, features = ["derive", "rc"] } serde_bytes = "0.11" -async-trait = "0.1.30" futures = "0.3" futures-util = "0.3" tokio = { version = "1.0", default-features = false, features = ["rt", "rt-multi-thread", "io-util", "net", "time", "macros", "sync", "signal", "fs"] } diff --git a/src/rpc/Cargo.toml b/src/rpc/Cargo.toml index fc066bef..fbe826a8 100644 --- a/src/rpc/Cargo.toml +++ b/src/rpc/Cargo.toml @@ -15,22 +15,20 @@ path = "lib.rs" [dependencies] garage_util = { version = "0.1.1", path = "../util" } -bytes = "0.4" -rand = "0.7" -hex = "0.3" -sha2 = "0.8" -arc-swap = "0.4" +bytes = "1.0" +hex = "0.4" +arc-swap = "1.0" gethostname = "0.2" log = "0.4" -rmp-serde = "0.14.3" +rmp-serde = "0.15" serde = { version = "1.0", default-features = false, features = ["derive", "rc"] } serde_json = "1.0" futures = "0.3" futures-util = "0.3" tokio = { version = "1.0", default-features = false, features = ["rt", "rt-multi-thread", "io-util", "net", "time", "macros", "sync", "signal", "fs"] } -tokio-stream = {version = "0.1", features = ["net"] } +tokio-stream = { version = "0.1", features = ["net"] } http = "0.2" hyper = { version = "0.14", features = ["full"] } diff --git a/src/rpc/ring.rs b/src/rpc/ring.rs index 490fb1de..2e997523 100644 --- a/src/rpc/ring.rs +++ b/src/rpc/ring.rs @@ -22,7 +22,6 @@ const PARTITION_MASK_U16: u16 = ((1 << PARTITION_BITS) - 1) << (16 - PARTITION_B // (most deployments use a replication factor of 3, so...) pub const MAX_REPLICATION: usize = 3; - #[derive(Clone, Debug, Serialize, Deserialize)] pub struct NetworkConfig { pub members: HashMap, diff --git a/src/rpc/rpc_client.rs b/src/rpc/rpc_client.rs index cffcf106..eb4f6620 100644 --- a/src/rpc/rpc_client.rs +++ b/src/rpc/rpc_client.rs @@ -7,7 +7,6 @@ use std::sync::Arc; use std::time::Duration; use arc_swap::ArcSwapOption; -use bytes::IntoBuf; use futures::future::Future; use futures::stream::futures_unordered::FuturesUnordered; use futures::stream::StreamExt; @@ -333,7 +332,7 @@ impl RpcHttpClient { let body = hyper::body::to_bytes(resp.into_body()).await?; drop(slot); - match rmp_serde::decode::from_read::<_, Result>(body.into_buf())? { + match rmp_serde::decode::from_read::<_, Result>(&body[..])? { Err(e) => Ok(Err(Error::RemoteError(e, status))), Ok(x) => Ok(Ok(x)), } diff --git a/src/rpc/rpc_server.rs b/src/rpc/rpc_server.rs index 0c5bf6f9..0d82d796 100644 --- a/src/rpc/rpc_server.rs +++ b/src/rpc/rpc_server.rs @@ -4,7 +4,6 @@ use std::pin::Pin; use std::sync::Arc; use std::time::Instant; -use bytes::IntoBuf; use futures::future::Future; use futures_util::future::*; use futures_util::stream::*; @@ -48,7 +47,7 @@ where { let begin_time = Instant::now(); let whole_body = hyper::body::to_bytes(req.into_body()).await?; - let msg = rmp_serde::decode::from_read::<_, M>(whole_body.into_buf())?; + let msg = rmp_serde::decode::from_read::<_, M>(&whole_body[..])?; trace!( "Request message: {}", diff --git a/src/table/Cargo.toml b/src/table/Cargo.toml index 8f73470e..f9d98dec 100644 --- a/src/table/Cargo.toml +++ b/src/table/Cargo.toml @@ -16,19 +16,17 @@ path = "lib.rs" garage_util = { version = "0.1.1", path = "../util" } garage_rpc = { version = "0.1.1", path = "../rpc" } -bytes = "0.4" -rand = "0.7" -hex = "0.3" +bytes = "1.0" +rand = "0.8" log = "0.4" hexdump = "0.1" sled = "0.34" -rmp-serde = "0.14.3" +rmp-serde = "0.15" serde = { version = "1.0", default-features = false, features = ["derive", "rc"] } serde_bytes = "0.11" -async-trait = "0.1.30" futures = "0.3" futures-util = "0.3" tokio = { version = "1.0", default-features = false, features = ["rt", "rt-multi-thread", "io-util", "net", "time", "macros", "sync", "signal", "fs"] } diff --git a/src/table/merkle.rs b/src/table/merkle.rs index db05cca4..8a8eb342 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -20,7 +20,6 @@ use crate::data::*; use crate::replication::*; use crate::schema::*; - // This modules partitions the data in 2**16 partitions, based on the top // 16 bits (two bytes) of item's partition keys' hashes. // It builds one Merkle tree for each of these 2**16 partitions. @@ -73,10 +72,7 @@ where F: TableSchema + 'static, R: TableReplication + 'static, { - pub(crate) fn launch( - background: &BackgroundRunner, - data: Arc>, - ) -> Arc { + pub(crate) fn launch(background: &BackgroundRunner, data: Arc>) -> Arc { let empty_node_hash = blake2sum(&rmp_to_vec_all_named(&MerkleNode::Empty).unwrap()[..]); let ret = Arc::new(Self { @@ -132,7 +128,10 @@ where }; let key = MerkleNodeKey { - partition: self.data.replication.partition_of(&Hash::try_from(&k[0..32]).unwrap()), + partition: self + .data + .replication + .partition_of(&Hash::try_from(&k[0..32]).unwrap()), prefix: vec![], }; self.data diff --git a/src/table/sync.rs b/src/table/sync.rs index f5c2ef33..3130abe8 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -244,12 +244,8 @@ where ))); } } else { - self.offload_partition( - &partition.begin, - &partition.end, - must_exit, - ) - .await?; + self.offload_partition(&partition.begin, &partition.end, must_exit) + .await?; } Ok(()) @@ -399,9 +395,7 @@ where ); return Ok(()); } - SyncRPC::RootCkDifferent(true) => { - VecDeque::from(vec![root_ck_key]) - } + SyncRPC::RootCkDifferent(true) => VecDeque::from(vec![root_ck_key]), x => { return Err(Error::Message(format!( "Invalid respone to RootCkHash RPC: {}", @@ -550,7 +544,7 @@ impl SyncTodo { let begin = partitions[i].1; let end = if i + 1 < partitions.len() { - partitions[i+1].1 + partitions[i + 1].1 } else { [0xFFu8; 32].into() }; @@ -579,7 +573,7 @@ impl SyncTodo { return None; } - let i = rand::thread_rng().gen_range::(0, self.todo.len()); + let i = rand::thread_rng().gen_range(0..self.todo.len()); if i == self.todo.len() - 1 { self.todo.pop() } else { diff --git a/src/util/Cargo.toml b/src/util/Cargo.toml index 2ae4796c..4698a04f 100644 --- a/src/util/Cargo.toml +++ b/src/util/Cargo.toml @@ -13,31 +13,26 @@ path = "lib.rs" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html [dependencies] -rand = "0.7" -hex = "0.3" -sha2 = "0.8" +rand = "0.8" +hex = "0.4" +sha2 = "0.9" blake2 = "0.9" -err-derive = "0.2.3" +err-derive = "0.3" log = "0.4" fasthash = "0.4" sled = "0.34" toml = "0.5" -rmp-serde = "0.14.3" +rmp-serde = "0.15" serde = { version = "1.0", default-features = false, features = ["derive", "rc"] } serde_json = "1.0" chrono = "0.4" -arc-swap = "1.2" futures = "0.3" -futures-util = "0.3" tokio = { version = "1.0", default-features = false, features = ["rt", "rt-multi-thread", "io-util", "net", "time", "macros", "sync", "signal", "fs"] } http = "0.2" hyper = "0.14" rustls = "0.19" webpki = "0.21" - -roxmltree = "0.11" - diff --git a/src/util/background.rs b/src/util/background.rs index 35d41d9f..b5eb8bc8 100644 --- a/src/util/background.rs +++ b/src/util/background.rs @@ -37,7 +37,7 @@ impl BackgroundRunner { None => break, } } - _ = tokio::time::sleep(Duration::from_secs(10)).fuse() => { + _ = tokio::time::sleep(Duration::from_secs(5)).fuse() => { if *stop_signal_2.borrow() { break; } else { @@ -71,9 +71,9 @@ impl BackgroundRunner { // because the sending side was dropped. Exit now. None => break, }, - _ = tokio::time::sleep(Duration::from_secs(10)).fuse() => { + _ = tokio::time::sleep(Duration::from_secs(5)).fuse() => { if *stop_signal.borrow() { - // Nothing has been going on for 10 secs, and we are shutting + // Nothing has been going on for 5 secs, and we are shutting // down. Exit now. break; } else { diff --git a/src/util/data.rs b/src/util/data.rs index 591b7605..cb784730 100644 --- a/src/util/data.rs +++ b/src/util/data.rs @@ -87,9 +87,9 @@ pub fn sha256sum(data: &[u8]) -> Hash { use sha2::{Digest, Sha256}; let mut hasher = Sha256::new(); - hasher.input(data); + hasher.update(data); let mut hash = [0u8; 32]; - hash.copy_from_slice(&hasher.result()[..]); + hash.copy_from_slice(&hasher.finalize()[..]); hash.into() } diff --git a/src/web/Cargo.toml b/src/web/Cargo.toml index 8c340f6b..9aabfe81 100644 --- a/src/web/Cargo.toml +++ b/src/web/Cargo.toml @@ -18,11 +18,10 @@ 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" +err-derive = "0.3" log = "0.4" futures = "0.3" http = "0.2" hyper = "0.14" percent-encoding = "2.1.0" -roxmltree = "0.11" idna = "0.2" -- cgit v1.2.3 From 3fadc5cbbd5067e160dbe9cf23e301f74a19186e Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 16 Mar 2021 16:35:10 +0100 Subject: Small changes --- src/garage/admin_rpc.rs | 36 ++++++++++++++++++------------------ src/table/gc.rs | 13 ++++++++----- 2 files changed, 26 insertions(+), 23 deletions(-) diff --git a/src/garage/admin_rpc.rs b/src/garage/admin_rpc.rs index aec791a4..10087f74 100644 --- a/src/garage/admin_rpc.rs +++ b/src/garage/admin_rpc.rs @@ -443,12 +443,14 @@ impl AdminRpcHandler { self.gather_table_stats(&mut ret, &self.garage.block_ref_table, &opt)?; writeln!(&mut ret, "\nBlock manager stats:").unwrap(); - writeln!( - &mut ret, - " number of blocks: {}", - self.garage.block_manager.rc_len() - ) - .unwrap(); + if opt.detailed { + writeln!( + &mut ret, + " number of blocks: {}", + self.garage.block_manager.rc_len() + ) + .unwrap(); + } writeln!( &mut ret, " resync queue length: {}", @@ -456,10 +458,6 @@ impl AdminRpcHandler { ) .unwrap(); - if opt.detailed { - writeln!(&mut ret, "\nDetailed stats not implemented yet.").unwrap(); - } - Ok(ret) } @@ -467,26 +465,28 @@ impl AdminRpcHandler { &self, to: &mut String, t: &Arc>, - _opt: &StatsOpt, + opt: &StatsOpt, ) -> Result<(), Error> where F: TableSchema + 'static, R: TableReplication + 'static, { writeln!(to, "\nTable stats for {}", t.data.name).unwrap(); - writeln!(to, " number of items: {}", t.data.store.len()).unwrap(); + if opt.detailed { + writeln!(to, " number of items: {}", t.data.store.len()).unwrap(); + writeln!( + to, + " Merkle tree size: {}", + t.merkle_updater.merkle_tree_len() + ) + .unwrap(); + } writeln!( to, " Merkle updater todo queue length: {}", t.merkle_updater.todo_len() ) .unwrap(); - writeln!( - to, - " Merkle tree size: {}", - t.merkle_updater.merkle_tree_len() - ) - .unwrap(); writeln!(to, " GC todo queue length: {}", t.data.gc_todo_len()).unwrap(); Ok(()) } diff --git a/src/table/gc.rs b/src/table/gc.rs index 20ebe3c7..a37c052f 100644 --- a/src/table/gc.rs +++ b/src/table/gc.rs @@ -146,16 +146,19 @@ where .map(|(nodes, items)| self.try_send_and_delete(nodes, items)), ) .await; + + let mut errs = vec![]; for resp in resps { if let Err(e) = resp { - warn!( - "({}) Unable to send and delete for GC: {}", - self.data.name, e - ); + errs.push(e); } } - Ok(true) + if errs.is_empty() { + Ok(true) + } else { + Err(Error::Message(errs.into_iter().map(|x| format!("{}", x)).collect::>().join(", "))) + } } async fn try_send_and_delete( -- cgit v1.2.3 From 1bfe9c129e2cc3780e784b0c4a784fa1679a3f97 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 16 Mar 2021 16:35:46 +0100 Subject: Switch to AGPL --- LICENSE | 148 +++++++++++++++++++++++++++++----------------------------------- 1 file changed, 67 insertions(+), 81 deletions(-) diff --git a/LICENSE b/LICENSE index 53d1f3d0..be3f7b28 100644 --- a/LICENSE +++ b/LICENSE @@ -1,5 +1,5 @@ - GNU GENERAL PUBLIC LICENSE - Version 3, 29 June 2007 + GNU AFFERO GENERAL PUBLIC LICENSE + Version 3, 19 November 2007 Copyright (C) 2007 Free Software Foundation, Inc. Everyone is permitted to copy and distribute verbatim copies @@ -7,17 +7,15 @@ Preamble - The GNU General Public License is a free, copyleft license for -software and other kinds of works. + The GNU Affero General Public License is a free, copyleft license for +software and other kinds of works, specifically designed to ensure +cooperation with the community in the case of network server software. The licenses for most software and other practical works are designed to take away your freedom to share and change the works. By contrast, -the GNU General Public License is intended to guarantee your freedom to +our General Public Licenses are intended to guarantee your freedom to share and change all versions of a program--to make sure it remains free -software for all its users. We, the Free Software Foundation, use the -GNU General Public License for most of our software; it applies also to -any other work released this way by its authors. You can apply it to -your programs, too. +software for all its users. When we speak of free software, we are referring to freedom, not price. Our General Public Licenses are designed to make sure that you @@ -26,44 +24,34 @@ them if you wish), that you receive source code or can get it if you want it, that you can change the software or use pieces of it in new free programs, and that you know you can do these things. - To protect your rights, we need to prevent others from denying you -these rights or asking you to surrender the rights. Therefore, you have -certain responsibilities if you distribute copies of the software, or if -you modify it: responsibilities to respect the freedom of others. - - For example, if you distribute copies of such a program, whether -gratis or for a fee, you must pass on to the recipients the same -freedoms that you received. You must make sure that they, too, receive -or can get the source code. And you must show them these terms so they -know their rights. - - Developers that use the GNU GPL protect your rights with two steps: -(1) assert copyright on the software, and (2) offer you this License -giving you legal permission to copy, distribute and/or modify it. - - For the developers' and authors' protection, the GPL clearly explains -that there is no warranty for this free software. For both users' and -authors' sake, the GPL requires that modified versions be marked as -changed, so that their problems will not be attributed erroneously to -authors of previous versions. - - Some devices are designed to deny users access to install or run -modified versions of the software inside them, although the manufacturer -can do so. This is fundamentally incompatible with the aim of -protecting users' freedom to change the software. The systematic -pattern of such abuse occurs in the area of products for individuals to -use, which is precisely where it is most unacceptable. Therefore, we -have designed this version of the GPL to prohibit the practice for those -products. If such problems arise substantially in other domains, we -stand ready to extend this provision to those domains in future versions -of the GPL, as needed to protect the freedom of users. - - Finally, every program is threatened constantly by software patents. -States should not allow patents to restrict development and use of -software on general-purpose computers, but in those that do, we wish to -avoid the special danger that patents applied to a free program could -make it effectively proprietary. To prevent this, the GPL assures that -patents cannot be used to render the program non-free. + Developers that use our General Public Licenses protect your rights +with two steps: (1) assert copyright on the software, and (2) offer +you this License which gives you legal permission to copy, distribute +and/or modify the software. + + A secondary benefit of defending all users' freedom is that +improvements made in alternate versions of the program, if they +receive widespread use, become available for other developers to +incorporate. Many developers of free software are heartened and +encouraged by the resulting cooperation. However, in the case of +software used on network servers, this result may fail to come about. +The GNU General Public License permits making a modified version and +letting the public access it on a server without ever releasing its +source code to the public. + + The GNU Affero General Public License is designed specifically to +ensure that, in such cases, the modified source code becomes available +to the community. It requires the operator of a network server to +provide the source code of the modified version running there to the +users of that server. Therefore, public use of a modified version, on +a publicly accessible server, gives the public access to the source +code of the modified version. + + An older license, called the Affero General Public License and +published by Affero, was designed to accomplish similar goals. This is +a different license, not a version of the Affero GPL, but Affero has +released a new version of the Affero GPL which permits relicensing under +this license. The precise terms and conditions for copying, distribution and modification follow. @@ -72,7 +60,7 @@ modification follow. 0. Definitions. - "This License" refers to version 3 of the GNU General Public License. + "This License" refers to version 3 of the GNU Affero General Public License. "Copyright" also means copyright-like laws that apply to other kinds of works, such as semiconductor masks. @@ -549,35 +537,45 @@ to collect a royalty for further conveying from those to whom you convey the Program, the only way you could satisfy both those terms and this License would be to refrain entirely from conveying the Program. - 13. Use with the GNU Affero General Public License. + 13. Remote Network Interaction; Use with the GNU General Public License. + + Notwithstanding any other provision of this License, if you modify the +Program, your modified version must prominently offer all users +interacting with it remotely through a computer network (if your version +supports such interaction) an opportunity to receive the Corresponding +Source of your version by providing access to the Corresponding Source +from a network server at no charge, through some standard or customary +means of facilitating copying of software. This Corresponding Source +shall include the Corresponding Source for any work covered by version 3 +of the GNU General Public License that is incorporated pursuant to the +following paragraph. Notwithstanding any other provision of this License, you have permission to link or combine any covered work with a work licensed -under version 3 of the GNU Affero General Public License into a single +under version 3 of the GNU General Public License into a single combined work, and to convey the resulting work. The terms of this License will continue to apply to the part which is the covered work, -but the special requirements of the GNU Affero General Public License, -section 13, concerning interaction through a network will apply to the -combination as such. +but the work with which it is combined will remain governed by version +3 of the GNU General Public License. 14. Revised Versions of this License. The Free Software Foundation may publish revised and/or new versions of -the GNU General Public License from time to time. Such new versions will -be similar in spirit to the present version, but may differ in detail to +the GNU Affero General Public License from time to time. Such new versions +will be similar in spirit to the present version, but may differ in detail to address new problems or concerns. Each version is given a distinguishing version number. If the -Program specifies that a certain numbered version of the GNU General +Program specifies that a certain numbered version of the GNU Affero General Public License "or any later version" applies to it, you have the option of following the terms and conditions either of that numbered version or of any later version published by the Free Software Foundation. If the Program does not specify a version number of the -GNU General Public License, you may choose any version ever published +GNU Affero General Public License, you may choose any version ever published by the Free Software Foundation. If the Program specifies that a proxy can decide which future -versions of the GNU General Public License can be used, that proxy's +versions of the GNU Affero General Public License can be used, that proxy's public statement of acceptance of a version permanently authorizes you to choose that version for the Program. @@ -635,41 +633,29 @@ the "copyright" line and a pointer to where the full notice is found. Copyright (C) This program is free software: you can redistribute it and/or modify - it under the terms of the GNU General Public License as published by + it under the terms of the GNU Affero General Public License as published by the Free Software Foundation, either version 3 of the License, or (at your option) any later version. This program is distributed in the hope that it will be useful, but WITHOUT ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - GNU General Public License for more details. + GNU Affero General Public License for more details. - You should have received a copy of the GNU General Public License + You should have received a copy of the GNU Affero General Public License along with this program. If not, see . Also add information on how to contact you by electronic and paper mail. - If the program does terminal interaction, make it output a short -notice like this when it starts in an interactive mode: - - Copyright (C) - This program comes with ABSOLUTELY NO WARRANTY; for details type `show w'. - This is free software, and you are welcome to redistribute it - under certain conditions; type `show c' for details. - -The hypothetical commands `show w' and `show c' should show the appropriate -parts of the General Public License. Of course, your program's commands -might be different; for a GUI interface, you would use an "about box". + If your software can interact with users remotely through a computer +network, you should also make sure that it provides a way for users to +get its source. For example, if your program is a web application, its +interface could display a "Source" link that leads users to an archive +of the code. There are many ways you could offer source, and different +solutions will be better for different programs; see section 13 for the +specific requirements. You should also get your employer (if you work as a programmer) or school, if any, to sign a "copyright disclaimer" for the program, if necessary. -For more information on this, and how to apply and follow the GNU GPL, see +For more information on this, and how to apply and follow the GNU AGPL, see . - - The GNU General Public License does not permit incorporating your program -into proprietary programs. If your program is a subroutine library, you -may consider it more useful to permit linking proprietary applications with -the library. If this is what you want to do, use the GNU Lesser General -Public License instead of this License. But first, please read -. - -- cgit v1.2.3 From 08bcd5195654ae073afe31c341f0ce4b36094da0 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 16 Mar 2021 16:51:15 +0100 Subject: GC object table in a specific case --- src/model/block.rs | 2 +- src/model/object_table.rs | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/model/block.rs b/src/model/block.rs index 41729685..0d9af38f 100644 --- a/src/model/block.rs +++ b/src/model/block.rs @@ -259,7 +259,7 @@ impl BlockManager { if let Err(e) = self.resync_iter(&mut must_exit).await { warn!("Error in block resync loop: {}", e); select! { - _ = tokio::time::sleep(Duration::from_secs(10)).fuse() => (), + _ = tokio::time::sleep(Duration::from_secs(1)).fuse() => (), _ = must_exit.changed().fuse() => (), } } diff --git a/src/model/object_table.rs b/src/model/object_table.rs index 99fad3ce..62606df4 100644 --- a/src/model/object_table.rs +++ b/src/model/object_table.rs @@ -146,6 +146,9 @@ impl Entry for Object { fn sort_key(&self) -> &String { &self.key } + fn is_tombstone(&self) -> bool { + self.versions.len() == 1 && self.versions[0].state == ObjectVersionState::Complete(ObjectVersionData::DeleteMarker) + } } impl CRDT for Object { -- cgit v1.2.3 From 7b10245dfb741b7f801d1f3eaa56c6cb4f385d65 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 16 Mar 2021 18:42:33 +0100 Subject: Leader-based GC --- src/table/data.rs | 20 +++++++++++++++++--- src/table/table.rs | 2 +- 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/src/table/data.rs b/src/table/data.rs index 9aa2a3bc..e07a21d2 100644 --- a/src/table/data.rs +++ b/src/table/data.rs @@ -9,13 +9,16 @@ use tokio::sync::Notify; use garage_util::data::*; use garage_util::error::*; +use garage_rpc::membership::System; + use crate::crdt::CRDT; use crate::replication::*; use crate::schema::*; pub struct TableData { - pub name: String, + system: Arc, + pub name: String, pub(crate) instance: F, pub(crate) replication: R, @@ -32,7 +35,7 @@ where F: TableSchema, R: TableReplication, { - pub fn new(name: String, instance: F, replication: R, db: &sled::Db) -> Arc { + pub fn new(system: Arc, name: String, instance: F, replication: R, db: &sled::Db) -> Arc { let store = db .open_tree(&format!("{}:table", name)) .expect("Unable to open DB tree"); @@ -49,6 +52,7 @@ where .expect("Unable to open DB tree"); Arc::new(Self { + system, name, instance, replication, @@ -157,7 +161,17 @@ where self.instance.updated(old_entry, Some(new_entry)); self.merkle_todo_notify.notify_one(); if is_tombstone { - self.gc_todo.insert(&tree_key, new_bytes_hash.as_slice())?; + // We are only responsible for GC'ing this item if we are the + // "leader" of the partition, i.e. the first node in the + // set of nodes that replicates this partition. + // This avoids GC loops and does not change the termination properties + // of the GC algorithm, as in all cases GC is suspended if + // any node of the partition is unavailable. + let pk_hash = Hash::try_from(&tree_key[..32]).unwrap(); + let nodes = self.replication.write_nodes(&pk_hash); + if nodes.first() == Some(&self.system.id) { + self.gc_todo.insert(&tree_key, new_bytes_hash.as_slice())?; + } } } diff --git a/src/table/table.rs b/src/table/table.rs index 421c8bf5..e203b178 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -64,7 +64,7 @@ where let rpc_path = format!("table_{}", name); let rpc_client = system.rpc_client::>(&rpc_path); - let data = TableData::new(name, instance, replication, db); + let data = TableData::new(system.clone(), name, instance, replication, db); let merkle_updater = MerkleUpdater::launch(&system.background, data.clone()); -- cgit v1.2.3 From 390ab02f41c32e75e1df2b6893dfa0fd484e8b4b Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 16 Mar 2021 20:10:41 +0100 Subject: Todo make a test for the Merkle updater --- src/table/merkle.rs | 103 ++++++++++++++++++++++++++++++---------------------- 1 file changed, 59 insertions(+), 44 deletions(-) diff --git a/src/table/merkle.rs b/src/table/merkle.rs index 8a8eb342..3001786f 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -136,7 +136,7 @@ where }; self.data .merkle_tree - .transaction(|tx| self.update_item_rec(tx, k, khash, &key, new_vhash))?; + .transaction(|tx| self.update_item_rec(tx, k, &khash, &key, new_vhash))?; let deleted = self .data @@ -157,7 +157,7 @@ where &self, tx: &TransactionalTree, k: &[u8], - khash: Hash, + khash: &Hash, key: &MerkleNodeKey, new_vhash: Option, ) -> ConflictableTransactionResult, Error> { @@ -195,11 +195,22 @@ where Some(MerkleNode::Empty) } else if children.len() == 1 { // We now have a single node (case when the update deleted one of only two - // children). Move that single child to this level of the tree. + // children). If that node is a leaf, move it to this level. let key_sub = key.add_byte(children[0].0); let subnode = self.read_node_txn(tx, &key_sub)?; - tx.remove(key_sub.encode())?; - Some(subnode) + match subnode { + MerkleNode::Empty => { + warn!("({}) Single subnode in tree is empty Merkle node", self.data.name); + Some(MerkleNode::Empty) + } + MerkleNode::Intermediate(_) => { + Some(MerkleNode::Intermediate(children)) + } + x @ MerkleNode::Leaf(_, _) => { + tx.remove(key_sub.encode())?; + Some(x) + } + } } else { Some(MerkleNode::Intermediate(children)) } @@ -208,37 +219,41 @@ where None } } - MerkleNode::Leaf(exlf_key, exlf_hash) => { - if exlf_key == k { + MerkleNode::Leaf(exlf_k, exlf_vhash) => { + if exlf_k == k { // This leaf is for the same key that the one we are updating match new_vhash { - Some(vhv) if vhv == exlf_hash => None, + Some(vhv) if vhv == exlf_vhash => None, Some(vhv) => Some(MerkleNode::Leaf(k.to_vec(), vhv)), None => Some(MerkleNode::Empty), } } else { // This is an only leaf for another key - if let Some(vhv) = new_vhash { + if new_vhash.is_some() { // Move that other key to a subnode, create another subnode for our // insertion and replace current node by an intermediary node - let (pos1, h1) = { - let key2 = key.next_key(blake2sum(&exlf_key[..])); - let subhash = self.put_node_txn( - tx, - &key2, - &MerkleNode::Leaf(exlf_key, exlf_hash), - )?; - (key2.prefix[i], subhash) - }; - let (pos2, h2) = { - let key2 = key.next_key(khash); - let subhash = - self.put_node_txn(tx, &key2, &MerkleNode::Leaf(k.to_vec(), vhv))?; - (key2.prefix[i], subhash) - }; let mut int = vec![]; - intermediate_set_child(&mut int, pos1, h1); - intermediate_set_child(&mut int, pos2, h2); + + let exlf_khash = blake2sum(&exlf_k[..]); + assert_eq!(khash.as_slice()[..i], exlf_khash.as_slice()[..i]); + + { + let exlf_subkey = key.next_key(&exlf_khash); + let exlf_sub_hash = self.update_item_rec(tx, &exlf_k[..], &exlf_khash, &exlf_subkey, Some(exlf_vhash))?.unwrap(); + intermediate_set_child(&mut int, exlf_subkey.prefix[i], exlf_sub_hash); + assert_eq!(int.len(), 1); + } + + { + let key2 = key.next_key(khash); + let subhash = self.update_item_rec(tx, k, khash, &key2, new_vhash)?.unwrap(); + intermediate_set_child(&mut int, key2.prefix[i], subhash); + if exlf_khash.as_slice()[i] == khash.as_slice()[i] { + assert_eq!(int.len(), 1); + } else { + assert_eq!(int.len(), 2); + } + } Some(MerkleNode::Intermediate(int)) } else { // Nothing to do, we don't want to insert this value because it is None, @@ -266,11 +281,7 @@ where k: &MerkleNodeKey, ) -> ConflictableTransactionResult { let ent = tx.get(k.encode())?; - match ent { - None => Ok(MerkleNode::Empty), - Some(v) => Ok(rmp_serde::decode::from_read_ref::<_, MerkleNode>(&v[..]) - .map_err(|e| ConflictableTransactionError::Abort(e.into()))?), - } + MerkleNode::decode_opt(ent).map_err(ConflictableTransactionError::Abort) } fn put_node_txn( @@ -295,10 +306,7 @@ where // Access a node in the Merkle tree, used by the sync protocol pub(crate) fn read_node(&self, k: &MerkleNodeKey) -> Result { let ent = self.data.merkle_tree.get(k.encode())?; - match ent { - None => Ok(MerkleNode::Empty), - Some(v) => Ok(rmp_serde::decode::from_read_ref::<_, MerkleNode>(&v[..])?), - } + MerkleNode::decode_opt(ent) } pub fn merkle_tree_len(&self) -> usize { @@ -318,8 +326,8 @@ impl MerkleNodeKey { ret } - pub fn next_key(&self, h: Hash) -> Self { - assert!(&h.as_slice()[0..self.prefix.len()] == &self.prefix[..]); + pub fn next_key(&self, h: &Hash) -> Self { + assert_eq!(h.as_slice()[0..self.prefix.len()], self.prefix[..]); let mut s2 = self.clone(); s2.prefix.push(h.as_slice()[self.prefix.len()]); s2 @@ -332,6 +340,19 @@ impl MerkleNodeKey { } } +impl MerkleNode { + fn decode_opt(ent: Option) -> Result { + match ent { + None => Ok(MerkleNode::Empty), + Some(v) => Ok(rmp_serde::decode::from_read_ref::<_, MerkleNode>(&v[..])?), + } + } + + pub fn is_empty(&self) -> bool { + *self == MerkleNode::Empty + } +} + fn intermediate_set_child(ch: &mut Vec<(u8, Hash)>, pos: u8, v: Hash) { for i in 0..ch.len() { if ch[i].0 == pos { @@ -342,7 +363,7 @@ fn intermediate_set_child(ch: &mut Vec<(u8, Hash)>, pos: u8, v: Hash) { return; } } - ch.insert(ch.len(), (pos, v)); + ch.push((pos, v)); } fn intermediate_rm_child(ch: &mut Vec<(u8, Hash)>, pos: u8) { @@ -431,9 +452,3 @@ fn test_intermediate_aux() { ] ); } - -impl MerkleNode { - pub fn is_empty(&self) -> bool { - *self == MerkleNode::Empty - } -} -- cgit v1.2.3 From 46c7226fe446a2a7238aee085f0b35f095ada7ee Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 17 Mar 2021 08:41:07 +0100 Subject: New logo!! --- doc/logo/garage-dark-notext.png | Bin 0 -> 8254 bytes doc/logo/garage-dark-notext.svg | 113 ++++++++++++++++++++++ doc/logo/garage-dark.svg | 174 +++++++++++++++++++++++++++++++++ doc/logo/garage-notext.png | Bin 0 -> 7987 bytes doc/logo/garage-notext.svg | 146 ++++++++++++++++++++++++++++ doc/logo/garage.svg | 206 ++++++++++++++++++++++++++++++++++++++++ garage.png | Bin 15433 -> 0 bytes garage.svg | 119 ----------------------- 8 files changed, 639 insertions(+), 119 deletions(-) create mode 100644 doc/logo/garage-dark-notext.png create mode 100644 doc/logo/garage-dark-notext.svg create mode 100644 doc/logo/garage-dark.svg create mode 100644 doc/logo/garage-notext.png create mode 100644 doc/logo/garage-notext.svg create mode 100644 doc/logo/garage.svg delete mode 100644 garage.png delete mode 100644 garage.svg diff --git a/doc/logo/garage-dark-notext.png b/doc/logo/garage-dark-notext.png new file mode 100644 index 00000000..a2c7f5e2 Binary files /dev/null and b/doc/logo/garage-dark-notext.png differ diff --git a/doc/logo/garage-dark-notext.svg b/doc/logo/garage-dark-notext.svg new file mode 100644 index 00000000..2d1c1873 --- /dev/null +++ b/doc/logo/garage-dark-notext.svg @@ -0,0 +1,113 @@ + + + + + + + + image/svg+xml + + + + + + + + + + + + + + + + + + diff --git a/doc/logo/garage-dark.svg b/doc/logo/garage-dark.svg new file mode 100644 index 00000000..3c0699ae --- /dev/null +++ b/doc/logo/garage-dark.svg @@ -0,0 +1,174 @@ + + + + + + + + image/svg+xml + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/doc/logo/garage-notext.png b/doc/logo/garage-notext.png new file mode 100644 index 00000000..3ac607f4 Binary files /dev/null and b/doc/logo/garage-notext.png differ diff --git a/doc/logo/garage-notext.svg b/doc/logo/garage-notext.svg new file mode 100644 index 00000000..e48e14ea --- /dev/null +++ b/doc/logo/garage-notext.svg @@ -0,0 +1,146 @@ + + + + + + image/svg+xml + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/doc/logo/garage.svg b/doc/logo/garage.svg new file mode 100644 index 00000000..a4a3dbb2 --- /dev/null +++ b/doc/logo/garage.svg @@ -0,0 +1,206 @@ + + + + + + image/svg+xml + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/garage.png b/garage.png deleted file mode 100644 index d70d187e..00000000 Binary files a/garage.png and /dev/null differ diff --git a/garage.svg b/garage.svg deleted file mode 100644 index 4f08c212..00000000 --- a/garage.svg +++ /dev/null @@ -1,119 +0,0 @@ - - - - - - - - image/svg+xml - - - - - - - - - - - - - - - - - - - - - - -- cgit v1.2.3 From 797cda1c33d0b908492d2c08ece1c51c9b10f36d Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 17 Mar 2021 10:26:50 +0100 Subject: Add logo in repo --- README.md | 11 ++++++++--- doc/logo/garage.png | Bin 0 -> 11871 bytes doc/logo/garage.svg | 12 ++++++------ 3 files changed, 14 insertions(+), 9 deletions(-) create mode 100644 doc/logo/garage.png diff --git a/README.md b/README.md index 6f462493..c736a8a9 100644 --- a/README.md +++ b/README.md @@ -1,6 +1,11 @@ -# Garage - -[![Build Status](https://drone.deuxfleurs.fr/api/badges/Deuxfleurs/garage/status.svg)](https://drone.deuxfleurs.fr/Deuxfleurs/garage) +Garage [![Build Status](https://drone.deuxfleurs.fr/api/badges/Deuxfleurs/garage/status.svg)](https://drone.deuxfleurs.fr/Deuxfleurs/garage) +=== + +

+ + Garage logo + +

Garage is a lightweight S3-compatible distributed object store, with the following goals: diff --git a/doc/logo/garage.png b/doc/logo/garage.png new file mode 100644 index 00000000..2519e219 Binary files /dev/null and b/doc/logo/garage.png differ diff --git a/doc/logo/garage.svg b/doc/logo/garage.svg index a4a3dbb2..130c607d 100644 --- a/doc/logo/garage.svg +++ b/doc/logo/garage.svg @@ -15,7 +15,7 @@ version="1.1" sodipodi:docname="garage.svg" inkscape:version="1.0.2 (e86c870879, 2021-01-15)" - inkscape:export-filename="/home/lx/Deuxfleurs/garage-logo/v2-2c.png" + inkscape:export-filename="/home/lx/Deuxfleurs/garage/doc/logo/garage.png" inkscape:export-xdpi="96" inkscape:export-ydpi="96"> image/svg+xml - + @@ -40,14 +40,14 @@ inkscape:pageopacity="1" inkscape:pageshadow="2" inkscape:window-width="1920" - inkscape:window-height="1039" + inkscape:window-height="1080" id="namedview31" showgrid="false" inkscape:zoom="2.1842656" - inkscape:cx="179.89962" + inkscape:cx="90.853672" inkscape:cy="123.63257" inkscape:window-x="0" - inkscape:window-y="20" + inkscape:window-y="0" inkscape:window-maximized="0" inkscape:current-layer="Calque_1" inkscape:document-rotation="0" @@ -83,7 +83,7 @@ y="-0.31321606" /> + transform="matrix(1.7099534,0,0,1.7099534,-88.607712,-87.994557)"> Date: Thu, 18 Mar 2021 17:45:26 +0100 Subject: try fixing CI maybe absolute path break caching, who knows --- .drone.yml | 36 +++++++++++------------------------- 1 file changed, 11 insertions(+), 25 deletions(-) diff --git a/.drone.yml b/.drone.yml index 76379e40..68094a31 100644 --- a/.drone.yml +++ b/.drone.yml @@ -2,20 +2,9 @@ kind: pipeline name: default workspace: - base: /drone - -clone: - disable: true + base: /drone/garage steps: - - name: clone - image: alpine/git - commands: - - mkdir -p cargo - - git clone $DRONE_GIT_HTTP_URL - - cd garage - - git checkout $DRONE_COMMIT - - name: restore-cache image: meltwater/drone-cache:dev environment: @@ -31,11 +20,11 @@ steps: cache_key: '{{ .Repo.Name }}_{{ checksum "garage/Cargo.lock" }}_{{ arch }}_{{ os }}_gzip' region: garage mount: - - 'garage/target' - - 'cargo/registry/index' - - 'cargo/registry/cache' - - 'cargo/bin' - - 'cargo/git/db' + - 'target' + - '/drone/cargo/registry/index' + - '/drone/cargo/registry/cache' + - '/drone/cargo/bin' + - '/drone/cargo/git/db' path_style: true endpoint: https://garage.deuxfleurs.fr @@ -47,7 +36,6 @@ steps: - apt-get update - apt-get install --yes libsodium-dev - pwd - - cd garage - cargo build - name: cargo-test @@ -57,7 +45,6 @@ steps: commands: - apt-get update - apt-get install --yes libsodium-dev - - cd garage - cargo test - name: rebuild-cache @@ -75,11 +62,11 @@ steps: cache_key: '{{ .Repo.Name }}_{{ checksum "garage/Cargo.lock" }}_{{ arch }}_{{ os }}_gzip' region: garage mount: - - 'garage/target' - - 'cargo/registry/index' - - 'cargo/registry/cache' - - 'cargo/git/db' - - 'cargo/bin' + - 'target' + - '/drone/cargo/registry/index' + - '/drone/cargo/registry/cache' + - '/drone/cargo/git/db' + - '/drone/cargo/bin' path_style: true endpoint: https://garage.deuxfleurs.fr @@ -91,5 +78,4 @@ steps: - apt-get update - apt-get install --yes libsodium-dev awscli python-pip - pip install s3cmd - - cd garage - ./script/test-smoke.sh || (cat /tmp/garage.log; false) -- cgit v1.2.3 From 4eb16e886388f35d2bdee52b16922421004cf132 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 18 Mar 2021 19:24:59 +0100 Subject: Allow to import keys from previous Garage instance --- src/garage/admin_rpc.rs | 11 +++++++++++ src/garage/cli.rs | 17 +++++++++++++++++ src/model/key_table.rs | 9 +++++++++ 3 files changed, 37 insertions(+) diff --git a/src/garage/admin_rpc.rs b/src/garage/admin_rpc.rs index 10087f74..df00fcaf 100644 --- a/src/garage/admin_rpc.rs +++ b/src/garage/admin_rpc.rs @@ -245,6 +245,17 @@ impl AdminRpcHandler { key.key_id ))) } + KeyOperation::Import(query) => { + let prev_key = self.garage.key_table.get(&EmptyKey, &query.key_id) + .await?; + if prev_key.is_some() { + return Err(Error::Message(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); + self.garage.key_table.insert(&imported_key).await?; + Ok(AdminRPC::KeyInfo(imported_key)) + + } } } diff --git a/src/garage/cli.rs b/src/garage/cli.rs index e74f59a2..21bafebd 100644 --- a/src/garage/cli.rs +++ b/src/garage/cli.rs @@ -194,6 +194,10 @@ pub enum KeyOperation { /// Delete key #[structopt(name = "delete")] Delete(KeyDeleteOpt), + + /// Import key + #[structopt(name = "import")] + Import(KeyImportOpt), } #[derive(Serialize, Deserialize, StructOpt, Debug)] @@ -228,6 +232,19 @@ pub struct KeyDeleteOpt { pub yes: bool, } +#[derive(Serialize, Deserialize, StructOpt, Debug)] +pub struct KeyImportOpt { + /// Access key ID + pub key_id: String, + + /// Secret access key + pub secret_key: String, + + /// Key name + #[structopt(short = "n", default_value = "Imported key")] + pub name: String, +} + #[derive(Serialize, Deserialize, StructOpt, Debug, Clone)] pub struct RepairOpt { /// Launch repair operation on all nodes diff --git a/src/model/key_table.rs b/src/model/key_table.rs index 02dcf68c..fcca3835 100644 --- a/src/model/key_table.rs +++ b/src/model/key_table.rs @@ -34,6 +34,15 @@ impl Key { authorized_buckets: crdt::LWWMap::new(), } } + pub fn import(key_id: &str, secret_key: &str, name: &str) -> Self { + Self { + key_id: key_id.to_string(), + secret_key: secret_key.to_string(), + name: crdt::LWW::new(name.to_string()), + deleted: crdt::Bool::new(false), + authorized_buckets: crdt::LWWMap::new(), + } + } pub fn delete(key_id: String) -> Self { Self { key_id, -- cgit v1.2.3