diff options
Diffstat (limited to 'src/rpc')
-rw-r--r-- | src/rpc/Cargo.toml | 2 | ||||
-rw-r--r-- | src/rpc/layout/graph_algo.rs (renamed from src/rpc/graph_algo.rs) | 10 | ||||
-rw-r--r-- | src/rpc/layout/helper.rs | 298 | ||||
-rw-r--r-- | src/rpc/layout/history.rs | 306 | ||||
-rw-r--r-- | src/rpc/layout/manager.rs | 381 | ||||
-rw-r--r-- | src/rpc/layout/mod.rs | 478 | ||||
-rw-r--r-- | src/rpc/layout/test.rs | 158 | ||||
-rw-r--r-- | src/rpc/layout/version.rs (renamed from src/rpc/layout.rs) | 713 | ||||
-rw-r--r-- | src/rpc/lib.rs | 2 | ||||
-rw-r--r-- | src/rpc/replication_mode.rs | 119 | ||||
-rw-r--r-- | src/rpc/ring.rs | 164 | ||||
-rw-r--r-- | src/rpc/rpc_helper.rs | 492 | ||||
-rw-r--r-- | src/rpc/system.rs | 453 | ||||
-rw-r--r-- | src/rpc/system_metrics.rs | 75 |
14 files changed, 2421 insertions, 1230 deletions
diff --git a/src/rpc/Cargo.toml b/src/rpc/Cargo.toml index 823fab99..3e7ac635 100644 --- a/src/rpc/Cargo.toml +++ b/src/rpc/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "garage_rpc" -version = "0.9.3" +version = "0.10.0" authors = ["Alex Auvolat <alex@adnab.me>"] edition = "2018" license = "AGPL-3.0" diff --git a/src/rpc/graph_algo.rs b/src/rpc/layout/graph_algo.rs index d8c6c9b9..bd33e97f 100644 --- a/src/rpc/graph_algo.rs +++ b/src/rpc/layout/graph_algo.rs @@ -114,16 +114,6 @@ impl Graph<FlowEdge> { Ok(result) } - /// This function returns the value of the flow incoming to v. - pub fn get_inflow(&self, v: Vertex) -> Result<i64, String> { - 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<i64, String> { let idv = self.get_vertex_id(&v)?; diff --git a/src/rpc/layout/helper.rs b/src/rpc/layout/helper.rs new file mode 100644 index 00000000..2835347a --- /dev/null +++ b/src/rpc/layout/helper.rs @@ -0,0 +1,298 @@ +use std::collections::HashMap; +use std::ops::Deref; +use std::sync::atomic::{AtomicUsize, Ordering}; + +use serde::{Deserialize, Serialize}; + +use garage_util::data::*; + +use super::*; +use crate::replication_mode::*; + +#[derive(Debug, Clone, Serialize, Deserialize, Default, PartialEq, Eq)] +pub struct RpcLayoutDigest { + /// 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, +} + +#[derive(Debug, Clone, Copy, Eq, PartialEq)] +pub struct SyncLayoutDigest { + current: u64, + ack_map_min: u64, + min_stored: u64, +} + +pub struct LayoutHelper { + replication_factor: ReplicationFactor, + consistency_mode: ConsistencyMode, + layout: Option<LayoutHistory>, + + // cached values + ack_map_min: u64, + sync_map_min: u64, + + all_nodes: Vec<Uuid>, + all_nongateway_nodes: Vec<Uuid>, + + 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<u64, AtomicUsize>, +} + +impl Deref for LayoutHelper { + type Target = LayoutHistory; + fn deref(&self) -> &LayoutHistory { + self.layout() + } +} + +impl LayoutHelper { + pub fn new( + replication_factor: ReplicationFactor, + consistency_mode: ConsistencyMode, + mut layout: LayoutHistory, + mut ack_lock: HashMap<u64, AtomicUsize>, + ) -> 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 consistency_mode != ConsistencyMode::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(); + let all_nongateway_nodes = layout.get_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_among(&all_nodes, min_version); + + // 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 + // 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). + // 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_factor, &all_nongateway_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 { + replication_factor, + consistency_mode, + 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<F>(&mut self, f: F) -> bool + where + F: FnOnce(&mut LayoutHistory) -> bool, + { + let changed = f(self.layout.as_mut().unwrap()); + if changed { + *self = Self::new( + self.replication_factor, + self.consistency_mode, + 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 ack_map_min(&self) -> u64 { + self.ack_map_min + } + + pub fn sync_map_min(&self) -> u64 { + self.sync_map_min + } + + 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<Uuid> { + 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 fn storage_sets_of(&self, position: &Hash) -> Vec<Vec<Uuid>> { + self.layout() + .versions + .iter() + .map(|x| x.nodes_of(position, x.replication_factor).collect()) + .collect() + } + + pub fn storage_nodes_of(&self, position: &Hash) -> Vec<Uuid> { + 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 + } + + pub fn digest(&self) -> RpcLayoutDigest { + RpcLayoutDigest { + 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) { + // 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); + + 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.min_stored(); + 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) + .skip_while(|v| { + self.ack_lock + .get(v) + .map(|x| x.load(Ordering::Relaxed) == 0) + .unwrap_or(true) + }) + .next() + .unwrap_or(self.current().version) + } +} diff --git a/src/rpc/layout/history.rs b/src/rpc/layout/history.rs new file mode 100644 index 00000000..290f058d --- /dev/null +++ b/src/rpc/layout/history.rs @@ -0,0 +1,306 @@ +use std::collections::HashSet; + +use garage_util::crdt::{Crdt, Lww, LwwMap}; +use garage_util::data::*; +use garage_util::encode::nonversioned_encode; +use garage_util::error::*; + +use super::*; +use crate::replication_mode::*; + +impl LayoutHistory { + pub fn new(replication_factor: ReplicationFactor) -> Self { + let version = LayoutVersion::new(replication_factor.into()); + + let staging = LayoutStaging { + parameters: Lww::<LayoutParameters>::new(version.parameters), + roles: LwwMap::new(), + }; + + LayoutHistory { + versions: vec![version], + old_versions: vec![], + update_trackers: Default::default(), + staging: Lww::raw(0, staging), + } + } + + // ------------------ who stores what now? --------------- + + pub fn current(&self) -> &LayoutVersion { + self.versions.last().as_ref().unwrap() + } + + pub fn min_stored(&self) -> u64 { + self.versions.first().as_ref().unwrap().version + } + + pub fn get_all_nodes(&self) -> Vec<Uuid> { + if self.versions.len() == 1 { + self.versions[0].all_nodes().to_vec() + } else { + let set = self + .versions + .iter() + .flat_map(|x| x.all_nodes()) + .collect::<HashSet<_>>(); + set.into_iter().copied().collect::<Vec<_>>() + } + } + + pub(crate) fn get_all_nongateway_nodes(&self) -> Vec<Uuid> { + if self.versions.len() == 1 { + self.versions[0].nongateway_nodes().to_vec() + } else { + let set = self + .versions + .iter() + .flat_map(|x| x.nongateway_nodes()) + .collect::<HashSet<_>>(); + set.into_iter().copied().collect::<Vec<_>>() + } + } + + // ---- 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() { + 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 (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. + 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 { + 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]) { + 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); + } + } + + pub(crate) fn calculate_sync_map_min_with_quorum( + &self, + replication_factor: ReplicationFactor, + 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_factor.write_quorum(ConsistencyMode::Consistent); + + 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::<Vec<Uuid>>::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::<Vec<Uuid>>(); + 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::<Vec<_>>(); + 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()[..]) + } + + pub(crate) 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 { + let mut changed = false; + + // 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); + } + } 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 { + self.versions.push(v2.clone()); + changed = true; + } + } + + // Merge trackers + 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 = changed || self.staging != prev_staging; + } + + changed + } + + pub fn apply_staged_changes(mut self, version: Option<u64>) -> 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())); + } + } + } + + // Compute new version and add it to history + let (new_version, msg) = self + .current() + .clone() + .calculate_next_version(self.staging.get())?; + + self.versions.push(new_version); + self.cleanup_old_versions(); + + // Reset the staged layout changes + self.staging.update(LayoutStaging { + parameters: self.staging.get().parameters.clone(), + roles: LwwMap::new(), + }); + + Ok((self, msg)) + } + + pub fn revert_staged_changes(mut self) -> Result<Self, Error> { + self.staging.update(LayoutStaging { + parameters: Lww::new(self.current().parameters), + roles: LwwMap::new(), + }); + + Ok(self) + } + + pub fn check(&self) -> Result<(), String> { + // TODO: anything more ? + self.current().check() + } +} diff --git a/src/rpc/layout/manager.rs b/src/rpc/layout/manager.rs new file mode 100644 index 00000000..8a6eb1c3 --- /dev/null +++ b/src/rpc/layout/manager.rs @@ -0,0 +1,381 @@ +use std::collections::HashMap; +use std::sync::{atomic::Ordering, Arc, Mutex, RwLock, RwLockReadGuard}; +use std::time::Duration; + +use tokio::sync::Notify; + +use garage_net::endpoint::Endpoint; +use garage_net::peering::PeeringManager; +use garage_net::NodeID; + +use garage_util::config::Config; +use garage_util::data::*; +use garage_util::error::*; +use garage_util::persister::Persister; + +use super::*; +use crate::replication_mode::*; +use crate::rpc_helper::*; +use crate::system::*; + +pub struct LayoutManager { + node_id: Uuid, + replication_factor: ReplicationFactor, + persist_cluster_layout: Persister<LayoutHistory>, + + layout: Arc<RwLock<LayoutHelper>>, + pub(crate) change_notify: Arc<Notify>, + + table_sync_version: Mutex<HashMap<String, u64>>, + + pub(crate) rpc_helper: RpcHelper, + system_endpoint: Arc<Endpoint<SystemRpc, System>>, +} + +impl LayoutManager { + pub fn new( + config: &Config, + node_id: NodeID, + system_endpoint: Arc<Endpoint<SystemRpc, System>>, + peering: Arc<PeeringManager>, + replication_factor: ReplicationFactor, + consistency_mode: ConsistencyMode, + ) -> Result<Arc<Self>, Error> { + let persist_cluster_layout: Persister<LayoutHistory> = + Persister::new(&config.metadata_dir, "cluster_layout"); + + let cluster_layout = match persist_cluster_layout.load() { + Ok(x) => { + if x.current().replication_factor != 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.replication_factor() + ))); + } + x + } + Err(e) => { + info!( + "No valid previous cluster layout stored ({}), starting fresh.", + e + ); + LayoutHistory::new(replication_factor) + } + }; + + let mut cluster_layout = LayoutHelper::new( + replication_factor, + consistency_mode, + 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()); + + let rpc_helper = RpcHelper::new( + node_id.into(), + peering, + layout.clone(), + config.rpc_timeout_msec.map(Duration::from_millis), + ); + + Ok(Arc::new(Self { + node_id: node_id.into(), + replication_factor, + persist_cluster_layout, + layout, + change_notify, + table_sync_version: Mutex::new(HashMap::new()), + system_endpoint, + rpc_helper, + })) + } + + // ---- PUBLIC INTERFACE ---- + + pub fn layout(&self) -> RwLockReadGuard<'_, LayoutHelper> { + self.layout.read().unwrap() + } + + pub async fn update_cluster_layout( + self: &Arc<Self>, + layout: &LayoutHistory, + ) -> Result<(), Error> { + self.handle_advertise_cluster_layout(layout).await?; + 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<Self>, 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).min().unwrap(); + drop(table_sync_version); + + let mut layout = self.layout.write().unwrap(); + if layout.update(|l| l.update_trackers.sync_map.set_max(self.node_id, sync_until)) { + info!("sync_until updated to {}", sync_until); + self.broadcast_update(SystemRpc::AdvertiseClusterLayoutTrackers( + layout.update_trackers.clone(), + )); + } + } + + fn ack_new_version(self: &Arc<Self>) { + 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<Self>, position: &Hash) -> WriteLock<Vec<Vec<Uuid>>> { + let layout = self.layout(); + let version = layout.current().version; + let nodes = layout.storage_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<LayoutHistory> { + 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() { + if layout.update(|l| l.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."); + } + assert!(layout.digest() != prev_digest); + return Some(layout.clone()); + } + } + + None + } + + fn merge_layout_trackers(&self, adv: &UpdateTrackers) -> Option<UpdateTrackers> { + 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()); + } + } + + None + } + + async fn pull_cluster_layout(self: &Arc<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 { + if let Err(e) = self.handle_advertise_cluster_layout(&layout).await { + warn!("In pull_cluster_layout: {}", e); + } + } + } + + async fn pull_cluster_layout_trackers(self: &Arc<Self>, 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 cluster layout data to disk + async fn save_cluster_layout(&self) -> Result<(), Error> { + let layout = self.layout.read().unwrap().clone(); + self.persist_cluster_layout + .save_async(&layout) + .await + .expect("Cannot save current cluster layout"); + Ok(()) + } + + fn broadcast_update(self: &Arc<Self>, 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); + } + } + }); + } + + // ---- RPC HANDLERS ---- + + pub(crate) fn handle_advertise_status(self: &Arc<Self>, from: Uuid, remote: &RpcLayoutDigest) { + 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.trackers_hash != local.trackers_hash { + tokio::spawn({ + let this = self.clone(); + async move { this.pull_cluster_layout_trackers(from).await } + }); + } + } + + pub(crate) fn handle_pull_cluster_layout(&self) -> SystemRpc { + let layout = self.layout.read().unwrap().clone(); + 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<Self>, + adv: &LayoutHistory, + ) -> Result<SystemRpc, Error> { + 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.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.replication_factor() + ); + error!("{}", msg); + return Err(Error::Message(msg)); + } + + 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?; + } + + Ok(SystemRpc::Ok) + } + + pub(crate) async fn handle_advertise_cluster_layout_trackers( + self: &Arc<Self>, + trackers: &UpdateTrackers, + ) -> Result<SystemRpc, Error> { + 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)); + self.save_cluster_layout().await?; + } + + Ok(SystemRpc::Ok) + } +} + +// ---- ack lock ---- + +pub struct WriteLock<T> { + layout_version: u64, + layout_manager: Arc<LayoutManager>, + value: T, +} + +impl<T> WriteLock<T> { + fn new(version: u64, layout_manager: &Arc<LayoutManager>, value: T) -> Self { + Self { + layout_version: version, + layout_manager: layout_manager.clone(), + value, + } + } +} + +impl<T> AsRef<T> for WriteLock<T> { + fn as_ref(&self) -> &T { + &self.value + } +} + +impl<T> AsMut<T> for WriteLock<T> { + fn as_mut(&mut self) -> &mut T { + &mut self.value + } +} + +impl<T> Drop for WriteLock<T> { + 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 new file mode 100644 index 00000000..33676c37 --- /dev/null +++ b/src/rpc/layout/mod.rs @@ -0,0 +1,478 @@ +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 version; + +#[cfg(test)] +mod test; + +pub mod manager; + +// ---- re-exports ---- + +pub use helper::{LayoutHelper, RpcLayoutDigest, SyncLayoutDigest}; +pub use manager::WriteLock; +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; + +// ======== 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<Uuid, NodeRoleV>, + + // see comments in v010::ClusterLayout + pub node_id_vec: Vec<Uuid>, + #[serde(with = "serde_bytes")] + pub ring_assignation_data: Vec<CompactNodeType>, + + /// Role changes which are staged for the next version of the layout + pub staging: LwwMap<Uuid, NodeRoleV>, + pub staging_hash: Hash, + } + + #[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Debug, Serialize, Deserialize)] + pub struct NodeRoleV(pub Option<NodeRole>); + + /// 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<u64>, + /// A set of tags to recognize the node + pub tags: Vec<String>, + } + + 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<Uuid, NodeRoleV>, + + // see comments in v010::ClusterLayout + pub node_id_vec: Vec<Uuid>, + #[serde(with = "serde_bytes")] + pub ring_assignment_data: Vec<CompactNodeType>, + + /// Parameters to be used in the next partition assignment computation. + pub staging_parameters: Lww<LayoutParameters>, + /// Role changes which are staged for the next version of the layout + pub staging_roles: LwwMap<Uuid, NodeRoleV>, + 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<Uuid, NodeRoleV>, + mul: u64, + ) -> LwwMap<Uuid, NodeRoleV> { + 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<LayoutVersion>, + /// 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<LayoutVersion>, + + /// Update trackers + pub update_trackers: UpdateTrackers, + + /// Staged changes for the next version + pub staging: Lww<LayoutStaging>, + } + + /// 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<Uuid, NodeRoleV>, + /// 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<Uuid>, + /// 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<CompactNodeType>, + } + + /// 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<LayoutParameters>, + /// Role changes which are staged for the next version of the layout + pub roles: LwwMap<Uuid, NodeRoleV>, + } + + /// 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<Uuid, u64>); + + 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::<BTreeMap<Uuid, u64>>(), + ); + 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, + }, + 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<Self, Self::Err> { + match s { + "none" | "max" | "maximum" => Ok(ZoneRedundancy::Maximum), + x => { + let v = x + .parse::<usize>() + .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/test.rs b/src/rpc/layout/test.rs new file mode 100644 index 00000000..fcbb9dfc --- /dev/null +++ b/src/rpc/layout/test.rs @@ -0,0 +1,158 @@ +use std::cmp::min; +use std::collections::HashMap; + +use garage_util::crdt::Crdt; +use garage_util::error::*; + +use crate::layout::*; +use crate::replication_mode::ReplicationFactor; + +// 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<bool, Error> { + let over_size = cl.partition_size + 1; + let mut zone_token = HashMap::<String, usize>::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.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), + ); + } + + // 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), + }); +} + +#[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(ReplicationFactor::new(3).unwrap()); + 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.rs b/src/rpc/layout/version.rs index e02a180b..ee4b2821 100644 --- a/src/rpc/layout.rs +++ b/src/rpc/layout/version.rs @@ -1,375 +1,55 @@ -use std::cmp::Ordering; 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, Crdt, Lww, LwwMap}; +use garage_util::crdt::{Crdt, LwwMap}; use garage_util::data::*; -use garage_util::encode::nonversioned_encode; use garage_util::error::*; -use crate::graph_algo::*; - -use crate::ring::*; - -use std::convert::TryInto; - -const NB_PARTITIONS: usize = 1usize << PARTITION_BITS; +use super::graph_algo::*; +use super::*; // The Message type will be used to collect information on the algorithm. -type Message = Vec<String>; - -mod v08 { - use crate::ring::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<Uuid, NodeRoleV>, - - /// 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<Uuid>, - /// the assignation of data partitions to node, the values - /// are indices in node_id_vec - #[serde(with = "serde_bytes")] - pub ring_assignation_data: Vec<CompactNodeType>, - - /// Role changes which are staged for the next version of the layout - pub staging: LwwMap<Uuid, NodeRoleV>, - pub staging_hash: Hash, - } - - #[derive(PartialEq, Eq, PartialOrd, Ord, Clone, Debug, Serialize, Deserialize)] - pub struct NodeRoleV(pub Option<NodeRole>); - - /// 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<u64>, - /// A set of tags to recognize the node - pub tags: Vec<String>, - } - - impl garage_util::migrate::InitialFormat for ClusterLayout {} -} - -mod v09 { - use super::v08; - use crate::ring::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<Uuid, NodeRoleV>, - - /// see comment in v08::ClusterLayout - pub node_id_vec: Vec<Uuid>, - /// see comment in v08::ClusterLayout - #[serde(with = "serde_bytes")] - pub ring_assignment_data: Vec<CompactNodeType>, - - /// Parameters to be used in the next partition assignment computation. - pub staging_parameters: Lww<LayoutParameters>, - /// Role changes which are staged for the next version of the layout - pub staging_roles: LwwMap<Uuid, NodeRoleV>, - pub staging_hash: Hash, - } +pub type Message = Vec<String>; - /// 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<Uuid, NodeRoleV>, - mul: u64, - ) -> LwwMap<Uuid, NodeRoleV> { - 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<Self, Self::Err> { - match s { - "none" | "max" | "maximum" => Ok(ZoneRedundancy::Maximum), - x => { - let v = x - .parse::<usize>() - .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 { +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, }; - let staging_parameters = Lww::<LayoutParameters>::new(parameters); - - let empty_lwwmap = LwwMap::new(); - let mut ret = ClusterLayout { + LayoutVersion { version: 0, replication_factor, partition_size: 0, roles: LwwMap::new(), node_id_vec: Vec::new(), + nongateway_node_count: 0, 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 - } - - 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<u64>) -> 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(); + // ===================== accessors ====================== - let msg = self.calculate_partition_assignment()?; - - self.version += 1; - - Ok((self, msg)) - } - - pub fn revert_staged_changes(mut self, version: Option<u64>) -> Result<Self, 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.staging_roles.clear(); - self.staging_parameters.update(self.parameters); - self.staging_hash = self.calculate_staging_hash(); - - self.version += 1; - - Ok(self) - } - - /// 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[..] } - pub fn num_nodes(&self) -> usize { - self.node_id_vec.len() + /// 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] } - /// 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), @@ -377,41 +57,23 @@ 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<Uuid> { - let mut result = Vec::<Uuid>::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<String, Error> { - 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<u64, Error> { + /// Returns the capacity of a node in the layout, if it has one + pub fn get_node_capacity(&self, uuid: &Uuid) -> Option<u64> { 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, + } + } + + /// 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, } } @@ -435,17 +97,65 @@ To know the correct value of the new layout version, invoke `garage layout show` )) } + /// 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) -> impl Iterator<Item = (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)) + }) + } + + /// Return the n servers in which data for this hash should be replicated + pub fn nodes_of(&self, position: &Hash, n: usize) -> impl Iterator<Item = Uuid> + '_ { + assert_eq!(n, self.replication_factor); + + let data = &self.ring_assignment_data; + + 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!"); + &[] + }; + + partition_nodes + .iter() + .map(move |i| self.node_id_vec[*i as usize]) + } + + // ===================== internal information extractors ====================== + + 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<u64, Error> { + fn get_total_capacity(&self) -> u64 { 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.expect_get_node_capacity(uuid); } - Ok(total_capacity) + total_capacity } /// 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 => { @@ -465,10 +175,14 @@ To know the correct value of the new layout version, invoke `garage layout show` /// (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 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 @@ -486,16 +200,6 @@ To know the correct value of the new layout version, invoke `garage layout show` 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 @@ -524,10 +228,7 @@ To know the correct value of the new layout version, invoke `garage layout show` // 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::<Vec<_>>(); if zones_of_p.iter().unique().count() < zone_redundancy { return Err(format!( @@ -546,7 +247,7 @@ To know the correct value of the new layout version, invoke `garage layout show` 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 ({})", @@ -574,12 +275,24 @@ To know the correct value of the new layout version, invoke `garage layout show` Ok(()) } -} -// ==================================================================================== + // ================== 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)) + } -// 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 @@ -609,12 +322,12 @@ impl ClusterLayout { // 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 { + 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 { @@ -712,12 +425,14 @@ impl ClusterLayout { .map(|(k, _, _)| *k) .collect(); - let mut new_node_id_vec = Vec::<Uuid>::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); - let old_node_id_vec = self.node_id_vec.clone(); - self.node_id_vec = new_node_id_vec.clone(); + 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 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 @@ -756,7 +471,7 @@ impl ClusterLayout { } } - // We write the ring + // We clear the ring assignemnt data self.ring_assignment_data = Vec::<CompactNodeType>::new(); Ok(Some(old_assignment)) @@ -764,7 +479,9 @@ impl ClusterLayout { /// This function generates ids for the zone of the nodes appearing in /// self.node_id_vec. - fn generate_nongateway_zone_ids(&self) -> Result<(Vec<String>, HashMap<String, usize>), Error> { + pub(crate) fn generate_nongateway_zone_ids( + &self, + ) -> Result<(Vec<String>, HashMap<String, usize>), Error> { let mut id_to_zone = Vec::<String>::new(); let mut zone_to_id = HashMap::<String, usize>::new(); @@ -797,7 +514,7 @@ impl ClusterLayout { } 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, @@ -846,7 +563,7 @@ impl ClusterLayout { zone_redundancy: usize, ) -> Result<Graph<FlowEdge>, Error> { let vertices = - ClusterLayout::generate_graph_vertices(zone_to_id.len(), self.nongateway_nodes().len()); + LayoutVersion::generate_graph_vertices(zone_to_id.len(), self.nongateway_nodes().len()); let mut g = Graph::<FlowEdge>::new(&vertices); let nb_zones = zone_to_id.len(); for p in 0..NB_PARTITIONS { @@ -866,8 +583,8 @@ 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_capacity = self.expect_get_node_capacity(&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)) { @@ -913,7 +630,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.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()?; @@ -933,7 +650,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.expect_get_node_zone(&self.node_id_vec[*n])]; cost.insert((Vertex::PZ(p, node_zone), Vertex::N(*n)), -1); } } @@ -988,7 +705,7 @@ impl ClusterLayout { 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} %)", @@ -1035,7 +752,7 @@ impl ClusterLayout { let mut old_zones_of_p = Vec::<usize>::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; @@ -1077,7 +794,7 @@ impl ClusterLayout { for z in 0..id_to_zone.len() { let mut nodes_of_z = Vec::<usize>::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); } } @@ -1091,13 +808,13 @@ impl ClusterLayout { 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}%)", @@ -1127,167 +844,3 @@ impl ClusterLayout { 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<bool, Error> { - let over_size = cl.partition_size + 1; - let mut zone_token = HashMap::<String, usize>::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<u8>, - node_capacity_vec: &Vec<u64>, - node_zone_vec: &Vec<String>, - 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/lib.rs b/src/rpc/lib.rs index a5f8fc6e..b5b31c05 100644 --- a/src/rpc/lib.rs +++ b/src/rpc/lib.rs @@ -11,10 +11,8 @@ mod consul; #[cfg(feature = "kubernetes-discovery")] 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/replication_mode.rs b/src/rpc/replication_mode.rs index e244e063..a3a94085 100644 --- a/src/rpc/replication_mode.rs +++ b/src/rpc/replication_mode.rs @@ -1,57 +1,94 @@ -#[derive(Clone, Copy)] -pub enum ReplicationMode { - None, - TwoWay, - TwoWayDangerous, - ThreeWay, - ThreeWayDegraded, - ThreeWayDangerous, +use garage_util::config::Config; +use garage_util::crdt::AutoCrdt; +use garage_util::error::*; +use serde::{Deserialize, Serialize}; + +#[derive(Debug, Clone, Copy, PartialEq, Serialize, Deserialize)] +#[serde(transparent)] +pub struct ReplicationFactor(usize); + +#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Default, Serialize, Deserialize)] +#[serde(rename_all = "lowercase")] +pub enum ConsistencyMode { + /// Read- and Write-quorum are 1 + Dangerous, + /// Read-quorum is 1 + Degraded, + /// Read- and Write-quorum are determined for read-after-write-consistency + #[default] + Consistent, } -impl ReplicationMode { - pub fn parse(v: &str) -> Option<Self> { - match v { - "none" | "1" => Some(Self::None), - "2" => Some(Self::TwoWay), - "2-dangerous" => Some(Self::TwoWayDangerous), - "3" => Some(Self::ThreeWay), - "3-degraded" => Some(Self::ThreeWayDegraded), - "3-dangerous" => Some(Self::ThreeWayDangerous), - _ => None, - } +impl ConsistencyMode { + pub fn parse(s: &str) -> Option<Self> { + serde_json::from_value(serde_json::Value::String(s.to_string())).ok() } +} + +impl AutoCrdt for ConsistencyMode { + const WARN_IF_DIFFERENT: bool = true; +} - pub fn control_write_max_faults(&self) -> usize { - match self { - Self::None => 0, - _ => 1, +impl ReplicationFactor { + pub fn new(replication_factor: usize) -> Option<Self> { + if replication_factor < 1 { + None + } else { + Some(Self(replication_factor)) } } pub fn replication_factor(&self) -> usize { - match self { - Self::None => 1, - Self::TwoWay | Self::TwoWayDangerous => 2, - Self::ThreeWay | Self::ThreeWayDegraded | Self::ThreeWayDangerous => 3, - } + self.0 } - pub fn read_quorum(&self) -> usize { - match self { - Self::None => 1, - Self::TwoWay | Self::TwoWayDangerous => 1, - Self::ThreeWay => 2, - Self::ThreeWayDegraded | Self::ThreeWayDangerous => 1, + pub fn read_quorum(&self, consistency_mode: ConsistencyMode) -> usize { + match consistency_mode { + ConsistencyMode::Dangerous | ConsistencyMode::Degraded => 1, + ConsistencyMode::Consistent => self.replication_factor().div_ceil(2), } } - pub fn write_quorum(&self) -> usize { - match self { - Self::None => 1, - Self::TwoWay => 2, - Self::TwoWayDangerous => 1, - Self::ThreeWay | Self::ThreeWayDegraded => 2, - Self::ThreeWayDangerous => 1, + pub fn write_quorum(&self, consistency_mode: ConsistencyMode) -> usize { + match consistency_mode { + ConsistencyMode::Dangerous => 1, + ConsistencyMode::Degraded | ConsistencyMode::Consistent => { + (self.replication_factor() + 1) - self.read_quorum(ConsistencyMode::Consistent) + } } } } + +impl std::convert::From<ReplicationFactor> for usize { + fn from(replication_factor: ReplicationFactor) -> usize { + replication_factor.0 + } +} + +pub fn parse_replication_mode( + config: &Config, +) -> Result<(ReplicationFactor, ConsistencyMode), Error> { + match (&config.replication_mode, config.replication_factor, config.consistency_mode.as_str()) { + (Some(replication_mode), None, "consistent") => { + tracing::warn!("Legacy config option replication_mode in use. Please migrate to replication_factor and consistency_mode"); + let parsed_replication_mode = match replication_mode.as_str() { + "1" | "none" => Some((ReplicationFactor(1), ConsistencyMode::Consistent)), + "2" => Some((ReplicationFactor(2), ConsistencyMode::Consistent)), + "2-dangerous" => Some((ReplicationFactor(2), ConsistencyMode::Dangerous)), + "3" => Some((ReplicationFactor(3), ConsistencyMode::Consistent)), + "3-degraded" => Some((ReplicationFactor(3), ConsistencyMode::Degraded)), + "3-dangerous" => Some((ReplicationFactor(3), ConsistencyMode::Dangerous)), + _ => None, + }; + Some(parsed_replication_mode.ok_or_message("Invalid replication_mode in config file.")?) + }, + (None, Some(replication_factor), consistency_mode) => { + let replication_factor = ReplicationFactor::new(replication_factor) + .ok_or_message("Invalid replication_factor in config file.")?; + let consistency_mode = ConsistencyMode::parse(consistency_mode) + .ok_or_message("Invalid consistency_mode in config file.")?; + Some((replication_factor, consistency_mode)) + } + _ => None, + }.ok_or_message("Either the legacy replication_mode or replication_level and consistency_mode can be set, not both.") +} 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<Uuid>, - - // The list of entries in the ring - ring: Vec<RingEntry>, -} - -// 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::<Vec<_>>(); - - 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<Uuid> { - 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::<Vec<_>>() - } -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_ring_entry_size() { - assert_eq!(std::mem::size_of::<RingEntry>(), 8); - } -} diff --git a/src/rpc/rpc_helper.rs b/src/rpc/rpc_helper.rs index c46e577f..977c6ed8 100644 --- a/src/rpc/rpc_helper.rs +++ b/src/rpc/rpc_helper.rs @@ -1,12 +1,12 @@ //! Contain structs related to making RPCs -use std::sync::Arc; +use std::collections::HashMap; +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::{ @@ -26,8 +26,8 @@ use garage_util::data::*; use garage_util::error::Error; use garage_util::metrics::RecordDuration; +use crate::layout::{LayoutHelper, LayoutHistory}; use crate::metrics::RpcMetrics; -use crate::ring::Ring; // Default RPC timeout = 5 minutes const DEFAULT_TIMEOUT: Duration = Duration::from_secs(300); @@ -36,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<usize>, - /// Should requests be dropped after enough response are received - pub rs_interrupt_after_quorum: bool, + rs_quorum: Option<usize>, + /// Send all requests at once + rs_send_all_at_once: Option<bool>, /// Request priority - pub rs_priority: RequestPriority, + rs_priority: RequestPriority, /// Custom timeout for this request rs_timeout: Timeout, } @@ -57,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, } @@ -67,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 @@ -91,7 +90,7 @@ pub struct RpcHelper(Arc<RpcHelperInner>); struct RpcHelperInner { our_node_id: Uuid, peering: Arc<PeeringManager>, - ring: watch::Receiver<Arc<Ring>>, + layout: Arc<RwLock<LayoutHelper>>, metrics: RpcMetrics, rpc_timeout: Duration, } @@ -100,7 +99,7 @@ impl RpcHelper { pub(crate) fn new( our_node_id: Uuid, peering: Arc<PeeringManager>, - ring: watch::Receiver<Arc<Ring>>, + layout: Arc<RwLock<LayoutHelper>>, rpc_timeout: Option<Duration>, ) -> Self { let metrics = RpcMetrics::new(); @@ -108,7 +107,7 @@ impl RpcHelper { Self(Arc::new(RpcHelperInner { our_node_id, peering, - ring, + layout, metrics, rpc_timeout: rpc_timeout.unwrap_or(DEFAULT_TIMEOUT), })) @@ -130,6 +129,12 @@ impl RpcHelper { N: IntoReq<M> + Send, H: StreamingEndpointHandler<M>, { + 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)), @@ -141,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 { @@ -183,12 +189,17 @@ impl RpcHelper { N: IntoReq<M>, H: StreamingEndpointHandler<M>, { + 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(garage_net::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() @@ -220,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<M, N, H, S>( &self, endpoint: &Arc<Endpoint<M, H>>, @@ -236,31 +263,24 @@ 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!( + "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))); 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<M, N, H, S>( + async fn try_call_many_inner<M, N, H, S>( &self, endpoint: &Arc<Endpoint<M, H>>, to: &[Uuid], @@ -274,129 +294,238 @@ impl RpcHelper { H: StreamingEndpointHandler<M> + 'static, S: Send + 'static, { - let msg = msg.into_req().map_err(garage_net::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. + + // 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(&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 // 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(garage_net::error::Error::from)?; + let mut requests = request_order.into_iter().map(|to| { 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 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::<Vec<_>>(); - 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(fut) = requests.next() { + resp_stream.push(fut) + } else { + break; + } + } + + if successes.len() + resp_stream.len() < quorum { + // We know we won't ever reach quorum + 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; - } + // Wait for one request to terminate + match resp_stream.next().await.unwrap() { + Ok(msg) => { + successes.push(msg); } - assert!(!resp_stream.is_empty()); // because of loop invariants - - // Wait for one request to terminate - match resp_stream.next().await.unwrap().unwrap() { - Ok(msg) => { - successes.push(msg); - } - Err(e) => { - errors.push(e); - } + Err(e) => { + errors.push(e); } } + } + + if successes.len() >= quorum { + Ok(successes) } 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::<FuturesUnordered<_>>(); - - 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); - } - } - } + let errors = errors.iter().map(|e| format!("{}", e)).collect::<Vec<_>>(); + Err(Error::Quorum( + quorum, + None, + successes.len(), + to.len(), + errors, + )) + } + } + + /// 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<M, N, H, S>( + &self, + endpoint: &Arc<Endpoint<M, H>>, + to_sets: &[Vec<Uuid>], + msg: N, + strategy: RequestStrategy, + ) -> Result<Vec<S>, Error> + where + M: Rpc<Response = Result<S, Error>> + 'static, + N: IntoReq<M>, + H: StreamingEndpointHandler<M> + 'static, + S: Send + 'static, + { + let quorum = strategy + .rs_quorum + .expect("internal error: missing quorum value in try_write_many_sets"); + + let tracer = opentelemetry::global::tracer("garage"); + let span_name = format!( + "RPC [{}] try_write_many_sets (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<M, N, H, S>( + &self, + endpoint: &Arc<Endpoint<M, H>>, + to_sets: &[Vec<Uuid>], + msg: N, + strategy: RequestStrategy, + quorum: usize, + ) -> Result<Vec<S>, Error> + where + M: Rpc<Response = Result<S, Error>> + 'static, + N: IntoReq<M>, + H: StreamingEndpointHandler<M> + 'static, + S: Send + 'static, + { + // 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(garage_net::error::Error::from)?; + let requests = result_tracker.nodes.keys().map(|peer| { + let self2 = self.clone(); + let msg = msg.clone(); + let endpoint2 = endpoint.clone(); + let to = *peer; + async move { (to, self2.call(&endpoint2, to, msg, strategy).await) } + }); + let mut resp_stream = requests.collect::<FuturesUnordered<_>>(); + + // 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 + result_tracker.register_result(node, resp); - 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. + // If we have a quorum of ok in all quorum sets, then it's a success! + if result_tracker.all_quorums_ok() { + // Continue all other requets in background tokio::spawn(async move { - resp_stream.collect::<Vec<Result<_, _>>>().await; + resp_stream.collect::<Vec<(Uuid, Result<_, _>)>>().await; }); + + 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 result_tracker.too_many_failures() { + break; } } - if successes.len() >= quorum { - Ok(successes) - } else { - let errors = errors.iter().map(|e| format!("{}", e)).collect::<Vec<_>>(); - Err(Error::Quorum(quorum, successes.len(), to.len(), errors)) + // 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 + 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<Uuid> { + 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.sync_map_min() { + 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 } - pub fn request_order(&self, nodes: &[Uuid]) -> Vec<Uuid> { + fn request_order( + &self, + layout: &LayoutHistory, + nodes: impl Iterator<Item = Uuid>, + ) -> Vec<Uuid> { // Retrieve some status variables that we will use to sort requests let peer_list = self.0.peering.get_peer_list(); - let ring: Arc<Ring> = self.0.ring.borrow().clone(); - let our_zone = match ring.layout.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: @@ -405,22 +534,18 @@ 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 ring.layout.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()) .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::<Vec<_>>(); @@ -434,3 +559,108 @@ impl RpcHelper { .collect::<Vec<_>>() } } + +// ------- utility for tracking successes/errors among write sets -------- + +pub struct QuorumSetResultTracker<S, E> { + /// The set of nodes and the index of the quorum sets they belong to + pub nodes: HashMap<Uuid, Vec<usize>>, + /// The quorum value, i.e. number of success responses to await in each set + pub quorum: usize, + + /// The success responses received + pub successes: Vec<(Uuid, S)>, + /// The error responses received + pub failures: Vec<(Uuid, E)>, + + /// 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<S, E> QuorumSetResultTracker<S, E> +where + E: std::fmt::Display, +{ + pub fn new<A>(sets: &[A], quorum: usize) -> Self + where + A: AsRef<[Uuid]>, + { + let mut nodes = HashMap::<Uuid, Vec<usize>>::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, + 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::<Vec<_>>() + .into_boxed_slice(), + } + } + + pub fn register_result(&mut self, node: Uuid, result: Result<S, E>) { + 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<S> { + self.successes + .into_iter() + .map(|(_, x)| x) + .collect::<Vec<_>>() + } + + pub fn quorum_error(self) -> Error { + let errors = self + .failures + .iter() + .map(|(n, e)| format!("{:?}: {}", n, e)) + .collect::<Vec<_>>(); + Error::Quorum( + self.quorum, + Some(self.set_lens.len()), + self.successes.len(), + self.nodes.len(), + errors, + ) + } +} diff --git a/src/rpc/system.rs b/src/rpc/system.rs index 9e475717..54d589d2 100644 --- a/src/rpc/system.rs +++ b/src/rpc/system.rs @@ -1,9 +1,9 @@ //! 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}; -use std::sync::{Arc, RwLock}; +use std::sync::{Arc, RwLock, RwLockReadGuard}; use std::time::{Duration, Instant}; use arc_swap::ArcSwapOption; @@ -12,8 +12,7 @@ use futures::join; use serde::{Deserialize, Serialize}; use sodiumoxide::crypto::sign::ed25519; use tokio::select; -use tokio::sync::watch; -use tokio::sync::Mutex; +use tokio::sync::{watch, Notify}; use garage_net::endpoint::{Endpoint, EndpointHandler}; use garage_net::message::*; @@ -33,9 +32,10 @@ use garage_util::time::*; use crate::consul::ConsulDiscovery; #[cfg(feature = "kubernetes-discovery")] use crate::kubernetes::*; -use crate::layout::*; +use crate::layout::{ + self, manager::LayoutManager, LayoutHelper, LayoutHistory, NodeRoleV, RpcLayoutDigest, +}; use crate::replication_mode::*; -use crate::ring::*; use crate::rpc_helper::*; use crate::system_metrics::*; @@ -46,10 +46,10 @@ 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"; +pub const SYSTEM_RPC_PATH: &str = "garage_rpc/system.rs/SystemRpc"; /// RPC messages related to membership #[derive(Debug, Serialize, Deserialize, Clone)] @@ -58,17 +58,22 @@ pub enum SystemRpc { Ok, /// Request to connect to a specific node (in <pubkey>@<host>:<port> format, pubkey = full-length node ID) 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(ClusterLayout), /// Get known nodes states GetKnownNodes, /// Return known nodes ReturnKnownNodes(Vec<KnownNodeInfo>), + + /// 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 { @@ -84,7 +89,6 @@ pub struct System { /// The id of this node pub id: Uuid, - persist_cluster_layout: Persister<ClusterLayout>, persist_peer_list: Persister<PeerList>, pub(crate) local_status: RwLock<NodeStatus>, @@ -92,9 +96,8 @@ pub struct System { pub netapp: Arc<NetApp>, peering: Arc<PeeringManager>, - pub rpc: RpcHelper, - system_endpoint: Arc<Endpoint<SystemRpc, System>>, + pub(crate) system_endpoint: Arc<Endpoint<SystemRpc, System>>, rpc_listen_addr: SocketAddr, rpc_public_addr: Option<SocketAddr>, @@ -105,14 +108,11 @@ pub struct System { #[cfg(feature = "kubernetes-discovery")] kubernetes_discovery: Option<KubernetesDiscoveryConfig>, - metrics: ArcSwapOption<SystemMetrics>, + pub layout_manager: Arc<LayoutManager>, - replication_mode: ReplicationMode, - pub(crate) replication_factor: usize, + metrics: ArcSwapOption<SystemMetrics>, - /// The ring - pub ring: watch::Receiver<Arc<Ring>>, - update_ring: Mutex<watch::Sender<Arc<Ring>>>, + pub(crate) replication_factor: ReplicationFactor, /// Path to metadata directory pub metadata_dir: PathBuf, @@ -123,14 +123,13 @@ pub struct System { #[derive(Debug, Clone, Serialize, Deserialize)] pub struct NodeStatus { /// Hostname of the node - pub hostname: String, + pub hostname: Option<String>, /// Replication factor configured on the node pub replication_factor: usize, - /// Cluster layout version - pub cluster_layout_version: u64, - /// Hash of cluster layout staging data - pub cluster_layout_staging_hash: Hash, + + /// Cluster layout digest + pub layout_digest: RpcLayoutDigest, /// Disk usage on partition containing metadata directory (tuple: `(avail, total)`) #[serde(default)] @@ -243,11 +242,11 @@ impl System { /// Create this node's membership manager pub fn new( network_key: NetworkKey, - replication_mode: ReplicationMode, + replication_factor: ReplicationFactor, + consistency_mode: ConsistencyMode, config: &Config, ) -> Result<Arc<Self>, 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!( @@ -255,83 +254,39 @@ impl System { hex::encode(&node_key.public_key()[..8]) ); - let persist_cluster_layout: Persister<ClusterLayout> = - 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 { - 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, - replication_factor - ))); - } - x - } - Err(e) => { - info!( - "No valid previous cluster layout stored ({}), starting fresh.", - e - ); - ClusterLayout::new(replication_factor) - } - }; - - let mut local_status = NodeStatus::initial(replication_factor, &cluster_layout); - local_status.update_disk_usage(&config.metadata_dir, &config.data_dir); + let bind_outgoing_to = Some(config) + .filter(|x| x.rpc_bind_outgoing) + .map(|x| x.rpc_bind_addr.ip()); + let netapp = NetApp::new(GARAGE_VERSION_TAG, network_key, node_key, bind_outgoing_to); + let system_endpoint = netapp.endpoint(SYSTEM_RPC_PATH.into()); - let ring = Ring::new(cluster_layout, replication_factor); - let (update_ring, ring) = watch::channel(Arc::new(ring)); - - 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::<Vec<_>>(); - 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 bind_outgoing_to = Some(config) - .filter(|x| x.rpc_bind_outgoing) - .map(|x| x.rpc_bind_addr.ip()); - let netapp = NetApp::new(GARAGE_VERSION_TAG, network_key, node_key, bind_outgoing_to); let peering = PeeringManager::new(netapp.clone(), vec![], rpc_public_addr); if let Some(ping_timeout) = config.rpc_ping_timeout_msec { peering.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 layout_manager = LayoutManager::new( + config, + netapp.id, + system_endpoint.clone(), + peering.clone(), + replication_factor, + consistency_mode, + )?; + let mut local_status = NodeStatus::initial(replication_factor, &layout_manager); + local_status.update_disk_usage(&config.metadata_dir, &config.data_dir); + + // ---- if enabled, set up additionnal peer discovery methods ---- #[cfg(feature = "consul-discovery")] let consul_discovery = match &config.consul_discovery { Some(cfg) => Some( @@ -350,22 +305,15 @@ impl System { warn!("Kubernetes discovery is not enabled in this build."); } + // ---- almost done ---- let sys = Arc::new(System { id: netapp.id.into(), - persist_cluster_layout, persist_peer_list, local_status: RwLock::new(local_status), node_status: RwLock::new(HashMap::new()), netapp: netapp.clone(), peering: peering.clone(), - rpc: RpcHelper::new( - netapp.id.into(), - peering, - ring.clone(), - config.rpc_timeout_msec.map(Duration::from_millis), - ), system_endpoint, - replication_mode, replication_factor, rpc_listen_addr: config.rpc_bind_addr, rpc_public_addr, @@ -374,10 +322,9 @@ impl System { consul_discovery, #[cfg(feature = "kubernetes-discovery")] kubernetes_discovery: config.kubernetes_discovery.clone(), + layout_manager, metrics: ArcSwapOption::new(None), - ring, - update_ring: Mutex::new(update_ring), metadata_dir: config.metadata_dir.clone(), data_dir: config.data_dir.clone(), }); @@ -409,6 +356,20 @@ impl System { self.metrics.store(None); } + // ---- Public utilities / accessors ---- + + pub fn cluster_layout(&self) -> RwLockReadGuard<'_, LayoutHelper> { + self.layout_manager.layout() + } + + pub fn layout_notify(&self) -> Arc<Notify> { + self.layout_manager.change_notify.clone() + } + + pub fn rpc_helper(&self) -> &RpcHelper { + &self.layout_manager.rpc_helper + } + // ---- Administrative operations (directly available and // also available through RPC) ---- @@ -435,18 +396,6 @@ impl System { known_nodes } - pub fn get_cluster_layout(&self) -> ClusterLayout { - self.ring.borrow().layout.clone() - } - - pub async fn update_cluster_layout( - self: &Arc<Self>, - layout: &ClusterLayout, - ) -> 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 @@ -476,47 +425,65 @@ impl System { } pub fn health(&self) -> ClusterHealth { - let ring: Arc<_> = self.ring.borrow().clone(); - let quorum = self.replication_mode.write_quorum(); - let replication_factor = self.replication_factor; - + let quorum = self + .replication_factor + .write_quorum(ConsistencyMode::Consistent); + + // 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::<HashMap<Uuid, _>>(); 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::<Uuid>::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(); + + // 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::<Vec<_>>(); + let mut partitions_quorum = 0; + let mut partitions_all_ok = 0; + for (_, hash) in partitions.iter() { + 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.all(|mut set| set.all(|x| node_up(&x))); + if has_quorum { + partitions_quorum += 1; + } + if all_ok { + partitions_all_ok += 1; + } + } - let storage_nodes = ring - .layout - .roles - .items() - .iter() - .filter(|(_, _, v)| matches!(v, NodeRoleV(Some(r)) if r.capacity.is_some())) - .collect::<Vec<_>>(); - let storage_nodes_ok = storage_nodes - .iter() - .filter(|(x, _, _)| nodes.get(x).map(|n| n.is_up).unwrap_or(false)) - .count(); - - let partitions = ring.partitions(); - let partitions_n_up = partitions - .iter() - .map(|(_, h)| { - let pn = ring.get_nodes(h, ring.replication_factor); - pn.iter() - .filter(|x| nodes.get(x).map(|n| n.is_up).unwrap_or(false)) - .count() - }) - .collect::<Vec<usize>>(); - 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(); - + // 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 @@ -553,7 +520,7 @@ impl System { } }; - let hostname = self.local_status.read().unwrap().hostname.clone(); + let hostname = self.local_status.read().unwrap().hostname.clone().unwrap(); if let Err(e) = c .publish_consul_service(self.netapp.id, &hostname, rpc_public_addr) .await @@ -577,30 +544,16 @@ impl System { } }; - let hostname = self.local_status.read().unwrap().hostname.clone(); + let hostname = self.local_status.read().unwrap().hostname.clone().unwrap(); if let Err(e) = publish_kubernetes_node(k, self.netapp.id, &hostname, rpc_public_addr).await { error!("Error while publishing node to Kubernetes: {}", e); } } - /// Save network configuration to disc - async fn save_cluster_layout(&self) -> Result<(), Error> { - let ring: Arc<Ring> = self.ring.borrow().clone(); - self.persist_cluster_layout - .save_async(&ring.layout) - .await - .expect("Cannot save current cluster layout"); - Ok(()) - } - fn update_local_status(&self) { let mut local_status = self.local_status.write().unwrap(); - - let ring = self.ring.borrow(); - local_status.cluster_layout_version = ring.layout.version; - local_status.cluster_layout_staging_hash = ring.layout.staging_hash; - + local_status.layout_digest = self.layout_manager.layout().digest(); local_status.update_disk_usage(&self.metadata_dir, &self.data_dir); } @@ -611,11 +564,6 @@ impl System { Ok(SystemRpc::Ok) } - fn handle_pull_cluster_layout(&self) -> SystemRpc { - let ring = self.ring.borrow().clone(); - SystemRpc::AdvertiseClusterLayout(ring.layout.clone()) - } - fn handle_get_known_nodes(&self) -> SystemRpc { let known_nodes = self.get_known_nodes(); SystemRpc::ReturnKnownNodes(known_nodes) @@ -635,11 +583,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(self.clone().pull_cluster_layout(from)); - } + self.layout_manager + .handle_advertise_status(from, &info.layout_digest); drop(local_info); @@ -651,57 +596,6 @@ impl System { Ok(SystemRpc::Ok) } - async fn handle_advertise_cluster_layout( - self: &Arc<Self>, - adv: &ClusterLayout, - ) -> Result<SystemRpc, Error> { - if adv.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, - self.replication_factor - ); - error!("{}", msg); - return Err(Error::Message(msg)); - } - - let update_ring = self.update_ring.lock().await; - let mut layout: ClusterLayout = self.ring.borrow().layout.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(), - )); - } - - let ring = Ring::new(layout.clone(), self.replication_factor); - update_ring.send(Arc::new(ring))?; - drop(update_ring); - - 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<bool>) { while !*stop_signal.borrow() { let restart_at = Instant::now() + STATUS_EXCHANGE_INTERVAL; @@ -711,7 +605,7 @@ impl System { let local_status: NodeStatus = self.local_status.read().unwrap().clone(); let _ = self - .rpc + .rpc_helper() .broadcast( &self.system_endpoint, SystemRpc::AdvertiseStatus(local_status), @@ -736,10 +630,9 @@ impl System { .filter(|p| p.is_up()) .count(); - let not_configured = self.ring.borrow().layout.check().is_err(); - let no_peers = n_connected < self.replication_factor; - - let expected_n_nodes = self.ring.borrow().layout.num_nodes(); + let not_configured = self.cluster_layout().check().is_err(); + let no_peers = n_connected < self.replication_factor.into(); + let expected_n_nodes = self.cluster_layout().all_nodes().len(); let bad_peers = n_connected != expected_n_nodes; if not_configured || no_peers || bad_peers { @@ -791,8 +684,8 @@ impl System { // If the layout is configured, and we already have some connections // to other nodes in the cluster, we can skip trying to connect to // nodes that are not in the cluster layout. - let ring = self.ring.borrow(); - ping_list.retain(|(id, _)| ring.layout.node_ids().contains(&(*id).into())); + let layout = self.cluster_layout(); + ping_list.retain(|(id, _)| layout.all_nodes().contains(&(*id).into())); } for (node_id, node_addr) in ping_list { @@ -847,48 +740,49 @@ impl System { .save_async(&PeerList(peer_list)) .await } - - async fn pull_cluster_layout(self: Arc<Self>, 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<SystemRpc> for System { async fn handle(self: &Arc<Self>, msg: &SystemRpc, from: NodeID) -> Result<SystemRpc, Error> { 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()), + 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)), } } } impl NodeStatus { - fn initial(replication_factor: usize, layout: &ClusterLayout) -> Self { + fn initial(replication_factor: ReplicationFactor, layout_manager: &LayoutManager) -> Self { NodeStatus { - hostname: gethostname::gethostname() - .into_string() - .unwrap_or_else(|_| "<invalid utf-8>".to_string()), - replication_factor, - cluster_layout_version: layout.version, - cluster_layout_staging_hash: layout.staging_hash, + hostname: Some( + gethostname::gethostname() + .into_string() + .unwrap_or_else(|_| "<invalid utf-8>".to_string()), + ), + replication_factor: replication_factor.into(), + layout_digest: layout_manager.layout().digest(), meta_disk_avail: None, data_disk_avail: None, } @@ -896,10 +790,9 @@ impl NodeStatus { fn unknown() -> Self { NodeStatus { - hostname: "?".to_string(), + hostname: None, replication_factor: 0, - cluster_layout_version: 0, - cluster_layout_staging_hash: Hash::from([0u8; 32]), + layout_digest: Default::default(), meta_disk_avail: None, data_disk_avail: None, } @@ -952,6 +845,40 @@ fn get_default_ip() -> Option<IpAddr> { .map(|a| a.ip()) } +fn get_rpc_public_addr(config: &Config) -> Option<SocketAddr> { + 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::<Vec<_>>(); + 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/rpc/system_metrics.rs b/src/rpc/system_metrics.rs index ffbef6df..a64daec8 100644 --- a/src/rpc/system_metrics.rs +++ b/src/rpc/system_metrics.rs @@ -3,7 +3,6 @@ use std::time::{Duration, Instant}; use opentelemetry::{global, metrics::*, KeyValue}; -use crate::ring::Ring; use crate::system::{ClusterHealthStatus, System}; /// TableMetrics reference all counter used for metrics @@ -69,7 +68,7 @@ impl SystemMetrics { let replication_factor = system.replication_factor; meter .u64_value_observer("garage_replication_factor", move |observer| { - observer.observe(replication_factor as u64, &[]) + observer.observe(replication_factor.replication_factor() as u64, &[]) }) .with_description("Garage replication factor setting") .init() @@ -215,14 +214,14 @@ impl SystemMetrics { let system = system.clone(); meter .u64_value_observer("cluster_layout_node_connected", move |observer| { - let ring: Arc<Ring> = system.ring.borrow().clone(); + let layout = system.cluster_layout(); let nodes = system.get_known_nodes(); - for (id, _, config) in ring.layout.roles.items().iter() { - if let Some(role) = &config.0 { - let mut kv = vec![ - KeyValue::new("id", format!("{:?}", id)), - KeyValue::new("role_zone", role.zone.clone()), - ]; + for id in layout.all_nodes().iter() { + let mut kv = vec![KeyValue::new("id", format!("{:?}", id))]; + if let Some(role) = + layout.current().roles.get(id).and_then(|r| r.0.as_ref()) + { + kv.push(KeyValue::new("role_zone", role.zone.clone())); match role.capacity { Some(cap) => { kv.push(KeyValue::new("role_capacity", cap as i64)); @@ -232,24 +231,24 @@ impl SystemMetrics { kv.push(KeyValue::new("role_gateway", 1)); } } + } - let value; - if let Some(node) = nodes.iter().find(|n| n.id == *id) { - value = if node.is_up { 1 } else { 0 }; + let value; + if let Some(node) = nodes.iter().find(|n| n.id == *id) { // TODO: if we add address and hostname, and those change, we // get duplicate metrics, due to bad otel aggregation :( // Can probably be fixed when we upgrade opentelemetry // kv.push(KeyValue::new("address", node.addr.to_string())); // kv.push(KeyValue::new( - // "hostname", - // node.status.hostname.clone(), + // "hostname", + // node.status.hostname.clone(), // )); - } else { - value = 0; - } - - observer.observe(value, &kv); + value = if node.is_up { 1 } else { 0 }; + } else { + value = 0; } + + observer.observe(value, &kv); } }) .with_description("Connection status for nodes in the cluster layout") @@ -259,14 +258,14 @@ impl SystemMetrics { let system = system.clone(); meter .u64_value_observer("cluster_layout_node_disconnected_time", move |observer| { - let ring: Arc<Ring> = system.ring.borrow().clone(); + let layout = system.cluster_layout(); let nodes = system.get_known_nodes(); - for (id, _, config) in ring.layout.roles.items().iter() { - if let Some(role) = &config.0 { - let mut kv = vec![ - KeyValue::new("id", format!("{:?}", id)), - KeyValue::new("role_zone", role.zone.clone()), - ]; + for id in layout.all_nodes().iter() { + let mut kv = vec![KeyValue::new("id", format!("{:?}", id))]; + if let Some(role) = + layout.current().roles.get(id).and_then(|r| r.0.as_ref()) + { + kv.push(KeyValue::new("role_zone", role.zone.clone())); match role.capacity { Some(cap) => { kv.push(KeyValue::new("role_capacity", cap as i64)); @@ -276,19 +275,19 @@ impl SystemMetrics { kv.push(KeyValue::new("role_gateway", 1)); } } + } - if let Some(node) = nodes.iter().find(|n| n.id == *id) { - // TODO: see comment above - // kv.push(KeyValue::new("address", node.addr.to_string())); - // kv.push(KeyValue::new( - // "hostname", - // node.status.hostname.clone(), - // )); - if node.is_up { - observer.observe(0, &kv); - } else if let Some(secs) = node.last_seen_secs_ago { - observer.observe(secs, &kv); - } + if let Some(node) = nodes.iter().find(|n| n.id == *id) { + // TODO: see comment above + // kv.push(KeyValue::new("address", node.addr.to_string())); + // kv.push(KeyValue::new( + // "hostname", + // node.status.hostname.clone(), + // )); + if node.is_up { + observer.observe(0, &kv); + } else if let Some(secs) = node.last_seen_secs_ago { + observer.observe(secs, &kv); } } } |