From 0962313ebd45abb116d6ad2ee0eb754f587fc299 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 8 Nov 2023 13:11:13 +0100 Subject: garage_rpc: reorder functions in layout.rs --- src/rpc/layout.rs | 223 +++++++++++++++++++++++++++--------------------------- 1 file changed, 113 insertions(+), 110 deletions(-) diff --git a/src/rpc/layout.rs b/src/rpc/layout.rs index e02a180b..368a9d2c 100644 --- a/src/rpc/layout.rs +++ b/src/rpc/layout.rs @@ -278,86 +278,7 @@ impl ClusterLayout { ret } - fn calculate_staging_hash(&self) -> Hash { - let hashed_tuple = (&self.staging_roles, &self.staging_parameters); - blake2sum(&nonversioned_encode(&hashed_tuple).unwrap()[..]) - } - - pub fn merge(&mut self, other: &ClusterLayout) -> bool { - match other.version.cmp(&self.version) { - Ordering::Greater => { - *self = other.clone(); - true - } - Ordering::Equal => { - self.staging_parameters.merge(&other.staging_parameters); - self.staging_roles.merge(&other.staging_roles); - - let new_staging_hash = self.calculate_staging_hash(); - let changed = new_staging_hash != self.staging_hash; - - self.staging_hash = new_staging_hash; - - changed - } - Ordering::Less => false, - } - } - - pub fn apply_staged_changes(mut self, version: Option) -> Result<(Self, Message), Error> { - match version { - None => { - let error = r#" -Please pass the new layout version number to ensure that you are writing the correct version of the cluster layout. -To know the correct value of the new layout version, invoke `garage layout show` and review the proposed changes. - "#; - return Err(Error::Message(error.into())); - } - Some(v) => { - if v != self.version + 1 { - return Err(Error::Message("Invalid new layout version".into())); - } - } - } - - self.roles.merge(&self.staging_roles); - self.roles.retain(|(_, _, v)| v.0.is_some()); - self.parameters = *self.staging_parameters.get(); - - self.staging_roles.clear(); - self.staging_hash = self.calculate_staging_hash(); - - let msg = self.calculate_partition_assignment()?; - - self.version += 1; - - Ok((self, msg)) - } - - pub fn revert_staged_changes(mut self, version: Option) -> Result { - match version { - None => { - let error = r#" -Please pass the new layout version number to ensure that you are writing the correct version of the cluster layout. -To know the correct value of the new layout version, invoke `garage layout show` and review the proposed changes. - "#; - return Err(Error::Message(error.into())); - } - Some(v) => { - if v != self.version + 1 { - return Err(Error::Message("Invalid new layout version".into())); - } - } - } - - self.staging_roles.clear(); - self.staging_parameters.update(self.parameters); - self.staging_hash = self.calculate_staging_hash(); - - self.version += 1; - - Ok(self) - } + // ===================== accessors ====================== /// Returns a list of IDs of nodes that currently have /// a role in the cluster @@ -377,28 +298,6 @@ To know the correct value of the new layout version, invoke `garage layout show` } } - /// Returns the uuids of the non_gateway nodes in self.node_id_vec. - fn nongateway_nodes(&self) -> Vec { - let mut result = Vec::::new(); - for uuid in self.node_id_vec.iter() { - match self.node_role(uuid) { - Some(role) if role.capacity.is_some() => result.push(*uuid), - _ => (), - } - } - result - } - - /// Given a node uuids, this function returns the label of its zone - fn get_node_zone(&self, uuid: &Uuid) -> Result { - match self.node_role(uuid) { - Some(role) => Ok(role.zone.clone()), - _ => Err(Error::Message( - "The Uuid does not correspond to a node present in the cluster.".into(), - )), - } - } - /// Given a node uuids, this function returns its capacity or fails if it does not have any pub fn get_node_capacity(&self, uuid: &Uuid) -> Result { match self.node_role(uuid) { @@ -435,6 +334,30 @@ To know the correct value of the new layout version, invoke `garage layout show` )) } + // ===================== internal information extractors ====================== + + /// Returns the uuids of the non_gateway nodes in self.node_id_vec. + fn nongateway_nodes(&self) -> Vec { + let mut result = Vec::::new(); + for uuid in self.node_id_vec.iter() { + match self.node_role(uuid) { + Some(role) if role.capacity.is_some() => result.push(*uuid), + _ => (), + } + } + result + } + + /// Given a node uuids, this function returns the label of its zone + fn get_node_zone(&self, uuid: &Uuid) -> Result<&str, Error> { + match self.node_role(uuid) { + Some(role) => Ok(&role.zone), + _ => Err(Error::Message( + "The Uuid does not correspond to a node present in the cluster.".into(), + )), + } + } + /// Returns the sum of capacities of non gateway nodes in the cluster fn get_total_capacity(&self) -> Result { let mut total_capacity = 0; @@ -461,6 +384,89 @@ To know the correct value of the new layout version, invoke `garage layout show` } } + fn calculate_staging_hash(&self) -> Hash { + let hashed_tuple = (&self.staging_roles, &self.staging_parameters); + blake2sum(&nonversioned_encode(&hashed_tuple).unwrap()[..]) + } + + // ================== updates to layout, public interface =================== + + pub fn merge(&mut self, other: &ClusterLayout) -> bool { + match other.version.cmp(&self.version) { + Ordering::Greater => { + *self = other.clone(); + true + } + Ordering::Equal => { + self.staging_parameters.merge(&other.staging_parameters); + self.staging_roles.merge(&other.staging_roles); + + let new_staging_hash = self.calculate_staging_hash(); + let changed = new_staging_hash != self.staging_hash; + + self.staging_hash = new_staging_hash; + + changed + } + Ordering::Less => false, + } + } + + pub fn apply_staged_changes(mut self, version: Option) -> Result<(Self, Message), Error> { + match version { + None => { + let error = r#" +Please pass the new layout version number to ensure that you are writing the correct version of the cluster layout. +To know the correct value of the new layout version, invoke `garage layout show` and review the proposed changes. + "#; + return Err(Error::Message(error.into())); + } + Some(v) => { + if v != self.version + 1 { + return Err(Error::Message("Invalid new layout version".into())); + } + } + } + + self.roles.merge(&self.staging_roles); + self.roles.retain(|(_, _, v)| v.0.is_some()); + self.parameters = *self.staging_parameters.get(); + + self.staging_roles.clear(); + self.staging_hash = self.calculate_staging_hash(); + + let msg = self.calculate_partition_assignment()?; + + self.version += 1; + + Ok((self, msg)) + } + + pub fn revert_staged_changes(mut self, version: Option) -> Result { + match version { + None => { + let error = r#" +Please pass the new layout version number to ensure that you are writing the correct version of the cluster layout. +To know the correct value of the new layout version, invoke `garage layout show` and review the proposed changes. + "#; + return Err(Error::Message(error.into())); + } + Some(v) => { + if v != self.version + 1 { + return Err(Error::Message("Invalid new layout version".into())); + } + } + } + + self.staging_roles.clear(); + self.staging_parameters.update(self.parameters); + self.staging_hash = self.calculate_staging_hash(); + + self.version += 1; + + Ok(self) + } + /// Check a cluster layout for internal consistency /// (assignment, roles, parameters, partition size) /// returns true if consistent, false if error @@ -574,12 +580,9 @@ To know the correct value of the new layout version, invoke `garage layout show` Ok(()) } -} -// ==================================================================================== + // ================== updates to layout, internals =================== -// Implementation of the ClusterLayout methods related to the assignment algorithm. -impl ClusterLayout { /// This function calculates a new partition-to-node assignment. /// The computed assignment respects the node replication factor /// and the zone redundancy parameter It maximizes the capacity of a @@ -867,7 +870,7 @@ impl ClusterLayout { } for n in 0..self.nongateway_nodes().len() { let node_capacity = self.get_node_capacity(&self.node_id_vec[n])?; - let node_zone = zone_to_id[&self.get_node_zone(&self.node_id_vec[n])?]; + let node_zone = zone_to_id[self.get_node_zone(&self.node_id_vec[n])?]; g.add_edge(Vertex::N(n), Vertex::Sink, node_capacity / partition_size)?; for p in 0..NB_PARTITIONS { if !exclude_assoc.contains(&(p, n)) { @@ -913,7 +916,7 @@ impl ClusterLayout { // The algorithm is such that it will start with the flow that we just computed // and find ameliorating paths from that. for (p, n) in exclude_edge.iter() { - let node_zone = zone_to_id[&self.get_node_zone(&self.node_id_vec[*n])?]; + let node_zone = zone_to_id[self.get_node_zone(&self.node_id_vec[*n])?]; g.add_edge(Vertex::PZ(*p, node_zone), Vertex::N(*n), 1)?; } g.compute_maximal_flow()?; @@ -933,7 +936,7 @@ impl ClusterLayout { let mut cost = CostFunction::new(); for (p, assoc_p) in prev_assign.iter().enumerate() { for n in assoc_p.iter() { - let node_zone = zone_to_id[&self.get_node_zone(&self.node_id_vec[*n])?]; + let node_zone = zone_to_id[self.get_node_zone(&self.node_id_vec[*n])?]; cost.insert((Vertex::PZ(p, node_zone), Vertex::N(*n)), -1); } } @@ -1035,7 +1038,7 @@ impl ClusterLayout { let mut old_zones_of_p = Vec::::new(); for n in prev_assign[p].iter() { old_zones_of_p - .push(zone_to_id[&self.get_node_zone(&self.node_id_vec[*n])?]); + .push(zone_to_id[self.get_node_zone(&self.node_id_vec[*n])?]); } if !old_zones_of_p.contains(&z) { new_partitions_zone[z] += 1; -- cgit v1.2.3 From 12d1dbfc6b884be488e2d79c0b9e3c47490f5442 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 8 Nov 2023 15:41:24 +0100 Subject: remove Ring and use ClusterLayout everywhere --- src/api/admin/bucket.rs | 4 +- src/api/k2v/index.rs | 8 +- src/api/s3/put.rs | 2 +- src/garage/admin/bucket.rs | 4 +- src/garage/admin/mod.rs | 20 ++--- src/model/helper/bucket.rs | 6 +- src/model/index_counter.rs | 6 +- src/rpc/layout.rs | 72 ++++++++++++++-- src/rpc/lib.rs | 1 - src/rpc/ring.rs | 164 ------------------------------------ src/rpc/rpc_helper.rs | 14 +-- src/rpc/system.rs | 55 ++++++------ src/table/merkle.rs | 2 +- src/table/replication/fullcopy.rs | 8 +- src/table/replication/parameters.rs | 2 +- src/table/replication/sharded.rs | 14 +-- src/table/sync.rs | 20 ++--- 17 files changed, 148 insertions(+), 254 deletions(-) delete mode 100644 src/rpc/ring.rs diff --git a/src/api/admin/bucket.rs b/src/api/admin/bucket.rs index 17f46c30..6bff7e9f 100644 --- a/src/api/admin/bucket.rs +++ b/src/api/admin/bucket.rs @@ -122,7 +122,7 @@ async fn bucket_info_results( .table .get(&bucket_id, &EmptyKey) .await? - .map(|x| x.filtered_values(&garage.system.ring.borrow())) + .map(|x| x.filtered_values(&garage.system.layout_watch.borrow())) .unwrap_or_default(); let mpu_counters = garage @@ -130,7 +130,7 @@ async fn bucket_info_results( .table .get(&bucket_id, &EmptyKey) .await? - .map(|x| x.filtered_values(&garage.system.ring.borrow())) + .map(|x| x.filtered_values(&garage.system.layout_watch.borrow())) .unwrap_or_default(); let mut relevant_keys = HashMap::new(); diff --git a/src/api/k2v/index.rs b/src/api/k2v/index.rs index 6c1d4a91..ff8beda3 100644 --- a/src/api/k2v/index.rs +++ b/src/api/k2v/index.rs @@ -5,7 +5,7 @@ use serde::Serialize; use garage_util::data::*; -use garage_rpc::ring::Ring; +use garage_rpc::layout::ClusterLayout; use garage_table::util::*; use garage_model::garage::Garage; @@ -26,7 +26,7 @@ pub async fn handle_read_index( ) -> Result, Error> { let reverse = reverse.unwrap_or(false); - let ring: Arc = garage.system.ring.borrow().clone(); + let layout: Arc = garage.system.layout_watch.borrow().clone(); let (partition_keys, more, next_start) = read_range( &garage.k2v.counter_table.table, @@ -35,7 +35,7 @@ pub async fn handle_read_index( &start, &end, limit, - Some((DeletedFilter::NotDeleted, ring.layout.node_id_vec.clone())), + Some((DeletedFilter::NotDeleted, layout.node_id_vec.clone())), EnumerationOrder::from_reverse(reverse), ) .await?; @@ -54,7 +54,7 @@ pub async fn handle_read_index( partition_keys: partition_keys .into_iter() .map(|part| { - let vals = part.filtered_values(&ring); + let vals = part.filtered_values(&layout); ReadIndexResponseEntry { pk: part.sk, entries: *vals.get(&s_entries).unwrap_or(&0), diff --git a/src/api/s3/put.rs b/src/api/s3/put.rs index 606facc4..fc17ed03 100644 --- a/src/api/s3/put.rs +++ b/src/api/s3/put.rs @@ -253,7 +253,7 @@ pub(crate) async fn check_quotas( .await?; let counters = counters - .map(|x| x.filtered_values(&garage.system.ring.borrow())) + .map(|x| x.filtered_values(&garage.system.layout_watch.borrow())) .unwrap_or_default(); let (prev_cnt_obj, prev_cnt_size) = match prev_object { diff --git a/src/garage/admin/bucket.rs b/src/garage/admin/bucket.rs index 0781cb8b..34e48292 100644 --- a/src/garage/admin/bucket.rs +++ b/src/garage/admin/bucket.rs @@ -70,7 +70,7 @@ impl AdminRpcHandler { .table .get(&bucket_id, &EmptyKey) .await? - .map(|x| x.filtered_values(&self.garage.system.ring.borrow())) + .map(|x| x.filtered_values(&self.garage.system.layout_watch.borrow())) .unwrap_or_default(); let mpu_counters = self @@ -79,7 +79,7 @@ impl AdminRpcHandler { .table .get(&bucket_id, &EmptyKey) .await? - .map(|x| x.filtered_values(&self.garage.system.ring.borrow())) + .map(|x| x.filtered_values(&self.garage.system.layout_watch.borrow())) .unwrap_or_default(); let mut relevant_keys = HashMap::new(); diff --git a/src/garage/admin/mod.rs b/src/garage/admin/mod.rs index b6f9c426..006f71cd 100644 --- a/src/garage/admin/mod.rs +++ b/src/garage/admin/mod.rs @@ -18,7 +18,7 @@ use garage_util::error::Error as GarageError; use garage_table::replication::*; use garage_table::*; -use garage_rpc::ring::PARTITION_BITS; +use garage_rpc::layout::PARTITION_BITS; use garage_rpc::*; use garage_block::manager::BlockResyncErrorInfo; @@ -126,8 +126,8 @@ impl AdminRpcHandler { opt_to_send.all_nodes = false; let mut failures = vec![]; - let ring = self.garage.system.ring.borrow().clone(); - for node in ring.layout.node_ids().iter() { + let layout = self.garage.system.layout_watch.borrow().clone(); + for node in layout.node_ids().iter() { let node = (*node).into(); let resp = self .endpoint @@ -163,9 +163,9 @@ 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(); + let layout = self.garage.system.layout_watch.borrow().clone(); - for node in ring.layout.node_ids().iter() { + for node in layout.node_ids().iter() { let mut opt = opt.clone(); opt.all_nodes = false; opt.skip_global = true; @@ -275,7 +275,7 @@ impl AdminRpcHandler { let mut ret = String::new(); // Gather storage node and free space statistics - let layout = &self.garage.system.ring.borrow().layout; + let layout = &self.garage.system.layout_watch.borrow(); let mut node_partition_count = HashMap::::new(); for short_id in layout.ring_assignment_data.iter() { let id = layout.node_id_vec[*short_id as usize]; @@ -440,8 +440,8 @@ impl AdminRpcHandler { ) -> Result { if all_nodes { let mut ret = vec![]; - let ring = self.garage.system.ring.borrow().clone(); - for node in ring.layout.node_ids().iter() { + let layout = self.garage.system.layout_watch.borrow().clone(); + for node in layout.node_ids().iter() { let node = (*node).into(); match self .endpoint @@ -488,8 +488,8 @@ impl AdminRpcHandler { ) -> Result { if all_nodes { let mut ret = vec![]; - let ring = self.garage.system.ring.borrow().clone(); - for node in ring.layout.node_ids().iter() { + let layout = self.garage.system.layout_watch.borrow().clone(); + for node in layout.node_ids().iter() { let node = (*node).into(); match self .endpoint diff --git a/src/model/helper/bucket.rs b/src/model/helper/bucket.rs index 576d03f3..d43d7e96 100644 --- a/src/model/helper/bucket.rs +++ b/src/model/helper/bucket.rs @@ -450,10 +450,10 @@ impl<'a> BucketHelper<'a> { #[cfg(feature = "k2v")] { - use garage_rpc::ring::Ring; + use garage_rpc::layout::ClusterLayout; use std::sync::Arc; - let ring: Arc = self.0.system.ring.borrow().clone(); + let layout: Arc = self.0.system.layout_watch.borrow().clone(); let k2vindexes = self .0 .k2v @@ -462,7 +462,7 @@ impl<'a> BucketHelper<'a> { .get_range( &bucket_id, None, - Some((DeletedFilter::NotDeleted, ring.layout.node_id_vec.clone())), + Some((DeletedFilter::NotDeleted, layout.node_id_vec.clone())), 10, EnumerationOrder::Forward, ) diff --git a/src/model/index_counter.rs b/src/model/index_counter.rs index a46c165f..d514cb06 100644 --- a/src/model/index_counter.rs +++ b/src/model/index_counter.rs @@ -7,7 +7,7 @@ use serde::{Deserialize, Serialize}; use garage_db as db; -use garage_rpc::ring::Ring; +use garage_rpc::layout::ClusterLayout; use garage_rpc::system::System; use garage_util::background::BackgroundRunner; use garage_util::data::*; @@ -83,8 +83,8 @@ impl Entry for CounterEntry { } impl CounterEntry { - pub fn filtered_values(&self, ring: &Ring) -> HashMap { - let nodes = &ring.layout.node_id_vec[..]; + pub fn filtered_values(&self, layout: &ClusterLayout) -> HashMap { + let nodes = &layout.node_id_vec[..]; self.filtered_values_with_nodes(nodes) } diff --git a/src/rpc/layout.rs b/src/rpc/layout.rs index 368a9d2c..2b5b6606 100644 --- a/src/rpc/layout.rs +++ b/src/rpc/layout.rs @@ -13,17 +13,39 @@ use garage_util::error::*; use crate::graph_algo::*; -use crate::ring::*; - use std::convert::TryInto; +// ---- defines: partitions ---- + +/// A partition id, which is stored 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. +// Maximum value : 16 +/// How many bits from the hash are used to make partitions. Higher numbers means more fairness in +/// presence of numerous nodes, but exponentially bigger ring. Max 16 +pub const PARTITION_BITS: usize = 8; + const NB_PARTITIONS: usize = 1usize << PARTITION_BITS; +// ---- defines: nodes ---- + +// Type to store compactly the id of a node in the system +// Change this to u16 the day we want to have more than 256 nodes in a cluster +pub type CompactNodeType = u8; +pub const MAX_NODE_NUMBER: usize = 256; + +// ---- defines: other ---- + // The Message type will be used to collect information on the algorithm. -type Message = Vec; +pub type Message = Vec; mod v08 { - use crate::ring::CompactNodeType; + use super::CompactNodeType; use garage_util::crdt::LwwMap; use garage_util::data::{Hash, Uuid}; use serde::{Deserialize, Serialize}; @@ -76,7 +98,7 @@ mod v08 { mod v09 { use super::v08; - use crate::ring::CompactNodeType; + use super::CompactNodeType; use garage_util::crdt::{Lww, LwwMap}; use garage_util::data::{Hash, Uuid}; use serde::{Deserialize, Serialize}; @@ -334,6 +356,46 @@ impl ClusterLayout { )) } + /// Get the partition in which data would fall on + pub fn partition_of(&self, position: &Hash) -> Partition { + let top = u16::from_be_bytes(position.as_slice()[0..2].try_into().unwrap()); + top >> (16 - PARTITION_BITS) + } + + /// Get the list of partitions and the first hash of a partition key that would fall in it + pub fn partitions(&self) -> Vec<(Partition, Hash)> { + (0..(1 << PARTITION_BITS)) + .map(|i| { + let top = (i as u16) << (16 - PARTITION_BITS); + let mut location = [0u8; 32]; + location[..2].copy_from_slice(&u16::to_be_bytes(top)[..]); + (i as u16, Hash::from(location)) + }) + .collect::>() + } + + /// Walk the ring to find the n servers in which data should be replicated + pub fn nodes_of(&self, position: &Hash, n: usize) -> Vec { + assert_eq!(n, self.replication_factor); + + let data = &self.ring_assignment_data; + + if data.len() != self.replication_factor * (1 << PARTITION_BITS) { + warn!("Ring not yet ready, read/writes will be lost!"); + return vec![]; + } + + let partition_idx = self.partition_of(position) as usize; + let partition_start = partition_idx * self.replication_factor; + let partition_end = (partition_idx + 1) * self.replication_factor; + let partition_nodes = &data[partition_start..partition_end]; + + partition_nodes + .iter() + .map(|i| self.node_id_vec[*i as usize]) + .collect::>() + } + // ===================== internal information extractors ====================== /// Returns the uuids of the non_gateway nodes in self.node_id_vec. diff --git a/src/rpc/lib.rs b/src/rpc/lib.rs index a5f8fc6e..1af8b78e 100644 --- a/src/rpc/lib.rs +++ b/src/rpc/lib.rs @@ -14,7 +14,6 @@ mod kubernetes; pub mod graph_algo; pub mod layout; pub mod replication_mode; -pub mod ring; pub mod system; pub mod rpc_helper; diff --git a/src/rpc/ring.rs b/src/rpc/ring.rs deleted file mode 100644 index 6a2e5c72..00000000 --- a/src/rpc/ring.rs +++ /dev/null @@ -1,164 +0,0 @@ -//! Module containing types related to computing nodes which should receive a copy of data blocks -//! and metadata -use std::convert::TryInto; - -use garage_util::data::*; - -use crate::layout::ClusterLayout; - -/// A partition id, which is stored 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. -// Maximum value : 16 -/// How many bits from the hash are used to make partitions. Higher numbers means more fairness in -/// presence of numerous nodes, but exponentially bigger ring. Max 16 -pub const PARTITION_BITS: usize = 8; - -const PARTITION_MASK_U16: u16 = ((1 << PARTITION_BITS) - 1) << (16 - PARTITION_BITS); - -/// A ring distributing fairly objects to nodes -#[derive(Clone)] -pub struct Ring { - /// The replication factor for this ring - pub replication_factor: usize, - - /// The network configuration used to generate this ring - pub layout: ClusterLayout, - - // Internal order of nodes used to make a more compact representation of the ring - nodes: Vec, - - // The list of entries in the ring - ring: Vec, -} - -// Type to store compactly the id of a node in the system -// Change this to u16 the day we want to have more than 256 nodes in a cluster -pub type CompactNodeType = u8; -pub const MAX_NODE_NUMBER: usize = 256; - -// The maximum number of times an object might get replicated -// This must be at least 3 because Garage supports 3-way replication -// Here we use 6 so that the size of a ring entry is 8 bytes -// (2 bytes partition id, 6 bytes node numbers as u8s) -const MAX_REPLICATION: usize = 6; - -/// An entry in the ring -#[derive(Clone, Debug)] -struct RingEntry { - // The two first bytes of the first hash that goes in this partition - // (the next bytes are zeroes) - hash_prefix: u16, - // The nodes that store this partition, stored as a list of positions in the `nodes` - // field of the Ring structure - // Only items 0 up to ring.replication_factor - 1 are used, others are zeros - nodes_buf: [CompactNodeType; MAX_REPLICATION], -} - -impl Ring { - pub(crate) fn new(layout: ClusterLayout, replication_factor: usize) -> Self { - if replication_factor != layout.replication_factor { - warn!("Could not build ring: replication factor does not match between local configuration and network role assignment."); - return Self::empty(layout, replication_factor); - } - - if layout.ring_assignment_data.len() != replication_factor * (1 << PARTITION_BITS) { - warn!("Could not build ring: network role assignment data has invalid length"); - return Self::empty(layout, replication_factor); - } - - let nodes = layout.node_id_vec.clone(); - let ring = (0..(1 << PARTITION_BITS)) - .map(|i| { - let top = (i as u16) << (16 - PARTITION_BITS); - let mut nodes_buf = [0u8; MAX_REPLICATION]; - nodes_buf[..replication_factor].copy_from_slice( - &layout.ring_assignment_data - [replication_factor * i..replication_factor * (i + 1)], - ); - RingEntry { - hash_prefix: top, - nodes_buf, - } - }) - .collect::>(); - - Self { - replication_factor, - layout, - nodes, - ring, - } - } - - fn empty(layout: ClusterLayout, replication_factor: usize) -> Self { - Self { - replication_factor, - layout, - nodes: vec![], - ring: vec![], - } - } - - /// Get the partition in which data would fall on - pub fn partition_of(&self, position: &Hash) -> Partition { - let top = u16::from_be_bytes(position.as_slice()[0..2].try_into().unwrap()); - top >> (16 - PARTITION_BITS) - } - - /// Get the list of partitions and the first hash of a partition key that would fall in it - pub fn partitions(&self) -> Vec<(Partition, Hash)> { - let mut ret = vec![]; - - for (i, entry) in self.ring.iter().enumerate() { - let mut location = [0u8; 32]; - location[..2].copy_from_slice(&u16::to_be_bytes(entry.hash_prefix)[..]); - ret.push((i as u16, location.into())); - } - if !ret.is_empty() { - assert_eq!(ret[0].1, [0u8; 32].into()); - } - - ret - } - - /// Walk the ring to find the n servers in which data should be replicated - pub fn get_nodes(&self, position: &Hash, n: usize) -> Vec { - if self.ring.len() != 1 << PARTITION_BITS { - warn!("Ring not yet ready, read/writes will be lost!"); - return vec![]; - } - - let partition_idx = self.partition_of(position) as usize; - let partition = &self.ring[partition_idx]; - - let top = u16::from_be_bytes(position.as_slice()[0..2].try_into().unwrap()); - // Check that we haven't messed up our partition table, i.e. that this partition - // table entrey indeed corresponds to the item we are storing - assert_eq!( - partition.hash_prefix & PARTITION_MASK_U16, - top & PARTITION_MASK_U16 - ); - - assert!(n <= self.replication_factor); - partition.nodes_buf[..n] - .iter() - .map(|i| self.nodes[*i as usize]) - .collect::>() - } -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_ring_entry_size() { - assert_eq!(std::mem::size_of::(), 8); - } -} diff --git a/src/rpc/rpc_helper.rs b/src/rpc/rpc_helper.rs index e59c372a..56bef2f3 100644 --- a/src/rpc/rpc_helper.rs +++ b/src/rpc/rpc_helper.rs @@ -26,8 +26,8 @@ use garage_util::data::*; use garage_util::error::Error; use garage_util::metrics::RecordDuration; +use crate::layout::ClusterLayout; use crate::metrics::RpcMetrics; -use crate::ring::Ring; // Default RPC timeout = 5 minutes const DEFAULT_TIMEOUT: Duration = Duration::from_secs(300); @@ -91,7 +91,7 @@ pub struct RpcHelper(Arc); struct RpcHelperInner { our_node_id: Uuid, fullmesh: Arc, - ring: watch::Receiver>, + layout_watch: watch::Receiver>, metrics: RpcMetrics, rpc_timeout: Duration, } @@ -100,7 +100,7 @@ impl RpcHelper { pub(crate) fn new( our_node_id: Uuid, fullmesh: Arc, - ring: watch::Receiver>, + layout_watch: watch::Receiver>, rpc_timeout: Option, ) -> Self { let metrics = RpcMetrics::new(); @@ -108,7 +108,7 @@ impl RpcHelper { Self(Arc::new(RpcHelperInner { our_node_id, fullmesh, - ring, + layout_watch, metrics, rpc_timeout: rpc_timeout.unwrap_or(DEFAULT_TIMEOUT), })) @@ -392,8 +392,8 @@ impl RpcHelper { pub fn request_order(&self, nodes: &[Uuid]) -> Vec { // Retrieve some status variables that we will use to sort requests let peer_list = self.0.fullmesh.get_peer_list(); - let ring: Arc = self.0.ring.borrow().clone(); - let our_zone = match ring.layout.node_role(&self.0.our_node_id) { + let layout: Arc = self.0.layout_watch.borrow().clone(); + let our_zone = match layout.node_role(&self.0.our_node_id) { Some(pc) => &pc.zone, None => "", }; @@ -407,7 +407,7 @@ impl RpcHelper { let mut nodes = nodes .iter() .map(|to| { - let peer_zone = match ring.layout.node_role(to) { + let peer_zone = match layout.node_role(to) { Some(pc) => &pc.zone, None => "", }; diff --git a/src/rpc/system.rs b/src/rpc/system.rs index 4b40bec4..106e9f8c 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -36,7 +36,6 @@ use crate::consul::ConsulDiscovery; use crate::kubernetes::*; use crate::layout::*; use crate::replication_mode::*; -use crate::ring::*; use crate::rpc_helper::*; use crate::system_metrics::*; @@ -112,9 +111,9 @@ pub struct System { replication_mode: ReplicationMode, replication_factor: usize, - /// The ring - pub ring: watch::Receiver>, - update_ring: Mutex>>, + /// The layout + pub layout_watch: watch::Receiver>, + update_layout: Mutex>>, /// Path to metadata directory pub metadata_dir: PathBuf, @@ -286,8 +285,7 @@ impl System { let mut local_status = NodeStatus::initial(replication_factor, &cluster_layout); local_status.update_disk_usage(&config.metadata_dir, &config.data_dir, &metrics); - let ring = Ring::new(cluster_layout, replication_factor); - let (update_ring, ring) = watch::channel(Arc::new(ring)); + let (update_layout, layout_watch) = watch::channel(Arc::new(cluster_layout)); let rpc_public_addr = match &config.rpc_public_addr { Some(a_str) => { @@ -362,7 +360,7 @@ impl System { rpc: RpcHelper::new( netapp.id.into(), fullmesh, - ring.clone(), + layout_watch.clone(), config.rpc_timeout_msec.map(Duration::from_millis), ), system_endpoint, @@ -378,8 +376,8 @@ impl System { kubernetes_discovery: config.kubernetes_discovery.clone(), metrics, - ring, - update_ring: Mutex::new(update_ring), + layout_watch, + update_layout: Mutex::new(update_layout), metadata_dir: config.metadata_dir.clone(), data_dir: config.data_dir.clone(), }); @@ -426,7 +424,7 @@ impl System { } pub fn get_cluster_layout(&self) -> ClusterLayout { - self.ring.borrow().layout.clone() + self.layout_watch.borrow().as_ref().clone() } pub async fn update_cluster_layout( @@ -466,7 +464,7 @@ impl System { } pub fn health(&self) -> ClusterHealth { - let ring: Arc<_> = self.ring.borrow().clone(); + let layout: Arc<_> = self.layout_watch.borrow().clone(); let quorum = self.replication_mode.write_quorum(); let replication_factor = self.replication_factor; @@ -477,8 +475,7 @@ impl System { .collect::>(); let connected_nodes = nodes.iter().filter(|(_, n)| n.is_up).count(); - let storage_nodes = ring - .layout + let storage_nodes = layout .roles .items() .iter() @@ -489,11 +486,11 @@ impl System { .filter(|(x, _, _)| nodes.get(x).map(|n| n.is_up).unwrap_or(false)) .count(); - let partitions = ring.partitions(); + let partitions = layout.partitions(); let partitions_n_up = partitions .iter() .map(|(_, h)| { - let pn = ring.get_nodes(h, ring.replication_factor); + let pn = layout.nodes_of(h, layout.replication_factor); pn.iter() .filter(|x| nodes.get(x).map(|n| n.is_up).unwrap_or(false)) .count() @@ -584,9 +581,9 @@ impl System { /// Save network configuration to disc async fn save_cluster_layout(&self) -> Result<(), Error> { - let ring: Arc = self.ring.borrow().clone(); + let layout: Arc = self.layout_watch.borrow().clone(); self.persist_cluster_layout - .save_async(&ring.layout) + .save_async(&layout) .await .expect("Cannot save current cluster layout"); Ok(()) @@ -595,9 +592,9 @@ impl System { fn update_local_status(&self) { let mut new_si: NodeStatus = self.local_status.load().as_ref().clone(); - let ring = self.ring.borrow(); - new_si.cluster_layout_version = ring.layout.version; - new_si.cluster_layout_staging_hash = ring.layout.staging_hash; + let layout = self.layout_watch.borrow(); + new_si.cluster_layout_version = layout.version; + new_si.cluster_layout_staging_hash = layout.staging_hash; new_si.update_disk_usage(&self.metadata_dir, &self.data_dir, &self.metrics); @@ -612,8 +609,8 @@ impl System { } fn handle_pull_cluster_layout(&self) -> SystemRpc { - let ring = self.ring.borrow().clone(); - SystemRpc::AdvertiseClusterLayout(ring.layout.clone()) + let layout = self.layout_watch.borrow().as_ref().clone(); + SystemRpc::AdvertiseClusterLayout(layout) } fn handle_get_known_nodes(&self) -> SystemRpc { @@ -663,8 +660,9 @@ impl System { return Err(Error::Message(msg)); } - let update_ring = self.update_ring.lock().await; - let mut layout: ClusterLayout = self.ring.borrow().layout.clone(); + let update_layout = self.update_layout.lock().await; + // TODO: don't clone each time an AdvertiseClusterLayout is received + let mut layout: ClusterLayout = self.layout_watch.borrow().as_ref().clone(); let prev_layout_check = layout.check().is_ok(); if layout.merge(adv) { @@ -675,9 +673,8 @@ impl System { )); } - let ring = Ring::new(layout.clone(), self.replication_factor); - update_ring.send(Arc::new(ring))?; - drop(update_ring); + update_layout.send(Arc::new(layout.clone()))?; + drop(update_layout); let self2 = self.clone(); tokio::spawn(async move { @@ -725,9 +722,9 @@ impl System { async fn discovery_loop(self: &Arc, mut stop_signal: watch::Receiver) { while !*stop_signal.borrow() { - let not_configured = self.ring.borrow().layout.check().is_err(); + let not_configured = self.layout_watch.borrow().check().is_err(); let no_peers = self.fullmesh.get_peer_list().len() < self.replication_factor; - let expected_n_nodes = self.ring.borrow().layout.num_nodes(); + let expected_n_nodes = self.layout_watch.borrow().num_nodes(); let bad_peers = self .fullmesh .get_peer_list() diff --git a/src/table/merkle.rs b/src/table/merkle.rs index 4577f872..01271c58 100644 --- a/src/table/merkle.rs +++ b/src/table/merkle.rs @@ -13,7 +13,7 @@ use garage_util::data::*; use garage_util::encode::{nonversioned_decode, nonversioned_encode}; use garage_util::error::Error; -use garage_rpc::ring::*; +use garage_rpc::layout::*; use crate::data::*; use crate::replication::*; diff --git a/src/table/replication/fullcopy.rs b/src/table/replication/fullcopy.rs index 18682ace..f8b7cacc 100644 --- a/src/table/replication/fullcopy.rs +++ b/src/table/replication/fullcopy.rs @@ -1,6 +1,6 @@ use std::sync::Arc; -use garage_rpc::ring::*; +use garage_rpc::layout::*; use garage_rpc::system::System; use garage_util::data::*; @@ -27,11 +27,11 @@ impl TableReplication for TableFullReplication { } fn write_nodes(&self, _hash: &Hash) -> Vec { - let ring = self.system.ring.borrow(); - ring.layout.node_ids().to_vec() + let layout = self.system.layout_watch.borrow(); + layout.node_ids().to_vec() } fn write_quorum(&self) -> usize { - let nmembers = self.system.ring.borrow().layout.node_ids().len(); + let nmembers = self.system.layout_watch.borrow().node_ids().len(); if nmembers > self.max_faults { nmembers - self.max_faults } else { diff --git a/src/table/replication/parameters.rs b/src/table/replication/parameters.rs index f00815a2..19b306f2 100644 --- a/src/table/replication/parameters.rs +++ b/src/table/replication/parameters.rs @@ -1,4 +1,4 @@ -use garage_rpc::ring::*; +use garage_rpc::layout::*; use garage_util::data::*; /// Trait to describe how a table shall be replicated diff --git a/src/table/replication/sharded.rs b/src/table/replication/sharded.rs index 1cf964af..95901a5a 100644 --- a/src/table/replication/sharded.rs +++ b/src/table/replication/sharded.rs @@ -1,6 +1,6 @@ use std::sync::Arc; -use garage_rpc::ring::*; +use garage_rpc::layout::*; use garage_rpc::system::System; use garage_util::data::*; @@ -26,16 +26,16 @@ pub struct TableShardedReplication { impl TableReplication for TableShardedReplication { fn read_nodes(&self, hash: &Hash) -> Vec { - let ring = self.system.ring.borrow(); - ring.get_nodes(hash, self.replication_factor) + let layout = self.system.layout_watch.borrow(); + layout.nodes_of(hash, self.replication_factor) } fn read_quorum(&self) -> usize { self.read_quorum } fn write_nodes(&self, hash: &Hash) -> Vec { - let ring = self.system.ring.borrow(); - ring.get_nodes(hash, self.replication_factor) + let layout = self.system.layout_watch.borrow(); + layout.nodes_of(hash, self.replication_factor) } fn write_quorum(&self) -> usize { self.write_quorum @@ -45,9 +45,9 @@ impl TableReplication for TableShardedReplication { } fn partition_of(&self, hash: &Hash) -> Partition { - self.system.ring.borrow().partition_of(hash) + self.system.layout_watch.borrow().partition_of(hash) } fn partitions(&self) -> Vec<(Partition, Hash)> { - self.system.ring.borrow().partitions() + self.system.layout_watch.borrow().partitions() } } diff --git a/src/table/sync.rs b/src/table/sync.rs index 92a353c6..b2600013 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -17,7 +17,7 @@ use garage_util::data::*; use garage_util::encode::{debug_serialize, nonversioned_encode}; use garage_util::error::{Error, OkOrMessage}; -use garage_rpc::ring::*; +use garage_rpc::layout::*; use garage_rpc::system::System; use garage_rpc::*; @@ -91,8 +91,8 @@ impl TableSyncer { bg.spawn_worker(SyncWorker { syncer: self.clone(), - ring_recv: self.system.ring.clone(), - ring: self.system.ring.borrow().clone(), + layout_watch: self.system.layout_watch.clone(), + layout: self.system.layout_watch.borrow().clone(), add_full_sync_rx, todo: vec![], next_full_sync: Instant::now() + Duration::from_secs(20), @@ -492,8 +492,8 @@ impl EndpointHandler for TableSync struct SyncWorker { syncer: Arc>, - ring_recv: watch::Receiver>, - ring: Arc, + layout_watch: watch::Receiver>, + layout: Arc, add_full_sync_rx: mpsc::UnboundedReceiver<()>, todo: Vec, next_full_sync: Instant, @@ -593,11 +593,11 @@ impl Worker for SyncWorker { self.add_full_sync(); } }, - _ = self.ring_recv.changed() => { - let new_ring = self.ring_recv.borrow(); - if !Arc::ptr_eq(&new_ring, &self.ring) { - self.ring = new_ring.clone(); - drop(new_ring); + _ = self.layout_watch.changed() => { + let new_layout = self.layout_watch.borrow(); + if !Arc::ptr_eq(&new_layout, &self.layout) { + self.layout = new_layout.clone(); + drop(new_layout); debug!("({}) Ring changed, adding full sync to syncer todo list", F::TABLE_NAME); self.add_full_sync(); } -- cgit v1.2.3 From 4a9c94514f49aa4e9880a8e0f5cf5a52d11ae993 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 8 Nov 2023 16:41:00 +0100 Subject: avoid using layout_watch in System directly --- src/api/admin/bucket.rs | 4 ++-- src/api/admin/cluster.rs | 10 +++++----- src/api/k2v/index.rs | 2 +- src/api/s3/put.rs | 2 +- src/garage/admin/bucket.rs | 4 ++-- src/garage/admin/mod.rs | 10 +++++----- src/model/helper/bucket.rs | 2 +- src/rpc/system.rs | 4 ++-- src/table/replication/fullcopy.rs | 4 ++-- src/table/replication/sharded.rs | 8 ++++---- src/table/sync.rs | 2 +- 11 files changed, 26 insertions(+), 26 deletions(-) diff --git a/src/api/admin/bucket.rs b/src/api/admin/bucket.rs index 6bff7e9f..65929d61 100644 --- a/src/api/admin/bucket.rs +++ b/src/api/admin/bucket.rs @@ -122,7 +122,7 @@ async fn bucket_info_results( .table .get(&bucket_id, &EmptyKey) .await? - .map(|x| x.filtered_values(&garage.system.layout_watch.borrow())) + .map(|x| x.filtered_values(&garage.system.cluster_layout())) .unwrap_or_default(); let mpu_counters = garage @@ -130,7 +130,7 @@ async fn bucket_info_results( .table .get(&bucket_id, &EmptyKey) .await? - .map(|x| x.filtered_values(&garage.system.layout_watch.borrow())) + .map(|x| x.filtered_values(&garage.system.cluster_layout())) .unwrap_or_default(); let mut relevant_keys = HashMap::new(); diff --git a/src/api/admin/cluster.rs b/src/api/admin/cluster.rs index c8107b82..01ff9885 100644 --- a/src/api/admin/cluster.rs +++ b/src/api/admin/cluster.rs @@ -33,7 +33,7 @@ pub async fn handle_get_cluster_status(garage: &Arc) -> Result) -> Result, Error> { - let res = format_cluster_layout(&garage.system.get_cluster_layout()); + let res = format_cluster_layout(&garage.system.cluster_layout()); Ok(json_ok_response(&res)?) } @@ -207,7 +207,7 @@ pub async fn handle_update_cluster_layout( ) -> Result, Error> { let updates = parse_json_body::(req).await?; - let mut layout = garage.system.get_cluster_layout(); + let mut layout = garage.system.cluster_layout().as_ref().clone(); let mut roles = layout.roles.clone(); roles.merge(&layout.staging_roles); @@ -247,7 +247,7 @@ pub async fn handle_apply_cluster_layout( ) -> Result, Error> { let param = parse_json_body::(req).await?; - let layout = garage.system.get_cluster_layout(); + let layout = garage.system.cluster_layout().as_ref().clone(); let (layout, msg) = layout.apply_staged_changes(Some(param.version))?; garage.system.update_cluster_layout(&layout).await?; @@ -265,7 +265,7 @@ pub async fn handle_revert_cluster_layout( ) -> Result, Error> { let param = parse_json_body::(req).await?; - let layout = garage.system.get_cluster_layout(); + let layout = garage.system.cluster_layout().as_ref().clone(); let layout = layout.revert_staged_changes(Some(param.version))?; garage.system.update_cluster_layout(&layout).await?; diff --git a/src/api/k2v/index.rs b/src/api/k2v/index.rs index ff8beda3..3fc39de6 100644 --- a/src/api/k2v/index.rs +++ b/src/api/k2v/index.rs @@ -26,7 +26,7 @@ pub async fn handle_read_index( ) -> Result, Error> { let reverse = reverse.unwrap_or(false); - let layout: Arc = garage.system.layout_watch.borrow().clone(); + let layout: Arc = garage.system.cluster_layout().clone(); let (partition_keys, more, next_start) = read_range( &garage.k2v.counter_table.table, diff --git a/src/api/s3/put.rs b/src/api/s3/put.rs index fc17ed03..d1c88a76 100644 --- a/src/api/s3/put.rs +++ b/src/api/s3/put.rs @@ -253,7 +253,7 @@ pub(crate) async fn check_quotas( .await?; let counters = counters - .map(|x| x.filtered_values(&garage.system.layout_watch.borrow())) + .map(|x| x.filtered_values(&garage.system.cluster_layout())) .unwrap_or_default(); let (prev_cnt_obj, prev_cnt_size) = match prev_object { diff --git a/src/garage/admin/bucket.rs b/src/garage/admin/bucket.rs index 34e48292..9e642f57 100644 --- a/src/garage/admin/bucket.rs +++ b/src/garage/admin/bucket.rs @@ -70,7 +70,7 @@ impl AdminRpcHandler { .table .get(&bucket_id, &EmptyKey) .await? - .map(|x| x.filtered_values(&self.garage.system.layout_watch.borrow())) + .map(|x| x.filtered_values(&self.garage.system.cluster_layout())) .unwrap_or_default(); let mpu_counters = self @@ -79,7 +79,7 @@ impl AdminRpcHandler { .table .get(&bucket_id, &EmptyKey) .await? - .map(|x| x.filtered_values(&self.garage.system.layout_watch.borrow())) + .map(|x| x.filtered_values(&self.garage.system.cluster_layout())) .unwrap_or_default(); let mut relevant_keys = HashMap::new(); diff --git a/src/garage/admin/mod.rs b/src/garage/admin/mod.rs index 006f71cd..c3fa801a 100644 --- a/src/garage/admin/mod.rs +++ b/src/garage/admin/mod.rs @@ -126,7 +126,7 @@ impl AdminRpcHandler { opt_to_send.all_nodes = false; let mut failures = vec![]; - let layout = self.garage.system.layout_watch.borrow().clone(); + let layout = self.garage.system.cluster_layout().clone(); for node in layout.node_ids().iter() { let node = (*node).into(); let resp = self @@ -163,7 +163,7 @@ impl AdminRpcHandler { async fn handle_stats(&self, opt: StatsOpt) -> Result { if opt.all_nodes { let mut ret = String::new(); - let layout = self.garage.system.layout_watch.borrow().clone(); + let layout = self.garage.system.cluster_layout().clone(); for node in layout.node_ids().iter() { let mut opt = opt.clone(); @@ -275,7 +275,7 @@ impl AdminRpcHandler { let mut ret = String::new(); // Gather storage node and free space statistics - let layout = &self.garage.system.layout_watch.borrow(); + let layout = &self.garage.system.cluster_layout(); let mut node_partition_count = HashMap::::new(); for short_id in layout.ring_assignment_data.iter() { let id = layout.node_id_vec[*short_id as usize]; @@ -440,7 +440,7 @@ impl AdminRpcHandler { ) -> Result { if all_nodes { let mut ret = vec![]; - let layout = self.garage.system.layout_watch.borrow().clone(); + let layout = self.garage.system.cluster_layout().clone(); for node in layout.node_ids().iter() { let node = (*node).into(); match self @@ -488,7 +488,7 @@ impl AdminRpcHandler { ) -> Result { if all_nodes { let mut ret = vec![]; - let layout = self.garage.system.layout_watch.borrow().clone(); + let layout = self.garage.system.cluster_layout().clone(); for node in layout.node_ids().iter() { let node = (*node).into(); match self diff --git a/src/model/helper/bucket.rs b/src/model/helper/bucket.rs index d43d7e96..8cd5b27b 100644 --- a/src/model/helper/bucket.rs +++ b/src/model/helper/bucket.rs @@ -453,7 +453,7 @@ impl<'a> BucketHelper<'a> { use garage_rpc::layout::ClusterLayout; use std::sync::Arc; - let layout: Arc = self.0.system.layout_watch.borrow().clone(); + let layout: Arc = self.0.system.cluster_layout().clone(); let k2vindexes = self .0 .k2v diff --git a/src/rpc/system.rs b/src/rpc/system.rs index 106e9f8c..93144e39 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -423,8 +423,8 @@ impl System { known_nodes } - pub fn get_cluster_layout(&self) -> ClusterLayout { - self.layout_watch.borrow().as_ref().clone() + pub fn cluster_layout(&self) -> watch::Ref> { + self.layout_watch.borrow() } pub async fn update_cluster_layout( diff --git a/src/table/replication/fullcopy.rs b/src/table/replication/fullcopy.rs index f8b7cacc..34807e3d 100644 --- a/src/table/replication/fullcopy.rs +++ b/src/table/replication/fullcopy.rs @@ -27,11 +27,11 @@ impl TableReplication for TableFullReplication { } fn write_nodes(&self, _hash: &Hash) -> Vec { - let layout = self.system.layout_watch.borrow(); + let layout = self.system.cluster_layout(); layout.node_ids().to_vec() } fn write_quorum(&self) -> usize { - let nmembers = self.system.layout_watch.borrow().node_ids().len(); + let nmembers = self.system.cluster_layout().node_ids().len(); if nmembers > self.max_faults { nmembers - self.max_faults } else { diff --git a/src/table/replication/sharded.rs b/src/table/replication/sharded.rs index 95901a5a..60c95cb4 100644 --- a/src/table/replication/sharded.rs +++ b/src/table/replication/sharded.rs @@ -26,7 +26,7 @@ pub struct TableShardedReplication { impl TableReplication for TableShardedReplication { fn read_nodes(&self, hash: &Hash) -> Vec { - let layout = self.system.layout_watch.borrow(); + let layout = self.system.cluster_layout(); layout.nodes_of(hash, self.replication_factor) } fn read_quorum(&self) -> usize { @@ -34,7 +34,7 @@ impl TableReplication for TableShardedReplication { } fn write_nodes(&self, hash: &Hash) -> Vec { - let layout = self.system.layout_watch.borrow(); + let layout = self.system.cluster_layout(); layout.nodes_of(hash, self.replication_factor) } fn write_quorum(&self) -> usize { @@ -45,9 +45,9 @@ impl TableReplication for TableShardedReplication { } fn partition_of(&self, hash: &Hash) -> Partition { - self.system.layout_watch.borrow().partition_of(hash) + self.system.cluster_layout().partition_of(hash) } fn partitions(&self) -> Vec<(Partition, Hash)> { - self.system.layout_watch.borrow().partitions() + self.system.cluster_layout().partitions() } } diff --git a/src/table/sync.rs b/src/table/sync.rs index b2600013..65eff7cd 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -92,7 +92,7 @@ impl TableSyncer { bg.spawn_worker(SyncWorker { syncer: self.clone(), layout_watch: self.system.layout_watch.clone(), - layout: self.system.layout_watch.borrow().clone(), + layout: self.system.cluster_layout().clone(), add_full_sync_rx, todo: vec![], next_full_sync: Instant::now() + Duration::from_secs(20), -- cgit v1.2.3 From fe9af1dcaae31a117528a9cfa10c422c9a850201 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 8 Nov 2023 17:49:06 +0100 Subject: WIP: garage_rpc: store layout version history --- src/rpc/layout.rs | 1358 --------------------------------------------- src/rpc/layout/history.rs | 170 ++++++ src/rpc/layout/mod.rs | 32 ++ src/rpc/layout/schema.rs | 286 ++++++++++ src/rpc/layout/tracker.rs | 21 + src/rpc/layout/version.rs | 1052 +++++++++++++++++++++++++++++++++++ src/rpc/rpc_helper.rs | 12 +- src/rpc/system.rs | 44 +- 8 files changed, 1590 insertions(+), 1385 deletions(-) delete mode 100644 src/rpc/layout.rs create mode 100644 src/rpc/layout/history.rs create mode 100644 src/rpc/layout/mod.rs create mode 100644 src/rpc/layout/schema.rs create mode 100644 src/rpc/layout/tracker.rs create mode 100644 src/rpc/layout/version.rs diff --git a/src/rpc/layout.rs b/src/rpc/layout.rs deleted file mode 100644 index 2b5b6606..00000000 --- a/src/rpc/layout.rs +++ /dev/null @@ -1,1358 +0,0 @@ -use std::cmp::Ordering; -use std::collections::HashMap; -use std::collections::HashSet; -use std::fmt; - -use bytesize::ByteSize; -use itertools::Itertools; - -use garage_util::crdt::{AutoCrdt, Crdt, Lww, LwwMap}; -use garage_util::data::*; -use garage_util::encode::nonversioned_encode; -use garage_util::error::*; - -use crate::graph_algo::*; - -use std::convert::TryInto; - -// ---- defines: partitions ---- - -/// A partition id, which is stored 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. -// Maximum value : 16 -/// How many bits from the hash are used to make partitions. Higher numbers means more fairness in -/// presence of numerous nodes, but exponentially bigger ring. Max 16 -pub const PARTITION_BITS: usize = 8; - -const NB_PARTITIONS: usize = 1usize << PARTITION_BITS; - -// ---- defines: nodes ---- - -// Type to store compactly the id of a node in the system -// Change this to u16 the day we want to have more than 256 nodes in a cluster -pub type CompactNodeType = u8; -pub const MAX_NODE_NUMBER: usize = 256; - -// ---- defines: other ---- - -// The Message type will be used to collect information on the algorithm. -pub type Message = Vec; - -mod v08 { - use super::CompactNodeType; - use garage_util::crdt::LwwMap; - use garage_util::data::{Hash, Uuid}; - use serde::{Deserialize, Serialize}; - - /// The layout of the cluster, i.e. the list of roles - /// which are assigned to each cluster node - #[derive(Clone, Debug, Serialize, Deserialize)] - pub struct ClusterLayout { - pub version: u64, - - pub replication_factor: usize, - pub roles: LwwMap, - - /// node_id_vec: a vector of node IDs with a role assigned - /// in the system (this includes gateway nodes). - /// The order here is different than the vec stored by `roles`, because: - /// 1. non-gateway nodes are first so that they have lower numbers - /// 2. nodes that don't have a role are excluded (but they need to - /// stay in the CRDT as tombstones) - pub node_id_vec: Vec, - /// the assignation of data partitions to node, the values - /// are indices in node_id_vec - #[serde(with = "serde_bytes")] - pub ring_assignation_data: Vec, - - /// Role changes which are staged for the next version of the layout - pub staging: LwwMap, - pub staging_hash: Hash, - } - - #[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Debug, Serialize, Deserialize)] - pub struct NodeRoleV(pub Option); - - /// The user-assigned roles of cluster nodes - #[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Debug, Serialize, Deserialize)] - pub struct NodeRole { - /// Datacenter at which this entry belong. This information is used to - /// perform a better geodistribution - pub zone: String, - /// The capacity of the node - /// If this is set to None, the node does not participate in storing data for the system - /// and is only active as an API gateway to other nodes - pub capacity: Option, - /// A set of tags to recognize the node - pub tags: Vec, - } - - impl garage_util::migrate::InitialFormat for ClusterLayout {} -} - -mod v09 { - use super::v08; - use super::CompactNodeType; - use garage_util::crdt::{Lww, LwwMap}; - use garage_util::data::{Hash, Uuid}; - use serde::{Deserialize, Serialize}; - pub use v08::{NodeRole, NodeRoleV}; - - /// The layout of the cluster, i.e. the list of roles - /// which are assigned to each cluster node - #[derive(Clone, Debug, Serialize, Deserialize)] - pub struct ClusterLayout { - pub version: u64, - - pub replication_factor: usize, - - /// This attribute is only used to retain the previously computed partition size, - /// to know to what extent does it change with the layout update. - pub partition_size: u64, - /// Parameters used to compute the assignment currently given by - /// ring_assignment_data - pub parameters: LayoutParameters, - - pub roles: LwwMap, - - /// see comment in v08::ClusterLayout - pub node_id_vec: Vec, - /// see comment in v08::ClusterLayout - #[serde(with = "serde_bytes")] - pub ring_assignment_data: Vec, - - /// Parameters to be used in the next partition assignment computation. - pub staging_parameters: Lww, - /// Role changes which are staged for the next version of the layout - pub staging_roles: LwwMap, - pub staging_hash: Hash, - } - - /// This struct is used to set the parameters to be used in the assignment computation - /// algorithm. It is stored as a Crdt. - #[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Copy, Debug, Serialize, Deserialize)] - pub struct LayoutParameters { - pub zone_redundancy: ZoneRedundancy, - } - - /// Zone redundancy: if set to AtLeast(x), the layout calculation will aim to store copies - /// of each partition on at least that number of different zones. - /// Otherwise, copies will be stored on the maximum possible number of zones. - #[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Copy, Debug, Serialize, Deserialize)] - pub enum ZoneRedundancy { - AtLeast(usize), - Maximum, - } - - impl garage_util::migrate::Migrate for ClusterLayout { - const VERSION_MARKER: &'static [u8] = b"G09layout"; - - type Previous = v08::ClusterLayout; - - fn migrate(previous: Self::Previous) -> Self { - use itertools::Itertools; - - // In the old layout, capacities are in an arbitrary unit, - // but in the new layout they are in bytes. - // Here we arbitrarily multiply everything by 1G, - // such that 1 old capacity unit = 1GB in the new units. - // This is totally arbitrary and won't work for most users. - let cap_mul = 1024 * 1024 * 1024; - let roles = multiply_all_capacities(previous.roles, cap_mul); - let staging_roles = multiply_all_capacities(previous.staging, cap_mul); - let node_id_vec = previous.node_id_vec; - - // Determine partition size - let mut tmp = previous.ring_assignation_data.clone(); - tmp.sort(); - let partition_size = tmp - .into_iter() - .dedup_with_count() - .map(|(npart, node)| { - roles - .get(&node_id_vec[node as usize]) - .and_then(|p| p.0.as_ref().and_then(|r| r.capacity)) - .unwrap_or(0) / npart as u64 - }) - .min() - .unwrap_or(0); - - // By default, zone_redundancy is maximum possible value - let parameters = LayoutParameters { - zone_redundancy: ZoneRedundancy::Maximum, - }; - - let mut res = Self { - version: previous.version, - replication_factor: previous.replication_factor, - partition_size, - parameters, - roles, - node_id_vec, - ring_assignment_data: previous.ring_assignation_data, - staging_parameters: Lww::new(parameters), - staging_roles, - staging_hash: [0u8; 32].into(), - }; - res.staging_hash = res.calculate_staging_hash(); - res - } - } - - fn multiply_all_capacities( - old_roles: LwwMap, - mul: u64, - ) -> LwwMap { - let mut new_roles = LwwMap::new(); - for (node, ts, role) in old_roles.items() { - let mut role = role.clone(); - if let NodeRoleV(Some(NodeRole { - capacity: Some(ref mut cap), - .. - })) = role - { - *cap *= mul; - } - new_roles.merge_raw(node, *ts, &role); - } - new_roles - } -} - -pub use v09::*; - -impl AutoCrdt for LayoutParameters { - const WARN_IF_DIFFERENT: bool = true; -} - -impl AutoCrdt for NodeRoleV { - const WARN_IF_DIFFERENT: bool = true; -} - -impl NodeRole { - pub fn capacity_string(&self) -> String { - match self.capacity { - Some(c) => ByteSize::b(c).to_string_as(false), - None => "gateway".to_string(), - } - } - - pub fn tags_string(&self) -> String { - self.tags.join(",") - } -} - -impl fmt::Display for ZoneRedundancy { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match self { - ZoneRedundancy::Maximum => write!(f, "maximum"), - ZoneRedundancy::AtLeast(x) => write!(f, "{}", x), - } - } -} - -impl core::str::FromStr for ZoneRedundancy { - type Err = &'static str; - fn from_str(s: &str) -> Result { - match s { - "none" | "max" | "maximum" => Ok(ZoneRedundancy::Maximum), - x => { - let v = x - .parse::() - .map_err(|_| "zone redundancy must be 'none'/'max' or an integer")?; - Ok(ZoneRedundancy::AtLeast(v)) - } - } - } -} - -// Implementation of the ClusterLayout methods unrelated to the assignment algorithm. -impl ClusterLayout { - pub fn new(replication_factor: usize) -> Self { - // We set the default zone redundancy to be Maximum, meaning that the maximum - // possible value will be used depending on the cluster topology - let parameters = LayoutParameters { - zone_redundancy: ZoneRedundancy::Maximum, - }; - let staging_parameters = Lww::::new(parameters); - - let empty_lwwmap = LwwMap::new(); - - let mut ret = ClusterLayout { - version: 0, - replication_factor, - partition_size: 0, - roles: LwwMap::new(), - node_id_vec: Vec::new(), - ring_assignment_data: Vec::new(), - parameters, - staging_parameters, - staging_roles: empty_lwwmap, - staging_hash: [0u8; 32].into(), - }; - ret.staging_hash = ret.calculate_staging_hash(); - ret - } - - // ===================== accessors ====================== - - /// Returns a list of IDs of nodes that currently have - /// a role in the cluster - pub fn node_ids(&self) -> &[Uuid] { - &self.node_id_vec[..] - } - - pub fn num_nodes(&self) -> usize { - self.node_id_vec.len() - } - - /// Returns the role of a node in the layout - pub fn node_role(&self, node: &Uuid) -> Option<&NodeRole> { - match self.roles.get(node) { - Some(NodeRoleV(Some(v))) => Some(v), - _ => None, - } - } - - /// Given a node uuids, this function returns its capacity or fails if it does not have any - pub fn get_node_capacity(&self, uuid: &Uuid) -> Result { - match self.node_role(uuid) { - Some(NodeRole { - capacity: Some(cap), - zone: _, - tags: _, - }) => Ok(*cap), - _ => Err(Error::Message( - "The Uuid does not correspond to a node present in the \ - cluster or this node does not have a positive capacity." - .into(), - )), - } - } - - /// Returns the number of partitions associated to this node in the ring - pub fn get_node_usage(&self, uuid: &Uuid) -> Result { - for (i, id) in self.node_id_vec.iter().enumerate() { - if id == uuid { - let mut count = 0; - for nod in self.ring_assignment_data.iter() { - if i as u8 == *nod { - count += 1 - } - } - return Ok(count); - } - } - Err(Error::Message( - "The Uuid does not correspond to a node present in the \ - cluster or this node does not have a positive capacity." - .into(), - )) - } - - /// Get the partition in which data would fall on - pub fn partition_of(&self, position: &Hash) -> Partition { - let top = u16::from_be_bytes(position.as_slice()[0..2].try_into().unwrap()); - top >> (16 - PARTITION_BITS) - } - - /// Get the list of partitions and the first hash of a partition key that would fall in it - pub fn partitions(&self) -> Vec<(Partition, Hash)> { - (0..(1 << PARTITION_BITS)) - .map(|i| { - let top = (i as u16) << (16 - PARTITION_BITS); - let mut location = [0u8; 32]; - location[..2].copy_from_slice(&u16::to_be_bytes(top)[..]); - (i as u16, Hash::from(location)) - }) - .collect::>() - } - - /// Walk the ring to find the n servers in which data should be replicated - pub fn nodes_of(&self, position: &Hash, n: usize) -> Vec { - assert_eq!(n, self.replication_factor); - - let data = &self.ring_assignment_data; - - if data.len() != self.replication_factor * (1 << PARTITION_BITS) { - warn!("Ring not yet ready, read/writes will be lost!"); - return vec![]; - } - - let partition_idx = self.partition_of(position) as usize; - let partition_start = partition_idx * self.replication_factor; - let partition_end = (partition_idx + 1) * self.replication_factor; - let partition_nodes = &data[partition_start..partition_end]; - - partition_nodes - .iter() - .map(|i| self.node_id_vec[*i as usize]) - .collect::>() - } - - // ===================== internal information extractors ====================== - - /// Returns the uuids of the non_gateway nodes in self.node_id_vec. - fn nongateway_nodes(&self) -> Vec { - let mut result = Vec::::new(); - for uuid in self.node_id_vec.iter() { - match self.node_role(uuid) { - Some(role) if role.capacity.is_some() => result.push(*uuid), - _ => (), - } - } - result - } - - /// Given a node uuids, this function returns the label of its zone - fn get_node_zone(&self, uuid: &Uuid) -> Result<&str, Error> { - match self.node_role(uuid) { - Some(role) => Ok(&role.zone), - _ => Err(Error::Message( - "The Uuid does not correspond to a node present in the cluster.".into(), - )), - } - } - - /// Returns the sum of capacities of non gateway nodes in the cluster - fn get_total_capacity(&self) -> Result { - let mut total_capacity = 0; - for uuid in self.nongateway_nodes().iter() { - total_capacity += self.get_node_capacity(uuid)?; - } - Ok(total_capacity) - } - - /// Returns the effective value of the zone_redundancy parameter - fn effective_zone_redundancy(&self) -> usize { - match self.parameters.zone_redundancy { - ZoneRedundancy::AtLeast(v) => v, - ZoneRedundancy::Maximum => { - let n_zones = self - .roles - .items() - .iter() - .filter_map(|(_, _, role)| role.0.as_ref().map(|x| x.zone.as_str())) - .collect::>() - .len(); - std::cmp::min(n_zones, self.replication_factor) - } - } - } - - fn calculate_staging_hash(&self) -> Hash { - let hashed_tuple = (&self.staging_roles, &self.staging_parameters); - blake2sum(&nonversioned_encode(&hashed_tuple).unwrap()[..]) - } - - // ================== updates to layout, public interface =================== - - pub fn merge(&mut self, other: &ClusterLayout) -> bool { - match other.version.cmp(&self.version) { - Ordering::Greater => { - *self = other.clone(); - true - } - Ordering::Equal => { - self.staging_parameters.merge(&other.staging_parameters); - self.staging_roles.merge(&other.staging_roles); - - let new_staging_hash = self.calculate_staging_hash(); - let changed = new_staging_hash != self.staging_hash; - - self.staging_hash = new_staging_hash; - - changed - } - Ordering::Less => false, - } - } - - pub fn apply_staged_changes(mut self, version: Option) -> Result<(Self, Message), Error> { - match version { - None => { - let error = r#" -Please pass the new layout version number to ensure that you are writing the correct version of the cluster layout. -To know the correct value of the new layout version, invoke `garage layout show` and review the proposed changes. - "#; - return Err(Error::Message(error.into())); - } - Some(v) => { - if v != self.version + 1 { - return Err(Error::Message("Invalid new layout version".into())); - } - } - } - - self.roles.merge(&self.staging_roles); - self.roles.retain(|(_, _, v)| v.0.is_some()); - self.parameters = *self.staging_parameters.get(); - - self.staging_roles.clear(); - self.staging_hash = self.calculate_staging_hash(); - - let msg = self.calculate_partition_assignment()?; - - self.version += 1; - - Ok((self, msg)) - } - - pub fn revert_staged_changes(mut self, version: Option) -> Result { - match version { - None => { - let error = r#" -Please pass the new layout version number to ensure that you are writing the correct version of the cluster layout. -To know the correct value of the new layout version, invoke `garage layout show` and review the proposed changes. - "#; - return Err(Error::Message(error.into())); - } - Some(v) => { - if v != self.version + 1 { - return Err(Error::Message("Invalid new layout version".into())); - } - } - } - - self.staging_roles.clear(); - self.staging_parameters.update(self.parameters); - self.staging_hash = self.calculate_staging_hash(); - - self.version += 1; - - Ok(self) - } - - /// Check a cluster layout for internal consistency - /// (assignment, roles, parameters, partition size) - /// returns true if consistent, false if error - pub fn check(&self) -> Result<(), String> { - // Check that the hash of the staging data is correct - let staging_hash = self.calculate_staging_hash(); - if staging_hash != self.staging_hash { - return Err("staging_hash is incorrect".into()); - } - - // Check that node_id_vec contains the correct list of nodes - let mut expected_nodes = self - .roles - .items() - .iter() - .filter(|(_, _, v)| v.0.is_some()) - .map(|(id, _, _)| *id) - .collect::>(); - expected_nodes.sort(); - let mut node_id_vec = self.node_id_vec.clone(); - node_id_vec.sort(); - if expected_nodes != node_id_vec { - return Err(format!("node_id_vec does not contain the correct set of nodes\nnode_id_vec: {:?}\nexpected: {:?}", node_id_vec, expected_nodes)); - } - - // Check that the assignment data has the correct length - let expected_assignment_data_len = (1 << PARTITION_BITS) * self.replication_factor; - if self.ring_assignment_data.len() != expected_assignment_data_len { - return Err(format!( - "ring_assignment_data has incorrect length {} instead of {}", - self.ring_assignment_data.len(), - expected_assignment_data_len - )); - } - - // Check that the assigned nodes are correct identifiers - // of nodes that are assigned a role - // and that role is not the role of a gateway nodes - for x in self.ring_assignment_data.iter() { - if *x as usize >= self.node_id_vec.len() { - return Err(format!( - "ring_assignment_data contains invalid node id {}", - *x - )); - } - let node = self.node_id_vec[*x as usize]; - match self.roles.get(&node) { - Some(NodeRoleV(Some(x))) if x.capacity.is_some() => (), - _ => return Err("ring_assignment_data contains id of a gateway node".into()), - } - } - - // Check that every partition is associated to distinct nodes - let zone_redundancy = self.effective_zone_redundancy(); - let rf = self.replication_factor; - for p in 0..(1 << PARTITION_BITS) { - let nodes_of_p = self.ring_assignment_data[rf * p..rf * (p + 1)].to_vec(); - if nodes_of_p.iter().unique().count() != rf { - return Err(format!("partition does not contain {} unique node ids", rf)); - } - // Check that every partition is spread over at least zone_redundancy zones. - let zones_of_p = nodes_of_p - .iter() - .map(|n| { - self.get_node_zone(&self.node_id_vec[*n as usize]) - .expect("Zone not found.") - }) - .collect::>(); - if zones_of_p.iter().unique().count() < zone_redundancy { - return Err(format!( - "nodes of partition are in less than {} distinct zones", - zone_redundancy - )); - } - } - - // Check that the nodes capacities is consistent with the stored partitions - let mut node_usage = vec![0; MAX_NODE_NUMBER]; - for n in self.ring_assignment_data.iter() { - node_usage[*n as usize] += 1; - } - for (n, usage) in node_usage.iter().enumerate() { - if *usage > 0 { - let uuid = self.node_id_vec[n]; - let partusage = usage * self.partition_size; - let nodecap = self.get_node_capacity(&uuid).unwrap(); - if partusage > nodecap { - return Err(format!( - "node usage ({}) is bigger than node capacity ({})", - usage * self.partition_size, - nodecap - )); - } - } - } - - // Check that the partition size stored is the one computed by the asignation - // algorithm. - let cl2 = self.clone(); - let (_, zone_to_id) = cl2.generate_nongateway_zone_ids().unwrap(); - match cl2.compute_optimal_partition_size(&zone_to_id, zone_redundancy) { - Ok(s) if s != self.partition_size => { - return Err(format!( - "partition_size ({}) is different than optimal value ({})", - self.partition_size, s - )) - } - Err(e) => return Err(format!("could not calculate optimal partition size: {}", e)), - _ => (), - } - - Ok(()) - } - - // ================== updates to layout, internals =================== - - /// This function calculates a new partition-to-node assignment. - /// The computed assignment respects the node replication factor - /// and the zone redundancy parameter It maximizes the capacity of a - /// partition (assuming all partitions have the same size). - /// Among such optimal assignment, it minimizes the distance to - /// the former assignment (if any) to minimize the amount of - /// data to be moved. - /// Staged role changes must be merged with nodes roles before calling this function, - /// hence it must only be called from apply_staged_changes() and hence is not public. - fn calculate_partition_assignment(&mut self) -> Result { - // We update the node ids, since the node role list might have changed with the - // changes in the layout. We retrieve the old_assignment reframed with new ids - let old_assignment_opt = self.update_node_id_vec()?; - - let zone_redundancy = self.effective_zone_redundancy(); - - let mut msg = Message::new(); - msg.push("==== COMPUTATION OF A NEW PARTITION ASSIGNATION ====".into()); - msg.push("".into()); - msg.push(format!( - "Partitions are \ - replicated {} times on at least {} distinct zones.", - self.replication_factor, zone_redundancy - )); - - // We generate for once numerical ids for the zones of non gateway nodes, - // to use them as indices in the flow graphs. - let (id_to_zone, zone_to_id) = self.generate_nongateway_zone_ids()?; - - let nb_nongateway_nodes = self.nongateway_nodes().len(); - if nb_nongateway_nodes < self.replication_factor { - return Err(Error::Message(format!( - "The number of nodes with positive \ - capacity ({}) is smaller than the replication factor ({}).", - nb_nongateway_nodes, self.replication_factor - ))); - } - if id_to_zone.len() < zone_redundancy { - return Err(Error::Message(format!( - "The number of zones with non-gateway \ - nodes ({}) is smaller than the redundancy parameter ({})", - id_to_zone.len(), - zone_redundancy - ))); - } - - // We compute the optimal partition size - // Capacities should be given in a unit so that partition size is at least 100. - // In this case, integer rounding plays a marginal role in the percentages of - // optimality. - let partition_size = self.compute_optimal_partition_size(&zone_to_id, zone_redundancy)?; - - msg.push("".into()); - if old_assignment_opt.is_some() { - msg.push(format!( - "Optimal partition size: {} ({} in previous layout)", - ByteSize::b(partition_size).to_string_as(false), - ByteSize::b(self.partition_size).to_string_as(false) - )); - } else { - msg.push(format!( - "Optimal partition size: {}", - ByteSize::b(partition_size).to_string_as(false) - )); - } - // We write the partition size. - self.partition_size = partition_size; - - if partition_size < 100 { - msg.push( - "WARNING: The partition size is low (< 100), make sure the capacities of your nodes are correct and are of at least a few MB" - .into(), - ); - } - - // We compute a first flow/assignment that is heuristically close to the previous - // assignment - let mut gflow = - self.compute_candidate_assignment(&zone_to_id, &old_assignment_opt, zone_redundancy)?; - if let Some(assoc) = &old_assignment_opt { - // We minimize the distance to the previous assignment. - self.minimize_rebalance_load(&mut gflow, &zone_to_id, assoc)?; - } - - // We display statistics of the computation - msg.extend(self.output_stat(&gflow, &old_assignment_opt, &zone_to_id, &id_to_zone)?); - - // We update the layout structure - self.update_ring_from_flow(id_to_zone.len(), &gflow)?; - - if let Err(e) = self.check() { - return Err(Error::Message( - format!("Layout check returned an error: {}\nOriginal result of computation: <<<<\n{}\n>>>>", e, msg.join("\n")) - )); - } - - Ok(msg) - } - - /// The LwwMap of node roles might have changed. This function updates the node_id_vec - /// and returns the assignment given by ring, with the new indices of the nodes, and - /// None if the node is not present anymore. - /// We work with the assumption that only this function and calculate_new_assignment - /// do modify assignment_ring and node_id_vec. - fn update_node_id_vec(&mut self) -> Result>>, Error> { - // (1) We compute the new node list - // Non gateway nodes should be coded on 8bits, hence they must be first in the list - // We build the new node ids - let new_non_gateway_nodes: Vec = self - .roles - .items() - .iter() - .filter(|(_, _, v)| matches!(&v.0, Some(r) if r.capacity.is_some())) - .map(|(k, _, _)| *k) - .collect(); - - if new_non_gateway_nodes.len() > MAX_NODE_NUMBER { - return Err(Error::Message(format!( - "There are more than {} non-gateway nodes in the new \ - layout. This is not allowed.", - MAX_NODE_NUMBER - ))); - } - - let new_gateway_nodes: Vec = self - .roles - .items() - .iter() - .filter(|(_, _, v)| matches!(v, NodeRoleV(Some(r)) if r.capacity.is_none())) - .map(|(k, _, _)| *k) - .collect(); - - let mut new_node_id_vec = Vec::::new(); - new_node_id_vec.extend(new_non_gateway_nodes); - new_node_id_vec.extend(new_gateway_nodes); - - let old_node_id_vec = self.node_id_vec.clone(); - self.node_id_vec = new_node_id_vec.clone(); - - // (2) We retrieve the old association - // We rewrite the old association with the new indices. We only consider partition - // to node assignments where the node is still in use. - if self.ring_assignment_data.is_empty() { - // This is a new association - return Ok(None); - } - - if self.ring_assignment_data.len() != NB_PARTITIONS * self.replication_factor { - return Err(Error::Message( - "The old assignment does not have a size corresponding to \ - the old replication factor or the number of partitions." - .into(), - )); - } - - // We build a translation table between the uuid and new ids - let mut uuid_to_new_id = HashMap::::new(); - - // We add the indices of only the new non-gateway nodes that can be used in the - // association ring - for (i, uuid) in new_node_id_vec.iter().enumerate() { - uuid_to_new_id.insert(*uuid, i); - } - - let mut old_assignment = vec![Vec::::new(); NB_PARTITIONS]; - let rf = self.replication_factor; - - for (p, old_assign_p) in old_assignment.iter_mut().enumerate() { - for old_id in &self.ring_assignment_data[p * rf..(p + 1) * rf] { - let uuid = old_node_id_vec[*old_id as usize]; - if uuid_to_new_id.contains_key(&uuid) { - old_assign_p.push(uuid_to_new_id[&uuid]); - } - } - } - - // We write the ring - self.ring_assignment_data = Vec::::new(); - - Ok(Some(old_assignment)) - } - - /// This function generates ids for the zone of the nodes appearing in - /// self.node_id_vec. - fn generate_nongateway_zone_ids(&self) -> Result<(Vec, HashMap), Error> { - let mut id_to_zone = Vec::::new(); - let mut zone_to_id = HashMap::::new(); - - for uuid in self.nongateway_nodes().iter() { - let r = self.node_role(uuid).unwrap(); - if !zone_to_id.contains_key(&r.zone) && r.capacity.is_some() { - zone_to_id.insert(r.zone.clone(), id_to_zone.len()); - id_to_zone.push(r.zone.clone()); - } - } - Ok((id_to_zone, zone_to_id)) - } - - /// This function computes by dichotomy the largest realizable partition size, given - /// the layout roles and parameters. - fn compute_optimal_partition_size( - &self, - zone_to_id: &HashMap, - zone_redundancy: usize, - ) -> Result { - let empty_set = HashSet::<(usize, usize)>::new(); - let mut g = self.generate_flow_graph(1, zone_to_id, &empty_set, zone_redundancy)?; - g.compute_maximal_flow()?; - if g.get_flow_value()? < (NB_PARTITIONS * self.replication_factor) as i64 { - return Err(Error::Message( - "The storage capacity of he cluster is to small. It is \ - impossible to store partitions of size 1." - .into(), - )); - } - - let mut s_down = 1; - let mut s_up = self.get_total_capacity()?; - while s_down + 1 < s_up { - g = self.generate_flow_graph( - (s_down + s_up) / 2, - zone_to_id, - &empty_set, - zone_redundancy, - )?; - g.compute_maximal_flow()?; - if g.get_flow_value()? < (NB_PARTITIONS * self.replication_factor) as i64 { - s_up = (s_down + s_up) / 2; - } else { - s_down = (s_down + s_up) / 2; - } - } - - Ok(s_down) - } - - fn generate_graph_vertices(nb_zones: usize, nb_nodes: usize) -> Vec { - let mut vertices = vec![Vertex::Source, Vertex::Sink]; - for p in 0..NB_PARTITIONS { - vertices.push(Vertex::Pup(p)); - vertices.push(Vertex::Pdown(p)); - for z in 0..nb_zones { - vertices.push(Vertex::PZ(p, z)); - } - } - for n in 0..nb_nodes { - vertices.push(Vertex::N(n)); - } - vertices - } - - /// Generates the graph to compute the maximal flow corresponding to the optimal - /// partition assignment. - /// exclude_assoc is the set of (partition, node) association that we are forbidden - /// to use (hence we do not add the corresponding edge to the graph). This parameter - /// is used to compute a first flow that uses only edges appearing in the previous - /// assignment. This produces a solution that heuristically should be close to the - /// previous one. - fn generate_flow_graph( - &self, - partition_size: u64, - zone_to_id: &HashMap, - exclude_assoc: &HashSet<(usize, usize)>, - zone_redundancy: usize, - ) -> Result, Error> { - let vertices = - ClusterLayout::generate_graph_vertices(zone_to_id.len(), self.nongateway_nodes().len()); - let mut g = Graph::::new(&vertices); - let nb_zones = zone_to_id.len(); - for p in 0..NB_PARTITIONS { - g.add_edge(Vertex::Source, Vertex::Pup(p), zone_redundancy as u64)?; - g.add_edge( - Vertex::Source, - Vertex::Pdown(p), - (self.replication_factor - zone_redundancy) as u64, - )?; - for z in 0..nb_zones { - g.add_edge(Vertex::Pup(p), Vertex::PZ(p, z), 1)?; - g.add_edge( - Vertex::Pdown(p), - Vertex::PZ(p, z), - self.replication_factor as u64, - )?; - } - } - for n in 0..self.nongateway_nodes().len() { - let node_capacity = self.get_node_capacity(&self.node_id_vec[n])?; - let node_zone = zone_to_id[self.get_node_zone(&self.node_id_vec[n])?]; - g.add_edge(Vertex::N(n), Vertex::Sink, node_capacity / partition_size)?; - for p in 0..NB_PARTITIONS { - if !exclude_assoc.contains(&(p, n)) { - g.add_edge(Vertex::PZ(p, node_zone), Vertex::N(n), 1)?; - } - } - } - Ok(g) - } - - /// This function computes a first optimal assignment (in the form of a flow graph). - fn compute_candidate_assignment( - &self, - zone_to_id: &HashMap, - prev_assign_opt: &Option>>, - zone_redundancy: usize, - ) -> Result, Error> { - // We list the (partition,node) associations that are not used in the - // previous assignment - let mut exclude_edge = HashSet::<(usize, usize)>::new(); - if let Some(prev_assign) = prev_assign_opt { - let nb_nodes = self.nongateway_nodes().len(); - for (p, prev_assign_p) in prev_assign.iter().enumerate() { - for n in 0..nb_nodes { - exclude_edge.insert((p, n)); - } - for n in prev_assign_p.iter() { - exclude_edge.remove(&(p, *n)); - } - } - } - - // We compute the best flow using only the edges used in the previous assignment - let mut g = self.generate_flow_graph( - self.partition_size, - zone_to_id, - &exclude_edge, - zone_redundancy, - )?; - g.compute_maximal_flow()?; - - // We add the excluded edges and compute the maximal flow with the full graph. - // The algorithm is such that it will start with the flow that we just computed - // and find ameliorating paths from that. - for (p, n) in exclude_edge.iter() { - let node_zone = zone_to_id[self.get_node_zone(&self.node_id_vec[*n])?]; - g.add_edge(Vertex::PZ(*p, node_zone), Vertex::N(*n), 1)?; - } - g.compute_maximal_flow()?; - Ok(g) - } - - /// This function updates the flow graph gflow to minimize the distance between - /// its corresponding assignment and the previous one - fn minimize_rebalance_load( - &self, - gflow: &mut Graph, - zone_to_id: &HashMap, - prev_assign: &[Vec], - ) -> Result<(), Error> { - // We define a cost function on the edges (pairs of vertices) corresponding - // to the distance between the two assignments. - let mut cost = CostFunction::new(); - for (p, assoc_p) in prev_assign.iter().enumerate() { - for n in assoc_p.iter() { - let node_zone = zone_to_id[self.get_node_zone(&self.node_id_vec[*n])?]; - cost.insert((Vertex::PZ(p, node_zone), Vertex::N(*n)), -1); - } - } - - // We compute the maximal length of a simple path in gflow. It is used in the - // Bellman-Ford algorithm in optimize_flow_with_cost to set the number - // of iterations. - let nb_nodes = self.nongateway_nodes().len(); - let path_length = 4 * nb_nodes; - gflow.optimize_flow_with_cost(&cost, path_length)?; - - Ok(()) - } - - /// This function updates the assignment ring from the flow graph. - fn update_ring_from_flow( - &mut self, - nb_zones: usize, - gflow: &Graph, - ) -> Result<(), Error> { - self.ring_assignment_data = Vec::::new(); - for p in 0..NB_PARTITIONS { - for z in 0..nb_zones { - let assoc_vertex = gflow.get_positive_flow_from(Vertex::PZ(p, z))?; - for vertex in assoc_vertex.iter() { - if let Vertex::N(n) = vertex { - self.ring_assignment_data.push((*n).try_into().unwrap()); - } - } - } - } - - if self.ring_assignment_data.len() != NB_PARTITIONS * self.replication_factor { - return Err(Error::Message( - "Critical Error : the association ring we produced does not \ - have the right size." - .into(), - )); - } - Ok(()) - } - - /// This function returns a message summing up the partition repartition of the new - /// layout, and other statistics of the partition assignment computation. - fn output_stat( - &self, - gflow: &Graph, - prev_assign_opt: &Option>>, - zone_to_id: &HashMap, - id_to_zone: &[String], - ) -> Result { - let mut msg = Message::new(); - - let used_cap = self.partition_size * NB_PARTITIONS as u64 * self.replication_factor as u64; - let total_cap = self.get_total_capacity()?; - let percent_cap = 100.0 * (used_cap as f32) / (total_cap as f32); - msg.push(format!( - "Usable capacity / total cluster capacity: {} / {} ({:.1} %)", - ByteSize::b(used_cap).to_string_as(false), - ByteSize::b(total_cap).to_string_as(false), - percent_cap - )); - msg.push(format!( - "Effective capacity (replication factor {}): {}", - self.replication_factor, - ByteSize::b(used_cap / self.replication_factor as u64).to_string_as(false) - )); - if percent_cap < 80. { - msg.push("".into()); - msg.push( - "If the percentage is too low, it might be that the \ - cluster topology and redundancy constraints are forcing the use of nodes/zones with small \ - storage capacities." - .into(), - ); - msg.push( - "You might want to move storage capacity between zones or relax the redundancy constraint." - .into(), - ); - msg.push( - "See the detailed statistics below and look for saturated nodes/zones.".into(), - ); - } - - // We define and fill in the following tables - let storing_nodes = self.nongateway_nodes(); - let mut new_partitions = vec![0; storing_nodes.len()]; - let mut stored_partitions = vec![0; storing_nodes.len()]; - - let mut new_partitions_zone = vec![0; id_to_zone.len()]; - let mut stored_partitions_zone = vec![0; id_to_zone.len()]; - - for p in 0..NB_PARTITIONS { - for z in 0..id_to_zone.len() { - let pz_nodes = gflow.get_positive_flow_from(Vertex::PZ(p, z))?; - if !pz_nodes.is_empty() { - stored_partitions_zone[z] += 1; - if let Some(prev_assign) = prev_assign_opt { - let mut old_zones_of_p = Vec::::new(); - for n in prev_assign[p].iter() { - old_zones_of_p - .push(zone_to_id[self.get_node_zone(&self.node_id_vec[*n])?]); - } - if !old_zones_of_p.contains(&z) { - new_partitions_zone[z] += 1; - } - } - } - for vert in pz_nodes.iter() { - if let Vertex::N(n) = *vert { - stored_partitions[n] += 1; - if let Some(prev_assign) = prev_assign_opt { - if !prev_assign[p].contains(&n) { - new_partitions[n] += 1; - } - } - } - } - } - } - - if prev_assign_opt.is_none() { - new_partitions = stored_partitions.clone(); - //new_partitions_zone = stored_partitions_zone.clone(); - } - - // We display the statistics - - msg.push("".into()); - if prev_assign_opt.is_some() { - let total_new_partitions: usize = new_partitions.iter().sum(); - msg.push(format!( - "A total of {} new copies of partitions need to be \ - transferred.", - total_new_partitions - )); - msg.push("".into()); - } - - let mut table = vec![]; - for z in 0..id_to_zone.len() { - let mut nodes_of_z = Vec::::new(); - for n in 0..storing_nodes.len() { - if self.get_node_zone(&self.node_id_vec[n])? == id_to_zone[z] { - nodes_of_z.push(n); - } - } - let replicated_partitions: usize = - nodes_of_z.iter().map(|n| stored_partitions[*n]).sum(); - table.push(format!( - "{}\tTags\tPartitions\tCapacity\tUsable capacity", - id_to_zone[z] - )); - - let available_cap_z: u64 = self.partition_size * replicated_partitions as u64; - let mut total_cap_z = 0; - for n in nodes_of_z.iter() { - total_cap_z += self.get_node_capacity(&self.node_id_vec[*n])?; - } - let percent_cap_z = 100.0 * (available_cap_z as f32) / (total_cap_z as f32); - - for n in nodes_of_z.iter() { - let available_cap_n = stored_partitions[*n] as u64 * self.partition_size; - let total_cap_n = self.get_node_capacity(&self.node_id_vec[*n])?; - let tags_n = (self.node_role(&self.node_id_vec[*n]).ok_or(""))?.tags_string(); - table.push(format!( - " {:?}\t{}\t{} ({} new)\t{}\t{} ({:.1}%)", - self.node_id_vec[*n], - tags_n, - stored_partitions[*n], - new_partitions[*n], - ByteSize::b(total_cap_n).to_string_as(false), - ByteSize::b(available_cap_n).to_string_as(false), - (available_cap_n as f32) / (total_cap_n as f32) * 100.0, - )); - } - - table.push(format!( - " TOTAL\t\t{} ({} unique)\t{}\t{} ({:.1}%)", - replicated_partitions, - stored_partitions_zone[z], - //new_partitions_zone[z], - ByteSize::b(total_cap_z).to_string_as(false), - ByteSize::b(available_cap_z).to_string_as(false), - percent_cap_z - )); - table.push("".into()); - } - msg.push(format_table::format_table_to_string(table)); - - Ok(msg) - } -} - -// ==================================================================================== - -#[cfg(test)] -mod tests { - use super::{Error, *}; - use std::cmp::min; - - // This function checks that the partition size S computed is at least better than the - // one given by a very naive algorithm. To do so, we try to run the naive algorithm - // assuming a partion size of S+1. If we succed, it means that the optimal assignment - // was not optimal. The naive algorithm is the following : - // - we compute the max number of partitions associated to every node, capped at the - // partition number. It gives the number of tokens of every node. - // - every zone has a number of tokens equal to the sum of the tokens of its nodes. - // - we cycle over the partitions and associate zone tokens while respecting the - // zone redundancy constraint. - // NOTE: the naive algorithm is not optimal. Counter example: - // take nb_partition = 3 ; replication_factor = 5; redundancy = 4; - // number of tokens by zone : (A, 4), (B,1), (C,4), (D, 4), (E, 2) - // With these parameters, the naive algo fails, whereas there is a solution: - // (A,A,C,D,E) , (A,B,C,D,D) (A,C,C,D,E) - fn check_against_naive(cl: &ClusterLayout) -> Result { - let over_size = cl.partition_size + 1; - let mut zone_token = HashMap::::new(); - - let (zones, zone_to_id) = cl.generate_nongateway_zone_ids()?; - - if zones.is_empty() { - return Ok(false); - } - - for z in zones.iter() { - zone_token.insert(z.clone(), 0); - } - for uuid in cl.nongateway_nodes().iter() { - let z = cl.get_node_zone(uuid)?; - let c = cl.get_node_capacity(uuid)?; - zone_token.insert( - z.clone(), - zone_token[&z] + min(NB_PARTITIONS, (c / over_size) as usize), - ); - } - - // For every partition, we count the number of zone already associated and - // the name of the last zone associated - - let mut id_zone_token = vec![0; zones.len()]; - for (z, t) in zone_token.iter() { - id_zone_token[zone_to_id[z]] = *t; - } - - let mut nb_token = vec![0; NB_PARTITIONS]; - let mut last_zone = vec![zones.len(); NB_PARTITIONS]; - - let mut curr_zone = 0; - - let redundancy = cl.effective_zone_redundancy(); - - for replic in 0..cl.replication_factor { - for p in 0..NB_PARTITIONS { - while id_zone_token[curr_zone] == 0 - || (last_zone[p] == curr_zone - && redundancy - nb_token[p] <= cl.replication_factor - replic) - { - curr_zone += 1; - if curr_zone >= zones.len() { - return Ok(true); - } - } - id_zone_token[curr_zone] -= 1; - if last_zone[p] != curr_zone { - nb_token[p] += 1; - last_zone[p] = curr_zone; - } - } - } - - return Ok(false); - } - - fn show_msg(msg: &Message) { - for s in msg.iter() { - println!("{}", s); - } - } - - fn update_layout( - cl: &mut ClusterLayout, - node_id_vec: &Vec, - node_capacity_vec: &Vec, - node_zone_vec: &Vec, - zone_redundancy: usize, - ) { - for i in 0..node_id_vec.len() { - if let Some(x) = FixedBytes32::try_from(&[i as u8; 32]) { - cl.node_id_vec.push(x); - } - - let update = cl.staging_roles.update_mutator( - cl.node_id_vec[i], - NodeRoleV(Some(NodeRole { - zone: (node_zone_vec[i].to_string()), - capacity: (Some(node_capacity_vec[i])), - tags: (vec![]), - })), - ); - cl.staging_roles.merge(&update); - } - cl.staging_parameters.update(LayoutParameters { - zone_redundancy: ZoneRedundancy::AtLeast(zone_redundancy), - }); - cl.staging_hash = cl.calculate_staging_hash(); - } - - #[test] - fn test_assignment() { - let mut node_id_vec = vec![1, 2, 3]; - let mut node_capacity_vec = vec![4000, 1000, 2000]; - let mut node_zone_vec = vec!["A", "B", "C"] - .into_iter() - .map(|x| x.to_string()) - .collect(); - - let mut cl = ClusterLayout::new(3); - update_layout(&mut cl, &node_id_vec, &node_capacity_vec, &node_zone_vec, 3); - let v = cl.version; - let (mut cl, msg) = cl.apply_staged_changes(Some(v + 1)).unwrap(); - show_msg(&msg); - assert_eq!(cl.check(), Ok(())); - assert!(matches!(check_against_naive(&cl), Ok(true))); - - node_id_vec = vec![1, 2, 3, 4, 5, 6, 7, 8, 9]; - node_capacity_vec = vec![4000, 1000, 1000, 3000, 1000, 1000, 2000, 10000, 2000]; - node_zone_vec = vec!["A", "B", "C", "C", "C", "B", "G", "H", "I"] - .into_iter() - .map(|x| x.to_string()) - .collect(); - update_layout(&mut cl, &node_id_vec, &node_capacity_vec, &node_zone_vec, 2); - let v = cl.version; - let (mut cl, msg) = cl.apply_staged_changes(Some(v + 1)).unwrap(); - show_msg(&msg); - assert_eq!(cl.check(), Ok(())); - assert!(matches!(check_against_naive(&cl), Ok(true))); - - node_capacity_vec = vec![4000, 1000, 2000, 7000, 1000, 1000, 2000, 10000, 2000]; - update_layout(&mut cl, &node_id_vec, &node_capacity_vec, &node_zone_vec, 3); - let v = cl.version; - let (mut cl, msg) = cl.apply_staged_changes(Some(v + 1)).unwrap(); - show_msg(&msg); - assert_eq!(cl.check(), Ok(())); - assert!(matches!(check_against_naive(&cl), Ok(true))); - - node_capacity_vec = vec![ - 4000000, 4000000, 2000000, 7000000, 1000000, 9000000, 2000000, 10000, 2000000, - ]; - update_layout(&mut cl, &node_id_vec, &node_capacity_vec, &node_zone_vec, 1); - let v = cl.version; - let (cl, msg) = cl.apply_staged_changes(Some(v + 1)).unwrap(); - show_msg(&msg); - assert_eq!(cl.check(), Ok(())); - assert!(matches!(check_against_naive(&cl), Ok(true))); - } -} diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs new file mode 100644 index 00000000..b3019f58 --- /dev/null +++ b/src/rpc/layout/history.rs @@ -0,0 +1,170 @@ +use std::cmp::Ordering; +use std::sync::Arc; + +use garage_util::crdt::{Crdt, Lww, LwwMap}; +use garage_util::data::*; +use garage_util::encode::nonversioned_encode; +use garage_util::error::*; + +use super::schema::*; +use super::*; + +impl LayoutHistory { + pub fn new(replication_factor: usize) -> Self { + let version = LayoutVersion::new(replication_factor); + + let staging_parameters = Lww::::new(version.parameters); + let empty_lwwmap = LwwMap::new(); + + let mut ret = LayoutHistory { + versions: vec![version].into_boxed_slice().into(), + update_trackers: Default::default(), + staging_parameters, + staging_roles: empty_lwwmap, + staging_hash: [0u8; 32].into(), + }; + ret.staging_hash = ret.calculate_staging_hash(); + ret + } + + pub fn current(&self) -> &LayoutVersion { + self.versions.last().as_ref().unwrap() + } + + pub(crate) fn calculate_staging_hash(&self) -> Hash { + let hashed_tuple = (&self.staging_roles, &self.staging_parameters); + blake2sum(&nonversioned_encode(&hashed_tuple).unwrap()[..]) + } + + // ================== updates to layout, public interface =================== + + pub fn merge(&mut self, other: &LayoutHistory) -> bool { + let mut changed = false; + + // Merge staged layout changes + match other.current().version.cmp(&self.current().version) { + Ordering::Greater => { + self.staging_parameters = other.staging_parameters.clone(); + self.staging_roles = other.staging_roles.clone(); + self.staging_hash = other.staging_hash; + changed = true; + } + Ordering::Equal => { + self.staging_parameters.merge(&other.staging_parameters); + self.staging_roles.merge(&other.staging_roles); + + let new_staging_hash = self.calculate_staging_hash(); + if new_staging_hash != self.staging_hash { + changed = true; + } + + self.staging_hash = new_staging_hash; + } + Ordering::Less => (), + } + + // Add any new versions to history + let mut versions = self.versions.to_vec(); + for v2 in other.versions.iter() { + if let Some(v1) = versions.iter().find(|v| v.version == v2.version) { + if v1 != v2 { + error!("Inconsistent layout histories: different layout compositions for version {}. Your cluster will be broken as long as this layout version is not replaced.", v2.version); + } + } else if versions.iter().all(|v| v.version != v2.version - 1) { + error!( + "Cannot receive new layout version {}, version {} is missing", + v2.version, + v2.version - 1 + ); + } else { + versions.push(v2.clone()); + changed = true; + } + } + self.versions = Arc::from(versions.into_boxed_slice()); + + // Merge trackers + self.update_trackers.merge(&other.update_trackers); + + changed + } + + pub fn apply_staged_changes(mut self, version: Option) -> Result<(Self, Message), Error> { + match version { + None => { + let error = r#" +Please pass the new layout version number to ensure that you are writing the correct version of the cluster layout. +To know the correct value of the new layout version, invoke `garage layout show` and review the proposed changes. + "#; + return Err(Error::Message(error.into())); + } + Some(v) => { + if v != self.current().version + 1 { + return Err(Error::Message("Invalid new layout version".into())); + } + } + } + + let mut new_version = self.current().clone(); + new_version.version += 1; + + new_version.roles.merge(&self.staging_roles); + new_version.roles.retain(|(_, _, v)| v.0.is_some()); + new_version.parameters = *self.staging_parameters.get(); + + self.staging_roles.clear(); + self.staging_hash = self.calculate_staging_hash(); + + let msg = new_version.calculate_partition_assignment()?; + + let mut versions = self.versions.to_vec(); + versions.push(new_version); + self.versions = Arc::from(versions.into_boxed_slice()); + + Ok((self, msg)) + } + + pub fn revert_staged_changes(mut self, version: Option) -> Result { + match version { + None => { + let error = r#" +Please pass the new layout version number to ensure that you are writing the correct version of the cluster layout. +To know the correct value of the new layout version, invoke `garage layout show` and review the proposed changes. + "#; + return Err(Error::Message(error.into())); + } + Some(v) => { + if v != self.current().version + 1 { + return Err(Error::Message("Invalid new layout version".into())); + } + } + } + + self.staging_roles.clear(); + self.staging_parameters.update(self.current().parameters); + self.staging_hash = self.calculate_staging_hash(); + + // TODO this is stupid, we should have a separate version counter/LWW + // for the staging params + let mut new_version = self.current().clone(); + new_version.version += 1; + + let mut versions = self.versions.to_vec(); + versions.push(new_version); + self.versions = Arc::from(versions.into_boxed_slice()); + + Ok(self) + } + + pub fn check(&self) -> Result<(), String> { + // Check that the hash of the staging data is correct + let staging_hash = self.calculate_staging_hash(); + if staging_hash != self.staging_hash { + return Err("staging_hash is incorrect".into()); + } + + // TODO: anythign more ? + + self.current().check() + } +} diff --git a/src/rpc/layout/mod.rs b/src/rpc/layout/mod.rs new file mode 100644 index 00000000..122d4b65 --- /dev/null +++ b/src/rpc/layout/mod.rs @@ -0,0 +1,32 @@ +mod history; +mod schema; +mod tracker; +mod version; + +pub use history::*; +pub use schema::*; +pub use version::*; + +// ---- defines: partitions ---- + +/// A partition id, which is stored 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. +// Maximum value : 16 +/// How many bits from the hash are used to make partitions. Higher numbers means more fairness in +/// presence of numerous nodes, but exponentially bigger ring. Max 16 +pub const PARTITION_BITS: usize = 8; + +const NB_PARTITIONS: usize = 1usize << PARTITION_BITS; + +// ---- defines: nodes ---- + +// Type to store compactly the id of a node in the system +// Change this to u16 the day we want to have more than 256 nodes in a cluster +pub type CompactNodeType = u8; +pub const MAX_NODE_NUMBER: usize = 256; diff --git a/src/rpc/layout/schema.rs b/src/rpc/layout/schema.rs new file mode 100644 index 00000000..fa0822fa --- /dev/null +++ b/src/rpc/layout/schema.rs @@ -0,0 +1,286 @@ +mod v08 { + use crate::layout::CompactNodeType; + use garage_util::crdt::LwwMap; + use garage_util::data::{Hash, Uuid}; + use serde::{Deserialize, Serialize}; + + /// The layout of the cluster, i.e. the list of roles + /// which are assigned to each cluster node + #[derive(Clone, Debug, Serialize, Deserialize)] + pub struct ClusterLayout { + pub version: u64, + + pub replication_factor: usize, + pub roles: LwwMap, + + /// node_id_vec: a vector of node IDs with a role assigned + /// in the system (this includes gateway nodes). + /// The order here is different than the vec stored by `roles`, because: + /// 1. non-gateway nodes are first so that they have lower numbers + /// 2. nodes that don't have a role are excluded (but they need to + /// stay in the CRDT as tombstones) + pub node_id_vec: Vec, + /// the assignation of data partitions to node, the values + /// are indices in node_id_vec + #[serde(with = "serde_bytes")] + pub ring_assignation_data: Vec, + + /// Role changes which are staged for the next version of the layout + pub staging: LwwMap, + pub staging_hash: Hash, + } + + #[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Debug, Serialize, Deserialize)] + pub struct NodeRoleV(pub Option); + + /// The user-assigned roles of cluster nodes + #[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Debug, Serialize, Deserialize)] + pub struct NodeRole { + /// Datacenter at which this entry belong. This information is used to + /// perform a better geodistribution + pub zone: String, + /// The capacity of the node + /// If this is set to None, the node does not participate in storing data for the system + /// and is only active as an API gateway to other nodes + pub capacity: Option, + /// A set of tags to recognize the node + pub tags: Vec, + } + + impl garage_util::migrate::InitialFormat for ClusterLayout {} +} + +mod v09 { + use super::v08; + use crate::layout::CompactNodeType; + use garage_util::crdt::{Lww, LwwMap}; + use garage_util::data::{Hash, Uuid}; + use serde::{Deserialize, Serialize}; + pub use v08::{NodeRole, NodeRoleV}; + + /// The layout of the cluster, i.e. the list of roles + /// which are assigned to each cluster node + #[derive(Clone, Debug, Serialize, Deserialize)] + pub struct ClusterLayout { + pub version: u64, + + pub replication_factor: usize, + + /// This attribute is only used to retain the previously computed partition size, + /// to know to what extent does it change with the layout update. + pub partition_size: u64, + /// Parameters used to compute the assignment currently given by + /// ring_assignment_data + pub parameters: LayoutParameters, + + pub roles: LwwMap, + + /// see comment in v08::ClusterLayout + pub node_id_vec: Vec, + /// see comment in v08::ClusterLayout + #[serde(with = "serde_bytes")] + pub ring_assignment_data: Vec, + + /// Parameters to be used in the next partition assignment computation. + pub staging_parameters: Lww, + /// Role changes which are staged for the next version of the layout + pub staging_roles: LwwMap, + pub staging_hash: Hash, + } + + /// This struct is used to set the parameters to be used in the assignment computation + /// algorithm. It is stored as a Crdt. + #[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Copy, Debug, Serialize, Deserialize)] + pub struct LayoutParameters { + pub zone_redundancy: ZoneRedundancy, + } + + /// Zone redundancy: if set to AtLeast(x), the layout calculation will aim to store copies + /// of each partition on at least that number of different zones. + /// Otherwise, copies will be stored on the maximum possible number of zones. + #[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Copy, Debug, Serialize, Deserialize)] + pub enum ZoneRedundancy { + AtLeast(usize), + Maximum, + } + + impl garage_util::migrate::Migrate for ClusterLayout { + const VERSION_MARKER: &'static [u8] = b"G09layout"; + + type Previous = v08::ClusterLayout; + + fn migrate(previous: Self::Previous) -> Self { + use itertools::Itertools; + + // In the old layout, capacities are in an arbitrary unit, + // but in the new layout they are in bytes. + // Here we arbitrarily multiply everything by 1G, + // such that 1 old capacity unit = 1GB in the new units. + // This is totally arbitrary and won't work for most users. + let cap_mul = 1024 * 1024 * 1024; + let roles = multiply_all_capacities(previous.roles, cap_mul); + let staging_roles = multiply_all_capacities(previous.staging, cap_mul); + let node_id_vec = previous.node_id_vec; + + // Determine partition size + let mut tmp = previous.ring_assignation_data.clone(); + tmp.sort(); + let partition_size = tmp + .into_iter() + .dedup_with_count() + .map(|(npart, node)| { + roles + .get(&node_id_vec[node as usize]) + .and_then(|p| p.0.as_ref().and_then(|r| r.capacity)) + .unwrap_or(0) / npart as u64 + }) + .min() + .unwrap_or(0); + + // By default, zone_redundancy is maximum possible value + let parameters = LayoutParameters { + zone_redundancy: ZoneRedundancy::Maximum, + }; + + Self { + version: previous.version, + replication_factor: previous.replication_factor, + partition_size, + parameters, + roles, + node_id_vec, + ring_assignment_data: previous.ring_assignation_data, + staging_parameters: Lww::new(parameters), + staging_roles, + staging_hash: [0u8; 32].into(), // will be set in the next migration + } + } + } + + fn multiply_all_capacities( + old_roles: LwwMap, + mul: u64, + ) -> LwwMap { + let mut new_roles = LwwMap::new(); + for (node, ts, role) in old_roles.items() { + let mut role = role.clone(); + if let NodeRoleV(Some(NodeRole { + capacity: Some(ref mut cap), + .. + })) = role + { + *cap *= mul; + } + new_roles.merge_raw(node, *ts, &role); + } + new_roles + } +} + +mod v010 { + use super::v09; + use crate::layout::CompactNodeType; + use garage_util::crdt::{Lww, LwwMap}; + use garage_util::data::{Hash, Uuid}; + use serde::{Deserialize, Serialize}; + use std::collections::HashMap; + use std::sync::Arc; + pub use v09::{LayoutParameters, NodeRole, NodeRoleV, ZoneRedundancy}; + + /// The layout of the cluster, i.e. the list of roles + /// which are assigned to each cluster node + #[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] + pub struct LayoutVersion { + pub version: u64, + + pub replication_factor: usize, + + /// This attribute is only used to retain the previously computed partition size, + /// to know to what extent does it change with the layout update. + pub partition_size: u64, + /// Parameters used to compute the assignment currently given by + /// ring_assignment_data + pub parameters: LayoutParameters, + + pub roles: LwwMap, + + /// see comment in v08::ClusterLayout + pub node_id_vec: Vec, + /// see comment in v08::ClusterLayout + #[serde(with = "serde_bytes")] + pub ring_assignment_data: Vec, + } + + /// The history of cluster layouts + #[derive(Clone, Debug, Serialize, Deserialize)] + pub struct LayoutHistory { + /// The versions currently in use in the cluster + pub versions: Arc<[LayoutVersion]>, + + /// Update trackers + pub update_trackers: UpdateTrackers, + + /// Parameters to be used in the next partition assignment computation. + pub staging_parameters: Lww, + /// Role changes which are staged for the next version of the layout + pub staging_roles: LwwMap, + /// Hash of the serialized staging_parameters + staging_roles + pub staging_hash: Hash, + } + + /// The tracker of acknowlegments and data syncs around the cluster + #[derive(Clone, Debug, Serialize, Deserialize, Default)] + pub struct UpdateTrackers { + /// The highest layout version number each node has ack'ed + pub ack_map: UpdateTracker, + /// The highest layout version number each node has synced data for + pub sync_map: UpdateTracker, + /// The highest layout version number each node has + /// ack'ed that all other nodes have synced data for + pub sync_ack_map: UpdateTracker, + } + + /// The history of cluster layouts + #[derive(Clone, Debug, Serialize, Deserialize, Default)] + pub struct UpdateTracker(pub HashMap); + + impl garage_util::migrate::Migrate for LayoutHistory { + const VERSION_MARKER: &'static [u8] = b"G010lh"; + + type Previous = v09::ClusterLayout; + + fn migrate(previous: Self::Previous) -> Self { + let version = LayoutVersion { + version: previous.version, + replication_factor: previous.replication_factor, + partition_size: previous.partition_size, + parameters: previous.parameters, + roles: previous.roles, + node_id_vec: previous.node_id_vec, + ring_assignment_data: previous.ring_assignment_data, + }; + let update_tracker = UpdateTracker( + version + .nongateway_nodes() + .iter() + .map(|x| (*x, version.version)) + .collect::>(), + ); + let mut ret = Self { + versions: Arc::from(vec![version].into_boxed_slice()), + update_trackers: UpdateTrackers { + ack_map: update_tracker.clone(), + sync_map: update_tracker.clone(), + sync_ack_map: update_tracker.clone(), + }, + staging_parameters: previous.staging_parameters, + staging_roles: previous.staging_roles, + staging_hash: [0u8; 32].into(), + }; + ret.staging_hash = ret.calculate_staging_hash(); + ret + } + } +} + +pub use v010::*; diff --git a/src/rpc/layout/tracker.rs b/src/rpc/layout/tracker.rs new file mode 100644 index 00000000..778121e4 --- /dev/null +++ b/src/rpc/layout/tracker.rs @@ -0,0 +1,21 @@ +use super::*; + +impl UpdateTracker { + fn merge(&mut self, other: &UpdateTracker) { + for (k, v) in other.0.iter() { + if let Some(v_mut) = self.0.get_mut(k) { + *v_mut = std::cmp::max(*v_mut, *v); + } else { + self.0.insert(*k, *v); + } + } + } +} + +impl UpdateTrackers { + pub(crate) fn merge(&mut self, other: &UpdateTrackers) { + self.ack_map.merge(&other.ack_map); + self.sync_map.merge(&other.sync_map); + self.sync_ack_map.merge(&other.sync_ack_map); + } +} diff --git a/src/rpc/layout/version.rs b/src/rpc/layout/version.rs new file mode 100644 index 00000000..363bc204 --- /dev/null +++ b/src/rpc/layout/version.rs @@ -0,0 +1,1052 @@ +use std::collections::HashMap; +use std::collections::HashSet; +use std::fmt; + +use bytesize::ByteSize; +use itertools::Itertools; + +use garage_util::crdt::{AutoCrdt, LwwMap}; +use garage_util::data::*; +use garage_util::error::*; + +use crate::graph_algo::*; + +use std::convert::TryInto; + +use super::schema::*; +use super::*; + +// The Message type will be used to collect information on the algorithm. +pub type Message = Vec; + +impl AutoCrdt for LayoutParameters { + const WARN_IF_DIFFERENT: bool = true; +} + +impl AutoCrdt for NodeRoleV { + const WARN_IF_DIFFERENT: bool = true; +} + +impl NodeRole { + pub fn capacity_string(&self) -> String { + match self.capacity { + Some(c) => ByteSize::b(c).to_string_as(false), + None => "gateway".to_string(), + } + } + + pub fn tags_string(&self) -> String { + self.tags.join(",") + } +} + +impl fmt::Display for ZoneRedundancy { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + ZoneRedundancy::Maximum => write!(f, "maximum"), + ZoneRedundancy::AtLeast(x) => write!(f, "{}", x), + } + } +} + +impl core::str::FromStr for ZoneRedundancy { + type Err = &'static str; + fn from_str(s: &str) -> Result { + match s { + "none" | "max" | "maximum" => Ok(ZoneRedundancy::Maximum), + x => { + let v = x + .parse::() + .map_err(|_| "zone redundancy must be 'none'/'max' or an integer")?; + Ok(ZoneRedundancy::AtLeast(v)) + } + } + } +} + +impl LayoutVersion { + pub fn new(replication_factor: usize) -> Self { + // We set the default zone redundancy to be Maximum, meaning that the maximum + // possible value will be used depending on the cluster topology + let parameters = LayoutParameters { + zone_redundancy: ZoneRedundancy::Maximum, + }; + + LayoutVersion { + version: 0, + replication_factor, + partition_size: 0, + roles: LwwMap::new(), + node_id_vec: Vec::new(), + ring_assignment_data: Vec::new(), + parameters, + } + } + + // ===================== accessors ====================== + + /// Returns a list of IDs of nodes that currently have + /// a role in the cluster + pub fn node_ids(&self) -> &[Uuid] { + &self.node_id_vec[..] + } + + pub fn num_nodes(&self) -> usize { + self.node_id_vec.len() + } + + /// Returns the role of a node in the layout + pub fn node_role(&self, node: &Uuid) -> Option<&NodeRole> { + match self.roles.get(node) { + Some(NodeRoleV(Some(v))) => Some(v), + _ => None, + } + } + + /// Given a node uuids, this function returns its capacity or fails if it does not have any + pub fn get_node_capacity(&self, uuid: &Uuid) -> Result { + match self.node_role(uuid) { + Some(NodeRole { + capacity: Some(cap), + zone: _, + tags: _, + }) => Ok(*cap), + _ => Err(Error::Message( + "The Uuid does not correspond to a node present in the \ + cluster or this node does not have a positive capacity." + .into(), + )), + } + } + + /// Returns the number of partitions associated to this node in the ring + pub fn get_node_usage(&self, uuid: &Uuid) -> Result { + for (i, id) in self.node_id_vec.iter().enumerate() { + if id == uuid { + let mut count = 0; + for nod in self.ring_assignment_data.iter() { + if i as u8 == *nod { + count += 1 + } + } + return Ok(count); + } + } + Err(Error::Message( + "The Uuid does not correspond to a node present in the \ + cluster or this node does not have a positive capacity." + .into(), + )) + } + + /// Get the partition in which data would fall on + pub fn partition_of(&self, position: &Hash) -> Partition { + let top = u16::from_be_bytes(position.as_slice()[0..2].try_into().unwrap()); + top >> (16 - PARTITION_BITS) + } + + /// Get the list of partitions and the first hash of a partition key that would fall in it + pub fn partitions(&self) -> Vec<(Partition, Hash)> { + (0..(1 << PARTITION_BITS)) + .map(|i| { + let top = (i as u16) << (16 - PARTITION_BITS); + let mut location = [0u8; 32]; + location[..2].copy_from_slice(&u16::to_be_bytes(top)[..]); + (i as u16, Hash::from(location)) + }) + .collect::>() + } + + /// Walk the ring to find the n servers in which data should be replicated + pub fn nodes_of(&self, position: &Hash, n: usize) -> Vec { + assert_eq!(n, self.replication_factor); + + let data = &self.ring_assignment_data; + + if data.len() != self.replication_factor * (1 << PARTITION_BITS) { + warn!("Ring not yet ready, read/writes will be lost!"); + return vec![]; + } + + let partition_idx = self.partition_of(position) as usize; + let partition_start = partition_idx * self.replication_factor; + let partition_end = (partition_idx + 1) * self.replication_factor; + let partition_nodes = &data[partition_start..partition_end]; + + partition_nodes + .iter() + .map(|i| self.node_id_vec[*i as usize]) + .collect::>() + } + + // ===================== internal information extractors ====================== + + /// Returns the uuids of the non_gateway nodes in self.node_id_vec. + pub(crate) fn nongateway_nodes(&self) -> Vec { + let mut result = Vec::::new(); + for uuid in self.node_id_vec.iter() { + match self.node_role(uuid) { + Some(role) if role.capacity.is_some() => result.push(*uuid), + _ => (), + } + } + result + } + + /// Given a node uuids, this function returns the label of its zone + fn get_node_zone(&self, uuid: &Uuid) -> Result<&str, Error> { + match self.node_role(uuid) { + Some(role) => Ok(&role.zone), + _ => Err(Error::Message( + "The Uuid does not correspond to a node present in the cluster.".into(), + )), + } + } + + /// Returns the sum of capacities of non gateway nodes in the cluster + fn get_total_capacity(&self) -> Result { + let mut total_capacity = 0; + for uuid in self.nongateway_nodes().iter() { + total_capacity += self.get_node_capacity(uuid)?; + } + Ok(total_capacity) + } + + /// Returns the effective value of the zone_redundancy parameter + fn effective_zone_redundancy(&self) -> usize { + match self.parameters.zone_redundancy { + ZoneRedundancy::AtLeast(v) => v, + ZoneRedundancy::Maximum => { + let n_zones = self + .roles + .items() + .iter() + .filter_map(|(_, _, role)| role.0.as_ref().map(|x| x.zone.as_str())) + .collect::>() + .len(); + std::cmp::min(n_zones, self.replication_factor) + } + } + } + + /// Check a cluster layout for internal consistency + /// (assignment, roles, parameters, partition size) + /// returns true if consistent, false if error + pub fn check(&self) -> Result<(), String> { + // Check that node_id_vec contains the correct list of nodes + let mut expected_nodes = self + .roles + .items() + .iter() + .filter(|(_, _, v)| v.0.is_some()) + .map(|(id, _, _)| *id) + .collect::>(); + expected_nodes.sort(); + let mut node_id_vec = self.node_id_vec.clone(); + node_id_vec.sort(); + if expected_nodes != node_id_vec { + return Err(format!("node_id_vec does not contain the correct set of nodes\nnode_id_vec: {:?}\nexpected: {:?}", node_id_vec, expected_nodes)); + } + + // Check that the assignment data has the correct length + let expected_assignment_data_len = (1 << PARTITION_BITS) * self.replication_factor; + if self.ring_assignment_data.len() != expected_assignment_data_len { + return Err(format!( + "ring_assignment_data has incorrect length {} instead of {}", + self.ring_assignment_data.len(), + expected_assignment_data_len + )); + } + + // Check that the assigned nodes are correct identifiers + // of nodes that are assigned a role + // and that role is not the role of a gateway nodes + for x in self.ring_assignment_data.iter() { + if *x as usize >= self.node_id_vec.len() { + return Err(format!( + "ring_assignment_data contains invalid node id {}", + *x + )); + } + let node = self.node_id_vec[*x as usize]; + match self.roles.get(&node) { + Some(NodeRoleV(Some(x))) if x.capacity.is_some() => (), + _ => return Err("ring_assignment_data contains id of a gateway node".into()), + } + } + + // Check that every partition is associated to distinct nodes + let zone_redundancy = self.effective_zone_redundancy(); + let rf = self.replication_factor; + for p in 0..(1 << PARTITION_BITS) { + let nodes_of_p = self.ring_assignment_data[rf * p..rf * (p + 1)].to_vec(); + if nodes_of_p.iter().unique().count() != rf { + return Err(format!("partition does not contain {} unique node ids", rf)); + } + // Check that every partition is spread over at least zone_redundancy zones. + let zones_of_p = nodes_of_p + .iter() + .map(|n| { + self.get_node_zone(&self.node_id_vec[*n as usize]) + .expect("Zone not found.") + }) + .collect::>(); + if zones_of_p.iter().unique().count() < zone_redundancy { + return Err(format!( + "nodes of partition are in less than {} distinct zones", + zone_redundancy + )); + } + } + + // Check that the nodes capacities is consistent with the stored partitions + let mut node_usage = vec![0; MAX_NODE_NUMBER]; + for n in self.ring_assignment_data.iter() { + node_usage[*n as usize] += 1; + } + for (n, usage) in node_usage.iter().enumerate() { + if *usage > 0 { + let uuid = self.node_id_vec[n]; + let partusage = usage * self.partition_size; + let nodecap = self.get_node_capacity(&uuid).unwrap(); + if partusage > nodecap { + return Err(format!( + "node usage ({}) is bigger than node capacity ({})", + usage * self.partition_size, + nodecap + )); + } + } + } + + // Check that the partition size stored is the one computed by the asignation + // algorithm. + let cl2 = self.clone(); + let (_, zone_to_id) = cl2.generate_nongateway_zone_ids().unwrap(); + match cl2.compute_optimal_partition_size(&zone_to_id, zone_redundancy) { + Ok(s) if s != self.partition_size => { + return Err(format!( + "partition_size ({}) is different than optimal value ({})", + self.partition_size, s + )) + } + Err(e) => return Err(format!("could not calculate optimal partition size: {}", e)), + _ => (), + } + + Ok(()) + } + + // ================== updates to layout, internals =================== + + /// This function calculates a new partition-to-node assignment. + /// The computed assignment respects the node replication factor + /// and the zone redundancy parameter It maximizes the capacity of a + /// partition (assuming all partitions have the same size). + /// Among such optimal assignment, it minimizes the distance to + /// the former assignment (if any) to minimize the amount of + /// data to be moved. + /// Staged role changes must be merged with nodes roles before calling this function, + /// hence it must only be called from apply_staged_changes() and hence is not public. + pub(crate) fn calculate_partition_assignment(&mut self) -> Result { + // We update the node ids, since the node role list might have changed with the + // changes in the layout. We retrieve the old_assignment reframed with new ids + let old_assignment_opt = self.update_node_id_vec()?; + + let zone_redundancy = self.effective_zone_redundancy(); + + let mut msg = Message::new(); + msg.push("==== COMPUTATION OF A NEW PARTITION ASSIGNATION ====".into()); + msg.push("".into()); + msg.push(format!( + "Partitions are \ + replicated {} times on at least {} distinct zones.", + self.replication_factor, zone_redundancy + )); + + // We generate for once numerical ids for the zones of non gateway nodes, + // to use them as indices in the flow graphs. + let (id_to_zone, zone_to_id) = self.generate_nongateway_zone_ids()?; + + let nb_nongateway_nodes = self.nongateway_nodes().len(); + if nb_nongateway_nodes < self.replication_factor { + return Err(Error::Message(format!( + "The number of nodes with positive \ + capacity ({}) is smaller than the replication factor ({}).", + nb_nongateway_nodes, self.replication_factor + ))); + } + if id_to_zone.len() < zone_redundancy { + return Err(Error::Message(format!( + "The number of zones with non-gateway \ + nodes ({}) is smaller than the redundancy parameter ({})", + id_to_zone.len(), + zone_redundancy + ))); + } + + // We compute the optimal partition size + // Capacities should be given in a unit so that partition size is at least 100. + // In this case, integer rounding plays a marginal role in the percentages of + // optimality. + let partition_size = self.compute_optimal_partition_size(&zone_to_id, zone_redundancy)?; + + msg.push("".into()); + if old_assignment_opt.is_some() { + msg.push(format!( + "Optimal partition size: {} ({} in previous layout)", + ByteSize::b(partition_size).to_string_as(false), + ByteSize::b(self.partition_size).to_string_as(false) + )); + } else { + msg.push(format!( + "Optimal partition size: {}", + ByteSize::b(partition_size).to_string_as(false) + )); + } + // We write the partition size. + self.partition_size = partition_size; + + if partition_size < 100 { + msg.push( + "WARNING: The partition size is low (< 100), make sure the capacities of your nodes are correct and are of at least a few MB" + .into(), + ); + } + + // We compute a first flow/assignment that is heuristically close to the previous + // assignment + let mut gflow = + self.compute_candidate_assignment(&zone_to_id, &old_assignment_opt, zone_redundancy)?; + if let Some(assoc) = &old_assignment_opt { + // We minimize the distance to the previous assignment. + self.minimize_rebalance_load(&mut gflow, &zone_to_id, assoc)?; + } + + // We display statistics of the computation + msg.extend(self.output_stat(&gflow, &old_assignment_opt, &zone_to_id, &id_to_zone)?); + + // We update the layout structure + self.update_ring_from_flow(id_to_zone.len(), &gflow)?; + + if let Err(e) = self.check() { + return Err(Error::Message( + format!("Layout check returned an error: {}\nOriginal result of computation: <<<<\n{}\n>>>>", e, msg.join("\n")) + )); + } + + Ok(msg) + } + + /// The LwwMap of node roles might have changed. This function updates the node_id_vec + /// and returns the assignment given by ring, with the new indices of the nodes, and + /// None if the node is not present anymore. + /// We work with the assumption that only this function and calculate_new_assignment + /// do modify assignment_ring and node_id_vec. + fn update_node_id_vec(&mut self) -> Result>>, Error> { + // (1) We compute the new node list + // Non gateway nodes should be coded on 8bits, hence they must be first in the list + // We build the new node ids + let new_non_gateway_nodes: Vec = self + .roles + .items() + .iter() + .filter(|(_, _, v)| matches!(&v.0, Some(r) if r.capacity.is_some())) + .map(|(k, _, _)| *k) + .collect(); + + if new_non_gateway_nodes.len() > MAX_NODE_NUMBER { + return Err(Error::Message(format!( + "There are more than {} non-gateway nodes in the new \ + layout. This is not allowed.", + MAX_NODE_NUMBER + ))); + } + + let new_gateway_nodes: Vec = self + .roles + .items() + .iter() + .filter(|(_, _, v)| matches!(v, NodeRoleV(Some(r)) if r.capacity.is_none())) + .map(|(k, _, _)| *k) + .collect(); + + let mut new_node_id_vec = Vec::::new(); + new_node_id_vec.extend(new_non_gateway_nodes); + new_node_id_vec.extend(new_gateway_nodes); + + let old_node_id_vec = self.node_id_vec.clone(); + self.node_id_vec = new_node_id_vec.clone(); + + // (2) We retrieve the old association + // We rewrite the old association with the new indices. We only consider partition + // to node assignments where the node is still in use. + if self.ring_assignment_data.is_empty() { + // This is a new association + return Ok(None); + } + + if self.ring_assignment_data.len() != NB_PARTITIONS * self.replication_factor { + return Err(Error::Message( + "The old assignment does not have a size corresponding to \ + the old replication factor or the number of partitions." + .into(), + )); + } + + // We build a translation table between the uuid and new ids + let mut uuid_to_new_id = HashMap::::new(); + + // We add the indices of only the new non-gateway nodes that can be used in the + // association ring + for (i, uuid) in new_node_id_vec.iter().enumerate() { + uuid_to_new_id.insert(*uuid, i); + } + + let mut old_assignment = vec![Vec::::new(); NB_PARTITIONS]; + let rf = self.replication_factor; + + for (p, old_assign_p) in old_assignment.iter_mut().enumerate() { + for old_id in &self.ring_assignment_data[p * rf..(p + 1) * rf] { + let uuid = old_node_id_vec[*old_id as usize]; + if uuid_to_new_id.contains_key(&uuid) { + old_assign_p.push(uuid_to_new_id[&uuid]); + } + } + } + + // We write the ring + self.ring_assignment_data = Vec::::new(); + + Ok(Some(old_assignment)) + } + + /// This function generates ids for the zone of the nodes appearing in + /// self.node_id_vec. + fn generate_nongateway_zone_ids(&self) -> Result<(Vec, HashMap), Error> { + let mut id_to_zone = Vec::::new(); + let mut zone_to_id = HashMap::::new(); + + for uuid in self.nongateway_nodes().iter() { + let r = self.node_role(uuid).unwrap(); + if !zone_to_id.contains_key(&r.zone) && r.capacity.is_some() { + zone_to_id.insert(r.zone.clone(), id_to_zone.len()); + id_to_zone.push(r.zone.clone()); + } + } + Ok((id_to_zone, zone_to_id)) + } + + /// This function computes by dichotomy the largest realizable partition size, given + /// the layout roles and parameters. + fn compute_optimal_partition_size( + &self, + zone_to_id: &HashMap, + zone_redundancy: usize, + ) -> Result { + let empty_set = HashSet::<(usize, usize)>::new(); + let mut g = self.generate_flow_graph(1, zone_to_id, &empty_set, zone_redundancy)?; + g.compute_maximal_flow()?; + if g.get_flow_value()? < (NB_PARTITIONS * self.replication_factor) as i64 { + return Err(Error::Message( + "The storage capacity of he cluster is to small. It is \ + impossible to store partitions of size 1." + .into(), + )); + } + + let mut s_down = 1; + let mut s_up = self.get_total_capacity()?; + while s_down + 1 < s_up { + g = self.generate_flow_graph( + (s_down + s_up) / 2, + zone_to_id, + &empty_set, + zone_redundancy, + )?; + g.compute_maximal_flow()?; + if g.get_flow_value()? < (NB_PARTITIONS * self.replication_factor) as i64 { + s_up = (s_down + s_up) / 2; + } else { + s_down = (s_down + s_up) / 2; + } + } + + Ok(s_down) + } + + fn generate_graph_vertices(nb_zones: usize, nb_nodes: usize) -> Vec { + let mut vertices = vec![Vertex::Source, Vertex::Sink]; + for p in 0..NB_PARTITIONS { + vertices.push(Vertex::Pup(p)); + vertices.push(Vertex::Pdown(p)); + for z in 0..nb_zones { + vertices.push(Vertex::PZ(p, z)); + } + } + for n in 0..nb_nodes { + vertices.push(Vertex::N(n)); + } + vertices + } + + /// Generates the graph to compute the maximal flow corresponding to the optimal + /// partition assignment. + /// exclude_assoc is the set of (partition, node) association that we are forbidden + /// to use (hence we do not add the corresponding edge to the graph). This parameter + /// is used to compute a first flow that uses only edges appearing in the previous + /// assignment. This produces a solution that heuristically should be close to the + /// previous one. + fn generate_flow_graph( + &self, + partition_size: u64, + zone_to_id: &HashMap, + exclude_assoc: &HashSet<(usize, usize)>, + zone_redundancy: usize, + ) -> Result, Error> { + let vertices = + LayoutVersion::generate_graph_vertices(zone_to_id.len(), self.nongateway_nodes().len()); + let mut g = Graph::::new(&vertices); + let nb_zones = zone_to_id.len(); + for p in 0..NB_PARTITIONS { + g.add_edge(Vertex::Source, Vertex::Pup(p), zone_redundancy as u64)?; + g.add_edge( + Vertex::Source, + Vertex::Pdown(p), + (self.replication_factor - zone_redundancy) as u64, + )?; + for z in 0..nb_zones { + g.add_edge(Vertex::Pup(p), Vertex::PZ(p, z), 1)?; + g.add_edge( + Vertex::Pdown(p), + Vertex::PZ(p, z), + self.replication_factor as u64, + )?; + } + } + for n in 0..self.nongateway_nodes().len() { + let node_capacity = self.get_node_capacity(&self.node_id_vec[n])?; + let node_zone = zone_to_id[self.get_node_zone(&self.node_id_vec[n])?]; + g.add_edge(Vertex::N(n), Vertex::Sink, node_capacity / partition_size)?; + for p in 0..NB_PARTITIONS { + if !exclude_assoc.contains(&(p, n)) { + g.add_edge(Vertex::PZ(p, node_zone), Vertex::N(n), 1)?; + } + } + } + Ok(g) + } + + /// This function computes a first optimal assignment (in the form of a flow graph). + fn compute_candidate_assignment( + &self, + zone_to_id: &HashMap, + prev_assign_opt: &Option>>, + zone_redundancy: usize, + ) -> Result, Error> { + // We list the (partition,node) associations that are not used in the + // previous assignment + let mut exclude_edge = HashSet::<(usize, usize)>::new(); + if let Some(prev_assign) = prev_assign_opt { + let nb_nodes = self.nongateway_nodes().len(); + for (p, prev_assign_p) in prev_assign.iter().enumerate() { + for n in 0..nb_nodes { + exclude_edge.insert((p, n)); + } + for n in prev_assign_p.iter() { + exclude_edge.remove(&(p, *n)); + } + } + } + + // We compute the best flow using only the edges used in the previous assignment + let mut g = self.generate_flow_graph( + self.partition_size, + zone_to_id, + &exclude_edge, + zone_redundancy, + )?; + g.compute_maximal_flow()?; + + // We add the excluded edges and compute the maximal flow with the full graph. + // The algorithm is such that it will start with the flow that we just computed + // and find ameliorating paths from that. + for (p, n) in exclude_edge.iter() { + let node_zone = zone_to_id[self.get_node_zone(&self.node_id_vec[*n])?]; + g.add_edge(Vertex::PZ(*p, node_zone), Vertex::N(*n), 1)?; + } + g.compute_maximal_flow()?; + Ok(g) + } + + /// This function updates the flow graph gflow to minimize the distance between + /// its corresponding assignment and the previous one + fn minimize_rebalance_load( + &self, + gflow: &mut Graph, + zone_to_id: &HashMap, + prev_assign: &[Vec], + ) -> Result<(), Error> { + // We define a cost function on the edges (pairs of vertices) corresponding + // to the distance between the two assignments. + let mut cost = CostFunction::new(); + for (p, assoc_p) in prev_assign.iter().enumerate() { + for n in assoc_p.iter() { + let node_zone = zone_to_id[self.get_node_zone(&self.node_id_vec[*n])?]; + cost.insert((Vertex::PZ(p, node_zone), Vertex::N(*n)), -1); + } + } + + // We compute the maximal length of a simple path in gflow. It is used in the + // Bellman-Ford algorithm in optimize_flow_with_cost to set the number + // of iterations. + let nb_nodes = self.nongateway_nodes().len(); + let path_length = 4 * nb_nodes; + gflow.optimize_flow_with_cost(&cost, path_length)?; + + Ok(()) + } + + /// This function updates the assignment ring from the flow graph. + fn update_ring_from_flow( + &mut self, + nb_zones: usize, + gflow: &Graph, + ) -> Result<(), Error> { + self.ring_assignment_data = Vec::::new(); + for p in 0..NB_PARTITIONS { + for z in 0..nb_zones { + let assoc_vertex = gflow.get_positive_flow_from(Vertex::PZ(p, z))?; + for vertex in assoc_vertex.iter() { + if let Vertex::N(n) = vertex { + self.ring_assignment_data.push((*n).try_into().unwrap()); + } + } + } + } + + if self.ring_assignment_data.len() != NB_PARTITIONS * self.replication_factor { + return Err(Error::Message( + "Critical Error : the association ring we produced does not \ + have the right size." + .into(), + )); + } + Ok(()) + } + + /// This function returns a message summing up the partition repartition of the new + /// layout, and other statistics of the partition assignment computation. + fn output_stat( + &self, + gflow: &Graph, + prev_assign_opt: &Option>>, + zone_to_id: &HashMap, + id_to_zone: &[String], + ) -> Result { + let mut msg = Message::new(); + + let used_cap = self.partition_size * NB_PARTITIONS as u64 * self.replication_factor as u64; + let total_cap = self.get_total_capacity()?; + let percent_cap = 100.0 * (used_cap as f32) / (total_cap as f32); + msg.push(format!( + "Usable capacity / total cluster capacity: {} / {} ({:.1} %)", + ByteSize::b(used_cap).to_string_as(false), + ByteSize::b(total_cap).to_string_as(false), + percent_cap + )); + msg.push(format!( + "Effective capacity (replication factor {}): {}", + self.replication_factor, + ByteSize::b(used_cap / self.replication_factor as u64).to_string_as(false) + )); + if percent_cap < 80. { + msg.push("".into()); + msg.push( + "If the percentage is too low, it might be that the \ + cluster topology and redundancy constraints are forcing the use of nodes/zones with small \ + storage capacities." + .into(), + ); + msg.push( + "You might want to move storage capacity between zones or relax the redundancy constraint." + .into(), + ); + msg.push( + "See the detailed statistics below and look for saturated nodes/zones.".into(), + ); + } + + // We define and fill in the following tables + let storing_nodes = self.nongateway_nodes(); + let mut new_partitions = vec![0; storing_nodes.len()]; + let mut stored_partitions = vec![0; storing_nodes.len()]; + + let mut new_partitions_zone = vec![0; id_to_zone.len()]; + let mut stored_partitions_zone = vec![0; id_to_zone.len()]; + + for p in 0..NB_PARTITIONS { + for z in 0..id_to_zone.len() { + let pz_nodes = gflow.get_positive_flow_from(Vertex::PZ(p, z))?; + if !pz_nodes.is_empty() { + stored_partitions_zone[z] += 1; + if let Some(prev_assign) = prev_assign_opt { + let mut old_zones_of_p = Vec::::new(); + for n in prev_assign[p].iter() { + old_zones_of_p + .push(zone_to_id[self.get_node_zone(&self.node_id_vec[*n])?]); + } + if !old_zones_of_p.contains(&z) { + new_partitions_zone[z] += 1; + } + } + } + for vert in pz_nodes.iter() { + if let Vertex::N(n) = *vert { + stored_partitions[n] += 1; + if let Some(prev_assign) = prev_assign_opt { + if !prev_assign[p].contains(&n) { + new_partitions[n] += 1; + } + } + } + } + } + } + + if prev_assign_opt.is_none() { + new_partitions = stored_partitions.clone(); + //new_partitions_zone = stored_partitions_zone.clone(); + } + + // We display the statistics + + msg.push("".into()); + if prev_assign_opt.is_some() { + let total_new_partitions: usize = new_partitions.iter().sum(); + msg.push(format!( + "A total of {} new copies of partitions need to be \ + transferred.", + total_new_partitions + )); + msg.push("".into()); + } + + let mut table = vec![]; + for z in 0..id_to_zone.len() { + let mut nodes_of_z = Vec::::new(); + for n in 0..storing_nodes.len() { + if self.get_node_zone(&self.node_id_vec[n])? == id_to_zone[z] { + nodes_of_z.push(n); + } + } + let replicated_partitions: usize = + nodes_of_z.iter().map(|n| stored_partitions[*n]).sum(); + table.push(format!( + "{}\tTags\tPartitions\tCapacity\tUsable capacity", + id_to_zone[z] + )); + + let available_cap_z: u64 = self.partition_size * replicated_partitions as u64; + let mut total_cap_z = 0; + for n in nodes_of_z.iter() { + total_cap_z += self.get_node_capacity(&self.node_id_vec[*n])?; + } + let percent_cap_z = 100.0 * (available_cap_z as f32) / (total_cap_z as f32); + + for n in nodes_of_z.iter() { + let available_cap_n = stored_partitions[*n] as u64 * self.partition_size; + let total_cap_n = self.get_node_capacity(&self.node_id_vec[*n])?; + let tags_n = (self.node_role(&self.node_id_vec[*n]).ok_or(""))?.tags_string(); + table.push(format!( + " {:?}\t{}\t{} ({} new)\t{}\t{} ({:.1}%)", + self.node_id_vec[*n], + tags_n, + stored_partitions[*n], + new_partitions[*n], + ByteSize::b(total_cap_n).to_string_as(false), + ByteSize::b(available_cap_n).to_string_as(false), + (available_cap_n as f32) / (total_cap_n as f32) * 100.0, + )); + } + + table.push(format!( + " TOTAL\t\t{} ({} unique)\t{}\t{} ({:.1}%)", + replicated_partitions, + stored_partitions_zone[z], + //new_partitions_zone[z], + ByteSize::b(total_cap_z).to_string_as(false), + ByteSize::b(available_cap_z).to_string_as(false), + percent_cap_z + )); + table.push("".into()); + } + msg.push(format_table::format_table_to_string(table)); + + Ok(msg) + } +} + +// ==================================================================================== + +#[cfg(test)] +mod tests { + use super::{Error, *}; + use std::cmp::min; + + // This function checks that the partition size S computed is at least better than the + // one given by a very naive algorithm. To do so, we try to run the naive algorithm + // assuming a partion size of S+1. If we succed, it means that the optimal assignment + // was not optimal. The naive algorithm is the following : + // - we compute the max number of partitions associated to every node, capped at the + // partition number. It gives the number of tokens of every node. + // - every zone has a number of tokens equal to the sum of the tokens of its nodes. + // - we cycle over the partitions and associate zone tokens while respecting the + // zone redundancy constraint. + // NOTE: the naive algorithm is not optimal. Counter example: + // take nb_partition = 3 ; replication_factor = 5; redundancy = 4; + // number of tokens by zone : (A, 4), (B,1), (C,4), (D, 4), (E, 2) + // With these parameters, the naive algo fails, whereas there is a solution: + // (A,A,C,D,E) , (A,B,C,D,D) (A,C,C,D,E) + fn check_against_naive(cl: &LayoutVersion) -> Result { + let over_size = cl.partition_size + 1; + let mut zone_token = HashMap::::new(); + + let (zones, zone_to_id) = cl.generate_nongateway_zone_ids()?; + + if zones.is_empty() { + return Ok(false); + } + + for z in zones.iter() { + zone_token.insert(z.clone(), 0); + } + for uuid in cl.nongateway_nodes().iter() { + let z = cl.get_node_zone(uuid)?; + let c = cl.get_node_capacity(uuid)?; + zone_token.insert( + z.clone(), + zone_token[&z] + min(NB_PARTITIONS, (c / over_size) as usize), + ); + } + + // For every partition, we count the number of zone already associated and + // the name of the last zone associated + + let mut id_zone_token = vec![0; zones.len()]; + for (z, t) in zone_token.iter() { + id_zone_token[zone_to_id[z]] = *t; + } + + let mut nb_token = vec![0; NB_PARTITIONS]; + let mut last_zone = vec![zones.len(); NB_PARTITIONS]; + + let mut curr_zone = 0; + + let redundancy = cl.effective_zone_redundancy(); + + for replic in 0..cl.replication_factor { + for p in 0..NB_PARTITIONS { + while id_zone_token[curr_zone] == 0 + || (last_zone[p] == curr_zone + && redundancy - nb_token[p] <= cl.replication_factor - replic) + { + curr_zone += 1; + if curr_zone >= zones.len() { + return Ok(true); + } + } + id_zone_token[curr_zone] -= 1; + if last_zone[p] != curr_zone { + nb_token[p] += 1; + last_zone[p] = curr_zone; + } + } + } + + return Ok(false); + } + + fn show_msg(msg: &Message) { + for s in msg.iter() { + println!("{}", s); + } + } + + fn update_layout( + cl: &mut LayoutVersion, + node_id_vec: &Vec, + node_capacity_vec: &Vec, + node_zone_vec: &Vec, + zone_redundancy: usize, + ) { + for i in 0..node_id_vec.len() { + if let Some(x) = FixedBytes32::try_from(&[i as u8; 32]) { + cl.node_id_vec.push(x); + } + + let update = cl.staging_roles.update_mutator( + cl.node_id_vec[i], + NodeRoleV(Some(NodeRole { + zone: (node_zone_vec[i].to_string()), + capacity: (Some(node_capacity_vec[i])), + tags: (vec![]), + })), + ); + cl.staging_roles.merge(&update); + } + cl.staging_parameters.update(LayoutParameters { + zone_redundancy: ZoneRedundancy::AtLeast(zone_redundancy), + }); + cl.staging_hash = cl.calculate_staging_hash(); + } + + #[test] + fn test_assignment() { + let mut node_id_vec = vec![1, 2, 3]; + let mut node_capacity_vec = vec![4000, 1000, 2000]; + let mut node_zone_vec = vec!["A", "B", "C"] + .into_iter() + .map(|x| x.to_string()) + .collect(); + + let mut cl = LayoutVersion::new(3); + update_layout(&mut cl, &node_id_vec, &node_capacity_vec, &node_zone_vec, 3); + let v = cl.version; + let (mut cl, msg) = cl.apply_staged_changes(Some(v + 1)).unwrap(); + show_msg(&msg); + assert_eq!(cl.check(), Ok(())); + assert!(matches!(check_against_naive(&cl), Ok(true))); + + node_id_vec = vec![1, 2, 3, 4, 5, 6, 7, 8, 9]; + node_capacity_vec = vec![4000, 1000, 1000, 3000, 1000, 1000, 2000, 10000, 2000]; + node_zone_vec = vec!["A", "B", "C", "C", "C", "B", "G", "H", "I"] + .into_iter() + .map(|x| x.to_string()) + .collect(); + update_layout(&mut cl, &node_id_vec, &node_capacity_vec, &node_zone_vec, 2); + let v = cl.version; + let (mut cl, msg) = cl.apply_staged_changes(Some(v + 1)).unwrap(); + show_msg(&msg); + assert_eq!(cl.check(), Ok(())); + assert!(matches!(check_against_naive(&cl), Ok(true))); + + node_capacity_vec = vec![4000, 1000, 2000, 7000, 1000, 1000, 2000, 10000, 2000]; + update_layout(&mut cl, &node_id_vec, &node_capacity_vec, &node_zone_vec, 3); + let v = cl.version; + let (mut cl, msg) = cl.apply_staged_changes(Some(v + 1)).unwrap(); + show_msg(&msg); + assert_eq!(cl.check(), Ok(())); + assert!(matches!(check_against_naive(&cl), Ok(true))); + + node_capacity_vec = vec![ + 4000000, 4000000, 2000000, 7000000, 1000000, 9000000, 2000000, 10000, 2000000, + ]; + update_layout(&mut cl, &node_id_vec, &node_capacity_vec, &node_zone_vec, 1); + let v = cl.version; + let (cl, msg) = cl.apply_staged_changes(Some(v + 1)).unwrap(); + show_msg(&msg); + assert_eq!(cl.check(), Ok(())); + assert!(matches!(check_against_naive(&cl), Ok(true))); + } +} diff --git a/src/rpc/rpc_helper.rs b/src/rpc/rpc_helper.rs index 56bef2f3..3fdb4acd 100644 --- a/src/rpc/rpc_helper.rs +++ b/src/rpc/rpc_helper.rs @@ -26,7 +26,7 @@ use garage_util::data::*; use garage_util::error::Error; use garage_util::metrics::RecordDuration; -use crate::layout::ClusterLayout; +use crate::layout::LayoutHistory; use crate::metrics::RpcMetrics; // Default RPC timeout = 5 minutes @@ -91,7 +91,7 @@ pub struct RpcHelper(Arc); struct RpcHelperInner { our_node_id: Uuid, fullmesh: Arc, - layout_watch: watch::Receiver>, + layout_watch: watch::Receiver>, metrics: RpcMetrics, rpc_timeout: Duration, } @@ -100,7 +100,7 @@ impl RpcHelper { pub(crate) fn new( our_node_id: Uuid, fullmesh: Arc, - layout_watch: watch::Receiver>, + layout_watch: watch::Receiver>, rpc_timeout: Option, ) -> Self { let metrics = RpcMetrics::new(); @@ -392,8 +392,8 @@ impl RpcHelper { pub fn request_order(&self, nodes: &[Uuid]) -> Vec { // Retrieve some status variables that we will use to sort requests let peer_list = self.0.fullmesh.get_peer_list(); - let layout: Arc = self.0.layout_watch.borrow().clone(); - let our_zone = match layout.node_role(&self.0.our_node_id) { + let layout: Arc = self.0.layout_watch.borrow().clone(); + let our_zone = match layout.current().node_role(&self.0.our_node_id) { Some(pc) => &pc.zone, None => "", }; @@ -407,7 +407,7 @@ impl RpcHelper { let mut nodes = nodes .iter() .map(|to| { - let peer_zone = match layout.node_role(to) { + let peer_zone = match layout.current().node_role(to) { Some(pc) => &pc.zone, None => "", }; diff --git a/src/rpc/system.rs b/src/rpc/system.rs index 93144e39..86d724f1 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -64,7 +64,7 @@ pub enum SystemRpc { /// Exchanged with every node on a regular basis. AdvertiseStatus(NodeStatus), /// Advertisement of cluster layout. Sent spontanously or in response to PullClusterLayout - AdvertiseClusterLayout(ClusterLayout), + AdvertiseClusterLayout(LayoutHistory), /// Get known nodes states GetKnownNodes, /// Return known nodes @@ -84,7 +84,7 @@ pub struct System { /// The id of this node pub id: Uuid, - persist_cluster_layout: Persister, + persist_cluster_layout: Persister, persist_peer_list: Persister, local_status: ArcSwap, @@ -112,8 +112,8 @@ pub struct System { replication_factor: usize, /// The layout - pub layout_watch: watch::Receiver>, - update_layout: Mutex>>, + pub layout_watch: watch::Receiver>, + update_layout: Mutex>>, /// Path to metadata directory pub metadata_dir: PathBuf, @@ -256,16 +256,16 @@ impl System { hex::encode(&node_key.public_key()[..8]) ); - let persist_cluster_layout: Persister = + let persist_cluster_layout: Persister = Persister::new(&config.metadata_dir, "cluster_layout"); let persist_peer_list = Persister::new(&config.metadata_dir, "peer_list"); let cluster_layout = match persist_cluster_layout.load() { Ok(x) => { - if x.replication_factor != replication_factor { + if x.current().replication_factor != replication_factor { return Err(Error::Message(format!( "Prevous cluster layout has replication factor {}, which is different than the one specified in the config file ({}). The previous cluster layout can be purged, if you know what you are doing, simply by deleting the `cluster_layout` file in your metadata directory.", - x.replication_factor, + x.current().replication_factor, replication_factor ))); } @@ -276,7 +276,7 @@ impl System { "No valid previous cluster layout stored ({}), starting fresh.", e ); - ClusterLayout::new(replication_factor) + LayoutHistory::new(replication_factor) } }; @@ -423,13 +423,13 @@ impl System { known_nodes } - pub fn cluster_layout(&self) -> watch::Ref> { + pub fn cluster_layout(&self) -> watch::Ref> { self.layout_watch.borrow() } pub async fn update_cluster_layout( self: &Arc, - layout: &ClusterLayout, + layout: &LayoutHistory, ) -> Result<(), Error> { self.handle_advertise_cluster_layout(layout).await?; Ok(()) @@ -475,7 +475,9 @@ impl System { .collect::>(); let connected_nodes = nodes.iter().filter(|(_, n)| n.is_up).count(); + // TODO: not only layout.current() let storage_nodes = layout + .current() .roles .items() .iter() @@ -486,11 +488,11 @@ impl System { .filter(|(x, _, _)| nodes.get(x).map(|n| n.is_up).unwrap_or(false)) .count(); - let partitions = layout.partitions(); + let partitions = layout.current().partitions(); let partitions_n_up = partitions .iter() .map(|(_, h)| { - let pn = layout.nodes_of(h, layout.replication_factor); + let pn = layout.current().nodes_of(h, replication_factor); pn.iter() .filter(|x| nodes.get(x).map(|n| n.is_up).unwrap_or(false)) .count() @@ -581,7 +583,7 @@ impl System { /// Save network configuration to disc async fn save_cluster_layout(&self) -> Result<(), Error> { - let layout: Arc = self.layout_watch.borrow().clone(); + let layout: Arc = self.layout_watch.borrow().clone(); self.persist_cluster_layout .save_async(&layout) .await @@ -593,7 +595,7 @@ impl System { let mut new_si: NodeStatus = self.local_status.load().as_ref().clone(); let layout = self.layout_watch.borrow(); - new_si.cluster_layout_version = layout.version; + new_si.cluster_layout_version = layout.current().version; new_si.cluster_layout_staging_hash = layout.staging_hash; new_si.update_disk_usage(&self.metadata_dir, &self.data_dir, &self.metrics); @@ -648,12 +650,12 @@ impl System { async fn handle_advertise_cluster_layout( self: &Arc, - adv: &ClusterLayout, + adv: &LayoutHistory, ) -> Result { - if adv.replication_factor != self.replication_factor { + if adv.current().replication_factor != self.replication_factor { let msg = format!( "Received a cluster layout from another node with replication factor {}, which is different from what we have in our configuration ({}). Discarding the cluster layout we received.", - adv.replication_factor, + adv.current().replication_factor, self.replication_factor ); error!("{}", msg); @@ -662,7 +664,7 @@ impl System { let update_layout = self.update_layout.lock().await; // TODO: don't clone each time an AdvertiseClusterLayout is received - let mut layout: ClusterLayout = self.layout_watch.borrow().as_ref().clone(); + let mut layout: LayoutHistory = self.layout_watch.borrow().as_ref().clone(); let prev_layout_check = layout.check().is_ok(); if layout.merge(adv) { @@ -724,7 +726,7 @@ impl System { while !*stop_signal.borrow() { let not_configured = self.layout_watch.borrow().check().is_err(); let no_peers = self.fullmesh.get_peer_list().len() < self.replication_factor; - let expected_n_nodes = self.layout_watch.borrow().num_nodes(); + let expected_n_nodes = self.layout_watch.borrow().current().num_nodes(); let bad_peers = self .fullmesh .get_peer_list() @@ -863,13 +865,13 @@ impl EndpointHandler for System { } impl NodeStatus { - fn initial(replication_factor: usize, layout: &ClusterLayout) -> Self { + fn initial(replication_factor: usize, layout: &LayoutHistory) -> Self { NodeStatus { hostname: gethostname::gethostname() .into_string() .unwrap_or_else(|_| "".to_string()), replication_factor, - cluster_layout_version: layout.version, + cluster_layout_version: layout.current().version, cluster_layout_staging_hash: layout.staging_hash, meta_disk_avail: None, data_disk_avail: None, -- cgit v1.2.3 From 8dccee3ccfe7793c42203f28c1e91c6f989b6899 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 8 Nov 2023 19:28:36 +0100 Subject: cluster layout: adapt all uses of ClusterLayout to LayoutHistory --- src/api/admin/cluster.rs | 9 +++++---- src/api/k2v/index.rs | 9 ++++++--- src/garage/admin/mod.rs | 14 +++++++------- src/garage/cli/cmd.rs | 17 +++++++++++------ src/garage/cli/layout.rs | 38 +++++++++++++++++++++----------------- src/model/helper/bucket.rs | 9 ++++++--- src/model/index_counter.rs | 6 +++--- src/rpc/layout/history.rs | 17 +++++------------ src/rpc/layout/schema.rs | 5 ++--- src/table/replication/fullcopy.rs | 5 ++--- src/table/replication/sharded.rs | 16 ++++++++++------ src/table/sync.rs | 4 ++-- 12 files changed, 80 insertions(+), 69 deletions(-) diff --git a/src/api/admin/cluster.rs b/src/api/admin/cluster.rs index 01ff9885..6dd2e8da 100644 --- a/src/api/admin/cluster.rs +++ b/src/api/admin/cluster.rs @@ -89,8 +89,9 @@ pub async fn handle_get_cluster_layout(garage: &Arc) -> Result GetClusterLayoutResponse { +fn format_cluster_layout(layout: &layout::LayoutHistory) -> GetClusterLayoutResponse { let roles = layout + .current() .roles .items() .iter() @@ -107,7 +108,7 @@ fn format_cluster_layout(layout: &layout::ClusterLayout) -> GetClusterLayoutResp .staging_roles .items() .iter() - .filter(|(k, _, v)| layout.roles.get(k) != Some(v)) + .filter(|(k, _, v)| layout.current().roles.get(k) != Some(v)) .map(|(k, _, v)| match &v.0 { None => NodeRoleChange { id: hex::encode(k), @@ -125,7 +126,7 @@ fn format_cluster_layout(layout: &layout::ClusterLayout) -> GetClusterLayoutResp .collect::>(); GetClusterLayoutResponse { - version: layout.version, + version: layout.current().version, roles, staged_role_changes, } @@ -209,7 +210,7 @@ pub async fn handle_update_cluster_layout( let mut layout = garage.system.cluster_layout().as_ref().clone(); - let mut roles = layout.roles.clone(); + let mut roles = layout.current().roles.clone(); roles.merge(&layout.staging_roles); for change in updates { diff --git a/src/api/k2v/index.rs b/src/api/k2v/index.rs index 3fc39de6..a9bc3826 100644 --- a/src/api/k2v/index.rs +++ b/src/api/k2v/index.rs @@ -5,7 +5,7 @@ use serde::Serialize; use garage_util::data::*; -use garage_rpc::layout::ClusterLayout; +use garage_rpc::layout::LayoutHistory; use garage_table::util::*; use garage_model::garage::Garage; @@ -26,7 +26,7 @@ pub async fn handle_read_index( ) -> Result, Error> { let reverse = reverse.unwrap_or(false); - let layout: Arc = garage.system.cluster_layout().clone(); + let layout: Arc = garage.system.cluster_layout().clone(); let (partition_keys, more, next_start) = read_range( &garage.k2v.counter_table.table, @@ -35,7 +35,10 @@ pub async fn handle_read_index( &start, &end, limit, - Some((DeletedFilter::NotDeleted, layout.node_id_vec.clone())), + Some(( + DeletedFilter::NotDeleted, + layout.current().node_id_vec.clone(), + )), EnumerationOrder::from_reverse(reverse), ) .await?; diff --git a/src/garage/admin/mod.rs b/src/garage/admin/mod.rs index c3fa801a..e3ba6d35 100644 --- a/src/garage/admin/mod.rs +++ b/src/garage/admin/mod.rs @@ -127,7 +127,7 @@ impl AdminRpcHandler { let mut failures = vec![]; let layout = self.garage.system.cluster_layout().clone(); - for node in layout.node_ids().iter() { + for node in layout.current().node_ids().iter() { let node = (*node).into(); let resp = self .endpoint @@ -165,7 +165,7 @@ impl AdminRpcHandler { let mut ret = String::new(); let layout = self.garage.system.cluster_layout().clone(); - for node in layout.node_ids().iter() { + for node in layout.current().node_ids().iter() { let mut opt = opt.clone(); opt.all_nodes = false; opt.skip_global = true; @@ -277,8 +277,8 @@ impl AdminRpcHandler { // Gather storage node and free space statistics let layout = &self.garage.system.cluster_layout(); let mut node_partition_count = HashMap::::new(); - for short_id in layout.ring_assignment_data.iter() { - let id = layout.node_id_vec[*short_id as usize]; + for short_id in layout.current().ring_assignment_data.iter() { + let id = layout.current().node_id_vec[*short_id as usize]; *node_partition_count.entry(id).or_default() += 1; } let node_info = self @@ -293,7 +293,7 @@ impl AdminRpcHandler { for (id, parts) in node_partition_count.iter() { let info = node_info.get(id); let status = info.map(|x| &x.status); - let role = layout.roles.get(id).and_then(|x| x.0.as_ref()); + let role = layout.current().roles.get(id).and_then(|x| x.0.as_ref()); let hostname = status.map(|x| x.hostname.as_str()).unwrap_or("?"); let zone = role.map(|x| x.zone.as_str()).unwrap_or("?"); let capacity = role @@ -441,7 +441,7 @@ impl AdminRpcHandler { if all_nodes { let mut ret = vec![]; let layout = self.garage.system.cluster_layout().clone(); - for node in layout.node_ids().iter() { + for node in layout.current().node_ids().iter() { let node = (*node).into(); match self .endpoint @@ -489,7 +489,7 @@ impl AdminRpcHandler { if all_nodes { let mut ret = vec![]; let layout = self.garage.system.cluster_layout().clone(); - for node in layout.node_ids().iter() { + for node in layout.current().node_ids().iter() { let node = (*node).into(); match self .endpoint diff --git a/src/garage/cli/cmd.rs b/src/garage/cli/cmd.rs index 48359614..8be43873 100644 --- a/src/garage/cli/cmd.rs +++ b/src/garage/cli/cmd.rs @@ -62,7 +62,7 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> let mut healthy_nodes = vec!["ID\tHostname\tAddress\tTags\tZone\tCapacity\tDataAvail".to_string()]; for adv in status.iter().filter(|adv| adv.is_up) { - match layout.roles.get(&adv.id) { + match layout.current().roles.get(&adv.id) { Some(NodeRoleV(Some(cfg))) => { let data_avail = match &adv.status.data_disk_avail { _ if cfg.capacity.is_none() => "N/A".into(), @@ -102,10 +102,15 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> format_table(healthy_nodes); let status_keys = status.iter().map(|adv| adv.id).collect::>(); - let failure_case_1 = status - .iter() - .any(|adv| !adv.is_up && matches!(layout.roles.get(&adv.id), Some(NodeRoleV(Some(_))))); + let failure_case_1 = status.iter().any(|adv| { + !adv.is_up + && matches!( + layout.current().roles.get(&adv.id), + Some(NodeRoleV(Some(_))) + ) + }); let failure_case_2 = layout + .current() .roles .items() .iter() @@ -115,7 +120,7 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> let mut failed_nodes = vec!["ID\tHostname\tAddress\tTags\tZone\tCapacity\tLast seen".to_string()]; for adv in status.iter().filter(|adv| !adv.is_up) { - if let Some(NodeRoleV(Some(cfg))) = layout.roles.get(&adv.id) { + if let Some(NodeRoleV(Some(cfg))) = layout.current().roles.get(&adv.id) { let tf = timeago::Formatter::new(); failed_nodes.push(format!( "{id:?}\t{host}\t{addr}\t[{tags}]\t{zone}\t{capacity}\t{last_seen}", @@ -132,7 +137,7 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> )); } } - for (id, _, role_v) in layout.roles.items().iter() { + for (id, _, role_v) in layout.current().roles.items().iter() { if let NodeRoleV(Some(cfg)) = role_v { if !status_keys.contains(id) { failed_nodes.push(format!( diff --git a/src/garage/cli/layout.rs b/src/garage/cli/layout.rs index ce2b11e0..4a617337 100644 --- a/src/garage/cli/layout.rs +++ b/src/garage/cli/layout.rs @@ -58,17 +58,18 @@ pub async fn cmd_assign_role( status .iter() .map(|adv| adv.id) - .chain(layout.node_ids().iter().cloned()), + .chain(layout.current().node_ids().iter().cloned()), node_id, ) }) .collect::, _>>()?; - let mut roles = layout.roles.clone(); + let mut roles = layout.current().roles.clone(); roles.merge(&layout.staging_roles); for replaced in args.replace.iter() { - let replaced_node = find_matching_node(layout.node_ids().iter().cloned(), replaced)?; + let replaced_node = + find_matching_node(layout.current().node_ids().iter().cloned(), replaced)?; match roles.get(&replaced_node) { Some(NodeRoleV(Some(_))) => { layout @@ -149,7 +150,7 @@ pub async fn cmd_remove_role( ) -> Result<(), Error> { let mut layout = fetch_layout(rpc_cli, rpc_host).await?; - let mut roles = layout.roles.clone(); + let mut roles = layout.current().roles.clone(); roles.merge(&layout.staging_roles); let deleted_node = @@ -174,13 +175,16 @@ pub async fn cmd_show_layout( let layout = fetch_layout(rpc_cli, rpc_host).await?; println!("==== CURRENT CLUSTER LAYOUT ===="); - print_cluster_layout(&layout, "No nodes currently have a role in the cluster.\nSee `garage status` to view available nodes."); + print_cluster_layout(layout.current(), "No nodes currently have a role in the cluster.\nSee `garage status` to view available nodes."); println!(); - println!("Current cluster layout version: {}", layout.version); + println!( + "Current cluster layout version: {}", + layout.current().version + ); let has_role_changes = print_staging_role_changes(&layout); if has_role_changes { - let v = layout.version; + let v = layout.current().version; let res_apply = layout.apply_staged_changes(Some(v + 1)); // this will print the stats of what partitions @@ -189,7 +193,7 @@ pub async fn cmd_show_layout( Ok((layout, msg)) => { println!(); println!("==== NEW CLUSTER LAYOUT AFTER APPLYING CHANGES ===="); - print_cluster_layout(&layout, "No nodes have a role in the new layout."); + print_cluster_layout(layout.current(), "No nodes have a role in the new layout."); println!(); for line in msg.iter() { @@ -266,11 +270,11 @@ pub async fn cmd_config_layout( .parse::() .ok_or_message("invalid zone redundancy value")?; if let ZoneRedundancy::AtLeast(r_int) = r { - if r_int > layout.replication_factor { + if r_int > layout.current().replication_factor { return Err(Error::Message(format!( "The zone redundancy must be smaller or equal to the \ replication factor ({}).", - layout.replication_factor + layout.current().replication_factor ))); } else if r_int < 1 { return Err(Error::Message( @@ -302,7 +306,7 @@ pub async fn cmd_config_layout( pub async fn fetch_layout( rpc_cli: &Endpoint, rpc_host: NodeID, -) -> Result { +) -> Result { match rpc_cli .call(&rpc_host, SystemRpc::PullClusterLayout, PRIO_NORMAL) .await?? @@ -315,7 +319,7 @@ pub async fn fetch_layout( pub async fn send_layout( rpc_cli: &Endpoint, rpc_host: NodeID, - layout: ClusterLayout, + layout: LayoutHistory, ) -> Result<(), Error> { rpc_cli .call( @@ -327,7 +331,7 @@ pub async fn send_layout( Ok(()) } -pub fn print_cluster_layout(layout: &ClusterLayout, empty_msg: &str) { +pub fn print_cluster_layout(layout: &LayoutVersion, empty_msg: &str) { let mut table = vec!["ID\tTags\tZone\tCapacity\tUsable capacity".to_string()]; for (id, _, role) in layout.roles.items().iter() { let role = match &role.0 { @@ -366,13 +370,13 @@ pub fn print_cluster_layout(layout: &ClusterLayout, empty_msg: &str) { } } -pub fn print_staging_role_changes(layout: &ClusterLayout) -> bool { +pub fn print_staging_role_changes(layout: &LayoutHistory) -> bool { let has_role_changes = layout .staging_roles .items() .iter() - .any(|(k, _, v)| layout.roles.get(k) != Some(v)); - let has_layout_changes = *layout.staging_parameters.get() != layout.parameters; + .any(|(k, _, v)| layout.current().roles.get(k) != Some(v)); + let has_layout_changes = *layout.staging_parameters.get() != layout.current().parameters; if has_role_changes || has_layout_changes { println!(); @@ -380,7 +384,7 @@ pub fn print_staging_role_changes(layout: &ClusterLayout) -> bool { if has_role_changes { let mut table = vec!["ID\tTags\tZone\tCapacity".to_string()]; for (id, _, role) in layout.staging_roles.items().iter() { - if layout.roles.get(id) == Some(role) { + if layout.current().roles.get(id) == Some(role) { continue; } if let Some(role) = &role.0 { diff --git a/src/model/helper/bucket.rs b/src/model/helper/bucket.rs index 8cd5b27b..18904c8d 100644 --- a/src/model/helper/bucket.rs +++ b/src/model/helper/bucket.rs @@ -450,10 +450,10 @@ impl<'a> BucketHelper<'a> { #[cfg(feature = "k2v")] { - use garage_rpc::layout::ClusterLayout; + use garage_rpc::layout::LayoutHistory; use std::sync::Arc; - let layout: Arc = self.0.system.cluster_layout().clone(); + let layout: Arc = self.0.system.cluster_layout().clone(); let k2vindexes = self .0 .k2v @@ -462,7 +462,10 @@ impl<'a> BucketHelper<'a> { .get_range( &bucket_id, None, - Some((DeletedFilter::NotDeleted, layout.node_id_vec.clone())), + Some(( + DeletedFilter::NotDeleted, + layout.current().node_id_vec.clone(), + )), 10, EnumerationOrder::Forward, ) diff --git a/src/model/index_counter.rs b/src/model/index_counter.rs index d514cb06..9637cc4c 100644 --- a/src/model/index_counter.rs +++ b/src/model/index_counter.rs @@ -7,7 +7,7 @@ use serde::{Deserialize, Serialize}; use garage_db as db; -use garage_rpc::layout::ClusterLayout; +use garage_rpc::layout::LayoutHistory; use garage_rpc::system::System; use garage_util::background::BackgroundRunner; use garage_util::data::*; @@ -83,8 +83,8 @@ impl Entry for CounterEntry { } impl CounterEntry { - pub fn filtered_values(&self, layout: &ClusterLayout) -> HashMap { - let nodes = &layout.node_id_vec[..]; + pub fn filtered_values(&self, layout: &LayoutHistory) -> HashMap { + let nodes = &layout.current().node_id_vec[..]; self.filtered_values_with_nodes(nodes) } diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index b3019f58..e59c9e9c 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -1,5 +1,4 @@ use std::cmp::Ordering; -use std::sync::Arc; use garage_util::crdt::{Crdt, Lww, LwwMap}; use garage_util::data::*; @@ -64,24 +63,22 @@ impl LayoutHistory { } // Add any new versions to history - let mut versions = self.versions.to_vec(); for v2 in other.versions.iter() { - if let Some(v1) = versions.iter().find(|v| v.version == v2.version) { + if let Some(v1) = self.versions.iter().find(|v| v.version == v2.version) { if v1 != v2 { error!("Inconsistent layout histories: different layout compositions for version {}. Your cluster will be broken as long as this layout version is not replaced.", v2.version); } - } else if versions.iter().all(|v| v.version != v2.version - 1) { + } else if self.versions.iter().all(|v| v.version != v2.version - 1) { error!( "Cannot receive new layout version {}, version {} is missing", v2.version, v2.version - 1 ); } else { - versions.push(v2.clone()); + self.versions.push(v2.clone()); changed = true; } } - self.versions = Arc::from(versions.into_boxed_slice()); // Merge trackers self.update_trackers.merge(&other.update_trackers); @@ -117,9 +114,7 @@ To know the correct value of the new layout version, invoke `garage layout show` let msg = new_version.calculate_partition_assignment()?; - let mut versions = self.versions.to_vec(); - versions.push(new_version); - self.versions = Arc::from(versions.into_boxed_slice()); + self.versions.push(new_version); Ok((self, msg)) } @@ -149,9 +144,7 @@ To know the correct value of the new layout version, invoke `garage layout show` let mut new_version = self.current().clone(); new_version.version += 1; - let mut versions = self.versions.to_vec(); - versions.push(new_version); - self.versions = Arc::from(versions.into_boxed_slice()); + self.versions.push(new_version); Ok(self) } diff --git a/src/rpc/layout/schema.rs b/src/rpc/layout/schema.rs index fa0822fa..14e797be 100644 --- a/src/rpc/layout/schema.rs +++ b/src/rpc/layout/schema.rs @@ -184,7 +184,6 @@ mod v010 { use garage_util::data::{Hash, Uuid}; use serde::{Deserialize, Serialize}; use std::collections::HashMap; - use std::sync::Arc; pub use v09::{LayoutParameters, NodeRole, NodeRoleV, ZoneRedundancy}; /// The layout of the cluster, i.e. the list of roles @@ -215,7 +214,7 @@ mod v010 { #[derive(Clone, Debug, Serialize, Deserialize)] pub struct LayoutHistory { /// The versions currently in use in the cluster - pub versions: Arc<[LayoutVersion]>, + pub versions: Vec, /// Update trackers pub update_trackers: UpdateTrackers, @@ -267,7 +266,7 @@ mod v010 { .collect::>(), ); let mut ret = Self { - versions: Arc::from(vec![version].into_boxed_slice()), + versions: vec![version], update_trackers: UpdateTrackers { ack_map: update_tracker.clone(), sync_map: update_tracker.clone(), diff --git a/src/table/replication/fullcopy.rs b/src/table/replication/fullcopy.rs index 34807e3d..a5c83d0f 100644 --- a/src/table/replication/fullcopy.rs +++ b/src/table/replication/fullcopy.rs @@ -27,11 +27,10 @@ impl TableReplication for TableFullReplication { } fn write_nodes(&self, _hash: &Hash) -> Vec { - let layout = self.system.cluster_layout(); - layout.node_ids().to_vec() + self.system.cluster_layout().current().node_ids().to_vec() } fn write_quorum(&self) -> usize { - let nmembers = self.system.cluster_layout().node_ids().len(); + let nmembers = self.system.cluster_layout().current().node_ids().len(); if nmembers > self.max_faults { nmembers - self.max_faults } else { diff --git a/src/table/replication/sharded.rs b/src/table/replication/sharded.rs index 60c95cb4..793d87fd 100644 --- a/src/table/replication/sharded.rs +++ b/src/table/replication/sharded.rs @@ -26,16 +26,20 @@ pub struct TableShardedReplication { impl TableReplication for TableShardedReplication { fn read_nodes(&self, hash: &Hash) -> Vec { - let layout = self.system.cluster_layout(); - layout.nodes_of(hash, self.replication_factor) + self.system + .cluster_layout() + .current() + .nodes_of(hash, self.replication_factor) } fn read_quorum(&self) -> usize { self.read_quorum } fn write_nodes(&self, hash: &Hash) -> Vec { - let layout = self.system.cluster_layout(); - layout.nodes_of(hash, self.replication_factor) + self.system + .cluster_layout() + .current() + .nodes_of(hash, self.replication_factor) } fn write_quorum(&self) -> usize { self.write_quorum @@ -45,9 +49,9 @@ impl TableReplication for TableShardedReplication { } fn partition_of(&self, hash: &Hash) -> Partition { - self.system.cluster_layout().partition_of(hash) + self.system.cluster_layout().current().partition_of(hash) } fn partitions(&self) -> Vec<(Partition, Hash)> { - self.system.cluster_layout().partitions() + self.system.cluster_layout().current().partitions() } } diff --git a/src/table/sync.rs b/src/table/sync.rs index 65eff7cd..620d83b9 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -492,8 +492,8 @@ impl EndpointHandler for TableSync struct SyncWorker { syncer: Arc>, - layout_watch: watch::Receiver>, - layout: Arc, + layout_watch: watch::Receiver>, + layout: Arc, add_full_sync_rx: mpsc::UnboundedReceiver<()>, todo: Vec, next_full_sync: Instant, -- cgit v1.2.3 From 1da0a5676edcd20fc5c7412596edb5772da9f606 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 8 Nov 2023 19:30:58 +0100 Subject: bump garage protocol version tag to 0x000A (0.10) --- src/rpc/system.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/rpc/system.rs b/src/rpc/system.rs index 86d724f1..a7433b68 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -46,7 +46,7 @@ const STATUS_EXCHANGE_INTERVAL: Duration = Duration::from_secs(10); /// Version tag used for version check upon Netapp connection. /// Cluster nodes with different version tags are deemed /// incompatible and will refuse to connect. -pub const GARAGE_VERSION_TAG: u64 = 0x6761726167650008; // garage 0x0008 +pub const GARAGE_VERSION_TAG: u64 = 0x676172616765000A; // garage 0x000A /// RPC endpoint used for calls related to membership pub const SYSTEM_RPC_PATH: &str = "garage_rpc/membership.rs/SystemRpc"; -- cgit v1.2.3 From 523d2ecb9511f74e144cd116b942d6c1bf0f546d Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 9 Nov 2023 11:19:43 +0100 Subject: layout: use separate CRDT for staged layout changes --- src/api/admin/api_server.rs | 2 +- src/api/admin/cluster.rs | 23 ++- src/garage/cli/cmd.rs | 2 +- src/garage/cli/layout.rs | 47 +++-- src/garage/cli/structs.rs | 6 +- src/rpc/graph_algo.rs | 415 ------------------------------------------- src/rpc/layout/graph_algo.rs | 405 +++++++++++++++++++++++++++++++++++++++++ src/rpc/layout/history.rs | 82 +++------ src/rpc/layout/mod.rs | 4 +- src/rpc/layout/schema.rs | 106 ++++++++++- src/rpc/layout/tracker.rs | 21 --- src/rpc/layout/version.rs | 54 +----- src/rpc/lib.rs | 1 - 13 files changed, 580 insertions(+), 588 deletions(-) delete mode 100644 src/rpc/graph_algo.rs create mode 100644 src/rpc/layout/graph_algo.rs delete mode 100644 src/rpc/layout/tracker.rs diff --git a/src/api/admin/api_server.rs b/src/api/admin/api_server.rs index 4779f924..d9bd600e 100644 --- a/src/api/admin/api_server.rs +++ b/src/api/admin/api_server.rs @@ -279,7 +279,7 @@ impl ApiHandler for AdminApiServer { Endpoint::GetClusterLayout => handle_get_cluster_layout(&self.garage).await, Endpoint::UpdateClusterLayout => handle_update_cluster_layout(&self.garage, req).await, Endpoint::ApplyClusterLayout => handle_apply_cluster_layout(&self.garage, req).await, - Endpoint::RevertClusterLayout => handle_revert_cluster_layout(&self.garage, req).await, + Endpoint::RevertClusterLayout => handle_revert_cluster_layout(&self.garage).await, // Keys Endpoint::ListKeys => handle_list_keys(&self.garage).await, Endpoint::GetKeyInfo { diff --git a/src/api/admin/cluster.rs b/src/api/admin/cluster.rs index 6dd2e8da..fe8e8764 100644 --- a/src/api/admin/cluster.rs +++ b/src/api/admin/cluster.rs @@ -105,7 +105,9 @@ fn format_cluster_layout(layout: &layout::LayoutHistory) -> GetClusterLayoutResp .collect::>(); let staged_role_changes = layout - .staging_roles + .staging + .get() + .roles .items() .iter() .filter(|(k, _, v)| layout.current().roles.get(k) != Some(v)) @@ -211,7 +213,7 @@ pub async fn handle_update_cluster_layout( let mut layout = garage.system.cluster_layout().as_ref().clone(); let mut roles = layout.current().roles.clone(); - roles.merge(&layout.staging_roles); + roles.merge(&layout.staging.get().roles); for change in updates { let node = hex::decode(&change.id).ok_or_bad_request("Invalid node identifier")?; @@ -232,7 +234,9 @@ pub async fn handle_update_cluster_layout( }; layout - .staging_roles + .staging + .get_mut() + .roles .merge(&roles.update_mutator(node, layout::NodeRoleV(new_role))); } @@ -246,7 +250,7 @@ pub async fn handle_apply_cluster_layout( garage: &Arc, req: Request, ) -> Result, Error> { - let param = parse_json_body::(req).await?; + let param = parse_json_body::(req).await?; let layout = garage.system.cluster_layout().as_ref().clone(); let (layout, msg) = layout.apply_staged_changes(Some(param.version))?; @@ -260,14 +264,9 @@ pub async fn handle_apply_cluster_layout( Ok(json_ok_response(&res)?) } -pub async fn handle_revert_cluster_layout( - garage: &Arc, - req: Request, -) -> Result, Error> { - let param = parse_json_body::(req).await?; - +pub async fn handle_revert_cluster_layout(garage: &Arc) -> Result, Error> { let layout = garage.system.cluster_layout().as_ref().clone(); - let layout = layout.revert_staged_changes(Some(param.version))?; + let layout = layout.revert_staged_changes()?; garage.system.update_cluster_layout(&layout).await?; let res = format_cluster_layout(&layout); @@ -280,7 +279,7 @@ type UpdateClusterLayoutRequest = Vec; #[derive(Deserialize)] #[serde(rename_all = "camelCase")] -struct ApplyRevertLayoutRequest { +struct ApplyLayoutRequest { version: u64, } diff --git a/src/garage/cli/cmd.rs b/src/garage/cli/cmd.rs index 8be43873..1a054025 100644 --- a/src/garage/cli/cmd.rs +++ b/src/garage/cli/cmd.rs @@ -85,7 +85,7 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> )); } _ => { - let new_role = match layout.staging_roles.get(&adv.id) { + let new_role = match layout.staging.get().roles.get(&adv.id) { Some(NodeRoleV(Some(_))) => "(pending)", _ => "NO ROLE ASSIGNED", }; diff --git a/src/garage/cli/layout.rs b/src/garage/cli/layout.rs index 4a617337..269d92f4 100644 --- a/src/garage/cli/layout.rs +++ b/src/garage/cli/layout.rs @@ -65,7 +65,7 @@ pub async fn cmd_assign_role( .collect::, _>>()?; let mut roles = layout.current().roles.clone(); - roles.merge(&layout.staging_roles); + roles.merge(&layout.staging.get().roles); for replaced in args.replace.iter() { let replaced_node = @@ -73,7 +73,9 @@ pub async fn cmd_assign_role( match roles.get(&replaced_node) { Some(NodeRoleV(Some(_))) => { layout - .staging_roles + .staging + .get_mut() + .roles .merge(&roles.update_mutator(replaced_node, NodeRoleV(None))); } _ => { @@ -131,7 +133,9 @@ pub async fn cmd_assign_role( }; layout - .staging_roles + .staging + .get_mut() + .roles .merge(&roles.update_mutator(added_node, NodeRoleV(Some(new_entry)))); } @@ -151,13 +155,15 @@ pub async fn cmd_remove_role( let mut layout = fetch_layout(rpc_cli, rpc_host).await?; let mut roles = layout.current().roles.clone(); - roles.merge(&layout.staging_roles); + roles.merge(&layout.staging.get().roles); let deleted_node = find_matching_node(roles.items().iter().map(|(id, _, _)| *id), &args.node_id)?; layout - .staging_roles + .staging + .get_mut() + .roles .merge(&roles.update_mutator(deleted_node, NodeRoleV(None))); send_layout(rpc_cli, rpc_host, layout).await?; @@ -203,16 +209,12 @@ pub async fn cmd_show_layout( println!(); println!(" garage layout apply --version {}", v + 1); println!(); - println!( - "You can also revert all proposed changes with: garage layout revert --version {}", - v + 1) + println!("You can also revert all proposed changes with: garage layout revert"); } Err(e) => { println!("Error while trying to compute the assignment: {}", e); println!("This new layout cannot yet be applied."); - println!( - "You can also revert all proposed changes with: garage layout revert --version {}", - v + 1) + println!("You can also revert all proposed changes with: garage layout revert"); } } } @@ -245,9 +247,15 @@ pub async fn cmd_revert_layout( rpc_host: NodeID, revert_opt: RevertLayoutOpt, ) -> Result<(), Error> { + if !revert_opt.yes { + return Err(Error::Message( + "Please add the --yes flag to run the layout revert operation".into(), + )); + } + let layout = fetch_layout(rpc_cli, rpc_host).await?; - let layout = layout.revert_staged_changes(revert_opt.version)?; + let layout = layout.revert_staged_changes()?; send_layout(rpc_cli, rpc_host, layout).await?; @@ -284,7 +292,9 @@ pub async fn cmd_config_layout( } layout - .staging_parameters + .staging + .get_mut() + .parameters .update(LayoutParameters { zone_redundancy: r }); println!("The zone redundancy parameter has been set to '{}'.", r); did_something = true; @@ -371,19 +381,20 @@ pub fn print_cluster_layout(layout: &LayoutVersion, empty_msg: &str) { } pub fn print_staging_role_changes(layout: &LayoutHistory) -> bool { - let has_role_changes = layout - .staging_roles + let staging = layout.staging.get(); + let has_role_changes = staging + .roles .items() .iter() .any(|(k, _, v)| layout.current().roles.get(k) != Some(v)); - let has_layout_changes = *layout.staging_parameters.get() != layout.current().parameters; + let has_layout_changes = *staging.parameters.get() != layout.current().parameters; if has_role_changes || has_layout_changes { println!(); println!("==== STAGED ROLE CHANGES ===="); if has_role_changes { let mut table = vec!["ID\tTags\tZone\tCapacity".to_string()]; - for (id, _, role) in layout.staging_roles.items().iter() { + for (id, _, role) in staging.roles.items().iter() { if layout.current().roles.get(id) == Some(role) { continue; } @@ -406,7 +417,7 @@ pub fn print_staging_role_changes(layout: &LayoutHistory) -> bool { if has_layout_changes { println!( "Zone redundancy: {}", - layout.staging_parameters.get().zone_redundancy + staging.parameters.get().zone_redundancy ); } true diff --git a/src/garage/cli/structs.rs b/src/garage/cli/structs.rs index aba57551..3badc447 100644 --- a/src/garage/cli/structs.rs +++ b/src/garage/cli/structs.rs @@ -164,9 +164,9 @@ pub struct ApplyLayoutOpt { #[derive(StructOpt, Debug)] pub struct RevertLayoutOpt { - /// Version number of old configuration to which to revert - #[structopt(long = "version")] - pub(crate) version: Option, + /// The revert operation will not be ran unless this flag is added + #[structopt(long = "yes")] + pub(crate) yes: bool, } #[derive(Serialize, Deserialize, StructOpt, Debug)] diff --git a/src/rpc/graph_algo.rs b/src/rpc/graph_algo.rs deleted file mode 100644 index d8c6c9b9..00000000 --- a/src/rpc/graph_algo.rs +++ /dev/null @@ -1,415 +0,0 @@ -//! This module deals with graph algorithms. -//! It is used in layout.rs to build the partition to node assignment. - -use rand::prelude::{SeedableRng, SliceRandom}; -use std::cmp::{max, min}; -use std::collections::HashMap; -use std::collections::VecDeque; - -/// Vertex data structures used in all the graphs used in layout.rs. -/// usize parameters correspond to node/zone/partitions ids. -/// To understand the vertex roles below, please refer to the formal description -/// of the layout computation algorithm. -#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash)] -pub enum Vertex { - Source, - Pup(usize), // The vertex p+ of partition p - Pdown(usize), // The vertex p- of partition p - PZ(usize, usize), // The vertex corresponding to x_(partition p, zone z) - N(usize), // The vertex corresponding to node n - Sink, -} - -/// Edge data structure for the flow algorithm. -#[derive(Clone, Copy, Debug)] -pub struct FlowEdge { - cap: u64, // flow maximal capacity of the edge - flow: i64, // flow value on the edge - dest: usize, // destination vertex id - rev: usize, // index of the reversed edge (v, self) in the edge list of vertex v -} - -/// Edge data structure for the detection of negative cycles. -#[derive(Clone, Copy, Debug)] -pub struct WeightedEdge { - w: i64, // weight of the edge - dest: usize, -} - -pub trait Edge: Clone + Copy {} -impl Edge for FlowEdge {} -impl Edge for WeightedEdge {} - -/// Struct for the graph structure. We do encapsulation here to be able to both -/// provide user friendly Vertex enum to address vertices, and to use internally usize -/// indices and Vec instead of HashMap in the graph algorithm to optimize execution speed. -pub struct Graph { - vertex_to_id: HashMap, - id_to_vertex: Vec, - - // The graph is stored as an adjacency list - graph: Vec>, -} - -pub type CostFunction = HashMap<(Vertex, Vertex), i64>; - -impl Graph { - pub fn new(vertices: &[Vertex]) -> Self { - let mut map = HashMap::::new(); - for (i, vert) in vertices.iter().enumerate() { - map.insert(*vert, i); - } - Graph:: { - vertex_to_id: map, - id_to_vertex: vertices.to_vec(), - graph: vec![Vec::::new(); vertices.len()], - } - } - - fn get_vertex_id(&self, v: &Vertex) -> Result { - self.vertex_to_id - .get(v) - .cloned() - .ok_or_else(|| format!("The graph does not contain vertex {:?}", v)) - } -} - -impl Graph { - /// This function adds a directed edge to the graph with capacity c, and the - /// corresponding reversed edge with capacity 0. - pub fn add_edge(&mut self, u: Vertex, v: Vertex, c: u64) -> Result<(), String> { - let idu = self.get_vertex_id(&u)?; - let idv = self.get_vertex_id(&v)?; - if idu == idv { - return Err("Cannot add edge from vertex to itself in flow graph".into()); - } - - let rev_u = self.graph[idu].len(); - let rev_v = self.graph[idv].len(); - self.graph[idu].push(FlowEdge { - cap: c, - dest: idv, - flow: 0, - rev: rev_v, - }); - self.graph[idv].push(FlowEdge { - cap: 0, - dest: idu, - flow: 0, - rev: rev_u, - }); - Ok(()) - } - - /// This function returns the list of vertices that receive a positive flow from - /// vertex v. - pub fn get_positive_flow_from(&self, v: Vertex) -> Result, String> { - let idv = self.get_vertex_id(&v)?; - let mut result = Vec::::new(); - for edge in self.graph[idv].iter() { - if edge.flow > 0 { - result.push(self.id_to_vertex[edge.dest]); - } - } - Ok(result) - } - - /// This function returns the value of the flow incoming to v. - pub fn get_inflow(&self, v: Vertex) -> Result { - let idv = self.get_vertex_id(&v)?; - let mut result = 0; - for edge in self.graph[idv].iter() { - result += max(0, self.graph[edge.dest][edge.rev].flow); - } - Ok(result) - } - - /// This function returns the value of the flow outgoing from v. - pub fn get_outflow(&self, v: Vertex) -> Result { - let idv = self.get_vertex_id(&v)?; - let mut result = 0; - for edge in self.graph[idv].iter() { - result += max(0, edge.flow); - } - Ok(result) - } - - /// This function computes the flow total value by computing the outgoing flow - /// from the source. - pub fn get_flow_value(&mut self) -> Result { - self.get_outflow(Vertex::Source) - } - - /// This function shuffles the order of the edge lists. It keeps the ids of the - /// reversed edges consistent. - fn shuffle_edges(&mut self) { - // We use deterministic randomness so that the layout calculation algorihtm - // will output the same thing every time it is run. This way, the results - // pre-calculated in `garage layout show` will match exactly those used - // in practice with `garage layout apply` - let mut rng = rand::rngs::StdRng::from_seed([0x12u8; 32]); - for i in 0..self.graph.len() { - self.graph[i].shuffle(&mut rng); - // We need to update the ids of the reverse edges. - for j in 0..self.graph[i].len() { - let target_v = self.graph[i][j].dest; - let target_rev = self.graph[i][j].rev; - self.graph[target_v][target_rev].rev = j; - } - } - } - - /// Computes an upper bound of the flow on the graph - pub fn flow_upper_bound(&self) -> Result { - let idsource = self.get_vertex_id(&Vertex::Source)?; - let mut flow_upper_bound = 0; - for edge in self.graph[idsource].iter() { - flow_upper_bound += edge.cap; - } - Ok(flow_upper_bound) - } - - /// This function computes the maximal flow using Dinic's algorithm. It starts with - /// the flow values already present in the graph. So it is possible to add some edge to - /// the graph, compute a flow, add other edges, update the flow. - pub fn compute_maximal_flow(&mut self) -> Result<(), String> { - let idsource = self.get_vertex_id(&Vertex::Source)?; - let idsink = self.get_vertex_id(&Vertex::Sink)?; - - let nb_vertices = self.graph.len(); - - let flow_upper_bound = self.flow_upper_bound()?; - - // To ensure the dispersion of the associations generated by the - // assignment, we shuffle the neighbours of the nodes. Hence, - // the vertices do not consider their neighbours in the same order. - self.shuffle_edges(); - - // We run Dinic's max flow algorithm - loop { - // We build the level array from Dinic's algorithm. - let mut level = vec![None; nb_vertices]; - - let mut fifo = VecDeque::new(); - fifo.push_back((idsource, 0)); - while let Some((id, lvl)) = fifo.pop_front() { - if level[id].is_none() { - // it means id has not yet been reached - level[id] = Some(lvl); - for edge in self.graph[id].iter() { - if edge.cap as i64 - edge.flow > 0 { - fifo.push_back((edge.dest, lvl + 1)); - } - } - } - } - if level[idsink].is_none() { - // There is no residual flow - break; - } - // Now we run DFS respecting the level array - let mut next_nbd = vec![0; nb_vertices]; - let mut lifo = Vec::new(); - - lifo.push((idsource, flow_upper_bound)); - - while let Some((id, f)) = lifo.last().cloned() { - if id == idsink { - // The DFS reached the sink, we can add a - // residual flow. - lifo.pop(); - while let Some((id, _)) = lifo.pop() { - let nbd = next_nbd[id]; - self.graph[id][nbd].flow += f as i64; - let id_rev = self.graph[id][nbd].dest; - let nbd_rev = self.graph[id][nbd].rev; - self.graph[id_rev][nbd_rev].flow -= f as i64; - } - lifo.push((idsource, flow_upper_bound)); - continue; - } - // else we did not reach the sink - let nbd = next_nbd[id]; - if nbd >= self.graph[id].len() { - // There is nothing to explore from id anymore - lifo.pop(); - if let Some((parent, _)) = lifo.last() { - next_nbd[*parent] += 1; - } - continue; - } - // else we can try to send flow from id to its nbd - let new_flow = min( - f as i64, - self.graph[id][nbd].cap as i64 - self.graph[id][nbd].flow, - ) as u64; - if new_flow == 0 { - next_nbd[id] += 1; - continue; - } - if let (Some(lvldest), Some(lvlid)) = (level[self.graph[id][nbd].dest], level[id]) { - if lvldest <= lvlid { - // We cannot send flow to nbd. - next_nbd[id] += 1; - continue; - } - } - // otherwise, we send flow to nbd. - lifo.push((self.graph[id][nbd].dest, new_flow)); - } - } - Ok(()) - } - - /// This function takes a flow, and a cost function on the edges, and tries to find an - /// equivalent flow with a better cost, by finding improving overflow cycles. It uses - /// as subroutine the Bellman Ford algorithm run up to path_length. - /// We assume that the cost of edge (u,v) is the opposite of the cost of (v,u), and - /// only one needs to be present in the cost function. - pub fn optimize_flow_with_cost( - &mut self, - cost: &CostFunction, - path_length: usize, - ) -> Result<(), String> { - // We build the weighted graph g where we will look for negative cycle - let mut gf = self.build_cost_graph(cost)?; - let mut cycles = gf.list_negative_cycles(path_length); - while !cycles.is_empty() { - // we enumerate negative cycles - for c in cycles.iter() { - for i in 0..c.len() { - // We add one flow unit to the edge (u,v) of cycle c - let idu = self.vertex_to_id[&c[i]]; - let idv = self.vertex_to_id[&c[(i + 1) % c.len()]]; - for j in 0..self.graph[idu].len() { - // since idu appears at most once in the cycles, we enumerate every - // edge at most once. - let edge = self.graph[idu][j]; - if edge.dest == idv { - self.graph[idu][j].flow += 1; - self.graph[idv][edge.rev].flow -= 1; - break; - } - } - } - } - - gf = self.build_cost_graph(cost)?; - cycles = gf.list_negative_cycles(path_length); - } - Ok(()) - } - - /// Construct the weighted graph G_f from the flow and the cost function - fn build_cost_graph(&self, cost: &CostFunction) -> Result, String> { - let mut g = Graph::::new(&self.id_to_vertex); - let nb_vertices = self.id_to_vertex.len(); - for i in 0..nb_vertices { - for edge in self.graph[i].iter() { - if edge.cap as i64 - edge.flow > 0 { - // It is possible to send overflow through this edge - let u = self.id_to_vertex[i]; - let v = self.id_to_vertex[edge.dest]; - if cost.contains_key(&(u, v)) { - g.add_edge(u, v, cost[&(u, v)])?; - } else if cost.contains_key(&(v, u)) { - g.add_edge(u, v, -cost[&(v, u)])?; - } else { - g.add_edge(u, v, 0)?; - } - } - } - } - Ok(g) - } -} - -impl Graph { - /// This function adds a single directed weighted edge to the graph. - pub fn add_edge(&mut self, u: Vertex, v: Vertex, w: i64) -> Result<(), String> { - let idu = self.get_vertex_id(&u)?; - let idv = self.get_vertex_id(&v)?; - self.graph[idu].push(WeightedEdge { w, dest: idv }); - Ok(()) - } - - /// This function lists the negative cycles it manages to find after path_length - /// iterations of the main loop of the Bellman-Ford algorithm. For the classical - /// algorithm, path_length needs to be equal to the number of vertices. However, - /// for particular graph structures like in our case, the algorithm is still correct - /// when path_length is the length of the longest possible simple path. - /// See the formal description of the algorithm for more details. - fn list_negative_cycles(&self, path_length: usize) -> Vec> { - let nb_vertices = self.graph.len(); - - // We start with every vertex at distance 0 of some imaginary extra -1 vertex. - let mut distance = vec![0; nb_vertices]; - // The prev vector collects for every vertex from where does the shortest path come - let mut prev = vec![None; nb_vertices]; - - for _ in 0..path_length + 1 { - for id in 0..nb_vertices { - for e in self.graph[id].iter() { - if distance[id] + e.w < distance[e.dest] { - distance[e.dest] = distance[id] + e.w; - prev[e.dest] = Some(id); - } - } - } - } - - // If self.graph contains a negative cycle, then at this point the graph described - // by prev (which is a directed 1-forest/functional graph) - // must contain a cycle. We list the cycles of prev. - let cycles_prev = cycles_of_1_forest(&prev); - - // Remark that the cycle in prev is in the reverse order compared to the cycle - // in the graph. Thus the .rev(). - return cycles_prev - .iter() - .map(|cycle| { - cycle - .iter() - .rev() - .map(|id| self.id_to_vertex[*id]) - .collect() - }) - .collect(); - } -} - -/// This function returns the list of cycles of a directed 1 forest. It does not -/// check for the consistency of the input. -fn cycles_of_1_forest(forest: &[Option]) -> Vec> { - let mut cycles = Vec::>::new(); - let mut time_of_discovery = vec![None; forest.len()]; - - for t in 0..forest.len() { - let mut id = t; - // while we are on a valid undiscovered node - while time_of_discovery[id].is_none() { - time_of_discovery[id] = Some(t); - if let Some(i) = forest[id] { - id = i; - } else { - break; - } - } - if forest[id].is_some() && time_of_discovery[id] == Some(t) { - // We discovered an id that we explored at this iteration t. - // It means we are on a cycle - let mut cy = vec![id; 1]; - let mut id2 = id; - while let Some(id_next) = forest[id2] { - id2 = id_next; - if id2 != id { - cy.push(id2); - } else { - break; - } - } - cycles.push(cy); - } - } - cycles -} diff --git a/src/rpc/layout/graph_algo.rs b/src/rpc/layout/graph_algo.rs new file mode 100644 index 00000000..bd33e97f --- /dev/null +++ b/src/rpc/layout/graph_algo.rs @@ -0,0 +1,405 @@ +//! This module deals with graph algorithms. +//! It is used in layout.rs to build the partition to node assignment. + +use rand::prelude::{SeedableRng, SliceRandom}; +use std::cmp::{max, min}; +use std::collections::HashMap; +use std::collections::VecDeque; + +/// Vertex data structures used in all the graphs used in layout.rs. +/// usize parameters correspond to node/zone/partitions ids. +/// To understand the vertex roles below, please refer to the formal description +/// of the layout computation algorithm. +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash)] +pub enum Vertex { + Source, + Pup(usize), // The vertex p+ of partition p + Pdown(usize), // The vertex p- of partition p + PZ(usize, usize), // The vertex corresponding to x_(partition p, zone z) + N(usize), // The vertex corresponding to node n + Sink, +} + +/// Edge data structure for the flow algorithm. +#[derive(Clone, Copy, Debug)] +pub struct FlowEdge { + cap: u64, // flow maximal capacity of the edge + flow: i64, // flow value on the edge + dest: usize, // destination vertex id + rev: usize, // index of the reversed edge (v, self) in the edge list of vertex v +} + +/// Edge data structure for the detection of negative cycles. +#[derive(Clone, Copy, Debug)] +pub struct WeightedEdge { + w: i64, // weight of the edge + dest: usize, +} + +pub trait Edge: Clone + Copy {} +impl Edge for FlowEdge {} +impl Edge for WeightedEdge {} + +/// Struct for the graph structure. We do encapsulation here to be able to both +/// provide user friendly Vertex enum to address vertices, and to use internally usize +/// indices and Vec instead of HashMap in the graph algorithm to optimize execution speed. +pub struct Graph { + vertex_to_id: HashMap, + id_to_vertex: Vec, + + // The graph is stored as an adjacency list + graph: Vec>, +} + +pub type CostFunction = HashMap<(Vertex, Vertex), i64>; + +impl Graph { + pub fn new(vertices: &[Vertex]) -> Self { + let mut map = HashMap::::new(); + for (i, vert) in vertices.iter().enumerate() { + map.insert(*vert, i); + } + Graph:: { + vertex_to_id: map, + id_to_vertex: vertices.to_vec(), + graph: vec![Vec::::new(); vertices.len()], + } + } + + fn get_vertex_id(&self, v: &Vertex) -> Result { + self.vertex_to_id + .get(v) + .cloned() + .ok_or_else(|| format!("The graph does not contain vertex {:?}", v)) + } +} + +impl Graph { + /// This function adds a directed edge to the graph with capacity c, and the + /// corresponding reversed edge with capacity 0. + pub fn add_edge(&mut self, u: Vertex, v: Vertex, c: u64) -> Result<(), String> { + let idu = self.get_vertex_id(&u)?; + let idv = self.get_vertex_id(&v)?; + if idu == idv { + return Err("Cannot add edge from vertex to itself in flow graph".into()); + } + + let rev_u = self.graph[idu].len(); + let rev_v = self.graph[idv].len(); + self.graph[idu].push(FlowEdge { + cap: c, + dest: idv, + flow: 0, + rev: rev_v, + }); + self.graph[idv].push(FlowEdge { + cap: 0, + dest: idu, + flow: 0, + rev: rev_u, + }); + Ok(()) + } + + /// This function returns the list of vertices that receive a positive flow from + /// vertex v. + pub fn get_positive_flow_from(&self, v: Vertex) -> Result, String> { + let idv = self.get_vertex_id(&v)?; + let mut result = Vec::::new(); + for edge in self.graph[idv].iter() { + if edge.flow > 0 { + result.push(self.id_to_vertex[edge.dest]); + } + } + Ok(result) + } + + /// This function returns the value of the flow outgoing from v. + pub fn get_outflow(&self, v: Vertex) -> Result { + let idv = self.get_vertex_id(&v)?; + let mut result = 0; + for edge in self.graph[idv].iter() { + result += max(0, edge.flow); + } + Ok(result) + } + + /// This function computes the flow total value by computing the outgoing flow + /// from the source. + pub fn get_flow_value(&mut self) -> Result { + self.get_outflow(Vertex::Source) + } + + /// This function shuffles the order of the edge lists. It keeps the ids of the + /// reversed edges consistent. + fn shuffle_edges(&mut self) { + // We use deterministic randomness so that the layout calculation algorihtm + // will output the same thing every time it is run. This way, the results + // pre-calculated in `garage layout show` will match exactly those used + // in practice with `garage layout apply` + let mut rng = rand::rngs::StdRng::from_seed([0x12u8; 32]); + for i in 0..self.graph.len() { + self.graph[i].shuffle(&mut rng); + // We need to update the ids of the reverse edges. + for j in 0..self.graph[i].len() { + let target_v = self.graph[i][j].dest; + let target_rev = self.graph[i][j].rev; + self.graph[target_v][target_rev].rev = j; + } + } + } + + /// Computes an upper bound of the flow on the graph + pub fn flow_upper_bound(&self) -> Result { + let idsource = self.get_vertex_id(&Vertex::Source)?; + let mut flow_upper_bound = 0; + for edge in self.graph[idsource].iter() { + flow_upper_bound += edge.cap; + } + Ok(flow_upper_bound) + } + + /// This function computes the maximal flow using Dinic's algorithm. It starts with + /// the flow values already present in the graph. So it is possible to add some edge to + /// the graph, compute a flow, add other edges, update the flow. + pub fn compute_maximal_flow(&mut self) -> Result<(), String> { + let idsource = self.get_vertex_id(&Vertex::Source)?; + let idsink = self.get_vertex_id(&Vertex::Sink)?; + + let nb_vertices = self.graph.len(); + + let flow_upper_bound = self.flow_upper_bound()?; + + // To ensure the dispersion of the associations generated by the + // assignment, we shuffle the neighbours of the nodes. Hence, + // the vertices do not consider their neighbours in the same order. + self.shuffle_edges(); + + // We run Dinic's max flow algorithm + loop { + // We build the level array from Dinic's algorithm. + let mut level = vec![None; nb_vertices]; + + let mut fifo = VecDeque::new(); + fifo.push_back((idsource, 0)); + while let Some((id, lvl)) = fifo.pop_front() { + if level[id].is_none() { + // it means id has not yet been reached + level[id] = Some(lvl); + for edge in self.graph[id].iter() { + if edge.cap as i64 - edge.flow > 0 { + fifo.push_back((edge.dest, lvl + 1)); + } + } + } + } + if level[idsink].is_none() { + // There is no residual flow + break; + } + // Now we run DFS respecting the level array + let mut next_nbd = vec![0; nb_vertices]; + let mut lifo = Vec::new(); + + lifo.push((idsource, flow_upper_bound)); + + while let Some((id, f)) = lifo.last().cloned() { + if id == idsink { + // The DFS reached the sink, we can add a + // residual flow. + lifo.pop(); + while let Some((id, _)) = lifo.pop() { + let nbd = next_nbd[id]; + self.graph[id][nbd].flow += f as i64; + let id_rev = self.graph[id][nbd].dest; + let nbd_rev = self.graph[id][nbd].rev; + self.graph[id_rev][nbd_rev].flow -= f as i64; + } + lifo.push((idsource, flow_upper_bound)); + continue; + } + // else we did not reach the sink + let nbd = next_nbd[id]; + if nbd >= self.graph[id].len() { + // There is nothing to explore from id anymore + lifo.pop(); + if let Some((parent, _)) = lifo.last() { + next_nbd[*parent] += 1; + } + continue; + } + // else we can try to send flow from id to its nbd + let new_flow = min( + f as i64, + self.graph[id][nbd].cap as i64 - self.graph[id][nbd].flow, + ) as u64; + if new_flow == 0 { + next_nbd[id] += 1; + continue; + } + if let (Some(lvldest), Some(lvlid)) = (level[self.graph[id][nbd].dest], level[id]) { + if lvldest <= lvlid { + // We cannot send flow to nbd. + next_nbd[id] += 1; + continue; + } + } + // otherwise, we send flow to nbd. + lifo.push((self.graph[id][nbd].dest, new_flow)); + } + } + Ok(()) + } + + /// This function takes a flow, and a cost function on the edges, and tries to find an + /// equivalent flow with a better cost, by finding improving overflow cycles. It uses + /// as subroutine the Bellman Ford algorithm run up to path_length. + /// We assume that the cost of edge (u,v) is the opposite of the cost of (v,u), and + /// only one needs to be present in the cost function. + pub fn optimize_flow_with_cost( + &mut self, + cost: &CostFunction, + path_length: usize, + ) -> Result<(), String> { + // We build the weighted graph g where we will look for negative cycle + let mut gf = self.build_cost_graph(cost)?; + let mut cycles = gf.list_negative_cycles(path_length); + while !cycles.is_empty() { + // we enumerate negative cycles + for c in cycles.iter() { + for i in 0..c.len() { + // We add one flow unit to the edge (u,v) of cycle c + let idu = self.vertex_to_id[&c[i]]; + let idv = self.vertex_to_id[&c[(i + 1) % c.len()]]; + for j in 0..self.graph[idu].len() { + // since idu appears at most once in the cycles, we enumerate every + // edge at most once. + let edge = self.graph[idu][j]; + if edge.dest == idv { + self.graph[idu][j].flow += 1; + self.graph[idv][edge.rev].flow -= 1; + break; + } + } + } + } + + gf = self.build_cost_graph(cost)?; + cycles = gf.list_negative_cycles(path_length); + } + Ok(()) + } + + /// Construct the weighted graph G_f from the flow and the cost function + fn build_cost_graph(&self, cost: &CostFunction) -> Result, String> { + let mut g = Graph::::new(&self.id_to_vertex); + let nb_vertices = self.id_to_vertex.len(); + for i in 0..nb_vertices { + for edge in self.graph[i].iter() { + if edge.cap as i64 - edge.flow > 0 { + // It is possible to send overflow through this edge + let u = self.id_to_vertex[i]; + let v = self.id_to_vertex[edge.dest]; + if cost.contains_key(&(u, v)) { + g.add_edge(u, v, cost[&(u, v)])?; + } else if cost.contains_key(&(v, u)) { + g.add_edge(u, v, -cost[&(v, u)])?; + } else { + g.add_edge(u, v, 0)?; + } + } + } + } + Ok(g) + } +} + +impl Graph { + /// This function adds a single directed weighted edge to the graph. + pub fn add_edge(&mut self, u: Vertex, v: Vertex, w: i64) -> Result<(), String> { + let idu = self.get_vertex_id(&u)?; + let idv = self.get_vertex_id(&v)?; + self.graph[idu].push(WeightedEdge { w, dest: idv }); + Ok(()) + } + + /// This function lists the negative cycles it manages to find after path_length + /// iterations of the main loop of the Bellman-Ford algorithm. For the classical + /// algorithm, path_length needs to be equal to the number of vertices. However, + /// for particular graph structures like in our case, the algorithm is still correct + /// when path_length is the length of the longest possible simple path. + /// See the formal description of the algorithm for more details. + fn list_negative_cycles(&self, path_length: usize) -> Vec> { + let nb_vertices = self.graph.len(); + + // We start with every vertex at distance 0 of some imaginary extra -1 vertex. + let mut distance = vec![0; nb_vertices]; + // The prev vector collects for every vertex from where does the shortest path come + let mut prev = vec![None; nb_vertices]; + + for _ in 0..path_length + 1 { + for id in 0..nb_vertices { + for e in self.graph[id].iter() { + if distance[id] + e.w < distance[e.dest] { + distance[e.dest] = distance[id] + e.w; + prev[e.dest] = Some(id); + } + } + } + } + + // If self.graph contains a negative cycle, then at this point the graph described + // by prev (which is a directed 1-forest/functional graph) + // must contain a cycle. We list the cycles of prev. + let cycles_prev = cycles_of_1_forest(&prev); + + // Remark that the cycle in prev is in the reverse order compared to the cycle + // in the graph. Thus the .rev(). + return cycles_prev + .iter() + .map(|cycle| { + cycle + .iter() + .rev() + .map(|id| self.id_to_vertex[*id]) + .collect() + }) + .collect(); + } +} + +/// This function returns the list of cycles of a directed 1 forest. It does not +/// check for the consistency of the input. +fn cycles_of_1_forest(forest: &[Option]) -> Vec> { + let mut cycles = Vec::>::new(); + let mut time_of_discovery = vec![None; forest.len()]; + + for t in 0..forest.len() { + let mut id = t; + // while we are on a valid undiscovered node + while time_of_discovery[id].is_none() { + time_of_discovery[id] = Some(t); + if let Some(i) = forest[id] { + id = i; + } else { + break; + } + } + if forest[id].is_some() && time_of_discovery[id] == Some(t) { + // We discovered an id that we explored at this iteration t. + // It means we are on a cycle + let mut cy = vec![id; 1]; + let mut id2 = id; + while let Some(id_next) = forest[id2] { + id2 = id_next; + if id2 != id { + cy.push(id2); + } else { + break; + } + } + cycles.push(cy); + } + } + cycles +} diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index e59c9e9c..9ae28887 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -1,5 +1,3 @@ -use std::cmp::Ordering; - use garage_util::crdt::{Crdt, Lww, LwwMap}; use garage_util::data::*; use garage_util::encode::nonversioned_encode; @@ -12,14 +10,15 @@ impl LayoutHistory { pub fn new(replication_factor: usize) -> Self { let version = LayoutVersion::new(replication_factor); - let staging_parameters = Lww::::new(version.parameters); - let empty_lwwmap = LwwMap::new(); + let staging = LayoutStaging { + parameters: Lww::::new(version.parameters), + roles: LwwMap::new(), + }; let mut ret = LayoutHistory { versions: vec![version].into_boxed_slice().into(), update_trackers: Default::default(), - staging_parameters, - staging_roles: empty_lwwmap, + staging: Lww::raw(0, staging), staging_hash: [0u8; 32].into(), }; ret.staging_hash = ret.calculate_staging_hash(); @@ -31,8 +30,7 @@ impl LayoutHistory { } pub(crate) fn calculate_staging_hash(&self) -> Hash { - let hashed_tuple = (&self.staging_roles, &self.staging_parameters); - blake2sum(&nonversioned_encode(&hashed_tuple).unwrap()[..]) + blake2sum(&nonversioned_encode(&self.staging).unwrap()[..]) } // ================== updates to layout, public interface =================== @@ -41,26 +39,10 @@ impl LayoutHistory { let mut changed = false; // Merge staged layout changes - match other.current().version.cmp(&self.current().version) { - Ordering::Greater => { - self.staging_parameters = other.staging_parameters.clone(); - self.staging_roles = other.staging_roles.clone(); - self.staging_hash = other.staging_hash; - changed = true; - } - Ordering::Equal => { - self.staging_parameters.merge(&other.staging_parameters); - self.staging_roles.merge(&other.staging_roles); - - let new_staging_hash = self.calculate_staging_hash(); - if new_staging_hash != self.staging_hash { - changed = true; - } - - self.staging_hash = new_staging_hash; - } - Ordering::Less => (), + if self.staging != other.staging { + changed = true; } + self.staging.merge(&other.staging); // Add any new versions to history for v2 in other.versions.iter() { @@ -102,50 +84,34 @@ To know the correct value of the new layout version, invoke `garage layout show` } } + // Compute new version and add it to history let mut new_version = self.current().clone(); new_version.version += 1; - new_version.roles.merge(&self.staging_roles); + new_version.roles.merge(&self.staging.get().roles); new_version.roles.retain(|(_, _, v)| v.0.is_some()); - new_version.parameters = *self.staging_parameters.get(); - - self.staging_roles.clear(); - self.staging_hash = self.calculate_staging_hash(); + new_version.parameters = *self.staging.get().parameters.get(); let msg = new_version.calculate_partition_assignment()?; - self.versions.push(new_version); + // Reset the staged layout changes + self.staging.update(LayoutStaging { + parameters: self.staging.get().parameters.clone(), + roles: LwwMap::new(), + }); + self.staging_hash = self.calculate_staging_hash(); + Ok((self, msg)) } - pub fn revert_staged_changes(mut self, version: Option) -> Result { - match version { - None => { - let error = r#" -Please pass the new layout version number to ensure that you are writing the correct version of the cluster layout. -To know the correct value of the new layout version, invoke `garage layout show` and review the proposed changes. - "#; - return Err(Error::Message(error.into())); - } - Some(v) => { - if v != self.current().version + 1 { - return Err(Error::Message("Invalid new layout version".into())); - } - } - } - - self.staging_roles.clear(); - self.staging_parameters.update(self.current().parameters); + pub fn revert_staged_changes(mut self) -> Result { + self.staging.update(LayoutStaging { + parameters: Lww::new(self.current().parameters.clone()), + roles: LwwMap::new(), + }); self.staging_hash = self.calculate_staging_hash(); - // TODO this is stupid, we should have a separate version counter/LWW - // for the staging params - let mut new_version = self.current().clone(); - new_version.version += 1; - - self.versions.push(new_version); - Ok(self) } diff --git a/src/rpc/layout/mod.rs b/src/rpc/layout/mod.rs index 122d4b65..7c15988a 100644 --- a/src/rpc/layout/mod.rs +++ b/src/rpc/layout/mod.rs @@ -1,8 +1,10 @@ +mod graph_algo; mod history; mod schema; -mod tracker; mod version; +// ---- re-exports ---- + pub use history::*; pub use schema::*; pub use version::*; diff --git a/src/rpc/layout/schema.rs b/src/rpc/layout/schema.rs index 14e797be..c5b9b1d3 100644 --- a/src/rpc/layout/schema.rs +++ b/src/rpc/layout/schema.rs @@ -1,3 +1,9 @@ +use std::fmt; + +use bytesize::ByteSize; + +use garage_util::crdt::{AutoCrdt, Crdt}; + mod v08 { use crate::layout::CompactNodeType; use garage_util::crdt::LwwMap; @@ -210,6 +216,15 @@ mod v010 { pub ring_assignment_data: Vec, } + /// The staged changes for the next layout version + #[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] + pub struct LayoutStaging { + /// Parameters to be used in the next partition assignment computation. + pub parameters: Lww, + /// Role changes which are staged for the next version of the layout + pub roles: LwwMap, + } + /// The history of cluster layouts #[derive(Clone, Debug, Serialize, Deserialize)] pub struct LayoutHistory { @@ -219,10 +234,8 @@ mod v010 { /// Update trackers pub update_trackers: UpdateTrackers, - /// Parameters to be used in the next partition assignment computation. - pub staging_parameters: Lww, - /// Role changes which are staged for the next version of the layout - pub staging_roles: LwwMap, + /// Staged changes for the next version + pub staging: Lww, /// Hash of the serialized staging_parameters + staging_roles pub staging_hash: Hash, } @@ -265,6 +278,10 @@ mod v010 { .map(|x| (*x, version.version)) .collect::>(), ); + let staging = LayoutStaging { + parameters: previous.staging_parameters, + roles: previous.staging_roles, + }; let mut ret = Self { versions: vec![version], update_trackers: UpdateTrackers { @@ -272,8 +289,7 @@ mod v010 { sync_map: update_tracker.clone(), sync_ack_map: update_tracker.clone(), }, - staging_parameters: previous.staging_parameters, - staging_roles: previous.staging_roles, + staging: Lww::raw(previous.version, staging), staging_hash: [0u8; 32].into(), }; ret.staging_hash = ret.calculate_staging_hash(); @@ -283,3 +299,81 @@ mod v010 { } pub use v010::*; + +// ---- utility functions ---- + +impl AutoCrdt for LayoutParameters { + const WARN_IF_DIFFERENT: bool = true; +} + +impl AutoCrdt for NodeRoleV { + const WARN_IF_DIFFERENT: bool = true; +} + +impl Crdt for LayoutStaging { + fn merge(&mut self, other: &LayoutStaging) { + self.parameters.merge(&other.parameters); + self.roles.merge(&other.roles); + } +} + +impl NodeRole { + pub fn capacity_string(&self) -> String { + match self.capacity { + Some(c) => ByteSize::b(c).to_string_as(false), + None => "gateway".to_string(), + } + } + + pub fn tags_string(&self) -> String { + self.tags.join(",") + } +} + +impl fmt::Display for ZoneRedundancy { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + ZoneRedundancy::Maximum => write!(f, "maximum"), + ZoneRedundancy::AtLeast(x) => write!(f, "{}", x), + } + } +} + +impl core::str::FromStr for ZoneRedundancy { + type Err = &'static str; + fn from_str(s: &str) -> Result { + match s { + "none" | "max" | "maximum" => Ok(ZoneRedundancy::Maximum), + x => { + let v = x + .parse::() + .map_err(|_| "zone redundancy must be 'none'/'max' or an integer")?; + Ok(ZoneRedundancy::AtLeast(v)) + } + } + } +} + +impl UpdateTracker { + fn merge(&mut self, other: &UpdateTracker) { + for (k, v) in other.0.iter() { + if let Some(v_mut) = self.0.get_mut(k) { + *v_mut = std::cmp::max(*v_mut, *v); + } else { + self.0.insert(*k, *v); + } + } + } + + pub(crate) fn min(&self) -> u64 { + self.0.iter().map(|(_, v)| *v).min().unwrap_or(0) + } +} + +impl UpdateTrackers { + pub(crate) fn merge(&mut self, other: &UpdateTrackers) { + self.ack_map.merge(&other.ack_map); + self.sync_map.merge(&other.sync_map); + self.sync_ack_map.merge(&other.sync_ack_map); + } +} diff --git a/src/rpc/layout/tracker.rs b/src/rpc/layout/tracker.rs deleted file mode 100644 index 778121e4..00000000 --- a/src/rpc/layout/tracker.rs +++ /dev/null @@ -1,21 +0,0 @@ -use super::*; - -impl UpdateTracker { - fn merge(&mut self, other: &UpdateTracker) { - for (k, v) in other.0.iter() { - if let Some(v_mut) = self.0.get_mut(k) { - *v_mut = std::cmp::max(*v_mut, *v); - } else { - self.0.insert(*k, *v); - } - } - } -} - -impl UpdateTrackers { - pub(crate) fn merge(&mut self, other: &UpdateTrackers) { - self.ack_map.merge(&other.ack_map); - self.sync_map.merge(&other.sync_map); - self.sync_ack_map.merge(&other.sync_ack_map); - } -} diff --git a/src/rpc/layout/version.rs b/src/rpc/layout/version.rs index 363bc204..6918fdf9 100644 --- a/src/rpc/layout/version.rs +++ b/src/rpc/layout/version.rs @@ -1,69 +1,21 @@ use std::collections::HashMap; use std::collections::HashSet; -use std::fmt; +use std::convert::TryInto; use bytesize::ByteSize; use itertools::Itertools; -use garage_util::crdt::{AutoCrdt, LwwMap}; +use garage_util::crdt::LwwMap; use garage_util::data::*; use garage_util::error::*; -use crate::graph_algo::*; - -use std::convert::TryInto; - +use super::graph_algo::*; use super::schema::*; use super::*; // The Message type will be used to collect information on the algorithm. pub type Message = Vec; -impl AutoCrdt for LayoutParameters { - const WARN_IF_DIFFERENT: bool = true; -} - -impl AutoCrdt for NodeRoleV { - const WARN_IF_DIFFERENT: bool = true; -} - -impl NodeRole { - pub fn capacity_string(&self) -> String { - match self.capacity { - Some(c) => ByteSize::b(c).to_string_as(false), - None => "gateway".to_string(), - } - } - - pub fn tags_string(&self) -> String { - self.tags.join(",") - } -} - -impl fmt::Display for ZoneRedundancy { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match self { - ZoneRedundancy::Maximum => write!(f, "maximum"), - ZoneRedundancy::AtLeast(x) => write!(f, "{}", x), - } - } -} - -impl core::str::FromStr for ZoneRedundancy { - type Err = &'static str; - fn from_str(s: &str) -> Result { - match s { - "none" | "max" | "maximum" => Ok(ZoneRedundancy::Maximum), - x => { - let v = x - .parse::() - .map_err(|_| "zone redundancy must be 'none'/'max' or an integer")?; - Ok(ZoneRedundancy::AtLeast(v)) - } - } - } -} - impl LayoutVersion { pub fn new(replication_factor: usize) -> Self { // We set the default zone redundancy to be Maximum, meaning that the maximum diff --git a/src/rpc/lib.rs b/src/rpc/lib.rs index 1af8b78e..b5b31c05 100644 --- a/src/rpc/lib.rs +++ b/src/rpc/lib.rs @@ -11,7 +11,6 @@ mod consul; #[cfg(feature = "kubernetes-discovery")] mod kubernetes; -pub mod graph_algo; pub mod layout; pub mod replication_mode; pub mod system; -- cgit v1.2.3 From 8a2b1dd422fb57abe611d8c1cf3cb0b55f487189 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 9 Nov 2023 12:55:36 +0100 Subject: wip: split out layout management from System into separate LayoutManager --- src/api/admin/cluster.rs | 18 ++- src/block/manager.rs | 10 +- src/block/resync.rs | 4 +- src/model/k2v/rpc.rs | 20 ++-- src/rpc/layout/manager.rs | 177 ++++++++++++++++++++++++++++ src/rpc/layout/mod.rs | 2 + src/rpc/system.rs | 295 +++++++++++++++++----------------------------- src/table/gc.rs | 4 +- src/table/sync.rs | 10 +- src/table/table.rs | 10 +- 10 files changed, 331 insertions(+), 219 deletions(-) create mode 100644 src/rpc/layout/manager.rs diff --git a/src/api/admin/cluster.rs b/src/api/admin/cluster.rs index fe8e8764..f5483451 100644 --- a/src/api/admin/cluster.rs +++ b/src/api/admin/cluster.rs @@ -240,7 +240,11 @@ pub async fn handle_update_cluster_layout( .merge(&roles.update_mutator(node, layout::NodeRoleV(new_role))); } - garage.system.update_cluster_layout(&layout).await?; + garage + .system + .layout_manager + .update_cluster_layout(&layout) + .await?; let res = format_cluster_layout(&layout); Ok(json_ok_response(&res)?) @@ -255,7 +259,11 @@ pub async fn handle_apply_cluster_layout( let layout = garage.system.cluster_layout().as_ref().clone(); let (layout, msg) = layout.apply_staged_changes(Some(param.version))?; - garage.system.update_cluster_layout(&layout).await?; + garage + .system + .layout_manager + .update_cluster_layout(&layout) + .await?; let res = ApplyClusterLayoutResponse { message: msg, @@ -267,7 +275,11 @@ pub async fn handle_apply_cluster_layout( pub async fn handle_revert_cluster_layout(garage: &Arc) -> Result, Error> { let layout = garage.system.cluster_layout().as_ref().clone(); let layout = layout.revert_staged_changes()?; - garage.system.update_cluster_layout(&layout).await?; + garage + .system + .layout_manager + .update_cluster_layout(&layout) + .await?; let res = format_cluster_layout(&layout); Ok(json_ok_response(&res)?) diff --git a/src/block/manager.rs b/src/block/manager.rs index 2d1b5c67..72b4ea66 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -265,7 +265,7 @@ impl BlockManager { Fut: futures::Future>, { let who = self.replication.read_nodes(hash); - let who = self.system.rpc.request_order(&who); + let who = self.system.rpc_helper().request_order(&who); for node in who.iter() { let node_id = NodeID::from(*node); @@ -305,7 +305,7 @@ impl BlockManager { // if the first one doesn't succeed rapidly // TODO: keep first request running when initiating a new one and take the // one that finishes earlier - _ = tokio::time::sleep(self.system.rpc.rpc_timeout()) => { + _ = tokio::time::sleep(self.system.rpc_helper().rpc_timeout()) => { debug!("Get block {:?}: node {:?} didn't return block in time, trying next.", hash, node); } }; @@ -363,7 +363,7 @@ impl BlockManager { Req::new(BlockRpc::PutBlock { hash, header })?.with_stream_from_buffer(bytes); self.system - .rpc + .rpc_helper() .try_call_many( &self.endpoint, &who[..], @@ -439,7 +439,7 @@ impl BlockManager { tokio::spawn(async move { if let Err(e) = this .resync - .put_to_resync(&hash, 2 * this.system.rpc.rpc_timeout()) + .put_to_resync(&hash, 2 * this.system.rpc_helper().rpc_timeout()) { error!("Block {:?} could not be put in resync queue: {}.", hash, e); } @@ -533,7 +533,7 @@ impl BlockManager { None => { // Not found but maybe we should have had it ?? self.resync - .put_to_resync(hash, 2 * self.system.rpc.rpc_timeout())?; + .put_to_resync(hash, 2 * self.system.rpc_helper().rpc_timeout())?; return Err(Error::Message(format!( "block {:?} not found on node", hash diff --git a/src/block/resync.rs b/src/block/resync.rs index 9c1da4a7..fedcd6f5 100644 --- a/src/block/resync.rs +++ b/src/block/resync.rs @@ -385,7 +385,7 @@ impl BlockResyncManager { let who_needs_resps = manager .system - .rpc + .rpc_helper() .call_many( &manager.endpoint, &who, @@ -431,7 +431,7 @@ impl BlockResyncManager { .with_stream_from_buffer(bytes); manager .system - .rpc + .rpc_helper() .try_call_many( &manager.endpoint, &need_nodes[..], diff --git a/src/model/k2v/rpc.rs b/src/model/k2v/rpc.rs index 37e142f6..2f548ad7 100644 --- a/src/model/k2v/rpc.rs +++ b/src/model/k2v/rpc.rs @@ -131,7 +131,7 @@ impl K2VRpcHandler { who.sort(); self.system - .rpc + .rpc_helper() .try_call_many( &self.endpoint, &who[..], @@ -187,7 +187,7 @@ impl K2VRpcHandler { let call_futures = call_list.into_iter().map(|(nodes, items)| async move { let resp = self .system - .rpc + .rpc_helper() .try_call_many( &self.endpoint, &nodes[..], @@ -229,7 +229,7 @@ impl K2VRpcHandler { .replication .write_nodes(&poll_key.partition.hash()); - let rpc = self.system.rpc.try_call_many( + let rpc = self.system.rpc_helper().try_call_many( &self.endpoint, &nodes[..], K2VRpc::PollItem { @@ -241,7 +241,8 @@ impl K2VRpcHandler { .with_quorum(self.item_table.data.replication.read_quorum()) .without_timeout(), ); - let timeout_duration = Duration::from_millis(timeout_msec) + self.system.rpc.rpc_timeout(); + let timeout_duration = + Duration::from_millis(timeout_msec) + self.system.rpc_helper().rpc_timeout(); let resps = select! { r = rpc => r?, _ = tokio::time::sleep(timeout_duration) => return Ok(None), @@ -300,7 +301,11 @@ impl K2VRpcHandler { let rs = RequestStrategy::with_priority(PRIO_NORMAL).without_timeout(); let mut requests = nodes .iter() - .map(|node| self.system.rpc.call(&self.endpoint, *node, msg.clone(), rs)) + .map(|node| { + self.system + .rpc_helper() + .call(&self.endpoint, *node, msg.clone(), rs) + }) .collect::>(); // Fetch responses. This procedure stops fetching responses when any of the following @@ -316,8 +321,9 @@ impl K2VRpcHandler { // kind: all items produced by that node until time ts have been returned, so we can // bump the entry in the global vector clock and possibly remove some item-specific // vector clocks) - let mut deadline = - Instant::now() + Duration::from_millis(timeout_msec) + self.system.rpc.rpc_timeout(); + let mut deadline = Instant::now() + + Duration::from_millis(timeout_msec) + + self.system.rpc_helper().rpc_timeout(); let mut resps = vec![]; let mut errors = vec![]; loop { diff --git a/src/rpc/layout/manager.rs b/src/rpc/layout/manager.rs new file mode 100644 index 00000000..a8a77139 --- /dev/null +++ b/src/rpc/layout/manager.rs @@ -0,0 +1,177 @@ +use std::sync::Arc; +use std::time::Duration; + +use tokio::sync::watch; +use tokio::sync::Mutex; + +use netapp::endpoint::Endpoint; +use netapp::peering::fullmesh::FullMeshPeeringStrategy; +use netapp::NodeID; + +use garage_util::config::Config; +use garage_util::data::*; +use garage_util::error::*; +use garage_util::persister::Persister; + +use super::*; +use crate::rpc_helper::*; +use crate::system::*; + +pub struct LayoutManager { + replication_factor: usize, + persist_cluster_layout: Persister, + + pub layout_watch: watch::Receiver>, + update_layout: Mutex>>, + + pub(crate) rpc_helper: RpcHelper, + system_endpoint: Arc>, +} + +impl LayoutManager { + pub fn new( + config: &Config, + node_id: NodeID, + system_endpoint: Arc>, + fullmesh: Arc, + replication_factor: usize, + ) -> Result { + let persist_cluster_layout: Persister = + Persister::new(&config.metadata_dir, "cluster_layout"); + + let cluster_layout = match persist_cluster_layout.load() { + Ok(x) => { + if x.current().replication_factor != replication_factor { + return Err(Error::Message(format!( + "Prevous cluster layout has replication factor {}, which is different than the one specified in the config file ({}). The previous cluster layout can be purged, if you know what you are doing, simply by deleting the `cluster_layout` file in your metadata directory.", + x.current().replication_factor, + replication_factor + ))); + } + x + } + Err(e) => { + info!( + "No valid previous cluster layout stored ({}), starting fresh.", + e + ); + LayoutHistory::new(replication_factor) + } + }; + + let (update_layout, layout_watch) = watch::channel(Arc::new(cluster_layout)); + + let rpc_helper = RpcHelper::new( + node_id.into(), + fullmesh, + layout_watch.clone(), + config.rpc_timeout_msec.map(Duration::from_millis), + ); + + Ok(Self { + replication_factor, + persist_cluster_layout, + layout_watch, + update_layout: Mutex::new(update_layout), + system_endpoint, + rpc_helper, + }) + } + + // ---- PUBLIC INTERFACE ---- + + pub async fn update_cluster_layout(&self, layout: &LayoutHistory) -> Result<(), Error> { + self.handle_advertise_cluster_layout(layout).await?; + Ok(()) + } + + pub fn history(&self) -> watch::Ref> { + self.layout_watch.borrow() + } + + pub(crate) async fn pull_cluster_layout(&self, peer: Uuid) { + let resp = self + .rpc_helper + .call( + &self.system_endpoint, + peer, + SystemRpc::PullClusterLayout, + RequestStrategy::with_priority(PRIO_HIGH), + ) + .await; + if let Ok(SystemRpc::AdvertiseClusterLayout(layout)) = resp { + let _: Result<_, _> = self.handle_advertise_cluster_layout(&layout).await; + } + } + + // ---- INTERNALS --- + + /// Save network configuration to disc + async fn save_cluster_layout(&self) -> Result<(), Error> { + let layout: Arc = self.layout_watch.borrow().clone(); + self.persist_cluster_layout + .save_async(&layout) + .await + .expect("Cannot save current cluster layout"); + Ok(()) + } + + // ---- RPC HANDLERS ---- + + pub(crate) fn handle_pull_cluster_layout(&self) -> SystemRpc { + let layout = self.layout_watch.borrow().as_ref().clone(); + SystemRpc::AdvertiseClusterLayout(layout) + } + + pub(crate) async fn handle_advertise_cluster_layout( + &self, + adv: &LayoutHistory, + ) -> Result { + if adv.current().replication_factor != self.replication_factor { + let msg = format!( + "Received a cluster layout from another node with replication factor {}, which is different from what we have in our configuration ({}). Discarding the cluster layout we received.", + adv.current().replication_factor, + self.replication_factor + ); + error!("{}", msg); + return Err(Error::Message(msg)); + } + + let update_layout = self.update_layout.lock().await; + // TODO: don't clone each time an AdvertiseClusterLayout is received + let mut layout: LayoutHistory = self.layout_watch.borrow().as_ref().clone(); + + let prev_layout_check = layout.check().is_ok(); + if layout.merge(adv) { + if prev_layout_check && layout.check().is_err() { + error!("New cluster layout is invalid, discarding."); + return Err(Error::Message( + "New cluster layout is invalid, discarding.".into(), + )); + } + + update_layout.send(Arc::new(layout.clone()))?; + drop(update_layout); + + /* TODO + tokio::spawn(async move { + if let Err(e) = system + .rpc_helper() + .broadcast( + &system.system_endpoint, + SystemRpc::AdvertiseClusterLayout(layout), + RequestStrategy::with_priority(PRIO_HIGH), + ) + .await + { + warn!("Error while broadcasting new cluster layout: {}", e); + } + }); + */ + + self.save_cluster_layout().await?; + } + + Ok(SystemRpc::Ok) + } +} diff --git a/src/rpc/layout/mod.rs b/src/rpc/layout/mod.rs index 7c15988a..cd3764bc 100644 --- a/src/rpc/layout/mod.rs +++ b/src/rpc/layout/mod.rs @@ -3,6 +3,8 @@ mod history; mod schema; mod version; +pub mod manager; + // ---- re-exports ---- pub use history::*; diff --git a/src/rpc/system.rs b/src/rpc/system.rs index a7433b68..a8e88425 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -14,7 +14,6 @@ use serde::{Deserialize, Serialize}; use sodiumoxide::crypto::sign::ed25519; use tokio::select; use tokio::sync::watch; -use tokio::sync::Mutex; use netapp::endpoint::{Endpoint, EndpointHandler}; use netapp::message::*; @@ -34,6 +33,7 @@ use garage_util::time::*; use crate::consul::ConsulDiscovery; #[cfg(feature = "kubernetes-discovery")] use crate::kubernetes::*; +use crate::layout::manager::LayoutManager; use crate::layout::*; use crate::replication_mode::*; use crate::rpc_helper::*; @@ -49,7 +49,7 @@ const STATUS_EXCHANGE_INTERVAL: Duration = Duration::from_secs(10); pub const GARAGE_VERSION_TAG: u64 = 0x676172616765000A; // garage 0x000A /// RPC endpoint used for calls related to membership -pub const SYSTEM_RPC_PATH: &str = "garage_rpc/membership.rs/SystemRpc"; +pub const SYSTEM_RPC_PATH: &str = "garage_rpc/system.rs/SystemRpc"; /// RPC messages related to membership #[derive(Debug, Serialize, Deserialize, Clone)] @@ -58,17 +58,17 @@ pub enum SystemRpc { Ok, /// Request to connect to a specific node (in @: format) Connect(String), - /// Ask other node its cluster layout. Answered with AdvertiseClusterLayout - PullClusterLayout, /// Advertise Garage status. Answered with another AdvertiseStatus. /// Exchanged with every node on a regular basis. AdvertiseStatus(NodeStatus), - /// Advertisement of cluster layout. Sent spontanously or in response to PullClusterLayout - AdvertiseClusterLayout(LayoutHistory), /// Get known nodes states GetKnownNodes, /// Return known nodes ReturnKnownNodes(Vec), + /// Ask other node its cluster layout. Answered with AdvertiseClusterLayout + PullClusterLayout, + /// Advertisement of cluster layout. Sent spontanously or in response to PullClusterLayout + AdvertiseClusterLayout(LayoutHistory), } impl Rpc for SystemRpc { @@ -84,7 +84,6 @@ pub struct System { /// The id of this node pub id: Uuid, - persist_cluster_layout: Persister, persist_peer_list: Persister, local_status: ArcSwap, @@ -92,9 +91,8 @@ pub struct System { pub netapp: Arc, fullmesh: Arc, - pub rpc: RpcHelper, - system_endpoint: Arc>, + pub(crate) system_endpoint: Arc>, rpc_listen_addr: SocketAddr, #[cfg(any(feature = "consul-discovery", feature = "kubernetes-discovery"))] @@ -106,15 +104,13 @@ pub struct System { #[cfg(feature = "kubernetes-discovery")] kubernetes_discovery: Option, + pub layout_manager: LayoutManager, + metrics: SystemMetrics, replication_mode: ReplicationMode, replication_factor: usize, - /// The layout - pub layout_watch: watch::Receiver>, - update_layout: Mutex>>, - /// Path to metadata directory pub metadata_dir: PathBuf, /// Path to data directory @@ -128,8 +124,11 @@ pub struct NodeStatus { /// Replication factor configured on the node pub replication_factor: usize, + /// Cluster layout version pub cluster_layout_version: u64, + /// Hash of cluster layout update trackers + // (TODO) pub cluster_layout_trackers_hash: Hash, /// Hash of cluster layout staging data pub cluster_layout_staging_hash: Hash, @@ -247,8 +246,7 @@ impl System { replication_mode: ReplicationMode, config: &Config, ) -> Result, Error> { - let replication_factor = replication_mode.replication_factor(); - + // ---- setup netapp RPC protocol ---- let node_key = gen_node_key(&config.metadata_dir).expect("Unable to read or generate node ID"); info!( @@ -256,81 +254,40 @@ impl System { hex::encode(&node_key.public_key()[..8]) ); - let persist_cluster_layout: Persister = - Persister::new(&config.metadata_dir, "cluster_layout"); - let persist_peer_list = Persister::new(&config.metadata_dir, "peer_list"); - - let cluster_layout = match persist_cluster_layout.load() { - Ok(x) => { - if x.current().replication_factor != replication_factor { - return Err(Error::Message(format!( - "Prevous cluster layout has replication factor {}, which is different than the one specified in the config file ({}). The previous cluster layout can be purged, if you know what you are doing, simply by deleting the `cluster_layout` file in your metadata directory.", - x.current().replication_factor, - replication_factor - ))); - } - x - } - Err(e) => { - info!( - "No valid previous cluster layout stored ({}), starting fresh.", - e - ); - LayoutHistory::new(replication_factor) - } - }; - - let metrics = SystemMetrics::new(replication_factor); - - let mut local_status = NodeStatus::initial(replication_factor, &cluster_layout); - local_status.update_disk_usage(&config.metadata_dir, &config.data_dir, &metrics); + let netapp = NetApp::new(GARAGE_VERSION_TAG, network_key, node_key); + let system_endpoint = netapp.endpoint(SYSTEM_RPC_PATH.into()); - let (update_layout, layout_watch) = watch::channel(Arc::new(cluster_layout)); - - let rpc_public_addr = match &config.rpc_public_addr { - Some(a_str) => { - use std::net::ToSocketAddrs; - match a_str.to_socket_addrs() { - Err(e) => { - error!( - "Cannot resolve rpc_public_addr {} from config file: {}.", - a_str, e - ); - None - } - Ok(a) => { - let a = a.collect::>(); - if a.is_empty() { - error!("rpc_public_addr {} resolve to no known IP address", a_str); - } - if a.len() > 1 { - warn!("Multiple possible resolutions for rpc_public_addr: {:?}. Taking the first one.", a); - } - a.into_iter().next() - } - } - } - None => { - let addr = - get_default_ip().map(|ip| SocketAddr::new(ip, config.rpc_bind_addr.port())); - if let Some(a) = addr { - warn!("Using autodetected rpc_public_addr: {}. Consider specifying it explicitly in configuration file if possible.", a); - } - addr - } - }; + // ---- setup netapp public listener and full mesh peering strategy ---- + let rpc_public_addr = get_rpc_public_addr(config); if rpc_public_addr.is_none() { warn!("This Garage node does not know its publicly reachable RPC address, this might hamper intra-cluster communication."); } - let netapp = NetApp::new(GARAGE_VERSION_TAG, network_key, node_key); let fullmesh = FullMeshPeeringStrategy::new(netapp.clone(), vec![], rpc_public_addr); if let Some(ping_timeout) = config.rpc_ping_timeout_msec { fullmesh.set_ping_timeout_millis(ping_timeout); } - let system_endpoint = netapp.endpoint(SYSTEM_RPC_PATH.into()); + let persist_peer_list = Persister::new(&config.metadata_dir, "peer_list"); + // ---- setup cluster layout and layout manager ---- + let replication_factor = replication_mode.replication_factor(); + + let layout_manager = LayoutManager::new( + config, + netapp.id, + system_endpoint.clone(), + fullmesh.clone(), + replication_factor, + )?; + + // ---- set up metrics and status exchange ---- + let metrics = SystemMetrics::new(replication_factor); + + let mut local_status = NodeStatus::initial(replication_factor, &layout_manager.history()); + local_status.update_disk_usage(&config.metadata_dir, &config.data_dir, &metrics); + + // ---- if enabled, set up additionnal peer discovery methods ---- #[cfg(feature = "consul-discovery")] let consul_discovery = match &config.consul_discovery { Some(cfg) => Some( @@ -349,20 +306,14 @@ impl System { warn!("Kubernetes discovery is not enabled in this build."); } + // ---- done ---- let sys = Arc::new(System { id: netapp.id.into(), - persist_cluster_layout, persist_peer_list, local_status: ArcSwap::new(Arc::new(local_status)), node_status: RwLock::new(HashMap::new()), netapp: netapp.clone(), fullmesh: fullmesh.clone(), - rpc: RpcHelper::new( - netapp.id.into(), - fullmesh, - layout_watch.clone(), - config.rpc_timeout_msec.map(Duration::from_millis), - ), system_endpoint, replication_mode, replication_factor, @@ -374,10 +325,9 @@ impl System { consul_discovery, #[cfg(feature = "kubernetes-discovery")] kubernetes_discovery: config.kubernetes_discovery.clone(), + layout_manager, metrics, - layout_watch, - update_layout: Mutex::new(update_layout), metadata_dir: config.metadata_dir.clone(), data_dir: config.data_dir.clone(), }); @@ -397,6 +347,20 @@ impl System { ); } + // ---- Public utilities / accessors ---- + + pub fn cluster_layout(&self) -> watch::Ref> { + self.layout_manager.history() + } + + pub fn layout_watch(&self) -> watch::Receiver> { + self.layout_manager.layout_watch.clone() + } + + pub fn rpc_helper(&self) -> &RpcHelper { + &self.layout_manager.rpc_helper + } + // ---- Administrative operations (directly available and // also available through RPC) ---- @@ -423,18 +387,6 @@ impl System { known_nodes } - pub fn cluster_layout(&self) -> watch::Ref> { - self.layout_watch.borrow() - } - - pub async fn update_cluster_layout( - self: &Arc, - layout: &LayoutHistory, - ) -> Result<(), Error> { - self.handle_advertise_cluster_layout(layout).await?; - Ok(()) - } - pub async fn connect(&self, node: &str) -> Result<(), Error> { let (pubkey, addrs) = parse_and_resolve_peer_addr_async(node) .await @@ -464,7 +416,7 @@ impl System { } pub fn health(&self) -> ClusterHealth { - let layout: Arc<_> = self.layout_watch.borrow().clone(); + let layout: Arc<_> = self.cluster_layout().clone(); let quorum = self.replication_mode.write_quorum(); let replication_factor = self.replication_factor; @@ -581,20 +533,10 @@ impl System { } } - /// Save network configuration to disc - async fn save_cluster_layout(&self) -> Result<(), Error> { - let layout: Arc = self.layout_watch.borrow().clone(); - self.persist_cluster_layout - .save_async(&layout) - .await - .expect("Cannot save current cluster layout"); - Ok(()) - } - fn update_local_status(&self) { let mut new_si: NodeStatus = self.local_status.load().as_ref().clone(); - let layout = self.layout_watch.borrow(); + let layout = self.cluster_layout(); new_si.cluster_layout_version = layout.current().version; new_si.cluster_layout_staging_hash = layout.staging_hash; @@ -610,11 +552,6 @@ impl System { Ok(SystemRpc::Ok) } - fn handle_pull_cluster_layout(&self) -> SystemRpc { - let layout = self.layout_watch.borrow().as_ref().clone(); - SystemRpc::AdvertiseClusterLayout(layout) - } - fn handle_get_known_nodes(&self) -> SystemRpc { let known_nodes = self.get_known_nodes(); SystemRpc::ReturnKnownNodes(known_nodes) @@ -637,7 +574,10 @@ impl System { if info.cluster_layout_version > local_info.cluster_layout_version || info.cluster_layout_staging_hash != local_info.cluster_layout_staging_hash { - tokio::spawn(self.clone().pull_cluster_layout(from)); + tokio::spawn({ + let system = self.clone(); + async move { system.layout_manager.pull_cluster_layout(from).await } + }); } self.node_status @@ -648,57 +588,6 @@ impl System { Ok(SystemRpc::Ok) } - async fn handle_advertise_cluster_layout( - self: &Arc, - adv: &LayoutHistory, - ) -> Result { - if adv.current().replication_factor != self.replication_factor { - let msg = format!( - "Received a cluster layout from another node with replication factor {}, which is different from what we have in our configuration ({}). Discarding the cluster layout we received.", - adv.current().replication_factor, - self.replication_factor - ); - error!("{}", msg); - return Err(Error::Message(msg)); - } - - let update_layout = self.update_layout.lock().await; - // TODO: don't clone each time an AdvertiseClusterLayout is received - let mut layout: LayoutHistory = self.layout_watch.borrow().as_ref().clone(); - - let prev_layout_check = layout.check().is_ok(); - if layout.merge(adv) { - if prev_layout_check && layout.check().is_err() { - error!("New cluster layout is invalid, discarding."); - return Err(Error::Message( - "New cluster layout is invalid, discarding.".into(), - )); - } - - update_layout.send(Arc::new(layout.clone()))?; - drop(update_layout); - - let self2 = self.clone(); - tokio::spawn(async move { - if let Err(e) = self2 - .rpc - .broadcast( - &self2.system_endpoint, - SystemRpc::AdvertiseClusterLayout(layout), - RequestStrategy::with_priority(PRIO_HIGH), - ) - .await - { - warn!("Error while broadcasting new cluster layout: {}", e); - } - }); - - self.save_cluster_layout().await?; - } - - Ok(SystemRpc::Ok) - } - async fn status_exchange_loop(&self, mut stop_signal: watch::Receiver) { while !*stop_signal.borrow() { let restart_at = Instant::now() + STATUS_EXCHANGE_INTERVAL; @@ -706,7 +595,7 @@ impl System { self.update_local_status(); let local_status: NodeStatus = self.local_status.load().as_ref().clone(); let _ = self - .rpc + .rpc_helper() .broadcast( &self.system_endpoint, SystemRpc::AdvertiseStatus(local_status), @@ -724,9 +613,9 @@ impl System { async fn discovery_loop(self: &Arc, mut stop_signal: watch::Receiver) { while !*stop_signal.borrow() { - let not_configured = self.layout_watch.borrow().check().is_err(); + let not_configured = self.cluster_layout().check().is_err(); let no_peers = self.fullmesh.get_peer_list().len() < self.replication_factor; - let expected_n_nodes = self.layout_watch.borrow().current().num_nodes(); + let expected_n_nodes = self.cluster_layout().current().num_nodes(); let bad_peers = self .fullmesh .get_peer_list() @@ -831,34 +720,26 @@ impl System { .save_async(&PeerList(peer_list)) .await } - - async fn pull_cluster_layout(self: Arc, peer: Uuid) { - let resp = self - .rpc - .call( - &self.system_endpoint, - peer, - SystemRpc::PullClusterLayout, - RequestStrategy::with_priority(PRIO_HIGH), - ) - .await; - if let Ok(SystemRpc::AdvertiseClusterLayout(layout)) = resp { - let _: Result<_, _> = self.handle_advertise_cluster_layout(&layout).await; - } - } } #[async_trait] impl EndpointHandler for System { async fn handle(self: &Arc, msg: &SystemRpc, from: NodeID) -> Result { match msg { + // ---- system functions -> System ---- SystemRpc::Connect(node) => self.handle_connect(node).await, - SystemRpc::PullClusterLayout => Ok(self.handle_pull_cluster_layout()), SystemRpc::AdvertiseStatus(adv) => self.handle_advertise_status(from.into(), adv).await, + SystemRpc::GetKnownNodes => Ok(self.handle_get_known_nodes()), + + // ---- layout functions -> LayoutManager ---- + SystemRpc::PullClusterLayout => Ok(self.layout_manager.handle_pull_cluster_layout()), SystemRpc::AdvertiseClusterLayout(adv) => { - self.clone().handle_advertise_cluster_layout(adv).await + self.layout_manager + .handle_advertise_cluster_layout(adv) + .await } - SystemRpc::GetKnownNodes => Ok(self.handle_get_known_nodes()), + + // ---- other -> Error ---- m => Err(Error::unexpected_rpc_message(m)), } } @@ -962,6 +843,40 @@ fn get_default_ip() -> Option { .map(|a| a.ip()) } +fn get_rpc_public_addr(config: &Config) -> Option { + match &config.rpc_public_addr { + Some(a_str) => { + use std::net::ToSocketAddrs; + match a_str.to_socket_addrs() { + Err(e) => { + error!( + "Cannot resolve rpc_public_addr {} from config file: {}.", + a_str, e + ); + None + } + Ok(a) => { + let a = a.collect::>(); + if a.is_empty() { + error!("rpc_public_addr {} resolve to no known IP address", a_str); + } + if a.len() > 1 { + warn!("Multiple possible resolutions for rpc_public_addr: {:?}. Taking the first one.", a); + } + a.into_iter().next() + } + } + } + None => { + let addr = get_default_ip().map(|ip| SocketAddr::new(ip, config.rpc_bind_addr.port())); + if let Some(a) = addr { + warn!("Using autodetected rpc_public_addr: {}. Consider specifying it explicitly in configuration file if possible.", a); + } + addr + } + } +} + async fn resolve_peers(peers: &[String]) -> Vec<(NodeID, SocketAddr)> { let mut ret = vec![]; diff --git a/src/table/gc.rs b/src/table/gc.rs index 5b9124a7..2135a358 100644 --- a/src/table/gc.rs +++ b/src/table/gc.rs @@ -227,7 +227,7 @@ impl TableGc { // GC'ing is not a critical function of the system, so it's not a big // deal if we can't do it right now. self.system - .rpc + .rpc_helper() .try_call_many( &self.endpoint, &nodes[..], @@ -248,7 +248,7 @@ impl TableGc { // it means that the garbage collection wasn't completed and has // to be retried later. self.system - .rpc + .rpc_helper() .try_call_many( &self.endpoint, &nodes[..], diff --git a/src/table/sync.rs b/src/table/sync.rs index 620d83b9..2da1bfe7 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -91,7 +91,7 @@ impl TableSyncer { bg.spawn_worker(SyncWorker { syncer: self.clone(), - layout_watch: self.system.layout_watch.clone(), + layout_watch: self.system.layout_watch(), layout: self.system.cluster_layout().clone(), add_full_sync_rx, todo: vec![], @@ -244,7 +244,7 @@ impl TableSyncer { } self.system - .rpc + .rpc_helper() .try_call_many( &self.endpoint, nodes, @@ -305,7 +305,7 @@ impl TableSyncer { // If so, do nothing. let root_resp = self .system - .rpc + .rpc_helper() .call( &self.endpoint, who, @@ -361,7 +361,7 @@ impl TableSyncer { // and compare it with local node let remote_node = match self .system - .rpc + .rpc_helper() .call( &self.endpoint, who, @@ -437,7 +437,7 @@ impl TableSyncer { let rpc_resp = self .system - .rpc + .rpc_helper() .call( &self.endpoint, who, diff --git a/src/table/table.rs b/src/table/table.rs index 7ad79677..3e3fd138 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -123,7 +123,7 @@ impl Table { let rpc = TableRpc::::Update(vec![e_enc]); self.system - .rpc + .rpc_helper() .try_call_many( &self.endpoint, &who[..], @@ -181,7 +181,7 @@ impl Table { let resp = self .system - .rpc + .rpc_helper() .call( &self.endpoint, node, @@ -236,7 +236,7 @@ impl Table { let rpc = TableRpc::::ReadEntry(partition_key.clone(), sort_key.clone()); let resps = self .system - .rpc + .rpc_helper() .try_call_many( &self.endpoint, &who[..], @@ -332,7 +332,7 @@ impl Table { let resps = self .system - .rpc + .rpc_helper() .try_call_many( &self.endpoint, &who[..], @@ -411,7 +411,7 @@ impl Table { async fn repair_on_read(&self, who: &[Uuid], what: F::E) -> Result<(), Error> { let what_enc = Arc::new(ByteBuf::from(what.encode()?)); self.system - .rpc + .rpc_helper() .try_call_many( &self.endpoint, who, -- cgit v1.2.3 From 19ef1ec8e7fee3a6c670e6e35dfcc83f0801e604 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 9 Nov 2023 13:34:14 +0100 Subject: layout: more refactoring --- src/garage/cli/layout.rs | 6 ++- src/rpc/layout/manager.rs | 116 ++++++++++++++++++++++++++++++---------------- src/rpc/layout/schema.rs | 6 +-- src/rpc/system.rs | 40 ++++++---------- 4 files changed, 97 insertions(+), 71 deletions(-) diff --git a/src/garage/cli/layout.rs b/src/garage/cli/layout.rs index 269d92f4..bffc81d3 100644 --- a/src/garage/cli/layout.rs +++ b/src/garage/cli/layout.rs @@ -1,3 +1,5 @@ +use std::sync::Arc; + use bytesize::ByteSize; use format_table::format_table; @@ -321,7 +323,7 @@ pub async fn fetch_layout( .call(&rpc_host, SystemRpc::PullClusterLayout, PRIO_NORMAL) .await?? { - SystemRpc::AdvertiseClusterLayout(t) => Ok(t), + SystemRpc::AdvertiseClusterLayout(t) => Ok(Arc::try_unwrap(t).unwrap()), resp => Err(Error::Message(format!("Invalid RPC response: {:?}", resp))), } } @@ -334,7 +336,7 @@ pub async fn send_layout( rpc_cli .call( &rpc_host, - SystemRpc::AdvertiseClusterLayout(layout), + SystemRpc::AdvertiseClusterLayout(Arc::new(layout)), PRIO_NORMAL, ) .await??; diff --git a/src/rpc/layout/manager.rs b/src/rpc/layout/manager.rs index a8a77139..351e0959 100644 --- a/src/rpc/layout/manager.rs +++ b/src/rpc/layout/manager.rs @@ -1,6 +1,8 @@ use std::sync::Arc; use std::time::Duration; +use serde::{Deserialize, Serialize}; + use tokio::sync::watch; use tokio::sync::Mutex; @@ -28,6 +30,16 @@ pub struct LayoutManager { system_endpoint: Arc>, } +#[derive(Debug, Clone, Serialize, Deserialize, Default)] +pub struct LayoutStatus { + /// Cluster layout version + pub cluster_layout_version: u64, + /// Hash of cluster layout update trackers + // (TODO) pub cluster_layout_trackers_hash: Hash, + /// Hash of cluster layout staging data + pub cluster_layout_staging_hash: Hash, +} + impl LayoutManager { pub fn new( config: &Config, @@ -35,7 +47,7 @@ impl LayoutManager { system_endpoint: Arc>, fullmesh: Arc, replication_factor: usize, - ) -> Result { + ) -> Result, Error> { let persist_cluster_layout: Persister = Persister::new(&config.metadata_dir, "cluster_layout"); @@ -68,28 +80,39 @@ impl LayoutManager { config.rpc_timeout_msec.map(Duration::from_millis), ); - Ok(Self { + Ok(Arc::new(Self { replication_factor, persist_cluster_layout, layout_watch, update_layout: Mutex::new(update_layout), system_endpoint, rpc_helper, - }) + })) } // ---- PUBLIC INTERFACE ---- - pub async fn update_cluster_layout(&self, layout: &LayoutHistory) -> Result<(), Error> { + pub fn status(&self) -> LayoutStatus { + let layout = self.layout(); + LayoutStatus { + cluster_layout_version: layout.current().version, + cluster_layout_staging_hash: layout.staging_hash, + } + } + + pub async fn update_cluster_layout( + self: &Arc, + layout: &LayoutHistory, + ) -> Result<(), Error> { self.handle_advertise_cluster_layout(layout).await?; Ok(()) } - pub fn history(&self) -> watch::Ref> { + pub fn layout(&self) -> watch::Ref> { self.layout_watch.borrow() } - pub(crate) async fn pull_cluster_layout(&self, peer: Uuid) { + pub(crate) async fn pull_cluster_layout(self: &Arc, peer: Uuid) { let resp = self .rpc_helper .call( @@ -118,13 +141,25 @@ impl LayoutManager { // ---- RPC HANDLERS ---- + pub(crate) fn handle_advertise_status(self: &Arc, from: Uuid, status: &LayoutStatus) { + let local_status = self.status(); + if status.cluster_layout_version > local_status.cluster_layout_version + || status.cluster_layout_staging_hash != local_status.cluster_layout_staging_hash + { + tokio::spawn({ + let this = self.clone(); + async move { this.pull_cluster_layout(from).await } + }); + } + } + pub(crate) fn handle_pull_cluster_layout(&self) -> SystemRpc { - let layout = self.layout_watch.borrow().as_ref().clone(); + let layout = self.layout_watch.borrow().clone(); SystemRpc::AdvertiseClusterLayout(layout) } pub(crate) async fn handle_advertise_cluster_layout( - &self, + self: &Arc, adv: &LayoutHistory, ) -> Result { if adv.current().replication_factor != self.replication_factor { @@ -137,39 +172,42 @@ impl LayoutManager { return Err(Error::Message(msg)); } - let update_layout = self.update_layout.lock().await; - // TODO: don't clone each time an AdvertiseClusterLayout is received - let mut layout: LayoutHistory = self.layout_watch.borrow().as_ref().clone(); - - let prev_layout_check = layout.check().is_ok(); - if layout.merge(adv) { - if prev_layout_check && layout.check().is_err() { - error!("New cluster layout is invalid, discarding."); - return Err(Error::Message( - "New cluster layout is invalid, discarding.".into(), - )); - } - - update_layout.send(Arc::new(layout.clone()))?; - drop(update_layout); - - /* TODO - tokio::spawn(async move { - if let Err(e) = system - .rpc_helper() - .broadcast( - &system.system_endpoint, - SystemRpc::AdvertiseClusterLayout(layout), - RequestStrategy::with_priority(PRIO_HIGH), - ) - .await - { - warn!("Error while broadcasting new cluster layout: {}", e); + if *adv != **self.layout_watch.borrow() { + let update_layout = self.update_layout.lock().await; + let mut layout: LayoutHistory = self.layout_watch.borrow().as_ref().clone(); + + let prev_layout_check = layout.check().is_ok(); + if layout.merge(adv) { + if prev_layout_check && layout.check().is_err() { + error!("New cluster layout is invalid, discarding."); + return Err(Error::Message( + "New cluster layout is invalid, discarding.".into(), + )); } - }); - */ - self.save_cluster_layout().await?; + let layout = Arc::new(layout); + update_layout.send(layout.clone())?; + drop(update_layout); // release mutex + + tokio::spawn({ + let this = self.clone(); + async move { + if let Err(e) = this + .rpc_helper + .broadcast( + &this.system_endpoint, + SystemRpc::AdvertiseClusterLayout(layout), + RequestStrategy::with_priority(PRIO_HIGH), + ) + .await + { + warn!("Error while broadcasting new cluster layout: {}", e); + } + } + }); + + self.save_cluster_layout().await?; + } } Ok(SystemRpc::Ok) diff --git a/src/rpc/layout/schema.rs b/src/rpc/layout/schema.rs index c5b9b1d3..d587a6cb 100644 --- a/src/rpc/layout/schema.rs +++ b/src/rpc/layout/schema.rs @@ -226,7 +226,7 @@ mod v010 { } /// The history of cluster layouts - #[derive(Clone, Debug, Serialize, Deserialize)] + #[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] pub struct LayoutHistory { /// The versions currently in use in the cluster pub versions: Vec, @@ -241,7 +241,7 @@ mod v010 { } /// The tracker of acknowlegments and data syncs around the cluster - #[derive(Clone, Debug, Serialize, Deserialize, Default)] + #[derive(Clone, Debug, Serialize, Deserialize, Default, PartialEq)] pub struct UpdateTrackers { /// The highest layout version number each node has ack'ed pub ack_map: UpdateTracker, @@ -253,7 +253,7 @@ mod v010 { } /// The history of cluster layouts - #[derive(Clone, Debug, Serialize, Deserialize, Default)] + #[derive(Clone, Debug, Serialize, Deserialize, Default, PartialEq)] pub struct UpdateTracker(pub HashMap); impl garage_util::migrate::Migrate for LayoutHistory { diff --git a/src/rpc/system.rs b/src/rpc/system.rs index a8e88425..88c4d443 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -33,7 +33,7 @@ use garage_util::time::*; use crate::consul::ConsulDiscovery; #[cfg(feature = "kubernetes-discovery")] use crate::kubernetes::*; -use crate::layout::manager::LayoutManager; +use crate::layout::manager::{LayoutManager, LayoutStatus}; use crate::layout::*; use crate::replication_mode::*; use crate::rpc_helper::*; @@ -68,7 +68,7 @@ pub enum SystemRpc { /// Ask other node its cluster layout. Answered with AdvertiseClusterLayout PullClusterLayout, /// Advertisement of cluster layout. Sent spontanously or in response to PullClusterLayout - AdvertiseClusterLayout(LayoutHistory), + AdvertiseClusterLayout(Arc), } impl Rpc for SystemRpc { @@ -104,7 +104,7 @@ pub struct System { #[cfg(feature = "kubernetes-discovery")] kubernetes_discovery: Option, - pub layout_manager: LayoutManager, + pub layout_manager: Arc, metrics: SystemMetrics, @@ -125,12 +125,8 @@ pub struct NodeStatus { /// Replication factor configured on the node pub replication_factor: usize, - /// Cluster layout version - pub cluster_layout_version: u64, - /// Hash of cluster layout update trackers - // (TODO) pub cluster_layout_trackers_hash: Hash, - /// Hash of cluster layout staging data - pub cluster_layout_staging_hash: Hash, + /// Layout status + pub layout_status: LayoutStatus, /// Disk usage on partition containing metadata directory (tuple: `(avail, total)`) #[serde(default)] @@ -284,7 +280,7 @@ impl System { // ---- set up metrics and status exchange ---- let metrics = SystemMetrics::new(replication_factor); - let mut local_status = NodeStatus::initial(replication_factor, &layout_manager.history()); + let mut local_status = NodeStatus::initial(replication_factor, &layout_manager); local_status.update_disk_usage(&config.metadata_dir, &config.data_dir, &metrics); // ---- if enabled, set up additionnal peer discovery methods ---- @@ -350,7 +346,7 @@ impl System { // ---- Public utilities / accessors ---- pub fn cluster_layout(&self) -> watch::Ref> { - self.layout_manager.history() + self.layout_manager.layout() } pub fn layout_watch(&self) -> watch::Receiver> { @@ -536,9 +532,7 @@ impl System { fn update_local_status(&self) { let mut new_si: NodeStatus = self.local_status.load().as_ref().clone(); - let layout = self.cluster_layout(); - new_si.cluster_layout_version = layout.current().version; - new_si.cluster_layout_staging_hash = layout.staging_hash; + new_si.layout_status = self.layout_manager.status(); new_si.update_disk_usage(&self.metadata_dir, &self.data_dir, &self.metrics); @@ -571,14 +565,8 @@ impl System { std::process::exit(1); } - if info.cluster_layout_version > local_info.cluster_layout_version - || info.cluster_layout_staging_hash != local_info.cluster_layout_staging_hash - { - tokio::spawn({ - let system = self.clone(); - async move { system.layout_manager.pull_cluster_layout(from).await } - }); - } + self.layout_manager + .handle_advertise_status(from, &info.layout_status); self.node_status .write() @@ -746,14 +734,13 @@ impl EndpointHandler for System { } impl NodeStatus { - fn initial(replication_factor: usize, layout: &LayoutHistory) -> Self { + fn initial(replication_factor: usize, layout_manager: &LayoutManager) -> Self { NodeStatus { hostname: gethostname::gethostname() .into_string() .unwrap_or_else(|_| "".to_string()), replication_factor, - cluster_layout_version: layout.current().version, - cluster_layout_staging_hash: layout.staging_hash, + layout_status: layout_manager.status(), meta_disk_avail: None, data_disk_avail: None, } @@ -763,8 +750,7 @@ impl NodeStatus { NodeStatus { hostname: "?".to_string(), replication_factor: 0, - cluster_layout_version: 0, - cluster_layout_staging_hash: Hash::from([0u8; 32]), + layout_status: Default::default(), meta_disk_avail: None, data_disk_avail: None, } -- cgit v1.2.3 From bfb1845fdc981a370539d641a5d80f438f184f07 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 9 Nov 2023 14:12:05 +0100 Subject: layout: refactor to use a RwLock on LayoutHistory --- src/api/admin/cluster.rs | 6 +-- src/api/k2v/index.rs | 11 ++---- src/garage/cli/layout.rs | 6 +-- src/model/helper/bucket.rs | 11 ++---- src/rpc/layout/manager.rs | 93 +++++++++++++++++++++++----------------------- src/rpc/rpc_helper.rs | 11 +++--- src/rpc/system.rs | 15 ++++---- src/table/sync.rs | 21 +++++------ 8 files changed, 82 insertions(+), 92 deletions(-) diff --git a/src/api/admin/cluster.rs b/src/api/admin/cluster.rs index f5483451..593bd778 100644 --- a/src/api/admin/cluster.rs +++ b/src/api/admin/cluster.rs @@ -210,7 +210,7 @@ pub async fn handle_update_cluster_layout( ) -> Result, Error> { let updates = parse_json_body::(req).await?; - let mut layout = garage.system.cluster_layout().as_ref().clone(); + let mut layout = garage.system.cluster_layout().clone(); let mut roles = layout.current().roles.clone(); roles.merge(&layout.staging.get().roles); @@ -256,7 +256,7 @@ pub async fn handle_apply_cluster_layout( ) -> Result, Error> { let param = parse_json_body::(req).await?; - let layout = garage.system.cluster_layout().as_ref().clone(); + let layout = garage.system.cluster_layout().clone(); let (layout, msg) = layout.apply_staged_changes(Some(param.version))?; garage @@ -273,7 +273,7 @@ pub async fn handle_apply_cluster_layout( } pub async fn handle_revert_cluster_layout(garage: &Arc) -> Result, Error> { - let layout = garage.system.cluster_layout().as_ref().clone(); + let layout = garage.system.cluster_layout().clone(); let layout = layout.revert_staged_changes()?; garage .system diff --git a/src/api/k2v/index.rs b/src/api/k2v/index.rs index a9bc3826..3c2f51a9 100644 --- a/src/api/k2v/index.rs +++ b/src/api/k2v/index.rs @@ -5,7 +5,6 @@ use serde::Serialize; use garage_util::data::*; -use garage_rpc::layout::LayoutHistory; use garage_table::util::*; use garage_model::garage::Garage; @@ -26,7 +25,8 @@ pub async fn handle_read_index( ) -> Result, Error> { let reverse = reverse.unwrap_or(false); - let layout: Arc = garage.system.cluster_layout().clone(); + // TODO: not only current + let node_id_vec = garage.system.cluster_layout().current().node_ids().to_vec(); let (partition_keys, more, next_start) = read_range( &garage.k2v.counter_table.table, @@ -35,10 +35,7 @@ pub async fn handle_read_index( &start, &end, limit, - Some(( - DeletedFilter::NotDeleted, - layout.current().node_id_vec.clone(), - )), + Some((DeletedFilter::NotDeleted, node_id_vec)), EnumerationOrder::from_reverse(reverse), ) .await?; @@ -57,7 +54,7 @@ pub async fn handle_read_index( partition_keys: partition_keys .into_iter() .map(|part| { - let vals = part.filtered_values(&layout); + let vals = part.filtered_values(&garage.system.cluster_layout()); ReadIndexResponseEntry { pk: part.sk, entries: *vals.get(&s_entries).unwrap_or(&0), diff --git a/src/garage/cli/layout.rs b/src/garage/cli/layout.rs index bffc81d3..269d92f4 100644 --- a/src/garage/cli/layout.rs +++ b/src/garage/cli/layout.rs @@ -1,5 +1,3 @@ -use std::sync::Arc; - use bytesize::ByteSize; use format_table::format_table; @@ -323,7 +321,7 @@ pub async fn fetch_layout( .call(&rpc_host, SystemRpc::PullClusterLayout, PRIO_NORMAL) .await?? { - SystemRpc::AdvertiseClusterLayout(t) => Ok(Arc::try_unwrap(t).unwrap()), + SystemRpc::AdvertiseClusterLayout(t) => Ok(t), resp => Err(Error::Message(format!("Invalid RPC response: {:?}", resp))), } } @@ -336,7 +334,7 @@ pub async fn send_layout( rpc_cli .call( &rpc_host, - SystemRpc::AdvertiseClusterLayout(Arc::new(layout)), + SystemRpc::AdvertiseClusterLayout(layout), PRIO_NORMAL, ) .await??; diff --git a/src/model/helper/bucket.rs b/src/model/helper/bucket.rs index 18904c8d..2a9c0fb1 100644 --- a/src/model/helper/bucket.rs +++ b/src/model/helper/bucket.rs @@ -450,10 +450,8 @@ impl<'a> BucketHelper<'a> { #[cfg(feature = "k2v")] { - use garage_rpc::layout::LayoutHistory; - use std::sync::Arc; - - let layout: Arc = self.0.system.cluster_layout().clone(); + // TODO: not only current + let node_id_vec = self.0.system.cluster_layout().current().node_ids().to_vec(); let k2vindexes = self .0 .k2v @@ -462,10 +460,7 @@ impl<'a> BucketHelper<'a> { .get_range( &bucket_id, None, - Some(( - DeletedFilter::NotDeleted, - layout.current().node_id_vec.clone(), - )), + Some((DeletedFilter::NotDeleted, node_id_vec)), 10, EnumerationOrder::Forward, ) diff --git a/src/rpc/layout/manager.rs b/src/rpc/layout/manager.rs index 351e0959..c021039b 100644 --- a/src/rpc/layout/manager.rs +++ b/src/rpc/layout/manager.rs @@ -1,10 +1,9 @@ -use std::sync::Arc; +use std::sync::{Arc, RwLock, RwLockReadGuard}; use std::time::Duration; use serde::{Deserialize, Serialize}; -use tokio::sync::watch; -use tokio::sync::Mutex; +use tokio::sync::Notify; use netapp::endpoint::Endpoint; use netapp::peering::fullmesh::FullMeshPeeringStrategy; @@ -23,8 +22,8 @@ pub struct LayoutManager { replication_factor: usize, persist_cluster_layout: Persister, - pub layout_watch: watch::Receiver>, - update_layout: Mutex>>, + layout: Arc>, + pub(crate) change_notify: Arc, pub(crate) rpc_helper: RpcHelper, system_endpoint: Arc>, @@ -71,20 +70,21 @@ impl LayoutManager { } }; - let (update_layout, layout_watch) = watch::channel(Arc::new(cluster_layout)); + let layout = Arc::new(RwLock::new(cluster_layout)); + let change_notify = Arc::new(Notify::new()); let rpc_helper = RpcHelper::new( node_id.into(), fullmesh, - layout_watch.clone(), + layout.clone(), config.rpc_timeout_msec.map(Duration::from_millis), ); Ok(Arc::new(Self { replication_factor, persist_cluster_layout, - layout_watch, - update_layout: Mutex::new(update_layout), + layout, + change_notify, system_endpoint, rpc_helper, })) @@ -108,8 +108,8 @@ impl LayoutManager { Ok(()) } - pub fn layout(&self) -> watch::Ref> { - self.layout_watch.borrow() + pub fn layout(&self) -> RwLockReadGuard<'_, LayoutHistory> { + self.layout.read().unwrap() } pub(crate) async fn pull_cluster_layout(self: &Arc, peer: Uuid) { @@ -131,7 +131,7 @@ impl LayoutManager { /// Save network configuration to disc async fn save_cluster_layout(&self) -> Result<(), Error> { - let layout: Arc = self.layout_watch.borrow().clone(); + let layout = self.layout.read().unwrap().clone(); // TODO: avoid cloning self.persist_cluster_layout .save_async(&layout) .await @@ -139,6 +139,22 @@ impl LayoutManager { Ok(()) } + fn merge_layout(&self, adv: &LayoutHistory) -> Option { + let mut layout = self.layout.write().unwrap(); + let prev_layout_check = layout.check().is_ok(); + + if !prev_layout_check || adv.check().is_ok() { + if layout.merge(adv) { + if prev_layout_check && layout.check().is_err() { + panic!("Merged two correct layouts and got an incorrect layout."); + } + + return Some(layout.clone()); + } + } + None + } + // ---- RPC HANDLERS ---- pub(crate) fn handle_advertise_status(self: &Arc, from: Uuid, status: &LayoutStatus) { @@ -154,7 +170,7 @@ impl LayoutManager { } pub(crate) fn handle_pull_cluster_layout(&self) -> SystemRpc { - let layout = self.layout_watch.borrow().clone(); + let layout = self.layout.read().unwrap().clone(); // TODO: avoid cloning SystemRpc::AdvertiseClusterLayout(layout) } @@ -172,42 +188,27 @@ impl LayoutManager { return Err(Error::Message(msg)); } - if *adv != **self.layout_watch.borrow() { - let update_layout = self.update_layout.lock().await; - let mut layout: LayoutHistory = self.layout_watch.borrow().as_ref().clone(); + if let Some(new_layout) = self.merge_layout(adv) { + self.change_notify.notify_waiters(); - let prev_layout_check = layout.check().is_ok(); - if layout.merge(adv) { - if prev_layout_check && layout.check().is_err() { - error!("New cluster layout is invalid, discarding."); - return Err(Error::Message( - "New cluster layout is invalid, discarding.".into(), - )); - } - - let layout = Arc::new(layout); - update_layout.send(layout.clone())?; - drop(update_layout); // release mutex - - tokio::spawn({ - let this = self.clone(); - async move { - if let Err(e) = this - .rpc_helper - .broadcast( - &this.system_endpoint, - SystemRpc::AdvertiseClusterLayout(layout), - RequestStrategy::with_priority(PRIO_HIGH), - ) - .await - { - warn!("Error while broadcasting new cluster layout: {}", e); - } + tokio::spawn({ + let this = self.clone(); + async move { + if let Err(e) = this + .rpc_helper + .broadcast( + &this.system_endpoint, + SystemRpc::AdvertiseClusterLayout(new_layout), + RequestStrategy::with_priority(PRIO_HIGH), + ) + .await + { + warn!("Error while broadcasting new cluster layout: {}", e); } - }); + } + }); - self.save_cluster_layout().await?; - } + self.save_cluster_layout().await?; } Ok(SystemRpc::Ok) diff --git a/src/rpc/rpc_helper.rs b/src/rpc/rpc_helper.rs index 3fdb4acd..ce291068 100644 --- a/src/rpc/rpc_helper.rs +++ b/src/rpc/rpc_helper.rs @@ -1,12 +1,11 @@ //! Contain structs related to making RPCs -use std::sync::Arc; +use std::sync::{Arc, RwLock}; use std::time::Duration; use futures::future::join_all; use futures::stream::futures_unordered::FuturesUnordered; use futures::stream::StreamExt; use tokio::select; -use tokio::sync::watch; use opentelemetry::KeyValue; use opentelemetry::{ @@ -91,7 +90,7 @@ pub struct RpcHelper(Arc); struct RpcHelperInner { our_node_id: Uuid, fullmesh: Arc, - layout_watch: watch::Receiver>, + layout: Arc>, metrics: RpcMetrics, rpc_timeout: Duration, } @@ -100,7 +99,7 @@ impl RpcHelper { pub(crate) fn new( our_node_id: Uuid, fullmesh: Arc, - layout_watch: watch::Receiver>, + layout: Arc>, rpc_timeout: Option, ) -> Self { let metrics = RpcMetrics::new(); @@ -108,7 +107,7 @@ impl RpcHelper { Self(Arc::new(RpcHelperInner { our_node_id, fullmesh, - layout_watch, + layout, metrics, rpc_timeout: rpc_timeout.unwrap_or(DEFAULT_TIMEOUT), })) @@ -392,7 +391,7 @@ impl RpcHelper { pub fn request_order(&self, nodes: &[Uuid]) -> Vec { // Retrieve some status variables that we will use to sort requests let peer_list = self.0.fullmesh.get_peer_list(); - let layout: Arc = self.0.layout_watch.borrow().clone(); + let layout = self.0.layout.read().unwrap(); let our_zone = match layout.current().node_role(&self.0.our_node_id) { Some(pc) => &pc.zone, None => "", diff --git a/src/rpc/system.rs b/src/rpc/system.rs index 88c4d443..cb3af3fe 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -4,7 +4,7 @@ use std::io::{Read, Write}; use std::net::{IpAddr, SocketAddr}; use std::path::{Path, PathBuf}; use std::sync::atomic::Ordering; -use std::sync::{Arc, RwLock}; +use std::sync::{Arc, RwLock, RwLockReadGuard}; use std::time::{Duration, Instant}; use arc_swap::ArcSwap; @@ -13,7 +13,7 @@ use futures::join; use serde::{Deserialize, Serialize}; use sodiumoxide::crypto::sign::ed25519; use tokio::select; -use tokio::sync::watch; +use tokio::sync::{watch, Notify}; use netapp::endpoint::{Endpoint, EndpointHandler}; use netapp::message::*; @@ -68,7 +68,7 @@ pub enum SystemRpc { /// Ask other node its cluster layout. Answered with AdvertiseClusterLayout PullClusterLayout, /// Advertisement of cluster layout. Sent spontanously or in response to PullClusterLayout - AdvertiseClusterLayout(Arc), + AdvertiseClusterLayout(LayoutHistory), } impl Rpc for SystemRpc { @@ -345,12 +345,12 @@ impl System { // ---- Public utilities / accessors ---- - pub fn cluster_layout(&self) -> watch::Ref> { + pub fn cluster_layout(&self) -> RwLockReadGuard<'_, LayoutHistory> { self.layout_manager.layout() } - pub fn layout_watch(&self) -> watch::Receiver> { - self.layout_manager.layout_watch.clone() + pub fn layout_notify(&self) -> Arc { + self.layout_manager.change_notify.clone() } pub fn rpc_helper(&self) -> &RpcHelper { @@ -412,7 +412,6 @@ impl System { } pub fn health(&self) -> ClusterHealth { - let layout: Arc<_> = self.cluster_layout().clone(); let quorum = self.replication_mode.write_quorum(); let replication_factor = self.replication_factor; @@ -423,6 +422,8 @@ impl System { .collect::>(); let connected_nodes = nodes.iter().filter(|(_, n)| n.is_up).count(); + let layout = self.cluster_layout(); // acquires a rwlock + // TODO: not only layout.current() let storage_nodes = layout .current() diff --git a/src/table/sync.rs b/src/table/sync.rs index 2da1bfe7..4355bd9e 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -10,7 +10,7 @@ use rand::Rng; use serde::{Deserialize, Serialize}; use serde_bytes::ByteBuf; use tokio::select; -use tokio::sync::{mpsc, watch}; +use tokio::sync::{mpsc, watch, Notify}; use garage_util::background::*; use garage_util::data::*; @@ -91,8 +91,8 @@ impl TableSyncer { bg.spawn_worker(SyncWorker { syncer: self.clone(), - layout_watch: self.system.layout_watch(), - layout: self.system.cluster_layout().clone(), + layout_notify: self.system.layout_notify(), + layout_version: self.system.cluster_layout().current().version, add_full_sync_rx, todo: vec![], next_full_sync: Instant::now() + Duration::from_secs(20), @@ -492,8 +492,8 @@ impl EndpointHandler for TableSync struct SyncWorker { syncer: Arc>, - layout_watch: watch::Receiver>, - layout: Arc, + layout_notify: Arc, + layout_version: u64, add_full_sync_rx: mpsc::UnboundedReceiver<()>, todo: Vec, next_full_sync: Instant, @@ -593,12 +593,11 @@ impl Worker for SyncWorker { self.add_full_sync(); } }, - _ = self.layout_watch.changed() => { - let new_layout = self.layout_watch.borrow(); - if !Arc::ptr_eq(&new_layout, &self.layout) { - self.layout = new_layout.clone(); - drop(new_layout); - debug!("({}) Ring changed, adding full sync to syncer todo list", F::TABLE_NAME); + _ = self.layout_notify.notified() => { + let new_version = self.syncer.system.cluster_layout().current().version; + if new_version > self.layout_version { + self.layout_version = new_version; + debug!("({}) Layout changed, adding full sync to syncer todo list", F::TABLE_NAME); self.add_full_sync(); } }, -- cgit v1.2.3 From 94caf9c0c1342ce1d2ba3ac7af39fb133721ee83 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 9 Nov 2023 14:53:34 +0100 Subject: layout: separate code path for synchronizing update trackers only --- src/rpc/layout/history.rs | 51 ++++++++++++----- src/rpc/layout/manager.rs | 140 +++++++++++++++++++++++++++++++++------------- src/rpc/layout/schema.rs | 23 ++++++-- src/rpc/system.rs | 15 ++++- 4 files changed, 168 insertions(+), 61 deletions(-) diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index 9ae28887..357b9d62 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -18,10 +18,11 @@ impl LayoutHistory { let mut ret = LayoutHistory { versions: vec![version].into_boxed_slice().into(), update_trackers: Default::default(), + trackers_hash: [0u8; 32].into(), staging: Lww::raw(0, staging), staging_hash: [0u8; 32].into(), }; - ret.staging_hash = ret.calculate_staging_hash(); + ret.update_hashes(); ret } @@ -29,6 +30,15 @@ impl LayoutHistory { self.versions.last().as_ref().unwrap() } + pub(crate) fn update_hashes(&mut self) { + self.trackers_hash = self.calculate_trackers_hash(); + self.staging_hash = self.calculate_staging_hash(); + } + + pub(crate) fn calculate_trackers_hash(&self) -> Hash { + blake2sum(&nonversioned_encode(&self.update_trackers).unwrap()[..]) + } + pub(crate) fn calculate_staging_hash(&self) -> Hash { blake2sum(&nonversioned_encode(&self.staging).unwrap()[..]) } @@ -38,12 +48,6 @@ impl LayoutHistory { pub fn merge(&mut self, other: &LayoutHistory) -> bool { let mut changed = false; - // Merge staged layout changes - if self.staging != other.staging { - changed = true; - } - self.staging.merge(&other.staging); - // Add any new versions to history for v2 in other.versions.iter() { if let Some(v1) = self.versions.iter().find(|v| v.version == v2.version) { @@ -63,7 +67,21 @@ impl LayoutHistory { } // Merge trackers - self.update_trackers.merge(&other.update_trackers); + if self.update_trackers != other.update_trackers { + let c = self.update_trackers.merge(&other.update_trackers); + changed = changed || c; + } + + // Merge staged layout changes + if self.staging != other.staging { + self.staging.merge(&other.staging); + changed = true; + } + + // Update hashes if there are changes + if changed { + self.update_hashes(); + } changed } @@ -100,7 +118,7 @@ To know the correct value of the new layout version, invoke `garage layout show` parameters: self.staging.get().parameters.clone(), roles: LwwMap::new(), }); - self.staging_hash = self.calculate_staging_hash(); + self.update_hashes(); Ok((self, msg)) } @@ -110,20 +128,25 @@ To know the correct value of the new layout version, invoke `garage layout show` parameters: Lww::new(self.current().parameters.clone()), roles: LwwMap::new(), }); - self.staging_hash = self.calculate_staging_hash(); + self.update_hashes(); Ok(self) } pub fn check(&self) -> Result<(), String> { // Check that the hash of the staging data is correct - let staging_hash = self.calculate_staging_hash(); - if staging_hash != self.staging_hash { + if self.trackers_hash != self.calculate_trackers_hash() { + return Err("trackers_hash is incorrect".into()); + } + if self.staging_hash != self.calculate_staging_hash() { return Err("staging_hash is incorrect".into()); } - // TODO: anythign more ? + for version in self.versions.iter() { + version.check()?; + } - self.current().check() + // TODO: anythign more ? + Ok(()) } } diff --git a/src/rpc/layout/manager.rs b/src/rpc/layout/manager.rs index c021039b..a2502f58 100644 --- a/src/rpc/layout/manager.rs +++ b/src/rpc/layout/manager.rs @@ -19,6 +19,7 @@ use crate::rpc_helper::*; use crate::system::*; pub struct LayoutManager { + node_id: Uuid, replication_factor: usize, persist_cluster_layout: Persister, @@ -34,7 +35,7 @@ pub struct LayoutStatus { /// Cluster layout version pub cluster_layout_version: u64, /// Hash of cluster layout update trackers - // (TODO) pub cluster_layout_trackers_hash: Hash, + pub cluster_layout_trackers_hash: Hash, /// Hash of cluster layout staging data pub cluster_layout_staging_hash: Hash, } @@ -81,6 +82,7 @@ impl LayoutManager { ); Ok(Arc::new(Self { + node_id: node_id.into(), replication_factor, persist_cluster_layout, layout, @@ -92,10 +94,15 @@ impl LayoutManager { // ---- PUBLIC INTERFACE ---- + pub fn layout(&self) -> RwLockReadGuard<'_, LayoutHistory> { + self.layout.read().unwrap() + } + pub fn status(&self) -> LayoutStatus { let layout = self.layout(); LayoutStatus { cluster_layout_version: layout.current().version, + cluster_layout_trackers_hash: layout.trackers_hash, cluster_layout_staging_hash: layout.staging_hash, } } @@ -108,11 +115,35 @@ impl LayoutManager { Ok(()) } - pub fn layout(&self) -> RwLockReadGuard<'_, LayoutHistory> { - self.layout.read().unwrap() + // ---- INTERNALS --- + + fn merge_layout(&self, adv: &LayoutHistory) -> Option { + let mut layout = self.layout.write().unwrap(); + let prev_layout_check = layout.check().is_ok(); + + if !prev_layout_check || adv.check().is_ok() { + if layout.merge(adv) { + if prev_layout_check && layout.check().is_err() { + panic!("Merged two correct layouts and got an incorrect layout."); + } + + return Some(layout.clone()); + } + } + None } - pub(crate) async fn pull_cluster_layout(self: &Arc, peer: Uuid) { + fn merge_layout_trackers(&self, adv: &UpdateTrackers) -> Option { + let mut layout = self.layout.write().unwrap(); + if layout.update_trackers != *adv { + if layout.update_trackers.merge(adv) { + return Some(layout.update_trackers.clone()); + } + } + None + } + + async fn pull_cluster_layout(self: &Arc, peer: Uuid) { let resp = self .rpc_helper .call( @@ -123,15 +154,35 @@ impl LayoutManager { ) .await; if let Ok(SystemRpc::AdvertiseClusterLayout(layout)) = resp { - let _: Result<_, _> = self.handle_advertise_cluster_layout(&layout).await; + if let Err(e) = self.handle_advertise_cluster_layout(&layout).await { + warn!("In pull_cluster_layout: {}", e); + } } } - // ---- INTERNALS --- + async fn pull_cluster_layout_trackers(self: &Arc, peer: Uuid) { + let resp = self + .rpc_helper + .call( + &self.system_endpoint, + peer, + SystemRpc::PullClusterLayoutTrackers, + RequestStrategy::with_priority(PRIO_HIGH), + ) + .await; + if let Ok(SystemRpc::AdvertiseClusterLayoutTrackers(trackers)) = resp { + if let Err(e) = self + .handle_advertise_cluster_layout_trackers(&trackers) + .await + { + warn!("In pull_cluster_layout_trackers: {}", e); + } + } + } - /// Save network configuration to disc + /// Save cluster layout data to disk async fn save_cluster_layout(&self) -> Result<(), Error> { - let layout = self.layout.read().unwrap().clone(); // TODO: avoid cloning + let layout = self.layout.read().unwrap().clone(); self.persist_cluster_layout .save_async(&layout) .await @@ -139,33 +190,41 @@ impl LayoutManager { Ok(()) } - fn merge_layout(&self, adv: &LayoutHistory) -> Option { - let mut layout = self.layout.write().unwrap(); - let prev_layout_check = layout.check().is_ok(); - - if !prev_layout_check || adv.check().is_ok() { - if layout.merge(adv) { - if prev_layout_check && layout.check().is_err() { - panic!("Merged two correct layouts and got an incorrect layout."); + fn broadcast_update(self: &Arc, rpc: SystemRpc) { + tokio::spawn({ + let this = self.clone(); + async move { + if let Err(e) = this + .rpc_helper + .broadcast( + &this.system_endpoint, + rpc, + RequestStrategy::with_priority(PRIO_HIGH), + ) + .await + { + warn!("Error while broadcasting new cluster layout: {}", e); } - - return Some(layout.clone()); } - } - None + }); } // ---- RPC HANDLERS ---- - pub(crate) fn handle_advertise_status(self: &Arc, from: Uuid, status: &LayoutStatus) { - let local_status = self.status(); - if status.cluster_layout_version > local_status.cluster_layout_version - || status.cluster_layout_staging_hash != local_status.cluster_layout_staging_hash + pub(crate) fn handle_advertise_status(self: &Arc, from: Uuid, remote: &LayoutStatus) { + let local = self.status(); + if remote.cluster_layout_version > local.cluster_layout_version + || remote.cluster_layout_staging_hash != local.cluster_layout_staging_hash { tokio::spawn({ let this = self.clone(); async move { this.pull_cluster_layout(from).await } }); + } else if remote.cluster_layout_trackers_hash != local.cluster_layout_trackers_hash { + tokio::spawn({ + let this = self.clone(); + async move { this.pull_cluster_layout_trackers(from).await } + }); } } @@ -174,6 +233,11 @@ impl LayoutManager { SystemRpc::AdvertiseClusterLayout(layout) } + pub(crate) fn handle_pull_cluster_layout_trackers(&self) -> SystemRpc { + let layout = self.layout.read().unwrap(); + SystemRpc::AdvertiseClusterLayoutTrackers(layout.update_trackers.clone()) + } + pub(crate) async fn handle_advertise_cluster_layout( self: &Arc, adv: &LayoutHistory, @@ -190,24 +254,20 @@ impl LayoutManager { if let Some(new_layout) = self.merge_layout(adv) { self.change_notify.notify_waiters(); + self.broadcast_update(SystemRpc::AdvertiseClusterLayout(new_layout)); + self.save_cluster_layout().await?; + } - tokio::spawn({ - let this = self.clone(); - async move { - if let Err(e) = this - .rpc_helper - .broadcast( - &this.system_endpoint, - SystemRpc::AdvertiseClusterLayout(new_layout), - RequestStrategy::with_priority(PRIO_HIGH), - ) - .await - { - warn!("Error while broadcasting new cluster layout: {}", e); - } - } - }); + Ok(SystemRpc::Ok) + } + pub(crate) async fn handle_advertise_cluster_layout_trackers( + self: &Arc, + trackers: &UpdateTrackers, + ) -> Result { + if let Some(new_trackers) = self.merge_layout_trackers(trackers) { + self.change_notify.notify_waiters(); + self.broadcast_update(SystemRpc::AdvertiseClusterLayoutTrackers(new_trackers)); self.save_cluster_layout().await?; } diff --git a/src/rpc/layout/schema.rs b/src/rpc/layout/schema.rs index d587a6cb..abae5bd8 100644 --- a/src/rpc/layout/schema.rs +++ b/src/rpc/layout/schema.rs @@ -233,6 +233,8 @@ mod v010 { /// Update trackers pub update_trackers: UpdateTrackers, + /// Hash of the update trackers + pub trackers_hash: Hash, /// Staged changes for the next version pub staging: Lww, @@ -289,10 +291,12 @@ mod v010 { sync_map: update_tracker.clone(), sync_ack_map: update_tracker.clone(), }, + trackers_hash: [0u8; 32].into(), staging: Lww::raw(previous.version, staging), staging_hash: [0u8; 32].into(), }; ret.staging_hash = ret.calculate_staging_hash(); + ret.trackers_hash = ret.calculate_trackers_hash(); ret } } @@ -355,14 +359,20 @@ impl core::str::FromStr for ZoneRedundancy { } impl UpdateTracker { - fn merge(&mut self, other: &UpdateTracker) { + fn merge(&mut self, other: &UpdateTracker) -> bool { + let mut changed = false; for (k, v) in other.0.iter() { if let Some(v_mut) = self.0.get_mut(k) { - *v_mut = std::cmp::max(*v_mut, *v); + if *v > *v_mut { + *v_mut = *v; + changed = true; + } } else { self.0.insert(*k, *v); + changed = true; } } + changed } pub(crate) fn min(&self) -> u64 { @@ -371,9 +381,10 @@ impl UpdateTracker { } impl UpdateTrackers { - pub(crate) fn merge(&mut self, other: &UpdateTrackers) { - self.ack_map.merge(&other.ack_map); - self.sync_map.merge(&other.sync_map); - self.sync_ack_map.merge(&other.sync_ack_map); + pub(crate) fn merge(&mut self, other: &UpdateTrackers) -> bool { + let c1 = self.ack_map.merge(&other.ack_map); + let c2 = self.sync_map.merge(&other.sync_map); + let c3 = self.sync_ack_map.merge(&other.sync_ack_map); + c1 || c2 || c3 } } diff --git a/src/rpc/system.rs b/src/rpc/system.rs index cb3af3fe..6ce13d0d 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -34,7 +34,7 @@ use crate::consul::ConsulDiscovery; #[cfg(feature = "kubernetes-discovery")] use crate::kubernetes::*; use crate::layout::manager::{LayoutManager, LayoutStatus}; -use crate::layout::*; +use crate::layout::{self, LayoutHistory, NodeRoleV}; use crate::replication_mode::*; use crate::rpc_helper::*; @@ -65,10 +65,15 @@ pub enum SystemRpc { GetKnownNodes, /// Return known nodes ReturnKnownNodes(Vec), + /// Ask other node its cluster layout. Answered with AdvertiseClusterLayout PullClusterLayout, /// Advertisement of cluster layout. Sent spontanously or in response to PullClusterLayout AdvertiseClusterLayout(LayoutHistory), + /// Ask other node its cluster layout update trackers. + PullClusterLayoutTrackers, + /// Advertisement of cluster layout update trackers. + AdvertiseClusterLayoutTrackers(layout::UpdateTrackers), } impl Rpc for SystemRpc { @@ -727,6 +732,14 @@ impl EndpointHandler for System { .handle_advertise_cluster_layout(adv) .await } + SystemRpc::PullClusterLayoutTrackers => { + Ok(self.layout_manager.handle_pull_cluster_layout_trackers()) + } + SystemRpc::AdvertiseClusterLayoutTrackers(adv) => { + self.layout_manager + .handle_advertise_cluster_layout_trackers(adv) + .await + } // ---- other -> Error ---- m => Err(Error::unexpected_rpc_message(m)), -- cgit v1.2.3 From 03ebf18830dff1983f09abe6ecb8d8d26daeb446 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 9 Nov 2023 15:31:59 +0100 Subject: layout: begin managing the update tracker values --- src/rpc/layout/history.rs | 74 +++++++++++++++++++++++++++++++++++++++++++---- src/rpc/layout/manager.rs | 7 +++-- src/rpc/layout/schema.rs | 15 +++++++--- src/rpc/layout/version.rs | 46 +++++++++++++++-------------- 4 files changed, 109 insertions(+), 33 deletions(-) diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index 357b9d62..347f03db 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -1,3 +1,5 @@ +use std::collections::HashSet; + use garage_util::crdt::{Crdt, Lww, LwwMap}; use garage_util::data::*; use garage_util::encode::nonversioned_encode; @@ -30,6 +32,14 @@ impl LayoutHistory { self.versions.last().as_ref().unwrap() } + pub fn all_storage_nodes(&self) -> HashSet { + self.versions + .iter() + .map(|x| x.nongateway_nodes()) + .flatten() + .collect::>() + } + pub(crate) fn update_hashes(&mut self) { self.trackers_hash = self.calculate_trackers_hash(); self.staging_hash = self.calculate_staging_hash(); @@ -43,6 +53,65 @@ impl LayoutHistory { blake2sum(&nonversioned_encode(&self.staging).unwrap()[..]) } + // ------------------ update tracking --------------- + + pub(crate) fn update_trackers(&mut self, node_id: Uuid) { + // Ensure trackers for this node's values are up-to-date + + // 1. Acknowledge the last layout version in the history + self.ack_last(node_id); + + // 2. Assume the data on this node is sync'ed up at least to + // the first layout version in the history + self.sync_first(node_id); + + // 3. Acknowledge everyone has synced up to min(self.sync_map) + self.sync_ack(node_id); + + // 4. Cleanup layout versions that are not needed anymore + self.cleanup_old_versions(); + + info!("ack_map: {:?}", self.update_trackers.ack_map); + info!("sync_map: {:?}", self.update_trackers.sync_map); + info!("sync_ack_map: {:?}", self.update_trackers.sync_ack_map); + + // Finally, update hashes + self.update_hashes(); + } + + pub(crate) fn ack_last(&mut self, node: Uuid) { + let last_version = self.current().version; + self.update_trackers.ack_map.set_max(node, last_version); + } + + pub(crate) fn sync_first(&mut self, node: Uuid) { + let first_version = self.versions.first().as_ref().unwrap().version; + self.update_trackers.sync_map.set_max(node, first_version); + } + + pub(crate) fn sync_ack(&mut self, node: Uuid) { + self.update_trackers.sync_ack_map.set_max( + node, + self.calculate_global_min(&self.update_trackers.sync_map), + ); + } + + pub(crate) fn cleanup_old_versions(&mut self) { + let min_sync_ack = self.calculate_global_min(&self.update_trackers.sync_ack_map); + while self.versions.first().as_ref().unwrap().version < min_sync_ack { + self.versions.remove(0); + } + } + + pub(crate) fn calculate_global_min(&self, tracker: &UpdateTracker) -> u64 { + let storage_nodes = self.all_storage_nodes(); + storage_nodes + .iter() + .map(|x| tracker.0.get(x).copied().unwrap_or(0)) + .min() + .unwrap_or(0) + } + // ================== updates to layout, public interface =================== pub fn merge(&mut self, other: &LayoutHistory) -> bool { @@ -78,11 +147,6 @@ impl LayoutHistory { changed = true; } - // Update hashes if there are changes - if changed { - self.update_hashes(); - } - changed } diff --git a/src/rpc/layout/manager.rs b/src/rpc/layout/manager.rs index a2502f58..ffcc938b 100644 --- a/src/rpc/layout/manager.rs +++ b/src/rpc/layout/manager.rs @@ -51,7 +51,7 @@ impl LayoutManager { let persist_cluster_layout: Persister = Persister::new(&config.metadata_dir, "cluster_layout"); - let cluster_layout = match persist_cluster_layout.load() { + let mut cluster_layout = match persist_cluster_layout.load() { Ok(x) => { if x.current().replication_factor != replication_factor { return Err(Error::Message(format!( @@ -71,6 +71,8 @@ impl LayoutManager { } }; + cluster_layout.update_trackers(node_id.into()); + let layout = Arc::new(RwLock::new(cluster_layout)); let change_notify = Arc::new(Notify::new()); @@ -126,7 +128,7 @@ impl LayoutManager { if prev_layout_check && layout.check().is_err() { panic!("Merged two correct layouts and got an incorrect layout."); } - + layout.update_trackers(self.node_id); return Some(layout.clone()); } } @@ -137,6 +139,7 @@ impl LayoutManager { let mut layout = self.layout.write().unwrap(); if layout.update_trackers != *adv { if layout.update_trackers.merge(adv) { + layout.update_trackers(self.node_id); return Some(layout.update_trackers.clone()); } } diff --git a/src/rpc/layout/schema.rs b/src/rpc/layout/schema.rs index abae5bd8..9f5d6f62 100644 --- a/src/rpc/layout/schema.rs +++ b/src/rpc/layout/schema.rs @@ -3,6 +3,7 @@ use std::fmt; use bytesize::ByteSize; use garage_util::crdt::{AutoCrdt, Crdt}; +use garage_util::data::Uuid; mod v08 { use crate::layout::CompactNodeType; @@ -276,8 +277,7 @@ mod v010 { let update_tracker = UpdateTracker( version .nongateway_nodes() - .iter() - .map(|x| (*x, version.version)) + .map(|x| (x, version.version)) .collect::>(), ); let staging = LayoutStaging { @@ -375,8 +375,15 @@ impl UpdateTracker { changed } - pub(crate) fn min(&self) -> u64 { - self.0.iter().map(|(_, v)| *v).min().unwrap_or(0) + pub(crate) fn set_max(&mut self, peer: Uuid, value: u64) { + match self.0.get_mut(&peer) { + Some(e) => { + *e = std::cmp::max(*e, value); + } + None => { + self.0.insert(peer, value); + } + } } } diff --git a/src/rpc/layout/version.rs b/src/rpc/layout/version.rs index 6918fdf9..65c62f63 100644 --- a/src/rpc/layout/version.rs +++ b/src/rpc/layout/version.rs @@ -134,15 +134,14 @@ impl LayoutVersion { // ===================== internal information extractors ====================== /// Returns the uuids of the non_gateway nodes in self.node_id_vec. - pub(crate) fn nongateway_nodes(&self) -> Vec { - let mut result = Vec::::new(); - for uuid in self.node_id_vec.iter() { - match self.node_role(uuid) { - Some(role) if role.capacity.is_some() => result.push(*uuid), - _ => (), - } - } - result + pub(crate) fn nongateway_nodes(&self) -> impl Iterator + '_ { + self.node_id_vec + .iter() + .copied() + .filter(move |uuid| match self.node_role(uuid) { + Some(role) if role.capacity.is_some() => true, + _ => false, + }) } /// Given a node uuids, this function returns the label of its zone @@ -158,8 +157,8 @@ impl LayoutVersion { /// Returns the sum of capacities of non gateway nodes in the cluster fn get_total_capacity(&self) -> Result { let mut total_capacity = 0; - for uuid in self.nongateway_nodes().iter() { - total_capacity += self.get_node_capacity(uuid)?; + for uuid in self.nongateway_nodes() { + total_capacity += self.get_node_capacity(&uuid)?; } Ok(total_capacity) } @@ -320,7 +319,7 @@ impl LayoutVersion { // to use them as indices in the flow graphs. let (id_to_zone, zone_to_id) = self.generate_nongateway_zone_ids()?; - let nb_nongateway_nodes = self.nongateway_nodes().len(); + let nb_nongateway_nodes = self.nongateway_nodes().count(); if nb_nongateway_nodes < self.replication_factor { return Err(Error::Message(format!( "The number of nodes with positive \ @@ -479,7 +478,8 @@ impl LayoutVersion { let mut id_to_zone = Vec::::new(); let mut zone_to_id = HashMap::::new(); - for uuid in self.nongateway_nodes().iter() { + let nongateway_nodes = self.nongateway_nodes().collect::>(); + for uuid in nongateway_nodes.iter() { let r = self.node_role(uuid).unwrap(); if !zone_to_id.contains_key(&r.zone) && r.capacity.is_some() { zone_to_id.insert(r.zone.clone(), id_to_zone.len()); @@ -556,8 +556,10 @@ impl LayoutVersion { exclude_assoc: &HashSet<(usize, usize)>, zone_redundancy: usize, ) -> Result, Error> { - let vertices = - LayoutVersion::generate_graph_vertices(zone_to_id.len(), self.nongateway_nodes().len()); + let vertices = LayoutVersion::generate_graph_vertices( + zone_to_id.len(), + self.nongateway_nodes().count(), + ); let mut g = Graph::::new(&vertices); let nb_zones = zone_to_id.len(); for p in 0..NB_PARTITIONS { @@ -576,7 +578,7 @@ impl LayoutVersion { )?; } } - for n in 0..self.nongateway_nodes().len() { + for n in 0..self.nongateway_nodes().count() { let node_capacity = self.get_node_capacity(&self.node_id_vec[n])?; let node_zone = zone_to_id[self.get_node_zone(&self.node_id_vec[n])?]; g.add_edge(Vertex::N(n), Vertex::Sink, node_capacity / partition_size)?; @@ -600,7 +602,7 @@ impl LayoutVersion { // previous assignment let mut exclude_edge = HashSet::<(usize, usize)>::new(); if let Some(prev_assign) = prev_assign_opt { - let nb_nodes = self.nongateway_nodes().len(); + let nb_nodes = self.nongateway_nodes().count(); for (p, prev_assign_p) in prev_assign.iter().enumerate() { for n in 0..nb_nodes { exclude_edge.insert((p, n)); @@ -652,7 +654,7 @@ impl LayoutVersion { // We compute the maximal length of a simple path in gflow. It is used in the // Bellman-Ford algorithm in optimize_flow_with_cost to set the number // of iterations. - let nb_nodes = self.nongateway_nodes().len(); + let nb_nodes = self.nongateway_nodes().count(); let path_length = 4 * nb_nodes; gflow.optimize_flow_with_cost(&cost, path_length)?; @@ -730,7 +732,7 @@ impl LayoutVersion { } // We define and fill in the following tables - let storing_nodes = self.nongateway_nodes(); + let storing_nodes = self.nongateway_nodes().collect::>(); let mut new_partitions = vec![0; storing_nodes.len()]; let mut stored_partitions = vec![0; storing_nodes.len()]; @@ -873,9 +875,9 @@ mod tests { for z in zones.iter() { zone_token.insert(z.clone(), 0); } - for uuid in cl.nongateway_nodes().iter() { - let z = cl.get_node_zone(uuid)?; - let c = cl.get_node_capacity(uuid)?; + for uuid in cl.nongateway_nodes() { + let z = cl.get_node_zone(&uuid)?; + let c = cl.get_node_capacity(&uuid)?; zone_token.insert( z.clone(), zone_token[&z] + min(NB_PARTITIONS, (c / over_size) as usize), -- cgit v1.2.3 From bad7cc812ead88e9f334405c5c082d79c14c8898 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 9 Nov 2023 15:42:10 +0100 Subject: layout admin: add missing calls to update_hash --- src/api/admin/cluster.rs | 1 + src/garage/cli/layout.rs | 3 ++- src/rpc/layout/history.rs | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/api/admin/cluster.rs b/src/api/admin/cluster.rs index 593bd778..d912b58f 100644 --- a/src/api/admin/cluster.rs +++ b/src/api/admin/cluster.rs @@ -240,6 +240,7 @@ pub async fn handle_update_cluster_layout( .merge(&roles.update_mutator(node, layout::NodeRoleV(new_role))); } + layout.update_hashes(); garage .system .layout_manager diff --git a/src/garage/cli/layout.rs b/src/garage/cli/layout.rs index 269d92f4..15727448 100644 --- a/src/garage/cli/layout.rs +++ b/src/garage/cli/layout.rs @@ -329,8 +329,9 @@ pub async fn fetch_layout( pub async fn send_layout( rpc_cli: &Endpoint, rpc_host: NodeID, - layout: LayoutHistory, + mut layout: LayoutHistory, ) -> Result<(), Error> { + layout.update_hashes(); rpc_cli .call( &rpc_host, diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index 347f03db..e17a1c77 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -40,7 +40,7 @@ impl LayoutHistory { .collect::>() } - pub(crate) fn update_hashes(&mut self) { + pub fn update_hashes(&mut self) { self.trackers_hash = self.calculate_trackers_hash(); self.staging_hash = self.calculate_staging_hash(); } -- cgit v1.2.3 From 9d95f6f7040c1899715ae4f984313427b1432758 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 9 Nov 2023 15:52:45 +0100 Subject: layout: fix tracker bugs --- src/rpc/layout/manager.rs | 8 +++++++- src/rpc/layout/schema.rs | 6 +++--- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/src/rpc/layout/manager.rs b/src/rpc/layout/manager.rs index ffcc938b..c1417dac 100644 --- a/src/rpc/layout/manager.rs +++ b/src/rpc/layout/manager.rs @@ -125,10 +125,10 @@ impl LayoutManager { if !prev_layout_check || adv.check().is_ok() { if layout.merge(adv) { + layout.update_trackers(self.node_id); if prev_layout_check && layout.check().is_err() { panic!("Merged two correct layouts and got an incorrect layout."); } - layout.update_trackers(self.node_id); return Some(layout.clone()); } } @@ -245,6 +245,8 @@ impl LayoutManager { self: &Arc, adv: &LayoutHistory, ) -> Result { + debug!("handle_advertise_cluster_layout: {:?}", adv); + if adv.current().replication_factor != self.replication_factor { let msg = format!( "Received a cluster layout from another node with replication factor {}, which is different from what we have in our configuration ({}). Discarding the cluster layout we received.", @@ -256,6 +258,8 @@ impl LayoutManager { } if let Some(new_layout) = self.merge_layout(adv) { + debug!("handle_advertise_cluster_layout: some changes were added to the current stuff"); + self.change_notify.notify_waiters(); self.broadcast_update(SystemRpc::AdvertiseClusterLayout(new_layout)); self.save_cluster_layout().await?; @@ -268,6 +272,8 @@ impl LayoutManager { self: &Arc, trackers: &UpdateTrackers, ) -> Result { + debug!("handle_advertise_cluster_layout_trackers: {:?}", trackers); + if let Some(new_trackers) = self.merge_layout_trackers(trackers) { self.change_notify.notify_waiters(); self.broadcast_update(SystemRpc::AdvertiseClusterLayoutTrackers(new_trackers)); diff --git a/src/rpc/layout/schema.rs b/src/rpc/layout/schema.rs index 9f5d6f62..db60c806 100644 --- a/src/rpc/layout/schema.rs +++ b/src/rpc/layout/schema.rs @@ -190,7 +190,7 @@ mod v010 { use garage_util::crdt::{Lww, LwwMap}; use garage_util::data::{Hash, Uuid}; use serde::{Deserialize, Serialize}; - use std::collections::HashMap; + use std::collections::BTreeMap; pub use v09::{LayoutParameters, NodeRole, NodeRoleV, ZoneRedundancy}; /// The layout of the cluster, i.e. the list of roles @@ -257,7 +257,7 @@ mod v010 { /// The history of cluster layouts #[derive(Clone, Debug, Serialize, Deserialize, Default, PartialEq)] - pub struct UpdateTracker(pub HashMap); + pub struct UpdateTracker(pub BTreeMap); impl garage_util::migrate::Migrate for LayoutHistory { const VERSION_MARKER: &'static [u8] = b"G010lh"; @@ -278,7 +278,7 @@ mod v010 { version .nongateway_nodes() .map(|x| (x, version.version)) - .collect::>(), + .collect::>(), ); let staging = LayoutStaging { parameters: previous.staging_parameters, -- cgit v1.2.3 From df36cf3099f6010c4fc62109b85d4d1e62f160cc Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 9 Nov 2023 16:32:31 +0100 Subject: layout: add helpers to LayoutHistory and prepare integration with Table --- src/rpc/layout/history.rs | 41 +++++++++++++++++++++++++++++++++-------- src/rpc/layout/manager.rs | 33 ++++++++++++++++++++++++++++++++- src/rpc/layout/schema.rs | 9 ++++++--- src/rpc/layout/version.rs | 2 +- src/table/table.rs | 2 ++ 5 files changed, 74 insertions(+), 13 deletions(-) diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index e17a1c77..dbb02269 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -32,14 +32,6 @@ impl LayoutHistory { self.versions.last().as_ref().unwrap() } - pub fn all_storage_nodes(&self) -> HashSet { - self.versions - .iter() - .map(|x| x.nongateway_nodes()) - .flatten() - .collect::>() - } - pub fn update_hashes(&mut self) { self.trackers_hash = self.calculate_trackers_hash(); self.staging_hash = self.calculate_staging_hash(); @@ -53,6 +45,39 @@ impl LayoutHistory { blake2sum(&nonversioned_encode(&self.staging).unwrap()[..]) } + // ------------------ who stores what now? --------------- + + pub fn max_ack(&self) -> u64 { + self.calculate_global_min(&self.update_trackers.ack_map) + } + + pub fn all_storage_nodes(&self) -> HashSet { + // TODO: cache this + self.versions + .iter() + .map(|x| x.nongateway_nodes()) + .flatten() + .collect::>() + } + + pub fn read_nodes_of(&self, position: &Hash) -> Vec { + let sync_min = self.calculate_global_min(&self.update_trackers.sync_map); + let version = self + .versions + .iter() + .find(|x| x.version == sync_min) + .or(self.versions.last()) + .unwrap(); + version.nodes_of(position, version.replication_factor) + } + + pub fn write_sets_of(&self, position: &Hash) -> Vec> { + self.versions + .iter() + .map(|x| x.nodes_of(position, x.replication_factor)) + .collect::>() + } + // ------------------ update tracking --------------- pub(crate) fn update_trackers(&mut self, node_id: Uuid) { diff --git a/src/rpc/layout/manager.rs b/src/rpc/layout/manager.rs index c1417dac..b0302b12 100644 --- a/src/rpc/layout/manager.rs +++ b/src/rpc/layout/manager.rs @@ -1,4 +1,5 @@ -use std::sync::{Arc, RwLock, RwLockReadGuard}; +use std::collections::HashMap; +use std::sync::{Arc, Mutex, RwLock, RwLockReadGuard}; use std::time::Duration; use serde::{Deserialize, Serialize}; @@ -26,6 +27,8 @@ pub struct LayoutManager { layout: Arc>, pub(crate) change_notify: Arc, + table_sync_version: Mutex>, + pub(crate) rpc_helper: RpcHelper, system_endpoint: Arc>, } @@ -117,6 +120,34 @@ impl LayoutManager { Ok(()) } + pub fn add_table(&self, table_name: &'static str) { + let first_version = self.layout().versions.first().unwrap().version; + + self.table_sync_version + .lock() + .unwrap() + .insert(table_name.to_string(), first_version); + } + + pub fn sync_table_until(self: &Arc, table_name: &'static str, version: u64) { + let mut table_sync_version = self.table_sync_version.lock().unwrap(); + *table_sync_version.get_mut(table_name).unwrap() = version; + let sync_until = table_sync_version.iter().map(|(_, v)| *v).max().unwrap(); + drop(table_sync_version); + + let mut layout = self.layout.write().unwrap(); + if layout + .update_trackers + .sync_map + .set_max(self.node_id, sync_until) + { + layout.update_hashes(); + self.broadcast_update(SystemRpc::AdvertiseClusterLayoutTrackers( + layout.update_trackers.clone(), + )); + } + } + // ---- INTERNALS --- fn merge_layout(&self, adv: &LayoutHistory) -> Option { diff --git a/src/rpc/layout/schema.rs b/src/rpc/layout/schema.rs index db60c806..89f5c361 100644 --- a/src/rpc/layout/schema.rs +++ b/src/rpc/layout/schema.rs @@ -375,14 +375,17 @@ impl UpdateTracker { changed } - pub(crate) fn set_max(&mut self, peer: Uuid, value: u64) { + pub(crate) fn set_max(&mut self, peer: Uuid, value: u64) -> bool { match self.0.get_mut(&peer) { - Some(e) => { - *e = std::cmp::max(*e, value); + Some(e) if *e < value => { + *e = value; + true } None => { self.0.insert(peer, value); + true } + _ => false, } } } diff --git a/src/rpc/layout/version.rs b/src/rpc/layout/version.rs index 65c62f63..8133672a 100644 --- a/src/rpc/layout/version.rs +++ b/src/rpc/layout/version.rs @@ -109,7 +109,7 @@ impl LayoutVersion { .collect::>() } - /// Walk the ring to find the n servers in which data should be replicated + /// Return the n servers in which data for this hash should be replicated pub fn nodes_of(&self, position: &Hash, n: usize) -> Vec { assert_eq!(n, self.replication_factor); diff --git a/src/table/table.rs b/src/table/table.rs index 3e3fd138..997fd7dc 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -80,6 +80,8 @@ impl Table { let syncer = TableSyncer::new(system.clone(), data.clone(), merkle_updater.clone()); let gc = TableGc::new(system.clone(), data.clone()); + system.layout_manager.add_table(F::TABLE_NAME); + let table = Arc::new(Self { system, data, -- cgit v1.2.3 From ce89d1ddabe3b9e638b0173949726522ae9a0311 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Sat, 11 Nov 2023 12:08:32 +0100 Subject: table sync: adapt to new layout history --- src/rpc/layout/history.rs | 21 +++-- src/rpc/layout/manager.rs | 1 + src/rpc/layout/version.rs | 16 ++-- src/rpc/system.rs | 2 +- src/table/replication/fullcopy.rs | 25 ++++- src/table/replication/parameters.rs | 19 +++- src/table/replication/sharded.rs | 39 +++++++- src/table/sync.rs | 178 +++++++++++++++--------------------- 8 files changed, 172 insertions(+), 129 deletions(-) diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index dbb02269..185dbb27 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -47,11 +47,19 @@ impl LayoutHistory { // ------------------ who stores what now? --------------- - pub fn max_ack(&self) -> u64 { + pub fn all_ack(&self) -> u64 { self.calculate_global_min(&self.update_trackers.ack_map) } - pub fn all_storage_nodes(&self) -> HashSet { + pub fn min_stored(&self) -> u64 { + self.versions.first().as_ref().unwrap().version + } + + pub fn sync_versions(&self) -> (u64, u64, u64) { + (self.current().version, self.all_ack(), self.min_stored()) + } + + pub fn all_nongateway_nodes(&self) -> HashSet { // TODO: cache this self.versions .iter() @@ -71,11 +79,10 @@ impl LayoutHistory { version.nodes_of(position, version.replication_factor) } - pub fn write_sets_of(&self, position: &Hash) -> Vec> { + pub fn write_sets_of<'a>(&'a self, position: &'a Hash) -> impl Iterator> + 'a { self.versions .iter() - .map(|x| x.nodes_of(position, x.replication_factor)) - .collect::>() + .map(move |x| x.nodes_of(position, x.replication_factor)) } // ------------------ update tracking --------------- @@ -129,7 +136,9 @@ impl LayoutHistory { } pub(crate) fn calculate_global_min(&self, tracker: &UpdateTracker) -> u64 { - let storage_nodes = self.all_storage_nodes(); + // TODO: for TableFullReplication, counting gateway nodes might be + // necessary? Think about this more. + let storage_nodes = self.all_nongateway_nodes(); storage_nodes .iter() .map(|x| tracker.0.get(x).copied().unwrap_or(0)) diff --git a/src/rpc/layout/manager.rs b/src/rpc/layout/manager.rs index b0302b12..7d60bae6 100644 --- a/src/rpc/layout/manager.rs +++ b/src/rpc/layout/manager.rs @@ -92,6 +92,7 @@ impl LayoutManager { persist_cluster_layout, layout, change_notify, + table_sync_version: Mutex::new(HashMap::new()), system_endpoint, rpc_helper, })) diff --git a/src/rpc/layout/version.rs b/src/rpc/layout/version.rs index 8133672a..f45a3c35 100644 --- a/src/rpc/layout/version.rs +++ b/src/rpc/layout/version.rs @@ -98,15 +98,13 @@ impl LayoutVersion { } /// Get the list of partitions and the first hash of a partition key that would fall in it - pub fn partitions(&self) -> Vec<(Partition, Hash)> { - (0..(1 << PARTITION_BITS)) - .map(|i| { - let top = (i as u16) << (16 - PARTITION_BITS); - let mut location = [0u8; 32]; - location[..2].copy_from_slice(&u16::to_be_bytes(top)[..]); - (i as u16, Hash::from(location)) - }) - .collect::>() + pub fn partitions(&self) -> impl Iterator + '_ { + (0..(1 << PARTITION_BITS)).map(|i| { + let top = (i as u16) << (16 - PARTITION_BITS); + let mut location = [0u8; 32]; + location[..2].copy_from_slice(&u16::to_be_bytes(top)[..]); + (i as u16, Hash::from(location)) + }) } /// Return the n servers in which data for this hash should be replicated diff --git a/src/rpc/system.rs b/src/rpc/system.rs index 6ce13d0d..3418600b 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -442,7 +442,7 @@ impl System { .filter(|(x, _, _)| nodes.get(x).map(|n| n.is_up).unwrap_or(false)) .count(); - let partitions = layout.current().partitions(); + let partitions = layout.current().partitions().collect::>(); let partitions_n_up = partitions .iter() .map(|(_, h)| { diff --git a/src/table/replication/fullcopy.rs b/src/table/replication/fullcopy.rs index a5c83d0f..5653a229 100644 --- a/src/table/replication/fullcopy.rs +++ b/src/table/replication/fullcopy.rs @@ -1,3 +1,4 @@ +use std::iter::FromIterator; use std::sync::Arc; use garage_rpc::layout::*; @@ -6,10 +7,17 @@ use garage_util::data::*; use crate::replication::*; +// TODO: find a way to track layout changes for this as well +// The hard thing is that this data is stored also on gateway nodes, +// whereas sharded data is stored only on non-Gateway nodes (storage nodes) +// Also we want to be more tolerant to failures of gateways so we don't +// want to do too much holding back of data when progress of gateway +// nodes is not reported in the layout history's ack/sync/sync_ack maps. + /// 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 +/// Inconvenient: if some writes fail, nodes will read outdated data #[derive(Clone)] pub struct TableFullReplication { /// The membership manager of this node @@ -44,7 +52,18 @@ impl TableReplication for TableFullReplication { fn partition_of(&self, _hash: &Hash) -> Partition { 0u16 } - fn partitions(&self) -> Vec<(Partition, Hash)> { - vec![(0u16, [0u8; 32].into())] + + fn sync_partitions(&self) -> SyncPartitions { + let layout = self.system.cluster_layout(); + let layout_version = layout.current().version; + SyncPartitions { + layout_version, + partitions: vec![SyncPartition { + partition: 0u16, + first_hash: [0u8; 32].into(), + last_hash: [0xff; 32].into(), + storage_nodes: Vec::from_iter(layout.current().node_ids().to_vec()), + }], + } } } diff --git a/src/table/replication/parameters.rs b/src/table/replication/parameters.rs index 19b306f2..2a7d3585 100644 --- a/src/table/replication/parameters.rs +++ b/src/table/replication/parameters.rs @@ -20,6 +20,21 @@ pub trait TableReplication: Send + Sync + 'static { // Accessing partitions, for Merkle tree & sync /// Get partition for data with given hash fn partition_of(&self, hash: &Hash) -> Partition; - /// List of existing partitions - fn partitions(&self) -> Vec<(Partition, Hash)>; + + /// List of partitions and nodes to sync with in current layout + fn sync_partitions(&self) -> SyncPartitions; +} + +#[derive(Debug)] +pub struct SyncPartitions { + pub layout_version: u64, + pub partitions: Vec, +} + +#[derive(Debug)] +pub struct SyncPartition { + pub partition: Partition, + pub first_hash: Hash, + pub last_hash: Hash, + pub storage_nodes: Vec, } diff --git a/src/table/replication/sharded.rs b/src/table/replication/sharded.rs index 793d87fd..f02b1d66 100644 --- a/src/table/replication/sharded.rs +++ b/src/table/replication/sharded.rs @@ -51,7 +51,42 @@ impl TableReplication for TableShardedReplication { fn partition_of(&self, hash: &Hash) -> Partition { self.system.cluster_layout().current().partition_of(hash) } - fn partitions(&self) -> Vec<(Partition, Hash)> { - self.system.cluster_layout().current().partitions() + + fn sync_partitions(&self) -> SyncPartitions { + let layout = self.system.cluster_layout(); + let layout_version = layout.all_ack(); + + let mut partitions = layout + .current() + .partitions() + .map(|(partition, first_hash)| { + let mut storage_nodes = layout + .write_sets_of(&first_hash) + .map(|x| x.into_iter()) + .flatten() + .collect::>(); + storage_nodes.sort(); + storage_nodes.dedup(); + SyncPartition { + partition, + first_hash, + last_hash: [0u8; 32].into(), // filled in just after + storage_nodes, + } + }) + .collect::>(); + + for i in 0..partitions.len() { + partitions[i].last_hash = if i + 1 < partitions.len() { + partitions[i + 1].first_hash + } else { + [0xFFu8; 32].into() + }; + } + + SyncPartitions { + layout_version, + partitions, + } } } diff --git a/src/table/sync.rs b/src/table/sync.rs index 4355bd9e..43636faa 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -6,7 +6,7 @@ use arc_swap::ArcSwapOption; use async_trait::async_trait; use futures_util::stream::*; use opentelemetry::KeyValue; -use rand::Rng; +use rand::prelude::*; use serde::{Deserialize, Serialize}; use serde_bytes::ByteBuf; use tokio::select; @@ -52,16 +52,6 @@ impl Rpc for SyncRpc { type Response = Result; } -#[derive(Debug, Clone)] -struct TodoPartition { - partition: Partition, - begin: Hash, - end: Hash, - - // Are we a node that stores this partition or not? - retain: bool, -} - impl TableSyncer { pub(crate) fn new( system: Arc, @@ -92,9 +82,9 @@ impl TableSyncer { bg.spawn_worker(SyncWorker { syncer: self.clone(), layout_notify: self.system.layout_notify(), - layout_version: self.system.cluster_layout().current().version, + layout_versions: self.system.cluster_layout().sync_versions(), add_full_sync_rx, - todo: vec![], + todo: None, next_full_sync: Instant::now() + Duration::from_secs(20), }); } @@ -112,31 +102,26 @@ impl TableSyncer { async fn sync_partition( self: &Arc, - partition: &TodoPartition, + partition: &SyncPartition, must_exit: &mut watch::Receiver, ) -> Result<(), Error> { - if partition.retain { - let my_id = self.system.id; - - let nodes = self - .data - .replication - .write_nodes(&partition.begin) - .into_iter() - .filter(|node| *node != my_id) - .collect::>(); + let my_id = self.system.id; + let retain = partition.storage_nodes.contains(&my_id); + if retain { debug!( "({}) Syncing {:?} with {:?}...", F::TABLE_NAME, partition, - nodes + partition.storage_nodes ); - let mut sync_futures = nodes + let mut sync_futures = partition + .storage_nodes .iter() + .filter(|node| **node != my_id) .map(|node| { self.clone() - .do_sync_with(partition.clone(), *node, must_exit.clone()) + .do_sync_with(&partition, *node, must_exit.clone()) }) .collect::>(); @@ -147,14 +132,14 @@ impl TableSyncer { warn!("({}) Sync error: {}", F::TABLE_NAME, e); } } - if n_errors > self.data.replication.max_write_errors() { + if n_errors > 0 { return Err(Error::Message(format!( - "Sync failed with too many nodes (should have been: {:?}).", - nodes + "Sync failed with {} nodes.", + n_errors ))); } } else { - self.offload_partition(&partition.begin, &partition.end, must_exit) + self.offload_partition(&partition.first_hash, &partition.last_hash, must_exit) .await?; } @@ -285,7 +270,7 @@ impl TableSyncer { async fn do_sync_with( self: Arc, - partition: TodoPartition, + partition: &SyncPartition, who: Uuid, must_exit: watch::Receiver, ) -> Result<(), Error> { @@ -492,76 +477,23 @@ impl EndpointHandler for TableSync struct SyncWorker { syncer: Arc>, + layout_notify: Arc, - layout_version: u64, + layout_versions: (u64, u64, u64), + add_full_sync_rx: mpsc::UnboundedReceiver<()>, - todo: Vec, next_full_sync: Instant, + + todo: Option, } impl SyncWorker { fn add_full_sync(&mut self) { - let system = &self.syncer.system; - let data = &self.syncer.data; - - let my_id = system.id; - - self.todo.clear(); - - let partitions = data.replication.partitions(); - - for i in 0..partitions.len() { - let begin = partitions[i].1; - - let end = if i + 1 < partitions.len() { - partitions[i + 1].1 - } else { - [0xFFu8; 32].into() - }; - - 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 - match data.store.range(begin..end) { - Ok(mut iter) => { - if iter.next().is_none() { - continue; - } - } - Err(e) => { - warn!("DB error in add_full_sync: {}", e); - continue; - } - } - } - - self.todo.push(TodoPartition { - partition: partitions[i].0, - begin, - end, - retain, - }); - } - + let mut partitions = self.syncer.data.replication.sync_partitions(); + partitions.partitions.shuffle(&mut thread_rng()); + self.todo = Some(partitions); self.next_full_sync = Instant::now() + ANTI_ENTROPY_INTERVAL; } - - 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) - } - } } #[async_trait] @@ -572,18 +504,46 @@ impl Worker for SyncWorker { fn status(&self) -> WorkerStatus { WorkerStatus { - queue_length: Some(self.todo.len() as u64), + queue_length: Some(self.todo.as_ref().map(|x| x.partitions.len()).unwrap_or(0) as u64), ..Default::default() } } async fn work(&mut self, must_exit: &mut watch::Receiver) -> Result { - if let Some(partition) = self.pop_task() { - self.syncer.sync_partition(&partition, must_exit).await?; - Ok(WorkerState::Busy) - } else { - Ok(WorkerState::Idle) + if let Some(todo) = &mut self.todo { + let partition = todo.partitions.pop().unwrap(); + + // process partition + if let Err(e) = self.syncer.sync_partition(&partition, must_exit).await { + error!( + "{}: Failed to sync partition {:?}: {}", + F::TABLE_NAME, + partition, + e + ); + // if error, put partition back at the other side of the queue, + // so that other partitions will be tried in the meantime + todo.partitions.insert(0, partition); + // TODO: returning an error here will cause the background job worker + // to delay this task for some time, but maybe we don't want to + // delay it if there are lots of failures from nodes that are gone + // (we also don't want zero delays as that will cause lots of useless retries) + return Err(e); + } + + // done + if !todo.partitions.is_empty() { + return Ok(WorkerState::Busy); + } + + self.syncer + .system + .layout_manager + .sync_table_until(F::TABLE_NAME, todo.layout_version); } + + self.todo = None; + Ok(WorkerState::Idle) } async fn wait_for_work(&mut self) -> WorkerState { @@ -594,10 +554,16 @@ impl Worker for SyncWorker { } }, _ = self.layout_notify.notified() => { - let new_version = self.syncer.system.cluster_layout().current().version; - if new_version > self.layout_version { - self.layout_version = new_version; - debug!("({}) Layout changed, adding full sync to syncer todo list", F::TABLE_NAME); + let layout_versions = self.syncer.system.cluster_layout().sync_versions(); + if layout_versions != self.layout_versions { + self.layout_versions = layout_versions; + debug!( + "({}) Layout versions changed (max={}, ack={}, min stored={}), adding full sync to syncer todo list", + F::TABLE_NAME, + layout_versions.0, + layout_versions.1, + layout_versions.2 + ); self.add_full_sync(); } }, @@ -605,9 +571,9 @@ impl Worker for SyncWorker { self.add_full_sync(); } } - match self.todo.is_empty() { - false => WorkerState::Busy, - true => WorkerState::Idle, + match self.todo.is_some() { + true => WorkerState::Busy, + false => WorkerState::Idle, } } } -- cgit v1.2.3 From df24bb806d64d5d5e748c35efe3f49ad3dda709e Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Sat, 11 Nov 2023 12:37:33 +0100 Subject: layout/sync: fix bugs and add tracing --- src/rpc/layout/history.rs | 3 ++- src/rpc/layout/manager.rs | 10 ++++++-- src/table/sync.rs | 60 ++++++++++++++++++++++++++++++----------------- 3 files changed, 48 insertions(+), 25 deletions(-) diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index 185dbb27..cef56647 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -131,7 +131,8 @@ impl LayoutHistory { pub(crate) fn cleanup_old_versions(&mut self) { let min_sync_ack = self.calculate_global_min(&self.update_trackers.sync_ack_map); while self.versions.first().as_ref().unwrap().version < min_sync_ack { - self.versions.remove(0); + let removed = self.versions.remove(0); + info!("Layout history: pruning old version {}", removed.version); } } diff --git a/src/rpc/layout/manager.rs b/src/rpc/layout/manager.rs index 7d60bae6..ce8b6f61 100644 --- a/src/rpc/layout/manager.rs +++ b/src/rpc/layout/manager.rs @@ -133,7 +133,7 @@ impl LayoutManager { pub fn sync_table_until(self: &Arc, table_name: &'static str, version: u64) { let mut table_sync_version = self.table_sync_version.lock().unwrap(); *table_sync_version.get_mut(table_name).unwrap() = version; - let sync_until = table_sync_version.iter().map(|(_, v)| *v).max().unwrap(); + let sync_until = table_sync_version.iter().map(|(_, v)| *v).min().unwrap(); drop(table_sync_version); let mut layout = self.layout.write().unwrap(); @@ -142,6 +142,7 @@ impl LayoutManager { .sync_map .set_max(self.node_id, sync_until) { + debug!("sync_until updated to {}", sync_until); layout.update_hashes(); self.broadcast_update(SystemRpc::AdvertiseClusterLayoutTrackers( layout.update_trackers.clone(), @@ -277,7 +278,12 @@ impl LayoutManager { self: &Arc, adv: &LayoutHistory, ) -> Result { - debug!("handle_advertise_cluster_layout: {:?}", adv); + debug!( + "handle_advertise_cluster_layout: {} versions, last={}, trackers={:?}", + adv.versions.len(), + adv.current().version, + adv.update_trackers + ); if adv.current().replication_factor != self.replication_factor { let msg = format!( diff --git a/src/table/sync.rs b/src/table/sync.rs index 43636faa..8c21db8b 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -488,8 +488,29 @@ struct SyncWorker { } impl SyncWorker { + fn check_add_full_sync(&mut self) { + let layout_versions = self.syncer.system.cluster_layout().sync_versions(); + if layout_versions != self.layout_versions { + self.layout_versions = layout_versions; + info!( + "({}) Layout versions changed (max={}, ack={}, min stored={}), adding full sync to syncer todo list", + F::TABLE_NAME, + layout_versions.0, + layout_versions.1, + layout_versions.2 + ); + self.add_full_sync(); + } + } + fn add_full_sync(&mut self) { let mut partitions = self.syncer.data.replication.sync_partitions(); + info!( + "{}: Adding full sync for ack layout version {}", + F::TABLE_NAME, + partitions.layout_version + ); + partitions.partitions.shuffle(&mut thread_rng()); self.todo = Some(partitions); self.next_full_sync = Instant::now() + ANTI_ENTROPY_INTERVAL; @@ -510,6 +531,8 @@ impl Worker for SyncWorker { } async fn work(&mut self, must_exit: &mut watch::Receiver) -> Result { + self.check_add_full_sync(); + if let Some(todo) = &mut self.todo { let partition = todo.partitions.pop().unwrap(); @@ -531,19 +554,23 @@ impl Worker for SyncWorker { return Err(e); } - // done - if !todo.partitions.is_empty() { - return Ok(WorkerState::Busy); + if todo.partitions.is_empty() { + info!( + "{}: Completed full sync for ack layout version {}", + F::TABLE_NAME, + todo.layout_version + ); + self.syncer + .system + .layout_manager + .sync_table_until(F::TABLE_NAME, todo.layout_version); + self.todo = None; } - self.syncer - .system - .layout_manager - .sync_table_until(F::TABLE_NAME, todo.layout_version); + Ok(WorkerState::Busy) + } else { + Ok(WorkerState::Idle) } - - self.todo = None; - Ok(WorkerState::Idle) } async fn wait_for_work(&mut self) -> WorkerState { @@ -554,18 +581,7 @@ impl Worker for SyncWorker { } }, _ = self.layout_notify.notified() => { - let layout_versions = self.syncer.system.cluster_layout().sync_versions(); - if layout_versions != self.layout_versions { - self.layout_versions = layout_versions; - debug!( - "({}) Layout versions changed (max={}, ack={}, min stored={}), adding full sync to syncer todo list", - F::TABLE_NAME, - layout_versions.0, - layout_versions.1, - layout_versions.2 - ); - self.add_full_sync(); - } + self.check_add_full_sync(); }, _ = tokio::time::sleep_until(self.next_full_sync.into()) => { self.add_full_sync(); -- cgit v1.2.3 From 9a491fa1372a23e91c793ee1d2b313607752826a Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Sat, 11 Nov 2023 13:10:59 +0100 Subject: layout: fix test --- src/rpc/layout/history.rs | 9 ++- src/rpc/layout/mod.rs | 3 + src/rpc/layout/test.rs | 159 ++++++++++++++++++++++++++++++++++++++++++ src/rpc/layout/version.rs | 172 ++-------------------------------------------- 4 files changed, 174 insertions(+), 169 deletions(-) create mode 100644 src/rpc/layout/test.rs diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index cef56647..050f5d0a 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -18,7 +18,7 @@ impl LayoutHistory { }; let mut ret = LayoutHistory { - versions: vec![version].into_boxed_slice().into(), + versions: vec![version], update_trackers: Default::default(), trackers_hash: [0u8; 32].into(), staging: Lww::raw(0, staging), @@ -211,6 +211,11 @@ To know the correct value of the new layout version, invoke `garage layout show` let msg = new_version.calculate_partition_assignment()?; self.versions.push(new_version); + if self.current().check().is_ok() { + while self.versions.first().unwrap().check().is_err() { + self.versions.remove(0); + } + } // Reset the staged layout changes self.staging.update(LayoutStaging { @@ -245,7 +250,7 @@ To know the correct value of the new layout version, invoke `garage layout show` version.check()?; } - // TODO: anythign more ? + // TODO: anything more ? Ok(()) } } diff --git a/src/rpc/layout/mod.rs b/src/rpc/layout/mod.rs index cd3764bc..577b32fb 100644 --- a/src/rpc/layout/mod.rs +++ b/src/rpc/layout/mod.rs @@ -3,6 +3,9 @@ mod history; mod schema; mod version; +#[cfg(test)] +mod test; + pub mod manager; // ---- re-exports ---- diff --git a/src/rpc/layout/test.rs b/src/rpc/layout/test.rs new file mode 100644 index 00000000..0ce090d2 --- /dev/null +++ b/src/rpc/layout/test.rs @@ -0,0 +1,159 @@ +use std::cmp::min; +use std::collections::HashMap; + +use garage_util::crdt::Crdt; +use garage_util::error::*; + +use crate::layout::*; + +// This function checks that the partition size S computed is at least better than the +// one given by a very naive algorithm. To do so, we try to run the naive algorithm +// assuming a partion size of S+1. If we succed, it means that the optimal assignment +// was not optimal. The naive algorithm is the following : +// - we compute the max number of partitions associated to every node, capped at the +// partition number. It gives the number of tokens of every node. +// - every zone has a number of tokens equal to the sum of the tokens of its nodes. +// - we cycle over the partitions and associate zone tokens while respecting the +// zone redundancy constraint. +// NOTE: the naive algorithm is not optimal. Counter example: +// take nb_partition = 3 ; replication_factor = 5; redundancy = 4; +// number of tokens by zone : (A, 4), (B,1), (C,4), (D, 4), (E, 2) +// With these parameters, the naive algo fails, whereas there is a solution: +// (A,A,C,D,E) , (A,B,C,D,D) (A,C,C,D,E) +fn check_against_naive(cl: &LayoutVersion) -> Result { + let over_size = cl.partition_size + 1; + let mut zone_token = HashMap::::new(); + + let (zones, zone_to_id) = cl.generate_nongateway_zone_ids()?; + + if zones.is_empty() { + return Ok(false); + } + + for z in zones.iter() { + zone_token.insert(z.clone(), 0); + } + for uuid in cl.nongateway_nodes() { + let z = cl.get_node_zone(&uuid)?; + let c = cl.get_node_capacity(&uuid)?; + zone_token.insert( + z.to_string(), + zone_token[z] + min(NB_PARTITIONS, (c / over_size) as usize), + ); + } + + // For every partition, we count the number of zone already associated and + // the name of the last zone associated + + let mut id_zone_token = vec![0; zones.len()]; + for (z, t) in zone_token.iter() { + id_zone_token[zone_to_id[z]] = *t; + } + + let mut nb_token = vec![0; NB_PARTITIONS]; + let mut last_zone = vec![zones.len(); NB_PARTITIONS]; + + let mut curr_zone = 0; + + let redundancy = cl.effective_zone_redundancy(); + + for replic in 0..cl.replication_factor { + for p in 0..NB_PARTITIONS { + while id_zone_token[curr_zone] == 0 + || (last_zone[p] == curr_zone + && redundancy - nb_token[p] <= cl.replication_factor - replic) + { + curr_zone += 1; + if curr_zone >= zones.len() { + return Ok(true); + } + } + id_zone_token[curr_zone] -= 1; + if last_zone[p] != curr_zone { + nb_token[p] += 1; + last_zone[p] = curr_zone; + } + } + } + + return Ok(false); +} + +fn show_msg(msg: &Message) { + for s in msg.iter() { + println!("{}", s); + } +} + +fn update_layout( + cl: &mut LayoutHistory, + node_capacity_vec: &[u64], + node_zone_vec: &[&'static str], + zone_redundancy: usize, +) { + let staging = cl.staging.get_mut(); + + for (i, (capacity, zone)) in node_capacity_vec + .iter() + .zip(node_zone_vec.iter()) + .enumerate() + { + let node_id = [i as u8; 32].into(); + + let update = staging.roles.update_mutator( + node_id, + NodeRoleV(Some(NodeRole { + zone: zone.to_string(), + capacity: Some(*capacity), + tags: (vec![]), + })), + ); + staging.roles.merge(&update); + } + staging.parameters.update(LayoutParameters { + zone_redundancy: ZoneRedundancy::AtLeast(zone_redundancy), + }); + + cl.update_hashes(); +} + +#[test] +fn test_assignment() { + let mut node_capacity_vec = vec![4000, 1000, 2000]; + let mut node_zone_vec = vec!["A", "B", "C"]; + + let mut cl = LayoutHistory::new(3); + update_layout(&mut cl, &node_capacity_vec, &node_zone_vec, 3); + let v = cl.current().version; + let (mut cl, msg) = cl.apply_staged_changes(Some(v + 1)).unwrap(); + show_msg(&msg); + assert_eq!(cl.check(), Ok(())); + assert!(check_against_naive(cl.current()).unwrap()); + + node_capacity_vec = vec![4000, 1000, 1000, 3000, 1000, 1000, 2000, 10000, 2000]; + node_zone_vec = vec!["A", "B", "C", "C", "C", "B", "G", "H", "I"]; + update_layout(&mut cl, &node_capacity_vec, &node_zone_vec, 2); + let v = cl.current().version; + let (mut cl, msg) = cl.apply_staged_changes(Some(v + 1)).unwrap(); + show_msg(&msg); + assert_eq!(cl.check(), Ok(())); + assert!(check_against_naive(cl.current()).unwrap()); + + node_capacity_vec = vec![4000, 1000, 2000, 7000, 1000, 1000, 2000, 10000, 2000]; + update_layout(&mut cl, &node_capacity_vec, &node_zone_vec, 3); + let v = cl.current().version; + let (mut cl, msg) = cl.apply_staged_changes(Some(v + 1)).unwrap(); + show_msg(&msg); + assert_eq!(cl.check(), Ok(())); + assert!(check_against_naive(cl.current()).unwrap()); + + node_capacity_vec = vec![ + 4000000, 4000000, 2000000, 7000000, 1000000, 9000000, 2000000, 10000, 2000000, + ]; + update_layout(&mut cl, &node_capacity_vec, &node_zone_vec, 1); + let v = cl.current().version; + let (cl, msg) = cl.apply_staged_changes(Some(v + 1)).unwrap(); + show_msg(&msg); + assert_eq!(cl.check(), Ok(())); + assert!(check_against_naive(cl.current()).unwrap()); +} diff --git a/src/rpc/layout/version.rs b/src/rpc/layout/version.rs index f45a3c35..ffbdf277 100644 --- a/src/rpc/layout/version.rs +++ b/src/rpc/layout/version.rs @@ -143,7 +143,7 @@ impl LayoutVersion { } /// Given a node uuids, this function returns the label of its zone - fn get_node_zone(&self, uuid: &Uuid) -> Result<&str, Error> { + pub(crate) fn get_node_zone(&self, uuid: &Uuid) -> Result<&str, Error> { match self.node_role(uuid) { Some(role) => Ok(&role.zone), _ => Err(Error::Message( @@ -162,7 +162,7 @@ impl LayoutVersion { } /// Returns the effective value of the zone_redundancy parameter - fn effective_zone_redundancy(&self) -> usize { + pub(crate) fn effective_zone_redundancy(&self) -> usize { match self.parameters.zone_redundancy { ZoneRedundancy::AtLeast(v) => v, ZoneRedundancy::Maximum => { @@ -472,7 +472,9 @@ impl LayoutVersion { /// This function generates ids for the zone of the nodes appearing in /// self.node_id_vec. - fn generate_nongateway_zone_ids(&self) -> Result<(Vec, HashMap), Error> { + pub(crate) fn generate_nongateway_zone_ids( + &self, + ) -> Result<(Vec, HashMap), Error> { let mut id_to_zone = Vec::::new(); let mut zone_to_id = HashMap::::new(); @@ -838,167 +840,3 @@ impl LayoutVersion { Ok(msg) } } - -// ==================================================================================== - -#[cfg(test)] -mod tests { - use super::{Error, *}; - use std::cmp::min; - - // This function checks that the partition size S computed is at least better than the - // one given by a very naive algorithm. To do so, we try to run the naive algorithm - // assuming a partion size of S+1. If we succed, it means that the optimal assignment - // was not optimal. The naive algorithm is the following : - // - we compute the max number of partitions associated to every node, capped at the - // partition number. It gives the number of tokens of every node. - // - every zone has a number of tokens equal to the sum of the tokens of its nodes. - // - we cycle over the partitions and associate zone tokens while respecting the - // zone redundancy constraint. - // NOTE: the naive algorithm is not optimal. Counter example: - // take nb_partition = 3 ; replication_factor = 5; redundancy = 4; - // number of tokens by zone : (A, 4), (B,1), (C,4), (D, 4), (E, 2) - // With these parameters, the naive algo fails, whereas there is a solution: - // (A,A,C,D,E) , (A,B,C,D,D) (A,C,C,D,E) - fn check_against_naive(cl: &LayoutVersion) -> Result { - let over_size = cl.partition_size + 1; - let mut zone_token = HashMap::::new(); - - let (zones, zone_to_id) = cl.generate_nongateway_zone_ids()?; - - if zones.is_empty() { - return Ok(false); - } - - for z in zones.iter() { - zone_token.insert(z.clone(), 0); - } - for uuid in cl.nongateway_nodes() { - let z = cl.get_node_zone(&uuid)?; - let c = cl.get_node_capacity(&uuid)?; - zone_token.insert( - z.clone(), - zone_token[&z] + min(NB_PARTITIONS, (c / over_size) as usize), - ); - } - - // For every partition, we count the number of zone already associated and - // the name of the last zone associated - - let mut id_zone_token = vec![0; zones.len()]; - for (z, t) in zone_token.iter() { - id_zone_token[zone_to_id[z]] = *t; - } - - let mut nb_token = vec![0; NB_PARTITIONS]; - let mut last_zone = vec![zones.len(); NB_PARTITIONS]; - - let mut curr_zone = 0; - - let redundancy = cl.effective_zone_redundancy(); - - for replic in 0..cl.replication_factor { - for p in 0..NB_PARTITIONS { - while id_zone_token[curr_zone] == 0 - || (last_zone[p] == curr_zone - && redundancy - nb_token[p] <= cl.replication_factor - replic) - { - curr_zone += 1; - if curr_zone >= zones.len() { - return Ok(true); - } - } - id_zone_token[curr_zone] -= 1; - if last_zone[p] != curr_zone { - nb_token[p] += 1; - last_zone[p] = curr_zone; - } - } - } - - return Ok(false); - } - - fn show_msg(msg: &Message) { - for s in msg.iter() { - println!("{}", s); - } - } - - fn update_layout( - cl: &mut LayoutVersion, - node_id_vec: &Vec, - node_capacity_vec: &Vec, - node_zone_vec: &Vec, - zone_redundancy: usize, - ) { - for i in 0..node_id_vec.len() { - if let Some(x) = FixedBytes32::try_from(&[i as u8; 32]) { - cl.node_id_vec.push(x); - } - - let update = cl.staging_roles.update_mutator( - cl.node_id_vec[i], - NodeRoleV(Some(NodeRole { - zone: (node_zone_vec[i].to_string()), - capacity: (Some(node_capacity_vec[i])), - tags: (vec![]), - })), - ); - cl.staging_roles.merge(&update); - } - cl.staging_parameters.update(LayoutParameters { - zone_redundancy: ZoneRedundancy::AtLeast(zone_redundancy), - }); - cl.staging_hash = cl.calculate_staging_hash(); - } - - #[test] - fn test_assignment() { - let mut node_id_vec = vec![1, 2, 3]; - let mut node_capacity_vec = vec![4000, 1000, 2000]; - let mut node_zone_vec = vec!["A", "B", "C"] - .into_iter() - .map(|x| x.to_string()) - .collect(); - - let mut cl = LayoutVersion::new(3); - update_layout(&mut cl, &node_id_vec, &node_capacity_vec, &node_zone_vec, 3); - let v = cl.version; - let (mut cl, msg) = cl.apply_staged_changes(Some(v + 1)).unwrap(); - show_msg(&msg); - assert_eq!(cl.check(), Ok(())); - assert!(matches!(check_against_naive(&cl), Ok(true))); - - node_id_vec = vec![1, 2, 3, 4, 5, 6, 7, 8, 9]; - node_capacity_vec = vec![4000, 1000, 1000, 3000, 1000, 1000, 2000, 10000, 2000]; - node_zone_vec = vec!["A", "B", "C", "C", "C", "B", "G", "H", "I"] - .into_iter() - .map(|x| x.to_string()) - .collect(); - update_layout(&mut cl, &node_id_vec, &node_capacity_vec, &node_zone_vec, 2); - let v = cl.version; - let (mut cl, msg) = cl.apply_staged_changes(Some(v + 1)).unwrap(); - show_msg(&msg); - assert_eq!(cl.check(), Ok(())); - assert!(matches!(check_against_naive(&cl), Ok(true))); - - node_capacity_vec = vec![4000, 1000, 2000, 7000, 1000, 1000, 2000, 10000, 2000]; - update_layout(&mut cl, &node_id_vec, &node_capacity_vec, &node_zone_vec, 3); - let v = cl.version; - let (mut cl, msg) = cl.apply_staged_changes(Some(v + 1)).unwrap(); - show_msg(&msg); - assert_eq!(cl.check(), Ok(())); - assert!(matches!(check_against_naive(&cl), Ok(true))); - - node_capacity_vec = vec![ - 4000000, 4000000, 2000000, 7000000, 1000000, 9000000, 2000000, 10000, 2000000, - ]; - update_layout(&mut cl, &node_id_vec, &node_capacity_vec, &node_zone_vec, 1); - let v = cl.version; - let (cl, msg) = cl.apply_staged_changes(Some(v + 1)).unwrap(); - show_msg(&msg); - assert_eq!(cl.check(), Ok(())); - assert!(matches!(check_against_naive(&cl), Ok(true))); - } -} -- cgit v1.2.3 From 8e292e06b3fde1d3b5b019a26eabd4f0d9ac22c3 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 14 Nov 2023 12:48:38 +0100 Subject: layout: some refactoring of nongateway nodes --- src/api/k2v/index.rs | 7 +++- src/model/index_counter.rs | 4 +- src/rpc/layout/history.rs | 30 ++++++++------ src/rpc/layout/schema.rs | 17 ++++++++ src/rpc/layout/version.rs | 98 +++++++++++++++++++++++++--------------------- 5 files changed, 95 insertions(+), 61 deletions(-) diff --git a/src/api/k2v/index.rs b/src/api/k2v/index.rs index 3c2f51a9..c189232a 100644 --- a/src/api/k2v/index.rs +++ b/src/api/k2v/index.rs @@ -25,8 +25,11 @@ pub async fn handle_read_index( ) -> Result, Error> { let reverse = reverse.unwrap_or(false); - // TODO: not only current - let node_id_vec = garage.system.cluster_layout().current().node_ids().to_vec(); + let node_id_vec = garage + .system + .cluster_layout() + .all_nongateway_nodes() + .into_owned(); let (partition_keys, more, next_start) = read_range( &garage.k2v.counter_table.table, diff --git a/src/model/index_counter.rs b/src/model/index_counter.rs index 9637cc4c..2d968733 100644 --- a/src/model/index_counter.rs +++ b/src/model/index_counter.rs @@ -84,8 +84,8 @@ impl Entry for CounterEntry { impl CounterEntry { pub fn filtered_values(&self, layout: &LayoutHistory) -> HashMap { - let nodes = &layout.current().node_id_vec[..]; - self.filtered_values_with_nodes(nodes) + let nodes = layout.all_nongateway_nodes(); + self.filtered_values_with_nodes(&nodes) } pub fn filtered_values_with_nodes(&self, nodes: &[Uuid]) -> HashMap { diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index 050f5d0a..877ad3a7 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -1,3 +1,4 @@ +use std::borrow::Cow; use std::collections::HashSet; use garage_util::crdt::{Crdt, Lww, LwwMap}; @@ -59,13 +60,19 @@ impl LayoutHistory { (self.current().version, self.all_ack(), self.min_stored()) } - pub fn all_nongateway_nodes(&self) -> HashSet { + pub fn all_nongateway_nodes(&self) -> Cow<'_, [Uuid]> { // TODO: cache this - self.versions - .iter() - .map(|x| x.nongateway_nodes()) - .flatten() - .collect::>() + if self.versions.len() == 1 { + self.versions[0].nongateway_nodes().into() + } else { + let set = self + .versions + .iter() + .map(|x| x.nongateway_nodes()) + .flatten() + .collect::>(); + set.into_iter().copied().collect::>().into() + } } pub fn read_nodes_of(&self, position: &Hash) -> Vec { @@ -202,14 +209,11 @@ To know the correct value of the new layout version, invoke `garage layout show` } // Compute new version and add it to history - let mut new_version = self.current().clone(); - new_version.version += 1; - - new_version.roles.merge(&self.staging.get().roles); - new_version.roles.retain(|(_, _, v)| v.0.is_some()); - new_version.parameters = *self.staging.get().parameters.get(); + let (new_version, msg) = self + .current() + .clone() + .calculate_next_version(&self.staging.get())?; - let msg = new_version.calculate_partition_assignment()?; self.versions.push(new_version); if self.current().check().is_ok() { while self.versions.first().unwrap().check().is_err() { diff --git a/src/rpc/layout/schema.rs b/src/rpc/layout/schema.rs index 89f5c361..db298ee6 100644 --- a/src/rpc/layout/schema.rs +++ b/src/rpc/layout/schema.rs @@ -212,6 +212,8 @@ mod v010 { /// see comment in v08::ClusterLayout pub node_id_vec: Vec, + /// number of non-gateway nodes, which are the first ids in node_id_vec + pub nongateway_node_count: usize, /// see comment in v08::ClusterLayout #[serde(with = "serde_bytes")] pub ring_assignment_data: Vec, @@ -265,6 +267,18 @@ mod v010 { type Previous = v09::ClusterLayout; fn migrate(previous: Self::Previous) -> Self { + let nongateway_node_count = previous + .node_id_vec + .iter() + .enumerate() + .filter(|(_, uuid)| { + let role = previous.roles.get(uuid); + matches!(role, Some(NodeRoleV(Some(role))) if role.capacity.is_some()) + }) + .map(|(i, _)| i) + .max() + .unwrap_or(0); + let version = LayoutVersion { version: previous.version, replication_factor: previous.replication_factor, @@ -272,11 +286,14 @@ mod v010 { parameters: previous.parameters, roles: previous.roles, node_id_vec: previous.node_id_vec, + nongateway_node_count, ring_assignment_data: previous.ring_assignment_data, }; let update_tracker = UpdateTracker( version .nongateway_nodes() + .iter() + .copied() .map(|x| (x, version.version)) .collect::>(), ); diff --git a/src/rpc/layout/version.rs b/src/rpc/layout/version.rs index ffbdf277..a7f387b6 100644 --- a/src/rpc/layout/version.rs +++ b/src/rpc/layout/version.rs @@ -5,7 +5,7 @@ use std::convert::TryInto; use bytesize::ByteSize; use itertools::Itertools; -use garage_util::crdt::LwwMap; +use garage_util::crdt::{Crdt, LwwMap}; use garage_util::data::*; use garage_util::error::*; @@ -30,6 +30,7 @@ impl LayoutVersion { partition_size: 0, roles: LwwMap::new(), node_id_vec: Vec::new(), + nongateway_node_count: 0, ring_assignment_data: Vec::new(), parameters, } @@ -43,6 +44,11 @@ impl LayoutVersion { &self.node_id_vec[..] } + /// Returns the uuids of the non_gateway nodes in self.node_id_vec. + pub fn nongateway_nodes(&self) -> &[Uuid] { + &self.node_id_vec[..self.nongateway_node_count] + } + pub fn num_nodes(&self) -> usize { self.node_id_vec.len() } @@ -56,18 +62,14 @@ impl LayoutVersion { } /// Given a node uuids, this function returns its capacity or fails if it does not have any - pub fn get_node_capacity(&self, uuid: &Uuid) -> Result { + pub fn get_node_capacity(&self, uuid: &Uuid) -> Option { match self.node_role(uuid) { Some(NodeRole { capacity: Some(cap), zone: _, tags: _, - }) => Ok(*cap), - _ => Err(Error::Message( - "The Uuid does not correspond to a node present in the \ - cluster or this node does not have a positive capacity." - .into(), - )), + }) => Some(*cap), + _ => None, } } @@ -131,17 +133,6 @@ impl LayoutVersion { // ===================== internal information extractors ====================== - /// Returns the uuids of the non_gateway nodes in self.node_id_vec. - pub(crate) fn nongateway_nodes(&self) -> impl Iterator + '_ { - self.node_id_vec - .iter() - .copied() - .filter(move |uuid| match self.node_role(uuid) { - Some(role) if role.capacity.is_some() => true, - _ => false, - }) - } - /// Given a node uuids, this function returns the label of its zone pub(crate) fn get_node_zone(&self, uuid: &Uuid) -> Result<&str, Error> { match self.node_role(uuid) { @@ -152,11 +143,16 @@ impl LayoutVersion { } } + fn expect_get_node_capacity(&self, uuid: &Uuid) -> u64 { + self.get_node_capacity(&uuid) + .expect("non-gateway node with zero capacity") + } + /// Returns the sum of capacities of non gateway nodes in the cluster fn get_total_capacity(&self) -> Result { let mut total_capacity = 0; for uuid in self.nongateway_nodes() { - total_capacity += self.get_node_capacity(&uuid)?; + total_capacity += self.expect_get_node_capacity(&uuid); } Ok(total_capacity) } @@ -257,7 +253,7 @@ impl LayoutVersion { if *usage > 0 { let uuid = self.node_id_vec[n]; let partusage = usage * self.partition_size; - let nodecap = self.get_node_capacity(&uuid).unwrap(); + let nodecap = self.expect_get_node_capacity(&uuid); if partusage > nodecap { return Err(format!( "node usage ({}) is bigger than node capacity ({})", @@ -288,6 +284,21 @@ impl LayoutVersion { // ================== updates to layout, internals =================== + pub(crate) fn calculate_next_version( + mut self, + staging: &LayoutStaging, + ) -> Result<(Self, Message), Error> { + self.version += 1; + + self.roles.merge(&staging.roles); + self.roles.retain(|(_, _, v)| v.0.is_some()); + self.parameters = *staging.parameters.get(); + + let msg = self.calculate_partition_assignment()?; + + Ok((self, msg)) + } + /// This function calculates a new partition-to-node assignment. /// The computed assignment respects the node replication factor /// and the zone redundancy parameter It maximizes the capacity of a @@ -297,7 +308,7 @@ impl LayoutVersion { /// data to be moved. /// Staged role changes must be merged with nodes roles before calling this function, /// hence it must only be called from apply_staged_changes() and hence is not public. - pub(crate) fn calculate_partition_assignment(&mut self) -> Result { + fn calculate_partition_assignment(&mut self) -> Result { // We update the node ids, since the node role list might have changed with the // changes in the layout. We retrieve the old_assignment reframed with new ids let old_assignment_opt = self.update_node_id_vec()?; @@ -317,12 +328,12 @@ impl LayoutVersion { // to use them as indices in the flow graphs. let (id_to_zone, zone_to_id) = self.generate_nongateway_zone_ids()?; - let nb_nongateway_nodes = self.nongateway_nodes().count(); - if nb_nongateway_nodes < self.replication_factor { + if self.nongateway_nodes().len() < self.replication_factor { return Err(Error::Message(format!( "The number of nodes with positive \ capacity ({}) is smaller than the replication factor ({}).", - nb_nongateway_nodes, self.replication_factor + self.nongateway_nodes().len(), + self.replication_factor ))); } if id_to_zone.len() < zone_redundancy { @@ -420,12 +431,14 @@ impl LayoutVersion { .map(|(k, _, _)| *k) .collect(); - let mut new_node_id_vec = Vec::::new(); - new_node_id_vec.extend(new_non_gateway_nodes); - new_node_id_vec.extend(new_gateway_nodes); + let old_node_id_vec = std::mem::take(&mut self.node_id_vec); + + self.nongateway_node_count = new_non_gateway_nodes.len(); + self.node_id_vec.clear(); + self.node_id_vec.extend(new_non_gateway_nodes); + self.node_id_vec.extend(new_gateway_nodes); - let old_node_id_vec = self.node_id_vec.clone(); - self.node_id_vec = new_node_id_vec.clone(); + let new_node_id_vec = &self.node_id_vec; // (2) We retrieve the old association // We rewrite the old association with the new indices. We only consider partition @@ -464,7 +477,7 @@ impl LayoutVersion { } } - // We write the ring + // We clear the ring assignemnt data self.ring_assignment_data = Vec::::new(); Ok(Some(old_assignment)) @@ -478,8 +491,7 @@ impl LayoutVersion { let mut id_to_zone = Vec::::new(); let mut zone_to_id = HashMap::::new(); - let nongateway_nodes = self.nongateway_nodes().collect::>(); - for uuid in nongateway_nodes.iter() { + for uuid in self.nongateway_nodes().iter() { let r = self.node_role(uuid).unwrap(); if !zone_to_id.contains_key(&r.zone) && r.capacity.is_some() { zone_to_id.insert(r.zone.clone(), id_to_zone.len()); @@ -556,10 +568,8 @@ impl LayoutVersion { exclude_assoc: &HashSet<(usize, usize)>, zone_redundancy: usize, ) -> Result, Error> { - let vertices = LayoutVersion::generate_graph_vertices( - zone_to_id.len(), - self.nongateway_nodes().count(), - ); + let vertices = + LayoutVersion::generate_graph_vertices(zone_to_id.len(), self.nongateway_nodes().len()); let mut g = Graph::::new(&vertices); let nb_zones = zone_to_id.len(); for p in 0..NB_PARTITIONS { @@ -578,8 +588,8 @@ impl LayoutVersion { )?; } } - for n in 0..self.nongateway_nodes().count() { - let node_capacity = self.get_node_capacity(&self.node_id_vec[n])?; + for n in 0..self.nongateway_nodes().len() { + let node_capacity = self.expect_get_node_capacity(&self.node_id_vec[n]); let node_zone = zone_to_id[self.get_node_zone(&self.node_id_vec[n])?]; g.add_edge(Vertex::N(n), Vertex::Sink, node_capacity / partition_size)?; for p in 0..NB_PARTITIONS { @@ -602,7 +612,7 @@ impl LayoutVersion { // previous assignment let mut exclude_edge = HashSet::<(usize, usize)>::new(); if let Some(prev_assign) = prev_assign_opt { - let nb_nodes = self.nongateway_nodes().count(); + let nb_nodes = self.nongateway_nodes().len(); for (p, prev_assign_p) in prev_assign.iter().enumerate() { for n in 0..nb_nodes { exclude_edge.insert((p, n)); @@ -654,7 +664,7 @@ impl LayoutVersion { // We compute the maximal length of a simple path in gflow. It is used in the // Bellman-Ford algorithm in optimize_flow_with_cost to set the number // of iterations. - let nb_nodes = self.nongateway_nodes().count(); + let nb_nodes = self.nongateway_nodes().len(); let path_length = 4 * nb_nodes; gflow.optimize_flow_with_cost(&cost, path_length)?; @@ -732,7 +742,7 @@ impl LayoutVersion { } // We define and fill in the following tables - let storing_nodes = self.nongateway_nodes().collect::>(); + let storing_nodes = self.nongateway_nodes(); let mut new_partitions = vec![0; storing_nodes.len()]; let mut stored_partitions = vec![0; storing_nodes.len()]; @@ -804,13 +814,13 @@ impl LayoutVersion { let available_cap_z: u64 = self.partition_size * replicated_partitions as u64; let mut total_cap_z = 0; for n in nodes_of_z.iter() { - total_cap_z += self.get_node_capacity(&self.node_id_vec[*n])?; + total_cap_z += self.expect_get_node_capacity(&self.node_id_vec[*n]); } let percent_cap_z = 100.0 * (available_cap_z as f32) / (total_cap_z as f32); for n in nodes_of_z.iter() { let available_cap_n = stored_partitions[*n] as u64 * self.partition_size; - let total_cap_n = self.get_node_capacity(&self.node_id_vec[*n])?; + let total_cap_n = self.expect_get_node_capacity(&self.node_id_vec[*n]); let tags_n = (self.node_role(&self.node_id_vec[*n]).ok_or(""))?.tags_string(); table.push(format!( " {:?}\t{}\t{} ({} new)\t{}\t{} ({:.1}%)", -- cgit v1.2.3 From 1aab1f4e688ebc3f3adcb41c817c16c688a3291c Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 14 Nov 2023 13:06:16 +0100 Subject: layout: refactoring of all_nodes --- src/garage/admin/mod.rs | 17 +++++++++-------- src/garage/cli/layout.rs | 6 +++--- src/model/helper/bucket.rs | 8 ++++++-- src/rpc/layout/history.rs | 15 +++++++++++++++ src/rpc/layout/version.rs | 17 +++++++---------- src/rpc/system.rs | 2 +- src/table/replication/fullcopy.rs | 6 +++--- 7 files changed, 44 insertions(+), 27 deletions(-) diff --git a/src/garage/admin/mod.rs b/src/garage/admin/mod.rs index e3ba6d35..77918a0f 100644 --- a/src/garage/admin/mod.rs +++ b/src/garage/admin/mod.rs @@ -126,8 +126,8 @@ impl AdminRpcHandler { opt_to_send.all_nodes = false; let mut failures = vec![]; - let layout = self.garage.system.cluster_layout().clone(); - for node in layout.current().node_ids().iter() { + let all_nodes = self.garage.system.cluster_layout().all_nodes().to_vec(); + for node in all_nodes.iter() { let node = (*node).into(); let resp = self .endpoint @@ -163,9 +163,9 @@ impl AdminRpcHandler { async fn handle_stats(&self, opt: StatsOpt) -> Result { if opt.all_nodes { let mut ret = String::new(); - let layout = self.garage.system.cluster_layout().clone(); + let all_nodes = self.garage.system.cluster_layout().all_nodes().to_vec(); - for node in layout.current().node_ids().iter() { + for node in all_nodes.iter() { let mut opt = opt.clone(); opt.all_nodes = false; opt.skip_global = true; @@ -275,6 +275,7 @@ impl AdminRpcHandler { let mut ret = String::new(); // Gather storage node and free space statistics + // TODO: not only layout.current() ??? let layout = &self.garage.system.cluster_layout(); let mut node_partition_count = HashMap::::new(); for short_id in layout.current().ring_assignment_data.iter() { @@ -440,8 +441,8 @@ impl AdminRpcHandler { ) -> Result { if all_nodes { let mut ret = vec![]; - let layout = self.garage.system.cluster_layout().clone(); - for node in layout.current().node_ids().iter() { + let all_nodes = self.garage.system.cluster_layout().all_nodes().to_vec(); + for node in all_nodes.iter() { let node = (*node).into(); match self .endpoint @@ -488,8 +489,8 @@ impl AdminRpcHandler { ) -> Result { if all_nodes { let mut ret = vec![]; - let layout = self.garage.system.cluster_layout().clone(); - for node in layout.current().node_ids().iter() { + let all_nodes = self.garage.system.cluster_layout().all_nodes().to_vec(); + for node in all_nodes.iter() { let node = (*node).into(); match self .endpoint diff --git a/src/garage/cli/layout.rs b/src/garage/cli/layout.rs index 15727448..0f01a37a 100644 --- a/src/garage/cli/layout.rs +++ b/src/garage/cli/layout.rs @@ -49,6 +49,7 @@ pub async fn cmd_assign_role( }; let mut layout = fetch_layout(rpc_cli, rpc_host).await?; + let all_nodes = layout.all_nodes().into_owned(); let added_nodes = args .node_ids @@ -58,7 +59,7 @@ pub async fn cmd_assign_role( status .iter() .map(|adv| adv.id) - .chain(layout.current().node_ids().iter().cloned()), + .chain(all_nodes.iter().cloned()), node_id, ) }) @@ -68,8 +69,7 @@ pub async fn cmd_assign_role( roles.merge(&layout.staging.get().roles); for replaced in args.replace.iter() { - let replaced_node = - find_matching_node(layout.current().node_ids().iter().cloned(), replaced)?; + let replaced_node = find_matching_node(all_nodes.iter().cloned(), replaced)?; match roles.get(&replaced_node) { Some(NodeRoleV(Some(_))) => { layout diff --git a/src/model/helper/bucket.rs b/src/model/helper/bucket.rs index 2a9c0fb1..2cb53424 100644 --- a/src/model/helper/bucket.rs +++ b/src/model/helper/bucket.rs @@ -450,8 +450,12 @@ impl<'a> BucketHelper<'a> { #[cfg(feature = "k2v")] { - // TODO: not only current - let node_id_vec = self.0.system.cluster_layout().current().node_ids().to_vec(); + let node_id_vec = self + .0 + .system + .cluster_layout() + .all_nongateway_nodes() + .into_owned(); let k2vindexes = self .0 .k2v diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index 877ad3a7..69348873 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -60,6 +60,21 @@ impl LayoutHistory { (self.current().version, self.all_ack(), self.min_stored()) } + pub fn all_nodes(&self) -> Cow<'_, [Uuid]> { + // TODO: cache this + if self.versions.len() == 1 { + self.versions[0].all_nodes().into() + } else { + let set = self + .versions + .iter() + .map(|x| x.all_nodes()) + .flatten() + .collect::>(); + set.into_iter().copied().collect::>().into() + } + } + pub fn all_nongateway_nodes(&self) -> Cow<'_, [Uuid]> { // TODO: cache this if self.versions.len() == 1 { diff --git a/src/rpc/layout/version.rs b/src/rpc/layout/version.rs index a7f387b6..2cbdcee2 100644 --- a/src/rpc/layout/version.rs +++ b/src/rpc/layout/version.rs @@ -38,22 +38,19 @@ impl LayoutVersion { // ===================== accessors ====================== - /// Returns a list of IDs of nodes that currently have - /// a role in the cluster - pub fn node_ids(&self) -> &[Uuid] { + /// Returns a list of IDs of nodes that have a role in this + /// version of the cluster layout, including gateway nodes + pub fn all_nodes(&self) -> &[Uuid] { &self.node_id_vec[..] } - /// Returns the uuids of the non_gateway nodes in self.node_id_vec. + /// Returns a list of IDs of nodes that have a storage capacity + /// assigned in this version of the cluster layout pub fn nongateway_nodes(&self) -> &[Uuid] { &self.node_id_vec[..self.nongateway_node_count] } - pub fn num_nodes(&self) -> usize { - self.node_id_vec.len() - } - - /// Returns the role of a node in the layout + /// Returns the role of a node in the layout, if it has one pub fn node_role(&self, node: &Uuid) -> Option<&NodeRole> { match self.roles.get(node) { Some(NodeRoleV(Some(v))) => Some(v), @@ -61,7 +58,7 @@ impl LayoutVersion { } } - /// Given a node uuids, this function returns its capacity or fails if it does not have any + /// Returns the capacity of a node in the layout, if it has one pub fn get_node_capacity(&self, uuid: &Uuid) -> Option { match self.node_role(uuid) { Some(NodeRole { diff --git a/src/rpc/system.rs b/src/rpc/system.rs index 3418600b..ab3c96b8 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -609,7 +609,7 @@ impl System { while !*stop_signal.borrow() { let not_configured = self.cluster_layout().check().is_err(); let no_peers = self.fullmesh.get_peer_list().len() < self.replication_factor; - let expected_n_nodes = self.cluster_layout().current().num_nodes(); + let expected_n_nodes = self.cluster_layout().all_nodes().len(); let bad_peers = self .fullmesh .get_peer_list() diff --git a/src/table/replication/fullcopy.rs b/src/table/replication/fullcopy.rs index 5653a229..beaacc2b 100644 --- a/src/table/replication/fullcopy.rs +++ b/src/table/replication/fullcopy.rs @@ -35,10 +35,10 @@ impl TableReplication for TableFullReplication { } fn write_nodes(&self, _hash: &Hash) -> Vec { - self.system.cluster_layout().current().node_ids().to_vec() + self.system.cluster_layout().current().all_nodes().to_vec() } fn write_quorum(&self) -> usize { - let nmembers = self.system.cluster_layout().current().node_ids().len(); + let nmembers = self.system.cluster_layout().current().all_nodes().len(); if nmembers > self.max_faults { nmembers - self.max_faults } else { @@ -62,7 +62,7 @@ impl TableReplication for TableFullReplication { partition: 0u16, first_hash: [0u8; 32].into(), last_hash: [0xff; 32].into(), - storage_nodes: Vec::from_iter(layout.current().node_ids().to_vec()), + storage_nodes: Vec::from_iter(layout.current().all_nodes().to_vec()), }], } } -- cgit v1.2.3 From 83a11374ca45831a6f54928dfe726fac65493b00 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 14 Nov 2023 13:29:26 +0100 Subject: layout: fixes in schema --- src/rpc/layout/schema.rs | 42 +++++++++++++++++++++--------------------- 1 file changed, 21 insertions(+), 21 deletions(-) diff --git a/src/rpc/layout/schema.rs b/src/rpc/layout/schema.rs index db298ee6..79440a47 100644 --- a/src/rpc/layout/schema.rs +++ b/src/rpc/layout/schema.rs @@ -193,7 +193,25 @@ mod v010 { use std::collections::BTreeMap; pub use v09::{LayoutParameters, NodeRole, NodeRoleV, ZoneRedundancy}; - /// The layout of the cluster, i.e. the list of roles + /// The history of cluster layouts, with trackers to keep a record + /// of which nodes are up-to-date to current cluster data + #[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] + pub struct LayoutHistory { + /// The versions currently in use in the cluster + pub versions: Vec, + + /// Update trackers + pub update_trackers: UpdateTrackers, + /// Hash of the update trackers + pub trackers_hash: Hash, + + /// Staged changes for the next version + pub staging: Lww, + /// Hash of the serialized staging_parameters + staging_roles + pub staging_hash: Hash, + } + + /// A version of the layout of the cluster, i.e. the list of roles /// which are assigned to each cluster node #[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] pub struct LayoutVersion { @@ -228,23 +246,6 @@ mod v010 { pub roles: LwwMap, } - /// The history of cluster layouts - #[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] - pub struct LayoutHistory { - /// The versions currently in use in the cluster - pub versions: Vec, - - /// Update trackers - pub update_trackers: UpdateTrackers, - /// Hash of the update trackers - pub trackers_hash: Hash, - - /// Staged changes for the next version - pub staging: Lww, - /// Hash of the serialized staging_parameters + staging_roles - pub staging_hash: Hash, - } - /// The tracker of acknowlegments and data syncs around the cluster #[derive(Clone, Debug, Serialize, Deserialize, Default, PartialEq)] pub struct UpdateTrackers { @@ -275,7 +276,7 @@ mod v010 { let role = previous.roles.get(uuid); matches!(role, Some(NodeRoleV(Some(role))) if role.capacity.is_some()) }) - .map(|(i, _)| i) + .map(|(i, _)| i + 1) .max() .unwrap_or(0); @@ -312,8 +313,7 @@ mod v010 { staging: Lww::raw(previous.version, staging), staging_hash: [0u8; 32].into(), }; - ret.staging_hash = ret.calculate_staging_hash(); - ret.trackers_hash = ret.calculate_trackers_hash(); + ret.update_hashes(); ret } } -- cgit v1.2.3 From 866196750fca74c1911ade2a90611f3663e60046 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 14 Nov 2023 13:36:58 +0100 Subject: system: add todo wrt new layout --- src/rpc/system.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/rpc/system.rs b/src/rpc/system.rs index ab3c96b8..86c02e86 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -417,6 +417,9 @@ impl System { } pub fn health(&self) -> ClusterHealth { + // TODO: adapt this function to take into account layout history + // when estimating cluster health, and not just use current layout + let quorum = self.replication_mode.write_quorum(); let replication_factor = self.replication_factor; @@ -429,7 +432,6 @@ impl System { let layout = self.cluster_layout(); // acquires a rwlock - // TODO: not only layout.current() let storage_nodes = layout .current() .roles -- cgit v1.2.3 From 3b361d2959e3d577bdae6f8a5ccb0c9d5526b7ea Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 14 Nov 2023 14:28:16 +0100 Subject: layout: prepare for write sets --- src/block/manager.rs | 3 ++- src/block/resync.rs | 2 +- src/model/k2v/rpc.rs | 10 ++++++---- src/rpc/layout/history.rs | 19 ++++++++++++++++--- src/rpc/layout/version.rs | 21 ++++++++++----------- src/rpc/system.rs | 3 +-- src/table/data.rs | 3 ++- src/table/gc.rs | 2 +- src/table/replication/fullcopy.rs | 9 +++++++-- src/table/replication/parameters.rs | 8 +++++--- src/table/replication/sharded.rs | 24 ++++++++---------------- src/table/sync.rs | 2 +- src/table/table.rs | 6 ++++-- 13 files changed, 64 insertions(+), 48 deletions(-) diff --git a/src/block/manager.rs b/src/block/manager.rs index 72b4ea66..2bb9c23d 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -354,7 +354,8 @@ impl BlockManager { /// Send block to nodes that should have it pub async fn rpc_put_block(&self, hash: Hash, data: Bytes) -> Result<(), Error> { - let who = self.replication.write_nodes(&hash); + // TODO: use quorums among latest write set + let who = self.replication.storage_nodes(&hash); let (header, bytes) = DataBlock::from_buffer(data, self.compression_level) .await diff --git a/src/block/resync.rs b/src/block/resync.rs index fedcd6f5..122d0142 100644 --- a/src/block/resync.rs +++ b/src/block/resync.rs @@ -377,7 +377,7 @@ impl BlockResyncManager { info!("Resync block {:?}: offloading and deleting", hash); let existing_path = existing_path.unwrap(); - let mut who = manager.replication.write_nodes(hash); + let mut who = manager.replication.storage_nodes(hash); if who.len() < manager.replication.write_quorum() { return Err(Error::Message("Not trying to offload block because we don't have a quorum of nodes to write to".to_string())); } diff --git a/src/model/k2v/rpc.rs b/src/model/k2v/rpc.rs index 2f548ad7..aa3323d5 100644 --- a/src/model/k2v/rpc.rs +++ b/src/model/k2v/rpc.rs @@ -127,7 +127,7 @@ impl K2VRpcHandler { .item_table .data .replication - .write_nodes(&partition.hash()); + .storage_nodes(&partition.hash()); who.sort(); self.system @@ -168,7 +168,7 @@ impl K2VRpcHandler { .item_table .data .replication - .write_nodes(&partition.hash()); + .storage_nodes(&partition.hash()); who.sort(); call_list.entry(who).or_default().push(InsertedItem { @@ -223,11 +223,12 @@ impl K2VRpcHandler { }, sort_key, }; + // TODO figure this out with write sets, does it still work???? let nodes = self .item_table .data .replication - .write_nodes(&poll_key.partition.hash()); + .read_nodes(&poll_key.partition.hash()); let rpc = self.system.rpc_helper().try_call_many( &self.endpoint, @@ -284,11 +285,12 @@ impl K2VRpcHandler { seen.restrict(&range); // Prepare PollRange RPC to send to the storage nodes responsible for the parititon + // TODO figure this out with write sets, does it still work???? let nodes = self .item_table .data .replication - .write_nodes(&range.partition.hash()); + .read_nodes(&range.partition.hash()); let quorum = self.item_table.data.replication.read_quorum(); let msg = K2VRpc::PollRange { range, diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index 69348873..dce492c9 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -98,13 +98,26 @@ impl LayoutHistory { .find(|x| x.version == sync_min) .or(self.versions.last()) .unwrap(); - version.nodes_of(position, version.replication_factor) + version + .nodes_of(position, version.replication_factor) + .collect() } - pub fn write_sets_of<'a>(&'a self, position: &'a Hash) -> impl Iterator> + 'a { + pub fn write_sets_of(&self, position: &Hash) -> Vec> { self.versions .iter() - .map(move |x| x.nodes_of(position, x.replication_factor)) + .map(|x| x.nodes_of(position, x.replication_factor).collect()) + .collect() + } + + pub fn storage_nodes_of(&self, position: &Hash) -> Vec { + let mut ret = vec![]; + for version in self.versions.iter() { + ret.extend(version.nodes_of(position, version.replication_factor)); + } + ret.sort(); + ret.dedup(); + ret } // ------------------ update tracking --------------- diff --git a/src/rpc/layout/version.rs b/src/rpc/layout/version.rs index 2cbdcee2..912ee538 100644 --- a/src/rpc/layout/version.rs +++ b/src/rpc/layout/version.rs @@ -107,25 +107,24 @@ impl LayoutVersion { } /// Return the n servers in which data for this hash should be replicated - pub fn nodes_of(&self, position: &Hash, n: usize) -> Vec { + pub fn nodes_of(&self, position: &Hash, n: usize) -> impl Iterator + '_ { assert_eq!(n, self.replication_factor); let data = &self.ring_assignment_data; - if data.len() != self.replication_factor * (1 << PARTITION_BITS) { + let partition_nodes = if data.len() == self.replication_factor * (1 << PARTITION_BITS) { + let partition_idx = self.partition_of(position) as usize; + let partition_start = partition_idx * self.replication_factor; + let partition_end = (partition_idx + 1) * self.replication_factor; + &data[partition_start..partition_end] + } else { warn!("Ring not yet ready, read/writes will be lost!"); - return vec![]; - } - - let partition_idx = self.partition_of(position) as usize; - let partition_start = partition_idx * self.replication_factor; - let partition_end = (partition_idx + 1) * self.replication_factor; - let partition_nodes = &data[partition_start..partition_end]; + &[] + }; partition_nodes .iter() - .map(|i| self.node_id_vec[*i as usize]) - .collect::>() + .map(move |i| self.node_id_vec[*i as usize]) } // ===================== internal information extractors ====================== diff --git a/src/rpc/system.rs b/src/rpc/system.rs index 86c02e86..31d78bf6 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -449,8 +449,7 @@ impl System { .iter() .map(|(_, h)| { let pn = layout.current().nodes_of(h, replication_factor); - pn.iter() - .filter(|x| nodes.get(x).map(|n| n.is_up).unwrap_or(false)) + pn.filter(|x| nodes.get(x).map(|n| n.is_up).unwrap_or(false)) .count() }) .collect::>(); diff --git a/src/table/data.rs b/src/table/data.rs index bbfdf58b..7f6b7847 100644 --- a/src/table/data.rs +++ b/src/table/data.rs @@ -254,7 +254,8 @@ impl TableData { // 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); + // TODO: this probably breaks when the layout changes + let nodes = self.replication.storage_nodes(&pk_hash); if nodes.first() == Some(&self.system.id) { GcTodoEntry::new(tree_key, new_bytes_hash).save(&self.gc_todo)?; } diff --git a/src/table/gc.rs b/src/table/gc.rs index 2135a358..002cfbf4 100644 --- a/src/table/gc.rs +++ b/src/table/gc.rs @@ -152,7 +152,7 @@ impl TableGc { let mut partitions = HashMap::new(); for entry in entries { let pkh = Hash::try_from(&entry.key[..32]).unwrap(); - let mut nodes = self.data.replication.write_nodes(&pkh); + let mut nodes = self.data.replication.storage_nodes(&pkh); nodes.retain(|x| *x != self.system.id); nodes.sort(); diff --git a/src/table/replication/fullcopy.rs b/src/table/replication/fullcopy.rs index beaacc2b..cb5471af 100644 --- a/src/table/replication/fullcopy.rs +++ b/src/table/replication/fullcopy.rs @@ -27,6 +27,11 @@ pub struct TableFullReplication { } impl TableReplication for TableFullReplication { + fn storage_nodes(&self, _hash: &Hash) -> Vec { + let layout = self.system.cluster_layout(); + layout.current().all_nodes().to_vec() + } + fn read_nodes(&self, _hash: &Hash) -> Vec { vec![self.system.id] } @@ -34,8 +39,8 @@ impl TableReplication for TableFullReplication { 1 } - fn write_nodes(&self, _hash: &Hash) -> Vec { - self.system.cluster_layout().current().all_nodes().to_vec() + fn write_sets(&self, hash: &Hash) -> Vec> { + vec![self.storage_nodes(hash)] } fn write_quorum(&self) -> usize { let nmembers = self.system.cluster_layout().current().all_nodes().len(); diff --git a/src/table/replication/parameters.rs b/src/table/replication/parameters.rs index 2a7d3585..2f842409 100644 --- a/src/table/replication/parameters.rs +++ b/src/table/replication/parameters.rs @@ -6,21 +6,23 @@ pub trait TableReplication: Send + Sync + 'static { // See examples in table_sharded.rs and table_fullcopy.rs // To understand various replication methods + /// The entire list of all nodes that store a partition + fn storage_nodes(&self, hash: &Hash) -> Vec; + /// Which nodes to send read requests to fn read_nodes(&self, hash: &Hash) -> Vec; /// Responses needed to consider a read succesfull fn read_quorum(&self) -> usize; /// Which nodes to send writes to - fn write_nodes(&self, hash: &Hash) -> Vec; - /// Responses needed to consider a write succesfull + fn write_sets(&self, hash: &Hash) -> Vec>; + /// Responses needed to consider a write succesfull in each set fn write_quorum(&self) -> usize; fn max_write_errors(&self) -> usize; // Accessing partitions, for Merkle tree & sync /// Get partition for data with given hash fn partition_of(&self, hash: &Hash) -> Partition; - /// List of partitions and nodes to sync with in current layout fn sync_partitions(&self) -> SyncPartitions; } diff --git a/src/table/replication/sharded.rs b/src/table/replication/sharded.rs index f02b1d66..1320a189 100644 --- a/src/table/replication/sharded.rs +++ b/src/table/replication/sharded.rs @@ -25,21 +25,19 @@ pub struct TableShardedReplication { } impl TableReplication for TableShardedReplication { + fn storage_nodes(&self, hash: &Hash) -> Vec { + self.system.cluster_layout().storage_nodes_of(hash) + } + fn read_nodes(&self, hash: &Hash) -> Vec { - self.system - .cluster_layout() - .current() - .nodes_of(hash, self.replication_factor) + self.system.cluster_layout().read_nodes_of(hash) } fn read_quorum(&self) -> usize { self.read_quorum } - fn write_nodes(&self, hash: &Hash) -> Vec { - self.system - .cluster_layout() - .current() - .nodes_of(hash, self.replication_factor) + fn write_sets(&self, hash: &Hash) -> Vec> { + self.system.cluster_layout().write_sets_of(hash) } fn write_quorum(&self) -> usize { self.write_quorum @@ -60,13 +58,7 @@ impl TableReplication for TableShardedReplication { .current() .partitions() .map(|(partition, first_hash)| { - let mut storage_nodes = layout - .write_sets_of(&first_hash) - .map(|x| x.into_iter()) - .flatten() - .collect::>(); - storage_nodes.sort(); - storage_nodes.dedup(); + let storage_nodes = layout.storage_nodes_of(&first_hash); SyncPartition { partition, first_hash, diff --git a/src/table/sync.rs b/src/table/sync.rs index 8c21db8b..b67cdd79 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -176,7 +176,7 @@ impl TableSyncer { let nodes = self .data .replication - .write_nodes(begin) + .storage_nodes(begin) .into_iter() .collect::>(); if nodes.contains(&self.system.id) { diff --git a/src/table/table.rs b/src/table/table.rs index 997fd7dc..bf08d5a0 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -119,7 +119,8 @@ impl Table { async fn insert_internal(&self, e: &F::E) -> Result<(), Error> { let hash = e.partition_key().hash(); - let who = self.data.replication.write_nodes(&hash); + // TODO: use write sets + let who = self.data.replication.storage_nodes(&hash); let e_enc = Arc::new(ByteBuf::from(e.encode()?)); let rpc = TableRpc::::Update(vec![e_enc]); @@ -171,7 +172,8 @@ impl Table { for entry in entries.into_iter() { let entry = entry.borrow(); let hash = entry.partition_key().hash(); - let who = self.data.replication.write_nodes(&hash); + // TODO: use write sets + let who = self.data.replication.storage_nodes(&hash); let e_enc = Arc::new(ByteBuf::from(entry.encode()?)); for node in who { call_list.entry(node).or_default().push(e_enc.clone()); -- cgit v1.2.3 From 90e1619b1e9f5d81e59da371f04717f0c4fe5afc Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 14 Nov 2023 15:40:46 +0100 Subject: table: take into account multiple write sets in inserts --- src/block/manager.rs | 7 +- src/block/resync.rs | 2 +- src/model/k2v/rpc.rs | 15 ++- src/rpc/rpc_helper.rs | 278 ++++++++++++++++++++++++++++++-------------------- src/table/gc.rs | 4 +- src/table/table.rs | 17 ++- 6 files changed, 189 insertions(+), 134 deletions(-) diff --git a/src/block/manager.rs b/src/block/manager.rs index 2bb9c23d..0ca8bc31 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -354,8 +354,7 @@ impl BlockManager { /// Send block to nodes that should have it pub async fn rpc_put_block(&self, hash: Hash, data: Bytes) -> Result<(), Error> { - // TODO: use quorums among latest write set - let who = self.replication.storage_nodes(&hash); + let who = self.replication.write_sets(&hash); let (header, bytes) = DataBlock::from_buffer(data, self.compression_level) .await @@ -365,9 +364,9 @@ impl BlockManager { self.system .rpc_helper() - .try_call_many( + .try_write_many_sets( &self.endpoint, - &who[..], + &who, put_block_rpc, RequestStrategy::with_priority(PRIO_NORMAL | PRIO_SECONDARY) .with_quorum(self.replication.write_quorum()), diff --git a/src/block/resync.rs b/src/block/resync.rs index 122d0142..15f210e4 100644 --- a/src/block/resync.rs +++ b/src/block/resync.rs @@ -434,7 +434,7 @@ impl BlockResyncManager { .rpc_helper() .try_call_many( &manager.endpoint, - &need_nodes[..], + &need_nodes, put_block_message, RequestStrategy::with_priority(PRIO_BACKGROUND) .with_quorum(need_nodes.len()), diff --git a/src/model/k2v/rpc.rs b/src/model/k2v/rpc.rs index aa3323d5..863a068a 100644 --- a/src/model/k2v/rpc.rs +++ b/src/model/k2v/rpc.rs @@ -134,16 +134,14 @@ impl K2VRpcHandler { .rpc_helper() .try_call_many( &self.endpoint, - &who[..], + &who, K2VRpc::InsertItem(InsertedItem { partition, sort_key, causal_context, value, }), - RequestStrategy::with_priority(PRIO_NORMAL) - .with_quorum(1) - .interrupt_after_quorum(true), + RequestStrategy::with_priority(PRIO_NORMAL).with_quorum(1), ) .await?; @@ -192,9 +190,7 @@ impl K2VRpcHandler { &self.endpoint, &nodes[..], K2VRpc::InsertManyItems(items), - RequestStrategy::with_priority(PRIO_NORMAL) - .with_quorum(1) - .interrupt_after_quorum(true), + RequestStrategy::with_priority(PRIO_NORMAL).with_quorum(1), ) .await?; Ok::<_, Error>((nodes, resp)) @@ -223,7 +219,7 @@ impl K2VRpcHandler { }, sort_key, }; - // TODO figure this out with write sets, does it still work???? + // TODO figure this out with write sets, is it still appropriate??? let nodes = self .item_table .data @@ -232,7 +228,7 @@ impl K2VRpcHandler { let rpc = self.system.rpc_helper().try_call_many( &self.endpoint, - &nodes[..], + &nodes, K2VRpc::PollItem { key: poll_key, causal_context, @@ -240,6 +236,7 @@ impl K2VRpcHandler { }, RequestStrategy::with_priority(PRIO_NORMAL) .with_quorum(self.item_table.data.replication.read_quorum()) + .send_all_at_once(true) .without_timeout(), ); let timeout_duration = diff --git a/src/rpc/rpc_helper.rs b/src/rpc/rpc_helper.rs index ce291068..12d073b6 100644 --- a/src/rpc/rpc_helper.rs +++ b/src/rpc/rpc_helper.rs @@ -1,4 +1,5 @@ //! Contain structs related to making RPCs +use std::collections::HashMap; use std::sync::{Arc, RwLock}; use std::time::Duration; @@ -35,11 +36,11 @@ const DEFAULT_TIMEOUT: Duration = Duration::from_secs(300); #[derive(Copy, Clone)] pub struct RequestStrategy { /// Min number of response to consider the request successful - pub rs_quorum: Option, - /// Should requests be dropped after enough response are received - pub rs_interrupt_after_quorum: bool, + rs_quorum: Option, + /// Send all requests at once + rs_send_all_at_once: Option, /// Request priority - pub rs_priority: RequestPriority, + rs_priority: RequestPriority, /// Custom timeout for this request rs_timeout: Timeout, } @@ -56,7 +57,7 @@ impl RequestStrategy { pub fn with_priority(prio: RequestPriority) -> Self { RequestStrategy { rs_quorum: None, - rs_interrupt_after_quorum: false, + rs_send_all_at_once: None, rs_priority: prio, rs_timeout: Timeout::Default, } @@ -66,10 +67,9 @@ impl RequestStrategy { self.rs_quorum = Some(quorum); self } - /// Set if requests can be dropped after quorum has been reached - /// In general true for read requests, and false for write - pub fn interrupt_after_quorum(mut self, interrupt: bool) -> Self { - self.rs_interrupt_after_quorum = interrupt; + /// Set quorum to be reached for request + pub fn send_all_at_once(mut self, value: bool) -> Self { + self.rs_send_all_at_once = Some(value); self } /// Deactivate timeout for this request @@ -235,31 +235,19 @@ impl RpcHelper { let quorum = strategy.rs_quorum.unwrap_or(to.len()); let tracer = opentelemetry::global::tracer("garage"); - let span_name = if strategy.rs_interrupt_after_quorum { - format!("RPC {} to {} of {}", endpoint.path(), quorum, to.len()) - } else { - format!( - "RPC {} to {} (quorum {})", - endpoint.path(), - to.len(), - quorum - ) - }; + let span_name = format!("Read RPC {} to {} of {}", endpoint.path(), quorum, to.len()); + let mut span = tracer.start(span_name); span.set_attribute(KeyValue::new("from", format!("{:?}", self.0.our_node_id))); span.set_attribute(KeyValue::new("to", format!("{:?}", to))); span.set_attribute(KeyValue::new("quorum", quorum as i64)); - span.set_attribute(KeyValue::new( - "interrupt_after_quorum", - strategy.rs_interrupt_after_quorum.to_string(), - )); - self.try_call_many_internal(endpoint, to, msg, strategy, quorum) + self.try_call_many_inner(endpoint, to, msg, strategy, quorum) .with_context(Context::current_with_span(span)) .await } - async fn try_call_many_internal( + async fn try_call_many_inner( &self, endpoint: &Arc>, to: &[Uuid], @@ -273,12 +261,20 @@ impl RpcHelper { H: StreamingEndpointHandler + 'static, S: Send + 'static, { - let msg = msg.into_req().map_err(netapp::error::Error::from)?; + // Once quorum is reached, other requests don't matter. + // What we do here is only send the required number of requests + // to reach a quorum, priorizing nodes with the lowest latency. + // When there are errors, we start new requests to compensate. + + // Reorder requests to priorize closeness / low latency + let request_order = self.request_order(to); + let send_all_at_once = strategy.rs_send_all_at_once.unwrap_or(false); // Build future for each request // They are not started now: they are added below in a FuturesUnordered // object that will take care of polling them (see below) - let requests = to.iter().cloned().map(|to| { + let msg = msg.into_req().map_err(netapp::error::Error::from)?; + let mut requests = request_order.into_iter().map(|to| { let self2 = self.clone(); let msg = msg.clone(); let endpoint2 = endpoint.clone(); @@ -291,93 +287,40 @@ impl RpcHelper { let mut successes = vec![]; let mut errors = vec![]; - if strategy.rs_interrupt_after_quorum { - // Case 1: once quorum is reached, other requests don't matter. - // What we do here is only send the required number of requests - // to reach a quorum, priorizing nodes with the lowest latency. - // When there are errors, we start new requests to compensate. - - // Reorder requests to priorize closeness / low latency - let request_order = self.request_order(to); - let mut ord_requests = vec![(); request_order.len()] - .into_iter() - .map(|_| None) - .collect::>(); - for (to, fut) in requests { - let i = request_order.iter().position(|x| *x == to).unwrap(); - ord_requests[i] = Some((to, fut)); + // resp_stream will contain all of the requests that are currently in flight. + // (for the moment none, they will be added in the loop below) + let mut resp_stream = FuturesUnordered::new(); + + // Do some requests and collect results + while successes.len() < quorum { + // If the current set of requests that are running is not enough to possibly + // reach quorum, start some new requests. + while send_all_at_once || successes.len() + resp_stream.len() < quorum { + if let Some((req_to, fut)) = requests.next() { + let tracer = opentelemetry::global::tracer("garage"); + let span = tracer.start(format!("RPC to {:?}", req_to)); + resp_stream.push(tokio::spawn( + fut.with_context(Context::current_with_span(span)), + )); + } else { + break; + } } - // Make an iterator to take requests in their sorted order - let mut requests = ord_requests.into_iter().map(Option::unwrap); - - // resp_stream will contain all of the requests that are currently in flight. - // (for the moment none, they will be added in the loop below) - let mut resp_stream = FuturesUnordered::new(); - - // Do some requests and collect results - 'request_loop: while successes.len() < quorum { - // If the current set of requests that are running is not enough to possibly - // reach quorum, start some new requests. - while successes.len() + resp_stream.len() < quorum { - if let Some((req_to, fut)) = requests.next() { - let tracer = opentelemetry::global::tracer("garage"); - let span = tracer.start(format!("RPC to {:?}", req_to)); - resp_stream.push(tokio::spawn( - fut.with_context(Context::current_with_span(span)), - )); - } else { - // If we have no request to add, we know that we won't ever - // reach quorum: bail out now. - break 'request_loop; - } - } - assert!(!resp_stream.is_empty()); // because of loop invariants + if successes.len() + resp_stream.len() < quorum { + // We know we won't ever reach quorum + break; + } - // Wait for one request to terminate - match resp_stream.next().await.unwrap().unwrap() { - Ok(msg) => { - successes.push(msg); - } - Err(e) => { - errors.push(e); - } + // Wait for one request to terminate + match resp_stream.next().await.unwrap().unwrap() { + Ok(msg) => { + successes.push(msg); } - } - } else { - // Case 2: all of the requests need to be sent in all cases, - // and need to terminate. (this is the case for writes that - // must be spread to n nodes) - // Just start all the requests in parallel and return as soon - // as the quorum is reached. - let mut resp_stream = requests - .map(|(_, fut)| fut) - .collect::>(); - - while let Some(resp) = resp_stream.next().await { - match resp { - Ok(msg) => { - successes.push(msg); - if successes.len() >= quorum { - break; - } - } - Err(e) => { - errors.push(e); - } + Err(e) => { + errors.push(e); } } - - if !resp_stream.is_empty() { - // Continue remaining requests in background. - // Note: these requests can get interrupted on process shutdown, - // we must not count on them being executed for certain. - // For all background things that have to happen with certainty, - // they have to be put in a proper queue that is persisted to disk. - tokio::spawn(async move { - resp_stream.collect::>>().await; - }); - } } if successes.len() >= quorum { @@ -432,4 +375,123 @@ impl RpcHelper { .map(|(_, _, _, to)| to) .collect::>() } + + pub async fn try_write_many_sets( + &self, + endpoint: &Arc>, + to_sets: &[Vec], + msg: N, + strategy: RequestStrategy, + ) -> Result, Error> + where + M: Rpc> + 'static, + N: IntoReq, + H: StreamingEndpointHandler + 'static, + S: Send + 'static, + { + let quorum = strategy + .rs_quorum + .expect("internal error: missing quroum in try_write_many_sets"); + + let tracer = opentelemetry::global::tracer("garage"); + let span_name = format!( + "Write RPC {} (quorum {} in {} sets)", + endpoint.path(), + quorum, + to_sets.len() + ); + + let mut span = tracer.start(span_name); + span.set_attribute(KeyValue::new("from", format!("{:?}", self.0.our_node_id))); + span.set_attribute(KeyValue::new("to", format!("{:?}", to_sets))); + span.set_attribute(KeyValue::new("quorum", quorum as i64)); + + self.try_write_many_sets_inner(endpoint, to_sets, msg, strategy, quorum) + .with_context(Context::current_with_span(span)) + .await + } + + async fn try_write_many_sets_inner( + &self, + endpoint: &Arc>, + to_sets: &[Vec], + msg: N, + strategy: RequestStrategy, + quorum: usize, + ) -> Result, Error> + where + M: Rpc> + 'static, + N: IntoReq, + H: StreamingEndpointHandler + 'static, + S: Send + 'static, + { + let msg = msg.into_req().map_err(netapp::error::Error::from)?; + + let mut peers = HashMap::>::new(); + for (i, set) in to_sets.iter().enumerate() { + for peer in set.iter() { + peers.entry(*peer).or_default().push(i); + } + } + + let requests = peers.iter().map(|(peer, _)| { + let self2 = self.clone(); + let msg = msg.clone(); + let endpoint2 = endpoint.clone(); + let to = *peer; + let tracer = opentelemetry::global::tracer("garage"); + let span = tracer.start(format!("RPC to {:?}", to)); + let fut = async move { (to, self2.call(&endpoint2, to, msg, strategy).await) }; + tokio::spawn(fut.with_context(Context::current_with_span(span))) + }); + let mut resp_stream = requests.collect::>(); + + let mut successes = vec![]; + let mut errors = vec![]; + + let mut set_counters = vec![(0, 0); to_sets.len()]; + + while !resp_stream.is_empty() { + let (node, resp) = resp_stream.next().await.unwrap().unwrap(); + + match resp { + Ok(msg) => { + for set in peers.get(&node).unwrap().iter() { + set_counters[*set].0 += 1; + } + successes.push(msg); + } + Err(e) => { + for set in peers.get(&node).unwrap().iter() { + set_counters[*set].1 += 1; + } + errors.push(e); + } + } + + if set_counters.iter().all(|x| x.0 > quorum) { + // Success + + // Continue all other requets in background + tokio::spawn(async move { + resp_stream.collect::>>().await; + }); + + return Ok(successes); + } + + if set_counters + .iter() + .enumerate() + .any(|(i, x)| x.1 + quorum > to_sets[i].len()) + { + // Too many errors in this set, we know we won't get a quorum + break; + } + } + + // Failure, could not get quorum + let errors = errors.iter().map(|e| format!("{}", e)).collect::>(); + Err(Error::Quorum(quorum, successes.len(), peers.len(), errors)) + } } diff --git a/src/table/gc.rs b/src/table/gc.rs index 002cfbf4..ef788749 100644 --- a/src/table/gc.rs +++ b/src/table/gc.rs @@ -230,7 +230,7 @@ impl TableGc { .rpc_helper() .try_call_many( &self.endpoint, - &nodes[..], + &nodes, GcRpc::Update(updates), RequestStrategy::with_priority(PRIO_BACKGROUND).with_quorum(nodes.len()), ) @@ -251,7 +251,7 @@ impl TableGc { .rpc_helper() .try_call_many( &self.endpoint, - &nodes[..], + &nodes, GcRpc::DeleteIfEqualHash(deletes), RequestStrategy::with_priority(PRIO_BACKGROUND).with_quorum(nodes.len()), ) diff --git a/src/table/table.rs b/src/table/table.rs index bf08d5a0..c2efaeaf 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -119,17 +119,16 @@ impl Table { async fn insert_internal(&self, e: &F::E) -> Result<(), Error> { let hash = e.partition_key().hash(); - // TODO: use write sets - let who = self.data.replication.storage_nodes(&hash); + let who = self.data.replication.write_sets(&hash); let e_enc = Arc::new(ByteBuf::from(e.encode()?)); let rpc = TableRpc::::Update(vec![e_enc]); self.system .rpc_helper() - .try_call_many( + .try_write_many_sets( &self.endpoint, - &who[..], + &who, rpc, RequestStrategy::with_priority(PRIO_NORMAL) .with_quorum(self.data.replication.write_quorum()), @@ -243,11 +242,10 @@ impl Table { .rpc_helper() .try_call_many( &self.endpoint, - &who[..], + &who, rpc, RequestStrategy::with_priority(PRIO_NORMAL) - .with_quorum(self.data.replication.read_quorum()) - .interrupt_after_quorum(true), + .with_quorum(self.data.replication.read_quorum()), ) .await?; @@ -339,11 +337,10 @@ impl Table { .rpc_helper() .try_call_many( &self.endpoint, - &who[..], + &who, rpc, RequestStrategy::with_priority(PRIO_NORMAL) - .with_quorum(self.data.replication.read_quorum()) - .interrupt_after_quorum(true), + .with_quorum(self.data.replication.read_quorum()), ) .await?; -- cgit v1.2.3 From 7ef2c231208073db5a0a0a8674e2dd2d2ecb2222 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 14 Nov 2023 15:45:01 +0100 Subject: layout: fix test --- src/rpc/layout/test.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/rpc/layout/test.rs b/src/rpc/layout/test.rs index 0ce090d2..e9639073 100644 --- a/src/rpc/layout/test.rs +++ b/src/rpc/layout/test.rs @@ -35,7 +35,7 @@ fn check_against_naive(cl: &LayoutVersion) -> Result { } for uuid in cl.nongateway_nodes() { let z = cl.get_node_zone(&uuid)?; - let c = cl.get_node_capacity(&uuid)?; + let c = cl.get_node_capacity(&uuid).unwrap(); zone_token.insert( z.to_string(), zone_token[z] + min(NB_PARTITIONS, (c / over_size) as usize), -- cgit v1.2.3 From b3e729f4b8ec3b06593f8d3b161c76b1263d9f13 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 15 Nov 2023 12:15:58 +0100 Subject: layout history merge: rm invalid versions when valid versions are added --- src/rpc/layout/history.rs | 18 ++++++++++++++++++ src/rpc/layout/version.rs | 20 ++++++++++---------- 2 files changed, 28 insertions(+), 10 deletions(-) diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index dce492c9..2346b14a 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -211,6 +211,24 @@ impl LayoutHistory { changed = changed || c; } + // If there are invalid versions before valid versions, remove them, + // and increment update trackers + if self.versions.len() > 1 && self.current().check().is_ok() { + while self.versions.first().unwrap().check().is_err() { + self.versions.remove(0); + changed = true; + } + if changed { + let min_v = self.versions.first().unwrap().version; + let nodes = self.all_nongateway_nodes().into_owned(); + for node in nodes { + self.update_trackers.ack_map.set_max(node, min_v); + self.update_trackers.sync_map.set_max(node, min_v); + self.update_trackers.sync_ack_map.set_max(node, min_v); + } + } + } + // Merge staged layout changes if self.staging != other.staging { self.staging.merge(&other.staging); diff --git a/src/rpc/layout/version.rs b/src/rpc/layout/version.rs index 912ee538..947fab56 100644 --- a/src/rpc/layout/version.rs +++ b/src/rpc/layout/version.rs @@ -174,6 +174,16 @@ impl LayoutVersion { /// (assignment, roles, parameters, partition size) /// returns true if consistent, false if error pub fn check(&self) -> Result<(), String> { + // Check that the assignment data has the correct length + let expected_assignment_data_len = (1 << PARTITION_BITS) * self.replication_factor; + if self.ring_assignment_data.len() != expected_assignment_data_len { + return Err(format!( + "ring_assignment_data has incorrect length {} instead of {}", + self.ring_assignment_data.len(), + expected_assignment_data_len + )); + } + // Check that node_id_vec contains the correct list of nodes let mut expected_nodes = self .roles @@ -189,16 +199,6 @@ impl LayoutVersion { return Err(format!("node_id_vec does not contain the correct set of nodes\nnode_id_vec: {:?}\nexpected: {:?}", node_id_vec, expected_nodes)); } - // Check that the assignment data has the correct length - let expected_assignment_data_len = (1 << PARTITION_BITS) * self.replication_factor; - if self.ring_assignment_data.len() != expected_assignment_data_len { - return Err(format!( - "ring_assignment_data has incorrect length {} instead of {}", - self.ring_assignment_data.len(), - expected_assignment_data_len - )); - } - // Check that the assigned nodes are correct identifiers // of nodes that are assigned a role // and that role is not the role of a gateway nodes -- cgit v1.2.3 From 46007bf01dd2e5b489d145ca0a5499ffa7257b96 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 15 Nov 2023 12:56:52 +0100 Subject: integration test: print stdout and stderr on subcommand crash --- src/garage/tests/common/ext/process.rs | 44 +++++++++------------------------- src/garage/tests/common/garage.rs | 2 +- 2 files changed, 12 insertions(+), 34 deletions(-) diff --git a/src/garage/tests/common/ext/process.rs b/src/garage/tests/common/ext/process.rs index ba533b6c..8e20bf7c 100644 --- a/src/garage/tests/common/ext/process.rs +++ b/src/garage/tests/common/ext/process.rs @@ -14,42 +14,20 @@ impl CommandExt for process::Command { } fn expect_success_status(&mut self, msg: &str) -> process::ExitStatus { - let status = self.status().expect(msg); - status.expect_success(msg); - status + self.expect_success_output(msg).status } fn expect_success_output(&mut self, msg: &str) -> process::Output { let output = self.output().expect(msg); - output.expect_success(msg); - output - } -} - -pub trait OutputExt { - fn expect_success(&self, msg: &str); -} - -impl OutputExt for process::Output { - fn expect_success(&self, msg: &str) { - self.status.expect_success(msg) - } -} - -pub trait ExitStatusExt { - fn expect_success(&self, msg: &str); -} - -impl ExitStatusExt for process::ExitStatus { - fn expect_success(&self, msg: &str) { - if !self.success() { - match self.code() { - Some(code) => panic!( - "Command exited with code {code}: {msg}", - code = code, - msg = msg - ), - None => panic!("Command exited with signal: {msg}", msg = msg), - } + if !output.status.success() { + panic!( + "{}: command {:?} exited with error {:?}\nSTDOUT: {}\nSTDERR: {}", + msg, + self, + output.status.code(), + String::from_utf8_lossy(&output.stdout), + String::from_utf8_lossy(&output.stderr) + ); } + output } } diff --git a/src/garage/tests/common/garage.rs b/src/garage/tests/common/garage.rs index d1f0867a..ebc82f37 100644 --- a/src/garage/tests/common/garage.rs +++ b/src/garage/tests/common/garage.rs @@ -96,7 +96,7 @@ api_bind_addr = "127.0.0.1:{admin_port}" .arg("server") .stdout(stdout) .stderr(stderr) - .env("RUST_LOG", "garage=info,garage_api=trace") + .env("RUST_LOG", "garage=debug,garage_api=trace") .spawn() .expect("Could not start garage"); -- cgit v1.2.3 From acd49de9f97bd27409232691262bd5827983388d Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 15 Nov 2023 13:07:42 +0100 Subject: rpc: fix write set quorums --- src/api/common_error.rs | 8 ++------ src/model/k2v/rpc.rs | 2 +- src/rpc/rpc_helper.rs | 18 +++++++++++++++--- src/util/error.rs | 7 ++++--- 4 files changed, 22 insertions(+), 13 deletions(-) diff --git a/src/api/common_error.rs b/src/api/common_error.rs index 20f9f266..ecb22fd8 100644 --- a/src/api/common_error.rs +++ b/src/api/common_error.rs @@ -53,9 +53,7 @@ impl CommonError { pub fn http_status_code(&self) -> StatusCode { match self { CommonError::InternalError( - GarageError::Timeout - | GarageError::RemoteError(_) - | GarageError::Quorum(_, _, _, _), + GarageError::Timeout | GarageError::RemoteError(_) | GarageError::Quorum(..), ) => StatusCode::SERVICE_UNAVAILABLE, CommonError::InternalError(_) | CommonError::Hyper(_) | CommonError::Http(_) => { StatusCode::INTERNAL_SERVER_ERROR @@ -72,9 +70,7 @@ impl CommonError { match self { CommonError::Forbidden(_) => "AccessDenied", CommonError::InternalError( - GarageError::Timeout - | GarageError::RemoteError(_) - | GarageError::Quorum(_, _, _, _), + GarageError::Timeout | GarageError::RemoteError(_) | GarageError::Quorum(..), ) => "ServiceUnavailable", CommonError::InternalError(_) | CommonError::Hyper(_) | CommonError::Http(_) => { "InternalError" diff --git a/src/model/k2v/rpc.rs b/src/model/k2v/rpc.rs index 863a068a..3c759181 100644 --- a/src/model/k2v/rpc.rs +++ b/src/model/k2v/rpc.rs @@ -344,7 +344,7 @@ impl K2VRpcHandler { } if errors.len() > nodes.len() - quorum { let errors = errors.iter().map(|e| format!("{}", e)).collect::>(); - return Err(Error::Quorum(quorum, resps.len(), nodes.len(), errors).into()); + return Err(Error::Quorum(quorum, None, resps.len(), nodes.len(), errors).into()); } // Take all returned items into account to produce the response. diff --git a/src/rpc/rpc_helper.rs b/src/rpc/rpc_helper.rs index 12d073b6..1bad495b 100644 --- a/src/rpc/rpc_helper.rs +++ b/src/rpc/rpc_helper.rs @@ -327,7 +327,13 @@ impl RpcHelper { Ok(successes) } else { let errors = errors.iter().map(|e| format!("{}", e)).collect::>(); - Err(Error::Quorum(quorum, successes.len(), to.len(), errors)) + Err(Error::Quorum( + quorum, + None, + successes.len(), + to.len(), + errors, + )) } } @@ -469,7 +475,7 @@ impl RpcHelper { } } - if set_counters.iter().all(|x| x.0 > quorum) { + if set_counters.iter().all(|x| x.0 >= quorum) { // Success // Continue all other requets in background @@ -492,6 +498,12 @@ impl RpcHelper { // Failure, could not get quorum let errors = errors.iter().map(|e| format!("{}", e)).collect::>(); - Err(Error::Quorum(quorum, successes.len(), peers.len(), errors)) + Err(Error::Quorum( + quorum, + Some(to_sets.len()), + successes.len(), + peers.len(), + errors, + )) } } diff --git a/src/util/error.rs b/src/util/error.rs index 3fcee71d..be7cdfdb 100644 --- a/src/util/error.rs +++ b/src/util/error.rs @@ -55,13 +55,14 @@ pub enum Error { Timeout, #[error( - display = "Could not reach quorum of {}. {} of {} request succeeded, others returned errors: {:?}", + display = "Could not reach quorum of {} (sets={:?}). {} of {} request succeeded, others returned errors: {:?}", _0, _1, _2, - _3 + _3, + _4 )] - Quorum(usize, usize, usize, Vec), + Quorum(usize, Option, usize, usize, Vec), #[error(display = "Unexpected RPC message: {}", _0)] UnexpectedRpcMessage(String), -- cgit v1.2.3 From 65066c70640371cc318faddfb4c05c96de18e86d Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 15 Nov 2023 13:28:30 +0100 Subject: layout: wip cache global mins --- src/rpc/layout/history.rs | 46 +++++++++++++++++++++++++++------------------- src/rpc/layout/manager.rs | 6 +++--- src/rpc/layout/schema.rs | 36 +++++++++++++++++++++++++++--------- 3 files changed, 57 insertions(+), 31 deletions(-) diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index 2346b14a..1684918e 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -9,6 +9,7 @@ use garage_util::error::*; use super::schema::*; use super::*; + impl LayoutHistory { pub fn new(replication_factor: usize) -> Self { let version = LayoutVersion::new(replication_factor); @@ -49,7 +50,7 @@ impl LayoutHistory { // ------------------ who stores what now? --------------- pub fn all_ack(&self) -> u64 { - self.calculate_global_min(&self.update_trackers.ack_map) + self.update_trackers.ack_map.current_min } pub fn min_stored(&self) -> u64 { @@ -91,7 +92,7 @@ impl LayoutHistory { } pub fn read_nodes_of(&self, position: &Hash) -> Vec { - let sync_min = self.calculate_global_min(&self.update_trackers.sync_map); + let sync_min = self.update_trackers.sync_map.current_min; let version = self .versions .iter() @@ -122,7 +123,7 @@ impl LayoutHistory { // ------------------ update tracking --------------- - pub(crate) fn update_trackers(&mut self, node_id: Uuid) { + pub(crate) fn update_trackers_of(&mut self, node_id: Uuid) { // Ensure trackers for this node's values are up-to-date // 1. Acknowledge the last layout version in the history @@ -138,6 +139,9 @@ impl LayoutHistory { // 4. Cleanup layout versions that are not needed anymore self.cleanup_old_versions(); + // 5. Recalculate global minima + self.update_trackers_min(); + info!("ack_map: {:?}", self.update_trackers.ack_map); info!("sync_map: {:?}", self.update_trackers.sync_map); info!("sync_ack_map: {:?}", self.update_trackers.sync_ack_map); @@ -146,42 +150,41 @@ impl LayoutHistory { self.update_hashes(); } + fn update_trackers_min(&mut self) { + // TODO: for TableFullReplication, counting gateway nodes might be + // necessary? Think about this more. + let storage_nodes = self.all_nongateway_nodes().into_owned(); + let min_version = self.versions.first().unwrap().version; + self.update_trackers.update_min(&storage_nodes, min_version); + } + pub(crate) fn ack_last(&mut self, node: Uuid) { let last_version = self.current().version; self.update_trackers.ack_map.set_max(node, last_version); + self.update_trackers_min(); } pub(crate) fn sync_first(&mut self, node: Uuid) { let first_version = self.versions.first().as_ref().unwrap().version; self.update_trackers.sync_map.set_max(node, first_version); + self.update_trackers_min(); } pub(crate) fn sync_ack(&mut self, node: Uuid) { - self.update_trackers.sync_ack_map.set_max( - node, - self.calculate_global_min(&self.update_trackers.sync_map), - ); + self.update_trackers + .sync_ack_map + .set_max(node, self.update_trackers.sync_map.current_min); + self.update_trackers_min(); } pub(crate) fn cleanup_old_versions(&mut self) { - let min_sync_ack = self.calculate_global_min(&self.update_trackers.sync_ack_map); + let min_sync_ack = self.update_trackers.sync_ack_map.current_min; while self.versions.first().as_ref().unwrap().version < min_sync_ack { let removed = self.versions.remove(0); info!("Layout history: pruning old version {}", removed.version); } } - pub(crate) fn calculate_global_min(&self, tracker: &UpdateTracker) -> u64 { - // TODO: for TableFullReplication, counting gateway nodes might be - // necessary? Think about this more. - let storage_nodes = self.all_nongateway_nodes(); - storage_nodes - .iter() - .map(|x| tracker.0.get(x).copied().unwrap_or(0)) - .min() - .unwrap_or(0) - } - // ================== updates to layout, public interface =================== pub fn merge(&mut self, other: &LayoutHistory) -> bool { @@ -229,6 +232,11 @@ impl LayoutHistory { } } + // Update the current_min value in trackers if anything changed + if changed { + self.update_trackers_min(); + } + // Merge staged layout changes if self.staging != other.staging { self.staging.merge(&other.staging); diff --git a/src/rpc/layout/manager.rs b/src/rpc/layout/manager.rs index ce8b6f61..21ec2d8d 100644 --- a/src/rpc/layout/manager.rs +++ b/src/rpc/layout/manager.rs @@ -74,7 +74,7 @@ impl LayoutManager { } }; - cluster_layout.update_trackers(node_id.into()); + cluster_layout.update_trackers_of(node_id.into()); let layout = Arc::new(RwLock::new(cluster_layout)); let change_notify = Arc::new(Notify::new()); @@ -158,7 +158,7 @@ impl LayoutManager { if !prev_layout_check || adv.check().is_ok() { if layout.merge(adv) { - layout.update_trackers(self.node_id); + layout.update_trackers_of(self.node_id); if prev_layout_check && layout.check().is_err() { panic!("Merged two correct layouts and got an incorrect layout."); } @@ -172,7 +172,7 @@ impl LayoutManager { let mut layout = self.layout.write().unwrap(); if layout.update_trackers != *adv { if layout.update_trackers.merge(adv) { - layout.update_trackers(self.node_id); + layout.update_trackers_of(self.node_id); return Some(layout.update_trackers.clone()); } } diff --git a/src/rpc/layout/schema.rs b/src/rpc/layout/schema.rs index 79440a47..969f5a0b 100644 --- a/src/rpc/layout/schema.rs +++ b/src/rpc/layout/schema.rs @@ -260,7 +260,10 @@ mod v010 { /// The history of cluster layouts #[derive(Clone, Debug, Serialize, Deserialize, Default, PartialEq)] - pub struct UpdateTracker(pub BTreeMap); + pub struct UpdateTracker { + pub values: BTreeMap, + pub current_min: u64, + } impl garage_util::migrate::Migrate for LayoutHistory { const VERSION_MARKER: &'static [u8] = b"G010lh"; @@ -290,14 +293,15 @@ mod v010 { nongateway_node_count, ring_assignment_data: previous.ring_assignment_data, }; - let update_tracker = UpdateTracker( - version + let update_tracker = UpdateTracker { + values: version .nongateway_nodes() .iter() .copied() .map(|x| (x, version.version)) .collect::>(), - ); + current_min: 0, + }; let staging = LayoutStaging { parameters: previous.staging_parameters, roles: previous.staging_roles, @@ -378,14 +382,14 @@ impl core::str::FromStr for ZoneRedundancy { impl UpdateTracker { fn merge(&mut self, other: &UpdateTracker) -> bool { let mut changed = false; - for (k, v) in other.0.iter() { - if let Some(v_mut) = self.0.get_mut(k) { + for (k, v) in other.values.iter() { + if let Some(v_mut) = self.values.get_mut(k) { if *v > *v_mut { *v_mut = *v; changed = true; } } else { - self.0.insert(*k, *v); + self.values.insert(*k, *v); changed = true; } } @@ -393,18 +397,26 @@ impl UpdateTracker { } pub(crate) fn set_max(&mut self, peer: Uuid, value: u64) -> bool { - match self.0.get_mut(&peer) { + match self.values.get_mut(&peer) { Some(e) if *e < value => { *e = value; true } None => { - self.0.insert(peer, value); + self.values.insert(peer, value); true } _ => false, } } + + fn update_min(&mut self, storage_nodes: &[Uuid], min_version: u64) { + self.current_min = storage_nodes + .iter() + .map(|x| self.values.get(x).copied().unwrap_or(min_version)) + .min() + .unwrap_or(min_version) + } } impl UpdateTrackers { @@ -414,4 +426,10 @@ impl UpdateTrackers { let c3 = self.sync_ack_map.merge(&other.sync_ack_map); c1 || c2 || c3 } + + pub(crate) fn update_min(&mut self, storage_nodes: &[Uuid], min_version: u64) { + self.ack_map.update_min(&storage_nodes, min_version); + self.sync_map.update_min(&storage_nodes, min_version); + self.sync_ack_map.update_min(&storage_nodes, min_version); + } } -- cgit v1.2.3 From 393c4d4515e0cdadadc8de8ae2df12e4371cff88 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 15 Nov 2023 14:20:50 +0100 Subject: layout: add helper for cached/external values to centralize recomputation --- src/api/admin/cluster.rs | 1 - src/api/k2v/index.rs | 2 +- src/garage/cli/layout.rs | 3 +- src/model/helper/bucket.rs | 2 +- src/model/index_counter.rs | 4 +- src/rpc/layout/history.rs | 311 +++++++++++++++++++++++++++------------------ src/rpc/layout/manager.rs | 22 ++-- src/rpc/layout/schema.rs | 48 +++---- src/rpc/rpc_helper.rs | 6 +- src/rpc/system.rs | 4 +- 10 files changed, 222 insertions(+), 181 deletions(-) diff --git a/src/api/admin/cluster.rs b/src/api/admin/cluster.rs index d912b58f..593bd778 100644 --- a/src/api/admin/cluster.rs +++ b/src/api/admin/cluster.rs @@ -240,7 +240,6 @@ pub async fn handle_update_cluster_layout( .merge(&roles.update_mutator(node, layout::NodeRoleV(new_role))); } - layout.update_hashes(); garage .system .layout_manager diff --git a/src/api/k2v/index.rs b/src/api/k2v/index.rs index c189232a..e8cd1fba 100644 --- a/src/api/k2v/index.rs +++ b/src/api/k2v/index.rs @@ -29,7 +29,7 @@ pub async fn handle_read_index( .system .cluster_layout() .all_nongateway_nodes() - .into_owned(); + .to_vec(); let (partition_keys, more, next_start) = read_range( &garage.k2v.counter_table.table, diff --git a/src/garage/cli/layout.rs b/src/garage/cli/layout.rs index 0f01a37a..51774314 100644 --- a/src/garage/cli/layout.rs +++ b/src/garage/cli/layout.rs @@ -49,7 +49,7 @@ pub async fn cmd_assign_role( }; let mut layout = fetch_layout(rpc_cli, rpc_host).await?; - let all_nodes = layout.all_nodes().into_owned(); + let all_nodes = layout.get_all_nodes(); let added_nodes = args .node_ids @@ -331,7 +331,6 @@ pub async fn send_layout( rpc_host: NodeID, mut layout: LayoutHistory, ) -> Result<(), Error> { - layout.update_hashes(); rpc_cli .call( &rpc_host, diff --git a/src/model/helper/bucket.rs b/src/model/helper/bucket.rs index 2cb53424..efa3e27b 100644 --- a/src/model/helper/bucket.rs +++ b/src/model/helper/bucket.rs @@ -455,7 +455,7 @@ impl<'a> BucketHelper<'a> { .system .cluster_layout() .all_nongateway_nodes() - .into_owned(); + .to_vec(); let k2vindexes = self .0 .k2v diff --git a/src/model/index_counter.rs b/src/model/index_counter.rs index 2d968733..e8702bf1 100644 --- a/src/model/index_counter.rs +++ b/src/model/index_counter.rs @@ -7,7 +7,7 @@ use serde::{Deserialize, Serialize}; use garage_db as db; -use garage_rpc::layout::LayoutHistory; +use garage_rpc::layout::LayoutHelper; use garage_rpc::system::System; use garage_util::background::BackgroundRunner; use garage_util::data::*; @@ -83,7 +83,7 @@ impl Entry for CounterEntry { } impl CounterEntry { - pub fn filtered_values(&self, layout: &LayoutHistory) -> HashMap { + pub fn filtered_values(&self, layout: &LayoutHelper) -> HashMap { let nodes = layout.all_nongateway_nodes(); self.filtered_values_with_nodes(&nodes) } diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index 1684918e..b6f0e495 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -1,5 +1,5 @@ -use std::borrow::Cow; use std::collections::HashSet; +use std::ops::Deref; use garage_util::crdt::{Crdt, Lww, LwwMap}; use garage_util::data::*; @@ -9,95 +9,106 @@ use garage_util::error::*; use super::schema::*; use super::*; +pub struct LayoutHelper { + layout: Option, -impl LayoutHistory { - pub fn new(replication_factor: usize) -> Self { - let version = LayoutVersion::new(replication_factor); + // cached values + ack_map_min: u64, + sync_map_min: u64, - let staging = LayoutStaging { - parameters: Lww::::new(version.parameters), - roles: LwwMap::new(), - }; + all_nodes: Vec, + all_nongateway_nodes: Vec, - let mut ret = LayoutHistory { - versions: vec![version], - update_trackers: Default::default(), - trackers_hash: [0u8; 32].into(), - staging: Lww::raw(0, staging), - staging_hash: [0u8; 32].into(), - }; - ret.update_hashes(); - ret - } + trackers_hash: Hash, + staging_hash: Hash, +} - pub fn current(&self) -> &LayoutVersion { - self.versions.last().as_ref().unwrap() +impl Deref for LayoutHelper { + type Target = LayoutHistory; + fn deref(&self) -> &LayoutHistory { + self.layout() } +} - pub fn update_hashes(&mut self) { - self.trackers_hash = self.calculate_trackers_hash(); - self.staging_hash = self.calculate_staging_hash(); +impl LayoutHelper { + pub fn new(mut layout: LayoutHistory) -> Self { + layout.cleanup_old_versions(); + + let all_nongateway_nodes = layout.get_all_nongateway_nodes(); + layout.clamp_update_trackers(&all_nongateway_nodes); + + let min_version = layout.min_stored(); + let ack_map_min = layout + .update_trackers + .ack_map + .min(&all_nongateway_nodes, min_version); + let sync_map_min = layout + .update_trackers + .sync_map + .min(&all_nongateway_nodes, min_version); + + let all_nodes = layout.get_all_nodes(); + let trackers_hash = layout.calculate_trackers_hash(); + let staging_hash = layout.calculate_staging_hash(); + + LayoutHelper { + layout: Some(layout), + ack_map_min, + sync_map_min, + all_nodes, + all_nongateway_nodes, + trackers_hash, + staging_hash, + } } - pub(crate) fn calculate_trackers_hash(&self) -> Hash { - blake2sum(&nonversioned_encode(&self.update_trackers).unwrap()[..]) - } + // ------------------ single updating function -------------- - pub(crate) fn calculate_staging_hash(&self) -> Hash { - blake2sum(&nonversioned_encode(&self.staging).unwrap()[..]) + fn layout(&self) -> &LayoutHistory { + self.layout.as_ref().unwrap() } - // ------------------ who stores what now? --------------- - - pub fn all_ack(&self) -> u64 { - self.update_trackers.ack_map.current_min + pub(crate) fn update(&mut self, f: F) -> bool + where + F: FnOnce(&mut LayoutHistory) -> bool, + { + let changed = f(&mut self.layout.as_mut().unwrap()); + if changed { + *self = Self::new(self.layout.take().unwrap()); + } + changed } - pub fn min_stored(&self) -> u64 { - self.versions.first().as_ref().unwrap().version + // ------------------ read helpers --------------- + + pub fn all_nodes(&self) -> &[Uuid] { + &self.all_nodes } - pub fn sync_versions(&self) -> (u64, u64, u64) { - (self.current().version, self.all_ack(), self.min_stored()) + pub fn all_nongateway_nodes(&self) -> &[Uuid] { + &self.all_nongateway_nodes } - pub fn all_nodes(&self) -> Cow<'_, [Uuid]> { - // TODO: cache this - if self.versions.len() == 1 { - self.versions[0].all_nodes().into() - } else { - let set = self - .versions - .iter() - .map(|x| x.all_nodes()) - .flatten() - .collect::>(); - set.into_iter().copied().collect::>().into() - } + pub fn all_ack(&self) -> u64 { + self.ack_map_min } - pub fn all_nongateway_nodes(&self) -> Cow<'_, [Uuid]> { - // TODO: cache this - if self.versions.len() == 1 { - self.versions[0].nongateway_nodes().into() - } else { - let set = self - .versions - .iter() - .map(|x| x.nongateway_nodes()) - .flatten() - .collect::>(); - set.into_iter().copied().collect::>().into() - } + pub fn sync_versions(&self) -> (u64, u64, u64) { + ( + self.layout().current().version, + self.all_ack(), + self.layout().min_stored(), + ) } pub fn read_nodes_of(&self, position: &Hash) -> Vec { - let sync_min = self.update_trackers.sync_map.current_min; + let sync_min = self.sync_map_min; let version = self + .layout() .versions .iter() .find(|x| x.version == sync_min) - .or(self.versions.last()) + .or(self.layout().versions.last()) .unwrap(); version .nodes_of(position, version.replication_factor) @@ -105,7 +116,8 @@ impl LayoutHistory { } pub fn write_sets_of(&self, position: &Hash) -> Vec> { - self.versions + self.layout() + .versions .iter() .map(|x| x.nodes_of(position, x.replication_factor).collect()) .collect() @@ -113,7 +125,7 @@ impl LayoutHistory { pub fn storage_nodes_of(&self, position: &Hash) -> Vec { let mut ret = vec![]; - for version in self.versions.iter() { + for version in self.layout().versions.iter() { ret.extend(version.nodes_of(position, version.replication_factor)); } ret.sort(); @@ -121,7 +133,35 @@ impl LayoutHistory { ret } - // ------------------ update tracking --------------- + pub fn trackers_hash(&self) -> Hash { + self.trackers_hash + } + + pub fn staging_hash(&self) -> Hash { + self.staging_hash + } + + // ------------------ helpers for update tracking --------------- + + pub(crate) fn sync_first(&mut self, node: Uuid) { + let first_version = self.versions.first().as_ref().unwrap().version; + self.update(|layout| layout.update_trackers.sync_map.set_max(node, first_version)); + } + + pub(crate) fn sync_ack(&mut self, node: Uuid) { + let sync_map_min = self.sync_map_min; + self.update(|layout| { + layout + .update_trackers + .sync_ack_map + .set_max(node, sync_map_min) + }); + } + + pub(crate) fn ack_last(&mut self, node: Uuid) { + let last_version = self.current().version; + self.update(|layout| layout.update_trackers.ack_map.set_max(node, last_version)); + } pub(crate) fn update_trackers_of(&mut self, node_id: Uuid) { // Ensure trackers for this node's values are up-to-date @@ -136,55 +176,104 @@ impl LayoutHistory { // 3. Acknowledge everyone has synced up to min(self.sync_map) self.sync_ack(node_id); - // 4. Cleanup layout versions that are not needed anymore - self.cleanup_old_versions(); - - // 5. Recalculate global minima - self.update_trackers_min(); - info!("ack_map: {:?}", self.update_trackers.ack_map); info!("sync_map: {:?}", self.update_trackers.sync_map); info!("sync_ack_map: {:?}", self.update_trackers.sync_ack_map); + } +} - // Finally, update hashes - self.update_hashes(); +// ---- + +impl LayoutHistory { + pub fn new(replication_factor: usize) -> Self { + let version = LayoutVersion::new(replication_factor); + + let staging = LayoutStaging { + parameters: Lww::::new(version.parameters), + roles: LwwMap::new(), + }; + + LayoutHistory { + versions: vec![version], + update_trackers: Default::default(), + staging: Lww::raw(0, staging), + } } - fn update_trackers_min(&mut self) { - // TODO: for TableFullReplication, counting gateway nodes might be - // necessary? Think about this more. - let storage_nodes = self.all_nongateway_nodes().into_owned(); - let min_version = self.versions.first().unwrap().version; - self.update_trackers.update_min(&storage_nodes, min_version); + // ------------------ who stores what now? --------------- + + pub fn current(&self) -> &LayoutVersion { + self.versions.last().as_ref().unwrap() } - pub(crate) fn ack_last(&mut self, node: Uuid) { - let last_version = self.current().version; - self.update_trackers.ack_map.set_max(node, last_version); - self.update_trackers_min(); + pub fn min_stored(&self) -> u64 { + self.versions.first().as_ref().unwrap().version } - pub(crate) fn sync_first(&mut self, node: Uuid) { - let first_version = self.versions.first().as_ref().unwrap().version; - self.update_trackers.sync_map.set_max(node, first_version); - self.update_trackers_min(); + pub fn get_all_nodes(&self) -> Vec { + if self.versions.len() == 1 { + self.versions[0].all_nodes().to_vec() + } else { + let set = self + .versions + .iter() + .map(|x| x.all_nodes()) + .flatten() + .collect::>(); + set.into_iter().copied().collect::>() + } } - pub(crate) fn sync_ack(&mut self, node: Uuid) { - self.update_trackers - .sync_ack_map - .set_max(node, self.update_trackers.sync_map.current_min); - self.update_trackers_min(); + fn get_all_nongateway_nodes(&self) -> Vec { + if self.versions.len() == 1 { + self.versions[0].nongateway_nodes().to_vec() + } else { + let set = self + .versions + .iter() + .map(|x| x.nongateway_nodes()) + .flatten() + .collect::>(); + set.into_iter().copied().collect::>() + } } - pub(crate) fn cleanup_old_versions(&mut self) { - let min_sync_ack = self.update_trackers.sync_ack_map.current_min; - while self.versions.first().as_ref().unwrap().version < min_sync_ack { - let removed = self.versions.remove(0); - info!("Layout history: pruning old version {}", removed.version); + // ---- housekeeping (all invoked by LayoutHelper) ---- + + fn cleanup_old_versions(&mut self) { + loop { + let all_nongateway_nodes = self.get_all_nongateway_nodes(); + let min_version = self.min_stored(); + let sync_ack_map_min = self + .update_trackers + .sync_ack_map + .min(&all_nongateway_nodes, min_version); + if self.min_stored() < sync_ack_map_min { + let removed = self.versions.remove(0); + info!("Layout history: pruning old version {}", removed.version); + } else { + break; + } } } + fn clamp_update_trackers(&mut self, nodes: &[Uuid]) { + let min_v = self.min_stored(); + for node in nodes { + self.update_trackers.ack_map.set_max(*node, min_v); + self.update_trackers.sync_map.set_max(*node, min_v); + self.update_trackers.sync_ack_map.set_max(*node, min_v); + } + } + + fn calculate_trackers_hash(&self) -> Hash { + blake2sum(&nonversioned_encode(&self.update_trackers).unwrap()[..]) + } + + fn calculate_staging_hash(&self) -> Hash { + blake2sum(&nonversioned_encode(&self.staging).unwrap()[..]) + } + // ================== updates to layout, public interface =================== pub fn merge(&mut self, other: &LayoutHistory) -> bool { @@ -221,20 +310,6 @@ impl LayoutHistory { self.versions.remove(0); changed = true; } - if changed { - let min_v = self.versions.first().unwrap().version; - let nodes = self.all_nongateway_nodes().into_owned(); - for node in nodes { - self.update_trackers.ack_map.set_max(node, min_v); - self.update_trackers.sync_map.set_max(node, min_v); - self.update_trackers.sync_ack_map.set_max(node, min_v); - } - } - } - - // Update the current_min value in trackers if anything changed - if changed { - self.update_trackers_min(); } // Merge staged layout changes @@ -280,7 +355,6 @@ To know the correct value of the new layout version, invoke `garage layout show` parameters: self.staging.get().parameters.clone(), roles: LwwMap::new(), }); - self.update_hashes(); Ok((self, msg)) } @@ -290,20 +364,11 @@ To know the correct value of the new layout version, invoke `garage layout show` parameters: Lww::new(self.current().parameters.clone()), roles: LwwMap::new(), }); - self.update_hashes(); Ok(self) } pub fn check(&self) -> Result<(), String> { - // Check that the hash of the staging data is correct - if self.trackers_hash != self.calculate_trackers_hash() { - return Err("trackers_hash is incorrect".into()); - } - if self.staging_hash != self.calculate_staging_hash() { - return Err("staging_hash is incorrect".into()); - } - for version in self.versions.iter() { version.check()?; } diff --git a/src/rpc/layout/manager.rs b/src/rpc/layout/manager.rs index 21ec2d8d..e270ad21 100644 --- a/src/rpc/layout/manager.rs +++ b/src/rpc/layout/manager.rs @@ -24,7 +24,7 @@ pub struct LayoutManager { replication_factor: usize, persist_cluster_layout: Persister, - layout: Arc>, + layout: Arc>, pub(crate) change_notify: Arc, table_sync_version: Mutex>, @@ -54,7 +54,7 @@ impl LayoutManager { let persist_cluster_layout: Persister = Persister::new(&config.metadata_dir, "cluster_layout"); - let mut cluster_layout = match persist_cluster_layout.load() { + let cluster_layout = match persist_cluster_layout.load() { Ok(x) => { if x.current().replication_factor != replication_factor { return Err(Error::Message(format!( @@ -74,6 +74,7 @@ impl LayoutManager { } }; + let mut cluster_layout = LayoutHelper::new(cluster_layout); cluster_layout.update_trackers_of(node_id.into()); let layout = Arc::new(RwLock::new(cluster_layout)); @@ -100,7 +101,7 @@ impl LayoutManager { // ---- PUBLIC INTERFACE ---- - pub fn layout(&self) -> RwLockReadGuard<'_, LayoutHistory> { + pub fn layout(&self) -> RwLockReadGuard<'_, LayoutHelper> { self.layout.read().unwrap() } @@ -108,8 +109,8 @@ impl LayoutManager { let layout = self.layout(); LayoutStatus { cluster_layout_version: layout.current().version, - cluster_layout_trackers_hash: layout.trackers_hash, - cluster_layout_staging_hash: layout.staging_hash, + cluster_layout_trackers_hash: layout.trackers_hash(), + cluster_layout_staging_hash: layout.staging_hash(), } } @@ -137,13 +138,8 @@ impl LayoutManager { drop(table_sync_version); let mut layout = self.layout.write().unwrap(); - if layout - .update_trackers - .sync_map - .set_max(self.node_id, sync_until) - { + if layout.update(|l| l.update_trackers.sync_map.set_max(self.node_id, sync_until)) { debug!("sync_until updated to {}", sync_until); - layout.update_hashes(); self.broadcast_update(SystemRpc::AdvertiseClusterLayoutTrackers( layout.update_trackers.clone(), )); @@ -157,7 +153,7 @@ impl LayoutManager { let prev_layout_check = layout.check().is_ok(); if !prev_layout_check || adv.check().is_ok() { - if layout.merge(adv) { + if layout.update(|l| l.merge(adv)) { layout.update_trackers_of(self.node_id); if prev_layout_check && layout.check().is_err() { panic!("Merged two correct layouts and got an incorrect layout."); @@ -171,7 +167,7 @@ impl LayoutManager { fn merge_layout_trackers(&self, adv: &UpdateTrackers) -> Option { let mut layout = self.layout.write().unwrap(); if layout.update_trackers != *adv { - if layout.update_trackers.merge(adv) { + if layout.update(|l| l.update_trackers.merge(adv)) { layout.update_trackers_of(self.node_id); return Some(layout.update_trackers.clone()); } diff --git a/src/rpc/layout/schema.rs b/src/rpc/layout/schema.rs index 969f5a0b..00a2c017 100644 --- a/src/rpc/layout/schema.rs +++ b/src/rpc/layout/schema.rs @@ -188,7 +188,7 @@ mod v010 { use super::v09; use crate::layout::CompactNodeType; use garage_util::crdt::{Lww, LwwMap}; - use garage_util::data::{Hash, Uuid}; + use garage_util::data::Uuid; use serde::{Deserialize, Serialize}; use std::collections::BTreeMap; pub use v09::{LayoutParameters, NodeRole, NodeRoleV, ZoneRedundancy}; @@ -202,13 +202,9 @@ mod v010 { /// Update trackers pub update_trackers: UpdateTrackers, - /// Hash of the update trackers - pub trackers_hash: Hash, /// Staged changes for the next version pub staging: Lww, - /// Hash of the serialized staging_parameters + staging_roles - pub staging_hash: Hash, } /// A version of the layout of the cluster, i.e. the list of roles @@ -260,10 +256,7 @@ mod v010 { /// The history of cluster layouts #[derive(Clone, Debug, Serialize, Deserialize, Default, PartialEq)] - pub struct UpdateTracker { - pub values: BTreeMap, - pub current_min: u64, - } + pub struct UpdateTracker(pub BTreeMap); impl garage_util::migrate::Migrate for LayoutHistory { const VERSION_MARKER: &'static [u8] = b"G010lh"; @@ -293,32 +286,27 @@ mod v010 { nongateway_node_count, ring_assignment_data: previous.ring_assignment_data, }; - let update_tracker = UpdateTracker { - values: version + let update_tracker = UpdateTracker( + version .nongateway_nodes() .iter() .copied() .map(|x| (x, version.version)) .collect::>(), - current_min: 0, - }; + ); let staging = LayoutStaging { parameters: previous.staging_parameters, roles: previous.staging_roles, }; - let mut ret = Self { + Self { versions: vec![version], update_trackers: UpdateTrackers { ack_map: update_tracker.clone(), sync_map: update_tracker.clone(), sync_ack_map: update_tracker.clone(), }, - trackers_hash: [0u8; 32].into(), staging: Lww::raw(previous.version, staging), - staging_hash: [0u8; 32].into(), - }; - ret.update_hashes(); - ret + } } } } @@ -382,14 +370,14 @@ impl core::str::FromStr for ZoneRedundancy { impl UpdateTracker { fn merge(&mut self, other: &UpdateTracker) -> bool { let mut changed = false; - for (k, v) in other.values.iter() { - if let Some(v_mut) = self.values.get_mut(k) { + for (k, v) in other.0.iter() { + if let Some(v_mut) = self.0.get_mut(k) { if *v > *v_mut { *v_mut = *v; changed = true; } } else { - self.values.insert(*k, *v); + self.0.insert(*k, *v); changed = true; } } @@ -397,23 +385,23 @@ impl UpdateTracker { } pub(crate) fn set_max(&mut self, peer: Uuid, value: u64) -> bool { - match self.values.get_mut(&peer) { + match self.0.get_mut(&peer) { Some(e) if *e < value => { *e = value; true } None => { - self.values.insert(peer, value); + self.0.insert(peer, value); true } _ => false, } } - fn update_min(&mut self, storage_nodes: &[Uuid], min_version: u64) { - self.current_min = storage_nodes + pub(crate) fn min(&self, storage_nodes: &[Uuid], min_version: u64) -> u64 { + storage_nodes .iter() - .map(|x| self.values.get(x).copied().unwrap_or(min_version)) + .map(|x| self.0.get(x).copied().unwrap_or(min_version)) .min() .unwrap_or(min_version) } @@ -426,10 +414,4 @@ impl UpdateTrackers { let c3 = self.sync_ack_map.merge(&other.sync_ack_map); c1 || c2 || c3 } - - pub(crate) fn update_min(&mut self, storage_nodes: &[Uuid], min_version: u64) { - self.ack_map.update_min(&storage_nodes, min_version); - self.sync_map.update_min(&storage_nodes, min_version); - self.sync_ack_map.update_min(&storage_nodes, min_version); - } } diff --git a/src/rpc/rpc_helper.rs b/src/rpc/rpc_helper.rs index 1bad495b..e269ddaa 100644 --- a/src/rpc/rpc_helper.rs +++ b/src/rpc/rpc_helper.rs @@ -26,7 +26,7 @@ use garage_util::data::*; use garage_util::error::Error; use garage_util::metrics::RecordDuration; -use crate::layout::LayoutHistory; +use crate::layout::LayoutHelper; use crate::metrics::RpcMetrics; // Default RPC timeout = 5 minutes @@ -90,7 +90,7 @@ pub struct RpcHelper(Arc); struct RpcHelperInner { our_node_id: Uuid, fullmesh: Arc, - layout: Arc>, + layout: Arc>, metrics: RpcMetrics, rpc_timeout: Duration, } @@ -99,7 +99,7 @@ impl RpcHelper { pub(crate) fn new( our_node_id: Uuid, fullmesh: Arc, - layout: Arc>, + layout: Arc>, rpc_timeout: Option, ) -> Self { let metrics = RpcMetrics::new(); diff --git a/src/rpc/system.rs b/src/rpc/system.rs index 31d78bf6..d74dc2a1 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -34,7 +34,7 @@ use crate::consul::ConsulDiscovery; #[cfg(feature = "kubernetes-discovery")] use crate::kubernetes::*; use crate::layout::manager::{LayoutManager, LayoutStatus}; -use crate::layout::{self, LayoutHistory, NodeRoleV}; +use crate::layout::{self, LayoutHelper, LayoutHistory, NodeRoleV}; use crate::replication_mode::*; use crate::rpc_helper::*; @@ -350,7 +350,7 @@ impl System { // ---- Public utilities / accessors ---- - pub fn cluster_layout(&self) -> RwLockReadGuard<'_, LayoutHistory> { + pub fn cluster_layout(&self) -> RwLockReadGuard<'_, LayoutHelper> { self.layout_manager.layout() } -- cgit v1.2.3 From 33c8a489b0a9c0e869282bfc19c548f5a3e02e8c Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 15 Nov 2023 15:40:44 +0100 Subject: layou: implement ack locking --- src/block/manager.rs | 2 +- src/garage/cli/layout.rs | 2 +- src/rpc/layout/history.rs | 98 +++++++++++++++++++++++++++---------- src/rpc/layout/manager.rs | 74 +++++++++++++++++++++++++--- src/rpc/layout/mod.rs | 1 + src/table/replication/fullcopy.rs | 4 +- src/table/replication/parameters.rs | 4 +- src/table/replication/sharded.rs | 6 ++- src/table/sync.rs | 9 +--- src/table/table.rs | 2 +- 10 files changed, 156 insertions(+), 46 deletions(-) diff --git a/src/block/manager.rs b/src/block/manager.rs index 0ca8bc31..be2e4951 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -366,7 +366,7 @@ impl BlockManager { .rpc_helper() .try_write_many_sets( &self.endpoint, - &who, + who.as_ref(), put_block_rpc, RequestStrategy::with_priority(PRIO_NORMAL | PRIO_SECONDARY) .with_quorum(self.replication.write_quorum()), diff --git a/src/garage/cli/layout.rs b/src/garage/cli/layout.rs index 51774314..0be8278f 100644 --- a/src/garage/cli/layout.rs +++ b/src/garage/cli/layout.rs @@ -329,7 +329,7 @@ pub async fn fetch_layout( pub async fn send_layout( rpc_cli: &Endpoint, rpc_host: NodeID, - mut layout: LayoutHistory, + layout: LayoutHistory, ) -> Result<(), Error> { rpc_cli .call( diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index b6f0e495..dd38efa7 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -1,5 +1,7 @@ +use std::collections::HashMap; use std::collections::HashSet; use std::ops::Deref; +use std::sync::atomic::{AtomicUsize, Ordering}; use garage_util::crdt::{Crdt, Lww, LwwMap}; use garage_util::data::*; @@ -21,6 +23,11 @@ pub struct LayoutHelper { trackers_hash: Hash, staging_hash: Hash, + + // ack lock: counts in-progress write operations for each + // layout version ; we don't increase the ack update tracker + // while this lock is nonzero + pub(crate) ack_lock: HashMap, } impl Deref for LayoutHelper { @@ -31,7 +38,7 @@ impl Deref for LayoutHelper { } impl LayoutHelper { - pub fn new(mut layout: LayoutHistory) -> Self { + pub fn new(mut layout: LayoutHistory, mut ack_lock: HashMap) -> Self { layout.cleanup_old_versions(); let all_nongateway_nodes = layout.get_all_nongateway_nodes(); @@ -51,6 +58,11 @@ impl LayoutHelper { let trackers_hash = layout.calculate_trackers_hash(); let staging_hash = layout.calculate_staging_hash(); + ack_lock.retain(|_, cnt| *cnt.get_mut() > 0); + ack_lock + .entry(layout.current().version) + .or_insert(AtomicUsize::new(0)); + LayoutHelper { layout: Some(layout), ack_map_min, @@ -59,6 +71,7 @@ impl LayoutHelper { all_nongateway_nodes, trackers_hash, staging_hash, + ack_lock, } } @@ -74,7 +87,10 @@ impl LayoutHelper { { let changed = f(&mut self.layout.as_mut().unwrap()); if changed { - *self = Self::new(self.layout.take().unwrap()); + *self = Self::new( + self.layout.take().unwrap(), + std::mem::take(&mut self.ack_lock), + ); } changed } @@ -115,7 +131,7 @@ impl LayoutHelper { .collect() } - pub fn write_sets_of(&self, position: &Hash) -> Vec> { + pub(crate) fn write_sets_of(&self, position: &Hash) -> Vec> { self.layout() .versions .iter() @@ -143,42 +159,72 @@ impl LayoutHelper { // ------------------ helpers for update tracking --------------- - pub(crate) fn sync_first(&mut self, node: Uuid) { + pub(crate) fn update_trackers(&mut self, local_node_id: Uuid) { + // Ensure trackers for this node's values are up-to-date + + // 1. Acknowledge the last layout version which is not currently + // locked by an in-progress write operation + self.ack_max_free(local_node_id); + + // 2. Assume the data on this node is sync'ed up at least to + // the first layout version in the history + self.sync_first(local_node_id); + + // 3. Acknowledge everyone has synced up to min(self.sync_map) + self.sync_ack(local_node_id); + + info!("ack_map: {:?}", self.update_trackers.ack_map); + info!("sync_map: {:?}", self.update_trackers.sync_map); + info!("sync_ack_map: {:?}", self.update_trackers.sync_ack_map); + } + + fn sync_first(&mut self, local_node_id: Uuid) { let first_version = self.versions.first().as_ref().unwrap().version; - self.update(|layout| layout.update_trackers.sync_map.set_max(node, first_version)); + self.update(|layout| { + layout + .update_trackers + .sync_map + .set_max(local_node_id, first_version) + }); } - pub(crate) fn sync_ack(&mut self, node: Uuid) { + fn sync_ack(&mut self, local_node_id: Uuid) { let sync_map_min = self.sync_map_min; self.update(|layout| { layout .update_trackers .sync_ack_map - .set_max(node, sync_map_min) + .set_max(local_node_id, sync_map_min) }); } - pub(crate) fn ack_last(&mut self, node: Uuid) { - let last_version = self.current().version; - self.update(|layout| layout.update_trackers.ack_map.set_max(node, last_version)); + pub(crate) fn ack_max_free(&mut self, local_node_id: Uuid) -> bool { + let max_ack = self.max_free_ack(); + let changed = self.update(|layout| { + layout + .update_trackers + .ack_map + .set_max(local_node_id, max_ack) + }); + if changed { + info!("ack_until updated to {}", max_ack); + } + changed } - pub(crate) fn update_trackers_of(&mut self, node_id: Uuid) { - // Ensure trackers for this node's values are up-to-date - - // 1. Acknowledge the last layout version in the history - self.ack_last(node_id); - - // 2. Assume the data on this node is sync'ed up at least to - // the first layout version in the history - self.sync_first(node_id); - - // 3. Acknowledge everyone has synced up to min(self.sync_map) - self.sync_ack(node_id); - - info!("ack_map: {:?}", self.update_trackers.ack_map); - info!("sync_map: {:?}", self.update_trackers.sync_map); - info!("sync_ack_map: {:?}", self.update_trackers.sync_ack_map); + pub(crate) fn max_free_ack(&self) -> u64 { + self.layout() + .versions + .iter() + .map(|x| x.version) + .take_while(|v| { + self.ack_lock + .get(v) + .map(|x| x.load(Ordering::Relaxed) == 0) + .unwrap_or(true) + }) + .max() + .unwrap_or(self.min_stored()) } } diff --git a/src/rpc/layout/manager.rs b/src/rpc/layout/manager.rs index e270ad21..4e073d1f 100644 --- a/src/rpc/layout/manager.rs +++ b/src/rpc/layout/manager.rs @@ -1,5 +1,5 @@ use std::collections::HashMap; -use std::sync::{Arc, Mutex, RwLock, RwLockReadGuard}; +use std::sync::{atomic::Ordering, Arc, Mutex, RwLock, RwLockReadGuard}; use std::time::Duration; use serde::{Deserialize, Serialize}; @@ -74,8 +74,8 @@ impl LayoutManager { } }; - let mut cluster_layout = LayoutHelper::new(cluster_layout); - cluster_layout.update_trackers_of(node_id.into()); + let mut cluster_layout = LayoutHelper::new(cluster_layout, Default::default()); + cluster_layout.update_trackers(node_id.into()); let layout = Arc::new(RwLock::new(cluster_layout)); let change_notify = Arc::new(Notify::new()); @@ -139,13 +139,36 @@ impl LayoutManager { let mut layout = self.layout.write().unwrap(); if layout.update(|l| l.update_trackers.sync_map.set_max(self.node_id, sync_until)) { - debug!("sync_until updated to {}", sync_until); + info!("sync_until updated to {}", sync_until); self.broadcast_update(SystemRpc::AdvertiseClusterLayoutTrackers( layout.update_trackers.clone(), )); } } + fn ack_new_version(self: &Arc) { + let mut layout = self.layout.write().unwrap(); + if layout.ack_max_free(self.node_id) { + self.broadcast_update(SystemRpc::AdvertiseClusterLayoutTrackers( + layout.update_trackers.clone(), + )); + } + } + + // ---- ACK LOCKING ---- + + pub fn write_sets_of(self: &Arc, position: &Hash) -> WriteLock>> { + let layout = self.layout(); + let version = layout.current().version; + let nodes = layout.write_sets_of(position); + layout + .ack_lock + .get(&version) + .unwrap() + .fetch_add(1, Ordering::Relaxed); + WriteLock::new(version, self, nodes) + } + // ---- INTERNALS --- fn merge_layout(&self, adv: &LayoutHistory) -> Option { @@ -154,7 +177,7 @@ impl LayoutManager { if !prev_layout_check || adv.check().is_ok() { if layout.update(|l| l.merge(adv)) { - layout.update_trackers_of(self.node_id); + layout.update_trackers(self.node_id); if prev_layout_check && layout.check().is_err() { panic!("Merged two correct layouts and got an incorrect layout."); } @@ -168,7 +191,7 @@ impl LayoutManager { let mut layout = self.layout.write().unwrap(); if layout.update_trackers != *adv { if layout.update(|l| l.update_trackers.merge(adv)) { - layout.update_trackers_of(self.node_id); + layout.update_trackers(self.node_id); return Some(layout.update_trackers.clone()); } } @@ -317,3 +340,42 @@ impl LayoutManager { Ok(SystemRpc::Ok) } } + +// ---- ack lock ---- + +pub struct WriteLock { + layout_version: u64, + layout_manager: Arc, + value: T, +} + +impl WriteLock { + fn new(version: u64, layout_manager: &Arc, value: T) -> Self { + Self { + layout_version: version, + layout_manager: layout_manager.clone(), + value, + } + } +} + +impl AsRef for WriteLock { + fn as_ref(&self) -> &T { + &self.value + } +} + +impl Drop for WriteLock { + fn drop(&mut self) { + let layout = self.layout_manager.layout(); // acquire read lock + if let Some(counter) = layout.ack_lock.get(&self.layout_version) { + let prev_lock = counter.fetch_sub(1, Ordering::Relaxed); + if prev_lock == 1 && layout.current().version > self.layout_version { + drop(layout); // release read lock, write lock will be acquired + self.layout_manager.ack_new_version(); + } + } else { + error!("Could not find ack lock counter for layout version {}. This probably indicates a bug in Garage.", self.layout_version); + } + } +} diff --git a/src/rpc/layout/mod.rs b/src/rpc/layout/mod.rs index 577b32fb..859287c8 100644 --- a/src/rpc/layout/mod.rs +++ b/src/rpc/layout/mod.rs @@ -11,6 +11,7 @@ pub mod manager; // ---- re-exports ---- pub use history::*; +pub use manager::WriteLock; pub use schema::*; pub use version::*; diff --git a/src/table/replication/fullcopy.rs b/src/table/replication/fullcopy.rs index cb5471af..df930224 100644 --- a/src/table/replication/fullcopy.rs +++ b/src/table/replication/fullcopy.rs @@ -27,6 +27,8 @@ pub struct TableFullReplication { } impl TableReplication for TableFullReplication { + type WriteSets = Vec>; + fn storage_nodes(&self, _hash: &Hash) -> Vec { let layout = self.system.cluster_layout(); layout.current().all_nodes().to_vec() @@ -39,7 +41,7 @@ impl TableReplication for TableFullReplication { 1 } - fn write_sets(&self, hash: &Hash) -> Vec> { + fn write_sets(&self, hash: &Hash) -> Self::WriteSets { vec![self.storage_nodes(hash)] } fn write_quorum(&self) -> usize { diff --git a/src/table/replication/parameters.rs b/src/table/replication/parameters.rs index 2f842409..a4e701bb 100644 --- a/src/table/replication/parameters.rs +++ b/src/table/replication/parameters.rs @@ -3,6 +3,8 @@ use garage_util::data::*; /// Trait to describe how a table shall be replicated pub trait TableReplication: Send + Sync + 'static { + type WriteSets: AsRef>> + Send + Sync + 'static; + // See examples in table_sharded.rs and table_fullcopy.rs // To understand various replication methods @@ -15,7 +17,7 @@ pub trait TableReplication: Send + Sync + 'static { fn read_quorum(&self) -> usize; /// Which nodes to send writes to - fn write_sets(&self, hash: &Hash) -> Vec>; + fn write_sets(&self, hash: &Hash) -> Self::WriteSets; /// Responses needed to consider a write succesfull in each set fn write_quorum(&self) -> usize; fn max_write_errors(&self) -> usize; diff --git a/src/table/replication/sharded.rs b/src/table/replication/sharded.rs index 1320a189..2a16bc0c 100644 --- a/src/table/replication/sharded.rs +++ b/src/table/replication/sharded.rs @@ -25,6 +25,8 @@ pub struct TableShardedReplication { } impl TableReplication for TableShardedReplication { + type WriteSets = WriteLock>>; + fn storage_nodes(&self, hash: &Hash) -> Vec { self.system.cluster_layout().storage_nodes_of(hash) } @@ -36,8 +38,8 @@ impl TableReplication for TableShardedReplication { self.read_quorum } - fn write_sets(&self, hash: &Hash) -> Vec> { - self.system.cluster_layout().write_sets_of(hash) + fn write_sets(&self, hash: &Hash) -> Self::WriteSets { + self.system.layout_manager.write_sets_of(hash) } fn write_quorum(&self) -> usize { self.write_quorum diff --git a/src/table/sync.rs b/src/table/sync.rs index b67cdd79..efeac402 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -173,12 +173,7 @@ impl TableSyncer { } if !items.is_empty() { - let nodes = self - .data - .replication - .storage_nodes(begin) - .into_iter() - .collect::>(); + let nodes = self.data.replication.storage_nodes(begin); if nodes.contains(&self.system.id) { warn!( "({}) Interrupting offload as partitions seem to have changed", @@ -202,7 +197,7 @@ impl TableSyncer { end, counter ); - self.offload_items(&items, &nodes[..]).await?; + self.offload_items(&items, &nodes).await?; } else { break; } diff --git a/src/table/table.rs b/src/table/table.rs index c2efaeaf..5ec9eb0a 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -128,7 +128,7 @@ impl Table { .rpc_helper() .try_write_many_sets( &self.endpoint, - &who, + who.as_ref(), rpc, RequestStrategy::with_priority(PRIO_NORMAL) .with_quorum(self.data.replication.write_quorum()), -- cgit v1.2.3 From d4df03424f1c7f3cc1eaba9e16d2e1d049131b97 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 15 Nov 2023 15:56:57 +0100 Subject: layout: fix test --- src/rpc/layout/test.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/rpc/layout/test.rs b/src/rpc/layout/test.rs index e9639073..bb072c97 100644 --- a/src/rpc/layout/test.rs +++ b/src/rpc/layout/test.rs @@ -113,8 +113,6 @@ fn update_layout( staging.parameters.update(LayoutParameters { zone_redundancy: ZoneRedundancy::AtLeast(zone_redundancy), }); - - cl.update_hashes(); } #[test] -- cgit v1.2.3 From ad5c6f779f7fdfdc0569920c830c59197023515a Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 16 Nov 2023 13:26:43 +0100 Subject: layout: split helper in separate file; more precise difference tracking --- src/rpc/layout/helper.rs | 224 +++++++++++++++++++++++++++++++++++++ src/rpc/layout/history.rs | 278 +++++----------------------------------------- src/rpc/layout/manager.rs | 5 +- src/rpc/layout/mod.rs | 3 +- 4 files changed, 256 insertions(+), 254 deletions(-) create mode 100644 src/rpc/layout/helper.rs diff --git a/src/rpc/layout/helper.rs b/src/rpc/layout/helper.rs new file mode 100644 index 00000000..ed3da498 --- /dev/null +++ b/src/rpc/layout/helper.rs @@ -0,0 +1,224 @@ +use std::collections::HashMap; +use std::ops::Deref; +use std::sync::atomic::{AtomicUsize, Ordering}; + +use garage_util::data::*; + +use super::schema::*; + +pub struct LayoutHelper { + layout: Option, + + // cached values + ack_map_min: u64, + sync_map_min: u64, + + all_nodes: Vec, + all_nongateway_nodes: Vec, + + pub(crate) trackers_hash: Hash, + pub(crate) staging_hash: Hash, + + // ack lock: counts in-progress write operations for each + // layout version ; we don't increase the ack update tracker + // while this lock is nonzero + pub(crate) ack_lock: HashMap, +} + +impl Deref for LayoutHelper { + type Target = LayoutHistory; + fn deref(&self) -> &LayoutHistory { + self.layout() + } +} + +impl LayoutHelper { + pub fn new(mut layout: LayoutHistory, mut ack_lock: HashMap) -> Self { + layout.cleanup_old_versions(); + + let all_nongateway_nodes = layout.get_all_nongateway_nodes(); + layout.clamp_update_trackers(&all_nongateway_nodes); + + let min_version = layout.min_stored(); + let ack_map_min = layout + .update_trackers + .ack_map + .min(&all_nongateway_nodes, min_version); + let sync_map_min = layout + .update_trackers + .sync_map + .min(&all_nongateway_nodes, min_version); + + let all_nodes = layout.get_all_nodes(); + let trackers_hash = layout.calculate_trackers_hash(); + let staging_hash = layout.calculate_staging_hash(); + + ack_lock.retain(|_, cnt| *cnt.get_mut() > 0); + ack_lock + .entry(layout.current().version) + .or_insert(AtomicUsize::new(0)); + + LayoutHelper { + layout: Some(layout), + ack_map_min, + sync_map_min, + all_nodes, + all_nongateway_nodes, + trackers_hash, + staging_hash, + ack_lock, + } + } + + // ------------------ single updating function -------------- + + fn layout(&self) -> &LayoutHistory { + self.layout.as_ref().unwrap() + } + + pub(crate) fn update(&mut self, f: F) -> bool + where + F: FnOnce(&mut LayoutHistory) -> bool, + { + let changed = f(&mut self.layout.as_mut().unwrap()); + if changed { + *self = Self::new( + self.layout.take().unwrap(), + std::mem::take(&mut self.ack_lock), + ); + } + changed + } + + // ------------------ read helpers --------------- + + pub fn all_nodes(&self) -> &[Uuid] { + &self.all_nodes + } + + pub fn all_nongateway_nodes(&self) -> &[Uuid] { + &self.all_nongateway_nodes + } + + pub fn all_ack(&self) -> u64 { + self.ack_map_min + } + + pub fn sync_versions(&self) -> (u64, u64, u64) { + ( + self.layout().current().version, + self.all_ack(), + self.layout().min_stored(), + ) + } + + pub fn read_nodes_of(&self, position: &Hash) -> Vec { + let sync_min = self.sync_map_min; + let version = self + .layout() + .versions + .iter() + .find(|x| x.version == sync_min) + .or(self.layout().versions.last()) + .unwrap(); + version + .nodes_of(position, version.replication_factor) + .collect() + } + + pub(crate) fn write_sets_of(&self, position: &Hash) -> Vec> { + self.layout() + .versions + .iter() + .map(|x| x.nodes_of(position, x.replication_factor).collect()) + .collect() + } + + pub fn storage_nodes_of(&self, position: &Hash) -> Vec { + let mut ret = vec![]; + for version in self.layout().versions.iter() { + ret.extend(version.nodes_of(position, version.replication_factor)); + } + ret.sort(); + ret.dedup(); + ret + } + + pub fn trackers_hash(&self) -> Hash { + self.trackers_hash + } + + pub fn staging_hash(&self) -> Hash { + self.staging_hash + } + + // ------------------ helpers for update tracking --------------- + + pub(crate) fn update_trackers(&mut self, local_node_id: Uuid) { + // Ensure trackers for this node's values are up-to-date + + // 1. Acknowledge the last layout version which is not currently + // locked by an in-progress write operation + self.ack_max_free(local_node_id); + + // 2. Assume the data on this node is sync'ed up at least to + // the first layout version in the history + self.sync_first(local_node_id); + + // 3. Acknowledge everyone has synced up to min(self.sync_map) + self.sync_ack(local_node_id); + + info!("ack_map: {:?}", self.update_trackers.ack_map); + info!("sync_map: {:?}", self.update_trackers.sync_map); + info!("sync_ack_map: {:?}", self.update_trackers.sync_ack_map); + } + + fn sync_first(&mut self, local_node_id: Uuid) { + let first_version = self.versions.first().as_ref().unwrap().version; + self.update(|layout| { + layout + .update_trackers + .sync_map + .set_max(local_node_id, first_version) + }); + } + + fn sync_ack(&mut self, local_node_id: Uuid) { + let sync_map_min = self.sync_map_min; + self.update(|layout| { + layout + .update_trackers + .sync_ack_map + .set_max(local_node_id, sync_map_min) + }); + } + + pub(crate) fn ack_max_free(&mut self, local_node_id: Uuid) -> bool { + let max_ack = self.max_free_ack(); + let changed = self.update(|layout| { + layout + .update_trackers + .ack_map + .set_max(local_node_id, max_ack) + }); + if changed { + info!("ack_until updated to {}", max_ack); + } + changed + } + + pub(crate) fn max_free_ack(&self) -> u64 { + self.layout() + .versions + .iter() + .map(|x| x.version) + .take_while(|v| { + self.ack_lock + .get(v) + .map(|x| x.load(Ordering::Relaxed) == 0) + .unwrap_or(true) + }) + .max() + .unwrap_or(self.min_stored()) + } +} diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index dd38efa7..0a139549 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -1,7 +1,4 @@ -use std::collections::HashMap; use std::collections::HashSet; -use std::ops::Deref; -use std::sync::atomic::{AtomicUsize, Ordering}; use garage_util::crdt::{Crdt, Lww, LwwMap}; use garage_util::data::*; @@ -11,225 +8,6 @@ use garage_util::error::*; use super::schema::*; use super::*; -pub struct LayoutHelper { - layout: Option, - - // cached values - ack_map_min: u64, - sync_map_min: u64, - - all_nodes: Vec, - all_nongateway_nodes: Vec, - - trackers_hash: Hash, - staging_hash: Hash, - - // ack lock: counts in-progress write operations for each - // layout version ; we don't increase the ack update tracker - // while this lock is nonzero - pub(crate) ack_lock: HashMap, -} - -impl Deref for LayoutHelper { - type Target = LayoutHistory; - fn deref(&self) -> &LayoutHistory { - self.layout() - } -} - -impl LayoutHelper { - pub fn new(mut layout: LayoutHistory, mut ack_lock: HashMap) -> Self { - layout.cleanup_old_versions(); - - let all_nongateway_nodes = layout.get_all_nongateway_nodes(); - layout.clamp_update_trackers(&all_nongateway_nodes); - - let min_version = layout.min_stored(); - let ack_map_min = layout - .update_trackers - .ack_map - .min(&all_nongateway_nodes, min_version); - let sync_map_min = layout - .update_trackers - .sync_map - .min(&all_nongateway_nodes, min_version); - - let all_nodes = layout.get_all_nodes(); - let trackers_hash = layout.calculate_trackers_hash(); - let staging_hash = layout.calculate_staging_hash(); - - ack_lock.retain(|_, cnt| *cnt.get_mut() > 0); - ack_lock - .entry(layout.current().version) - .or_insert(AtomicUsize::new(0)); - - LayoutHelper { - layout: Some(layout), - ack_map_min, - sync_map_min, - all_nodes, - all_nongateway_nodes, - trackers_hash, - staging_hash, - ack_lock, - } - } - - // ------------------ single updating function -------------- - - fn layout(&self) -> &LayoutHistory { - self.layout.as_ref().unwrap() - } - - pub(crate) fn update(&mut self, f: F) -> bool - where - F: FnOnce(&mut LayoutHistory) -> bool, - { - let changed = f(&mut self.layout.as_mut().unwrap()); - if changed { - *self = Self::new( - self.layout.take().unwrap(), - std::mem::take(&mut self.ack_lock), - ); - } - changed - } - - // ------------------ read helpers --------------- - - pub fn all_nodes(&self) -> &[Uuid] { - &self.all_nodes - } - - pub fn all_nongateway_nodes(&self) -> &[Uuid] { - &self.all_nongateway_nodes - } - - pub fn all_ack(&self) -> u64 { - self.ack_map_min - } - - pub fn sync_versions(&self) -> (u64, u64, u64) { - ( - self.layout().current().version, - self.all_ack(), - self.layout().min_stored(), - ) - } - - pub fn read_nodes_of(&self, position: &Hash) -> Vec { - let sync_min = self.sync_map_min; - let version = self - .layout() - .versions - .iter() - .find(|x| x.version == sync_min) - .or(self.layout().versions.last()) - .unwrap(); - version - .nodes_of(position, version.replication_factor) - .collect() - } - - pub(crate) fn write_sets_of(&self, position: &Hash) -> Vec> { - self.layout() - .versions - .iter() - .map(|x| x.nodes_of(position, x.replication_factor).collect()) - .collect() - } - - pub fn storage_nodes_of(&self, position: &Hash) -> Vec { - let mut ret = vec![]; - for version in self.layout().versions.iter() { - ret.extend(version.nodes_of(position, version.replication_factor)); - } - ret.sort(); - ret.dedup(); - ret - } - - pub fn trackers_hash(&self) -> Hash { - self.trackers_hash - } - - pub fn staging_hash(&self) -> Hash { - self.staging_hash - } - - // ------------------ helpers for update tracking --------------- - - pub(crate) fn update_trackers(&mut self, local_node_id: Uuid) { - // Ensure trackers for this node's values are up-to-date - - // 1. Acknowledge the last layout version which is not currently - // locked by an in-progress write operation - self.ack_max_free(local_node_id); - - // 2. Assume the data on this node is sync'ed up at least to - // the first layout version in the history - self.sync_first(local_node_id); - - // 3. Acknowledge everyone has synced up to min(self.sync_map) - self.sync_ack(local_node_id); - - info!("ack_map: {:?}", self.update_trackers.ack_map); - info!("sync_map: {:?}", self.update_trackers.sync_map); - info!("sync_ack_map: {:?}", self.update_trackers.sync_ack_map); - } - - fn sync_first(&mut self, local_node_id: Uuid) { - let first_version = self.versions.first().as_ref().unwrap().version; - self.update(|layout| { - layout - .update_trackers - .sync_map - .set_max(local_node_id, first_version) - }); - } - - fn sync_ack(&mut self, local_node_id: Uuid) { - let sync_map_min = self.sync_map_min; - self.update(|layout| { - layout - .update_trackers - .sync_ack_map - .set_max(local_node_id, sync_map_min) - }); - } - - pub(crate) fn ack_max_free(&mut self, local_node_id: Uuid) -> bool { - let max_ack = self.max_free_ack(); - let changed = self.update(|layout| { - layout - .update_trackers - .ack_map - .set_max(local_node_id, max_ack) - }); - if changed { - info!("ack_until updated to {}", max_ack); - } - changed - } - - pub(crate) fn max_free_ack(&self) -> u64 { - self.layout() - .versions - .iter() - .map(|x| x.version) - .take_while(|v| { - self.ack_lock - .get(v) - .map(|x| x.load(Ordering::Relaxed) == 0) - .unwrap_or(true) - }) - .max() - .unwrap_or(self.min_stored()) - } -} - -// ---- - impl LayoutHistory { pub fn new(replication_factor: usize) -> Self { let version = LayoutVersion::new(replication_factor); @@ -270,7 +48,7 @@ impl LayoutHistory { } } - fn get_all_nongateway_nodes(&self) -> Vec { + pub(crate) fn get_all_nongateway_nodes(&self) -> Vec { if self.versions.len() == 1 { self.versions[0].nongateway_nodes().to_vec() } else { @@ -286,8 +64,21 @@ impl LayoutHistory { // ---- housekeeping (all invoked by LayoutHelper) ---- - fn cleanup_old_versions(&mut self) { - loop { + pub(crate) fn cleanup_old_versions(&mut self) { + // If there are invalid versions before valid versions, remove them + if self.versions.len() > 1 && self.current().check().is_ok() { + while self.versions.len() > 1 && self.versions.first().unwrap().check().is_err() { + let removed = self.versions.remove(0); + info!( + "Layout history: pruning old invalid version {}", + removed.version + ); + } + } + + // If there are old versions that no one is reading from anymore, + // remove them + while self.versions.len() > 1 { let all_nongateway_nodes = self.get_all_nongateway_nodes(); let min_version = self.min_stored(); let sync_ack_map_min = self @@ -303,7 +94,7 @@ impl LayoutHistory { } } - fn clamp_update_trackers(&mut self, nodes: &[Uuid]) { + pub(crate) fn clamp_update_trackers(&mut self, nodes: &[Uuid]) { let min_v = self.min_stored(); for node in nodes { self.update_trackers.ack_map.set_max(*node, min_v); @@ -312,11 +103,11 @@ impl LayoutHistory { } } - fn calculate_trackers_hash(&self) -> Hash { + pub(crate) fn calculate_trackers_hash(&self) -> Hash { blake2sum(&nonversioned_encode(&self.update_trackers).unwrap()[..]) } - fn calculate_staging_hash(&self) -> Hash { + pub(crate) fn calculate_staging_hash(&self) -> Hash { blake2sum(&nonversioned_encode(&self.staging).unwrap()[..]) } @@ -328,6 +119,7 @@ impl LayoutHistory { // Add any new versions to history for v2 in other.versions.iter() { if let Some(v1) = self.versions.iter().find(|v| v.version == v2.version) { + // Version is already present, check consistency if v1 != v2 { error!("Inconsistent layout histories: different layout compositions for version {}. Your cluster will be broken as long as this layout version is not replaced.", v2.version); } @@ -344,24 +136,14 @@ impl LayoutHistory { } // Merge trackers - if self.update_trackers != other.update_trackers { - let c = self.update_trackers.merge(&other.update_trackers); - changed = changed || c; - } - - // If there are invalid versions before valid versions, remove them, - // and increment update trackers - if self.versions.len() > 1 && self.current().check().is_ok() { - while self.versions.first().unwrap().check().is_err() { - self.versions.remove(0); - changed = true; - } - } + let c = self.update_trackers.merge(&other.update_trackers); + changed = changed || c; // Merge staged layout changes if self.staging != other.staging { + let prev_staging = self.staging.clone(); self.staging.merge(&other.staging); - changed = true; + changed = changed || self.staging != prev_staging; } changed @@ -390,11 +172,7 @@ To know the correct value of the new layout version, invoke `garage layout show` .calculate_next_version(&self.staging.get())?; self.versions.push(new_version); - if self.current().check().is_ok() { - while self.versions.first().unwrap().check().is_err() { - self.versions.remove(0); - } - } + self.cleanup_old_versions(); // Reset the staged layout changes self.staging.update(LayoutStaging { @@ -415,11 +193,7 @@ To know the correct value of the new layout version, invoke `garage layout show` } pub fn check(&self) -> Result<(), String> { - for version in self.versions.iter() { - version.check()?; - } - // TODO: anything more ? - Ok(()) + self.current().check() } } diff --git a/src/rpc/layout/manager.rs b/src/rpc/layout/manager.rs index 4e073d1f..85d94ffa 100644 --- a/src/rpc/layout/manager.rs +++ b/src/rpc/layout/manager.rs @@ -184,17 +184,20 @@ impl LayoutManager { return Some(layout.clone()); } } + None } fn merge_layout_trackers(&self, adv: &UpdateTrackers) -> Option { let mut layout = self.layout.write().unwrap(); + if layout.update_trackers != *adv { if layout.update(|l| l.update_trackers.merge(adv)) { layout.update_trackers(self.node_id); return Some(layout.update_trackers.clone()); } } + None } @@ -284,7 +287,7 @@ impl LayoutManager { } pub(crate) fn handle_pull_cluster_layout(&self) -> SystemRpc { - let layout = self.layout.read().unwrap().clone(); // TODO: avoid cloning + let layout = self.layout.read().unwrap().clone(); SystemRpc::AdvertiseClusterLayout(layout) } diff --git a/src/rpc/layout/mod.rs b/src/rpc/layout/mod.rs index 859287c8..91151ab4 100644 --- a/src/rpc/layout/mod.rs +++ b/src/rpc/layout/mod.rs @@ -1,4 +1,5 @@ mod graph_algo; +mod helper; mod history; mod schema; mod version; @@ -10,7 +11,7 @@ pub mod manager; // ---- re-exports ---- -pub use history::*; +pub use helper::LayoutHelper; pub use manager::WriteLock; pub use schema::*; pub use version::*; -- cgit v1.2.3 From 707442f5de416fdbed4681a33b739f0a787b7834 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 16 Nov 2023 13:51:40 +0100 Subject: layout: refactor digests and add "!=" assertions before epidemic bcast --- src/rpc/layout/helper.rs | 27 +++++++++++++++++++++++++-- src/rpc/layout/history.rs | 1 - src/rpc/layout/manager.rs | 36 ++++++++++-------------------------- src/rpc/layout/mod.rs | 2 +- src/rpc/system.rs | 17 +++++++++-------- 5 files changed, 45 insertions(+), 38 deletions(-) diff --git a/src/rpc/layout/helper.rs b/src/rpc/layout/helper.rs index ed3da498..0d746ea3 100644 --- a/src/rpc/layout/helper.rs +++ b/src/rpc/layout/helper.rs @@ -2,10 +2,24 @@ use std::collections::HashMap; use std::ops::Deref; use std::sync::atomic::{AtomicUsize, Ordering}; +use serde::{Deserialize, Serialize}; + use garage_util::data::*; use super::schema::*; +#[derive(Debug, Clone, Serialize, Deserialize, Default, PartialEq, Eq)] +pub struct LayoutDigest { + /// Cluster layout version + pub current_version: u64, + /// Number of active layout versions + pub active_versions: usize, + /// Hash of cluster layout update trackers + pub trackers_hash: Hash, + /// Hash of cluster layout staging data + pub staging_hash: Hash, +} + pub struct LayoutHelper { layout: Option, @@ -16,8 +30,8 @@ pub struct LayoutHelper { all_nodes: Vec, all_nongateway_nodes: Vec, - pub(crate) trackers_hash: Hash, - pub(crate) staging_hash: Hash, + trackers_hash: Hash, + staging_hash: Hash, // ack lock: counts in-progress write operations for each // layout version ; we don't increase the ack update tracker @@ -152,6 +166,15 @@ impl LayoutHelper { self.staging_hash } + pub fn digest(&self) -> LayoutDigest { + LayoutDigest { + current_version: self.current().version, + active_versions: self.versions.len(), + trackers_hash: self.trackers_hash, + staging_hash: self.staging_hash, + } + } + // ------------------ helpers for update tracking --------------- pub(crate) fn update_trackers(&mut self, local_node_id: Uuid) { diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index 0a139549..653d2a48 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -5,7 +5,6 @@ use garage_util::data::*; use garage_util::encode::nonversioned_encode; use garage_util::error::*; -use super::schema::*; use super::*; impl LayoutHistory { diff --git a/src/rpc/layout/manager.rs b/src/rpc/layout/manager.rs index 85d94ffa..c65831a2 100644 --- a/src/rpc/layout/manager.rs +++ b/src/rpc/layout/manager.rs @@ -2,8 +2,6 @@ use std::collections::HashMap; use std::sync::{atomic::Ordering, Arc, Mutex, RwLock, RwLockReadGuard}; use std::time::Duration; -use serde::{Deserialize, Serialize}; - use tokio::sync::Notify; use netapp::endpoint::Endpoint; @@ -33,16 +31,6 @@ pub struct LayoutManager { system_endpoint: Arc>, } -#[derive(Debug, Clone, Serialize, Deserialize, Default)] -pub struct LayoutStatus { - /// Cluster layout version - pub cluster_layout_version: u64, - /// Hash of cluster layout update trackers - pub cluster_layout_trackers_hash: Hash, - /// Hash of cluster layout staging data - pub cluster_layout_staging_hash: Hash, -} - impl LayoutManager { pub fn new( config: &Config, @@ -105,15 +93,6 @@ impl LayoutManager { self.layout.read().unwrap() } - pub fn status(&self) -> LayoutStatus { - let layout = self.layout(); - LayoutStatus { - cluster_layout_version: layout.current().version, - cluster_layout_trackers_hash: layout.trackers_hash(), - cluster_layout_staging_hash: layout.staging_hash(), - } - } - pub async fn update_cluster_layout( self: &Arc, layout: &LayoutHistory, @@ -173,6 +152,7 @@ impl LayoutManager { fn merge_layout(&self, adv: &LayoutHistory) -> Option { let mut layout = self.layout.write().unwrap(); + let prev_digest = layout.digest(); let prev_layout_check = layout.check().is_ok(); if !prev_layout_check || adv.check().is_ok() { @@ -181,6 +161,7 @@ impl LayoutManager { if prev_layout_check && layout.check().is_err() { panic!("Merged two correct layouts and got an incorrect layout."); } + assert!(layout.digest() != prev_digest); return Some(layout.clone()); } } @@ -190,10 +171,12 @@ impl LayoutManager { fn merge_layout_trackers(&self, adv: &UpdateTrackers) -> Option { let mut layout = self.layout.write().unwrap(); + let prev_digest = layout.digest(); if layout.update_trackers != *adv { if layout.update(|l| l.update_trackers.merge(adv)) { layout.update_trackers(self.node_id); + assert!(layout.digest() != prev_digest); return Some(layout.update_trackers.clone()); } } @@ -269,16 +252,17 @@ impl LayoutManager { // ---- RPC HANDLERS ---- - pub(crate) fn handle_advertise_status(self: &Arc, from: Uuid, remote: &LayoutStatus) { - let local = self.status(); - if remote.cluster_layout_version > local.cluster_layout_version - || remote.cluster_layout_staging_hash != local.cluster_layout_staging_hash + pub(crate) fn handle_advertise_status(self: &Arc, from: Uuid, remote: &LayoutDigest) { + let local = self.layout().digest(); + if remote.current_version > local.current_version + || remote.active_versions != local.active_versions + || remote.staging_hash != local.staging_hash { tokio::spawn({ let this = self.clone(); async move { this.pull_cluster_layout(from).await } }); - } else if remote.cluster_layout_trackers_hash != local.cluster_layout_trackers_hash { + } else if remote.trackers_hash != local.trackers_hash { tokio::spawn({ let this = self.clone(); async move { this.pull_cluster_layout_trackers(from).await } diff --git a/src/rpc/layout/mod.rs b/src/rpc/layout/mod.rs index 91151ab4..eb127fda 100644 --- a/src/rpc/layout/mod.rs +++ b/src/rpc/layout/mod.rs @@ -11,7 +11,7 @@ pub mod manager; // ---- re-exports ---- -pub use helper::LayoutHelper; +pub use helper::{LayoutDigest, LayoutHelper}; pub use manager::WriteLock; pub use schema::*; pub use version::*; diff --git a/src/rpc/system.rs b/src/rpc/system.rs index d74dc2a1..dc127afb 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -33,8 +33,9 @@ use garage_util::time::*; use crate::consul::ConsulDiscovery; #[cfg(feature = "kubernetes-discovery")] use crate::kubernetes::*; -use crate::layout::manager::{LayoutManager, LayoutStatus}; -use crate::layout::{self, LayoutHelper, LayoutHistory, NodeRoleV}; +use crate::layout::{ + self, manager::LayoutManager, LayoutDigest, LayoutHelper, LayoutHistory, NodeRoleV, +}; use crate::replication_mode::*; use crate::rpc_helper::*; @@ -130,8 +131,8 @@ pub struct NodeStatus { /// Replication factor configured on the node pub replication_factor: usize, - /// Layout status - pub layout_status: LayoutStatus, + /// Cluster layout digest + pub layout_digest: LayoutDigest, /// Disk usage on partition containing metadata directory (tuple: `(avail, total)`) #[serde(default)] @@ -539,7 +540,7 @@ impl System { fn update_local_status(&self) { let mut new_si: NodeStatus = self.local_status.load().as_ref().clone(); - new_si.layout_status = self.layout_manager.status(); + new_si.layout_digest = self.layout_manager.layout().digest(); new_si.update_disk_usage(&self.metadata_dir, &self.data_dir, &self.metrics); @@ -573,7 +574,7 @@ impl System { } self.layout_manager - .handle_advertise_status(from, &info.layout_status); + .handle_advertise_status(from, &info.layout_digest); self.node_status .write() @@ -755,7 +756,7 @@ impl NodeStatus { .into_string() .unwrap_or_else(|_| "".to_string()), replication_factor, - layout_status: layout_manager.status(), + layout_digest: layout_manager.layout().digest(), meta_disk_avail: None, data_disk_avail: None, } @@ -765,7 +766,7 @@ impl NodeStatus { NodeStatus { hostname: "?".to_string(), replication_factor: 0, - layout_status: Default::default(), + layout_digest: Default::default(), meta_disk_avail: None, data_disk_avail: None, } -- cgit v1.2.3 From 22f38808e744ea5b30ad771fcb344a29579b56d4 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 16 Nov 2023 16:34:01 +0100 Subject: rpc_helper: don't use tokio::spawn for individual requests --- src/rpc/rpc_helper.rs | 18 +++++++----------- 1 file changed, 7 insertions(+), 11 deletions(-) diff --git a/src/rpc/rpc_helper.rs b/src/rpc/rpc_helper.rs index e269ddaa..7e9fabd7 100644 --- a/src/rpc/rpc_helper.rs +++ b/src/rpc/rpc_helper.rs @@ -299,9 +299,7 @@ impl RpcHelper { if let Some((req_to, fut)) = requests.next() { let tracer = opentelemetry::global::tracer("garage"); let span = tracer.start(format!("RPC to {:?}", req_to)); - resp_stream.push(tokio::spawn( - fut.with_context(Context::current_with_span(span)), - )); + resp_stream.push(fut.with_context(Context::current_with_span(span))); } else { break; } @@ -313,7 +311,7 @@ impl RpcHelper { } // Wait for one request to terminate - match resp_stream.next().await.unwrap().unwrap() { + match resp_stream.next().await.unwrap() { Ok(msg) => { successes.push(msg); } @@ -448,7 +446,7 @@ impl RpcHelper { let tracer = opentelemetry::global::tracer("garage"); let span = tracer.start(format!("RPC to {:?}", to)); let fut = async move { (to, self2.call(&endpoint2, to, msg, strategy).await) }; - tokio::spawn(fut.with_context(Context::current_with_span(span))) + fut.with_context(Context::current_with_span(span)) }); let mut resp_stream = requests.collect::>(); @@ -457,9 +455,7 @@ impl RpcHelper { let mut set_counters = vec![(0, 0); to_sets.len()]; - while !resp_stream.is_empty() { - let (node, resp) = resp_stream.next().await.unwrap().unwrap(); - + while let Some((node, resp)) = resp_stream.next().await { match resp { Ok(msg) => { for set in peers.get(&node).unwrap().iter() { @@ -475,12 +471,12 @@ impl RpcHelper { } } - if set_counters.iter().all(|x| x.0 >= quorum) { + if set_counters.iter().all(|(ok_cnt, _)| *ok_cnt >= quorum) { // Success // Continue all other requets in background tokio::spawn(async move { - resp_stream.collect::>>().await; + resp_stream.collect::)>>().await; }); return Ok(successes); @@ -489,7 +485,7 @@ impl RpcHelper { if set_counters .iter() .enumerate() - .any(|(i, x)| x.1 + quorum > to_sets[i].len()) + .any(|(i, (_, err_cnt))| err_cnt + quorum > to_sets[i].len()) { // Too many errors in this set, we know we won't get a quorum break; -- cgit v1.2.3 From 3ecd14b9f6202ad3c5513c6ad7422bd408134002 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 16 Nov 2023 16:41:45 +0100 Subject: table: implement write sets for insert_many --- src/table/table.rs | 157 +++++++++++++++++++++++++++++++++++++++++++---------- 1 file changed, 127 insertions(+), 30 deletions(-) diff --git a/src/table/table.rs b/src/table/table.rs index 5ec9eb0a..7d1ff31c 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -143,7 +143,7 @@ impl Table { self.data.queue_insert(tx, e) } - pub async fn insert_many(&self, entries: I) -> Result<(), Error> + pub async fn insert_many(self: &Arc, entries: I) -> Result<(), Error> where I: IntoIterator + Send + Sync, IE: Borrow + Send + Sync, @@ -161,52 +161,149 @@ impl Table { Ok(()) } - async fn insert_many_internal(&self, entries: I) -> Result<(), Error> + async fn insert_many_internal(self: &Arc, entries: I) -> Result<(), Error> where I: IntoIterator + Send + Sync, IE: Borrow + Send + Sync, { - let mut call_list: HashMap<_, Vec<_>> = HashMap::new(); - + // The different items will have to be stored on possibly different nodes. + // We will here batch all items into a single request for each concerned + // node, with all of the entries it must store within that request. + // Each entry has to be saved to a specific list of "write sets", i.e. a set + // of node within wich a quorum must be achieved. In normal operation, there + // is a single write set which corresponds to the quorum in the current + // cluster layout, but when the layout is updated, multiple write sets might + // have to be handled at once. Here, since we are sending many entries, we + // will have to handle many write sets in all cases. The algorihtm is thus + // to send one request to each node with all the items it must save, + // and keep track of the OK responses within each write set: if for all sets + // a quorum of nodes has answered OK, then the insert has succeeded and + // consistency properties (read-after-write) are preserved. + + // Some code here might feel redundant with RpcHelper::try_write_many_sets, + // but I think deduplicating could lead to more spaghetti instead of + // improving the readability, so I'm leaving as is. + + let quorum = self.data.replication.write_quorum(); + + // Serialize all entries and compute the write sets for each of them. + // In the case of sharded table replication, this also takes an "ack lock" + // to the layout manager to avoid ack'ing newer versions which are not + // taken into account by writes in progress (the ack can happen later, once + // all writes that didn't take the new layout into account are finished). + // These locks are released when entries_vec is dropped, i.e. when this + // function returns. + let mut entries_vec = Vec::new(); for entry in entries.into_iter() { let entry = entry.borrow(); let hash = entry.partition_key().hash(); - // TODO: use write sets - let who = self.data.replication.storage_nodes(&hash); + let write_sets = self.data.replication.write_sets(&hash); let e_enc = Arc::new(ByteBuf::from(entry.encode()?)); - for node in who { - call_list.entry(node).or_default().push(e_enc.clone()); + entries_vec.push((write_sets, e_enc)); + } + + // Compute a deduplicated list of all of the write sets, + // and compute an index from each node to the position of the sets in which + // it takes part, to optimize the detection of a quorum. + let mut write_sets = entries_vec + .iter() + .map(|(wss, _)| wss.as_ref().iter().map(|ws| ws.as_slice())) + .flatten() + .collect::>(); + write_sets.sort(); + write_sets.dedup(); + let mut write_set_index = HashMap::<&Uuid, Vec>::new(); + for (i, write_set) in write_sets.iter().enumerate() { + for node in write_set.iter() { + write_set_index.entry(node).or_default().push(i); } } - let call_futures = call_list.drain().map(|(node, entries)| async move { - let rpc = TableRpc::::Update(entries); - - let resp = self - .system - .rpc_helper() - .call( - &self.endpoint, - node, - rpc, - RequestStrategy::with_priority(PRIO_NORMAL), - ) - .await?; - Ok::<_, Error>((node, resp)) + // Build a map of all nodes to the entries that must be sent to that node. + let mut call_list: HashMap> = HashMap::new(); + for (write_sets, entry_enc) in entries_vec.iter() { + for write_set in write_sets.as_ref().iter() { + for node in write_set.iter() { + call_list.entry(*node).or_default().push(entry_enc.clone()) + } + } + } + + // Build futures to actually perform each of the corresponding RPC calls + let call_count = call_list.len(); + let call_futures = call_list.into_iter().map(|(node, entries)| { + let this = self.clone(); + let tracer = opentelemetry::global::tracer("garage"); + let span = tracer.start(format!("RPC to {:?}", node)); + let fut = async move { + let rpc = TableRpc::::Update(entries); + let resp = this + .system + .rpc_helper() + .call( + &this.endpoint, + node, + rpc, + RequestStrategy::with_priority(PRIO_NORMAL).with_quorum(quorum), + ) + .await; + (node, resp) + }; + fut.with_context(Context::current_with_span(span)) }); + + // Run all requests in parallel thanks to FuturesUnordered, and collect results. let mut resps = call_futures.collect::>(); + let mut set_counters = vec![(0, 0); write_sets.len()]; + let mut successes = 0; let mut errors = vec![]; - while let Some(resp) = resps.next().await { - if let Err(e) = resp { - errors.push(e); + while let Some((node, resp)) = resps.next().await { + match resp { + Ok(_) => { + successes += 1; + for set in write_set_index.get(&node).unwrap().iter() { + set_counters[*set].0 += 1; + } + } + Err(e) => { + errors.push(e); + for set in write_set_index.get(&node).unwrap().iter() { + set_counters[*set].1 += 1; + } + } + } + + if set_counters.iter().all(|(ok_cnt, _)| *ok_cnt >= quorum) { + // Success + + // Continue all other requests in background + tokio::spawn(async move { + resps.collect::)>>().await; + }); + + return Ok(()); + } + + if set_counters + .iter() + .enumerate() + .any(|(i, (_, err_cnt))| err_cnt + quorum > write_sets[i].len()) + { + // Too many errors in this set, we know we won't get a quorum + break; } } - if errors.len() > self.data.replication.max_write_errors() { - Err(Error::Message("Too many errors".into())) - } else { - Ok(()) - } + + // Failure, could not get quorum within at least one set + let errors = errors.iter().map(|e| format!("{}", e)).collect::>(); + Err(Error::Quorum( + quorum, + Some(write_sets.len()), + successes, + call_count, + errors, + )) } pub async fn get( -- cgit v1.2.3 From d6d239fc7909cbd017da6ea35cceb3d561a87cca Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 27 Nov 2023 11:52:57 +0100 Subject: block manager: read_block using old layout versions if necessary --- src/block/manager.rs | 6 ++++-- src/rpc/layout/helper.rs | 23 +++++++++++++++++++++++ src/rpc/layout/history.rs | 12 +++++++++++- src/rpc/layout/schema.rs | 7 +++++++ src/rpc/rpc_helper.rs | 11 +++++------ 5 files changed, 50 insertions(+), 9 deletions(-) diff --git a/src/block/manager.rs b/src/block/manager.rs index be2e4951..47111160 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -264,8 +264,10 @@ impl BlockManager { F: Fn(DataBlockHeader, ByteStream) -> Fut, Fut: futures::Future>, { - let who = self.replication.read_nodes(hash); - let who = self.system.rpc_helper().request_order(&who); + let who = self + .system + .cluster_layout() + .block_read_nodes_of(hash, self.system.rpc_helper()); for node in who.iter() { let node_id = NodeID::from(*node); diff --git a/src/rpc/layout/helper.rs b/src/rpc/layout/helper.rs index 0d746ea3..5d159f3e 100644 --- a/src/rpc/layout/helper.rs +++ b/src/rpc/layout/helper.rs @@ -7,6 +7,7 @@ use serde::{Deserialize, Serialize}; use garage_util::data::*; use super::schema::*; +use crate::rpc_helper::RpcHelper; #[derive(Debug, Clone, Serialize, Deserialize, Default, PartialEq, Eq)] pub struct LayoutDigest { @@ -140,6 +141,28 @@ impl LayoutHelper { .collect() } + pub fn block_read_nodes_of(&self, position: &Hash, rpc_helper: &RpcHelper) -> Vec { + let mut ret = Vec::with_capacity(12); + let ver_iter = self + .layout() + .versions + .iter() + .rev() + .chain(self.layout().old_versions.iter().rev()); + for ver in ver_iter { + if ver.version > self.sync_map_min { + continue; + } + let nodes = ver.nodes_of(position, ver.replication_factor); + for node in rpc_helper.request_order(nodes) { + if !ret.contains(&node) { + ret.push(node); + } + } + } + ret + } + pub(crate) fn write_sets_of(&self, position: &Hash) -> Vec> { self.layout() .versions diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index 653d2a48..7d4a1b48 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -18,6 +18,7 @@ impl LayoutHistory { LayoutHistory { versions: vec![version], + old_versions: vec![], update_trackers: Default::default(), staging: Lww::raw(0, staging), } @@ -86,11 +87,20 @@ impl LayoutHistory { .min(&all_nongateway_nodes, min_version); if self.min_stored() < sync_ack_map_min { let removed = self.versions.remove(0); - info!("Layout history: pruning old version {}", removed.version); + info!( + "Layout history: moving version {} to old_versions", + removed.version + ); + self.old_versions.push(removed); } else { break; } } + + while self.old_versions.len() > OLD_VERSION_COUNT { + let removed = self.old_versions.remove(0); + info!("Layout history: removing old_version {}", removed.version); + } } pub(crate) fn clamp_update_trackers(&mut self, nodes: &[Uuid]) { diff --git a/src/rpc/layout/schema.rs b/src/rpc/layout/schema.rs index 00a2c017..08db44ca 100644 --- a/src/rpc/layout/schema.rs +++ b/src/rpc/layout/schema.rs @@ -193,12 +193,18 @@ mod v010 { use std::collections::BTreeMap; pub use v09::{LayoutParameters, NodeRole, NodeRoleV, ZoneRedundancy}; + pub const OLD_VERSION_COUNT: usize = 5; + /// The history of cluster layouts, with trackers to keep a record /// of which nodes are up-to-date to current cluster data #[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] pub struct LayoutHistory { /// The versions currently in use in the cluster pub versions: Vec, + /// At most 5 of the previous versions, not used by the garage_table + /// module, but usefull for the garage_block module to find data blocks + /// that have not yet been moved + pub old_versions: Vec, /// Update trackers pub update_trackers: UpdateTrackers, @@ -300,6 +306,7 @@ mod v010 { }; Self { versions: vec![version], + old_versions: vec![], update_trackers: UpdateTrackers { ack_map: update_tracker.clone(), sync_map: update_tracker.clone(), diff --git a/src/rpc/rpc_helper.rs b/src/rpc/rpc_helper.rs index 7e9fabd7..e9a9143f 100644 --- a/src/rpc/rpc_helper.rs +++ b/src/rpc/rpc_helper.rs @@ -267,7 +267,7 @@ impl RpcHelper { // When there are errors, we start new requests to compensate. // Reorder requests to priorize closeness / low latency - let request_order = self.request_order(to); + let request_order = self.request_order(to.iter().copied()); let send_all_at_once = strategy.rs_send_all_at_once.unwrap_or(false); // Build future for each request @@ -335,7 +335,7 @@ impl RpcHelper { } } - pub fn request_order(&self, nodes: &[Uuid]) -> Vec { + pub fn request_order(&self, nodes: impl Iterator) -> Vec { // Retrieve some status variables that we will use to sort requests let peer_list = self.0.fullmesh.get_peer_list(); let layout = self.0.layout.read().unwrap(); @@ -351,9 +351,8 @@ impl RpcHelper { // By sorting this vec, we priorize ourself, then nodes in the same zone, // and within a same zone we priorize nodes with the lowest latency. let mut nodes = nodes - .iter() .map(|to| { - let peer_zone = match layout.current().node_role(to) { + let peer_zone = match layout.current().node_role(&to) { Some(pc) => &pc.zone, None => "", }; @@ -363,10 +362,10 @@ impl RpcHelper { .and_then(|pi| pi.avg_ping) .unwrap_or_else(|| Duration::from_secs(10)); ( - *to != self.0.our_node_id, + to != self.0.our_node_id, peer_zone != our_zone, peer_avg_ping, - *to, + to, ) }) .collect::>(); -- cgit v1.2.3 From 78362140f5a177340a06690d9c9ea98bd831e7a4 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 27 Nov 2023 12:10:21 +0100 Subject: rpc: update system::health to take into account write sets for all partitions --- src/rpc/system.rs | 77 ++++++++++++++++++++++++++++++++----------------------- 1 file changed, 45 insertions(+), 32 deletions(-) diff --git a/src/rpc/system.rs b/src/rpc/system.rs index dc127afb..c7d41ee4 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -1,5 +1,5 @@ //! Module containing structs related to membership management -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::io::{Read, Write}; use std::net::{IpAddr, SocketAddr}; use std::path::{Path, PathBuf}; @@ -418,48 +418,61 @@ impl System { } pub fn health(&self) -> ClusterHealth { - // TODO: adapt this function to take into account layout history - // when estimating cluster health, and not just use current layout - let quorum = self.replication_mode.write_quorum(); - let replication_factor = self.replication_factor; + // Gather information about running nodes. + // Technically, `nodes` contains currently running nodes, as well + // as nodes that this Garage process has been connected to at least + // once since it started. let nodes = self .get_known_nodes() .into_iter() .map(|n| (n.id, n)) .collect::>(); let connected_nodes = nodes.iter().filter(|(_, n)| n.is_up).count(); + let node_up = |x: &Uuid| nodes.get(x).map(|n| n.is_up).unwrap_or(false); + + // Acquire a rwlock read-lock to the current cluster layout + let layout = self.cluster_layout(); + + // Obtain information about nodes that have a role as storage nodes + // in one of the active layout versions + let mut storage_nodes = HashSet::::with_capacity(16); + for ver in layout.versions.iter() { + storage_nodes.extend( + ver.roles + .items() + .iter() + .filter(|(_, _, v)| matches!(v, NodeRoleV(Some(r)) if r.capacity.is_some())) + .map(|(n, _, _)| *n), + ) + } + let storage_nodes_ok = storage_nodes.iter().filter(|x| node_up(x)).count(); - let layout = self.cluster_layout(); // acquires a rwlock - - let storage_nodes = layout - .current() - .roles - .items() - .iter() - .filter(|(_, _, v)| matches!(v, NodeRoleV(Some(r)) if r.capacity.is_some())) - .collect::>(); - let storage_nodes_ok = storage_nodes - .iter() - .filter(|(x, _, _)| nodes.get(x).map(|n| n.is_up).unwrap_or(false)) - .count(); - + // Determine the number of partitions that have: + // - a quorum of up nodes for all write sets (i.e. are available) + // - for which all nodes in all write sets are up (i.e. are fully healthy) let partitions = layout.current().partitions().collect::>(); - let partitions_n_up = partitions - .iter() - .map(|(_, h)| { - let pn = layout.current().nodes_of(h, replication_factor); - pn.filter(|x| nodes.get(x).map(|n| n.is_up).unwrap_or(false)) - .count() - }) - .collect::>(); - let partitions_all_ok = partitions_n_up - .iter() - .filter(|c| **c == replication_factor) - .count(); - let partitions_quorum = partitions_n_up.iter().filter(|c| **c >= quorum).count(); + let mut partitions_quorum = 0; + let mut partitions_all_ok = 0; + for (_, hash) in partitions.iter() { + let write_sets = layout + .versions + .iter() + .map(|x| x.nodes_of(hash, x.replication_factor)); + let has_quorum = write_sets + .clone() + .all(|set| set.filter(|x| node_up(x)).count() >= quorum); + let all_ok = write_sets.clone().all(|mut set| set.all(|x| node_up(&x))); + if has_quorum { + partitions_quorum += 1; + } + if all_ok { + partitions_all_ok += 1; + } + } + // Determine overall cluster status let status = if partitions_quorum == partitions.len() && storage_nodes_ok == storage_nodes.len() { ClusterHealthStatus::Healthy -- cgit v1.2.3 From 539a920313fff010b8a4291aeef58ec9a14ee635 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 27 Nov 2023 13:18:59 +0100 Subject: cli: show when nodes are draining metadata --- src/garage/cli/cmd.rs | 172 +++++++++++++++++++++++++++++++------------------- 1 file changed, 108 insertions(+), 64 deletions(-) diff --git a/src/garage/cli/cmd.rs b/src/garage/cli/cmd.rs index 1a054025..c99243b9 100644 --- a/src/garage/cli/cmd.rs +++ b/src/garage/cli/cmd.rs @@ -1,4 +1,4 @@ -use std::collections::HashSet; +use std::collections::{HashMap, HashSet}; use std::time::Duration; use format_table::format_table; @@ -62,35 +62,69 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> let mut healthy_nodes = vec!["ID\tHostname\tAddress\tTags\tZone\tCapacity\tDataAvail".to_string()]; for adv in status.iter().filter(|adv| adv.is_up) { - match layout.current().roles.get(&adv.id) { - Some(NodeRoleV(Some(cfg))) => { - let data_avail = match &adv.status.data_disk_avail { - _ if cfg.capacity.is_none() => "N/A".into(), - Some((avail, total)) => { - let pct = (*avail as f64) / (*total as f64) * 100.; - let avail = bytesize::ByteSize::b(*avail); - format!("{} ({:.1}%)", avail, pct) - } - None => "?".into(), - }; + if let Some(NodeRoleV(Some(cfg))) = layout.current().roles.get(&adv.id) { + let data_avail = match &adv.status.data_disk_avail { + _ if cfg.capacity.is_none() => "N/A".into(), + Some((avail, total)) => { + let pct = (*avail as f64) / (*total as f64) * 100.; + let avail = bytesize::ByteSize::b(*avail); + format!("{} ({:.1}%)", avail, pct) + } + None => "?".into(), + }; + healthy_nodes.push(format!( + "{id:?}\t{host}\t{addr}\t[{tags}]\t{zone}\t{capacity}\t{data_avail}", + id = adv.id, + host = adv.status.hostname, + addr = adv.addr, + tags = cfg.tags.join(","), + zone = cfg.zone, + capacity = cfg.capacity_string(), + data_avail = data_avail, + )); + } else { + let prev_role = layout + .versions + .iter() + .rev() + .find_map(|x| match x.roles.get(&adv.id) { + Some(NodeRoleV(Some(cfg))) => Some(cfg), + _ => None, + }); + let historic_role = + layout + .old_versions + .iter() + .rev() + .find_map(|x| match x.roles.get(&adv.id) { + Some(NodeRoleV(Some(cfg))) => Some(cfg), + _ => None, + }); + if let Some(cfg) = prev_role { healthy_nodes.push(format!( - "{id:?}\t{host}\t{addr}\t[{tags}]\t{zone}\t{capacity}\t{data_avail}", + "{id:?}\t{host}\t{addr}\t[{tags}]\t{zone}\tdraining metadata...", id = adv.id, host = adv.status.hostname, addr = adv.addr, tags = cfg.tags.join(","), zone = cfg.zone, - capacity = cfg.capacity_string(), - data_avail = data_avail, )); - } - _ => { + } else if let Some(cfg) = historic_role { + healthy_nodes.push(format!( + "{id:?}\t{host}\t{addr}\t[{tags}]\t{zone}\tremoved, metadata drained", + id = adv.id, + host = adv.status.hostname, + addr = adv.addr, + tags = cfg.tags.join(","), + zone = cfg.zone, + )); + } else { let new_role = match layout.staging.get().roles.get(&adv.id) { - Some(NodeRoleV(Some(_))) => "(pending)", + Some(NodeRoleV(Some(_))) => "pending...", _ => "NO ROLE ASSIGNED", }; healthy_nodes.push(format!( - "{id:?}\t{h}\t{addr}\t{new_role}", + "{id:?}\t{h}\t{addr}\t\t\t{new_role}", id = adv.id, h = adv.status.hostname, addr = adv.addr, @@ -101,55 +135,65 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> } format_table(healthy_nodes); - let status_keys = status.iter().map(|adv| adv.id).collect::>(); - let failure_case_1 = status.iter().any(|adv| { - !adv.is_up - && matches!( - layout.current().roles.get(&adv.id), - Some(NodeRoleV(Some(_))) - ) - }); - let failure_case_2 = layout - .current() - .roles - .items() + // Determine which nodes are unhealthy and print that to stdout + let status_map = status .iter() - .any(|(id, _, v)| !status_keys.contains(id) && v.0.is_some()); - if failure_case_1 || failure_case_2 { - println!("\n==== FAILED NODES ===="); - let mut failed_nodes = - vec!["ID\tHostname\tAddress\tTags\tZone\tCapacity\tLast seen".to_string()]; - for adv in status.iter().filter(|adv| !adv.is_up) { - if let Some(NodeRoleV(Some(cfg))) = layout.current().roles.get(&adv.id) { - let tf = timeago::Formatter::new(); - failed_nodes.push(format!( - "{id:?}\t{host}\t{addr}\t[{tags}]\t{zone}\t{capacity}\t{last_seen}", - id = adv.id, - host = adv.status.hostname, - addr = adv.addr, - tags = cfg.tags.join(","), - zone = cfg.zone, - capacity = cfg.capacity_string(), - last_seen = adv - .last_seen_secs_ago - .map(|s| tf.convert(Duration::from_secs(s))) - .unwrap_or_else(|| "never seen".into()), - )); + .map(|adv| (adv.id, adv)) + .collect::>(); + + let tf = timeago::Formatter::new(); + let mut failed_nodes = + vec!["ID\tHostname\tAddress\tTags\tZone\tCapacity\tLast seen".to_string()]; + let mut listed = HashSet::new(); + for ver in layout.versions.iter().rev() { + for (node, _, role) in ver.roles.items().iter() { + let cfg = match role { + NodeRoleV(Some(role)) if role.capacity.is_some() => role, + _ => continue, + }; + + if listed.contains(node) { + continue; } - } - for (id, _, role_v) in layout.current().roles.items().iter() { - if let NodeRoleV(Some(cfg)) = role_v { - if !status_keys.contains(id) { - failed_nodes.push(format!( - "{id:?}\t??\t??\t[{tags}]\t{zone}\t{capacity}\tnever seen", - id = id, - tags = cfg.tags.join(","), - zone = cfg.zone, - capacity = cfg.capacity_string(), - )); - } + listed.insert(*node); + + let adv = status_map.get(node); + if adv.map(|x| x.is_up).unwrap_or(false) { + continue; } + + // Node is in a layout version, is not a gateway node, and is not up: + // it is in a failed state, add proper line to the output + let (host, addr, last_seen) = match adv { + Some(adv) => ( + adv.status.hostname.as_str(), + adv.addr.to_string(), + adv.last_seen_secs_ago + .map(|s| tf.convert(Duration::from_secs(s))) + .unwrap_or_else(|| "never seen".into()), + ), + None => ("??", "??".into(), "never seen".into()), + }; + let capacity = if ver.version == layout.current().version { + cfg.capacity_string() + } else { + "draining metadata...".to_string() + }; + failed_nodes.push(format!( + "{id:?}\t{host}\t{addr}\t[{tags}]\t{zone}\t{capacity}\t{last_seen}", + id = node, + host = host, + addr = addr, + tags = cfg.tags.join(","), + zone = cfg.zone, + capacity = capacity, + last_seen = last_seen, + )); } + } + + if failed_nodes.len() > 1 { + println!("\n==== FAILED NODES ===="); format_table(failed_nodes); } -- cgit v1.2.3 From 11e6fef93ce3ca56584fc99223b71da77d320dd7 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 27 Nov 2023 16:17:41 +0100 Subject: cli: add layout history and layout assume-sync commands --- src/garage/cli/cmd.rs | 14 +++++- src/garage/cli/layout.rs | 111 ++++++++++++++++++++++++++++++++++++++++++++++ src/garage/cli/structs.rs | 16 +++++++ src/rpc/layout/schema.rs | 9 +++- 4 files changed, 147 insertions(+), 3 deletions(-) diff --git a/src/garage/cli/cmd.rs b/src/garage/cli/cmd.rs index c99243b9..08ed00cf 100644 --- a/src/garage/cli/cmd.rs +++ b/src/garage/cli/cmd.rs @@ -135,13 +135,14 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> } format_table(healthy_nodes); - // Determine which nodes are unhealthy and print that to stdout + // Determine which nodes are unhealthy and print that to stdout let status_map = status .iter() .map(|adv| (adv.id, adv)) .collect::>(); let tf = timeago::Formatter::new(); + let mut drain_msg = false; let mut failed_nodes = vec!["ID\tHostname\tAddress\tTags\tZone\tCapacity\tLast seen".to_string()]; let mut listed = HashSet::new(); @@ -163,7 +164,7 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> } // Node is in a layout version, is not a gateway node, and is not up: - // it is in a failed state, add proper line to the output + // it is in a failed state, add proper line to the output let (host, addr, last_seen) = match adv { Some(adv) => ( adv.status.hostname.as_str(), @@ -177,6 +178,7 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> let capacity = if ver.version == layout.current().version { cfg.capacity_string() } else { + drain_msg = true; "draining metadata...".to_string() }; failed_nodes.push(format!( @@ -195,6 +197,14 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> if failed_nodes.len() > 1 { println!("\n==== FAILED NODES ===="); format_table(failed_nodes); + if drain_msg { + println!(); + println!("Your cluster is expecting to drain data from nodes that are currently unavailable."); + println!("If these nodes are definitely dead, please review the layout history with"); + println!( + "`garage layout history` and use `garage layout assume-sync` to force progress." + ); + } } if print_staging_role_changes(&layout) { diff --git a/src/garage/cli/layout.rs b/src/garage/cli/layout.rs index 0be8278f..3c7843bd 100644 --- a/src/garage/cli/layout.rs +++ b/src/garage/cli/layout.rs @@ -32,6 +32,10 @@ pub async fn cli_layout_command_dispatch( LayoutOperation::Config(config_opt) => { cmd_config_layout(system_rpc_endpoint, rpc_host, config_opt).await } + LayoutOperation::History => cmd_layout_history(system_rpc_endpoint, rpc_host).await, + LayoutOperation::AssumeSync(assume_sync_opt) => { + cmd_layout_assume_sync(system_rpc_endpoint, rpc_host, assume_sync_opt).await + } } } @@ -311,6 +315,113 @@ pub async fn cmd_config_layout( Ok(()) } +pub async fn cmd_layout_history( + rpc_cli: &Endpoint, + rpc_host: NodeID, +) -> Result<(), Error> { + let layout = fetch_layout(rpc_cli, rpc_host).await?; + let min_stored = layout.min_stored(); + + println!("==== LAYOUT HISTORY ===="); + let mut table = vec!["Version\tStatus\tStorage nodes\tGateway nodes".to_string()]; + for ver in layout + .versions + .iter() + .rev() + .chain(layout.old_versions.iter().rev()) + { + let status = if ver.version == layout.current().version { + "current" + } else if ver.version >= min_stored { + "draining" + } else { + "historical" + }; + table.push(format!( + "#{}\t{}\t{}\t{}", + ver.version, + status, + ver.roles + .items() + .iter() + .filter(|(_, _, x)| matches!(x, NodeRoleV(Some(c)) if c.capacity.is_some())) + .count(), + ver.roles + .items() + .iter() + .filter(|(_, _, x)| matches!(x, NodeRoleV(Some(c)) if c.capacity.is_none())) + .count(), + )); + } + format_table(table); + + println!(); + println!("==== UPDATE TRACKERS ===="); + println!("This is the internal data that Garage stores to know which nodes have what data."); + println!(); + let mut table = vec!["Node\tAck\tSync\tSync_ack".to_string()]; + let all_nodes = layout.get_all_nodes(); + for node in all_nodes.iter() { + table.push(format!( + "{:?}\t#{}\t#{}\t#{}", + node, + layout.update_trackers.ack_map.get(node), + layout.update_trackers.sync_map.get(node), + layout.update_trackers.sync_ack_map.get(node), + )); + } + table[1..].sort(); + format_table(table); + + if layout.versions.len() > 1 { + println!(); + println!( + "If some nodes are not catching up to the latest layout version in the update tracker," + ); + println!("it might be because they are offline or unable to complete a sync successfully."); + println!( + "You may force progress using `garage layout assume-sync --version {}`", + layout.current().version + ); + } + + Ok(()) +} + +pub async fn cmd_layout_assume_sync( + rpc_cli: &Endpoint, + rpc_host: NodeID, + opt: AssumeSyncOpt, +) -> Result<(), Error> { + let mut layout = fetch_layout(rpc_cli, rpc_host).await?; + + let min_v = layout.min_stored(); + if opt.version <= min_v || opt.version > layout.current().version { + return Err(Error::Message(format!( + "Invalid version, you may use the following version numbers: {}", + (min_v + 1..=layout.current().version) + .map(|x| x.to_string()) + .collect::>() + .join(" ") + ))); + } + + let all_nodes = layout.get_all_nodes(); + for node in all_nodes.iter() { + layout.update_trackers.ack_map.set_max(*node, opt.version); + layout.update_trackers.sync_map.set_max(*node, opt.version); + layout + .update_trackers + .sync_ack_map + .set_max(*node, opt.version); + } + + send_layout(rpc_cli, rpc_host, layout).await?; + println!("Success."); + + Ok(()) +} + // --- utility --- pub async fn fetch_layout( diff --git a/src/garage/cli/structs.rs b/src/garage/cli/structs.rs index 3badc447..c4b400f4 100644 --- a/src/garage/cli/structs.rs +++ b/src/garage/cli/structs.rs @@ -112,6 +112,14 @@ pub enum LayoutOperation { /// Revert staged changes to cluster layout #[structopt(name = "revert", version = garage_version())] Revert(RevertLayoutOpt), + + /// View the history of layouts in the cluster + #[structopt(name = "history", version = garage_version())] + History, + + /// Assume all nodes are synchronized up to a certain layout version + #[structopt(name = "assume-sync", version = garage_version())] + AssumeSync(AssumeSyncOpt), } #[derive(StructOpt, Debug)] @@ -169,6 +177,14 @@ pub struct RevertLayoutOpt { pub(crate) yes: bool, } +#[derive(StructOpt, Debug)] +pub struct AssumeSyncOpt { + /// Version number of the layout to assume is currently up-to-date. + /// This will generally be the current layout version. + #[structopt(long = "version")] + pub(crate) version: u64, +} + #[derive(Serialize, Deserialize, StructOpt, Debug)] pub enum BucketOperation { /// List buckets diff --git a/src/rpc/layout/schema.rs b/src/rpc/layout/schema.rs index 08db44ca..cb36297d 100644 --- a/src/rpc/layout/schema.rs +++ b/src/rpc/layout/schema.rs @@ -391,7 +391,10 @@ impl UpdateTracker { changed } - pub(crate) fn set_max(&mut self, peer: Uuid, value: u64) -> bool { + /// This bumps the update tracker for a given node up to the specified value. + /// This has potential impacts on the correctness of Garage and should only + /// be used in very specific circumstances. + pub fn set_max(&mut self, peer: Uuid, value: u64) -> bool { match self.0.get_mut(&peer) { Some(e) if *e < value => { *e = value; @@ -412,6 +415,10 @@ impl UpdateTracker { .min() .unwrap_or(min_version) } + + pub fn get(&self, node: &Uuid) -> u64 { + self.0.get(node).copied().unwrap_or(0) + } } impl UpdateTrackers { -- cgit v1.2.3 From c539077d30809c9d2232aa0fe107a9652dcb7c26 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 27 Nov 2023 16:20:19 +0100 Subject: cli: remove historic layout info from status --- src/garage/cli/cmd.rs | 18 ------------------ 1 file changed, 18 deletions(-) diff --git a/src/garage/cli/cmd.rs b/src/garage/cli/cmd.rs index 08ed00cf..4d1306b6 100644 --- a/src/garage/cli/cmd.rs +++ b/src/garage/cli/cmd.rs @@ -91,15 +91,6 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> Some(NodeRoleV(Some(cfg))) => Some(cfg), _ => None, }); - let historic_role = - layout - .old_versions - .iter() - .rev() - .find_map(|x| match x.roles.get(&adv.id) { - Some(NodeRoleV(Some(cfg))) => Some(cfg), - _ => None, - }); if let Some(cfg) = prev_role { healthy_nodes.push(format!( "{id:?}\t{host}\t{addr}\t[{tags}]\t{zone}\tdraining metadata...", @@ -109,15 +100,6 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> tags = cfg.tags.join(","), zone = cfg.zone, )); - } else if let Some(cfg) = historic_role { - healthy_nodes.push(format!( - "{id:?}\t{host}\t{addr}\t[{tags}]\t{zone}\tremoved, metadata drained", - id = adv.id, - host = adv.status.hostname, - addr = adv.addr, - tags = cfg.tags.join(","), - zone = cfg.zone, - )); } else { let new_role = match layout.staging.get().roles.get(&adv.id) { Some(NodeRoleV(Some(_))) => "pending...", -- cgit v1.2.3 From 539af6eac434bd94acbcabcc5bb5c10450b71c5d Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 28 Nov 2023 11:12:39 +0100 Subject: rpc helper: write comments + small refactoring of tracing --- src/rpc/rpc_helper.rs | 105 ++++++++++++++++++++++++++++++++++++++++++-------- 1 file changed, 88 insertions(+), 17 deletions(-) diff --git a/src/rpc/rpc_helper.rs b/src/rpc/rpc_helper.rs index e9a9143f..f71f5ae7 100644 --- a/src/rpc/rpc_helper.rs +++ b/src/rpc/rpc_helper.rs @@ -129,6 +129,12 @@ impl RpcHelper { N: IntoReq + Send, H: StreamingEndpointHandler, { + let tracer = opentelemetry::global::tracer("garage"); + let span_name = format!("RPC [{}] to {:?}", endpoint.path(), to); + let mut span = tracer.start(span_name); + span.set_attribute(KeyValue::new("from", format!("{:?}", self.0.our_node_id))); + span.set_attribute(KeyValue::new("to", format!("{:?}", to))); + let metric_tags = [ KeyValue::new("rpc_endpoint", endpoint.path().to_string()), KeyValue::new("from", format!("{:?}", self.0.our_node_id)), @@ -140,6 +146,7 @@ impl RpcHelper { let node_id = to.into(); let rpc_call = endpoint .call_streaming(&node_id, msg, strat.rs_priority) + .with_context(Context::current_with_span(span)) .record_duration(&self.0.metrics.rpc_duration, &metric_tags); let timeout = async { @@ -182,12 +189,17 @@ impl RpcHelper { N: IntoReq, H: StreamingEndpointHandler, { + let tracer = opentelemetry::global::tracer("garage"); + let span_name = format!("RPC [{}] call_many {} nodes", endpoint.path(), to.len()); + let span = tracer.start(span_name); + let msg = msg.into_req().map_err(netapp::error::Error::from)?; let resps = join_all( to.iter() .map(|to| self.call(endpoint, *to, msg.clone(), strat)), ) + .with_context(Context::current_with_span(span)) .await; Ok(to .iter() @@ -219,6 +231,22 @@ impl RpcHelper { /// Make a RPC call to multiple servers, returning either a Vec of responses, /// or an error if quorum could not be reached due to too many errors + /// + /// If RequestStrategy has send_all_at_once set, then all requests will be + /// sent at once, and `try_call_many` will return as soon as a quorum of + /// responses is achieved, dropping and cancelling the remaining requests. + /// + /// Otherwise, `quorum` requests will be sent at the same time, and if an + /// error response is received, a new request will be sent to replace it. + /// The ordering of nodes to which requests are sent is determined by + /// the `RpcHelper::request_order` function, which takes into account + /// parameters such as node zones and measured ping values. + /// + /// In both cases, the basic contract of this function is that even in the + /// absence of failures, the RPC call might not be driven to completion + /// on all of the specified nodes. It is therefore unfit for broadcast + /// write operations where we expect all nodes to successfully store + /// the written date. pub async fn try_call_many( &self, endpoint: &Arc>, @@ -235,7 +263,12 @@ impl RpcHelper { let quorum = strategy.rs_quorum.unwrap_or(to.len()); let tracer = opentelemetry::global::tracer("garage"); - let span_name = format!("Read RPC {} to {} of {}", endpoint.path(), quorum, to.len()); + let span_name = format!( + "RPC [{}] try_call_many (quorum {}/{})", + endpoint.path(), + quorum, + to.len() + ); let mut span = tracer.start(span_name); span.set_attribute(KeyValue::new("from", format!("{:?}", self.0.our_node_id))); @@ -266,6 +299,10 @@ impl RpcHelper { // to reach a quorum, priorizing nodes with the lowest latency. // When there are errors, we start new requests to compensate. + // TODO: this could be made more aggressive, e.g. if after 2x the + // average ping of a given request, the response is not yet received, + // preemptively send an additional request to any remaining nodes. + // Reorder requests to priorize closeness / low latency let request_order = self.request_order(to.iter().copied()); let send_all_at_once = strategy.rs_send_all_at_once.unwrap_or(false); @@ -278,9 +315,7 @@ impl RpcHelper { let self2 = self.clone(); let msg = msg.clone(); let endpoint2 = endpoint.clone(); - (to, async move { - self2.call(&endpoint2, to, msg, strategy).await - }) + async move { self2.call(&endpoint2, to, msg, strategy).await } }); // Vectors in which success results and errors will be collected @@ -296,10 +331,8 @@ impl RpcHelper { // If the current set of requests that are running is not enough to possibly // reach quorum, start some new requests. while send_all_at_once || successes.len() + resp_stream.len() < quorum { - if let Some((req_to, fut)) = requests.next() { - let tracer = opentelemetry::global::tracer("garage"); - let span = tracer.start(format!("RPC to {:?}", req_to)); - resp_stream.push(fut.with_context(Context::current_with_span(span))); + if let Some(fut) = requests.next() { + resp_stream.push(fut) } else { break; } @@ -379,6 +412,25 @@ impl RpcHelper { .collect::>() } + /// Make a RPC call to multiple servers, returning either a Vec of responses, + /// or an error if quorum could not be reached due to too many errors + /// + /// Contrary to try_call_many, this fuction is especially made for broadcast + /// write operations. In particular: + /// + /// - The request are sent to all specified nodes as soon as `try_write_many_sets` + /// is invoked. + /// + /// - When `try_write_many_sets` returns, all remaining requests that haven't + /// completed move to a background task so that they have a chance to + /// complete successfully if there are no failures. + /// + /// In addition, the nodes to which requests should be sent are divided in + /// "quorum sets", and `try_write_many_sets` only returns once a quorum + /// has been validated in each set. This is used in the case of cluster layout + /// changes, where data has to be written both in the old layout and in the + /// new one as long as all nodes have not successfully tranisitionned and + /// moved all data to the new layout. pub async fn try_write_many_sets( &self, endpoint: &Arc>, @@ -394,11 +446,11 @@ impl RpcHelper { { let quorum = strategy .rs_quorum - .expect("internal error: missing quroum in try_write_many_sets"); + .expect("internal error: missing quorum value in try_write_many_sets"); let tracer = opentelemetry::global::tracer("garage"); let span_name = format!( - "Write RPC {} (quorum {} in {} sets)", + "RPC [{}] try_write_many_sets (quorum {} in {} sets)", endpoint.path(), quorum, to_sets.len() @@ -430,6 +482,8 @@ impl RpcHelper { { let msg = msg.into_req().map_err(netapp::error::Error::from)?; + // Peers may appear in many quorum sets. Here, build a list of peers, + // mapping to the index of the quorum sets in which they appear. let mut peers = HashMap::>::new(); for (i, set) in to_sets.iter().enumerate() { for peer in set.iter() { @@ -437,24 +491,30 @@ impl RpcHelper { } } + // Send one request to each peer of the quorum sets let requests = peers.iter().map(|(peer, _)| { let self2 = self.clone(); let msg = msg.clone(); let endpoint2 = endpoint.clone(); let to = *peer; - let tracer = opentelemetry::global::tracer("garage"); - let span = tracer.start(format!("RPC to {:?}", to)); - let fut = async move { (to, self2.call(&endpoint2, to, msg, strategy).await) }; - fut.with_context(Context::current_with_span(span)) + async move { (to, self2.call(&endpoint2, to, msg, strategy).await) } }); let mut resp_stream = requests.collect::>(); + // Success and error responses will be collected in these two vectors let mut successes = vec![]; let mut errors = vec![]; + // `set_counters` is used to keep track of how many success and error + // responses are received within each quorum set. When a node returns + // its response, it counts as a sucess/an error for all of the quorum + // sets which it is part of. let mut set_counters = vec![(0, 0); to_sets.len()]; + // Drive requests to completion while let Some((node, resp)) = resp_stream.next().await { + // Store the response in the correct vector and increment the + // appropriate counters match resp { Ok(msg) => { for set in peers.get(&node).unwrap().iter() { @@ -470,9 +530,8 @@ impl RpcHelper { } } + // If we have a quorum of ok in all quorum sets, then it's a success! if set_counters.iter().all(|(ok_cnt, _)| *ok_cnt >= quorum) { - // Success - // Continue all other requets in background tokio::spawn(async move { resp_stream.collect::)>>().await; @@ -481,16 +540,28 @@ impl RpcHelper { return Ok(successes); } + // If there is a quorum set for which too many errors were received, + // we know it's impossible to get a quorum, so return immediately. if set_counters .iter() .enumerate() .any(|(i, (_, err_cnt))| err_cnt + quorum > to_sets[i].len()) { - // Too many errors in this set, we know we won't get a quorum break; } } + // At this point, there is no quorum and we know that a quorum + // will never be achieved. Currently, we drop all remaining requests. + // Should we still move them to background so that they can continue + // for non-failed nodes? Not doing so has no impact on correctness, + // but it means that more cancellation messages will be sent. Idk. + // (When an in-progress request future is dropped, Netapp automatically + // sends a cancellation message to the remote node to inform it that + // the result is no longer needed. In turn, if the remote node receives + // the cancellation message in time, it interrupts the task of the + // running request handler.) + // Failure, could not get quorum let errors = errors.iter().map(|e| format!("{}", e)).collect::>(); Err(Error::Quorum( -- cgit v1.2.3 From c04dd8788a3764da2f307b1d10c2d56b7b0e4a61 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 28 Nov 2023 14:25:04 +0100 Subject: admin: more info in admin GetClusterStatus --- doc/drafts/admin-api.md | 139 +++++++++++++++++++++++++---------------------- src/api/admin/cluster.rs | 122 ++++++++++++++++++++++++++++++++++------- src/garage/admin/mod.rs | 2 +- src/garage/cli/cmd.rs | 9 +-- src/rpc/system.rs | 12 ++-- 5 files changed, 190 insertions(+), 94 deletions(-) diff --git a/doc/drafts/admin-api.md b/doc/drafts/admin-api.md index 411f6418..274bd5c4 100644 --- a/doc/drafts/admin-api.md +++ b/doc/drafts/admin-api.md @@ -69,8 +69,8 @@ Example response body: ```json { - "node": "ec79480e0ce52ae26fd00c9da684e4fa56658d9c64cdcecb094e936de0bfe71f", - "garageVersion": "git:v0.9.0-dev", + "node": "b10c110e4e854e5aa3f4637681befac755154b20059ec163254ddbfae86b09df", + "garageVersion": "v0.10.0", "garageFeatures": [ "k2v", "sled", @@ -81,83 +81,92 @@ Example response body: ], "rustVersion": "1.68.0", "dbEngine": "LMDB (using Heed crate)", - "knownNodes": [ + "layoutVersion": 5, + "nodes": [ { - "id": "ec79480e0ce52ae26fd00c9da684e4fa56658d9c64cdcecb094e936de0bfe71f", - "addr": "10.0.0.11:3901", + "id": "62b218d848e86a64f7fe1909735f29a4350547b54c4b204f91246a14eb0a1a8c", + "role": { + "id": "62b218d848e86a64f7fe1909735f29a4350547b54c4b204f91246a14eb0a1a8c", + "zone": "dc1", + "capacity": 100000000000, + "tags": [] + }, + "addr": "10.0.0.3:3901", + "hostname": "node3", "isUp": true, - "lastSeenSecsAgo": 9, - "hostname": "node1" + "lastSeenSecsAgo": 12, + "draining": false, + "dataPartition": { + "available": 660270088192, + "total": 873862266880 + }, + "metadataPartition": { + "available": 660270088192, + "total": 873862266880 + } }, { - "id": "4a6ae5a1d0d33bf895f5bb4f0a418b7dc94c47c0dd2eb108d1158f3c8f60b0ff", - "addr": "10.0.0.12:3901", + "id": "a11c7cf18af297379eff8688360155fe68d9061654449ba0ce239252f5a7487f", + "role": null, + "addr": "10.0.0.2:3901", + "hostname": "node2", "isUp": true, - "lastSeenSecsAgo": 1, - "hostname": "node2" + "lastSeenSecsAgo": 11, + "draining": true, + "dataPartition": { + "available": 660270088192, + "total": 873862266880 + }, + "metadataPartition": { + "available": 660270088192, + "total": 873862266880 + } }, { - "id": "23ffd0cdd375ebff573b20cc5cef38996b51c1a7d6dbcf2c6e619876e507cf27", - "addr": "10.0.0.21:3901", + "id": "a235ac7695e0c54d7b403943025f57504d500fdcc5c3e42c71c5212faca040a2", + "role": { + "id": "a235ac7695e0c54d7b403943025f57504d500fdcc5c3e42c71c5212faca040a2", + "zone": "dc1", + "capacity": 100000000000, + "tags": [] + }, + "addr": "127.0.0.1:3904", + "hostname": "lindy", "isUp": true, - "lastSeenSecsAgo": 7, - "hostname": "node3" + "lastSeenSecsAgo": 2, + "draining": false, + "dataPartition": { + "available": 660270088192, + "total": 873862266880 + }, + "metadataPartition": { + "available": 660270088192, + "total": 873862266880 + } }, { - "id": "e2ee7984ee65b260682086ec70026165903c86e601a4a5a501c1900afe28d84b", - "addr": "10.0.0.22:3901", - "isUp": true, - "lastSeenSecsAgo": 1, - "hostname": "node4" - } - ], - "layout": { - "version": 12, - "roles": [ - { - "id": "ec79480e0ce52ae26fd00c9da684e4fa56658d9c64cdcecb094e936de0bfe71f", + "id": "b10c110e4e854e5aa3f4637681befac755154b20059ec163254ddbfae86b09df", + "role": { + "id": "b10c110e4e854e5aa3f4637681befac755154b20059ec163254ddbfae86b09df", "zone": "dc1", - "capacity": 10737418240, - "tags": [ - "node1" - ] + "capacity": 100000000000, + "tags": [] }, - { - "id": "4a6ae5a1d0d33bf895f5bb4f0a418b7dc94c47c0dd2eb108d1158f3c8f60b0ff", - "zone": "dc1", - "capacity": 10737418240, - "tags": [ - "node2" - ] + "addr": "10.0.0.1:3901", + "hostname": "node1", + "isUp": true, + "lastSeenSecsAgo": 3, + "draining": false, + "dataPartition": { + "available": 660270088192, + "total": 873862266880 }, - { - "id": "23ffd0cdd375ebff573b20cc5cef38996b51c1a7d6dbcf2c6e619876e507cf27", - "zone": "dc2", - "capacity": 10737418240, - "tags": [ - "node3" - ] + "metadataPartition": { + "available": 660270088192, + "total": 873862266880 } - ], - "stagedRoleChanges": [ - { - "id": "e2ee7984ee65b260682086ec70026165903c86e601a4a5a501c1900afe28d84b", - "remove": false, - "zone": "dc2", - "capacity": 10737418240, - "tags": [ - "node4" - ] - } - { - "id": "23ffd0cdd375ebff573b20cc5cef38996b51c1a7d6dbcf2c6e619876e507cf27", - "remove": true, - "zone": null, - "capacity": null, - "tags": null, - } - ] - } + } + ] } ``` diff --git a/src/api/admin/cluster.rs b/src/api/admin/cluster.rs index 593bd778..3ce1b254 100644 --- a/src/api/admin/cluster.rs +++ b/src/api/admin/cluster.rs @@ -1,3 +1,4 @@ +use std::collections::HashMap; use std::net::SocketAddr; use std::sync::Arc; @@ -15,25 +16,95 @@ use crate::admin::error::*; use crate::helpers::{json_ok_response, parse_json_body}; pub async fn handle_get_cluster_status(garage: &Arc) -> Result, Error> { + let layout = garage.system.cluster_layout(); + let mut nodes = garage + .system + .get_known_nodes() + .into_iter() + .map(|i| { + ( + i.id, + NodeResp { + id: hex::encode(i.id), + addr: Some(i.addr), + hostname: i.status.hostname, + is_up: i.is_up, + last_seen_secs_ago: i.last_seen_secs_ago, + data_partition: i + .status + .data_disk_avail + .map(|(avail, total)| FreeSpaceResp { + available: avail, + total, + }), + metadata_partition: i.status.meta_disk_avail.map(|(avail, total)| { + FreeSpaceResp { + available: avail, + total, + } + }), + ..Default::default() + }, + ) + }) + .collect::>(); + + for (id, _, role) in layout.current().roles.items().iter() { + if let layout::NodeRoleV(Some(r)) = role { + let role = NodeRoleResp { + id: hex::encode(id), + zone: r.zone.to_string(), + capacity: r.capacity, + tags: r.tags.clone(), + }; + match nodes.get_mut(id) { + None => { + nodes.insert( + *id, + NodeResp { + id: hex::encode(id), + role: Some(role), + ..Default::default() + }, + ); + } + Some(n) => { + if n.role.is_none() { + n.role = Some(role); + } + } + } + } + } + + for ver in layout.versions.iter().rev().skip(1) { + for (id, _, role) in ver.roles.items().iter() { + if let layout::NodeRoleV(Some(r)) = role { + if !nodes.contains_key(id) && r.capacity.is_some() { + nodes.insert( + *id, + NodeResp { + id: hex::encode(id), + draining: true, + ..Default::default() + }, + ); + } + } + } + } + + let mut nodes = nodes.into_iter().map(|(_, v)| v).collect::>(); + nodes.sort_by(|x, y| x.id.cmp(&y.id)); + let res = GetClusterStatusResponse { node: hex::encode(garage.system.id), garage_version: garage_util::version::garage_version(), garage_features: garage_util::version::garage_features(), rust_version: garage_util::version::rust_version(), db_engine: garage.db.engine(), - known_nodes: garage - .system - .get_known_nodes() - .into_iter() - .map(|i| KnownNodeResp { - id: hex::encode(i.id), - addr: i.addr, - is_up: i.is_up, - last_seen_secs_ago: i.last_seen_secs_ago, - hostname: i.status.hostname, - }) - .collect(), - layout: format_cluster_layout(&garage.system.cluster_layout()), + layout_version: layout.current().version, + nodes, }; Ok(json_ok_response(&res)?) @@ -157,8 +228,8 @@ struct GetClusterStatusResponse { garage_features: Option<&'static [&'static str]>, rust_version: &'static str, db_engine: String, - known_nodes: Vec, - layout: GetClusterLayoutResponse, + layout_version: u64, + nodes: Vec, } #[derive(Serialize)] @@ -192,14 +263,27 @@ struct NodeRoleResp { tags: Vec, } -#[derive(Serialize)] +#[derive(Serialize, Default)] +#[serde(rename_all = "camelCase")] +struct FreeSpaceResp { + available: u64, + total: u64, +} + +#[derive(Serialize, Default)] #[serde(rename_all = "camelCase")] -struct KnownNodeResp { +struct NodeResp { id: String, - addr: SocketAddr, + role: Option, + addr: Option, + hostname: Option, is_up: bool, last_seen_secs_ago: Option, - hostname: String, + draining: bool, + #[serde(skip_serializing_if = "Option::is_none")] + data_partition: Option, + #[serde(skip_serializing_if = "Option::is_none")] + metadata_partition: Option, } // ---- update functions ---- diff --git a/src/garage/admin/mod.rs b/src/garage/admin/mod.rs index 77918a0f..da4226cf 100644 --- a/src/garage/admin/mod.rs +++ b/src/garage/admin/mod.rs @@ -295,7 +295,7 @@ impl AdminRpcHandler { let info = node_info.get(id); let status = info.map(|x| &x.status); let role = layout.current().roles.get(id).and_then(|x| x.0.as_ref()); - let hostname = status.map(|x| x.hostname.as_str()).unwrap_or("?"); + let hostname = status.and_then(|x| x.hostname.as_deref()).unwrap_or("?"); let zone = role.map(|x| x.zone.as_str()).unwrap_or("?"); let capacity = role .map(|x| x.capacity_string()) diff --git a/src/garage/cli/cmd.rs b/src/garage/cli/cmd.rs index 4d1306b6..c7f0ad2b 100644 --- a/src/garage/cli/cmd.rs +++ b/src/garage/cli/cmd.rs @@ -62,6 +62,7 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> let mut healthy_nodes = vec!["ID\tHostname\tAddress\tTags\tZone\tCapacity\tDataAvail".to_string()]; for adv in status.iter().filter(|adv| adv.is_up) { + let host = adv.status.hostname.as_deref().unwrap_or("?"); if let Some(NodeRoleV(Some(cfg))) = layout.current().roles.get(&adv.id) { let data_avail = match &adv.status.data_disk_avail { _ if cfg.capacity.is_none() => "N/A".into(), @@ -75,7 +76,7 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> healthy_nodes.push(format!( "{id:?}\t{host}\t{addr}\t[{tags}]\t{zone}\t{capacity}\t{data_avail}", id = adv.id, - host = adv.status.hostname, + host = host, addr = adv.addr, tags = cfg.tags.join(","), zone = cfg.zone, @@ -95,7 +96,7 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> healthy_nodes.push(format!( "{id:?}\t{host}\t{addr}\t[{tags}]\t{zone}\tdraining metadata...", id = adv.id, - host = adv.status.hostname, + host = host, addr = adv.addr, tags = cfg.tags.join(","), zone = cfg.zone, @@ -108,7 +109,7 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> healthy_nodes.push(format!( "{id:?}\t{h}\t{addr}\t\t\t{new_role}", id = adv.id, - h = adv.status.hostname, + h = host, addr = adv.addr, new_role = new_role, )); @@ -149,7 +150,7 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> // it is in a failed state, add proper line to the output let (host, addr, last_seen) = match adv { Some(adv) => ( - adv.status.hostname.as_str(), + adv.status.hostname.as_deref().unwrap_or("?"), adv.addr.to_string(), adv.last_seen_secs_ago .map(|s| tf.convert(Duration::from_secs(s))) diff --git a/src/rpc/system.rs b/src/rpc/system.rs index c7d41ee4..be4aefa2 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -126,7 +126,7 @@ pub struct System { #[derive(Debug, Clone, Serialize, Deserialize)] pub struct NodeStatus { /// Hostname of the node - pub hostname: String, + pub hostname: Option, /// Replication factor configured on the node pub replication_factor: usize, @@ -765,9 +765,11 @@ impl EndpointHandler for System { impl NodeStatus { fn initial(replication_factor: usize, layout_manager: &LayoutManager) -> Self { NodeStatus { - hostname: gethostname::gethostname() - .into_string() - .unwrap_or_else(|_| "".to_string()), + hostname: Some( + gethostname::gethostname() + .into_string() + .unwrap_or_else(|_| "".to_string()), + ), replication_factor, layout_digest: layout_manager.layout().digest(), meta_disk_avail: None, @@ -777,7 +779,7 @@ impl NodeStatus { fn unknown() -> Self { NodeStatus { - hostname: "?".to_string(), + hostname: None, replication_factor: 0, layout_digest: Default::default(), meta_disk_avail: None, -- cgit v1.2.3 From c8356a91d9bf1d1488ec288099f2a55a1019918f Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 7 Dec 2023 10:30:26 +0100 Subject: layout updates: fix the set of nodes among which minima are calculated --- src/rpc/layout/helper.rs | 25 +++++++++++++++++++++---- src/rpc/layout/history.rs | 8 +++++--- src/rpc/layout/schema.rs | 2 +- 3 files changed, 27 insertions(+), 8 deletions(-) diff --git a/src/rpc/layout/helper.rs b/src/rpc/layout/helper.rs index 5d159f3e..881a039e 100644 --- a/src/rpc/layout/helper.rs +++ b/src/rpc/layout/helper.rs @@ -51,20 +51,37 @@ impl LayoutHelper { pub fn new(mut layout: LayoutHistory, mut ack_lock: HashMap) -> Self { layout.cleanup_old_versions(); + let all_nodes = layout.get_all_nodes(); let all_nongateway_nodes = layout.get_all_nongateway_nodes(); - layout.clamp_update_trackers(&all_nongateway_nodes); + + layout.clamp_update_trackers(&all_nodes); let min_version = layout.min_stored(); + + // ack_map_min is the minimum value of ack_map among all nodes + // in the cluster (gateway, non-gateway, current and previous layouts). + // It is the highest layout version which all of these nodes have + // acknowledged, indicating that they are aware of it and are no + // longer processing write operations that did not take it into account. let ack_map_min = layout .update_trackers .ack_map - .min(&all_nongateway_nodes, min_version); + .min_among(&all_nodes, min_version); + + // sync_map_min is the minimum value of sync_map among all storage nodes + // in the cluster (non-gateway nodes only, current and previous layouts). + // It is the highest layout version for which we know that all relevant + // storage nodes have fullfilled a sync, and therefore it is safe to + // use a read quorum within that layout to ensure consistency. + // Gateway nodes are excluded here because they hold no relevant data + // (they store the bucket and access key tables, but we don't have + // consistency on those). + // TODO: this value could take quorums into account instead. let sync_map_min = layout .update_trackers .sync_map - .min(&all_nongateway_nodes, min_version); + .min_among(&all_nongateway_nodes, min_version); - let all_nodes = layout.get_all_nodes(); let trackers_hash = layout.calculate_trackers_hash(); let staging_hash = layout.calculate_staging_hash(); diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index 7d4a1b48..c448ac24 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -77,14 +77,16 @@ impl LayoutHistory { } // If there are old versions that no one is reading from anymore, - // remove them + // remove them (keep them in self.old_versions). + // ASSUMPTION: we only care about where nodes in the current layout version + // are reading from, as we assume older nodes are being discarded. while self.versions.len() > 1 { - let all_nongateway_nodes = self.get_all_nongateway_nodes(); + let current_nodes = &self.current().node_id_vec; let min_version = self.min_stored(); let sync_ack_map_min = self .update_trackers .sync_ack_map - .min(&all_nongateway_nodes, min_version); + .min_among(¤t_nodes, min_version); if self.min_stored() < sync_ack_map_min { let removed = self.versions.remove(0); info!( diff --git a/src/rpc/layout/schema.rs b/src/rpc/layout/schema.rs index cb36297d..49e84420 100644 --- a/src/rpc/layout/schema.rs +++ b/src/rpc/layout/schema.rs @@ -408,7 +408,7 @@ impl UpdateTracker { } } - pub(crate) fn min(&self, storage_nodes: &[Uuid], min_version: u64) -> u64 { + pub(crate) fn min_among(&self, storage_nodes: &[Uuid], min_version: u64) -> u64 { storage_nodes .iter() .map(|x| self.0.get(x).copied().unwrap_or(min_version)) -- cgit v1.2.3 From 95eb13eb08d517d328e3c8aeb222440a27211ee9 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 7 Dec 2023 10:55:15 +0100 Subject: rpc: refactor result tracking for quorum sets --- src/rpc/layout/manager.rs | 6 ++ src/rpc/rpc_helper.rs | 147 +++++++++++++++++++++++++----------- src/table/replication/parameters.rs | 2 +- src/table/table.rs | 54 +++---------- 4 files changed, 121 insertions(+), 88 deletions(-) diff --git a/src/rpc/layout/manager.rs b/src/rpc/layout/manager.rs index c65831a2..17465019 100644 --- a/src/rpc/layout/manager.rs +++ b/src/rpc/layout/manager.rs @@ -352,6 +352,12 @@ impl AsRef for WriteLock { } } +impl AsMut for WriteLock { + fn as_mut(&mut self) -> &mut T { + &mut self.value + } +} + impl Drop for WriteLock { fn drop(&mut self) { let layout = self.layout_manager.layout(); // acquire read lock diff --git a/src/rpc/rpc_helper.rs b/src/rpc/rpc_helper.rs index f71f5ae7..c6dcbe75 100644 --- a/src/rpc/rpc_helper.rs +++ b/src/rpc/rpc_helper.rs @@ -484,15 +484,10 @@ impl RpcHelper { // Peers may appear in many quorum sets. Here, build a list of peers, // mapping to the index of the quorum sets in which they appear. - let mut peers = HashMap::>::new(); - for (i, set) in to_sets.iter().enumerate() { - for peer in set.iter() { - peers.entry(*peer).or_default().push(i); - } - } + let mut result_tracker = QuorumSetResultTracker::new(to_sets, quorum); // Send one request to each peer of the quorum sets - let requests = peers.iter().map(|(peer, _)| { + let requests = result_tracker.nodes.iter().map(|(peer, _)| { let self2 = self.clone(); let msg = msg.clone(); let endpoint2 = endpoint.clone(); @@ -501,52 +496,25 @@ impl RpcHelper { }); let mut resp_stream = requests.collect::>(); - // Success and error responses will be collected in these two vectors - let mut successes = vec![]; - let mut errors = vec![]; - - // `set_counters` is used to keep track of how many success and error - // responses are received within each quorum set. When a node returns - // its response, it counts as a sucess/an error for all of the quorum - // sets which it is part of. - let mut set_counters = vec![(0, 0); to_sets.len()]; - // Drive requests to completion while let Some((node, resp)) = resp_stream.next().await { // Store the response in the correct vector and increment the // appropriate counters - match resp { - Ok(msg) => { - for set in peers.get(&node).unwrap().iter() { - set_counters[*set].0 += 1; - } - successes.push(msg); - } - Err(e) => { - for set in peers.get(&node).unwrap().iter() { - set_counters[*set].1 += 1; - } - errors.push(e); - } - } + result_tracker.register_result(node, resp); // If we have a quorum of ok in all quorum sets, then it's a success! - if set_counters.iter().all(|(ok_cnt, _)| *ok_cnt >= quorum) { + if result_tracker.all_quorums_ok() { // Continue all other requets in background tokio::spawn(async move { resp_stream.collect::)>>().await; }); - return Ok(successes); + return Ok(result_tracker.success_values()); } // If there is a quorum set for which too many errors were received, // we know it's impossible to get a quorum, so return immediately. - if set_counters - .iter() - .enumerate() - .any(|(i, (_, err_cnt))| err_cnt + quorum > to_sets[i].len()) - { + if result_tracker.too_many_failures() { break; } } @@ -563,13 +531,104 @@ impl RpcHelper { // running request handler.) // Failure, could not get quorum - let errors = errors.iter().map(|e| format!("{}", e)).collect::>(); - Err(Error::Quorum( + Err(result_tracker.quorum_error()) + } +} + +// ------- utility for tracking successes/errors among write sets -------- + +pub struct QuorumSetResultTracker { + // The set of nodes and the quorum sets they belong to + pub nodes: HashMap>, + pub quorum: usize, + + // The success and error responses received + pub successes: Vec<(Uuid, S)>, + pub failures: Vec<(Uuid, E)>, + + // The counters for successes and failures in each set + pub success_counters: Box<[usize]>, + pub failure_counters: Box<[usize]>, + pub set_lens: Box<[usize]>, +} + +impl QuorumSetResultTracker { + pub fn new(sets: &[A], quorum: usize) -> Self + where + A: AsRef<[Uuid]>, + { + let mut nodes = HashMap::>::new(); + for (i, set) in sets.iter().enumerate() { + for node in set.as_ref().iter() { + nodes.entry(*node).or_default().push(i); + } + } + + let num_nodes = nodes.len(); + Self { + nodes, quorum, - Some(to_sets.len()), - successes.len(), - peers.len(), + successes: Vec::with_capacity(num_nodes), + failures: vec![], + success_counters: vec![0; sets.len()].into_boxed_slice(), + failure_counters: vec![0; sets.len()].into_boxed_slice(), + set_lens: sets + .iter() + .map(|x| x.as_ref().len()) + .collect::>() + .into_boxed_slice(), + } + } + + pub fn register_result(&mut self, node: Uuid, result: Result) { + match result { + Ok(s) => { + self.successes.push((node, s)); + for set in self.nodes.get(&node).unwrap().iter() { + self.success_counters[*set] += 1; + } + } + Err(e) => { + self.failures.push((node, e)); + for set in self.nodes.get(&node).unwrap().iter() { + self.failure_counters[*set] += 1; + } + } + } + } + + pub fn all_quorums_ok(&self) -> bool { + self.success_counters + .iter() + .all(|ok_cnt| *ok_cnt >= self.quorum) + } + + pub fn too_many_failures(&self) -> bool { + self.failure_counters + .iter() + .zip(self.set_lens.iter()) + .any(|(err_cnt, set_len)| *err_cnt + self.quorum > *set_len) + } + + pub fn success_values(self) -> Vec { + self.successes + .into_iter() + .map(|(_, x)| x) + .collect::>() + } + + pub fn quorum_error(self) -> Error { + let errors = self + .failures + .iter() + .map(|(n, e)| format!("{:?}: {}", n, e)) + .collect::>(); + Error::Quorum( + self.quorum, + Some(self.set_lens.len()), + self.successes.len(), + self.nodes.len(), errors, - )) + ) } } diff --git a/src/table/replication/parameters.rs b/src/table/replication/parameters.rs index a4e701bb..db11ff5f 100644 --- a/src/table/replication/parameters.rs +++ b/src/table/replication/parameters.rs @@ -3,7 +3,7 @@ use garage_util::data::*; /// Trait to describe how a table shall be replicated pub trait TableReplication: Send + Sync + 'static { - type WriteSets: AsRef>> + Send + Sync + 'static; + type WriteSets: AsRef>> + AsMut>> + Send + Sync + 'static; // See examples in table_sharded.rs and table_fullcopy.rs // To understand various replication methods diff --git a/src/table/table.rs b/src/table/table.rs index 7d1ff31c..6508cf5d 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -20,6 +20,7 @@ use garage_util::error::Error; use garage_util::metrics::RecordDuration; use garage_util::migrate::Migrate; +use garage_rpc::rpc_helper::QuorumSetResultTracker; use garage_rpc::system::System; use garage_rpc::*; @@ -180,10 +181,6 @@ impl Table { // a quorum of nodes has answered OK, then the insert has succeeded and // consistency properties (read-after-write) are preserved. - // Some code here might feel redundant with RpcHelper::try_write_many_sets, - // but I think deduplicating could lead to more spaghetti instead of - // improving the readability, so I'm leaving as is. - let quorum = self.data.replication.write_quorum(); // Serialize all entries and compute the write sets for each of them. @@ -197,7 +194,10 @@ impl Table { for entry in entries.into_iter() { let entry = entry.borrow(); let hash = entry.partition_key().hash(); - let write_sets = self.data.replication.write_sets(&hash); + let mut write_sets = self.data.replication.write_sets(&hash); + for set in write_sets.as_mut().iter_mut() { + set.sort(); + } let e_enc = Arc::new(ByteBuf::from(entry.encode()?)); entries_vec.push((write_sets, e_enc)); } @@ -212,12 +212,8 @@ impl Table { .collect::>(); write_sets.sort(); write_sets.dedup(); - let mut write_set_index = HashMap::<&Uuid, Vec>::new(); - for (i, write_set) in write_sets.iter().enumerate() { - for node in write_set.iter() { - write_set_index.entry(node).or_default().push(i); - } - } + + let mut result_tracker = QuorumSetResultTracker::new(&write_sets, quorum); // Build a map of all nodes to the entries that must be sent to that node. let mut call_list: HashMap> = HashMap::new(); @@ -230,7 +226,6 @@ impl Table { } // Build futures to actually perform each of the corresponding RPC calls - let call_count = call_list.len(); let call_futures = call_list.into_iter().map(|(node, entries)| { let this = self.clone(); let tracer = opentelemetry::global::tracer("garage"); @@ -254,27 +249,11 @@ impl Table { // Run all requests in parallel thanks to FuturesUnordered, and collect results. let mut resps = call_futures.collect::>(); - let mut set_counters = vec![(0, 0); write_sets.len()]; - let mut successes = 0; - let mut errors = vec![]; while let Some((node, resp)) = resps.next().await { - match resp { - Ok(_) => { - successes += 1; - for set in write_set_index.get(&node).unwrap().iter() { - set_counters[*set].0 += 1; - } - } - Err(e) => { - errors.push(e); - for set in write_set_index.get(&node).unwrap().iter() { - set_counters[*set].1 += 1; - } - } - } + result_tracker.register_result(node, resp.map(|_| ())); - if set_counters.iter().all(|(ok_cnt, _)| *ok_cnt >= quorum) { + if result_tracker.all_quorums_ok() { // Success // Continue all other requests in background @@ -285,25 +264,14 @@ impl Table { return Ok(()); } - if set_counters - .iter() - .enumerate() - .any(|(i, (_, err_cnt))| err_cnt + quorum > write_sets[i].len()) - { + if result_tracker.too_many_failures() { // Too many errors in this set, we know we won't get a quorum break; } } // Failure, could not get quorum within at least one set - let errors = errors.iter().map(|e| format!("{}", e)).collect::>(); - Err(Error::Quorum( - quorum, - Some(write_sets.len()), - successes, - call_count, - errors, - )) + Err(result_tracker.quorum_error()) } pub async fn get( -- cgit v1.2.3 From d90de365b3b30cb631b22fcd62c98bddb5a91549 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 7 Dec 2023 11:16:10 +0100 Subject: table sync: use write quorums to report global success or failure of sync --- src/rpc/layout/helper.rs | 2 +- src/rpc/layout/manager.rs | 2 +- src/table/replication/fullcopy.rs | 3 +-- src/table/replication/parameters.rs | 2 +- src/table/replication/sharded.rs | 4 +-- src/table/sync.rs | 51 ++++++++++++++++++++++--------------- 6 files changed, 36 insertions(+), 28 deletions(-) diff --git a/src/rpc/layout/helper.rs b/src/rpc/layout/helper.rs index 881a039e..0aa7c6aa 100644 --- a/src/rpc/layout/helper.rs +++ b/src/rpc/layout/helper.rs @@ -180,7 +180,7 @@ impl LayoutHelper { ret } - pub(crate) fn write_sets_of(&self, position: &Hash) -> Vec> { + pub fn storage_sets_of(&self, position: &Hash) -> Vec> { self.layout() .versions .iter() diff --git a/src/rpc/layout/manager.rs b/src/rpc/layout/manager.rs index 17465019..dc963ba0 100644 --- a/src/rpc/layout/manager.rs +++ b/src/rpc/layout/manager.rs @@ -139,7 +139,7 @@ impl LayoutManager { pub fn write_sets_of(self: &Arc, position: &Hash) -> WriteLock>> { let layout = self.layout(); let version = layout.current().version; - let nodes = layout.write_sets_of(position); + let nodes = layout.storage_sets_of(position); layout .ack_lock .get(&version) diff --git a/src/table/replication/fullcopy.rs b/src/table/replication/fullcopy.rs index df930224..30122f39 100644 --- a/src/table/replication/fullcopy.rs +++ b/src/table/replication/fullcopy.rs @@ -1,4 +1,3 @@ -use std::iter::FromIterator; use std::sync::Arc; use garage_rpc::layout::*; @@ -69,7 +68,7 @@ impl TableReplication for TableFullReplication { partition: 0u16, first_hash: [0u8; 32].into(), last_hash: [0xff; 32].into(), - storage_nodes: Vec::from_iter(layout.current().all_nodes().to_vec()), + storage_sets: vec![layout.current().all_nodes().to_vec()], }], } } diff --git a/src/table/replication/parameters.rs b/src/table/replication/parameters.rs index db11ff5f..78470f35 100644 --- a/src/table/replication/parameters.rs +++ b/src/table/replication/parameters.rs @@ -40,5 +40,5 @@ pub struct SyncPartition { pub partition: Partition, pub first_hash: Hash, pub last_hash: Hash, - pub storage_nodes: Vec, + pub storage_sets: Vec>, } diff --git a/src/table/replication/sharded.rs b/src/table/replication/sharded.rs index 2a16bc0c..55d0029d 100644 --- a/src/table/replication/sharded.rs +++ b/src/table/replication/sharded.rs @@ -60,12 +60,12 @@ impl TableReplication for TableShardedReplication { .current() .partitions() .map(|(partition, first_hash)| { - let storage_nodes = layout.storage_nodes_of(&first_hash); + let storage_sets = layout.storage_sets_of(&first_hash); SyncPartition { partition, first_hash, last_hash: [0u8; 32].into(), // filled in just after - storage_nodes, + storage_sets, } }) .collect::>(); diff --git a/src/table/sync.rs b/src/table/sync.rs index efeac402..cfcbc4b5 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -18,6 +18,7 @@ use garage_util::encode::{debug_serialize, nonversioned_encode}; use garage_util::error::{Error, OkOrMessage}; use garage_rpc::layout::*; +use garage_rpc::rpc_helper::QuorumSetResultTracker; use garage_rpc::system::System; use garage_rpc::*; @@ -106,44 +107,52 @@ impl TableSyncer { must_exit: &mut watch::Receiver, ) -> Result<(), Error> { let my_id = self.system.id; - let retain = partition.storage_nodes.contains(&my_id); + let retain = partition.storage_sets.iter().any(|x| x.contains(&my_id)); if retain { debug!( "({}) Syncing {:?} with {:?}...", F::TABLE_NAME, partition, - partition.storage_nodes + partition.storage_sets ); - let mut sync_futures = partition - .storage_nodes + let mut result_tracker = QuorumSetResultTracker::new( + &partition.storage_sets, + self.data.replication.write_quorum(), + ); + + let mut sync_futures = result_tracker + .nodes .iter() - .filter(|node| **node != my_id) + .map(|(node, _)| *node) .map(|node| { - self.clone() - .do_sync_with(&partition, *node, must_exit.clone()) + let must_exit = must_exit.clone(); + async move { + if node == my_id { + (node, Ok(())) + } else { + (node, self.do_sync_with(&partition, node, must_exit).await) + } + } }) .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: {}", F::TABLE_NAME, e); + while let Some((node, res)) = sync_futures.next().await { + if let Err(e) = &res { + warn!("({}) Sync error with {:?}: {}", F::TABLE_NAME, node, e); } + result_tracker.register_result(node, res); } - if n_errors > 0 { - return Err(Error::Message(format!( - "Sync failed with {} nodes.", - n_errors - ))); + + if result_tracker.too_many_failures() { + return Err(result_tracker.quorum_error()); + } else { + Ok(()) } } else { self.offload_partition(&partition.first_hash, &partition.last_hash, must_exit) - .await?; + .await } - - Ok(()) } // Offload partition: this partition is not something we are storing, @@ -264,7 +273,7 @@ impl TableSyncer { } async fn do_sync_with( - self: Arc, + self: &Arc, partition: &SyncPartition, who: Uuid, must_exit: watch::Receiver, -- cgit v1.2.3 From aa59059a910eb6e1e824b84413a66909d697ef8a Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 7 Dec 2023 11:50:00 +0100 Subject: layout cli: safer skip-dead-nodes command --- src/garage/cli/cmd.rs | 23 ++++++++++++++++------- src/garage/cli/layout.rs | 35 +++++++++++++++++++++++++---------- src/garage/cli/structs.rs | 12 ++++++++---- 3 files changed, 49 insertions(+), 21 deletions(-) diff --git a/src/garage/cli/cmd.rs b/src/garage/cli/cmd.rs index c7f0ad2b..196c0cb3 100644 --- a/src/garage/cli/cmd.rs +++ b/src/garage/cli/cmd.rs @@ -49,13 +49,7 @@ pub async fn cli_command_dispatch( } pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> Result<(), Error> { - let status = match rpc_cli - .call(&rpc_host, SystemRpc::GetKnownNodes, PRIO_NORMAL) - .await?? - { - SystemRpc::ReturnKnownNodes(nodes) => nodes, - resp => return Err(Error::Message(format!("Invalid RPC response: {:?}", resp))), - }; + let status = fetch_status(rpc_cli, rpc_host).await?; let layout = fetch_layout(rpc_cli, rpc_host).await?; println!("==== HEALTHY NODES ===="); @@ -268,3 +262,18 @@ pub async fn cmd_admin( } Ok(()) } + +// ---- utility ---- + +pub async fn fetch_status( + rpc_cli: &Endpoint, + rpc_host: NodeID, +) -> Result, Error> { + match rpc_cli + .call(&rpc_host, SystemRpc::GetKnownNodes, PRIO_NORMAL) + .await?? + { + SystemRpc::ReturnKnownNodes(nodes) => Ok(nodes), + resp => Err(Error::Message(format!("Invalid RPC response: {:?}", resp))), + } +} diff --git a/src/garage/cli/layout.rs b/src/garage/cli/layout.rs index 3c7843bd..cdf77c04 100644 --- a/src/garage/cli/layout.rs +++ b/src/garage/cli/layout.rs @@ -33,8 +33,8 @@ pub async fn cli_layout_command_dispatch( cmd_config_layout(system_rpc_endpoint, rpc_host, config_opt).await } LayoutOperation::History => cmd_layout_history(system_rpc_endpoint, rpc_host).await, - LayoutOperation::AssumeSync(assume_sync_opt) => { - cmd_layout_assume_sync(system_rpc_endpoint, rpc_host, assume_sync_opt).await + LayoutOperation::SkipDeadNodes(assume_sync_opt) => { + cmd_layout_skip_dead_nodes(system_rpc_endpoint, rpc_host, assume_sync_opt).await } } } @@ -388,13 +388,21 @@ pub async fn cmd_layout_history( Ok(()) } -pub async fn cmd_layout_assume_sync( +pub async fn cmd_layout_skip_dead_nodes( rpc_cli: &Endpoint, rpc_host: NodeID, - opt: AssumeSyncOpt, + opt: SkipDeadNodesOpt, ) -> Result<(), Error> { + let status = fetch_status(rpc_cli, rpc_host).await?; let mut layout = fetch_layout(rpc_cli, rpc_host).await?; + if layout.versions.len() == 1 { + return Err(Error::Message( + "This command cannot be called when there is only one live cluster layout version" + .into(), + )); + } + let min_v = layout.min_stored(); if opt.version <= min_v || opt.version > layout.current().version { return Err(Error::Message(format!( @@ -408,12 +416,19 @@ pub async fn cmd_layout_assume_sync( let all_nodes = layout.get_all_nodes(); for node in all_nodes.iter() { - layout.update_trackers.ack_map.set_max(*node, opt.version); - layout.update_trackers.sync_map.set_max(*node, opt.version); - layout - .update_trackers - .sync_ack_map - .set_max(*node, opt.version); + if status.iter().any(|x| x.id == *node && x.is_up) { + continue; + } + + if layout.update_trackers.ack_map.set_max(*node, opt.version) { + println!("Increased the ACK tracker for node {:?}", node); + } + + if opt.allow_missing_data { + if layout.update_trackers.sync_map.set_max(*node, opt.version) { + println!("Increased the SYNC tracker for node {:?}", node); + } + } } send_layout(rpc_cli, rpc_host, layout).await?; diff --git a/src/garage/cli/structs.rs b/src/garage/cli/structs.rs index c4b400f4..6bc3da22 100644 --- a/src/garage/cli/structs.rs +++ b/src/garage/cli/structs.rs @@ -117,9 +117,9 @@ pub enum LayoutOperation { #[structopt(name = "history", version = garage_version())] History, - /// Assume all nodes are synchronized up to a certain layout version - #[structopt(name = "assume-sync", version = garage_version())] - AssumeSync(AssumeSyncOpt), + /// Skip dead nodes when awaiting for a new layout version to be synchronized + #[structopt(name = "skip-dead-nodes", version = garage_version())] + SkipDeadNodes(SkipDeadNodesOpt), } #[derive(StructOpt, Debug)] @@ -178,11 +178,15 @@ pub struct RevertLayoutOpt { } #[derive(StructOpt, Debug)] -pub struct AssumeSyncOpt { +pub struct SkipDeadNodesOpt { /// Version number of the layout to assume is currently up-to-date. /// This will generally be the current layout version. #[structopt(long = "version")] pub(crate) version: u64, + /// Allow the skip even if a quorum of ndoes could not be found for + /// the data among the remaining nodes + #[structopt(long = "allow-missing-data")] + pub(crate) allow_missing_data: bool, } #[derive(Serialize, Deserialize, StructOpt, Debug)] -- cgit v1.2.3 From 9cecea64d4509e95ac9793b29c947e2ecf9bb0b8 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 7 Dec 2023 14:27:53 +0100 Subject: layout: allow sync update tracker to progress with only quorums --- src/garage/cli/layout.rs | 6 +-- src/rpc/layout/helper.rs | 33 ++++++++++++--- src/rpc/layout/history.rs | 101 ++++++++++++++++++++++++++++++++++++++++++++ src/rpc/layout/manager.rs | 18 +++++--- src/rpc/layout/schema.rs | 6 +-- src/rpc/replication_mode.rs | 7 +++ src/rpc/system.rs | 2 +- 7 files changed, 152 insertions(+), 21 deletions(-) diff --git a/src/garage/cli/layout.rs b/src/garage/cli/layout.rs index cdf77c04..fac826f5 100644 --- a/src/garage/cli/layout.rs +++ b/src/garage/cli/layout.rs @@ -365,9 +365,9 @@ pub async fn cmd_layout_history( table.push(format!( "{:?}\t#{}\t#{}\t#{}", node, - layout.update_trackers.ack_map.get(node), - layout.update_trackers.sync_map.get(node), - layout.update_trackers.sync_ack_map.get(node), + layout.update_trackers.ack_map.get(node, min_stored), + layout.update_trackers.sync_map.get(node, min_stored), + layout.update_trackers.sync_ack_map.get(node, min_stored), )); } table[1..].sort(); diff --git a/src/rpc/layout/helper.rs b/src/rpc/layout/helper.rs index 0aa7c6aa..eeaf4ffa 100644 --- a/src/rpc/layout/helper.rs +++ b/src/rpc/layout/helper.rs @@ -7,6 +7,7 @@ use serde::{Deserialize, Serialize}; use garage_util::data::*; use super::schema::*; +use crate::replication_mode::ReplicationMode; use crate::rpc_helper::RpcHelper; #[derive(Debug, Clone, Serialize, Deserialize, Default, PartialEq, Eq)] @@ -22,6 +23,7 @@ pub struct LayoutDigest { } pub struct LayoutHelper { + replication_mode: ReplicationMode, layout: Option, // cached values @@ -48,7 +50,23 @@ impl Deref for LayoutHelper { } impl LayoutHelper { - pub fn new(mut layout: LayoutHistory, mut ack_lock: HashMap) -> Self { + pub fn new( + replication_mode: ReplicationMode, + mut layout: LayoutHistory, + mut ack_lock: HashMap, + ) -> Self { + // In the new() function of the helper, we do a bunch of cleanup + // and calculations on the layout history to make sure things are + // correct and we have rapid access to important values such as + // the layout versions to use when reading to ensure consistency. + + if !replication_mode.is_read_after_write_consistent() { + // Fast path for when no consistency is required. + // In this case we only need to keep the last version of the layout, + // we don't care about coordinating stuff in the cluster. + layout.keep_current_version_only(); + } + layout.cleanup_old_versions(); let all_nodes = layout.get_all_nodes(); @@ -68,7 +86,7 @@ impl LayoutHelper { .ack_map .min_among(&all_nodes, min_version); - // sync_map_min is the minimum value of sync_map among all storage nodes + // sync_map_min is the minimum value of sync_map among storage nodes // in the cluster (non-gateway nodes only, current and previous layouts). // It is the highest layout version for which we know that all relevant // storage nodes have fullfilled a sync, and therefore it is safe to @@ -76,11 +94,10 @@ impl LayoutHelper { // Gateway nodes are excluded here because they hold no relevant data // (they store the bucket and access key tables, but we don't have // consistency on those). - // TODO: this value could take quorums into account instead. - let sync_map_min = layout - .update_trackers - .sync_map - .min_among(&all_nongateway_nodes, min_version); + // This value is calculated using quorums to allow progress even + // if not all nodes have successfully completed a sync. + let sync_map_min = + layout.calculate_sync_map_min_with_quorum(replication_mode, &all_nongateway_nodes); let trackers_hash = layout.calculate_trackers_hash(); let staging_hash = layout.calculate_staging_hash(); @@ -91,6 +108,7 @@ impl LayoutHelper { .or_insert(AtomicUsize::new(0)); LayoutHelper { + replication_mode, layout: Some(layout), ack_map_min, sync_map_min, @@ -115,6 +133,7 @@ impl LayoutHelper { let changed = f(&mut self.layout.as_mut().unwrap()); if changed { *self = Self::new( + self.replication_mode, self.layout.take().unwrap(), std::mem::take(&mut self.ack_lock), ); diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index c448ac24..a53256cc 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -6,6 +6,7 @@ use garage_util::encode::nonversioned_encode; use garage_util::error::*; use super::*; +use crate::replication_mode::ReplicationMode; impl LayoutHistory { pub fn new(replication_factor: usize) -> Self { @@ -64,6 +65,13 @@ impl LayoutHistory { // ---- housekeeping (all invoked by LayoutHelper) ---- + pub(crate) fn keep_current_version_only(&mut self) { + while self.versions.len() > 1 { + let removed = self.versions.remove(0); + self.old_versions.push(removed); + } + } + pub(crate) fn cleanup_old_versions(&mut self) { // If there are invalid versions before valid versions, remove them if self.versions.len() > 1 && self.current().check().is_ok() { @@ -114,6 +122,99 @@ impl LayoutHistory { } } + pub(crate) fn calculate_sync_map_min_with_quorum( + &self, + replication_mode: ReplicationMode, + all_nongateway_nodes: &[Uuid], + ) -> u64 { + // This function calculates the minimum layout version from which + // it is safe to read if we want to maintain read-after-write consistency. + // In the general case the computation can be a bit expensive so + // we try to optimize it in several ways. + + // If there is only one layout version, we know that's the one + // we need to read from. + if self.versions.len() == 1 { + return self.current().version; + } + + let quorum = replication_mode.write_quorum(); + + let min_version = self.min_stored(); + let global_min = self + .update_trackers + .sync_map + .min_among(&all_nongateway_nodes, min_version); + + // If the write quorums are equal to the total number of nodes, + // i.e. no writes can succeed while they are not written to all nodes, + // then we must in all case wait for all nodes to complete a sync. + // This is represented by reading from the layout with version + // number global_min, the smallest layout version for which all nodes + // have completed a sync. + if quorum == self.current().replication_factor { + return global_min; + } + + // In the general case, we need to look at all write sets for all partitions, + // and find a safe layout version to read for that partition. We then + // take the minimum value among all partition as the safe layout version + // to read in all cases (the layout version to which all reads are directed). + let mut current_min = self.current().version; + let mut sets_done = HashSet::>::new(); + + for (_, p_hash) in self.current().partitions() { + for v in self.versions.iter() { + if v.version == self.current().version { + // We don't care about whether nodes in the latest layout version + // have completed a sync or not, as the sync is push-only + // and by definition nodes in the latest layout version do not + // hold data that must be pushed to nodes in the latest layout + // version, since that's the same version (any data that's + // already in the latest version is assumed to have been written + // by an operation that ensured a quorum of writes within + // that version). + continue; + } + + // Determine set of nodes for partition p in layout version v. + // Sort the node set to avoid duplicate computations. + let mut set = v + .nodes_of(&p_hash, v.replication_factor) + .collect::>(); + set.sort(); + + // If this set was already processed, skip it. + if sets_done.contains(&set) { + continue; + } + + // Find the value of the sync update trackers that is the + // highest possible minimum within a quorum of nodes. + let mut sync_values = set + .iter() + .map(|x| self.update_trackers.sync_map.get(x, min_version)) + .collect::>(); + sync_values.sort(); + let set_min = sync_values[sync_values.len() - quorum]; + if set_min < current_min { + current_min = set_min; + } + // defavorable case, we know we are at the smallest possible version, + // so we can stop early + assert!(current_min >= global_min); + if current_min == global_min { + return current_min; + } + + // Add set to already processed sets + sets_done.insert(set); + } + } + + current_min + } + pub(crate) fn calculate_trackers_hash(&self) -> Hash { blake2sum(&nonversioned_encode(&self.update_trackers).unwrap()[..]) } diff --git a/src/rpc/layout/manager.rs b/src/rpc/layout/manager.rs index dc963ba0..ec8a2a15 100644 --- a/src/rpc/layout/manager.rs +++ b/src/rpc/layout/manager.rs @@ -14,12 +14,13 @@ use garage_util::error::*; use garage_util::persister::Persister; use super::*; +use crate::replication_mode::ReplicationMode; use crate::rpc_helper::*; use crate::system::*; pub struct LayoutManager { node_id: Uuid, - replication_factor: usize, + replication_mode: ReplicationMode, persist_cluster_layout: Persister, layout: Arc>, @@ -37,14 +38,16 @@ impl LayoutManager { node_id: NodeID, system_endpoint: Arc>, fullmesh: Arc, - replication_factor: usize, + replication_mode: ReplicationMode, ) -> Result, Error> { + let replication_factor = replication_mode.replication_factor(); + let persist_cluster_layout: Persister = Persister::new(&config.metadata_dir, "cluster_layout"); let cluster_layout = match persist_cluster_layout.load() { Ok(x) => { - if x.current().replication_factor != replication_factor { + if x.current().replication_factor != replication_mode.replication_factor() { return Err(Error::Message(format!( "Prevous cluster layout has replication factor {}, which is different than the one specified in the config file ({}). The previous cluster layout can be purged, if you know what you are doing, simply by deleting the `cluster_layout` file in your metadata directory.", x.current().replication_factor, @@ -62,7 +65,8 @@ impl LayoutManager { } }; - let mut cluster_layout = LayoutHelper::new(cluster_layout, Default::default()); + let mut cluster_layout = + LayoutHelper::new(replication_mode, cluster_layout, Default::default()); cluster_layout.update_trackers(node_id.into()); let layout = Arc::new(RwLock::new(cluster_layout)); @@ -77,7 +81,7 @@ impl LayoutManager { Ok(Arc::new(Self { node_id: node_id.into(), - replication_factor, + replication_mode, persist_cluster_layout, layout, change_notify, @@ -291,11 +295,11 @@ impl LayoutManager { adv.update_trackers ); - if adv.current().replication_factor != self.replication_factor { + if adv.current().replication_factor != self.replication_mode.replication_factor() { let msg = format!( "Received a cluster layout from another node with replication factor {}, which is different from what we have in our configuration ({}). Discarding the cluster layout we received.", adv.current().replication_factor, - self.replication_factor + self.replication_mode.replication_factor() ); error!("{}", msg); return Err(Error::Message(msg)); diff --git a/src/rpc/layout/schema.rs b/src/rpc/layout/schema.rs index 49e84420..df949906 100644 --- a/src/rpc/layout/schema.rs +++ b/src/rpc/layout/schema.rs @@ -411,13 +411,13 @@ impl UpdateTracker { pub(crate) fn min_among(&self, storage_nodes: &[Uuid], min_version: u64) -> u64 { storage_nodes .iter() - .map(|x| self.0.get(x).copied().unwrap_or(min_version)) + .map(|x| self.get(x, min_version)) .min() .unwrap_or(min_version) } - pub fn get(&self, node: &Uuid) -> u64 { - self.0.get(node).copied().unwrap_or(0) + pub fn get(&self, node: &Uuid, min_version: u64) -> u64 { + self.0.get(node).copied().unwrap_or(min_version) } } diff --git a/src/rpc/replication_mode.rs b/src/rpc/replication_mode.rs index e244e063..2f7e2fec 100644 --- a/src/rpc/replication_mode.rs +++ b/src/rpc/replication_mode.rs @@ -54,4 +54,11 @@ impl ReplicationMode { Self::ThreeWayDangerous => 1, } } + + pub fn is_read_after_write_consistent(&self) -> bool { + match self { + Self::None | Self::TwoWay | Self::ThreeWay => true, + _ => false, + } + } } diff --git a/src/rpc/system.rs b/src/rpc/system.rs index be4aefa2..81a47ff3 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -280,7 +280,7 @@ impl System { netapp.id, system_endpoint.clone(), fullmesh.clone(), - replication_factor, + replication_mode, )?; // ---- set up metrics and status exchange ---- -- cgit v1.2.3 From 431b28e0cfdc9cac6c649193cf602108a8b02997 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Thu, 7 Dec 2023 15:15:59 +0100 Subject: fix build with discovery features --- src/rpc/system.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/rpc/system.rs b/src/rpc/system.rs index 81a47ff3..adfef6b6 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -514,7 +514,7 @@ impl System { if let Err(e) = c .publish_consul_service( self.netapp.id, - &self.local_status.load_full().hostname, + &self.local_status.load_full().hostname.as_deref().unwrap(), rpc_public_addr, ) .await @@ -541,7 +541,7 @@ impl System { if let Err(e) = publish_kubernetes_node( k, self.netapp.id, - &self.local_status.load_full().hostname, + &self.local_status.load_full().hostname.as_deref().unwrap(), rpc_public_addr, ) .await -- cgit v1.2.3 From 91b874c4efa40e64663368369a712e0a5a389e53 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 8 Dec 2023 10:36:37 +0100 Subject: rpc: fix system::health --- src/rpc/system.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/rpc/system.rs b/src/rpc/system.rs index adfef6b6..a8f12852 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -456,14 +456,14 @@ impl System { let mut partitions_quorum = 0; let mut partitions_all_ok = 0; for (_, hash) in partitions.iter() { - let write_sets = layout + let mut write_sets = layout .versions .iter() .map(|x| x.nodes_of(hash, x.replication_factor)); let has_quorum = write_sets .clone() .all(|set| set.filter(|x| node_up(x)).count() >= quorum); - let all_ok = write_sets.clone().all(|mut set| set.all(|x| node_up(&x))); + let all_ok = write_sets.all(|mut set| set.all(|x| node_up(&x))); if has_quorum { partitions_quorum += 1; } @@ -474,7 +474,7 @@ impl System { // Determine overall cluster status let status = - if partitions_quorum == partitions.len() && storage_nodes_ok == storage_nodes.len() { + if partitions_all_ok == partitions.len() && storage_nodes_ok == storage_nodes.len() { ClusterHealthStatus::Healthy } else if partitions_quorum == partitions.len() { ClusterHealthStatus::Degraded -- cgit v1.2.3 From 7f2541101f15614c79020b35d3d7dab767c32676 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 8 Dec 2023 11:24:23 +0100 Subject: cli: improvements to the layout commands when multiple layouts are live --- src/garage/admin/mod.rs | 3 +-- src/garage/cli/cmd.rs | 4 +-- src/garage/cli/layout.rs | 67 +++++++++++++++++++++++++++++++----------------- src/garage/cli/util.rs | 4 ++- 4 files changed, 49 insertions(+), 29 deletions(-) diff --git a/src/garage/admin/mod.rs b/src/garage/admin/mod.rs index da4226cf..de7851e1 100644 --- a/src/garage/admin/mod.rs +++ b/src/garage/admin/mod.rs @@ -274,8 +274,7 @@ impl AdminRpcHandler { fn gather_cluster_stats(&self) -> String { let mut ret = String::new(); - // Gather storage node and free space statistics - // TODO: not only layout.current() ??? + // Gather storage node and free space statistics for current nodes let layout = &self.garage.system.cluster_layout(); let mut node_partition_count = HashMap::::new(); for short_id in layout.current().ring_assignment_data.iter() { diff --git a/src/garage/cli/cmd.rs b/src/garage/cli/cmd.rs index 196c0cb3..fb6dface 100644 --- a/src/garage/cli/cmd.rs +++ b/src/garage/cli/cmd.rs @@ -179,7 +179,7 @@ pub async fn cmd_status(rpc_cli: &Endpoint, rpc_host: NodeID) -> println!("Your cluster is expecting to drain data from nodes that are currently unavailable."); println!("If these nodes are definitely dead, please review the layout history with"); println!( - "`garage layout history` and use `garage layout assume-sync` to force progress." + "`garage layout history` and use `garage layout skip-dead-nodes` to force progress." ); } } @@ -274,6 +274,6 @@ pub async fn fetch_status( .await?? { SystemRpc::ReturnKnownNodes(nodes) => Ok(nodes), - resp => Err(Error::Message(format!("Invalid RPC response: {:?}", resp))), + resp => Err(Error::unexpected_rpc_message(resp)), } } diff --git a/src/garage/cli/layout.rs b/src/garage/cli/layout.rs index fac826f5..f76e33c5 100644 --- a/src/garage/cli/layout.rs +++ b/src/garage/cli/layout.rs @@ -354,35 +354,44 @@ pub async fn cmd_layout_history( )); } format_table(table); - - println!(); - println!("==== UPDATE TRACKERS ===="); - println!("This is the internal data that Garage stores to know which nodes have what data."); println!(); - let mut table = vec!["Node\tAck\tSync\tSync_ack".to_string()]; - let all_nodes = layout.get_all_nodes(); - for node in all_nodes.iter() { - table.push(format!( - "{:?}\t#{}\t#{}\t#{}", - node, - layout.update_trackers.ack_map.get(node, min_stored), - layout.update_trackers.sync_map.get(node, min_stored), - layout.update_trackers.sync_ack_map.get(node, min_stored), - )); - } - table[1..].sort(); - format_table(table); if layout.versions.len() > 1 { + println!("==== UPDATE TRACKERS ===="); + println!("Several layout versions are currently live in the version, and data is being migrated."); + println!( + "This is the internal data that Garage stores to know which nodes have what data." + ); + println!(); + let mut table = vec!["Node\tAck\tSync\tSync_ack".to_string()]; + let all_nodes = layout.get_all_nodes(); + for node in all_nodes.iter() { + table.push(format!( + "{:?}\t#{}\t#{}\t#{}", + node, + layout.update_trackers.ack_map.get(node, min_stored), + layout.update_trackers.sync_map.get(node, min_stored), + layout.update_trackers.sync_ack_map.get(node, min_stored), + )); + } + table[1..].sort(); + format_table(table); + println!(); println!( - "If some nodes are not catching up to the latest layout version in the update tracker," + "If some nodes are not catching up to the latest layout version in the update trackers," ); println!("it might be because they are offline or unable to complete a sync successfully."); println!( - "You may force progress using `garage layout assume-sync --version {}`", + "You may force progress using `garage layout skip-dead-nodes --version {}`", layout.current().version ); + } else { + println!("Your cluster is currently in a stable state with a single live layout version."); + println!("No metadata migration is in progress. Note that the migration of data blocks is not tracked,"); + println!( + "so you might want to keep old nodes online until their data directories become empty." + ); } Ok(()) @@ -415,6 +424,7 @@ pub async fn cmd_layout_skip_dead_nodes( } let all_nodes = layout.get_all_nodes(); + let mut did_something = false; for node in all_nodes.iter() { if status.iter().any(|x| x.id == *node && x.is_up) { continue; @@ -422,19 +432,28 @@ pub async fn cmd_layout_skip_dead_nodes( if layout.update_trackers.ack_map.set_max(*node, opt.version) { println!("Increased the ACK tracker for node {:?}", node); + did_something = true; } if opt.allow_missing_data { if layout.update_trackers.sync_map.set_max(*node, opt.version) { println!("Increased the SYNC tracker for node {:?}", node); + did_something = true; } } } - send_layout(rpc_cli, rpc_host, layout).await?; - println!("Success."); - - Ok(()) + if did_something { + send_layout(rpc_cli, rpc_host, layout).await?; + println!("Success."); + Ok(()) + } else if !opt.allow_missing_data { + Err(Error::Message("Nothing was done, try passing the `--allow-missing-data` flag to force progress even when not enough nodes can complete a metadata sync.".into())) + } else { + Err(Error::Message( + "Sorry, there is nothing I can do for you. Please wait patiently. If you ask for help, please send the output of the `garage layout history` command.".into(), + )) + } } // --- utility --- @@ -448,7 +467,7 @@ pub async fn fetch_layout( .await?? { SystemRpc::AdvertiseClusterLayout(t) => Ok(t), - resp => Err(Error::Message(format!("Invalid RPC response: {:?}", resp))), + resp => Err(Error::unexpected_rpc_message(resp)), } } diff --git a/src/garage/cli/util.rs b/src/garage/cli/util.rs index 2232d395..0511e2b1 100644 --- a/src/garage/cli/util.rs +++ b/src/garage/cli/util.rs @@ -450,6 +450,8 @@ pub fn print_block_info( if refcount != nondeleted_count { println!(); - println!("Warning: refcount does not match number of non-deleted versions"); + println!( + "Warning: refcount does not match number of non-deleted versions (see issue #644)." + ); } } -- cgit v1.2.3 From 063294dd569e10c6d85e29eb6507249eece00956 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 8 Dec 2023 11:50:58 +0100 Subject: layout version: refactor get_node_zone --- src/rpc/layout/test.rs | 4 ++-- src/rpc/layout/version.rs | 47 +++++++++++++++++++++++------------------------ 2 files changed, 25 insertions(+), 26 deletions(-) diff --git a/src/rpc/layout/test.rs b/src/rpc/layout/test.rs index bb072c97..88eb518e 100644 --- a/src/rpc/layout/test.rs +++ b/src/rpc/layout/test.rs @@ -34,8 +34,8 @@ fn check_against_naive(cl: &LayoutVersion) -> Result { zone_token.insert(z.clone(), 0); } for uuid in cl.nongateway_nodes() { - let z = cl.get_node_zone(&uuid)?; - let c = cl.get_node_capacity(&uuid).unwrap(); + let z = cl.expect_get_node_zone(&uuid); + let c = cl.expect_get_node_capacity(&uuid); zone_token.insert( z.to_string(), zone_token[z] + min(NB_PARTITIONS, (c / over_size) as usize), diff --git a/src/rpc/layout/version.rs b/src/rpc/layout/version.rs index 947fab56..cbfbee94 100644 --- a/src/rpc/layout/version.rs +++ b/src/rpc/layout/version.rs @@ -70,6 +70,14 @@ impl LayoutVersion { } } + /// Given a node uuids, this function returns the label of its zone if it has one + pub fn get_node_zone(&self, uuid: &Uuid) -> Option<&str> { + match self.node_role(uuid) { + Some(role) => Some(&role.zone), + _ => None, + } + } + /// Returns the number of partitions associated to this node in the ring pub fn get_node_usage(&self, uuid: &Uuid) -> Result { for (i, id) in self.node_id_vec.iter().enumerate() { @@ -129,28 +137,22 @@ impl LayoutVersion { // ===================== internal information extractors ====================== - /// Given a node uuids, this function returns the label of its zone - pub(crate) fn get_node_zone(&self, uuid: &Uuid) -> Result<&str, Error> { - match self.node_role(uuid) { - Some(role) => Ok(&role.zone), - _ => Err(Error::Message( - "The Uuid does not correspond to a node present in the cluster.".into(), - )), - } - } - - fn expect_get_node_capacity(&self, uuid: &Uuid) -> u64 { + pub(crate) fn expect_get_node_capacity(&self, uuid: &Uuid) -> u64 { self.get_node_capacity(&uuid) .expect("non-gateway node with zero capacity") } + pub(crate) fn expect_get_node_zone(&self, uuid: &Uuid) -> &str { + self.get_node_zone(&uuid).expect("node without a zone") + } + /// Returns the sum of capacities of non gateway nodes in the cluster - fn get_total_capacity(&self) -> Result { + fn get_total_capacity(&self) -> u64 { let mut total_capacity = 0; for uuid in self.nongateway_nodes() { total_capacity += self.expect_get_node_capacity(&uuid); } - Ok(total_capacity) + total_capacity } /// Returns the effective value of the zone_redundancy parameter @@ -227,10 +229,7 @@ impl LayoutVersion { // Check that every partition is spread over at least zone_redundancy zones. let zones_of_p = nodes_of_p .iter() - .map(|n| { - self.get_node_zone(&self.node_id_vec[*n as usize]) - .expect("Zone not found.") - }) + .map(|n| self.expect_get_node_zone(&self.node_id_vec[*n as usize])) .collect::>(); if zones_of_p.iter().unique().count() < zone_redundancy { return Err(format!( @@ -516,7 +515,7 @@ impl LayoutVersion { } let mut s_down = 1; - let mut s_up = self.get_total_capacity()?; + let mut s_up = self.get_total_capacity(); while s_down + 1 < s_up { g = self.generate_flow_graph( (s_down + s_up) / 2, @@ -586,7 +585,7 @@ impl LayoutVersion { } for n in 0..self.nongateway_nodes().len() { let node_capacity = self.expect_get_node_capacity(&self.node_id_vec[n]); - let node_zone = zone_to_id[self.get_node_zone(&self.node_id_vec[n])?]; + let node_zone = zone_to_id[self.expect_get_node_zone(&self.node_id_vec[n])]; g.add_edge(Vertex::N(n), Vertex::Sink, node_capacity / partition_size)?; for p in 0..NB_PARTITIONS { if !exclude_assoc.contains(&(p, n)) { @@ -632,7 +631,7 @@ impl LayoutVersion { // The algorithm is such that it will start with the flow that we just computed // and find ameliorating paths from that. for (p, n) in exclude_edge.iter() { - let node_zone = zone_to_id[self.get_node_zone(&self.node_id_vec[*n])?]; + let node_zone = zone_to_id[self.expect_get_node_zone(&self.node_id_vec[*n])]; g.add_edge(Vertex::PZ(*p, node_zone), Vertex::N(*n), 1)?; } g.compute_maximal_flow()?; @@ -652,7 +651,7 @@ impl LayoutVersion { let mut cost = CostFunction::new(); for (p, assoc_p) in prev_assign.iter().enumerate() { for n in assoc_p.iter() { - let node_zone = zone_to_id[self.get_node_zone(&self.node_id_vec[*n])?]; + let node_zone = zone_to_id[self.expect_get_node_zone(&self.node_id_vec[*n])]; cost.insert((Vertex::PZ(p, node_zone), Vertex::N(*n)), -1); } } @@ -707,7 +706,7 @@ impl LayoutVersion { let mut msg = Message::new(); let used_cap = self.partition_size * NB_PARTITIONS as u64 * self.replication_factor as u64; - let total_cap = self.get_total_capacity()?; + let total_cap = self.get_total_capacity(); let percent_cap = 100.0 * (used_cap as f32) / (total_cap as f32); msg.push(format!( "Usable capacity / total cluster capacity: {} / {} ({:.1} %)", @@ -754,7 +753,7 @@ impl LayoutVersion { let mut old_zones_of_p = Vec::::new(); for n in prev_assign[p].iter() { old_zones_of_p - .push(zone_to_id[self.get_node_zone(&self.node_id_vec[*n])?]); + .push(zone_to_id[self.expect_get_node_zone(&self.node_id_vec[*n])]); } if !old_zones_of_p.contains(&z) { new_partitions_zone[z] += 1; @@ -796,7 +795,7 @@ impl LayoutVersion { for z in 0..id_to_zone.len() { let mut nodes_of_z = Vec::::new(); for n in 0..storing_nodes.len() { - if self.get_node_zone(&self.node_id_vec[n])? == id_to_zone[z] { + if self.expect_get_node_zone(&self.node_id_vec[n]) == id_to_zone[z] { nodes_of_z.push(n); } } -- cgit v1.2.3 From 5dd200c015aed786173f0e11541b0505f95dd6d1 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 8 Dec 2023 12:02:24 +0100 Subject: layout: move block_read_nodes_of to rpc_helper to avoid double-locking (in theory, this could have caused a deadlock) --- src/block/manager.rs | 2 +- src/rpc/layout/helper.rs | 27 ++--------- src/rpc/rpc_helper.rs | 121 +++++++++++++++++++++++++++++------------------ 3 files changed, 80 insertions(+), 70 deletions(-) diff --git a/src/block/manager.rs b/src/block/manager.rs index 47111160..bfd390ee 100644 --- a/src/block/manager.rs +++ b/src/block/manager.rs @@ -266,7 +266,7 @@ impl BlockManager { { let who = self .system - .cluster_layout() + .rpc_helper() .block_read_nodes_of(hash, self.system.rpc_helper()); for node in who.iter() { diff --git a/src/rpc/layout/helper.rs b/src/rpc/layout/helper.rs index eeaf4ffa..147c8b4f 100644 --- a/src/rpc/layout/helper.rs +++ b/src/rpc/layout/helper.rs @@ -8,7 +8,6 @@ use garage_util::data::*; use super::schema::*; use crate::replication_mode::ReplicationMode; -use crate::rpc_helper::RpcHelper; #[derive(Debug, Clone, Serialize, Deserialize, Default, PartialEq, Eq)] pub struct LayoutDigest { @@ -155,6 +154,10 @@ impl LayoutHelper { self.ack_map_min } + pub fn all_sync(&self) -> u64 { + self.sync_map_min + } + pub fn sync_versions(&self) -> (u64, u64, u64) { ( self.layout().current().version, @@ -177,28 +180,6 @@ impl LayoutHelper { .collect() } - pub fn block_read_nodes_of(&self, position: &Hash, rpc_helper: &RpcHelper) -> Vec { - let mut ret = Vec::with_capacity(12); - let ver_iter = self - .layout() - .versions - .iter() - .rev() - .chain(self.layout().old_versions.iter().rev()); - for ver in ver_iter { - if ver.version > self.sync_map_min { - continue; - } - let nodes = ver.nodes_of(position, ver.replication_factor); - for node in rpc_helper.request_order(nodes) { - if !ret.contains(&node) { - ret.push(node); - } - } - } - ret - } - pub fn storage_sets_of(&self, position: &Hash) -> Vec> { self.layout() .versions diff --git a/src/rpc/rpc_helper.rs b/src/rpc/rpc_helper.rs index c6dcbe75..7e1387ed 100644 --- a/src/rpc/rpc_helper.rs +++ b/src/rpc/rpc_helper.rs @@ -26,7 +26,7 @@ use garage_util::data::*; use garage_util::error::Error; use garage_util::metrics::RecordDuration; -use crate::layout::LayoutHelper; +use crate::layout::{LayoutHelper, LayoutHistory}; use crate::metrics::RpcMetrics; // Default RPC timeout = 5 minutes @@ -304,7 +304,7 @@ impl RpcHelper { // preemptively send an additional request to any remaining nodes. // Reorder requests to priorize closeness / low latency - let request_order = self.request_order(to.iter().copied()); + let request_order = self.request_order(&self.0.layout.read().unwrap(), to.iter().copied()); let send_all_at_once = strategy.rs_send_all_at_once.unwrap_or(false); // Build future for each request @@ -368,50 +368,6 @@ impl RpcHelper { } } - pub fn request_order(&self, nodes: impl Iterator) -> Vec { - // Retrieve some status variables that we will use to sort requests - let peer_list = self.0.fullmesh.get_peer_list(); - let layout = self.0.layout.read().unwrap(); - let our_zone = match layout.current().node_role(&self.0.our_node_id) { - Some(pc) => &pc.zone, - None => "", - }; - - // Augment requests with some information used to sort them. - // The tuples are as follows: - // (is another node?, is another zone?, latency, node ID, request future) - // We store all of these tuples in a vec that we can sort. - // By sorting this vec, we priorize ourself, then nodes in the same zone, - // and within a same zone we priorize nodes with the lowest latency. - let mut nodes = nodes - .map(|to| { - let peer_zone = match layout.current().node_role(&to) { - Some(pc) => &pc.zone, - None => "", - }; - let peer_avg_ping = peer_list - .iter() - .find(|x| x.id.as_ref() == to.as_slice()) - .and_then(|pi| pi.avg_ping) - .unwrap_or_else(|| Duration::from_secs(10)); - ( - to != self.0.our_node_id, - peer_zone != our_zone, - peer_avg_ping, - to, - ) - }) - .collect::>(); - - // Sort requests by (priorize ourself, priorize same zone, priorize low latency) - nodes.sort_by_key(|(diffnode, diffzone, ping, _to)| (*diffnode, *diffzone, *ping)); - - nodes - .into_iter() - .map(|(_, _, _, to)| to) - .collect::>() - } - /// Make a RPC call to multiple servers, returning either a Vec of responses, /// or an error if quorum could not be reached due to too many errors /// @@ -533,6 +489,79 @@ impl RpcHelper { // Failure, could not get quorum Err(result_tracker.quorum_error()) } + + // ---- functions not related to MAKING RPCs, but just determining to what nodes + // they should be made and in which order ---- + + pub fn block_read_nodes_of(&self, position: &Hash, rpc_helper: &RpcHelper) -> Vec { + let layout = self.0.layout.read().unwrap(); + + let mut ret = Vec::with_capacity(12); + let ver_iter = layout + .versions + .iter() + .rev() + .chain(layout.old_versions.iter().rev()); + for ver in ver_iter { + if ver.version > layout.all_sync() { + continue; + } + let nodes = ver.nodes_of(position, ver.replication_factor); + for node in rpc_helper.request_order(&layout, nodes) { + if !ret.contains(&node) { + ret.push(node); + } + } + } + ret + } + + fn request_order( + &self, + layout: &LayoutHistory, + nodes: impl Iterator, + ) -> Vec { + // Retrieve some status variables that we will use to sort requests + let peer_list = self.0.fullmesh.get_peer_list(); + let our_zone = match layout.current().node_role(&self.0.our_node_id) { + Some(pc) => &pc.zone, + None => "", + }; + + // Augment requests with some information used to sort them. + // The tuples are as follows: + // (is another node?, is another zone?, latency, node ID, request future) + // We store all of these tuples in a vec that we can sort. + // By sorting this vec, we priorize ourself, then nodes in the same zone, + // and within a same zone we priorize nodes with the lowest latency. + let mut nodes = nodes + .map(|to| { + let peer_zone = match layout.current().node_role(&to) { + Some(pc) => &pc.zone, + None => "", + }; + let peer_avg_ping = peer_list + .iter() + .find(|x| x.id.as_ref() == to.as_slice()) + .and_then(|pi| pi.avg_ping) + .unwrap_or_else(|| Duration::from_secs(10)); + ( + to != self.0.our_node_id, + peer_zone != our_zone, + peer_avg_ping, + to, + ) + }) + .collect::>(); + + // Sort requests by (priorize ourself, priorize same zone, priorize low latency) + nodes.sort_by_key(|(diffnode, diffzone, ping, _to)| (*diffnode, *diffzone, *ping)); + + nodes + .into_iter() + .map(|(_, _, _, to)| to) + .collect::>() + } } // ------- utility for tracking successes/errors among write sets -------- -- cgit v1.2.3 From 64a6e557a4ff6aa1ad833a1b25ef8c85cf9ee3f3 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 8 Dec 2023 12:18:12 +0100 Subject: rpc helper: small refactorings --- src/rpc/rpc_helper.rs | 31 +++++++++++++++++-------------- 1 file changed, 17 insertions(+), 14 deletions(-) diff --git a/src/rpc/rpc_helper.rs b/src/rpc/rpc_helper.rs index 7e1387ed..65af8901 100644 --- a/src/rpc/rpc_helper.rs +++ b/src/rpc/rpc_helper.rs @@ -436,13 +436,12 @@ impl RpcHelper { H: StreamingEndpointHandler + 'static, S: Send + 'static, { - let msg = msg.into_req().map_err(netapp::error::Error::from)?; - // Peers may appear in many quorum sets. Here, build a list of peers, // mapping to the index of the quorum sets in which they appear. let mut result_tracker = QuorumSetResultTracker::new(to_sets, quorum); // Send one request to each peer of the quorum sets + let msg = msg.into_req().map_err(netapp::error::Error::from)?; let requests = result_tracker.nodes.iter().map(|(peer, _)| { let self2 = self.clone(); let msg = msg.clone(); @@ -523,10 +522,10 @@ impl RpcHelper { ) -> Vec { // Retrieve some status variables that we will use to sort requests let peer_list = self.0.fullmesh.get_peer_list(); - let our_zone = match layout.current().node_role(&self.0.our_node_id) { - Some(pc) => &pc.zone, - None => "", - }; + let our_zone = layout + .current() + .get_node_zone(&self.0.our_node_id) + .unwrap_or(""); // Augment requests with some information used to sort them. // The tuples are as follows: @@ -536,10 +535,7 @@ impl RpcHelper { // and within a same zone we priorize nodes with the lowest latency. let mut nodes = nodes .map(|to| { - let peer_zone = match layout.current().node_role(&to) { - Some(pc) => &pc.zone, - None => "", - }; + let peer_zone = layout.current().get_node_zone(&to).unwrap_or(""); let peer_avg_ping = peer_list .iter() .find(|x| x.id.as_ref() == to.as_slice()) @@ -567,21 +563,28 @@ impl RpcHelper { // ------- utility for tracking successes/errors among write sets -------- pub struct QuorumSetResultTracker { - // The set of nodes and the quorum sets they belong to + /// The set of nodes and the index of the quorum sets they belong to pub nodes: HashMap>, + /// The quorum value, i.e. number of success responses to await in each set pub quorum: usize, - // The success and error responses received + /// The success responses received pub successes: Vec<(Uuid, S)>, + /// The error responses received pub failures: Vec<(Uuid, E)>, - // The counters for successes and failures in each set + /// The counters for successes in each set pub success_counters: Box<[usize]>, + /// The counters for failures in each set pub failure_counters: Box<[usize]>, + /// The total number of nodes in each set pub set_lens: Box<[usize]>, } -impl QuorumSetResultTracker { +impl QuorumSetResultTracker +where + E: std::fmt::Display, +{ pub fn new(sets: &[A], quorum: usize) -> Self where A: AsRef<[Uuid]>, -- cgit v1.2.3 From 4dbf254512327ef4e7abbd5525b89bfa5b7ecb6f Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 8 Dec 2023 14:15:52 +0100 Subject: layout: refactoring, merge two files --- src/rpc/layout/helper.rs | 2 +- src/rpc/layout/mod.rs | 441 +++++++++++++++++++++++++++++++++++++++++++++- src/rpc/layout/schema.rs | 431 -------------------------------------------- src/rpc/layout/version.rs | 1 - 4 files changed, 440 insertions(+), 435 deletions(-) delete mode 100644 src/rpc/layout/schema.rs diff --git a/src/rpc/layout/helper.rs b/src/rpc/layout/helper.rs index 147c8b4f..2ba010b8 100644 --- a/src/rpc/layout/helper.rs +++ b/src/rpc/layout/helper.rs @@ -6,7 +6,7 @@ use serde::{Deserialize, Serialize}; use garage_util::data::*; -use super::schema::*; +use super::*; use crate::replication_mode::ReplicationMode; #[derive(Debug, Clone, Serialize, Deserialize, Default, PartialEq, Eq)] diff --git a/src/rpc/layout/mod.rs b/src/rpc/layout/mod.rs index eb127fda..facdb2ce 100644 --- a/src/rpc/layout/mod.rs +++ b/src/rpc/layout/mod.rs @@ -1,7 +1,13 @@ +use std::fmt; + +use bytesize::ByteSize; + +use garage_util::crdt::{AutoCrdt, Crdt}; +use garage_util::data::Uuid; + mod graph_algo; mod helper; mod history; -mod schema; mod version; #[cfg(test)] @@ -13,7 +19,6 @@ pub mod manager; pub use helper::{LayoutDigest, LayoutHelper}; pub use manager::WriteLock; -pub use schema::*; pub use version::*; // ---- defines: partitions ---- @@ -39,3 +44,435 @@ const NB_PARTITIONS: usize = 1usize << PARTITION_BITS; // Change this to u16 the day we want to have more than 256 nodes in a cluster pub type CompactNodeType = u8; pub const MAX_NODE_NUMBER: usize = 256; + +// ======== actual data structures for the layout data ======== +// ======== that is persisted to disk ======== +// some small utility impls are at the end of this file, +// but most of the code that actually computes stuff is in +// version.rs, history.rs and helper.rs + +mod v08 { + use crate::layout::CompactNodeType; + use garage_util::crdt::LwwMap; + use garage_util::data::{Hash, Uuid}; + use serde::{Deserialize, Serialize}; + + /// The layout of the cluster, i.e. the list of roles + /// which are assigned to each cluster node + #[derive(Clone, Debug, Serialize, Deserialize)] + pub struct ClusterLayout { + pub version: u64, + + pub replication_factor: usize, + pub roles: LwwMap, + + // see comments in v010::ClusterLayout + pub node_id_vec: Vec, + #[serde(with = "serde_bytes")] + pub ring_assignation_data: Vec, + + /// Role changes which are staged for the next version of the layout + pub staging: LwwMap, + pub staging_hash: Hash, + } + + #[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Debug, Serialize, Deserialize)] + pub struct NodeRoleV(pub Option); + + /// The user-assigned roles of cluster nodes + #[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Debug, Serialize, Deserialize)] + pub struct NodeRole { + /// Datacenter at which this entry belong. This information is used to + /// perform a better geodistribution + pub zone: String, + /// The capacity of the node + /// If this is set to None, the node does not participate in storing data for the system + /// and is only active as an API gateway to other nodes + pub capacity: Option, + /// A set of tags to recognize the node + pub tags: Vec, + } + + impl garage_util::migrate::InitialFormat for ClusterLayout {} +} + +mod v09 { + use super::v08; + use crate::layout::CompactNodeType; + use garage_util::crdt::{Lww, LwwMap}; + use garage_util::data::{Hash, Uuid}; + use serde::{Deserialize, Serialize}; + pub use v08::{NodeRole, NodeRoleV}; + + /// The layout of the cluster, i.e. the list of roles + /// which are assigned to each cluster node + #[derive(Clone, Debug, Serialize, Deserialize)] + pub struct ClusterLayout { + pub version: u64, + + pub replication_factor: usize, + + /// This attribute is only used to retain the previously computed partition size, + /// to know to what extent does it change with the layout update. + pub partition_size: u64, + /// Parameters used to compute the assignment currently given by + /// ring_assignment_data + pub parameters: LayoutParameters, + + pub roles: LwwMap, + + // see comments in v010::ClusterLayout + pub node_id_vec: Vec, + #[serde(with = "serde_bytes")] + pub ring_assignment_data: Vec, + + /// Parameters to be used in the next partition assignment computation. + pub staging_parameters: Lww, + /// Role changes which are staged for the next version of the layout + pub staging_roles: LwwMap, + pub staging_hash: Hash, + } + + /// This struct is used to set the parameters to be used in the assignment computation + /// algorithm. It is stored as a Crdt. + #[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Copy, Debug, Serialize, Deserialize)] + pub struct LayoutParameters { + pub zone_redundancy: ZoneRedundancy, + } + + /// Zone redundancy: if set to AtLeast(x), the layout calculation will aim to store copies + /// of each partition on at least that number of different zones. + /// Otherwise, copies will be stored on the maximum possible number of zones. + #[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Copy, Debug, Serialize, Deserialize)] + pub enum ZoneRedundancy { + AtLeast(usize), + Maximum, + } + + impl garage_util::migrate::Migrate for ClusterLayout { + const VERSION_MARKER: &'static [u8] = b"G09layout"; + + type Previous = v08::ClusterLayout; + + fn migrate(previous: Self::Previous) -> Self { + use itertools::Itertools; + + // In the old layout, capacities are in an arbitrary unit, + // but in the new layout they are in bytes. + // Here we arbitrarily multiply everything by 1G, + // such that 1 old capacity unit = 1GB in the new units. + // This is totally arbitrary and won't work for most users. + let cap_mul = 1024 * 1024 * 1024; + let roles = multiply_all_capacities(previous.roles, cap_mul); + let staging_roles = multiply_all_capacities(previous.staging, cap_mul); + let node_id_vec = previous.node_id_vec; + + // Determine partition size + let mut tmp = previous.ring_assignation_data.clone(); + tmp.sort(); + let partition_size = tmp + .into_iter() + .dedup_with_count() + .map(|(npart, node)| { + roles + .get(&node_id_vec[node as usize]) + .and_then(|p| p.0.as_ref().and_then(|r| r.capacity)) + .unwrap_or(0) / npart as u64 + }) + .min() + .unwrap_or(0); + + // By default, zone_redundancy is maximum possible value + let parameters = LayoutParameters { + zone_redundancy: ZoneRedundancy::Maximum, + }; + + Self { + version: previous.version, + replication_factor: previous.replication_factor, + partition_size, + parameters, + roles, + node_id_vec, + ring_assignment_data: previous.ring_assignation_data, + staging_parameters: Lww::new(parameters), + staging_roles, + staging_hash: [0u8; 32].into(), // will be set in the next migration + } + } + } + + fn multiply_all_capacities( + old_roles: LwwMap, + mul: u64, + ) -> LwwMap { + let mut new_roles = LwwMap::new(); + for (node, ts, role) in old_roles.items() { + let mut role = role.clone(); + if let NodeRoleV(Some(NodeRole { + capacity: Some(ref mut cap), + .. + })) = role + { + *cap *= mul; + } + new_roles.merge_raw(node, *ts, &role); + } + new_roles + } +} + +mod v010 { + use super::v09; + use crate::layout::CompactNodeType; + use garage_util::crdt::{Lww, LwwMap}; + use garage_util::data::Uuid; + use serde::{Deserialize, Serialize}; + use std::collections::BTreeMap; + pub use v09::{LayoutParameters, NodeRole, NodeRoleV, ZoneRedundancy}; + + /// Number of old (non-live) versions to keep, see LayoutHistory::old_versions + pub const OLD_VERSION_COUNT: usize = 5; + + /// The history of cluster layouts, with trackers to keep a record + /// of which nodes are up-to-date to current cluster data + #[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] + pub struct LayoutHistory { + /// The versions currently in use in the cluster + pub versions: Vec, + /// At most 5 of the previous versions, not used by the garage_table + /// module, but usefull for the garage_block module to find data blocks + /// that have not yet been moved + pub old_versions: Vec, + + /// Update trackers + pub update_trackers: UpdateTrackers, + + /// Staged changes for the next version + pub staging: Lww, + } + + /// A version of the layout of the cluster, i.e. the list of roles + /// which are assigned to each cluster node + #[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] + pub struct LayoutVersion { + /// The number of this version + pub version: u64, + + /// Roles assigned to nodes in this version + pub roles: LwwMap, + /// Parameters used to compute the assignment currently given by + /// ring_assignment_data + pub parameters: LayoutParameters, + + /// The number of replicas for each data partition + pub replication_factor: usize, + /// This attribute is only used to retain the previously computed partition size, + /// to know to what extent does it change with the layout update. + pub partition_size: u64, + + /// node_id_vec: a vector of node IDs with a role assigned + /// in the system (this includes gateway nodes). + /// The order here is different than the vec stored by `roles`, because: + /// 1. non-gateway nodes are first so that they have lower numbers + /// 2. nodes that don't have a role are excluded (but they need to + /// stay in the CRDT as tombstones) + pub node_id_vec: Vec, + /// number of non-gateway nodes, which are the first ids in node_id_vec + pub nongateway_node_count: usize, + /// The assignation of data partitions to nodes, the values + /// are indices in node_id_vec + #[serde(with = "serde_bytes")] + pub ring_assignment_data: Vec, + } + + /// The staged changes for the next layout version + #[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] + pub struct LayoutStaging { + /// Parameters to be used in the next partition assignment computation. + pub parameters: Lww, + /// Role changes which are staged for the next version of the layout + pub roles: LwwMap, + } + + /// The tracker of acknowlegments and data syncs around the cluster + #[derive(Clone, Debug, Serialize, Deserialize, Default, PartialEq)] + pub struct UpdateTrackers { + /// The highest layout version number each node has ack'ed + pub ack_map: UpdateTracker, + /// The highest layout version number each node has synced data for + pub sync_map: UpdateTracker, + /// The highest layout version number each node has + /// ack'ed that all other nodes have synced data for + pub sync_ack_map: UpdateTracker, + } + + /// Generic update tracker struct + #[derive(Clone, Debug, Serialize, Deserialize, Default, PartialEq)] + pub struct UpdateTracker(pub BTreeMap); + + impl garage_util::migrate::Migrate for LayoutHistory { + const VERSION_MARKER: &'static [u8] = b"G010lh"; + + type Previous = v09::ClusterLayout; + + fn migrate(previous: Self::Previous) -> Self { + let nongateway_node_count = previous + .node_id_vec + .iter() + .enumerate() + .filter(|(_, uuid)| { + let role = previous.roles.get(uuid); + matches!(role, Some(NodeRoleV(Some(role))) if role.capacity.is_some()) + }) + .map(|(i, _)| i + 1) + .max() + .unwrap_or(0); + + let version = LayoutVersion { + version: previous.version, + replication_factor: previous.replication_factor, + partition_size: previous.partition_size, + parameters: previous.parameters, + roles: previous.roles, + node_id_vec: previous.node_id_vec, + nongateway_node_count, + ring_assignment_data: previous.ring_assignment_data, + }; + let update_tracker = UpdateTracker( + version + .nongateway_nodes() + .iter() + .copied() + .map(|x| (x, version.version)) + .collect::>(), + ); + let staging = LayoutStaging { + parameters: previous.staging_parameters, + roles: previous.staging_roles, + }; + Self { + versions: vec![version], + old_versions: vec![], + update_trackers: UpdateTrackers { + ack_map: update_tracker.clone(), + sync_map: update_tracker.clone(), + sync_ack_map: update_tracker.clone(), + }, + staging: Lww::raw(previous.version, staging), + } + } + } +} + +pub use v010::*; + +// ---- utility functions ---- + +impl AutoCrdt for LayoutParameters { + const WARN_IF_DIFFERENT: bool = true; +} + +impl AutoCrdt for NodeRoleV { + const WARN_IF_DIFFERENT: bool = true; +} + +impl Crdt for LayoutStaging { + fn merge(&mut self, other: &LayoutStaging) { + self.parameters.merge(&other.parameters); + self.roles.merge(&other.roles); + } +} + +impl NodeRole { + pub fn capacity_string(&self) -> String { + match self.capacity { + Some(c) => ByteSize::b(c).to_string_as(false), + None => "gateway".to_string(), + } + } + + pub fn tags_string(&self) -> String { + self.tags.join(",") + } +} + +impl fmt::Display for ZoneRedundancy { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + ZoneRedundancy::Maximum => write!(f, "maximum"), + ZoneRedundancy::AtLeast(x) => write!(f, "{}", x), + } + } +} + +impl core::str::FromStr for ZoneRedundancy { + type Err = &'static str; + fn from_str(s: &str) -> Result { + match s { + "none" | "max" | "maximum" => Ok(ZoneRedundancy::Maximum), + x => { + let v = x + .parse::() + .map_err(|_| "zone redundancy must be 'none'/'max' or an integer")?; + Ok(ZoneRedundancy::AtLeast(v)) + } + } + } +} + +impl UpdateTracker { + fn merge(&mut self, other: &UpdateTracker) -> bool { + let mut changed = false; + for (k, v) in other.0.iter() { + if let Some(v_mut) = self.0.get_mut(k) { + if *v > *v_mut { + *v_mut = *v; + changed = true; + } + } else { + self.0.insert(*k, *v); + changed = true; + } + } + changed + } + + /// This bumps the update tracker for a given node up to the specified value. + /// This has potential impacts on the correctness of Garage and should only + /// be used in very specific circumstances. + pub fn set_max(&mut self, peer: Uuid, value: u64) -> bool { + match self.0.get_mut(&peer) { + Some(e) if *e < value => { + *e = value; + true + } + None => { + self.0.insert(peer, value); + true + } + _ => false, + } + } + + pub(crate) fn min_among(&self, storage_nodes: &[Uuid], min_version: u64) -> u64 { + storage_nodes + .iter() + .map(|x| self.get(x, min_version)) + .min() + .unwrap_or(min_version) + } + + pub fn get(&self, node: &Uuid, min_version: u64) -> u64 { + self.0.get(node).copied().unwrap_or(min_version) + } +} + +impl UpdateTrackers { + pub(crate) fn merge(&mut self, other: &UpdateTrackers) -> bool { + let c1 = self.ack_map.merge(&other.ack_map); + let c2 = self.sync_map.merge(&other.sync_map); + let c3 = self.sync_ack_map.merge(&other.sync_ack_map); + c1 || c2 || c3 + } +} diff --git a/src/rpc/layout/schema.rs b/src/rpc/layout/schema.rs deleted file mode 100644 index df949906..00000000 --- a/src/rpc/layout/schema.rs +++ /dev/null @@ -1,431 +0,0 @@ -use std::fmt; - -use bytesize::ByteSize; - -use garage_util::crdt::{AutoCrdt, Crdt}; -use garage_util::data::Uuid; - -mod v08 { - use crate::layout::CompactNodeType; - use garage_util::crdt::LwwMap; - use garage_util::data::{Hash, Uuid}; - use serde::{Deserialize, Serialize}; - - /// The layout of the cluster, i.e. the list of roles - /// which are assigned to each cluster node - #[derive(Clone, Debug, Serialize, Deserialize)] - pub struct ClusterLayout { - pub version: u64, - - pub replication_factor: usize, - pub roles: LwwMap, - - /// node_id_vec: a vector of node IDs with a role assigned - /// in the system (this includes gateway nodes). - /// The order here is different than the vec stored by `roles`, because: - /// 1. non-gateway nodes are first so that they have lower numbers - /// 2. nodes that don't have a role are excluded (but they need to - /// stay in the CRDT as tombstones) - pub node_id_vec: Vec, - /// the assignation of data partitions to node, the values - /// are indices in node_id_vec - #[serde(with = "serde_bytes")] - pub ring_assignation_data: Vec, - - /// Role changes which are staged for the next version of the layout - pub staging: LwwMap, - pub staging_hash: Hash, - } - - #[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Debug, Serialize, Deserialize)] - pub struct NodeRoleV(pub Option); - - /// The user-assigned roles of cluster nodes - #[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Debug, Serialize, Deserialize)] - pub struct NodeRole { - /// Datacenter at which this entry belong. This information is used to - /// perform a better geodistribution - pub zone: String, - /// The capacity of the node - /// If this is set to None, the node does not participate in storing data for the system - /// and is only active as an API gateway to other nodes - pub capacity: Option, - /// A set of tags to recognize the node - pub tags: Vec, - } - - impl garage_util::migrate::InitialFormat for ClusterLayout {} -} - -mod v09 { - use super::v08; - use crate::layout::CompactNodeType; - use garage_util::crdt::{Lww, LwwMap}; - use garage_util::data::{Hash, Uuid}; - use serde::{Deserialize, Serialize}; - pub use v08::{NodeRole, NodeRoleV}; - - /// The layout of the cluster, i.e. the list of roles - /// which are assigned to each cluster node - #[derive(Clone, Debug, Serialize, Deserialize)] - pub struct ClusterLayout { - pub version: u64, - - pub replication_factor: usize, - - /// This attribute is only used to retain the previously computed partition size, - /// to know to what extent does it change with the layout update. - pub partition_size: u64, - /// Parameters used to compute the assignment currently given by - /// ring_assignment_data - pub parameters: LayoutParameters, - - pub roles: LwwMap, - - /// see comment in v08::ClusterLayout - pub node_id_vec: Vec, - /// see comment in v08::ClusterLayout - #[serde(with = "serde_bytes")] - pub ring_assignment_data: Vec, - - /// Parameters to be used in the next partition assignment computation. - pub staging_parameters: Lww, - /// Role changes which are staged for the next version of the layout - pub staging_roles: LwwMap, - pub staging_hash: Hash, - } - - /// This struct is used to set the parameters to be used in the assignment computation - /// algorithm. It is stored as a Crdt. - #[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Copy, Debug, Serialize, Deserialize)] - pub struct LayoutParameters { - pub zone_redundancy: ZoneRedundancy, - } - - /// Zone redundancy: if set to AtLeast(x), the layout calculation will aim to store copies - /// of each partition on at least that number of different zones. - /// Otherwise, copies will be stored on the maximum possible number of zones. - #[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Copy, Debug, Serialize, Deserialize)] - pub enum ZoneRedundancy { - AtLeast(usize), - Maximum, - } - - impl garage_util::migrate::Migrate for ClusterLayout { - const VERSION_MARKER: &'static [u8] = b"G09layout"; - - type Previous = v08::ClusterLayout; - - fn migrate(previous: Self::Previous) -> Self { - use itertools::Itertools; - - // In the old layout, capacities are in an arbitrary unit, - // but in the new layout they are in bytes. - // Here we arbitrarily multiply everything by 1G, - // such that 1 old capacity unit = 1GB in the new units. - // This is totally arbitrary and won't work for most users. - let cap_mul = 1024 * 1024 * 1024; - let roles = multiply_all_capacities(previous.roles, cap_mul); - let staging_roles = multiply_all_capacities(previous.staging, cap_mul); - let node_id_vec = previous.node_id_vec; - - // Determine partition size - let mut tmp = previous.ring_assignation_data.clone(); - tmp.sort(); - let partition_size = tmp - .into_iter() - .dedup_with_count() - .map(|(npart, node)| { - roles - .get(&node_id_vec[node as usize]) - .and_then(|p| p.0.as_ref().and_then(|r| r.capacity)) - .unwrap_or(0) / npart as u64 - }) - .min() - .unwrap_or(0); - - // By default, zone_redundancy is maximum possible value - let parameters = LayoutParameters { - zone_redundancy: ZoneRedundancy::Maximum, - }; - - Self { - version: previous.version, - replication_factor: previous.replication_factor, - partition_size, - parameters, - roles, - node_id_vec, - ring_assignment_data: previous.ring_assignation_data, - staging_parameters: Lww::new(parameters), - staging_roles, - staging_hash: [0u8; 32].into(), // will be set in the next migration - } - } - } - - fn multiply_all_capacities( - old_roles: LwwMap, - mul: u64, - ) -> LwwMap { - let mut new_roles = LwwMap::new(); - for (node, ts, role) in old_roles.items() { - let mut role = role.clone(); - if let NodeRoleV(Some(NodeRole { - capacity: Some(ref mut cap), - .. - })) = role - { - *cap *= mul; - } - new_roles.merge_raw(node, *ts, &role); - } - new_roles - } -} - -mod v010 { - use super::v09; - use crate::layout::CompactNodeType; - use garage_util::crdt::{Lww, LwwMap}; - use garage_util::data::Uuid; - use serde::{Deserialize, Serialize}; - use std::collections::BTreeMap; - pub use v09::{LayoutParameters, NodeRole, NodeRoleV, ZoneRedundancy}; - - pub const OLD_VERSION_COUNT: usize = 5; - - /// The history of cluster layouts, with trackers to keep a record - /// of which nodes are up-to-date to current cluster data - #[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] - pub struct LayoutHistory { - /// The versions currently in use in the cluster - pub versions: Vec, - /// At most 5 of the previous versions, not used by the garage_table - /// module, but usefull for the garage_block module to find data blocks - /// that have not yet been moved - pub old_versions: Vec, - - /// Update trackers - pub update_trackers: UpdateTrackers, - - /// Staged changes for the next version - pub staging: Lww, - } - - /// A version of the layout of the cluster, i.e. the list of roles - /// which are assigned to each cluster node - #[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] - pub struct LayoutVersion { - pub version: u64, - - pub replication_factor: usize, - - /// This attribute is only used to retain the previously computed partition size, - /// to know to what extent does it change with the layout update. - pub partition_size: u64, - /// Parameters used to compute the assignment currently given by - /// ring_assignment_data - pub parameters: LayoutParameters, - - pub roles: LwwMap, - - /// see comment in v08::ClusterLayout - pub node_id_vec: Vec, - /// number of non-gateway nodes, which are the first ids in node_id_vec - pub nongateway_node_count: usize, - /// see comment in v08::ClusterLayout - #[serde(with = "serde_bytes")] - pub ring_assignment_data: Vec, - } - - /// The staged changes for the next layout version - #[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] - pub struct LayoutStaging { - /// Parameters to be used in the next partition assignment computation. - pub parameters: Lww, - /// Role changes which are staged for the next version of the layout - pub roles: LwwMap, - } - - /// The tracker of acknowlegments and data syncs around the cluster - #[derive(Clone, Debug, Serialize, Deserialize, Default, PartialEq)] - pub struct UpdateTrackers { - /// The highest layout version number each node has ack'ed - pub ack_map: UpdateTracker, - /// The highest layout version number each node has synced data for - pub sync_map: UpdateTracker, - /// The highest layout version number each node has - /// ack'ed that all other nodes have synced data for - pub sync_ack_map: UpdateTracker, - } - - /// The history of cluster layouts - #[derive(Clone, Debug, Serialize, Deserialize, Default, PartialEq)] - pub struct UpdateTracker(pub BTreeMap); - - impl garage_util::migrate::Migrate for LayoutHistory { - const VERSION_MARKER: &'static [u8] = b"G010lh"; - - type Previous = v09::ClusterLayout; - - fn migrate(previous: Self::Previous) -> Self { - let nongateway_node_count = previous - .node_id_vec - .iter() - .enumerate() - .filter(|(_, uuid)| { - let role = previous.roles.get(uuid); - matches!(role, Some(NodeRoleV(Some(role))) if role.capacity.is_some()) - }) - .map(|(i, _)| i + 1) - .max() - .unwrap_or(0); - - let version = LayoutVersion { - version: previous.version, - replication_factor: previous.replication_factor, - partition_size: previous.partition_size, - parameters: previous.parameters, - roles: previous.roles, - node_id_vec: previous.node_id_vec, - nongateway_node_count, - ring_assignment_data: previous.ring_assignment_data, - }; - let update_tracker = UpdateTracker( - version - .nongateway_nodes() - .iter() - .copied() - .map(|x| (x, version.version)) - .collect::>(), - ); - let staging = LayoutStaging { - parameters: previous.staging_parameters, - roles: previous.staging_roles, - }; - Self { - versions: vec![version], - old_versions: vec![], - update_trackers: UpdateTrackers { - ack_map: update_tracker.clone(), - sync_map: update_tracker.clone(), - sync_ack_map: update_tracker.clone(), - }, - staging: Lww::raw(previous.version, staging), - } - } - } -} - -pub use v010::*; - -// ---- utility functions ---- - -impl AutoCrdt for LayoutParameters { - const WARN_IF_DIFFERENT: bool = true; -} - -impl AutoCrdt for NodeRoleV { - const WARN_IF_DIFFERENT: bool = true; -} - -impl Crdt for LayoutStaging { - fn merge(&mut self, other: &LayoutStaging) { - self.parameters.merge(&other.parameters); - self.roles.merge(&other.roles); - } -} - -impl NodeRole { - pub fn capacity_string(&self) -> String { - match self.capacity { - Some(c) => ByteSize::b(c).to_string_as(false), - None => "gateway".to_string(), - } - } - - pub fn tags_string(&self) -> String { - self.tags.join(",") - } -} - -impl fmt::Display for ZoneRedundancy { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match self { - ZoneRedundancy::Maximum => write!(f, "maximum"), - ZoneRedundancy::AtLeast(x) => write!(f, "{}", x), - } - } -} - -impl core::str::FromStr for ZoneRedundancy { - type Err = &'static str; - fn from_str(s: &str) -> Result { - match s { - "none" | "max" | "maximum" => Ok(ZoneRedundancy::Maximum), - x => { - let v = x - .parse::() - .map_err(|_| "zone redundancy must be 'none'/'max' or an integer")?; - Ok(ZoneRedundancy::AtLeast(v)) - } - } - } -} - -impl UpdateTracker { - fn merge(&mut self, other: &UpdateTracker) -> bool { - let mut changed = false; - for (k, v) in other.0.iter() { - if let Some(v_mut) = self.0.get_mut(k) { - if *v > *v_mut { - *v_mut = *v; - changed = true; - } - } else { - self.0.insert(*k, *v); - changed = true; - } - } - changed - } - - /// This bumps the update tracker for a given node up to the specified value. - /// This has potential impacts on the correctness of Garage and should only - /// be used in very specific circumstances. - pub fn set_max(&mut self, peer: Uuid, value: u64) -> bool { - match self.0.get_mut(&peer) { - Some(e) if *e < value => { - *e = value; - true - } - None => { - self.0.insert(peer, value); - true - } - _ => false, - } - } - - pub(crate) fn min_among(&self, storage_nodes: &[Uuid], min_version: u64) -> u64 { - storage_nodes - .iter() - .map(|x| self.get(x, min_version)) - .min() - .unwrap_or(min_version) - } - - pub fn get(&self, node: &Uuid, min_version: u64) -> u64 { - self.0.get(node).copied().unwrap_or(min_version) - } -} - -impl UpdateTrackers { - pub(crate) fn merge(&mut self, other: &UpdateTrackers) -> bool { - let c1 = self.ack_map.merge(&other.ack_map); - let c2 = self.sync_map.merge(&other.sync_map); - let c3 = self.sync_ack_map.merge(&other.sync_ack_map); - c1 || c2 || c3 - } -} diff --git a/src/rpc/layout/version.rs b/src/rpc/layout/version.rs index cbfbee94..5b307156 100644 --- a/src/rpc/layout/version.rs +++ b/src/rpc/layout/version.rs @@ -10,7 +10,6 @@ use garage_util::data::*; use garage_util::error::*; use super::graph_algo::*; -use super::schema::*; use super::*; // The Message type will be used to collect information on the algorithm. -- cgit v1.2.3 From f8df90b79b93e4a1391839435718bad8c697246d Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Fri, 8 Dec 2023 14:54:11 +0100 Subject: table: fix insert_many to not send duplicates --- src/table/table.rs | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/src/table/table.rs b/src/table/table.rs index 6508cf5d..59cfdd07 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -196,6 +196,8 @@ impl Table { let hash = entry.partition_key().hash(); let mut write_sets = self.data.replication.write_sets(&hash); for set in write_sets.as_mut().iter_mut() { + // Sort nodes in each write sets to merge write sets with same + // nodes but in possibly different orders set.sort(); } let e_enc = Arc::new(ByteBuf::from(entry.encode()?)); @@ -220,7 +222,16 @@ impl Table { for (write_sets, entry_enc) in entries_vec.iter() { for write_set in write_sets.as_ref().iter() { for node in write_set.iter() { - call_list.entry(*node).or_default().push(entry_enc.clone()) + let node_entries = call_list.entry(*node).or_default(); + match node_entries.last() { + Some(x) if Arc::ptr_eq(x, entry_enc) => { + // skip if entry already in list to send to this node + // (could happen if node is in several write sets for this entry) + } + _ => { + node_entries.push(entry_enc.clone()); + } + } } } } -- cgit v1.2.3 From e4f493b48156e6e30f16fba10f300f6cb5fe0b0d Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 11 Dec 2023 14:57:42 +0100 Subject: table: remove redundant tracing in insert_many --- src/table/table.rs | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/src/table/table.rs b/src/table/table.rs index 59cfdd07..05a0dab1 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -239,9 +239,7 @@ impl Table { // Build futures to actually perform each of the corresponding RPC calls let call_futures = call_list.into_iter().map(|(node, entries)| { let this = self.clone(); - let tracer = opentelemetry::global::tracer("garage"); - let span = tracer.start(format!("RPC to {:?}", node)); - let fut = async move { + async move { let rpc = TableRpc::::Update(entries); let resp = this .system @@ -254,8 +252,7 @@ impl Table { ) .await; (node, resp) - }; - fut.with_context(Context::current_with_span(span)) + } }); // Run all requests in parallel thanks to FuturesUnordered, and collect results. -- cgit v1.2.3 From 85b5a6bcd11c0a7651e4c589569e1935a3d18e46 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 11 Dec 2023 15:31:47 +0100 Subject: fix some clippy lints --- src/api/admin/cluster.rs | 2 +- src/rpc/layout/helper.rs | 2 +- src/rpc/layout/history.rs | 14 ++++++-------- src/rpc/layout/mod.rs | 2 +- src/rpc/layout/version.rs | 6 +++--- src/rpc/rpc_helper.rs | 2 +- src/rpc/system.rs | 2 +- src/table/sync.rs | 8 ++++---- src/table/table.rs | 3 +-- 9 files changed, 19 insertions(+), 22 deletions(-) diff --git a/src/api/admin/cluster.rs b/src/api/admin/cluster.rs index 3ce1b254..8677257d 100644 --- a/src/api/admin/cluster.rs +++ b/src/api/admin/cluster.rs @@ -94,7 +94,7 @@ pub async fn handle_get_cluster_status(garage: &Arc) -> Result>(); + let mut nodes = nodes.into_values().collect::>(); nodes.sort_by(|x, y| x.id.cmp(&y.id)); let res = GetClusterStatusResponse { diff --git a/src/rpc/layout/helper.rs b/src/rpc/layout/helper.rs index 2ba010b8..7e5d37e9 100644 --- a/src/rpc/layout/helper.rs +++ b/src/rpc/layout/helper.rs @@ -129,7 +129,7 @@ impl LayoutHelper { where F: FnOnce(&mut LayoutHistory) -> bool, { - let changed = f(&mut self.layout.as_mut().unwrap()); + let changed = f(self.layout.as_mut().unwrap()); if changed { *self = Self::new( self.replication_mode, diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index a53256cc..23196aee 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -42,8 +42,7 @@ impl LayoutHistory { let set = self .versions .iter() - .map(|x| x.all_nodes()) - .flatten() + .flat_map(|x| x.all_nodes()) .collect::>(); set.into_iter().copied().collect::>() } @@ -56,8 +55,7 @@ impl LayoutHistory { let set = self .versions .iter() - .map(|x| x.nongateway_nodes()) - .flatten() + .flat_map(|x| x.nongateway_nodes()) .collect::>(); set.into_iter().copied().collect::>() } @@ -94,7 +92,7 @@ impl LayoutHistory { let sync_ack_map_min = self .update_trackers .sync_ack_map - .min_among(¤t_nodes, min_version); + .min_among(current_nodes, min_version); if self.min_stored() < sync_ack_map_min { let removed = self.versions.remove(0); info!( @@ -144,7 +142,7 @@ impl LayoutHistory { let global_min = self .update_trackers .sync_map - .min_among(&all_nongateway_nodes, min_version); + .min_among(all_nongateway_nodes, min_version); // If the write quorums are equal to the total number of nodes, // i.e. no writes can succeed while they are not written to all nodes, @@ -281,7 +279,7 @@ To know the correct value of the new layout version, invoke `garage layout show` let (new_version, msg) = self .current() .clone() - .calculate_next_version(&self.staging.get())?; + .calculate_next_version(self.staging.get())?; self.versions.push(new_version); self.cleanup_old_versions(); @@ -297,7 +295,7 @@ To know the correct value of the new layout version, invoke `garage layout show` pub fn revert_staged_changes(mut self) -> Result { self.staging.update(LayoutStaging { - parameters: Lww::new(self.current().parameters.clone()), + parameters: Lww::new(self.current().parameters), roles: LwwMap::new(), }); diff --git a/src/rpc/layout/mod.rs b/src/rpc/layout/mod.rs index facdb2ce..162e3c6e 100644 --- a/src/rpc/layout/mod.rs +++ b/src/rpc/layout/mod.rs @@ -357,7 +357,7 @@ mod v010 { update_trackers: UpdateTrackers { ack_map: update_tracker.clone(), sync_map: update_tracker.clone(), - sync_ack_map: update_tracker.clone(), + sync_ack_map: update_tracker, }, staging: Lww::raw(previous.version, staging), } diff --git a/src/rpc/layout/version.rs b/src/rpc/layout/version.rs index 5b307156..ee4b2821 100644 --- a/src/rpc/layout/version.rs +++ b/src/rpc/layout/version.rs @@ -137,19 +137,19 @@ impl LayoutVersion { // ===================== internal information extractors ====================== pub(crate) fn expect_get_node_capacity(&self, uuid: &Uuid) -> u64 { - self.get_node_capacity(&uuid) + self.get_node_capacity(uuid) .expect("non-gateway node with zero capacity") } pub(crate) fn expect_get_node_zone(&self, uuid: &Uuid) -> &str { - self.get_node_zone(&uuid).expect("node without a zone") + self.get_node_zone(uuid).expect("node without a zone") } /// Returns the sum of capacities of non gateway nodes in the cluster fn get_total_capacity(&self) -> u64 { let mut total_capacity = 0; for uuid in self.nongateway_nodes() { - total_capacity += self.expect_get_node_capacity(&uuid); + total_capacity += self.expect_get_node_capacity(uuid); } total_capacity } diff --git a/src/rpc/rpc_helper.rs b/src/rpc/rpc_helper.rs index 65af8901..77a36ca1 100644 --- a/src/rpc/rpc_helper.rs +++ b/src/rpc/rpc_helper.rs @@ -442,7 +442,7 @@ impl RpcHelper { // Send one request to each peer of the quorum sets let msg = msg.into_req().map_err(netapp::error::Error::from)?; - let requests = result_tracker.nodes.iter().map(|(peer, _)| { + let requests = result_tracker.nodes.keys().map(|peer| { let self2 = self.clone(); let msg = msg.clone(); let endpoint2 = endpoint.clone(); diff --git a/src/rpc/system.rs b/src/rpc/system.rs index a8f12852..41d76177 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -315,7 +315,7 @@ impl System { local_status: ArcSwap::new(Arc::new(local_status)), node_status: RwLock::new(HashMap::new()), netapp: netapp.clone(), - fullmesh: fullmesh.clone(), + fullmesh, system_endpoint, replication_mode, replication_factor, diff --git a/src/table/sync.rs b/src/table/sync.rs index cfcbc4b5..1561a2e5 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -123,15 +123,15 @@ impl TableSyncer { let mut sync_futures = result_tracker .nodes - .iter() - .map(|(node, _)| *node) + .keys() + .copied() .map(|node| { let must_exit = must_exit.clone(); async move { if node == my_id { (node, Ok(())) } else { - (node, self.do_sync_with(&partition, node, must_exit).await) + (node, self.do_sync_with(partition, node, must_exit).await) } } }) @@ -145,7 +145,7 @@ impl TableSyncer { } if result_tracker.too_many_failures() { - return Err(result_tracker.quorum_error()); + Err(result_tracker.quorum_error()) } else { Ok(()) } diff --git a/src/table/table.rs b/src/table/table.rs index 05a0dab1..a5be2910 100644 --- a/src/table/table.rs +++ b/src/table/table.rs @@ -209,8 +209,7 @@ impl Table { // it takes part, to optimize the detection of a quorum. let mut write_sets = entries_vec .iter() - .map(|(wss, _)| wss.as_ref().iter().map(|ws| ws.as_slice())) - .flatten() + .flat_map(|(wss, _)| wss.as_ref().iter().map(|ws| ws.as_slice())) .collect::>(); write_sets.sort(); write_sets.dedup(); -- cgit v1.2.3 From adccce1145d5d82581e4a5da707be35badb2d5a6 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 11 Dec 2023 15:45:14 +0100 Subject: layout: refactor/fix bad while loop --- src/rpc/layout/history.rs | 31 ++++++++++++++----------------- 1 file changed, 14 insertions(+), 17 deletions(-) diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs index 23196aee..b8cc27da 100644 --- a/src/rpc/layout/history.rs +++ b/src/rpc/layout/history.rs @@ -86,23 +86,20 @@ impl LayoutHistory { // remove them (keep them in self.old_versions). // ASSUMPTION: we only care about where nodes in the current layout version // are reading from, as we assume older nodes are being discarded. - while self.versions.len() > 1 { - let current_nodes = &self.current().node_id_vec; - let min_version = self.min_stored(); - let sync_ack_map_min = self - .update_trackers - .sync_ack_map - .min_among(current_nodes, min_version); - if self.min_stored() < sync_ack_map_min { - let removed = self.versions.remove(0); - info!( - "Layout history: moving version {} to old_versions", - removed.version - ); - self.old_versions.push(removed); - } else { - break; - } + let current_nodes = &self.current().node_id_vec; + let min_version = self.min_stored(); + let sync_ack_map_min = self + .update_trackers + .sync_ack_map + .min_among(current_nodes, min_version); + while self.min_stored() < sync_ack_map_min { + assert!(self.versions.len() > 1); + let removed = self.versions.remove(0); + info!( + "Layout history: moving version {} to old_versions", + removed.version + ); + self.old_versions.push(removed); } while self.old_versions.len() > OLD_VERSION_COUNT { -- cgit v1.2.3 From 0041b013a473e3ae72f50209d8f79db75a72848b Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Mon, 11 Dec 2023 16:09:22 +0100 Subject: layout: refactoring and fix in layout helper --- src/rpc/layout/helper.rs | 43 +++++++++++++++++++++++----------------- src/rpc/layout/manager.rs | 2 +- src/rpc/layout/mod.rs | 2 +- src/rpc/rpc_helper.rs | 2 +- src/rpc/system.rs | 4 ++-- src/table/replication/sharded.rs | 2 +- src/table/sync.rs | 16 +++++++-------- 7 files changed, 38 insertions(+), 33 deletions(-) diff --git a/src/rpc/layout/helper.rs b/src/rpc/layout/helper.rs index 7e5d37e9..9fb738ea 100644 --- a/src/rpc/layout/helper.rs +++ b/src/rpc/layout/helper.rs @@ -10,7 +10,7 @@ use super::*; use crate::replication_mode::ReplicationMode; #[derive(Debug, Clone, Serialize, Deserialize, Default, PartialEq, Eq)] -pub struct LayoutDigest { +pub struct RpcLayoutDigest { /// Cluster layout version pub current_version: u64, /// Number of active layout versions @@ -21,6 +21,13 @@ pub struct LayoutDigest { pub staging_hash: Hash, } +#[derive(Debug, Clone, Copy, Eq, PartialEq)] +pub struct SyncLayoutDigest { + current: u64, + ack_map_min: u64, + min_stored: u64, +} + pub struct LayoutHelper { replication_mode: ReplicationMode, layout: Option, @@ -150,20 +157,20 @@ impl LayoutHelper { &self.all_nongateway_nodes } - pub fn all_ack(&self) -> u64 { + pub fn ack_map_min(&self) -> u64 { self.ack_map_min } - pub fn all_sync(&self) -> u64 { + pub fn sync_map_min(&self) -> u64 { self.sync_map_min } - pub fn sync_versions(&self) -> (u64, u64, u64) { - ( - self.layout().current().version, - self.all_ack(), - self.layout().min_stored(), - ) + pub fn sync_digest(&self) -> SyncLayoutDigest { + SyncLayoutDigest { + current: self.layout().current().version, + ack_map_min: self.ack_map_min(), + min_stored: self.layout().min_stored(), + } } pub fn read_nodes_of(&self, position: &Hash) -> Vec { @@ -206,8 +213,8 @@ impl LayoutHelper { self.staging_hash } - pub fn digest(&self) -> LayoutDigest { - LayoutDigest { + pub fn digest(&self) -> RpcLayoutDigest { + RpcLayoutDigest { current_version: self.current().version, active_versions: self.versions.len(), trackers_hash: self.trackers_hash, @@ -231,13 +238,13 @@ impl LayoutHelper { // 3. Acknowledge everyone has synced up to min(self.sync_map) self.sync_ack(local_node_id); - info!("ack_map: {:?}", self.update_trackers.ack_map); - info!("sync_map: {:?}", self.update_trackers.sync_map); - info!("sync_ack_map: {:?}", self.update_trackers.sync_ack_map); + debug!("ack_map: {:?}", self.update_trackers.ack_map); + debug!("sync_map: {:?}", self.update_trackers.sync_map); + debug!("sync_ack_map: {:?}", self.update_trackers.sync_ack_map); } fn sync_first(&mut self, local_node_id: Uuid) { - let first_version = self.versions.first().as_ref().unwrap().version; + let first_version = self.min_stored(); self.update(|layout| { layout .update_trackers @@ -275,13 +282,13 @@ impl LayoutHelper { .versions .iter() .map(|x| x.version) - .take_while(|v| { + .skip_while(|v| { self.ack_lock .get(v) .map(|x| x.load(Ordering::Relaxed) == 0) .unwrap_or(true) }) - .max() - .unwrap_or(self.min_stored()) + .next() + .unwrap_or(self.current().version) } } diff --git a/src/rpc/layout/manager.rs b/src/rpc/layout/manager.rs index ec8a2a15..6747b79d 100644 --- a/src/rpc/layout/manager.rs +++ b/src/rpc/layout/manager.rs @@ -256,7 +256,7 @@ impl LayoutManager { // ---- RPC HANDLERS ---- - pub(crate) fn handle_advertise_status(self: &Arc, from: Uuid, remote: &LayoutDigest) { + pub(crate) fn handle_advertise_status(self: &Arc, from: Uuid, remote: &RpcLayoutDigest) { let local = self.layout().digest(); if remote.current_version > local.current_version || remote.active_versions != local.active_versions diff --git a/src/rpc/layout/mod.rs b/src/rpc/layout/mod.rs index 162e3c6e..33676c37 100644 --- a/src/rpc/layout/mod.rs +++ b/src/rpc/layout/mod.rs @@ -17,7 +17,7 @@ pub mod manager; // ---- re-exports ---- -pub use helper::{LayoutDigest, LayoutHelper}; +pub use helper::{LayoutHelper, RpcLayoutDigest, SyncLayoutDigest}; pub use manager::WriteLock; pub use version::*; diff --git a/src/rpc/rpc_helper.rs b/src/rpc/rpc_helper.rs index 77a36ca1..ae3a19c4 100644 --- a/src/rpc/rpc_helper.rs +++ b/src/rpc/rpc_helper.rs @@ -502,7 +502,7 @@ impl RpcHelper { .rev() .chain(layout.old_versions.iter().rev()); for ver in ver_iter { - if ver.version > layout.all_sync() { + if ver.version > layout.sync_map_min() { continue; } let nodes = ver.nodes_of(position, ver.replication_factor); diff --git a/src/rpc/system.rs b/src/rpc/system.rs index 41d76177..83cc6816 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -34,7 +34,7 @@ use crate::consul::ConsulDiscovery; #[cfg(feature = "kubernetes-discovery")] use crate::kubernetes::*; use crate::layout::{ - self, manager::LayoutManager, LayoutDigest, LayoutHelper, LayoutHistory, NodeRoleV, + self, manager::LayoutManager, LayoutHelper, LayoutHistory, NodeRoleV, RpcLayoutDigest, }; use crate::replication_mode::*; use crate::rpc_helper::*; @@ -132,7 +132,7 @@ pub struct NodeStatus { pub replication_factor: usize, /// Cluster layout digest - pub layout_digest: LayoutDigest, + pub layout_digest: RpcLayoutDigest, /// Disk usage on partition containing metadata directory (tuple: `(avail, total)`) #[serde(default)] diff --git a/src/table/replication/sharded.rs b/src/table/replication/sharded.rs index 55d0029d..8ba3700f 100644 --- a/src/table/replication/sharded.rs +++ b/src/table/replication/sharded.rs @@ -54,7 +54,7 @@ impl TableReplication for TableShardedReplication { fn sync_partitions(&self) -> SyncPartitions { let layout = self.system.cluster_layout(); - let layout_version = layout.all_ack(); + let layout_version = layout.ack_map_min(); let mut partitions = layout .current() diff --git a/src/table/sync.rs b/src/table/sync.rs index 1561a2e5..cd080df0 100644 --- a/src/table/sync.rs +++ b/src/table/sync.rs @@ -83,7 +83,7 @@ impl TableSyncer { bg.spawn_worker(SyncWorker { syncer: self.clone(), layout_notify: self.system.layout_notify(), - layout_versions: self.system.cluster_layout().sync_versions(), + layout_digest: self.system.cluster_layout().sync_digest(), add_full_sync_rx, todo: None, next_full_sync: Instant::now() + Duration::from_secs(20), @@ -483,7 +483,7 @@ struct SyncWorker { syncer: Arc>, layout_notify: Arc, - layout_versions: (u64, u64, u64), + layout_digest: SyncLayoutDigest, add_full_sync_rx: mpsc::UnboundedReceiver<()>, next_full_sync: Instant, @@ -493,15 +493,13 @@ struct SyncWorker { impl SyncWorker { fn check_add_full_sync(&mut self) { - let layout_versions = self.syncer.system.cluster_layout().sync_versions(); - if layout_versions != self.layout_versions { - self.layout_versions = layout_versions; + let layout_digest = self.syncer.system.cluster_layout().sync_digest(); + if layout_digest != self.layout_digest { + self.layout_digest = layout_digest; info!( - "({}) Layout versions changed (max={}, ack={}, min stored={}), adding full sync to syncer todo list", + "({}) Layout versions changed ({:?}), adding full sync to syncer todo list", F::TABLE_NAME, - layout_versions.0, - layout_versions.1, - layout_versions.2 + layout_digest, ); self.add_full_sync(); } -- cgit v1.2.3