├── .gitignore ├── Cargo.toml ├── README.md ├── llvm-lines ├── Cargo.toml └── main.rs └── src ├── NOTES.md ├── TODO.md ├── agreement.rs ├── database.rs ├── globaltime.rs ├── language.rs ├── lib.rs ├── main.rs ├── network.rs ├── quorum.rs ├── replication.rs ├── storage.rs ├── test.rs ├── tidmgr.rs ├── transaction.rs └── watermarks.rs /.gitignore: -------------------------------------------------------------------------------- 1 | # Generated by Cargo 2 | # will have compiled files and executables 3 | /target/ 4 | 5 | # Remove Cargo.lock from gitignore if creating an executable, leave it for libraries 6 | # More information here https://doc.rust-lang.org/cargo/guide/cargo-toml-vs-cargo-lock.html 7 | Cargo.lock 8 | 9 | # These are backup files generated by rustfmt 10 | **/*.rs.bk 11 | -------------------------------------------------------------------------------- /Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "clepsydra" 3 | description = "small distributed database protocol" 4 | keywords = ["database", "mvcc", "distributed", "transaction"] 5 | version = "0.2.1-alpha.0" 6 | authors = ["Graydon Hoare "] 7 | edition = "2018" 8 | license = "MIT OR Apache-2.0" 9 | repository = "http://github.com/graydon/clepsydra" 10 | readme = "README.md" 11 | 12 | # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html 13 | [lib] 14 | name = "clepsydra" 15 | path = "src/lib.rs" 16 | 17 | [[bin]] 18 | name = "clepsydra-test" 19 | path = "src/main.rs" 20 | 21 | [dependencies] 22 | pergola = "0.9.0" 23 | concorde = "0.7.0" 24 | edelcrantz = "0.6.1" 25 | serde = { version = "1.0", features = ["derive"] } 26 | thiserror = "1.0.22" 27 | 28 | async-std = { version = "1.8.0", features = ["unstable"] } 29 | futures = "0.3.8" 30 | futures-util = "0.3.8" 31 | 32 | tracing = "0.1.22" 33 | tracing-futures = "0.2.4" 34 | 35 | tracing-tracy = {version="0.4.0", features = ["enable"], optional=true} 36 | 37 | [features] 38 | default = [] 39 | tracy = ["tracing-tracy"] 40 | 41 | [dev-dependencies] 42 | sluice = "0.5.3" 43 | duplexify = "1.2.2" 44 | tracing-subscriber = {version="0.2.15", features = ["chrono", "env-filter", "fmt"]} 45 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # clepsydra 2 | 3 | ## Overview 4 | 5 | This is a work-in-progress implementation of a core protocol for a minimalist 6 | distributed database. It strives to be as small and simple as possible while 7 | attempting to provide relatively challenging features: 8 | 9 | - Strict Serializability 10 | 11 | - Online Reconfiguration 12 | 13 | - Fault tolerance 14 | 15 | - High throughput 16 | 17 | The implementation is based on a simplified version of the "Ocean Vista" 18 | (OV) protocol, and uses its terminology wherever possible. OV combines 19 | replication, transaction commitment and concurrency control into a single 20 | protocol. 21 | 22 | ### Summary 23 | 24 | The short version of the protocol is: 25 | 26 | - Transactions are represented as deterministic thunks over snapshots. 27 | 28 | - Each transaction is assigned a globally-unique timestamp. 29 | 30 | - Transactions are separated into two phases: S-phase and E-phase. 31 | 32 | - S-phase (storage) consists of coordination-free "blind quorum-writes" 33 | replicating the thunks into their MVCC order on each replica. 34 | 35 | - A watermark tracking minimum transaction timestamps-being-written is 36 | gossiped between peers, increasing as quorum-writes complete. 37 | 38 | - A transaction only enters E-phase after the watermark advances past it. 39 | 40 | - E-phase (evaluation) quorum-reads and evaluates thunks from consistent 41 | snapshots below the watermark, lazily resolving any earlier thunks. 42 | Everything below the watermark is coordination-free and deterministic. 43 | 44 | ### Caveats 45 | 46 | Nothing's perfect, and this crate is anything but: 47 | 48 | - This crate is very incomplete and does not work yet. Don't use it for 49 | anything other than experiments and toys. Recovery, reconfiguration, 50 | timeouts and nontrivial fault tolerance paths _definitely_ don't work. 51 | 52 | - It also (somewhat recklessly) attempts to combine OV's reconfiguration 53 | and gossip protocols into an instance of the [concorde] reconfigurable 54 | lattice agreement protocol. This might not even be _theoretically_ safe. 55 | 56 | - It is much more minimal than the full OV protocol: there's no support 57 | for sharding, nor the two-level peer-vs-datacenter locality organization. 58 | This crate treats its whole peer group as a single symmetric shard. 59 | 60 | - As a result, performance won't be "webscale" or anything. It will scale 61 | vertically if you throw cores at it, but no better, and its latency will 62 | always have speed-of-light WAN RTT factors in it. It's distributed for 63 | fault tolerance, not horizontal scaling. 64 | 65 | - As with OV, this crate does require partial clock synchronization. It 66 | doesn't need to be very tight: clock drift only causes increased 67 | latency as the watermarks progress as the minimum of all times; it 68 | doesn't affect correctness. Normal weak-NTP-level sync should be ok. 69 | 70 | - As with OV, Calvin, and all deterministic databases: your txns have to be 71 | deterministic and must have deterministic _read and write sets_. If they 72 | cannot have their read and write sets statically computed (eg. if they 73 | rely on the data to decide read and write set) you have to build slightly 74 | awkward multi-phase txns. The term in the literature is "reconnaisance 75 | queries". 76 | 77 | ### Reference 78 | 79 | Hua Fan and Wojciech Golab. Ocean Vista: Gossip-Based Visibility Control for 80 | Speedy Geo-Distributed Transactions. PVLDB, 12(11): 1471-1484, 2019. 81 | 82 | DOI: 83 | 84 | 85 | 86 | ### Name 87 | 88 | Wikipedia: 89 | 90 | > A water clock or clepsydra (Greek κλεψύδρα from κλέπτειν kleptein, 'to 91 | > steal'; ὕδωρ hydor, 'water') is any timepiece by which time is measured by 92 | > the regulated flow of liquid into (inflow type) or out from (outflow type) 93 | > a vessel, and where the amount is then measured. 94 | 95 | 96 | License: MIT OR Apache-2.0 97 | -------------------------------------------------------------------------------- /llvm-lines/Cargo.toml: -------------------------------------------------------------------------------- 1 | [package] 2 | name = "clepsydra-llvm-lines" 3 | version = "0.1.0" 4 | authors = ["Graydon Hoare "] 5 | edition = "2018" 6 | publish = false 7 | 8 | [[bin]] 9 | name = "clepsydra-test" 10 | path = "main.rs" 11 | 12 | [dependencies] 13 | clepsydra = { path = ".." } 14 | 15 | serde = { version = "1.0", features = ["derive"] } 16 | 17 | async-std = { version = "1.8.0", features = ["unstable"] } 18 | futures = "0.3.8" 19 | 20 | tracing = "0.1.22" 21 | tracing-subscriber = {version="0.2.15", features = ["chrono", "env-filter", "fmt"]} 22 | 23 | sluice = "0.5.3" 24 | duplexify = "1.2.2" -------------------------------------------------------------------------------- /llvm-lines/main.rs: -------------------------------------------------------------------------------- 1 | pub use clepsydra::*; 2 | 3 | #[path = "../src/test.rs"] 4 | mod test; 5 | 6 | // This version of the test is a non-#[cfg(test)] binary so that 7 | // cargo llvm-lines can find it. See also ../src/main.rs 8 | 9 | pub fn main() { 10 | test::multi_txn_test(); 11 | } -------------------------------------------------------------------------------- /src/NOTES.md: -------------------------------------------------------------------------------- 1 | Module structure 2 | ================ 3 | 4 | Due to the way many parts of the OV protocol feed back on one another, we wind 5 | up defining basically everything _on_ this singular Database type. Methods are 6 | grouped by sub-protocols into separate modules (agreement, replication, etc.) 7 | but essentially there's no good way around the "god object" model here because 8 | that's just how the protocol is defined. 9 | 10 | 11 | Clone-self pattern on Database 12 | ============================== 13 | 14 | Database is Clone -- perhaps counterintuitively -- because all of its fields are 15 | `Arc<...>` and we clone it _entirely_ in order to capture its state into various 16 | task closures we spawn and futures we build in async move blocks. 17 | 18 | Subsequent methods are defined directly on Database rather than being 19 | free-standing functions that take a slew of cloned Arc<>s. Thanks to 20 | https://www.philipdaniels.com/blog/2020/self-cloning-for-multiple-threads-in-rust/ 21 | for the suggestion. 22 | 23 | 24 | Large number of traits required for Lang (pergola::Deftraits) 25 | ============================================================= 26 | 27 | There's no _good_ reason for Lang itself to extend all the bounds it requires, 28 | but it's necessary (as per bug https://github.com/rust-lang/rust/issues/26925) 29 | to make derive(...) work on structs that are themselves parameterized by a Lang 30 | instance. 31 | 32 | 33 | Concurrency bugs from dropped quorum futures 34 | ============================================ 35 | 36 | A note concerning sources of deadlock or other concurrency bugs in this program: 37 | if you're debugging what looks like a deadlock which manifests as service code 38 | for some RPC mysteriously "not finishing" (eg. not writing back to store, not 39 | releasing locks, etc.) there is a good chance this is because you've got a 40 | quorum-read or quorum-write threshold involved and the RPC's future is literally 41 | being abandoned part way through execution (at some await). Ideally this would 42 | never happen (tasks spawned, futures polled to completion, etc.) but it's an 43 | easy mistake to make, I've made it repeatedly. 44 | 45 | 46 | Strict serializability 47 | ====================== 48 | 49 | A note concerning strict serializability. Taking the jepsen.io definition: 50 | 51 | > if operation A completes before operation B begins, 52 | > then A should appear to precede B in the serialization order. 53 | 54 | Suppose A and B are both write-only transactions (the simplest kind) issued by 55 | different hosts with skewed clocks, so A is _issued_ before B in real time, but 56 | A has a later timestamp and B has an earlier timestamp, such that the 57 | serialization order winds up being B, A. Can we have A _complete_ before B? No: 58 | even write-only txs are only _acknowledged to the client_ after the 59 | visibility-watermark advances above the timestamp, so the fact that the 60 | timestamp order differs from the real-time order makes the txs appear to be 61 | "concurrent" from the perspective of the system, and the order in which it 62 | _completes_ a transaction will be the order of the watermark advancing past the 63 | txs, which will be the same _in real time_ as the serialization order. So it is 64 | in fact strictly serializable. The clock skew causes the system to have higher 65 | latency / a larger window of txs considered "concurrent" and thereby potentially 66 | having a different completion order than their issue order; so you do want the 67 | window to be kept small! But you get strict serializability regardless of any 68 | skew window. 69 | -------------------------------------------------------------------------------- /src/TODO.md: -------------------------------------------------------------------------------- 1 | 2 | # TODO 3 | 4 | Short term: 5 | =========== 6 | 7 | - [ ] find non-opt + debug-log-level concurrency bug (endless test execution) 8 | - [ ] refactor replication read and write functions 9 | - [ ] avoid double-read on store during eval -- integrate store-read with locks? 10 | - [ ] figure out sensible trigger conditions on lattice agreement rounds 11 | - [ ] maybe clean up the communication plumbing a bit, it's a mess 12 | - [ ] address question of straggling in watermarks. do we definitely wait on everyone? I guess so? 13 | - [ ] figure out where the reconfiguration safe-points are and how to detect / 14 | what to do about in-progress txns during reconfigurations. 15 | - [ ] switch everything to sorted_vec_maps 16 | 17 | Longer term 18 | =========== 19 | 20 | - [ ] evaluate the paper's "GC watermark" concept, see if it fits 21 | - [ ] implement some nontrivial Lang for testing at least 22 | - [ ] write preliminary log-structured storage backend with column sketches for Store 23 | - [ ] figure out if there's a nice way to have epochs in the peer-set lattice (& 24 | peer ID space -- or just stick with big-and-random IDs?) 25 | - [ ] add some kind of table / database / application-space identifiers 26 | - [ ] everything else... many TODOs scattered through code 27 | 28 | Done 29 | ==== 30 | 31 | - [X] clean up and document at least the public parts 32 | - [X] switch everything to tracing 33 | - [X] add fully-evaluated watermark to collect dead eval locks 34 | - [X] track successful write-set, and only finalize each after it's actually been written 35 | - [X] switch edelcrantz to postcard 36 | - [X] implement put() at end of evaluation 37 | - [X] implement read-from-local when under fully-replicated watermark 38 | - [X] fix locking / implement proper lock manager / deal with deadlock 39 | - [X] figure out correct arrangement of test/bin 40 | - [X] WONTFIX: fix lattice agreement / sending gets before vwatermark advances 41 | - [X] decide on a representation for GlobalTime. Maybe avoid being clever with 42 | bit-packing and let the storage layer do that per-segment via range and dictionary coding. 43 | -------------------------------------------------------------------------------- /src/agreement.rs: -------------------------------------------------------------------------------- 1 | // Copyright 2020 Graydon Hoare 2 | // Licensed under the MIT and Apache-2.0 licenses. 3 | 4 | //! This module instantiates the [concorde] reconfigurable lattice agreement 5 | //! crate with the [PeerID] and [crate::watermarks] types in clepsydra, such that 6 | //! we can use lattice agreement in place of the watermark-gossip and 7 | //! peer-configuration protocols in Ocean Vista. At least in theory. In practice 8 | //! this is still quite rough, but it does cause time to advange. 9 | 10 | use std::collections::BTreeMap; 11 | 12 | use crate::{ 13 | network::OneWay, Database, GlobalTime, GroupWatermarks, Lang, PeerID, RWatermark, 14 | ServerWatermarksLD, ServerWatermarksLE, ServerWatermarksLEExt, Store, Svw, VWatermark, 15 | }; 16 | use concorde::{CfgLE, CfgLEExt, StateLE, StateLEExt}; 17 | use futures::{channel::mpsc::UnboundedReceiver, stream::FusedStream, StreamExt}; 18 | use pergola::{BTreeMapWithUnion, LatticeElt}; 19 | use tracing::{debug, trace}; 20 | 21 | pub(crate) type Msg = concorde::Message; 22 | pub(crate) type Participant = concorde::Participant; 23 | 24 | /// The object lattice of our reconfigurable lattice agreement is a map from 25 | /// peer to the maximum watermarks on that peer. This means that during lattice 26 | /// agreement we'll "see" the advance of some peer's local Svw and Srw 27 | /// watermarks _when a quorum sees them_; but we'll later take the _minimum_ of 28 | /// all such "seen" per-peer watermarks when calculating the [[GroupWatermarks]] 29 | /// we decide to advance-to ourselves. 30 | pub(crate) type ObjLD = BTreeMapWithUnion; 31 | pub(crate) type ObjLE = LatticeElt; 32 | pub(crate) trait ObjLEExt { 33 | /// Return the minimum of all the per-peer [[Svw]] and [[Srw]] local watermarks in 34 | /// the system. This should be used to advance the local watermarks. 35 | /// 36 | /// Panics if any expected peer is missing. To avoid this, build a new ObjLE 37 | /// with default values for all expected peers and join it with this ObjLE. 38 | fn get_min_watermarks(&self, expected_peers: &Vec) -> GroupWatermarks; 39 | 40 | /// Update the watermark for the local peer. Panics if existing watermarks lattice 41 | /// element for local peer is not <= new element. 42 | fn advance_local_watermark( 43 | &mut self, 44 | local_peer: PeerID, 45 | server_watermarks: &ServerWatermarksLE, 46 | ); 47 | 48 | /// Builds a new ObjLE with a default watermark value for each peer. 49 | fn default_for_peers(peers: &Vec) -> Self; 50 | } 51 | 52 | impl ObjLEExt for LatticeElt { 53 | fn default_for_peers(peers: &Vec) -> Self { 54 | let mut m = BTreeMap::new(); 55 | for p in peers.iter() { 56 | m.insert(*p, ServerWatermarksLE::new_from_peer_zero_time(*p)); 57 | } 58 | LatticeElt::from(m) 59 | } 60 | 61 | fn advance_local_watermark( 62 | &mut self, 63 | local_peer: PeerID, 64 | server_watermarks: &ServerWatermarksLE, 65 | ) { 66 | let newle = ServerWatermarksLE::from(server_watermarks.clone()); 67 | match self.value.get(&local_peer) { 68 | None => (), 69 | Some(oldle) => { 70 | assert!(oldle <= &newle); 71 | } 72 | } 73 | self.value.insert(local_peer, newle); 74 | } 75 | 76 | fn get_min_watermarks(&self, expected_peers: &Vec) -> GroupWatermarks { 77 | if self.value.len() == 0 { 78 | return GroupWatermarks { 79 | visibility_watermark: VWatermark(GlobalTime::default()), 80 | replication_watermark: RWatermark(GlobalTime::default()), 81 | }; 82 | } 83 | for p in expected_peers.iter() { 84 | if !self.value.contains_key(p) { 85 | panic!("missing expected peer {:?}", p); 86 | } 87 | } 88 | let mut vw = GlobalTime::max_for(expected_peers[0]); 89 | let mut rw = GlobalTime::max_for(expected_peers[0]); 90 | for (_, v) in self.value.iter() { 91 | vw = std::cmp::min(vw, v.svw().value.0); 92 | rw = std::cmp::min(rw, v.srw().value.0); 93 | } 94 | GroupWatermarks { 95 | visibility_watermark: VWatermark(vw), 96 | replication_watermark: RWatermark(rw), 97 | } 98 | } 99 | } 100 | 101 | impl> Database { 102 | async fn get_expected_peers(&self) -> Vec { 103 | let guard = self.connections.read().await; 104 | let mut ps: Vec<_> = guard.keys().cloned().collect(); 105 | if !guard.contains_key(&self.self_id) { 106 | ps.push(self.self_id); 107 | } 108 | ps 109 | } 110 | 111 | /// Retrieves the final _group_ watermarks -- the minimum of all the server watermarks we 112 | /// learned about from lattice agreement. This should be sent to [[publish]]. 113 | pub(crate) async fn get_current_agreement(&self) -> Option { 114 | let expected_peers: Vec = self.get_expected_peers().await; 115 | match &self.participant.read().await.final_state { 116 | None => None, 117 | Some(statele) => Some(statele.object().get_min_watermarks(&expected_peers)), 118 | } 119 | } 120 | 121 | /// Starts a new proposal cycle of lattice-agreement based on the last lattice-agreement 122 | /// state and the current tidmgr's watermarks. 123 | pub(crate) async fn begin_new_proposal(&self) { 124 | let expected_peers: Vec = self.get_expected_peers().await; 125 | let server_watermarks = self.tidmgr.write().await.server_watermarks(); 126 | let (cfg, mut obj): (CfgLE, ObjLE) = 127 | match &self.participant.read().await.final_state { 128 | Some(sle) => (sle.config().clone(), sle.object().clone()), 129 | None => { 130 | let mut cfg = CfgLE::default(); 131 | let obj = ObjLE::default_for_peers(&expected_peers); 132 | for p in expected_peers.iter() { 133 | cfg.added_peers_mut().insert(*p); 134 | } 135 | (cfg, obj) 136 | } 137 | }; 138 | obj.advance_local_watermark(self.self_id, &server_watermarks); 139 | let prop: StateLE = (obj, cfg).into(); 140 | trace!( 141 | "begin new lattice-agreement proposal with {:?} members in config", 142 | prop.config().members().len() 143 | ); 144 | trace!("tidmgr wq_set: {:?}", self.tidmgr.read().await.wq_set); 145 | self.participant.write().await.propose(&prop) 146 | } 147 | 148 | pub(crate) async fn propose_step(&self, msgs: &[Msg]) { 149 | let mut out: Vec = Vec::new(); 150 | self.participant 151 | .write() 152 | .await 153 | .propose_step(msgs.iter(), &mut out); 154 | trace!( 155 | "lattice-agreement propose step, {} msgs in, {} msgs out", 156 | msgs.len(), 157 | out.len() 158 | ); 159 | for msg in out { 160 | match msg { 161 | concorde::Message::Request { to, .. } | concorde::Message::Response { to, .. } => { 162 | let ow = OneWay::LatticeAgreementMsg(msg); 163 | self.connections 164 | .read() 165 | .await 166 | .get(&to) 167 | .unwrap() 168 | .1 169 | .enqueue_oneway(ow) 170 | .await 171 | .unwrap(); 172 | } 173 | concorde::Message::Commit { .. } => { 174 | for pair in self.connections.read().await.values().cloned() { 175 | let ow = OneWay::LatticeAgreementMsg(msg.clone()); 176 | pair.1.enqueue_oneway(ow).await.unwrap(); 177 | } 178 | } 179 | } 180 | } 181 | } 182 | 183 | // Loops continuously reading from agreement_recv and feeding the concorde 184 | // state machine. 185 | // TODO: integrate better with edelcrantz messaging model 186 | // TODO: maybe insert a delay before cycling, and/or wait on received 187 | // changes, or changes to own tidmgr. 188 | 189 | const AGREE_LOOP_DELAY: std::time::Duration = std::time::Duration::from_millis(50); 190 | pub(crate) async fn run_proposal_loop(&self, mut agreement_recv: UnboundedReceiver) { 191 | while !agreement_recv.is_terminated() { 192 | self.begin_new_proposal().await; 193 | self.propose_step(&[]).await; 194 | while let Some(msg) = agreement_recv.next().await { 195 | self.propose_step(&[msg]).await; 196 | if self.participant.read().await.propose_is_fini() { 197 | break; 198 | } 199 | } 200 | match self.get_current_agreement().await { 201 | None => (), 202 | Some(gwm) => { 203 | self.publish(gwm).await; 204 | } 205 | } 206 | // TODO: This isn't how we want to throttle proposal loops. 207 | // let never = future::pending::<()>(); 208 | // let _ = future::timeout(Self::AGREE_LOOP_DELAY, never).await; 209 | } 210 | } 211 | 212 | /// From paper - Algorithm 1, procedure 'Publish' 213 | /// Called from network receiving an updated gossip watermark. 214 | // 215 | // TODO: maybe don't return Svw, seems pointless / related to DC-exchange we're not doing. 216 | // TODO: "the DB servers automatically set all versions below Rwatermark as finalized" 217 | pub(crate) async fn publish(&self, new_gwm: GroupWatermarks) -> Svw { 218 | { 219 | trace!( 220 | "saw lattice-agreement publish group watermarks {:?}", 221 | new_gwm 222 | ); 223 | let (lock, cvar) = &*self.group_wm; 224 | let mut gwm = lock.lock().await; 225 | let max_vwm = new_gwm 226 | .visibility_watermark 227 | .max(gwm.visibility_watermark.clone()); 228 | let max_rwm = new_gwm 229 | .replication_watermark 230 | .max(gwm.replication_watermark.clone()); 231 | if max_rwm != gwm.replication_watermark { 232 | debug!("Rwatermark advancing to {:?}", max_rwm.0); 233 | gwm.replication_watermark = max_rwm; 234 | } 235 | if max_vwm != gwm.visibility_watermark { 236 | debug!("Vwatermark advancing to {:?}", max_vwm.0); 237 | gwm.visibility_watermark = max_vwm; 238 | // Wake up transactions that are waiting on vwatermark. 239 | cvar.notify_all(); 240 | } 241 | } 242 | return self.tidmgr.write().await.svw(); 243 | } 244 | } 245 | -------------------------------------------------------------------------------- /src/database.rs: -------------------------------------------------------------------------------- 1 | // Copyright 2020 Graydon Hoare 2 | // Licensed under the MIT and Apache-2.0 licenses. 3 | 4 | use crate::{ 5 | agreement, network, Clock, Error, ExtVal, GlobalTime, GroupWatermarks, KeyVer, Lang, PeerID, 6 | RWatermark, Sdw, Store, Svw, SyncBoxFuture, TidMgr, Txn, VWatermark, 7 | }; 8 | use async_std::{ 9 | sync::{Arc, Condvar, Mutex, RwLock}, 10 | task, 11 | }; 12 | use futures::{stream::FuturesUnordered, Future, TryStreamExt}; 13 | use std::collections::{BTreeMap, BTreeSet, HashMap}; 14 | use tracing::{debug, debug_span, instrument, Instrument}; 15 | 16 | /// Main object that clients instantiate. Encapsulates the remainder of the 17 | /// system. Clients need to provide a [Clock] and [Store], as well as some 18 | /// number of async IO connections to other peers. 19 | pub struct Database> { 20 | pub self_id: PeerID, 21 | 22 | /// The current tidmgr, used to hand out transaction IDs / timestamps, and track 23 | /// the set of outstanding timestamps in various phases, and their minima. 24 | pub(crate) tidmgr: Arc>, 25 | 26 | // The Condvar here is used to wait on the completion of the S-phase 27 | // of a transaction. When the GroupWatermarks advances, transactions 28 | // waiting on it are woken and can consider whether it's time for them 29 | // keep waiting or proceed to E-phase. 30 | pub(crate) group_wm: Arc<(Mutex, Condvar)>, 31 | 32 | /// The current concorde state machine, used to propagate the watermark gossip. 33 | pub(crate) participant: Arc>, 34 | 35 | /// The map of network connections, served by network service loops. 36 | pub(crate) connections: Arc>>>, 37 | 38 | // A set of locks we hold during RPC demand-evaluation of a keyver to avoid 39 | // reentry; theoretically harmless to re-evaluate but since this is a lazy 40 | // system it can easily create multiplicative churn that's better avoided by 41 | // waiting for an existing evaluator call to finish. 42 | // 43 | // Paired with a usize to count the number of lock accesses; once every 44 | // LOCK_GC_FREQUENCY we purge old eval-locks below the delay watermark 45 | // since they will never be accessed again. 46 | pub(crate) eval_locks: Arc, Arc>>)>>, 47 | 48 | // Finally the store itself, where we read and write Store::Entries. 49 | pub(crate) store: Arc>, 50 | } 51 | 52 | impl> Clone for Database { 53 | fn clone(&self) -> Self { 54 | Database { 55 | self_id: self.self_id.clone(), 56 | tidmgr: self.tidmgr.clone(), 57 | group_wm: self.group_wm.clone(), 58 | participant: self.participant.clone(), 59 | connections: self.connections.clone(), 60 | eval_locks: self.eval_locks.clone(), 61 | store: self.store.clone(), 62 | } 63 | } 64 | } 65 | 66 | impl> Database { 67 | /// Construct a new Database for a given [Clock] and [Store]. The [PeerID] 68 | /// should be unique among peers in a group, and all peers should use 69 | /// compatible and roughly-synchronized clocks. 70 | pub fn new(self_id: PeerID, clock: Box, store: S) -> Self { 71 | let gwm = GroupWatermarks::new_from_peer_zero_time(self_id); 72 | Database { 73 | self_id: self_id.clone(), 74 | tidmgr: Arc::new(RwLock::new(TidMgr::new(self_id, clock))), 75 | group_wm: Arc::new((Mutex::new(gwm), Condvar::new())), 76 | participant: Arc::new(RwLock::new(concorde::Participant::new(self_id))), 77 | connections: Arc::new(RwLock::new(HashMap::new())), 78 | eval_locks: Arc::new(Mutex::new((0, HashMap::new()))), 79 | store: Arc::new(RwLock::new(store)), 80 | } 81 | } 82 | 83 | const LOCK_GC_FREQUENCY: usize = 128; 84 | 85 | async fn get_delayed_watermark(&self) -> Sdw { 86 | match self.store.read().await.get_delayed_watermark() { 87 | None => Sdw(self.tidmgr.write().await.create_timestamp()), 88 | Some(sdw) => sdw, 89 | } 90 | } 91 | 92 | pub(crate) async fn get_keylock(&self, kv: &KeyVer) -> Arc> { 93 | let mut guard = self.eval_locks.lock().await; 94 | 95 | // Maybe GC locks before issuing new one. 96 | guard.0 += 1; 97 | if guard.0 > Self::LOCK_GC_FREQUENCY { 98 | let pre_count = guard.1.len(); 99 | let Sdw(min_delayed_ver) = self.get_delayed_watermark().await; 100 | guard.1.retain(|k, _| k.ver >= min_delayed_ver); 101 | let post_count = guard.1.len(); 102 | debug!( 103 | "GC'ed {:?} key-locks, {:?} remaining", 104 | pre_count - post_count, 105 | post_count 106 | ); 107 | guard.0 = 0; 108 | } 109 | 110 | guard 111 | .1 112 | .entry(kv.clone()) 113 | .or_insert_with(|| Arc::new(Mutex::new(()))) 114 | .clone() 115 | } 116 | 117 | #[instrument(skip(self, quorum_futs))] 118 | async fn finalize_quorum_write_or_abort( 119 | &self, 120 | ver: GlobalTime, 121 | quorum_futs: FuturesUnordered, 122 | ) -> Result<(), Error> 123 | where 124 | QF: Future>, 125 | RF: Future, Error>> + Send + Sync + 'static, 126 | { 127 | // First we wait for all the quorum-{write,abort} futures to yield their 128 | // residual-{write,abort} futures. If anything goes wrong here we'll 129 | // return an Error and quorum-{write,abort} will not have succeeded. 130 | let residual_futs: FuturesUnordered<_> = quorum_futs.try_collect().await?; 131 | 132 | // Then we inform tidmgr that we've successfully made a quorum-{write,abort} so it 133 | // can advance the server-local visibility watermark Svw. 134 | debug!( 135 | "marking {:?} as stored (un-gating local server visibility watermark)", 136 | ver 137 | ); 138 | self.tidmgr.write().await.stored(ver); 139 | 140 | // Finally we spawn a sub-task that will drain the residual futures and 141 | // advance the tidmgr's server-local replication watermark Srw when we 142 | // have finished the {write,abort}-all phase. 143 | let this = self.clone(); 144 | task::spawn(async move { 145 | // TODO: the paper gives no guidance on what to do if there's a 146 | // failure while resolving the residual futures when finishing 147 | // up full replication. 148 | let _ = residual_futs.try_collect::>().await; 149 | debug!( 150 | "marking {:?} as fully replicated (un-gating local server repilcation watermark)", 151 | ver 152 | ); 153 | this.tidmgr.write().await.fully_replicated(ver); 154 | }); 155 | Ok(()) 156 | } 157 | 158 | /// Main entrypoint for clients submitting transactions. Call this and await 159 | /// the response. If the response is `Ok`, a transaction containing `stmt` 160 | /// and `vals` was successfully replicated to a quorum of peers and executed 161 | /// at a specific [GlobalTime] in the consensus sequential order. The result 162 | /// will contain the that timestamp as a map populated with any keys and 163 | /// their values (evaluated at the transaction's time) specified by 164 | /// `Lang::get_eval_set(stmt)`. 165 | #[instrument(skip(self))] 166 | pub fn coordinate( 167 | &self, 168 | stmt: L::Stmt, 169 | vals: Vec, 170 | ) -> SyncBoxFuture>), Error>> { 171 | let this = self.clone(); 172 | Box::pin(async move { 173 | let ver = this 174 | .tidmgr 175 | .write() 176 | .await 177 | .create_watermark_and_start_s_phase(); 178 | let span = debug_span!("coordinate", peer=?this.self_id, tid=?ver); 179 | this.coordinate_(ver, stmt, vals).instrument(span).await 180 | }) 181 | } 182 | 183 | // From paper -- Algorithm 1, procedure 'Coordinate' 184 | // Called from client initiating a new txn. 185 | async fn coordinate_( 186 | &self, 187 | ver: GlobalTime, 188 | stmt: L::Stmt, 189 | vals: Vec, 190 | ) -> Result<(GlobalTime, BTreeMap>), Error> { 191 | let txn = Txn { 192 | time: ver.clone(), 193 | stmt, 194 | vals, 195 | }; 196 | debug!("begin coordinate"); 197 | { 198 | debug!("begin S-phase"); 199 | let write_set = L::get_write_set(&txn.stmt); 200 | let quorum_write_futs: FuturesUnordered<_> = write_set 201 | .iter() 202 | .map(|(k, e)| { 203 | self.write( 204 | KeyVer { 205 | key: k.clone(), 206 | ver, 207 | }, 208 | e.clone(), 209 | txn.vals.clone(), 210 | ) 211 | }) 212 | .collect(); 213 | match self 214 | .finalize_quorum_write_or_abort(ver, quorum_write_futs) 215 | .await 216 | { 217 | Err(_) => { 218 | // We had a failure during quorum-write, so we now have to 219 | // quorum-abort. 220 | debug!("begin abort"); 221 | let quorum_abort_futs: FuturesUnordered<_> = write_set 222 | .keys() 223 | .map(|k| { 224 | self.abort_txn(KeyVer { 225 | key: k.clone(), 226 | ver, 227 | }) 228 | }) 229 | .collect(); 230 | 231 | // TODO: the paper is not at all clear about what we should 232 | // do if there's an error during quorum-abort, besides "this 233 | // cannot happen". For now we just ignore them. 234 | let _ = self 235 | .finalize_quorum_write_or_abort(ver, quorum_abort_futs) 236 | .await; 237 | return Err(Error::TxnAbort); 238 | } 239 | Ok(()) => (), 240 | } 241 | debug!("end S-phase"); 242 | } 243 | // In the paper this step is a bit unclear/incoherent, and 244 | // the actual call to "execute()" is launched from publish(). 245 | // This doesn't really make sense in our implementation, so we 246 | // pause here until it's time and then call execute() ourselves. 247 | // publish() un-gates us instead. It's _fairly_ close to the same. 248 | self.wait_for_visibility_watermark(txn.time).await; 249 | self.execute(txn, ver).await 250 | } 251 | 252 | pub(crate) async fn wait_for_visibility_watermark(&self, ver: GlobalTime) { 253 | let (lock, cvar) = &*self.group_wm; 254 | let tsvw = VWatermark(ver); 255 | debug!("waiting for VWatermark to advance past {:?}", ver); 256 | cvar.wait_until(lock.lock().await, |gw| tsvw < gw.visibility_watermark) 257 | .await; 258 | debug!("Vwatermark advanced past {:?}", ver); 259 | } 260 | 261 | #[instrument(skip(self))] 262 | fn execute( 263 | &self, 264 | tx: Txn, 265 | ts: GlobalTime, 266 | ) -> SyncBoxFuture>), Error>> { 267 | let this = self.clone(); 268 | Box::pin(async move { this.execute_(tx, ts).await }) 269 | } 270 | 271 | // From paper -- Algorithm 1, procedure 'Execute' 272 | // Called from publish, below. 273 | async fn execute_( 274 | &self, 275 | tx: Txn, 276 | ts: GlobalTime, 277 | ) -> Result<(GlobalTime, BTreeMap>), Error> { 278 | debug!("begin E-phase"); 279 | let write_set = L::get_write_set(&tx.stmt); 280 | let read_set: BTreeSet = write_set 281 | .iter() 282 | .map(|(_, expr)| L::get_read_set(expr)) 283 | .flatten() 284 | .collect(); 285 | 286 | let eval_set = L::get_eval_set(&tx.stmt); 287 | let read_set: BTreeSet = read_set.union(&eval_set).cloned().collect(); 288 | let ts_pre = ts.prev_event(); 289 | 290 | debug!("acquiring {:?} eval-locks", write_set.len()); 291 | let mut eval_arcs = Vec::new(); 292 | for k in write_set.keys() { 293 | let kv = KeyVer { 294 | key: k.clone(), 295 | ver: ts, 296 | }; 297 | let arc = self.get_keylock(&kv).await; 298 | eval_arcs.push(arc); 299 | } 300 | let mut eval_locks = Vec::new(); 301 | for arc in eval_arcs.iter() { 302 | eval_locks.push(arc.lock().await); 303 | } 304 | 305 | debug!("retrieving read set from previous timestamp {:?}", ts_pre); 306 | let read_futs: FuturesUnordered<_> = read_set 307 | .iter() 308 | .map(|k| { 309 | self.read(KeyVer { 310 | key: k.clone(), 311 | ver: ts_pre, 312 | }) 313 | }) 314 | .collect(); 315 | let env: BTreeMap<_, _> = read_futs.try_collect().await?; 316 | let mut out = BTreeMap::new(); 317 | debug!("evaluating exprs into write set"); 318 | for (key, expr) in write_set { 319 | out.insert(key, L::eval_expr(&expr, &tx.vals, &env)); 320 | } 321 | debug!("putting write set"); 322 | self.put(ts, &out).await?; 323 | // Finally return what the query asked for as a return value. 324 | debug!("calculating final value"); 325 | let mut res = BTreeMap::new(); 326 | for k in eval_set { 327 | match out.get(&k) { 328 | Some(v) => { 329 | res.insert(k, v.clone()); 330 | } 331 | None => match env.get(&k) { 332 | Some(v) => { 333 | res.insert(k, v.clone()); 334 | } 335 | None => return Err(Error::MissingKey), 336 | }, 337 | } 338 | } 339 | debug!("end E-phase"); 340 | Ok((ts, res)) 341 | } 342 | 343 | // Testing-only interface; bypasses consensus! 344 | pub(crate) async fn publish_an_hour_from_now(&self) -> Svw { 345 | let mut ts = self.tidmgr.write().await.create_timestamp(); 346 | ts.milli_secs += 3600 * 1000; 347 | let gwm = GroupWatermarks { 348 | visibility_watermark: VWatermark(ts), 349 | replication_watermark: RWatermark(ts), 350 | }; 351 | self.publish(gwm).await 352 | } 353 | } 354 | -------------------------------------------------------------------------------- /src/globaltime.rs: -------------------------------------------------------------------------------- 1 | // Copyright 2020 Graydon Hoare 2 | // Licensed under the MIT and Apache-2.0 licenses. 3 | 4 | use crate::PeerID; 5 | use serde::{Deserialize, Serialize}; 6 | 7 | /// GlobalTimes are the fundamental timekeeping type in the system. 8 | /// 9 | /// They have some interesting properties: 10 | /// 11 | /// - They are totally ordered. 12 | /// - They are assumed globally unique: there's a unique peer-ID component in 13 | /// each. 14 | /// - They are issued monotonicaly-increasing: there's an event-counter at the 15 | /// end in case real time moves backwards or stalls. 16 | /// - They are issued at each peer, _without_ coordination. 17 | /// - They do assume some level of real-time-clock availability on all peers 18 | /// generating them, but those clocks do not need to be very tightly 19 | /// synchronized. Larger skew will imply longer latency on each transaction 20 | /// but will not affect throughput as transactions execute asynchronously 21 | /// and in parallel. 22 | /// 23 | /// The `GlobalTime` type is used in a variety of contexts, some of which are 24 | /// wrapped in newtypes to make it a bit less likely to misuse them. They are 25 | /// sometimes called "transaction IDs" or "tids", or "timestamps", or "ts", or 26 | /// "versions", or the `ver` component of a [crate::KeyVer]. All these synonyms 27 | /// arise from the OV paper and are reproduced here to attempt to retain similar 28 | /// terminology in each algorithm and protocol message. The various "watermarks" 29 | /// tracking the sets of transactions in each phase across the system are also 30 | /// `GlobalTime`s. 31 | #[derive(Clone, Copy, Default, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 32 | pub struct GlobalTime { 33 | pub milli_secs: u64, 34 | pub peer: PeerID, 35 | pub event: u64, 36 | } 37 | 38 | impl std::fmt::Debug for GlobalTime { 39 | fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { 40 | f.write_fmt(format_args!( 41 | "{}.{}@{}", 42 | self.milli_secs, self.event, self.peer.0 43 | )) 44 | } 45 | } 46 | 47 | impl GlobalTime { 48 | pub fn new(peer: PeerID, milli_secs: u64, event: u64) -> GlobalTime { 49 | GlobalTime { 50 | peer, 51 | milli_secs, 52 | event, 53 | } 54 | } 55 | pub fn max_for(peer: PeerID) -> GlobalTime { 56 | GlobalTime::new(peer, u64::MAX, u64::MAX) 57 | } 58 | pub fn time_zero_for(peer: PeerID) -> GlobalTime { 59 | // We issue "time zero" at 1,1 because it's slightly nicer 60 | // to read logs that ask for "the previous event" when it 61 | // doesn't wrap around to u64::MAX 62 | GlobalTime::new(peer, 0, 1) 63 | } 64 | pub fn with_milli_sec(&self, milli_secs: u64) -> GlobalTime { 65 | Self { 66 | milli_secs, 67 | ..*self 68 | } 69 | } 70 | pub fn prev_milli_sec(&self) -> GlobalTime { 71 | let mut t = *self; 72 | t.milli_secs = t 73 | .milli_secs 74 | .checked_sub(1) 75 | .expect("globaltime millisecond underflow"); 76 | t 77 | } 78 | pub fn next_milli_sec(&self) -> GlobalTime { 79 | let mut t = *self; 80 | t.milli_secs = t 81 | .milli_secs 82 | .checked_add(1) 83 | .expect("globaltime millisecond overflow"); 84 | t 85 | } 86 | pub fn with_event(&self, event: u64) -> GlobalTime { 87 | Self { event, ..*self } 88 | } 89 | pub fn prev_event(&self) -> GlobalTime { 90 | match self.event { 91 | 0 => self.prev_milli_sec().with_event(u64::MAX), 92 | v => self.with_event(v - 1), 93 | } 94 | } 95 | pub fn next_event(&self) -> GlobalTime { 96 | match self.event { 97 | u64::MAX => self.next_milli_sec().with_event(0), 98 | v => self.with_event(v + 1), 99 | } 100 | } 101 | } 102 | -------------------------------------------------------------------------------- /src/language.rs: -------------------------------------------------------------------------------- 1 | // Copyright 2020 Graydon Hoare 2 | // Licensed under the MIT and Apache-2.0 licenses. 3 | 4 | use pergola::DefTraits; 5 | use serde::{Deserialize, Serialize}; 6 | use std::collections::{BTreeMap, BTreeSet}; 7 | 8 | /// An `ExtVal` extends the normal [Lang::Val] type with two extra sentinel 9 | /// values to represent not-yet-written or deleted data in a [crate::Store]. 10 | #[serde(bound = "")] 11 | #[derive(Clone, Debug, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 12 | pub enum ExtVal { 13 | Initial, 14 | Defined(L::Val), 15 | Deleted, 16 | } 17 | 18 | /// A `Lang` provides a skeletal interface between the database and a given 19 | /// client's query language. Clients of this crate need to define a `Lang` that 20 | /// models their language at least enough that the database can call it back to 21 | /// extract key-sets and perform (deterministic) evaluation. 22 | pub trait Lang: DefTraits + 'static { 23 | type Key: DefTraits + Send + Sync; 24 | type Val: DefTraits + Send + Sync; 25 | type Stmt: DefTraits + Send + Sync; 26 | type Expr: DefTraits + Send + Sync; 27 | 28 | fn get_write_set(s: &Self::Stmt) -> BTreeMap; 29 | fn get_read_set(e: &Self::Expr) -> BTreeSet; 30 | fn get_eval_set(s: &Self::Stmt) -> BTreeSet; 31 | fn eval_expr( 32 | e: &Self::Expr, 33 | vals: &[Self::Val], 34 | env: &BTreeMap>, 35 | ) -> ExtVal; 36 | } 37 | -------------------------------------------------------------------------------- /src/lib.rs: -------------------------------------------------------------------------------- 1 | // Copyright 2020 Graydon Hoare 2 | // Licensed under the MIT and Apache-2.0 licenses. 3 | 4 | //! # Overview 5 | //! 6 | //! This is a work-in-progress implementation of a core protocol for a minimalist 7 | //! distributed database. It strives to be as small and simple as possible while 8 | //! attempting to provide relatively challenging features: 9 | //! 10 | //! - Strict Serializability 11 | //! 12 | //! - Online Reconfiguration 13 | //! 14 | //! - Fault tolerance 15 | //! 16 | //! - High throughput 17 | //! 18 | //! The implementation is based on a simplified version of the "Ocean Vista" 19 | //! (OV) protocol, and uses its terminology wherever possible. OV combines 20 | //! replication, transaction commitment and concurrency control into a single 21 | //! protocol. 22 | //! 23 | //! ## Summary 24 | //! 25 | //! The short version of the protocol is: 26 | //! 27 | //! - Transactions are represented as deterministic thunks over snapshots. 28 | //! 29 | //! - Each transaction is assigned a globally-unique timestamp. 30 | //! 31 | //! - Transactions are separated into two phases: S-phase and E-phase. 32 | //! 33 | //! - S-phase (storage) consists of coordination-free "blind quorum-writes" 34 | //! replicating the thunks into their MVCC order on each replica. 35 | //! 36 | //! - A watermark tracking minimum transaction timestamps-being-written is 37 | //! gossiped between peers, increasing as quorum-writes complete. 38 | //! 39 | //! - A transaction only enters E-phase after the watermark advances past it. 40 | //! 41 | //! - E-phase (evaluation) quorum-reads and evaluates thunks from consistent 42 | //! snapshots below the watermark, lazily resolving any earlier thunks. 43 | //! Everything below the watermark is coordination-free and deterministic. 44 | //! 45 | //! ## Caveats 46 | //! 47 | //! Nothing's perfect, and this crate is anything but: 48 | //! 49 | //! - This crate is very incomplete and does not work yet. Don't use it for 50 | //! anything other than experiments and toys. Recovery, reconfiguration, 51 | //! timeouts and nontrivial fault tolerance paths _definitely_ don't work. 52 | //! 53 | //! - It also (somewhat recklessly) attempts to combine OV's reconfiguration 54 | //! and gossip protocols into an instance of the [concorde] reconfigurable 55 | //! lattice agreement protocol. This might not even be _theoretically_ safe. 56 | //! 57 | //! - It is much more minimal than the full OV protocol: there's no support 58 | //! for sharding, nor the two-level peer-vs-datacenter locality organization. 59 | //! This crate treats its whole peer group as a single symmetric shard. 60 | //! 61 | //! - As a result, performance won't be "webscale" or anything. It will scale 62 | //! vertically if you throw cores at it, but no better, and its latency will 63 | //! always have speed-of-light WAN RTT factors in it. It's distributed for 64 | //! fault tolerance, not horizontal scaling. 65 | //! 66 | //! - As with OV, this crate does require partial clock synchronization. It 67 | //! doesn't need to be very tight: clock drift only causes increased 68 | //! latency as the watermarks progress as the minimum of all times; it 69 | //! doesn't affect correctness. Normal weak-NTP-level sync should be ok. 70 | //! 71 | //! - As with OV, Calvin, and all deterministic databases: your txns have to be 72 | //! deterministic and must have deterministic _read and write sets_. If they 73 | //! cannot have their read and write sets statically computed (eg. if they 74 | //! rely on the data to decide read and write set) you have to build slightly 75 | //! awkward multi-phase txns. The term in the literature is "reconnaisance 76 | //! queries". 77 | //! 78 | //! ## Reference 79 | //! 80 | //! Hua Fan and Wojciech Golab. Ocean Vista: Gossip-Based Visibility Control for 81 | //! Speedy Geo-Distributed Transactions. PVLDB, 12(11): 1471-1484, 2019. 82 | //! 83 | //! DOI: 84 | //! 85 | //! 86 | //! 87 | //! ## Name 88 | //! 89 | //! Wikipedia: 90 | //! 91 | //! > A water clock or clepsydra (Greek κλεψύδρα from κλέπτειν kleptein, 'to 92 | //! > steal'; ὕδωρ hydor, 'water') is any timepiece by which time is measured by 93 | //! > the regulated flow of liquid into (inflow type) or out from (outflow type) 94 | //! > a vessel, and where the amount is then measured. 95 | //! 96 | 97 | #![allow(dead_code)] 98 | 99 | use futures::Future; 100 | use serde::{Deserialize, Serialize}; 101 | use std::{fmt::Debug, pin::Pin}; 102 | use thiserror::Error; 103 | 104 | #[derive(Error, Debug, Clone, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 105 | pub enum Error { 106 | #[error("Txn was aborted")] 107 | TxnAbort, 108 | #[error("Requested key was neither read nor written")] 109 | MissingKey, 110 | #[error("Replication failed")] 111 | ReplicationFailed, 112 | #[error("Read failed")] 113 | ReadFailed, 114 | #[error("Inconsistent replicas")] 115 | InconsistentReplicas, 116 | #[error("Too few replicas")] 117 | TooFewReplicas, 118 | #[error("Networking error")] 119 | NetworkingError, 120 | #[error("Unexpected response")] 121 | UnexpectedResponse, 122 | } 123 | 124 | impl From for Error { 125 | fn from(_: edelcrantz::Error) -> Self { 126 | Error::NetworkingError 127 | } 128 | } 129 | 130 | mod agreement; 131 | mod database; 132 | mod globaltime; 133 | mod language; 134 | mod network; 135 | mod quorum; 136 | mod replication; 137 | mod storage; 138 | mod tidmgr; 139 | mod transaction; 140 | mod watermarks; 141 | 142 | // We define a BoxFuture-like wrapper type here and wrap most of our nontrivial 143 | // async fn calls in it, for compilation and code footprint reasons: it costs an 144 | // extra heap allocation per async call, but means the library compiles faster, 145 | // can handle recursive futures, and doesn't require compiler pragmas to 146 | // override the maximum allowed type size. 147 | // 148 | // We don't use the standard BoxFuture type because we want our boxed futures to 149 | // also implement Sync, which the standard one doesn't. 150 | type SyncBoxFuture = Pin + 'static + Send + Sync>>; 151 | 152 | pub use database::Database; 153 | pub use globaltime::GlobalTime; 154 | pub use language::{ExtVal, Lang}; 155 | pub use network::PeerID; 156 | pub(crate) use quorum::{majority_quorum, super_quorum}; 157 | pub use replication::ReplicationTag; 158 | pub use storage::{Entry, KeyVer, Store}; 159 | pub(crate) use tidmgr::TidMgr; 160 | pub use tidmgr::{Clock, RealClock, TestClock}; 161 | pub(crate) use transaction::Txn; 162 | pub use watermarks::Sdw; 163 | pub(crate) use watermarks::{ 164 | GroupWatermarks, RWatermark, ServerWatermarksLD, ServerWatermarksLE, ServerWatermarksLEExt, 165 | Srw, Svw, VWatermark, 166 | }; 167 | -------------------------------------------------------------------------------- /src/main.rs: -------------------------------------------------------------------------------- 1 | // Copyright 2020 Graydon Hoare 2 | // Licensed under the MIT and Apache-2.0 licenses. 3 | 4 | pub use clepsydra::*; 5 | 6 | #[cfg(test)] 7 | mod test; 8 | 9 | // We have this test external from the crate for two reasons: 10 | // 11 | // 1. to make sure the public API is usable without accidentally relying on 12 | // crate-level-visibility stuff. 13 | // 14 | // 2. to use `cargo llvm-lines` in the llvm-lines/ subdirectory, to measure 15 | // footprint of final codegen when everything's actually instantiated. 16 | 17 | #[test] 18 | fn multi_txn_test() { 19 | test::multi_txn_test(); 20 | } 21 | 22 | pub fn main() { 23 | println!("please run `cargo test` instead"); 24 | } 25 | -------------------------------------------------------------------------------- /src/network.rs: -------------------------------------------------------------------------------- 1 | // Copyright 2020 Graydon Hoare 2 | // Licensed under the MIT and Apache-2.0 licenses. 3 | 4 | //! This module glues together a set of peer connections and their IO service 5 | //! loops provided by edelcrantz, a set of RPC sending and receiving async 6 | //! methods on Database, and the concorde lattice agreement state machine. It's 7 | //! a mess of uninteresting plumbing that should be cleaned up eventually. 8 | 9 | use crate::{ 10 | replication::{self}, 11 | Database, Store, SyncBoxFuture, 12 | }; 13 | use crate::{KeyVer, Lang}; 14 | use async_std::{ 15 | sync::{Arc, Mutex}, 16 | task, 17 | }; 18 | use futures::{ 19 | channel::mpsc::{self, UnboundedSender}, 20 | stream::FuturesUnordered, 21 | StreamExt, 22 | }; 23 | use serde::{Deserialize, Serialize}; 24 | use tracing::{debug, debug_span, trace, warn, Instrument}; 25 | 26 | /// A simple "peer identifier" which should be unique across any present or 27 | /// future configuration of a peer group. A randomly-chosen u64 should suffice. 28 | /// 29 | /// Used when tracking quorums and configuration members both in this crate and 30 | /// as a parameter to the lattice agreement protocol in [concorde]. 31 | #[derive(Clone, Copy, Default, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 32 | pub struct PeerID(pub u64); 33 | 34 | impl std::fmt::Debug for PeerID { 35 | fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { 36 | f.write_fmt(format_args!("#{}", self.0)) 37 | } 38 | } 39 | 40 | type Conn = edelcrantz::Connection, Req, Res>; 41 | type Queue = edelcrantz::Queue, Req, Res>; 42 | 43 | pub(crate) type Connection = (Arc>>, Queue); 44 | 45 | pub(crate) type ResponseFuture = SyncBoxFuture<(PeerID, Result, edelcrantz::Error>)>; 46 | 47 | pub(crate) type ResponseFutures = FuturesUnordered>; 48 | 49 | #[serde(bound = "")] 50 | #[derive(Clone, Debug, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 51 | pub(crate) enum Req { 52 | WriteReq(replication::WriteRequest), 53 | GetReq(replication::GetRequest), 54 | } 55 | #[serde(bound = "")] 56 | #[derive(Clone, Debug, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 57 | pub(crate) enum Res { 58 | WriteRes(replication::WriteResponse), 59 | GetRes(replication::GetResponse), 60 | } 61 | #[serde(bound = "")] 62 | #[derive(Clone, Debug, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 63 | pub(crate) enum OneWay { 64 | LatticeAgreementMsg(crate::agreement::Msg), 65 | PutMsg(replication::PutOneWay), 66 | } 67 | 68 | impl> Database { 69 | /// Adds a connection to a remote [PeerID]. The `io` handle should be a 70 | /// full-duplex `AsyncRead+AsyncWrite` type. 71 | pub async fn connect(&self, peer: PeerID, io: IO) { 72 | debug!( 73 | "connecting peers self={:?} to other={:?}", 74 | self.self_id, peer 75 | ); 76 | let conn = edelcrantz::Connection::new(io); 77 | let queue = conn.queue.clone(); 78 | let pair = (Arc::new(Mutex::new(conn)), queue); 79 | let mut net_guard = self.connections.write().await; 80 | net_guard.insert(peer, pair); 81 | } 82 | 83 | pub(crate) fn serve_req(&self, req: Req) -> SyncBoxFuture> { 84 | let this = self.clone(); 85 | Box::pin(async move { this.serve_req_(req).await }) 86 | } 87 | 88 | pub(crate) async fn serve_req_(&self, req: Req) -> Res { 89 | match req { 90 | Req::WriteReq(wr) => { 91 | trace!("write request {:?}", wr); 92 | let res = self.serve_write(wr).await; 93 | trace!("write response {:?}", res); 94 | Res::WriteRes(res) 95 | } 96 | Req::GetReq(gr) => { 97 | trace!("get request {:?}", gr); 98 | let res = self.serve_get(gr).await; 99 | trace!("get response {:?}", res); 100 | Res::GetRes(res) 101 | } 102 | } 103 | } 104 | 105 | pub(crate) fn serve_oneway( 106 | &self, 107 | remote: PeerID, 108 | agreement_send: UnboundedSender, 109 | putmsg_send: UnboundedSender<(PeerID, crate::replication::PutOneWay)>, 110 | ow: OneWay, 111 | ) { 112 | match ow { 113 | OneWay::LatticeAgreementMsg(msg) => { 114 | trace!("agreement msg {:?}", msg); 115 | agreement_send.unbounded_send(msg).unwrap(); 116 | } 117 | OneWay::PutMsg(put) => { 118 | trace!("put msg {:?}", put); 119 | putmsg_send.unbounded_send((remote, put)).unwrap(); 120 | } 121 | } 122 | } 123 | 124 | /// Start network-service worker tasks on this database. Should be called 125 | /// before any calls to [Database::coordinate]. 126 | pub fn launch_workers(&self) -> SyncBoxFuture<()> { 127 | let this: Database = self.clone(); 128 | let fut = async move { 129 | // TODO: the plumbing around the oneways is awkward. They 130 | // could/should share service task and/or the agreement 131 | // state-machine's requests and responses could be routed into 132 | // edelcrantz requests and responses also. And/or oneway processing 133 | // inside edelcrantz could itself be made async with an internal 134 | // mpsc. Many options for tidying this up. 135 | let (agreement_send, agreement_recv) = mpsc::unbounded(); 136 | let (putmsg_send, mut putmsg_recv) = mpsc::unbounded(); 137 | task::spawn({ 138 | let this = this.clone(); 139 | let span = debug_span!("put", peer=?this.self_id); 140 | (async move { 141 | let mut p: usize = 0; 142 | while let Some((remote, msg)) = putmsg_recv.next().await { 143 | // TODO: maybe don't ignore put-failures. 144 | p += 1; 145 | let this = this.clone(); 146 | let span = debug_span!("put-rpc-task", peer=?this.self_id, ?remote, ?p); 147 | task::spawn(async move { this.serve_put(msg).await }.instrument(span)); 148 | } 149 | }) 150 | .instrument(span) 151 | }); 152 | task::spawn({ 153 | let this = this.clone(); 154 | let span = debug_span!("agreement", peer=?this.self_id); 155 | async move { 156 | this.run_proposal_loop(agreement_recv) 157 | .instrument(span) 158 | .await; 159 | } 160 | }); 161 | for (id, conn) in this.connections.read().await.iter() { 162 | let id = id.clone(); 163 | let conn = conn.0.clone(); 164 | let a_send = agreement_send.clone(); 165 | let p_send = putmsg_send.clone(); 166 | task::spawn({ 167 | let this = this.clone(); 168 | async move { 169 | let mut n: usize = 0; 170 | loop { 171 | n += 1; 172 | let sreq = { 173 | let this = this.clone(); 174 | let span = 175 | debug_span!("rpc-task", peer=?this.self_id, remote=?id, ?n); 176 | move |req| { 177 | task::spawn( 178 | async move { this.serve_req(req).await }.instrument(span), 179 | ) 180 | } 181 | }; 182 | let sow = { 183 | let this = this.clone(); 184 | let a_send = a_send.clone(); 185 | let p_send = p_send.clone(); 186 | move |ow| this.serve_oneway(id, a_send, p_send, ow) 187 | }; 188 | let mut guard = conn.lock().await; 189 | trace!( 190 | "service worker task on {:?} talking to {:?} advancing (step {:?})", 191 | this.self_id, 192 | id, 193 | n 194 | ); 195 | let span = debug_span!("service", peer=?this.self_id, remote=?id); 196 | let res = guard.advance(sreq, sow).instrument(span).await; 197 | trace!( 198 | "service worker task on {:?} talking to {:?} advanced (step {:?}), got result {:?}", 199 | this.self_id, 200 | id, 201 | n, 202 | res 203 | ); 204 | match res { 205 | Ok(()) => (), 206 | // TODO: A dropped response-channel means that someone 207 | // sent a request (eg. a quorum-read) and then 208 | // stopped listening for the response (eg. 209 | // because they already got a quorum from some 210 | // other responses). This is not an error from 211 | // our perspective, currently. It might be in 212 | // the future, if we rework things. 213 | Err(edelcrantz::Error::ResponseChannelDropped(_)) => (), 214 | Err(_) => break, 215 | } 216 | } 217 | warn!( 218 | "service worker task on {:?} talking to {:?} exited", 219 | this.self_id, id 220 | ); 221 | } 222 | }); 223 | } 224 | }; 225 | Box::pin(fut) 226 | } 227 | 228 | pub(crate) async fn send_write_to_all( 229 | &self, 230 | kv: KeyVer, 231 | e: L::Expr, 232 | vals: Vec, 233 | ) -> ResponseFutures { 234 | let req = Req::WriteReq(replication::WriteRequest::Write(kv, e, vals)); 235 | self.send_req_to_all(req).await 236 | } 237 | 238 | pub(crate) async fn send_abort_to_all(&self, kv: KeyVer) -> ResponseFutures { 239 | let req = Req::WriteReq(replication::WriteRequest::Abort(kv)); 240 | self.send_req_to_all(req).await 241 | } 242 | 243 | pub(crate) async fn send_finalize_to_peers( 244 | &self, 245 | kv: KeyVer, 246 | peers: Vec, 247 | ) -> ResponseFutures { 248 | let req = Req::WriteReq(replication::WriteRequest::Finalize(kv)); 249 | self.send_req_to_peers(req, peers).await 250 | } 251 | 252 | pub(crate) async fn send_get_to_all(&self, kv: KeyVer) -> ResponseFutures { 253 | let req = Req::GetReq(replication::GetRequest(kv)); 254 | self.send_req_to_all(req).await 255 | } 256 | 257 | pub(crate) fn send_req_to_self(&self, req: Req) -> ResponseFuture { 258 | let this = self.clone(); 259 | Box::pin(async move { (this.self_id, Ok(this.serve_req(req).await)) }) 260 | } 261 | 262 | pub(crate) async fn send_req_to_all(&self, req: Req) -> ResponseFutures { 263 | let mut peers: Vec = self.connections.read().await.keys().cloned().collect(); 264 | peers.push(self.self_id); 265 | self.send_req_to_peers(req, peers).await 266 | } 267 | 268 | pub(crate) async fn send_req_to_peer(&self, req: Req, peer: PeerID) -> ResponseFuture { 269 | if peer == self.self_id { 270 | self.send_req_to_self(req.clone()) 271 | } else { 272 | match self.connections.read().await.get(&peer.clone()) { 273 | None => Box::pin({ 274 | let peer = peer.clone(); 275 | async move { (peer, Err(edelcrantz::Error::Queue)) } 276 | }), 277 | Some((_, queue)) => { 278 | let fut = queue.enqueue_request(req.clone()); 279 | Box::pin(async move { (peer, fut.await) }) 280 | } 281 | } 282 | } 283 | } 284 | 285 | pub(crate) async fn send_req_to_peers( 286 | &self, 287 | req: Req, 288 | peers: Vec, 289 | ) -> ResponseFutures { 290 | trace!("broadcasting request {:?} to peers {:?}", req, peers); 291 | let futs: ResponseFutures = FuturesUnordered::new(); 292 | for peer in peers { 293 | futs.push(self.send_req_to_peer(req.clone(), peer).await); 294 | } 295 | futs 296 | } 297 | } 298 | -------------------------------------------------------------------------------- /src/quorum.rs: -------------------------------------------------------------------------------- 1 | // Copyright 2020 Graydon Hoare 2 | // Licensed under the MIT and Apache-2.0 licenses. 3 | 4 | //! This module defines arithmetic of quorum sizes and tests that that 5 | //! arithmetic has expected values. It's very small but this is one of the minor 6 | //! bits of fiddly arithmetic that it's easy to mess up in a quorum-based 7 | //! system. 8 | 9 | // Many systems derive their operational thresholds from a user-supplied value 10 | // `f` for the number of tolerable failures, then reporting to the user a given 11 | // set of peers to provide. 12 | // 13 | // We take a slightly different approach and derive our thresholds from the 14 | // number of replicas that exist, _reporting_ the number of tolerable failures 15 | // with that number of replicas. 16 | // 17 | // We do this because peers may join or leave the system dynamically and we want 18 | // to be able to answer questions about quorum-sizes based on the number of 19 | // replicas we currently _have_ rather than the ideal configuration. 20 | // 21 | // The user can be notified -- and the system can optionally halt -- if we're 22 | // operating with fewer peers than one needs to achieve the user's preference 23 | // for failure-tolerance, but if we're at-or-above it we still want to calculate 24 | // the thresholds for switching between algorithm modes based on the number of 25 | // peers we _have_. 26 | // 27 | // 28 | // Formulating in terms of failures tolerated gives: 29 | // 30 | // failures | total | majority | super | 31 | // tolerated | replicas | quorum | quorum | 32 | // `f` | `2f + 1` | `f + 1` | 'ceil(3/2 * f) + 1` | 33 | // -------------|-----------|----------|----------------------| 34 | // 0 | 1 | 1 | 1 | 35 | // 1 | 3 | 2 | 3 | 36 | // 2 | 5 | 3 | 4 | 37 | // 3 | 7 | 4 | 6 | 38 | // 4 | 9 | 5 | 7 | 39 | // 40 | // 41 | // Reformulated in terms of number of replicas gives: 42 | // 43 | // failures | total | majority | super | 44 | // tolerated | replicas | quorum | quorum | 45 | // `(n-1)/2` | `n` | `(n/2) + 1` | `(3n/4) + 1` | 46 | // -------------|-----------|--------------|---------------| 47 | // 0 | 1 | 1 | 1 | 48 | // 0 | 2 | 2 | 2 | 49 | // 1 | 3 | 2 | 3 | 50 | // 1 | 4 | 3 | 4 | 51 | // 2 | 5 | 3 | 4 | 52 | // 2 | 6 | 4 | 5 | 53 | // 3 | 7 | 4 | 6 | 54 | // 3 | 8 | 5 | 7 | 55 | // 4 | 9 | 5 | 7 | 56 | // 4 | 10 | 6 | 8 | 57 | // 58 | 59 | // Returns the usize corresponding to ceil(x/y) 60 | fn ceil_div(x: usize, y: usize) -> usize { 61 | if x == 0 { 62 | 0 63 | } else { 64 | 1 + ((x - 1) / y) 65 | } 66 | } 67 | 68 | pub(crate) fn failures_tolerated(replica_count: usize) -> usize { 69 | if replica_count == 0 { 70 | 0 71 | } else { 72 | (replica_count - 1) / 2 73 | } 74 | } 75 | 76 | pub(crate) fn majority_quorum(replica_count: usize) -> usize { 77 | (replica_count / 2) + 1 78 | } 79 | 80 | pub(crate) fn super_quorum(replica_count: usize) -> usize { 81 | ((3 * replica_count) / 4) + 1 82 | } 83 | 84 | #[test] 85 | fn test_quorums() { 86 | assert_eq!(failures_tolerated(1), 0); 87 | assert_eq!(failures_tolerated(2), 0); 88 | assert_eq!(failures_tolerated(3), 1); 89 | assert_eq!(failures_tolerated(4), 1); 90 | assert_eq!(failures_tolerated(5), 2); 91 | assert_eq!(failures_tolerated(6), 2); 92 | assert_eq!(failures_tolerated(7), 3); 93 | assert_eq!(failures_tolerated(8), 3); 94 | assert_eq!(failures_tolerated(9), 4); 95 | assert_eq!(failures_tolerated(10), 4); 96 | 97 | assert_eq!(majority_quorum(1), 1); 98 | assert_eq!(majority_quorum(2), 2); 99 | assert_eq!(majority_quorum(3), 2); 100 | assert_eq!(majority_quorum(4), 3); 101 | assert_eq!(majority_quorum(5), 3); 102 | assert_eq!(majority_quorum(6), 4); 103 | assert_eq!(majority_quorum(7), 4); 104 | assert_eq!(majority_quorum(8), 5); 105 | assert_eq!(majority_quorum(9), 5); 106 | assert_eq!(majority_quorum(10), 6); 107 | 108 | assert_eq!(super_quorum(1), 1); 109 | assert_eq!(super_quorum(2), 2); 110 | assert_eq!(super_quorum(3), 3); 111 | assert_eq!(super_quorum(4), 4); 112 | assert_eq!(super_quorum(5), 4); 113 | assert_eq!(super_quorum(6), 5); 114 | assert_eq!(super_quorum(7), 6); 115 | assert_eq!(super_quorum(8), 7); 116 | assert_eq!(super_quorum(9), 7); 117 | assert_eq!(super_quorum(10), 8); 118 | } 119 | -------------------------------------------------------------------------------- /src/replication.rs: -------------------------------------------------------------------------------- 1 | // Copyright 2020 Graydon Hoare 2 | // Licensed under the MIT and Apache-2.0 licenses. 3 | 4 | //! The replication sub-protocol of OV attempts to ensure that: 5 | //! 6 | //! - Every write is sufficiently replicated during S-phase. The writing tx is 7 | //! un-blocked once all its writes are acknowledged from a quorum ("WQ") but 8 | //! writing (and finalizing) then continues _asynchronously_ to all 9 | //! remaining peers including stragglers ("WA"). If the WQ part hears 10 | //! success from a super-quorum it can unblock its writing tx after only 1 11 | //! WAN RTT, otherwise (if only a simple-majority quorum responds in time) 12 | //! it takes a second RTTs to finalize and receive confirmations. 13 | //! 14 | //! - Every read succeeds (up to fault-tolerance limit) while being serviced 15 | //! by as small a number of replicas as possible: if the read is below 16 | //! Rwatermark, we can fetch from any store directly on any single peer and 17 | //! expect success, including the local peer ("RO"). If the read is above 18 | //! Rwatermark, we have to do a quorum-read ("RQ") because the value might 19 | //! not be fully replicated yet. 20 | //! 21 | //! In other words, in the good/common case we should be able to get by with 22 | //! 1-RTT WQRO, which is the best you can really do if you want replication at 23 | //! all. 24 | 25 | use crate::{ 26 | majority_quorum, network, super_quorum, Database, Entry, Error, ExtVal, KeyVer, PeerID, 27 | RWatermark, Store, SyncBoxFuture, 28 | }; 29 | use crate::{GlobalTime, Lang}; 30 | use async_std::future; 31 | use futures::TryStreamExt; 32 | use futures_util::{stream::FuturesUnordered, StreamExt}; 33 | use serde::{Deserialize, Serialize}; 34 | use std::{ 35 | collections::{BTreeMap, BTreeSet}, 36 | future::Future, 37 | time::SystemTime, 38 | }; 39 | use tracing::{debug, debug_span, error, Instrument}; 40 | 41 | #[serde(bound = "")] 42 | #[derive(Clone, Debug, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 43 | pub(crate) enum WriteRequest { 44 | Write(KeyVer, L::Expr, Vec), 45 | Abort(KeyVer), 46 | Finalize(KeyVer), 47 | } 48 | 49 | #[serde(bound = "")] 50 | #[derive(Clone, Debug, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 51 | pub(crate) enum WriteResponse { 52 | Success, // First-round response to Write or Abort (writes can't fail) 53 | Confirm, // Second-round response to Finalize (tentative writes _can_ time out) 54 | Failure, // Something about write failed; should be impossible, represents a 55 | // corrupt packet or logic error or more failed nodes than we can tolerate 56 | // or something. 57 | } 58 | 59 | /// Designates which phase of quorum-writing a given [crate::Entry] was stored 60 | /// in. Must be stored and retieved along with entries in the [crate::Store]. 61 | #[serde(bound = "")] 62 | #[derive(Clone, Debug, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 63 | pub enum ReplicationTag { 64 | Tentative, 65 | Finalized, 66 | } 67 | 68 | /// At the end of evaluation, coordinators `Put` the finished ExtVal of each key 69 | /// back to each replica. This is a one-way message and if it fails or is lost 70 | /// the ExtVal can and will be lazily but deterministically re-evaluated on each 71 | /// replica as needed. 72 | #[serde(bound = "")] 73 | #[derive(Clone, Debug, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 74 | pub(crate) struct PutOneWay(KeyVer, ExtVal); 75 | 76 | #[serde(bound = "")] 77 | #[derive(Clone, Debug, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 78 | pub(crate) struct GetRequest(pub KeyVer); 79 | 80 | #[serde(bound = "")] 81 | #[derive(Clone, Debug, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 82 | pub(crate) enum GetResponse { 83 | SettledEntry(GlobalTime, ReplicationTag, ExtVal), 84 | // TODO: it's not clear that an explicit AbortedEntry response is necessary 85 | // or even makes sense; maybe storage should just return the last 86 | // non-aborted before query? 87 | AbortedEntry(GlobalTime), 88 | Failure, // Again, something unforeseen failed. Probably a quorum-read failing because 89 | // of too many dead nodes while doing a recursive read within get. 90 | } 91 | 92 | fn drain_responses( 93 | fu: FuturesUnordered, 94 | expected: WriteResponse, 95 | ) -> SyncBoxFuture, Error>> 96 | where 97 | L: Lang, 98 | F: Future, edelcrantz::Error>)> 99 | + 'static 100 | + Send 101 | + Sync, 102 | { 103 | Box::pin(drain_responses_(fu, expected)) 104 | } 105 | 106 | async fn drain_responses_( 107 | mut fu: FuturesUnordered, 108 | expected: WriteResponse, 109 | ) -> Result, Error> 110 | where 111 | L: Lang, 112 | F: Future, edelcrantz::Error>)>, 113 | { 114 | let mut peers = Vec::new(); 115 | while let Some(res) = fu.next().await { 116 | match res { 117 | (peer, Ok(network::Res::WriteRes(x))) if x == expected => { 118 | peers.push(peer); 119 | } 120 | (_, Ok(_)) => return Err(Error::UnexpectedResponse), 121 | (_, Err(_)) => return Err(Error::NetworkingError), 122 | } 123 | } 124 | Ok(peers) 125 | } 126 | 127 | impl> Database { 128 | // This async function, like write(), resolves to a `Result::Ok(fu)` when it 129 | // has successfully replicated an abort command to a quorum; the residual 130 | // value `fu` is a future that will drain the remaining aborts-in-flight or 131 | // return an error if anything unexpected happens while draining. 132 | // 133 | // All such futures should be placed into an asynchronous task that drains 134 | // them along with all the other aborts for this txn, then calls 135 | // tidmgr.fullyReplicated to move the Srw local replication-watermark. 136 | 137 | pub(crate) fn abort_txn( 138 | &self, 139 | kv: KeyVer, 140 | ) -> SyncBoxFuture, Error>>, Error>> { 141 | let this = self.clone(); 142 | let span = debug_span!("abort_txn", ?kv); 143 | Box::pin(async move { this.abort_txn(kv).instrument(span).await }) 144 | } 145 | 146 | pub(crate) async fn abort_txn_( 147 | &self, 148 | kv: KeyVer, 149 | ) -> Result, Error>>, Error> { 150 | let mut responses = self.send_abort_to_all(kv.clone()).await; 151 | let replica_count: usize = responses.len(); 152 | debug!( 153 | "sent broadcast-abort of {:?} to {:?} peers", 154 | kv, replica_count 155 | ); 156 | let mut n_aborted_received: usize = 0; 157 | // TODO: handle timeout / "majority unavailable" case. 158 | while let Some(res) = responses.next().await { 159 | match res { 160 | (_, Ok(network::Res::WriteRes(WriteResponse::Success))) => { 161 | n_aborted_received += 1; 162 | if n_aborted_received >= majority_quorum(replica_count) { 163 | return Ok(drain_responses(responses, WriteResponse::Success)); 164 | } 165 | } 166 | 167 | (_, Ok(_)) => return Err(Error::UnexpectedResponse), 168 | (_, Err(_)) => return Err(Error::NetworkingError), 169 | } 170 | } 171 | Err(Error::TooFewReplicas) 172 | } 173 | 174 | // Section 4.1 Write-only operations. Ensures that a txn is sufficiently 175 | // replicated to allow the local visibility watermark to advance past it, 176 | // either through a super-quorum fast path or a majority-quorum slow path. 177 | // 178 | // If successful -- either writing a super-quorum or majority-quorum -- it 179 | // returns Ok(fu) where fu is a future containing the _residual_ 180 | // writes-in-flight (and the residual finalizations). The caller should 181 | // spawn a background task containing all such residual futures from all 182 | // writes, then call tidmgr.fullyReplicated when the last of the writes 183 | // completes: this will move the Srw replication watermark. 184 | pub(crate) fn write( 185 | &self, 186 | kv: KeyVer, 187 | e: L::Expr, 188 | vals: Vec, 189 | ) -> SyncBoxFuture, Error>>, Error>> { 190 | let this = self.clone(); 191 | let span = debug_span!("write", ?kv); 192 | Box::pin(async move { this.write_(kv, e, vals).instrument(span).await }) 193 | } 194 | 195 | // TODO: refactor this, maybe introduce a helper type that holds both the 196 | // ongoing writes and ongoing finalizes. 197 | pub(crate) async fn write_( 198 | &self, 199 | kv: KeyVer, 200 | e: L::Expr, 201 | vals: Vec, 202 | ) -> Result, Error>>, Error> { 203 | let start = self.tidmgr.read().await.current_time(); 204 | 205 | let mut write_responses = self.send_write_to_all(kv.clone(), e, vals).await; 206 | let replica_count = write_responses.len(); 207 | debug!( 208 | "sent broadcast-write of {:?} to {:?} peers", 209 | kv.clone(), 210 | replica_count 211 | ); 212 | let mut successes: BTreeSet = BTreeSet::new(); 213 | while let Some(res) = write_responses.next().await { 214 | debug!( 215 | "write got response {:?} ({:?}/{:?})", 216 | res, 217 | successes.len(), 218 | replica_count 219 | ); 220 | match res { 221 | (peer, Ok(network::Res::WriteRes(WriteResponse::Success))) => { 222 | successes.insert(peer); 223 | if self.within_fast_path(start).await { 224 | if successes.len() >= super_quorum(replica_count) { 225 | // We received a super-quorum within the fast-path 226 | // timeout, we can consider it safely replicated and 227 | // un-gate our caller, sending the finalizes 228 | // asynchronously and draining both. 229 | let early_write_peers: Vec = 230 | successes.iter().cloned().collect(); 231 | let early_finalize_responses = self 232 | .send_finalize_to_peers(kv.clone(), early_write_peers) 233 | .await; 234 | let this = self.clone(); 235 | let drainer = async move { 236 | let late_write_peers: Vec = 237 | drain_responses(write_responses, WriteResponse::Success) 238 | .await?; 239 | let late_finalize_responses = 240 | this.send_finalize_to_peers(kv, late_write_peers).await; 241 | drain_responses(early_finalize_responses, WriteResponse::Confirm) 242 | .await?; 243 | drain_responses(late_finalize_responses, WriteResponse::Confirm) 244 | .await 245 | }; 246 | return Ok(Box::pin(drainer)); 247 | } else { 248 | // We're in the fast-path still but haven't got a 249 | // super-majority, keep waiting. 250 | } 251 | } else { 252 | // We're past the fast-path so we're waiting for a 253 | // majority. 254 | if successes.len() >= majority_quorum(replica_count) { 255 | // We got a majority, proceed to majority-finalize below. 256 | break; 257 | } 258 | // TODO: handle timeout / "majority unavailable" case on the slow path. 259 | } 260 | } 261 | (_, Ok(r)) => { 262 | error!("unexpected write response: {:?}", r); 263 | return Err(Error::UnexpectedResponse); 264 | } 265 | (_, Err(e)) => { 266 | error!("networking error while writing: {:?}", e); 267 | return Err(Error::NetworkingError); 268 | } 269 | } 270 | } 271 | 272 | if successes.len() < majority_quorum(replica_count) { 273 | error!( 274 | "write hit too few replicas: {:?}/{:?}, needed quorum of {:?}", 275 | successes.len(), 276 | replica_count, 277 | majority_quorum(replica_count) 278 | ); 279 | return Err(Error::TooFewReplicas); 280 | } 281 | 282 | let early_write_peers = successes.iter().cloned().collect(); 283 | let mut early_finalize_responses = self 284 | .send_finalize_to_peers(kv.clone(), early_write_peers) 285 | .await; 286 | let mut n_finalize_received: usize = 0; 287 | while let Some(res) = early_finalize_responses.next().await { 288 | match res { 289 | (_, Ok(network::Res::WriteRes(WriteResponse::Confirm))) => { 290 | n_finalize_received += 1; 291 | if n_finalize_received >= majority_quorum(replica_count) { 292 | let this = self.clone(); 293 | let drainer = async move { 294 | let late_write_peers: Vec = 295 | drain_responses(write_responses, WriteResponse::Success).await?; 296 | let late_finalize_responses = 297 | this.send_finalize_to_peers(kv, late_write_peers).await; 298 | drain_responses(early_finalize_responses, WriteResponse::Confirm) 299 | .await?; 300 | drain_responses(late_finalize_responses, WriteResponse::Confirm).await 301 | }; 302 | return Ok(Box::pin(drainer)); 303 | } 304 | } 305 | (_, Ok(_)) => return Err(Error::UnexpectedResponse), 306 | (_, Err(_)) => return Err(Error::NetworkingError), 307 | } 308 | } 309 | 310 | Err(Error::TooFewReplicas) 311 | } 312 | 313 | const MAX_RETRIES: usize = 64; 314 | const RETRY_DELAY: std::time::Duration = std::time::Duration::from_millis(100); 315 | const FAST_PATH: std::time::Duration = std::time::Duration::from_millis(200); 316 | 317 | // See crate::SyncBoxFuture for explanation of this wrapper function. 318 | pub(crate) fn read(&self, kv: KeyVer) -> SyncBoxFuture), Error>> { 319 | let this = self.clone(); 320 | let span = debug_span!("read", ?kv); 321 | Box::pin(async move { this.read_(kv).instrument(span).await }) 322 | } 323 | 324 | // Section 4.3 The Read-Only Operation (a.k.a. "Algorithm 3") 325 | pub(crate) async fn read_(&self, kv: KeyVer) -> Result<(L::Key, ExtVal), Error> { 326 | // If we're reading from below the fully-replicated watermark, we can just 327 | // read from our local copy. 328 | { 329 | let RWatermark(rw) = self.group_wm.0.lock().await.replication_watermark.clone(); 330 | if kv.ver <= rw { 331 | let mut target = kv.ver; 332 | debug!( 333 | "read of {:?} is below RWatermark {:?}, reading locally", 334 | kv, rw 335 | ); 336 | loop { 337 | let gr = GetRequest(KeyVer { 338 | key: kv.key.clone(), 339 | ver: target, 340 | }); 341 | let res = self.serve_get(gr).await; 342 | match res { 343 | GetResponse::SettledEntry(_, _, ev) => return Ok((kv.key, ev)), 344 | GetResponse::AbortedEntry(time) => target = time.prev_event(), 345 | GetResponse::Failure => return Err(Error::ReadFailed), 346 | } 347 | } 348 | } 349 | } 350 | 351 | // We loop until we meet a "MATCH CONDITION" which is defined as 352 | // 353 | // the highest version ver <= ts for which at least one 354 | // replica is FINALIZED or at least f+1 replicas are TENTATIVE 355 | 356 | #[derive(Debug)] 357 | struct ReadResponse { 358 | num_tentative: usize, 359 | num_finalized: usize, 360 | num_aborted: usize, 361 | value: Option>, 362 | } 363 | 364 | impl Default for ReadResponse { 365 | fn default() -> Self { 366 | Self { 367 | num_tentative: 0, 368 | num_finalized: 0, 369 | num_aborted: 0, 370 | value: None, 371 | } 372 | } 373 | } 374 | 375 | let mut num_retries = 0; 376 | let mut target = kv.ver; 377 | 'target: loop { 378 | debug!("read fetching keyver {:?}, target={:?}", kv, target); 379 | // We get() the target version. Every version we're requesting 380 | // here is _visible_, which means consensus visible-time has 381 | // advanced enough that one of the following conditions holds: 382 | // 383 | // - write() failed at target, abort_txn succeeded at target, 384 | // so there are at most f TENTATIVE replicas for target, 385 | // and no FINALIZED replicas for target. 386 | // - write() succeeded at target: 387 | // - in the fast path, producing super_quorum 388 | // TENTATIVE-or-FINALIZED replicas; or else 389 | // - in the slow path, producing majority_quorum 390 | // FINALIZED replicas. 391 | // 392 | // If target was aborted, we read the previous version _before_ 393 | // target, because new-values-at-target are not part of the 394 | // consensus history. 395 | // 396 | // We therefore loop here from target=ts downward until we find 397 | // a version that _did_ get written. 398 | let rkv = KeyVer { 399 | key: kv.key.clone(), 400 | ver: target, 401 | }; 402 | let mut futs: FuturesUnordered<_> = self.send_get_to_all(rkv.clone()).await; 403 | let replica_count: usize = futs.len(); 404 | if replica_count == 0 { 405 | return Err(crate::Error::TooFewReplicas); 406 | } 407 | let mut responses = BTreeMap::new(); 408 | debug!( 409 | "sent broadcast-get of {:?} to {:?} peers (retry count {:?})", 410 | rkv, replica_count, num_retries 411 | ); 412 | while let Some(result) = futs.next().await { 413 | debug!("read response {:?}", result); 414 | match result { 415 | (_, Ok(network::Res::GetRes(GetResponse::AbortedEntry(ts)))) => { 416 | let rr = responses 417 | .entry(ts) 418 | .or_insert_with(|| ReadResponse::::default()); 419 | rr.num_aborted += 1; 420 | } 421 | (_, Ok(network::Res::GetRes(GetResponse::SettledEntry(ts, tag, val)))) => { 422 | let rr = responses 423 | .entry(ts) 424 | .or_insert_with(|| ReadResponse::::default()); 425 | match tag { 426 | ReplicationTag::Tentative => rr.num_tentative += 1, 427 | ReplicationTag::Finalized => rr.num_finalized += 1, 428 | } 429 | match &rr.value { 430 | None => rr.value = Some(val), 431 | Some(other) if *other != val => { 432 | return Err(Error::InconsistentReplicas) 433 | } 434 | Some(_) => (), 435 | } 436 | } 437 | _ => { 438 | // Replicated reads are fault tolerant, we're going to 439 | // retry until we hit a retry limit or a successful quorum-read. 440 | } 441 | } 442 | // Each time we receive a response, we run from max-to-min in 443 | // the set of received responses checking to see if we've got a 444 | // threshold-to-act on any of the versions yet. 445 | for (ts, rr) in responses.iter().rev() { 446 | if rr.num_aborted >= majority_quorum(replica_count) { 447 | debug!( 448 | "read explicit quorum-read aborted key at {:?}, trying earlier version", 449 | ts 450 | ); 451 | target = responses.iter().next_back().unwrap().0.prev_event(); 452 | continue 'target; 453 | } 454 | if rr.num_tentative + rr.num_finalized >= super_quorum(replica_count) 455 | || rr.num_finalized >= majority_quorum(replica_count) 456 | { 457 | if let Some(v) = &rr.value { 458 | let kv = KeyVer:: { 459 | key: kv.key, 460 | ver: *ts, 461 | }; 462 | debug!("read quorum-read {:?}={:?}", kv, v); 463 | return Ok((kv.key, v.clone())); 464 | } 465 | } 466 | } 467 | } 468 | num_retries += 1; 469 | if num_retries > Self::MAX_RETRIES { 470 | error!("read of {:?} retried {:?} times, failing", kv, num_retries); 471 | return Err(Error::ReadFailed); 472 | } 473 | let never = future::pending::<()>(); 474 | error!( 475 | "read of {:?} failed to find quorum, retrying after {:?}", 476 | kv, 477 | Self::RETRY_DELAY 478 | ); 479 | let _ = future::timeout(Self::RETRY_DELAY, never).await; 480 | } 481 | } 482 | 483 | pub(crate) async fn put( 484 | &self, 485 | ver: GlobalTime, 486 | out: &BTreeMap>, 487 | ) -> Result<(), Error> { 488 | for (_, queue) in self.connections.read().await.values() { 489 | for (k, v) in out { 490 | let kv = KeyVer { 491 | key: k.clone(), 492 | ver, 493 | }; 494 | let ow = network::OneWay::PutMsg(PutOneWay(kv, v.clone())); 495 | queue.enqueue_oneway(ow).await? 496 | } 497 | } 498 | Ok(()) 499 | } 500 | 501 | async fn within_fast_path(&self, start: SystemTime) -> bool { 502 | let now = self.tidmgr.read().await.current_time(); 503 | match now.duration_since(start) { 504 | Ok(dur) => dur < Self::FAST_PATH, 505 | Err(_) => { 506 | // If the clock moves backwards, we treat it as timeout. 507 | false 508 | } 509 | } 510 | } 511 | 512 | // See crate::SyncBoxFuture for explanation of this wrapper function. 513 | pub(crate) fn serve_write(&self, wr: WriteRequest) -> SyncBoxFuture { 514 | let this = self.clone(); 515 | let (ty, kv) = match &wr { 516 | WriteRequest::Write(kv, _, _) => ("write", kv), 517 | WriteRequest::Abort(kv) => ("abort", kv), 518 | WriteRequest::Finalize(kv) => ("finalize", kv), 519 | }; 520 | let span = debug_span!("serve_write", ?kv, ty); 521 | Box::pin(async move { this.serve_write_(wr).instrument(span).await }) 522 | } 523 | 524 | pub(crate) async fn serve_write_(&self, wr: WriteRequest) -> WriteResponse { 525 | use crate::Entry::{Aborted, Delayed, Settled}; 526 | use ReplicationTag::{Finalized, Tentative}; 527 | match wr { 528 | WriteRequest::Write(kv, x, vals) => { 529 | let e = Delayed(x, vals, Tentative); 530 | self.store.write().await.put_key_at_time(&kv, &e); 531 | WriteResponse::Success 532 | } 533 | WriteRequest::Abort(kv) => { 534 | self.store.write().await.put_key_at_time(&kv, &Aborted); 535 | WriteResponse::Success 536 | } 537 | WriteRequest::Finalize(kv) => { 538 | let mut guard = self.store.write().await; 539 | match guard.get_key_at_or_before_time(&kv) { 540 | None => { 541 | error!("finalizing {:?}: no key found", kv); 542 | WriteResponse::Failure 543 | } 544 | Some((existing_ts, existing_entry)) => { 545 | if existing_ts != kv.ver { 546 | // Sender sent us an erroneous request, or our store 547 | // is corrupt. Either way, we're done for. 548 | error!( 549 | "finalizing {:?}: timestamp mismatch with existing {:?}", 550 | kv, existing_ts 551 | ); 552 | WriteResponse::Failure 553 | } else { 554 | match existing_entry { 555 | Settled(_, Finalized) | Aborted | Delayed(_, _, Finalized) => (), // Nothing to do 556 | Delayed(x, vals, Tentative) => { 557 | guard.put_key_at_time(&kv, &Delayed(x, vals, Finalized)); 558 | } 559 | Settled(v, Tentative) => { 560 | guard.put_key_at_time(&kv, &Settled(v, Finalized)) 561 | } 562 | }; 563 | WriteResponse::Confirm 564 | } 565 | } 566 | } 567 | } 568 | } 569 | } 570 | 571 | // See lib::SyncBoxFuture for explanation of this wrapper function. 572 | pub(crate) fn serve_put(&self, pow: PutOneWay) -> SyncBoxFuture> { 573 | let this = self.clone(); 574 | let span = debug_span!("serve_put", kv=?pow.0); 575 | Box::pin(async move { this.serve_put_(pow).instrument(span).await }) 576 | } 577 | 578 | pub(crate) async fn serve_put_(&self, pow: PutOneWay) -> Result<(), Error> { 579 | let PutOneWay(kv, ev) = pow; 580 | let mut guard = self.store.write().await; 581 | let tagopt = match guard.get_key_at_or_before_time(&kv) { 582 | // TODO: is a put to a version we haven't seen yet tentative? Unsure. 583 | // It probably should never happen but we're not presently tracking the 584 | // quorum-write set between write() and put(). 585 | Some((ts, _)) if ts != kv.ver => Some(ReplicationTag::Tentative), 586 | Some((_, Entry::Delayed(_, _, tag))) => Some(tag), 587 | Some((_, Entry::Settled(ov, _))) if ov != ev => { 588 | return Err(Error::InconsistentReplicas) 589 | } 590 | _ => None, 591 | }; 592 | match tagopt { 593 | None => (), 594 | Some(tag) => { 595 | guard.put_key_at_time(&kv, &Entry::Settled(ev, tag)); 596 | } 597 | } 598 | Ok(()) 599 | } 600 | 601 | // See lib::SyncBoxFuture for explanation of this wrapper function. 602 | pub(crate) fn serve_get(&self, gr: GetRequest) -> SyncBoxFuture> { 603 | let this = self.clone(); 604 | let span = debug_span!("serve_get", kv=?gr.0); 605 | Box::pin(async move { this.serve_get_(gr).instrument(span).await }) 606 | } 607 | 608 | // The contract with 'get' is that it's never called with a timestamp 609 | // later (greater) than Vwatermark (the global minimum) which was only advanced 610 | // when there's lattice-agreement from a quorum to advance it; so we 611 | // can definitely expect to be able to resolve all the thunks (recursively) 612 | // that the `get` depends on. If this is not so, there's a protocol-level 613 | // error. 614 | pub(crate) async fn serve_get_(&self, gr: GetRequest) -> GetResponse { 615 | use crate::Entry::{Aborted, Delayed, Settled}; 616 | let GetRequest(kv) = gr; 617 | 618 | // NB: the version being requested is always "below VWatermark" in the 619 | // sense of an omnicient observer (or indeed in the eyes of the peer 620 | // that sent the get request); but _this peer_ might not have received 621 | // enough information from lattice agreement to know that yet -- the get 622 | // request might be racing with the watermark propagation message -- so 623 | // we might need to pause briefly here to see our own evidence of that 624 | // watermark advance. 625 | // 626 | // TODO: this wait might be superfluous. Argument: if VWatermark 627 | // advanced on the sender of the request, we can assume it's because 628 | // they saw all server watermarks advance past kv.ver, so the causal 629 | // order below kv.ver is fixed and it doesn't really matter if this peer 630 | // knows that yet. 631 | self.wait_for_visibility_watermark(kv.ver).await; 632 | 633 | // NB: this needs to occur before the match-head in order to release the 634 | // read() lock inside the match arms; inside the head it's an lval and 635 | // the rwlock guard's lifetime is extended to the whole match expr. 636 | let first_read = self.store.read().await.get_key_at_or_before_time(&kv); 637 | match first_read { 638 | None => { 639 | debug!("don't have {:?} yet", kv); 640 | GetResponse::Failure 641 | } 642 | Some((ver, e)) => { 643 | match e { 644 | Delayed(x, vals, tag) => { 645 | let kv = KeyVer { key: kv.key, ver }; 646 | let kl = self.get_keylock(&kv).await; 647 | let _lockguard = kl.lock().await; 648 | // re-check that someone didn't resolve on us while we 649 | // were locking. 650 | let second_read = self.store.read().await.get_key_at_or_before_time(&kv); 651 | match second_read { 652 | Some((v, Settled(val, tag))) if v == ver => { 653 | return GetResponse::SettledEntry(ver, tag, val); 654 | } 655 | _ => (), 656 | } 657 | let ver_pre = ver.prev_event(); 658 | let read_set: BTreeSet = L::get_read_set(&x); 659 | debug!( 660 | "resolving delayed expr {:?} for {:?} by fetching read-set {:?}", 661 | x, kv, read_set 662 | ); 663 | let read_futs: FuturesUnordered<_> = read_set 664 | .iter() 665 | .map(|k| { 666 | self.read(KeyVer { 667 | key: k.clone(), 668 | ver: ver_pre, 669 | }) 670 | }) 671 | .collect(); 672 | let maybe_env: Result, Error> = 673 | read_futs.try_collect().await; 674 | match maybe_env { 675 | Ok(env) => { 676 | let settled = L::eval_expr(&x, &vals, &env); 677 | self.store 678 | .write() 679 | .await 680 | .put_key_at_time(&kv, &Settled(settled.clone(), tag.clone())); 681 | GetResponse::SettledEntry(ver, tag, settled) 682 | } 683 | Err(_) => GetResponse::Failure, 684 | } 685 | } 686 | Settled(val, tag) => GetResponse::SettledEntry(ver, tag, val), 687 | Aborted => GetResponse::AbortedEntry(ver), 688 | } 689 | } 690 | } 691 | } 692 | } 693 | -------------------------------------------------------------------------------- /src/storage.rs: -------------------------------------------------------------------------------- 1 | // Copyright 2020 Graydon Hoare 2 | // Licensed under the MIT and Apache-2.0 licenses. 3 | 4 | use crate::{ExtVal, GlobalTime, Lang, ReplicationTag, Sdw}; 5 | use serde::{Deserialize, Serialize}; 6 | 7 | /// An [Entry] is associated with each [KeyVer] (i.e. a [Lang::Key] at some 8 | /// [GlobalTime]) in the [Store], and is either an unevaluated expression of 9 | /// type [Entry::Delayed] (at one of two possible [ReplicationTag] levels), or 10 | /// an [Entry::Aborted] tombstone (if replication fails), or an [Entry::Settled] 11 | /// value carrying a fully-evaluated [ExtVal]. 12 | #[serde(bound = "")] 13 | #[derive(Clone, Debug, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 14 | pub enum Entry { 15 | Delayed(L::Expr, Vec, ReplicationTag), 16 | Settled(ExtVal, ReplicationTag), 17 | Aborted, 18 | } 19 | 20 | /// A `KeyVer` is a [Lang::Key] augmented with a [GlobalTime]. All reads and 21 | /// writes -- both inside the distributed protocol and against the [Store] -- 22 | /// happen in terms of `KeyVer`s. 23 | #[derive(Clone, Default, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 24 | pub struct KeyVer { 25 | pub key: L::Key, 26 | pub ver: GlobalTime, 27 | } 28 | 29 | impl std::fmt::Debug for KeyVer { 30 | fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { 31 | f.write_fmt(format_args!("{:?}:{:?}", self.key, self.ver)) 32 | } 33 | } 34 | 35 | /// A `Store` is responsible for durable storage. Clients of the library should 36 | /// provide an implementation and pass an instance in to the constructor of 37 | /// [crate::Database]. 38 | /// 39 | /// Stores are presumed to model something like maps over [KeyVer]s. That is, 40 | /// they are "multi-version" maps, supporting the "multi-version concurrency 41 | /// control" (MVCC) protocols, of which Ocean Vista is a distributed variant. 42 | /// 43 | /// When writing, initially an un-evaluated expression-entry of type 44 | /// [Entry::Delayed] is written to the `Store`. Later the same `KeyVer` will be 45 | /// _updated_ with an [Entry::Settled], when watermark time has advanced to 46 | /// the point that it's safe to [Lang::eval_expr] the delayed expression. If at 47 | /// any point the writing phase of the transaction aborts, an [Entry::Aborted] 48 | /// entry will be written instead. All this happens inside the [crate::Database] 49 | /// though; all Store has to do is write to some backing store. 50 | pub trait Store: Send + Sync + 'static { 51 | fn get_key_at_or_before_time(&self, kv: &KeyVer) -> Option<(GlobalTime, Entry)>; 52 | fn put_key_at_time(&mut self, kv: &KeyVer, v: &Entry); 53 | fn get_delayed_watermark(&self) -> Option; 54 | } 55 | -------------------------------------------------------------------------------- /src/test.rs: -------------------------------------------------------------------------------- 1 | // Copyright 2020 Graydon Hoare 2 | // Licensed under the MIT and Apache-2.0 licenses. 3 | 4 | use clepsydra::{ 5 | Clock, Database, Entry, ExtVal, GlobalTime, KeyVer, Lang, PeerID, Sdw, Store, TestClock, 6 | }; 7 | 8 | use async_std::task; 9 | use duplexify::Duplex; 10 | use futures::{stream::FuturesUnordered, StreamExt}; 11 | use tracing::{info, warn}; 12 | 13 | #[cfg(feature = "tracy")] 14 | use tracing_subscriber::{self, layer::SubscriberExt}; 15 | 16 | #[cfg(feature = "tracy")] 17 | use tracing_tracy; 18 | 19 | use serde::{Deserialize, Serialize}; 20 | use sluice::pipe::{pipe, PipeReader, PipeWriter}; 21 | use std::{ 22 | collections::{BTreeMap, BTreeSet}, 23 | ops::Bound, 24 | }; 25 | 26 | #[derive(Clone, Debug, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 27 | enum TExpr { 28 | Add(Box, Box), 29 | Var(String), 30 | Lit(u64), 31 | } 32 | 33 | impl Default for TExpr { 34 | fn default() -> Self { 35 | TExpr::Lit(0) 36 | } 37 | } 38 | 39 | #[derive(Clone, Debug, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 40 | enum TStmt { 41 | Set(String, TExpr), 42 | Get(String), 43 | Pass, 44 | } 45 | 46 | impl Default for TStmt { 47 | fn default() -> Self { 48 | TStmt::Pass 49 | } 50 | } 51 | 52 | #[derive(Clone, Debug, Default, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 53 | struct TLang; 54 | impl Lang for TLang { 55 | type Key = String; 56 | type Val = u64; 57 | type Expr = TExpr; 58 | type Stmt = TStmt; 59 | fn get_write_set(s: &Self::Stmt) -> BTreeMap { 60 | let mut m = BTreeMap::new(); 61 | match s { 62 | TStmt::Get(_) | TStmt::Pass => (), 63 | TStmt::Set(v, e) => { 64 | m.insert(v.clone(), e.clone()); 65 | } 66 | } 67 | m 68 | } 69 | fn get_read_set(e: &Self::Expr) -> BTreeSet { 70 | fn get_vars(s: &mut BTreeSet, x: &TExpr) { 71 | match x { 72 | TExpr::Add(a, b) => { 73 | get_vars(s, &**a); 74 | get_vars(s, &**b); 75 | } 76 | TExpr::Var(v) => { 77 | s.insert((*v).clone()); 78 | } 79 | TExpr::Lit(_) => (), 80 | } 81 | } 82 | let mut s = BTreeSet::new(); 83 | get_vars(&mut s, e); 84 | s 85 | } 86 | fn get_eval_set(s: &Self::Stmt) -> BTreeSet { 87 | let mut m = BTreeSet::new(); 88 | match s { 89 | TStmt::Set(_, _) | TStmt::Pass => (), 90 | TStmt::Get(v) => { 91 | m.insert(v.clone()); 92 | } 93 | } 94 | m 95 | } 96 | fn eval_expr( 97 | e: &Self::Expr, 98 | _vals: &[Self::Val], 99 | env: &BTreeMap>, 100 | ) -> ExtVal { 101 | match e { 102 | TExpr::Add(a, b) => match ( 103 | Self::eval_expr(&**a, &[], env), 104 | Self::eval_expr(&**b, &[], env), 105 | ) { 106 | (ExtVal::Defined(p), ExtVal::Defined(q)) => ExtVal::Defined(p + q), 107 | _ => ExtVal::Initial, 108 | }, 109 | TExpr::Var(v) => match env.get(v) { 110 | None => ExtVal::Initial, 111 | Some(x) => x.clone(), 112 | }, 113 | TExpr::Lit(x) => ExtVal::Defined(*x), 114 | } 115 | } 116 | } 117 | 118 | struct TStore { 119 | map: BTreeMap, Entry>, 120 | delay_counts: BTreeMap, 121 | } 122 | impl TStore { 123 | pub fn new() -> Self { 124 | TStore { 125 | map: BTreeMap::new(), 126 | delay_counts: BTreeMap::new(), 127 | } 128 | } 129 | } 130 | impl Store for TStore { 131 | fn get_key_at_or_before_time(&self, kv: &KeyVer) -> Option<(GlobalTime, Entry)> { 132 | let lo: Bound> = Bound::Unbounded; 133 | let hi: Bound> = Bound::Included(kv.clone()); 134 | let lookup = self.map.range((lo, hi)).next_back(); 135 | if let Some((k, e)) = lookup { 136 | if k.key == kv.key { 137 | return Some((k.ver, e.clone())); 138 | } 139 | } 140 | None 141 | } 142 | fn put_key_at_time(&mut self, kv: &KeyVer, v: &Entry) { 143 | let oldval = self.map.insert(kv.clone(), v.clone()); 144 | let delta: isize = match (oldval, v) { 145 | (None, Entry::Delayed(_, _, _)) => 1, 146 | (None, _) => return, 147 | (Some(Entry::Delayed(_, _, _)), Entry::Delayed(_, _, _)) => return, 148 | (Some(_), Entry::Delayed(_, _, _)) => 1, // shouldn't happen? 149 | (Some(Entry::Delayed(_, _, _)), _) => -1, 150 | (Some(_), _) => return, 151 | }; 152 | let entry = self.delay_counts.entry(kv.ver).or_insert(0); 153 | if delta > 0 { 154 | *entry += 1; 155 | } else if delta < 0 { 156 | if *entry == 0 { 157 | warn!("delay-count underflow"); 158 | } else { 159 | *entry -= 1; 160 | } 161 | if *entry == 0 { 162 | self.delay_counts.remove(&kv.ver); 163 | } 164 | } 165 | } 166 | 167 | fn get_delayed_watermark(&self) -> Option { 168 | match self.delay_counts.iter().next() { 169 | None => None, 170 | Some((ts, _)) => Some(Sdw(*ts)), 171 | } 172 | } 173 | } 174 | 175 | fn mk_db(i: u64) -> Database> { 176 | let clock: Box = Box::new(TestClock::new()); 177 | let peer = PeerID(i); 178 | let store = TStore::new(); 179 | Database::new(peer, clock, store) 180 | } 181 | 182 | type PipeRw = Duplex; 183 | fn duplex_pair() -> (PipeRw, PipeRw) { 184 | let (a_recv, b_send) = pipe(); 185 | let (b_recv, a_send) = pipe(); 186 | let a_end = Duplex::new(a_recv, a_send); 187 | let b_end = Duplex::new(b_recv, b_send); 188 | (a_end, b_end) 189 | } 190 | 191 | async fn connect_dbs>(a: &mut Database, b: &mut Database) { 192 | let (a_end, b_end) = duplex_pair(); 193 | a.connect(b.self_id, a_end).await; 194 | b.connect(a.self_id, b_end).await; 195 | } 196 | 197 | #[cfg(feature = "tracy")] 198 | fn setup_tracing_subscriber() { 199 | static ONCE: std::sync::Once = std::sync::Once::new(); 200 | ONCE.call_once(|| { 201 | tracing::subscriber::set_global_default( 202 | tracing_subscriber::registry().with(tracing_tracy::TracyLayer::new()), 203 | ) 204 | .unwrap(); 205 | }); 206 | } 207 | 208 | #[cfg(not(feature = "tracy"))] 209 | fn setup_tracing_subscriber() { 210 | let _ = tracing_subscriber::fmt::try_init(); 211 | } 212 | 213 | pub fn multi_txn_test() { 214 | setup_tracing_subscriber(); 215 | 216 | let mut db_1 = mk_db(1); 217 | let mut db_2 = mk_db(2); 218 | let mut db_3 = mk_db(3); 219 | let sz = 75; 220 | 221 | task::block_on(async move { 222 | connect_dbs(&mut db_1, &mut db_2).await; 223 | connect_dbs(&mut db_2, &mut db_3).await; 224 | connect_dbs(&mut db_1, &mut db_3).await; 225 | 226 | db_1.launch_workers().await; 227 | db_2.launch_workers().await; 228 | db_3.launch_workers().await; 229 | 230 | let var = "a".to_string(); 231 | fn add(a: TExpr, b: TExpr) -> TExpr { 232 | TExpr::Add(Box::new(a), Box::new(b)) 233 | } 234 | fn lit(i: u64) -> TExpr { 235 | TExpr::Lit(i) 236 | } 237 | let stmt_0 = TStmt::Set(var.clone(), lit(0)); 238 | let stmt_n = TStmt::Set(var.clone(), add(lit(1), TExpr::Var(var.clone()))); 239 | let final_stmt = TStmt::Get(var.clone()); 240 | 241 | let mut fu = FuturesUnordered::new(); 242 | for i in 1..sz { 243 | let stmt = if i == 1 { 244 | stmt_0.clone() 245 | } else { 246 | stmt_n.clone() 247 | }; 248 | let vals = Vec::new(); 249 | fu.push(db_1.coordinate(stmt, vals)); 250 | } 251 | // db_1.publish_an_hour_from_now().await; 252 | while let Some(r) = fu.next().await { 253 | info!("resolved txn {:?}", r); 254 | } 255 | let final_res = db_1.coordinate(final_stmt, Vec::new()).await; 256 | info!("resolved final txn val {:?}", final_res); 257 | }); 258 | } 259 | -------------------------------------------------------------------------------- /src/tidmgr.rs: -------------------------------------------------------------------------------- 1 | // Copyright 2020 Graydon Hoare 2 | // Licensed under the MIT and Apache-2.0 licenses. 3 | 4 | use crate::{GlobalTime, PeerID, ServerWatermarksLE, ServerWatermarksLEExt, Srw, Svw}; 5 | use async_std::sync::Arc; 6 | use std::time::{SystemTime, UNIX_EPOCH}; 7 | use std::{ 8 | collections::BTreeSet, 9 | sync::atomic::{AtomicU64, Ordering}, 10 | time::Duration, 11 | }; 12 | 13 | /// Trait to support multiple sorts of clock-source. 14 | pub trait Clock: Send + Sync { 15 | fn current_time(&self) -> SystemTime; 16 | } 17 | 18 | /// An implementation of [Clock] that calls [std::time::SystemTime::now]. 19 | pub struct RealClock; 20 | impl Clock for RealClock { 21 | fn current_time(&self) -> SystemTime { 22 | std::time::SystemTime::now() 23 | } 24 | } 25 | 26 | /// An implementation of [Clock] that holds a shared [AtomicU64] representing 27 | /// the current millisecond count since the epoch, that increments on each 28 | /// call to `Clock::current_time`. 29 | pub struct TestClock(Arc); 30 | impl TestClock { 31 | pub fn new() -> Self { 32 | TestClock(Arc::new(AtomicU64::from(0))) 33 | } 34 | } 35 | impl Clock for TestClock { 36 | fn current_time(&self) -> SystemTime { 37 | SystemTime::UNIX_EPOCH + Duration::from_millis(self.0.fetch_add(1, Ordering::SeqCst)) 38 | } 39 | } 40 | /// As in the paper: TidMgr tracks the set of "Transaction IDs" ([GlobalTime]s) 41 | /// coordinated by the current peer. This includes the local server visibility 42 | /// watermark ([Svw]) and server replication watermark ([Srw]) values. 43 | pub struct TidMgr { 44 | /// The clock source we'll use to draw system time from. 45 | clock: Box, 46 | 47 | /// The previously-issued GlobalTime, will only advance monotonically 48 | /// regardless of movement of the clock. 49 | prev: GlobalTime, 50 | 51 | /// Set of all locally-issued timestamps of transactions that are writing 52 | /// but not yet written to a quorum, so still in S-phase. The svw is derived 53 | /// as the minimum value of this set (or the next-issuable timestamp if the 54 | /// set is empty). This is called the TSset in the paper, not very 55 | /// helpfully. 56 | pub(crate) wq_set: BTreeSet, 57 | 58 | /// Set of all locally-issued timestamps of transactions that are writing 59 | /// but not yet written to all peers, may or may not still be in S-phase. 60 | /// The srw is derived as the minimum value of this set (or the 61 | /// next-issueable timestamp if the set is empty). This set isn't named in 62 | /// the paper, it's left as an exercise to the reader. 63 | wa_set: BTreeSet, 64 | } 65 | 66 | impl TidMgr { 67 | pub(crate) fn new(self_id: PeerID, clock: Box) -> Self { 68 | TidMgr { 69 | clock, 70 | prev: GlobalTime::time_zero_for(self_id), 71 | wq_set: Default::default(), 72 | wa_set: Default::default(), 73 | } 74 | } 75 | 76 | pub(crate) fn current_time(&self) -> SystemTime { 77 | self.clock.current_time() 78 | } 79 | 80 | pub(crate) fn self_id(&self) -> PeerID { 81 | self.prev.peer 82 | } 83 | 84 | /// Issues a monotonically-increasing timestamp for the current server, 85 | /// adding the new timestamp to the tset for this TidMgr. 86 | /// 87 | /// Usually this moves forward with the system clock, but if the system 88 | /// clock stalls, goes backwards, or otherwise misbehaves, we just call 89 | /// `GlobalTime.next_event()` on the previously-issued timestamp, which in 90 | /// the worst case may increment the millisecond count if the 91 | /// per-millisecond event count overflows. This is the best we can do. 92 | pub(crate) fn create_timestamp(&mut self) -> GlobalTime { 93 | let now = self.clock.current_time(); 94 | let next_millis = match now.duration_since(UNIX_EPOCH) { 95 | Err(_) => None, 96 | Ok(dur) => { 97 | let secs: u64 = dur.as_secs(); 98 | let millis: u32 = dur.subsec_millis(); 99 | // Seconds since the unix epoch here should be _way_ less 100 | // than 64bit; it won't exceed 33 bits in my lifetime 101 | // or that of anyone currently living. 102 | let mut ms = secs 103 | .checked_mul(1000) 104 | .expect("create_timestamp sec-to-ms overflow"); 105 | ms = ms 106 | .checked_add(millis as u64) 107 | .expect("create_timestamp ms-addition overflow"); 108 | Some(ms) 109 | } 110 | }; 111 | self.prev = match next_millis { 112 | // As with "time zero" we issue timestamps counting from event 1 because it's slightly 113 | // nicer to read "the previous event" in logs when it's 0 not u64::max. 114 | Some(millis) if millis > self.prev.milli_secs => { 115 | self.prev.with_milli_sec(millis).with_event(1) 116 | } 117 | _ => self.prev.next_event(), 118 | }; 119 | self.prev 120 | } 121 | 122 | pub(crate) fn create_watermark_and_start_s_phase(&mut self) -> GlobalTime { 123 | let ts = self.create_timestamp(); 124 | self.wq_set.insert(ts); 125 | self.wa_set.insert(ts); 126 | ts 127 | } 128 | 129 | /// Removes `ts` from `self.wq_set`, indicating that the transaction in 130 | /// question has been stored on a quorum, and is therefore finished S-phase. 131 | /// Panics if the `ts` was not in `self.wq_set`. 132 | pub(crate) fn stored(&mut self, ts: GlobalTime) { 133 | assert!(self.wq_set.remove(&ts)); 134 | } 135 | 136 | /// Removes `ts` from `self.rset`, indicating that the transaction in 137 | /// question has been stored on _all_ peers, and can therefore be read 138 | /// from any peer directly, without performing a quorum-read. 139 | pub(crate) fn fully_replicated(&mut self, ts: GlobalTime) { 140 | assert!(!self.wq_set.contains(&ts)); 141 | assert!(self.wa_set.remove(&ts)); 142 | } 143 | 144 | /// Returns the local server visibility watermark to gossip to other 145 | /// peers, describing the minimum transaction coordinated by this peer 146 | /// and still in S-phase. If no txn is in S-phase, we return a new 147 | /// timestamp to indicate a lower bound on any txn we _could_ issue 148 | /// in the future. 149 | pub(crate) fn svw(&mut self) -> Svw { 150 | match self.wq_set.iter().next() { 151 | None => Svw(self.create_timestamp()), 152 | Some(ts) => Svw(ts.clone()), 153 | } 154 | } 155 | 156 | pub(crate) fn srw(&mut self) -> Srw { 157 | match self.wa_set.iter().next() { 158 | None => Srw(self.create_timestamp()), 159 | Some(ts) => Srw(ts.clone()), 160 | } 161 | } 162 | 163 | pub(crate) fn server_watermarks(&mut self) -> ServerWatermarksLE { 164 | ServerWatermarksLE::new(self.svw(), self.srw()) 165 | } 166 | } 167 | -------------------------------------------------------------------------------- /src/transaction.rs: -------------------------------------------------------------------------------- 1 | // Copyright 2020 Graydon Hoare 2 | // Licensed under the MIT and Apache-2.0 licenses. 3 | 4 | //! A Transaction has a timestamp which also serves as its unique identifier -- 5 | //! the system assigns a GlobalTime per transaction and increments the 6 | //! GlobalTime event counter (at least) after each such assignment. 7 | //! 8 | //! A Transaction also declares read and write sets (of keys), a set of 9 | //! positional parameter values, and a language statement. 10 | 11 | use crate::{GlobalTime, Lang}; 12 | use serde::{Deserialize, Serialize}; 13 | 14 | #[derive(Clone, Debug, Default, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 15 | pub struct Txn { 16 | pub time: GlobalTime, 17 | pub stmt: L::Stmt, 18 | pub vals: Vec, 19 | } 20 | -------------------------------------------------------------------------------- /src/watermarks.rs: -------------------------------------------------------------------------------- 1 | // Copyright 2020 Graydon Hoare 2 | // Licensed under the MIT and Apache-2.0 licenses. 3 | 4 | use crate::{GlobalTime, PeerID}; 5 | use pergola::{LatticeElt, MaxDef, MaxUnitDefault, Tuple2}; 6 | /** 7 | * All nodes track and propagate a pair of watermarks. These are 8 | * monotonically increasing global timestamps. 9 | * 10 | * The visibility watermark supports a so-called "asynchronous concurrency 11 | * control (ACC) protocol", which runs transactions in 2 phases: 12 | * 13 | * 1. "S-phase": "storage phase" in which blind writes happen. These write 14 | * deterministic _thunks_ (called "functors" in the paper) into the 15 | * database at globally unique versions/timestamps, and replicate them to 16 | * other peers for fault-tolerance. This phase establishes the global order 17 | * of transactions, but does not _run_ them. 18 | * 19 | * 2. "E-phase": "execute phase" in which stored thunks are forced, in order 20 | * to produce any _reads_ (to return to the client) and/or write final 21 | * values back to the version allocated to the thunk. This phase is gated 22 | * by collective commitment to advancing a timestamp watermark, and runs 23 | * those transactions beneath the watermark against a world in which the 24 | * meaning of _any earlier version_ has been fixed by that advance (if not 25 | * yet computed -- thunks may force one another recursively). 26 | * 27 | * Concurrency control consists solely of a monotonically increasing watermark 28 | * that gates the transition of transactions from S-phase to E-phase: neither 29 | * phase needs any other concurrency control mechanisms inside of it, and can 30 | * execute in full parallelism; the only thing worth adding is a bit of (local 31 | * inter-thread) coordination on the forcing of each thunk, to avoid wasting CPU 32 | * by forcing the same thunk multiple times in parallel if it's needed as input 33 | * to multiple transactions. 34 | * 35 | * In addition, the progress of _replication_ is also tracked using a watermark, 36 | * the "replication watermark", and versions below it can be read off from any 37 | * replica (eg. a local replica) without requiring a quorum read. This gives us 38 | * horizontal scaling for snapshot reads in the past. 39 | * 40 | */ 41 | use serde::{Deserialize, Serialize}; 42 | 43 | // GlobalTimes are themselves reasoned about in a few different forms, and we 44 | // want to carry as much information about those forms as we can in the type 45 | // system, because it's important not to get them confused! 46 | 47 | /// A server-specific visibility watermark, denoting the minimum of the server's 48 | /// issued transactions that are still being replicated to a quorum. 49 | #[derive(Clone, Debug, Default, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 50 | pub struct Svw(pub GlobalTime); 51 | 52 | /// A server-specific replication watermark, denoting the minimum of the server's 53 | /// issued transactions that are still being replicated to all peers. 54 | #[derive(Clone, Debug, Default, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 55 | pub struct Srw(pub GlobalTime); 56 | 57 | /// A server-specific delayed-evaluation watermark, denoting the minimum of the 58 | /// [crate::KeyVer]s held in the server's [crate::Store] that remain in state 59 | /// [crate::Entry::Delayed] (not-yet-evaluated). 60 | #[derive(Clone, Debug, Default, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 61 | pub struct Sdw(pub GlobalTime); 62 | 63 | /// A group-wide visibility watermark, denoting the minimum of _all_ Svws 64 | /// observed across the group of servers. 65 | #[derive(Clone, Debug, Default, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 66 | pub struct VWatermark(pub GlobalTime); 67 | 68 | /// A group-wide replication watermark, denoting the minimum of _all_ Srws 69 | /// observed across the group of servers. 70 | #[derive(Clone, Debug, Default, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 71 | pub struct RWatermark(pub GlobalTime); 72 | 73 | // This says "GlobalTime::default() is the lattice unit for pergola::MaxDef". 74 | impl MaxUnitDefault for Svw {} 75 | 76 | // This says "GlobalTime::default() is the lattice unit for pergola::MaxDef". 77 | impl MaxUnitDefault for Srw {} 78 | 79 | /// A structure carrying the group-wide watermarks, the first of which gates tx 80 | /// execution phase, the second of which switches replicated-read strategies. 81 | #[derive(Clone, Debug, Default, PartialOrd, Ord, PartialEq, Eq, Hash, Serialize, Deserialize)] 82 | pub struct GroupWatermarks { 83 | /// Watermark below which all transactions have completed their write- 84 | /// only operations (S-phase), and can thereby enter E-phase. 85 | pub visibility_watermark: VWatermark, 86 | 87 | /// Watermark below which all transactions have _fully replicated_ their 88 | /// write-only (S-phase) operations on all participant replicas. Read-only 89 | /// operations for versions below replication_watermark can access the value 90 | /// directly from any replica; until then they must use quorum reads. 91 | pub replication_watermark: RWatermark, 92 | } 93 | 94 | impl GroupWatermarks { 95 | pub fn new_from_peer_zero_time(peer: PeerID) -> GroupWatermarks { 96 | let z = GlobalTime::time_zero_for(peer); 97 | GroupWatermarks { 98 | visibility_watermark: VWatermark(z.clone()), 99 | replication_watermark: RWatermark(z.clone()), 100 | } 101 | } 102 | } 103 | 104 | pub type ServerWatermarksLD = Tuple2, MaxDef>; 105 | pub type ServerWatermarksLE = LatticeElt; 106 | pub trait ServerWatermarksLEExt { 107 | fn new(svw: Svw, srw: Srw) -> Self; 108 | fn new_from_peer_zero_time(p: PeerID) -> Self; 109 | fn svw(&self) -> &LatticeElt>; 110 | fn svw_mut(&mut self) -> &mut LatticeElt>; 111 | fn srw(&self) -> &LatticeElt>; 112 | fn srw_mut(&mut self) -> &mut LatticeElt>; 113 | } 114 | 115 | impl ServerWatermarksLEExt for LatticeElt { 116 | fn new(svw: Svw, srw: Srw) -> Self { 117 | LatticeElt::from((svw.into(), srw.into())) 118 | } 119 | fn new_from_peer_zero_time(p: PeerID) -> Self { 120 | let z = GlobalTime::time_zero_for(p); 121 | Self::new(Svw(z), Srw(z)) 122 | } 123 | fn svw(&self) -> &LatticeElt> { 124 | &self.value.0 125 | } 126 | fn svw_mut(&mut self) -> &mut LatticeElt> { 127 | &mut self.value.0 128 | } 129 | fn srw(&self) -> &LatticeElt> { 130 | &self.value.1 131 | } 132 | fn srw_mut(&mut self) -> &mut LatticeElt> { 133 | &mut self.value.1 134 | } 135 | } 136 | --------------------------------------------------------------------------------