From a48e2e0cb2bdc75e14dfde199dbca0a779b1316b Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 10 Jan 2023 10:30:59 +0100 Subject: K2V: Subscription to ranges of items --- src/model/k2v/item_table.rs | 2 +- src/model/k2v/mod.rs | 3 +- src/model/k2v/poll.rs | 50 --------------------- src/model/k2v/rpc.rs | 10 ++--- src/model/k2v/sub.rs | 107 ++++++++++++++++++++++++++++++++++++++++++++ 5 files changed, 115 insertions(+), 57 deletions(-) delete mode 100644 src/model/k2v/poll.rs create mode 100644 src/model/k2v/sub.rs (limited to 'src/model/k2v') diff --git a/src/model/k2v/item_table.rs b/src/model/k2v/item_table.rs index ce3e4129..a22df68a 100644 --- a/src/model/k2v/item_table.rs +++ b/src/model/k2v/item_table.rs @@ -11,7 +11,7 @@ use garage_table::*; use crate::index_counter::*; use crate::k2v::causality::*; -use crate::k2v::poll::*; +use crate::k2v::sub::*; pub const ENTRIES: &str = "entries"; pub const CONFLICTS: &str = "conflicts"; diff --git a/src/model/k2v/mod.rs b/src/model/k2v/mod.rs index f6a96151..83ad2512 100644 --- a/src/model/k2v/mod.rs +++ b/src/model/k2v/mod.rs @@ -2,5 +2,6 @@ pub mod causality; pub mod item_table; -pub mod poll; pub mod rpc; + +pub(crate) mod sub; diff --git a/src/model/k2v/poll.rs b/src/model/k2v/poll.rs deleted file mode 100644 index 93105207..00000000 --- a/src/model/k2v/poll.rs +++ /dev/null @@ -1,50 +0,0 @@ -use std::collections::HashMap; -use std::sync::Mutex; - -use serde::{Deserialize, Serialize}; -use tokio::sync::broadcast; - -use crate::k2v::item_table::*; - -#[derive(Debug, Hash, Clone, PartialEq, Eq, Serialize, Deserialize)] -pub struct PollKey { - pub partition: K2VItemPartition, - pub sort_key: String, -} - -#[derive(Default)] -pub struct SubscriptionManager { - subscriptions: Mutex>>, -} - -impl SubscriptionManager { - pub fn new() -> Self { - Self::default() - } - - pub fn subscribe(&self, key: &PollKey) -> broadcast::Receiver { - let mut subs = self.subscriptions.lock().unwrap(); - if let Some(s) = subs.get(key) { - s.subscribe() - } else { - let (tx, rx) = broadcast::channel(8); - subs.insert(key.clone(), tx); - rx - } - } - - pub fn notify(&self, item: &K2VItem) { - let key = PollKey { - partition: item.partition.clone(), - sort_key: item.sort_key.clone(), - }; - let mut subs = self.subscriptions.lock().unwrap(); - if let Some(s) = subs.get(&key) { - if s.send(item.clone()).is_err() { - // no more subscribers, remove channel from here - // (we will re-create it later if we need to subscribe again) - subs.remove(&key); - } - } - } -} diff --git a/src/model/k2v/rpc.rs b/src/model/k2v/rpc.rs index f64a7984..8860676b 100644 --- a/src/model/k2v/rpc.rs +++ b/src/model/k2v/rpc.rs @@ -27,7 +27,7 @@ use garage_table::{PartitionKey, Table}; use crate::k2v::causality::*; use crate::k2v::item_table::*; -use crate::k2v::poll::*; +use crate::k2v::sub::*; /// RPC messages for K2V #[derive(Debug, Serialize, Deserialize)] @@ -181,7 +181,7 @@ impl K2VRpcHandler { Ok(()) } - pub async fn poll( + pub async fn poll_item( &self, bucket_id: Uuid, partition_key: String, @@ -288,8 +288,8 @@ impl K2VRpcHandler { }) } - async fn handle_poll(&self, key: &PollKey, ct: &CausalContext) -> Result { - let mut chan = self.subscriptions.subscribe(key); + async fn handle_poll_item(&self, key: &PollKey, ct: &CausalContext) -> Result { + let mut chan = self.subscriptions.subscribe_item(key); let mut value = self .item_table @@ -326,7 +326,7 @@ impl EndpointHandler for K2VRpcHandler { } => { let delay = tokio::time::sleep(Duration::from_millis(*timeout_msec)); select! { - ret = self.handle_poll(key, causal_context) => ret.map(Some).map(K2VRpc::PollItemResponse), + ret = self.handle_poll_item(key, causal_context) => ret.map(Some).map(K2VRpc::PollItemResponse), _ = delay => Ok(K2VRpc::PollItemResponse(None)), } } diff --git a/src/model/k2v/sub.rs b/src/model/k2v/sub.rs new file mode 100644 index 00000000..c4273dba --- /dev/null +++ b/src/model/k2v/sub.rs @@ -0,0 +1,107 @@ +use std::collections::HashMap; +use std::sync::Mutex; + +use serde::{Deserialize, Serialize}; +use tokio::sync::broadcast; + +use crate::k2v::item_table::*; + +#[derive(Debug, Hash, Clone, PartialEq, Eq, Serialize, Deserialize)] +pub struct PollKey { + pub partition: K2VItemPartition, + pub sort_key: String, +} + +#[derive(Debug, Hash, Clone, PartialEq, Eq, Serialize, Deserialize)] +pub struct PollRange { + pub partition: K2VItemPartition, + pub prefix: Option, + pub start: Option, + pub end: Option, +} + +#[derive(Default)] +pub struct SubscriptionManager(Mutex); + +#[derive(Default)] +pub struct SubscriptionManagerInner { + item_subscriptions: HashMap>, + range_subscriptions: HashMap>, +} + +impl SubscriptionManager { + pub fn new() -> Self { + Self::default() + } + + pub fn subscribe_item(&self, key: &PollKey) -> broadcast::Receiver { + let mut inner = self.0.lock().unwrap(); + if let Some(s) = inner.item_subscriptions.get(key) { + s.subscribe() + } else { + let (tx, rx) = broadcast::channel(8); + inner.item_subscriptions.insert(key.clone(), tx); + rx + } + } + + pub fn subscribe_range(&self, key: &PollRange) -> broadcast::Receiver { + let mut inner = self.0.lock().unwrap(); + if let Some(s) = inner.range_subscriptions.get(key) { + s.subscribe() + } else { + let (tx, rx) = broadcast::channel(8); + inner.range_subscriptions.insert(key.clone(), tx); + rx + } + } + + pub fn notify(&self, item: &K2VItem) { + let mut inner = self.0.lock().unwrap(); + + // 1. Notify single item subscribers, + // removing subscriptions with no more listeners if any + let key = PollKey { + partition: item.partition.clone(), + sort_key: item.sort_key.clone(), + }; + if let Some(s) = inner.item_subscriptions.get(&key) { + if s.send(item.clone()).is_err() { + // no more subscribers, remove channel from here + // (we will re-create it later if we need to subscribe again) + inner.item_subscriptions.remove(&key); + } + } + + // 2. Notify range subscribers, + // removing subscriptions with no more listeners if any + inner.range_subscriptions.retain(|sub, chan| { + if sub.matches(&item) { + chan.send(item.clone()).is_ok() + } else { + chan.receiver_count() != 0 + } + }); + } +} + +impl PollRange { + fn matches(&self, item: &K2VItem) -> bool { + item.partition == self.partition + && self + .prefix + .as_ref() + .map(|x| item.sort_key.starts_with(x)) + .unwrap_or(true) + && self + .start + .as_ref() + .map(|x| item.sort_key >= *x) + .unwrap_or(true) + && self + .end + .as_ref() + .map(|x| item.sort_key < *x) + .unwrap_or(true) + } +} -- cgit v1.2.3 From 9f5419f465de3ddbb9afc91464d0eb00636049b9 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 10 Jan 2023 11:01:49 +0100 Subject: Make K2V item timestamps globally increasing on each node --- src/model/k2v/item_table.rs | 7 +++++-- src/model/k2v/rpc.rs | 45 +++++++++++++++++++++++++++++++++++++++++---- 2 files changed, 46 insertions(+), 6 deletions(-) (limited to 'src/model/k2v') diff --git a/src/model/k2v/item_table.rs b/src/model/k2v/item_table.rs index a22df68a..1ba9bb46 100644 --- a/src/model/k2v/item_table.rs +++ b/src/model/k2v/item_table.rs @@ -73,7 +73,8 @@ impl K2VItem { this_node: Uuid, context: &Option, new_value: DvvsValue, - ) { + node_ts: u64, + ) -> u64 { if let Some(context) = context { for (node, t_discard) in context.vector_clock.iter() { if let Some(e) = self.items.get_mut(node) { @@ -98,7 +99,9 @@ impl K2VItem { values: vec![], }); let t_prev = e.max_time(); - e.values.push((t_prev + 1, new_value)); + let t_new = std::cmp::max(t_prev + 1, node_ts + 1); + e.values.push((t_new, new_value)); + t_new } /// Extract the causality context of a K2V Item diff --git a/src/model/k2v/rpc.rs b/src/model/k2v/rpc.rs index 8860676b..e5497215 100644 --- a/src/model/k2v/rpc.rs +++ b/src/model/k2v/rpc.rs @@ -6,7 +6,8 @@ //! mean the vector clock gets much larger than needed). use std::collections::HashMap; -use std::sync::Arc; +use std::convert::TryInto; +use std::sync::{Arc, Mutex}; use std::time::Duration; use async_trait::async_trait; @@ -15,9 +16,12 @@ use futures::StreamExt; use serde::{Deserialize, Serialize}; use tokio::select; +use garage_db as db; + use garage_util::crdt::*; use garage_util::data::*; use garage_util::error::*; +use garage_util::time::now_msec; use garage_rpc::system::System; use garage_rpc::*; @@ -29,6 +33,8 @@ use crate::k2v::causality::*; use crate::k2v::item_table::*; use crate::k2v::sub::*; +const TIMESTAMP_KEY: &'static [u8] = b"timestamp"; + /// RPC messages for K2V #[derive(Debug, Serialize, Deserialize)] enum K2VRpc { @@ -59,6 +65,7 @@ impl Rpc for K2VRpc { pub struct K2VRpcHandler { system: Arc, item_table: Arc>, + local_timestamp_tree: Mutex, endpoint: Arc>, subscriptions: Arc, } @@ -66,14 +73,19 @@ pub struct K2VRpcHandler { impl K2VRpcHandler { pub fn new( system: Arc, + db: &db::Db, item_table: Arc>, subscriptions: Arc, ) -> Arc { + let local_timestamp_tree = db + .open_tree("k2v_local_timestamp") + .expect("Unable to open DB tree for k2v local timestamp"); let endpoint = system.netapp.endpoint("garage_model/k2v/Rpc".to_string()); let rpc_handler = Arc::new(Self { system, item_table, + local_timestamp_tree: Mutex::new(local_timestamp_tree), endpoint, subscriptions, }); @@ -273,9 +285,22 @@ impl K2VRpcHandler { } fn local_insert(&self, item: &InsertedItem) -> Result, Error> { + // Using a mutex on the local_timestamp_tree is not strictly necessary, + // but it helps to not try to do several inserts at the same time, + // which would create transaction conflicts and force many useless retries. + let local_timestamp_tree = self.local_timestamp_tree.lock().unwrap(); + + let now = now_msec(); + self.item_table .data - .update_entry_with(&item.partition, &item.sort_key, |ent| { + .update_entry_with(&item.partition, &item.sort_key, |tx, ent| { + let old_local_timestamp = tx + .get(&local_timestamp_tree, TIMESTAMP_KEY)? + .and_then(|x| x.try_into().ok()) + .map(u64::from_be_bytes) + .unwrap_or_default(); + let mut ent = ent.unwrap_or_else(|| { K2VItem::new( item.partition.bucket_id, @@ -283,8 +308,20 @@ impl K2VRpcHandler { item.sort_key.clone(), ) }); - ent.update(self.system.id, &item.causal_context, item.value.clone()); - ent + let new_local_timestamp = ent.update( + self.system.id, + &item.causal_context, + item.value.clone(), + std::cmp::max(old_local_timestamp, now), + ); + + tx.insert( + &local_timestamp_tree, + TIMESTAMP_KEY, + u64::to_be_bytes(new_local_timestamp), + )?; + + Ok(ent) }) } -- cgit v1.2.3 From 789540ca379efd710f1e2699fd458b4821f36a5b Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 10 Jan 2023 11:59:57 +0100 Subject: Type definition for range seen marker --- src/model/k2v/causality.rs | 39 ++++++++++++++++----- src/model/k2v/item_table.rs | 2 +- src/model/k2v/mod.rs | 1 + src/model/k2v/seen.rs | 85 +++++++++++++++++++++++++++++++++++++++++++++ 4 files changed, 117 insertions(+), 10 deletions(-) create mode 100644 src/model/k2v/seen.rs (limited to 'src/model/k2v') diff --git a/src/model/k2v/causality.rs b/src/model/k2v/causality.rs index 9a692870..665b02b2 100644 --- a/src/model/k2v/causality.rs +++ b/src/model/k2v/causality.rs @@ -1,3 +1,13 @@ +//! Implements a CausalContext, which is a set of timestamps for each +//! node -- a vector clock --, indicating that the versions with +//! timestamps <= these numbers have been seen and can be +//! overwritten by a subsequent write. +//! +//! The textual representation of a CausalContext, which we call a +//! "causality token", is used in the API and must be sent along with +//! each write or delete operation to indicate the previously seen +//! versions that we want to overwrite or delete. + use std::collections::BTreeMap; use std::convert::TryInto; @@ -9,23 +19,36 @@ use garage_util::data::*; /// of full Garage node IDs which are 256-bit UUIDs. pub type K2VNodeId = u64; +pub type VectorClock = BTreeMap; + pub fn make_node_id(node_id: Uuid) -> K2VNodeId { let mut tmp = [0u8; 8]; tmp.copy_from_slice(&node_id.as_slice()[..8]); u64::from_be_bytes(tmp) } -#[derive(PartialEq, Eq, Debug, Serialize, Deserialize)] +pub fn vclock_gt(a: &VectorClock, b: &VectorClock) -> bool { + a.iter().any(|(n, ts)| ts > b.get(n).unwrap_or(&0)) +} + +pub fn vclock_max(a: &VectorClock, b: &VectorClock) -> VectorClock { + let mut ret = a.clone(); + for (n, ts) in b.iter() { + let ent = ret.entry(*n).or_insert(0); + *ent = std::cmp::max(*ts, *ent); + } + ret +} + +#[derive(PartialEq, Eq, Debug, Serialize, Deserialize, Default)] pub struct CausalContext { - pub vector_clock: BTreeMap, + pub vector_clock: VectorClock, } impl CausalContext { /// Empty causality context - pub fn new_empty() -> Self { - Self { - vector_clock: BTreeMap::new(), - } + pub fn new() -> Self { + Self::default() } /// Make binary representation and encode in base64 pub fn serialize(&self) -> String { @@ -72,9 +95,7 @@ impl CausalContext { } /// Check if this causal context contains newer items than another one pub fn is_newer_than(&self, other: &Self) -> bool { - self.vector_clock - .iter() - .any(|(k, v)| v > other.vector_clock.get(k).unwrap_or(&0)) + vclock_gt(&self.vector_clock, &other.vector_clock) } } diff --git a/src/model/k2v/item_table.rs b/src/model/k2v/item_table.rs index 1ba9bb46..bc2b1aef 100644 --- a/src/model/k2v/item_table.rs +++ b/src/model/k2v/item_table.rs @@ -106,7 +106,7 @@ impl K2VItem { /// Extract the causality context of a K2V Item pub fn causal_context(&self) -> CausalContext { - let mut cc = CausalContext::new_empty(); + let mut cc = CausalContext::new(); for (node, ent) in self.items.iter() { cc.vector_clock.insert(*node, ent.max_time()); } diff --git a/src/model/k2v/mod.rs b/src/model/k2v/mod.rs index 83ad2512..c488b4c6 100644 --- a/src/model/k2v/mod.rs +++ b/src/model/k2v/mod.rs @@ -1,4 +1,5 @@ pub mod causality; +pub mod seen; pub mod item_table; diff --git a/src/model/k2v/seen.rs b/src/model/k2v/seen.rs new file mode 100644 index 00000000..b8f4ff27 --- /dev/null +++ b/src/model/k2v/seen.rs @@ -0,0 +1,85 @@ +//! Implements a RangeSeenMarker, a data type used in the PollRange API +//! to indicate which items in the range have already been seen +//! and which have not been seen yet. +//! +//! It consists of a vector clock that indicates that for each node, +//! all items produced by that node with timestamps <= the value in the +//! vector clock has been seen, as well as a set of causal contexts for +//! individual items. + +use std::collections::BTreeMap; + +use serde::{Deserialize, Serialize}; + +use garage_util::data::Uuid; +use garage_util::encode::{nonversioned_decode, nonversioned_encode}; +use garage_util::error::{Error, OkOrMessage}; + +use crate::k2v::causality::*; +use crate::k2v::item_table::*; + +#[derive(Debug, Serialize, Deserialize, Default)] +pub struct RangeSeenMarker { + vector_clock: VectorClock, + items: BTreeMap, +} + +impl RangeSeenMarker { + pub fn new() -> Self { + Self::default() + } + + pub fn mark_seen_node_items<'a, I: IntoIterator>( + &mut self, + node: Uuid, + items: I, + ) { + let node = make_node_id(node); + for item in items.into_iter() { + let cc = item.causal_context(); + + if let Some(ts) = cc.vector_clock.get(&node) { + let ent = self.vector_clock.entry(node).or_insert(0); + *ent = std::cmp::max(*ent, *ts); + } + + if vclock_gt(&cc.vector_clock, &self.vector_clock) { + match self.items.get_mut(&item.sort_key) { + None => { + self.items.insert(item.sort_key.clone(), cc.vector_clock); + } + Some(ent) => *ent = vclock_max(&ent, &cc.vector_clock), + } + } + } + } + + pub fn canonicalize(&mut self) { + let self_vc = &self.vector_clock; + self.items.retain(|_sk, vc| vclock_gt(&vc, self_vc)) + } + + pub fn encode(&mut self) -> Result { + self.canonicalize(); + + let bytes = nonversioned_encode(&self)?; + let bytes = zstd::stream::encode_all(&mut &bytes[..], zstd::DEFAULT_COMPRESSION_LEVEL)?; + Ok(base64::encode(&bytes)) + } + + pub fn decode(s: &str) -> Result { + let bytes = base64::decode(&s).ok_or_message("invalid base64")?; + let bytes = zstd::stream::decode_all(&mut &bytes[..])?; + Ok(nonversioned_decode(&bytes)?) + } + + pub fn is_new_item(&self, item: &K2VItem) -> bool { + let cc = item.causal_context(); + vclock_gt(&cc.vector_clock, &self.vector_clock) + && self + .items + .get(&item.sort_key) + .map(|vc| vclock_gt(&cc.vector_clock, &vc)) + .unwrap_or(true) + } +} -- cgit v1.2.3 From 43fd6c1526339fb444e5e25021f42d0aee252b27 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 10 Jan 2023 12:54:24 +0100 Subject: PollRange RPC --- src/model/k2v/rpc.rs | 133 ++++++++++++++++++++++++++++++++++++++++++++++++-- src/model/k2v/seen.rs | 13 +++++ src/model/k2v/sub.rs | 31 ++++++------ 3 files changed, 159 insertions(+), 18 deletions(-) (limited to 'src/model/k2v') diff --git a/src/model/k2v/rpc.rs b/src/model/k2v/rpc.rs index e5497215..8b070885 100644 --- a/src/model/k2v/rpc.rs +++ b/src/model/k2v/rpc.rs @@ -5,7 +5,7 @@ //! node does not process the entry directly, as this would //! mean the vector clock gets much larger than needed). -use std::collections::HashMap; +use std::collections::{BTreeMap, HashMap}; use std::convert::TryInto; use std::sync::{Arc, Mutex}; use std::time::Duration; @@ -31,6 +31,7 @@ use garage_table::{PartitionKey, Table}; use crate::k2v::causality::*; use crate::k2v::item_table::*; +use crate::k2v::seen::*; use crate::k2v::sub::*; const TIMESTAMP_KEY: &'static [u8] = b"timestamp"; @@ -46,7 +47,13 @@ enum K2VRpc { causal_context: CausalContext, timeout_msec: u64, }, + PollRange { + range: PollRange, + seen_str: Option, + timeout_msec: u64, + }, PollItemResponse(Option), + PollRangeResponse(Uuid, Vec), } #[derive(Debug, Serialize, Deserialize)] @@ -242,9 +249,7 @@ impl K2VRpcHandler { resp = Some(x); } } - K2VRpc::PollItemResponse(None) => { - return Ok(None); - } + K2VRpc::PollItemResponse(None) => (), v => return Err(Error::unexpected_rpc_message(v)), } } @@ -252,6 +257,69 @@ impl K2VRpcHandler { Ok(resp) } + pub async fn poll_range( + &self, + range: PollRange, + seen_str: Option, + timeout_msec: u64, + ) -> Result, String)>, Error> { + let mut seen = seen_str + .as_deref() + .map(RangeSeenMarker::decode) + .transpose()? + .unwrap_or_default(); + seen.restrict(&range); + + let nodes = self + .item_table + .data + .replication + .write_nodes(&range.partition.hash()); + + let rpc = self.system.rpc.try_call_many( + &self.endpoint, + &nodes[..], + K2VRpc::PollRange { + range, + seen_str, + timeout_msec, + }, + RequestStrategy::with_priority(PRIO_NORMAL) + .with_quorum(self.item_table.data.replication.read_quorum()) + .without_timeout(), + ); + let timeout_duration = Duration::from_millis(timeout_msec) + self.system.rpc.rpc_timeout(); + let resps = select! { + r = rpc => r?, + _ = tokio::time::sleep(timeout_duration) => return Ok(None), + }; + + let mut new_items = BTreeMap::::new(); + for v in resps { + if let K2VRpc::PollRangeResponse(node, items) = v { + seen.mark_seen_node_items(node, items.iter()); + for item in items.into_iter() { + match new_items.get_mut(&item.sort_key) { + Some(ent) => { + ent.merge(&item); + } + None => { + new_items.insert(item.sort_key.clone(), item); + } + } + } + } else { + return Err(Error::unexpected_rpc_message(v)); + } + } + + if new_items.is_empty() { + Ok(None) + } else { + Ok(Some((new_items, seen.encode()?))) + } + } + // ---- internal handlers ---- async fn handle_insert(&self, item: &InsertedItem) -> Result { @@ -348,6 +416,52 @@ impl K2VRpcHandler { Ok(value) } + + async fn handle_poll_range( + &self, + range: &PollRange, + seen_str: &Option, + ) -> Result, Error> { + let seen = seen_str + .as_deref() + .map(RangeSeenMarker::decode) + .transpose()? + .unwrap_or_default(); + let mut new_items = vec![]; + + let mut chan = self.subscriptions.subscribe_partition(&range.partition); + + // Read current state of the specified range to check new items + let partition_hash = range.partition.hash(); + let first_key = match &range.start { + None => partition_hash.to_vec(), + Some(sk) => self.item_table.data.tree_key(&range.partition, sk), + }; + for item in self.item_table.data.store.range(first_key..)? { + let (key, value) = item?; + if &key[..32] != partition_hash.as_slice() { + break; + } + let item = self.item_table.data.decode_entry(&value)?; + if !range.matches(&item) { + break; + } + if seen.is_new_item(&item) { + new_items.push(item); + } + } + + // If we found no new items, wait for a matching item to arrive + // on the channel + while new_items.is_empty() { + let item = chan.recv().await?; + if range.matches(&item) && seen.is_new_item(&item) { + new_items.push(item); + } + } + + Ok(new_items) + } } #[async_trait] @@ -367,6 +481,17 @@ impl EndpointHandler for K2VRpcHandler { _ = delay => Ok(K2VRpc::PollItemResponse(None)), } } + K2VRpc::PollRange { + range, + seen_str, + timeout_msec, + } => { + let delay = tokio::time::sleep(Duration::from_millis(*timeout_msec)); + select! { + ret = self.handle_poll_range(range, seen_str) => ret.map(|items| K2VRpc::PollRangeResponse(self.system.id, items)), + _ = delay => Ok(K2VRpc::PollRangeResponse(self.system.id, vec![])), + } + } m => Err(Error::unexpected_rpc_message(m)), } } diff --git a/src/model/k2v/seen.rs b/src/model/k2v/seen.rs index b8f4ff27..d2cd54c7 100644 --- a/src/model/k2v/seen.rs +++ b/src/model/k2v/seen.rs @@ -17,6 +17,7 @@ use garage_util::error::{Error, OkOrMessage}; use crate::k2v::causality::*; use crate::k2v::item_table::*; +use crate::k2v::sub::*; #[derive(Debug, Serialize, Deserialize, Default)] pub struct RangeSeenMarker { @@ -29,6 +30,18 @@ impl RangeSeenMarker { Self::default() } + pub fn restrict(&mut self, range: &PollRange) { + if let Some(start) = &range.start { + self.items = self.items.split_off(start); + } + if let Some(end) = &range.end { + self.items.split_off(end); + } + if let Some(pfx) = &range.prefix { + self.items.retain(|k, _v| k.starts_with(pfx)); + } + } + pub fn mark_seen_node_items<'a, I: IntoIterator>( &mut self, node: Uuid, diff --git a/src/model/k2v/sub.rs b/src/model/k2v/sub.rs index c4273dba..b1daa271 100644 --- a/src/model/k2v/sub.rs +++ b/src/model/k2v/sub.rs @@ -26,7 +26,7 @@ pub struct SubscriptionManager(Mutex); #[derive(Default)] pub struct SubscriptionManagerInner { item_subscriptions: HashMap>, - range_subscriptions: HashMap>, + part_subscriptions: HashMap>, } impl SubscriptionManager { @@ -34,7 +34,7 @@ impl SubscriptionManager { Self::default() } - pub fn subscribe_item(&self, key: &PollKey) -> broadcast::Receiver { + pub(crate) fn subscribe_item(&self, key: &PollKey) -> broadcast::Receiver { let mut inner = self.0.lock().unwrap(); if let Some(s) = inner.item_subscriptions.get(key) { s.subscribe() @@ -45,18 +45,21 @@ impl SubscriptionManager { } } - pub fn subscribe_range(&self, key: &PollRange) -> broadcast::Receiver { + pub(crate) fn subscribe_partition( + &self, + part: &K2VItemPartition, + ) -> broadcast::Receiver { let mut inner = self.0.lock().unwrap(); - if let Some(s) = inner.range_subscriptions.get(key) { + if let Some(s) = inner.part_subscriptions.get(part) { s.subscribe() } else { let (tx, rx) = broadcast::channel(8); - inner.range_subscriptions.insert(key.clone(), tx); + inner.part_subscriptions.insert(part.clone(), tx); rx } } - pub fn notify(&self, item: &K2VItem) { + pub(crate) fn notify(&self, item: &K2VItem) { let mut inner = self.0.lock().unwrap(); // 1. Notify single item subscribers, @@ -73,20 +76,20 @@ impl SubscriptionManager { } } - // 2. Notify range subscribers, + // 2. Notify partition subscribers, // removing subscriptions with no more listeners if any - inner.range_subscriptions.retain(|sub, chan| { - if sub.matches(&item) { - chan.send(item.clone()).is_ok() - } else { - chan.receiver_count() != 0 + if let Some(s) = inner.part_subscriptions.get(&item.partition) { + if s.send(item.clone()).is_err() { + // no more subscribers, remove channel from here + // (we will re-create it later if we need to subscribe again) + inner.part_subscriptions.remove(&item.partition); } - }); + } } } impl PollRange { - fn matches(&self, item: &K2VItem) -> bool { + pub fn matches(&self, item: &K2VItem) -> bool { item.partition == self.partition && self .prefix -- cgit v1.2.3 From b83517d521b1bea7585ce45a803fad373f28225c Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Tue, 10 Jan 2023 15:22:25 +0100 Subject: Implement PollRange API endpoint --- src/model/k2v/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) (limited to 'src/model/k2v') diff --git a/src/model/k2v/mod.rs b/src/model/k2v/mod.rs index c488b4c6..acc1fcdc 100644 --- a/src/model/k2v/mod.rs +++ b/src/model/k2v/mod.rs @@ -5,4 +5,4 @@ pub mod item_table; pub mod rpc; -pub(crate) mod sub; +pub mod sub; -- cgit v1.2.3 From 09a3dad0f2c1641e4e300809dbdb3599b32efc03 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 11 Jan 2023 11:35:36 +0100 Subject: Lock once for insert_many --- src/model/k2v/rpc.rs | 34 ++++++++++++++++++++++------------ 1 file changed, 22 insertions(+), 12 deletions(-) (limited to 'src/model/k2v') diff --git a/src/model/k2v/rpc.rs b/src/model/k2v/rpc.rs index 8b070885..04ab3ab9 100644 --- a/src/model/k2v/rpc.rs +++ b/src/model/k2v/rpc.rs @@ -7,7 +7,7 @@ use std::collections::{BTreeMap, HashMap}; use std::convert::TryInto; -use std::sync::{Arc, Mutex}; +use std::sync::{Arc, Mutex, MutexGuard}; use std::time::Duration; use async_trait::async_trait; @@ -72,7 +72,12 @@ impl Rpc for K2VRpc { pub struct K2VRpcHandler { system: Arc, item_table: Arc>, + + // Using a mutex on the local_timestamp_tree is not strictly necessary, + // but it helps to not try to do several inserts at the same time, + // which would create transaction conflicts and force many useless retries. local_timestamp_tree: Mutex, + endpoint: Arc>, subscriptions: Arc, } @@ -323,7 +328,10 @@ impl K2VRpcHandler { // ---- internal handlers ---- async fn handle_insert(&self, item: &InsertedItem) -> Result { - let new = self.local_insert(item)?; + let new = { + let local_timestamp_tree = self.local_timestamp_tree.lock().unwrap(); + self.local_insert(&local_timestamp_tree, item)? + }; // Propagate to rest of network if let Some(updated) = new { @@ -336,11 +344,14 @@ impl K2VRpcHandler { async fn handle_insert_many(&self, items: &[InsertedItem]) -> Result { let mut updated_vec = vec![]; - for item in items { - let new = self.local_insert(item)?; + { + let local_timestamp_tree = self.local_timestamp_tree.lock().unwrap(); + for item in items { + let new = self.local_insert(&local_timestamp_tree, item)?; - if let Some(updated) = new { - updated_vec.push(updated); + if let Some(updated) = new { + updated_vec.push(updated); + } } } @@ -352,12 +363,11 @@ impl K2VRpcHandler { Ok(K2VRpc::Ok) } - fn local_insert(&self, item: &InsertedItem) -> Result, Error> { - // Using a mutex on the local_timestamp_tree is not strictly necessary, - // but it helps to not try to do several inserts at the same time, - // which would create transaction conflicts and force many useless retries. - let local_timestamp_tree = self.local_timestamp_tree.lock().unwrap(); - + fn local_insert( + &self, + local_timestamp_tree: &MutexGuard<'_, db::Tree>, + item: &InsertedItem, + ) -> Result, Error> { let now = now_msec(); self.item_table -- cgit v1.2.3 From ba384e61c0951036b0c4fb394011f3498abf67ca Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 11 Jan 2023 12:03:17 +0100 Subject: PollRange: return immediately if no seen marker is provided --- src/model/k2v/rpc.rs | 57 +++++++++++++++++++++++++++++++++++----------------- 1 file changed, 39 insertions(+), 18 deletions(-) (limited to 'src/model/k2v') diff --git a/src/model/k2v/rpc.rs b/src/model/k2v/rpc.rs index 04ab3ab9..04801ebf 100644 --- a/src/model/k2v/rpc.rs +++ b/src/model/k2v/rpc.rs @@ -268,6 +268,8 @@ impl K2VRpcHandler { seen_str: Option, timeout_msec: u64, ) -> Result, String)>, Error> { + let has_seen_marker = seen_str.is_some(); + let mut seen = seen_str .as_deref() .map(RangeSeenMarker::decode) @@ -318,7 +320,7 @@ impl K2VRpcHandler { } } - if new_items.is_empty() { + if new_items.is_empty() && has_seen_marker { Ok(None) } else { Ok(Some((new_items, seen.encode()?))) @@ -432,16 +434,44 @@ impl K2VRpcHandler { range: &PollRange, seen_str: &Option, ) -> Result, Error> { - let seen = seen_str - .as_deref() - .map(RangeSeenMarker::decode) - .transpose()? - .unwrap_or_default(); - let mut new_items = vec![]; + if let Some(seen_str) = seen_str { + let seen = RangeSeenMarker::decode(seen_str)?; + + // Subscribe now to all changes on that partition, + // so that new items that are inserted while we are reading the range + // will be seen in the loop below + let mut chan = self.subscriptions.subscribe_partition(&range.partition); + + // Check for the presence of any new items already stored in the item table + let mut new_items = self.poll_range_read_range(range, &seen)?; + + // If we found no new items, wait for a matching item to arrive + // on the channel + while new_items.is_empty() { + let item = chan.recv().await?; + if range.matches(&item) && seen.is_new_item(&item) { + new_items.push(item); + } + } + + Ok(new_items) + } else { + // If no seen marker was specified, we do not poll for anything. + // We return immediately with the set of known items (even if + // it is empty), which will give the client an inital view of + // the dataset and an initial seen marker for further + // PollRange calls. + self.poll_range_read_range(range, &RangeSeenMarker::default()) + } + } - let mut chan = self.subscriptions.subscribe_partition(&range.partition); + fn poll_range_read_range( + &self, + range: &PollRange, + seen: &RangeSeenMarker, + ) -> Result, Error> { + let mut new_items = vec![]; - // Read current state of the specified range to check new items let partition_hash = range.partition.hash(); let first_key = match &range.start { None => partition_hash.to_vec(), @@ -461,15 +491,6 @@ impl K2VRpcHandler { } } - // If we found no new items, wait for a matching item to arrive - // on the channel - while new_items.is_empty() { - let item = chan.recv().await?; - if range.matches(&item) && seen.is_new_item(&item) { - new_items.push(item); - } - } - Ok(new_items) } } -- cgit v1.2.3 From bba13f40fc2e411347ea83960935b39cedb0a7c4 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 11 Jan 2023 12:27:19 +0100 Subject: Correctly return bad requests when seeh marker is invalid --- src/model/k2v/causality.rs | 23 ++++++++++++++++------- src/model/k2v/rpc.rs | 13 +++++++------ src/model/k2v/seen.rs | 16 +++++++++++----- 3 files changed, 34 insertions(+), 18 deletions(-) (limited to 'src/model/k2v') diff --git a/src/model/k2v/causality.rs b/src/model/k2v/causality.rs index 665b02b2..b1ec8035 100644 --- a/src/model/k2v/causality.rs +++ b/src/model/k2v/causality.rs @@ -15,6 +15,8 @@ use serde::{Deserialize, Serialize}; use garage_util::data::*; +use crate::helper::error::{Error as HelperError, OkOrBadRequest}; + /// Node IDs used in K2V are u64 integers that are the abbreviation /// of full Garage node IDs which are 256-bit UUIDs. pub type K2VNodeId = u64; @@ -50,6 +52,7 @@ impl CausalContext { pub fn new() -> Self { Self::default() } + /// Make binary representation and encode in base64 pub fn serialize(&self) -> String { let mut ints = Vec::with_capacity(2 * self.vector_clock.len()); @@ -66,12 +69,13 @@ impl CausalContext { base64::encode_config(bytes, base64::URL_SAFE_NO_PAD) } - /// Parse from base64-encoded binary representation - pub fn parse(s: &str) -> Result { - let bytes = base64::decode_config(s, base64::URL_SAFE_NO_PAD) - .map_err(|e| format!("bad causality token base64: {}", e))?; + + /// Parse from base64-encoded binary representation. + /// Returns None on error. + pub fn parse(s: &str) -> Option { + let bytes = base64::decode_config(s, base64::URL_SAFE_NO_PAD).ok()?; if bytes.len() % 16 != 8 || bytes.len() < 8 { - return Err("bad causality token length".into()); + return None; } let checksum = u64::from_be_bytes(bytes[..8].try_into().unwrap()); @@ -88,11 +92,16 @@ impl CausalContext { let check = ret.vector_clock.iter().fold(0, |acc, (n, t)| acc ^ *n ^ *t); if check != checksum { - return Err("bad causality token checksum".into()); + return None; } - Ok(ret) + Some(ret) } + + pub fn parse_helper(s: &str) -> Result { + Self::parse(s).ok_or_bad_request("Invalid causality token") + } + /// Check if this causal context contains newer items than another one pub fn is_newer_than(&self, other: &Self) -> bool { vclock_gt(&self.vector_clock, &other.vector_clock) diff --git a/src/model/k2v/rpc.rs b/src/model/k2v/rpc.rs index 04801ebf..3b4d7465 100644 --- a/src/model/k2v/rpc.rs +++ b/src/model/k2v/rpc.rs @@ -29,6 +29,7 @@ use garage_rpc::*; use garage_table::replication::{TableReplication, TableShardedReplication}; use garage_table::{PartitionKey, Table}; +use crate::helper::error::Error as HelperError; use crate::k2v::causality::*; use crate::k2v::item_table::*; use crate::k2v::seen::*; @@ -212,7 +213,7 @@ impl K2VRpcHandler { sort_key: String, causal_context: CausalContext, timeout_msec: u64, - ) -> Result, Error> { + ) -> Result, HelperError> { let poll_key = PollKey { partition: K2VItemPartition { bucket_id, @@ -255,7 +256,7 @@ impl K2VRpcHandler { } } K2VRpc::PollItemResponse(None) => (), - v => return Err(Error::unexpected_rpc_message(v)), + v => return Err(Error::unexpected_rpc_message(v).into()), } } @@ -267,12 +268,12 @@ impl K2VRpcHandler { range: PollRange, seen_str: Option, timeout_msec: u64, - ) -> Result, String)>, Error> { + ) -> Result, String)>, HelperError> { let has_seen_marker = seen_str.is_some(); let mut seen = seen_str .as_deref() - .map(RangeSeenMarker::decode) + .map(RangeSeenMarker::decode_helper) .transpose()? .unwrap_or_default(); seen.restrict(&range); @@ -316,7 +317,7 @@ impl K2VRpcHandler { } } } else { - return Err(Error::unexpected_rpc_message(v)); + return Err(Error::unexpected_rpc_message(v).into()); } } @@ -435,7 +436,7 @@ impl K2VRpcHandler { seen_str: &Option, ) -> Result, Error> { if let Some(seen_str) = seen_str { - let seen = RangeSeenMarker::decode(seen_str)?; + let seen = RangeSeenMarker::decode(seen_str).ok_or_message("Invalid seenMarker")?; // Subscribe now to all changes on that partition, // so that new items that are inserted while we are reading the range diff --git a/src/model/k2v/seen.rs b/src/model/k2v/seen.rs index d2cd54c7..314d0f9e 100644 --- a/src/model/k2v/seen.rs +++ b/src/model/k2v/seen.rs @@ -13,8 +13,9 @@ use serde::{Deserialize, Serialize}; use garage_util::data::Uuid; use garage_util::encode::{nonversioned_decode, nonversioned_encode}; -use garage_util::error::{Error, OkOrMessage}; +use garage_util::error::Error; +use crate::helper::error::{Error as HelperError, OkOrBadRequest}; use crate::k2v::causality::*; use crate::k2v::item_table::*; use crate::k2v::sub::*; @@ -80,10 +81,15 @@ impl RangeSeenMarker { Ok(base64::encode(&bytes)) } - pub fn decode(s: &str) -> Result { - let bytes = base64::decode(&s).ok_or_message("invalid base64")?; - let bytes = zstd::stream::decode_all(&mut &bytes[..])?; - Ok(nonversioned_decode(&bytes)?) + /// Decode from msgpack+zstd+b64 representation, returns None on error. + pub fn decode(s: &str) -> Option { + let bytes = base64::decode(&s).ok()?; + let bytes = zstd::stream::decode_all(&mut &bytes[..]).ok()?; + nonversioned_decode(&bytes).ok() + } + + pub fn decode_helper(s: &str) -> Result { + Self::decode(s).ok_or_bad_request("Invalid causality token") } pub fn is_new_item(&self, item: &K2VItem) -> bool { -- cgit v1.2.3 From 5b5ca63cf6d108e7bc7d83da68667163a278d30a Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 11 Jan 2023 15:17:27 +0100 Subject: Poll cleanup --- src/model/k2v/rpc.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) (limited to 'src/model/k2v') diff --git a/src/model/k2v/rpc.rs b/src/model/k2v/rpc.rs index 3b4d7465..f5d8ffc2 100644 --- a/src/model/k2v/rpc.rs +++ b/src/model/k2v/rpc.rs @@ -213,7 +213,7 @@ impl K2VRpcHandler { sort_key: String, causal_context: CausalContext, timeout_msec: u64, - ) -> Result, HelperError> { + ) -> Result, Error> { let poll_key = PollKey { partition: K2VItemPartition { bucket_id, @@ -256,7 +256,7 @@ impl K2VRpcHandler { } } K2VRpc::PollItemResponse(None) => (), - v => return Err(Error::unexpected_rpc_message(v).into()), + v => return Err(Error::unexpected_rpc_message(v)), } } -- cgit v1.2.3 From 638c5a3ce006eba9d90a6358655ba2091423efd7 Mon Sep 17 00:00:00 2001 From: Alex Auvolat Date: Wed, 11 Jan 2023 16:12:07 +0100 Subject: PollRange: add extra RPC delay after quorum is achieved, to give a chance to the 3rd node to respond --- src/model/k2v/rpc.rs | 77 ++++++++++++++++++++++++++++++++++++++++------------ 1 file changed, 59 insertions(+), 18 deletions(-) (limited to 'src/model/k2v') diff --git a/src/model/k2v/rpc.rs b/src/model/k2v/rpc.rs index f5d8ffc2..117103b6 100644 --- a/src/model/k2v/rpc.rs +++ b/src/model/k2v/rpc.rs @@ -8,7 +8,7 @@ use std::collections::{BTreeMap, HashMap}; use std::convert::TryInto; use std::sync::{Arc, Mutex, MutexGuard}; -use std::time::Duration; +use std::time::{Duration, Instant}; use async_trait::async_trait; use futures::stream::FuturesUnordered; @@ -35,6 +35,8 @@ use crate::k2v::item_table::*; use crate::k2v::seen::*; use crate::k2v::sub::*; +const POLL_RANGE_EXTRA_DELAY: Duration = Duration::from_millis(200); + const TIMESTAMP_KEY: &'static [u8] = b"timestamp"; /// RPC messages for K2V @@ -271,6 +273,8 @@ impl K2VRpcHandler { ) -> Result, String)>, HelperError> { let has_seen_marker = seen_str.is_some(); + // Parse seen marker, we will use it below. This is also the first check + // that it is valid, which returns a bad request error if not. let mut seen = seen_str .as_deref() .map(RangeSeenMarker::decode_helper) @@ -278,30 +282,67 @@ impl K2VRpcHandler { .unwrap_or_default(); seen.restrict(&range); + // Prepare PollRange RPC to send to the storage nodes responsible for the parititon let nodes = self .item_table .data .replication .write_nodes(&range.partition.hash()); - - let rpc = self.system.rpc.try_call_many( - &self.endpoint, - &nodes[..], - K2VRpc::PollRange { - range, - seen_str, - timeout_msec, - }, - RequestStrategy::with_priority(PRIO_NORMAL) - .with_quorum(self.item_table.data.replication.read_quorum()) - .without_timeout(), - ); - let timeout_duration = Duration::from_millis(timeout_msec) + self.system.rpc.rpc_timeout(); - let resps = select! { - r = rpc => r?, - _ = tokio::time::sleep(timeout_duration) => return Ok(None), + let quorum = self.item_table.data.replication.read_quorum(); + let msg = K2VRpc::PollRange { + range, + seen_str, + timeout_msec, }; + // Send the request to all nodes, use FuturesUnordered to get the responses in any order + let msg = msg.into_req().map_err(netapp::error::Error::from)?; + let rs = RequestStrategy::with_priority(PRIO_NORMAL).without_timeout(); + let mut requests = nodes + .iter() + .map(|node| self.system.rpc.call(&self.endpoint, *node, msg.clone(), rs)) + .collect::>(); + + // Fetch responses. This procedure stops fetching responses when any of the following + // conditions arise: + // - we have a response to all requests + // - we have a response to a read quorum of requests (e.g. 2/3), and an extra delay + // has passed since the quorum was achieved + // - a global RPC timeout expired + // The extra delay after a quorum was received is usefull if the third response was to + // arrive during this short interval: this would allow us to consider all the data seen + // by that last node in the response we produce, and would likely help reduce the + // size of the seen marker that we will return (because we would have an info of the + // kind: all items produced by that node until time ts have been returned, so we can + // bump the entry in the global vector clock and possibly remove some item-specific + // vector clocks) + let mut deadline = + Instant::now() + Duration::from_millis(timeout_msec) + self.system.rpc.rpc_timeout(); + let mut resps = vec![]; + let mut errors = vec![]; + loop { + select! { + _ = tokio::time::sleep_until(deadline.into()) => { + break; + } + res = requests.next() => match res { + None => break, + Some(Err(e)) => errors.push(e), + Some(Ok(r)) => { + resps.push(r); + if resps.len() >= quorum { + deadline = std::cmp::min(deadline, Instant::now() + POLL_RANGE_EXTRA_DELAY); + } + } + } + } + } + if errors.len() > nodes.len() - quorum { + let errors = errors.iter().map(|e| format!("{}", e)).collect::>(); + return Err(Error::Quorum(quorum, resps.len(), nodes.len(), errors).into()); + } + + // Take all returned items into account to produce the response. let mut new_items = BTreeMap::::new(); for v in resps { if let K2VRpc::PollRangeResponse(node, items) = v { -- cgit v1.2.3